Browse Source

testing: remove background contexts (#7509)

pull/7516/head
Sam Kleinman 2 years ago
committed by GitHub
parent
commit
332163ede6
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
65 changed files with 954 additions and 553 deletions
  1. +6
    -1
      abci/client/grpc_client.go
  2. +2
    -2
      abci/example/example_test.go
  3. +10
    -7
      cmd/tendermint/commands/debug/dump.go
  4. +4
    -3
      cmd/tendermint/commands/debug/kill.go
  5. +6
    -6
      cmd/tendermint/commands/debug/util.go
  6. +6
    -3
      cmd/tendermint/commands/reindex_event_test.go
  7. +6
    -5
      internal/blocksync/reactor_test.go
  8. +1
    -1
      internal/consensus/byzantine_test.go
  9. +11
    -6
      internal/consensus/common_test.go
  10. +5
    -5
      internal/consensus/mempool_test.go
  11. +5
    -1
      internal/consensus/msgs_test.go
  12. +4
    -3
      internal/consensus/reactor_test.go
  13. +2
    -2
      internal/consensus/replay_test.go
  14. +31
    -31
      internal/consensus/state_test.go
  15. +1
    -1
      internal/consensus/types/height_vote_set_test.go
  16. +2
    -1
      internal/eventbus/event_bus_test.go
  17. +71
    -28
      internal/evidence/pool_test.go
  18. +31
    -29
      internal/evidence/reactor_test.go
  19. +63
    -40
      internal/evidence/verify_test.go
  20. +9
    -3
      internal/libs/queue/queue_test.go
  21. +1
    -1
      internal/mempool/mempool_bench_test.go
  22. +6
    -6
      internal/mempool/mempool_test.go
  23. +4
    -4
      internal/proxy/app_conn_test.go
  24. +3
    -2
      internal/state/execution_test.go
  25. +5
    -4
      internal/state/helpers_test.go
  26. +4
    -1
      internal/state/indexer/block/kv/kv_test.go
  27. +4
    -1
      internal/state/indexer/sink/kv/kv_test.go
  28. +17
    -5
      internal/state/rollback_test.go
  29. +18
    -8
      internal/state/store_test.go
  30. +5
    -2
      internal/state/test/factory/block.go
  31. +6
    -2
      internal/state/validation_test.go
  32. +30
    -10
      internal/statesync/block_queue_test.go
  33. +7
    -4
      internal/statesync/dispatcher_test.go
  34. +16
    -14
      internal/statesync/reactor_test.go
  35. +5
    -6
      internal/test/factory/commit.go
  36. +5
    -2
      internal/test/factory/genesis.go
  37. +4
    -4
      internal/test/factory/validator.go
  38. +3
    -2
      internal/test/factory/vote.go
  39. +11
    -5
      light/client_benchmark_test.go
  40. +8
    -5
      light/client_test.go
  41. +1
    -1
      light/example_test.go
  42. +7
    -7
      light/provider/http/http_test.go
  43. +24
    -8
      light/store/db/db_test.go
  44. +8
    -4
      node/node_test.go
  45. +21
    -9
      privval/file_test.go
  46. +3
    -3
      rpc/client/examples_test.go
  47. +2
    -2
      rpc/client/helpers.go
  48. +12
    -4
      rpc/client/helpers_test.go
  49. +23
    -14
      rpc/client/mock/abci_test.go
  50. +4
    -1
      rpc/client/mock/status_test.go
  51. +11
    -5
      rpc/client/rpc_test.go
  52. +10
    -10
      rpc/jsonrpc/jsonrpc_test.go
  53. +10
    -9
      test/e2e/runner/evidence.go
  54. +65
    -29
      types/block_test.go
  55. +12
    -9
      types/evidence.go
  56. +48
    -22
      types/evidence_test.go
  57. +16
    -7
      types/light_test.go
  58. +5
    -5
      types/priv_validator.go
  59. +21
    -6
      types/proposal_test.go
  60. +5
    -5
      types/test_util.go
  61. +34
    -17
      types/validation_test.go
  62. +37
    -18
      types/validator_set_test.go
  63. +9
    -2
      types/validator_test.go
  64. +79
    -58
      types/vote_set_test.go
  65. +49
    -32
      types/vote_test.go

+ 6
- 1
abci/client/grpc_client.go View File

@ -2,6 +2,7 @@ package abciclient
import (
"context"
"errors"
"fmt"
"net"
"sync"
@ -125,10 +126,14 @@ RETRY_LOOP:
ENSURE_CONNECTED:
for {
_, err := client.Echo(context.Background(), &types.RequestEcho{Message: "hello"}, grpc.WaitForReady(true))
_, err := client.Echo(ctx, &types.RequestEcho{Message: "hello"}, grpc.WaitForReady(true))
if err == nil {
break ENSURE_CONNECTED
}
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return err
}
cli.logger.Error("Echo failed", "err", err)
time.Sleep(time.Second * echoRetryIntervalSeconds)
}


+ 2
- 2
abci/example/example_test.go View File

@ -112,7 +112,7 @@ func testStream(ctx context.Context, t *testing.T, logger log.Logger, app types.
// Sometimes send flush messages
if counter%128 == 0 {
err = client.FlushSync(context.Background())
err = client.FlushSync(ctx)
require.NoError(t, err)
}
}
@ -162,7 +162,7 @@ func testGRPCSync(ctx context.Context, t *testing.T, logger log.Logger, app type
// Write requests
for counter := 0; counter < numDeliverTxs; counter++ {
// Send request
response, err := client.DeliverTx(context.Background(), &types.RequestDeliverTx{Tx: []byte("test")})
response, err := client.DeliverTx(ctx, &types.RequestDeliverTx{Tx: []byte("test")})
require.NoError(t, err, "Error in GRPC DeliverTx")
counter++


+ 10
- 7
cmd/tendermint/commands/debug/dump.go View File

@ -1,6 +1,7 @@
package debug
import (
"context"
"errors"
"fmt"
"os"
@ -42,7 +43,7 @@ func init() {
)
}
func dumpCmdHandler(_ *cobra.Command, args []string) error {
func dumpCmdHandler(cmd *cobra.Command, args []string) error {
outDir := args[0]
if outDir == "" {
return errors.New("invalid output directory")
@ -63,22 +64,24 @@ func dumpCmdHandler(_ *cobra.Command, args []string) error {
return fmt.Errorf("failed to create new http client: %w", err)
}
ctx := cmd.Context()
home := viper.GetString(cli.HomeFlag)
conf := config.DefaultConfig()
conf = conf.SetRoot(home)
config.EnsureRoot(conf.RootDir)
dumpDebugData(outDir, conf, rpc)
dumpDebugData(ctx, outDir, conf, rpc)
ticker := time.NewTicker(time.Duration(frequency) * time.Second)
for range ticker.C {
dumpDebugData(outDir, conf, rpc)
dumpDebugData(ctx, outDir, conf, rpc)
}
return nil
}
func dumpDebugData(outDir string, conf *config.Config, rpc *rpchttp.HTTP) {
func dumpDebugData(ctx context.Context, outDir string, conf *config.Config, rpc *rpchttp.HTTP) {
start := time.Now().UTC()
tmpDir, err := os.MkdirTemp(outDir, "tendermint_debug_tmp")
@ -89,19 +92,19 @@ func dumpDebugData(outDir string, conf *config.Config, rpc *rpchttp.HTTP) {
defer os.RemoveAll(tmpDir)
logger.Info("getting node status...")
if err := dumpStatus(rpc, tmpDir, "status.json"); err != nil {
if err := dumpStatus(ctx, rpc, tmpDir, "status.json"); err != nil {
logger.Error("failed to dump node status", "error", err)
return
}
logger.Info("getting node network info...")
if err := dumpNetInfo(rpc, tmpDir, "net_info.json"); err != nil {
if err := dumpNetInfo(ctx, rpc, tmpDir, "net_info.json"); err != nil {
logger.Error("failed to dump node network info", "error", err)
return
}
logger.Info("getting node consensus state...")
if err := dumpConsensusState(rpc, tmpDir, "consensus_state.json"); err != nil {
if err := dumpConsensusState(ctx, rpc, tmpDir, "consensus_state.json"); err != nil {
logger.Error("failed to dump node consensus state", "error", err)
return
}


+ 4
- 3
cmd/tendermint/commands/debug/kill.go View File

@ -33,6 +33,7 @@ $ tendermint debug kill 34255 /path/to/tm-debug.zip`,
}
func killCmdHandler(cmd *cobra.Command, args []string) error {
ctx := cmd.Context()
pid, err := strconv.ParseInt(args[0], 10, 64)
if err != nil {
return err
@ -62,17 +63,17 @@ func killCmdHandler(cmd *cobra.Command, args []string) error {
defer os.RemoveAll(tmpDir)
logger.Info("getting node status...")
if err := dumpStatus(rpc, tmpDir, "status.json"); err != nil {
if err := dumpStatus(ctx, rpc, tmpDir, "status.json"); err != nil {
return err
}
logger.Info("getting node network info...")
if err := dumpNetInfo(rpc, tmpDir, "net_info.json"); err != nil {
if err := dumpNetInfo(ctx, rpc, tmpDir, "net_info.json"); err != nil {
return err
}
logger.Info("getting node consensus state...")
if err := dumpConsensusState(rpc, tmpDir, "consensus_state.json"); err != nil {
if err := dumpConsensusState(ctx, rpc, tmpDir, "consensus_state.json"); err != nil {
return err
}


+ 6
- 6
cmd/tendermint/commands/debug/util.go View File

@ -15,8 +15,8 @@ import (
// dumpStatus gets node status state dump from the Tendermint RPC and writes it
// to file. It returns an error upon failure.
func dumpStatus(rpc *rpchttp.HTTP, dir, filename string) error {
status, err := rpc.Status(context.Background())
func dumpStatus(ctx context.Context, rpc *rpchttp.HTTP, dir, filename string) error {
status, err := rpc.Status(ctx)
if err != nil {
return fmt.Errorf("failed to get node status: %w", err)
}
@ -26,8 +26,8 @@ func dumpStatus(rpc *rpchttp.HTTP, dir, filename string) error {
// dumpNetInfo gets network information state dump from the Tendermint RPC and
// writes it to file. It returns an error upon failure.
func dumpNetInfo(rpc *rpchttp.HTTP, dir, filename string) error {
netInfo, err := rpc.NetInfo(context.Background())
func dumpNetInfo(ctx context.Context, rpc *rpchttp.HTTP, dir, filename string) error {
netInfo, err := rpc.NetInfo(ctx)
if err != nil {
return fmt.Errorf("failed to get node network information: %w", err)
}
@ -37,8 +37,8 @@ func dumpNetInfo(rpc *rpchttp.HTTP, dir, filename string) error {
// dumpConsensusState gets consensus state dump from the Tendermint RPC and
// writes it to file. It returns an error upon failure.
func dumpConsensusState(rpc *rpchttp.HTTP, dir, filename string) error {
consDump, err := rpc.DumpConsensusState(context.Background())
func dumpConsensusState(ctx context.Context, rpc *rpchttp.HTTP, dir, filename string) error {
consDump, err := rpc.DumpConsensusState(ctx)
if err != nil {
return fmt.Errorf("failed to get node consensus dump: %w", err)
}


+ 6
- 3
cmd/tendermint/commands/reindex_event_test.go View File

@ -25,13 +25,13 @@ const (
base int64 = 2
)
func setupReIndexEventCmd() *cobra.Command {
func setupReIndexEventCmd(ctx context.Context) *cobra.Command {
reIndexEventCmd := &cobra.Command{
Use: ReIndexEventCmd.Use,
Run: func(cmd *cobra.Command, args []string) {},
}
_ = reIndexEventCmd.ExecuteContext(context.Background())
_ = reIndexEventCmd.ExecuteContext(ctx)
return reIndexEventCmd
}
@ -177,11 +177,14 @@ func TestReIndexEvent(t *testing.T) {
{height, height, false},
}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
for _, tc := range testCases {
startHeight = tc.startHeight
endHeight = tc.endHeight
err := eventReIndex(setupReIndexEventCmd(), []indexer.EventSink{mockEventSink}, mockBlockStore, mockStateStore)
err := eventReIndex(setupReIndexEventCmd(ctx), []indexer.EventSink{mockEventSink}, mockBlockStore, mockStateStore)
if tc.reIndexErr {
require.Error(t, err)
} else {


+ 6
- 5
internal/blocksync/reactor_test.go View File

@ -136,6 +136,7 @@ func (rts *reactorTestSuite) addNode(
lastBlock := blockStore.LoadBlock(blockHeight - 1)
vote, err := factory.MakeVote(
ctx,
privVal,
lastBlock.Header.ChainID, 0,
lastBlock.Header.Height, 0, 2,
@ -200,7 +201,7 @@ func TestReactor_AbruptDisconnect(t *testing.T) {
require.NoError(t, err)
defer os.RemoveAll(cfg.RootDir)
genDoc, privVals := factory.RandGenesisDoc(cfg, 1, false, 30)
genDoc, privVals := factory.RandGenesisDoc(ctx, cfg, 1, false, 30)
maxBlockHeight := int64(64)
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0)
@ -239,7 +240,7 @@ func TestReactor_SyncTime(t *testing.T) {
require.NoError(t, err)
defer os.RemoveAll(cfg.RootDir)
genDoc, privVals := factory.RandGenesisDoc(cfg, 1, false, 30)
genDoc, privVals := factory.RandGenesisDoc(ctx, cfg, 1, false, 30)
maxBlockHeight := int64(101)
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0)
@ -267,7 +268,7 @@ func TestReactor_NoBlockResponse(t *testing.T) {
defer os.RemoveAll(cfg.RootDir)
genDoc, privVals := factory.RandGenesisDoc(cfg, 1, false, 30)
genDoc, privVals := factory.RandGenesisDoc(ctx, cfg, 1, false, 30)
maxBlockHeight := int64(65)
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0}, 0)
@ -319,7 +320,7 @@ func TestReactor_BadBlockStopsPeer(t *testing.T) {
defer os.RemoveAll(cfg.RootDir)
maxBlockHeight := int64(48)
genDoc, privVals := factory.RandGenesisDoc(cfg, 1, false, 30)
genDoc, privVals := factory.RandGenesisDoc(ctx, cfg, 1, false, 30)
rts := setup(ctx, t, genDoc, privVals[0], []int64{maxBlockHeight, 0, 0, 0, 0}, 1000)
@ -353,7 +354,7 @@ func TestReactor_BadBlockStopsPeer(t *testing.T) {
//
// XXX: This causes a potential race condition.
// See: https://github.com/tendermint/tendermint/issues/6005
otherGenDoc, otherPrivVals := factory.RandGenesisDoc(cfg, 1, false, 30)
otherGenDoc, otherPrivVals := factory.RandGenesisDoc(ctx, cfg, 1, false, 30)
newNode := rts.network.MakeNode(ctx, t, p2ptest.NodeOptions{
MaxPeers: uint16(len(rts.nodes) + 1),
MaxConnected: uint16(len(rts.nodes) + 1),


+ 1
- 1
internal/consensus/byzantine_test.go View File

@ -41,7 +41,7 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
tickerFunc := newMockTickerFunc(true)
appFunc := newKVStore
genDoc, privVals := factory.RandGenesisDoc(config, nValidators, false, 30)
genDoc, privVals := factory.RandGenesisDoc(ctx, config, nValidators, false, 30)
states := make([]*State, nValidators)
for i := 0; i < nValidators; i++ {


+ 11
- 6
internal/consensus/common_test.go View File

@ -501,12 +501,13 @@ func loadPrivValidator(t *testing.T, cfg *config.Config) *privval.FilePV {
func randState(
ctx context.Context,
t *testing.T,
cfg *config.Config,
logger log.Logger,
nValidators int,
) (*State, []*validatorStub, error) {
// Get State
state, privVals := randGenesisState(cfg, nValidators, false, 10)
state, privVals := randGenesisState(ctx, t, cfg, nValidators, false, 10)
vss := make([]*validatorStub, nValidators)
@ -764,7 +765,7 @@ func randConsensusState(
configOpts ...func(*config.Config),
) ([]*State, cleanupFunc) {
genDoc, privVals := factory.RandGenesisDoc(cfg, nValidators, false, 30)
genDoc, privVals := factory.RandGenesisDoc(ctx, cfg, nValidators, false, 30)
css := make([]*State, nValidators)
logger := consensusLogger()
@ -821,7 +822,7 @@ func randConsensusNetWithPeers(
tickerFunc func() TimeoutTicker,
appFunc func(string) abci.Application,
) ([]*State, *types.GenesisDoc, *config.Config, cleanupFunc) {
genDoc, privVals := factory.RandGenesisDoc(cfg, nValidators, false, testMinPower)
genDoc, privVals := factory.RandGenesisDoc(ctx, cfg, nValidators, false, testMinPower)
css := make([]*State, nPeers)
logger := consensusLogger()
@ -878,13 +879,17 @@ func randConsensusNetWithPeers(
}
func randGenesisState(
ctx context.Context,
t *testing.T,
cfg *config.Config,
numValidators int,
randPower bool,
minPower int64) (sm.State, []types.PrivValidator) {
minPower int64,
) (sm.State, []types.PrivValidator) {
genDoc, privValidators := factory.RandGenesisDoc(cfg, numValidators, randPower, minPower)
s0, _ := sm.MakeGenesisState(genDoc)
genDoc, privValidators := factory.RandGenesisDoc(ctx, cfg, numValidators, randPower, minPower)
s0, err := sm.MakeGenesisState(genDoc)
require.NoError(t, err)
return s0, privValidators
}


+ 5
- 5
internal/consensus/mempool_test.go View File

@ -37,7 +37,7 @@ func TestMempoolNoProgressUntilTxsAvailable(t *testing.T) {
t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) })
config.Consensus.CreateEmptyBlocks = false
state, privVals := randGenesisState(baseConfig, 1, false, 10)
state, privVals := randGenesisState(ctx, t, baseConfig, 1, false, 10)
cs := newStateWithConfig(ctx, log.TestingLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
@ -62,7 +62,7 @@ func TestMempoolProgressAfterCreateEmptyBlocksInterval(t *testing.T) {
t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) })
config.Consensus.CreateEmptyBlocksInterval = ensureTimeout
state, privVals := randGenesisState(baseConfig, 1, false, 10)
state, privVals := randGenesisState(ctx, t, baseConfig, 1, false, 10)
cs := newStateWithConfig(ctx, log.TestingLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(cs.txNotifier).EnableTxsAvailable()
@ -85,7 +85,7 @@ func TestMempoolProgressInHigherRound(t *testing.T) {
t.Cleanup(func() { _ = os.RemoveAll(config.RootDir) })
config.Consensus.CreateEmptyBlocks = false
state, privVals := randGenesisState(baseConfig, 1, false, 10)
state, privVals := randGenesisState(ctx, t, baseConfig, 1, false, 10)
cs := newStateWithConfig(ctx, log.TestingLogger(), config, state, privVals[0], NewCounterApplication())
assertMempool(cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
@ -136,7 +136,7 @@ func TestMempoolTxConcurrentWithCommit(t *testing.T) {
config := configSetup(t)
logger := log.TestingLogger()
state, privVals := randGenesisState(config, 1, false, 10)
state, privVals := randGenesisState(ctx, t, config, 1, false, 10)
stateStore := sm.NewStore(dbm.NewMemDB())
blockStore := store.NewBlockStore(dbm.NewMemDB())
@ -168,7 +168,7 @@ func TestMempoolRmBadTx(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
state, privVals := randGenesisState(config, 1, false, 10)
state, privVals := randGenesisState(ctx, t, config, 1, false, 10)
app := NewCounterApplication()
stateStore := sm.NewStore(dbm.NewMemDB())
blockStore := store.NewBlockStore(dbm.NewMemDB())


+ 5
- 1
internal/consensus/msgs_test.go View File

@ -1,6 +1,7 @@
package consensus
import (
"context"
"encoding/hex"
"fmt"
"math"
@ -24,6 +25,9 @@ import (
)
func TestMsgToProto(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
psh := types.PartSetHeader{
Total: 1,
Hash: tmrand.Bytes(32),
@ -62,7 +66,7 @@ func TestMsgToProto(t *testing.T) {
pbProposal := proposal.ToProto()
pv := types.NewMockPV()
vote, err := factory.MakeVote(pv, factory.DefaultTestChainID,
vote, err := factory.MakeVote(ctx, pv, factory.DefaultTestChainID,
0, 1, 0, 2, types.BlockID{}, time.Now())
require.NoError(t, err)
pbVote := vote.ToProto()


+ 4
- 3
internal/consensus/reactor_test.go View File

@ -367,7 +367,7 @@ func TestReactorWithEvidence(t *testing.T) {
tickerFunc := newMockTickerFunc(true)
appFunc := newKVStore
genDoc, privVals := factory.RandGenesisDoc(cfg, n, false, 30)
genDoc, privVals := factory.RandGenesisDoc(ctx, cfg, n, false, 30)
states := make([]*State, n)
logger := consensusLogger()
@ -410,7 +410,8 @@ func TestReactorWithEvidence(t *testing.T) {
// everyone includes evidence of another double signing
vIdx := (i + 1) % n
ev := types.NewMockDuplicateVoteEvidenceWithValidator(1, defaultTestTime, privVals[vIdx], cfg.ChainID())
ev, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx, 1, defaultTestTime, privVals[vIdx], cfg.ChainID())
require.NoError(t, err)
evpool := &statemocks.EvidencePool{}
evpool.On("CheckEvidence", mock.AnythingOfType("types.EvidenceList")).Return(nil)
evpool.On("PendingEvidence", mock.AnythingOfType("int64")).Return([]types.Evidence{
@ -721,7 +722,7 @@ func TestReactorValidatorSetChanges(t *testing.T) {
// map of active validators
activeVals := make(map[string]struct{})
for i := 0; i < nVals; i++ {
pubKey, err := states[i].privValidator.GetPubKey(context.Background())
pubKey, err := states[i].privValidator.GetPubKey(ctx)
require.NoError(t, err)
activeVals[string(pubKey.Address())] = struct{}{}


+ 2
- 2
internal/consensus/replay_test.go View File

@ -1016,7 +1016,7 @@ func TestHandshakePanicsIfAppReturnsWrongAppHash(t *testing.T) {
genDoc, _ := sm.MakeGenesisDocFromFile(cfg.GenesisFile())
state.LastValidators = state.Validators.Copy()
// mode = 0 for committing all the blocks
blocks, err := sf.MakeBlocks(3, &state, privVal)
blocks, err := sf.MakeBlocks(ctx, 3, &state, privVal)
require.NoError(t, err)
store.chain = blocks
@ -1298,7 +1298,7 @@ func TestHandshakeUpdatesValidators(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
val, _ := factory.RandValidator(true, 10)
val, _ := factory.RandValidator(ctx, true, 10)
vals := types.NewValidatorSet([]*types.Validator{val})
app := &initChainApp{vals: types.TM2PB.ValidatorUpdates(vals)}
clientCreator := abciclient.NewLocalCreator(app)


+ 31
- 31
internal/consensus/state_test.go View File

@ -60,7 +60,7 @@ func TestStateProposerSelection0(t *testing.T) {
defer cancel()
config := configSetup(t)
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
height, round := cs1.Height, cs1.Round
@ -115,7 +115,7 @@ func TestStateProposerSelection2(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4) // test needs more work for more than 3 validators
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4) // test needs more work for more than 3 validators
require.NoError(t, err)
height := cs1.Height
@ -158,7 +158,7 @@ func TestStateEnterProposeNoPrivValidator(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs, _, err := randState(ctx, config, log.TestingLogger(), 1)
cs, _, err := randState(ctx, t, config, log.TestingLogger(), 1)
require.NoError(t, err)
cs.SetPrivValidator(ctx, nil)
height, round := cs.Height, cs.Round
@ -182,7 +182,7 @@ func TestStateEnterProposeYesPrivValidator(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs, _, err := randState(ctx, config, log.TestingLogger(), 1)
cs, _, err := randState(ctx, t, config, log.TestingLogger(), 1)
require.NoError(t, err)
height, round := cs.Height, cs.Round
@ -217,7 +217,7 @@ func TestStateBadProposal(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 2)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 2)
require.NoError(t, err)
height, round := cs1.Height, cs1.Round
vs2 := vss[1]
@ -288,7 +288,7 @@ func TestStateOversizedBlock(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 2)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 2)
require.NoError(t, err)
cs1.state.ConsensusParams.Block.MaxBytes = 2000
height, round := cs1.Height, cs1.Round
@ -365,7 +365,7 @@ func TestStateFullRound1(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs, vss, err := randState(ctx, config, logger, 1)
cs, vss, err := randState(ctx, t, config, logger, 1)
require.NoError(t, err)
height, round := cs.Height, cs.Round
@ -410,7 +410,7 @@ func TestStateFullRoundNil(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs, vss, err := randState(ctx, config, log.TestingLogger(), 1)
cs, vss, err := randState(ctx, t, config, log.TestingLogger(), 1)
require.NoError(t, err)
height, round := cs.Height, cs.Round
@ -433,7 +433,7 @@ func TestStateFullRound2(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 2)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 2)
require.NoError(t, err)
vs2 := vss[1]
height, round := cs1.Height, cs1.Round
@ -478,7 +478,7 @@ func TestStateLockNoPOL(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 2)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 2)
require.NoError(t, err)
vs2 := vss[1]
height, round := cs1.Height, cs1.Round
@ -626,7 +626,7 @@ func TestStateLockNoPOL(t *testing.T) {
ensureNewTimeout(timeoutWaitCh, height, round, cs1.config.Precommit(round).Nanoseconds())
// needed so generated block is different than locked block
cs2, _, err := randState(ctx, config, log.TestingLogger(), 2)
cs2, _, err := randState(ctx, t, config, log.TestingLogger(), 2)
require.NoError(t, err)
// before we time out into new round, set next proposal block
prop, propBlock := decideProposal(ctx, t, cs2, vs2, vs2.Height, vs2.Round+1)
@ -691,7 +691,7 @@ func TestStateLockPOLRelock(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, logger, 4)
cs1, vss, err := randState(ctx, t, config, logger, 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -797,7 +797,7 @@ func TestStateLockPOLUnlock(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -896,7 +896,7 @@ func TestStateLockPOLUnlockOnUnknownBlock(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, logger, 4)
cs1, vss, err := randState(ctx, t, config, logger, 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -1032,7 +1032,7 @@ func TestStateLockPOLSafety1(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -1162,7 +1162,7 @@ func TestStateLockPOLSafety2(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -1266,7 +1266,7 @@ func TestProposeValidBlock(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -1365,7 +1365,7 @@ func TestSetValidBlockOnDelayedPrevote(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -1433,7 +1433,7 @@ func TestSetValidBlockOnDelayedProposal(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -1496,7 +1496,7 @@ func TestWaitingTimeoutOnNilPolka(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -1522,7 +1522,7 @@ func TestWaitingTimeoutProposeOnNewRound(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -1563,7 +1563,7 @@ func TestRoundSkipOnNilPolkaFromHigherRound(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -1604,7 +1604,7 @@ func TestWaitTimeoutProposeOnNilPolkaForTheCurrentRound(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, int32(1)
@ -1636,7 +1636,7 @@ func TestEmitNewValidBlockEventOnCommitWithoutBlock(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, int32(1)
@ -1676,7 +1676,7 @@ func TestCommitFromPreviousRound(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, int32(1)
@ -1736,7 +1736,7 @@ func TestStartNextHeightCorrectlyAfterTimeout(t *testing.T) {
defer cancel()
config.Consensus.SkipTimeoutCommit = false
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
cs1.txNotifier = &fakeTxNotifier{ch: make(chan struct{})}
@ -1802,7 +1802,7 @@ func TestResetTimeoutPrecommitUponNewHeight(t *testing.T) {
defer cancel()
config.Consensus.SkipTimeoutCommit = false
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
@ -1949,7 +1949,7 @@ func TestStateHalt1(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs1, vss, err := randState(ctx, config, log.TestingLogger(), 4)
cs1, vss, err := randState(ctx, t, config, log.TestingLogger(), 4)
require.NoError(t, err)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
height, round := cs1.Height, cs1.Round
@ -2023,7 +2023,7 @@ func TestStateOutputsBlockPartsStats(t *testing.T) {
defer cancel()
// create dummy peer
cs, _, err := randState(ctx, config, log.TestingLogger(), 1)
cs, _, err := randState(ctx, t, config, log.TestingLogger(), 1)
require.NoError(t, err)
peerID, err := types.NewNodeID("AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA")
require.NoError(t, err)
@ -2071,7 +2071,7 @@ func TestStateOutputVoteStats(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
cs, vss, err := randState(ctx, config, log.TestingLogger(), 2)
cs, vss, err := randState(ctx, t, config, log.TestingLogger(), 2)
require.NoError(t, err)
// create dummy peer
peerID, err := types.NewNodeID("AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA")
@ -2110,7 +2110,7 @@ func TestSignSameVoteTwice(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
_, vss, err := randState(ctx, config, log.TestingLogger(), 2)
_, vss, err := randState(ctx, t, config, log.TestingLogger(), 2)
require.NoError(t, err)
randBytes := tmrand.Bytes(tmhash.Size)


+ 1
- 1
internal/consensus/types/height_vote_set_test.go View File

@ -32,7 +32,7 @@ func TestPeerCatchupRounds(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
valSet, privVals := factory.RandValidatorSet(10, 1)
valSet, privVals := factory.RandValidatorSet(ctx, 10, 1)
hvs := NewHeightVoteSet(cfg.ChainID(), 1, valSet)


+ 2
- 1
internal/eventbus/event_bus_test.go View File

@ -309,7 +309,8 @@ func TestEventBusPublishEventNewEvidence(t *testing.T) {
err := eventBus.Start(ctx)
require.NoError(t, err)
ev := types.NewMockDuplicateVoteEvidence(1, time.Now(), "test-chain-id")
ev, err := types.NewMockDuplicateVoteEvidence(ctx, 1, time.Now(), "test-chain-id")
require.NoError(t, err)
const query = `tm.event='NewEvidence'`
evSub, err := eventBus.SubscribeWithArgs(ctx, tmpubsub.SubscribeArgs{


+ 71
- 28
internal/evidence/pool_test.go View File

@ -38,7 +38,10 @@ func TestEvidencePoolBasic(t *testing.T) {
blockStore = &mocks.BlockStore{}
)
valSet, privVals := factory.RandValidatorSet(1, 10)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
valSet, privVals := factory.RandValidatorSet(ctx, 1, 10)
blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return(
&types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}},
@ -54,8 +57,8 @@ func TestEvidencePoolBasic(t *testing.T) {
require.Equal(t, 0, len(evs))
require.Zero(t, size)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime, privVals[0], evidenceChainID)
ev, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx, height, defaultEvidenceTime, privVals[0], evidenceChainID)
require.NoError(t, err)
// good evidence
evAdded := make(chan struct{})
go func() {
@ -88,10 +91,13 @@ func TestEvidencePoolBasic(t *testing.T) {
// Tests inbound evidence for the right time and height
func TestAddExpiredEvidence(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var (
val = types.NewMockPV()
height = int64(30)
stateStore = initializeValidatorState(t, val, height)
stateStore = initializeValidatorState(ctx, t, val, height)
evidenceDB = dbm.NewMemDB()
blockStore = &mocks.BlockStore{}
expiredEvidenceTime = time.Date(2018, 1, 1, 0, 0, 0, 0, time.UTC)
@ -126,8 +132,12 @@ func TestAddExpiredEvidence(t *testing.T) {
tc := tc
t.Run(tc.evDescription, func(t *testing.T) {
ev := types.NewMockDuplicateVoteEvidenceWithValidator(tc.evHeight, tc.evTime, val, evidenceChainID)
err := pool.AddEvidence(ev)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ev, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx, tc.evHeight, tc.evTime, val, evidenceChainID)
require.NoError(t, err)
err = pool.AddEvidence(ev)
if tc.expErr {
require.Error(t, err)
} else {
@ -140,9 +150,14 @@ func TestAddExpiredEvidence(t *testing.T) {
func TestReportConflictingVotes(t *testing.T) {
var height int64 = 10
pool, pv := defaultTestPool(t, height)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
pool, pv := defaultTestPool(ctx, t, height)
val := types.NewValidator(pv.PrivKey.PubKey(), 10)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height+1, defaultEvidenceTime, pv, evidenceChainID)
ev, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx, height+1, defaultEvidenceTime, pv, evidenceChainID)
require.NoError(t, err)
pool.ReportConflictingVotes(ev.VoteA, ev.VoteB)
@ -174,33 +189,40 @@ func TestReportConflictingVotes(t *testing.T) {
func TestEvidencePoolUpdate(t *testing.T) {
height := int64(21)
pool, val := defaultTestPool(t, height)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
pool, val := defaultTestPool(ctx, t, height)
state := pool.State()
// create two lots of old evidence that we expect to be pruned when we update
prunedEv := types.NewMockDuplicateVoteEvidenceWithValidator(
prunedEv, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx,
1,
defaultEvidenceTime.Add(1*time.Minute),
val,
evidenceChainID,
)
require.NoError(t, err)
notPrunedEv := types.NewMockDuplicateVoteEvidenceWithValidator(
notPrunedEv, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx,
2,
defaultEvidenceTime.Add(2*time.Minute),
val,
evidenceChainID,
)
require.NoError(t, err)
require.NoError(t, pool.AddEvidence(prunedEv))
require.NoError(t, pool.AddEvidence(notPrunedEv))
ev := types.NewMockDuplicateVoteEvidenceWithValidator(
ev, err := types.NewMockDuplicateVoteEvidenceWithValidator(
ctx,
height,
defaultEvidenceTime.Add(21*time.Minute),
val,
evidenceChainID,
)
require.NoError(t, err)
lastCommit := makeCommit(height, val.PrivKey.PubKey().Address())
block := types.MakeBlock(height+1, []types.Tx{}, lastCommit, []types.Evidence{ev})
@ -227,7 +249,7 @@ func TestEvidencePoolUpdate(t *testing.T) {
require.Equal(t, []types.Evidence{notPrunedEv}, evList)
// b) If we try to check this evidence again it should fail because it has already been committed
err := pool.CheckEvidence(types.EvidenceList{ev})
err = pool.CheckEvidence(types.EvidenceList{ev})
if assert.Error(t, err) {
assert.Equal(t, "evidence was already committed", err.(*types.ErrInvalidEvidence).Reason.Error())
}
@ -236,29 +258,40 @@ func TestEvidencePoolUpdate(t *testing.T) {
func TestVerifyPendingEvidencePasses(t *testing.T) {
var height int64 = 1
pool, val := defaultTestPool(t, height)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
pool, val := defaultTestPool(ctx, t, height)
ev, err := types.NewMockDuplicateVoteEvidenceWithValidator(
ctx,
height,
defaultEvidenceTime.Add(1*time.Minute),
val,
evidenceChainID,
)
require.NoError(t, err)
require.NoError(t, pool.AddEvidence(ev))
require.NoError(t, pool.CheckEvidence(types.EvidenceList{ev}))
}
func TestVerifyDuplicatedEvidenceFails(t *testing.T) {
var height int64 = 1
pool, val := defaultTestPool(t, height)
ev := types.NewMockDuplicateVoteEvidenceWithValidator(
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
pool, val := defaultTestPool(ctx, t, height)
ev, err := types.NewMockDuplicateVoteEvidenceWithValidator(
ctx,
height,
defaultEvidenceTime.Add(1*time.Minute),
val,
evidenceChainID,
)
err := pool.CheckEvidence(types.EvidenceList{ev, ev})
require.NoError(t, err)
err = pool.CheckEvidence(types.EvidenceList{ev, ev})
if assert.Error(t, err) {
assert.Equal(t, "duplicate evidence", err.(*types.ErrInvalidEvidence).Reason.Error())
}
@ -271,8 +304,10 @@ func TestLightClientAttackEvidenceLifecycle(t *testing.T) {
height int64 = 100
commonHeight int64 = 90
)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ev, trusted, common := makeLunaticEvidence(t, height, commonHeight,
ev, trusted, common := makeLunaticEvidence(ctx, t, height, commonHeight,
10, 5, 5, defaultEvidenceTime, defaultEvidenceTime.Add(1*time.Hour))
state := sm.State{
@ -326,11 +361,14 @@ func TestLightClientAttackEvidenceLifecycle(t *testing.T) {
// Tests that restarting the evidence pool after a potential failure will recover the
// pending evidence and continue to gossip it
func TestRecoverPendingEvidence(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
height := int64(10)
val := types.NewMockPV()
valAddress := val.PrivKey.PubKey().Address()
evidenceDB := dbm.NewMemDB()
stateStore := initializeValidatorState(t, val, height)
stateStore := initializeValidatorState(ctx, t, val, height)
state, err := stateStore.Load()
require.NoError(t, err)
@ -342,18 +380,22 @@ func TestRecoverPendingEvidence(t *testing.T) {
pool, err := evidence.NewPool(log.TestingLogger(), evidenceDB, stateStore, blockStore)
require.NoError(t, err)
goodEvidence := types.NewMockDuplicateVoteEvidenceWithValidator(
goodEvidence, err := types.NewMockDuplicateVoteEvidenceWithValidator(
ctx,
height,
defaultEvidenceTime.Add(10*time.Minute),
val,
evidenceChainID,
)
expiredEvidence := types.NewMockDuplicateVoteEvidenceWithValidator(
require.NoError(t, err)
expiredEvidence, err := types.NewMockDuplicateVoteEvidenceWithValidator(
ctx,
int64(1),
defaultEvidenceTime.Add(1*time.Minute),
val,
evidenceChainID,
)
require.NoError(t, err)
require.NoError(t, pool.AddEvidence(goodEvidence))
require.NoError(t, pool.AddEvidence(expiredEvidence))
@ -420,8 +462,8 @@ func initializeStateFromValidatorSet(t *testing.T, valSet *types.ValidatorSet, h
return stateStore
}
func initializeValidatorState(t *testing.T, privVal types.PrivValidator, height int64) sm.Store {
pubKey, _ := privVal.GetPubKey(context.Background())
func initializeValidatorState(ctx context.Context, t *testing.T, privVal types.PrivValidator, height int64) sm.Store {
pubKey, _ := privVal.GetPubKey(ctx)
validator := &types.Validator{Address: pubKey.Address(), VotingPower: 10, PubKey: pubKey}
// create validator set and state
@ -471,13 +513,14 @@ func makeCommit(height int64, valAddr []byte) *types.Commit {
return types.NewCommit(height, 0, types.BlockID{}, commitSigs)
}
func defaultTestPool(t *testing.T, height int64) (*evidence.Pool, types.MockPV) {
func defaultTestPool(ctx context.Context, t *testing.T, height int64) (*evidence.Pool, types.MockPV) {
t.Helper()
val := types.NewMockPV()
valAddress := val.PrivKey.PubKey().Address()
evidenceDB := dbm.NewMemDB()
stateStore := initializeValidatorState(t, val, height)
state, _ := stateStore.Load()
stateStore := initializeValidatorState(ctx, t, val, height)
state, err := stateStore.Load()
require.NoError(t, err)
blockStore, err := initializeBlockStore(dbm.NewMemDB(), state, valAddress)
require.NoError(t, err)


+ 31
- 29
internal/evidence/reactor_test.go View File

@ -192,6 +192,7 @@ func (rts *reactorTestSuite) waitForEvidence(t *testing.T, evList types.Evidence
}
func createEvidenceList(
ctx context.Context,
t *testing.T,
pool *evidence.Pool,
val types.PrivValidator,
@ -202,13 +203,14 @@ func createEvidenceList(
evList := make([]types.Evidence, numEvidence)
for i := 0; i < numEvidence; i++ {
ev := types.NewMockDuplicateVoteEvidenceWithValidator(
ev, err := types.NewMockDuplicateVoteEvidenceWithValidator(
ctx,
int64(i+1),
time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC),
val,
evidenceChainID,
)
require.NoError(t, err)
require.NoError(t, pool.AddEvidence(ev),
"adding evidence it#%d of %d to pool with height %d",
i, numEvidence, pool.State().LastBlockHeight)
@ -225,14 +227,14 @@ func TestReactorMultiDisconnect(t *testing.T) {
val := types.NewMockPV()
height := int64(numEvidence) + 10
stateDB1 := initializeValidatorState(t, val, height)
stateDB2 := initializeValidatorState(t, val, height)
stateDB1 := initializeValidatorState(ctx, t, val, height)
stateDB2 := initializeValidatorState(ctx, t, val, height)
rts := setup(ctx, t, []sm.Store{stateDB1, stateDB2}, 20)
primary := rts.nodes[0]
secondary := rts.nodes[1]
_ = createEvidenceList(t, rts.pools[primary.NodeID], val, numEvidence)
_ = createEvidenceList(ctx, t, rts.pools[primary.NodeID], val, numEvidence)
require.Equal(t, primary.PeerManager.Status(secondary.NodeID), p2p.PeerStatusDown)
@ -259,6 +261,9 @@ func TestReactorMultiDisconnect(t *testing.T) {
func TestReactorBroadcastEvidence(t *testing.T) {
numPeers := 7
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// create a stateDB for all test suites (nodes)
stateDBs := make([]sm.Store, numPeers)
val := types.NewMockPV()
@ -267,12 +272,9 @@ func TestReactorBroadcastEvidence(t *testing.T) {
// evidence for.
height := int64(numEvidence) + 10
for i := 0; i < numPeers; i++ {
stateDBs[i] = initializeValidatorState(t, val, height)
stateDBs[i] = initializeValidatorState(ctx, t, val, height)
}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
rts := setup(ctx, t, stateDBs, 0)
rts.start(ctx, t)
@ -292,7 +294,7 @@ func TestReactorBroadcastEvidence(t *testing.T) {
secondaryIDs = append(secondaryIDs, id)
}
evList := createEvidenceList(t, rts.pools[primary.NodeID], val, numEvidence)
evList := createEvidenceList(ctx, t, rts.pools[primary.NodeID], val, numEvidence)
// Add each secondary suite (node) as a peer to the primary suite (node). This
// will cause the primary to gossip all evidence to the secondaries.
@ -320,14 +322,14 @@ func TestReactorBroadcastEvidence_Lagging(t *testing.T) {
height1 := int64(numEvidence) + 10
height2 := int64(numEvidence) / 2
// stateDB1 is ahead of stateDB2, where stateDB1 has all heights (1-20) and
// stateDB2 only has heights 1-5.
stateDB1 := initializeValidatorState(t, val, height1)
stateDB2 := initializeValidatorState(t, val, height2)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// stateDB1 is ahead of stateDB2, where stateDB1 has all heights (1-20) and
// stateDB2 only has heights 1-5.
stateDB1 := initializeValidatorState(ctx, t, val, height1)
stateDB2 := initializeValidatorState(ctx, t, val, height2)
rts := setup(ctx, t, []sm.Store{stateDB1, stateDB2}, 100)
rts.start(ctx, t)
@ -336,7 +338,7 @@ func TestReactorBroadcastEvidence_Lagging(t *testing.T) {
// Send a list of valid evidence to the first reactor's, the one that is ahead,
// evidence pool.
evList := createEvidenceList(t, rts.pools[primary.NodeID], val, numEvidence)
evList := createEvidenceList(ctx, t, rts.pools[primary.NodeID], val, numEvidence)
// Add each secondary suite (node) as a peer to the primary suite (node). This
// will cause the primary to gossip all evidence to the secondaries.
@ -356,17 +358,17 @@ func TestReactorBroadcastEvidence_Pending(t *testing.T) {
val := types.NewMockPV()
height := int64(10)
stateDB1 := initializeValidatorState(t, val, height)
stateDB2 := initializeValidatorState(t, val, height)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stateDB1 := initializeValidatorState(ctx, t, val, height)
stateDB2 := initializeValidatorState(ctx, t, val, height)
rts := setup(ctx, t, []sm.Store{stateDB1, stateDB2}, 100)
primary := rts.nodes[0]
secondary := rts.nodes[1]
evList := createEvidenceList(t, rts.pools[primary.NodeID], val, numEvidence)
evList := createEvidenceList(ctx, t, rts.pools[primary.NodeID], val, numEvidence)
// Manually add half the evidence to the secondary which will mark them as
// pending.
@ -396,19 +398,19 @@ func TestReactorBroadcastEvidence_Committed(t *testing.T) {
val := types.NewMockPV()
height := int64(10)
stateDB1 := initializeValidatorState(t, val, height)
stateDB2 := initializeValidatorState(t, val, height)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stateDB1 := initializeValidatorState(ctx, t, val, height)
stateDB2 := initializeValidatorState(ctx, t, val, height)
rts := setup(ctx, t, []sm.Store{stateDB1, stateDB2}, 0)
primary := rts.nodes[0]
secondary := rts.nodes[1]
// add all evidence to the primary reactor
evList := createEvidenceList(t, rts.pools[primary.NodeID], val, numEvidence)
evList := createEvidenceList(ctx, t, rts.pools[primary.NodeID], val, numEvidence)
// Manually add half the evidence to the secondary which will mark them as
// pending.
@ -450,20 +452,20 @@ func TestReactorBroadcastEvidence_FullyConnected(t *testing.T) {
stateDBs := make([]sm.Store, numPeers)
val := types.NewMockPV()
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// We need all validators saved for heights at least as high as we have
// evidence for.
height := int64(numEvidence) + 10
for i := 0; i < numPeers; i++ {
stateDBs[i] = initializeValidatorState(t, val, height)
stateDBs[i] = initializeValidatorState(ctx, t, val, height)
}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
rts := setup(ctx, t, stateDBs, 0)
rts.start(ctx, t)
evList := createEvidenceList(t, rts.pools[rts.network.RandomNode().NodeID], val, numEvidence)
evList := createEvidenceList(ctx, t, rts.pools[rts.network.RandomNode().NodeID], val, numEvidence)
// every suite (reactor) connects to every other suite (reactor)
for outerID, outerChan := range rts.peerChans {


+ 63
- 40
internal/evidence/verify_test.go View File

@ -33,9 +33,12 @@ func TestVerifyLightClientAttack_Lunatic(t *testing.T) {
totalVals = 10
byzVals = 4
)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
attackTime := defaultEvidenceTime.Add(1 * time.Hour)
// create valid lunatic evidence
ev, trusted, common := makeLunaticEvidence(
ev, trusted, common := makeLunaticEvidence(ctx,
t, height, commonHeight, totalVals, byzVals, totalVals-byzVals, defaultEvidenceTime, attackTime)
require.NoError(t, ev.ValidateBasic())
@ -57,7 +60,7 @@ func TestVerifyLightClientAttack_Lunatic(t *testing.T) {
assert.Error(t, ev.ValidateABCI(common.ValidatorSet, trusted.SignedHeader, defaultEvidenceTime))
// evidence without enough malicious votes should fail
ev, trusted, common = makeLunaticEvidence(
ev, trusted, common = makeLunaticEvidence(ctx,
t, height, commonHeight, totalVals, byzVals-1, totalVals-byzVals, defaultEvidenceTime, attackTime)
err = evidence.VerifyLightClientAttack(ev, common.SignedHeader, trusted.SignedHeader, common.ValidatorSet,
defaultEvidenceTime.Add(2*time.Hour), 3*time.Hour)
@ -71,9 +74,12 @@ func TestVerify_LunaticAttackAgainstState(t *testing.T) {
totalVals = 10
byzVals = 4
)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
attackTime := defaultEvidenceTime.Add(1 * time.Hour)
// create valid lunatic evidence
ev, trusted, common := makeLunaticEvidence(
ev, trusted, common := makeLunaticEvidence(ctx,
t, height, commonHeight, totalVals, byzVals, totalVals-byzVals, defaultEvidenceTime, attackTime)
// now we try to test verification against state
@ -141,8 +147,10 @@ func TestVerify_ForwardLunaticAttack(t *testing.T) {
)
attackTime := defaultEvidenceTime.Add(1 * time.Hour)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// create a forward lunatic attack
ev, trusted, common := makeLunaticEvidence(
ev, trusted, common := makeLunaticEvidence(ctx,
t, attackHeight, commonHeight, totalVals, byzVals, totalVals-byzVals, defaultEvidenceTime, attackTime)
// now we try to test verification against state
@ -190,7 +198,10 @@ func TestVerify_ForwardLunaticAttack(t *testing.T) {
}
func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
conflictingVals, conflictingPrivVals := factory.RandValidatorSet(5, 10)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
conflictingVals, conflictingPrivVals := factory.RandValidatorSet(ctx, 5, 10)
conflictingHeader, err := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
@ -215,7 +226,7 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
// except the last validator vote twice
blockID := factory.MakeBlockIDWithHash(conflictingHeader.Hash())
voteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
commit, err := factory.MakeCommit(blockID, 10, 1, voteSet, conflictingPrivVals[:4], defaultEvidenceTime)
commit, err := factory.MakeCommit(ctx, blockID, 10, 1, voteSet, conflictingPrivVals[:4], defaultEvidenceTime)
require.NoError(t, err)
ev := &types.LightClientAttackEvidence{
ConflictingBlock: &types.LightBlock{
@ -233,7 +244,7 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash"))
trustedVoteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals)
trustedCommit, err := factory.MakeCommit(trustedBlockID, 10, 1,
trustedCommit, err := factory.MakeCommit(ctx, trustedBlockID, 10, 1,
trustedVoteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
trustedSignedHeader := &types.SignedHeader{
@ -285,7 +296,11 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) {
func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
var height int64 = 10
conflictingVals, conflictingPrivVals := factory.RandValidatorSet(5, 10)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
conflictingVals, conflictingPrivVals := factory.RandValidatorSet(ctx, 5, 10)
conflictingHeader, err := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
@ -310,7 +325,7 @@ func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
// except the last validator vote twice. However this time the commits are of different rounds.
blockID := makeBlockID(conflictingHeader.Hash(), 1000, []byte("partshash"))
voteSet := types.NewVoteSet(evidenceChainID, height, 0, tmproto.SignedMsgType(2), conflictingVals)
commit, err := factory.MakeCommit(blockID, height, 0, voteSet, conflictingPrivVals, defaultEvidenceTime)
commit, err := factory.MakeCommit(ctx, blockID, height, 0, voteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
ev := &types.LightClientAttackEvidence{
ConflictingBlock: &types.LightBlock{
@ -328,7 +343,7 @@ func TestVerifyLightClientAttack_Amnesia(t *testing.T) {
trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash"))
trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), conflictingVals)
trustedCommit, err := factory.MakeCommit(trustedBlockID, height, 1,
trustedCommit, err := factory.MakeCommit(ctx, trustedBlockID, height, 1,
trustedVoteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
trustedSignedHeader := &types.SignedHeader{
@ -376,9 +391,12 @@ type voteData struct {
}
func TestVerifyDuplicateVoteEvidence(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
val := types.NewMockPV()
val2 := types.NewMockPV()
valSet := types.NewValidatorSet([]*types.Validator{val.ExtractIntoValidator(1)})
valSet := types.NewValidatorSet([]*types.Validator{val.ExtractIntoValidator(ctx, 1)})
blockID := makeBlockID([]byte("blockhash"), 1000, []byte("partshash"))
blockID2 := makeBlockID([]byte("blockhash2"), 1000, []byte("partshash"))
@ -387,30 +405,30 @@ func TestVerifyDuplicateVoteEvidence(t *testing.T) {
const chainID = "mychain"
vote1 := makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultEvidenceTime)
vote1 := makeVote(ctx, t, val, chainID, 0, 10, 2, 1, blockID, defaultEvidenceTime)
v1 := vote1.ToProto()
err := val.SignVote(context.Background(), chainID, v1)
err := val.SignVote(ctx, chainID, v1)
require.NoError(t, err)
badVote := makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultEvidenceTime)
badVote := makeVote(ctx, t, val, chainID, 0, 10, 2, 1, blockID, defaultEvidenceTime)
bv := badVote.ToProto()
err = val2.SignVote(context.Background(), chainID, bv)
err = val2.SignVote(ctx, chainID, bv)
require.NoError(t, err)
vote1.Signature = v1.Signature
badVote.Signature = bv.Signature
cases := []voteData{
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID2, defaultEvidenceTime), true}, // different block ids
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID3, defaultEvidenceTime), true},
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID4, defaultEvidenceTime), true},
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultEvidenceTime), false}, // wrong block id
{vote1, makeVote(t, val, "mychain2", 0, 10, 2, 1, blockID2, defaultEvidenceTime), false}, // wrong chain id
{vote1, makeVote(t, val, chainID, 0, 11, 2, 1, blockID2, defaultEvidenceTime), false}, // wrong height
{vote1, makeVote(t, val, chainID, 0, 10, 3, 1, blockID2, defaultEvidenceTime), false}, // wrong round
{vote1, makeVote(t, val, chainID, 0, 10, 2, 2, blockID2, defaultEvidenceTime), false}, // wrong step
{vote1, makeVote(t, val2, chainID, 0, 10, 2, 1, blockID2, defaultEvidenceTime), false}, // wrong validator
{vote1, makeVote(ctx, t, val, chainID, 0, 10, 2, 1, blockID2, defaultEvidenceTime), true}, // different block ids
{vote1, makeVote(ctx, t, val, chainID, 0, 10, 2, 1, blockID3, defaultEvidenceTime), true},
{vote1, makeVote(ctx, t, val, chainID, 0, 10, 2, 1, blockID4, defaultEvidenceTime), true},
{vote1, makeVote(ctx, t, val, chainID, 0, 10, 2, 1, blockID, defaultEvidenceTime), false}, // wrong block id
{vote1, makeVote(ctx, t, val, "mychain2", 0, 10, 2, 1, blockID2, defaultEvidenceTime), false}, // wrong chain id
{vote1, makeVote(ctx, t, val, chainID, 0, 11, 2, 1, blockID2, defaultEvidenceTime), false}, // wrong height
{vote1, makeVote(ctx, t, val, chainID, 0, 10, 3, 1, blockID2, defaultEvidenceTime), false}, // wrong round
{vote1, makeVote(ctx, t, val, chainID, 0, 10, 2, 2, blockID2, defaultEvidenceTime), false}, // wrong step
{vote1, makeVote(ctx, t, val2, chainID, 0, 10, 2, 1, blockID2, defaultEvidenceTime), false}, // wrong validator
// a different vote time doesn't matter
{vote1, makeVote(t, val, chainID, 0, 10, 2, 1, blockID2, time.Date(2020, 1, 1, 0, 0, 0, 0, time.UTC)), true},
{vote1, makeVote(ctx, t, val, chainID, 0, 10, 2, 1, blockID2, time.Date(2020, 1, 1, 0, 0, 0, 0, time.UTC)), true},
{vote1, badVote, false}, // signed by wrong key
}
@ -431,11 +449,14 @@ func TestVerifyDuplicateVoteEvidence(t *testing.T) {
}
// create good evidence and correct validator power
goodEv := types.NewMockDuplicateVoteEvidenceWithValidator(10, defaultEvidenceTime, val, chainID)
goodEv, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx, 10, defaultEvidenceTime, val, chainID)
require.NoError(t, err)
goodEv.ValidatorPower = 1
goodEv.TotalVotingPower = 1
badEv := types.NewMockDuplicateVoteEvidenceWithValidator(10, defaultEvidenceTime, val, chainID)
badTimeEv := types.NewMockDuplicateVoteEvidenceWithValidator(10, defaultEvidenceTime.Add(1*time.Minute), val, chainID)
badEv, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx, 10, defaultEvidenceTime, val, chainID)
require.NoError(t, err)
badTimeEv, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx, 10, defaultEvidenceTime.Add(1*time.Minute), val, chainID)
require.NoError(t, err)
badTimeEv.ValidatorPower = 1
badTimeEv.TotalVotingPower = 1
state := sm.State{
@ -469,25 +490,26 @@ func TestVerifyDuplicateVoteEvidence(t *testing.T) {
}
func makeLunaticEvidence(
ctx context.Context,
t *testing.T,
height, commonHeight int64,
totalVals, byzVals, phantomVals int,
commonTime, attackTime time.Time,
) (ev *types.LightClientAttackEvidence, trusted *types.LightBlock, common *types.LightBlock) {
commonValSet, commonPrivVals := factory.RandValidatorSet(totalVals, defaultVotingPower)
commonValSet, commonPrivVals := factory.RandValidatorSet(ctx, totalVals, defaultVotingPower)
require.Greater(t, totalVals, byzVals)
// extract out the subset of byzantine validators in the common validator set
byzValSet, byzPrivVals := commonValSet.Validators[:byzVals], commonPrivVals[:byzVals]
phantomValSet, phantomPrivVals := factory.RandValidatorSet(phantomVals, defaultVotingPower)
phantomValSet, phantomPrivVals := factory.RandValidatorSet(ctx, phantomVals, defaultVotingPower)
conflictingVals := phantomValSet.Copy()
require.NoError(t, conflictingVals.UpdateWithChangeSet(byzValSet))
conflictingPrivVals := append(phantomPrivVals, byzPrivVals...)
conflictingPrivVals = orderPrivValsByValSet(t, conflictingVals, conflictingPrivVals)
conflictingPrivVals = orderPrivValsByValSet(ctx, t, conflictingVals, conflictingPrivVals)
commonHeader, err := factory.MakeHeader(&types.Header{
ChainID: evidenceChainID,
@ -512,7 +534,7 @@ func makeLunaticEvidence(
blockID := factory.MakeBlockIDWithHash(conflictingHeader.Hash())
voteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), conflictingVals)
commit, err := factory.MakeCommit(blockID, height, 1, voteSet, conflictingPrivVals, defaultEvidenceTime)
commit, err := factory.MakeCommit(ctx, blockID, height, 1, voteSet, conflictingPrivVals, defaultEvidenceTime)
require.NoError(t, err)
ev = &types.LightClientAttackEvidence{
ConflictingBlock: &types.LightBlock{
@ -537,9 +559,9 @@ func makeLunaticEvidence(
ValidatorSet: commonValSet,
}
trustedBlockID := factory.MakeBlockIDWithHash(trustedHeader.Hash())
trustedVals, privVals := factory.RandValidatorSet(totalVals, defaultVotingPower)
trustedVals, privVals := factory.RandValidatorSet(ctx, totalVals, defaultVotingPower)
trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), trustedVals)
trustedCommit, err := factory.MakeCommit(trustedBlockID, height, 1, trustedVoteSet, privVals, defaultEvidenceTime)
trustedCommit, err := factory.MakeCommit(ctx, trustedBlockID, height, 1, trustedVoteSet, privVals, defaultEvidenceTime)
require.NoError(t, err)
trusted = &types.LightBlock{
SignedHeader: &types.SignedHeader{
@ -552,9 +574,11 @@ func makeLunaticEvidence(
}
func makeVote(
ctx context.Context,
t *testing.T, val types.PrivValidator, chainID string, valIndex int32, height int64,
round int32, step int, blockID types.BlockID, time time.Time) *types.Vote {
pubKey, err := val.GetPubKey(context.Background())
round int32, step int, blockID types.BlockID, time time.Time,
) *types.Vote {
pubKey, err := val.GetPubKey(ctx)
require.NoError(t, err)
v := &types.Vote{
ValidatorAddress: pubKey.Address(),
@ -567,7 +591,7 @@ func makeVote(
}
vpb := v.ToProto()
err = val.SignVote(context.Background(), chainID, vpb)
err = val.SignVote(ctx, chainID, vpb)
if err != nil {
panic(err)
}
@ -591,12 +615,11 @@ func makeBlockID(hash []byte, partSetSize uint32, partSetHash []byte) types.Bloc
}
}
func orderPrivValsByValSet(
t *testing.T, vals *types.ValidatorSet, privVals []types.PrivValidator) []types.PrivValidator {
func orderPrivValsByValSet(ctx context.Context, t *testing.T, vals *types.ValidatorSet, privVals []types.PrivValidator) []types.PrivValidator {
output := make([]types.PrivValidator, len(privVals))
for idx, v := range vals.Validators {
for _, p := range privVals {
pubKey, err := p.GetPubKey(context.Background())
pubKey, err := p.GetPubKey(ctx)
require.NoError(t, err)
if bytes.Equal(v.Address, pubKey.Address()) {
output[idx] = p


+ 9
- 3
internal/libs/queue/queue_test.go View File

@ -125,6 +125,9 @@ func TestClose(t *testing.T) {
}
func TestWait(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
q := mustQueue(t, Options{SoftQuota: 2, HardLimit: 2})
// A wait on an empty queue should time out.
@ -141,10 +144,13 @@ func TestWait(t *testing.T) {
// A wait on a non-empty queue should report an item.
t.Run("WaitNonEmpty", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
const input = "figgy pudding"
q.mustAdd(input)
got, err := q.Wait(context.Background())
got, err := q.Wait(ctx)
if err != nil {
t.Errorf("Wait: unexpected error: %v", err)
} else if got != input {
@ -159,7 +165,7 @@ func TestWait(t *testing.T) {
done := make(chan struct{})
go func() {
defer close(done)
got, err := q.Wait(context.Background())
got, err := q.Wait(ctx)
if err != nil {
t.Errorf("Wait: unexpected error: %w", err)
} else if got != input {
@ -176,7 +182,7 @@ func TestWait(t *testing.T) {
done := make(chan struct{})
go func() {
defer close(done)
got, err := q.Wait(context.Background())
got, err := q.Wait(ctx)
if err != ErrQueueClosed {
t.Errorf("Wait: got (%v, %v), want %v", got, err, ErrQueueClosed)
}


+ 1
- 1
internal/mempool/mempool_bench_test.go View File

@ -29,6 +29,6 @@ func BenchmarkTxMempool_CheckTx(b *testing.B) {
tx := []byte(fmt.Sprintf("%X=%d", prefix, priority))
b.StartTimer()
require.NoError(b, txmp.CheckTx(context.Background(), tx, nil, TxInfo{}))
require.NoError(b, txmp.CheckTx(ctx, tx, nil, TxInfo{}))
}
}

+ 6
- 6
internal/mempool/mempool_test.go View File

@ -361,13 +361,13 @@ func TestTxMempool_CheckTxExceedsMaxSize(t *testing.T) {
_, err := rng.Read(tx)
require.NoError(t, err)
require.Error(t, txmp.CheckTx(context.Background(), tx, nil, TxInfo{SenderID: 0}))
require.Error(t, txmp.CheckTx(ctx, tx, nil, TxInfo{SenderID: 0}))
tx = make([]byte, txmp.config.MaxTxBytes-1)
_, err = rng.Read(tx)
require.NoError(t, err)
require.NoError(t, txmp.CheckTx(context.Background(), tx, nil, TxInfo{SenderID: 0}))
require.NoError(t, txmp.CheckTx(ctx, tx, nil, TxInfo{SenderID: 0}))
}
func TestTxMempool_CheckTxSamePeer(t *testing.T) {
@ -384,8 +384,8 @@ func TestTxMempool_CheckTxSamePeer(t *testing.T) {
tx := []byte(fmt.Sprintf("sender-0=%X=%d", prefix, 50))
require.NoError(t, txmp.CheckTx(context.Background(), tx, nil, TxInfo{SenderID: peerID}))
require.Error(t, txmp.CheckTx(context.Background(), tx, nil, TxInfo{SenderID: peerID}))
require.NoError(t, txmp.CheckTx(ctx, tx, nil, TxInfo{SenderID: peerID}))
require.Error(t, txmp.CheckTx(ctx, tx, nil, TxInfo{SenderID: peerID}))
}
func TestTxMempool_CheckTxSameSender(t *testing.T) {
@ -407,9 +407,9 @@ func TestTxMempool_CheckTxSameSender(t *testing.T) {
tx1 := []byte(fmt.Sprintf("sender-0=%X=%d", prefix1, 50))
tx2 := []byte(fmt.Sprintf("sender-0=%X=%d", prefix2, 50))
require.NoError(t, txmp.CheckTx(context.Background(), tx1, nil, TxInfo{SenderID: peerID}))
require.NoError(t, txmp.CheckTx(ctx, tx1, nil, TxInfo{SenderID: peerID}))
require.Equal(t, 1, txmp.Size())
require.NoError(t, txmp.CheckTx(context.Background(), tx2, nil, TxInfo{SenderID: peerID}))
require.NoError(t, txmp.CheckTx(ctx, tx2, nil, TxInfo{SenderID: peerID}))
require.Equal(t, 1, txmp.Size())
}


+ 4
- 4
internal/proxy/app_conn_test.go View File

@ -133,13 +133,13 @@ func BenchmarkEcho(b *testing.B) {
}
func TestInfo(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
sockPath := fmt.Sprintf("unix:///tmp/echo_%v.sock", tmrand.Str(6))
logger := log.TestingLogger()
clientCreator := abciclient.NewRemoteCreator(logger, sockPath, SOCKET, true)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// Start server
s := server.NewSocketServer(logger.With("module", "abci-server"), sockPath, kvstore.NewApplication())
require.NoError(t, s.Start(ctx), "Error starting socket server")
@ -154,7 +154,7 @@ func TestInfo(t *testing.T) {
proxy := newAppConnTest(cli)
t.Log("Connected")
resInfo, err := proxy.InfoSync(context.Background(), RequestInfo)
resInfo, err := proxy.InfoSync(ctx, RequestInfo)
require.NoError(t, err)
if resInfo.Data != "{\"size\":0}" {


+ 3
- 2
internal/state/execution_test.go View File

@ -168,7 +168,8 @@ func TestBeginBlockByzantineValidators(t *testing.T) {
}
// we don't need to worry about validating the evidence as long as they pass validate basic
dve := types.NewMockDuplicateVoteEvidenceWithValidator(3, defaultEvidenceTime, privVal, state.ChainID)
dve, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx, 3, defaultEvidenceTime, privVal, state.ChainID)
require.NoError(t, err)
dve.ValidatorPower = 1000
lcae := &types.LightClientAttackEvidence{
ConflictingBlock: &types.LightBlock{
@ -396,7 +397,7 @@ func TestEndBlockValidatorUpdates(t *testing.T) {
blockExec.SetEventBus(eventBus)
updatesSub, err := eventBus.SubscribeWithArgs(context.Background(), pubsub.SubscribeArgs{
updatesSub, err := eventBus.SubscribeWithArgs(ctx, pubsub.SubscribeArgs{
ClientID: "TestEndBlockValidatorUpdates",
Query: types.EventQueryValidatorSetUpdates,
})


+ 5
- 4
internal/state/helpers_test.go View File

@ -55,7 +55,7 @@ func makeAndCommitGoodBlock(
}
// Simulate a lastCommit for this block from all validators for the next height
commit, err := makeValidCommit(height, blockID, state.Validators, privVals)
commit, err := makeValidCommit(ctx, height, blockID, state.Validators, privVals)
if err != nil {
return state, types.BlockID{}, nil, err
}
@ -89,6 +89,7 @@ func makeAndApplyGoodBlock(
}
func makeValidCommit(
ctx context.Context,
height int64,
blockID types.BlockID,
vals *types.ValidatorSet,
@ -97,7 +98,7 @@ func makeValidCommit(
sigs := make([]types.CommitSig, 0)
for i := 0; i < vals.Size(); i++ {
_, val := vals.GetByIndex(int32(i))
vote, err := factory.MakeVote(privVals[val.Address.String()], chainID, int32(i), height, 0, 2, blockID, time.Now())
vote, err := factory.MakeVote(ctx, privVals[val.Address.String()], chainID, int32(i), height, 0, 2, blockID, time.Now())
if err != nil {
return nil, err
}
@ -264,9 +265,9 @@ func makeRandomStateFromValidatorSet(
}
}
func makeRandomStateFromConsensusParams(consensusParams *types.ConsensusParams,
func makeRandomStateFromConsensusParams(ctx context.Context, consensusParams *types.ConsensusParams,
height, lastHeightConsensusParamsChanged int64) sm.State {
val, _ := factory.RandValidator(true, 10)
val, _ := factory.RandValidator(ctx, true, 10)
valSet := types.NewValidatorSet([]*types.Validator{val})
return sm.State{
LastBlockHeight: height - 1,


+ 4
- 1
internal/state/indexer/block/kv/kv_test.go View File

@ -134,7 +134,10 @@ func TestBlockIndexer(t *testing.T) {
for name, tc := range testCases {
tc := tc
t.Run(name, func(t *testing.T) {
results, err := indexer.Search(context.Background(), tc.q)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
results, err := indexer.Search(ctx, tc.q)
require.NoError(t, err)
require.Equal(t, tc.results, results)
})


+ 4
- 1
internal/state/indexer/sink/kv/kv_test.go View File

@ -151,7 +151,10 @@ func TestBlockFuncs(t *testing.T) {
for name, tc := range testCases {
tc := tc
t.Run(name, func(t *testing.T) {
results, err := indexer.SearchBlockEvents(context.Background(), tc.q)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
results, err := indexer.SearchBlockEvents(ctx, tc.q)
require.NoError(t, err)
require.Equal(t, tc.results, results)
})


+ 17
- 5
internal/state/rollback_test.go View File

@ -1,6 +1,7 @@
package state_test
import (
"context"
"testing"
"github.com/stretchr/testify/require"
@ -18,8 +19,11 @@ func TestRollback(t *testing.T) {
height int64 = 100
nextHeight int64 = 101
)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
blockStore := &mocks.BlockStore{}
stateStore := setupStateStore(t, height)
stateStore := setupStateStore(ctx, t, height)
initialState, err := stateStore.Load()
require.NoError(t, err)
@ -78,7 +82,11 @@ func TestRollbackNoState(t *testing.T) {
func TestRollbackNoBlocks(t *testing.T) {
const height = int64(100)
stateStore := setupStateStore(t, height)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stateStore := setupStateStore(ctx, t, height)
blockStore := &mocks.BlockStore{}
blockStore.On("Height").Return(height)
blockStore.On("LoadBlockMeta", height-1).Return(nil)
@ -90,7 +98,11 @@ func TestRollbackNoBlocks(t *testing.T) {
func TestRollbackDifferentStateHeight(t *testing.T) {
const height = int64(100)
stateStore := setupStateStore(t, height)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stateStore := setupStateStore(ctx, t, height)
blockStore := &mocks.BlockStore{}
blockStore.On("Height").Return(height + 2)
@ -99,9 +111,9 @@ func TestRollbackDifferentStateHeight(t *testing.T) {
require.Equal(t, err.Error(), "statestore height (100) is not one below or equal to blockstore height (102)")
}
func setupStateStore(t *testing.T, height int64) state.Store {
func setupStateStore(ctx context.Context, t *testing.T, height int64) state.Store {
stateStore := state.NewStore(dbm.NewMemDB())
valSet, _ := factory.RandValidatorSet(5, 10)
valSet, _ := factory.RandValidatorSet(ctx, 5, 10)
params := types.DefaultConsensusParams()
params.Version.AppVersion = 10


+ 18
- 8
internal/state/store_test.go View File

@ -1,6 +1,7 @@
package state_test
import (
"context"
"fmt"
"os"
"testing"
@ -26,11 +27,14 @@ const (
)
func TestStoreBootstrap(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stateDB := dbm.NewMemDB()
stateStore := sm.NewStore(stateDB)
val, _ := factory.RandValidator(true, 10)
val2, _ := factory.RandValidator(true, 10)
val3, _ := factory.RandValidator(true, 10)
val, _ := factory.RandValidator(ctx, true, 10)
val2, _ := factory.RandValidator(ctx, true, 10)
val3, _ := factory.RandValidator(ctx, true, 10)
vals := types.NewValidatorSet([]*types.Validator{val, val2, val3})
bootstrapState := makeRandomStateFromValidatorSet(vals, 100, 100)
err := stateStore.Bootstrap(bootstrapState)
@ -52,11 +56,14 @@ func TestStoreBootstrap(t *testing.T) {
}
func TestStoreLoadValidators(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stateDB := dbm.NewMemDB()
stateStore := sm.NewStore(stateDB)
val, _ := factory.RandValidator(true, 10)
val2, _ := factory.RandValidator(true, 10)
val3, _ := factory.RandValidator(true, 10)
val, _ := factory.RandValidator(ctx, true, 10)
val2, _ := factory.RandValidator(ctx, true, 10)
val3, _ := factory.RandValidator(ctx, true, 10)
vals := types.NewValidatorSet([]*types.Validator{val, val2, val3})
// 1) LoadValidators loads validators using a height where they were last changed
@ -141,9 +148,12 @@ func BenchmarkLoadValidators(b *testing.B) {
}
func TestStoreLoadConsensusParams(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stateDB := dbm.NewMemDB()
stateStore := sm.NewStore(stateDB)
err := stateStore.Save(makeRandomStateFromConsensusParams(types.DefaultConsensusParams(), 1, 1))
err := stateStore.Save(makeRandomStateFromConsensusParams(ctx, types.DefaultConsensusParams(), 1, 1))
require.NoError(t, err)
params, err := stateStore.LoadConsensusParams(1)
require.NoError(t, err)
@ -153,7 +163,7 @@ func TestStoreLoadConsensusParams(t *testing.T) {
// it should save a pointer to the params at height 1
differentParams := types.DefaultConsensusParams()
differentParams.Block.MaxBytes = 20000
err = stateStore.Save(makeRandomStateFromConsensusParams(differentParams, 10, 1))
err = stateStore.Save(makeRandomStateFromConsensusParams(ctx, differentParams, 10, 1))
require.NoError(t, err)
res, err := stateStore.LoadConsensusParams(10)
require.NoError(t, err)


+ 5
- 2
internal/state/test/factory/block.go View File

@ -1,6 +1,7 @@
package factory
import (
"context"
"time"
sm "github.com/tendermint/tendermint/internal/state"
@ -8,7 +9,7 @@ import (
"github.com/tendermint/tendermint/types"
)
func MakeBlocks(n int, state *sm.State, privVal types.PrivValidator) ([]*types.Block, error) {
func MakeBlocks(ctx context.Context, n int, state *sm.State, privVal types.PrivValidator) ([]*types.Block, error) {
blocks := make([]*types.Block, n)
var (
@ -20,7 +21,7 @@ func MakeBlocks(n int, state *sm.State, privVal types.PrivValidator) ([]*types.B
for i := 0; i < n; i++ {
height := int64(i + 1)
block, parts, err := makeBlockAndPartSet(*state, prevBlock, prevBlockMeta, privVal, height)
block, parts, err := makeBlockAndPartSet(ctx, *state, prevBlock, prevBlockMeta, privVal, height)
if err != nil {
return nil, err
}
@ -55,6 +56,7 @@ func MakeBlock(state sm.State, height int64, c *types.Commit) (*types.Block, err
}
func makeBlockAndPartSet(
ctx context.Context,
state sm.State,
lastBlock *types.Block,
lastBlockMeta *types.BlockMeta,
@ -64,6 +66,7 @@ func makeBlockAndPartSet(
lastCommit := types.NewCommit(height-1, 0, types.BlockID{}, nil)
if height > 1 {
vote, _ := factory.MakeVote(
ctx,
privVal,
lastBlock.Header.ChainID,
1, lastBlock.Header.Height, 0, 2,


+ 6
- 2
internal/state/validation_test.go View File

@ -148,6 +148,7 @@ func TestValidateBlockCommit(t *testing.T) {
*/
// should be height-1 instead of height
wrongHeightVote, err := testfactory.MakeVote(
ctx,
privVals[proposerAddr.String()],
chainID,
1,
@ -205,6 +206,7 @@ func TestValidateBlockCommit(t *testing.T) {
wrongSigsCommit is fine except for the extra bad precommit
*/
goodVote, err := testfactory.MakeVote(
ctx,
privVals[proposerAddr.String()],
chainID,
1,
@ -284,8 +286,9 @@ func TestValidateBlockEvidence(t *testing.T) {
var currentBytes int64
// more bytes than the maximum allowed for evidence
for currentBytes <= maxBytesEvidence {
newEv := types.NewMockDuplicateVoteEvidenceWithValidator(height, time.Now(),
newEv, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx, height, time.Now(),
privVals[proposerAddr.String()], chainID)
require.NoError(t, err)
evidence = append(evidence, newEv)
currentBytes += int64(len(newEv.Bytes()))
}
@ -306,8 +309,9 @@ func TestValidateBlockEvidence(t *testing.T) {
var currentBytes int64
// precisely the amount of allowed evidence
for {
newEv := types.NewMockDuplicateVoteEvidenceWithValidator(height, defaultEvidenceTime,
newEv, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx, height, defaultEvidenceTime,
privVals[proposerAddr.String()], chainID)
require.NoError(t, err)
currentBytes += int64(len(newEv.Bytes()))
if currentBytes >= maxBytesEvidence {
break


+ 30
- 10
internal/statesync/block_queue_test.go View File

@ -1,6 +1,7 @@
package statesync
import (
"context"
"math/rand"
"sync"
"testing"
@ -22,6 +23,9 @@ var (
)
func TestBlockQueueBasic(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
peerID, err := types.NewNodeID("0011223344556677889900112233445566778899")
require.NoError(t, err)
@ -35,7 +39,7 @@ func TestBlockQueueBasic(t *testing.T) {
for {
select {
case height := <-queue.nextHeight():
queue.add(mockLBResp(t, peerID, height, endTime))
queue.add(mockLBResp(ctx, t, peerID, height, endTime))
case <-queue.done():
wg.Done()
return
@ -69,6 +73,9 @@ loop:
// Test with spurious failures and retries
func TestBlockQueueWithFailures(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
peerID, err := types.NewNodeID("0011223344556677889900112233445566778899")
require.NoError(t, err)
@ -85,7 +92,7 @@ func TestBlockQueueWithFailures(t *testing.T) {
if rand.Intn(failureRate) == 0 {
queue.retry(height)
} else {
queue.add(mockLBResp(t, peerID, height, endTime))
queue.add(mockLBResp(ctx, t, peerID, height, endTime))
}
case <-queue.done():
wg.Done()
@ -125,6 +132,9 @@ func TestBlockQueueBlocks(t *testing.T) {
expectedHeight := startHeight
retryHeight := stopHeight + 2
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
loop:
for {
select {
@ -132,7 +142,7 @@ loop:
require.Equal(t, height, expectedHeight)
require.GreaterOrEqual(t, height, stopHeight)
expectedHeight--
queue.add(mockLBResp(t, peerID, height, endTime))
queue.add(mockLBResp(ctx, t, peerID, height, endTime))
case <-time.After(1 * time.Second):
if expectedHeight >= stopHeight {
t.Fatalf("expected next height %d", expectedHeight)
@ -171,12 +181,15 @@ func TestBlockQueueAcceptsNoMoreBlocks(t *testing.T) {
queue := newBlockQueue(startHeight, stopHeight, 1, stopTime, 1)
defer queue.close()
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
loop:
for {
select {
case height := <-queue.nextHeight():
require.GreaterOrEqual(t, height, stopHeight)
queue.add(mockLBResp(t, peerID, height, endTime))
queue.add(mockLBResp(ctx, t, peerID, height, endTime))
case <-time.After(1 * time.Second):
break loop
}
@ -184,7 +197,7 @@ loop:
require.Len(t, queue.pending, int(startHeight-stopHeight)+1)
queue.add(mockLBResp(t, peerID, stopHeight-1, endTime))
queue.add(mockLBResp(ctx, t, peerID, stopHeight-1, endTime))
require.Len(t, queue.pending, int(startHeight-stopHeight)+1)
}
@ -197,6 +210,9 @@ func TestBlockQueueStopTime(t *testing.T) {
queue := newBlockQueue(startHeight, stopHeight, 1, stopTime, 1)
wg := &sync.WaitGroup{}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
baseTime := stopTime.Add(-50 * time.Second)
// asynchronously fetch blocks and add it to the queue
@ -207,7 +223,7 @@ func TestBlockQueueStopTime(t *testing.T) {
select {
case height := <-queue.nextHeight():
blockTime := baseTime.Add(time.Duration(height) * time.Second)
queue.add(mockLBResp(t, peerID, height, blockTime))
queue.add(mockLBResp(ctx, t, peerID, height, blockTime))
case <-queue.done():
wg.Done()
return
@ -241,6 +257,9 @@ func TestBlockQueueInitialHeight(t *testing.T) {
queue := newBlockQueue(startHeight, stopHeight, initialHeight, stopTime, 1)
wg := &sync.WaitGroup{}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// asynchronously fetch blocks and add it to the queue
for i := 0; i <= numWorkers; i++ {
wg.Add(1)
@ -249,7 +268,7 @@ func TestBlockQueueInitialHeight(t *testing.T) {
select {
case height := <-queue.nextHeight():
require.GreaterOrEqual(t, height, initialHeight)
queue.add(mockLBResp(t, peerID, height, endTime))
queue.add(mockLBResp(ctx, t, peerID, height, endTime))
case <-queue.done():
wg.Done()
return
@ -273,9 +292,10 @@ loop:
}
}
func mockLBResp(t *testing.T, peer types.NodeID, height int64, time time.Time) lightBlockResponse {
vals, pv := factory.RandValidatorSet(3, 10)
_, _, lb := mockLB(t, height, time, factory.MakeBlockID(), vals, pv)
func mockLBResp(ctx context.Context, t *testing.T, peer types.NodeID, height int64, time time.Time) lightBlockResponse {
t.Helper()
vals, pv := factory.RandValidatorSet(ctx, 3, 10)
_, _, lb := mockLB(ctx, t, height, time, factory.MakeBlockID(), vals, pv)
return lightBlockResponse{
block: lb,
peer: peer,


+ 7
- 4
internal/statesync/dispatcher_test.go View File

@ -54,7 +54,7 @@ func TestDispatcherBasic(t *testing.T) {
wg.Add(1)
go func(height int64) {
defer wg.Done()
lb, err := d.LightBlock(context.Background(), height, peers[height-1])
lb, err := d.LightBlock(ctx, height, peers[height-1])
require.NoError(t, err)
require.NotNil(t, lb)
require.Equal(t, lb.Height, height)
@ -133,7 +133,7 @@ func TestDispatcherProviders(t *testing.T) {
for i, p := range providers {
assert.Equal(t, string(peers[i]), p.String(), i)
lb, err := p.LightBlock(context.Background(), 10)
lb, err := p.LightBlock(ctx, 10)
assert.NoError(t, err)
assert.NotNil(t, lb)
}
@ -207,7 +207,10 @@ func TestEmptyPeerListReturnsWhenContextCanceled(t *testing.T) {
peerList := newPeerList()
require.Zero(t, peerList.Len())
doneCh := make(chan struct{})
ctx := context.Background()
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
wrapped, cancel := context.WithCancel(ctx)
go func() {
peerList.Pop(wrapped)
@ -304,7 +307,7 @@ func handleRequests(ctx context.Context, t *testing.T, d *Dispatcher, ch chan p2
case request := <-ch:
height := request.Message.(*ssproto.LightBlockRequest).Height
peer := request.To
resp := mockLBResp(t, peer, int64(height), time.Now())
resp := mockLBResp(ctx, t, peer, int64(height), time.Now())
block, _ := resp.block.ToProto()
require.NoError(t, d.Respond(block, resp.peer))
case <-ctx.Done():


+ 16
- 14
internal/statesync/reactor_test.go View File

@ -193,7 +193,7 @@ func TestReactor_Sync(t *testing.T) {
const snapshotHeight = 7
rts := setup(ctx, t, nil, nil, nil, 2)
chain := buildLightBlockChain(t, 1, 10, time.Now())
chain := buildLightBlockChain(ctx, t, 1, 10, time.Now())
// app accepts any snapshot
rts.conn.On("OfferSnapshotSync", ctx, mock.AnythingOfType("types.RequestOfferSnapshot")).
Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ACCEPT}, nil)
@ -422,8 +422,8 @@ func TestReactor_LightBlockResponse(t *testing.T) {
h := factory.MakeRandomHeader()
h.Height = height
blockID := factory.MakeBlockIDWithHash(h.Hash())
vals, pv := factory.RandValidatorSet(1, 10)
vote, err := factory.MakeVote(pv[0], h.ChainID, 0, h.Height, 0, 2,
vals, pv := factory.RandValidatorSet(ctx, 1, 10)
vote, err := factory.MakeVote(ctx, pv[0], h.ChainID, 0, h.Height, 0, 2,
blockID, factory.DefaultTestTime)
require.NoError(t, err)
@ -485,7 +485,7 @@ func TestReactor_BlockProviders(t *testing.T) {
closeCh := make(chan struct{})
defer close(closeCh)
chain := buildLightBlockChain(t, 1, 10, time.Now())
chain := buildLightBlockChain(ctx, t, 1, 10, time.Now())
go handleLightBlockRequests(ctx, t, chain, rts.blockOutCh, rts.blockInCh, closeCh, 0)
peers := rts.reactor.peers.All()
@ -544,7 +544,7 @@ func TestReactor_StateProviderP2P(t *testing.T) {
closeCh := make(chan struct{})
defer close(closeCh)
chain := buildLightBlockChain(t, 1, 10, time.Now())
chain := buildLightBlockChain(ctx, t, 1, 10, time.Now())
go handleLightBlockRequests(ctx, t, chain, rts.blockOutCh, rts.blockInCh, closeCh, 0)
go handleConsensusParamsRequest(ctx, t, rts.paramsOutCh, rts.paramsInCh, closeCh)
@ -630,7 +630,7 @@ func TestReactor_Backfill(t *testing.T) {
return nil
})
chain := buildLightBlockChain(t, stopHeight-1, startHeight+1, stopTime)
chain := buildLightBlockChain(ctx, t, stopHeight-1, startHeight+1, stopTime)
closeCh := make(chan struct{})
defer close(closeCh)
@ -713,8 +713,8 @@ func handleLightBlockRequests(
} else {
switch errorCount % 3 {
case 0: // send a different block
vals, pv := factory.RandValidatorSet(3, 10)
_, _, lb := mockLB(t, int64(msg.Height), factory.DefaultTestTime, factory.MakeBlockID(), vals, pv)
vals, pv := factory.RandValidatorSet(ctx, 3, 10)
_, _, lb := mockLB(ctx, t, int64(msg.Height), factory.DefaultTestTime, factory.MakeBlockID(), vals, pv)
differntLB, err := lb.ToProto()
require.NoError(t, err)
sending <- p2p.Envelope{
@ -777,22 +777,24 @@ func handleConsensusParamsRequest(
}
}
func buildLightBlockChain(t *testing.T, fromHeight, toHeight int64, startTime time.Time) map[int64]*types.LightBlock {
func buildLightBlockChain(ctx context.Context, t *testing.T, fromHeight, toHeight int64, startTime time.Time) map[int64]*types.LightBlock {
t.Helper()
chain := make(map[int64]*types.LightBlock, toHeight-fromHeight)
lastBlockID := factory.MakeBlockID()
blockTime := startTime.Add(time.Duration(fromHeight-toHeight) * time.Minute)
vals, pv := factory.RandValidatorSet(3, 10)
vals, pv := factory.RandValidatorSet(ctx, 3, 10)
for height := fromHeight; height < toHeight; height++ {
vals, pv, chain[height] = mockLB(t, height, blockTime, lastBlockID, vals, pv)
vals, pv, chain[height] = mockLB(ctx, t, height, blockTime, lastBlockID, vals, pv)
lastBlockID = factory.MakeBlockIDWithHash(chain[height].Header.Hash())
blockTime = blockTime.Add(1 * time.Minute)
}
return chain
}
func mockLB(t *testing.T, height int64, time time.Time, lastBlockID types.BlockID,
func mockLB(ctx context.Context, t *testing.T, height int64, time time.Time, lastBlockID types.BlockID,
currentVals *types.ValidatorSet, currentPrivVals []types.PrivValidator,
) (*types.ValidatorSet, []types.PrivValidator, *types.LightBlock) {
t.Helper()
header, err := factory.MakeHeader(&types.Header{
Height: height,
LastBlockID: lastBlockID,
@ -800,13 +802,13 @@ func mockLB(t *testing.T, height int64, time time.Time, lastBlockID types.BlockI
})
header.Version.App = testAppVersion
require.NoError(t, err)
nextVals, nextPrivVals := factory.RandValidatorSet(3, 10)
nextVals, nextPrivVals := factory.RandValidatorSet(ctx, 3, 10)
header.ValidatorsHash = currentVals.Hash()
header.NextValidatorsHash = nextVals.Hash()
header.ConsensusHash = types.DefaultConsensusParams().HashConsensusParams()
lastBlockID = factory.MakeBlockIDWithHash(header.Hash())
voteSet := types.NewVoteSet(factory.DefaultTestChainID, height, 0, tmproto.PrecommitType, currentVals)
commit, err := factory.MakeCommit(lastBlockID, height, 0, voteSet, currentPrivVals, time)
commit, err := factory.MakeCommit(ctx, lastBlockID, height, 0, voteSet, currentPrivVals, time)
require.NoError(t, err)
return nextVals, nextPrivVals, &types.LightBlock{
SignedHeader: &types.SignedHeader{


+ 5
- 6
internal/test/factory/commit.go View File

@ -9,12 +9,11 @@ import (
"github.com/tendermint/tendermint/types"
)
func MakeCommit(blockID types.BlockID, height int64, round int32,
voteSet *types.VoteSet, validators []types.PrivValidator, now time.Time) (*types.Commit, error) {
func MakeCommit(ctx context.Context, blockID types.BlockID, height int64, round int32, voteSet *types.VoteSet, validators []types.PrivValidator, now time.Time) (*types.Commit, error) {
// all sign
for i := 0; i < len(validators); i++ {
pubKey, err := validators[i].GetPubKey(context.Background())
pubKey, err := validators[i].GetPubKey(ctx)
if err != nil {
return nil, fmt.Errorf("can't get pubkey: %w", err)
}
@ -28,7 +27,7 @@ func MakeCommit(blockID types.BlockID, height int64, round int32,
Timestamp: now,
}
_, err = signAddVote(validators[i], vote, voteSet)
_, err = signAddVote(ctx, validators[i], vote, voteSet)
if err != nil {
return nil, err
}
@ -37,9 +36,9 @@ func MakeCommit(blockID types.BlockID, height int64, round int32,
return voteSet.MakeCommit(), nil
}
func signAddVote(privVal types.PrivValidator, vote *types.Vote, voteSet *types.VoteSet) (signed bool, err error) {
func signAddVote(ctx context.Context, privVal types.PrivValidator, vote *types.Vote, voteSet *types.VoteSet) (signed bool, err error) {
v := vote.ToProto()
err = privVal.SignVote(context.Background(), voteSet.ChainID(), v)
err = privVal.SignVote(ctx, voteSet.ChainID(), v)
if err != nil {
return false, err
}


+ 5
- 2
internal/test/factory/genesis.go View File

@ -1,6 +1,7 @@
package factory
import (
"context"
"sort"
"github.com/tendermint/tendermint/config"
@ -9,15 +10,17 @@ import (
)
func RandGenesisDoc(
ctx context.Context,
cfg *config.Config,
numValidators int,
randPower bool,
minPower int64) (*types.GenesisDoc, []types.PrivValidator) {
minPower int64,
) (*types.GenesisDoc, []types.PrivValidator) {
validators := make([]types.GenesisValidator, numValidators)
privValidators := make([]types.PrivValidator, numValidators)
for i := 0; i < numValidators; i++ {
val, privVal := RandValidator(randPower, minPower)
val, privVal := RandValidator(ctx, randPower, minPower)
validators[i] = types.GenesisValidator{
PubKey: val.PubKey,
Power: val.VotingPower,


+ 4
- 4
internal/test/factory/validator.go View File

@ -9,14 +9,14 @@ import (
"github.com/tendermint/tendermint/types"
)
func RandValidator(randPower bool, minPower int64) (*types.Validator, types.PrivValidator) {
func RandValidator(ctx context.Context, randPower bool, minPower int64) (*types.Validator, types.PrivValidator) {
privVal := types.NewMockPV()
votePower := minPower
if randPower {
// nolint:gosec // G404: Use of weak random number generator
votePower += int64(rand.Uint32())
}
pubKey, err := privVal.GetPubKey(context.Background())
pubKey, err := privVal.GetPubKey(ctx)
if err != nil {
panic(fmt.Errorf("could not retrieve pubkey %w", err))
}
@ -24,14 +24,14 @@ func RandValidator(randPower bool, minPower int64) (*types.Validator, types.Priv
return val, privVal
}
func RandValidatorSet(numValidators int, votingPower int64) (*types.ValidatorSet, []types.PrivValidator) {
func RandValidatorSet(ctx context.Context, numValidators int, votingPower int64) (*types.ValidatorSet, []types.PrivValidator) {
var (
valz = make([]*types.Validator, numValidators)
privValidators = make([]types.PrivValidator, numValidators)
)
for i := 0; i < numValidators; i++ {
val, privValidator := RandValidator(false, votingPower)
val, privValidator := RandValidator(ctx, false, votingPower)
valz[i] = val
privValidators[i] = privValidator
}


+ 3
- 2
internal/test/factory/vote.go View File

@ -9,6 +9,7 @@ import (
)
func MakeVote(
ctx context.Context,
val types.PrivValidator,
chainID string,
valIndex int32,
@ -18,7 +19,7 @@ func MakeVote(
blockID types.BlockID,
time time.Time,
) (*types.Vote, error) {
pubKey, err := val.GetPubKey(context.Background())
pubKey, err := val.GetPubKey(ctx)
if err != nil {
return nil, err
}
@ -33,7 +34,7 @@ func MakeVote(
}
vpb := v.ToProto()
err = val.SignVote(context.Background(), chainID, vpb)
err = val.SignVote(ctx, chainID, vpb)
if err != nil {
panic(err)
}


+ 11
- 5
light/client_benchmark_test.go View File

@ -61,14 +61,17 @@ func (impl *providerBenchmarkImpl) ReportEvidence(_ context.Context, _ types.Evi
}
func BenchmarkSequence(b *testing.B) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
headers, vals, _ := genLightBlocksWithKeys(chainID, 1000, 100, 1, bTime)
benchmarkFullNode := newProviderBenchmarkImpl(headers, vals)
genesisBlock, _ := benchmarkFullNode.LightBlock(context.Background(), 1)
genesisBlock, _ := benchmarkFullNode.LightBlock(ctx, 1)
logger := log.NewTestingLogger(b)
c, err := light.NewClient(
context.Background(),
ctx,
chainID,
light.TrustOptions{
Period: 24 * time.Hour,
@ -87,7 +90,7 @@ func BenchmarkSequence(b *testing.B) {
b.ResetTimer()
for n := 0; n < b.N; n++ {
_, err = c.VerifyLightBlockAtHeight(context.Background(), 1000, bTime.Add(1000*time.Minute))
_, err = c.VerifyLightBlockAtHeight(ctx, 1000, bTime.Add(1000*time.Minute))
if err != nil {
b.Fatal(err)
}
@ -95,9 +98,12 @@ func BenchmarkSequence(b *testing.B) {
}
func BenchmarkBisection(b *testing.B) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
headers, vals, _ := genLightBlocksWithKeys(chainID, 1000, 100, 1, bTime)
benchmarkFullNode := newProviderBenchmarkImpl(headers, vals)
genesisBlock, _ := benchmarkFullNode.LightBlock(context.Background(), 1)
genesisBlock, _ := benchmarkFullNode.LightBlock(ctx, 1)
logger := log.NewTestingLogger(b)
@ -120,7 +126,7 @@ func BenchmarkBisection(b *testing.B) {
b.ResetTimer()
for n := 0; n < b.N; n++ {
_, err = c.VerifyLightBlockAtHeight(context.Background(), 1000, bTime.Add(1000*time.Minute))
_, err = c.VerifyLightBlockAtHeight(ctx, 1000, bTime.Add(1000*time.Minute))
if err != nil {
b.Fatal(err)
}


+ 8
- 5
light/client_test.go View File

@ -109,9 +109,12 @@ func TestValidateTrustOptions(t *testing.T) {
}
func TestClient_SequentialVerification(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
newKeys := genPrivKeys(4)
newVals := newKeys.ToValidators(10, 1)
differentVals, _ := factory.RandValidatorSet(10, 100)
differentVals, _ := factory.RandValidatorSet(ctx, 10, 100)
testCases := []struct {
name string
@ -934,9 +937,12 @@ func TestClientRemovesWitnessIfItSendsUsIncorrectHeader(t *testing.T) {
}
func TestClient_TrustedValidatorSet(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
logger := log.NewTestingLogger(t)
differentVals, _ := factory.RandValidatorSet(10, 100)
differentVals, _ := factory.RandValidatorSet(ctx, 10, 100)
mockBadValSetNode := mockNodeFromHeadersAndVals(
map[int64]*types.SignedHeader{
1: h1,
@ -960,9 +966,6 @@ func TestClient_TrustedValidatorSet(t *testing.T) {
2: vals,
})
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
c, err := light.NewClient(
ctx,
chainID,


+ 1
- 1
light/example_test.go View File

@ -91,7 +91,7 @@ func ExampleClient() {
time.Sleep(2 * time.Second)
// veify the block at height 3
_, err = c.VerifyLightBlockAtHeight(context.Background(), 3, time.Now())
_, err = c.VerifyLightBlockAtHeight(ctx, 3, time.Now())
if err != nil {
stdlog.Fatal(err)
}


+ 7
- 7
light/provider/http/http_test.go View File

@ -59,11 +59,11 @@ func TestProvider(t *testing.T) {
require.NotNil(t, p)
// let it produce some blocks
err = rpcclient.WaitForHeight(c, 10, nil)
err = rpcclient.WaitForHeight(ctx, c, 10, nil)
require.NoError(t, err)
// let's get the highest block
lb, err := p.LightBlock(context.Background(), 0)
lb, err := p.LightBlock(ctx, 0)
require.NoError(t, err)
assert.True(t, lb.Height < 9001, "height=%d", lb.Height)
@ -72,17 +72,17 @@ func TestProvider(t *testing.T) {
// historical queries now work :)
lower := lb.Height - 3
lb, err = p.LightBlock(context.Background(), lower)
lb, err = p.LightBlock(ctx, lower)
require.NoError(t, err)
assert.Equal(t, lower, lb.Height)
// fetching missing heights (both future and pruned) should return appropriate errors
lb, err = p.LightBlock(context.Background(), 9001)
lb, err = p.LightBlock(ctx, 9001)
require.Error(t, err)
require.Nil(t, lb)
assert.Equal(t, provider.ErrHeightTooHigh, err)
lb, err = p.LightBlock(context.Background(), 1)
lb, err = p.LightBlock(ctx, 1)
require.Error(t, err)
require.Nil(t, lb)
assert.Equal(t, provider.ErrLightBlockNotFound, err)
@ -90,7 +90,7 @@ func TestProvider(t *testing.T) {
// if the provider is unable to provide four more blocks then we should return
// an unreliable peer error
for i := 0; i < 4; i++ {
_, err = p.LightBlock(context.Background(), 1)
_, err = p.LightBlock(ctx, 1)
}
assert.IsType(t, provider.ErrUnreliableProvider{}, err)
@ -99,7 +99,7 @@ func TestProvider(t *testing.T) {
cancel()
time.Sleep(10 * time.Second)
lb, err = p.LightBlock(context.Background(), lower+2)
lb, err = p.LightBlock(ctx, lower+2)
// we should see a connection refused
require.Error(t, err)
require.Nil(t, lb)


+ 24
- 8
light/store/db/db_test.go View File

@ -1,6 +1,7 @@
package db
import (
"context"
"sync"
"testing"
"time"
@ -18,6 +19,9 @@ import (
)
func TestLast_FirstLightBlockHeight(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
dbStore := New(dbm.NewMemDB())
// Empty store
@ -30,7 +34,7 @@ func TestLast_FirstLightBlockHeight(t *testing.T) {
assert.EqualValues(t, -1, height)
// 1 key
err = dbStore.SaveLightBlock(randLightBlock(int64(1)))
err = dbStore.SaveLightBlock(randLightBlock(ctx, int64(1)))
require.NoError(t, err)
height, err = dbStore.LastLightBlockHeight()
@ -45,13 +49,16 @@ func TestLast_FirstLightBlockHeight(t *testing.T) {
func Test_SaveLightBlock(t *testing.T) {
dbStore := New(dbm.NewMemDB())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// Empty store
h, err := dbStore.LightBlock(1)
require.Error(t, err)
assert.Nil(t, h)
// 1 key
err = dbStore.SaveLightBlock(randLightBlock(1))
err = dbStore.SaveLightBlock(randLightBlock(ctx, 1))
require.NoError(t, err)
size := dbStore.Size()
@ -80,7 +87,10 @@ func Test_LightBlockBefore(t *testing.T) {
_, _ = dbStore.LightBlockBefore(100)
})
err := dbStore.SaveLightBlock(randLightBlock(int64(2)))
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
err := dbStore.SaveLightBlock(randLightBlock(ctx, int64(2)))
require.NoError(t, err)
h, err := dbStore.LightBlockBefore(3)
@ -96,13 +106,16 @@ func Test_LightBlockBefore(t *testing.T) {
func Test_Prune(t *testing.T) {
dbStore := New(dbm.NewMemDB())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// Empty store
assert.EqualValues(t, 0, dbStore.Size())
err := dbStore.Prune(0)
require.NoError(t, err)
// One header
err = dbStore.SaveLightBlock(randLightBlock(2))
err = dbStore.SaveLightBlock(randLightBlock(ctx, 2))
require.NoError(t, err)
assert.EqualValues(t, 1, dbStore.Size())
@ -117,7 +130,7 @@ func Test_Prune(t *testing.T) {
// Multiple headers
for i := 1; i <= 10; i++ {
err = dbStore.SaveLightBlock(randLightBlock(int64(i)))
err = dbStore.SaveLightBlock(randLightBlock(ctx, int64(i)))
require.NoError(t, err)
}
@ -133,13 +146,16 @@ func Test_Prune(t *testing.T) {
func Test_Concurrency(t *testing.T) {
dbStore := New(dbm.NewMemDB())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var wg sync.WaitGroup
for i := 1; i <= 100; i++ {
wg.Add(1)
go func(i int64) {
defer wg.Done()
err := dbStore.SaveLightBlock(randLightBlock(i))
err := dbStore.SaveLightBlock(randLightBlock(ctx, i))
require.NoError(t, err)
_, err = dbStore.LightBlock(i)
@ -182,8 +198,8 @@ func Test_Concurrency(t *testing.T) {
wg.Wait()
}
func randLightBlock(height int64) *types.LightBlock {
vals, _ := factory.RandValidatorSet(2, 1)
func randLightBlock(ctx context.Context, height int64) *types.LightBlock {
vals, _ := factory.RandValidatorSet(ctx, 2, 1)
return &types.LightBlock{
SignedHeader: &types.SignedHeader{
Header: &types.Header{


+ 8
- 4
node/node_test.go View File

@ -294,7 +294,8 @@ func TestCreateProposalBlock(t *testing.T) {
// than can fit in a block
var currentBytes int64
for currentBytes <= maxEvidenceBytes {
ev := types.NewMockDuplicateVoteEvidenceWithValidator(height, time.Now(), privVals[0], "test-chain")
ev, err := types.NewMockDuplicateVoteEvidenceWithValidator(ctx, height, time.Now(), privVals[0], "test-chain")
require.NoError(t, err)
currentBytes += int64(len(ev.Bytes()))
evidencePool.ReportConflictingVotes(ev.VoteA, ev.VoteB)
}
@ -698,10 +699,13 @@ func state(t *testing.T, nVals int, height int64) (sm.State, dbm.DB, []types.Pri
}
func TestLoadStateFromGenesis(t *testing.T) {
_ = loadStatefromGenesis(t)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
_ = loadStatefromGenesis(ctx, t)
}
func loadStatefromGenesis(t *testing.T) sm.State {
func loadStatefromGenesis(ctx context.Context, t *testing.T) sm.State {
t.Helper()
stateDB := dbm.NewMemDB()
@ -713,7 +717,7 @@ func loadStatefromGenesis(t *testing.T) sm.State {
require.NoError(t, err)
require.True(t, loadedState.IsEmpty())
genDoc, _ := factory.RandGenesisDoc(cfg, 0, false, 10)
genDoc, _ := factory.RandGenesisDoc(ctx, cfg, 0, false, 10)
state, err := loadStateFromDBOrGenesisDocProvider(
stateStore,


+ 21
- 9
privval/file_test.go View File

@ -41,6 +41,9 @@ func TestGenLoadValidator(t *testing.T) {
}
func TestResetValidator(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
tempKeyFile, err := os.CreateTemp("", "priv_validator_key_")
require.NoError(t, err)
tempStateFile, err := os.CreateTemp("", "priv_validator_state_")
@ -59,7 +62,7 @@ func TestResetValidator(t *testing.T) {
randBytes := tmrand.Bytes(tmhash.Size)
blockID := types.BlockID{Hash: randBytes, PartSetHeader: types.PartSetHeader{}}
vote := newVote(privVal.Key.Address, 0, height, round, voteType, blockID)
err = privVal.SignVote(context.Background(), "mychainid", vote.ToProto())
err = privVal.SignVote(ctx, "mychainid", vote.ToProto())
assert.NoError(t, err, "expected no error signing vote")
// priv val after signing is not same as empty
@ -154,6 +157,9 @@ func TestUnmarshalValidatorKey(t *testing.T) {
}
func TestSignVote(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
tempKeyFile, err := os.CreateTemp("", "priv_validator_key_")
require.NoError(t, err)
tempStateFile, err := os.CreateTemp("", "priv_validator_state_")
@ -176,11 +182,12 @@ func TestSignVote(t *testing.T) {
// sign a vote for first time
vote := newVote(privVal.Key.Address, 0, height, round, voteType, block1)
v := vote.ToProto()
err = privVal.SignVote(context.Background(), "mychainid", v)
err = privVal.SignVote(ctx, "mychainid", v)
assert.NoError(t, err, "expected no error signing vote")
// try to sign the same vote again; should be fine
err = privVal.SignVote(context.Background(), "mychainid", v)
err = privVal.SignVote(ctx, "mychainid", v)
assert.NoError(t, err, "expected no error on signing same vote")
// now try some bad votes
@ -192,18 +199,22 @@ func TestSignVote(t *testing.T) {
}
for _, c := range cases {
assert.Error(t, privVal.SignVote(context.Background(), "mychainid", c.ToProto()),
assert.Error(t, privVal.SignVote(ctx, "mychainid", c.ToProto()),
"expected error on signing conflicting vote")
}
// try signing a vote with a different time stamp
sig := vote.Signature
vote.Timestamp = vote.Timestamp.Add(time.Duration(1000))
assert.NoError(t, privVal.SignVote(context.Background(), "mychainid", v))
err = privVal.SignVote(ctx, "mychainid", v)
assert.NoError(t, err)
assert.Equal(t, sig, vote.Signature)
}
func TestSignProposal(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
tempKeyFile, err := os.CreateTemp("", "priv_validator_key_")
require.NoError(t, err)
tempStateFile, err := os.CreateTemp("", "priv_validator_state_")
@ -224,11 +235,12 @@ func TestSignProposal(t *testing.T) {
// sign a proposal for first time
proposal := newProposal(height, round, block1)
pbp := proposal.ToProto()
err = privVal.SignProposal(context.Background(), "mychainid", pbp)
err = privVal.SignProposal(ctx, "mychainid", pbp)
assert.NoError(t, err, "expected no error signing proposal")
// try to sign the same proposal again; should be fine
err = privVal.SignProposal(context.Background(), "mychainid", pbp)
err = privVal.SignProposal(ctx, "mychainid", pbp)
assert.NoError(t, err, "expected no error on signing same proposal")
// now try some bad Proposals
@ -240,14 +252,14 @@ func TestSignProposal(t *testing.T) {
}
for _, c := range cases {
assert.Error(t, privVal.SignProposal(context.Background(), "mychainid", c.ToProto()),
assert.Error(t, privVal.SignProposal(ctx, "mychainid", c.ToProto()),
"expected error on signing conflicting proposal")
}
// try signing a proposal with a different time stamp
sig := proposal.Signature
proposal.Timestamp = proposal.Timestamp.Add(time.Duration(1000))
err = privVal.SignProposal(context.Background(), "mychainid", pbp)
err = privVal.SignProposal(ctx, "mychainid", pbp)
assert.NoError(t, err)
assert.Equal(t, sig, proposal.Signature)
}


+ 3
- 3
rpc/client/examples_test.go View File

@ -43,7 +43,7 @@ func TestHTTPSimple(t *testing.T) {
// Broadcast the transaction and wait for it to commit (rather use
// c.BroadcastTxSync though in production).
bres, err := c.BroadcastTxCommit(context.Background(), tx)
bres, err := c.BroadcastTxCommit(ctx, tx)
require.NoError(t, err)
if err != nil {
log.Fatal(err)
@ -53,7 +53,7 @@ func TestHTTPSimple(t *testing.T) {
}
// Now try to fetch the value for the key
qres, err := c.ABCIQuery(context.Background(), "/key", k)
qres, err := c.ABCIQuery(ctx, "/key", k)
require.NoError(t, err)
require.False(t, qres.Response.IsErr(), "ABCIQuery failed")
require.True(t, bytes.Equal(qres.Response.Key, k),
@ -118,7 +118,7 @@ func TestHTTPBatching(t *testing.T) {
// Now let's query for the original results as a batch
exists := 0
for _, key := range [][]byte{k1, k2} {
_, err := batch.ABCIQuery(context.Background(), "/key", key)
_, err := batch.ABCIQuery(ctx, "/key", key)
if err == nil {
exists++


+ 2
- 2
rpc/client/helpers.go View File

@ -34,13 +34,13 @@ func DefaultWaitStrategy(delta int64) (abort error) {
//
// If waiter is nil, we use DefaultWaitStrategy, but you can also
// provide your own implementation
func WaitForHeight(c StatusClient, h int64, waiter Waiter) error {
func WaitForHeight(ctx context.Context, c StatusClient, h int64, waiter Waiter) error {
if waiter == nil {
waiter = DefaultWaitStrategy
}
delta := int64(1)
for delta > 0 {
s, err := c.Status(context.Background())
s, err := c.Status(ctx)
if err != nil {
return err
}


+ 12
- 4
rpc/client/helpers_test.go View File

@ -1,6 +1,7 @@
package client_test
import (
"context"
"errors"
"strings"
"testing"
@ -14,6 +15,9 @@ import (
)
func TestWaitForHeight(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// test with error result - immediate failure
m := &mock.StatusMock{
Call: mock.Call{
@ -23,9 +27,10 @@ func TestWaitForHeight(t *testing.T) {
r := mock.NewStatusRecorder(m)
// connection failure always leads to error
err := client.WaitForHeight(r, 8, nil)
err := client.WaitForHeight(ctx, r, 8, nil)
require.Error(t, err)
require.Equal(t, "bye", err.Error())
// we called status once to check
require.Equal(t, 1, len(r.Calls))
@ -35,15 +40,17 @@ func TestWaitForHeight(t *testing.T) {
}
// we will not wait for more than 10 blocks
err = client.WaitForHeight(r, 40, nil)
err = client.WaitForHeight(ctx, r, 40, nil)
require.Error(t, err)
require.True(t, strings.Contains(err.Error(), "aborting"))
// we called status once more to check
require.Equal(t, 2, len(r.Calls))
// waiting for the past returns immediately
err = client.WaitForHeight(r, 5, nil)
err = client.WaitForHeight(ctx, r, 5, nil)
require.NoError(t, err)
// we called status once more to check
require.Equal(t, 3, len(r.Calls))
@ -56,8 +63,9 @@ func TestWaitForHeight(t *testing.T) {
}
// we wait for a few blocks
err = client.WaitForHeight(r, 12, myWaiter)
err = client.WaitForHeight(ctx, r, 12, myWaiter)
require.NoError(t, err)
// we called status once to check
require.Equal(t, 5, len(r.Calls))


+ 23
- 14
rpc/client/mock/abci_test.go View File

@ -19,6 +19,9 @@ import (
)
func TestABCIMock(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
key, value := []byte("foo"), []byte("bar")
height := int64(10)
goodTx := types.Tx{0x01, 0xff}
@ -44,12 +47,12 @@ func TestABCIMock(t *testing.T) {
}
// now, let's try to make some calls
_, err := m.ABCIInfo(context.Background())
_, err := m.ABCIInfo(ctx)
require.NoError(t, err)
assert.Equal(t, "foobar", err.Error())
// query always returns the response
_query, err := m.ABCIQueryWithOptions(context.Background(), "/", nil, client.ABCIQueryOptions{Prove: false})
_query, err := m.ABCIQueryWithOptions(ctx, "/", nil, client.ABCIQueryOptions{Prove: false})
query := _query.Response
require.NoError(t, err)
require.NotNil(t, query)
@ -58,18 +61,18 @@ func TestABCIMock(t *testing.T) {
assert.Equal(t, height, query.Height)
// non-commit calls always return errors
_, err = m.BroadcastTxSync(context.Background(), goodTx)
_, err = m.BroadcastTxSync(ctx, goodTx)
require.NoError(t, err)
assert.Equal(t, "must commit", err.Error())
_, err = m.BroadcastTxAsync(context.Background(), goodTx)
_, err = m.BroadcastTxAsync(ctx, goodTx)
require.Error(t, err)
assert.Equal(t, "must commit", err.Error())
// commit depends on the input
_, err = m.BroadcastTxCommit(context.Background(), badTx)
_, err = m.BroadcastTxCommit(ctx, badTx)
require.NoError(t, err)
assert.Equal(t, "bad tx", err.Error())
bres, err := m.BroadcastTxCommit(context.Background(), goodTx)
bres, err := m.BroadcastTxCommit(ctx, goodTx)
require.NoError(t, err, "%+v", err)
assert.EqualValues(t, 0, bres.CheckTx.Code)
assert.EqualValues(t, "stand", bres.CheckTx.Data)
@ -77,6 +80,9 @@ func TestABCIMock(t *testing.T) {
}
func TestABCIRecorder(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// This mock returns errors on everything but Query
m := mock.ABCIMock{
Info: mock.Call{Response: abci.ResponseInfo{
@ -91,11 +97,11 @@ func TestABCIRecorder(t *testing.T) {
require.Equal(t, 0, len(r.Calls))
_, err := r.ABCIInfo(context.Background())
_, err := r.ABCIInfo(ctx)
assert.NoError(t, err, "expected no err on info")
_, err = r.ABCIQueryWithOptions(
context.Background(),
ctx,
"path",
bytes.HexBytes("data"),
client.ABCIQueryOptions{Prove: false},
@ -127,11 +133,11 @@ func TestABCIRecorder(t *testing.T) {
// now add some broadcasts (should all err)
txs := []types.Tx{{1}, {2}, {3}}
_, err = r.BroadcastTxCommit(context.Background(), txs[0])
_, err = r.BroadcastTxCommit(ctx, txs[0])
assert.Error(t, err, "expected err on broadcast")
_, err = r.BroadcastTxSync(context.Background(), txs[1])
_, err = r.BroadcastTxSync(ctx, txs[1])
assert.Error(t, err, "expected err on broadcast")
_, err = r.BroadcastTxAsync(context.Background(), txs[2])
_, err = r.BroadcastTxAsync(ctx, txs[2])
assert.Error(t, err, "expected err on broadcast")
require.Equal(t, 5, len(r.Calls))
@ -159,15 +165,18 @@ func TestABCIApp(t *testing.T) {
app := kvstore.NewApplication()
m := mock.ABCIApp{app}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// get some info
info, err := m.ABCIInfo(context.Background())
info, err := m.ABCIInfo(ctx)
require.NoError(t, err)
assert.Equal(t, `{"size":0}`, info.Response.GetData())
// add a key
key, value := "foo", "bar"
tx := fmt.Sprintf("%s=%s", key, value)
res, err := m.BroadcastTxCommit(context.Background(), types.Tx(tx))
res, err := m.BroadcastTxCommit(ctx, types.Tx(tx))
require.NoError(t, err)
assert.True(t, res.CheckTx.IsOK())
require.NotNil(t, res.DeliverTx)
@ -181,7 +190,7 @@ func TestABCIApp(t *testing.T) {
// check the key
_qres, err := m.ABCIQueryWithOptions(
context.Background(),
ctx,
"/key",
bytes.HexBytes(key),
client.ABCIQueryOptions{Prove: true},


+ 4
- 1
rpc/client/mock/status_test.go View File

@ -14,6 +14,9 @@ import (
)
func TestStatus(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
m := &mock.StatusMock{
Call: mock.Call{
Response: &coretypes.ResultStatus{
@ -39,7 +42,7 @@ func TestStatus(t *testing.T) {
require.Equal(t, 0, len(r.Calls))
// make sure response works proper
status, err := r.Status(context.Background())
status, err := r.Status(ctx)
require.NoError(t, err)
assert.EqualValues(t, "block", status.SyncInfo.LatestBlockHash)
assert.EqualValues(t, 10, status.SyncInfo.LatestBlockHeight)


+ 11
- 5
rpc/client/rpc_test.go View File

@ -302,7 +302,7 @@ func TestClientMethodCalls(t *testing.T) {
apph := status.SyncInfo.LatestBlockHeight + 2 // this is where the tx will be applied to the state
// wait before querying
err = client.WaitForHeight(c, apph, nil)
err = client.WaitForHeight(ctx, c, apph, nil)
require.NoError(t, err)
res, err := c.ABCIQuery(ctx, "/key", k)
qres := res.Response
@ -331,7 +331,7 @@ func TestClientMethodCalls(t *testing.T) {
apph := txh + 1 // this is where the tx will be applied to the state
// wait before querying
err = client.WaitForHeight(c, apph, nil)
err = client.WaitForHeight(ctx, c, apph, nil)
require.NoError(t, err)
_qres, err := c.ABCIQueryWithOptions(ctx, "/key", k, client.ABCIQueryOptions{Prove: false})
@ -411,7 +411,10 @@ func TestClientMethodCalls(t *testing.T) {
// XXX Test proof
})
t.Run("BlockchainInfo", func(t *testing.T) {
err := client.WaitForHeight(c, 10, nil)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
err := client.WaitForHeight(ctx, c, 10, nil)
require.NoError(t, err)
res, err := c.BlockchainInfo(ctx, 0, 0)
@ -520,6 +523,9 @@ func TestClientMethodCalls(t *testing.T) {
})
t.Run("Evidence", func(t *testing.T) {
t.Run("BraodcastDuplicateVote", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
chainID := conf.ChainID()
correct, fakes := makeEvidences(t, pv, chainID)
@ -533,7 +539,7 @@ func TestClientMethodCalls(t *testing.T) {
status, err := c.Status(ctx)
require.NoError(t, err)
err = client.WaitForHeight(c, status.SyncInfo.LatestBlockHeight+2, nil)
err = client.WaitForHeight(ctx, c, status.SyncInfo.LatestBlockHeight+2, nil)
require.NoError(t, err)
ed25519pub := pv.Key.PubKey.(ed25519.PubKey)
@ -862,7 +868,7 @@ func testBatchedJSONRPCCalls(ctx context.Context, t *testing.T, c *rpchttp.HTTP)
require.Equal(t, *bresult2, *r2)
apph := tmmath.MaxInt64(bresult1.Height, bresult2.Height) + 1
err = client.WaitForHeight(c, apph, nil)
err = client.WaitForHeight(ctx, c, apph, nil)
require.NoError(t, err)
q1, err := batch.ABCIQuery(ctx, "/key", k1)


+ 10
- 10
rpc/jsonrpc/jsonrpc_test.go View File

@ -205,11 +205,11 @@ func testWithHTTPClient(ctx context.Context, t *testing.T, cl client.HTTPClient)
assert.Equal(t, got4, val4)
}
func echoViaWS(cl *client.WSClient, val string) (string, error) {
func echoViaWS(ctx context.Context, cl *client.WSClient, val string) (string, error) {
params := map[string]interface{}{
"arg": val,
}
err := cl.Call(context.Background(), "echo", params)
err := cl.Call(ctx, "echo", params)
if err != nil {
return "", err
}
@ -227,11 +227,11 @@ func echoViaWS(cl *client.WSClient, val string) (string, error) {
return result.Value, nil
}
func echoBytesViaWS(cl *client.WSClient, bytes []byte) ([]byte, error) {
func echoBytesViaWS(ctx context.Context, cl *client.WSClient, bytes []byte) ([]byte, error) {
params := map[string]interface{}{
"arg": bytes,
}
err := cl.Call(context.Background(), "echo_bytes", params)
err := cl.Call(ctx, "echo_bytes", params)
if err != nil {
return []byte{}, err
}
@ -249,14 +249,14 @@ func echoBytesViaWS(cl *client.WSClient, bytes []byte) ([]byte, error) {
return result.Value, nil
}
func testWithWSClient(t *testing.T, cl *client.WSClient) {
func testWithWSClient(ctx context.Context, t *testing.T, cl *client.WSClient) {
val := testVal
got, err := echoViaWS(cl, val)
got, err := echoViaWS(ctx, cl, val)
require.NoError(t, err)
assert.Equal(t, got, val)
val2 := randBytes(t)
got2, err := echoBytesViaWS(cl, val2)
got2, err := echoBytesViaWS(ctx, cl, val2)
require.NoError(t, err)
assert.Equal(t, got2, val2)
}
@ -286,7 +286,7 @@ func TestServersAndClientsBasic(t *testing.T) {
err = cl3.Start(ctx)
require.NoError(t, err)
fmt.Printf("=== testing server on %s using WS client", addr)
testWithWSClient(t, cl3)
testWithWSClient(ctx, t, cl3)
cancel()
})
}
@ -311,7 +311,7 @@ func TestWSNewWSRPCFunc(t *testing.T) {
params := map[string]interface{}{
"arg": val,
}
err = cl.Call(context.Background(), "echo_ws", params)
err = cl.Call(ctx, "echo_ws", params)
require.NoError(t, err)
msg := <-cl.ResponsesCh
@ -342,7 +342,7 @@ func TestWSHandlesArrayParams(t *testing.T) {
val := testVal
params := []interface{}{val}
err = cl.CallWithArrayParams(context.Background(), "echo_ws", params)
err = cl.CallWithArrayParams(ctx, "echo_ws", params)
require.NoError(t, err)
msg := <-cl.ResponsesCh


+ 10
- 9
test/e2e/runner/evidence.go View File

@ -83,11 +83,11 @@ func InjectEvidence(ctx context.Context, r *rand.Rand, testnet *e2e.Testnet, amo
var ev types.Evidence
for i := 1; i <= amount; i++ {
if i%lightClientEvidenceRatio == 0 {
ev, err = generateLightClientAttackEvidence(
ev, err = generateLightClientAttackEvidence(ctx,
privVals, evidenceHeight, valSet, testnet.Name, blockRes.Block.Time,
)
} else {
ev, err = generateDuplicateVoteEvidence(
ev, err = generateDuplicateVoteEvidence(ctx,
privVals, evidenceHeight, valSet, testnet.Name, blockRes.Block.Time,
)
}
@ -142,6 +142,7 @@ func getPrivateValidatorKeys(testnet *e2e.Testnet) ([]types.MockPV, error) {
// creates evidence of a lunatic attack. The height provided is the common height.
// The forged height happens 2 blocks later.
func generateLightClientAttackEvidence(
ctx context.Context,
privVals []types.MockPV,
height int64,
vals *types.ValidatorSet,
@ -156,7 +157,7 @@ func generateLightClientAttackEvidence(
// add a new bogus validator and remove an existing one to
// vary the validator set slightly
pv, conflictingVals, err := mutateValidatorSet(privVals, vals)
pv, conflictingVals, err := mutateValidatorSet(ctx, privVals, vals)
if err != nil {
return nil, err
}
@ -166,7 +167,7 @@ func generateLightClientAttackEvidence(
// create a commit for the forged header
blockID := makeBlockID(header.Hash(), 1000, []byte("partshash"))
voteSet := types.NewVoteSet(chainID, forgedHeight, 0, tmproto.SignedMsgType(2), conflictingVals)
commit, err := factory.MakeCommit(blockID, forgedHeight, 0, voteSet, pv, forgedTime)
commit, err := factory.MakeCommit(ctx, blockID, forgedHeight, 0, voteSet, pv, forgedTime)
if err != nil {
return nil, err
}
@ -192,6 +193,7 @@ func generateLightClientAttackEvidence(
// generateDuplicateVoteEvidence picks a random validator from the val set and
// returns duplicate vote evidence against the validator
func generateDuplicateVoteEvidence(
ctx context.Context,
privVals []types.MockPV,
height int64,
vals *types.ValidatorSet,
@ -202,11 +204,11 @@ func generateDuplicateVoteEvidence(
if err != nil {
return nil, err
}
voteA, err := factory.MakeVote(privVal, chainID, valIdx, height, 0, 2, makeRandomBlockID(), time)
voteA, err := factory.MakeVote(ctx, privVal, chainID, valIdx, height, 0, 2, makeRandomBlockID(), time)
if err != nil {
return nil, err
}
voteB, err := factory.MakeVote(privVal, chainID, valIdx, height, 0, 2, makeRandomBlockID(), time)
voteB, err := factory.MakeVote(ctx, privVal, chainID, valIdx, height, 0, 2, makeRandomBlockID(), time)
if err != nil {
return nil, err
}
@ -284,9 +286,8 @@ func makeBlockID(hash []byte, partSetSize uint32, partSetHash []byte) types.Bloc
}
}
func mutateValidatorSet(privVals []types.MockPV, vals *types.ValidatorSet,
) ([]types.PrivValidator, *types.ValidatorSet, error) {
newVal, newPrivVal := factory.RandValidator(false, 10)
func mutateValidatorSet(ctx context.Context, privVals []types.MockPV, vals *types.ValidatorSet) ([]types.PrivValidator, *types.ValidatorSet, error) {
newVal, newPrivVal := factory.RandValidator(ctx, false, 10)
var newVals *types.ValidatorSet
if vals.Size() > 2 {


+ 65
- 29
types/block_test.go View File

@ -35,15 +35,19 @@ func TestMain(m *testing.M) {
}
func TestBlockAddEvidence(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
txs := []Tx{Tx("foo"), Tx("bar")}
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, _, vals := randVoteSet(h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(lastID, h-1, 1, voteSet, vals, time.Now())
voteSet, _, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now())
require.NoError(t, err)
ev := NewMockDuplicateVoteEvidenceWithValidator(h, time.Now(), vals[0], "block-test-chain")
ev, err := NewMockDuplicateVoteEvidenceWithValidator(ctx, h, time.Now(), vals[0], "block-test-chain")
require.NoError(t, err)
evList := []Evidence{ev}
block := MakeBlock(h, txs, commit, evList)
@ -53,17 +57,21 @@ func TestBlockAddEvidence(t *testing.T) {
}
func TestBlockValidateBasic(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
require.Error(t, (*Block)(nil).ValidateBasic())
txs := []Tx{Tx("foo"), Tx("bar")}
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, valSet, vals := randVoteSet(h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(lastID, h-1, 1, voteSet, vals, time.Now())
voteSet, valSet, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now())
require.NoError(t, err)
ev := NewMockDuplicateVoteEvidenceWithValidator(h, time.Now(), vals[0], "block-test-chain")
ev, err := NewMockDuplicateVoteEvidenceWithValidator(ctx, h, time.Now(), vals[0], "block-test-chain")
require.NoError(t, err)
evList := []Evidence{ev}
testCases := []struct {
@ -134,6 +142,9 @@ func TestBlockMakePartSet(t *testing.T) {
}
func TestBlockMakePartSetWithEvidence(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
bps, err := (*Block)(nil).MakePartSet(2)
assert.Error(t, err)
assert.Nil(t, bps)
@ -141,11 +152,12 @@ func TestBlockMakePartSetWithEvidence(t *testing.T) {
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, _, vals := randVoteSet(h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(lastID, h-1, 1, voteSet, vals, time.Now())
voteSet, _, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now())
require.NoError(t, err)
ev := NewMockDuplicateVoteEvidenceWithValidator(h, time.Now(), vals[0], "block-test-chain")
ev, err := NewMockDuplicateVoteEvidenceWithValidator(ctx, h, time.Now(), vals[0], "block-test-chain")
require.NoError(t, err)
evList := []Evidence{ev}
partSet, err := MakeBlock(h, []Tx{Tx("Hello World")}, commit, evList).MakePartSet(512)
@ -156,15 +168,19 @@ func TestBlockMakePartSetWithEvidence(t *testing.T) {
}
func TestBlockHashesTo(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
assert.False(t, (*Block)(nil).HashesTo(nil))
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, valSet, vals := randVoteSet(h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(lastID, h-1, 1, voteSet, vals, time.Now())
voteSet, valSet, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now())
require.NoError(t, err)
ev := NewMockDuplicateVoteEvidenceWithValidator(h, time.Now(), vals[0], "block-test-chain")
ev, err := NewMockDuplicateVoteEvidenceWithValidator(ctx, h, time.Now(), vals[0], "block-test-chain")
require.NoError(t, err)
evList := []Evidence{ev}
block := MakeBlock(h, []Tx{Tx("Hello World")}, commit, evList)
@ -236,10 +252,13 @@ func TestNilDataHashDoesntCrash(t *testing.T) {
}
func TestCommit(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, _, vals := randVoteSet(h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(lastID, h-1, 1, voteSet, vals, time.Now())
voteSet, _, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now())
require.NoError(t, err)
assert.Equal(t, h-1, commit.Height)
@ -270,7 +289,10 @@ func TestCommitValidateBasic(t *testing.T) {
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
com := randCommit(time.Now())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
com := randCommit(ctx, t, time.Now())
tc.malleateCommit(com)
assert.Equal(t, tc.expectErr, com.ValidateBasic() != nil, "Validate Basic had an unexpected result")
})
@ -447,14 +469,14 @@ func TestMaxHeaderBytes(t *testing.T) {
assert.EqualValues(t, MaxHeaderBytes, int64(len(bz)))
}
func randCommit(now time.Time) *Commit {
func randCommit(ctx context.Context, t *testing.T, now time.Time) *Commit {
t.Helper()
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, _, vals := randVoteSet(h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(lastID, h-1, 1, voteSet, vals, now)
if err != nil {
panic(err)
}
voteSet, _, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, now)
require.NoError(t, err)
return commit
}
@ -531,8 +553,11 @@ func TestCommitToVoteSet(t *testing.T) {
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, valSet, vals := randVoteSet(h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(lastID, h-1, 1, voteSet, vals, time.Now())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
voteSet, valSet, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now())
assert.NoError(t, err)
chainID := voteSet.ChainID()
@ -577,7 +602,7 @@ func TestCommitToVoteSetWithVotesForNilBlock(t *testing.T) {
}
for _, tc := range testCases {
voteSet, valSet, vals := randVoteSet(height-1, round, tmproto.PrecommitType, tc.numValidators, 1)
voteSet, valSet, vals := randVoteSet(ctx, t, height-1, round, tmproto.PrecommitType, tc.numValidators, 1)
vi := int32(0)
for n := range tc.blockIDs {
@ -594,7 +619,7 @@ func TestCommitToVoteSetWithVotesForNilBlock(t *testing.T) {
Timestamp: tmtime.Now(),
}
added, err := signAddVote(vals[vi], vote, voteSet)
added, err := signAddVote(ctx, vals[vi], vote, voteSet)
assert.NoError(t, err)
assert.True(t, added)
@ -654,15 +679,19 @@ func TestBlockIDValidateBasic(t *testing.T) {
}
func TestBlockProtoBuf(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
h := mrand.Int63()
c1 := randCommit(time.Now())
c1 := randCommit(ctx, t, time.Now())
b1 := MakeBlock(h, []Tx{Tx([]byte{1})}, &Commit{Signatures: []CommitSig{}}, []Evidence{})
b1.ProposerAddress = tmrand.Bytes(crypto.AddressSize)
b2 := MakeBlock(h, []Tx{Tx([]byte{1})}, c1, []Evidence{})
b2.ProposerAddress = tmrand.Bytes(crypto.AddressSize)
evidenceTime := time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
evi := NewMockDuplicateVoteEvidence(h, evidenceTime, "block-test-chain")
evi, err := NewMockDuplicateVoteEvidence(ctx, h, evidenceTime, "block-test-chain")
require.NoError(t, err)
b2.Evidence = EvidenceData{Evidence: EvidenceList{evi}}
b2.EvidenceHash = b2.Evidence.Hash()
@ -725,8 +754,12 @@ func TestDataProtoBuf(t *testing.T) {
// TestEvidenceDataProtoBuf ensures parity in converting to and from proto.
func TestEvidenceDataProtoBuf(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
const chainID = "mychain"
ev := NewMockDuplicateVoteEvidence(math.MaxInt64, time.Now(), chainID)
ev, err := NewMockDuplicateVoteEvidence(ctx, math.MaxInt64, time.Now(), chainID)
require.NoError(t, err)
data := &EvidenceData{Evidence: EvidenceList{ev}}
_ = data.ByteSize()
testCases := []struct {
@ -840,7 +873,10 @@ func TestBlockIDProtoBuf(t *testing.T) {
}
func TestSignedHeaderProtoBuf(t *testing.T) {
commit := randCommit(time.Now())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
commit := randCommit(ctx, t, time.Now())
h := MakeRandHeader()
sh := SignedHeader{Header: &h, Commit: commit}


+ 12
- 9
types/evidence.go View File

@ -671,29 +671,32 @@ func (err *ErrEvidenceOverflow) Error() string {
// unstable - use only for testing
// assumes the round to be 0 and the validator index to be 0
func NewMockDuplicateVoteEvidence(height int64, time time.Time, chainID string) *DuplicateVoteEvidence {
func NewMockDuplicateVoteEvidence(ctx context.Context, height int64, time time.Time, chainID string) (*DuplicateVoteEvidence, error) {
val := NewMockPV()
return NewMockDuplicateVoteEvidenceWithValidator(height, time, val, chainID)
return NewMockDuplicateVoteEvidenceWithValidator(ctx, height, time, val, chainID)
}
// assumes voting power to be 10 and validator to be the only one in the set
func NewMockDuplicateVoteEvidenceWithValidator(height int64, time time.Time,
pv PrivValidator, chainID string) *DuplicateVoteEvidence {
pubKey, _ := pv.GetPubKey(context.Background())
func NewMockDuplicateVoteEvidenceWithValidator(ctx context.Context, height int64, time time.Time, pv PrivValidator, chainID string) (*DuplicateVoteEvidence, error) {
pubKey, err := pv.GetPubKey(ctx)
if err != nil {
return nil, err
}
val := NewValidator(pubKey, 10)
voteA := makeMockVote(height, 0, 0, pubKey.Address(), randBlockID(), time)
vA := voteA.ToProto()
_ = pv.SignVote(context.Background(), chainID, vA)
_ = pv.SignVote(ctx, chainID, vA)
voteA.Signature = vA.Signature
voteB := makeMockVote(height, 0, 0, pubKey.Address(), randBlockID(), time)
vB := voteB.ToProto()
_ = pv.SignVote(context.Background(), chainID, vB)
_ = pv.SignVote(ctx, chainID, vB)
voteB.Signature = vB.Signature
ev, err := NewDuplicateVoteEvidence(voteA, voteB, time, NewValidatorSet([]*Validator{val}))
if err != nil {
panic("constructing mock duplicate vote evidence: " + err.Error())
return nil, fmt.Errorf("constructing mock duplicate vote evidence: %w", err)
}
return ev
return ev, nil
}
func makeMockVote(height int64, round, index int32, addr Address,


+ 48
- 22
types/evidence_test.go View File

@ -22,7 +22,10 @@ import (
var defaultVoteTime = time.Date(2019, 1, 1, 0, 0, 0, 0, time.UTC)
func TestEvidenceList(t *testing.T) {
ev := randomDuplicateVoteEvidence(t)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ev := randomDuplicateVoteEvidence(ctx, t)
evl := EvidenceList([]Evidence{ev})
assert.NotNil(t, evl.Hash())
@ -30,14 +33,15 @@ func TestEvidenceList(t *testing.T) {
assert.False(t, evl.Has(&DuplicateVoteEvidence{}))
}
func randomDuplicateVoteEvidence(t *testing.T) *DuplicateVoteEvidence {
func randomDuplicateVoteEvidence(ctx context.Context, t *testing.T) *DuplicateVoteEvidence {
t.Helper()
val := NewMockPV()
blockID := makeBlockID([]byte("blockhash"), 1000, []byte("partshash"))
blockID2 := makeBlockID([]byte("blockhash2"), 1000, []byte("partshash"))
const chainID = "mychain"
return &DuplicateVoteEvidence{
VoteA: makeVote(t, val, chainID, 0, 10, 2, 1, blockID, defaultVoteTime),
VoteB: makeVote(t, val, chainID, 0, 10, 2, 1, blockID2, defaultVoteTime.Add(1*time.Minute)),
VoteA: makeVote(ctx, t, val, chainID, 0, 10, 2, 1, blockID, defaultVoteTime),
VoteB: makeVote(ctx, t, val, chainID, 0, 10, 2, 1, blockID2, defaultVoteTime.Add(1*time.Minute)),
TotalVotingPower: 30,
ValidatorPower: 10,
Timestamp: defaultVoteTime,
@ -46,7 +50,11 @@ func randomDuplicateVoteEvidence(t *testing.T) *DuplicateVoteEvidence {
func TestDuplicateVoteEvidence(t *testing.T) {
const height = int64(13)
ev := NewMockDuplicateVoteEvidence(height, time.Now(), "mock-chain-id")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
ev, err := NewMockDuplicateVoteEvidence(ctx, height, time.Now(), "mock-chain-id")
require.NoError(t, err)
assert.Equal(t, ev.Hash(), tmhash.Sum(ev.Bytes()))
assert.NotNil(t, ev.String())
assert.Equal(t, ev.Height(), height)
@ -58,6 +66,9 @@ func TestDuplicateVoteEvidenceValidation(t *testing.T) {
blockID2 := makeBlockID(tmhash.Sum([]byte("blockhash2")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
const chainID = "mychain"
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
testCases := []struct {
testName string
malleateEvidence func(*DuplicateVoteEvidence)
@ -71,7 +82,7 @@ func TestDuplicateVoteEvidenceValidation(t *testing.T) {
ev.VoteB = nil
}, true},
{"Invalid vote type", func(ev *DuplicateVoteEvidence) {
ev.VoteA = makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, math.MaxInt32, 0, blockID2, defaultVoteTime)
ev.VoteA = makeVote(ctx, t, val, chainID, math.MaxInt32, math.MaxInt64, math.MaxInt32, 0, blockID2, defaultVoteTime)
}, true},
{"Invalid vote order", func(ev *DuplicateVoteEvidence) {
swap := ev.VoteA.Copy()
@ -82,9 +93,9 @@ func TestDuplicateVoteEvidenceValidation(t *testing.T) {
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
vote1 := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, math.MaxInt32, 0x02, blockID, defaultVoteTime)
vote2 := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, math.MaxInt32, 0x02, blockID2, defaultVoteTime)
valSet := NewValidatorSet([]*Validator{val.ExtractIntoValidator(10)})
vote1 := makeVote(ctx, t, val, chainID, math.MaxInt32, math.MaxInt64, math.MaxInt32, 0x02, blockID, defaultVoteTime)
vote2 := makeVote(ctx, t, val, chainID, math.MaxInt32, math.MaxInt64, math.MaxInt32, 0x02, blockID2, defaultVoteTime)
valSet := NewValidatorSet([]*Validator{val.ExtractIntoValidator(ctx, 10)})
ev, err := NewDuplicateVoteEvidence(vote1, vote2, defaultVoteTime, valSet)
require.NoError(t, err)
tc.malleateEvidence(ev)
@ -94,14 +105,17 @@ func TestDuplicateVoteEvidenceValidation(t *testing.T) {
}
func TestLightClientAttackEvidenceBasic(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
height := int64(5)
commonHeight := height - 1
nValidators := 10
voteSet, valSet, privVals := randVoteSet(height, 1, tmproto.PrecommitType, nValidators, 1)
voteSet, valSet, privVals := randVoteSet(ctx, t, height, 1, tmproto.PrecommitType, nValidators, 1)
header := makeHeaderRandom()
header.Height = height
blockID := makeBlockID(tmhash.Sum([]byte("blockhash")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
commit, err := makeCommit(blockID, height, 1, voteSet, privVals, defaultVoteTime)
commit, err := makeCommit(ctx, blockID, height, 1, voteSet, privVals, defaultVoteTime)
require.NoError(t, err)
lcae := &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
@ -153,15 +167,18 @@ func TestLightClientAttackEvidenceBasic(t *testing.T) {
}
func TestLightClientAttackEvidenceValidation(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
height := int64(5)
commonHeight := height - 1
nValidators := 10
voteSet, valSet, privVals := randVoteSet(height, 1, tmproto.PrecommitType, nValidators, 1)
voteSet, valSet, privVals := randVoteSet(ctx, t, height, 1, tmproto.PrecommitType, nValidators, 1)
header := makeHeaderRandom()
header.Height = height
header.ValidatorsHash = valSet.Hash()
blockID := makeBlockID(header.Hash(), math.MaxInt32, tmhash.Sum([]byte("partshash")))
commit, err := makeCommit(blockID, height, 1, voteSet, privVals, time.Now())
commit, err := makeCommit(ctx, blockID, height, 1, voteSet, privVals, time.Now())
require.NoError(t, err)
lcae := &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{
@ -228,14 +245,20 @@ func TestLightClientAttackEvidenceValidation(t *testing.T) {
}
func TestMockEvidenceValidateBasic(t *testing.T) {
goodEvidence := NewMockDuplicateVoteEvidence(int64(1), time.Now(), "mock-chain-id")
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
goodEvidence, err := NewMockDuplicateVoteEvidence(ctx, int64(1), time.Now(), "mock-chain-id")
require.NoError(t, err)
assert.Nil(t, goodEvidence.ValidateBasic())
}
func makeVote(
ctx context.Context,
t *testing.T, val PrivValidator, chainID string, valIndex int32, height int64, round int32, step int, blockID BlockID,
time time.Time) *Vote {
pubKey, err := val.GetPubKey(context.Background())
time time.Time,
) *Vote {
pubKey, err := val.GetPubKey(ctx)
require.NoError(t, err)
v := &Vote{
ValidatorAddress: pubKey.Address(),
@ -248,7 +271,7 @@ func makeVote(
}
vpb := v.ToProto()
err = val.SignVote(context.Background(), chainID, vpb)
err = val.SignVote(ctx, chainID, vpb)
require.NoError(t, err)
v.Signature = vpb.Signature
@ -275,13 +298,16 @@ func makeHeaderRandom() *Header {
}
func TestEvidenceProto(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
// -------- Votes --------
val := NewMockPV()
blockID := makeBlockID(tmhash.Sum([]byte("blockhash")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
blockID2 := makeBlockID(tmhash.Sum([]byte("blockhash2")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
const chainID = "mychain"
v := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, 1, 0x01, blockID, defaultVoteTime)
v2 := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, 2, 0x01, blockID2, defaultVoteTime)
v := makeVote(ctx, t, val, chainID, math.MaxInt32, math.MaxInt64, 1, 0x01, blockID, defaultVoteTime)
v2 := makeVote(ctx, t, val, chainID, math.MaxInt32, math.MaxInt64, 2, 0x01, blockID2, defaultVoteTime)
tests := []struct {
testName string
@ -325,8 +351,8 @@ func TestEvidenceVectors(t *testing.T) {
blockID := makeBlockID(tmhash.Sum([]byte("blockhash")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
blockID2 := makeBlockID(tmhash.Sum([]byte("blockhash2")), math.MaxInt32, tmhash.Sum([]byte("partshash")))
const chainID = "mychain"
v := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, 1, 0x01, blockID, defaultVoteTime)
v2 := makeVote(t, val, chainID, math.MaxInt32, math.MaxInt64, 2, 0x01, blockID2, defaultVoteTime)
v := makeVote(ctx, t, val, chainID, math.MaxInt32, math.MaxInt64, 1, 0x01, blockID, defaultVoteTime)
v2 := makeVote(ctx, t, val, chainID, math.MaxInt32, math.MaxInt64, 2, 0x01, blockID2, defaultVoteTime)
// Data for LightClientAttackEvidence
height := int64(5)
@ -352,7 +378,7 @@ func TestEvidenceVectors(t *testing.T) {
ProposerAddress: []byte("2915b7b15f979e48ebc61774bb1d86ba3136b7eb"),
}
blockID3 := makeBlockID(header.Hash(), math.MaxInt32, tmhash.Sum([]byte("partshash")))
commit, err := makeCommit(blockID3, height, 1, voteSet, privVals, defaultVoteTime)
commit, err := makeCommit(ctx, blockID3, height, 1, voteSet, privVals, defaultVoteTime)
require.NoError(t, err)
lcae := &LightClientAttackEvidence{
ConflictingBlock: &LightBlock{


+ 16
- 7
types/light_test.go View File

@ -1,6 +1,7 @@
package types
import (
"context"
"math"
"testing"
"time"
@ -12,14 +13,17 @@ import (
)
func TestLightBlockValidateBasic(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
header := MakeRandHeader()
commit := randCommit(time.Now())
vals, _ := randValidatorPrivValSet(5, 1)
commit := randCommit(ctx, t, time.Now())
vals, _ := randValidatorPrivValSet(ctx, t, 5, 1)
header.Height = commit.Height
header.LastBlockID = commit.BlockID
header.ValidatorsHash = vals.Hash()
header.Version.Block = version.BlockProtocol
vals2, _ := randValidatorPrivValSet(3, 1)
vals2, _ := randValidatorPrivValSet(ctx, t, 3, 1)
vals3 := vals.Copy()
vals3.Proposer = &Validator{}
commit.BlockID.Hash = header.Hash()
@ -38,7 +42,7 @@ func TestLightBlockValidateBasic(t *testing.T) {
{"valid light block", sh, vals, false},
{"hashes don't match", sh, vals2, true},
{"invalid validator set", sh, vals3, true},
{"invalid signed header", &SignedHeader{Header: &header, Commit: randCommit(time.Now())}, vals, true},
{"invalid signed header", &SignedHeader{Header: &header, Commit: randCommit(ctx, t, time.Now())}, vals, true},
}
for _, tc := range testCases {
@ -57,9 +61,11 @@ func TestLightBlockValidateBasic(t *testing.T) {
}
func TestLightBlockProtobuf(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
header := MakeRandHeader()
commit := randCommit(time.Now())
vals, _ := randValidatorPrivValSet(5, 1)
commit := randCommit(ctx, t, time.Now())
vals, _ := randValidatorPrivValSet(ctx, t, 5, 1)
header.Height = commit.Height
header.LastBlockID = commit.BlockID
header.Version.Block = version.BlockProtocol
@ -110,7 +116,10 @@ func TestLightBlockProtobuf(t *testing.T) {
}
func TestSignedHeaderValidateBasic(t *testing.T) {
commit := randCommit(time.Now())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
commit := randCommit(ctx, t, time.Now())
chainID := "𠜎"
timestamp := time.Date(math.MaxInt64, 0, 0, 0, 0, 0, math.MaxInt64, time.UTC)
h := Header{


+ 5
- 5
types/priv_validator.go View File

@ -39,11 +39,11 @@ func (pvs PrivValidatorsByAddress) Len() int {
}
func (pvs PrivValidatorsByAddress) Less(i, j int) bool {
pvi, err := pvs[i].GetPubKey(context.Background())
pvi, err := pvs[i].GetPubKey(context.TODO())
if err != nil {
panic(err)
}
pvj, err := pvs[j].GetPubKey(context.Background())
pvj, err := pvs[j].GetPubKey(context.TODO())
if err != nil {
panic(err)
}
@ -114,8 +114,8 @@ func (pv MockPV) SignProposal(ctx context.Context, chainID string, proposal *tmp
return nil
}
func (pv MockPV) ExtractIntoValidator(votingPower int64) *Validator {
pubKey, _ := pv.GetPubKey(context.Background())
func (pv MockPV) ExtractIntoValidator(ctx context.Context, votingPower int64) *Validator {
pubKey, _ := pv.GetPubKey(ctx)
return &Validator{
Address: pubKey.Address(),
PubKey: pubKey,
@ -125,7 +125,7 @@ func (pv MockPV) ExtractIntoValidator(votingPower int64) *Validator {
// String returns a string representation of the MockPV.
func (pv MockPV) String() string {
mpv, _ := pv.GetPubKey(context.Background()) // mockPV will never return an error, ignored here
mpv, _ := pv.GetPubKey(context.TODO()) // mockPV will never return an error, ignored here
return fmt.Sprintf("MockPV{%v}", mpv.Address())
}


+ 21
- 6
types/proposal_test.go View File

@ -56,8 +56,11 @@ func TestProposalString(t *testing.T) {
}
func TestProposalVerifySignature(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
privVal := NewMockPV()
pubKey, err := privVal.GetPubKey(context.Background())
pubKey, err := privVal.GetPubKey(ctx)
require.NoError(t, err)
prop := NewProposal(
@ -67,7 +70,7 @@ func TestProposalVerifySignature(t *testing.T) {
signBytes := ProposalSignBytes("test_chain_id", p)
// sign it
err = privVal.SignProposal(context.Background(), "test_chain_id", p)
err = privVal.SignProposal(ctx, "test_chain_id", p)
require.NoError(t, err)
prop.Signature = p.Signature
@ -102,9 +105,13 @@ func BenchmarkProposalWriteSignBytes(b *testing.B) {
}
func BenchmarkProposalSign(b *testing.B) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
privVal := NewMockPV()
b.ResetTimer()
for i := 0; i < b.N; i++ {
err := privVal.SignProposal(context.Background(), "test_chain_id", pbp)
err := privVal.SignProposal(ctx, "test_chain_id", pbp)
if err != nil {
b.Error(err)
}
@ -112,10 +119,15 @@ func BenchmarkProposalSign(b *testing.B) {
}
func BenchmarkProposalVerifySignature(b *testing.B) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
b.ResetTimer()
privVal := NewMockPV()
err := privVal.SignProposal(context.Background(), "test_chain_id", pbp)
err := privVal.SignProposal(ctx, "test_chain_id", pbp)
require.NoError(b, err)
pubKey, err := privVal.GetPubKey(context.Background())
pubKey, err := privVal.GetPubKey(ctx)
require.NoError(b, err)
for i := 0; i < b.N; i++ {
@ -151,11 +163,14 @@ func TestProposalValidateBasic(t *testing.T) {
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
prop := NewProposal(
4, 2, 2,
blockID)
p := prop.ToProto()
err := privVal.SignProposal(context.Background(), "test_chain_id", p)
err := privVal.SignProposal(ctx, "test_chain_id", p)
prop.Signature = p.Signature
require.NoError(t, err)
tc.malleateProposal(prop)


+ 5
- 5
types/test_util.go View File

@ -8,12 +8,12 @@ import (
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
)
func makeCommit(blockID BlockID, height int64, round int32,
func makeCommit(ctx context.Context, blockID BlockID, height int64, round int32,
voteSet *VoteSet, validators []PrivValidator, now time.Time) (*Commit, error) {
// all sign
for i := 0; i < len(validators); i++ {
pubKey, err := validators[i].GetPubKey(context.Background())
pubKey, err := validators[i].GetPubKey(ctx)
if err != nil {
return nil, fmt.Errorf("can't get pubkey: %w", err)
}
@ -27,7 +27,7 @@ func makeCommit(blockID BlockID, height int64, round int32,
Timestamp: now,
}
_, err = signAddVote(validators[i], vote, voteSet)
_, err = signAddVote(ctx, validators[i], vote, voteSet)
if err != nil {
return nil, err
}
@ -36,9 +36,9 @@ func makeCommit(blockID BlockID, height int64, round int32,
return voteSet.MakeCommit(), nil
}
func signAddVote(privVal PrivValidator, vote *Vote, voteSet *VoteSet) (signed bool, err error) {
func signAddVote(ctx context.Context, privVal PrivValidator, vote *Vote, voteSet *VoteSet) (signed bool, err error) {
v := vote.ToProto()
err = privVal.SignVote(context.Background(), voteSet.ChainID(), v)
err = privVal.SignVote(ctx, voteSet.ChainID(), v)
if err != nil {
return false, err
}


+ 34
- 17
types/validation_test.go View File

@ -60,7 +60,10 @@ func TestValidatorSet_VerifyCommit_All(t *testing.T) {
for _, tc := range testCases {
tc := tc
t.Run(tc.description, func(t *testing.T) {
_, valSet, vals := randVoteSet(tc.height, round, tmproto.PrecommitType, tc.valSize, 10)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
_, valSet, vals := randVoteSet(ctx, t, tc.height, round, tmproto.PrecommitType, tc.valSize, 10)
totalVotes := tc.blockVotes + tc.absentVotes + tc.nilVotes
sigs := make([]CommitSig, totalVotes)
vi := 0
@ -71,7 +74,7 @@ func TestValidatorSet_VerifyCommit_All(t *testing.T) {
}
for i := 0; i < tc.blockVotes+tc.nilVotes; i++ {
pubKey, err := vals[vi%len(vals)].GetPubKey(context.Background())
pubKey, err := vals[vi%len(vals)].GetPubKey(ctx)
require.NoError(t, err)
vote := &Vote{
ValidatorAddress: pubKey.Address(),
@ -88,7 +91,7 @@ func TestValidatorSet_VerifyCommit_All(t *testing.T) {
v := vote.ToProto()
require.NoError(t, vals[vi%len(vals)].SignVote(context.Background(), tc.chainID, v))
require.NoError(t, vals[vi%len(vals)].SignVote(ctx, tc.chainID, v))
vote.Signature = v.Signature
sigs[vi] = vote.CommitSig()
@ -138,15 +141,18 @@ func TestValidatorSet_VerifyCommit_CheckAllSignatures(t *testing.T) {
blockID = makeBlockIDRandom()
)
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
voteSet, valSet, vals := randVoteSet(ctx, t, h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now())
require.NoError(t, err)
require.NoError(t, valSet.VerifyCommit(chainID, blockID, h, commit))
// malleate 4th signature
vote := voteSet.GetByIndex(3)
v := vote.ToProto()
err = vals[3].SignVote(context.Background(), "CentaurusA", v)
err = vals[3].SignVote(ctx, "CentaurusA", v)
require.NoError(t, err)
vote.Signature = v.Signature
commit.Signatures[3] = vote.CommitSig()
@ -164,15 +170,18 @@ func TestValidatorSet_VerifyCommitLight_ReturnsAsSoonAsMajorityOfVotingPowerSign
blockID = makeBlockIDRandom()
)
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
voteSet, valSet, vals := randVoteSet(ctx, t, h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now())
require.NoError(t, err)
require.NoError(t, valSet.VerifyCommit(chainID, blockID, h, commit))
// malleate 4th signature (3 signatures are enough for 2/3+)
vote := voteSet.GetByIndex(3)
v := vote.ToProto()
err = vals[3].SignVote(context.Background(), "CentaurusA", v)
err = vals[3].SignVote(ctx, "CentaurusA", v)
require.NoError(t, err)
vote.Signature = v.Signature
commit.Signatures[3] = vote.CommitSig()
@ -187,16 +196,18 @@ func TestValidatorSet_VerifyCommitLightTrusting_ReturnsAsSoonAsTrustLevelOfVotin
h = int64(3)
blockID = makeBlockIDRandom()
)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
voteSet, valSet, vals := randVoteSet(ctx, t, h, 0, tmproto.PrecommitType, 4, 10)
commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now())
require.NoError(t, err)
require.NoError(t, valSet.VerifyCommit(chainID, blockID, h, commit))
// malleate 3rd signature (2 signatures are enough for 1/3+ trust level)
vote := voteSet.GetByIndex(2)
v := vote.ToProto()
err = vals[2].SignVote(context.Background(), "CentaurusA", v)
err = vals[2].SignVote(ctx, "CentaurusA", v)
require.NoError(t, err)
vote.Signature = v.Signature
commit.Signatures[2] = vote.CommitSig()
@ -206,11 +217,14 @@ func TestValidatorSet_VerifyCommitLightTrusting_ReturnsAsSoonAsTrustLevelOfVotin
}
func TestValidatorSet_VerifyCommitLightTrusting(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var (
blockID = makeBlockIDRandom()
voteSet, originalValset, vals = randVoteSet(1, 1, tmproto.PrecommitType, 6, 1)
commit, err = makeCommit(blockID, 1, 1, voteSet, vals, time.Now())
newValSet, _ = randValidatorPrivValSet(2, 1)
voteSet, originalValset, vals = randVoteSet(ctx, t, 1, 1, tmproto.PrecommitType, 6, 1)
commit, err = makeCommit(ctx, blockID, 1, 1, voteSet, vals, time.Now())
newValSet, _ = randValidatorPrivValSet(ctx, t, 2, 1)
)
require.NoError(t, err)
@ -247,10 +261,13 @@ func TestValidatorSet_VerifyCommitLightTrusting(t *testing.T) {
}
func TestValidatorSet_VerifyCommitLightTrustingErrorsOnOverflow(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var (
blockID = makeBlockIDRandom()
voteSet, valSet, vals = randVoteSet(1, 1, tmproto.PrecommitType, 1, MaxTotalVotingPower)
commit, err = makeCommit(blockID, 1, 1, voteSet, vals, time.Now())
voteSet, valSet, vals = randVoteSet(ctx, t, 1, 1, tmproto.PrecommitType, 1, MaxTotalVotingPower)
commit, err = makeCommit(ctx, blockID, 1, 1, voteSet, vals, time.Now())
)
require.NoError(t, err)


+ 37
- 18
types/validator_set_test.go View File

@ -81,7 +81,11 @@ func TestValidatorSetBasic(t *testing.T) {
}
func TestValidatorSetValidateBasic(t *testing.T) {
val, _ := randValidator(false, 1)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
val, _, err := randValidator(ctx, false, 1)
require.NoError(t, err)
badVal := &Validator{}
testCases := []struct {
@ -382,18 +386,18 @@ func randModuloValidator(totalVotingPower int64) *Validator {
return val
}
func randValidator(randPower bool, minPower int64) (*Validator, PrivValidator) {
func randValidator(ctx context.Context, randPower bool, minPower int64) (*Validator, PrivValidator, error) {
privVal := NewMockPV()
votePower := minPower
if randPower {
votePower += int64(rand.Uint32())
}
pubKey, err := privVal.GetPubKey(context.Background())
pubKey, err := privVal.GetPubKey(ctx)
if err != nil {
panic(fmt.Errorf("could not retrieve pubkey %w", err))
return nil, nil, fmt.Errorf("could not retrieve pubkey %w", err)
}
val := NewValidator(pubKey, votePower)
return val, privVal
return val, privVal, nil
}
func randModuloValidatorSet(numValidators int) *ValidatorSet {
@ -1293,11 +1297,14 @@ func verifyValSetUpdatePriorityOrder(t *testing.T, valSet *ValidatorSet, cfg tes
}
func TestNewValidatorSetFromExistingValidators(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
size := 5
vals := make([]*Validator, size)
for i := 0; i < size; i++ {
pv := NewMockPV()
vals[i] = pv.ExtractIntoValidator(int64(i + 1))
vals[i] = pv.ExtractIntoValidator(ctx, int64(i+1))
}
valSet := NewValidatorSet(vals)
valSet.IncrementProposerPriority(5)
@ -1409,14 +1416,17 @@ func TestSafeMul(t *testing.T) {
}
func TestValidatorSetProtoBuf(t *testing.T) {
valset, _ := randValidatorPrivValSet(10, 100)
valset2, _ := randValidatorPrivValSet(10, 100)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
valset, _ := randValidatorPrivValSet(ctx, t, 10, 100)
valset2, _ := randValidatorPrivValSet(ctx, t, 10, 100)
valset2.Validators[0] = &Validator{}
valset3, _ := randValidatorPrivValSet(10, 100)
valset3, _ := randValidatorPrivValSet(ctx, t, 10, 100)
valset3.Proposer = nil
valset4, _ := randValidatorPrivValSet(10, 100)
valset4, _ := randValidatorPrivValSet(ctx, t, 10, 100)
valset4.Proposer = &Validator{}
testCases := []struct {
@ -1521,7 +1531,10 @@ func BenchmarkUpdates(b *testing.B) {
}
}
func BenchmarkValidatorSet_VerifyCommit_Ed25519(b *testing.B) {
func BenchmarkValidatorSet_VerifyCommit_Ed25519(b *testing.B) { // nolint
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
for _, n := range []int{1, 8, 64, 1024} {
n := n
var (
@ -1532,9 +1545,9 @@ func BenchmarkValidatorSet_VerifyCommit_Ed25519(b *testing.B) {
b.Run(fmt.Sprintf("valset size %d", n), func(b *testing.B) {
b.ReportAllocs()
// generate n validators
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, n, int64(n*5))
voteSet, valSet, vals := randVoteSet(ctx, b, h, 0, tmproto.PrecommitType, n, int64(n*5))
// create a commit with n validators
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now())
require.NoError(b, err)
for i := 0; i < b.N/n; i++ {
@ -1545,7 +1558,10 @@ func BenchmarkValidatorSet_VerifyCommit_Ed25519(b *testing.B) {
}
}
func BenchmarkValidatorSet_VerifyCommitLight_Ed25519(b *testing.B) {
func BenchmarkValidatorSet_VerifyCommitLight_Ed25519(b *testing.B) { // nolint
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
for _, n := range []int{1, 8, 64, 1024} {
n := n
var (
@ -1556,9 +1572,9 @@ func BenchmarkValidatorSet_VerifyCommitLight_Ed25519(b *testing.B) {
b.Run(fmt.Sprintf("valset size %d", n), func(b *testing.B) {
b.ReportAllocs()
// generate n validators
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, n, int64(n*5))
voteSet, valSet, vals := randVoteSet(ctx, b, h, 0, tmproto.PrecommitType, n, int64(n*5))
// create a commit with n validators
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now())
require.NoError(b, err)
for i := 0; i < b.N/n; i++ {
@ -1570,6 +1586,9 @@ func BenchmarkValidatorSet_VerifyCommitLight_Ed25519(b *testing.B) {
}
func BenchmarkValidatorSet_VerifyCommitLightTrusting_Ed25519(b *testing.B) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
for _, n := range []int{1, 8, 64, 1024} {
n := n
var (
@ -1580,9 +1599,9 @@ func BenchmarkValidatorSet_VerifyCommitLightTrusting_Ed25519(b *testing.B) {
b.Run(fmt.Sprintf("valset size %d", n), func(b *testing.B) {
b.ReportAllocs()
// generate n validators
voteSet, valSet, vals := randVoteSet(h, 0, tmproto.PrecommitType, n, int64(n*5))
voteSet, valSet, vals := randVoteSet(ctx, b, h, 0, tmproto.PrecommitType, n, int64(n*5))
// create a commit with n validators
commit, err := makeCommit(blockID, h, 0, voteSet, vals, time.Now())
commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now())
require.NoError(b, err)
for i := 0; i < b.N/n; i++ {


+ 9
- 2
types/validator_test.go View File

@ -11,7 +11,11 @@ import (
)
func TestValidatorProtoBuf(t *testing.T) {
val, _ := randValidator(true, 100)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
val, _, err := randValidator(ctx, true, 100)
require.NoError(t, err)
testCases := []struct {
msg string
v1 *Validator
@ -42,8 +46,11 @@ func TestValidatorProtoBuf(t *testing.T) {
}
func TestValidatorValidateBasic(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
priv := NewMockPV()
pubKey, _ := priv.GetPubKey(context.Background())
pubKey, _ := priv.GetPubKey(ctx)
testCases := []struct {
val *Validator
err bool


+ 79
- 58
types/vote_set_test.go View File

@ -16,11 +16,14 @@ import (
)
func TestVoteSet_AddVote_Good(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
height, round := int64(1), int32(0)
voteSet, _, privValidators := randVoteSet(height, round, tmproto.PrevoteType, 10, 1)
voteSet, _, privValidators := randVoteSet(ctx, t, height, round, tmproto.PrevoteType, 10, 1)
val0 := privValidators[0]
val0p, err := val0.GetPubKey(context.Background())
val0p, err := val0.GetPubKey(ctx)
require.NoError(t, err)
val0Addr := val0p.Address()
@ -38,7 +41,7 @@ func TestVoteSet_AddVote_Good(t *testing.T) {
Timestamp: tmtime.Now(),
BlockID: BlockID{nil, PartSetHeader{}},
}
_, err = signAddVote(val0, vote, voteSet)
_, err = signAddVote(ctx, val0, vote, voteSet)
require.NoError(t, err)
assert.NotNil(t, voteSet.GetByAddress(val0Addr))
@ -48,8 +51,11 @@ func TestVoteSet_AddVote_Good(t *testing.T) {
}
func TestVoteSet_AddVote_Bad(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
height, round := int64(1), int32(0)
voteSet, _, privValidators := randVoteSet(height, round, tmproto.PrevoteType, 10, 1)
voteSet, _, privValidators := randVoteSet(ctx, t, height, round, tmproto.PrevoteType, 10, 1)
voteProto := &Vote{
ValidatorAddress: nil,
@ -63,11 +69,11 @@ func TestVoteSet_AddVote_Bad(t *testing.T) {
// val0 votes for nil.
{
pubKey, err := privValidators[0].GetPubKey(context.Background())
pubKey, err := privValidators[0].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, 0)
added, err := signAddVote(privValidators[0], vote, voteSet)
added, err := signAddVote(ctx, privValidators[0], vote, voteSet)
if !added || err != nil {
t.Errorf("expected VoteSet.Add to succeed")
}
@ -75,11 +81,11 @@ func TestVoteSet_AddVote_Bad(t *testing.T) {
// val0 votes again for some block.
{
pubKey, err := privValidators[0].GetPubKey(context.Background())
pubKey, err := privValidators[0].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, tmrand.Bytes(32)), voteSet)
added, err := signAddVote(ctx, privValidators[0], withBlockHash(vote, tmrand.Bytes(32)), voteSet)
if added || err == nil {
t.Errorf("expected VoteSet.Add to fail, conflicting vote.")
}
@ -87,11 +93,11 @@ func TestVoteSet_AddVote_Bad(t *testing.T) {
// val1 votes on another height
{
pubKey, err := privValidators[1].GetPubKey(context.Background())
pubKey, err := privValidators[1].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, 1)
added, err := signAddVote(privValidators[1], withHeight(vote, height+1), voteSet)
added, err := signAddVote(ctx, privValidators[1], withHeight(vote, height+1), voteSet)
if added || err == nil {
t.Errorf("expected VoteSet.Add to fail, wrong height")
}
@ -99,11 +105,11 @@ func TestVoteSet_AddVote_Bad(t *testing.T) {
// val2 votes on another round
{
pubKey, err := privValidators[2].GetPubKey(context.Background())
pubKey, err := privValidators[2].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, 2)
added, err := signAddVote(privValidators[2], withRound(vote, round+1), voteSet)
added, err := signAddVote(ctx, privValidators[2], withRound(vote, round+1), voteSet)
if added || err == nil {
t.Errorf("expected VoteSet.Add to fail, wrong round")
}
@ -111,11 +117,11 @@ func TestVoteSet_AddVote_Bad(t *testing.T) {
// val3 votes of another type.
{
pubKey, err := privValidators[3].GetPubKey(context.Background())
pubKey, err := privValidators[3].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, 3)
added, err := signAddVote(privValidators[3], withType(vote, byte(tmproto.PrecommitType)), voteSet)
added, err := signAddVote(ctx, privValidators[3], withType(vote, byte(tmproto.PrecommitType)), voteSet)
if added || err == nil {
t.Errorf("expected VoteSet.Add to fail, wrong type")
}
@ -123,8 +129,11 @@ func TestVoteSet_AddVote_Bad(t *testing.T) {
}
func TestVoteSet_2_3Majority(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
height, round := int64(1), int32(0)
voteSet, _, privValidators := randVoteSet(height, round, tmproto.PrevoteType, 10, 1)
voteSet, _, privValidators := randVoteSet(ctx, t, height, round, tmproto.PrevoteType, 10, 1)
voteProto := &Vote{
ValidatorAddress: nil, // NOTE: must fill in
@ -137,11 +146,11 @@ func TestVoteSet_2_3Majority(t *testing.T) {
}
// 6 out of 10 voted for nil.
for i := int32(0); i < 6; i++ {
pubKey, err := privValidators[i].GetPubKey(context.Background())
pubKey, err := privValidators[i].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, i)
_, err = signAddVote(privValidators[i], vote, voteSet)
_, err = signAddVote(ctx, privValidators[i], vote, voteSet)
require.NoError(t, err)
}
blockID, ok := voteSet.TwoThirdsMajority()
@ -149,11 +158,11 @@ func TestVoteSet_2_3Majority(t *testing.T) {
// 7th validator voted for some blockhash
{
pubKey, err := privValidators[6].GetPubKey(context.Background())
pubKey, err := privValidators[6].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, 6)
_, err = signAddVote(privValidators[6], withBlockHash(vote, tmrand.Bytes(32)), voteSet)
_, err = signAddVote(ctx, privValidators[6], withBlockHash(vote, tmrand.Bytes(32)), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(), "there should be no 2/3 majority")
@ -161,11 +170,11 @@ func TestVoteSet_2_3Majority(t *testing.T) {
// 8th validator voted for nil.
{
pubKey, err := privValidators[7].GetPubKey(context.Background())
pubKey, err := privValidators[7].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, 7)
_, err = signAddVote(privValidators[7], vote, voteSet)
_, err = signAddVote(ctx, privValidators[7], vote, voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.True(t, ok || blockID.IsZero(), "there should be 2/3 majority for nil")
@ -173,8 +182,11 @@ func TestVoteSet_2_3Majority(t *testing.T) {
}
func TestVoteSet_2_3MajorityRedux(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
height, round := int64(1), int32(0)
voteSet, _, privValidators := randVoteSet(height, round, tmproto.PrevoteType, 100, 1)
voteSet, _, privValidators := randVoteSet(ctx, t, height, round, tmproto.PrevoteType, 100, 1)
blockHash := crypto.CRandBytes(32)
blockPartsTotal := uint32(123)
@ -192,11 +204,11 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
// 66 out of 100 voted for nil.
for i := int32(0); i < 66; i++ {
pubKey, err := privValidators[i].GetPubKey(context.Background())
pubKey, err := privValidators[i].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, i)
_, err = signAddVote(privValidators[i], vote, voteSet)
_, err = signAddVote(ctx, privValidators[i], vote, voteSet)
require.NoError(t, err)
}
blockID, ok := voteSet.TwoThirdsMajority()
@ -205,11 +217,11 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
// 67th validator voted for nil
{
pubKey, err := privValidators[66].GetPubKey(context.Background())
pubKey, err := privValidators[66].GetPubKey(ctx)
require.NoError(t, err)
adrr := pubKey.Address()
vote := withValidator(voteProto, adrr, 66)
_, err = signAddVote(privValidators[66], withBlockHash(vote, nil), voteSet)
_, err = signAddVote(ctx, privValidators[66], withBlockHash(vote, nil), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
@ -218,12 +230,12 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
// 68th validator voted for a different BlockParts PartSetHeader
{
pubKey, err := privValidators[67].GetPubKey(context.Background())
pubKey, err := privValidators[67].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, 67)
blockPartsHeader := PartSetHeader{blockPartsTotal, crypto.CRandBytes(32)}
_, err = signAddVote(privValidators[67], withBlockPartSetHeader(vote, blockPartsHeader), voteSet)
_, err = signAddVote(ctx, privValidators[67], withBlockPartSetHeader(vote, blockPartsHeader), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
@ -232,12 +244,12 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
// 69th validator voted for different BlockParts Total
{
pubKey, err := privValidators[68].GetPubKey(context.Background())
pubKey, err := privValidators[68].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, 68)
blockPartsHeader := PartSetHeader{blockPartsTotal + 1, blockPartSetHeader.Hash}
_, err = signAddVote(privValidators[68], withBlockPartSetHeader(vote, blockPartsHeader), voteSet)
_, err = signAddVote(ctx, privValidators[68], withBlockPartSetHeader(vote, blockPartsHeader), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
@ -246,11 +258,11 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
// 70th validator voted for different BlockHash
{
pubKey, err := privValidators[69].GetPubKey(context.Background())
pubKey, err := privValidators[69].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, 69)
_, err = signAddVote(privValidators[69], withBlockHash(vote, tmrand.Bytes(32)), voteSet)
_, err = signAddVote(ctx, privValidators[69], withBlockHash(vote, tmrand.Bytes(32)), voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.False(t, ok || !blockID.IsZero(),
@ -259,11 +271,11 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
// 71st validator voted for the right BlockHash & BlockPartSetHeader
{
pubKey, err := privValidators[70].GetPubKey(context.Background())
pubKey, err := privValidators[70].GetPubKey(ctx)
require.NoError(t, err)
addr := pubKey.Address()
vote := withValidator(voteProto, addr, 70)
_, err = signAddVote(privValidators[70], vote, voteSet)
_, err = signAddVote(ctx, privValidators[70], vote, voteSet)
require.NoError(t, err)
blockID, ok = voteSet.TwoThirdsMajority()
assert.True(t, ok && blockID.Equals(BlockID{blockHash, blockPartSetHeader}),
@ -272,8 +284,11 @@ func TestVoteSet_2_3MajorityRedux(t *testing.T) {
}
func TestVoteSet_Conflicts(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
height, round := int64(1), int32(0)
voteSet, _, privValidators := randVoteSet(height, round, tmproto.PrevoteType, 4, 1)
voteSet, _, privValidators := randVoteSet(ctx, t, height, round, tmproto.PrevoteType, 4, 1)
blockHash1 := tmrand.Bytes(32)
blockHash2 := tmrand.Bytes(32)
@ -287,14 +302,14 @@ func TestVoteSet_Conflicts(t *testing.T) {
BlockID: BlockID{nil, PartSetHeader{}},
}
val0, err := privValidators[0].GetPubKey(context.Background())
val0, err := privValidators[0].GetPubKey(ctx)
require.NoError(t, err)
val0Addr := val0.Address()
// val0 votes for nil.
{
vote := withValidator(voteProto, val0Addr, 0)
added, err := signAddVote(privValidators[0], vote, voteSet)
added, err := signAddVote(ctx, privValidators[0], vote, voteSet)
if !added || err != nil {
t.Errorf("expected VoteSet.Add to succeed")
}
@ -303,7 +318,7 @@ func TestVoteSet_Conflicts(t *testing.T) {
// val0 votes again for blockHash1.
{
vote := withValidator(voteProto, val0Addr, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, blockHash1), voteSet)
added, err := signAddVote(ctx, privValidators[0], withBlockHash(vote, blockHash1), voteSet)
assert.False(t, added, "conflicting vote")
assert.Error(t, err, "conflicting vote")
}
@ -315,7 +330,7 @@ func TestVoteSet_Conflicts(t *testing.T) {
// val0 votes again for blockHash1.
{
vote := withValidator(voteProto, val0Addr, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, blockHash1), voteSet)
added, err := signAddVote(ctx, privValidators[0], withBlockHash(vote, blockHash1), voteSet)
assert.True(t, added, "called SetPeerMaj23()")
assert.Error(t, err, "conflicting vote")
}
@ -327,18 +342,18 @@ func TestVoteSet_Conflicts(t *testing.T) {
// val0 votes again for blockHash1.
{
vote := withValidator(voteProto, val0Addr, 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, blockHash2), voteSet)
added, err := signAddVote(ctx, privValidators[0], withBlockHash(vote, blockHash2), voteSet)
assert.False(t, added, "duplicate SetPeerMaj23() from peerA")
assert.Error(t, err, "conflicting vote")
}
// val1 votes for blockHash1.
{
pv, err := privValidators[1].GetPubKey(context.Background())
pv, err := privValidators[1].GetPubKey(ctx)
assert.NoError(t, err)
addr := pv.Address()
vote := withValidator(voteProto, addr, 1)
added, err := signAddVote(privValidators[1], withBlockHash(vote, blockHash1), voteSet)
added, err := signAddVote(ctx, privValidators[1], withBlockHash(vote, blockHash1), voteSet)
if !added || err != nil {
t.Errorf("expected VoteSet.Add to succeed")
}
@ -354,11 +369,11 @@ func TestVoteSet_Conflicts(t *testing.T) {
// val2 votes for blockHash2.
{
pv, err := privValidators[2].GetPubKey(context.Background())
pv, err := privValidators[2].GetPubKey(ctx)
assert.NoError(t, err)
addr := pv.Address()
vote := withValidator(voteProto, addr, 2)
added, err := signAddVote(privValidators[2], withBlockHash(vote, blockHash2), voteSet)
added, err := signAddVote(ctx, privValidators[2], withBlockHash(vote, blockHash2), voteSet)
if !added || err != nil {
t.Errorf("expected VoteSet.Add to succeed")
}
@ -378,11 +393,11 @@ func TestVoteSet_Conflicts(t *testing.T) {
// val2 votes for blockHash1.
{
pv, err := privValidators[2].GetPubKey(context.Background())
pv, err := privValidators[2].GetPubKey(ctx)
assert.NoError(t, err)
addr := pv.Address()
vote := withValidator(voteProto, addr, 2)
added, err := signAddVote(privValidators[2], withBlockHash(vote, blockHash1), voteSet)
added, err := signAddVote(ctx, privValidators[2], withBlockHash(vote, blockHash1), voteSet)
assert.True(t, added)
assert.Error(t, err, "conflicting vote")
}
@ -401,8 +416,11 @@ func TestVoteSet_Conflicts(t *testing.T) {
}
func TestVoteSet_MakeCommit(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
height, round := int64(1), int32(0)
voteSet, _, privValidators := randVoteSet(height, round, tmproto.PrecommitType, 10, 1)
voteSet, _, privValidators := randVoteSet(ctx, t, height, round, tmproto.PrecommitType, 10, 1)
blockHash, blockPartSetHeader := crypto.CRandBytes(32), PartSetHeader{123, crypto.CRandBytes(32)}
voteProto := &Vote{
@ -417,11 +435,11 @@ func TestVoteSet_MakeCommit(t *testing.T) {
// 6 out of 10 voted for some block.
for i := int32(0); i < 6; i++ {
pv, err := privValidators[i].GetPubKey(context.Background())
pv, err := privValidators[i].GetPubKey(ctx)
assert.NoError(t, err)
addr := pv.Address()
vote := withValidator(voteProto, addr, i)
_, err = signAddVote(privValidators[i], vote, voteSet)
_, err = signAddVote(ctx, privValidators[i], vote, voteSet)
if err != nil {
t.Error(err)
}
@ -432,36 +450,36 @@ func TestVoteSet_MakeCommit(t *testing.T) {
// 7th voted for some other block.
{
pv, err := privValidators[6].GetPubKey(context.Background())
pv, err := privValidators[6].GetPubKey(ctx)
assert.NoError(t, err)
addr := pv.Address()
vote := withValidator(voteProto, addr, 6)
vote = withBlockHash(vote, tmrand.Bytes(32))
vote = withBlockPartSetHeader(vote, PartSetHeader{123, tmrand.Bytes(32)})
_, err = signAddVote(privValidators[6], vote, voteSet)
_, err = signAddVote(ctx, privValidators[6], vote, voteSet)
require.NoError(t, err)
}
// The 8th voted like everyone else.
{
pv, err := privValidators[7].GetPubKey(context.Background())
pv, err := privValidators[7].GetPubKey(ctx)
assert.NoError(t, err)
addr := pv.Address()
vote := withValidator(voteProto, addr, 7)
_, err = signAddVote(privValidators[7], vote, voteSet)
_, err = signAddVote(ctx, privValidators[7], vote, voteSet)
require.NoError(t, err)
}
// The 9th voted for nil.
{
pv, err := privValidators[8].GetPubKey(context.Background())
pv, err := privValidators[8].GetPubKey(ctx)
assert.NoError(t, err)
addr := pv.Address()
vote := withValidator(voteProto, addr, 8)
vote.BlockID = BlockID{}
_, err = signAddVote(privValidators[8], vote, voteSet)
_, err = signAddVote(ctx, privValidators[8], vote, voteSet)
require.NoError(t, err)
}
@ -478,13 +496,15 @@ func TestVoteSet_MakeCommit(t *testing.T) {
// NOTE: privValidators are in order
func randVoteSet(
ctx context.Context,
t testing.TB,
height int64,
round int32,
signedMsgType tmproto.SignedMsgType,
numValidators int,
votingPower int64,
) (*VoteSet, *ValidatorSet, []PrivValidator) {
valSet, privValidators := randValidatorPrivValSet(numValidators, votingPower)
valSet, privValidators := randValidatorPrivValSet(ctx, t, numValidators, votingPower)
return NewVoteSet("test_chain_id", height, round, signedMsgType, valSet), valSet, privValidators
}
@ -499,14 +519,15 @@ func deterministicVoteSet(
return NewVoteSet("test_chain_id", height, round, signedMsgType, valSet), valSet, privValidators
}
func randValidatorPrivValSet(numValidators int, votingPower int64) (*ValidatorSet, []PrivValidator) {
func randValidatorPrivValSet(ctx context.Context, t testing.TB, numValidators int, votingPower int64) (*ValidatorSet, []PrivValidator) {
var (
valz = make([]*Validator, numValidators)
privValidators = make([]PrivValidator, numValidators)
)
for i := 0; i < numValidators; i++ {
val, privValidator := randValidator(false, votingPower)
val, privValidator, err := randValidator(ctx, false, votingPower)
require.NoError(t, err)
valz[i] = val
privValidators[i] = privValidator
}


+ 49
- 32
types/vote_test.go View File

@ -16,19 +16,20 @@ import (
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
)
func examplePrevote() *Vote {
return exampleVote(byte(tmproto.PrevoteType))
func examplePrevote(t *testing.T) *Vote {
t.Helper()
return exampleVote(t, byte(tmproto.PrevoteType))
}
func examplePrecommit() *Vote {
return exampleVote(byte(tmproto.PrecommitType))
func examplePrecommit(t testing.TB) *Vote {
t.Helper()
return exampleVote(t, byte(tmproto.PrecommitType))
}
func exampleVote(t byte) *Vote {
func exampleVote(tb testing.TB, t byte) *Vote {
tb.Helper()
var stamp, err = time.Parse(TimeFormat, "2017-12-25T03:00:01.234Z")
if err != nil {
panic(err)
}
require.NoError(tb, err)
return &Vote{
Type: tmproto.SignedMsgType(t),
@ -46,9 +47,8 @@ func exampleVote(t byte) *Vote {
ValidatorIndex: 56789,
}
}
func TestVoteSignable(t *testing.T) {
vote := examplePrecommit()
vote := examplePrecommit(t)
v := vote.ToProto()
signBytes := VoteSignBytes("test_chain_id", v)
pb := CanonicalizeVote("test_chain_id", v)
@ -148,16 +148,19 @@ func TestVoteProposalNotEq(t *testing.T) {
}
func TestVoteVerifySignature(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
privVal := NewMockPV()
pubkey, err := privVal.GetPubKey(context.Background())
pubkey, err := privVal.GetPubKey(ctx)
require.NoError(t, err)
vote := examplePrecommit()
vote := examplePrecommit(t)
v := vote.ToProto()
signBytes := VoteSignBytes("test_chain_id", v)
// sign it
err = privVal.SignVote(context.Background(), "test_chain_id", v)
err = privVal.SignVote(ctx, "test_chain_id", v)
require.NoError(t, err)
// verify the same vote
@ -200,11 +203,14 @@ func TestIsVoteTypeValid(t *testing.T) {
}
func TestVoteVerify(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
privVal := NewMockPV()
pubkey, err := privVal.GetPubKey(context.Background())
pubkey, err := privVal.GetPubKey(ctx)
require.NoError(t, err)
vote := examplePrevote()
vote := examplePrevote(t)
vote.ValidatorAddress = pubkey.Address()
err = vote.Verify("test_chain_id", ed25519.GenPrivKey().PubKey())
@ -219,13 +225,13 @@ func TestVoteVerify(t *testing.T) {
}
func TestVoteString(t *testing.T) {
str := examplePrecommit().String()
str := examplePrecommit(t).String()
expected := `Vote{56789:6AF1F4111082 12345/02/SIGNED_MSG_TYPE_PRECOMMIT(Precommit) 8B01023386C3 000000000000 @ 2017-12-25T03:00:01.234Z}`
if str != expected {
t.Errorf("got unexpected string for Vote. Expected:\n%v\nGot:\n%v", expected, str)
}
str2 := examplePrevote().String()
str2 := examplePrevote(t).String()
expected = `Vote{56789:6AF1F4111082 12345/02/SIGNED_MSG_TYPE_PREVOTE(Prevote) 8B01023386C3 000000000000 @ 2017-12-25T03:00:01.234Z}`
if str2 != expected {
t.Errorf("got unexpected string for Vote. Expected:\n%v\nGot:\n%v", expected, str2)
@ -254,9 +260,12 @@ func TestVoteValidateBasic(t *testing.T) {
for _, tc := range testCases {
tc := tc
t.Run(tc.testName, func(t *testing.T) {
vote := examplePrecommit()
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
vote := examplePrecommit(t)
v := vote.ToProto()
err := privVal.SignVote(context.Background(), "test_chain_id", v)
err := privVal.SignVote(ctx, "test_chain_id", v)
vote.Signature = v.Signature
require.NoError(t, err)
tc.malleateVote(vote)
@ -266,10 +275,13 @@ func TestVoteValidateBasic(t *testing.T) {
}
func TestVoteProtobuf(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
privVal := NewMockPV()
vote := examplePrecommit()
vote := examplePrecommit(t)
v := vote.ToProto()
err := privVal.SignVote(context.Background(), "test_chain_id", v)
err := privVal.SignVote(ctx, "test_chain_id", v)
vote.Signature = v.Signature
require.NoError(t, err)
@ -297,23 +309,22 @@ func TestVoteProtobuf(t *testing.T) {
var sink interface{}
var protoVote *tmproto.Vote
var sampleCommit *Commit
func init() {
protoVote = examplePrecommit().ToProto()
func getSampleCommit(ctx context.Context, t testing.TB) *Commit {
t.Helper()
lastID := makeBlockIDRandom()
voteSet, _, vals := randVoteSet(2, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(lastID, 2, 1, voteSet, vals, time.Now())
if err != nil {
panic(err)
}
sampleCommit = commit
voteSet, _, vals := randVoteSet(ctx, t, 2, 1, tmproto.PrecommitType, 10, 1)
commit, err := makeCommit(ctx, lastID, 2, 1, voteSet, vals, time.Now())
require.NoError(t, err)
return commit
}
func BenchmarkVoteSignBytes(b *testing.B) {
protoVote := examplePrecommit(b).ToProto()
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
sink = VoteSignBytes("test_chain_id", protoVote)
@ -328,7 +339,13 @@ func BenchmarkVoteSignBytes(b *testing.B) {
}
func BenchmarkCommitVoteSignBytes(b *testing.B) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
sampleCommit := getSampleCommit(ctx, b)
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
for index := range sampleCommit.Signatures {


Loading…
Cancel
Save