diff --git a/CHANGELOG.md b/CHANGELOG.md index 962a98d088..8c94e6c58e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,28 @@ +## [1.3.0] - 2024-09-19 + +### Bug Fixes + +- Address already in use (#845) +- Active validators not always connected to each other (#844) +- Validators form islands on genesis (#850) +- Panic on block_results when consensus params change (#923) + +### Features + +- [**breaking**] Replace dash core quorum sign with quorum platformsign (#828) +- Allow overriding genesis time in InitChain (#847) + +### Miscellaneous Tasks + +- Update changelog and version to 1.1.0-dev.1 (#842) +- Update changelog and version to 1.1.0-dev.2 (#846) +- Update changelog and version to 1.1.0-dev.3 (#848) + +### Build + +- Bump bufbuild/buf-setup-action from 1.33.0 to 1.35.0 (#841) +- Run dependabot on default branch, not master (#843) + ## [1.2.1] - 2024-08-29 ### Bug Fixes @@ -8,6 +33,7 @@ ### Miscellaneous Tasks - Update changelog and version to 1.2.1-dev.1 +- Update changelog and version to 1.2.1 ### Build diff --git a/abci/example/kvstore/verify_test.go b/abci/example/kvstore/verify_test.go index 29c8a8207c..14460650c5 100644 --- a/abci/example/kvstore/verify_test.go +++ b/abci/example/kvstore/verify_test.go @@ -85,7 +85,7 @@ func (e *blockExecutor) createBlock(txs types.Txs, commit *types.Commit) *types. if commit == nil { commit = &types.Commit{} } - proposer := e.state.Validators.GetProposer() + proposer := e.state.GetProposerFromState(e.state.LastBlockHeight+1, 0) block := e.state.MakeBlock( e.state.LastBlockHeight+1, txs, diff --git a/dash/quorum/validator_conn_executor_test.go b/dash/quorum/validator_conn_executor_test.go index 8ea0bc2591..4430648700 100644 --- a/dash/quorum/validator_conn_executor_test.go +++ b/dash/quorum/validator_conn_executor_test.go @@ -621,7 +621,7 @@ func makeState(nVals int, height int64) (sm.State, dbm.DB) { } func makeBlock(ctx context.Context, t *testing.T, blockExec *sm.BlockExecutor, state sm.State, _height int64, commit *types.Commit) *types.Block { - block, crs, err := blockExec.CreateProposalBlock(ctx, 1, 0, state, commit, state.Validators.Proposer.ProTxHash, 1) + block, crs, err := blockExec.CreateProposalBlock(ctx, 1, 0, state, commit, state.Validators.Proposer().ProTxHash, 1) require.NoError(t, err) err = crs.UpdateBlock(block) diff --git a/dash/quorum/validator_map_test.go b/dash/quorum/validator_map_test.go index 38a3debc47..800cb6cb3e 100644 --- a/dash/quorum/validator_map_test.go +++ b/dash/quorum/validator_map_test.go @@ -3,9 +3,10 @@ package quorum import ( "testing" + "github.com/stretchr/testify/assert" + "github.com/dashpay/tenderdash/dash/quorum/mock" "github.com/dashpay/tenderdash/types" - "github.com/stretchr/testify/assert" ) func Test_validatorMap_String(t *testing.T) { @@ -30,11 +31,11 @@ func Test_validatorMap_String(t *testing.T) { { vm: newValidatorMap(vals), contains: []string{ - " VP:0 A:0 N:tcp://0100000000000000000000000000000000000000@127.0.0.1:1}", - " VP:0 A:0 N:tcp://0200000000000000000000000000000000000000@127.0.0.1:2}", - " VP:0 A:0 N:tcp://0300000000000000000000000000000000000000@127.0.0.1:3}", - " VP:0 A:0 N:tcp://0400000000000000000000000000000000000000@127.0.0.1:4}", - " VP:0 A:0 N:tcp://0500000000000000000000000000000000000000@127.0.0.1:5}", + " VP:0 N:tcp://0100000000000000000000000000000000000000@127.0.0.1:1}", + " VP:0 N:tcp://0200000000000000000000000000000000000000@127.0.0.1:2}", + " VP:0 N:tcp://0300000000000000000000000000000000000000@127.0.0.1:3}", + " VP:0 N:tcp://0400000000000000000000000000000000000000@127.0.0.1:4}", + " VP:0 N:tcp://0500000000000000000000000000000000000000@127.0.0.1:5}", }, }, } diff --git a/internal/consensus/gossiper.go b/internal/consensus/gossiper.go index c1ec94adb0..fda600cced 100644 --- a/internal/consensus/gossiper.go +++ b/internal/consensus/gossiper.go @@ -203,10 +203,10 @@ func (g *msgGossiper) GossipCommit(ctx context.Context, rs cstypes.RoundState, p if prs.HasCommit { return } - logger := g.logger.With([]any{ + logger := g.logger.With( "height", rs.Height, "peer_height", prs.Height, - }) + ) var commit *types.Commit blockStoreBase := g.blockStore.Base() if rs.Height == prs.Height+1 { diff --git a/internal/consensus/msgs_test.go b/internal/consensus/msgs_test.go index 419102e828..2dea8c9bed 100644 --- a/internal/consensus/msgs_test.go +++ b/internal/consensus/msgs_test.go @@ -78,7 +78,7 @@ func TestMsgToProto(t *testing.T) { vote, err := factory.MakeVote( ctx, pv, - &types.ValidatorSet{Proposer: val, Validators: []*types.Validator{val}, QuorumHash: quorumHash, ThresholdPublicKey: pk}, + &types.ValidatorSet{Validators: []*types.Validator{val}, QuorumHash: quorumHash, ThresholdPublicKey: pk}, "chainID", 0, 1, diff --git a/internal/consensus/pbts_test.go b/internal/consensus/pbts_test.go index 2811f296e4..1dc58ad2d3 100644 --- a/internal/consensus/pbts_test.go +++ b/internal/consensus/pbts_test.go @@ -293,7 +293,7 @@ type timestampedEvent struct { func (p *pbtsTestHarness) pickProposer() types.PrivValidator { stateData := p.observedState.GetStateData() - proposer := stateData.Validators.GetProposer() + proposer := stateData.ProposerSelector.MustGetProposer(p.currentHeight, p.currentRound) p.observedState.logger.Debug("picking proposer", "protxhash", proposer.ProTxHash) allVals := append(p.otherValidators, p.observedValidator) diff --git a/internal/consensus/peer_state.go b/internal/consensus/peer_state.go index 8ab8f90be6..77844c395d 100644 --- a/internal/consensus/peer_state.go +++ b/internal/consensus/peer_state.go @@ -14,6 +14,7 @@ import ( cstypes "github.com/dashpay/tenderdash/internal/consensus/types" "github.com/dashpay/tenderdash/libs/bits" "github.com/dashpay/tenderdash/libs/log" + "github.com/dashpay/tenderdash/libs/math" tmproto "github.com/dashpay/tenderdash/proto/tendermint/types" "github.com/dashpay/tenderdash/types" ) @@ -214,7 +215,11 @@ func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (*types.Vote, boo } if index, ok := votes.BitArray().Sub(psVotes).PickRandom(); ok { - vote := votes.GetByIndex(int32(index)) + idx, err := math.SafeConvertInt32(int64(index)) + if err != nil { + panic(fmt.Errorf("failed to convert index to int32: %w", err)) + } + vote := votes.GetByIndex(idx) if vote != nil { return vote, true } diff --git a/internal/consensus/reactor.go b/internal/consensus/reactor.go index 222f92e978..3b6e165290 100644 --- a/internal/consensus/reactor.go +++ b/internal/consensus/reactor.go @@ -223,7 +223,7 @@ func (r *Reactor) SwitchToConsensus(ctx context.Context, state sm.State, skipWAL // NOTE: The line below causes broadcastNewRoundStepRoutine() to broadcast a // NewRoundStepMessage. - stateData.updateToState(state, nil) + stateData.updateToState(state, nil, r.state.blockStore) err := r.state.stateDataStore.Update(stateData) if err != nil { panic(err) diff --git a/internal/consensus/replay.go b/internal/consensus/replay.go index d0b769b428..960585bb6c 100644 --- a/internal/consensus/replay.go +++ b/internal/consensus/replay.go @@ -74,7 +74,6 @@ func (cs *State) readReplayMessage(ctx context.Context, msg *TimedWALMessage, ne case *ProposalMessage: p := msg.Proposal if cs.config.WalSkipRoundsToLast && p.Round > stateData.Round { - stateData.Validators.IncrementProposerPriority(p.Round - stateData.Round) stateData.Votes.SetRound(p.Round) stateData.Round = p.Round } diff --git a/internal/consensus/replay_test.go b/internal/consensus/replay_test.go index ff6efde35f..671329c9ed 100644 --- a/internal/consensus/replay_test.go +++ b/internal/consensus/replay_test.go @@ -495,7 +495,7 @@ func createSignSendProposal(ctx context.Context, height := stateData.RoundState.Height round := stateData.RoundState.Round - proposer := stateData.Validators.GetProposer() + proposer := stateData.ProposerSelector.MustGetProposer(height, round) proposerVs := findValByProTxHash(ctx, t, vss, proposer.ProTxHash) proposerCs := findStateByProTxHash(t, css, proposer.ProTxHash) diff --git a/internal/consensus/replayer_test.go b/internal/consensus/replayer_test.go index cbff61ba2b..c9548a8628 100644 --- a/internal/consensus/replayer_test.go +++ b/internal/consensus/replayer_test.go @@ -188,7 +188,6 @@ func TestInitChainGenesisTime(t *testing.T) { ) require.NoError(t, err) vset.ThresholdPublicKey = recoveredThresholdPublicKey - proposerProTxHash := vset.GetProposer().ProTxHash genDoc := tmtypes.GenesisDoc{ ChainID: "test-chain", @@ -208,6 +207,8 @@ func TestInitChainGenesisTime(t *testing.T) { require.NoError(t, err) stateStore := sm.NewStore(dbm.NewMemDB()) blockStore := store.NewBlockStore(dbm.NewMemDB()) + proposer := smState.GetProposerFromState(1, 0) + proposerProTxHash := proposer.ProTxHash replayer := newBlockReplayer(stateStore, blockStore, &genDoc, eventBus, proxyApp, proposerProTxHash) // use replayer to call initChain diff --git a/internal/consensus/state.go b/internal/consensus/state.go index 5ad2522472..ab7d6b086d 100644 --- a/internal/consensus/state.go +++ b/internal/consensus/state.go @@ -334,7 +334,7 @@ func (cs *State) updateStateFromStore() error { } } - stateData.updateToState(state, nil) + stateData.updateToState(state, nil, cs.blockStore) err = cs.stateDataStore.Update(stateData) if err != nil { return err diff --git a/internal/consensus/state_apply_commit.go b/internal/consensus/state_apply_commit.go index 4fa2d9ad61..1ce9d8d3fa 100644 --- a/internal/consensus/state_apply_commit.go +++ b/internal/consensus/state_apply_commit.go @@ -34,9 +34,6 @@ func (c *ApplyCommitAction) Execute(ctx context.Context, stateEvent StateEvent) event := stateEvent.Data.(*ApplyCommitEvent) stateData := stateEvent.StateData commit := event.Commit - c.logger.Info("applying commit", "commit", commit) - - block, blockParts := stateData.ProposalBlock, stateData.ProposalBlockParts height := stateData.Height round := stateData.Round @@ -45,6 +42,9 @@ func (c *ApplyCommitAction) Execute(ctx context.Context, stateEvent StateEvent) height = commit.Height round = commit.Round } + c.logger.Info("applying commit", "commit", commit, "height", height, "round", round) + + block, blockParts := stateData.ProposalBlock, stateData.ProposalBlockParts c.blockExec.mustEnsureProcess(ctx, &stateData.RoundState, round) c.blockExec.mustValidate(ctx, stateData) @@ -88,7 +88,7 @@ func (c *ApplyCommitAction) Execute(ctx context.Context, stateEvent StateEvent) c.RecordMetrics(stateData, height, block, lastBlockMeta) // NewHeightStep! - stateData.updateToState(stateCopy, commit) + stateData.updateToState(stateCopy, commit, c.blockStore) err = stateData.Save() if err != nil { return err diff --git a/internal/consensus/state_data.go b/internal/consensus/state_data.go index 13739ad48f..0686b0f32d 100644 --- a/internal/consensus/state_data.go +++ b/internal/consensus/state_data.go @@ -9,6 +9,7 @@ import ( "github.com/dashpay/tenderdash/config" cstypes "github.com/dashpay/tenderdash/internal/consensus/types" + selectproposer "github.com/dashpay/tenderdash/internal/consensus/versioned/selectproposer" sm "github.com/dashpay/tenderdash/internal/state" "github.com/dashpay/tenderdash/libs/eventemitter" "github.com/dashpay/tenderdash/libs/log" @@ -144,7 +145,12 @@ func (s *StateData) Save() error { } func (s *StateData) isProposer(proTxHash types.ProTxHash) bool { - return proTxHash != nil && bytes.Equal(s.Validators.GetProposer().ProTxHash.Bytes(), proTxHash.Bytes()) + prop, err := s.ProposerSelector.GetProposer(s.Height, s.Round) + if err != nil { + s.logger.Error("error getting proposer", "err", err, "height", s.Height, "round", s.Round) + return false + } + return proTxHash != nil && bytes.Equal(prop.ProTxHash.Bytes(), proTxHash.Bytes()) } func (s *StateData) isValidator(proTxHash types.ProTxHash) bool { @@ -177,11 +183,17 @@ func (s *StateData) updateRoundStep(round int32, step cstypes.RoundStepType) { } s.Round = round s.Step = step + + if err := s.ProposerSelector.UpdateHeightRound(s.Height, round); err != nil { + s.logger.Error("error updating proposer scores", + "height", s.Height, "round", round, + "err", err) + } } // Updates State and increments height to match that of state. // The round becomes 0 and cs.Step becomes cstypes.RoundStepNewHeight. -func (s *StateData) updateToState(state sm.State, commit *types.Commit) { +func (s *StateData) updateToState(state sm.State, commit *types.Commit, blockStore selectproposer.BlockStore) { if s.CommitRound > -1 && 0 < s.Height && s.Height != state.LastBlockHeight { panic(fmt.Sprintf( "updateToState() expected state height of %v but found %v", @@ -221,7 +233,6 @@ func (s *StateData) updateToState(state sm.State, commit *types.Commit) { } // Reset fields based on state. - validators := state.Validators switch { case state.LastBlockHeight == 0: // Very first commit should be empty. @@ -252,11 +263,31 @@ func (s *StateData) updateToState(state sm.State, commit *types.Commit) { height = state.InitialHeight } - s.logger.Trace("updating state height", "newHeight", height) + // state.Validators contain validator set at (state.LastBlockHeight+1, 0) + validators := state.Validators - // RoundState fields - s.updateHeight(height) - s.updateRoundStep(0, cstypes.RoundStepNewHeight) + if s.Validators == nil || !bytes.Equal(s.Validators.QuorumHash, validators.QuorumHash) { + s.logger.Info("Updating validators", "from", s.Validators.BasicInfoString(), + "to", validators.BasicInfoString()) + } + + s.Validators = validators + var err error + + s.ProposerSelector, err = selectproposer.NewProposerSelector( + state.ConsensusParams, + s.Validators, + height, + 0, + blockStore, + s.logger, + ) + if err != nil { + s.logger.Error("error creating proposer selector", "height", height, "round", 0, "validators", s.Validators, "err", err) + panic(fmt.Sprintf("error creating proposer selector: %v", err)) + } + + s.logger.Trace("updating state height", "newHeight", height) if s.CommitTime.IsZero() { // "Now" makes it easier to sync up dev nodes. @@ -265,12 +296,6 @@ func (s *StateData) updateToState(state sm.State, commit *types.Commit) { s.StartTime = s.CommitTime } - if s.Validators == nil || !bytes.Equal(s.Validators.QuorumHash, validators.QuorumHash) { - s.logger.Info("Updating validators", "from", s.Validators.BasicInfoString(), - "to", validators.BasicInfoString()) - } - - s.Validators = validators s.Proposal = nil s.ProposalReceiveTime = time.Time{} s.ProposalBlock = nil @@ -288,6 +313,10 @@ func (s *StateData) updateToState(state sm.State, commit *types.Commit) { s.TriggeredTimeoutPrecommit = false s.state = state + + // RoundState fields + s.updateHeight(height) + s.updateRoundStep(0, cstypes.RoundStepNewHeight) } func (s *StateData) updateHeight(height int64) { diff --git a/internal/consensus/state_enter_new_round.go b/internal/consensus/state_enter_new_round.go index d38bed9e90..2aa9f67561 100644 --- a/internal/consensus/state_enter_new_round.go +++ b/internal/consensus/state_enter_new_round.go @@ -66,18 +66,10 @@ func (c *EnterNewRoundAction) Execute(ctx context.Context, stateEvent StateEvent "round", stateData.Round, "step", stateData.Step) - // increment validators if necessary - validators := stateData.Validators - if stateData.Round < round { - validators = validators.Copy() - validators.IncrementProposerPriority(round - stateData.Round) - } - // Setup new round // we don't fire newStep for this step, // but we fire an event, so update the round step first stateData.updateRoundStep(round, cstypes.RoundStepNewRound) - stateData.Validators = validators if round == 0 { // We've already reset these upon new height, // and meanwhile we might have received a proposal diff --git a/internal/consensus/state_enter_propose.go b/internal/consensus/state_enter_propose.go index e9ae2075c8..db23ef2248 100644 --- a/internal/consensus/state_enter_propose.go +++ b/internal/consensus/state_enter_propose.go @@ -84,11 +84,14 @@ func (c *EnterProposeAction) Execute(ctx context.Context, stateEvent StateEvent) c.scheduler.ScheduleTimeout(stateData.proposeTimeout(round), height, round, cstypes.RoundStepPropose) if !isProposer { + prop, err := stateData.ProposerSelector.GetProposer(stateData.Height, stateData.Round) + if err != nil { + logger.Error("failed to get proposer", "err", err) + return nil // not a critical error, as we don't propose anyway + } logger.Info("propose step; not our turn to propose", - "proposer_proTxHash", stateData.Validators.GetProposer().ProTxHash, - "node_proTxHash", proTxHash.String(), - "height", stateData.Height, - "round", stateData.Round, + "proposer_proTxHash", prop.ProTxHash.ShortString(), + "node_proTxHash", proTxHash.ShortString(), "step", stateData.Step) return nil } @@ -99,9 +102,8 @@ func (c *EnterProposeAction) Execute(ctx context.Context, stateEvent StateEvent) } logger.Info("propose step; our turn to propose", + "node_proTxHash", proTxHash.ShortString(), "proposer_proTxHash", proTxHash.ShortString(), - "height", stateData.Height, - "round", stateData.Round, "step", stateData.Step, ) // Flush the WAL. Otherwise, we may not recompute the same proposal to sign, diff --git a/internal/consensus/state_proposaler.go b/internal/consensus/state_proposaler.go index 75c0b6942e..04a19423a4 100644 --- a/internal/consensus/state_proposaler.go +++ b/internal/consensus/state_proposaler.go @@ -82,7 +82,11 @@ func (p *Proposaler) Set(proposal *types.Proposal, receivedAt time.Time, rs *cst rs.ProposalBlockParts = types.NewPartSetFromHeader(proposal.BlockID.PartSetHeader) } - p.logger.Info("received proposal", "proposal", proposal, "received", receivedAt) + p.logger.Info("received proposal", + "proposal", proposal, + "height", proposal.Height, + "round", proposal.Round, + "received", receivedAt) return nil } @@ -209,6 +213,11 @@ func (p *Proposaler) sendMessages(ctx context.Context, msgs ...Message) { } func (p *Proposaler) verifyProposal(proposal *types.Proposal, rs *cstypes.RoundState) error { + if proposal.Height != rs.Height || proposal.Round != rs.Round { + return fmt.Errorf("proposal for invalid height/round, proposal height %d, round %d, expected height %d, round %d", + proposal.Height, proposal.Round, rs.Height, rs.Round) + } + protoProposal := proposal.ToProto() stateValSet := p.committedState.Validators // Verify signature @@ -218,8 +227,12 @@ func (p *Proposaler) verifyProposal(proposal *types.Proposal, rs *cstypes.RoundS stateValSet.QuorumType, stateValSet.QuorumHash, ) - vset := rs.Validators - proposer := vset.GetProposer() + + proposer, err := rs.ProposerSelector.GetProposer(rs.Height, rs.Round) + if err != nil { + return fmt.Errorf("error getting proposer: %w", err) + } + if proposer.PubKey == nil { return p.verifyProposalForNonValidatorSet(proposal, *rs) } @@ -251,11 +264,18 @@ func (p *Proposaler) verifyProposalForNonValidatorSet(proposal *types.Proposal, // We might have a commit already for the Round State // We need to verify that the commit block id is equal to the proposal block id if !proposal.BlockID.Equals(commit.BlockID) { - proposer := rs.Validators.GetProposer() - p.logger.Error("proposal blockID isn't the same as the commit blockID", - "height", proposal.Height, - "round", proposal.Round, - "proposer_proTxHash", proposer.ProTxHash.ShortString()) + proposer, err := rs.ProposerSelector.GetProposer(proposal.Height, proposal.Round) + if err != nil { + p.logger.Error("error getting proposer", + "height", proposal.Height, + "round", proposal.Round, + "err", err) + } else { + p.logger.Error("proposal blockID isn't the same as the commit blockID", + "height", proposal.Height, + "round", proposal.Round, + "proposer_proTxHash", proposer.ProTxHash.ShortString()) + } return ErrInvalidProposalForCommit } return nil diff --git a/internal/consensus/state_proposaler_test.go b/internal/consensus/state_proposaler_test.go index 9d98ddf956..b49745f796 100644 --- a/internal/consensus/state_proposaler_test.go +++ b/internal/consensus/state_proposaler_test.go @@ -7,9 +7,11 @@ import ( "time" "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" cstypes "github.com/dashpay/tenderdash/internal/consensus/types" + selectproposer "github.com/dashpay/tenderdash/internal/consensus/versioned/selectproposer" sm "github.com/dashpay/tenderdash/internal/state" "github.com/dashpay/tenderdash/internal/state/mocks" "github.com/dashpay/tenderdash/internal/test/factory" @@ -23,6 +25,7 @@ type ProposalerTestSuite struct { suite.Suite proposer *Proposaler + proposerSelector selectproposer.ProposerSelector mockBlockExec *mocks.Executor mockPrivVals []types.PrivValidator mockValSet *types.ValidatorSet @@ -87,7 +90,20 @@ func (suite *ProposalerTestSuite) SetupTest() { blockExec: blockExec, committedState: suite.committedState, } - suite.proposerProTxHash = suite.mockValSet.GetProposer().ProTxHash + var err error + suite.proposerSelector, err = selectproposer.NewProposerSelector( + suite.committedState.ConsensusParams, + valSet, + 100, + 0, + nil, + logger, + ) + if err != nil { + panic(fmt.Errorf("failed to create validator scoring strategy: %w", err)) + } + + suite.proposerProTxHash = suite.proposerSelector.MustGetProposer(100, 0).ProTxHash suite.blockH100R0 = suite.committedState.MakeBlock(100, []types.Tx{}, &suite.commitH99R0, nil, suite.proposerProTxHash, 0) } @@ -145,7 +161,11 @@ func (suite *ProposalerTestSuite) TestSet() { wantErr: ErrInvalidProposalPOLRound.Error(), }, { - rs: cstypes.RoundState{Height: 100, Round: 0, Validators: suite.mockValSet}, + rs: cstypes.RoundState{Height: 100, + Round: 0, + Validators: suite.mockValSet, + ProposerSelector: suite.proposerSelector, + }, proposal: *proposalH100R0, receivedAt: receivedAt, wantProposal: proposalH100R0, @@ -170,6 +190,8 @@ func (suite *ProposalerTestSuite) TestDecide() { state := suite.committedState proposalH100R0 := types.NewProposal(100, state.LastCoreChainLockedBlockHeight, 0, 0, blockID, suite.blockH100R0.Header.Time) suite.signProposal(ctx, proposalH100R0) + vs, err := selectproposer.NewProposerSelector(types.ConsensusParams{}, suite.mockValSet, 100, 0, nil, nil) + suite.Require().NoError(err) testCases := []struct { height int64 round int32 @@ -182,12 +204,13 @@ func (suite *ProposalerTestSuite) TestDecide() { height: 100, round: 0, rs: cstypes.RoundState{ - Height: 100, - Round: 0, - Validators: suite.mockValSet, - ValidBlock: nil, - LastCommit: &suite.commitH99R0, - ValidRound: 0, + Height: 100, + Round: 0, + Validators: suite.mockValSet, + ValidBlock: nil, + LastCommit: &suite.commitH99R0, + ValidRound: 0, + ProposerSelector: vs, }, mockFn: func(rs cstypes.RoundState) { suite.mockBlockExec. @@ -216,6 +239,7 @@ func (suite *ProposalerTestSuite) TestDecide() { ValidBlockRecvTime: suite.blockH100R0.Time.Add(100 * time.Millisecond), LastCommit: &suite.commitH99R0, ValidRound: 0, + ProposerSelector: vs, }, wantProposal: proposalH100R0, }, @@ -252,10 +276,17 @@ func (suite *ProposalerTestSuite) TestVerifyProposal() { suite.signProposal(ctx, proposalH100R0) proposalH100R0wrongSig := *proposalH100R0 proposalH100R0wrongSig.Signature = make([]byte, 96) - valSet := *suite.mockValSet - proposer := valSet.Proposer.Copy() + valSet := *suite.mockValSet.Copy() + proposer := valSet.Proposer() proposer.PubKey = nil - valSet.Proposer = proposer + idx, _ := valSet.GetByProTxHash(proposer.ProTxHash) + valSet.Validators[idx] = proposer + + proposerSelectorNoPubkeys, err := selectproposer.NewProposerSelector(state.ConsensusParams, &valSet, + 100, 0, + nil, nil) + require.NoError(suite.T(), err) + testCases := []struct { proposal *types.Proposal rs cstypes.RoundState @@ -264,53 +295,74 @@ func (suite *ProposalerTestSuite) TestVerifyProposal() { { proposal: proposalH100R0, rs: cstypes.RoundState{ - Validators: suite.mockValSet, + Validators: suite.mockValSet, + ProposerSelector: suite.proposerSelector, + Height: proposalH100R0.Height, + Round: proposalH100R0.Round, }, }, { proposal: &proposalH100R0wrongSig, rs: cstypes.RoundState{ - Validators: suite.mockValSet, + Validators: suite.mockValSet, + ProposerSelector: suite.proposerSelector, + Height: proposalH100R0wrongSig.Height, + Round: proposalH100R0wrongSig.Round, }, wantErr: ErrInvalidProposalSignature.Error(), }, { proposal: proposalH100R0, rs: cstypes.RoundState{ - Commit: nil, - Validators: &valSet, + Commit: nil, + Validators: &valSet, + ProposerSelector: proposerSelectorNoPubkeys, + Height: proposalH100R0.Height, + Round: proposalH100R0.Round, }, wantErr: ErrUnableToVerifyProposal.Error(), }, { proposal: proposalH100R0, rs: cstypes.RoundState{ - Commit: &types.Commit{Height: 99}, - Validators: &valSet, + Commit: &types.Commit{Height: 99}, + Validators: &valSet, + ProposerSelector: proposerSelectorNoPubkeys, + Height: proposalH100R0.Height, + Round: proposalH100R0.Round, }, wantErr: ErrUnableToVerifyProposal.Error(), }, { proposal: proposalH100R0, rs: cstypes.RoundState{ - Commit: &types.Commit{Height: 100, Round: 1}, - Validators: &valSet, + Commit: &types.Commit{Height: 100, Round: 1}, + Validators: &valSet, + ProposerSelector: proposerSelectorNoPubkeys, + Height: proposalH100R0.Height, + Round: proposalH100R0.Round, }, wantErr: ErrUnableToVerifyProposal.Error(), }, { proposal: proposalH100R0, rs: cstypes.RoundState{ - Commit: &types.Commit{Height: 100, Round: 0, BlockID: types.BlockID{Hash: nil}}, - Validators: &valSet, + Commit: &types.Commit{Height: 100, Round: 0, BlockID: types.BlockID{Hash: nil}}, + Validators: &valSet, + ProposerSelector: proposerSelectorNoPubkeys, + Height: proposalH100R0.Height, + Round: proposalH100R0.Round, }, wantErr: ErrInvalidProposalForCommit.Error(), }, { proposal: proposalH100R0, rs: cstypes.RoundState{ - Commit: &types.Commit{Height: 100, Round: 0, BlockID: proposalH100R0.BlockID}, - Validators: &valSet, + Commit: &types.Commit{Height: 100, Round: 0, BlockID: proposalH100R0.BlockID}, + Validators: &valSet, + ProposerSelector: proposerSelectorNoPubkeys, + Height: proposalH100R0.Height, + Round: proposalH100R0.Round, }, }, } diff --git a/internal/consensus/state_test.go b/internal/consensus/state_test.go index 2c6cee49ba..5604c0cd44 100644 --- a/internal/consensus/state_test.go +++ b/internal/consensus/state_test.go @@ -92,7 +92,7 @@ func TestStateProposerSelection0(t *testing.T) { ensureNewRound(t, newRoundCh, height, round) // Commit a block and ensure proposer for the next height is correct. - prop := cs1.GetRoundState().Validators.GetProposer() + prop := stateData.ProposerSelector.MustGetProposer(height, round) proTxHash, err := cs1.privValidator.GetProTxHash(ctx) require.NoError(t, err) require.Truef(t, bytes.Equal(prop.ProTxHash, proTxHash), "expected proposer to be validator %d. Got %X", 0, prop.ProTxHash.ShortString()) @@ -106,7 +106,7 @@ func TestStateProposerSelection0(t *testing.T) { // Wait for new round so next validator is set. ensureNewRound(t, newRoundCh, height+1, 0) - prop = cs1.GetRoundState().Validators.GetProposer() + prop = stateData.ProposerSelector.MustGetProposer(height+1, 0) proTxHash, err = vss[1].GetProTxHash(ctx) require.NoError(t, err) require.True(t, bytes.Equal(prop.ProTxHash, proTxHash), "expected proposer to be validator %d. Got %X", 1, prop.ProTxHash.ShortString()) @@ -134,8 +134,8 @@ func TestStateProposerSelection2(t *testing.T) { // everyone just votes nil. we get a new proposer each round for i := int32(0); int(i) < len(vss); i++ { - prop := cs1.GetRoundState().Validators.GetProposer() - proTxHash, err := vss[int(i+round)%len(vss)].GetProTxHash(ctx) + prop := stateData.ProposerSelector.MustGetProposer(height, round) + proTxHash, err := vss[int(int32(height-1)+round)%len(vss)].GetProTxHash(ctx) require.NoError(t, err) correctProposer := proTxHash require.True(t, bytes.Equal(prop.ProTxHash, correctProposer), @@ -3252,7 +3252,7 @@ func TestStateTryAddCommitCallsProcessProposal(t *testing.T) { parts, err := block.MakePartSet(999999999) require.NoError(t, err) - peerID := css0StateData.Validators.Proposer.NodeAddress.NodeID + peerID := css0StateData.Validators.Proposer().NodeAddress.NodeID css1StateData.Proposal = proposal css1StateData.ProposalBlock = block css1StateData.ProposalBlockParts = parts diff --git a/internal/consensus/types/round_state.go b/internal/consensus/types/round_state.go index 1e93d8d8c7..a27f74af42 100644 --- a/internal/consensus/types/round_state.go +++ b/internal/consensus/types/round_state.go @@ -6,6 +6,7 @@ import ( "strconv" "time" + selectproposer "github.com/dashpay/tenderdash/internal/consensus/versioned/selectproposer" sm "github.com/dashpay/tenderdash/internal/state" "github.com/dashpay/tenderdash/libs/bytes" tmcons "github.com/dashpay/tenderdash/proto/tendermint/consensus" @@ -76,14 +77,15 @@ type RoundState struct { // Subjective time when +2/3 precommits for Block at Round were found CommitTime time.Time `json:"commit_time"` Validators *types.ValidatorSet `json:"validators"` - Proposal *types.Proposal `json:"proposal"` - ProposalReceiveTime time.Time `json:"proposal_receive_time"` - ProposalBlock *types.Block `json:"proposal_block"` - ProposalBlockParts *types.PartSet `json:"proposal_block_parts"` - LockedRound int32 `json:"locked_round"` - LockedBlock *types.Block `json:"locked_block"` - LockedBlockParts *types.PartSet `json:"locked_block_parts"` - Commit *types.Commit `json:"commit"` + ProposerSelector selectproposer.ProposerSelector + Proposal *types.Proposal `json:"proposal"` + ProposalReceiveTime time.Time `json:"proposal_receive_time"` + ProposalBlock *types.Block `json:"proposal_block"` + ProposalBlockParts *types.PartSet `json:"proposal_block_parts"` + LockedRound int32 `json:"locked_round"` + LockedBlock *types.Block `json:"locked_block"` + LockedBlockParts *types.PartSet `json:"locked_block_parts"` + Commit *types.Commit `json:"commit"` // The variables below starting with "Valid..." derive their name from // the algorithm presented in this paper: @@ -128,7 +130,12 @@ func (rs *RoundState) RoundStateSimple() RoundStateSimple { panic(err) } - proTxHash := rs.Validators.GetProposer().ProTxHash + proposer, err := rs.ProposerSelector.GetProposer(rs.Height, rs.Round) + if err != nil { + panic(err) + } + + proTxHash := proposer.ProTxHash idx, _ := rs.Validators.GetByProTxHash(proTxHash) return RoundStateSimple{ @@ -147,7 +154,11 @@ func (rs *RoundState) RoundStateSimple() RoundStateSimple { // NewRoundEvent returns the RoundState with proposer information as an event. func (rs *RoundState) NewRoundEvent() types.EventDataNewRound { - proTxHash := rs.Validators.GetProposer().ProTxHash + proposer, err := rs.ProposerSelector.GetProposer(rs.Height, rs.Round) + if err != nil { + panic(fmt.Errorf("failed to get proposer for height/round %d:%d: %v", rs.Height, rs.Round, err)) + } + proTxHash := proposer.ProTxHash idx, _ := rs.Validators.GetByProTxHash(proTxHash) return types.EventDataNewRound{ diff --git a/internal/consensus/versioned/selectproposer/height_proposer.go b/internal/consensus/versioned/selectproposer/height_proposer.go new file mode 100644 index 0000000000..4b2d87a691 --- /dev/null +++ b/internal/consensus/versioned/selectproposer/height_proposer.go @@ -0,0 +1,228 @@ +package selectproposer + +import ( + "fmt" + "sync" + "testing" + + "github.com/dashpay/tenderdash/libs/bytes" + "github.com/dashpay/tenderdash/libs/log" + "github.com/dashpay/tenderdash/types" +) + +type heightProposerSelector struct { + valSet *types.ValidatorSet + height int64 + bs BlockStore + logger log.Logger + mtx sync.Mutex +} + +// NewHeightProposerSelector creates a new height-based proposer selector. +// +// This selector goes through validators in a round-robin approach, increasing proposer index by 1 at each height. +// +// Subsequent rounds at the same height will select next proposer on the list, but not persist these changes, +// so that the proposer of height H and round 1 is selected again at height H+1 and round 0. +// +// It modifies `valSet` in place. +// +// ## Arguments +// +// * `vset` - the validator set; it must not be empty and can be modified in place +// * `currentHeight` - the current height for which vset has correct scores +// * `bs` - block store used to retrieve info about historical commits +// * `logger` - logger to use +func NewHeightProposerSelector(vset *types.ValidatorSet, currentHeight int64, bs BlockStore, logger log.Logger) (ProposerSelector, error) { + if vset.IsNilOrEmpty() { + return nil, fmt.Errorf("empty validator set") + } + if logger == nil { + logger = log.NewNopLogger() + } + + logger.Debug("new height proposer selector", "height", currentHeight) + + s := &heightProposerSelector{ + valSet: vset, + height: currentHeight, + bs: bs, + logger: logger, + } + + // if we have a block store, we can determine the proposer for the current height; + // otherwise we just trust the state of `vset` + if bs != nil && bs.Base() > 0 && currentHeight >= bs.Base() { + if err := s.proposerFromStore(currentHeight); err != nil { + return nil, fmt.Errorf("could not initialize proposer: %w", err) + } + } + return s, nil +} + +// proposerFromStore determines the proposer for the given height and round 0 +// based on current or previous block stored in the block store. +func (s *heightProposerSelector) proposerFromStore(height int64) error { + if s.bs == nil { + return fmt.Errorf("block store is nil") + } + + // special case for genesis + if height == 0 || height == 1 { + // we take first proposer from the validator set + if err := s.valSet.SetProposer(s.valSet.Validators[0].ProTxHash); err != nil { + return fmt.Errorf("could not determine proposer: %w", err) + } + + return nil + } + + var proposer bytes.HexBytes + indexIncrement := int64(0) + + meta := s.bs.LoadBlockMeta(height) + if meta != nil { + // block already saved to store, just take the proposer + if !meta.Header.ValidatorsHash.Equal(s.valSet.Hash()) { + // we loaded the same block, so quorum should be the same + s.logger.Error("quorum rotation detected but not expected", + "height", height, + "validators_hash", meta.Header.ValidatorsHash, "quorum_hash", s.valSet.QuorumHash, + "validators", s.valSet) + + return fmt.Errorf("quorum hash mismatch at height %d", height) + } + + proposer = meta.Header.ProposerProTxHash + // rewind rounds, as the proposer in header is for round `meta.Round` and we want round 0 + indexIncrement = int64(-meta.Round) + } else { + // block not found; we try previous height, and will just add 1 to proposer index + meta = s.bs.LoadBlockMeta(height - 1) + if meta == nil { + return fmt.Errorf("could not find block meta for previous height %d", height-1) + } + + // we are at previous height, so we need to increment proposer index by 1 to go to next height + indexIncrement = 1 + + if meta.Header.ValidatorsHash.Equal(s.valSet.Hash()) { + // validators hash matches, so we can take proposer from previous height + proposer = meta.Header.ProposerProTxHash + // rewind rounds, as this is how heightBasedScoringStrategy works + indexIncrement = indexIncrement - int64(meta.Round) + } else { + // quorum rotation happened - we select 1st validator as proposer, and don't rotate + // NOTE: We use index 1 due to bug in original code that causes first validator to never propose. + // We need to preserve the original bad behavior to avoid breaking consensus + proposer = s.valSet.GetByIndex(1).ProTxHash + indexIncrement = 0 + + s.logger.Debug("quorum rotation detected, setting proposer to 1st validator", + "height", height, + "validators_hash", meta.Header.ValidatorsHash, "quorum_hash", s.valSet.QuorumHash, + "validators", s.valSet, + "proposer_proTxHash", proposer) + } + } + + // we're done, set the proposer + if err := s.valSet.SetProposer(proposer); err != nil { + return fmt.Errorf("could not set proposer: %w", err) + } + + if indexIncrement != 0 { + s.valSet.IncProposerIndex(indexIncrement) + } + + return nil +} + +// UpdateHeightRound updates the scores of the validators to the given height. +// Here, we ignore the round, as we don't want to persist round info. +func (s *heightProposerSelector) UpdateHeightRound(newHeight int64, round int32) error { + s.mtx.Lock() + defer s.mtx.Unlock() + + return s.updateScores(newHeight, round) +} + +func (s *heightProposerSelector) updateScores(newHeight int64, _round int32) error { + heightDiff := newHeight - s.height + if heightDiff == 0 { + // NOOP + return nil + } + if heightDiff < 0 { + // TODO: handle going back in height + return fmt.Errorf("cannot go back in height: %d -> %d", s.height, newHeight) + } + + if heightDiff > 1 { + if s.bs == nil || s.bs.Base() > s.height { + return fmt.Errorf("cannot jump more than one height without data in block store: %d -> %d", s.height, newHeight) + } + // FIXME: we assume that no consensus version update happened in the meantime + + if err := s.proposerFromStore(newHeight); err != nil { + return fmt.Errorf("could not determine proposer: %w", err) + } + + s.height = newHeight + return nil + } + + s.valSet.IncProposerIndex(1) + + s.height = newHeight + + return nil +} + +func (s *heightProposerSelector) GetProposer(height int64, round int32) (*types.Validator, error) { + s.mtx.Lock() + defer s.mtx.Unlock() + + if err := s.updateScores(height, 0); err != nil { + return nil, err + } + if round == 0 { + return s.valSet.Proposer(), nil + } + + // advance a copy of the validator set to the correct round, but don't persist the changes + vs := s.valSet.Copy() + vs.IncProposerIndex(int64(round)) + return vs.Proposer(), nil +} + +func (s *heightProposerSelector) MustGetProposer(height int64, round int32) *types.Validator { + if !testing.Testing() { + panic("MustGetProposer should only be used in tests") + } + + proposer, err := s.GetProposer(height, round) + if err != nil { + panic(err) + } + return proposer +} + +func (s *heightProposerSelector) ValidatorSet() *types.ValidatorSet { + s.mtx.Lock() + defer s.mtx.Unlock() + + return s.valSet +} + +func (s *heightProposerSelector) Copy() ProposerSelector { + s.mtx.Lock() + defer s.mtx.Unlock() + + return &heightProposerSelector{ + valSet: s.valSet.Copy(), + height: s.height, + bs: s.bs, + logger: s.logger, + } +} diff --git a/internal/consensus/versioned/selectproposer/height_proposer_test.go b/internal/consensus/versioned/selectproposer/height_proposer_test.go new file mode 100644 index 0000000000..9e370078e4 --- /dev/null +++ b/internal/consensus/versioned/selectproposer/height_proposer_test.go @@ -0,0 +1,179 @@ +package selectproposer_test + +import ( + "bytes" + "math/rand" + "strings" + "testing" + + "github.com/dashpay/dashd-go/btcjson" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/dashpay/tenderdash/crypto" + "github.com/dashpay/tenderdash/crypto/bls12381" + selectproposer "github.com/dashpay/tenderdash/internal/consensus/versioned/selectproposer" + "github.com/dashpay/tenderdash/libs/log" + "github.com/dashpay/tenderdash/types" +) + +//------------------------------------------------------------------- + +func TestProposerSelection1(t *testing.T) { + fooProTxHash := crypto.ProTxHash(crypto.Checksum([]byte("foo"))) + barProTxHash := crypto.ProTxHash(crypto.Checksum([]byte("bar"))) + bazProTxHash := crypto.ProTxHash(crypto.Checksum([]byte("baz"))) + vset := types.NewValidatorSet([]*types.Validator{ + types.NewTestValidatorGeneratedFromProTxHash(fooProTxHash), + types.NewTestValidatorGeneratedFromProTxHash(barProTxHash), + types.NewTestValidatorGeneratedFromProTxHash(bazProTxHash), + }, bls12381.GenPrivKey().PubKey(), btcjson.LLMQType_5_60, crypto.RandQuorumHash(), true) + var proposers []string + + vs, err := selectproposer.NewProposerSelector(types.ConsensusParams{}, vset, 0, 0, nil, log.NewTestingLogger(t)) + require.NoError(t, err) + + for height := int64(0); height < 99; height++ { + val := vs.MustGetProposer(height, 0) + proposers = append(proposers, val.ProTxHash.ShortString()) + } + expected := `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + + `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + + `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + + `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + + `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + + `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + + `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B` + if expected != strings.Join(proposers, " ") { + t.Errorf("expected sequence of proposers was\n%v\nbut got \n%v", expected, strings.Join(proposers, " ")) + } +} + +func TestProposerSelection2(t *testing.T) { + proTxHashes := make([]crypto.ProTxHash, 3) + addresses := make([]crypto.Address, 3) + proTxHashes[0] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1} + proTxHashes[1] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2} + proTxHashes[2] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 3} + addresses[0] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2} + addresses[1] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1} + addresses[2] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0} + + vals, _ := types.GenerateValidatorSet(types.NewValSetParam(proTxHashes)) + vs, err := selectproposer.NewProposerSelector(types.ConsensusParams{}, vals.Copy(), 0, 0, nil, log.NewTestingLogger(t)) + require.NoError(t, err) + + height := 0 + + for ; height < len(proTxHashes)*5; height++ { + ii := (height) % len(proTxHashes) + prop := vs.MustGetProposer(int64(height), 0) + if !bytes.Equal(prop.ProTxHash, vals.Validators[ii].ProTxHash) { + t.Fatalf("(%d): Expected %X. Got %X", height, vals.Validators[ii].ProTxHash, prop.ProTxHash) + } + } + + prop := vs.MustGetProposer(int64(height), 0) + if !bytes.Equal(prop.ProTxHash, proTxHashes[0]) { + t.Fatalf("Expected proposer with smallest pro_tx_hash to be first proposer. Got %X", prop.ProTxHash) + } + + height++ + prop = vs.MustGetProposer(int64(height), 0) + if !bytes.Equal(prop.ProTxHash, proTxHashes[1]) { + t.Fatalf("Expected proposer with second smallest pro_tx_hash to be second proposer. Got %X", prop.ProTxHash) + } +} + +func TestProposerSelection3(t *testing.T) { + const initialHeight = 1 + + proTxHashes := make([]crypto.ProTxHash, 4) + proTxHashes[0] = crypto.Checksum([]byte("avalidator_address12")) + proTxHashes[1] = crypto.Checksum([]byte("bvalidator_address12")) + proTxHashes[2] = crypto.Checksum([]byte("cvalidator_address12")) + proTxHashes[3] = crypto.Checksum([]byte("dvalidator_address12")) + + vset, _ := types.GenerateValidatorSet(types.NewValSetParam(proTxHashes)) + + vs, err := selectproposer.NewProposerSelector(types.ConsensusParams{}, vset.Copy(), initialHeight, 0, nil, log.NewTestingLogger(t)) + require.NoError(t, err) + + // initialize proposers + proposerOrder := make([]*types.Validator, 4) + for i := 0; i < 4; i++ { + proposerOrder[i] = vs.MustGetProposer(int64(i+initialHeight), 0) + } + + // h for the loop + // j for the times + // we should go in order for ever, despite some IncrementProposerPriority with times > 1 + var ( + h int + j uint32 + ) + + vs, err = selectproposer.NewProposerSelector(types.ConsensusParams{}, vset.Copy(), 1, 0, nil, log.NewTestingLogger(t)) + require.NoError(t, err) + j = 0 + for h = 1; h <= 10000; h++ { + + got := vs.MustGetProposer(int64(h), 0).ProTxHash + expected := proposerOrder[j%4].ProTxHash + if !bytes.Equal(got, expected) { + t.Fatalf("vset.Proposer (%X) does not match expected proposer (%X) for (%d, %d)", got, expected, h, j) + } + + round := uint32(rand.Int31n(100)) + require.NoError(t, vs.UpdateHeightRound(int64(h), int32(round))) + j++ // height proposer strategy only increment by 1 each height, regardless of the rounds + } +} + +func setupTestHeightScore(t *testing.T, genesisHeight int64) ([]crypto.ProTxHash, selectproposer.ProposerSelector) { + var proTxHashes []crypto.ProTxHash + for i := byte(1); i <= 5; i++ { + protx := make([]byte, 32) + protx[0] = i + proTxHashes = append(proTxHashes, protx) + } + + vset, _ := types.GenerateValidatorSet(types.NewValSetParam(proTxHashes)) + + vs, err := selectproposer.NewProposerSelector(types.ConsensusParams{}, vset.Copy(), genesisHeight, 0, nil, log.NewTestingLogger(t)) + require.NoError(t, err) + + return proTxHashes, vs +} + +func TestHeightScoreH(t *testing.T) { + const genesisHeight = 1 + + proTxHashes, vs := setupTestHeightScore(t, genesisHeight) + + // test that the proposer changes after the height is updated + for h := int64(1); h < 100; h++ { + proposer := vs.MustGetProposer(h, 0) + pos := (h - genesisHeight) % int64(len(proTxHashes)) + assert.Equal(t, proTxHashes[pos], proposer.ProTxHash, "height %d", h) + require.NoError(t, vs.UpdateHeightRound(h, 0), "height %d", h) + } +} + +// TestHeightScoreRound tests that round proposers don't affect the height proposers +func TestHeightScoreHR(t *testing.T) { + const genesisHeight = 1 + + proTxHashes, vs := setupTestHeightScore(t, genesisHeight) + + // now test with rounds + for h := int64(1); h < 10; h++ { + for r := int32(0); r < 10; r++ { + proposer := vs.MustGetProposer(h, r) + pos := (h - genesisHeight + int64(r)) % int64(len(proTxHashes)) + require.Equal(t, proTxHashes[pos], proposer.ProTxHash, "height %d, round %d", h, r) + require.NoError(t, vs.UpdateHeightRound(h, r), "height %d, round %d", h, r) + } + } +} diff --git a/internal/consensus/versioned/selectproposer/height_round_proposer.go b/internal/consensus/versioned/selectproposer/height_round_proposer.go new file mode 100644 index 0000000000..b8c2e0d1ac --- /dev/null +++ b/internal/consensus/versioned/selectproposer/height_round_proposer.go @@ -0,0 +1,234 @@ +package selectproposer + +import ( + "fmt" + "sync" + "testing" + + "github.com/dashpay/tenderdash/libs/bytes" + "github.com/dashpay/tenderdash/libs/log" + "github.com/dashpay/tenderdash/types" +) + +type heightRoundProposerSelector struct { + valSet *types.ValidatorSet + height int64 + round int32 + bs BlockStore + logger log.Logger + mtx sync.Mutex +} + +// NewHeightRoundProposerSelector creates a new proposer selector that goes around the validator set and ensures +// every proposer votes once. +// +// Each height and round increases proposer index. For example, that if a proposer is selected at height H and round R +// and the block is committed, next proposer will be selected at height H+1 and round 0 (contrary to heightProposerSelector). +// +// It modifies `valSet` in place. +// +// ## Arguments +// +// * `vset` - the validator set; it must not be empty and can be modified in place +// * `currentHeight` - the current height for which vset has correct scores +// * `currentRound` - the current round for which vset has correct scores +// * `bs` - the block store to use for historical data; can be nil +// * `logger` - the logger to use; can be nil +func NewHeightRoundProposerSelector(vset *types.ValidatorSet, currentHeight int64, currentRound int32, bs BlockStore, logger log.Logger) (ProposerSelector, error) { + if vset.IsNilOrEmpty() { + return nil, fmt.Errorf("empty validator set") + } + if logger == nil { + logger = log.NewNopLogger() + } + + logger.Debug("new height round proposer selector", "height", currentHeight, "round", currentRound) + + s := &heightRoundProposerSelector{ + valSet: vset, + height: currentHeight, + round: currentRound, + bs: bs, + logger: logger, + } + + // if we have a block store, we can determine the proposer for the current height; + // otherwise we just trust the state of `vset` + if bs != nil && bs.Base() > 0 && currentHeight >= bs.Base() { + if err := s.proposerFromStore(currentHeight, currentRound); err != nil { + return nil, fmt.Errorf("could not initialize proposer: %w", err) + } + } + + return s, nil +} + +// proposerFromStore determines the proposer for the given height and round using current or previous block read from +// the block store. +func (s *heightRoundProposerSelector) proposerFromStore(height int64, round int32) error { + if s.bs == nil { + return fmt.Errorf("block store is nil") + } + + // special case for genesis + if height == 0 || height == 1 { + // we take first proposer from the validator set + if err := s.valSet.SetProposer(s.valSet.Validators[0].ProTxHash); err != nil { + return fmt.Errorf("could not determine proposer: %w", err) + } + + return nil + } + + var proposer bytes.HexBytes + indexIncrement := int32(0) + + meta := s.bs.LoadBlockMeta(height) + if meta != nil { + // block already saved to store, just take the proposer + if !meta.Header.ValidatorsHash.Equal(s.valSet.Hash()) { + // we loaded the same block, so quorum should be the same + s.logger.Error("quorum rotation detected but not expected", + "height", height, + "validators_hash", meta.Header.ValidatorsHash, "quorum_hash", s.valSet.QuorumHash, + "validators", s.valSet) + + return fmt.Errorf("quorum hash mismatch at height %d", height) + } + + proposer = meta.Header.ProposerProTxHash + // adjust round number to match the requested one + indexIncrement = round - meta.Round + } else { + // block not found; we try previous height, and will just add 1 to proposer index + meta = s.bs.LoadBlockMeta(height - 1) + if meta == nil { + return fmt.Errorf("could not find block meta for previous height %d", height-1) + } + + if meta.Header.ValidatorsHash.Equal(s.valSet.Hash()) { + // validators hash matches, so we can take proposer from previous height + proposer = meta.Header.ProposerProTxHash + // we are at previous height+prev committed round, so we need to increment proposer index by 1 to go to next height, + // and then by round number to match the requested round + indexIncrement = 1 + round + } else { + // quorum rotation happened - we select 1st validator as proposer, and only adjust round number + proposer = s.valSet.GetByIndex(0).ProTxHash + indexIncrement = round + + s.logger.Debug("quorum rotation detected, setting proposer to 1st validator", + "height", height, + "validators_hash", meta.Header.ValidatorsHash, "quorum_hash", s.valSet.QuorumHash, + "validators", s.valSet, + "proposer_proTxHash", proposer) + } + } + + // we're done, set the proposer + if err := s.valSet.SetProposer(proposer); err != nil { + return fmt.Errorf("could not set proposer: %w", err) + } + + if indexIncrement != 0 { + s.valSet.IncProposerIndex(int64(indexIncrement)) + } + + return nil +} + +// UpdateHeightRound updates the scores of the validators to the given height and round. +func (s *heightRoundProposerSelector) UpdateHeightRound(newHeight int64, newRound int32) error { + s.mtx.Lock() + defer s.mtx.Unlock() + + return s.updateScores(newHeight, newRound) +} + +func (s *heightRoundProposerSelector) updateScores(newHeight int64, newRound int32) error { + heightDiff := newHeight - s.height + roundDiff := int64(newRound - s.round) + if heightDiff == 0 && roundDiff == 0 { + // NOOP + return nil + } + + if heightDiff == 0 && roundDiff != 0 { + // only update round + s.valSet.IncProposerIndex(roundDiff) + s.round = newRound + return nil + } + + if heightDiff < 0 { + // TODO: handle going back in height + return fmt.Errorf("cannot go back in height: %d -> %d", s.height, newHeight) + } + + if heightDiff > 1 { + if s.bs == nil || s.bs.Base() > s.height { + return fmt.Errorf("cannot jump more than one height without data in block store: %d -> %d", s.height, newHeight) + } + // we assume that no consensus version update happened in the meantime + if err := s.proposerFromStore(newHeight, newRound); err != nil { + return fmt.Errorf("could not determine proposer: %w", err) + } + + s.height = newHeight + s.round = newRound + + return nil + } + + // heightDiff is 1; it means we go to the newHeight+ newRound + // Assuming s.round is the last one as we are not able to determine this + s.valSet.IncProposerIndex(heightDiff + int64(newRound)) + + s.height = newHeight + s.round = newRound + + return nil +} + +func (s *heightRoundProposerSelector) GetProposer(height int64, round int32) (*types.Validator, error) { + s.mtx.Lock() + defer s.mtx.Unlock() + + if err := s.updateScores(height, round); err != nil { + return nil, err + } + + return s.valSet.Proposer(), nil +} + +func (s *heightRoundProposerSelector) MustGetProposer(height int64, round int32) *types.Validator { + if !testing.Testing() { + panic("MustGetProposer should only be used in tests") + } + + proposer, err := s.GetProposer(height, round) + if err != nil { + panic(err) + } + return proposer +} + +func (s *heightRoundProposerSelector) ValidatorSet() *types.ValidatorSet { + s.mtx.Lock() + defer s.mtx.Unlock() + + return s.valSet +} + +func (s *heightRoundProposerSelector) Copy() ProposerSelector { + s.mtx.Lock() + defer s.mtx.Unlock() + + return &heightRoundProposerSelector{ + valSet: s.valSet.Copy(), + height: s.height, + round: s.round, + bs: s.bs, + logger: s.logger, + } +} diff --git a/internal/consensus/versioned/selectproposer/height_round_proposer_test.go b/internal/consensus/versioned/selectproposer/height_round_proposer_test.go new file mode 100644 index 0000000000..7ebefc885e --- /dev/null +++ b/internal/consensus/versioned/selectproposer/height_round_proposer_test.go @@ -0,0 +1,66 @@ +package selectproposer_test + +import ( + "bytes" + "math/big" + "math/rand" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/dashpay/tenderdash/crypto" + selectproposer "github.com/dashpay/tenderdash/internal/consensus/versioned/selectproposer" + "github.com/dashpay/tenderdash/libs/log" + tmtypes "github.com/dashpay/tenderdash/proto/tendermint/types" + "github.com/dashpay/tenderdash/types" +) + +func TestProposerSelectionHR(t *testing.T) { + const nVals = 4 + const initialHeight = 1 + + proTxHashes := make([]crypto.ProTxHash, 0, nVals) + for i := 0; i < nVals; i++ { + protx := make([]byte, crypto.ProTxHashSize) + big.NewInt(int64(i + 1)).FillBytes(protx) + proTxHashes = append(proTxHashes, protx) + } + + vset, _ := types.GenerateValidatorSet(types.NewValSetParam(proTxHashes)) + + // initialize proposers + proposerOrder := make([]*types.Validator, vset.Size()) + for i := 0; i < vset.Size(); i++ { + proposerOrder[i] = vset.Validators[i] + } + + var ( + h int + proposerIndex uint32 + ) + + // HEIGHT ROUND STRATEGY + + vs, err := selectproposer.NewProposerSelector(types.ConsensusParams{ + Version: types.VersionParams{ + ConsensusVersion: int32(tmtypes.VersionParams_CONSENSUS_VERSION_1), + }, + }, vset.Copy(), initialHeight, 0, nil, log.NewTestingLogger(t)) + require.NoError(t, err) + + proposerIndex = 0 + for h = 1; h <= 10000; h++ { + got := vs.MustGetProposer(int64(h), 0).ProTxHash + expected := proposerOrder[proposerIndex%nVals].ProTxHash + if !bytes.Equal(got, expected) { + t.Fatalf("vset.Proposer (%X) does not match expected proposer (%X) for (%d, %d)", got, expected, h, proposerIndex) + } + + round := uint32(rand.Int31n(100)) + require.NoError(t, vs.UpdateHeightRound(int64(h), int32(round))) + + // t.Logf("Height: %d, Round: %d, proposer index %d", h, round, proposerIndex) + // we expect proposer increase for each round, plus one for next height + proposerIndex += 1 + round + } +} diff --git a/internal/consensus/versioned/selectproposer/proposer_selector.go b/internal/consensus/versioned/selectproposer/proposer_selector.go new file mode 100644 index 0000000000..792df302fe --- /dev/null +++ b/internal/consensus/versioned/selectproposer/proposer_selector.go @@ -0,0 +1,66 @@ +package selectproposer + +import ( + "fmt" + + "github.com/dashpay/tenderdash/libs/log" + tmtypes "github.com/dashpay/tenderdash/proto/tendermint/types" + "github.com/dashpay/tenderdash/types" +) + +type ProposerSelector interface { + // GetProposer returns the proposer for the given height and round. It calls Update if necessary. + GetProposer(height int64, round int32) (*types.Validator, error) + + // MustGetProposer returns the proposer for the given height and round. It calls Update if necessary. + // It panics on any error. + // + // Only use in tests. + // See [GetProposer](#GetProposer) for details. + MustGetProposer(height int64, round int32) *types.Validator + // UpdateHeightRound updates proposer to match provided height and round. It should be called at least once for each round. + // - `height` is the height + // - `round` is the round + UpdateHeightRound(height int64, round int32) error + // Returns pointer to underlying validator set; not thread-safe, and should be modified with caution + ValidatorSet() *types.ValidatorSet + // Create deep copy of the strategy and its underlying validator set + Copy() ProposerSelector +} + +type BlockStore interface { + LoadBlockMeta(height int64) *types.BlockMeta + Base() int64 +} + +// NewProposerSelector creates an instance of ProposerSelector based on the given ConsensusParams. +// +// Original ValidatorSet should not be used anymore. Height and round should point to the height and round that +// is reflected in validator scores, eg. the one for which GetProposer() returns proposer that generates proposal +// at the given height and round. +// +// If block store is provided, it will be used to determine the proposer for the current height. +// +// ## Arguments +// +// - `cp` - ConsensusParams that determine scoring strategy to use +// - `valSet` - validator set to use +// - `valsetHeight` - current height of the validator set +// - `valsetRound` - current round of the validator set +// - `bs` - block store used to retreve info about historical commits +// - `logger` - logger to use; can be nil +func NewProposerSelector(cp types.ConsensusParams, valSet *types.ValidatorSet, valsetHeight int64, valsetRound int32, + bs BlockStore, logger log.Logger) (ProposerSelector, error) { + if logger == nil { + logger = log.NewNopLogger() + } + switch cp.Version.ConsensusVersion { + case int32(tmtypes.VersionParams_CONSENSUS_VERSION_0): + return NewHeightProposerSelector(valSet, valsetHeight, bs, logger) + case int32(tmtypes.VersionParams_CONSENSUS_VERSION_1): + + return NewHeightRoundProposerSelector(valSet, valsetHeight, valsetRound, bs, logger) + default: + return nil, fmt.Errorf("unknown consensus version: %v", cp.Version.ConsensusVersion) + } +} diff --git a/internal/consensus/wal.go b/internal/consensus/wal.go index 6f5fe608ad..1c3c249801 100644 --- a/internal/consensus/wal.go +++ b/internal/consensus/wal.go @@ -15,6 +15,7 @@ import ( "github.com/dashpay/tenderdash/internal/jsontypes" auto "github.com/dashpay/tenderdash/internal/libs/autofile" "github.com/dashpay/tenderdash/libs/log" + tmmath "github.com/dashpay/tenderdash/libs/math" tmos "github.com/dashpay/tenderdash/libs/os" "github.com/dashpay/tenderdash/libs/service" tmtime "github.com/dashpay/tenderdash/libs/time" @@ -331,7 +332,7 @@ func (enc *WALEncoder) Encode(v *TimedWALMessage) error { } crc := crc32.Checksum(data, crc32c) - length := uint32(len(data)) + length := tmmath.MustConvertUint32(len(data)) if length > maxMsgSizeBytes { return fmt.Errorf("msg is too big: %d bytes, max: %d bytes", length, maxMsgSizeBytes) } diff --git a/internal/evidence/mocks/block_store.go b/internal/evidence/mocks/block_store.go index 41fff57cde..0062c4c656 100644 --- a/internal/evidence/mocks/block_store.go +++ b/internal/evidence/mocks/block_store.go @@ -12,6 +12,24 @@ type BlockStore struct { mock.Mock } +// Base provides a mock function with given fields: +func (_m *BlockStore) Base() int64 { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for Base") + } + + var r0 int64 + if rf, ok := ret.Get(0).(func() int64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int64) + } + + return r0 +} + // Height provides a mock function with given fields: func (_m *BlockStore) Height() int64 { ret := _m.Called() diff --git a/internal/evidence/pool.go b/internal/evidence/pool.go index 8d8a9970c0..6c4db07a89 100644 --- a/internal/evidence/pool.go +++ b/internal/evidence/pool.go @@ -576,7 +576,7 @@ func (evpool *Pool) processConsensusBuffer(ctx context.Context, state sm.State) ) case voteSet.VoteA.Height < state.LastBlockHeight: - valSet, dbErr := evpool.stateDB.LoadValidators(voteSet.VoteA.Height) + valSet, dbErr := evpool.stateDB.LoadValidators(voteSet.VoteA.Height, evpool.blockStore) if dbErr != nil { evpool.logger.Error("failed to load validator set for conflicting votes", "height", voteSet.VoteA.Height, "err", err) diff --git a/internal/evidence/pool_test.go b/internal/evidence/pool_test.go index 652af1ac7e..4d8f263a29 100644 --- a/internal/evidence/pool_test.go +++ b/internal/evidence/pool_test.go @@ -58,7 +58,7 @@ func TestEvidencePoolBasic(t *testing.T) { blockStore.On("LoadBlockMeta", mock.AnythingOfType("int64")).Return( &types.BlockMeta{Header: types.Header{Time: defaultEvidenceTime}}, ) - stateStore.On("LoadValidators", mock.AnythingOfType("int64")).Return(valSet, nil) + stateStore.On("LoadValidators", mock.AnythingOfType("int64"), mock.Anything).Return(valSet, nil) stateStore.On("Load").Return(createState(height+1, valSet), nil) logger := log.NewTestingLogger(t) @@ -524,7 +524,6 @@ func initializeValidatorState( // create validator set and state valSet := &types.ValidatorSet{ Validators: []*types.Validator{validator}, - Proposer: validator, ThresholdPublicKey: validator.PubKey, QuorumType: quorumType, QuorumHash: quorumHash, @@ -541,7 +540,8 @@ func initializeBlockStore(db dbm.DB, state sm.State) (*store.BlockStore, error) for i := int64(1); i <= state.LastBlockHeight; i++ { lastCommit := makeCommit(i-1, state.Validators.QuorumHash) - block := state.MakeBlock(i, []types.Tx{}, lastCommit, nil, state.Validators.GetProposer().ProTxHash, 0) + prop := state.GetProposerFromState(i, 0) + block := state.MakeBlock(i, []types.Tx{}, lastCommit, nil, prop.ProTxHash, 0) block.Header.Time = defaultEvidenceTime.Add(time.Duration(i) * time.Minute) block.Header.Version = version.Consensus{Block: version.BlockProtocol, App: 1} diff --git a/internal/evidence/services.go b/internal/evidence/services.go index c71783a273..eb7b53aebc 100644 --- a/internal/evidence/services.go +++ b/internal/evidence/services.go @@ -9,5 +9,6 @@ import ( type BlockStore interface { LoadBlockMeta(height int64) *types.BlockMeta LoadBlockCommit(height int64) *types.Commit + Base() int64 Height() int64 } diff --git a/internal/evidence/verify.go b/internal/evidence/verify.go index 0010a11b7c..c63546e46f 100644 --- a/internal/evidence/verify.go +++ b/internal/evidence/verify.go @@ -60,7 +60,7 @@ func (evpool *Pool) verify(ctx context.Context, evidence types.Evidence) error { // apply the evidence-specific verification logic switch ev := evidence.(type) { case *types.DuplicateVoteEvidence: - valSet, err := evpool.stateDB.LoadValidators(evidence.Height()) + valSet, err := evpool.stateDB.LoadValidators(evidence.Height(), evpool.blockStore) if err != nil { return err } diff --git a/internal/inspect/inspect_test.go b/internal/inspect/inspect_test.go index bbbb4f9107..f5d9e66492 100644 --- a/internal/inspect/inspect_test.go +++ b/internal/inspect/inspect_test.go @@ -478,7 +478,7 @@ func TestValidators(t *testing.T) { }, } stateStoreMock := &statemocks.Store{} - stateStoreMock.On("LoadValidators", testHeight).Return(&testValidators, nil) + stateStoreMock.On("LoadValidators", testHeight, mock.Anything).Return(&testValidators, nil) blockStoreMock := &statemocks.BlockStore{} blockStoreMock.On("Height").Return(testHeight) diff --git a/internal/libs/test/utils.go b/internal/libs/test/utils.go new file mode 100644 index 0000000000..9bc0322ac0 --- /dev/null +++ b/internal/libs/test/utils.go @@ -0,0 +1,9 @@ +package test + +func Must[T any](v T, err error) T { + if err != nil { + panic(err) + } + + return v +} diff --git a/internal/p2p/router.go b/internal/p2p/router.go index 34896d77eb..4ac248e1d6 100644 --- a/internal/p2p/router.go +++ b/internal/p2p/router.go @@ -489,7 +489,7 @@ func (r *Router) openConnection(ctx context.Context, conn Connection) { case errors.Is(err, context.Canceled): return case err != nil: - r.logger.Error("peer handshake failed", "endpoint", conn, "err", err) + r.logger.Warn("peer handshake failed", "endpoint", conn, "err", err) return } if err := r.filterPeersID(ctx, peerInfo.NodeID); err != nil { diff --git a/internal/rpc/core/blocks_test.go b/internal/rpc/core/blocks_test.go index 2b761c3bf3..7caef89219 100644 --- a/internal/rpc/core/blocks_test.go +++ b/internal/rpc/core/blocks_test.go @@ -14,7 +14,9 @@ import ( sm "github.com/dashpay/tenderdash/internal/state" "github.com/dashpay/tenderdash/internal/state/mocks" "github.com/dashpay/tenderdash/proto/tendermint/state" + "github.com/dashpay/tenderdash/proto/tendermint/types" "github.com/dashpay/tenderdash/rpc/coretypes" + tmtypes "github.com/dashpay/tenderdash/types" ) func TestBlockchainInfo(t *testing.T) { @@ -77,6 +79,11 @@ func TestBlockResults(t *testing.T) { {Code: 0, Data: []byte{0x02}, Log: "ok", GasUsed: 5}, {Code: 1, Log: "not ok", GasUsed: 0}, }, + ConsensusParamUpdates: &types.ConsensusParams{ + Version: &types.VersionParams{ + AppVersion: 1, + }, + }, }, } @@ -98,25 +105,30 @@ func TestBlockResults(t *testing.T) { {0, true, nil}, {101, true, nil}, {100, false, &coretypes.ResultBlockResults{ - Height: 100, - TxsResults: results.ProcessProposal.TxResults, - TotalGasUsed: 15, - FinalizeBlockEvents: results.ProcessProposal.Events, - ValidatorSetUpdate: results.ProcessProposal.ValidatorSetUpdate, - ConsensusParamUpdates: consensusParamsPtrFromProtoPtr(results.ProcessProposal.ConsensusParamUpdates), + Height: 100, + TxsResults: results.ProcessProposal.TxResults, + TotalGasUsed: 15, + FinalizeBlockEvents: results.ProcessProposal.Events, + ValidatorSetUpdate: results.ProcessProposal.ValidatorSetUpdate, + ConsensusParamUpdates: &tmtypes.ConsensusParams{ + Version: tmtypes.VersionParams{AppVersion: 1}, + }, }}, } ctx := context.Background() for _, tc := range testCases { - res, err := env.BlockResults(ctx, &coretypes.RequestBlockInfo{ - Height: (*coretypes.Int64)(&tc.height), + t.Run("", func(t *testing.T) { + res, err := env.BlockResults(ctx, &coretypes.RequestBlockInfo{ + Height: (*coretypes.Int64)(&tc.height), + }) + if tc.wantErr { + assert.Error(t, err) + } else { + assert.NoError(t, err) + t.Logf("Consensus params: %+v", res.ConsensusParamUpdates) + assert.Equal(t, tc.wantRes, res) + } }) - if tc.wantErr { - assert.Error(t, err) - } else { - assert.NoError(t, err) - assert.Equal(t, tc.wantRes, res) - } } } diff --git a/internal/rpc/core/consensus.go b/internal/rpc/core/consensus.go index ff356e7d8d..983e041c44 100644 --- a/internal/rpc/core/consensus.go +++ b/internal/rpc/core/consensus.go @@ -22,7 +22,7 @@ func (env *Environment) Validators(_ctx context.Context, req *coretypes.RequestV return nil, err } - validators, err := env.StateStore.LoadValidators(height) + validators, err := env.StateStore.LoadValidators(height, env.BlockStore) if err != nil { return nil, err } @@ -38,11 +38,16 @@ func (env *Environment) Validators(_ctx context.Context, req *coretypes.RequestV v := validators.Validators[skipCount : skipCount+tmmath.MinInt(perPage, totalCount-skipCount)] + quorumHash := validators.QuorumHash.Copy() + result := &coretypes.ResultValidators{ - BlockHeight: height, - Validators: v, - Count: len(v), - Total: totalCount, + BlockHeight: height, + Validators: v, + Count: len(v), + Total: totalCount, + ThresholdPublicKey: &validators.ThresholdPublicKey, + QuorumType: validators.QuorumType, + QuorumHash: &quorumHash, } if libs.BoolValue(req.RequestQuorumInfo) { result.QuorumHash = &validators.QuorumHash diff --git a/internal/state/current_round_state.go b/internal/state/current_round_state.go index f16bc353d7..a6848ccea5 100644 --- a/internal/state/current_round_state.go +++ b/internal/state/current_round_state.go @@ -58,10 +58,10 @@ func NewCurrentRoundState(proTxHash types.ProTxHash, rp RoundParams, baseState S Params: rp, Round: rp.Round, } - err := candidate.populate() - if err != nil { + if err := candidate.populate(); err != nil { return CurrentRoundState{}, err } + return candidate, nil } @@ -92,6 +92,7 @@ func (candidate *CurrentRoundState) UpdateBlock(target *types.Block) error { // UpdateState updates state when the block is committed. State will contain data needed by next block. func (candidate *CurrentRoundState) UpdateState(target *State) error { + target.LastBlockRound = candidate.Round target.LastAppHash = candidate.AppHash target.LastResultsHash = candidate.ResultsHash target.ConsensusParams = candidate.NextConsensusParams @@ -105,12 +106,6 @@ func (candidate *CurrentRoundState) UpdateState(target *State) error { return nil } -// UpdateFunc implements UpdateFunc -func (candidate *CurrentRoundState) UpdateFunc(state State) (State, error) { - err := candidate.UpdateState(&state) - return state, err -} - // GetHeight returns height of current block func (candidate *CurrentRoundState) GetHeight() int64 { if candidate.Base.LastBlockHeight == 0 { @@ -200,11 +195,6 @@ func (candidate *CurrentRoundState) populateValsetUpdates() error { return fmt.Errorf("validator set updates: %w", err) } - if updateSource != InitChainSource { - // we take validator sets as they arrive from InitChainSource response - newValSet.IncrementProposerPriority(1) - } - candidate.NextValidators = newValSet if updateSource != InitChainSource && update != nil && len(update.ValidatorUpdates) > 0 { diff --git a/internal/state/events.go b/internal/state/events.go index 104ca2e5aa..71bdf06b4a 100644 --- a/internal/state/events.go +++ b/internal/state/events.go @@ -6,6 +6,7 @@ import ( "github.com/hashicorp/go-multierror" abci "github.com/dashpay/tenderdash/abci/types" + "github.com/dashpay/tenderdash/libs/math" "github.com/dashpay/tenderdash/types" ) @@ -87,7 +88,7 @@ func (e *EventSet) WithTxs(block *types.Block, txResults []*abci.ExecTxResult) * e.Txs[i] = types.EventDataTx{ TxResult: abci.TxResult{ Height: block.Height, - Index: uint32(i), + Index: math.MustConvertUint32(i), Tx: tx, Result: *(txResults[i]), }, diff --git a/internal/state/execution.go b/internal/state/execution.go index fbcbd2620e..b3598109c4 100644 --- a/internal/state/execution.go +++ b/internal/state/execution.go @@ -668,7 +668,7 @@ func (state State) Update( nextVersion := state.Version - // NOTE: LastStateIDHash, AppHash and VoteExtension has not been populated. + // NOTE: LastBlockRound, LastStateIDHash, AppHash and VoteExtension has not been populated. // It will be filled on state.Save. newState := State{ Version: nextVersion, diff --git a/internal/state/execution_test.go b/internal/state/execution_test.go index deeca658b2..c94602d3ab 100644 --- a/internal/state/execution_test.go +++ b/internal/state/execution_test.go @@ -326,13 +326,14 @@ func TestUpdateConsensusParams(t *testing.T) { ConsensusParamUpdates: &tmtypes.ConsensusParams{Block: &tmtypes.BlockParams{MaxBytes: 1024 * 1024}}, AppVersion: 1, }, nil).Once() + prop := state.GetProposerFromState(height, round) block1, _, err := blockExec.CreateProposalBlock( ctx, height, round, state, lastCommit, - state.Validators.GetProposer().ProTxHash, + prop.ProTxHash, 1, ) require.NoError(t, err) @@ -399,13 +400,14 @@ func TestOverrideAppVersion(t *testing.T) { AppVersion: appVersion, }, nil).Once() + proposer := state.GetProposerFromState(height, round) block1, _, err := blockExec.CreateProposalBlock( ctx, height, round, state, lastCommit, - state.Validators.GetProposer().ProTxHash, + proposer.ProTxHash, 1, ) require.NoError(t, err) diff --git a/internal/state/helpers_test.go b/internal/state/helpers_test.go index 2aa8e74713..180be9d749 100644 --- a/internal/state/helpers_test.go +++ b/internal/state/helpers_test.go @@ -11,6 +11,7 @@ import ( abci "github.com/dashpay/tenderdash/abci/types" "github.com/dashpay/tenderdash/crypto" + selectproposer "github.com/dashpay/tenderdash/internal/consensus/versioned/selectproposer" sm "github.com/dashpay/tenderdash/internal/state" sf "github.com/dashpay/tenderdash/internal/state/test/factory" "github.com/dashpay/tenderdash/internal/test/factory" @@ -185,13 +186,26 @@ func makeHeaderPartsResponsesParams( func makeRandomStateFromValidatorSet( lastValSet *types.ValidatorSet, height, lastHeightValidatorsChanged int64, + bs selectproposer.BlockStore, ) sm.State { + vs := lastValSet.Copy() + cp := types.DefaultConsensusParams() + expectedVS, err := selectproposer.NewProposerSelector(*cp, vs, lastHeightValidatorsChanged, 0, bs, nil) + if err != nil { + panic(err) + } + for h := lastHeightValidatorsChanged; h <= height; h++ { + if err := expectedVS.UpdateHeightRound(h, 0); err != nil { + panic(err) + } + } + return sm.State{ LastBlockHeight: height - 1, - Validators: lastValSet.CopyIncrementProposerPriority(1), - LastValidators: lastValSet.Copy(), + Validators: vs.Copy(), + LastValidators: vs.Copy(), LastHeightConsensusParamsChanged: height, - ConsensusParams: *types.DefaultConsensusParams(), + ConsensusParams: *cp, LastHeightValidatorsChanged: lastHeightValidatorsChanged, InitialHeight: 1, } @@ -209,7 +223,7 @@ func makeRandomStateFromConsensusParams( LastBlockHeight: height - 1, ConsensusParams: *consensusParams, LastHeightConsensusParamsChanged: lastHeightConsensusParamsChanged, - Validators: valSet.CopyIncrementProposerPriority(1), + Validators: valSet.Copy(), LastValidators: valSet.Copy(), LastHeightValidatorsChanged: height, InitialHeight: 1, diff --git a/internal/state/mocks/store.go b/internal/state/mocks/store.go index 4a7fa92190..006a8b4e03 100644 --- a/internal/state/mocks/store.go +++ b/internal/state/mocks/store.go @@ -3,6 +3,7 @@ package mocks import ( + selectproposer "github.com/dashpay/tenderdash/internal/consensus/versioned/selectproposer" state "github.com/dashpay/tenderdash/internal/state" mock "github.com/stretchr/testify/mock" @@ -138,9 +139,9 @@ func (_m *Store) LoadConsensusParams(_a0 int64) (types.ConsensusParams, error) { return r0, r1 } -// LoadValidators provides a mock function with given fields: _a0 -func (_m *Store) LoadValidators(_a0 int64) (*types.ValidatorSet, error) { - ret := _m.Called(_a0) +// LoadValidators provides a mock function with given fields: _a0, _a1 +func (_m *Store) LoadValidators(_a0 int64, _a1 selectproposer.BlockStore) (*types.ValidatorSet, error) { + ret := _m.Called(_a0, _a1) if len(ret) == 0 { panic("no return value specified for LoadValidators") @@ -148,19 +149,19 @@ func (_m *Store) LoadValidators(_a0 int64) (*types.ValidatorSet, error) { var r0 *types.ValidatorSet var r1 error - if rf, ok := ret.Get(0).(func(int64) (*types.ValidatorSet, error)); ok { - return rf(_a0) + if rf, ok := ret.Get(0).(func(int64, selectproposer.BlockStore) (*types.ValidatorSet, error)); ok { + return rf(_a0, _a1) } - if rf, ok := ret.Get(0).(func(int64) *types.ValidatorSet); ok { - r0 = rf(_a0) + if rf, ok := ret.Get(0).(func(int64, selectproposer.BlockStore) *types.ValidatorSet); ok { + r0 = rf(_a0, _a1) } else { if ret.Get(0) != nil { r0 = ret.Get(0).(*types.ValidatorSet) } } - if rf, ok := ret.Get(1).(func(int64) error); ok { - r1 = rf(_a0) + if rf, ok := ret.Get(1).(func(int64, selectproposer.BlockStore) error); ok { + r1 = rf(_a0, _a1) } else { r1 = ret.Error(1) } diff --git a/internal/state/rollback.go b/internal/state/rollback.go index 5f077c70b1..5690778629 100644 --- a/internal/state/rollback.go +++ b/internal/state/rollback.go @@ -48,7 +48,7 @@ func Rollback(bs BlockStore, ss Store) (int64, []byte, error) { return -1, nil, fmt.Errorf("block at height %d not found", invalidState.LastBlockHeight) } - previousLastValidatorSet, err := ss.LoadValidators(rollbackHeight) + previousLastValidatorSet, err := ss.LoadValidators(rollbackHeight, bs) if err != nil { return -1, nil, err } @@ -84,6 +84,7 @@ func Rollback(bs BlockStore, ss Store) (int64, []byte, error) { InitialHeight: invalidState.InitialHeight, LastBlockHeight: rollbackBlock.Header.Height, + LastBlockRound: invalidState.LastBlockRound, LastBlockID: rollbackBlock.BlockID, LastBlockTime: rollbackBlock.Header.Time, diff --git a/internal/state/rollback_test.go b/internal/state/rollback_test.go index 4ca126c110..6fca9fe147 100644 --- a/internal/state/rollback_test.go +++ b/internal/state/rollback_test.go @@ -44,24 +44,30 @@ func TestRollback(t *testing.T) { block := &types.BlockMeta{ BlockID: initialState.LastBlockID, Header: types.Header{ - Height: initialState.LastBlockHeight, - AppHash: factory.RandomHash(), - LastBlockID: factory.MakeBlockID(), - ResultsHash: initialState.LastResultsHash, + Height: initialState.LastBlockHeight, + AppHash: factory.RandomHash(), + LastBlockID: factory.MakeBlockID(), + ResultsHash: initialState.LastResultsHash, + ProposerProTxHash: initialState.Validators.Validators[0].ProTxHash, + ValidatorsHash: initialState.Validators.Hash(), + NextValidatorsHash: nextState.Validators.Hash(), }, } nextBlock := &types.BlockMeta{ BlockID: initialState.LastBlockID, Header: types.Header{ - Height: nextState.LastBlockHeight, - AppHash: initialState.LastAppHash, - LastBlockID: block.BlockID, - ResultsHash: nextState.LastResultsHash, + Height: nextState.LastBlockHeight, + AppHash: initialState.LastAppHash, + LastBlockID: block.BlockID, + ResultsHash: nextState.LastResultsHash, + ProposerProTxHash: nextState.Validators.Validators[1].ProTxHash, + ValidatorsHash: nextState.Validators.Hash(), }, } blockStore.On("LoadBlockMeta", height).Return(block) blockStore.On("LoadBlockMeta", nextHeight).Return(nextBlock) blockStore.On("Height").Return(nextHeight) + blockStore.On("Base").Return(height) // rollback the state rollbackHeight, rollbackHash, err := state.Rollback(blockStore, stateStore) @@ -133,7 +139,7 @@ func setupStateStore(t *testing.T, height int64) state.Store { LastResultsHash: factory.RandomHash(), LastBlockHeight: height, LastValidators: valSet, - Validators: valSet.CopyIncrementProposerPriority(1), + Validators: valSet.Copy(), LastHeightValidatorsChanged: height + 1, ConsensusParams: *params, LastHeightConsensusParamsChanged: height + 1, diff --git a/internal/state/state.go b/internal/state/state.go index 0b7b994d5b..cbffe452e3 100644 --- a/internal/state/state.go +++ b/internal/state/state.go @@ -6,11 +6,13 @@ import ( "errors" "fmt" "os" + "testing" "time" "github.com/gogo/protobuf/proto" "github.com/dashpay/tenderdash/dash" + selectproposer "github.com/dashpay/tenderdash/internal/consensus/versioned/selectproposer" tmbytes "github.com/dashpay/tenderdash/libs/bytes" tmtime "github.com/dashpay/tenderdash/libs/time" tmstate "github.com/dashpay/tenderdash/proto/tendermint/state" @@ -78,6 +80,7 @@ type State struct { // LastBlockHeight=0 at genesis (ie. block(H=0) does not exist) LastBlockHeight int64 + LastBlockRound int32 LastBlockID types.BlockID LastBlockTime time.Time @@ -126,6 +129,7 @@ func (state State) Copy() State { InitialHeight: state.InitialHeight, LastBlockHeight: state.LastBlockHeight, + LastBlockRound: state.LastBlockRound, LastBlockID: state.LastBlockID, LastBlockTime: state.LastBlockTime, @@ -188,6 +192,7 @@ func (state *State) ToProto() (*tmstate.State, error) { sm.ChainID = state.ChainID sm.InitialHeight = state.InitialHeight sm.LastBlockHeight = state.LastBlockHeight + sm.LastBlockRound = state.LastBlockRound sm.LastCoreChainLockedBlockHeight = state.LastCoreChainLockedBlockHeight @@ -234,6 +239,7 @@ func FromProto(pb *tmstate.State) (*State, error) { //nolint:golint } state.LastBlockID = *bi state.LastBlockHeight = pb.LastBlockHeight + state.LastBlockRound = pb.LastBlockRound state.LastBlockTime = pb.LastBlockTime state.LastCoreChainLockedBlockHeight = pb.LastCoreChainLockedBlockHeight @@ -308,6 +314,28 @@ func (state State) ValidatorsAtHeight(height int64) *types.ValidatorSet { } } +// GetProposerFromState is a helper function that returns the proposer for the given height and round, +// based on current state. Only use in tests. +func (state *State) GetProposerFromState(height int64, round int32) *types.Validator { + if !testing.Testing() { + panic("GetProposerFromState should only be used in tests") + } + + vs, err := selectproposer.NewProposerSelector( + state.ConsensusParams, + state.Validators.Copy(), + state.LastBlockHeight, + state.LastBlockRound, + nil, + nil, + ) + if err != nil { + panic(fmt.Errorf("failed to create validator scoring strategy: %w", err)) + } + proposer := vs.MustGetProposer(height, round) + return proposer +} + // NewStateChangeset returns a structure that will hold new changes to the state, that can be applied once the block is finalized func (state State) NewStateChangeset(ctx context.Context, rp RoundParams) (CurrentRoundState, error) { proTxHash, _ := dash.ProTxHashFromContext(ctx) @@ -354,7 +382,7 @@ func MakeGenesisState(genDoc *types.GenesisDoc) (State, error) { } var validatorSet *types.ValidatorSet - if genDoc.Validators == nil || len(genDoc.Validators) == 0 { + if len(genDoc.Validators) == 0 { validatorSet = types.NewValidatorSet(nil, nil, genDoc.QuorumType, nil, false) } else { validators := make([]*types.Validator, len(genDoc.Validators)) diff --git a/internal/state/state_test.go b/internal/state/state_test.go index 8fd12710b5..9b8bb59384 100644 --- a/internal/state/state_test.go +++ b/internal/state/state_test.go @@ -3,7 +3,6 @@ package state_test import ( "context" "fmt" - "math/big" "os" "testing" @@ -21,6 +20,7 @@ import ( "github.com/dashpay/tenderdash/crypto/merkle" "github.com/dashpay/tenderdash/dash" "github.com/dashpay/tenderdash/dash/llmq" + "github.com/dashpay/tenderdash/internal/evidence/mocks" sm "github.com/dashpay/tenderdash/internal/state" statefactory "github.com/dashpay/tenderdash/internal/state/test/factory" tmstate "github.com/dashpay/tenderdash/proto/tendermint/state" @@ -242,18 +242,27 @@ func TestValidatorSimpleSaveLoad(t *testing.T) { defer tearDown(t) statestore := sm.NewStore(stateDB) + blockStore := mocks.NewBlockStore(t) // Can't load anything for height 0. - _, err := statestore.LoadValidators(0) + _, err := statestore.LoadValidators(0, blockStore) assert.IsType(t, sm.ErrNoValSetForHeight{}, err, "expected err at height 0") // Should be able to load for height 1. - v, err := statestore.LoadValidators(1) + blockStore.On("Base").Return(int64(1)) + blockStore.On("LoadBlockMeta", int64(1)).Return(&types.BlockMeta{ + Header: types.Header{ + Height: 1, + ProposerProTxHash: state.Validators.GetByIndex((int32(state.LastBlockHeight) % int32(state.Validators.Size()))).ProTxHash, + }}) + + v, err := statestore.LoadValidators(1, blockStore) require.NoError(t, err, "expected no err at height 1") assert.Equal(t, v.Hash(), state.Validators.Hash(), "expected validator hashes to match") // Should NOT be able to load for height 2. - _, err = statestore.LoadValidators(2) + blockStore.On("LoadBlockMeta", int64(2)).Return(nil) + _, err = statestore.LoadValidators(2, blockStore) require.Error(t, err, "expected no err at height 2") // Increment height, save; should be able to load for next height. @@ -261,11 +270,11 @@ func TestValidatorSimpleSaveLoad(t *testing.T) { nextHeight := state.LastBlockHeight + 1 err = statestore.Save(state) require.NoError(t, err) - vp0, err := statestore.LoadValidators(nextHeight + 0) + vp0, err := statestore.LoadValidators(nextHeight+0, blockStore) assert.NoError(t, err) assert.Equal(t, vp0.Hash(), state.Validators.Hash(), "expected validator hashes to match") - _, err = statestore.LoadValidators(nextHeight + 1) + _, err = statestore.LoadValidators(nextHeight+1, blockStore) assert.Error(t, err) // assert.Equal(t, vp1.Hash(), state.NextValidators.Hash(), "expected next validator hashes to match") } @@ -275,6 +284,8 @@ func TestOneValidatorChangesSaveLoad(t *testing.T) { tearDown, stateDB, state := setupTestCase(t) defer tearDown(t) stateStore := sm.NewStore(stateDB) + blockStore := mocks.NewBlockStore(t) + blockStore.On("Base").Return(int64(1)) // Change vals at these heights. changeHeights := []int64{1, 2, 4, 5, 10, 15, 16, 17, 20} @@ -315,7 +326,12 @@ func TestOneValidatorChangesSaveLoad(t *testing.T) { state, err = state.Update(blockID, &header, &changes) + blockStore.On("LoadBlockMeta", header.Height).Return(&types.BlockMeta{ + Header: header, + }).Maybe() + require.NoError(t, err) + validator := state.Validators.Validators[0] keys[height+1] = validator.PubKey err = stateStore.Save(state) @@ -323,8 +339,8 @@ func TestOneValidatorChangesSaveLoad(t *testing.T) { } for height := int64(2); height < highestHeight; height++ { - pubKey := keys[height] // new validators are in effect in the next block - v, err := stateStore.LoadValidators(height) // load validators that validate block at `height`` + pubKey := keys[height] // new validators are in effect in the next block + v, err := stateStore.LoadValidators(height, blockStore) // load validators that validate block at `height`` require.NoError(t, err, fmt.Sprintf("expected no err at height %d", height)) assert.Equal(t, 1, v.Size(), "validator set size is greater than 1: %d", v.Size()) val := v.GetByIndex(0) @@ -366,448 +382,6 @@ func TestEmptyValidatorUpdates(t *testing.T) { assert.Equal(t, expectValidators, types.ValidatorListString(changes.NextValidators.Validators), "validator should not change") } -//func TestProposerFrequency(t *testing.T) { -// ctx, cancel := context.WithCancel(context.Background()) -// defer cancel() -// -// // some explicit test cases -// testCases := []struct { -// powers []int64 -// }{ -// // 2 vals -// {[]int64{1, 1}}, -// {[]int64{1, 2}}, -// {[]int64{1, 100}}, -// {[]int64{5, 5}}, -// {[]int64{5, 100}}, -// {[]int64{50, 50}}, -// {[]int64{50, 100}}, -// {[]int64{1, 1000}}, -// -// // 3 vals -// {[]int64{1, 1, 1}}, -// {[]int64{1, 2, 3}}, -// {[]int64{1, 2, 3}}, -// {[]int64{1, 1, 10}}, -// {[]int64{1, 1, 100}}, -// {[]int64{1, 10, 100}}, -// {[]int64{1, 1, 1000}}, -// {[]int64{1, 10, 1000}}, -// {[]int64{1, 100, 1000}}, -// -// // 4 vals -// {[]int64{1, 1, 1, 1}}, -// {[]int64{1, 2, 3, 4}}, -// {[]int64{1, 1, 1, 10}}, -// {[]int64{1, 1, 1, 100}}, -// {[]int64{1, 1, 1, 1000}}, -// {[]int64{1, 1, 10, 100}}, -// {[]int64{1, 1, 10, 1000}}, -// {[]int64{1, 1, 100, 1000}}, -// {[]int64{1, 10, 100, 1000}}, -// } -// -// for caseNum, testCase := range testCases { -// // run each case 5 times to sample different -// // initial priorities -// for i := 0; i < 5; i++ { -// valSet := genValSetWithPowers(testCase.powers) -// testProposerFreq(t, caseNum, valSet) -// } -// } -// -// // some random test cases with up to 100 validators -// maxVals := 100 -// maxPower := 1000 -// nTestCases := 5 -// for i := 0; i < nTestCases; i++ { -// N := mrand.Int()%maxVals + 1 -// vals := make([]*types.Validator, N) -// totalVotePower := int64(0) -// for j := 0; j < N; j++ { -// // make sure votePower > 0 -// votePower := int64(mrand.Int()%maxPower) + 1 -// totalVotePower += votePower -// privVal := types.NewMockPV() -// pubKey, err := privVal.GetPubKey(ctx) -// require.NoError(t, err) -// val := types.NewValidator(pubKey, votePower) -// val.ProposerPriority = mrand.Int63() -// vals[j] = val -// } -// valSet := types.NewValidatorSet(vals) -// valSet.RescalePriorities(totalVotePower) -// testProposerFreq(t, i, valSet) -// } -//} -// -//// new val set with given powers and random initial priorities -//func genValSetWithPowers(powers []int64) *types.ValidatorSet { -// size := len(powers) -// vals := make([]*types.Validator, size) -// totalVotePower := int64(0) -// for i := 0; i < size; i++ { -// totalVotePower += powers[i] -// val := types.NewValidator(ed25519.GenPrivKey().PubKey(), powers[i]) -// val.ProposerPriority = mrand.Int63() -// vals[i] = val -// } -// valSet := types.NewValidatorSet(vals) -// valSet.RescalePriorities(totalVotePower) -// return valSet -//} -// -//// test a proposer appears as frequently as expected -//func testProposerFreq(t *testing.T, caseNum int, valSet *types.ValidatorSet) { -// N := valSet.Size() -// totalPower := valSet.TotalVotingPower() -// -// // run the proposer selection and track frequencies -// runMult := 1 -// runs := int(totalPower) * runMult -// freqs := make([]int, N) -// for i := 0; i < runs; i++ { -// prop := valSet.GetProposer() -// idx, _ := valSet.GetByAddress(prop.Address) -// freqs[idx]++ -// valSet.IncrementProposerPriority(1) -// } -// -// // assert frequencies match expected (max off by 1) -// for i, freq := range freqs { -// _, val := valSet.GetByIndex(int32(i)) -// expectFreq := int(val.VotingPower) * runMult -// gotFreq := freq -// abs := int(math.Abs(float64(expectFreq - gotFreq))) -// -// // max bound on expected vs seen freq was proven -// // to be 1 for the 2 validator case in -// // https://github.com/cwgoes/tm-proposer-idris -// // and inferred to generalize to N-1 -// bound := N - 1 -// require.True( -// t, -// abs <= bound, -// fmt.Sprintf("Case %d val %d (%d): got %d, expected %d", caseNum, i, N, gotFreq, expectFreq), -// ) -// } -//} - -// TestProposerPriorityDoesNotGetResetToZero assert that we preserve accum when calling updateState -// see https://github.com/tendermint/tendermint/issues/2718 -func TestProposerPriorityDoesNotGetResetToZero(t *testing.T) { - tearDown, _, state := setupTestCase(t) - defer tearDown(t) - - ld := llmq.MustGenerate(crypto.RandProTxHashes(2)) - - val1VotingPower := types.DefaultDashVotingPower - val1ProTxHash := ld.ProTxHashes[0] - val1PubKey := ld.PubKeyShares[0] - val1 := &types.Validator{ProTxHash: val1ProTxHash, PubKey: val1PubKey, VotingPower: val1VotingPower} - - quorumHash := crypto.RandQuorumHash() - state.Validators = types.NewValidatorSet([]*types.Validator{val1}, val1PubKey, btcjson.LLMQType_5_60, quorumHash, true) - - // NewValidatorSet calls IncrementProposerPriority but uses on a copy of val1 - assert.EqualValues(t, 0, val1.ProposerPriority) - - block, err := statefactory.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit), 0) - require.NoError(t, err) - blockID := block.BlockID(nil) - require.NoError(t, err) - - // Any node pro tx hash should do - firstNode := state.Validators.GetByIndex(0) - ctx := dash.ContextWithProTxHash(context.Background(), firstNode.ProTxHash) - changes, err := state.NewStateChangeset(ctx, sm.RoundParams{}) - assert.NoError(t, err) - updatedState, err := state.Update(blockID, &block.Header, &changes) - assert.NoError(t, err) - curTotal := val1VotingPower - // one increment step and one validator: 0 + power - total_power == 0 - assert.Equal(t, 0+val1VotingPower-curTotal, updatedState.Validators.Validators[0].ProposerPriority) - - // add a validator - val2ProTxHash := ld.ProTxHashes[1] - val2PubKey := ld.PubKeyShares[1] - val2VotingPower := types.DefaultDashVotingPower - fvp, err := cryptoenc.PubKeyToProto(val2PubKey) - require.NoError(t, err) - - updateAddVal := abci.ValidatorUpdate{ProTxHash: val2ProTxHash, PubKey: &fvp, Power: val2VotingPower} - validatorSetUpdate := &abci.ValidatorSetUpdate{ - ValidatorUpdates: []abci.ValidatorUpdate{updateAddVal}, - ThresholdPublicKey: fvp, - QuorumHash: quorumHash, - } - - changes, err = updatedState.NewStateChangeset(ctx, sm.RoundParams{ValidatorSetUpdate: validatorSetUpdate}) - assert.NoError(t, err) - - updatedState2, err := updatedState.Update(blockID, &block.Header, &changes) - assert.NoError(t, err) - - require.Equal(t, len(updatedState2.Validators.Validators), 2) - _, updatedVal1 := updatedState2.Validators.GetByProTxHash(val1ProTxHash) - _, addedVal2 := updatedState2.Validators.GetByProTxHash(val2ProTxHash) - - // adding a validator should not lead to a ProposerPriority equal to zero (unless the combination of averaging and - // incrementing would cause so; which is not the case here) - // Steps from adding new validator: - // 0 - val1 prio is 0, TVP after add: - wantVal1Prio := int64(0) - totalPowerAfter := val1VotingPower + val2VotingPower - // 1. Add - Val2 should be initially added with (-123) => - wantVal2Prio := -(totalPowerAfter + (totalPowerAfter >> 3)) - // 2. Scale - noop - // 3. Center - with avg, resulting val2:-61, val1:62 - avg := big.NewInt(0).Add(big.NewInt(wantVal1Prio), big.NewInt(wantVal2Prio)) - avg.Div(avg, big.NewInt(2)) - wantVal2Prio -= avg.Int64() // -61 - wantVal1Prio -= avg.Int64() // 62 - - // 4. Steps from IncrementProposerPriority - wantVal1Prio += val1VotingPower // 72 - wantVal2Prio += val2VotingPower // 39 - wantVal1Prio -= totalPowerAfter // -38 as val1 is proposer - - assert.Equal(t, wantVal1Prio, updatedVal1.ProposerPriority) - assert.Equal(t, wantVal2Prio, addedVal2.ProposerPriority) - - // Updating validators does not reset the ProposerPriority to zero if we keep the same quorum: - // If we change quorums it will! - // 1. Add - Val2 VotingPower change to 1 => - abciValidatorUpdates := types.ValidatorUpdatesRegenerateOnProTxHashes(ld.ProTxHashes) - - // this will cause the diff of priorities (77) - // to be larger than threshold == 2*totalVotingPower (22): - abciValidatorUpdates.QuorumHash = quorumHash - changes, err = updatedState2.NewStateChangeset(ctx, sm.RoundParams{ValidatorSetUpdate: validatorSetUpdate}) - require.NoError(t, err) - - updatedState3, err := updatedState2.Update(blockID, &block.Header, &changes) - assert.NoError(t, err) - - require.Equal(t, len(updatedState3.Validators.Validators), 2) - _, prevVal1 := updatedState2.Validators.GetByProTxHash(val1ProTxHash) - _, prevVal2 := updatedState2.Validators.GetByProTxHash(val2ProTxHash) - _, updatedVal1 = updatedState3.Validators.GetByProTxHash(val1ProTxHash) - _, updatedVal2 := updatedState3.Validators.GetByProTxHash(val2ProTxHash) - - assert.NotZero(t, updatedVal1) - assert.NotZero(t, updatedVal2) - // 2. Scale - // old prios: v1(100):13, v2(100):-12 - wantVal1Prio = prevVal1.ProposerPriority - wantVal2Prio = prevVal2.ProposerPriority - // scale to diffMax = 400 = 2 * tvp, diff=13-(-12)=25 - // new totalPower - totalPower := updatedVal1.VotingPower + updatedVal2.VotingPower - dist := wantVal2Prio - wantVal1Prio - if dist < 0 { // get the absolute distance - dist *= -1 - } - // ratio := (dist + 2*totalPower - 1) / 2*totalPower = 224/200 = 1 - ratio := int64(float64(dist+2*totalPower-1) / float64(2*totalPower)) - // v1(100):13/1, v2(100):-12/1 - if ratio != 0 { - wantVal1Prio /= ratio // 13 - wantVal2Prio /= ratio // -12 - } - - // 3. Center - noop - // 4. IncrementProposerPriority() -> - // v1(100):13+100, v2(100):-12+100 -> v2 proposer so subtract tvp(11) - // v1(100):-87, v2(1):88 - wantVal2Prio += updatedVal2.VotingPower // 88 -> prop - wantVal1Prio += updatedVal1.VotingPower // 113 - wantVal1Prio -= totalPower // -87 - - assert.Equal(t, wantVal2Prio, updatedVal2.ProposerPriority) - assert.Equal(t, wantVal1Prio, updatedVal1.ProposerPriority) -} - -func TestProposerPriorityProposerAlternates(t *testing.T) { - // Regression test that would fail if the inner workings of - // IncrementProposerPriority change. - // Additionally, make sure that same power validators alternate if both - // have the same voting power (and the 2nd was added later). - tearDown, _, state := setupTestCase(t) - defer tearDown(t) - - ld := llmq.MustGenerate(crypto.RandProTxHashes(2)) - thresholdPublicKey := cryptoenc.MustPubKeyToProto(ld.ThresholdPubKey) - - val1VotingPower := types.DefaultDashVotingPower - val1ProTxHash := ld.ProTxHashes[0] - val1PubKey := ld.PubKeyShares[0] - val1 := &types.Validator{ProTxHash: val1ProTxHash, PubKey: val1PubKey, VotingPower: val1VotingPower} - - // reset state validators to above validator, the threshold key is just the validator key since there is only 1 validator - quorumHash := crypto.RandQuorumHash() - state.Validators = types.NewValidatorSet([]*types.Validator{val1}, val1PubKey, btcjson.LLMQType_5_60, quorumHash, true) - - // we only have one validator: - assert.Equal(t, val1ProTxHash, state.Validators.Proposer.ProTxHash) - - firstNode := state.Validators.GetByIndex(0) - ctx := dash.ContextWithProTxHash(context.Background(), firstNode.ProTxHash) - - block, err := statefactory.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit), 0) - require.NoError(t, err) - blockID := block.BlockID(nil) - require.NoError(t, err) - - // no updates: - changes, err := state.NewStateChangeset(ctx, sm.RoundParams{}) - assert.NoError(t, err) - - updatedState, err := state.Update(blockID, &block.Header, &changes) - assert.NoError(t, err) - - // 0 + 10 (initial prio) - 10 (avg) - 10 (mostest - total) = -10 - totalPower := val1VotingPower - wantVal1Prio := 0 + val1VotingPower - totalPower - assert.Equal(t, wantVal1Prio, updatedState.Validators.Validators[0].ProposerPriority) - assert.Equal(t, val1ProTxHash, updatedState.Validators.Proposer.ProTxHash) - - // add a validator with the same voting power as the first - val2ProTxHash := ld.ProTxHashes[1] - val2PubKey := ld.PubKeyShares[1] - fvp, err := cryptoenc.PubKeyToProto(val2PubKey) - require.NoError(t, err) - updateAddVal := abci.ValidatorUpdate{ProTxHash: val2ProTxHash, PubKey: &fvp, Power: val1VotingPower} - valsetUpdate := &abci.ValidatorSetUpdate{ - ValidatorUpdates: []abci.ValidatorUpdate{updateAddVal}, - ThresholdPublicKey: thresholdPublicKey, - QuorumHash: quorumHash, - } - changes, err = updatedState.NewStateChangeset(ctx, sm.RoundParams{ValidatorSetUpdate: valsetUpdate}) - assert.NoError(t, err) - - updatedState2, err := updatedState.Update(blockID, &block.Header, &changes) - assert.NoError(t, err) - - require.Equal(t, len(updatedState2.Validators.Validators), 2) - - // val1 will still be proposer as val2 just got added: - assert.Equal(t, val1ProTxHash, updatedState.Validators.Proposer.ProTxHash) - assert.Equal(t, updatedState2.Validators.Proposer.ProTxHash, updatedState2.Validators.Proposer.ProTxHash) - assert.Equal(t, updatedState.Validators.Proposer.ProTxHash, val1ProTxHash) - assert.Equal(t, updatedState2.Validators.Proposer.ProTxHash, val1ProTxHash) - - _, updatedVal1 := updatedState2.Validators.GetByProTxHash(val1ProTxHash) - _, oldVal1 := updatedState.Validators.GetByProTxHash(val1ProTxHash) - _, updatedVal2 := updatedState2.Validators.GetByProTxHash(val2ProTxHash) - - // 1. Add - val2VotingPower := val1VotingPower - totalPower = val1VotingPower + val2VotingPower // 20 - v2PrioWhenAddedVal2 := -(totalPower + (totalPower >> 3)) // -22 - // 2. Scale - noop - // 3. Center - avgSum := big.NewInt(0).Add(big.NewInt(v2PrioWhenAddedVal2), big.NewInt(oldVal1.ProposerPriority)) - avg := avgSum.Div(avgSum, big.NewInt(2)) // -11 - expectedVal2Prio := v2PrioWhenAddedVal2 - avg.Int64() // -11 - expectedVal1Prio := oldVal1.ProposerPriority - avg.Int64() // 11 - // 4. Increment - expectedVal2Prio += val2VotingPower // -11 + 10 = -1 - expectedVal1Prio += val1VotingPower // 11 + 10 == 21 - expectedVal1Prio -= totalPower // 1, val1 proposer - - assert.EqualValues(t, expectedVal1Prio, updatedVal1.ProposerPriority) - assert.EqualValues( - t, - expectedVal2Prio, - updatedVal2.ProposerPriority, - "unexpected proposer priority for validator: %v", - updatedVal2, - ) - - changes, err = updatedState2.NewStateChangeset(ctx, sm.RoundParams{ValidatorSetUpdate: valsetUpdate}) - assert.NoError(t, err) - updatedState3, err := updatedState2.Update(blockID, &block.Header, &changes) - assert.NoError(t, err) - - assert.Equal(t, updatedState3.Validators.Proposer.ProTxHash, updatedState3.Validators.Proposer.ProTxHash) - - // assert.Equal(t, updatedState3.Validators, updatedState2.Validators) - _, updatedVal1 = updatedState3.Validators.GetByProTxHash(val1ProTxHash) - _, updatedVal2 = updatedState3.Validators.GetByProTxHash(val2ProTxHash) - - // val1 will still be proposer: - assert.Equal(t, val1ProTxHash, updatedState3.Validators.Proposer.ProTxHash) - - // check if expected proposer prio is matched: - // Increment - expectedVal2Prio2 := expectedVal2Prio + val2VotingPower // -1 + 10 = 9 - expectedVal1Prio2 := expectedVal1Prio + val1VotingPower // 1 + 10 == 11 - expectedVal1Prio2 -= totalPower // -9, val1 proposer - - assert.EqualValues( - t, - expectedVal1Prio2, - updatedVal1.ProposerPriority, - "unexpected proposer priority for validator: %v", - updatedVal2, - ) - assert.EqualValues( - t, - expectedVal2Prio2, - updatedVal2.ProposerPriority, - "unexpected proposer priority for validator: %v", - updatedVal2, - ) - - // no changes in voting power and both validators have same voting power - // -> proposers should alternate: - oldState := updatedState3 - changes, err = oldState.NewStateChangeset(ctx, sm.RoundParams{}) - assert.NoError(t, err) - oldState, err = oldState.Update(blockID, &block.Header, &changes) - assert.NoError(t, err) - expectedVal1Prio2 = 13 - expectedVal2Prio2 = -12 - expectedVal1Prio = -87 - expectedVal2Prio = 88 - - for i := 0; i < 1000; i++ { - // no validator updates: - changes, err = oldState.NewStateChangeset(ctx, sm.RoundParams{}) - require.NoError(t, err) - - updatedState, err := oldState.Update(blockID, &block.Header, &changes) - assert.NoError(t, err) - // alternate (and cyclic priorities): - assert.NotEqual( - t, - updatedState.Validators.Proposer.ProTxHash, - oldState.Validators.Proposer.ProTxHash, - "iter: %v", - i, - ) - assert.Equal(t, oldState.LastValidators.Proposer.ProTxHash, updatedState.Validators.Proposer.ProTxHash, "iter: %v", i) - - _, updatedVal1 = updatedState.Validators.GetByProTxHash(val1ProTxHash) - assert.NotNil(t, updatedVal1) - _, updatedVal2 = updatedState.Validators.GetByProTxHash(val2ProTxHash) - assert.NotNil(t, updatedVal2) - - if i%2 == 0 { - assert.Equal(t, updatedState.Validators.Proposer.ProTxHash, val1ProTxHash) - assert.Equal(t, expectedVal1Prio, updatedVal1.ProposerPriority) // -19 - assert.Equal(t, expectedVal2Prio, updatedVal2.ProposerPriority) // 0 - } else { - assert.Equal(t, updatedState.Validators.Proposer.ProTxHash, val2ProTxHash) - assert.Equal(t, expectedVal1Prio2, updatedVal1.ProposerPriority) // -9 - assert.Equal(t, expectedVal2Prio2, updatedVal2.ProposerPriority) // -10 - } - // update for next iteration: - oldState = updatedState - } -} - func TestFourAddFourMinusOneGenesisValidators(t *testing.T) { tearDown, _, state := setupTestCase(t) defer tearDown(t) @@ -917,47 +491,19 @@ func TestFourAddFourMinusOneGenesisValidators(t *testing.T) { updatedState = execute(state, updatedState, changes) if i > numVals { // expect proposers to cycle through after the first iteration (of numVals blocks): if proposers[i%numVals] == nil { - proposers[i%numVals] = updatedState.Validators.Proposer + proposers[i%numVals] = updatedState.Validators.Proposer() } else { - assert.Equal(t, proposers[i%numVals], updatedState.Validators.Proposer) + assert.Equal(t, proposers[i%numVals], updatedState.Validators.Proposer()) } } } } -func TestStoreLoadValidatorsIncrementsProposerPriority(t *testing.T) { - const valSetSize = 2 - tearDown, stateDB, state := setupTestCase(t) - t.Cleanup(func() { tearDown(t) }) - stateStore := sm.NewStore(stateDB) - state.Validators, _ = types.RandValidatorSet(valSetSize) - err := stateStore.Save(state) - require.NoError(t, err) - - state2 := state.Copy() - state2.LastBlockHeight++ - state2.Validators = state.Validators.CopyIncrementProposerPriority(1) - state2.LastHeightValidatorsChanged = state2.LastBlockHeight + 1 - err = stateStore.Save(state2) - require.NoError(t, err) - - nextHeight := state.LastBlockHeight + 1 - - v0, err := stateStore.LoadValidators(nextHeight) - assert.NoError(t, err) - acc0 := v0.Validators[0].ProposerPriority - - v1, err := stateStore.LoadValidators(nextHeight + 1) - assert.NoError(t, err) - acc1 := v1.Validators[0].ProposerPriority - - assert.NotEqual(t, acc1, acc0, "expected ProposerPriority value to change between heights") -} - // TestValidatorChangesSaveLoad tests saving and loading a validator set with // changes. func TestManyValidatorChangesSaveLoad(t *testing.T) { const valSetSize = 7 + blockStore := mocks.NewBlockStore(t) // ====== GENESIS STATE, height 1 ====== // @@ -969,6 +515,13 @@ func TestManyValidatorChangesSaveLoad(t *testing.T) { err := stateStore.Save(state) require.NoError(t, err) + blockStore.On("LoadBlockMeta", state.LastBlockHeight).Return(&types.BlockMeta{ + Header: types.Header{ + Height: state.LastBlockHeight, + ProposerProTxHash: state.Validators.GetByIndex(0).ProTxHash, + }}).Maybe() + blockStore.On("Base").Return(state.LastBlockHeight) + // ====== HEIGHT 2 ====== // // We receive new validator set, which will be used at height 3 @@ -1003,9 +556,14 @@ func TestManyValidatorChangesSaveLoad(t *testing.T) { err = stateStore.Save(state) require.NoError(t, err) + blockStore.On("LoadBlockMeta", currentHeight).Return(&types.BlockMeta{ + Header: types.Header{ + Height: currentHeight, + ProposerProTxHash: state.Validators.GetByIndex(int32(currentHeight-state.InitialHeight) % valSetSize).ProTxHash, + }}).Maybe() // Load height, it should be the oldpubkey. - v0, err := stateStore.LoadValidators(currentHeight - 1) + v0, err := stateStore.LoadValidators(currentHeight-1, blockStore) assert.NoError(t, err) assert.Equal(t, valSetSize, v0.Size()) index, val := v0.GetByProTxHash(proTxHash) @@ -1016,7 +574,7 @@ func TestManyValidatorChangesSaveLoad(t *testing.T) { } // Load nextheight+1, it should be the new pubkey. - v1, err := stateStore.LoadValidators(currentHeight) + v1, err := stateStore.LoadValidators(currentHeight, blockStore) require.NoError(t, err) assert.Equal(t, valSetSize, v1.Size()) index, val = v1.GetByProTxHash(proTxHash) @@ -1032,12 +590,12 @@ func TestStateMakeBlock(t *testing.T) { tearDown, _, state := setupTestCase(t) defer tearDown(t) - proposerProTxHash := state.Validators.GetProposer().ProTxHash stateVersion := state.Version.Consensus // temporary workaround; state.Version.Consensus is deprecated and will be removed stateVersion.App = kvstore.ProtocolVersion var height int64 = 2 state.LastBlockHeight = height - 1 + proposerProTxHash := state.GetProposerFromState(height, 0).ProTxHash block, err := statefactory.MakeBlock(state, height, new(types.Commit), 0) require.NoError(t, err) diff --git a/internal/state/store.go b/internal/state/store.go index 0c97f9af30..1bd53067ba 100644 --- a/internal/state/store.go +++ b/internal/state/store.go @@ -10,6 +10,8 @@ import ( dbm "github.com/tendermint/tm-db" abci "github.com/dashpay/tenderdash/abci/types" + selectproposer "github.com/dashpay/tenderdash/internal/consensus/versioned/selectproposer" + "github.com/dashpay/tenderdash/libs/log" tmmath "github.com/dashpay/tenderdash/libs/math" tmstate "github.com/dashpay/tenderdash/proto/tendermint/state" tmproto "github.com/dashpay/tenderdash/proto/tendermint/types" @@ -92,7 +94,7 @@ type Store interface { // Load loads the current state of the blockchain Load() (State, error) // LoadValidators loads the validator set that is used to validate the given height - LoadValidators(int64) (*types.ValidatorSet, error) + LoadValidators(int64, selectproposer.BlockStore) (*types.ValidatorSet, error) // LoadABCIResponses loads the abciResponse for a given height LoadABCIResponses(int64) (*tmstate.ABCIResponses, error) // LoadConsensusParams loads the consensus params for a given height @@ -113,14 +115,15 @@ type Store interface { // dbStore wraps a db (github.com/tendermint/tm-db) type dbStore struct { - db dbm.DB + db dbm.DB + logger log.Logger } var _ Store = (*dbStore)(nil) // NewStore creates the dbStore of the state pkg. func NewStore(db dbm.DB) Store { - return dbStore{db} + return dbStore{db, log.NewNopLogger()} } // LoadState loads the State from the database. @@ -497,18 +500,19 @@ func (store dbStore) SaveValidatorSets(lowerHeight, upperHeight int64, vals *typ //----------------------------------------------------------------------------- -// LoadValidators loads the ValidatorSet for a given height. +// LoadValidators loads the ValidatorSet for a given height and round 0. +// // Returns ErrNoValSetForHeight if the validator set can't be found for this height. -func (store dbStore) LoadValidators(height int64) (*types.ValidatorSet, error) { - +func (store dbStore) LoadValidators(height int64, bs selectproposer.BlockStore) (*types.ValidatorSet, error) { valInfo, err := loadValidatorsInfo(store.db, height) if err != nil { return nil, ErrNoValSetForHeight{Height: height, Err: err} } + if valInfo.ValidatorSet == nil { lastStoredHeight := lastStoredHeightFor(height, valInfo.LastHeightChanged) - valInfo2, err := loadValidatorsInfo(store.db, lastStoredHeight) - if err != nil || valInfo2.ValidatorSet == nil { + valInfo, err = loadValidatorsInfo(store.db, lastStoredHeight) + if err != nil || valInfo.ValidatorSet == nil { return nil, fmt.Errorf("couldn't find validators at height %d (height %d was originally requested): %w", lastStoredHeight, @@ -516,33 +520,69 @@ func (store dbStore) LoadValidators(height int64) (*types.ValidatorSet, error) { err, ) } + } - vs, err := types.ValidatorSetFromProto(valInfo2.ValidatorSet) - if err != nil { - return nil, err + valSet, err := types.ValidatorSetFromProto(valInfo.ValidatorSet) + if err != nil { + return nil, err + } + + // FIND PROPOSER + + // As per definition, proposer at height 1 is the first validator in the validator set. + if height == 1 { + proposer := valSet.GetByIndex(0) + if err := valSet.SetProposer(proposer.ProTxHash); err != nil { + return nil, fmt.Errorf("could not set first proposer: %w", err) } - h, err := tmmath.SafeConvertInt32(height - lastStoredHeight) - if err != nil { - return nil, err + return valSet, nil + } + + // load consensus params to determine algorithm to use for proposer selection + cp, err := store.LoadConsensusParams(height) + if err != nil { + store.logger.Warn("failed to load consensus params, falling back to defaults", "height", height, "err", err) + cp = *types.DefaultConsensusParams() + } + + // if we have that block in block store, we just rolllback to round 0 + if meta := bs.LoadBlockMeta(height); meta != nil { + proposer := meta.Header.ProposerProTxHash + if err := valSet.SetProposer(proposer); err != nil { + return nil, fmt.Errorf("could not set proposer: %w", err) } - vs.IncrementProposerPriority(h) // mutate - vi2, err := vs.ToProto() + strategy, err := selectproposer.NewProposerSelector(cp, valSet, meta.Header.Height, meta.Round, bs, store.logger) if err != nil { - return nil, err + return nil, fmt.Errorf("failed to create validator scoring strategy: %w", err) } - - valInfo2.ValidatorSet = vi2 - valInfo = valInfo2 + if err := strategy.UpdateHeightRound(meta.Header.Height, 0); err != nil { + return nil, fmt.Errorf("failed to update validator scores at height %d, round 0: %w", meta.Header.Height, err) + } + return strategy.ValidatorSet(), nil } - vip, err := types.ValidatorSetFromProto(valInfo.ValidatorSet) + // If we have that height in the block store, we just take proposer from previous block and advance it. + // We don't use current height block because we want to return proposer at round 0. + prevMeta := bs.LoadBlockMeta(height - 1) + if prevMeta == nil { + return nil, fmt.Errorf("could not find block meta for height %d", height-1) + } + // Configure proposer strategy; first set proposer from previous block + if err := valSet.SetProposer(prevMeta.Header.ProposerProTxHash); err != nil { + return nil, fmt.Errorf("could not set proposer: %w", err) + } + strategy, err := selectproposer.NewProposerSelector(cp, valSet, prevMeta.Header.Height, prevMeta.Round, bs, store.logger) if err != nil { - return nil, err + return nil, fmt.Errorf("failed to create validator scoring strategy: %w", err) + } + + // now, advance to (height,0) + if err := strategy.UpdateHeightRound(height, 0); err != nil { + return nil, fmt.Errorf("failed to update validator scores at height %d, round 0: %w", height, err) } - // fmt.Printf("loaded validators at %d %v", height, vip) - return vip, nil + return strategy.ValidatorSet(), nil } func lastStoredHeightFor(height, lastHeightChanged int64) int64 { diff --git a/internal/state/store_test.go b/internal/state/store_test.go index 174f63ce9f..a696f5a8cc 100644 --- a/internal/state/store_test.go +++ b/internal/state/store_test.go @@ -7,6 +7,7 @@ import ( "testing" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" dbm "github.com/tendermint/tm-db" @@ -14,7 +15,10 @@ import ( "github.com/dashpay/tenderdash/config" "github.com/dashpay/tenderdash/crypto" "github.com/dashpay/tenderdash/crypto/bls12381" + selectproposer "github.com/dashpay/tenderdash/internal/consensus/versioned/selectproposer" + "github.com/dashpay/tenderdash/internal/evidence/mocks" sm "github.com/dashpay/tenderdash/internal/state" + "github.com/dashpay/tenderdash/libs/log" tmstate "github.com/dashpay/tenderdash/proto/tendermint/state" "github.com/dashpay/tenderdash/types" ) @@ -24,22 +28,47 @@ const ( valSetCheckpointInterval = 100000 ) +// mockBlockStoreForProposerSelector creates a mock block store that returns proposers based on the height. +// It assumes every block ends in round 0 and the proposer is the next validator in the validator set. +func mockBlockStoreForProposerSelector(t *testing.T, startHeight, endHeight int64, vals *types.ValidatorSet) selectproposer.BlockStore { + vals = vals.Copy() + valsHash := vals.Hash() + blockStore := mocks.NewBlockStore(t) + blockStore.On("Base").Return(startHeight).Maybe() + for h := startHeight; h <= endHeight; h++ { + blockStore.On("LoadBlockMeta", h). + Return(&types.BlockMeta{ + Header: types.Header{ + Height: h, + ProposerProTxHash: vals.Proposer().ProTxHash, + ValidatorsHash: valsHash, + NextValidatorsHash: valsHash, + }, + }).Maybe() + vals.IncProposerIndex(1) + } + + return blockStore +} + func TestStoreBootstrap(t *testing.T) { stateDB := dbm.NewMemDB() stateStore := sm.NewStore(stateDB) vals, _ := types.RandValidatorSet(3) - bootstrapState := makeRandomStateFromValidatorSet(vals, 100, 100) + blockStore := mockBlockStoreForProposerSelector(t, 99, 100, vals) + + bootstrapState := makeRandomStateFromValidatorSet(vals, 100, 100, blockStore) require.NoError(t, stateStore.Bootstrap(bootstrapState)) // bootstrap should also save the previous validator - _, err := stateStore.LoadValidators(99) + _, err := stateStore.LoadValidators(99, blockStore) require.NoError(t, err) - _, err = stateStore.LoadValidators(100) + _, err = stateStore.LoadValidators(100, blockStore) require.NoError(t, err) - _, err = stateStore.LoadValidators(101) + _, err = stateStore.LoadValidators(101, blockStore) require.Error(t, err) state, err := stateStore.Load() @@ -47,46 +76,95 @@ func TestStoreBootstrap(t *testing.T) { require.Equal(t, bootstrapState, state) } +// assertProposer checks if the proposer at height h is correct (assuming no rounds and we started at initial height 1) +func assertProposer(t *testing.T, valSet *types.ValidatorSet, h int64) { + t.Helper() + + const initialHeight = 1 + + // check if currently selected proposer is correct + idx, _ := valSet.GetByProTxHash(valSet.Proposer().ProTxHash) + exp := (h - initialHeight) % int64(valSet.Size()) + assert.EqualValues(t, exp, idx, "pre-set proposer at height %d", h) + + // check if GetProposer returns the same proposer + vs, err := selectproposer.NewProposerSelector(types.ConsensusParams{}, valSet.Copy(), h, 0, nil, log.NewTestingLogger(t)) + require.NoError(t, err) + + prop := vs.MustGetProposer(h, 0) + idx, _ = valSet.GetByProTxHash(prop.ProTxHash) + assert.EqualValues(t, exp, idx, "strategy-generated proposer at height %d", h) +} + func TestStoreLoadValidators(t *testing.T) { stateDB := dbm.NewMemDB() stateStore := sm.NewStore(stateDB) vals, _ := types.RandValidatorSet(3) + expectedVS, err := selectproposer.NewProposerSelector(types.ConsensusParams{}, vals.Copy(), 1, 0, nil, log.NewTestingLogger(t)) + require.NoError(t, err) + + // initialize block store - create mock validators for each height + blockStoreVS := expectedVS.Copy() + blockStore := mockBlockStoreForProposerSelector(t, 1, valSetCheckpointInterval, blockStoreVS.ValidatorSet()) + // 1) LoadValidators loads validators using a height where they were last changed // Note that only the current validators at height h are saved - require.NoError(t, stateStore.Save(makeRandomStateFromValidatorSet(vals, 1, 1))) - require.NoError(t, stateStore.Save(makeRandomStateFromValidatorSet(vals.CopyIncrementProposerPriority(1), 2, 1))) - loadedVals, err := stateStore.LoadValidators(2) + require.NoError(t, stateStore.Save(makeRandomStateFromValidatorSet(vals, 1, 1, blockStore))) + + require.NoError(t, stateStore.Save(makeRandomStateFromValidatorSet(vals, 2, 1, blockStore))) + + loadedValsH1, err := stateStore.LoadValidators(1, blockStore) + require.NoError(t, err) + assertProposer(t, loadedValsH1, 1) + + loadedValsH2, err := stateStore.LoadValidators(2, blockStore) require.NoError(t, err) + assertProposer(t, loadedValsH2, 2) - _, err = stateStore.LoadValidators(3) + _, err = stateStore.LoadValidators(3, blockStore) assert.Error(t, err, "no validator expected at this height") - require.Equal(t, vals.CopyIncrementProposerPriority(2), loadedVals) + err = expectedVS.UpdateHeightRound(2, 0) + require.NoError(t, err) + assertProposer(t, expectedVS.ValidatorSet(), 2) + + require.Equal(t, expectedVS.ValidatorSet(), loadedValsH2) // 2) LoadValidators loads validators using a checkpoint height // add a validator set after the checkpoint - state := makeRandomStateFromValidatorSet(vals, valSetCheckpointInterval+1, 1) + state := makeRandomStateFromValidatorSet(vals, valSetCheckpointInterval+1, 1, nil) err = stateStore.Save(state) require.NoError(t, err) // check that a request will go back to the last checkpoint - _, err = stateStore.LoadValidators(valSetCheckpointInterval + 1) + _, err = stateStore.LoadValidators(valSetCheckpointInterval+1, blockStore) require.Error(t, err) require.Equal(t, fmt.Sprintf("couldn't find validators at height %d (height %d was originally requested): "+ "value retrieved from db is empty", valSetCheckpointInterval, valSetCheckpointInterval+1), err.Error()) // now save a validator set at that checkpoint - err = stateStore.Save(makeRandomStateFromValidatorSet(vals, valSetCheckpointInterval, 1)) + err = stateStore.Save(makeRandomStateFromValidatorSet(vals, valSetCheckpointInterval, 1, blockStore)) require.NoError(t, err) - loadedVals, err = stateStore.LoadValidators(valSetCheckpointInterval) + valsAtCheckpoint, err := stateStore.LoadValidators(valSetCheckpointInterval, blockStore) require.NoError(t, err) - // ensure we have correct validator set loaded - require.Equal(t, vals.CopyIncrementProposerPriority(valSetCheckpointInterval), loadedVals) - require.NotEqual(t, vals.CopyIncrementProposerPriority(valSetCheckpointInterval-1), loadedVals) + + // ensure we have correct validator set loaded; at height h, we expcect `(h+1) % 3` + // (adding 1 as we start from initial height 1). + for h := int64(2); h <= valSetCheckpointInterval-1; h++ { + require.NoError(t, expectedVS.UpdateHeightRound(h, 0)) + } + expected := expectedVS.ValidatorSet() + assertProposer(t, expected, valSetCheckpointInterval-1) + require.NotEqual(t, expected, valsAtCheckpoint) + + require.NoError(t, expectedVS.UpdateHeightRound(valSetCheckpointInterval, 0)) + expected = expectedVS.ValidatorSet() + assertProposer(t, expected, valSetCheckpointInterval) + require.Equal(t, expected, valsAtCheckpoint) } // This benchmarks the speed of loading validators from different heights if there is no validator set change. @@ -95,6 +173,8 @@ func TestStoreLoadValidators(t *testing.T) { // and 2) retrieve the validator set at the aforementioned height 1. func BenchmarkLoadValidators(b *testing.B) { const valSetSize = 100 + blockStore := mocks.NewBlockStore(b) + blockStore.On("LoadBlockCommit", mock.Anything).Return(&types.Commit{}) cfg, err := config.ResetTestRoot(b.TempDir(), "state_") require.NoError(b, err) @@ -118,14 +198,14 @@ func BenchmarkLoadValidators(b *testing.B) { for i := 10; i < 10000000000; i *= 10 { // 10, 100, 1000, ... i := i err = stateStore.Save(makeRandomStateFromValidatorSet(state.Validators, - int64(i)-1, state.LastHeightValidatorsChanged)) + int64(i)-1, state.LastHeightValidatorsChanged, blockStore)) if err != nil { b.Fatalf("error saving store: %v", err) } b.Run(fmt.Sprintf("height=%d", i), func(b *testing.B) { for n := 0; n < b.N; n++ { - _, err := stateStore.LoadValidators(int64(i)) + _, err := stateStore.LoadValidators(int64(i), blockStore) if err != nil { b.Fatal(err) } @@ -192,7 +272,6 @@ func TestPruneStates(t *testing.T) { validator := &types.Validator{VotingPower: types.DefaultDashVotingPower, PubKey: pk, ProTxHash: proTxHash} validatorSet := &types.ValidatorSet{ Validators: []*types.Validator{validator}, - Proposer: validator, ThresholdPublicKey: validator.PubKey, QuorumHash: crypto.RandQuorumHash(), } @@ -245,8 +324,15 @@ func TestPruneStates(t *testing.T) { } require.NoError(t, err) + blockStore := mockBlockStoreForProposerSelector(t, tc.remainingValSetHeight, tc.endHeight, validatorSet) + // We initialize block store from remainingValSetHeight just to pass this test; in practive, it can be + // pruned. But here we want to check state store logic, not block store logic. + // for h := int64(1); h < tc.remainingValSetHeight; h++ { + // blockStore.On("LoadBlockMeta", h).Return(nil).Maybe() + // } + for h := tc.pruneHeight; h <= tc.endHeight; h++ { - vals, err := stateStore.LoadValidators(h) + vals, err := stateStore.LoadValidators(h, blockStore) require.NoError(t, err, h) require.NotNil(t, vals, h) @@ -262,7 +348,7 @@ func TestPruneStates(t *testing.T) { emptyParams := types.ConsensusParams{} for h := tc.startHeight; h < tc.pruneHeight; h++ { - vals, err := stateStore.LoadValidators(h) + vals, err := stateStore.LoadValidators(h, blockStore) if h == tc.remainingValSetHeight { require.NoError(t, err, h) require.NotNil(t, vals, h) diff --git a/internal/state/test/factory/block.go b/internal/state/test/factory/block.go index 49ad77261d..e6d25549b0 100644 --- a/internal/state/test/factory/block.go +++ b/internal/state/test/factory/block.go @@ -55,12 +55,13 @@ func MakeBlock(state sm.State, height int64, c *types.Commit, proposedAppVersion if state.LastBlockHeight != (height - 1) { return nil, fmt.Errorf("requested height %d should be 1 more than last block height %d", height, state.LastBlockHeight) } + proposer := state.GetProposerFromState(height, 0) block := state.MakeBlock( height, factory.MakeNTxs(state.LastBlockHeight, 10), c, nil, - state.Validators.GetProposer().ProTxHash, + proposer.ProTxHash, proposedAppVersion, ) var err error @@ -122,8 +123,8 @@ func makeBlockAndPartSet( }, ) } - - block := state.MakeBlock(height, []types.Tx{}, lastCommit, nil, state.Validators.GetProposer().ProTxHash, proposedAppVersion) + proposer := state.GetProposerFromState(height, 0) + block := state.MakeBlock(height, []types.Tx{}, lastCommit, nil, proposer.ProTxHash, proposedAppVersion) partSet, err := block.MakePartSet(types.BlockPartSizeBytes) require.NoError(t, err) diff --git a/internal/state/validation_test.go b/internal/state/validation_test.go index 4f92fdffa7..a8497cc6db 100644 --- a/internal/state/validation_test.go +++ b/internal/state/validation_test.go @@ -132,7 +132,8 @@ func TestValidateBlockHeader(t *testing.T) { // Build up state for multiple heights for height := int64(1); height < validationTestsStopHeight; height++ { - proposerProTxHash := state.Validators.GetProposer().ProTxHash + proposer := state.GetProposerFromState(height, 0) + proposerProTxHash := proposer.ProTxHash /* Invalid blocks don't pass */ @@ -209,7 +210,8 @@ func TestValidateBlockCommit(t *testing.T) { badPrivVal := types.NewMockPVForQuorum(badPrivValQuorumHash) for height := int64(1); height < validationTestsStopHeight; height++ { - proTxHash := state.Validators.GetProposer().ProTxHash + proposer := state.GetProposerFromState(height, 0) + proTxHash := proposer.ProTxHash if height > 1 { /* #2589: ensure state.LastValidators.VerifyCommit fails here @@ -386,7 +388,8 @@ func TestValidateBlockEvidence(t *testing.T) { lastCommit := types.NewCommit(0, 0, types.BlockID{}, nil, nil) for height := int64(1); height < validationTestsStopHeight; height++ { - proposerProTxHash := state.Validators.GetProposer().ProTxHash + proposer := state.GetProposerFromState(height, 0) + proposerProTxHash := proposer.ProTxHash maxBytesEvidence := state.ConsensusParams.Evidence.MaxBytes if height > 1 { /* diff --git a/internal/statesync/reactor.go b/internal/statesync/reactor.go index 0440e047ba..e31088ff4e 100644 --- a/internal/statesync/reactor.go +++ b/internal/statesync/reactor.go @@ -1089,7 +1089,7 @@ func (r *Reactor) fetchLightBlock(height uint64) (*types.LightBlock, error) { return nil, nil } - vals, err := r.stateStore.LoadValidators(h) + vals, err := r.stateStore.LoadValidators(h, r.blockStore) if err != nil { return nil, err } diff --git a/internal/statesync/reactor_test.go b/internal/statesync/reactor_test.go index 8178f313f2..40b06dad4c 100644 --- a/internal/statesync/reactor_test.go +++ b/internal/statesync/reactor_test.go @@ -495,7 +495,7 @@ func TestReactor_LightBlockResponse(t *testing.T) { require.NoError(t, rts.blockStore.SaveSignedHeader(sh, blockID)) - rts.stateStore.On("LoadValidators", height).Return(vals, nil) + rts.stateStore.On("LoadValidators", height, mock.Anything).Return(vals, nil) rts.blockInCh <- p2p.Envelope{ From: types.NodeID("aa"), diff --git a/internal/statesync/repo.go b/internal/statesync/repo.go index fa5fb4084d..900524e44a 100644 --- a/internal/statesync/repo.go +++ b/internal/statesync/repo.go @@ -34,7 +34,7 @@ func (r *LightBlockRepository) Get(height uint64) (*types.LightBlock, error) { return nil, nil } - vals, err := r.stateStore.LoadValidators(h) + vals, err := r.stateStore.LoadValidators(h, r.blockStore) if err != nil { return nil, err } diff --git a/internal/statesync/stateprovider.go b/internal/statesync/stateprovider.go index f30688c735..1fe8b93a86 100644 --- a/internal/statesync/stateprovider.go +++ b/internal/statesync/stateprovider.go @@ -155,6 +155,7 @@ func (s *stateProviderRPC) State(ctx context.Context, height uint64) (sm.State, Software: version.TMCoreSemVer, } state.LastBlockHeight = lastLightBlock.Height + state.LastBlockRound = lastLightBlock.Commit.Round state.LastBlockTime = lastLightBlock.Time state.LastBlockID = lastLightBlock.Commit.BlockID state.LastCoreChainLockedBlockHeight = lastLightBlock.Header.CoreChainLockedHeight @@ -293,6 +294,7 @@ func (s *stateProviderP2P) State(ctx context.Context, height uint64) (sm.State, Software: version.TMCoreSemVer, } state.LastBlockHeight = lastLightBlock.Height + state.LastBlockRound = lastLightBlock.Commit.Round state.LastBlockTime = lastLightBlock.Time state.LastBlockID = lastLightBlock.Commit.BlockID state.LastCoreChainLockedBlockHeight = lastLightBlock.Header.CoreChainLockedHeight diff --git a/internal/statesync/syncer_test.go b/internal/statesync/syncer_test.go index e5934608a1..00da83c9ed 100644 --- a/internal/statesync/syncer_test.go +++ b/internal/statesync/syncer_test.go @@ -91,10 +91,10 @@ func (suite *SyncerTestSuite) TestSyncAny() { LastAppHash: []byte("app_hash"), LastValidators: &types.ValidatorSet{ - Proposer: &types.Validator{ProTxHash: crypto.Checksum([]byte("val1"))}, + Validators: []*types.Validator{{ProTxHash: crypto.Checksum([]byte("val1"))}}, }, Validators: &types.ValidatorSet{ - Proposer: &types.Validator{ProTxHash: crypto.Checksum([]byte("val2"))}, + Validators: []*types.Validator{{ProTxHash: crypto.Checksum([]byte("val2"))}}, }, ConsensusParams: *types.DefaultConsensusParams(), diff --git a/libs/math/safemath.go b/libs/math/safemath.go index af40548ea9..9afb409b21 100644 --- a/libs/math/safemath.go +++ b/libs/math/safemath.go @@ -2,13 +2,38 @@ package math import ( "errors" + "fmt" "math" ) +var ErrOverflowInt64 = errors.New("int64 overflow") var ErrOverflowInt32 = errors.New("int32 overflow") +var ErrOverflowUint32 = errors.New("uint32 overflow") var ErrOverflowUint8 = errors.New("uint8 overflow") var ErrOverflowInt8 = errors.New("int8 overflow") +// SafeAddClipInt64 adds two int64 integers and clips the result to the int64 range. +func SafeAddClipInt64(a, b int64) int64 { + c, err := SafeAddInt64(a, b) + if err != nil { + if b < 0 { + return math.MinInt64 + } + return math.MaxInt64 + } + return c +} + +// SafeAddInt64 adds two int64 integers. +func SafeAddInt64(a, b int64) (int64, error) { + if b > 0 && (a > math.MaxInt64-b) { + return 0, ErrOverflowInt64 + } else if b < 0 && (a < math.MinInt64-b) { + return 0, ErrOverflowInt64 + } + return a + b, nil +} + // SafeAddInt32 adds two int32 integers. func SafeAddInt32(a, b int32) (int32, error) { if b > 0 && (a > math.MaxInt32-b) { @@ -19,6 +44,26 @@ func SafeAddInt32(a, b int32) (int32, error) { return a + b, nil } +func SafeSubInt64(a, b int64) (int64, bool) { + if b > 0 && a < math.MinInt64+b { + return -1, true + } else if b < 0 && a > math.MaxInt64+b { + return -1, true + } + return a - b, false +} + +func SafeSubClipInt64(a, b int64) int64 { + c, overflow := SafeSubInt64(a, b) + if overflow { + if b > 0 { + return math.MinInt64 + } + return math.MaxInt64 + } + return c +} + // SafeSubInt32 subtracts two int32 integers. func SafeSubInt32(a, b int32) (int32, error) { if b > 0 && (a < math.MinInt32+b) { @@ -30,15 +75,49 @@ func SafeSubInt32(a, b int32) (int32, error) { } // SafeConvertInt32 takes a int and checks if it overflows. -func SafeConvertInt32(a int64) (int32, error) { - if a > math.MaxInt32 { +func SafeConvertInt32[T Integer](a T) (int32, error) { + if int64(a) > math.MaxInt32 { return 0, ErrOverflowInt32 - } else if a < math.MinInt32 { + } else if int64(a) < math.MinInt32 { return 0, ErrOverflowInt32 } return int32(a), nil } +// SafeConvertInt32 takes a int and checks if it overflows. +func SafeConvertUint32[T Integer](a T) (uint32, error) { + if uint64(a) > math.MaxUint32 { + return 0, ErrOverflowUint32 + } else if a < 0 { + return 0, ErrOverflowUint32 + } + return uint32(a), nil +} + +type Integer interface { + ~int | ~int8 | ~int16 | ~int32 | ~int64 | ~uint | ~uint8 | ~uint16 | ~uint32 | ~uint64 +} + +// MustConvertInt32 takes an Integer and converts it to int32. +// Panics if the conversion overflows. +func MustConvertInt32[T Integer](a T) int32 { + i, err := SafeConvertInt32(a) + if err != nil { + panic(fmt.Errorf("cannot convert %d to int32: %w", a, err)) + } + return i +} + +// MustConvertInt32 takes an Integer and converts it to int32. +// Panics if the conversion overflows. +func MustConvertUint32[T Integer](a T) uint32 { + i, err := SafeConvertUint32(a) + if err != nil { + panic(fmt.Errorf("cannot convert %d to int32: %w", a, err)) + } + return i +} + // SafeConvertUint8 takes an int64 and checks if it overflows. func SafeConvertUint8(a int64) (uint8, error) { if a > math.MaxUint8 { @@ -58,3 +137,25 @@ func SafeConvertInt8(a int64) (int8, error) { } return int8(a), nil } + +func SafeMulInt64(a, b int64) (int64, bool) { + if a == 0 || b == 0 { + return 0, false + } + + absOfB := b + if b < 0 { + absOfB = -b + } + + absOfA := a + if a < 0 { + absOfA = -a + } + + if absOfA > math.MaxInt64/absOfB { + return 0, true + } + + return a * b, false +} diff --git a/libs/math/safemath_test.go b/libs/math/safemath_test.go new file mode 100644 index 0000000000..92a8f32110 --- /dev/null +++ b/libs/math/safemath_test.go @@ -0,0 +1,86 @@ +package math + +import ( + "math" + "testing" + "testing/quick" + + "github.com/stretchr/testify/assert" +) + +func TestSafeAdd(t *testing.T) { + f := func(a, b int64) bool { + c, overflow := SafeAddInt64(a, b) + return overflow != nil || c == a+b + } + if err := quick.Check(f, nil); err != nil { + t.Error(err) + } +} + +func TestSafeAddClip(t *testing.T) { + assert.EqualValues(t, math.MaxInt64, SafeAddClipInt64(math.MaxInt64, 10)) + assert.EqualValues(t, math.MaxInt64, SafeAddClipInt64(math.MaxInt64, math.MaxInt64)) + assert.EqualValues(t, math.MinInt64, SafeAddClipInt64(math.MinInt64, -10)) +} + +func TestSafeSubClip(t *testing.T) { + assert.EqualValues(t, math.MinInt64, SafeSubClipInt64(math.MinInt64, 10)) + assert.EqualValues(t, 0, SafeSubClipInt64(math.MinInt64, math.MinInt64)) + assert.EqualValues(t, math.MinInt64, SafeSubClipInt64(math.MinInt64, math.MaxInt64)) + assert.EqualValues(t, math.MaxInt64, SafeSubClipInt64(math.MaxInt64, -10)) +} + +func TestSafeConvertUint32(t *testing.T) { + testCases := []struct { + a int64 + overflow bool + }{ + {-1, true}, + {0, false}, + {1, false}, + {math.MaxInt64, true}, + {math.MaxInt32, false}, + {math.MaxUint32, false}, + {math.MaxUint32 + 1, true}, + {math.MaxInt32, false}, + } + + for i, tc := range testCases { + b, err := SafeConvertUint32(tc.a) + if tc.overflow { + assert.Error(t, err, "#%d", i) + assert.Panics(t, func() { MustConvertUint32(tc.a) }, "#%d", i) + } else { + assert.EqualValues(t, tc.a, b, "#%d", i) + assert.NotPanics(t, func() { MustConvertUint32(tc.a) }, "#%d", i) + } + + } +} + +func TestSafeMul(t *testing.T) { + testCases := []struct { + a int64 + b int64 + c int64 + overflow bool + }{ + 0: {0, 0, 0, false}, + 1: {1, 0, 0, false}, + 2: {2, 3, 6, false}, + 3: {2, -3, -6, false}, + 4: {-2, -3, 6, false}, + 5: {-2, 3, -6, false}, + 6: {math.MaxInt64, 1, math.MaxInt64, false}, + 7: {math.MaxInt64 / 2, 2, math.MaxInt64 - 1, false}, + 8: {math.MaxInt64 / 2, 3, 0, true}, + 9: {math.MaxInt64, 2, 0, true}, + } + + for i, tc := range testCases { + c, overflow := SafeMulInt64(tc.a, tc.b) + assert.Equal(t, tc.c, c, "#%d", i) + assert.Equal(t, tc.overflow, overflow, "#%d", i) + } +} diff --git a/light/client_test.go b/light/client_test.go index 34daf19e03..f0a74e3919 100644 --- a/light/client_test.go +++ b/light/client_test.go @@ -576,7 +576,6 @@ func TestClient(t *testing.T) { t.Run("EnsureValidHeadersAndValSets", func(t *testing.T) { emptyValSet := &types.ValidatorSet{ Validators: nil, - Proposer: nil, } testCases := []struct { diff --git a/light/provider/http/http.go b/light/provider/http/http.go index 546d2334d4..73451e6b44 100644 --- a/light/provider/http/http.go +++ b/light/provider/http/http.go @@ -138,7 +138,7 @@ func (p *http) LightBlock(ctx context.Context, height int64) (*types.LightBlock, Reason: fmt.Errorf("height %d responded doesn't match height %d requested", sh.Height, height), } } - vs, err := p.validatorSet(ctx, &sh.Height) + vs, err := p.validatorSet(ctx, &sh.Height, sh.Header.ProposerProTxHash) if err != nil { return nil, err } @@ -162,7 +162,7 @@ func (p *http) ReportEvidence(ctx context.Context, ev types.Evidence) error { return err } -func (p *http) validatorSet(ctx context.Context, height *int64) (*types.ValidatorSet, error) { +func (p *http) validatorSet(ctx context.Context, height *int64, proposer types.ProTxHash) (*types.ValidatorSet, error) { // Since the malicious node could report a massive number of pages, making us // spend a considerable time iterating, we restrict the number of pages here. // => 10000 validators max @@ -238,11 +238,19 @@ func (p *http) validatorSet(ctx context.Context, height *int64) (*types.Validato break } } + valSet := types.NewValidatorSet(vals, thresholdPubKey, quorumType, quorumHash, false) - valSet, err := types.ValidatorSetFromExistingValidators(vals, thresholdPubKey, quorumType, quorumHash) - if err != nil { - return nil, provider.ErrBadLightBlock{Reason: err} + if valSet == nil || valSet.IsNilOrEmpty() { + return nil, provider.ErrBadLightBlock{Reason: fmt.Errorf("retrieved nil or empty validator set")} + } + if err := valSet.ValidateBasic(); err != nil { + return nil, provider.ErrBadLightBlock{Reason: fmt.Errorf("invalid validator set retrieved: %w", err)} } + + if err := valSet.SetProposer(proposer); err != nil { + return nil, provider.ErrBadLightBlock{Reason: fmt.Errorf("failed to determine proposer: %w", err)} + } + return valSet, nil } diff --git a/proto/tendermint/state/types.pb.go b/proto/tendermint/state/types.pb.go index 1a53d15db6..dcd836a2a1 100644 --- a/proto/tendermint/state/types.pb.go +++ b/proto/tendermint/state/types.pb.go @@ -262,6 +262,8 @@ type State struct { AppHash []byte `protobuf:"bytes,13,opt,name=app_hash,json=appHash,proto3" json:"app_hash,omitempty"` LastCoreChainLockedBlockHeight uint32 `protobuf:"varint,14,opt,name=last_core_chain_locked_block_height,json=lastCoreChainLockedBlockHeight,proto3" json:"last_core_chain_locked_block_height,omitempty"` NextCoreChainLock *types1.CoreChainLock `protobuf:"bytes,15,opt,name=next_core_chain_lock,json=nextCoreChainLock,proto3" json:"next_core_chain_lock,omitempty"` + // last_block_round is the round number at which the last block was committed. + LastBlockRound int32 `protobuf:"varint,16,opt,name=last_block_round,json=lastBlockRound,proto3" json:"last_block_round,omitempty"` } func (m *State) Reset() { *m = State{} } @@ -402,6 +404,13 @@ func (m *State) GetNextCoreChainLock() *types1.CoreChainLock { return nil } +func (m *State) GetLastBlockRound() int32 { + if m != nil { + return m.LastBlockRound + } + return 0 +} + func init() { proto.RegisterType((*ABCIResponses)(nil), "tendermint.state.ABCIResponses") proto.RegisterType((*ValidatorsInfo)(nil), "tendermint.state.ValidatorsInfo") @@ -413,56 +422,57 @@ func init() { func init() { proto.RegisterFile("tendermint/state/types.proto", fileDescriptor_ccfacf933f22bf93) } var fileDescriptor_ccfacf933f22bf93 = []byte{ - // 771 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x55, 0xcd, 0x4e, 0xdb, 0x4c, - 0x14, 0x8d, 0x3f, 0x7e, 0x92, 0x4c, 0x48, 0x02, 0x03, 0x0b, 0x13, 0x3e, 0x9c, 0x34, 0xfd, 0x51, - 0xd4, 0x85, 0x23, 0xb5, 0xdd, 0x74, 0x53, 0x89, 0x04, 0xa9, 0x44, 0x45, 0x08, 0x99, 0x8a, 0x45, - 0x37, 0xd6, 0xc4, 0x1e, 0x62, 0xab, 0x89, 0xc7, 0xf2, 0x4c, 0x68, 0x79, 0x80, 0xee, 0xd9, 0xf6, - 0x61, 0xba, 0x67, 0xc9, 0xb2, 0x2b, 0x5a, 0x85, 0x17, 0xa9, 0xe6, 0xc7, 0xce, 0x38, 0x61, 0x41, - 0xd5, 0x9d, 0xe7, 0x9e, 0x7b, 0xcf, 0x3d, 0x73, 0xe7, 0x5c, 0x19, 0xfc, 0xcf, 0x70, 0xe4, 0xe3, - 0x64, 0x12, 0x46, 0xac, 0x4b, 0x19, 0x62, 0xb8, 0xcb, 0xae, 0x62, 0x4c, 0xed, 0x38, 0x21, 0x8c, - 0xc0, 0xcd, 0x39, 0x6a, 0x0b, 0xb4, 0xb1, 0x33, 0x22, 0x23, 0x22, 0xc0, 0x2e, 0xff, 0x92, 0x79, - 0x8d, 0x3d, 0x8d, 0x05, 0x0d, 0xbd, 0x50, 0x27, 0x69, 0xe8, 0x2d, 0x44, 0x3c, 0x87, 0xee, 0x2d, - 0xa1, 0x3e, 0xa2, 0x81, 0x02, 0x5b, 0x4b, 0xe0, 0x25, 0x1a, 0x87, 0x3e, 0x62, 0x24, 0x51, 0x19, - 0xfb, 0x4b, 0x19, 0x31, 0x4a, 0xd0, 0x24, 0x65, 0xb7, 0x34, 0xf8, 0x12, 0x27, 0x34, 0x24, 0x51, - 0xae, 0x7b, 0x73, 0x44, 0xc8, 0x68, 0x8c, 0xbb, 0xe2, 0x34, 0x9c, 0x5e, 0x74, 0x59, 0x38, 0xc1, - 0x94, 0xa1, 0x49, 0x2c, 0x13, 0xda, 0x3e, 0xa8, 0x1e, 0xf4, 0xfa, 0x03, 0x07, 0xd3, 0x98, 0x44, - 0x14, 0x53, 0x78, 0x06, 0x36, 0xe3, 0x84, 0x78, 0x98, 0x52, 0x37, 0x4e, 0x48, 0x4c, 0x28, 0x1a, - 0x9b, 0x46, 0xcb, 0xe8, 0x54, 0x5e, 0x75, 0x6c, 0x6d, 0x5a, 0x7c, 0x0a, 0x76, 0x5a, 0x75, 0x2a, - 0x0b, 0x4e, 0x55, 0xbe, 0x53, 0x8f, 0xf3, 0x81, 0xf6, 0x37, 0x03, 0xd4, 0xce, 0xd3, 0x9b, 0xd1, - 0x41, 0x74, 0x41, 0x60, 0x1f, 0x54, 0xb3, 0xbb, 0xba, 0x14, 0x33, 0xd5, 0xc4, 0xd2, 0x9b, 0xc8, - 0x9b, 0x64, 0x85, 0x67, 0x98, 0x39, 0x1b, 0x97, 0xda, 0x09, 0xda, 0x60, 0x7b, 0x8c, 0x28, 0x73, - 0x03, 0x1c, 0x8e, 0x02, 0xe6, 0x7a, 0x01, 0x8a, 0x46, 0xd8, 0x37, 0xff, 0x6b, 0x19, 0x9d, 0x15, - 0x67, 0x8b, 0x43, 0x47, 0x02, 0xe9, 0x4b, 0xa0, 0xfd, 0xdd, 0x00, 0xdb, 0x7d, 0xae, 0x38, 0xa2, - 0x53, 0x7a, 0x2a, 0x06, 0x29, 0xc4, 0x38, 0x60, 0xd3, 0x4b, 0xc3, 0xae, 0x1c, 0xb0, 0xd2, 0xf3, - 0x64, 0x59, 0xcf, 0x02, 0x41, 0x6f, 0xf5, 0xe6, 0xae, 0x59, 0x70, 0xea, 0x5e, 0x3e, 0xfc, 0xd7, - 0xda, 0x02, 0x50, 0x3c, 0x97, 0x2f, 0x08, 0x0f, 0x40, 0x39, 0x63, 0x53, 0x3a, 0xf6, 0x75, 0x1d, - 0xea, 0xa5, 0xe7, 0x4a, 0x94, 0x86, 0x79, 0x15, 0x6c, 0x80, 0x12, 0x25, 0x17, 0xec, 0x0b, 0x4a, - 0xb0, 0x68, 0x59, 0x76, 0xb2, 0x73, 0xfb, 0x47, 0x11, 0xac, 0x9d, 0x71, 0xb7, 0xc3, 0xb7, 0xa0, - 0xa8, 0xb8, 0x54, 0x9b, 0x5d, 0x7b, 0x71, 0x23, 0x6c, 0x25, 0x4a, 0xb5, 0x48, 0xf3, 0xe1, 0x0b, - 0x50, 0xf2, 0x02, 0x14, 0x46, 0x6e, 0x28, 0xef, 0x54, 0xee, 0x55, 0x66, 0x77, 0xcd, 0x62, 0x9f, - 0xc7, 0x06, 0x87, 0x4e, 0x51, 0x80, 0x03, 0x1f, 0x3e, 0x07, 0xb5, 0x30, 0x0a, 0x59, 0x88, 0xc6, - 0x6a, 0x12, 0xe6, 0x8a, 0x98, 0x40, 0x55, 0x45, 0xe5, 0x10, 0xe0, 0x4b, 0x20, 0x46, 0xe2, 0x0e, - 0xc7, 0xc4, 0xfb, 0x9c, 0x66, 0xae, 0x8a, 0xcc, 0x3a, 0x07, 0x7a, 0x3c, 0xae, 0x72, 0x1d, 0x50, - 0xd5, 0x72, 0x43, 0xdf, 0x5c, 0x5b, 0xd6, 0x2e, 0x9f, 0x4a, 0x54, 0x0d, 0x0e, 0x7b, 0xdb, 0x5c, - 0xfb, 0xec, 0xae, 0x59, 0x39, 0x4e, 0xa9, 0x06, 0x87, 0x4e, 0x25, 0xe3, 0x1d, 0xf8, 0xf0, 0x18, - 0xd4, 0x35, 0x4e, 0xbe, 0x25, 0xe6, 0xba, 0x60, 0x6d, 0xd8, 0x72, 0x85, 0xec, 0x74, 0x85, 0xec, - 0x8f, 0xe9, 0x0a, 0xf5, 0x4a, 0x9c, 0xf6, 0xfa, 0x57, 0xd3, 0x70, 0xaa, 0x19, 0x17, 0x47, 0xe1, - 0x3b, 0x00, 0x32, 0x9f, 0x52, 0xb3, 0xf8, 0x28, 0x67, 0x6b, 0x15, 0xf0, 0xbd, 0x52, 0xa3, 0x91, - 0x94, 0x1e, 0x45, 0x52, 0xe3, 0x65, 0xf3, 0x4d, 0x83, 0x7d, 0x60, 0xe9, 0x26, 0x9c, 0xf3, 0x65, - 0x7e, 0x2c, 0x8b, 0x19, 0xef, 0xcd, 0xfd, 0x38, 0xaf, 0x56, 0xce, 0x7c, 0x70, 0x3b, 0xc0, 0x3f, - 0x6e, 0xc7, 0x09, 0x78, 0x96, 0xdb, 0x8e, 0x05, 0xfe, 0x4c, 0x5e, 0x45, 0xc8, 0x6b, 0x69, 0xeb, - 0x92, 0x27, 0x4a, 0x35, 0xa6, 0xfe, 0x49, 0x30, 0x9d, 0x8e, 0x19, 0x75, 0x03, 0x44, 0x03, 0x73, - 0xa3, 0x65, 0x74, 0x36, 0xa4, 0x7f, 0x1c, 0x19, 0x3f, 0x42, 0x34, 0x80, 0xbb, 0xa0, 0x84, 0xe2, - 0x58, 0xa6, 0x54, 0x45, 0x4a, 0x11, 0xc5, 0xb1, 0x80, 0x3e, 0x80, 0xa7, 0x82, 0xc6, 0x23, 0x09, - 0x76, 0xa5, 0xbf, 0xf9, 0xa3, 0x62, 0x3f, 0x6f, 0xcc, 0x5a, 0xcb, 0xe8, 0x54, 0x1d, 0x31, 0xda, - 0x3e, 0x49, 0xb0, 0x30, 0xfd, 0xb1, 0xc8, 0xd3, 0x7d, 0x7a, 0x0e, 0x76, 0x22, 0xfc, 0x75, 0x89, - 0xcc, 0xac, 0x8b, 0xd9, 0x35, 0x1f, 0x9a, 0x9d, 0xc6, 0x25, 0x26, 0x67, 0x38, 0x5b, 0x9c, 0x22, - 0x0f, 0x9c, 0xdc, 0xcc, 0x2c, 0xe3, 0x76, 0x66, 0x19, 0xbf, 0x67, 0x96, 0x71, 0x7d, 0x6f, 0x15, - 0x6e, 0xef, 0xad, 0xc2, 0xcf, 0x7b, 0xab, 0xf0, 0xe9, 0xcd, 0x28, 0x64, 0xc1, 0x74, 0x68, 0x7b, - 0x64, 0x22, 0x7e, 0x33, 0x31, 0xba, 0xea, 0xca, 0x2e, 0xfc, 0x24, 0xff, 0x02, 0xdd, 0xc5, 0x3f, - 0xe2, 0x70, 0x5d, 0xc4, 0x5f, 0xff, 0x09, 0x00, 0x00, 0xff, 0xff, 0x0b, 0x4e, 0x41, 0x27, 0x2c, - 0x07, 0x00, 0x00, + // 794 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x55, 0x4d, 0x4f, 0xdb, 0x4a, + 0x14, 0x8d, 0x1f, 0x1f, 0x49, 0x26, 0xe4, 0x83, 0x81, 0x85, 0x09, 0x0f, 0x27, 0x2f, 0xef, 0xbd, + 0x2a, 0xea, 0xc2, 0x91, 0xda, 0x6e, 0xba, 0xa9, 0x44, 0x82, 0x54, 0xa2, 0x22, 0x84, 0x4c, 0xc5, + 0xa2, 0x1b, 0x6b, 0x62, 0x0f, 0xb1, 0xd5, 0xc4, 0x63, 0x79, 0x26, 0xb4, 0xfc, 0x80, 0xee, 0xd9, + 0xf6, 0x1f, 0xb1, 0x64, 0xd9, 0x15, 0xad, 0xc2, 0xa2, 0x7f, 0xa3, 0x9a, 0x0f, 0x3b, 0xe3, 0x84, + 0x05, 0x55, 0x77, 0x9e, 0x7b, 0xee, 0x3d, 0xf7, 0xcc, 0x9d, 0x73, 0x65, 0xf0, 0x37, 0xc3, 0x91, + 0x8f, 0x93, 0x69, 0x18, 0xb1, 0x1e, 0x65, 0x88, 0xe1, 0x1e, 0xbb, 0x8e, 0x31, 0xb5, 0xe3, 0x84, + 0x30, 0x02, 0x1b, 0x0b, 0xd4, 0x16, 0x68, 0x73, 0x77, 0x4c, 0xc6, 0x44, 0x80, 0x3d, 0xfe, 0x25, + 0xf3, 0x9a, 0xfb, 0x1a, 0x0b, 0x1a, 0x79, 0xa1, 0x4e, 0xd2, 0xd4, 0x5b, 0x88, 0x78, 0x0e, 0xdd, + 0x5f, 0x41, 0x7d, 0x44, 0x03, 0x05, 0xb6, 0x57, 0xc0, 0x2b, 0x34, 0x09, 0x7d, 0xc4, 0x48, 0xa2, + 0x32, 0x0e, 0x56, 0x32, 0x62, 0x94, 0xa0, 0x69, 0xca, 0x6e, 0x69, 0xf0, 0x15, 0x4e, 0x68, 0x48, + 0xa2, 0x5c, 0xf7, 0xd6, 0x98, 0x90, 0xf1, 0x04, 0xf7, 0xc4, 0x69, 0x34, 0xbb, 0xec, 0xb1, 0x70, + 0x8a, 0x29, 0x43, 0xd3, 0x58, 0x26, 0x74, 0x7c, 0x50, 0x3d, 0xec, 0x0f, 0x86, 0x0e, 0xa6, 0x31, + 0x89, 0x28, 0xa6, 0xf0, 0x1c, 0x34, 0xe2, 0x84, 0x78, 0x98, 0x52, 0x37, 0x4e, 0x48, 0x4c, 0x28, + 0x9a, 0x98, 0x46, 0xdb, 0xe8, 0x56, 0x5e, 0x74, 0x6d, 0x6d, 0x5a, 0x7c, 0x0a, 0x76, 0x5a, 0x75, + 0x26, 0x0b, 0xce, 0x54, 0xbe, 0x53, 0x8f, 0xf3, 0x81, 0xce, 0x17, 0x03, 0xd4, 0x2e, 0xd2, 0x9b, + 0xd1, 0x61, 0x74, 0x49, 0xe0, 0x00, 0x54, 0xb3, 0xbb, 0xba, 0x14, 0x33, 0xd5, 0xc4, 0xd2, 0x9b, + 0xc8, 0x9b, 0x64, 0x85, 0xe7, 0x98, 0x39, 0x5b, 0x57, 0xda, 0x09, 0xda, 0x60, 0x67, 0x82, 0x28, + 0x73, 0x03, 0x1c, 0x8e, 0x03, 0xe6, 0x7a, 0x01, 0x8a, 0xc6, 0xd8, 0x37, 0xff, 0x6a, 0x1b, 0xdd, + 0x35, 0x67, 0x9b, 0x43, 0xc7, 0x02, 0x19, 0x48, 0xa0, 0xf3, 0xd5, 0x00, 0x3b, 0x03, 0xae, 0x38, + 0xa2, 0x33, 0x7a, 0x26, 0x06, 0x29, 0xc4, 0x38, 0xa0, 0xe1, 0xa5, 0x61, 0x57, 0x0e, 0x58, 0xe9, + 0xf9, 0x67, 0x55, 0xcf, 0x12, 0x41, 0x7f, 0xfd, 0xf6, 0xbe, 0x55, 0x70, 0xea, 0x5e, 0x3e, 0xfc, + 0xdb, 0xda, 0x02, 0x50, 0xbc, 0x90, 0x2f, 0x08, 0x0f, 0x41, 0x39, 0x63, 0x53, 0x3a, 0x0e, 0x74, + 0x1d, 0xea, 0xa5, 0x17, 0x4a, 0x94, 0x86, 0x45, 0x15, 0x6c, 0x82, 0x12, 0x25, 0x97, 0xec, 0x13, + 0x4a, 0xb0, 0x68, 0x59, 0x76, 0xb2, 0x73, 0xe7, 0x67, 0x11, 0x6c, 0x9c, 0x73, 0xb7, 0xc3, 0xd7, + 0xa0, 0xa8, 0xb8, 0x54, 0x9b, 0x3d, 0x7b, 0x79, 0x23, 0x6c, 0x25, 0x4a, 0xb5, 0x48, 0xf3, 0xe1, + 0x33, 0x50, 0xf2, 0x02, 0x14, 0x46, 0x6e, 0x28, 0xef, 0x54, 0xee, 0x57, 0xe6, 0xf7, 0xad, 0xe2, + 0x80, 0xc7, 0x86, 0x47, 0x4e, 0x51, 0x80, 0x43, 0x1f, 0xfe, 0x0f, 0x6a, 0x61, 0x14, 0xb2, 0x10, + 0x4d, 0xd4, 0x24, 0xcc, 0x35, 0x31, 0x81, 0xaa, 0x8a, 0xca, 0x21, 0xc0, 0xe7, 0x40, 0x8c, 0xc4, + 0x1d, 0x4d, 0x88, 0xf7, 0x31, 0xcd, 0x5c, 0x17, 0x99, 0x75, 0x0e, 0xf4, 0x79, 0x5c, 0xe5, 0x3a, + 0xa0, 0xaa, 0xe5, 0x86, 0xbe, 0xb9, 0xb1, 0xaa, 0x5d, 0x3e, 0x95, 0xa8, 0x1a, 0x1e, 0xf5, 0x77, + 0xb8, 0xf6, 0xf9, 0x7d, 0xab, 0x72, 0x92, 0x52, 0x0d, 0x8f, 0x9c, 0x4a, 0xc6, 0x3b, 0xf4, 0xe1, + 0x09, 0xa8, 0x6b, 0x9c, 0x7c, 0x4b, 0xcc, 0x4d, 0xc1, 0xda, 0xb4, 0xe5, 0x0a, 0xd9, 0xe9, 0x0a, + 0xd9, 0xef, 0xd3, 0x15, 0xea, 0x97, 0x38, 0xed, 0xcd, 0xf7, 0x96, 0xe1, 0x54, 0x33, 0x2e, 0x8e, + 0xc2, 0x37, 0x00, 0x64, 0x3e, 0xa5, 0x66, 0xf1, 0x49, 0xce, 0xd6, 0x2a, 0xe0, 0x5b, 0xa5, 0x46, + 0x23, 0x29, 0x3d, 0x89, 0xa4, 0xc6, 0xcb, 0x16, 0x9b, 0x06, 0x07, 0xc0, 0xd2, 0x4d, 0xb8, 0xe0, + 0xcb, 0xfc, 0x58, 0x16, 0x33, 0xde, 0x5f, 0xf8, 0x71, 0x51, 0xad, 0x9c, 0xf9, 0xe8, 0x76, 0x80, + 0x3f, 0xdc, 0x8e, 0x53, 0xf0, 0x5f, 0x6e, 0x3b, 0x96, 0xf8, 0x33, 0x79, 0x15, 0x21, 0xaf, 0xad, + 0xad, 0x4b, 0x9e, 0x28, 0xd5, 0x98, 0xfa, 0x27, 0xc1, 0x74, 0x36, 0x61, 0xd4, 0x0d, 0x10, 0x0d, + 0xcc, 0xad, 0xb6, 0xd1, 0xdd, 0x92, 0xfe, 0x71, 0x64, 0xfc, 0x18, 0xd1, 0x00, 0xee, 0x81, 0x12, + 0x8a, 0x63, 0x99, 0x52, 0x15, 0x29, 0x45, 0x14, 0xc7, 0x02, 0x7a, 0x07, 0xfe, 0x15, 0x34, 0x1e, + 0x49, 0xb0, 0x2b, 0xfd, 0xcd, 0x1f, 0x15, 0xfb, 0x79, 0x63, 0xd6, 0xda, 0x46, 0xb7, 0xea, 0x88, + 0xd1, 0x0e, 0x48, 0x82, 0x85, 0xe9, 0x4f, 0x44, 0x9e, 0xee, 0xd3, 0x0b, 0xb0, 0x1b, 0xe1, 0xcf, + 0x2b, 0x64, 0x66, 0x5d, 0xcc, 0xae, 0xf5, 0xd8, 0xec, 0x34, 0x2e, 0x31, 0x39, 0xc3, 0xd9, 0xe6, + 0x14, 0x39, 0x00, 0x76, 0x41, 0x43, 0xf3, 0x6a, 0x42, 0x66, 0x91, 0x6f, 0x36, 0xda, 0x46, 0x77, + 0x43, 0x3e, 0xbf, 0x90, 0xe0, 0xf0, 0x68, 0xff, 0xf4, 0x76, 0x6e, 0x19, 0x77, 0x73, 0xcb, 0xf8, + 0x31, 0xb7, 0x8c, 0x9b, 0x07, 0xab, 0x70, 0xf7, 0x60, 0x15, 0xbe, 0x3d, 0x58, 0x85, 0x0f, 0xaf, + 0xc6, 0x21, 0x0b, 0x66, 0x23, 0xdb, 0x23, 0x53, 0xf1, 0x43, 0x8a, 0xd1, 0x75, 0x4f, 0xea, 0xe1, + 0x27, 0xf9, 0xbf, 0xe8, 0x2d, 0xff, 0x3b, 0x47, 0x9b, 0x22, 0xfe, 0xf2, 0x57, 0x00, 0x00, 0x00, + 0xff, 0xff, 0x4b, 0x9d, 0x59, 0xc2, 0x56, 0x07, 0x00, 0x00, } func (m *ABCIResponses) Marshal() (dAtA []byte, err error) { @@ -638,6 +648,13 @@ func (m *State) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.LastBlockRound != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.LastBlockRound)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x80 + } if m.NextCoreChainLock != nil { { size, err := m.NextCoreChainLock.MarshalToSizedBuffer(dAtA[:i]) @@ -883,6 +900,9 @@ func (m *State) Size() (n int) { l = m.NextCoreChainLock.Size() n += 1 + l + sovTypes(uint64(l)) } + if m.LastBlockRound != 0 { + n += 2 + sovTypes(uint64(m.LastBlockRound)) + } return n } @@ -1764,6 +1784,25 @@ func (m *State) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 16: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LastBlockRound", wireType) + } + m.LastBlockRound = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LastBlockRound |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipTypes(dAtA[iNdEx:]) diff --git a/proto/tendermint/state/types.proto b/proto/tendermint/state/types.proto index 7b87241a36..10dd0a0315 100644 --- a/proto/tendermint/state/types.proto +++ b/proto/tendermint/state/types.proto @@ -69,4 +69,7 @@ message State { bytes app_hash = 13; uint32 last_core_chain_locked_block_height = 14; tendermint.types.CoreChainLock next_core_chain_lock = 15 [(gogoproto.nullable) = true]; + + // last_block_round is the round number at which the last block was committed. + int32 last_block_round = 16; } diff --git a/proto/tendermint/types/params.pb.go b/proto/tendermint/types/params.pb.go index 0ae29224bd..9f0460c1ac 100644 --- a/proto/tendermint/types/params.pb.go +++ b/proto/tendermint/types/params.pb.go @@ -27,6 +27,33 @@ var _ = time.Kitchen // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package +type VersionParams_ConsensusVersion int32 + +const ( + // CONSENSUS_VERSION_0 is the original version of the consensus protocol. + VersionParams_CONSENSUS_VERSION_0 VersionParams_ConsensusVersion = 0 + // CONSENSUS_VERSION_1 changes proposer selection algorithm to not double-propose when previous proposer is offline. + VersionParams_CONSENSUS_VERSION_1 VersionParams_ConsensusVersion = 1 +) + +var VersionParams_ConsensusVersion_name = map[int32]string{ + 0: "CONSENSUS_VERSION_0", + 1: "CONSENSUS_VERSION_1", +} + +var VersionParams_ConsensusVersion_value = map[string]int32{ + "CONSENSUS_VERSION_0": 0, + "CONSENSUS_VERSION_1": 1, +} + +func (x VersionParams_ConsensusVersion) String() string { + return proto.EnumName(VersionParams_ConsensusVersion_name, int32(x)) +} + +func (VersionParams_ConsensusVersion) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_e12598271a686f57, []int{4, 0} +} + // ConsensusParams contains consensus critical parameters that determine the // validity of blocks. type ConsensusParams struct { @@ -300,9 +327,10 @@ func (m *ValidatorParams) GetPubKeyTypes() []string { } // VersionParams contains the ABCI application version. -// DEPRECATED. This will be removed in a future release. type VersionParams struct { AppVersion uint64 `protobuf:"varint,1,opt,name=app_version,json=appVersion,proto3" json:"app_version,omitempty"` + // Version of consensus protocol, used to upgrade consensus without hard forks + ConsensusVersion VersionParams_ConsensusVersion `protobuf:"varint,2,opt,name=consensus_version,json=consensusVersion,proto3,enum=tendermint.types.VersionParams_ConsensusVersion" json:"consensus_version,omitempty"` } func (m *VersionParams) Reset() { *m = VersionParams{} } @@ -345,12 +373,20 @@ func (m *VersionParams) GetAppVersion() uint64 { return 0 } +func (m *VersionParams) GetConsensusVersion() VersionParams_ConsensusVersion { + if m != nil { + return m.ConsensusVersion + } + return VersionParams_CONSENSUS_VERSION_0 +} + // HashedParams is a subset of ConsensusParams. // // It is hashed into the Header.ConsensusHash. type HashedParams struct { - BlockMaxBytes int64 `protobuf:"varint,1,opt,name=block_max_bytes,json=blockMaxBytes,proto3" json:"block_max_bytes,omitempty"` - BlockMaxGas int64 `protobuf:"varint,2,opt,name=block_max_gas,json=blockMaxGas,proto3" json:"block_max_gas,omitempty"` + BlockMaxBytes int64 `protobuf:"varint,1,opt,name=block_max_bytes,json=blockMaxBytes,proto3" json:"block_max_bytes,omitempty"` + BlockMaxGas int64 `protobuf:"varint,2,opt,name=block_max_gas,json=blockMaxGas,proto3" json:"block_max_gas,omitempty"` + ConsensusVersion int32 `protobuf:"varint,3,opt,name=consensus_version,json=consensusVersion,proto3" json:"consensus_version,omitempty"` } func (m *HashedParams) Reset() { *m = HashedParams{} } @@ -400,6 +436,13 @@ func (m *HashedParams) GetBlockMaxGas() int64 { return 0 } +func (m *HashedParams) GetConsensusVersion() int32 { + if m != nil { + return m.ConsensusVersion + } + return 0 +} + // SynchronyParams configure the bounds under which a proposed block's timestamp is considered valid. // These parameters are part of the proposer-based timestamps algorithm. For more information, // see the specification of proposer-based timestamps: @@ -601,6 +644,7 @@ func (m *ABCIParams) GetRecheckTx() bool { } func init() { + proto.RegisterEnum("tendermint.types.VersionParams_ConsensusVersion", VersionParams_ConsensusVersion_name, VersionParams_ConsensusVersion_value) proto.RegisterType((*ConsensusParams)(nil), "tendermint.types.ConsensusParams") proto.RegisterType((*BlockParams)(nil), "tendermint.types.BlockParams") proto.RegisterType((*EvidenceParams)(nil), "tendermint.types.EvidenceParams") @@ -615,53 +659,58 @@ func init() { func init() { proto.RegisterFile("tendermint/types/params.proto", fileDescriptor_e12598271a686f57) } var fileDescriptor_e12598271a686f57 = []byte{ - // 724 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x95, 0x4f, 0x6f, 0xd3, 0x48, - 0x18, 0xc6, 0xe3, 0xc6, 0xcd, 0x9f, 0x37, 0x4d, 0x13, 0x8d, 0x76, 0xb5, 0xde, 0xee, 0xd6, 0xe9, - 0xfa, 0xb0, 0xaa, 0x54, 0xc9, 0xae, 0xb6, 0xbb, 0x87, 0x95, 0xf8, 0xa3, 0xa6, 0x41, 0x40, 0x51, - 0x11, 0x32, 0x15, 0x87, 0x5e, 0xac, 0xb1, 0x33, 0x38, 0x56, 0x63, 0x8f, 0xe5, 0xb1, 0xa3, 0xf8, - 0x5b, 0x70, 0x42, 0x7c, 0x04, 0xb8, 0x20, 0x3e, 0x46, 0x8f, 0x3d, 0x72, 0x02, 0x94, 0x7e, 0x11, - 0x34, 0xe3, 0x71, 0x43, 0x52, 0x0a, 0x39, 0xc5, 0x33, 0xef, 0xf3, 0xf3, 0xe3, 0x79, 0xde, 0xd7, - 0x0e, 0x6c, 0xa7, 0x24, 0x1a, 0x92, 0x24, 0x0c, 0xa2, 0xd4, 0x4a, 0xf3, 0x98, 0x30, 0x2b, 0xc6, - 0x09, 0x0e, 0x99, 0x19, 0x27, 0x34, 0xa5, 0xa8, 0x3b, 0x2f, 0x9b, 0xa2, 0xbc, 0xf5, 0x8b, 0x4f, - 0x7d, 0x2a, 0x8a, 0x16, 0xbf, 0x2a, 0x74, 0x5b, 0xba, 0x4f, 0xa9, 0x3f, 0x26, 0x96, 0x58, 0xb9, - 0xd9, 0x4b, 0x6b, 0x98, 0x25, 0x38, 0x0d, 0x68, 0x54, 0xd4, 0x8d, 0xf7, 0x55, 0xe8, 0x1c, 0xd1, - 0x88, 0x91, 0x88, 0x65, 0xec, 0x99, 0x70, 0x40, 0x07, 0xb0, 0xee, 0x8e, 0xa9, 0x77, 0xae, 0x29, - 0x3b, 0xca, 0x6e, 0xeb, 0x9f, 0x6d, 0x73, 0xd9, 0xcb, 0xec, 0xf3, 0x72, 0xa1, 0xb6, 0x0b, 0x2d, - 0xba, 0x03, 0x0d, 0x32, 0x09, 0x86, 0x24, 0xf2, 0x88, 0xb6, 0x26, 0xb8, 0x9d, 0x9b, 0xdc, 0x03, - 0xa9, 0x90, 0xe8, 0x35, 0x81, 0xee, 0x43, 0x73, 0x82, 0xc7, 0xc1, 0x10, 0xa7, 0x34, 0xd1, 0xaa, - 0x02, 0xff, 0xeb, 0x26, 0xfe, 0xa2, 0x94, 0x48, 0x7e, 0xce, 0xa0, 0xff, 0xa1, 0x3e, 0x21, 0x09, - 0x0b, 0x68, 0xa4, 0xa9, 0x02, 0xef, 0x7d, 0x07, 0x2f, 0x04, 0x12, 0x2e, 0xf5, 0xdc, 0x9b, 0xe5, - 0x91, 0x37, 0x4a, 0x68, 0x94, 0x6b, 0xeb, 0xb7, 0x79, 0x3f, 0x2f, 0x25, 0xa5, 0xf7, 0x35, 0xc3, - 0xbd, 0xd3, 0x20, 0x24, 0x34, 0x4b, 0xb5, 0xda, 0x6d, 0xde, 0xa7, 0x85, 0xa0, 0xf4, 0x96, 0x7a, - 0xb4, 0x0f, 0x2a, 0x76, 0xbd, 0x40, 0xab, 0x0b, 0xee, 0xcf, 0x9b, 0xdc, 0x61, 0xff, 0xe8, 0xb1, - 0x84, 0x84, 0xd2, 0x38, 0x82, 0xd6, 0x37, 0xe9, 0xa3, 0x3f, 0xa0, 0x19, 0xe2, 0xa9, 0xe3, 0xe6, - 0x29, 0x61, 0xa2, 0x5f, 0x55, 0xbb, 0x11, 0xe2, 0x69, 0x9f, 0xaf, 0xd1, 0x6f, 0x50, 0xe7, 0x45, - 0x1f, 0x33, 0xd1, 0x92, 0xaa, 0x5d, 0x0b, 0xf1, 0xf4, 0x21, 0x66, 0xc6, 0x3b, 0x05, 0x36, 0x17, - 0x7b, 0x81, 0xf6, 0x00, 0x71, 0x2d, 0xf6, 0x89, 0x13, 0x65, 0xa1, 0x23, 0x9a, 0x5a, 0xde, 0xb1, - 0x13, 0xe2, 0xe9, 0xa1, 0x4f, 0x9e, 0x66, 0xa1, 0xb0, 0x66, 0xe8, 0x04, 0xba, 0xa5, 0xb8, 0x9c, - 0x27, 0xd9, 0xf4, 0xdf, 0xcd, 0x62, 0xe0, 0xcc, 0x72, 0xe0, 0xcc, 0x81, 0x14, 0xf4, 0x1b, 0x17, - 0x9f, 0x7a, 0x95, 0x37, 0x9f, 0x7b, 0x8a, 0xbd, 0x59, 0xdc, 0xaf, 0xac, 0x2c, 0x1e, 0xa2, 0xba, - 0x78, 0x08, 0xe3, 0x3f, 0xe8, 0x2c, 0xf5, 0x1d, 0x19, 0xd0, 0x8e, 0x33, 0xd7, 0x39, 0x27, 0xb9, - 0x23, 0x52, 0xd2, 0x94, 0x9d, 0xea, 0x6e, 0xd3, 0x6e, 0xc5, 0x99, 0xfb, 0x84, 0xe4, 0xa7, 0x7c, - 0xcb, 0xd8, 0x87, 0xf6, 0x42, 0xbf, 0x51, 0x0f, 0x5a, 0x38, 0x8e, 0x9d, 0x72, 0x4a, 0xf8, 0xc9, - 0x54, 0x1b, 0x70, 0x1c, 0x4b, 0x99, 0x71, 0x06, 0x1b, 0x8f, 0x30, 0x1b, 0x91, 0xa1, 0x04, 0xfe, - 0x86, 0x8e, 0x48, 0xc1, 0x59, 0x0e, 0xb8, 0x2d, 0xb6, 0x4f, 0xca, 0x94, 0x0d, 0x68, 0xcf, 0x75, - 0xf3, 0xac, 0x5b, 0xa5, 0x8a, 0x07, 0xfe, 0x5a, 0x81, 0xce, 0xd2, 0x04, 0xa1, 0x01, 0xb4, 0x43, - 0xc2, 0x98, 0x08, 0x91, 0x8c, 0x71, 0x2e, 0x5f, 0xb7, 0x1f, 0x24, 0xa8, 0x8a, 0xf4, 0x36, 0x24, - 0x35, 0xe0, 0x10, 0xba, 0x0b, 0xcd, 0x38, 0x21, 0x5e, 0xc0, 0x56, 0xea, 0x41, 0x71, 0x87, 0x39, - 0x61, 0x7c, 0x58, 0x83, 0xf6, 0xc2, 0x6c, 0xf2, 0x69, 0x8e, 0x13, 0x1a, 0x53, 0x46, 0x56, 0x7d, - 0xa0, 0x52, 0xcf, 0x4f, 0x24, 0x2f, 0xf9, 0x89, 0x52, 0xbc, 0xea, 0xf3, 0x6c, 0x48, 0x6a, 0xc0, - 0x21, 0x74, 0x00, 0xea, 0x84, 0xa6, 0x44, 0x7e, 0x06, 0x7e, 0x0a, 0x0b, 0x31, 0xba, 0x07, 0xc0, - 0x7f, 0xa5, 0xaf, 0xba, 0x62, 0x0e, 0x1c, 0x11, 0xa6, 0xc7, 0x6a, 0x63, 0xbd, 0x5b, 0x3b, 0x56, - 0x1b, 0xb5, 0x6e, 0xdd, 0xae, 0x79, 0x34, 0x0c, 0x83, 0xd4, 0xfe, 0xd5, 0xcd, 0x63, 0xcc, 0x98, - 0x53, 0x2c, 0x1d, 0xf9, 0xce, 0x1a, 0x7b, 0x00, 0xf3, 0xb7, 0x12, 0x6d, 0x03, 0x24, 0xc4, 0x1b, - 0x11, 0xef, 0xdc, 0x49, 0xa7, 0x22, 0xb1, 0x86, 0xdd, 0x94, 0x3b, 0xa7, 0xd3, 0xbe, 0xfd, 0x76, - 0xa6, 0x2b, 0x17, 0x33, 0x5d, 0xb9, 0x9c, 0xe9, 0xca, 0x97, 0x99, 0xae, 0xbc, 0xba, 0xd2, 0x2b, - 0x97, 0x57, 0x7a, 0xe5, 0xe3, 0x95, 0x5e, 0x39, 0xfb, 0xd7, 0x0f, 0xd2, 0x51, 0xe6, 0x9a, 0x1e, - 0x0d, 0xad, 0x21, 0x66, 0xa3, 0x18, 0xe7, 0x56, 0xf1, 0x09, 0xe0, 0xab, 0xe2, 0xa3, 0x6d, 0x2d, - 0xff, 0x11, 0xb8, 0x35, 0xb1, 0x7f, 0xf0, 0x35, 0x00, 0x00, 0xff, 0xff, 0x17, 0x03, 0x59, 0x60, - 0x23, 0x06, 0x00, 0x00, + // 801 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0x4f, 0x8f, 0xdb, 0x44, + 0x14, 0x8f, 0x1b, 0x6f, 0x36, 0x79, 0xd9, 0x6c, 0xcc, 0x00, 0xaa, 0x29, 0xac, 0xb3, 0xf8, 0x80, + 0x2a, 0x55, 0xb2, 0x97, 0x2e, 0x1c, 0x90, 0xf8, 0xa3, 0x26, 0x59, 0x95, 0x16, 0x35, 0x45, 0xce, + 0xd2, 0x03, 0x12, 0xb2, 0xc6, 0xce, 0xe0, 0x58, 0x1b, 0x7b, 0x2c, 0x8f, 0x1d, 0xc5, 0x9f, 0x80, + 0x2b, 0x27, 0xc4, 0x47, 0x80, 0x0b, 0xe2, 0x63, 0xf4, 0xd8, 0x0b, 0x12, 0x27, 0x40, 0xd9, 0x2f, + 0x82, 0x66, 0x3c, 0x4e, 0x48, 0x36, 0x65, 0x73, 0xb2, 0xe7, 0xbd, 0xdf, 0x6f, 0x7e, 0x7e, 0xf3, + 0x7e, 0x6f, 0x0c, 0x27, 0x19, 0x89, 0x27, 0x24, 0x8d, 0xc2, 0x38, 0xb3, 0xb3, 0x22, 0x21, 0xcc, + 0x4e, 0x70, 0x8a, 0x23, 0x66, 0x25, 0x29, 0xcd, 0x28, 0xd2, 0xd6, 0x69, 0x4b, 0xa4, 0xef, 0xbd, + 0x15, 0xd0, 0x80, 0x8a, 0xa4, 0xcd, 0xdf, 0x4a, 0xdc, 0x3d, 0x23, 0xa0, 0x34, 0x98, 0x11, 0x5b, + 0xac, 0xbc, 0xfc, 0x7b, 0x7b, 0x92, 0xa7, 0x38, 0x0b, 0x69, 0x5c, 0xe6, 0xcd, 0xdf, 0xea, 0xd0, + 0x1d, 0xd0, 0x98, 0x91, 0x98, 0xe5, 0xec, 0x6b, 0xa1, 0x80, 0xce, 0xe1, 0xc0, 0x9b, 0x51, 0xff, + 0x4a, 0x57, 0x4e, 0x95, 0xfb, 0xed, 0x87, 0x27, 0xd6, 0xb6, 0x96, 0xd5, 0xe7, 0xe9, 0x12, 0xed, + 0x94, 0x58, 0xf4, 0x29, 0x34, 0xc9, 0x3c, 0x9c, 0x90, 0xd8, 0x27, 0xfa, 0x1d, 0xc1, 0x3b, 0xbd, + 0xc9, 0xbb, 0x90, 0x08, 0x49, 0x5d, 0x31, 0xd0, 0x17, 0xd0, 0x9a, 0xe3, 0x59, 0x38, 0xc1, 0x19, + 0x4d, 0xf5, 0xba, 0xa0, 0xbf, 0x7f, 0x93, 0xfe, 0xa2, 0x82, 0x48, 0xfe, 0x9a, 0x83, 0x3e, 0x81, + 0xc3, 0x39, 0x49, 0x59, 0x48, 0x63, 0x5d, 0x15, 0xf4, 0xde, 0x0e, 0x7a, 0x09, 0x90, 0xe4, 0x0a, + 0xcf, 0xb5, 0x59, 0x11, 0xfb, 0xd3, 0x94, 0xc6, 0x85, 0x7e, 0xf0, 0x3a, 0xed, 0x71, 0x05, 0xa9, + 0xb4, 0x57, 0x1c, 0xae, 0x9d, 0x85, 0x11, 0xa1, 0x79, 0xa6, 0x37, 0x5e, 0xa7, 0x7d, 0x59, 0x02, + 0x2a, 0x6d, 0x89, 0x47, 0x67, 0xa0, 0x62, 0xcf, 0x0f, 0xf5, 0x43, 0xc1, 0x7b, 0xef, 0x26, 0xef, + 0x51, 0x7f, 0xf0, 0x44, 0x92, 0x04, 0xd2, 0x1c, 0x40, 0xfb, 0x3f, 0xa7, 0x8f, 0xde, 0x85, 0x56, + 0x84, 0x17, 0xae, 0x57, 0x64, 0x84, 0x89, 0x7e, 0xd5, 0x9d, 0x66, 0x84, 0x17, 0x7d, 0xbe, 0x46, + 0x77, 0xe1, 0x90, 0x27, 0x03, 0xcc, 0x44, 0x4b, 0xea, 0x4e, 0x23, 0xc2, 0x8b, 0xc7, 0x98, 0x99, + 0xbf, 0x2a, 0x70, 0xbc, 0xd9, 0x0b, 0xf4, 0x00, 0x10, 0xc7, 0xe2, 0x80, 0xb8, 0x71, 0x1e, 0xb9, + 0xa2, 0xa9, 0xd5, 0x8e, 0xdd, 0x08, 0x2f, 0x1e, 0x05, 0x64, 0x94, 0x47, 0x42, 0x9a, 0xa1, 0x67, + 0xa0, 0x55, 0xe0, 0xca, 0x4f, 0xb2, 0xe9, 0xef, 0x58, 0xa5, 0xe1, 0xac, 0xca, 0x70, 0xd6, 0x50, + 0x02, 0xfa, 0xcd, 0x97, 0x7f, 0xf5, 0x6a, 0x3f, 0xff, 0xdd, 0x53, 0x9c, 0xe3, 0x72, 0xbf, 0x2a, + 0xb3, 0x59, 0x44, 0x7d, 0xb3, 0x08, 0xf3, 0x63, 0xe8, 0x6e, 0xf5, 0x1d, 0x99, 0xd0, 0x49, 0x72, + 0xcf, 0xbd, 0x22, 0x85, 0x2b, 0x4e, 0x49, 0x57, 0x4e, 0xeb, 0xf7, 0x5b, 0x4e, 0x3b, 0xc9, 0xbd, + 0xaf, 0x48, 0x71, 0xc9, 0x43, 0xe6, 0x1f, 0x0a, 0x74, 0x36, 0x1a, 0x8e, 0x7a, 0xd0, 0xc6, 0x49, + 0xe2, 0x56, 0x36, 0xe1, 0xa5, 0xa9, 0x0e, 0xe0, 0x24, 0x91, 0x30, 0xf4, 0x1d, 0xbc, 0xe1, 0x57, + 0xa3, 0xb0, 0x82, 0xf1, 0xb2, 0x8e, 0x1f, 0x9e, 0xdd, 0xe2, 0x26, 0x6b, 0x35, 0x43, 0x32, 0xec, + 0x68, 0xfe, 0x56, 0xc4, 0x1c, 0x82, 0xb6, 0x8d, 0x42, 0x77, 0xe1, 0xcd, 0xc1, 0xf3, 0xd1, 0xf8, + 0x62, 0x34, 0xfe, 0x66, 0xec, 0xbe, 0xb8, 0x70, 0xc6, 0x4f, 0x9e, 0x8f, 0xdc, 0x33, 0xad, 0xb6, + 0x3b, 0xf1, 0xa1, 0xa6, 0x98, 0x3f, 0x28, 0x70, 0xf4, 0x25, 0x66, 0x53, 0x32, 0x91, 0x65, 0x7d, + 0x00, 0x5d, 0xd1, 0x2c, 0x77, 0xdb, 0x07, 0x1d, 0x11, 0x7e, 0x56, 0x99, 0xc1, 0x84, 0xce, 0x1a, + 0xb7, 0xb6, 0x44, 0xbb, 0x42, 0x3d, 0xc6, 0xdc, 0x04, 0x3b, 0x4e, 0x80, 0x37, 0xe4, 0x60, 0x47, + 0x3d, 0x3f, 0x29, 0xd0, 0xdd, 0x9a, 0x0a, 0x34, 0x84, 0x4e, 0x44, 0x18, 0x13, 0xc6, 0x20, 0x33, + 0x5c, 0xc8, 0x2b, 0xe4, 0x7f, 0x5c, 0xa1, 0x0a, 0x47, 0x1c, 0x49, 0xd6, 0x90, 0x93, 0xd0, 0x67, + 0xd0, 0x4a, 0x52, 0xe2, 0x87, 0x6c, 0x2f, 0x5f, 0x95, 0x3b, 0xac, 0x19, 0xe6, 0xef, 0x77, 0xa0, + 0xb3, 0x31, 0x6f, 0x7c, 0x42, 0x93, 0x94, 0x26, 0x94, 0x91, 0x7d, 0x3f, 0xa8, 0xc2, 0xf3, 0x8a, + 0xe4, 0x2b, 0xaf, 0x28, 0xc3, 0xfb, 0x7e, 0xcf, 0x91, 0x64, 0x0d, 0x39, 0x09, 0x9d, 0x83, 0x3a, + 0xa7, 0x19, 0x91, 0x57, 0xdb, 0xad, 0x64, 0x01, 0x46, 0x9f, 0x03, 0xf0, 0xa7, 0xd4, 0x55, 0xf7, + 0x3c, 0x07, 0x4e, 0x11, 0xa2, 0x4f, 0xd5, 0xe6, 0x81, 0xd6, 0x78, 0xaa, 0x36, 0x1b, 0xda, 0xa1, + 0xd3, 0xf0, 0x69, 0x14, 0x85, 0x99, 0xf3, 0xb6, 0x57, 0x24, 0x98, 0x31, 0xb7, 0x5c, 0xba, 0xf2, + 0x1e, 0x32, 0x1f, 0x00, 0xac, 0x6f, 0x1a, 0x74, 0x02, 0x90, 0x12, 0x7f, 0x4a, 0xfc, 0x2b, 0x37, + 0x5b, 0x88, 0x13, 0x6b, 0x3a, 0x2d, 0x19, 0xb9, 0x5c, 0xf4, 0x9d, 0x5f, 0x96, 0x86, 0xf2, 0x72, + 0x69, 0x28, 0xaf, 0x96, 0x86, 0xf2, 0xcf, 0xd2, 0x50, 0x7e, 0xbc, 0x36, 0x6a, 0xaf, 0xae, 0x8d, + 0xda, 0x9f, 0xd7, 0x46, 0xed, 0xdb, 0x8f, 0x82, 0x30, 0x9b, 0xe6, 0x9e, 0xe5, 0xd3, 0xc8, 0x9e, + 0x60, 0x36, 0x4d, 0x70, 0x61, 0x97, 0xc3, 0xc3, 0x57, 0xe5, 0x8f, 0xc8, 0xde, 0xfe, 0xb9, 0x79, + 0x0d, 0x11, 0x3f, 0xff, 0x37, 0x00, 0x00, 0xff, 0xff, 0xe5, 0x7d, 0x36, 0x05, 0xf7, 0x06, 0x00, + 0x00, } func (this *ConsensusParams) Equal(that interface{}) bool { @@ -814,6 +863,9 @@ func (this *VersionParams) Equal(that interface{}) bool { if this.AppVersion != that1.AppVersion { return false } + if this.ConsensusVersion != that1.ConsensusVersion { + return false + } return true } func (this *HashedParams) Equal(that interface{}) bool { @@ -841,6 +893,9 @@ func (this *HashedParams) Equal(that interface{}) bool { if this.BlockMaxGas != that1.BlockMaxGas { return false } + if this.ConsensusVersion != that1.ConsensusVersion { + return false + } return true } func (this *SynchronyParams) Equal(that interface{}) bool { @@ -1196,6 +1251,11 @@ func (m *VersionParams) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.ConsensusVersion != 0 { + i = encodeVarintParams(dAtA, i, uint64(m.ConsensusVersion)) + i-- + dAtA[i] = 0x10 + } if m.AppVersion != 0 { i = encodeVarintParams(dAtA, i, uint64(m.AppVersion)) i-- @@ -1224,6 +1284,11 @@ func (m *HashedParams) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.ConsensusVersion != 0 { + i = encodeVarintParams(dAtA, i, uint64(m.ConsensusVersion)) + i-- + dAtA[i] = 0x18 + } if m.BlockMaxGas != 0 { i = encodeVarintParams(dAtA, i, uint64(m.BlockMaxGas)) i-- @@ -1480,6 +1545,9 @@ func (m *VersionParams) Size() (n int) { if m.AppVersion != 0 { n += 1 + sovParams(uint64(m.AppVersion)) } + if m.ConsensusVersion != 0 { + n += 1 + sovParams(uint64(m.ConsensusVersion)) + } return n } @@ -1495,6 +1563,9 @@ func (m *HashedParams) Size() (n int) { if m.BlockMaxGas != 0 { n += 1 + sovParams(uint64(m.BlockMaxGas)) } + if m.ConsensusVersion != 0 { + n += 1 + sovParams(uint64(m.ConsensusVersion)) + } return n } @@ -2199,6 +2270,25 @@ func (m *VersionParams) Unmarshal(dAtA []byte) error { break } } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ConsensusVersion", wireType) + } + m.ConsensusVersion = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowParams + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ConsensusVersion |= VersionParams_ConsensusVersion(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipParams(dAtA[iNdEx:]) @@ -2287,6 +2377,25 @@ func (m *HashedParams) Unmarshal(dAtA []byte) error { break } } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ConsensusVersion", wireType) + } + m.ConsensusVersion = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowParams + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ConsensusVersion |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipParams(dAtA[iNdEx:]) diff --git a/proto/tendermint/types/params.proto b/proto/tendermint/types/params.proto index e69beb9364..3c4a7feec0 100644 --- a/proto/tendermint/types/params.proto +++ b/proto/tendermint/types/params.proto @@ -61,17 +61,26 @@ message ValidatorParams { } // VersionParams contains the ABCI application version. -// DEPRECATED. This will be removed in a future release. message VersionParams { uint64 app_version = 1; + // Version of consensus protocol, used to upgrade consensus without hard forks + ConsensusVersion consensus_version = 2; + + enum ConsensusVersion { + // CONSENSUS_VERSION_0 is the original version of the consensus protocol. + CONSENSUS_VERSION_0 = 0; + // CONSENSUS_VERSION_1 changes proposer selection algorithm to not double-propose when previous proposer is offline. + CONSENSUS_VERSION_1 = 1; + } } // HashedParams is a subset of ConsensusParams. // // It is hashed into the Header.ConsensusHash. message HashedParams { - int64 block_max_bytes = 1; - int64 block_max_gas = 2; + int64 block_max_bytes = 1; + int64 block_max_gas = 2; + int32 consensus_version = 3; } // SynchronyParams configure the bounds under which a proposed block's timestamp is considered valid. diff --git a/test/e2e/networks/rotate.toml b/test/e2e/networks/rotate.toml index 736e56765c..14190da099 100644 --- a/test/e2e/networks/rotate.toml +++ b/test/e2e/networks/rotate.toml @@ -50,6 +50,36 @@ validator04 = 100 validator05 = 100 validator09 = 100 +[validator_update.1060] +validator01 = 100 +validator02 = 100 +validator03 = 100 +validator04 = 100 +validator05 = 100 + +[validator_update.1061] +validator01 = 100 +validator07 = 100 +validator08 = 100 +validator09 = 100 +validator10 = 100 + + +[validator_update.1062] +validator01 = 100 +validator02 = 100 +validator03 = 100 +validator04 = 100 +validator05 = 100 + + +[validator_update.1063] +validator01 = 100 +validator03 = 100 +validator04 = 100 +validator05 = 100 +validator11 = 100 + [node.seed01] mode = "seed" perturb = ["restart"] diff --git a/test/e2e/runner/evidence.go b/test/e2e/runner/evidence.go index 84192cf2a1..0901990194 100644 --- a/test/e2e/runner/evidence.go +++ b/test/e2e/runner/evidence.go @@ -68,9 +68,15 @@ func InjectEvidence(ctx context.Context, logger log.Logger, r *rand.Rand, testne return errors.New("quorum hash must be returned when requested") } - valSet, err := types.ValidatorSetFromExistingValidators(valRes.Validators, *valRes.ThresholdPublicKey, valRes.QuorumType, *valRes.QuorumHash) - if err != nil { - return err + valSet := types.NewValidatorSet(valRes.Validators, *valRes.ThresholdPublicKey, + valRes.QuorumType, + *valRes.QuorumHash, + false) + if valSet == nil { + return fmt.Errorf("could not create validator set from response") + } + if err = valSet.SetProposer(blockRes.Block.ProposerProTxHash); err != nil { + return fmt.Errorf("could not set proposer: %w", err) } // get the private keys of all the validators in the network diff --git a/test/e2e/tests/validator_test.go b/test/e2e/tests/validator_test.go index 332f3e5a31..07e0fda853 100644 --- a/test/e2e/tests/validator_test.go +++ b/test/e2e/tests/validator_test.go @@ -10,6 +10,8 @@ import ( "github.com/dashpay/tenderdash/crypto" cryptoenc "github.com/dashpay/tenderdash/crypto/encoding" + selectproposer "github.com/dashpay/tenderdash/internal/consensus/versioned/selectproposer" + "github.com/dashpay/tenderdash/internal/libs/test" e2e "github.com/dashpay/tenderdash/test/e2e/pkg" "github.com/dashpay/tenderdash/types" ) @@ -60,7 +62,7 @@ func TestValidator_Sets(t *testing.T) { } // fmt.Printf("node %s(%X) validator set for height %d is %v\n", // node.Name, node.ProTxHash, h, valSchedule.Set) - for i, valScheduleValidator := range valSchedule.Set.Validators { + for i, valScheduleValidator := range valSchedule.Set.ValidatorSet().Validators { validator := validators[i] require.Equal(t, valScheduleValidator.ProTxHash, validator.ProTxHash, "mismatching validator proTxHashes at height %v (%X <=> %X", h, @@ -72,9 +74,9 @@ func TestValidator_Sets(t *testing.T) { // Validators in the schedule don't contain addresses validator.NodeAddress = types.ValidatorAddress{} } - require.Equal(t, valSchedule.Set.Validators, validators, + require.Equal(t, valSchedule.Set.ValidatorSet().Validators, validators, "incorrect validator set at height %v", h) - require.Equal(t, valSchedule.Set.ThresholdPublicKey, thresholdPublicKey, + require.Equal(t, valSchedule.Set.ValidatorSet().ThresholdPublicKey, thresholdPublicKey, "incorrect thresholdPublicKey at height %v", h) require.NoError(t, valSchedule.Increment(1)) } @@ -97,10 +99,16 @@ func TestValidator_Propose(t *testing.T) { expectCount := 0 proposeCount := 0 + var prevBlock *types.Block for _, block := range blocks { - if bytes.Equal(valSchedule.Set.Proposer.ProTxHash, proTxHash) { + if prevBlock == nil { + prevBlock = block + continue + } + + if bytes.Equal(valSchedule.Set.MustGetProposer(prevBlock.Height, block.LastCommit.Round).ProTxHash, proTxHash) { expectCount++ - if bytes.Equal(block.ProposerProTxHash, proTxHash) { + if bytes.Equal(prevBlock.ProposerProTxHash, proTxHash) { proposeCount++ } } @@ -121,7 +129,7 @@ func TestValidator_Propose(t *testing.T) { // validatorSchedule is a validator set iterator, which takes into account // validator set updates. type validatorSchedule struct { - Set *types.ValidatorSet + Set selectproposer.ProposerSelector height int64 updates map[int64]e2e.ValidatorsMap thresholdPublicKeyUpdates map[int64]crypto.PubKey @@ -147,10 +155,12 @@ func newValidatorSchedule(testnet e2e.Testnet) *validatorSchedule { panic("quorum hash key must be set for height 0 if validator changes") } } - + valset := types.NewValidatorSet(makeVals(valMap), thresholdPublicKey, quorumType, quorumHash, true) + vs := test.Must(selectproposer.NewProposerSelector(types.ConsensusParams{}, valset, + testnet.InitialHeight, 0, nil, nil)) return &validatorSchedule{ height: testnet.InitialHeight, - Set: types.NewValidatorSet(makeVals(valMap), thresholdPublicKey, quorumType, quorumHash, true), + Set: vs, updates: testnet.ValidatorUpdates, thresholdPublicKeyUpdates: testnet.ThresholdPublicKeyUpdates, quorumHashUpdates: testnet.QuorumHashUpdates, @@ -166,19 +176,24 @@ func (s *validatorSchedule) Increment(heights int64) error { if update, ok := s.updates[s.height-1]; ok { if thresholdPublicKeyUpdate, ok := s.thresholdPublicKeyUpdates[s.height-1]; ok { if quorumHashUpdate, ok := s.quorumHashUpdates[s.height-1]; ok { - if bytes.Equal(quorumHashUpdate, s.Set.QuorumHash) { - if err := s.Set.UpdateWithChangeSet(makeVals(update), thresholdPublicKeyUpdate, quorumHashUpdate); err != nil { + if bytes.Equal(quorumHashUpdate, s.Set.ValidatorSet().QuorumHash) { + if err := s.Set.ValidatorSet().UpdateWithChangeSet(makeVals(update), thresholdPublicKeyUpdate, quorumHashUpdate); err != nil { return err } } else { - s.Set = types.NewValidatorSet(makeVals(update), thresholdPublicKeyUpdate, btcjson.LLMQType_5_60, + + vset := types.NewValidatorSet(makeVals(update), thresholdPublicKeyUpdate, btcjson.LLMQType_5_60, quorumHashUpdate, true) + s.Set = test.Must(selectproposer.NewProposerSelector(types.ConsensusParams{}, vset, + s.height, 0, nil, nil)) } } } } } - s.Set.IncrementProposerPriority(1) + if err := s.Set.UpdateHeightRound(s.height, 0); err != nil { + return err + } } return nil } diff --git a/types/block_test.go b/types/block_test.go index 2803f8f1cc..0b6139e585 100644 --- a/types/block_test.go +++ b/types/block_test.go @@ -87,10 +87,9 @@ func TestBlockValidateBasic(t *testing.T) { malleateBlock func(*Block) expErr bool }{ - {"Make Block", func(blk *Block) {}, false}, + {"Make Block", func(_blk *Block) {}, false}, {"Make Block w/ proposer pro_tx_hash", func(blk *Block) { - blk.ProposerProTxHash = - valSet.GetProposer().ProTxHash + blk.ProposerProTxHash = valSet.Proposer().ProTxHash }, false}, {"Negative Height", func(blk *Block) { blk.Height = -1 }, true}, {"Modify the last Commit", func(blk *Block) { @@ -131,7 +130,7 @@ func TestBlockValidateBasic(t *testing.T) { j := i t.Run(tcRun.testName, func(t *testing.T) { block := MakeBlock(h, txs, commit, evList) - block.ProposerProTxHash = valSet.GetProposer().ProTxHash + block.ProposerProTxHash = valSet.Proposer().ProTxHash tcRun.malleateBlock(block) err = block.ValidateBasic() assert.Equal(t, tcRun.expErr, err != nil, "#%d: %v", j, err) @@ -322,7 +321,7 @@ func TestCommitValidateBasic(t *testing.T) { malleateCommit func(*Commit) expectErr bool }{ - {"Random Commit", func(com *Commit) {}, false}, + {"Random Commit", func(_com *Commit) {}, false}, {"Incorrect block signature", func(com *Commit) { com.ThresholdBlockSignature = []byte{0} }, true}, {"Incorrect height", func(com *Commit) { com.Height = int64(-100) }, true}, {"Incorrect round", func(com *Commit) { com.Round = -100 }, true}, diff --git a/types/evidence_test.go b/types/evidence_test.go index edee9d7248..19df99c4fe 100644 --- a/types/evidence_test.go +++ b/types/evidence_test.go @@ -136,7 +136,7 @@ func TestDuplicateVoteEvidenceValidation(t *testing.T) { malleateEvidence func(*DuplicateVoteEvidence) expectErr bool }{ - {"Good DuplicateVoteEvidence", func(ev *DuplicateVoteEvidence) {}, false}, + {"Good DuplicateVoteEvidence", func(_ *DuplicateVoteEvidence) {}, false}, {"Nil vote A", func(ev *DuplicateVoteEvidence) { ev.VoteA = nil }, true}, {"Nil vote B", func(ev *DuplicateVoteEvidence) { ev.VoteB = nil }, true}, {"Nil votes", func(ev *DuplicateVoteEvidence) { diff --git a/types/light_test.go b/types/light_test.go index bcb1bad17f..bba922c2bd 100644 --- a/types/light_test.go +++ b/types/light_test.go @@ -26,7 +26,7 @@ func TestLightBlockValidateBasic(t *testing.T) { header.Version.Block = version.BlockProtocol vals2, _ := RandValidatorSet(3) vals3 := vals.Copy() - vals3.Proposer = &Validator{} + vals3.QuorumHash = []byte("invalid") commit.BlockID.Hash = header.Hash() sh := &SignedHeader{ @@ -94,8 +94,6 @@ func TestLightBlockProtobuf(t *testing.T) { header.LastBlockID = commit.BlockID header.Version.Block = version.BlockProtocol header.ValidatorsHash = vals.Hash() - vals3 := vals.Copy() - vals3.Proposer = &Validator{} commit.BlockID.Hash = header.Hash() commit.QuorumHash = vals.QuorumHash diff --git a/types/params.go b/types/params.go index 3e2b7535bc..fa7243825f 100644 --- a/types/params.go +++ b/types/params.go @@ -73,7 +73,8 @@ type ValidatorParams struct { } type VersionParams struct { - AppVersion uint64 `json:"app_version,string"` + AppVersion uint64 `json:"app_version,string"` + ConsensusVersion int32 `json:"consensus,string"` } // SynchronyParams influence the validity of block timestamps. @@ -146,7 +147,8 @@ func DefaultValidatorParams() ValidatorParams { func DefaultVersionParams() VersionParams { return VersionParams{ - AppVersion: 0, + AppVersion: 0, + ConsensusVersion: 0, } } @@ -345,8 +347,9 @@ func (params ConsensusParams) ValidateConsensusParams() error { // TODO: We should hash the other parameters as well func (params ConsensusParams) HashConsensusParams() tmbytes.HexBytes { hp := tmproto.HashedParams{ - BlockMaxBytes: params.Block.MaxBytes, - BlockMaxGas: params.Block.MaxGas, + BlockMaxBytes: params.Block.MaxBytes, + BlockMaxGas: params.Block.MaxGas, + ConsensusVersion: params.Version.ConsensusVersion, } bz, err := hp.Marshal() @@ -395,6 +398,7 @@ func (params ConsensusParams) UpdateConsensusParams(params2 *tmproto.ConsensusPa } if params2.Version != nil { res.Version.AppVersion = params2.Version.AppVersion + res.Version.ConsensusVersion = int32(params2.Version.ConsensusVersion) } if params2.Synchrony != nil { if params2.Synchrony.MessageDelay != nil { @@ -439,7 +443,8 @@ func (params *ConsensusParams) ToProto() tmproto.ConsensusParams { PubKeyTypes: params.Validator.PubKeyTypes, }, Version: &tmproto.VersionParams{ - AppVersion: params.Version.AppVersion, + AppVersion: params.Version.AppVersion, + ConsensusVersion: tmproto.VersionParams_ConsensusVersion(params.Version.ConsensusVersion), }, Synchrony: &tmproto.SynchronyParams{ MessageDelay: ¶ms.Synchrony.MessageDelay, @@ -457,24 +462,38 @@ func (params *ConsensusParams) ToProto() tmproto.ConsensusParams { } } +// ConsensusParamsFromProto returns a ConsensusParams from a protobuf representation. func ConsensusParamsFromProto(pbParams tmproto.ConsensusParams) ConsensusParams { - c := ConsensusParams{ - Block: BlockParams{ + + c := ConsensusParams{} + if pbParams.Block != nil { + c.Block = BlockParams{ MaxBytes: pbParams.Block.MaxBytes, MaxGas: pbParams.Block.MaxGas, - }, - Evidence: EvidenceParams{ + } + } + + if pbParams.Evidence != nil { + c.Evidence = EvidenceParams{ MaxAgeNumBlocks: pbParams.Evidence.MaxAgeNumBlocks, MaxAgeDuration: pbParams.Evidence.MaxAgeDuration, MaxBytes: pbParams.Evidence.MaxBytes, - }, - Validator: ValidatorParams{ + } + } + + if pbParams.Validator != nil { + c.Validator = ValidatorParams{ PubKeyTypes: pbParams.Validator.PubKeyTypes, - }, - Version: VersionParams{ - AppVersion: pbParams.Version.AppVersion, - }, + } + } + + if pbParams.Version != nil { + c.Version = VersionParams{ + AppVersion: pbParams.Version.AppVersion, + ConsensusVersion: int32(pbParams.Version.ConsensusVersion), + } } + if pbParams.Synchrony != nil { if pbParams.Synchrony.MessageDelay != nil { c.Synchrony.MessageDelay = *pbParams.Synchrony.GetMessageDelay() @@ -483,6 +502,7 @@ func ConsensusParamsFromProto(pbParams tmproto.ConsensusParams) ConsensusParams c.Synchrony.Precision = *pbParams.Synchrony.GetPrecision() } } + if pbParams.Timeout != nil { if pbParams.Timeout.Propose != nil { c.Timeout.Propose = *pbParams.Timeout.GetPropose() @@ -497,8 +517,10 @@ func ConsensusParamsFromProto(pbParams tmproto.ConsensusParams) ConsensusParams c.Timeout.VoteDelta = *pbParams.Timeout.GetVoteDelta() } } + if pbParams.Abci != nil { c.ABCI.RecheckTx = pbParams.Abci.GetRecheckTx() } + return c } diff --git a/types/params_test.go b/types/params_test.go index a5abcfa6a4..ccd18e8e71 100644 --- a/types/params_test.go +++ b/types/params_test.go @@ -186,6 +186,8 @@ type makeParamsArgs struct { proposeDelta *time.Duration vote *time.Duration voteDelta *time.Duration + + consensusVersion int32 } func makeParams(args makeParamsArgs) ConsensusParams { @@ -205,7 +207,7 @@ func makeParams(args makeParamsArgs) ConsensusParams { args.voteDelta = durationPtr(1) } - return ConsensusParams{ + cp := ConsensusParams{ Block: BlockParams{ MaxBytes: args.blockBytes, MaxGas: args.blockGas, @@ -232,6 +234,15 @@ func makeParams(args makeParamsArgs) ConsensusParams { RecheckTx: args.recheck, }, } + + if args.consensusVersion != 0 { + cp.Version = VersionParams{ + AppVersion: 1, + ConsensusVersion: args.consensusVersion, + } + } + + return cp } func TestConsensusParamsHash(t *testing.T) { @@ -244,6 +255,7 @@ func TestConsensusParamsHash(t *testing.T) { makeParams(makeParamsArgs{blockBytes: 9, blockGas: 5, evidenceAge: 4, maxEvidenceBytes: 1}), makeParams(makeParamsArgs{blockBytes: 7, blockGas: 8, evidenceAge: 9, maxEvidenceBytes: 1}), makeParams(makeParamsArgs{blockBytes: 4, blockGas: 6, evidenceAge: 5, maxEvidenceBytes: 1}), + makeParams(makeParamsArgs{blockBytes: 4, blockGas: 6, evidenceAge: 5, maxEvidenceBytes: 1, consensusVersion: 1}), } hashes := make([][]byte, len(params)) @@ -352,6 +364,20 @@ func TestConsensusParamsUpdate(t *testing.T) { evidenceAge: 300, maxEvidenceBytes: 50, }), + }, { + initialParams: makeParams(makeParamsArgs{blockBytes: 100, blockGas: 200, evidenceAge: 300}), + updates: &tmproto.ConsensusParams{ + Version: &tmproto.VersionParams{ + AppVersion: 1, + ConsensusVersion: 1, + }, + }, + updatedParams: makeParams(makeParamsArgs{ + blockBytes: 100, + blockGas: 200, + evidenceAge: 300, + consensusVersion: 1, + }), }, } @@ -360,15 +386,16 @@ func TestConsensusParamsUpdate(t *testing.T) { } } -func TestConsensusParamsUpdate_AppVersion(t *testing.T) { +func TestConsensusParamsUpdate_Version(t *testing.T) { params := makeParams(makeParamsArgs{blockBytes: 1, blockGas: 2, evidenceAge: 3}) assert.EqualValues(t, 0, params.Version.AppVersion) updated := params.UpdateConsensusParams( - &tmproto.ConsensusParams{Version: &tmproto.VersionParams{AppVersion: 1}}) + &tmproto.ConsensusParams{Version: &tmproto.VersionParams{AppVersion: 1, ConsensusVersion: 1}}) assert.EqualValues(t, 1, updated.Version.AppVersion) + assert.EqualValues(t, 1, updated.Version.ConsensusVersion) } func TestProto(t *testing.T) { @@ -401,6 +428,17 @@ func TestProto(t *testing.T) { } } +func TestFromProto(t *testing.T) { + params := []tmproto.ConsensusParams{ + {}, + } + + for i := range params { + pbParams := params[i] + assert.NotPanics(t, func() { ConsensusParamsFromProto(pbParams) }) + } +} + func durationPtr(t time.Duration) *time.Duration { return &t } diff --git a/types/validator.go b/types/validator.go index 9316f96b53..72b229abd6 100644 --- a/types/validator.go +++ b/types/validator.go @@ -1,7 +1,6 @@ package types import ( - "bytes" "encoding/json" "errors" "fmt" @@ -25,8 +24,6 @@ type Validator struct { PubKey crypto.PubKey VotingPower int64 NodeAddress ValidatorAddress - - ProposerPriority int64 } type validatorJSON struct { @@ -39,9 +36,8 @@ type validatorJSON struct { func (v Validator) MarshalJSON() ([]byte, error) { val := validatorJSON{ - ProTxHash: v.ProTxHash, - VotingPower: v.VotingPower, - ProposerPriority: v.ProposerPriority, + ProTxHash: v.ProTxHash, + VotingPower: v.VotingPower, } if v.PubKey != nil { pk, err := jsontypes.Marshal(v.PubKey) @@ -63,23 +59,20 @@ func (v *Validator) UnmarshalJSON(data []byte) error { } v.ProTxHash = val.ProTxHash v.VotingPower = val.VotingPower - v.ProposerPriority = val.ProposerPriority return nil } func NewTestValidatorGeneratedFromProTxHash(proTxHash crypto.ProTxHash) *Validator { return &Validator{ - VotingPower: DefaultDashVotingPower, - ProposerPriority: 0, - ProTxHash: proTxHash, + VotingPower: DefaultDashVotingPower, + ProTxHash: proTxHash, } } func NewTestRemoveValidatorGeneratedFromProTxHash(proTxHash crypto.ProTxHash) *Validator { return &Validator{ - VotingPower: 0, - ProposerPriority: 0, - ProTxHash: proTxHash, + VotingPower: 0, + ProTxHash: proTxHash, } } @@ -100,11 +93,10 @@ func NewValidator(pubKey crypto.PubKey, votingPower int64, proTxHash ProTxHash, } } return &Validator{ - PubKey: pubKey, - VotingPower: votingPower, - ProposerPriority: 0, - ProTxHash: proTxHash, - NodeAddress: addr, + PubKey: pubKey, + VotingPower: votingPower, + ProTxHash: proTxHash, + NodeAddress: addr, } } @@ -154,29 +146,6 @@ func (v *Validator) Copy() *Validator { return &vCopy } -// CompareProposerPriority Returns the one with higher ProposerPriority. -func (v *Validator) CompareProposerPriority(other *Validator) *Validator { - if v == nil { - return other - } - switch { - case v.ProposerPriority > other.ProposerPriority: - return v - case v.ProposerPriority < other.ProposerPriority: - return other - default: - result := bytes.Compare(v.ProTxHash, other.ProTxHash) - switch { - case result < 0: - return v - case result > 0: - return other - default: - panic("Cannot compare identical validators") - } - } -} - // String returns a string representation of String. // // 1. address @@ -188,11 +157,10 @@ func (v *Validator) String() string { if v == nil { return "nil-Validator" } - return fmt.Sprintf("Validator{%v %v VP:%v A:%v N:%s}", + return fmt.Sprintf("Validator{%v %v VP:%v N:%s}", v.ProTxHash, v.PubKey, v.VotingPower, - v.ProposerPriority, v.NodeAddress.String()) } @@ -209,7 +177,6 @@ func (v *Validator) ShortStringBasic() string { func (v *Validator) MarshalZerologObject(e *zerolog.Event) { e.Str("protxhash", v.ProTxHash.ShortString()) e.Int64("voting_power", v.VotingPower) - e.Int64("proposer_priority", v.ProposerPriority) e.Str("address", v.NodeAddress.String()) if v.PubKey != nil { @@ -262,9 +229,8 @@ func (v *Validator) ToProto() (*tmproto.Validator, error) { } vp := tmproto.Validator{ - VotingPower: v.VotingPower, - ProposerPriority: v.ProposerPriority, - ProTxHash: v.ProTxHash, + VotingPower: v.VotingPower, + ProTxHash: v.ProTxHash, } if v.PubKey != nil && len(v.PubKey.Bytes()) > 0 { @@ -287,7 +253,6 @@ func ValidatorFromProto(vp *tmproto.Validator) (*Validator, error) { } v := new(Validator) v.VotingPower = vp.GetVotingPower() - v.ProposerPriority = vp.GetProposerPriority() v.ProTxHash = vp.ProTxHash var err error diff --git a/types/validator_set.go b/types/validator_set.go index 94dae79198..37aff76e32 100644 --- a/types/validator_set.go +++ b/types/validator_set.go @@ -5,7 +5,6 @@ import ( "errors" "fmt" "math" - "math/big" "sort" "strings" @@ -19,6 +18,8 @@ import ( "github.com/dashpay/tenderdash/crypto/merkle" "github.com/dashpay/tenderdash/dash/llmq" tmbytes "github.com/dashpay/tenderdash/libs/bytes" + tmmath "github.com/dashpay/tenderdash/libs/math" + tmproto "github.com/dashpay/tenderdash/proto/tendermint/types" ) @@ -50,27 +51,20 @@ var ( // ValidatorSet represent a set of *Validator at a given height. // // The validators can be fetched by address or index. -// The index is in order of .VotingPower, so the indices are fixed for all +// The index is in order of .ProTxHash, so the indices are fixed for all // rounds of a given blockchain height - ie. the validators are sorted by their -// voting power (descending). Secondary index - .ProTxHash (ascending). -// -// On the other hand, the .ProposerPriority of each validator and the -// designated .GetProposer() of a set changes every round, upon calling -// .IncrementProposerPriority(). +// .ProTxHash (ascending). // // NOTE: Not goroutine-safe. // NOTE: All get/set to validators should copy the value for safety. type ValidatorSet struct { // NOTE: persisted via reflect, must be exported. - Validators []*Validator `json:"validators"` - Proposer *Validator `json:"proposer"` + Validators []*Validator `json:"validators"` + proposerIndex int32 ThresholdPublicKey crypto.PubKey `json:"threshold_public_key"` QuorumHash crypto.QuorumHash `json:"quorum_hash"` QuorumType btcjson.LLMQType `json:"quorum_type"` HasPublicKeys bool `json:"has_public_keys"` - - // cached (unexported) - totalVotingPower int64 } // NewValidatorSet initializes a ValidatorSet by copying over the values from @@ -94,9 +88,7 @@ func NewValidatorSet(valz []*Validator, newThresholdPublicKey crypto.PubKey, quo if err != nil { panic(fmt.Sprintf("Cannot create validator set: %v", err)) } - if len(valz) > 0 { - vals.IncrementProposerPriority(1) - } + return vals } @@ -127,8 +119,13 @@ func (vals *ValidatorSet) ValidateBasic() error { return ErrValidatorSetNilOrEmpty } - if vals.Proposer == nil { - return errors.New("validator set proposer is not set") + nVals, err := tmmath.SafeConvertInt32(int64(vals.Size())) + if err != nil { + return fmt.Errorf("failed to convert int to int32: %w", err) + } + + if vals.proposerIndex >= nVals { + return fmt.Errorf("validator set proposer index %d out of range, expected < %d", vals.proposerIndex, vals.Size()) } for idx, val := range vals.Validators { @@ -151,7 +148,7 @@ func (vals *ValidatorSet) ValidateBasic() error { return fmt.Errorf("quorumHash error: %w", err) } - if err := vals.Proposer.ValidateBasic(); err != nil { + if err := vals.Proposer().ValidateBasic(); err != nil { return fmt.Errorf("proposer failed validate basic, error: %w", err) } @@ -237,135 +234,42 @@ func (vals *ValidatorSet) QuorumHashValid() error { return nil } -// CopyIncrementProposerPriority increments ProposerPriority and updates the -// proposer on a copy, and returns it. -func (vals *ValidatorSet) CopyIncrementProposerPriority(times int32) *ValidatorSet { - copy := vals.Copy() - copy.IncrementProposerPriority(times) - return copy -} - -// IncrementProposerPriority increments ProposerPriority of each validator and -// updates the proposer. Panics if validator set is empty. -// `times` must be positive. -func (vals *ValidatorSet) IncrementProposerPriority(times int32) { +// Proposer returns the proposer of the validator set. +// +// Panics on empty validator set. +func (vals *ValidatorSet) Proposer() *Validator { if vals.IsNilOrEmpty() { panic("empty validator set") } - if times <= 0 { - panic("Cannot call IncrementProposerPriority with non-positive times") - } - - // Cap the difference between priorities to be proportional to 2*totalPower by - // re-normalizing priorities, i.e., rescale all priorities by multiplying with: - // 2*totalVotingPower/(maxPriority - minPriority) - diffMax := PriorityWindowSizeFactor * vals.TotalVotingPower() - vals.RescalePriorities(diffMax) - vals.shiftByAvgProposerPriority() - - var proposer *Validator - // Call IncrementProposerPriority(1) times times. - for i := int32(0); i < times; i++ { - proposer = vals.incrementProposerPriority() - } - - vals.Proposer = proposer + return vals.GetByIndex(vals.proposerIndex) } -// RescalePriorities rescales the priorities such that the distance between the -// maximum and minimum is smaller than `diffMax`. Panics if validator set is -// empty. -func (vals *ValidatorSet) RescalePriorities(diffMax int64) { - if vals.IsNilOrEmpty() { - panic("empty validator set") - } - // NOTE: This check is merely a sanity check which could be - // removed if all tests would init. voting power appropriately; - // i.e. diffMax should always be > 0 - if diffMax <= 0 { - return +// SetProposer sets the proposer of the validator set. +func (vals *ValidatorSet) SetProposer(newProposer ProTxHash) error { + idx, _ := vals.GetByProTxHash(newProposer) + if idx < 0 { + return fmt.Errorf("proposer %X not found in validator set", newProposer) } - // Calculating ceil(diff/diffMax): - // Re-normalization is performed by dividing by an integer for simplicity. - // NOTE: This may make debugging priority issues easier as well. - diff := computeMaxMinPriorityDiff(vals) - ratio := (diff + diffMax - 1) / diffMax - if diff > diffMax && ratio != 0 { - for _, val := range vals.Validators { - val.ProposerPriority /= ratio - } - } + vals.proposerIndex = idx + return nil } -func (vals *ValidatorSet) incrementProposerPriority() *Validator { - for _, val := range vals.Validators { - // Check for overflow for sum. - newPrio := safeAddClip(val.ProposerPriority, val.VotingPower) - val.ProposerPriority = newPrio - } - // Decrement the validator with most ProposerPriority. - mostest := vals.getValWithMostPriority() - // Mind the underflow. - mostest.ProposerPriority = safeSubClip(mostest.ProposerPriority, vals.TotalVotingPower()) - - return mostest -} +// increaseProposerIndex increments the proposer index by `times`, wrapping around if necessary. +// It also supports negative `times` to decrement the index. +func (vals *ValidatorSet) IncProposerIndex(times int64) { + newIndex := int64(vals.proposerIndex) + times + nVals := int64(vals.Size()) -// Should not be called on an empty validator set. -func (vals *ValidatorSet) computeAvgProposerPriority() int64 { - n := int64(len(vals.Validators)) - sum := big.NewInt(0) - for _, val := range vals.Validators { - sum.Add(sum, big.NewInt(val.ProposerPriority)) + for newIndex < 0 { + newIndex += nVals } - avg := sum.Div(sum, big.NewInt(n)) - if avg.IsInt64() { - return avg.Int64() - } - - // This should never happen: each val.ProposerPriority is in bounds of int64. - panic(fmt.Sprintf("Cannot represent avg ProposerPriority as an int64 %v", avg)) -} -// Compute the difference between the max and min ProposerPriority of that set. -func computeMaxMinPriorityDiff(vals *ValidatorSet) int64 { - if vals.IsNilOrEmpty() { - panic("empty validator set") - } - max := int64(math.MinInt64) - min := int64(math.MaxInt64) - for _, v := range vals.Validators { - if v.ProposerPriority < min { - min = v.ProposerPriority - } - if v.ProposerPriority > max { - max = v.ProposerPriority - } - } - diff := max - min - if diff < 0 { - return -1 * diff - } - return diff -} - -func (vals *ValidatorSet) getValWithMostPriority() *Validator { - var res *Validator - for _, val := range vals.Validators { - res = res.CompareProposerPriority(val) - } - return res -} - -func (vals *ValidatorSet) shiftByAvgProposerPriority() { - if vals.IsNilOrEmpty() { - panic("empty validator set") - } - avgProposerPriority := vals.computeAvgProposerPriority() - for _, val := range vals.Validators { - val.ProposerPriority = safeSubClip(val.ProposerPriority, avgProposerPriority) + idx, err := tmmath.SafeConvertInt32(newIndex % nVals) + if err != nil { + panic(fmt.Errorf("failed to convert int64 to int32: %w", err)) } + vals.proposerIndex = idx } // Makes a copy of the validator list. @@ -385,15 +289,16 @@ func (vals *ValidatorSet) Copy() *ValidatorSet { if vals == nil { return nil } - return &ValidatorSet{ + valset := &ValidatorSet{ Validators: validatorListCopy(vals.Validators), - Proposer: vals.Proposer, - totalVotingPower: vals.totalVotingPower, ThresholdPublicKey: vals.ThresholdPublicKey, QuorumHash: vals.QuorumHash, QuorumType: vals.QuorumType, HasPublicKeys: vals.HasPublicKeys, + proposerIndex: vals.proposerIndex, } + + return valset } // HasProTxHash returns true if proTxHash given is in the validator set, false - @@ -415,7 +320,11 @@ func (vals *ValidatorSet) HasProTxHash(proTxHash crypto.ProTxHash) bool { func (vals *ValidatorSet) GetByProTxHash(proTxHash []byte) (index int32, val *Validator) { for idx, val := range vals.Validators { if bytes.Equal(val.ProTxHash, proTxHash) { - return int32(idx), val.Copy() + index, err := tmmath.SafeConvertInt32(int64(idx)) + if err != nil { + panic(fmt.Errorf("failed to convert int to int32: %w", err)) + } + return index, val.Copy() } } return -1, nil @@ -474,31 +383,10 @@ func (vals *ValidatorSet) Size() int { return len(vals.Validators) } -// Forces recalculation of the set's total voting power. -// Panics if total voting power is bigger than MaxTotalVotingPower. -func (vals *ValidatorSet) updateTotalVotingPower() { - sum := int64(0) - for _, val := range vals.Validators { - // mind overflow - sum = safeAddClip(sum, val.VotingPower) - if sum > MaxTotalVotingPower { - panic(fmt.Sprintf( - "Total voting power should be guarded to not exceed %v; got: %v", - MaxTotalVotingPower, - sum)) - } - } - - vals.totalVotingPower = sum -} - // TotalVotingPower returns the sum of the voting powers of all validators. // It recomputes the total voting power if required. func (vals *ValidatorSet) TotalVotingPower() int64 { - if vals.totalVotingPower == 0 { - vals.updateTotalVotingPower() - } - return vals.totalVotingPower + return int64(vals.Size()) * DefaultDashVotingPower } // QuorumVotingPower returns the voting power of the quorum if all the members existed. @@ -530,28 +418,6 @@ func (vals *ValidatorSet) QuorumTypeThresholdCount() int { return threshold } -// GetProposer returns the current proposer. If the validator set is empty, nil -// is returned. -func (vals *ValidatorSet) GetProposer() (proposer *Validator) { - if len(vals.Validators) == 0 { - return nil - } - if vals.Proposer == nil { - vals.Proposer = vals.findProposer() - } - return vals.Proposer.Copy() -} - -func (vals *ValidatorSet) findProposer() *Validator { - var proposer *Validator - for _, val := range vals.Validators { - if proposer == nil || !bytes.Equal(val.ProTxHash, proposer.ProTxHash) { - proposer = proposer.CompareProposerPriority(val) - } - } - return proposer -} - // Hash returns the Quorum Hash. func (vals *ValidatorSet) Hash() tmbytes.HexBytes { if vals == nil || vals.QuorumHash == nil || vals.ThresholdPublicKey == nil { @@ -680,39 +546,6 @@ func numNewValidators(updates []*Validator, vals *ValidatorSet) int { return numNewValidators } -// computeNewPriorities computes the proposer priority for the validators not present in the set based on -// 'updatedTotalVotingPower'. -// Leaves unchanged the priorities of validators that are changed. -// -// 'updates' parameter must be a list of unique validators to be added or updated. -// -// 'updatedTotalVotingPower' is the total voting power of a set where all updates would be applied but -// -// not the removals. It must be < 2*MaxTotalVotingPower and may be close to this limit if close to -// MaxTotalVotingPower will be removed. This is still safe from overflow since MaxTotalVotingPower is maxInt64/8. -// -// No changes are made to the validator set 'vals'. -func computeNewPriorities(updates []*Validator, vals *ValidatorSet, updatedTotalVotingPower int64) { - for _, valUpdate := range updates { - proTxHash := valUpdate.ProTxHash - _, val := vals.GetByProTxHash(proTxHash) - if val == nil { - // add val - // Set ProposerPriority to -C*totalVotingPower (with C ~= 1.125) to make sure validators can't - // un-bond and then re-bond to reset their (potentially previously negative) ProposerPriority to zero. - // - // Contract: updatedVotingPower < 2 * MaxTotalVotingPower to ensure ProposerPriority does - // not exceed the bounds of int64. - // - // Compute ProposerPriority = -1.125*totalVotingPower == -(updatedVotingPower + (updatedVotingPower >> 3)). - valUpdate.ProposerPriority = -(updatedTotalVotingPower + (updatedTotalVotingPower >> 3)) - } else { - valUpdate.ProposerPriority = val.ProposerPriority - } - } - -} - // Merges the vals' validator list with the updates list. // When two elements with same address are seen, the one from updates is selected. // Expects updates to be a list of updates sorted by proTxHash with no duplicates or errors, @@ -843,25 +676,15 @@ func (vals *ValidatorSet) updateWithChangeSet(changes []*Validator, allowDeletes // Verify that applying the 'updates' against 'vals' will not result in error. // Get the updated total voting power before removal. Note that this is < 2 * MaxTotalVotingPower - tvpAfterUpdatesBeforeRemovals, err := verifyUpdates(updates, vals, removedVotingPower) + _, err = verifyUpdates(updates, vals, removedVotingPower) if err != nil { return err } - - // Compute the priorities for updates. - computeNewPriorities(updates, vals, tvpAfterUpdatesBeforeRemovals) - // Apply updates and removals. vals.applyUpdates(updates) vals.applyRemovals(deletes) - vals.updateTotalVotingPower() // will panic if total voting power > MaxTotalVotingPower - - // Scale and center. - vals.RescalePriorities(PriorityWindowSizeFactor * vals.TotalVotingPower()) - vals.shiftByAvgProposerPriority() - - sort.Sort(ValidatorsByVotingPower(vals.Validators)) + sort.Sort(ValidatorsByProTxHashes(vals.Validators)) vals.ThresholdPublicKey = newThresholdPublicKey vals.QuorumHash = newQuorumHash @@ -927,24 +750,6 @@ func (vals *ValidatorSet) VerifyCommit(chainID string, blockID BlockID, return nil } -// findPreviousProposer reverses the compare proposer priority function to find the validator -// with the lowest proposer priority which would have been the previous proposer. -// -// Is used when recreating a validator set from an existing array of validators. -func (vals *ValidatorSet) findPreviousProposer() *Validator { - var previousProposer *Validator - for _, val := range vals.Validators { - if previousProposer == nil { - previousProposer = val - continue - } - if previousProposer == previousProposer.CompareProposerPriority(val) { - previousProposer = val - } - } - return previousProposer -} - //----------------- // IsErrNotEnoughVotingPowerSigned returns true if err is @@ -1000,13 +805,12 @@ func (vals *ValidatorSet) StringIndented(indent string) string { valStrings = append(valStrings, val.String()) return false }) + return fmt.Sprintf(`ValidatorSet{ -%s Proposer: %v %s QuorumHash: %v %s Validators: %s %v %s}`, - indent, vals.GetProposer().String(), indent, vals.QuorumHash.String(), indent, indent, strings.Join(valStrings, "\n"+indent+" "), @@ -1031,12 +835,10 @@ func (vals *ValidatorSet) StringIndentedBasic(indent string) string { return false }) return fmt.Sprintf(`ValidatorSet{ -%s Proposer: %v %s QuorumHash: %v %s Validators: %s %v %s}`, - indent, vals.GetProposer().ProTxHash.ShortString(), indent, vals.QuorumHash.String(), indent, indent, strings.Join(valStrings, "\n"+indent+" "), @@ -1049,7 +851,6 @@ func (vals *ValidatorSet) MarshalZerologObject(e *zerolog.Event) { if vals == nil { return } - e.Str("proposer", vals.GetProposer().ProTxHash.ShortString()) e.Str("quorum_hash", vals.QuorumHash.ShortString()) validators := zerolog.Arr() for _, item := range vals.Validators { @@ -1060,23 +861,6 @@ func (vals *ValidatorSet) MarshalZerologObject(e *zerolog.Event) { //------------------------------------- -// ValidatorsByVotingPower implements sort.Interface for []*Validator based on -// the VotingPower and Address fields. -type ValidatorsByVotingPower []*Validator - -func (valz ValidatorsByVotingPower) Len() int { return len(valz) } - -func (valz ValidatorsByVotingPower) Less(i, j int) bool { - if valz[i].VotingPower == valz[j].VotingPower { - return bytes.Compare(valz[i].ProTxHash, valz[j].ProTxHash) == -1 - } - return valz[i].VotingPower > valz[j].VotingPower -} - -func (valz ValidatorsByVotingPower) Swap(i, j int) { - valz[i], valz[j] = valz[j], valz[i] -} - // ValidatorsByAddress implements sort.Interface for []*Validator based on // the Address field. type ValidatorsByProTxHashes []*Validator @@ -1108,7 +892,7 @@ func (vals *ValidatorSet) ToProto() (*tmproto.ValidatorSet, error) { } vp.Validators = valsProto - valProposer, err := vals.Proposer.ToProto() + valProposer, err := vals.Proposer().ToProto() if err != nil { return nil, fmt.Errorf("toProto: validatorSet proposer error: %w", err) } @@ -1132,7 +916,11 @@ func (vals *ValidatorSet) ToProto() (*tmproto.ValidatorSet, error) { return nil, fmt.Errorf("toProto: quorumHash is incorrect size: %d", len(vals.QuorumHash)) } - vp.QuorumType = int32(vals.QuorumType) + quorumType, err := tmmath.SafeConvertInt32(int64(vals.QuorumType)) + if err != nil { + return nil, fmt.Errorf("toProto: quorumType error: %w", err) + } + vp.QuorumType = quorumType vp.QuorumHash = vals.QuorumHash @@ -1163,8 +951,7 @@ func ValidatorSetFromProto(vp *tmproto.ValidatorSet) (*ValidatorSet, error) { var err error proposer := vp.GetProposer() if proposer != nil { - vals.Proposer, err = ValidatorFromProto(vp.GetProposer()) - if err != nil { + if err := vals.SetProposer(proposer.GetProTxHash()); err != nil { return nil, fmt.Errorf("fromProto: validatorSet proposer error: %w", err) } } @@ -1192,42 +979,6 @@ func ValidatorSetFromProto(vp *tmproto.ValidatorSet) (*ValidatorSet, error) { return vals, vals.ValidateBasic() } -// ValidatorSetFromExistingValidators takes an existing array of validators and rebuilds -// the exact same validator set that corresponds to it without changing the proposer priority or power -// if any of the validators fail validate basic then an empty set is returned. -func ValidatorSetFromExistingValidators( - valz []*Validator, - thresholdPublicKey crypto.PubKey, - quorumType btcjson.LLMQType, - quorumHash crypto.QuorumHash, -) (*ValidatorSet, error) { - if len(valz) == 0 { - return nil, errors.New("validator set is empty") - } - hasPublicKeys := true - for _, val := range valz { - err := val.ValidateBasic() - if val.PubKey == nil { - hasPublicKeys = false - } - if err != nil { - return nil, fmt.Errorf("can't create validator set: %w", err) - } - } - - vals := &ValidatorSet{ - Validators: valz, - ThresholdPublicKey: thresholdPublicKey, - QuorumType: quorumType, - QuorumHash: quorumHash, - HasPublicKeys: hasPublicKeys, - } - vals.Proposer = vals.findPreviousProposer() - vals.updateTotalVotingPower() - sort.Sort(ValidatorsByVotingPower(vals.Validators)) - return vals, nil -} - //---------------------------------------- func ValidatorUpdatesRegenerateOnProTxHashes(proTxHashes []crypto.ProTxHash) abci.ValidatorSetUpdate { @@ -1251,67 +1002,3 @@ func ValidatorUpdatesRegenerateOnProTxHashes(proTxHashes []crypto.ProTxHash) abc QuorumHash: crypto.RandQuorumHash(), } } - -// safe addition/subtraction/multiplication - -func safeAdd(a, b int64) (int64, bool) { - if b > 0 && a > math.MaxInt64-b { - return -1, true - } else if b < 0 && a < math.MinInt64-b { - return -1, true - } - return a + b, false -} - -func safeSub(a, b int64) (int64, bool) { - if b > 0 && a < math.MinInt64+b { - return -1, true - } else if b < 0 && a > math.MaxInt64+b { - return -1, true - } - return a - b, false -} - -func safeAddClip(a, b int64) int64 { - c, overflow := safeAdd(a, b) - if overflow { - if b < 0 { - return math.MinInt64 - } - return math.MaxInt64 - } - return c -} - -func safeSubClip(a, b int64) int64 { - c, overflow := safeSub(a, b) - if overflow { - if b > 0 { - return math.MinInt64 - } - return math.MaxInt64 - } - return c -} - -func safeMul(a, b int64) (int64, bool) { - if a == 0 || b == 0 { - return 0, false - } - - absOfB := b - if b < 0 { - absOfB = -b - } - - absOfA := a - if a < 0 { - absOfA = -a - } - - if absOfA > math.MaxInt64/absOfB { - return 0, true - } - - return a * b, false -} diff --git a/types/validator_set_test.go b/types/validator_set_test.go index bc31df9dd8..c423ff7710 100644 --- a/types/validator_set_test.go +++ b/types/validator_set_test.go @@ -4,12 +4,11 @@ import ( "bytes" "context" "fmt" - "math" "math/rand" "sort" + "strconv" "strings" "testing" - "testing/quick" "github.com/dashpay/dashd-go/btcjson" "github.com/stretchr/testify/assert" @@ -25,12 +24,7 @@ import ( func TestValidatorSetBasic(t *testing.T) { // empty or nil validator lists are allowed, - // but attempting to IncrementProposerPriority on them will panic. vset := NewValidatorSet([]*Validator{}, nil, btcjson.LLMQType_5_60, nil, true) - assert.Panics(t, func() { vset.IncrementProposerPriority(1) }) - - vset = NewValidatorSet(nil, nil, btcjson.LLMQType_5_60, nil, true) - assert.Panics(t, func() { vset.IncrementProposerPriority(1) }) assert.EqualValues(t, vset, vset.Copy()) assert.False(t, vset.HasProTxHash([]byte("some val"))) @@ -45,10 +39,9 @@ func TestValidatorSetBasic(t *testing.T) { assert.Nil(t, val) assert.Zero(t, vset.Size()) assert.Equal(t, int64(0), vset.TotalVotingPower()) - assert.Nil(t, vset.GetProposer()) assert.Equal(t, tmbytes.HexBytes(nil), vset.Hash()) // add - val = randModuloValidator(vset.TotalVotingPower()) + val = randValidator() assert.NoError(t, vset.UpdateWithChangeSet([]*Validator{val}, val.PubKey, crypto.RandQuorumHash())) assert.True(t, vset.HasProTxHash(val.ProTxHash)) @@ -59,20 +52,12 @@ func TestValidatorSetBasic(t *testing.T) { assert.Equal(t, 1, vset.Size()) assert.Equal(t, val.VotingPower, vset.TotalVotingPower()) assert.NotNil(t, vset.Hash()) - assert.NotPanics(t, func() { vset.IncrementProposerPriority(1) }) - assert.Equal(t, val.ProTxHash, vset.GetProposer().ProTxHash) // update - val = randModuloValidator(vset.TotalVotingPower()) + val = randValidator() assert.NoError(t, vset.UpdateWithChangeSet([]*Validator{val}, val.PubKey, crypto.RandQuorumHash())) _, val = vset.GetByProTxHash(val.ProTxHash) val.PubKey = bls12381.GenPrivKey().PubKey() - proposerPriority := val.ProposerPriority - - val.ProposerPriority = 0 - assert.NoError(t, vset.UpdateWithChangeSet([]*Validator{val}, val.PubKey, crypto.RandQuorumHash())) - _, val = vset.GetByProTxHash(val.ProTxHash) - assert.Equal(t, proposerPriority, val.ProposerPriority) } @@ -136,8 +121,7 @@ func TestValidatorSetValidateBasic(t *testing.T) { QuorumHash: crypto.RandQuorumHash(), HasPublicKeys: true, }, - err: true, - msg: "validator set proposer is not set", + err: false, }, { testName: "Validator in set has wrong public key for threshold", @@ -146,7 +130,6 @@ func TestValidatorSetValidateBasic(t *testing.T) { ThresholdPublicKey: bls12381.GenPrivKey().PubKey(), QuorumHash: crypto.RandQuorumHash(), HasPublicKeys: true, - Proposer: val, }, err: true, msg: "thresholdPublicKey error: incorrect threshold public key", @@ -158,7 +141,6 @@ func TestValidatorSetValidateBasic(t *testing.T) { ThresholdPublicKey: bls12381.GenPrivKey().PubKey(), QuorumHash: crypto.RandQuorumHash(), HasPublicKeys: true, - Proposer: badValNoPublicKey, }, err: true, msg: "invalid validator pub key #0: validator does not have a public key", @@ -170,7 +152,6 @@ func TestValidatorSetValidateBasic(t *testing.T) { ThresholdPublicKey: bls12381.GenPrivKey().PubKey(), QuorumHash: crypto.RandQuorumHash(), HasPublicKeys: true, - Proposer: badValNoProTxHash, }, err: true, msg: "invalid validator #0: validator does not have a provider transaction hash", @@ -179,7 +160,6 @@ func TestValidatorSetValidateBasic(t *testing.T) { testName: "Validator set needs quorum hash", vals: ValidatorSet{ Validators: []*Validator{val}, - Proposer: val, ThresholdPublicKey: val.PubKey, HasPublicKeys: true, }, @@ -190,7 +170,6 @@ func TestValidatorSetValidateBasic(t *testing.T) { testName: "Validator set single val good", vals: ValidatorSet{ Validators: []*Validator{val}, - Proposer: val, ThresholdPublicKey: val.PubKey, QuorumHash: crypto.RandQuorumHash(), HasPublicKeys: true, @@ -202,7 +181,6 @@ func TestValidatorSetValidateBasic(t *testing.T) { testName: "Validator set needs threshold public key", vals: ValidatorSet{ Validators: []*Validator{val}, - Proposer: val, QuorumHash: crypto.RandQuorumHash(), HasPublicKeys: true, }, @@ -253,27 +231,17 @@ func TestCopy(t *testing.T) { } } -// Test that IncrementProposerPriority requires positive times. -func TestIncrementProposerPriorityPositiveTimes(t *testing.T) { - vset := NewValidatorSet([]*Validator{ - NewTestValidatorGeneratedFromProTxHash(crypto.Checksum([]byte("foo"))), - NewTestValidatorGeneratedFromProTxHash(crypto.Checksum([]byte("bar"))), - NewTestValidatorGeneratedFromProTxHash(crypto.Checksum([]byte("baz"))), - }, bls12381.GenPrivKey().PubKey(), btcjson.LLMQType_5_60, crypto.RandQuorumHash(), true) - - assert.Panics(t, func() { vset.IncrementProposerPriority(-1) }) - assert.Panics(t, func() { vset.IncrementProposerPriority(0) }) - vset.IncrementProposerPriority(1) -} - func BenchmarkValidatorSetCopy(b *testing.B) { b.StopTimer() vset := NewValidatorSet([]*Validator{}, nil, btcjson.LLMQType_5_60, nil, true) for i := 0; i < 1000; i++ { privKey := bls12381.GenPrivKey() pubKey := privKey.PubKey() - val := NewValidatorDefaultVotingPower(pubKey, crypto.ProTxHash{}) - err := vset.UpdateWithChangeSet([]*Validator{val}, nil, crypto.RandQuorumHash()) + + quorumKey := bls12381.GenPrivKey().PubKey() + ProTxHash := crypto.RandProTxHash() + val := NewValidatorDefaultVotingPower(pubKey, ProTxHash) + err := vset.UpdateWithChangeSet([]*Validator{val}, quorumKey, crypto.RandQuorumHash()) require.NoError(b, err) } b.StartTimer() @@ -283,139 +251,15 @@ func BenchmarkValidatorSetCopy(b *testing.B) { } } -//------------------------------------------------------------------- - -func TestProposerSelection1(t *testing.T) { - fooProTxHash := crypto.ProTxHash(crypto.Checksum([]byte("foo"))) - barProTxHash := crypto.ProTxHash(crypto.Checksum([]byte("bar"))) - bazProTxHash := crypto.ProTxHash(crypto.Checksum([]byte("baz"))) - vset := NewValidatorSet([]*Validator{ - NewTestValidatorGeneratedFromProTxHash(fooProTxHash), - NewTestValidatorGeneratedFromProTxHash(barProTxHash), - NewTestValidatorGeneratedFromProTxHash(bazProTxHash), - }, bls12381.GenPrivKey().PubKey(), btcjson.LLMQType_5_60, crypto.RandQuorumHash(), true) - var proposers []string - for i := 0; i < 99; i++ { - val := vset.GetProposer() - proposers = append(proposers, val.ProTxHash.ShortString()) - vset.IncrementProposerPriority(1) - } - expected := `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + - `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + - `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + - `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + - `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + - `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B ` + - `2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B 2C26B4 BAA5A0 FCDE2B` - if expected != strings.Join(proposers, " ") { - t.Errorf("expected sequence of proposers was\n%v\nbut got \n%v", expected, strings.Join(proposers, " ")) - } -} - -func TestProposerSelection2(t *testing.T) { - proTxHashes := make([]crypto.ProTxHash, 3) - addresses := make([]crypto.Address, 3) - proTxHashes[0] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1} - proTxHashes[1] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2} - proTxHashes[2] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 3} - addresses[0] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2} - addresses[1] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1} - addresses[2] = []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0} - - vals, _ := GenerateValidatorSet(NewValSetParam(proTxHashes)) - for i := 0; i < len(proTxHashes)*5; i++ { - ii := (i) % len(proTxHashes) - prop := vals.GetProposer() - if !bytes.Equal(prop.ProTxHash, vals.Validators[ii].ProTxHash) { - t.Fatalf("(%d): Expected %X. Got %X", i, vals.Validators[ii].ProTxHash, prop.ProTxHash) - } - vals.IncrementProposerPriority(1) - } - - prop := vals.GetProposer() - if !bytes.Equal(prop.ProTxHash, proTxHashes[0]) { - t.Fatalf("Expected proposer with smallest pro_tx_hash to be first proposer. Got %X", prop.ProTxHash) - } - vals.IncrementProposerPriority(1) - prop = vals.GetProposer() - if !bytes.Equal(prop.ProTxHash, proTxHashes[1]) { - t.Fatalf("Expected proposer with second smallest pro_tx_hash to be second proposer. Got %X", prop.ProTxHash) - } -} - -func TestProposerSelection3(t *testing.T) { - proTxHashes := make([]crypto.ProTxHash, 4) - proTxHashes[0] = crypto.Checksum([]byte("avalidator_address12")) - proTxHashes[1] = crypto.Checksum([]byte("bvalidator_address12")) - proTxHashes[2] = crypto.Checksum([]byte("cvalidator_address12")) - proTxHashes[3] = crypto.Checksum([]byte("dvalidator_address12")) - - vset, _ := GenerateValidatorSet(NewValSetParam(proTxHashes)) - - proposerOrder := make([]*Validator, 4) - for i := 0; i < 4; i++ { - proposerOrder[i] = vset.GetProposer() - vset.IncrementProposerPriority(1) - } - - // i for the loop - // j for the times - // we should go in order for ever, despite some IncrementProposerPriority with times > 1 - var ( - i int - j int32 - ) - for ; i < 10000; i++ { - got := vset.GetProposer().ProTxHash - expected := proposerOrder[j%4].ProTxHash - if !bytes.Equal(got, expected) { - t.Fatalf(fmt.Sprintf("vset.Proposer (%X) does not match expected proposer (%X) for (%d, %d)", got, expected, i, j)) - } - - // serialize, deserialize, check proposer - b := vset.toBytes(t) - vset = vset.fromBytes(t, b) - - computed := vset.GetProposer() // findGetProposer() - if i != 0 { - if !bytes.Equal(got, computed.ProTxHash) { - t.Fatalf( - fmt.Sprintf( - "vset.Proposer (%X) does not match computed proposer (%X) for (%d, %d)", - got, - computed.ProTxHash, - i, - j, - ), - ) - } - } - - // times is usually 1 - times := int32(1) - mod := (rand.Int() % 5) + 1 - if rand.Int()%mod > 0 { - // sometimes its up to 5 - times = (rand.Int31() % 4) + 1 - } - vset.IncrementProposerPriority(times) - - j += times - } -} - func randPubKey() crypto.PubKey { pubKey := make(bls12381.PubKey, bls12381.PubKeySize) copy(pubKey, tmrand.Bytes(32)) return bls12381.PubKey(tmrand.Bytes(32)) } -func randModuloValidator(totalVotingPower int64) *Validator { - // this modulo limits the ProposerPriority/VotingPower to stay in the - // bounds of MaxTotalVotingPower minus the already existing voting power: +func randValidator() *Validator { address := RandValidatorAddress().String() val := NewValidator(randPubKey(), DefaultDashVotingPower, crypto.RandProTxHash(), address) - val.ProposerPriority = rand.Int63() % (MaxTotalVotingPower - totalVotingPower) return val } @@ -432,138 +276,10 @@ func randValidatorInQuorum(ctx context.Context, t *testing.T, quorumHash crypto. return val, privVal } -func (vals *ValidatorSet) toBytes(t *testing.T) []byte { - pbvs, err := vals.ToProto() - require.NoError(t, err) - - bz, err := pbvs.Marshal() - require.NoError(t, err) - - return bz -} - -func (vals *ValidatorSet) fromBytes(t *testing.T, b []byte) *ValidatorSet { - pbvs := new(tmproto.ValidatorSet) - err := pbvs.Unmarshal(b) - require.NoError(t, err) - - vs, err := ValidatorSetFromProto(pbvs) - require.NoError(t, err) - - return vs -} - -func TestAvgProposerPriority(t *testing.T) { - // Create Validator set without calling IncrementProposerPriority: - tcs := []struct { - vs ValidatorSet - want int64 - }{ - 0: {ValidatorSet{Validators: []*Validator{{ProposerPriority: 0}, {ProposerPriority: 0}, {ProposerPriority: 0}}}, 0}, - 1: { - ValidatorSet{ - Validators: []*Validator{{ProposerPriority: math.MaxInt64}, {ProposerPriority: 0}, {ProposerPriority: 0}}, - }, math.MaxInt64 / 3, - }, - 2: { - ValidatorSet{ - Validators: []*Validator{{ProposerPriority: math.MaxInt64}, {ProposerPriority: 0}}, - }, math.MaxInt64 / 2, - }, - 3: { - ValidatorSet{ - Validators: []*Validator{{ProposerPriority: math.MaxInt64}, {ProposerPriority: math.MaxInt64}}, - }, math.MaxInt64, - }, - 4: { - ValidatorSet{ - Validators: []*Validator{{ProposerPriority: math.MinInt64}, {ProposerPriority: math.MinInt64}}, - }, math.MinInt64, - }, - } - for i, tc := range tcs { - got := tc.vs.computeAvgProposerPriority() - assert.Equal(t, tc.want, got, "test case: %v", i) - } -} - -func TestAveragingInIncrementProposerPriority(t *testing.T) { - // Test that the averaging works as expected inside of IncrementProposerPriority. - // Each validator comes with zero voting power which simplifies reasoning about - // the expected ProposerPriority. - tcs := []struct { - vs ValidatorSet - times int32 - avg int64 - }{ - 0: {ValidatorSet{ - Validators: []*Validator{ - {ProTxHash: []byte("a"), ProposerPriority: 1}, - {ProTxHash: []byte("b"), ProposerPriority: 2}, - {ProTxHash: []byte("c"), ProposerPriority: 3}}}, - 1, - 2, - }, - 1: {ValidatorSet{ - Validators: []*Validator{ - {ProTxHash: []byte("a"), ProposerPriority: 10}, - {ProTxHash: []byte("b"), ProposerPriority: -10}, - {ProTxHash: []byte("c"), ProposerPriority: 1}}}, - // this should average twice but the average should be 0 after the first iteration - // (voting power is 0 -> no changes) - // 1/3 -> 0 - 11, 0}, - 2: {ValidatorSet{ - Validators: []*Validator{ - {ProTxHash: []byte("a"), ProposerPriority: 100}, - {ProTxHash: []byte("b"), ProposerPriority: -10}, - {ProTxHash: []byte("c"), ProposerPriority: 1}}}, - 1, - 91 / 3, - }, - } - for i, tc := range tcs { - // work on copy to have the old ProposerPriorities: - newVset := tc.vs.CopyIncrementProposerPriority(tc.times) - for _, val := range tc.vs.Validators { - _, updatedVal := newVset.GetByProTxHash(val.ProTxHash) - assert.Equal(t, updatedVal.ProposerPriority, val.ProposerPriority-tc.avg, "test case: %v", i) - } - } -} - -func TestSafeAdd(t *testing.T) { - f := func(a, b int64) bool { - c, overflow := safeAdd(a, b) - return overflow || (!overflow && c == a+b) - } - if err := quick.Check(f, nil); err != nil { - t.Error(err) - } -} - -func TestSafeAddClip(t *testing.T) { - assert.EqualValues(t, math.MaxInt64, safeAddClip(math.MaxInt64, 10)) - assert.EqualValues(t, math.MaxInt64, safeAddClip(math.MaxInt64, math.MaxInt64)) - assert.EqualValues(t, math.MinInt64, safeAddClip(math.MinInt64, -10)) -} - -func TestSafeSubClip(t *testing.T) { - assert.EqualValues(t, math.MinInt64, safeSubClip(math.MinInt64, 10)) - assert.EqualValues(t, 0, safeSubClip(math.MinInt64, math.MinInt64)) - assert.EqualValues(t, math.MinInt64, safeSubClip(math.MinInt64, math.MaxInt64)) - assert.EqualValues(t, math.MaxInt64, safeSubClip(math.MaxInt64, -10)) -} - func TestEmptySet(t *testing.T) { var valList []*Validator valSet := NewValidatorSet(valList, bls12381.PubKey{}, btcjson.LLMQType_5_60, crypto.QuorumHash{}, true) - assert.Panics(t, func() { valSet.IncrementProposerPriority(1) }) - assert.Panics(t, func() { valSet.RescalePriorities(100) }) - assert.Panics(t, func() { valSet.shiftByAvgProposerPriority() }) - assert.Panics(t, func() { assert.Zero(t, computeMaxMinPriorityDiff(valSet)) }) - valSet.GetProposer() // Add to empty set proTxHashes := []crypto.ProTxHash{crypto.Checksum([]byte("v1")), crypto.Checksum([]byte("v2"))} @@ -606,9 +322,8 @@ func TestUpdatesForNewValidatorSet(t *testing.T) { // Verify set including validator with negative voting power cannot be created v1 = NewTestValidatorGeneratedFromProTxHash(crypto.Checksum([]byte("v1"))) v2 = &Validator{ - VotingPower: -20, - ProposerPriority: 0, - ProTxHash: crypto.Checksum([]byte("v2")), + VotingPower: -20, + ProTxHash: crypto.Checksum([]byte("v2")), } v3 = NewTestValidatorGeneratedFromProTxHash(crypto.Checksum([]byte("v3"))) valList = []*Validator{v1, v2, v3} @@ -702,36 +417,16 @@ func addValidatorsToValidatorSet(vals *ValidatorSet, testValList []testVal) ([]* } -func valSetTotalProposerPriority(valSet *ValidatorSet) int64 { - sum := int64(0) - for _, val := range valSet.Validators { - // mind overflow - sum = safeAddClip(sum, val.ProposerPriority) - } - return sum -} - func verifyValidatorSet(t *testing.T, valSet *ValidatorSet) { // verify that the capacity and length of validators is the same assert.Equal(t, len(valSet.Validators), cap(valSet.Validators)) // verify that the set's total voting power has been updated - tvp := valSet.totalVotingPower - valSet.updateTotalVotingPower() - expectedTvp := valSet.TotalVotingPower() - assert.Equal(t, expectedTvp, tvp, - "expected TVP %d. Got %d, valSet=%s", expectedTvp, tvp, valSet) - - // verify that validator priorities are centered - valsCount := int64(len(valSet.Validators)) - tpp := valSetTotalProposerPriority(valSet) - assert.True(t, tpp < valsCount && tpp > -valsCount, - "expected total priority in (-%d, %d). Got %d", valsCount, valsCount, tpp) - - // verify that priorities are scaled - dist := computeMaxMinPriorityDiff(valSet) - assert.True(t, dist <= PriorityWindowSizeFactor*tvp, - "expected priority distance < %d. Got %d", PriorityWindowSizeFactor*tvp, dist) + tvp := int64(0) + for _, v := range valSet.Validators { + tvp += v.VotingPower + } + assert.Equal(t, tvp, valSet.TotalVotingPower()) recoveredPublicKey, err := bls12381.RecoverThresholdPublicKeyFromPublicKeys(valSet.GetPublicKeys(), valSet.GetProTxHashesAsByteArrays()) assert.NoError(t, err) @@ -778,7 +473,7 @@ type valSetErrTestCaseWithErr struct { func executeValSetErrTestCaseIgnoreThresholdPublicKey(t *testing.T, idx int, tt valSetErrTestCaseWithErr) { // create a new set and apply updates, keeping copies for the checks valSet := createNewValidatorSet(tt.startVals) - valSetCopy := valSet.Copy() + valSetExpected := valSet.Copy() valList := createNewValidatorList(tt.updateVals) valListCopy := validatorListCopy(valList) err := valSet.UpdateWithChangeSet(valList, bls12381.GenPrivKey().PubKey(), crypto.RandQuorumHash()) @@ -787,7 +482,7 @@ func executeValSetErrTestCaseIgnoreThresholdPublicKey(t *testing.T, idx int, tt if assert.Error(t, err, "test %d", idx) { assert.Contains(t, err.Error(), tt.errString) } - assert.Equal(t, valSet, valSetCopy, "test %v", idx) + assert.Equal(t, valSetExpected, valSet, "test %v", idx) // check the parameter list has not changed assert.Equal(t, valList, valListCopy, "test %v", idx) @@ -873,7 +568,9 @@ func TestValSetUpdatesDuplicateEntries(t *testing.T) { } for i, tt := range testCases { - executeValSetErrTestCaseIgnoreThresholdPublicKey(t, i, tt) + t.Run(strconv.Itoa(i), func(t *testing.T) { + executeValSetErrTestCaseIgnoreThresholdPublicKey(t, i, tt) + }) } } @@ -1074,223 +771,16 @@ func TestValSetApplyUpdatesTestsExecute(t *testing.T) { } } -type testVSetCfg struct { - startVals []testVal - deletedVals []testVal - updatedVals []testVal - addedVals []testVal - expectedVals []testVal -} - -func randTestVSetCfg(t *testing.T, nBase, nAddMax int) testVSetCfg { - if nBase <= 0 || nAddMax < 0 { - t.Fatalf("bad parameters %v %v", nBase, nAddMax) - } - - var nOld, nDel, nChanged, nAdd int - - nOld = int(uint(rand.Int())%uint(nBase)) + 1 - if nBase-nOld > 0 { - nDel = int(uint(rand.Int()) % uint(nBase-nOld)) - } - nChanged = nBase - nOld - nDel - - if nAddMax > 0 { - nAdd = rand.Int()%nAddMax + 1 - } - - cfg := testVSetCfg{} - - cfg.startVals = make([]testVal, nBase) - cfg.deletedVals = make([]testVal, nDel) - cfg.addedVals = make([]testVal, nAdd) - cfg.updatedVals = make([]testVal, nChanged) - cfg.expectedVals = make([]testVal, nBase-nDel+nAdd) - - for i := 0; i < nBase; i++ { - cfg.startVals[i] = testVal{fmt.Sprintf("v%d", i), DefaultDashVotingPower} - if i < nOld { - cfg.expectedVals[i] = cfg.startVals[i] - } - if i >= nOld && i < nOld+nChanged { - cfg.updatedVals[i-nOld] = testVal{fmt.Sprintf("v%d", i), DefaultDashVotingPower} - cfg.expectedVals[i] = cfg.updatedVals[i-nOld] - } - if i >= nOld+nChanged { - cfg.deletedVals[i-nOld-nChanged] = testVal{fmt.Sprintf("v%d", i), 0} - } - } - - for i := nBase; i < nBase+nAdd; i++ { - cfg.addedVals[i-nBase] = testVal{fmt.Sprintf("v%d", i), DefaultDashVotingPower} - cfg.expectedVals[i-nDel] = cfg.addedVals[i-nBase] - } - - sort.Sort(testValsByVotingPower(cfg.startVals)) - sort.Sort(testValsByVotingPower(cfg.deletedVals)) - sort.Sort(testValsByVotingPower(cfg.updatedVals)) - sort.Sort(testValsByVotingPower(cfg.addedVals)) - sort.Sort(testValsByVotingPower(cfg.expectedVals)) - - return cfg - -} - -func applyChangesToValSet(t *testing.T, expErr error, valSet *ValidatorSet, valsLists ...[]testVal) { - changes := make([]testVal, 0) - for _, valsList := range valsLists { - changes = append(changes, valsList...) - } - valList, thresholdPublicKey := addValidatorsToValidatorSet(valSet, changes) - err := valSet.UpdateWithChangeSet(valList, thresholdPublicKey, crypto.RandQuorumHash()) - if expErr != nil { - assert.Equal(t, expErr, err) - } else { - assert.NoError(t, err) - } -} - -func TestValSetUpdatePriorityOrderTests(t *testing.T) { - if testing.Short() { - t.Skip("skipping test in short mode") - } - - const nMaxElections int32 = 5000 - - testCases := []testVSetCfg{ - 0: { // remove high power validator, keep old equal lower power validators - startVals: []testVal{{"v3", DefaultDashVotingPower}, {"v1", DefaultDashVotingPower}, {"v2", DefaultDashVotingPower}}, - deletedVals: []testVal{{"v3", 0}}, - updatedVals: []testVal{}, - addedVals: []testVal{}, - expectedVals: []testVal{{"v1", DefaultDashVotingPower}, {"v2", DefaultDashVotingPower}}, - }, - 1: { // remove high power validator, keep old different power validators - startVals: []testVal{{"v3", DefaultDashVotingPower}, {"v2", DefaultDashVotingPower}, {"v1", DefaultDashVotingPower}}, - deletedVals: []testVal{{"v3", 0}}, - updatedVals: []testVal{}, - addedVals: []testVal{}, - expectedVals: []testVal{{"v1", DefaultDashVotingPower}, {"v2", DefaultDashVotingPower}}, - }, - 2: { // remove high power validator, add new low power validators, keep old lower power - startVals: []testVal{{"v3", DefaultDashVotingPower}, {"v2", DefaultDashVotingPower}, {"v1", DefaultDashVotingPower}}, - deletedVals: []testVal{{"v3", 0}}, - updatedVals: []testVal{{"v2", DefaultDashVotingPower}}, - addedVals: []testVal{{"v4", DefaultDashVotingPower}, {"v5", DefaultDashVotingPower}}, - expectedVals: []testVal{{"v1", DefaultDashVotingPower}, {"v4", DefaultDashVotingPower}, {"v5", DefaultDashVotingPower}, {"v2", DefaultDashVotingPower}}, - }, - - // generate a configuration with 100 validators, - // randomly select validators for updates and deletes, and - // generate 10 new validators to be added - 3: randTestVSetCfg(t, 100, 10), - // - 4: randTestVSetCfg(t, 1000, 100), - // - 5: randTestVSetCfg(t, 10, 100), - // - 6: randTestVSetCfg(t, 100, 1000), - // - 7: randTestVSetCfg(t, 1000, 1000), - } - - for i, cfg := range testCases { - - // create a new validator set - valSet := createNewValidatorSet(cfg.startVals) - verifyValidatorSet(t, valSet) - - // run election up to nMaxElections times, apply changes and verify that the priority order is correct - verifyValSetUpdatePriorityOrder(t, valSet, cfg, nMaxElections, i) - } -} - -func verifyValSetUpdatePriorityOrder(t *testing.T, valSet *ValidatorSet, cfg testVSetCfg, nMaxElections int32, testNumber int) { - // Run election up to nMaxElections times, sort validators by priorities - valSet.IncrementProposerPriority(rand.Int31()%nMaxElections + 1) - - // apply the changes, get the updated validators, sort by priorities - applyChangesToValSet(t, nil, valSet, cfg.addedVals, cfg.updatedVals, cfg.deletedVals) - - // basic checks - testValSet := toTestProTxHashValList(valSet.Validators) - assert.Equal(t, switchToTestProTxHashValList(cfg.expectedVals), testValSet, "(0) test number %d", testNumber) - verifyValidatorSet(t, valSet) - - // verify that the added validators have the smallest priority: - // - they should be at the beginning of updatedValsPriSorted since it is - // sorted by priority - if len(cfg.addedVals) > 0 { - updatedValsPriSorted := validatorListCopy(valSet.Validators) - sort.Sort(validatorsByPriority(updatedValsPriSorted)) - - addedValsPriSlice := updatedValsPriSorted[:len(cfg.addedVals)] - sort.Sort(ValidatorsByVotingPower(addedValsPriSlice)) - assert.Equal(t, switchToTestProTxHashValList(cfg.addedVals), toTestProTxHashValList(addedValsPriSlice), "(1) test number %d", testNumber) - - // - and should all have the same priority - expectedPri := addedValsPriSlice[0].ProposerPriority - for _, val := range addedValsPriSlice[1:] { - assert.Equal(t, expectedPri, val.ProposerPriority, "(2) test number %d", testNumber) - } - } -} - -func TestNewValidatorSetFromExistingValidators(t *testing.T) { - size := 5 - valSet, _ := GenerateValidatorSet(NewValSetParam(crypto.RandProTxHashes(size))) - valSet.IncrementProposerPriority(3) - - newValSet0 := NewValidatorSet(valSet.Validators, valSet.ThresholdPublicKey, valSet.QuorumType, valSet.QuorumHash, true) - assert.NotEqual(t, valSet, newValSet0) - - valSet.IncrementProposerPriority(2) - newValSet1 := NewValidatorSet(valSet.Validators, valSet.ThresholdPublicKey, valSet.QuorumType, valSet.QuorumHash, true) - assert.Equal(t, valSet, newValSet1) - - existingValSet, err := ValidatorSetFromExistingValidators(valSet.Validators, valSet.ThresholdPublicKey, - valSet.QuorumType, valSet.QuorumHash) - assert.NoError(t, err) - assert.Equal(t, valSet, existingValSet) - assert.Equal(t, valSet.CopyIncrementProposerPriority(3), existingValSet.CopyIncrementProposerPriority(3)) -} - -func TestSafeMul(t *testing.T) { - testCases := []struct { - a int64 - b int64 - c int64 - overflow bool - }{ - 0: {0, 0, 0, false}, - 1: {1, 0, 0, false}, - 2: {2, 3, 6, false}, - 3: {2, -3, -6, false}, - 4: {-2, -3, 6, false}, - 5: {-2, 3, -6, false}, - 6: {math.MaxInt64, 1, math.MaxInt64, false}, - 7: {math.MaxInt64 / 2, 2, math.MaxInt64 - 1, false}, - 8: {math.MaxInt64 / 2, 3, 0, true}, - 9: {math.MaxInt64, 2, 0, true}, - } - - for i, tc := range testCases { - c, overflow := safeMul(tc.a, tc.b) - assert.Equal(t, tc.c, c, "#%d", i) - assert.Equal(t, tc.overflow, overflow, "#%d", i) - } -} - func TestValidatorSetProtoBuf(t *testing.T) { valset, _ := RandValidatorSet(10) valset2, _ := RandValidatorSet(10) valset2.Validators[0] = &Validator{} valset3, _ := RandValidatorSet(10) - valset3.Proposer = nil + valset3.Validators[0] = nil valset4, _ := RandValidatorSet(10) - valset4.Proposer = &Validator{} + valset4.Validators[0] = &Validator{} testCases := []struct { msg string @@ -1323,50 +813,6 @@ func TestValidatorSetProtoBuf(t *testing.T) { } } -// --------------------- -// Sort validators by priority and address -type validatorsByPriority []*Validator - -func (valz validatorsByPriority) Len() int { - return len(valz) -} - -func (valz validatorsByPriority) Less(i, j int) bool { - if valz[i].ProposerPriority < valz[j].ProposerPriority { - return true - } - if valz[i].ProposerPriority > valz[j].ProposerPriority { - return false - } - return bytes.Compare(valz[i].ProTxHash, valz[j].ProTxHash) < 0 -} - -func (valz validatorsByPriority) Swap(i, j int) { - valz[i], valz[j] = valz[j], valz[i] -} - -//------------------------------------- - -type testValsByVotingPower []testVal - -func (tvals testValsByVotingPower) Len() int { - return len(tvals) -} - -// Here we need to sort by the pro_tx_hash and not the name if the power is equal, in the test the pro_tx_hash is derived -// -// from the name by applying a single SHA256 -func (tvals testValsByVotingPower) Less(i, j int) bool { - if tvals[i].power == tvals[j].power { - return bytes.Compare(crypto.Checksum([]byte(tvals[i].name)), crypto.Checksum([]byte(tvals[j].name))) == -1 - } - return tvals[i].power > tvals[j].power -} - -func (tvals testValsByVotingPower) Swap(i, j int) { - tvals[i], tvals[j] = tvals[j], tvals[i] -} - // ------------------------------------- // Benchmark tests func BenchmarkUpdates(b *testing.B) { diff --git a/types/vote_test.go b/types/vote_test.go index b7d413a373..044024b0d2 100644 --- a/types/vote_test.go +++ b/types/vote_test.go @@ -236,7 +236,7 @@ func TestVoteVerifySignature(t *testing.T) { testCases := []testCase{ { name: "correct", - modify: func(v *tmproto.Vote) {}, + modify: func(_ *tmproto.Vote) {}, expectValid: true, }, { @@ -430,7 +430,7 @@ func TestIsVoteTypeValid(t *testing.T) { for _, tt := range tc { tt := tt - t.Run(tt.name, func(st *testing.T) { + t.Run(tt.name, func(_st *testing.T) { if rs := IsVoteTypeValid(tt.in); rs != tt.out { t.Errorf("got unexpected Vote type. Expected:\n%v\nGot:\n%v", rs, tt.out) } @@ -455,8 +455,6 @@ func TestVoteVerify(t *testing.T) { vote := examplePrevote(t) vote.ValidatorProTxHash = proTxHash - stateID := RandStateID() - stateID.Height = uint64(vote.Height - 1) pubKey := bls12381.GenPrivKey().PubKey() err = vote.Verify("test_chain_id", quorumType, quorumHash, pubKey, crypto.RandProTxHash()) @@ -531,7 +529,7 @@ func TestValidVotes(t *testing.T) { vote *Vote malleateVote func(*Vote) }{ - {"good prevote", examplePrevote(t), func(v *Vote) {}}, + {"good prevote", examplePrevote(t), func(_ *Vote) {}}, {"good precommit without vote extension", examplePrecommit(t), func(v *Vote) { v.VoteExtensions = nil }}, { "good precommit with vote extension", diff --git a/version/version.go b/version/version.go index 9a488a2623..1782ff03ad 100644 --- a/version/version.go +++ b/version/version.go @@ -9,9 +9,9 @@ var ( const ( // TMVersionDefault is the used as the fallback version for Tenderdash // when not using git describe. It is formatted with semantic versioning. - TMVersionDefault = "1.2.1" + TMVersionDefault = "1.3.0" // ABCISemVer is the semantic version of the ABCI library - ABCISemVer = "1.1.0" + ABCISemVer = "1.2.0" ABCIVersion = ABCISemVer )