From 06b1812094dba6086f53ec385ed6238ffbc255fa Mon Sep 17 00:00:00 2001 From: Thane Thomson Date: Wed, 11 May 2022 07:10:32 -0400 Subject: [PATCH] abci++: Propagate vote extensions (RFC 017) (#8433) * Add protos for ExtendedCommit Cherry-pick from e73f0178b72a16ee81f8e856aadf651f2c62ec6e just the changes to the .proto files, since we have deleted the .intermediate files. Signed-off-by: Thane Thomson * make proto-gen Signed-off-by: Thane Thomson * BlockStore holds extended commit Cherry-pick 8d504d4b50ec6afbdffe2df7ababbef30e15053d and fix conflicts. Signed-off-by: Thane Thomson * Reshuffle ExtendedCommit and ExtendedCommitSig Separate the data structures and functions from their Commit-oriented counterparts to adhere to the current coding style. Signed-off-by: Thane Thomson * Fix exit condition in blocksync * Add note to remove TxResult proto As Sergio pointed out in 3e31aa6f583cdc71e208ed03a82f1d804ec0de49, this proto message can probably be removed. We should do this in a separate PR. Signed-off-by: Thane Thomson * Lift termination condition into for loop Signed-off-by: Thane Thomson * Enforce vote extension signature requirement Signed-off-by: Thane Thomson * Expand on comment for PeekTwoBlocks for posterity Signed-off-by: Thane Thomson * Isolate TODO more clearly Signed-off-by: Thane Thomson * make mockery Signed-off-by: Thane Thomson * Fix comment Signed-off-by: Thane Thomson * Make panic output from BlockStore.SaveBlock more readable Signed-off-by: Thane Thomson * Add helper methods to ExtendedCommitSig and ExtendedCommit Signed-off-by: Thane Thomson * Fix most tests except TestHandshake* Signed-off-by: Thane Thomson * Fix store prefix collision Signed-off-by: Thane Thomson * Fix TestBlockFetchAtHeight Signed-off-by: Thane Thomson * Remove global state from store tests Signed-off-by: Thane Thomson * Apply suggestions from code review Co-authored-by: M. J. Fromberger Co-authored-by: Sergio Mena * blocksync: Just return error Signed-off-by: Thane Thomson * make format Signed-off-by: Thane Thomson * types: Remove unused/commented-out code Signed-off-by: Thane Thomson * blocksync: Change pool AddBlock function signature to return errors Signed-off-by: Thane Thomson * types: Improve legibility of switch statements Signed-off-by: Thane Thomson * blocksync: Expand on extended commit requirement in AddBlock description Signed-off-by: Thane Thomson * blocksync: Return error without also logging it Signed-off-by: Thane Thomson * consensus: Rename short-lived local variable Signed-off-by: Thane Thomson * consensus: Allocate TODO to Sergio Signed-off-by: Thane Thomson * evidence/pool_test: Inline slice construction Signed-off-by: Thane Thomson * state: Rename LoadBlockExtCommit to LoadBlockExtendedCommit Signed-off-by: Thane Thomson * proto: Remove TODO on TxResult Signed-off-by: Thane Thomson * types: Minor format Signed-off-by: Thane Thomson * types: Reformat ExtendedCommitSig.BlockID Signed-off-by: Thane Thomson * types: Remove NewExtendedCommit constructor Signed-off-by: Thane Thomson * types: Remove NewCommit constructor Signed-off-by: Thane Thomson * types: Shorten receiver names for ExtendedCommit Signed-off-by: Thane Thomson * types: Convert ExtendedCommit.Copy to a deep clone Signed-off-by: Thane Thomson * types: Assign TODO to Sergio Signed-off-by: Thane Thomson * types: Fix legibility nits Signed-off-by: Thane Thomson * types: Improve legibility Signed-off-by: Thane Thomson * store/state: Add TODO to move prefixes to common package Signed-off-by: Thane Thomson * Propagate validator info to PrepareProposal In order to propagate validator voting power through to PrepareProposal, we need to load the validator set info from the height corresponding to the extended commit that we're passing through to PrepareProposal as the "LocalLastCommit". Signed-off-by: Thane Thomson * Rename local var for clarity Signed-off-by: Thane Thomson * Fix TestMaxProposalBlockSize Signed-off-by: Thane Thomson * Rename local var for clarity Signed-off-by: Thane Thomson * Remove debug log Signed-off-by: Thane Thomson * Remove CommigSig.ForBlock helper Signed-off-by: Thane Thomson * Remove CommigSig.Absent helper Signed-off-by: Thane Thomson * Remove ExtendedCommitSig.ForBlock helper Signed-off-by: Thane Thomson * Remove ExtendedCommitSig.Absent helper Signed-off-by: Thane Thomson * There are no extended commits below the initial height Signed-off-by: Thane Thomson * Fix comment grammar Signed-off-by: Thane Thomson * Remove JSON encoding from ExtendedCommit Signed-off-by: Thane Thomson * Embed CommitSig into ExtendedCommitSig instead of duplicating fields Signed-off-by: Thane Thomson * Rename ExtendedCommit vote_extension field to extension for consistency with domain types Signed-off-by: Thane Thomson * blocksync: Panic if we peek a block without an extended commit Signed-off-by: Thane Thomson * Apply suggestions from code review Co-authored-by: M. J. Fromberger * Remove Sergio from TODO Signed-off-by: Thane Thomson * Increase hard-coded vote extension max size to 1MB Signed-off-by: Thane Thomson * state: Remove unnecessary comment Signed-off-by: Thane Thomson * state: Ensure no of commit sigs equals validator set length Signed-off-by: Thane Thomson * make format Signed-off-by: Thane Thomson * types: Minor legibility improvements Signed-off-by: Thane Thomson * Improve legibility Signed-off-by: Thane Thomson * types: Remove unused GetVotes function on VoteSet Signed-off-by: Thane Thomson * Refactor TestMaxProposalBlockSize to construct more realistic extended commit Signed-off-by: Thane Thomson * Refactor buildExtendedCommitInfo to resemble buildLastCommitInfo Signed-off-by: Thane Thomson * Apply suggestions from code review Co-authored-by: M. J. Fromberger * abci++: Disable VerifyVoteExtension call on nil precommits (#8491) Signed-off-by: Thane Thomson * types: Require vote extensions on non-nil precommits and not otherwise Signed-off-by: Thane Thomson * Disable lint Signed-off-by: Thane Thomson * Increase timeout for TestReactorVotingPowerChange to counter flakiness Signed-off-by: Thane Thomson * Only sign and verify vote extensions in non-nil precommits Signed-off-by: Thane Thomson * Revert "Disable lint" This reverts commit 6fffbf94028a1ae78289abbad1b602c251f6f652. Signed-off-by: Thane Thomson * Add missing non-nil check uncovered non-deterministically in TestHandshakeReplayAll Signed-off-by: Thane Thomson * Expand error message for accuracy Signed-off-by: Thane Thomson * Only call ExtendVote when we make non-nil precommits Signed-off-by: Thane Thomson * Revert "Increase timeout for TestReactorVotingPowerChange to counter flakiness" This reverts commit af514939dbdf72ce275ef290a34c390a5e982563. Signed-off-by: Thane Thomson * Refactor ValidateBasic for ExtendedCommitSig for legibility Signed-off-by: Thane Thomson Co-authored-by: Sergio Mena Co-authored-by: M. J. Fromberger --- internal/blocksync/pool.go | 58 +- internal/blocksync/pool_test.go | 9 +- internal/blocksync/reactor.go | 54 +- internal/blocksync/reactor_test.go | 52 +- internal/consensus/byzantine_test.go | 10 +- internal/consensus/common_test.go | 3 +- internal/consensus/mocks/cons_sync_reactor.go | 1 + internal/consensus/msgs_test.go | 2 +- internal/consensus/reactor.go | 6 +- internal/consensus/replay_test.go | 70 +- internal/consensus/state.go | 39 +- internal/consensus/state_test.go | 14 +- internal/evidence/pool_test.go | 31 +- internal/evidence/verify_test.go | 18 +- internal/state/execution.go | 100 +- internal/state/execution_test.go | 64 +- internal/state/helpers_test.go | 14 +- internal/state/indexer/mocks/event_sink.go | 1 + internal/state/mocks/block_store.go | 18 +- internal/state/mocks/evidence_pool.go | 1 + internal/state/mocks/store.go | 1 + internal/state/services.go | 3 +- internal/state/store.go | 3 + internal/state/test/factory/block.go | 10 +- internal/state/validation_test.go | 42 +- internal/statesync/mocks/state_provider.go | 1 + internal/statesync/reactor_test.go | 4 +- internal/store/store.go | 50 +- internal/store/store_test.go | 140 ++- internal/test/factory/commit.go | 4 +- light/helpers_test.go | 7 +- node/node_test.go | 54 +- privval/file.go | 8 +- proto/tendermint/blocksync/types.pb.go | 113 ++- proto/tendermint/blocksync/types.proto | 2 + proto/tendermint/types/types.pb.go | 919 ++++++++++++++++-- proto/tendermint/types/types.proto | 22 + scripts/confix/confix.go | 1 + test/e2e/runner/evidence.go | 4 +- types/block.go | 404 ++++++-- types/block_test.go | 49 +- types/evidence.go | 6 +- types/evidence_test.go | 12 +- types/part_set.go | 6 + types/priv_validator.go | 8 +- types/test_util.go | 16 +- types/validation.go | 8 +- types/validation_test.go | 29 +- types/validator_set_test.go | 9 +- types/vote.go | 46 +- types/vote_set.go | 36 +- types/vote_set_test.go | 8 +- types/vote_test.go | 30 +- 53 files changed, 1928 insertions(+), 692 deletions(-) diff --git a/internal/blocksync/pool.go b/internal/blocksync/pool.go index f00a2fab5..30bb6962e 100644 --- a/internal/blocksync/pool.go +++ b/internal/blocksync/pool.go @@ -200,16 +200,20 @@ func (pool *BlockPool) IsCaughtUp() bool { return pool.height >= (pool.maxPeerHeight - 1) } -// PeekTwoBlocks returns blocks at pool.height and pool.height+1. -// We need to see the second block's Commit to validate the first block. -// So we peek two blocks at a time. +// PeekTwoBlocks returns blocks at pool.height and pool.height+1. We need to +// see the second block's Commit to validate the first block. So we peek two +// blocks at a time. We return an extended commit, containing vote extensions +// and their associated signatures, as this is critical to consensus in ABCI++ +// as we switch from block sync to consensus mode. +// // The caller will verify the commit. -func (pool *BlockPool) PeekTwoBlocks() (first *types.Block, second *types.Block) { +func (pool *BlockPool) PeekTwoBlocks() (first, second *types.Block, firstExtCommit *types.ExtendedCommit) { pool.mtx.RLock() defer pool.mtx.RUnlock() if r := pool.requesters[pool.height]; r != nil { first = r.getBlock() + firstExtCommit = r.getExtendedCommit() } if r := pool.requesters[pool.height+1]; r != nil { second = r.getBlock() @@ -218,7 +222,8 @@ func (pool *BlockPool) PeekTwoBlocks() (first *types.Block, second *types.Block) } // PopRequest pops the first block at pool.height. -// It must have been validated by 'second'.Commit from PeekTwoBlocks(). +// It must have been validated by the second Commit from PeekTwoBlocks. +// TODO(thane): (?) and its corresponding ExtendedCommit. func (pool *BlockPool) PopRequest() { pool.mtx.Lock() defer pool.mtx.Unlock() @@ -262,16 +267,25 @@ func (pool *BlockPool) RedoRequest(height int64) types.NodeID { return peerID } -// AddBlock validates that the block comes from the peer it was expected from and calls the requester to store it. +// AddBlock validates that the block comes from the peer it was expected from +// and calls the requester to store it. +// +// This requires an extended commit at the same height as the supplied block - +// the block contains the last commit, but we need the latest commit in case we +// need to switch over from block sync to consensus at this height. If the +// height of the extended commit and the height of the block do not match, we +// do not add the block and return an error. // TODO: ensure that blocks come in order for each peer. -func (pool *BlockPool) AddBlock(peerID types.NodeID, block *types.Block, blockSize int) { +func (pool *BlockPool) AddBlock(peerID types.NodeID, block *types.Block, extCommit *types.ExtendedCommit, blockSize int) error { pool.mtx.Lock() defer pool.mtx.Unlock() + if block.Height != extCommit.Height { + return fmt.Errorf("heights don't match, not adding block (block height: %d, commit height: %d)", block.Height, extCommit.Height) + } + requester := pool.requesters[block.Height] if requester == nil { - pool.logger.Error("peer sent us a block we didn't expect", - "peer", peerID, "curHeight", pool.height, "blockHeight", block.Height) diff := pool.height - block.Height if diff < 0 { diff *= -1 @@ -279,10 +293,10 @@ func (pool *BlockPool) AddBlock(peerID types.NodeID, block *types.Block, blockSi if diff > maxDiffBetweenCurrentAndReceivedBlockHeight { pool.sendError(errors.New("peer sent us a block we didn't expect with a height too far ahead/behind"), peerID) } - return + return fmt.Errorf("peer sent us a block we didn't expect (peer: %s, current height: %d, block height: %d)", peerID, pool.height, block.Height) } - if requester.setBlock(block, peerID) { + if requester.setBlock(block, extCommit, peerID) { atomic.AddInt32(&pool.numPending, -1) peer := pool.peers[peerID] if peer != nil { @@ -290,9 +304,11 @@ func (pool *BlockPool) AddBlock(peerID types.NodeID, block *types.Block, blockSi } } else { err := errors.New("requester is different or block already exists") - pool.logger.Error(err.Error(), "peer", peerID, "requester", requester.getPeerID(), "blockHeight", block.Height) pool.sendError(err, peerID) + return fmt.Errorf("%w (peer: %s, requester: %s, block height: %d)", err, peerID, requester.getPeerID(), block.Height) } + + return nil } // MaxPeerHeight returns the highest reported height. @@ -456,6 +472,7 @@ func (pool *BlockPool) debug() string { } else { str += fmt.Sprintf("H(%v):", h) str += fmt.Sprintf("B?(%v) ", pool.requesters[h].block != nil) + str += fmt.Sprintf("C?(%v) ", pool.requesters[h].extCommit != nil) } } return str @@ -544,9 +561,10 @@ type bpRequester struct { gotBlockCh chan struct{} redoCh chan types.NodeID // redo may send multitime, add peerId to identify repeat - mtx sync.Mutex - peerID types.NodeID - block *types.Block + mtx sync.Mutex + peerID types.NodeID + block *types.Block + extCommit *types.ExtendedCommit } func newBPRequester(logger log.Logger, pool *BlockPool, height int64) *bpRequester { @@ -572,13 +590,14 @@ func (bpr *bpRequester) OnStart(ctx context.Context) error { func (*bpRequester) OnStop() {} // Returns true if the peer matches and block doesn't already exist. -func (bpr *bpRequester) setBlock(block *types.Block, peerID types.NodeID) bool { +func (bpr *bpRequester) setBlock(block *types.Block, extCommit *types.ExtendedCommit, peerID types.NodeID) bool { bpr.mtx.Lock() if bpr.block != nil || bpr.peerID != peerID { bpr.mtx.Unlock() return false } bpr.block = block + bpr.extCommit = extCommit bpr.mtx.Unlock() select { @@ -594,6 +613,12 @@ func (bpr *bpRequester) getBlock() *types.Block { return bpr.block } +func (bpr *bpRequester) getExtendedCommit() *types.ExtendedCommit { + bpr.mtx.Lock() + defer bpr.mtx.Unlock() + return bpr.extCommit +} + func (bpr *bpRequester) getPeerID() types.NodeID { bpr.mtx.Lock() defer bpr.mtx.Unlock() @@ -611,6 +636,7 @@ func (bpr *bpRequester) reset() { bpr.peerID = "" bpr.block = nil + bpr.extCommit = nil } // Tells bpRequester to pick another peer and try again. diff --git a/internal/blocksync/pool_test.go b/internal/blocksync/pool_test.go index 1cb8cca40..3c47b4a64 100644 --- a/internal/blocksync/pool_test.go +++ b/internal/blocksync/pool_test.go @@ -43,7 +43,10 @@ func (p testPeer) runInputRoutine() { // Request desired, pretend like we got the block immediately. func (p testPeer) simulateInput(input inputData) { block := &types.Block{Header: types.Header{Height: input.request.Height}} - input.pool.AddBlock(input.request.PeerID, block, 123) + extCommit := &types.ExtendedCommit{ + Height: input.request.Height, + } + _ = input.pool.AddBlock(input.request.PeerID, block, extCommit, 123) // TODO: uncommenting this creates a race which is detected by: // https://github.com/golang/go/blob/2bd767b1022dd3254bcec469f0ee164024726486/src/testing/testing.go#L854-L856 // see: https://github.com/tendermint/tendermint/issues/3390#issue-418379890 @@ -110,7 +113,7 @@ func TestBlockPoolBasic(t *testing.T) { if !pool.IsRunning() { return } - first, second := pool.PeekTwoBlocks() + first, second, _ := pool.PeekTwoBlocks() if first != nil && second != nil { pool.PopRequest() } else { @@ -164,7 +167,7 @@ func TestBlockPoolTimeout(t *testing.T) { if !pool.IsRunning() { return } - first, second := pool.PeekTwoBlocks() + first, second, _ := pool.PeekTwoBlocks() if first != nil && second != nil { pool.PopRequest() } else { diff --git a/internal/blocksync/reactor.go b/internal/blocksync/reactor.go index bf9845370..144595889 100644 --- a/internal/blocksync/reactor.go +++ b/internal/blocksync/reactor.go @@ -76,7 +76,7 @@ type Reactor struct { stateStore sm.Store blockExec *sm.BlockExecutor - store *store.BlockStore + store sm.BlockStore pool *BlockPool consReactor consensusReactor blockSync *atomicBool @@ -186,15 +186,21 @@ func (r *Reactor) OnStop() { func (r *Reactor) respondToPeer(ctx context.Context, msg *bcproto.BlockRequest, peerID types.NodeID, blockSyncCh *p2p.Channel) error { block := r.store.LoadBlock(msg.Height) if block != nil { + extCommit := r.store.LoadBlockExtendedCommit(msg.Height) + if extCommit == nil { + return fmt.Errorf("found block in store without extended commit: %v", block) + } blockProto, err := block.ToProto() if err != nil { - r.logger.Error("failed to convert msg to protobuf", "err", err) - return err + return fmt.Errorf("failed to convert block to protobuf: %w", err) } return blockSyncCh.Send(ctx, p2p.Envelope{ - To: peerID, - Message: &bcproto.BlockResponse{Block: blockProto}, + To: peerID, + Message: &bcproto.BlockResponse{ + Block: blockProto, + ExtCommit: extCommit.ToProto(), + }, }) } @@ -236,8 +242,17 @@ func (r *Reactor) handleMessage(ctx context.Context, envelope *p2p.Envelope, blo "err", err) return err } + extCommit, err := types.ExtendedCommitFromProto(msg.ExtCommit) + if err != nil { + r.logger.Error("failed to convert extended commit from proto", + "peer", envelope.From, + "err", err) + return err + } - r.pool.AddBlock(envelope.From, block, block.Size()) + if err := r.pool.AddBlock(envelope.From, block, extCommit, block.Size()); err != nil { + r.logger.Error("failed to add block", "err", err) + } case *bcproto.StatusRequest: return blockSyncCh.Send(ctx, p2p.Envelope{ @@ -448,6 +463,20 @@ func (r *Reactor) poolRoutine(ctx context.Context, stateSynced bool, blockSyncCh ) switch { + // TODO(sergio) Might be needed for implementing the upgrading solution. Remove after that + //case state.LastBlockHeight > 0 && r.store.LoadBlockExtCommit(state.LastBlockHeight) == nil: + case state.LastBlockHeight > 0 && blocksSynced == 0: + // Having state-synced, we need to blocksync at least one block + r.logger.Info( + "no seen commit yet", + "height", height, + "last_block_height", state.LastBlockHeight, + "initial_height", state.InitialHeight, + "max_peer_height", r.pool.MaxPeerHeight(), + "timeout_in", syncTimeout-time.Since(lastAdvance), + ) + continue + case r.pool.IsCaughtUp(): r.logger.Info("switching to consensus reactor", "height", height) @@ -490,9 +519,13 @@ func (r *Reactor) poolRoutine(ctx context.Context, stateSynced bool, blockSyncCh // TODO: Uncouple from request routine. // see if there are any blocks to sync - first, second := r.pool.PeekTwoBlocks() - if first == nil || second == nil { - // we need both to sync the first block + first, second, extCommit := r.pool.PeekTwoBlocks() + if first == nil || second == nil || extCommit == nil { + if first != nil && extCommit == nil { + // See https://github.com/tendermint/tendermint/pull/8433#discussion_r866790631 + panic(fmt.Errorf("peeked first block without extended commit at height %d - possible node store corruption", first.Height)) + } + // we need all to sync the first block continue } else { // try again quickly next loop @@ -517,6 +550,7 @@ func (r *Reactor) poolRoutine(ctx context.Context, stateSynced bool, blockSyncCh // NOTE: We can probably make this more efficient, but note that calling // first.Hash() doesn't verify the tx contents, so MakePartSet() is // currently necessary. + // TODO(sergio): Should we also validate against the extended commit? err = state.Validators.VerifyCommitLight(chainID, firstID, first.Height, second.LastCommit) if err == nil { @@ -559,7 +593,7 @@ func (r *Reactor) poolRoutine(ctx context.Context, stateSynced bool, blockSyncCh r.pool.PopRequest() // TODO: batch saves so we do not persist to disk every block - r.store.SaveBlock(first, firstParts, second.LastCommit) + r.store.SaveBlock(first, firstParts, extCommit) // TODO: Same thing for app - but we would need a way to get the hash // without persisting the state. diff --git a/internal/blocksync/reactor_test.go b/internal/blocksync/reactor_test.go index 857b0a519..1d4d7d4d6 100644 --- a/internal/blocksync/reactor_test.go +++ b/internal/blocksync/reactor_test.go @@ -147,39 +147,43 @@ func (rts *reactorTestSuite) addNode( sm.NopMetrics(), ) + var lastExtCommit *types.ExtendedCommit + + // The commit we are building for the current height. + seenExtCommit := &types.ExtendedCommit{} + for blockHeight := int64(1); blockHeight <= maxBlockHeight; blockHeight++ { - lastCommit := types.NewCommit(blockHeight-1, 0, types.BlockID{}, nil) + lastExtCommit = seenExtCommit.Clone() - if blockHeight > 1 { - lastBlockMeta := blockStore.LoadBlockMeta(blockHeight - 1) - lastBlock := blockStore.LoadBlock(blockHeight - 1) - - vote, err := factory.MakeVote( - ctx, - privVal, - lastBlock.Header.ChainID, 0, - lastBlock.Header.Height, 0, 2, - lastBlockMeta.BlockID, - time.Now(), - ) - require.NoError(t, err) - lastCommit = types.NewCommit( - vote.Height, - vote.Round, - lastBlockMeta.BlockID, - []types.CommitSig{vote.CommitSig()}, - ) - } - - thisBlock := sf.MakeBlock(state, blockHeight, lastCommit) + thisBlock := sf.MakeBlock(state, blockHeight, lastExtCommit.StripExtensions()) thisParts, err := thisBlock.MakePartSet(types.BlockPartSizeBytes) require.NoError(t, err) blockID := types.BlockID{Hash: thisBlock.Hash(), PartSetHeader: thisParts.Header()} + // Simulate a commit for the current height + vote, err := factory.MakeVote( + ctx, + privVal, + thisBlock.Header.ChainID, + 0, + thisBlock.Header.Height, + 0, + 2, + blockID, + time.Now(), + ) + require.NoError(t, err) + seenExtCommit = &types.ExtendedCommit{ + Height: vote.Height, + Round: vote.Round, + BlockID: blockID, + ExtendedSignatures: []types.ExtendedCommitSig{vote.ExtendedCommitSig()}, + } + state, err = blockExec.ApplyBlock(ctx, state, blockID, thisBlock) require.NoError(t, err) - blockStore.SaveBlock(thisBlock, thisParts, lastCommit) + blockStore.SaveBlock(thisBlock, thisParts, seenExtCommit) } rts.peerChans[nodeID] = make(chan p2p.PeerUpdate) diff --git a/internal/consensus/byzantine_test.go b/internal/consensus/byzantine_test.go index 804ebdb18..9c6f4a295 100644 --- a/internal/consensus/byzantine_test.go +++ b/internal/consensus/byzantine_test.go @@ -178,22 +178,22 @@ func TestByzantinePrevoteEquivocation(t *testing.T) { lazyNodeState.decideProposal = func(ctx context.Context, height int64, round int32) { require.NotNil(t, lazyNodeState.privValidator) - var commit *types.Commit + var extCommit *types.ExtendedCommit switch { case lazyNodeState.Height == lazyNodeState.state.InitialHeight: // We're creating a proposal for the first block. // The commit is empty, but not nil. - commit = types.NewCommit(0, 0, types.BlockID{}, nil) + extCommit = &types.ExtendedCommit{} case lazyNodeState.LastCommit.HasTwoThirdsMajority(): // Make the commit from LastCommit - commit = lazyNodeState.LastCommit.MakeCommit() + extCommit = lazyNodeState.LastCommit.MakeExtendedCommit() default: // This shouldn't happen. lazyNodeState.logger.Error("enterPropose: Cannot propose anything: No commit for the previous block") return } // omit the last signature in the commit - commit.Signatures[len(commit.Signatures)-1] = types.NewCommitSigAbsent() + extCommit.ExtendedSignatures[len(extCommit.ExtendedSignatures)-1] = types.NewExtendedCommitSigAbsent() if lazyNodeState.privValidatorPubKey == nil { // If this node is a validator & proposer in the current round, it will @@ -204,7 +204,7 @@ func TestByzantinePrevoteEquivocation(t *testing.T) { proposerAddr := lazyNodeState.privValidatorPubKey.Address() block, err := lazyNodeState.blockExec.CreateProposalBlock( - ctx, lazyNodeState.Height, lazyNodeState.state, commit, proposerAddr, lazyNodeState.LastCommit.GetVotes()) + ctx, lazyNodeState.Height, lazyNodeState.state, extCommit, proposerAddr) require.NoError(t, err) blockParts, err := block.MakePartSet(types.BlockPartSizeBytes) require.NoError(t, err) diff --git a/internal/consensus/common_test.go b/internal/consensus/common_test.go index ca1db8425..1dc92b33c 100644 --- a/internal/consensus/common_test.go +++ b/internal/consensus/common_test.go @@ -160,7 +160,8 @@ func signVote( blockID types.BlockID) *types.Vote { var ext []byte - if voteType == tmproto.PrecommitType { + // Only non-nil precommits are allowed to carry vote extensions. + if voteType == tmproto.PrecommitType && !blockID.IsNil() { ext = []byte("extension") } v, err := vs.signVote(ctx, voteType, chainID, blockID, ext) diff --git a/internal/consensus/mocks/cons_sync_reactor.go b/internal/consensus/mocks/cons_sync_reactor.go index f904e9129..3aa02e9fe 100644 --- a/internal/consensus/mocks/cons_sync_reactor.go +++ b/internal/consensus/mocks/cons_sync_reactor.go @@ -6,6 +6,7 @@ import ( testing "testing" mock "github.com/stretchr/testify/mock" + state "github.com/tendermint/tendermint/internal/state" ) diff --git a/internal/consensus/msgs_test.go b/internal/consensus/msgs_test.go index 5a6465294..b8d18a109 100644 --- a/internal/consensus/msgs_test.go +++ b/internal/consensus/msgs_test.go @@ -67,7 +67,7 @@ func TestMsgToProto(t *testing.T) { pv := types.NewMockPV() vote, err := factory.MakeVote(ctx, pv, factory.DefaultTestChainID, - 0, 1, 0, 2, types.BlockID{}, time.Now()) + 0, 1, 0, 2, bi, time.Now()) require.NoError(t, err) pbVote := vote.ToProto() diff --git a/internal/consensus/reactor.go b/internal/consensus/reactor.go index eea74b5e1..1a9d49057 100644 --- a/internal/consensus/reactor.go +++ b/internal/consensus/reactor.go @@ -794,10 +794,10 @@ func (r *Reactor) gossipVotesRoutine(ctx context.Context, ps *PeerState, voteCh // catchup logic -- if peer is lagging by more than 1, send Commit blockStoreBase := r.state.blockStore.Base() if blockStoreBase > 0 && prs.Height != 0 && rs.Height >= prs.Height+2 && prs.Height >= blockStoreBase { - // Load the block commit for prs.Height, which contains precommit + // Load the block's extended commit for prs.Height, which contains precommit // signatures for prs.Height. - if commit := r.state.blockStore.LoadBlockCommit(prs.Height); commit != nil { - if ok, err := r.pickSendVote(ctx, ps, commit, voteCh); err != nil { + if ec := r.state.blockStore.LoadBlockExtendedCommit(prs.Height); ec != nil { + if ok, err := r.pickSendVote(ctx, ps, ec, voteCh); err != nil { return } else if ok { logger.Debug("picked Catchup commit to send", "height", prs.Height) diff --git a/internal/consensus/replay_test.go b/internal/consensus/replay_test.go index f112f23e8..c8f04655b 100644 --- a/internal/consensus/replay_test.go +++ b/internal/consensus/replay_test.go @@ -297,7 +297,7 @@ type simulatorTestSuite struct { GenesisState sm.State Config *config.Config Chain []*types.Block - Commits []*types.Commit + ExtCommits []*types.ExtendedCommit CleanupFunc cleanupFunc Mempool mempool.Mempool @@ -578,11 +578,11 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite { } ensureNewRound(t, newRoundCh, height+1, 0) - sim.Chain = make([]*types.Block, 0) - sim.Commits = make([]*types.Commit, 0) + sim.Chain = []*types.Block{} + sim.ExtCommits = []*types.ExtendedCommit{} for i := 1; i <= numBlocks; i++ { sim.Chain = append(sim.Chain, css[0].blockStore.LoadBlock(int64(i))) - sim.Commits = append(sim.Commits, css[0].blockStore.LoadBlockCommit(int64(i))) + sim.ExtCommits = append(sim.ExtCommits, css[0].blockStore.LoadBlockExtendedCommit(int64(i))) } return sim @@ -679,7 +679,7 @@ func testHandshakeReplay( testValidatorsChange bool, ) { var chain []*types.Block - var commits []*types.Commit + var extCommits []*types.ExtendedCommit var store *mockBlockStore var stateDB dbm.DB var genesisState sm.State @@ -699,7 +699,7 @@ func testHandshakeReplay( genesisState = sim.GenesisState cfg = sim.Config chain = append([]*types.Block{}, sim.Chain...) // copy chain - commits = sim.Commits + extCommits = sim.ExtCommits store = newMockBlockStore(t, cfg, genesisState.ConsensusParams) } else { // test single node testConfig, err := ResetConfig(t.TempDir(), fmt.Sprintf("%s_%v_s", t.Name(), mode)) @@ -718,7 +718,7 @@ func testHandshakeReplay( err = wal.Start(ctx) require.NoError(t, err) t.Cleanup(func() { cancel(); wal.Wait() }) - chain, commits = makeBlockchainFromWAL(t, wal) + chain, extCommits = makeBlockchainFromWAL(t, wal) pubKey, err := privVal.GetPubKey(ctx) require.NoError(t, err) stateDB, genesisState, store = stateAndStore(t, cfg, pubKey, kvstore.ProtocolVersion) @@ -726,7 +726,7 @@ func testHandshakeReplay( } stateStore := sm.NewStore(stateDB) store.chain = chain - store.commits = commits + store.extCommits = extCommits state := genesisState.Copy() // run the chain through state.ApplyBlock to build up the tendermint state @@ -1034,7 +1034,7 @@ func (app *badApp) Commit(context.Context) (*abci.ResponseCommit, error) { //-------------------------- // utils for making blocks -func makeBlockchainFromWAL(t *testing.T, wal WAL) ([]*types.Block, []*types.Commit) { +func makeBlockchainFromWAL(t *testing.T, wal WAL) ([]*types.Block, []*types.ExtendedCommit) { t.Helper() var height int64 @@ -1047,10 +1047,10 @@ func makeBlockchainFromWAL(t *testing.T, wal WAL) ([]*types.Block, []*types.Comm // log.Notice("Build a blockchain by reading from the WAL") var ( - blocks []*types.Block - commits []*types.Commit - thisBlockParts *types.PartSet - thisBlockCommit *types.Commit + blocks []*types.Block + extCommits []*types.ExtendedCommit + thisBlockParts *types.PartSet + thisBlockExtCommit *types.ExtendedCommit ) dec := NewWALDecoder(gr) @@ -1082,12 +1082,12 @@ func makeBlockchainFromWAL(t *testing.T, wal WAL) ([]*types.Block, []*types.Comm require.Equal(t, block.Height, height+1, "read bad block from wal. got height %d, expected %d", block.Height, height+1) - commitHeight := thisBlockCommit.Height + commitHeight := thisBlockExtCommit.Height require.Equal(t, commitHeight, height+1, "commit doesnt match. got height %d, expected %d", commitHeight, height+1) blocks = append(blocks, block) - commits = append(commits, thisBlockCommit) + extCommits = append(extCommits, thisBlockExtCommit) height++ } case *types.PartSetHeader: @@ -1097,8 +1097,12 @@ func makeBlockchainFromWAL(t *testing.T, wal WAL) ([]*types.Block, []*types.Comm require.NoError(t, err) case *types.Vote: if p.Type == tmproto.PrecommitType { - thisBlockCommit = types.NewCommit(p.Height, p.Round, - p.BlockID, []types.CommitSig{p.CommitSig()}) + thisBlockExtCommit = &types.ExtendedCommit{ + Height: p.Height, + Round: p.Round, + BlockID: p.BlockID, + ExtendedSignatures: []types.ExtendedCommitSig{p.ExtendedCommitSig()}, + } } } } @@ -1113,12 +1117,12 @@ func makeBlockchainFromWAL(t *testing.T, wal WAL) ([]*types.Block, []*types.Comm require.NoError(t, err) require.Equal(t, block.Height, height+1, "read bad block from wal. got height %d, expected %d", block.Height, height+1) - commitHeight := thisBlockCommit.Height + commitHeight := thisBlockExtCommit.Height require.Equal(t, commitHeight, height+1, "commit does not match. got height %d, expected %d", commitHeight, height+1) blocks = append(blocks, block) - commits = append(commits, thisBlockCommit) - return blocks, commits + extCommits = append(extCommits, thisBlockExtCommit) + return blocks, extCommits } func readPieceFromWAL(msg *TimedWALMessage) interface{} { @@ -1162,14 +1166,16 @@ func stateAndStore( // mock block store type mockBlockStore struct { - cfg *config.Config - params types.ConsensusParams - chain []*types.Block - commits []*types.Commit - base int64 - t *testing.T + cfg *config.Config + params types.ConsensusParams + chain []*types.Block + extCommits []*types.ExtendedCommit + base int64 + t *testing.T } +var _ sm.BlockStore = &mockBlockStore{} + // TODO: NewBlockStore(db.NewMemDB) ... func newMockBlockStore(t *testing.T, cfg *config.Config, params types.ConsensusParams) *mockBlockStore { return &mockBlockStore{ @@ -1198,20 +1204,24 @@ func (bs *mockBlockStore) LoadBlockMeta(height int64) *types.BlockMeta { } } func (bs *mockBlockStore) LoadBlockPart(height int64, index int) *types.Part { return nil } -func (bs *mockBlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) { +func (bs *mockBlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.ExtendedCommit) { } + func (bs *mockBlockStore) LoadBlockCommit(height int64) *types.Commit { - return bs.commits[height-1] + return bs.extCommits[height-1].StripExtensions() } func (bs *mockBlockStore) LoadSeenCommit() *types.Commit { - return bs.commits[len(bs.commits)-1] + return bs.extCommits[len(bs.extCommits)-1].StripExtensions() +} +func (bs *mockBlockStore) LoadBlockExtendedCommit(height int64) *types.ExtendedCommit { + return bs.extCommits[height-1] } func (bs *mockBlockStore) PruneBlocks(height int64) (uint64, error) { pruned := uint64(0) for i := int64(0); i < height-1; i++ { bs.chain[i] = nil - bs.commits[i] = nil + bs.extCommits[i] = nil pruned++ } bs.base = height diff --git a/internal/consensus/state.go b/internal/consensus/state.go index 90efbab77..b016e2687 100644 --- a/internal/consensus/state.go +++ b/internal/consensus/state.go @@ -695,19 +695,15 @@ func (cs *State) sendInternalMessage(ctx context.Context, mi msgInfo) { // Reconstruct LastCommit from SeenCommit, which we saved along with the block, // (which happens even before saving the state) func (cs *State) reconstructLastCommit(state sm.State) { - commit := cs.blockStore.LoadSeenCommit() - if commit == nil || commit.Height != state.LastBlockHeight { - commit = cs.blockStore.LoadBlockCommit(state.LastBlockHeight) - } - - if commit == nil { + extCommit := cs.blockStore.LoadBlockExtendedCommit(state.LastBlockHeight) + if extCommit == nil { panic(fmt.Sprintf( "failed to reconstruct last commit; commit for height %v not found", state.LastBlockHeight, )) } - lastPrecommits := types.CommitToVoteSet(state.ChainID, commit, state.LastValidators) + lastPrecommits := extCommit.ToVoteSet(state.ChainID, state.LastValidators) if !lastPrecommits.HasTwoThirdsMajority() { panic("failed to reconstruct last commit; does not have +2/3 maj") } @@ -1401,16 +1397,17 @@ func (cs *State) createProposalBlock(ctx context.Context) (*types.Block, error) return nil, errors.New("entered createProposalBlock with privValidator being nil") } - var commit *types.Commit + // TODO(sergio): wouldn't it be easier if CreateProposalBlock accepted cs.LastCommit directly? + var lastExtCommit *types.ExtendedCommit switch { case cs.Height == cs.state.InitialHeight: // We're creating a proposal for the first block. // The commit is empty, but not nil. - commit = types.NewCommit(0, 0, types.BlockID{}, nil) + lastExtCommit = &types.ExtendedCommit{} case cs.LastCommit.HasTwoThirdsMajority(): // Make the commit from LastCommit - commit = cs.LastCommit.MakeCommit() + lastExtCommit = cs.LastCommit.MakeExtendedCommit() default: // This shouldn't happen. cs.logger.Error("propose step; cannot propose anything without commit for the previous block") @@ -1426,7 +1423,7 @@ func (cs *State) createProposalBlock(ctx context.Context) (*types.Block, error) proposerAddr := cs.privValidatorPubKey.Address() - ret, err := cs.blockExec.CreateProposalBlock(ctx, cs.Height, cs.state, commit, proposerAddr, cs.LastCommit.GetVotes()) + ret, err := cs.blockExec.CreateProposalBlock(ctx, cs.Height, cs.state, lastExtCommit, proposerAddr) if err != nil { panic(err) } @@ -1925,8 +1922,7 @@ func (cs *State) finalizeCommit(ctx context.Context, height int64) { // NOTE: the seenCommit is local justification to commit this block, // but may differ from the LastCommit included in the next block precommits := cs.Votes.Precommits(cs.CommitRound) - seenCommit := precommits.MakeCommit() - cs.blockStore.SaveBlock(block, blockParts, seenCommit) + cs.blockStore.SaveBlock(block, blockParts, precommits.MakeExtendedCommit()) } else { // Happens during replay if we already saved the block but didn't commit logger.Debug("calling finalizeCommit on already stored block", "height", block.Height) @@ -2028,7 +2024,7 @@ func (cs *State) RecordMetrics(height int64, block *types.Block) { for i, val := range cs.LastValidators.Validators { commitSig := block.LastCommit.Signatures[i] - if commitSig.Absent() { + if commitSig.BlockIDFlag == types.BlockIDFlagAbsent { missingValidators++ missingValidatorsPower += val.VotingPower } @@ -2038,7 +2034,7 @@ func (cs *State) RecordMetrics(height int64, block *types.Block) { "validator_address", val.Address.String(), } cs.metrics.ValidatorPower.With(label...).Set(float64(val.VotingPower)) - if commitSig.ForBlock() { + if commitSig.BlockIDFlag == types.BlockIDFlagCommit { cs.metrics.ValidatorLastSignedHeight.With(label...).Set(float64(height)) } else { cs.metrics.ValidatorMissedBlocks.With(label...).Add(float64(1)) @@ -2341,8 +2337,9 @@ func (cs *State) addVote( return } - // Verify VoteExtension if precommit - if vote.Type == tmproto.PrecommitType { + // Verify VoteExtension if precommit and not nil + // https://github.com/tendermint/tendermint/issues/8487 + if vote.Type == tmproto.PrecommitType && !vote.BlockID.IsNil() { err := cs.blockExec.VerifyVoteExtension(ctx, vote) cs.metrics.MarkVoteExtensionReceived(err == nil) if err != nil { @@ -2497,15 +2494,15 @@ func (cs *State) signVote( // use our local precommit Timeout as the max wait time for getting a singed commit. The same goes for prevote. timeout := cs.voteTimeout(cs.Round) - switch msgType { - case tmproto.PrecommitType: - // if the signedMessage type is for a precommit, add VoteExtension + if msgType == tmproto.PrecommitType && !vote.BlockID.IsNil() { + // if the signedMessage type is for a non-nil precommit, add + // VoteExtension ext, err := cs.blockExec.ExtendVote(ctx, vote) if err != nil { return nil, err } vote.Extension = ext - default: + } else { timeout = time.Second } diff --git a/internal/consensus/state_test.go b/internal/consensus/state_test.go index 93aa4a49d..6fa69a1a3 100644 --- a/internal/consensus/state_test.go +++ b/internal/consensus/state_test.go @@ -1950,7 +1950,7 @@ func TestFinalizeBlockCalled(t *testing.T) { expectCalled bool }{ { - name: "finalze block called when block committed", + name: "finalize block called when block committed", voteNil: false, expectCalled: true, }, @@ -1970,11 +1970,15 @@ func TestFinalizeBlockCalled(t *testing.T) { Status: abci.ResponseProcessProposal_ACCEPT, }, nil) m.On("PrepareProposal", mock.Anything, mock.Anything).Return(&abci.ResponsePrepareProposal{}, nil) - m.On("VerifyVoteExtension", mock.Anything, mock.Anything).Return(&abci.ResponseVerifyVoteExtension{ - Status: abci.ResponseVerifyVoteExtension_ACCEPT, - }, nil) + // We only expect VerifyVoteExtension to be called on non-nil precommits. + // https://github.com/tendermint/tendermint/issues/8487 + if !testCase.voteNil { + m.On("ExtendVote", mock.Anything, mock.Anything).Return(&abci.ResponseExtendVote{}, nil) + m.On("VerifyVoteExtension", mock.Anything, mock.Anything).Return(&abci.ResponseVerifyVoteExtension{ + Status: abci.ResponseVerifyVoteExtension_ACCEPT, + }, nil) + } m.On("FinalizeBlock", mock.Anything, mock.Anything).Return(&abci.ResponseFinalizeBlock{}, nil).Maybe() - m.On("ExtendVote", mock.Anything, mock.Anything).Return(&abci.ResponseExtendVote{}, nil) m.On("Commit", mock.Anything).Return(&abci.ResponseCommit{}, nil).Maybe() cs1, vss := makeState(ctx, t, makeStateArgs{config: config, application: m}) diff --git a/internal/evidence/pool_test.go b/internal/evidence/pool_test.go index dcf44a5df..4047d3e7f 100644 --- a/internal/evidence/pool_test.go +++ b/internal/evidence/pool_test.go @@ -249,8 +249,8 @@ func TestEvidencePoolUpdate(t *testing.T) { evidenceChainID, ) require.NoError(t, err) - lastCommit := makeCommit(height, val.PrivKey.PubKey().Address()) - block := types.MakeBlock(height+1, []types.Tx{}, lastCommit, []types.Evidence{ev}) + lastExtCommit := makeExtCommit(height, val.PrivKey.PubKey().Address()) + block := types.MakeBlock(height+1, []types.Tx{}, lastExtCommit.StripExtensions(), []types.Evidence{ev}) // update state (partially) state.LastBlockHeight = height + 1 @@ -568,8 +568,8 @@ func initializeBlockStore(db dbm.DB, state sm.State, valAddr []byte) (*store.Blo blockStore := store.NewBlockStore(db) for i := int64(1); i <= state.LastBlockHeight; i++ { - lastCommit := makeCommit(i-1, valAddr) - block := sf.MakeBlock(state, i, lastCommit) + lastCommit := makeExtCommit(i-1, valAddr) + block := sf.MakeBlock(state, i, lastCommit.StripExtensions()) block.Header.Time = defaultEvidenceTime.Add(time.Duration(i) * time.Minute) block.Header.Version = version.Consensus{Block: version.BlockProtocol, App: 1} @@ -579,22 +579,25 @@ func initializeBlockStore(db dbm.DB, state sm.State, valAddr []byte) (*store.Blo return nil, err } - seenCommit := makeCommit(i, valAddr) + seenCommit := makeExtCommit(i, valAddr) blockStore.SaveBlock(block, partSet, seenCommit) } return blockStore, nil } -func makeCommit(height int64, valAddr []byte) *types.Commit { - commitSigs := []types.CommitSig{{ - BlockIDFlag: types.BlockIDFlagCommit, - ValidatorAddress: valAddr, - Timestamp: defaultEvidenceTime, - Signature: []byte("Signature"), - }} - - return types.NewCommit(height, 0, types.BlockID{}, commitSigs) +func makeExtCommit(height int64, valAddr []byte) *types.ExtendedCommit { + return &types.ExtendedCommit{ + Height: height, + ExtendedSignatures: []types.ExtendedCommitSig{{ + CommitSig: types.CommitSig{ + BlockIDFlag: types.BlockIDFlagCommit, + ValidatorAddress: valAddr, + Timestamp: defaultEvidenceTime, + Signature: []byte("Signature"), + }, + }}, + } } func defaultTestPool(ctx context.Context, t *testing.T, height int64) (*evidence.Pool, types.MockPV, *eventbus.EventBus) { diff --git a/internal/evidence/verify_test.go b/internal/evidence/verify_test.go index b2056186f..2ed84fa69 100644 --- a/internal/evidence/verify_test.go +++ b/internal/evidence/verify_test.go @@ -234,8 +234,9 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) { // except the last validator vote twice blockID := factory.MakeBlockIDWithHash(conflictingHeader.Hash()) voteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals) - commit, err := factory.MakeCommit(ctx, blockID, 10, 1, voteSet, conflictingPrivVals[:4], defaultEvidenceTime) + extCommit, err := factory.MakeExtendedCommit(ctx, blockID, 10, 1, voteSet, conflictingPrivVals[:4], defaultEvidenceTime) require.NoError(t, err) + commit := extCommit.StripExtensions() ev := &types.LightClientAttackEvidence{ ConflictingBlock: &types.LightBlock{ @@ -253,9 +254,10 @@ func TestVerifyLightClientAttack_Equivocation(t *testing.T) { trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash")) trustedVoteSet := types.NewVoteSet(evidenceChainID, 10, 1, tmproto.SignedMsgType(2), conflictingVals) - trustedCommit, err := factory.MakeCommit(ctx, trustedBlockID, 10, 1, + trustedExtCommit, err := factory.MakeExtendedCommit(ctx, trustedBlockID, 10, 1, trustedVoteSet, conflictingPrivVals, defaultEvidenceTime) require.NoError(t, err) + trustedCommit := trustedExtCommit.StripExtensions() trustedSignedHeader := &types.SignedHeader{ Header: trustedHeader, @@ -335,8 +337,9 @@ func TestVerifyLightClientAttack_Amnesia(t *testing.T) { // except the last validator vote twice. However this time the commits are of different rounds. blockID := makeBlockID(conflictingHeader.Hash(), 1000, []byte("partshash")) voteSet := types.NewVoteSet(evidenceChainID, height, 0, tmproto.SignedMsgType(2), conflictingVals) - commit, err := factory.MakeCommit(ctx, blockID, height, 0, voteSet, conflictingPrivVals, defaultEvidenceTime) + extCommit, err := factory.MakeExtendedCommit(ctx, blockID, height, 0, voteSet, conflictingPrivVals, defaultEvidenceTime) require.NoError(t, err) + commit := extCommit.StripExtensions() ev := &types.LightClientAttackEvidence{ ConflictingBlock: &types.LightBlock{ @@ -354,9 +357,10 @@ func TestVerifyLightClientAttack_Amnesia(t *testing.T) { trustedBlockID := makeBlockID(trustedHeader.Hash(), 1000, []byte("partshash")) trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), conflictingVals) - trustedCommit, err := factory.MakeCommit(ctx, trustedBlockID, height, 1, + trustedExtCommit, err := factory.MakeExtendedCommit(ctx, trustedBlockID, height, 1, trustedVoteSet, conflictingPrivVals, defaultEvidenceTime) require.NoError(t, err) + trustedCommit := trustedExtCommit.StripExtensions() trustedSignedHeader := &types.SignedHeader{ Header: trustedHeader, @@ -550,8 +554,9 @@ func makeLunaticEvidence( blockID := factory.MakeBlockIDWithHash(conflictingHeader.Hash()) voteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), conflictingVals) - commit, err := factory.MakeCommit(ctx, blockID, height, 1, voteSet, conflictingPrivVals, defaultEvidenceTime) + extCommit, err := factory.MakeExtendedCommit(ctx, blockID, height, 1, voteSet, conflictingPrivVals, defaultEvidenceTime) require.NoError(t, err) + commit := extCommit.StripExtensions() ev = &types.LightClientAttackEvidence{ ConflictingBlock: &types.LightBlock{ @@ -578,8 +583,9 @@ func makeLunaticEvidence( trustedBlockID := factory.MakeBlockIDWithHash(trustedHeader.Hash()) trustedVals, privVals := factory.ValidatorSet(ctx, t, totalVals, defaultVotingPower) trustedVoteSet := types.NewVoteSet(evidenceChainID, height, 1, tmproto.SignedMsgType(2), trustedVals) - trustedCommit, err := factory.MakeCommit(ctx, trustedBlockID, height, 1, trustedVoteSet, privVals, defaultEvidenceTime) + trustedExtCommit, err := factory.MakeExtendedCommit(ctx, trustedBlockID, height, 1, trustedVoteSet, privVals, defaultEvidenceTime) require.NoError(t, err) + trustedCommit := trustedExtCommit.StripExtensions() trusted = &types.LightBlock{ SignedHeader: &types.SignedHeader{ diff --git a/internal/state/execution.go b/internal/state/execution.go index cfacb816d..2c88c793b 100644 --- a/internal/state/execution.go +++ b/internal/state/execution.go @@ -1,13 +1,13 @@ package state import ( + "bytes" "context" "fmt" "time" abciclient "github.com/tendermint/tendermint/abci/client" abci "github.com/tendermint/tendermint/abci/types" - "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/crypto/encoding" "github.com/tendermint/tendermint/crypto/merkle" "github.com/tendermint/tendermint/internal/eventbus" @@ -87,9 +87,8 @@ func (blockExec *BlockExecutor) CreateProposalBlock( ctx context.Context, height int64, state State, - commit *types.Commit, + lastExtCommit *types.ExtendedCommit, proposerAddr []byte, - votes []*types.Vote, ) (*types.Block, error) { maxBytes := state.ConsensusParams.Block.MaxBytes @@ -101,15 +100,15 @@ func (blockExec *BlockExecutor) CreateProposalBlock( maxDataBytes := types.MaxDataBytes(maxBytes, evSize, state.Validators.Size()) txs := blockExec.mempool.ReapMaxBytesMaxGas(maxDataBytes, maxGas) + commit := lastExtCommit.StripExtensions() block := state.MakeBlock(height, txs, commit, evidence, proposerAddr) - localLastCommit := buildLastCommitInfo(block, blockExec.store, state.InitialHeight) rpp, err := blockExec.appClient.PrepareProposal( ctx, &abci.RequestPrepareProposal{ MaxTxBytes: maxDataBytes, Txs: block.Txs.ToSliceOfBytes(), - LocalLastCommit: extendedCommitInfo(localLastCommit, votes), + LocalLastCommit: buildExtendedCommitInfo(lastExtCommit, blockExec.store, state.InitialHeight), ByzantineValidators: block.Evidence.ToABCI(), Height: block.Height, Time: block.Time, @@ -381,14 +380,14 @@ func (blockExec *BlockExecutor) Commit( func buildLastCommitInfo(block *types.Block, store Store, initialHeight int64) abci.CommitInfo { if block.Height == initialHeight { - // there is no last commmit for the initial height. + // there is no last commit for the initial height. // return an empty value. return abci.CommitInfo{} } lastValSet, err := store.LoadValidators(block.Height - 1) if err != nil { - panic(err) + panic(fmt.Errorf("failed to load validator set at height %d: %w", block.Height-1, err)) } var ( @@ -410,7 +409,7 @@ func buildLastCommitInfo(block *types.Block, store Store, initialHeight int64) a commitSig := block.LastCommit.Signatures[i] votes[i] = abci.VoteInfo{ Validator: types.TM2PB.Validator(val), - SignedLastBlock: !commitSig.Absent(), + SignedLastBlock: commitSig.BlockIDFlag != types.BlockIDFlagAbsent, } } @@ -420,44 +419,69 @@ func buildLastCommitInfo(block *types.Block, store Store, initialHeight int64) a } } -// extendedCommitInfo expects a CommitInfo struct along with all of the -// original votes relating to that commit, including their vote extensions. The -// order of votes does not matter. -func extendedCommitInfo(c abci.CommitInfo, votes []*types.Vote) abci.ExtendedCommitInfo { - if len(c.Votes) != len(votes) { - panic(fmt.Sprintf("extendedCommitInfo: number of votes from commit differ from the number of votes supplied (%d != %d)", len(c.Votes), len(votes))) +// buildExtendedCommitInfo populates an ABCI extended commit from the +// corresponding Tendermint extended commit ec, using the stored validator set +// from ec. It requires ec to include the original precommit votes along with +// the vote extensions from the last commit. +// +// For heights below the initial height, for which we do not have the required +// data, it returns an empty record. +// +// Assumes that the commit signatures are sorted according to validator index. +func buildExtendedCommitInfo(ec *types.ExtendedCommit, store Store, initialHeight int64) abci.ExtendedCommitInfo { + if ec.Height < initialHeight { + // There are no extended commits for heights below the initial height. + return abci.ExtendedCommitInfo{} } - votesByVal := make(map[string]*types.Vote) - for _, vote := range votes { - if vote != nil { - valAddr := vote.ValidatorAddress.String() - if _, ok := votesByVal[valAddr]; ok { - panic(fmt.Sprintf("extendedCommitInfo: found duplicate vote for validator with address %s", valAddr)) - } - votesByVal[valAddr] = vote + + valSet, err := store.LoadValidators(ec.Height) + if err != nil { + panic(fmt.Errorf("failed to load validator set at height %d, initial height %d: %w", ec.Height, initialHeight, err)) + } + + var ( + ecSize = ec.Size() + valSetLen = len(valSet.Validators) + ) + + // Ensure that the size of the validator set in the extended commit matches + // the size of the validator set in the state store. + if ecSize != valSetLen { + panic(fmt.Errorf( + "extended commit size (%d) does not match validator set length (%d) at height %d\n\n%v\n\n%v", + ecSize, valSetLen, ec.Height, ec.ExtendedSignatures, valSet.Validators, + )) + } + + votes := make([]abci.ExtendedVoteInfo, ecSize) + for i, val := range valSet.Validators { + ecs := ec.ExtendedSignatures[i] + + // Absent signatures have empty validator addresses, but otherwise we + // expect the validator addresses to be the same. + if ecs.BlockIDFlag != types.BlockIDFlagAbsent && !bytes.Equal(ecs.ValidatorAddress, val.Address) { + panic(fmt.Errorf("validator address of extended commit signature in position %d (%s) does not match the corresponding validator's at height %d (%s)", + i, ecs.ValidatorAddress, ec.Height, val.Address, + )) } - } - vs := make([]abci.ExtendedVoteInfo, len(c.Votes)) - for i := range vs { + var ext []byte - // votes[i] will be nil if c.Votes[i].SignedLastBlock is false - if c.Votes[i].SignedLastBlock { - valAddr := crypto.Address(c.Votes[i].Validator.Address).String() - vote, ok := votesByVal[valAddr] - if !ok || vote == nil { - panic(fmt.Sprintf("extendedCommitInfo: validator with address %s signed last block, but could not find vote for it", valAddr)) - } - ext = vote.Extension + if ecs.BlockIDFlag == types.BlockIDFlagCommit { + // We only care about vote extensions if a validator has voted to + // commit. + ext = ecs.Extension } - vs[i] = abci.ExtendedVoteInfo{ - Validator: c.Votes[i].Validator, - SignedLastBlock: c.Votes[i].SignedLastBlock, + + votes[i] = abci.ExtendedVoteInfo{ + Validator: types.TM2PB.Validator(val), + SignedLastBlock: ecs.BlockIDFlag != types.BlockIDFlagAbsent, VoteExtension: ext, } } + return abci.ExtendedCommitInfo{ - Round: c.Round, - Votes: vs, + Round: ec.Round, + Votes: votes, } } diff --git a/internal/state/execution_test.go b/internal/state/execution_test.go index 0937b9990..ffe9cb6f8 100644 --- a/internal/state/execution_test.go +++ b/internal/state/execution_test.go @@ -79,9 +79,10 @@ func TestApplyBlock(t *testing.T) { assert.EqualValues(t, 1, state.Version.Consensus.App, "App version wasn't updated") } -// TestFinalizeBlockDecidedLastCommit ensures we correctly send the DecidedLastCommit to the -// application. The test ensures that the DecidedLastCommit properly reflects -// which validators signed the preceding block. +// TestFinalizeBlockDecidedLastCommit ensures we correctly send the +// DecidedLastCommit to the application. The test ensures that the +// DecidedLastCommit properly reflects which validators signed the preceding +// block. func TestFinalizeBlockDecidedLastCommit(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -96,7 +97,7 @@ func TestFinalizeBlockDecidedLastCommit(t *testing.T) { state, stateDB, privVals := makeState(t, 7, 1) stateStore := sm.NewStore(stateDB) - absentSig := types.NewCommitSigAbsent() + absentSig := types.NewExtendedCommitSigAbsent() testCases := []struct { name string @@ -134,12 +135,12 @@ func TestFinalizeBlockDecidedLastCommit(t *testing.T) { for idx, isAbsent := range tc.absentCommitSigs { if isAbsent { - lastCommit.Signatures[idx] = absentSig + lastCommit.ExtendedSignatures[idx] = absentSig } } // block for height 2 - block := sf.MakeBlock(state, 2, lastCommit) + block := sf.MakeBlock(state, 2, lastCommit.StripExtensions()) bps, err := block.MakePartSet(testPartSize) require.NoError(t, err) blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()} @@ -198,12 +199,15 @@ func TestFinalizeBlockByzantineValidators(t *testing.T) { ConflictingBlock: &types.LightBlock{ SignedHeader: &types.SignedHeader{ Header: header, - Commit: types.NewCommit(10, 0, makeBlockID(header.Hash(), 100, []byte("partshash")), []types.CommitSig{{ - BlockIDFlag: types.BlockIDFlagNil, - ValidatorAddress: crypto.AddressHash([]byte("validator_address")), - Timestamp: defaultEvidenceTime, - Signature: crypto.CRandBytes(types.MaxSignatureSize), - }}), + Commit: &types.Commit{ + Height: 10, + BlockID: makeBlockID(header.Hash(), 100, []byte("partshash")), + Signatures: []types.CommitSig{{ + BlockIDFlag: types.BlockIDFlagNil, + ValidatorAddress: crypto.AddressHash([]byte("validator_address")), + Timestamp: defaultEvidenceTime, + Signature: crypto.CRandBytes(types.MaxSignatureSize)}}, + }, }, ValidatorSet: state.Validators, }, @@ -324,8 +328,10 @@ func TestProcessProposal(t *testing.T) { lastCommitSig = append(lastCommitSig, vote.CommitSig()) } - lastCommit := types.NewCommit(height-1, 0, types.BlockID{}, lastCommitSig) - block1 := sf.MakeBlock(state, height, lastCommit) + block1 := sf.MakeBlock(state, height, &types.Commit{ + Height: height - 1, + Signatures: lastCommitSig, + }) block1.Txs = txs expectedRpp := &abci.RequestProcessProposal{ @@ -653,8 +659,8 @@ func TestEmptyPrepareProposal(t *testing.T) { sm.NopMetrics(), ) pa, _ := state.Validators.GetByIndex(0) - commit, votes := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) - _, err = blockExec.CreateProposalBlock(ctx, height, state, commit, pa, votes) + commit, _ := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) + _, err = blockExec.CreateProposalBlock(ctx, height, state, commit, pa) require.NoError(t, err) } @@ -708,8 +714,8 @@ func TestPrepareProposalErrorOnNonExistingRemoved(t *testing.T) { sm.NopMetrics(), ) pa, _ := state.Validators.GetByIndex(0) - commit, votes := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) - block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa, votes) + commit, _ := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) + block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa) require.ErrorContains(t, err, "new transaction incorrectly marked as removed") require.Nil(t, block) @@ -764,8 +770,8 @@ func TestPrepareProposalRemoveTxs(t *testing.T) { sm.NopMetrics(), ) pa, _ := state.Validators.GetByIndex(0) - commit, votes := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) - block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa, votes) + commit, _ := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) + block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa) require.NoError(t, err) require.Len(t, block.Data.Txs.ToSliceOfBytes(), len(trs)-2) @@ -823,8 +829,8 @@ func TestPrepareProposalAddedTxsIncluded(t *testing.T) { sm.NopMetrics(), ) pa, _ := state.Validators.GetByIndex(0) - commit, votes := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) - block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa, votes) + commit, _ := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) + block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa) require.NoError(t, err) require.Equal(t, txs[0], block.Data.Txs[0]) @@ -879,8 +885,8 @@ func TestPrepareProposalReorderTxs(t *testing.T) { sm.NopMetrics(), ) pa, _ := state.Validators.GetByIndex(0) - commit, votes := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) - block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa, votes) + commit, _ := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) + block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa) require.NoError(t, err) for i, tx := range block.Data.Txs { require.Equal(t, types.Tx(trs[i].Tx), tx) @@ -939,9 +945,8 @@ func TestPrepareProposalErrorOnTooManyTxs(t *testing.T) { sm.NopMetrics(), ) pa, _ := state.Validators.GetByIndex(0) - commit, votes := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) - - block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa, votes) + commit, _ := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) + block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa) require.ErrorContains(t, err, "transaction data size exceeds maximum") require.Nil(t, block, "") @@ -991,9 +996,8 @@ func TestPrepareProposalErrorOnPrepareProposalError(t *testing.T) { sm.NopMetrics(), ) pa, _ := state.Validators.GetByIndex(0) - commit, votes := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) - - block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa, votes) + commit, _ := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals) + block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa) require.Nil(t, block) require.ErrorContains(t, err, "an injected error") diff --git a/internal/state/helpers_test.go b/internal/state/helpers_test.go index 07dd0d865..dec5afc66 100644 --- a/internal/state/helpers_test.go +++ b/internal/state/helpers_test.go @@ -39,7 +39,7 @@ func makeAndCommitGoodBlock( blockExec *sm.BlockExecutor, privVals map[string]types.PrivValidator, evidence []types.Evidence, -) (sm.State, types.BlockID, *types.Commit) { +) (sm.State, types.BlockID, *types.ExtendedCommit) { t.Helper() // A good block passes @@ -82,19 +82,23 @@ func makeValidCommit( blockID types.BlockID, vals *types.ValidatorSet, privVals map[string]types.PrivValidator, -) (*types.Commit, []*types.Vote) { +) (*types.ExtendedCommit, []*types.Vote) { t.Helper() - sigs := make([]types.CommitSig, vals.Size()) + sigs := make([]types.ExtendedCommitSig, vals.Size()) votes := make([]*types.Vote, vals.Size()) for i := 0; i < vals.Size(); i++ { _, val := vals.GetByIndex(int32(i)) vote, err := factory.MakeVote(ctx, privVals[val.Address.String()], chainID, int32(i), height, 0, 2, blockID, time.Now()) require.NoError(t, err) - sigs[i] = vote.CommitSig() + sigs[i] = vote.ExtendedCommitSig() votes[i] = vote } - return types.NewCommit(height, 0, blockID, sigs), votes + return &types.ExtendedCommit{ + Height: height, + BlockID: blockID, + ExtendedSignatures: sigs, + }, votes } func makeState(t *testing.T, nVals, height int) (sm.State, dbm.DB, map[string]types.PrivValidator) { diff --git a/internal/state/indexer/mocks/event_sink.go b/internal/state/indexer/mocks/event_sink.go index decf551ab..69abe3907 100644 --- a/internal/state/indexer/mocks/event_sink.go +++ b/internal/state/indexer/mocks/event_sink.go @@ -6,6 +6,7 @@ import ( context "context" mock "github.com/stretchr/testify/mock" + indexer "github.com/tendermint/tendermint/internal/state/indexer" query "github.com/tendermint/tendermint/internal/pubsub/query" diff --git a/internal/state/mocks/block_store.go b/internal/state/mocks/block_store.go index 7cc7fa883..4eafb1273 100644 --- a/internal/state/mocks/block_store.go +++ b/internal/state/mocks/block_store.go @@ -107,6 +107,22 @@ func (_m *BlockStore) LoadBlockCommit(height int64) *types.Commit { return r0 } +// LoadBlockExtendedCommit provides a mock function with given fields: height +func (_m *BlockStore) LoadBlockExtendedCommit(height int64) *types.ExtendedCommit { + ret := _m.Called(height) + + var r0 *types.ExtendedCommit + if rf, ok := ret.Get(0).(func(int64) *types.ExtendedCommit); ok { + r0 = rf(height) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.ExtendedCommit) + } + } + + return r0 +} + // LoadBlockMeta provides a mock function with given fields: height func (_m *BlockStore) LoadBlockMeta(height int64) *types.BlockMeta { ret := _m.Called(height) @@ -193,7 +209,7 @@ func (_m *BlockStore) PruneBlocks(height int64) (uint64, error) { } // SaveBlock provides a mock function with given fields: block, blockParts, seenCommit -func (_m *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) { +func (_m *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.ExtendedCommit) { _m.Called(block, blockParts, seenCommit) } diff --git a/internal/state/mocks/evidence_pool.go b/internal/state/mocks/evidence_pool.go index 49633269b..0ea3ba17b 100644 --- a/internal/state/mocks/evidence_pool.go +++ b/internal/state/mocks/evidence_pool.go @@ -6,6 +6,7 @@ import ( context "context" mock "github.com/stretchr/testify/mock" + state "github.com/tendermint/tendermint/internal/state" testing "testing" diff --git a/internal/state/mocks/store.go b/internal/state/mocks/store.go index 9b41f3c1b..1d9ef2f6f 100644 --- a/internal/state/mocks/store.go +++ b/internal/state/mocks/store.go @@ -4,6 +4,7 @@ package mocks import ( mock "github.com/stretchr/testify/mock" + state "github.com/tendermint/tendermint/internal/state" tendermintstate "github.com/tendermint/tendermint/proto/tendermint/state" diff --git a/internal/state/services.go b/internal/state/services.go index 5d04d2c82..35a91aa11 100644 --- a/internal/state/services.go +++ b/internal/state/services.go @@ -26,7 +26,7 @@ type BlockStore interface { LoadBlockMeta(height int64) *types.BlockMeta LoadBlock(height int64) *types.Block - SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) + SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.ExtendedCommit) PruneBlocks(height int64) (uint64, error) @@ -36,6 +36,7 @@ type BlockStore interface { LoadBlockCommit(height int64) *types.Commit LoadSeenCommit() *types.Commit + LoadBlockExtendedCommit(height int64) *types.ExtendedCommit } //----------------------------------------------------------------------------- diff --git a/internal/state/store.go b/internal/state/store.go index 87f5e0c4f..2d2e4dc81 100644 --- a/internal/state/store.go +++ b/internal/state/store.go @@ -26,6 +26,9 @@ const ( //------------------------------------------------------------------------ +// NB: Before modifying these, cross-check them with those in +// internal/store/store.go +// TODO(thane): Move these and the ones in internal/store/store.go to their own package. const ( // prefixes are unique across all tm db's prefixValidators = int64(5) diff --git a/internal/state/test/factory/block.go b/internal/state/test/factory/block.go index 1b3351363..0ccd46dcb 100644 --- a/internal/state/test/factory/block.go +++ b/internal/state/test/factory/block.go @@ -63,7 +63,7 @@ func makeBlockAndPartSet( ) (*types.Block, *types.PartSet) { t.Helper() - lastCommit := types.NewCommit(height-1, 0, types.BlockID{}, nil) + lastCommit := &types.Commit{Height: height - 1} if height > 1 { vote, err := factory.MakeVote( ctx, @@ -73,8 +73,12 @@ func makeBlockAndPartSet( lastBlockMeta.BlockID, time.Now()) require.NoError(t, err) - lastCommit = types.NewCommit(vote.Height, vote.Round, - lastBlockMeta.BlockID, []types.CommitSig{vote.CommitSig()}) + lastCommit = &types.Commit{ + Height: vote.Height, + Round: vote.Round, + BlockID: lastBlock.LastBlockID, + Signatures: []types.CommitSig{vote.CommitSig()}, + } } block := state.MakeBlock(height, []types.Tx{}, lastCommit, nil, state.Validators.GetProposer().Address) diff --git a/internal/state/validation_test.go b/internal/state/validation_test.go index 376ce61bc..b29cfd0f9 100644 --- a/internal/state/validation_test.go +++ b/internal/state/validation_test.go @@ -65,7 +65,8 @@ func TestValidateBlockHeader(t *testing.T) { eventBus, sm.NopMetrics(), ) - lastCommit := types.NewCommit(0, 0, types.BlockID{}, nil) + lastCommit := &types.Commit{} + var lastExtCommit *types.ExtendedCommit // some bad values wrongHash := crypto.Checksum([]byte("this hash is wrong")) @@ -100,7 +101,7 @@ func TestValidateBlockHeader(t *testing.T) { {"Proposer invalid", func(block *types.Block) { block.ProposerAddress = []byte("wrong size") }}, {"first LastCommit contains signatures", func(block *types.Block) { - block.LastCommit = types.NewCommit(0, 0, types.BlockID{}, []types.CommitSig{types.NewCommitSigAbsent()}) + block.LastCommit = &types.Commit{Signatures: []types.CommitSig{types.NewCommitSigAbsent()}} block.LastCommitHash = block.LastCommit.Hash() }}, } @@ -121,8 +122,9 @@ func TestValidateBlockHeader(t *testing.T) { /* A good block passes */ - state, _, lastCommit = makeAndCommitGoodBlock(ctx, t, + state, _, lastExtCommit = makeAndCommitGoodBlock(ctx, t, state, height, lastCommit, state.Validators.GetProposer().Address, blockExec, privVals, nil) + lastCommit = lastExtCommit.StripExtensions() } nextHeight := validationTestsStopHeight @@ -169,8 +171,9 @@ func TestValidateBlockCommit(t *testing.T) { eventBus, sm.NopMetrics(), ) - lastCommit := types.NewCommit(0, 0, types.BlockID{}, nil) - wrongSigsCommit := types.NewCommit(1, 0, types.BlockID{}, nil) + lastCommit := &types.Commit{} + var lastExtCommit *types.ExtendedCommit + wrongSigsCommit := &types.Commit{Height: 1} badPrivVal := types.NewMockPV() for height := int64(1); height < validationTestsStopHeight; height++ { @@ -192,12 +195,12 @@ func TestValidateBlockCommit(t *testing.T) { time.Now(), ) require.NoError(t, err) - wrongHeightCommit := types.NewCommit( - wrongHeightVote.Height, - wrongHeightVote.Round, - state.LastBlockID, - []types.CommitSig{wrongHeightVote.CommitSig()}, - ) + wrongHeightCommit := &types.Commit{ + Height: wrongHeightVote.Height, + Round: wrongHeightVote.Round, + BlockID: state.LastBlockID, + Signatures: []types.CommitSig{wrongHeightVote.CommitSig()}, + } block := statefactory.MakeBlock(state, height, wrongHeightCommit) err = blockExec.ValidateBlock(ctx, state, block) _, isErrInvalidCommitHeight := err.(types.ErrInvalidCommitHeight) @@ -220,7 +223,7 @@ func TestValidateBlockCommit(t *testing.T) { A good block passes */ var blockID types.BlockID - state, blockID, lastCommit = makeAndCommitGoodBlock( + state, blockID, lastExtCommit = makeAndCommitGoodBlock( ctx, t, state, @@ -231,6 +234,7 @@ func TestValidateBlockCommit(t *testing.T) { privVals, nil, ) + lastCommit = lastExtCommit.StripExtensions() /* wrongSigsCommit is fine except for the extra bad precommit @@ -270,8 +274,12 @@ func TestValidateBlockCommit(t *testing.T) { goodVote.Signature, badVote.Signature = g.Signature, b.Signature - wrongSigsCommit = types.NewCommit(goodVote.Height, goodVote.Round, - blockID, []types.CommitSig{goodVote.CommitSig(), badVote.CommitSig()}) + wrongSigsCommit = &types.Commit{ + Height: goodVote.Height, + Round: goodVote.Round, + BlockID: blockID, + Signatures: []types.CommitSig{goodVote.CommitSig(), badVote.CommitSig()}, + } } } @@ -319,7 +327,8 @@ func TestValidateBlockEvidence(t *testing.T) { eventBus, sm.NopMetrics(), ) - lastCommit := types.NewCommit(0, 0, types.BlockID{}, nil) + lastCommit := &types.Commit{} + var lastExtCommit *types.ExtendedCommit for height := int64(1); height < validationTestsStopHeight; height++ { proposerAddr := state.Validators.GetProposer().Address @@ -364,7 +373,7 @@ func TestValidateBlockEvidence(t *testing.T) { evidence = append(evidence, newEv) } - state, _, lastCommit = makeAndCommitGoodBlock( + state, _, lastExtCommit = makeAndCommitGoodBlock( ctx, t, state, @@ -375,6 +384,7 @@ func TestValidateBlockEvidence(t *testing.T) { privVals, evidence, ) + lastCommit = lastExtCommit.StripExtensions() } } diff --git a/internal/statesync/mocks/state_provider.go b/internal/statesync/mocks/state_provider.go index 582ebcd9c..82e4bd60e 100644 --- a/internal/statesync/mocks/state_provider.go +++ b/internal/statesync/mocks/state_provider.go @@ -6,6 +6,7 @@ import ( context "context" mock "github.com/stretchr/testify/mock" + state "github.com/tendermint/tendermint/internal/state" testing "testing" diff --git a/internal/statesync/reactor_test.go b/internal/statesync/reactor_test.go index 55a9fcf8c..904fb2b74 100644 --- a/internal/statesync/reactor_test.go +++ b/internal/statesync/reactor_test.go @@ -856,12 +856,12 @@ func mockLB(ctx context.Context, t *testing.T, height int64, time time.Time, las header.ConsensusHash = types.DefaultConsensusParams().HashConsensusParams() lastBlockID = factory.MakeBlockIDWithHash(header.Hash()) voteSet := types.NewVoteSet(factory.DefaultTestChainID, height, 0, tmproto.PrecommitType, currentVals) - commit, err := factory.MakeCommit(ctx, lastBlockID, height, 0, voteSet, currentPrivVals, time) + extCommit, err := factory.MakeExtendedCommit(ctx, lastBlockID, height, 0, voteSet, currentPrivVals, time) require.NoError(t, err) return nextVals, nextPrivVals, &types.LightBlock{ SignedHeader: &types.SignedHeader{ Header: header, - Commit: commit, + Commit: extCommit.StripExtensions(), }, ValidatorSet: currentVals, } diff --git a/internal/store/store.go b/internal/store/store.go index eb03e5fe6..5617674a2 100644 --- a/internal/store/store.go +++ b/internal/store/store.go @@ -273,11 +273,31 @@ func (bs *BlockStore) LoadBlockCommit(height int64) *types.Commit { } commit, err := types.CommitFromProto(pbc) if err != nil { - panic(fmt.Errorf("error reading block commit: %w", err)) + panic(fmt.Errorf("converting commit to proto: %w", err)) } return commit } +func (bs *BlockStore) LoadBlockExtendedCommit(height int64) *types.ExtendedCommit { + pbec := new(tmproto.ExtendedCommit) + bz, err := bs.db.Get(extCommitKey(height)) + if err != nil { + panic(fmt.Errorf("fetching extended commit: %w", err)) + } + if len(bz) == 0 { + return nil + } + err = proto.Unmarshal(bz, pbec) + if err != nil { + panic(fmt.Errorf("decoding extended commit: %w", err)) + } + extCommit, err := types.ExtendedCommitFromProto(pbec) + if err != nil { + panic(fmt.Errorf("converting extended commit: %w", err)) + } + return extCommit +} + // LoadSeenCommit returns the last locally seen Commit before being // cannonicalized. This is useful when we've seen a commit, but there // has not yet been a new block at `height + 1` that includes this @@ -298,7 +318,7 @@ func (bs *BlockStore) LoadSeenCommit() *types.Commit { commit, err := types.CommitFromProto(pbc) if err != nil { - panic(fmt.Errorf("error from proto commit: %w", err)) + panic(fmt.Errorf("converting seen commit: %w", err)) } return commit } @@ -446,7 +466,7 @@ func (bs *BlockStore) batchDelete( // If all the nodes restart after committing a block, // we need this to reload the precommits to catch-up nodes to the // most recent height. Otherwise they'd stall at H-1. -func (bs *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.Commit) { +func (bs *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, seenCommit *types.ExtendedCommit) { if block == nil { panic("BlockStore can only save a non-nil block") } @@ -462,6 +482,10 @@ func (bs *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, s if !blockParts.IsComplete() { panic("BlockStore can only save complete block part sets") } + if height != seenCommit.Height { + panic(fmt.Sprintf("BlockStore cannot save seen commit of a different height (block: %d, commit: %d)", + height, seenCommit.Height)) + } // Save block parts. This must be done before the block meta, since callers // typically load the block meta first as an indication that the block exists @@ -494,12 +518,18 @@ func (bs *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, s } // Save seen commit (seen +2/3 precommits for block) - pbsc := seenCommit.ToProto() + pbsc := seenCommit.StripExtensions().ToProto() seenCommitBytes := mustEncode(pbsc) if err := batch.Set(seenCommitKey(), seenCommitBytes); err != nil { panic(err) } + pbec := seenCommit.ToProto() + extCommitBytes := mustEncode(pbec) + if err := batch.Set(extCommitKey(height), extCommitBytes); err != nil { + panic(err) + } + if err := batch.WriteSync(); err != nil { panic(err) } @@ -579,6 +609,9 @@ func (bs *BlockStore) Close() error { //---------------------------------- KEY ENCODING ----------------------------------------- // key prefixes +// NB: Before modifying these, cross-check them with those in +// internal/state/store.go +// TODO(thane): Move these and the ones in internal/state/store.go to their own package. const ( // prefixes are unique across all tm db's prefixBlockMeta = int64(0) @@ -586,6 +619,7 @@ const ( prefixBlockCommit = int64(2) prefixSeenCommit = int64(3) prefixBlockHash = int64(4) + prefixExtCommit = int64(9) // 5..8 are used by state/store ) func blockMetaKey(height int64) []byte { @@ -635,6 +669,14 @@ func seenCommitKey() []byte { return key } +func extCommitKey(height int64) []byte { + key, err := orderedcode.Append(nil, prefixExtCommit, height) + if err != nil { + panic(err) + } + return key +} + func blockHashKey(hash []byte) []byte { key, err := orderedcode.Append(nil, prefixBlockHash, string(hash)) if err != nil { diff --git a/internal/store/store_test.go b/internal/store/store_test.go index 4fa577cc4..9df3eed9f 100644 --- a/internal/store/store_test.go +++ b/internal/store/store_test.go @@ -2,7 +2,6 @@ package store import ( "fmt" - stdlog "log" "os" "runtime/debug" "strings" @@ -27,22 +26,25 @@ import ( // test. type cleanupFunc func() -// make a Commit with a single vote containing just the height and a timestamp -func makeTestCommit(height int64, timestamp time.Time) *types.Commit { - commitSigs := []types.CommitSig{{ - BlockIDFlag: types.BlockIDFlagCommit, - ValidatorAddress: tmrand.Bytes(crypto.AddressSize), - Timestamp: timestamp, - Signature: []byte("Signature"), +// make an extended commit with a single vote containing just the height and a +// timestamp +func makeTestExtCommit(height int64, timestamp time.Time) *types.ExtendedCommit { + extCommitSigs := []types.ExtendedCommitSig{{ + CommitSig: types.CommitSig{ + BlockIDFlag: types.BlockIDFlagCommit, + ValidatorAddress: tmrand.Bytes(crypto.AddressSize), + Timestamp: timestamp, + Signature: []byte("Signature"), + }, }} - return types.NewCommit( - height, - 0, - types.BlockID{ + return &types.ExtendedCommit{ + Height: height, + BlockID: types.BlockID{ Hash: crypto.CRandBytes(32), PartSetHeader: types.PartSetHeader{Hash: crypto.CRandBytes(32), Total: 2}, }, - commitSigs) + ExtendedSignatures: extCommitSigs, + } } func makeStateAndBlockStore(dir string) (sm.State, *BlockStore, cleanupFunc, error) { @@ -59,47 +61,11 @@ func makeStateAndBlockStore(dir string) (sm.State, *BlockStore, cleanupFunc, err return state, NewBlockStore(blockDB), func() { os.RemoveAll(cfg.RootDir) }, nil } -func freshBlockStore() (*BlockStore, dbm.DB) { +func newInMemoryBlockStore() (*BlockStore, dbm.DB) { db := dbm.NewMemDB() return NewBlockStore(db), db } -var ( - state sm.State - block *types.Block - partSet *types.PartSet - part1 *types.Part - part2 *types.Part - seenCommit1 *types.Commit -) - -func TestMain(m *testing.M) { - dir, err := os.MkdirTemp("", "store_test") - if err != nil { - stdlog.Fatal(err) - } - var cleanup cleanupFunc - - state, _, cleanup, err = makeStateAndBlockStore(dir) - if err != nil { - stdlog.Fatal(err) - } - - block = factory.MakeBlock(state, 1, new(types.Commit)) - - partSet, err = block.MakePartSet(2) - if err != nil { - stdlog.Fatal(err) - } - part1 = partSet.GetPart(0) - part2 = partSet.GetPart(1) - seenCommit1 = makeTestCommit(10, tmtime.Now()) - code := m.Run() - cleanup() - os.RemoveAll(dir) // best-effort - os.Exit(code) -} - // TODO: This test should be simplified ... func TestBlockStoreSaveLoadBlock(t *testing.T) { state, bs, cleanup, err := makeStateAndBlockStore(t.TempDir()) @@ -120,8 +86,10 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) { block := factory.MakeBlock(state, bs.Height()+1, new(types.Commit)) validPartSet, err := block.MakePartSet(2) require.NoError(t, err) - seenCommit := makeTestCommit(10, tmtime.Now()) - bs.SaveBlock(block, partSet, seenCommit) + part2 := validPartSet.GetPart(1) + + seenCommit := makeTestExtCommit(block.Header.Height, tmtime.Now()) + bs.SaveBlock(block, validPartSet, seenCommit) require.EqualValues(t, 1, bs.Base(), "expecting the new height to be changed") require.EqualValues(t, block.Header.Height, bs.Height(), "expecting the new height to be changed") @@ -139,11 +107,11 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) { } // End of setup, test data - commitAtH10 := makeTestCommit(10, tmtime.Now()) + commitAtH10 := makeTestExtCommit(10, tmtime.Now()).StripExtensions() tuples := []struct { block *types.Block parts *types.PartSet - seenCommit *types.Commit + seenCommit *types.ExtendedCommit wantPanic string wantErr bool @@ -156,7 +124,7 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) { { block: newBlock(header1, commitAtH10), parts: validPartSet, - seenCommit: seenCommit1, + seenCommit: seenCommit, }, { @@ -172,10 +140,10 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) { ChainID: "block_test", Time: tmtime.Now(), ProposerAddress: tmrand.Bytes(crypto.AddressSize)}, - makeTestCommit(5, tmtime.Now()), + makeTestExtCommit(5, tmtime.Now()).StripExtensions(), ), parts: validPartSet, - seenCommit: makeTestCommit(5, tmtime.Now()), + seenCommit: makeTestExtCommit(5, tmtime.Now()), }, { @@ -187,7 +155,7 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) { { block: newBlock(header1, commitAtH10), parts: validPartSet, - seenCommit: seenCommit1, + seenCommit: seenCommit, corruptCommitInDB: true, // Corrupt the DB's commit entry wantPanic: "error reading block commit", }, @@ -195,7 +163,7 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) { { block: newBlock(header1, commitAtH10), parts: validPartSet, - seenCommit: seenCommit1, + seenCommit: seenCommit, wantPanic: "unmarshal to tmproto.BlockMeta", corruptBlockInDB: true, // Corrupt the DB's block entry }, @@ -203,7 +171,7 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) { { block: newBlock(header1, commitAtH10), parts: validPartSet, - seenCommit: seenCommit1, + seenCommit: seenCommit, // Expecting no error and we want a nil back eraseSeenCommitInDB: true, @@ -212,7 +180,7 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) { { block: newBlock(header1, commitAtH10), parts: validPartSet, - seenCommit: seenCommit1, + seenCommit: seenCommit, corruptSeenCommitInDB: true, wantPanic: "error reading block seen commit", @@ -221,7 +189,7 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) { { block: newBlock(header1, commitAtH10), parts: validPartSet, - seenCommit: seenCommit1, + seenCommit: seenCommit, // Expecting no error and we want a nil back eraseCommitInDB: true, @@ -238,7 +206,7 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) { for i, tuple := range tuples { tuple := tuple - bs, db := freshBlockStore() + bs, db := newInMemoryBlockStore() // SaveBlock res, err, panicErr := doFn(func() (interface{}, error) { bs.SaveBlock(tuple.block, tuple.parts, tuple.seenCommit) @@ -324,7 +292,7 @@ func TestLoadBaseMeta(t *testing.T) { block := factory.MakeBlock(state, h, new(types.Commit)) partSet, err := block.MakePartSet(2) require.NoError(t, err) - seenCommit := makeTestCommit(h, tmtime.Now()) + seenCommit := makeTestExtCommit(h, tmtime.Now()) bs.SaveBlock(block, partSet, seenCommit) } @@ -338,13 +306,19 @@ func TestLoadBaseMeta(t *testing.T) { } func TestLoadBlockPart(t *testing.T) { - bs, db := freshBlockStore() - height, index := int64(10), 1 + cfg, err := config.ResetTestRoot(t.TempDir(), "blockchain_reactor_test") + require.NoError(t, err) + + bs, db := newInMemoryBlockStore() + const height, index = 10, 1 loadPart := func() (interface{}, error) { part := bs.LoadBlockPart(height, index) return part, nil } + state, err := sm.MakeGenesisStateFromFile(cfg.GenesisFile()) + require.NoError(t, err) + // Initially no contents. // 1. Requesting for a non-existent block shouldn't fail res, _, panicErr := doFn(loadPart) @@ -352,13 +326,18 @@ func TestLoadBlockPart(t *testing.T) { require.Nil(t, res, "a non-existent block part should return nil") // 2. Next save a corrupted block then try to load it - err := db.Set(blockPartKey(height, index), []byte("Tendermint")) + err = db.Set(blockPartKey(height, index), []byte("Tendermint")) require.NoError(t, err) res, _, panicErr = doFn(loadPart) require.NotNil(t, panicErr, "expecting a non-nil panic") require.Contains(t, panicErr.Error(), "unmarshal to tmproto.Part failed") // 3. A good block serialized and saved to the DB should be retrievable + block := factory.MakeBlock(state, height, new(types.Commit)) + partSet, err := block.MakePartSet(2) + require.NoError(t, err) + part1 := partSet.GetPart(0) + pb1, err := part1.ToProto() require.NoError(t, err) err = db.Set(blockPartKey(height, index), mustEncode(pb1)) @@ -391,7 +370,7 @@ func TestPruneBlocks(t *testing.T) { block := factory.MakeBlock(state, h, new(types.Commit)) partSet, err := block.MakePartSet(2) require.NoError(t, err) - seenCommit := makeTestCommit(h, tmtime.Now()) + seenCommit := makeTestExtCommit(h, tmtime.Now()) bs.SaveBlock(block, partSet, seenCommit) } @@ -452,7 +431,7 @@ func TestPruneBlocks(t *testing.T) { } func TestLoadBlockMeta(t *testing.T) { - bs, db := freshBlockStore() + bs, db := newInMemoryBlockStore() height := int64(10) loadMeta := func() (interface{}, error) { meta := bs.LoadBlockMeta(height) @@ -499,7 +478,7 @@ func TestBlockFetchAtHeight(t *testing.T) { partSet, err := block.MakePartSet(2) require.NoError(t, err) - seenCommit := makeTestCommit(10, tmtime.Now()) + seenCommit := makeTestExtCommit(block.Header.Height, tmtime.Now()) bs.SaveBlock(block, partSet, seenCommit) require.Equal(t, bs.Height(), block.Header.Height, "expecting the new height to be changed") @@ -521,9 +500,12 @@ func TestBlockFetchAtHeight(t *testing.T) { } func TestSeenAndCanonicalCommit(t *testing.T) { - bs, _ := freshBlockStore() + state, store, cleanup, err := makeStateAndBlockStore(t.TempDir()) + defer cleanup() + require.NoError(t, err) + loadCommit := func() (interface{}, error) { - meta := bs.LoadSeenCommit() + meta := store.LoadSeenCommit() return meta, nil } @@ -536,19 +518,19 @@ func TestSeenAndCanonicalCommit(t *testing.T) { // produce a few blocks and check that the correct seen and cannoncial commits // are persisted. for h := int64(3); h <= 5; h++ { - blockCommit := makeTestCommit(h-1, tmtime.Now()) + blockCommit := makeTestExtCommit(h-1, tmtime.Now()).StripExtensions() block := factory.MakeBlock(state, h, blockCommit) partSet, err := block.MakePartSet(2) require.NoError(t, err) - seenCommit := makeTestCommit(h, tmtime.Now()) - bs.SaveBlock(block, partSet, seenCommit) - c3 := bs.LoadSeenCommit() + seenCommit := makeTestExtCommit(h, tmtime.Now()) + store.SaveBlock(block, partSet, seenCommit) + c3 := store.LoadSeenCommit() require.NotNil(t, c3) require.Equal(t, h, c3.Height) - require.Equal(t, seenCommit.Hash(), c3.Hash()) - c5 := bs.LoadBlockCommit(h) + require.Equal(t, seenCommit.StripExtensions().Hash(), c3.Hash()) + c5 := store.LoadBlockCommit(h) require.Nil(t, c5) - c6 := bs.LoadBlockCommit(h - 1) + c6 := store.LoadBlockCommit(h - 1) require.Equal(t, blockCommit.Hash(), c6.Hash()) } diff --git a/internal/test/factory/commit.go b/internal/test/factory/commit.go index bc4022499..1bd1c7ae6 100644 --- a/internal/test/factory/commit.go +++ b/internal/test/factory/commit.go @@ -8,7 +8,7 @@ import ( "github.com/tendermint/tendermint/types" ) -func MakeCommit(ctx context.Context, blockID types.BlockID, height int64, round int32, voteSet *types.VoteSet, validators []types.PrivValidator, now time.Time) (*types.Commit, error) { +func MakeExtendedCommit(ctx context.Context, blockID types.BlockID, height int64, round int32, voteSet *types.VoteSet, validators []types.PrivValidator, now time.Time) (*types.ExtendedCommit, error) { // all sign for i := 0; i < len(validators); i++ { pubKey, err := validators[i].GetPubKey(ctx) @@ -37,5 +37,5 @@ func MakeCommit(ctx context.Context, blockID types.BlockID, height int64, round } } - return voteSet.MakeCommit(), nil + return voteSet.MakeExtendedCommit(), nil } diff --git a/light/helpers_test.go b/light/helpers_test.go index d93735bb7..9187cc3c3 100644 --- a/light/helpers_test.go +++ b/light/helpers_test.go @@ -72,7 +72,12 @@ func (pkz privKeys) signHeader(t testing.TB, header *types.Header, valSet *types commitSigs[vote.ValidatorIndex] = vote.CommitSig() } - return types.NewCommit(header.Height, 1, blockID, commitSigs) + return &types.Commit{ + Height: header.Height, + Round: 1, + BlockID: blockID, + Signatures: commitSigs, + } } func makeVote(t testing.TB, header *types.Header, valset *types.ValidatorSet, key crypto.PrivKey, blockID types.BlockID) *types.Vote { diff --git a/node/node_test.go b/node/node_test.go index c5ff1f014..b1d7a9481 100644 --- a/node/node_test.go +++ b/node/node_test.go @@ -35,6 +35,7 @@ import ( "github.com/tendermint/tendermint/libs/service" tmtime "github.com/tendermint/tendermint/libs/time" "github.com/tendermint/tendermint/privval" + tmproto "github.com/tendermint/tendermint/proto/tendermint/types" "github.com/tendermint/tendermint/types" ) @@ -339,13 +340,13 @@ func TestCreateProposalBlock(t *testing.T) { sm.NopMetrics(), ) - commit := types.NewCommit(height-1, 0, types.BlockID{}, nil) + extCommit := &types.ExtendedCommit{Height: height - 1} block, err := blockExec.CreateProposalBlock( ctx, height, - state, commit, + state, + extCommit, proposerAddr, - nil, ) require.NoError(t, err) @@ -419,13 +420,13 @@ func TestMaxTxsProposalBlockSize(t *testing.T) { sm.NopMetrics(), ) - commit := types.NewCommit(height-1, 0, types.BlockID{}, nil) + extCommit := &types.ExtendedCommit{Height: height - 1} block, err := blockExec.CreateProposalBlock( ctx, height, - state, commit, + state, + extCommit, proposerAddr, - nil, ) require.NoError(t, err) @@ -525,38 +526,41 @@ func TestMaxProposalBlockSize(t *testing.T) { } state.ChainID = maxChainID - cs := types.CommitSig{ - BlockIDFlag: types.BlockIDFlagNil, - ValidatorAddress: crypto.AddressHash([]byte("validator_address")), - Timestamp: timestamp, - Signature: crypto.CRandBytes(types.MaxSignatureSize), - } - - commit := &types.Commit{ - Height: math.MaxInt64, - Round: math.MaxInt32, - BlockID: blockID, - } - - votes := make([]*types.Vote, types.MaxVotesCount) + voteSet := types.NewVoteSet(state.ChainID, math.MaxInt64-1, math.MaxInt32, tmproto.PrecommitType, state.Validators) // add maximum amount of signatures to a single commit for i := 0; i < types.MaxVotesCount; i++ { pubKey, err := privVals[i].GetPubKey(ctx) require.NoError(t, err) - votes[i] = &types.Vote{ - ValidatorAddress: pubKey.Address(), + valIdx, val := state.Validators.GetByAddress(pubKey.Address()) + require.NotNil(t, val) + + vote := &types.Vote{ + Type: tmproto.PrecommitType, + Height: math.MaxInt64 - 1, + Round: math.MaxInt32, + BlockID: blockID, + Timestamp: timestamp, + ValidatorAddress: val.Address, + ValidatorIndex: valIdx, + Extension: []byte("extension"), } - commit.Signatures = append(commit.Signatures, cs) + vpb := vote.ToProto() + require.NoError(t, privVals[i].SignVote(ctx, state.ChainID, vpb)) + vote.Signature = vpb.Signature + vote.ExtensionSignature = vpb.ExtensionSignature + + added, err := voteSet.AddVote(vote) + require.NoError(t, err) + require.True(t, added) } block, err := blockExec.CreateProposalBlock( ctx, math.MaxInt64, state, - commit, + voteSet.MakeExtendedCommit(), proposerAddr, - votes, ) require.NoError(t, err) partSet, err := block.MakePartSet(types.BlockPartSizeBytes) diff --git a/privval/file.go b/privval/file.go index bf5803632..a5d696093 100644 --- a/privval/file.go +++ b/privval/file.go @@ -375,17 +375,17 @@ func (pv *FilePV) signVote(chainID string, vote *tmproto.Vote) error { // Vote extensions are non-deterministic, so it is possible that an // application may have created a different extension. We therefore always - // re-sign the vote extensions of precommits. For prevotes, the extension - // signature will always be empty. + // re-sign the vote extensions of precommits. For prevotes and nil + // precommits, the extension signature will always be empty. var extSig []byte - if vote.Type == tmproto.PrecommitType { + if vote.Type == tmproto.PrecommitType && !types.ProtoBlockIDIsNil(&vote.BlockID) { extSignBytes := types.VoteExtensionSignBytes(chainID, vote) extSig, err = pv.Key.PrivKey.Sign(extSignBytes) if err != nil { return err } } else if len(vote.Extension) > 0 { - return errors.New("unexpected vote extension - extensions are only allowed in precommits") + return errors.New("unexpected vote extension - extensions are only allowed in non-nil precommits") } // We might crash before writing to the wal, diff --git a/proto/tendermint/blocksync/types.pb.go b/proto/tendermint/blocksync/types.pb.go index c00200322..8757f8ab3 100644 --- a/proto/tendermint/blocksync/types.pb.go +++ b/proto/tendermint/blocksync/types.pb.go @@ -116,7 +116,8 @@ func (m *NoBlockResponse) GetHeight() int64 { // BlockResponse returns block to the requested type BlockResponse struct { - Block *types.Block `protobuf:"bytes,1,opt,name=block,proto3" json:"block,omitempty"` + Block *types.Block `protobuf:"bytes,1,opt,name=block,proto3" json:"block,omitempty"` + ExtCommit *types.ExtendedCommit `protobuf:"bytes,2,opt,name=ext_commit,json=extCommit,proto3" json:"ext_commit,omitempty"` } func (m *BlockResponse) Reset() { *m = BlockResponse{} } @@ -159,6 +160,13 @@ func (m *BlockResponse) GetBlock() *types.Block { return nil } +func (m *BlockResponse) GetExtCommit() *types.ExtendedCommit { + if m != nil { + return m.ExtCommit + } + return nil +} + // StatusRequest requests the status of a peer. type StatusRequest struct { } @@ -385,30 +393,33 @@ func init() { func init() { proto.RegisterFile("tendermint/blocksync/types.proto", fileDescriptor_19b397c236e0fa07) } var fileDescriptor_19b397c236e0fa07 = []byte{ - // 368 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x93, 0x4d, 0x4f, 0xfa, 0x40, - 0x10, 0xc6, 0xdb, 0x7f, 0x81, 0x7f, 0x32, 0x50, 0x1a, 0x1b, 0xa3, 0xc4, 0x98, 0x86, 0xd4, 0x97, - 0xe8, 0xc1, 0x36, 0xc1, 0xa3, 0xc6, 0x03, 0x27, 0x4c, 0x7c, 0x49, 0x4a, 0xbc, 0x78, 0x21, 0x14, - 0x37, 0x40, 0x94, 0x2e, 0x32, 0xdb, 0x03, 0xdf, 0xc2, 0x2f, 0xe0, 0xf7, 0xf1, 0xc8, 0xd1, 0xa3, - 0x81, 0x2f, 0x62, 0x98, 0x2d, 0x65, 0x69, 0xb0, 0xb7, 0xdd, 0xe9, 0x33, 0xbf, 0x79, 0xfa, 0x64, - 0x16, 0xea, 0x82, 0x45, 0x2f, 0x6c, 0x32, 0x1a, 0x46, 0xc2, 0x0f, 0xdf, 0x78, 0xef, 0x15, 0xa7, - 0x51, 0xcf, 0x17, 0xd3, 0x31, 0x43, 0x6f, 0x3c, 0xe1, 0x82, 0xdb, 0xbb, 0x6b, 0x85, 0x97, 0x2a, - 0x0e, 0x0e, 0x95, 0x3e, 0x52, 0xcb, 0x6e, 0xd9, 0xe3, 0x9e, 0x42, 0xa5, 0xb9, 0xbc, 0x06, 0xec, - 0x3d, 0x66, 0x28, 0xec, 0x3d, 0x28, 0x0d, 0xd8, 0xb0, 0x3f, 0x10, 0x35, 0xbd, 0xae, 0x9f, 0x19, - 0x41, 0x72, 0x73, 0xcf, 0xc1, 0x7a, 0xe0, 0x89, 0x12, 0xc7, 0x3c, 0x42, 0xf6, 0xa7, 0xf4, 0x06, - 0xcc, 0x4d, 0xe1, 0x05, 0x14, 0x69, 0x24, 0xe9, 0xca, 0x8d, 0x7d, 0x4f, 0xf1, 0x29, 0xfd, 0x4b, - 0xbd, 0x54, 0xb9, 0x16, 0x98, 0x6d, 0xd1, 0x15, 0x31, 0x26, 0x9e, 0xdc, 0x6b, 0xa8, 0xae, 0x0a, - 0xf9, 0xa3, 0x6d, 0x1b, 0x0a, 0x61, 0x17, 0x59, 0xed, 0x1f, 0x55, 0xe9, 0xec, 0x7e, 0x1a, 0xf0, - 0xff, 0x9e, 0x21, 0x76, 0xfb, 0xcc, 0xbe, 0x05, 0x93, 0x66, 0x74, 0x26, 0x12, 0x9d, 0x38, 0x72, - 0xbd, 0x6d, 0xc9, 0x79, 0x6a, 0x30, 0x2d, 0x2d, 0xa8, 0x84, 0x6a, 0x50, 0x6d, 0xd8, 0x89, 0x78, - 0x67, 0x45, 0x93, 0xbe, 0x68, 0x6e, 0xb9, 0x71, 0xb2, 0x1d, 0x97, 0xc9, 0xaf, 0xa5, 0x05, 0x56, - 0x94, 0x89, 0xf4, 0x0e, 0xaa, 0x19, 0xa2, 0x41, 0xc4, 0xa3, 0x5c, 0x83, 0x29, 0xcf, 0x0c, 0xb3, - 0x34, 0xa4, 0xdc, 0xd2, 0xdf, 0x2d, 0xe4, 0xd1, 0x36, 0x42, 0x5f, 0xd2, 0x50, 0x2d, 0xd8, 0x8f, - 0x60, 0xa5, 0xb4, 0xc4, 0x5c, 0x91, 0x70, 0xc7, 0xf9, 0xb8, 0xd4, 0x5d, 0x15, 0x37, 0x2a, 0xcd, - 0x22, 0x18, 0x18, 0x8f, 0x9a, 0x4f, 0x5f, 0x73, 0x47, 0x9f, 0xcd, 0x1d, 0xfd, 0x67, 0xee, 0xe8, - 0x1f, 0x0b, 0x47, 0x9b, 0x2d, 0x1c, 0xed, 0x7b, 0xe1, 0x68, 0xcf, 0x57, 0xfd, 0xa1, 0x18, 0xc4, - 0xa1, 0xd7, 0xe3, 0x23, 0x5f, 0x5d, 0xe2, 0xf5, 0x91, 0x76, 0xd8, 0xdf, 0xf6, 0x30, 0xc2, 0x12, - 0x7d, 0xbb, 0xfc, 0x0d, 0x00, 0x00, 0xff, 0xff, 0xf5, 0x1c, 0xa3, 0x45, 0x37, 0x03, 0x00, 0x00, + // 404 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x93, 0xcd, 0x4a, 0xeb, 0x50, + 0x10, 0xc7, 0x93, 0x9b, 0xb6, 0x97, 0x3b, 0xb7, 0x69, 0xb8, 0xe1, 0xa2, 0x45, 0x24, 0x94, 0xf8, + 0x81, 0x2e, 0x4c, 0x40, 0x97, 0x0a, 0x42, 0x45, 0xa8, 0xe0, 0x07, 0xa4, 0xb8, 0x71, 0x53, 0x9a, + 0xf4, 0xd0, 0x06, 0x4d, 0x4e, 0xed, 0x39, 0x81, 0x76, 0xe5, 0x2b, 0xf8, 0x02, 0xbe, 0x8f, 0xcb, + 0x2e, 0x5d, 0x4a, 0xfb, 0x22, 0xd2, 0x39, 0x69, 0x9a, 0xc6, 0x98, 0xdd, 0x64, 0xce, 0x7f, 0x7e, + 0xf9, 0xcf, 0x0c, 0x03, 0x0d, 0x4e, 0xc2, 0x1e, 0x19, 0x05, 0x7e, 0xc8, 0x6d, 0xf7, 0x89, 0x7a, + 0x8f, 0x6c, 0x12, 0x7a, 0x36, 0x9f, 0x0c, 0x09, 0xb3, 0x86, 0x23, 0xca, 0xa9, 0xfe, 0x7f, 0xa5, + 0xb0, 0x12, 0xc5, 0xd6, 0x76, 0xaa, 0x0e, 0xd5, 0xa2, 0x5a, 0xd4, 0xe4, 0xbc, 0xa6, 0x88, 0xe6, + 0x3e, 0x54, 0x9b, 0x0b, 0xb1, 0x43, 0x9e, 0x23, 0xc2, 0xb8, 0xbe, 0x01, 0x95, 0x01, 0xf1, 0xfb, + 0x03, 0x5e, 0x97, 0x1b, 0xf2, 0x81, 0xe2, 0xc4, 0x5f, 0xe6, 0x21, 0x68, 0xb7, 0x34, 0x56, 0xb2, + 0x21, 0x0d, 0x19, 0xf9, 0x51, 0xfa, 0x02, 0xea, 0xba, 0xf0, 0x08, 0xca, 0x68, 0x08, 0x75, 0x7f, + 0x8f, 0x37, 0xad, 0x54, 0x17, 0xc2, 0x8b, 0xd0, 0x0b, 0x95, 0x7e, 0x0e, 0x40, 0xc6, 0xbc, 0xe3, + 0xd1, 0x20, 0xf0, 0x79, 0xfd, 0x17, 0xd6, 0x34, 0xbe, 0xd7, 0x5c, 0x8e, 0x31, 0xd5, 0xbb, 0x40, + 0x9d, 0xf3, 0x87, 0x8c, 0xb9, 0x08, 0x4d, 0x0d, 0xd4, 0x36, 0xef, 0xf2, 0x88, 0xc5, 0x4d, 0x99, + 0x67, 0x50, 0x5b, 0x26, 0x8a, 0xbd, 0xeb, 0x3a, 0x94, 0xdc, 0x2e, 0x23, 0xf8, 0x57, 0xc5, 0xc1, + 0xd8, 0x7c, 0x53, 0xe0, 0xf7, 0x0d, 0x61, 0xac, 0xdb, 0x27, 0xfa, 0x15, 0xa8, 0x68, 0xb2, 0x33, + 0x12, 0xe8, 0xb8, 0x25, 0xd3, 0xca, 0x5b, 0x8c, 0x95, 0x9e, 0x6c, 0x4b, 0x72, 0xaa, 0x6e, 0x7a, + 0xd2, 0x6d, 0xf8, 0x17, 0xd2, 0xce, 0x92, 0x26, 0x7c, 0xc5, 0xdd, 0xee, 0xe5, 0xe3, 0x32, 0x0b, + 0x68, 0x49, 0x8e, 0x16, 0x66, 0x76, 0x72, 0x0d, 0xb5, 0x0c, 0x51, 0x41, 0xe2, 0x4e, 0xa1, 0xc1, + 0x84, 0xa7, 0xba, 0x59, 0x1a, 0xc3, 0xb9, 0x25, 0xed, 0x96, 0x8a, 0x68, 0x6b, 0x43, 0x5f, 0xd0, + 0x58, 0x3a, 0xa1, 0xdf, 0x81, 0x96, 0xd0, 0x62, 0x73, 0x65, 0xc4, 0xed, 0x16, 0xe3, 0x12, 0x77, + 0x35, 0xb6, 0x96, 0x69, 0x96, 0x41, 0x61, 0x51, 0xd0, 0xbc, 0x7f, 0x9f, 0x19, 0xf2, 0x74, 0x66, + 0xc8, 0x9f, 0x33, 0x43, 0x7e, 0x9d, 0x1b, 0xd2, 0x74, 0x6e, 0x48, 0x1f, 0x73, 0x43, 0x7a, 0x38, + 0xed, 0xfb, 0x7c, 0x10, 0xb9, 0x96, 0x47, 0x03, 0x3b, 0x7d, 0x05, 0xab, 0x10, 0x8f, 0xc0, 0xce, + 0xbb, 0x3b, 0xb7, 0x82, 0x6f, 0x27, 0x5f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xdc, 0x13, 0x4f, 0x42, + 0x96, 0x03, 0x00, 0x00, } func (m *BlockRequest) Marshal() (dAtA []byte, err error) { @@ -487,6 +498,18 @@ func (m *BlockResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.ExtCommit != nil { + { + size, err := m.ExtCommit.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } if m.Block != nil { { size, err := m.Block.MarshalToSizedBuffer(dAtA[:i]) @@ -740,6 +763,10 @@ func (m *BlockResponse) Size() (n int) { l = m.Block.Size() n += 1 + l + sovTypes(uint64(l)) } + if m.ExtCommit != nil { + l = m.ExtCommit.Size() + n += 1 + l + sovTypes(uint64(l)) + } return n } @@ -1049,6 +1076,42 @@ func (m *BlockResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtCommit", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.ExtCommit == nil { + m.ExtCommit = &types.ExtendedCommit{} + } + if err := m.ExtCommit.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipTypes(dAtA[iNdEx:]) diff --git a/proto/tendermint/blocksync/types.proto b/proto/tendermint/blocksync/types.proto index 4febfd145..67da76dce 100644 --- a/proto/tendermint/blocksync/types.proto +++ b/proto/tendermint/blocksync/types.proto @@ -4,6 +4,7 @@ package tendermint.blocksync; option go_package = "github.com/tendermint/tendermint/proto/tendermint/blocksync"; import "tendermint/types/block.proto"; +import "tendermint/types/types.proto"; // BlockRequest requests a block for a specific height message BlockRequest { @@ -19,6 +20,7 @@ message NoBlockResponse { // BlockResponse returns block to the requested message BlockResponse { tendermint.types.Block block = 1; + tendermint.types.ExtendedCommit ext_commit = 2; } // StatusRequest requests the status of a peer. diff --git a/proto/tendermint/types/types.pb.go b/proto/tendermint/types/types.pb.go index 1904afcd1..fcfbc01f5 100644 --- a/proto/tendermint/types/types.pb.go +++ b/proto/tendermint/types/types.pb.go @@ -726,6 +726,162 @@ func (m *CommitSig) GetSignature() []byte { return nil } +type ExtendedCommit struct { + Height int64 `protobuf:"varint,1,opt,name=height,proto3" json:"height,omitempty"` + Round int32 `protobuf:"varint,2,opt,name=round,proto3" json:"round,omitempty"` + BlockID BlockID `protobuf:"bytes,3,opt,name=block_id,json=blockId,proto3" json:"block_id"` + ExtendedSignatures []ExtendedCommitSig `protobuf:"bytes,4,rep,name=extended_signatures,json=extendedSignatures,proto3" json:"extended_signatures"` +} + +func (m *ExtendedCommit) Reset() { *m = ExtendedCommit{} } +func (m *ExtendedCommit) String() string { return proto.CompactTextString(m) } +func (*ExtendedCommit) ProtoMessage() {} +func (*ExtendedCommit) Descriptor() ([]byte, []int) { + return fileDescriptor_d3a6e55e2345de56, []int{8} +} +func (m *ExtendedCommit) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExtendedCommit) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ExtendedCommit.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ExtendedCommit) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExtendedCommit.Merge(m, src) +} +func (m *ExtendedCommit) XXX_Size() int { + return m.Size() +} +func (m *ExtendedCommit) XXX_DiscardUnknown() { + xxx_messageInfo_ExtendedCommit.DiscardUnknown(m) +} + +var xxx_messageInfo_ExtendedCommit proto.InternalMessageInfo + +func (m *ExtendedCommit) GetHeight() int64 { + if m != nil { + return m.Height + } + return 0 +} + +func (m *ExtendedCommit) GetRound() int32 { + if m != nil { + return m.Round + } + return 0 +} + +func (m *ExtendedCommit) GetBlockID() BlockID { + if m != nil { + return m.BlockID + } + return BlockID{} +} + +func (m *ExtendedCommit) GetExtendedSignatures() []ExtendedCommitSig { + if m != nil { + return m.ExtendedSignatures + } + return nil +} + +// ExtendedCommitSig retains all the same fields as CommitSig but adds vote +// extension-related fields. +type ExtendedCommitSig struct { + BlockIdFlag BlockIDFlag `protobuf:"varint,1,opt,name=block_id_flag,json=blockIdFlag,proto3,enum=tendermint.types.BlockIDFlag" json:"block_id_flag,omitempty"` + ValidatorAddress []byte `protobuf:"bytes,2,opt,name=validator_address,json=validatorAddress,proto3" json:"validator_address,omitempty"` + Timestamp time.Time `protobuf:"bytes,3,opt,name=timestamp,proto3,stdtime" json:"timestamp"` + Signature []byte `protobuf:"bytes,4,opt,name=signature,proto3" json:"signature,omitempty"` + // Vote extension data + Extension []byte `protobuf:"bytes,5,opt,name=extension,proto3" json:"extension,omitempty"` + // Vote extension signature + ExtensionSignature []byte `protobuf:"bytes,6,opt,name=extension_signature,json=extensionSignature,proto3" json:"extension_signature,omitempty"` +} + +func (m *ExtendedCommitSig) Reset() { *m = ExtendedCommitSig{} } +func (m *ExtendedCommitSig) String() string { return proto.CompactTextString(m) } +func (*ExtendedCommitSig) ProtoMessage() {} +func (*ExtendedCommitSig) Descriptor() ([]byte, []int) { + return fileDescriptor_d3a6e55e2345de56, []int{9} +} +func (m *ExtendedCommitSig) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExtendedCommitSig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ExtendedCommitSig.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ExtendedCommitSig) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExtendedCommitSig.Merge(m, src) +} +func (m *ExtendedCommitSig) XXX_Size() int { + return m.Size() +} +func (m *ExtendedCommitSig) XXX_DiscardUnknown() { + xxx_messageInfo_ExtendedCommitSig.DiscardUnknown(m) +} + +var xxx_messageInfo_ExtendedCommitSig proto.InternalMessageInfo + +func (m *ExtendedCommitSig) GetBlockIdFlag() BlockIDFlag { + if m != nil { + return m.BlockIdFlag + } + return BlockIDFlagUnknown +} + +func (m *ExtendedCommitSig) GetValidatorAddress() []byte { + if m != nil { + return m.ValidatorAddress + } + return nil +} + +func (m *ExtendedCommitSig) GetTimestamp() time.Time { + if m != nil { + return m.Timestamp + } + return time.Time{} +} + +func (m *ExtendedCommitSig) GetSignature() []byte { + if m != nil { + return m.Signature + } + return nil +} + +func (m *ExtendedCommitSig) GetExtension() []byte { + if m != nil { + return m.Extension + } + return nil +} + +func (m *ExtendedCommitSig) GetExtensionSignature() []byte { + if m != nil { + return m.ExtensionSignature + } + return nil +} + type Proposal struct { Type SignedMsgType `protobuf:"varint,1,opt,name=type,proto3,enum=tendermint.types.SignedMsgType" json:"type,omitempty"` Height int64 `protobuf:"varint,2,opt,name=height,proto3" json:"height,omitempty"` @@ -740,7 +896,7 @@ func (m *Proposal) Reset() { *m = Proposal{} } func (m *Proposal) String() string { return proto.CompactTextString(m) } func (*Proposal) ProtoMessage() {} func (*Proposal) Descriptor() ([]byte, []int) { - return fileDescriptor_d3a6e55e2345de56, []int{8} + return fileDescriptor_d3a6e55e2345de56, []int{10} } func (m *Proposal) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -827,7 +983,7 @@ func (m *SignedHeader) Reset() { *m = SignedHeader{} } func (m *SignedHeader) String() string { return proto.CompactTextString(m) } func (*SignedHeader) ProtoMessage() {} func (*SignedHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_d3a6e55e2345de56, []int{9} + return fileDescriptor_d3a6e55e2345de56, []int{11} } func (m *SignedHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -879,7 +1035,7 @@ func (m *LightBlock) Reset() { *m = LightBlock{} } func (m *LightBlock) String() string { return proto.CompactTextString(m) } func (*LightBlock) ProtoMessage() {} func (*LightBlock) Descriptor() ([]byte, []int) { - return fileDescriptor_d3a6e55e2345de56, []int{10} + return fileDescriptor_d3a6e55e2345de56, []int{12} } func (m *LightBlock) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -933,7 +1089,7 @@ func (m *BlockMeta) Reset() { *m = BlockMeta{} } func (m *BlockMeta) String() string { return proto.CompactTextString(m) } func (*BlockMeta) ProtoMessage() {} func (*BlockMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_d3a6e55e2345de56, []int{11} + return fileDescriptor_d3a6e55e2345de56, []int{13} } func (m *BlockMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1002,7 +1158,7 @@ func (m *TxProof) Reset() { *m = TxProof{} } func (m *TxProof) String() string { return proto.CompactTextString(m) } func (*TxProof) ProtoMessage() {} func (*TxProof) Descriptor() ([]byte, []int) { - return fileDescriptor_d3a6e55e2345de56, []int{12} + return fileDescriptor_d3a6e55e2345de56, []int{14} } func (m *TxProof) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1063,6 +1219,8 @@ func init() { proto.RegisterType((*Vote)(nil), "tendermint.types.Vote") proto.RegisterType((*Commit)(nil), "tendermint.types.Commit") proto.RegisterType((*CommitSig)(nil), "tendermint.types.CommitSig") + proto.RegisterType((*ExtendedCommit)(nil), "tendermint.types.ExtendedCommit") + proto.RegisterType((*ExtendedCommitSig)(nil), "tendermint.types.ExtendedCommitSig") proto.RegisterType((*Proposal)(nil), "tendermint.types.Proposal") proto.RegisterType((*SignedHeader)(nil), "tendermint.types.SignedHeader") proto.RegisterType((*LightBlock)(nil), "tendermint.types.LightBlock") @@ -1073,91 +1231,95 @@ func init() { func init() { proto.RegisterFile("tendermint/types/types.proto", fileDescriptor_d3a6e55e2345de56) } var fileDescriptor_d3a6e55e2345de56 = []byte{ - // 1341 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x57, 0xcf, 0x73, 0xdb, 0xc4, - 0x17, 0x8f, 0x62, 0x25, 0xb6, 0x9f, 0xed, 0xc4, 0xd9, 0x6f, 0xda, 0xba, 0x6e, 0xe3, 0x68, 0xfc, - 0x1d, 0x20, 0x2d, 0x8c, 0x52, 0x52, 0x86, 0x1f, 0x07, 0x0e, 0xb6, 0x93, 0xb6, 0x9e, 0x26, 0x8e, - 0x91, 0xdd, 0x32, 0x70, 0xd1, 0xc8, 0xd6, 0xd6, 0x16, 0x95, 0x25, 0x8d, 0x76, 0x1d, 0x92, 0xfe, - 0x05, 0x4c, 0x4e, 0x3d, 0x71, 0xcb, 0x09, 0x0e, 0xdc, 0x39, 0x70, 0x65, 0x38, 0xf5, 0xd8, 0x1b, - 0x5c, 0x28, 0x4c, 0x3a, 0xc3, 0xdf, 0xc1, 0xec, 0x0f, 0xc9, 0x72, 0x9c, 0x40, 0xa7, 0xd3, 0xe1, - 0xe2, 0xd1, 0xbe, 0xf7, 0x79, 0x6f, 0xdf, 0x8f, 0xcf, 0xee, 0x5b, 0xc3, 0x75, 0x8a, 0x3d, 0x1b, - 0x87, 0x23, 0xc7, 0xa3, 0x9b, 0xf4, 0x28, 0xc0, 0x44, 0xfc, 0xea, 0x41, 0xe8, 0x53, 0x1f, 0x15, - 0x27, 0x5a, 0x9d, 0xcb, 0xcb, 0xab, 0x03, 0x7f, 0xe0, 0x73, 0xe5, 0x26, 0xfb, 0x12, 0xb8, 0xf2, - 0xfa, 0xc0, 0xf7, 0x07, 0x2e, 0xde, 0xe4, 0xab, 0xde, 0xf8, 0xd1, 0x26, 0x75, 0x46, 0x98, 0x50, - 0x6b, 0x14, 0x48, 0xc0, 0x5a, 0x62, 0x9b, 0x7e, 0x78, 0x14, 0x50, 0x9f, 0x61, 0xfd, 0x47, 0x52, - 0x5d, 0x49, 0xa8, 0x0f, 0x70, 0x48, 0x1c, 0xdf, 0x4b, 0xc6, 0x51, 0xd6, 0x66, 0xa2, 0x3c, 0xb0, - 0x5c, 0xc7, 0xb6, 0xa8, 0x1f, 0x0a, 0x44, 0xf5, 0x13, 0x28, 0xb4, 0xad, 0x90, 0x76, 0x30, 0xbd, - 0x87, 0x2d, 0x1b, 0x87, 0x68, 0x15, 0x16, 0xa8, 0x4f, 0x2d, 0xb7, 0xa4, 0x68, 0xca, 0x46, 0xc1, - 0x10, 0x0b, 0x84, 0x40, 0x1d, 0x5a, 0x64, 0x58, 0x9a, 0xd7, 0x94, 0x8d, 0xbc, 0xc1, 0xbf, 0xab, - 0x43, 0x50, 0x99, 0x29, 0xb3, 0x70, 0x3c, 0x1b, 0x1f, 0x46, 0x16, 0x7c, 0xc1, 0xa4, 0xbd, 0x23, - 0x8a, 0x89, 0x34, 0x11, 0x0b, 0xf4, 0x01, 0x2c, 0xf0, 0xf8, 0x4b, 0x29, 0x4d, 0xd9, 0xc8, 0x6d, - 0x95, 0xf4, 0x44, 0xa1, 0x44, 0x7e, 0x7a, 0x9b, 0xe9, 0xeb, 0xea, 0xb3, 0x17, 0xeb, 0x73, 0x86, - 0x00, 0x57, 0x5d, 0x48, 0xd7, 0x5d, 0xbf, 0xff, 0xb8, 0xb9, 0x1d, 0x07, 0xa2, 0x4c, 0x02, 0x41, - 0x7b, 0xb0, 0x1c, 0x58, 0x21, 0x35, 0x09, 0xa6, 0xe6, 0x90, 0x67, 0xc1, 0x37, 0xcd, 0x6d, 0xad, - 0xeb, 0x67, 0xfb, 0xa0, 0x4f, 0x25, 0x2b, 0x77, 0x29, 0x04, 0x49, 0x61, 0xf5, 0x2f, 0x15, 0x16, - 0x65, 0x31, 0x3e, 0x85, 0xb4, 0x2c, 0x2b, 0xdf, 0x30, 0xb7, 0xb5, 0x96, 0xf4, 0x28, 0x55, 0x7a, - 0xc3, 0xf7, 0x08, 0xf6, 0xc8, 0x98, 0x48, 0x7f, 0x91, 0x0d, 0x7a, 0x1b, 0x32, 0xfd, 0xa1, 0xe5, - 0x78, 0xa6, 0x63, 0xf3, 0x88, 0xb2, 0xf5, 0xdc, 0xe9, 0x8b, 0xf5, 0x74, 0x83, 0xc9, 0x9a, 0xdb, - 0x46, 0x9a, 0x2b, 0x9b, 0x36, 0xba, 0x0c, 0x8b, 0x43, 0xec, 0x0c, 0x86, 0x94, 0x97, 0x25, 0x65, - 0xc8, 0x15, 0xfa, 0x18, 0x54, 0x46, 0x88, 0x92, 0xca, 0xf7, 0x2e, 0xeb, 0x82, 0x2d, 0x7a, 0xc4, - 0x16, 0xbd, 0x1b, 0xb1, 0xa5, 0x9e, 0x61, 0x1b, 0x3f, 0xfd, 0x63, 0x5d, 0x31, 0xb8, 0x05, 0x6a, - 0x40, 0xc1, 0xb5, 0x08, 0x35, 0x7b, 0xac, 0x6c, 0x6c, 0xfb, 0x05, 0xee, 0xe2, 0xea, 0x6c, 0x41, - 0x64, 0x61, 0x65, 0xe8, 0x39, 0x66, 0x25, 0x44, 0x36, 0xda, 0x80, 0x22, 0x77, 0xd2, 0xf7, 0x47, - 0x23, 0x87, 0x9a, 0xbc, 0xee, 0x8b, 0xbc, 0xee, 0x4b, 0x4c, 0xde, 0xe0, 0xe2, 0x7b, 0xac, 0x03, - 0xd7, 0x20, 0x6b, 0x5b, 0xd4, 0x12, 0x90, 0x34, 0x87, 0x64, 0x98, 0x80, 0x2b, 0xdf, 0x81, 0xe5, - 0x98, 0x75, 0x44, 0x40, 0x32, 0xc2, 0xcb, 0x44, 0xcc, 0x81, 0xb7, 0x60, 0xd5, 0xc3, 0x87, 0xd4, - 0x3c, 0x8b, 0xce, 0x72, 0x34, 0x62, 0xba, 0x87, 0xd3, 0x16, 0x6f, 0xc1, 0x52, 0x3f, 0x2a, 0xbe, - 0xc0, 0x02, 0xc7, 0x16, 0x62, 0x29, 0x87, 0x5d, 0x85, 0x8c, 0x15, 0x04, 0x02, 0x90, 0xe3, 0x80, - 0xb4, 0x15, 0x04, 0x5c, 0x75, 0x13, 0x56, 0x78, 0x8e, 0x21, 0x26, 0x63, 0x97, 0x4a, 0x27, 0x79, - 0x8e, 0x59, 0x66, 0x0a, 0x43, 0xc8, 0x39, 0xf6, 0xff, 0x50, 0xc0, 0x07, 0x8e, 0x8d, 0xbd, 0x3e, - 0x16, 0xb8, 0x02, 0xc7, 0xe5, 0x23, 0x21, 0x07, 0xdd, 0x80, 0x62, 0x10, 0xfa, 0x81, 0x4f, 0x70, - 0x68, 0x5a, 0xb6, 0x1d, 0x62, 0x42, 0x4a, 0x4b, 0xc2, 0x5f, 0x24, 0xaf, 0x09, 0x71, 0xb5, 0x04, - 0xea, 0xb6, 0x45, 0x2d, 0x54, 0x84, 0x14, 0x3d, 0x24, 0x25, 0x45, 0x4b, 0x6d, 0xe4, 0x0d, 0xf6, - 0x59, 0xfd, 0x29, 0x05, 0xea, 0x43, 0x9f, 0x62, 0x74, 0x1b, 0x54, 0xd6, 0x26, 0xce, 0xbe, 0xa5, - 0xf3, 0xf8, 0xdc, 0x71, 0x06, 0x1e, 0xb6, 0xf7, 0xc8, 0xa0, 0x7b, 0x14, 0x60, 0x83, 0x83, 0x13, - 0x74, 0x9a, 0x9f, 0xa2, 0xd3, 0x2a, 0x2c, 0x84, 0xfe, 0xd8, 0xb3, 0x39, 0xcb, 0x16, 0x0c, 0xb1, - 0x40, 0x3b, 0x90, 0x89, 0x59, 0xa2, 0xfe, 0x1b, 0x4b, 0x96, 0x19, 0x4b, 0x18, 0x87, 0xa5, 0xc0, - 0x48, 0xf7, 0x24, 0x59, 0xea, 0x90, 0x8d, 0x2f, 0x2f, 0xc9, 0xb6, 0x57, 0x23, 0xec, 0xc4, 0x0c, - 0xbd, 0x0b, 0x2b, 0x71, 0xef, 0xe3, 0xe2, 0x09, 0xc6, 0x15, 0x63, 0x85, 0xac, 0xde, 0x14, 0xad, - 0x4c, 0x71, 0x01, 0xa5, 0x79, 0x5e, 0x13, 0x5a, 0x35, 0xf9, 0x4d, 0x74, 0x1d, 0xb2, 0xc4, 0x19, - 0x78, 0x16, 0x1d, 0x87, 0x58, 0x32, 0x6f, 0x22, 0x60, 0x5a, 0x7c, 0x48, 0xb1, 0xc7, 0x0f, 0xb9, - 0x60, 0xda, 0x44, 0x80, 0x36, 0xe1, 0x7f, 0xf1, 0xc2, 0x9c, 0x78, 0x11, 0x2c, 0x43, 0xb1, 0xaa, - 0x13, 0x69, 0xaa, 0x3f, 0x2b, 0xb0, 0x28, 0x0e, 0x46, 0xa2, 0x0d, 0xca, 0xf9, 0x6d, 0x98, 0xbf, - 0xa8, 0x0d, 0xa9, 0xd7, 0x6f, 0x43, 0x0d, 0x20, 0x0e, 0x93, 0x94, 0x54, 0x2d, 0xb5, 0x91, 0xdb, - 0xba, 0x36, 0xeb, 0x48, 0x84, 0xd8, 0x71, 0x06, 0xf2, 0xdc, 0x27, 0x8c, 0xaa, 0xbf, 0x2b, 0x90, - 0x8d, 0xf5, 0xa8, 0x06, 0x85, 0x28, 0x2e, 0xf3, 0x91, 0x6b, 0x0d, 0x24, 0x15, 0xd7, 0x2e, 0x0c, - 0xee, 0x8e, 0x6b, 0x0d, 0x8c, 0x9c, 0x8c, 0x87, 0x2d, 0xce, 0x6f, 0xeb, 0xfc, 0x05, 0x6d, 0x9d, - 0xe2, 0x51, 0xea, 0xf5, 0x78, 0x34, 0xd5, 0x71, 0xf5, 0x4c, 0xc7, 0xab, 0x3f, 0xce, 0x43, 0xa6, - 0xcd, 0x8f, 0xa2, 0xe5, 0xfe, 0x17, 0x07, 0xec, 0x1a, 0x64, 0x03, 0xdf, 0x35, 0x85, 0x46, 0xe5, - 0x9a, 0x4c, 0xe0, 0xbb, 0xc6, 0x4c, 0xdb, 0x17, 0xde, 0xd0, 0xe9, 0x5b, 0x7c, 0x03, 0x55, 0x4b, - 0x9f, 0xad, 0x5a, 0x08, 0x79, 0x51, 0x0a, 0x39, 0x1a, 0x6f, 0xb1, 0x1a, 0xf0, 0x59, 0xab, 0xcc, - 0x8e, 0x72, 0x11, 0xb6, 0x40, 0x1a, 0x12, 0xc7, 0x2c, 0xc4, 0x24, 0x91, 0xd3, 0xb9, 0x74, 0x11, - 0x2d, 0x0d, 0x89, 0xab, 0x7e, 0xab, 0x00, 0xec, 0xb2, 0xca, 0xf2, 0x7c, 0xd9, 0x50, 0x23, 0x3c, - 0x04, 0x73, 0x6a, 0xe7, 0xca, 0x45, 0x4d, 0x93, 0xfb, 0xe7, 0x49, 0x32, 0xee, 0x06, 0x14, 0x26, - 0x64, 0x24, 0x38, 0x0a, 0xe6, 0x1c, 0x27, 0xf1, 0xac, 0xe9, 0x60, 0x6a, 0xe4, 0x0f, 0x12, 0xab, - 0xea, 0x2f, 0x0a, 0x64, 0x79, 0x4c, 0x7b, 0x98, 0x5a, 0x53, 0x3d, 0x54, 0x5e, 0xbf, 0x87, 0x6b, - 0x00, 0xc2, 0x0d, 0x71, 0x9e, 0x60, 0xc9, 0xac, 0x2c, 0x97, 0x74, 0x9c, 0x27, 0x18, 0x7d, 0x18, - 0x17, 0x3c, 0xf5, 0xcf, 0x05, 0x97, 0x47, 0x3a, 0x2a, 0xfb, 0x15, 0x48, 0x7b, 0xe3, 0x91, 0xc9, - 0x26, 0x8c, 0x2a, 0xd8, 0xea, 0x8d, 0x47, 0xdd, 0x43, 0x52, 0xfd, 0x0a, 0xd2, 0xdd, 0x43, 0xfe, - 0xda, 0x62, 0x14, 0x0d, 0x7d, 0x5f, 0x8e, 0x78, 0xf1, 0xb4, 0xca, 0x30, 0x01, 0x9f, 0x68, 0x08, - 0x54, 0x36, 0xcb, 0xa3, 0xb7, 0x1f, 0xfb, 0x46, 0xfa, 0x2b, 0xbe, 0xe3, 0xe4, 0x0b, 0xee, 0xe6, - 0xaf, 0x0a, 0xe4, 0x12, 0xf7, 0x03, 0x7a, 0x1f, 0x2e, 0xd5, 0x77, 0xf7, 0x1b, 0xf7, 0xcd, 0xe6, - 0xb6, 0x79, 0x67, 0xb7, 0x76, 0xd7, 0x7c, 0xd0, 0xba, 0xdf, 0xda, 0xff, 0xbc, 0x55, 0x9c, 0x2b, - 0x5f, 0x3e, 0x3e, 0xd1, 0x50, 0x02, 0xfb, 0xc0, 0x7b, 0xec, 0xf9, 0x5f, 0xb3, 0xab, 0x78, 0x75, - 0xda, 0xa4, 0x56, 0xef, 0xec, 0xb4, 0xba, 0x45, 0xa5, 0x7c, 0xe9, 0xf8, 0x44, 0x5b, 0x49, 0x58, - 0xd4, 0x7a, 0x04, 0x7b, 0x74, 0xd6, 0xa0, 0xb1, 0xbf, 0xb7, 0xd7, 0xec, 0x16, 0xe7, 0x67, 0x0c, - 0xe4, 0x85, 0x7d, 0x03, 0x56, 0xa6, 0x0d, 0x5a, 0xcd, 0xdd, 0x62, 0xaa, 0x8c, 0x8e, 0x4f, 0xb4, - 0xa5, 0x04, 0xba, 0xe5, 0xb8, 0xe5, 0xcc, 0x37, 0xdf, 0x55, 0xe6, 0x7e, 0xf8, 0xbe, 0xa2, 0xb0, - 0xcc, 0x0a, 0x53, 0x77, 0x04, 0x7a, 0x0f, 0xae, 0x74, 0x9a, 0x77, 0x5b, 0x3b, 0xdb, 0xe6, 0x5e, - 0xe7, 0xae, 0xd9, 0xfd, 0xa2, 0xbd, 0x93, 0xc8, 0x6e, 0xf9, 0xf8, 0x44, 0xcb, 0xc9, 0x94, 0x2e, - 0x42, 0xb7, 0x8d, 0x9d, 0x87, 0xfb, 0xdd, 0x9d, 0xa2, 0x22, 0xd0, 0xed, 0x10, 0x1f, 0xf8, 0x14, - 0x73, 0xf4, 0x2d, 0xb8, 0x7a, 0x0e, 0x3a, 0x4e, 0x6c, 0xe5, 0xf8, 0x44, 0x2b, 0xb4, 0x43, 0x2c, - 0xce, 0x0f, 0xb7, 0xd0, 0xa1, 0x34, 0x6b, 0xb1, 0xdf, 0xde, 0xef, 0xd4, 0x76, 0x8b, 0x5a, 0xb9, - 0x78, 0x7c, 0xa2, 0xe5, 0xa3, 0xcb, 0x90, 0xe1, 0x27, 0x99, 0xd5, 0x3f, 0x7b, 0x76, 0x5a, 0x51, - 0x9e, 0x9f, 0x56, 0x94, 0x3f, 0x4f, 0x2b, 0xca, 0xd3, 0x97, 0x95, 0xb9, 0xe7, 0x2f, 0x2b, 0x73, - 0xbf, 0xbd, 0xac, 0xcc, 0x7d, 0xf9, 0xd1, 0xc0, 0xa1, 0xc3, 0x71, 0x4f, 0xef, 0xfb, 0xa3, 0xcd, - 0xe4, 0x3f, 0x8c, 0xc9, 0xa7, 0xf8, 0xa7, 0x73, 0xf6, 0xdf, 0x47, 0x6f, 0x91, 0xcb, 0x6f, 0xff, - 0x1d, 0x00, 0x00, 0xff, 0xff, 0xbb, 0xc0, 0x81, 0x37, 0x3e, 0x0d, 0x00, 0x00, + // 1396 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x57, 0x4b, 0x6f, 0x1b, 0xd5, + 0x17, 0xcf, 0xd8, 0xe3, 0xd7, 0xb1, 0x9d, 0x38, 0xf7, 0x9f, 0xb6, 0xae, 0xdb, 0x38, 0x96, 0xab, + 0x3f, 0xa4, 0x05, 0x39, 0x25, 0x45, 0x3c, 0x16, 0x2c, 0x6c, 0xc7, 0x6d, 0xad, 0x26, 0x8e, 0x19, + 0xbb, 0x45, 0x74, 0x33, 0x1a, 0x7b, 0x6e, 0xed, 0xa1, 0xf6, 0xcc, 0x68, 0xe6, 0x3a, 0x38, 0xfd, + 0x04, 0x28, 0xab, 0xae, 0xd8, 0x65, 0x05, 0x0b, 0xf6, 0x20, 0xb1, 0x45, 0xac, 0xba, 0xec, 0x0e, + 0x36, 0x14, 0x48, 0x25, 0x3e, 0x07, 0xba, 0x8f, 0x19, 0xcf, 0xc4, 0x31, 0x54, 0x51, 0x05, 0x12, + 0x9b, 0x68, 0xee, 0x39, 0xbf, 0x73, 0xee, 0x79, 0xfc, 0xee, 0xc9, 0x31, 0x5c, 0x25, 0xd8, 0xd4, + 0xb1, 0x33, 0x36, 0x4c, 0xb2, 0x45, 0x0e, 0x6d, 0xec, 0xf2, 0xbf, 0x15, 0xdb, 0xb1, 0x88, 0x85, + 0x72, 0x33, 0x6d, 0x85, 0xc9, 0x0b, 0x6b, 0x03, 0x6b, 0x60, 0x31, 0xe5, 0x16, 0xfd, 0xe2, 0xb8, + 0xc2, 0xc6, 0xc0, 0xb2, 0x06, 0x23, 0xbc, 0xc5, 0x4e, 0xbd, 0xc9, 0xa3, 0x2d, 0x62, 0x8c, 0xb1, + 0x4b, 0xb4, 0xb1, 0x2d, 0x00, 0xeb, 0x81, 0x6b, 0xfa, 0xce, 0xa1, 0x4d, 0x2c, 0x8a, 0xb5, 0x1e, + 0x09, 0x75, 0x31, 0xa0, 0x3e, 0xc0, 0x8e, 0x6b, 0x58, 0x66, 0x30, 0x8e, 0x42, 0x69, 0x2e, 0xca, + 0x03, 0x6d, 0x64, 0xe8, 0x1a, 0xb1, 0x1c, 0x8e, 0x28, 0x7f, 0x08, 0xd9, 0xb6, 0xe6, 0x90, 0x0e, + 0x26, 0x77, 0xb1, 0xa6, 0x63, 0x07, 0xad, 0x41, 0x8c, 0x58, 0x44, 0x1b, 0xe5, 0xa5, 0x92, 0xb4, + 0x99, 0x55, 0xf8, 0x01, 0x21, 0x90, 0x87, 0x9a, 0x3b, 0xcc, 0x47, 0x4a, 0xd2, 0x66, 0x46, 0x61, + 0xdf, 0xe5, 0x21, 0xc8, 0xd4, 0x94, 0x5a, 0x18, 0xa6, 0x8e, 0xa7, 0x9e, 0x05, 0x3b, 0x50, 0x69, + 0xef, 0x90, 0x60, 0x57, 0x98, 0xf0, 0x03, 0x7a, 0x17, 0x62, 0x2c, 0xfe, 0x7c, 0xb4, 0x24, 0x6d, + 0xa6, 0xb7, 0xf3, 0x95, 0x40, 0xa1, 0x78, 0x7e, 0x95, 0x36, 0xd5, 0xd7, 0xe4, 0x67, 0x2f, 0x36, + 0x96, 0x14, 0x0e, 0x2e, 0x8f, 0x20, 0x51, 0x1b, 0x59, 0xfd, 0xc7, 0xcd, 0x1d, 0x3f, 0x10, 0x69, + 0x16, 0x08, 0xda, 0x83, 0x15, 0x5b, 0x73, 0x88, 0xea, 0x62, 0xa2, 0x0e, 0x59, 0x16, 0xec, 0xd2, + 0xf4, 0xf6, 0x46, 0xe5, 0x74, 0x1f, 0x2a, 0xa1, 0x64, 0xc5, 0x2d, 0x59, 0x3b, 0x28, 0x2c, 0xff, + 0x21, 0x43, 0x5c, 0x14, 0xe3, 0x23, 0x48, 0x88, 0xb2, 0xb2, 0x0b, 0xd3, 0xdb, 0xeb, 0x41, 0x8f, + 0x42, 0x55, 0xa9, 0x5b, 0xa6, 0x8b, 0x4d, 0x77, 0xe2, 0x0a, 0x7f, 0x9e, 0x0d, 0x7a, 0x03, 0x92, + 0xfd, 0xa1, 0x66, 0x98, 0xaa, 0xa1, 0xb3, 0x88, 0x52, 0xb5, 0xf4, 0xc9, 0x8b, 0x8d, 0x44, 0x9d, + 0xca, 0x9a, 0x3b, 0x4a, 0x82, 0x29, 0x9b, 0x3a, 0xba, 0x08, 0xf1, 0x21, 0x36, 0x06, 0x43, 0xc2, + 0xca, 0x12, 0x55, 0xc4, 0x09, 0x7d, 0x00, 0x32, 0x25, 0x44, 0x5e, 0x66, 0x77, 0x17, 0x2a, 0x9c, + 0x2d, 0x15, 0x8f, 0x2d, 0x95, 0xae, 0xc7, 0x96, 0x5a, 0x92, 0x5e, 0xfc, 0xf4, 0xd7, 0x0d, 0x49, + 0x61, 0x16, 0xa8, 0x0e, 0xd9, 0x91, 0xe6, 0x12, 0xb5, 0x47, 0xcb, 0x46, 0xaf, 0x8f, 0x31, 0x17, + 0x97, 0xe7, 0x0b, 0x22, 0x0a, 0x2b, 0x42, 0x4f, 0x53, 0x2b, 0x2e, 0xd2, 0xd1, 0x26, 0xe4, 0x98, + 0x93, 0xbe, 0x35, 0x1e, 0x1b, 0x44, 0x65, 0x75, 0x8f, 0xb3, 0xba, 0x2f, 0x53, 0x79, 0x9d, 0x89, + 0xef, 0xd2, 0x0e, 0x5c, 0x81, 0x94, 0xae, 0x11, 0x8d, 0x43, 0x12, 0x0c, 0x92, 0xa4, 0x02, 0xa6, + 0x7c, 0x13, 0x56, 0x7c, 0xd6, 0xb9, 0x1c, 0x92, 0xe4, 0x5e, 0x66, 0x62, 0x06, 0xbc, 0x09, 0x6b, + 0x26, 0x9e, 0x12, 0xf5, 0x34, 0x3a, 0xc5, 0xd0, 0x88, 0xea, 0x1e, 0x84, 0x2d, 0xfe, 0x0f, 0xcb, + 0x7d, 0xaf, 0xf8, 0x1c, 0x0b, 0x0c, 0x9b, 0xf5, 0xa5, 0x0c, 0x76, 0x19, 0x92, 0x9a, 0x6d, 0x73, + 0x40, 0x9a, 0x01, 0x12, 0x9a, 0x6d, 0x33, 0xd5, 0x0d, 0x58, 0x65, 0x39, 0x3a, 0xd8, 0x9d, 0x8c, + 0x88, 0x70, 0x92, 0x61, 0x98, 0x15, 0xaa, 0x50, 0xb8, 0x9c, 0x61, 0xaf, 0x41, 0x16, 0x1f, 0x18, + 0x3a, 0x36, 0xfb, 0x98, 0xe3, 0xb2, 0x0c, 0x97, 0xf1, 0x84, 0x0c, 0x74, 0x1d, 0x72, 0xb6, 0x63, + 0xd9, 0x96, 0x8b, 0x1d, 0x55, 0xd3, 0x75, 0x07, 0xbb, 0x6e, 0x7e, 0x99, 0xfb, 0xf3, 0xe4, 0x55, + 0x2e, 0x2e, 0xe7, 0x41, 0xde, 0xd1, 0x88, 0x86, 0x72, 0x10, 0x25, 0x53, 0x37, 0x2f, 0x95, 0xa2, + 0x9b, 0x19, 0x85, 0x7e, 0x96, 0xbf, 0x8f, 0x82, 0xfc, 0xc0, 0x22, 0x18, 0xdd, 0x02, 0x99, 0xb6, + 0x89, 0xb1, 0x6f, 0xf9, 0x2c, 0x3e, 0x77, 0x8c, 0x81, 0x89, 0xf5, 0x3d, 0x77, 0xd0, 0x3d, 0xb4, + 0xb1, 0xc2, 0xc0, 0x01, 0x3a, 0x45, 0x42, 0x74, 0x5a, 0x83, 0x98, 0x63, 0x4d, 0x4c, 0x9d, 0xb1, + 0x2c, 0xa6, 0xf0, 0x03, 0x6a, 0x40, 0xd2, 0x67, 0x89, 0xfc, 0x77, 0x2c, 0x59, 0xa1, 0x2c, 0xa1, + 0x1c, 0x16, 0x02, 0x25, 0xd1, 0x13, 0x64, 0xa9, 0x41, 0xca, 0x1f, 0x5e, 0x82, 0x6d, 0xaf, 0x46, + 0xd8, 0x99, 0x19, 0x7a, 0x0b, 0x56, 0xfd, 0xde, 0xfb, 0xc5, 0xe3, 0x8c, 0xcb, 0xf9, 0x0a, 0x51, + 0xbd, 0x10, 0xad, 0x54, 0x3e, 0x80, 0x12, 0x2c, 0xaf, 0x19, 0xad, 0x9a, 0x6c, 0x12, 0x5d, 0x85, + 0x94, 0x6b, 0x0c, 0x4c, 0x8d, 0x4c, 0x1c, 0x2c, 0x98, 0x37, 0x13, 0x50, 0x2d, 0x9e, 0x12, 0x6c, + 0xb2, 0x47, 0xce, 0x99, 0x36, 0x13, 0xa0, 0x2d, 0xf8, 0x9f, 0x7f, 0x50, 0x67, 0x5e, 0x38, 0xcb, + 0x90, 0xaf, 0xea, 0x78, 0x9a, 0xf2, 0x0f, 0x12, 0xc4, 0xf9, 0xc3, 0x08, 0xb4, 0x41, 0x3a, 0xbb, + 0x0d, 0x91, 0x45, 0x6d, 0x88, 0x9e, 0xbf, 0x0d, 0x55, 0x00, 0x3f, 0x4c, 0x37, 0x2f, 0x97, 0xa2, + 0x9b, 0xe9, 0xed, 0x2b, 0xf3, 0x8e, 0x78, 0x88, 0x1d, 0x63, 0x20, 0xde, 0x7d, 0xc0, 0xa8, 0xfc, + 0x8b, 0x04, 0x29, 0x5f, 0x8f, 0xaa, 0x90, 0xf5, 0xe2, 0x52, 0x1f, 0x8d, 0xb4, 0x81, 0xa0, 0xe2, + 0xfa, 0xc2, 0xe0, 0x6e, 0x8f, 0xb4, 0x81, 0x92, 0x16, 0xf1, 0xd0, 0xc3, 0xd9, 0x6d, 0x8d, 0x2c, + 0x68, 0x6b, 0x88, 0x47, 0xd1, 0xf3, 0xf1, 0x28, 0xd4, 0x71, 0xf9, 0x54, 0xc7, 0xcb, 0xbf, 0x4b, + 0xb0, 0xdc, 0x98, 0xb2, 0xf0, 0xf5, 0x7f, 0xb3, 0x55, 0x0f, 0x05, 0xb7, 0x74, 0xac, 0xab, 0x73, + 0x3d, 0xbb, 0x36, 0xef, 0x31, 0x1c, 0xf3, 0xac, 0x77, 0xc8, 0xf3, 0xd2, 0x99, 0xf5, 0xf0, 0xbb, + 0x08, 0xac, 0xce, 0xe1, 0xff, 0x7b, 0xbd, 0x0c, 0xbf, 0xde, 0xd8, 0x2b, 0xbe, 0xde, 0xf8, 0xc2, + 0xd7, 0xfb, 0x6d, 0x04, 0x92, 0x6d, 0x36, 0xa5, 0xb5, 0xd1, 0x3f, 0x31, 0x7b, 0xaf, 0x40, 0xca, + 0xb6, 0x46, 0x2a, 0xd7, 0xc8, 0x4c, 0x93, 0xb4, 0xad, 0x91, 0x32, 0x47, 0xb3, 0xd8, 0x6b, 0x1a, + 0xcc, 0xf1, 0xd7, 0xd0, 0x84, 0xc4, 0xe9, 0x07, 0xe5, 0x40, 0x86, 0x97, 0x42, 0x6c, 0x4d, 0x37, + 0x69, 0x0d, 0xd8, 0x1a, 0x26, 0xcd, 0x6f, 0x79, 0x3c, 0x6c, 0x8e, 0x54, 0x04, 0x8e, 0x5a, 0xf0, + 0x25, 0x43, 0x2c, 0x6e, 0xf9, 0x45, 0x13, 0x4b, 0x11, 0xb8, 0xf2, 0x97, 0x12, 0xc0, 0x2e, 0xad, + 0x2c, 0xcb, 0x97, 0xee, 0x3b, 0x2e, 0x0b, 0x41, 0x0d, 0xdd, 0x5c, 0x5c, 0xd4, 0x34, 0x71, 0x7f, + 0xc6, 0x0d, 0xc6, 0x5d, 0x87, 0xec, 0x8c, 0xdb, 0x2e, 0xf6, 0x82, 0x39, 0xc3, 0x89, 0xbf, 0x86, + 0x74, 0x30, 0x51, 0x32, 0x07, 0x81, 0x53, 0xf9, 0x47, 0x09, 0x52, 0x2c, 0xa6, 0x3d, 0x4c, 0xb4, + 0x50, 0x0f, 0xa5, 0xf3, 0xf7, 0x70, 0x1d, 0x80, 0xbb, 0x71, 0x8d, 0x27, 0x58, 0x30, 0x2b, 0xc5, + 0x24, 0x1d, 0xe3, 0x09, 0x46, 0xef, 0xf9, 0x05, 0x8f, 0xfe, 0x75, 0xc1, 0xc5, 0xc4, 0xf0, 0xca, + 0x7e, 0x09, 0x12, 0xe6, 0x64, 0xac, 0xd2, 0xe5, 0x43, 0xe6, 0x6c, 0x35, 0x27, 0xe3, 0xee, 0xd4, + 0x2d, 0x7f, 0x06, 0x89, 0xee, 0x94, 0x2d, 0xe2, 0x94, 0xa2, 0x8e, 0x65, 0x89, 0xed, 0x8f, 0x6f, + 0xdd, 0x49, 0x2a, 0x60, 0xcb, 0x0e, 0x02, 0x99, 0xae, 0x79, 0xde, 0xcf, 0x02, 0xfa, 0x8d, 0x2a, + 0xaf, 0xb8, 0xe2, 0x8b, 0xe5, 0xfe, 0xc6, 0x4f, 0x12, 0xa4, 0x03, 0xe3, 0x06, 0xbd, 0x03, 0x17, + 0x6a, 0xbb, 0xfb, 0xf5, 0x7b, 0x6a, 0x73, 0x47, 0xbd, 0xbd, 0x5b, 0xbd, 0xa3, 0xde, 0x6f, 0xdd, + 0x6b, 0xed, 0x7f, 0xd2, 0xca, 0x2d, 0x15, 0x2e, 0x1e, 0x1d, 0x97, 0x50, 0x00, 0x7b, 0xdf, 0x7c, + 0x6c, 0x5a, 0x9f, 0xd3, 0x77, 0xbe, 0x16, 0x36, 0xa9, 0xd6, 0x3a, 0x8d, 0x56, 0x37, 0x27, 0x15, + 0x2e, 0x1c, 0x1d, 0x97, 0x56, 0x03, 0x16, 0xd5, 0x9e, 0x8b, 0x4d, 0x32, 0x6f, 0x50, 0xdf, 0xdf, + 0xdb, 0x6b, 0x76, 0x73, 0x91, 0x39, 0x03, 0xf1, 0x0f, 0xe2, 0x3a, 0xac, 0x86, 0x0d, 0x5a, 0xcd, + 0xdd, 0x5c, 0xb4, 0x80, 0x8e, 0x8e, 0x4b, 0xcb, 0x01, 0x74, 0xcb, 0x18, 0x15, 0x92, 0x5f, 0x7c, + 0x55, 0x5c, 0xfa, 0xe6, 0xeb, 0xa2, 0x44, 0x33, 0xcb, 0x86, 0x66, 0x04, 0x7a, 0x1b, 0x2e, 0x75, + 0x9a, 0x77, 0x5a, 0x8d, 0x1d, 0x75, 0xaf, 0x73, 0x47, 0xed, 0x7e, 0xda, 0x6e, 0x04, 0xb2, 0x5b, + 0x39, 0x3a, 0x2e, 0xa5, 0x45, 0x4a, 0x8b, 0xd0, 0x6d, 0xa5, 0xf1, 0x60, 0xbf, 0xdb, 0xc8, 0x49, + 0x1c, 0xdd, 0x76, 0xf0, 0x81, 0x45, 0x30, 0x43, 0xdf, 0x84, 0xcb, 0x67, 0xa0, 0xfd, 0xc4, 0x56, + 0x8f, 0x8e, 0x4b, 0xd9, 0xb6, 0x83, 0xf9, 0xfb, 0x61, 0x16, 0x15, 0xc8, 0xcf, 0x5b, 0xec, 0xb7, + 0xf7, 0x3b, 0xd5, 0xdd, 0x5c, 0xa9, 0x90, 0x3b, 0x3a, 0x2e, 0x65, 0xbc, 0x61, 0x48, 0xf1, 0xb3, + 0xcc, 0x6a, 0x1f, 0x3f, 0x3b, 0x29, 0x4a, 0xcf, 0x4f, 0x8a, 0xd2, 0x6f, 0x27, 0x45, 0xe9, 0xe9, + 0xcb, 0xe2, 0xd2, 0xf3, 0x97, 0xc5, 0xa5, 0x9f, 0x5f, 0x16, 0x97, 0x1e, 0xbe, 0x3f, 0x30, 0xc8, + 0x70, 0xd2, 0xab, 0xf4, 0xad, 0xf1, 0x56, 0xf0, 0xc7, 0xe7, 0xec, 0x93, 0xff, 0x08, 0x3e, 0xfd, + 0xc3, 0xb4, 0x17, 0x67, 0xf2, 0x5b, 0x7f, 0x06, 0x00, 0x00, 0xff, 0xff, 0x46, 0xcf, 0x37, 0x28, + 0x59, 0x0f, 0x00, 0x00, } func (m *PartSetHeader) Marshal() (dAtA []byte, err error) { @@ -1634,6 +1796,127 @@ func (m *CommitSig) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *ExtendedCommit) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExtendedCommit) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ExtendedCommit) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.ExtendedSignatures) > 0 { + for iNdEx := len(m.ExtendedSignatures) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.ExtendedSignatures[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + } + { + size, err := m.BlockID.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + if m.Round != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Round)) + i-- + dAtA[i] = 0x10 + } + if m.Height != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Height)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ExtendedCommitSig) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ExtendedCommitSig) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ExtendedCommitSig) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.ExtensionSignature) > 0 { + i -= len(m.ExtensionSignature) + copy(dAtA[i:], m.ExtensionSignature) + i = encodeVarintTypes(dAtA, i, uint64(len(m.ExtensionSignature))) + i-- + dAtA[i] = 0x32 + } + if len(m.Extension) > 0 { + i -= len(m.Extension) + copy(dAtA[i:], m.Extension) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Extension))) + i-- + dAtA[i] = 0x2a + } + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x22 + } + n11, err11 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Timestamp, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp):]) + if err11 != nil { + return 0, err11 + } + i -= n11 + i = encodeVarintTypes(dAtA, i, uint64(n11)) + i-- + dAtA[i] = 0x1a + if len(m.ValidatorAddress) > 0 { + i -= len(m.ValidatorAddress) + copy(dAtA[i:], m.ValidatorAddress) + i = encodeVarintTypes(dAtA, i, uint64(len(m.ValidatorAddress))) + i-- + dAtA[i] = 0x12 + } + if m.BlockIdFlag != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.BlockIdFlag)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func (m *Proposal) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1661,12 +1944,12 @@ func (m *Proposal) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x3a } - n10, err10 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Timestamp, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp):]) - if err10 != nil { - return 0, err10 + n12, err12 := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Timestamp, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp):]) + if err12 != nil { + return 0, err12 } - i -= n10 - i = encodeVarintTypes(dAtA, i, uint64(n10)) + i -= n12 + i = encodeVarintTypes(dAtA, i, uint64(n12)) i-- dAtA[i] = 0x32 { @@ -2117,6 +2400,59 @@ func (m *CommitSig) Size() (n int) { return n } +func (m *ExtendedCommit) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Height != 0 { + n += 1 + sovTypes(uint64(m.Height)) + } + if m.Round != 0 { + n += 1 + sovTypes(uint64(m.Round)) + } + l = m.BlockID.Size() + n += 1 + l + sovTypes(uint64(l)) + if len(m.ExtendedSignatures) > 0 { + for _, e := range m.ExtendedSignatures { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + return n +} + +func (m *ExtendedCommitSig) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BlockIdFlag != 0 { + n += 1 + sovTypes(uint64(m.BlockIdFlag)) + } + l = len(m.ValidatorAddress) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp) + n += 1 + l + sovTypes(uint64(l)) + l = len(m.Signature) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + l = len(m.Extension) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + l = len(m.ExtensionSignature) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + return n +} + func (m *Proposal) Size() (n int) { if m == nil { return 0 @@ -3823,6 +4159,399 @@ func (m *CommitSig) Unmarshal(dAtA []byte) error { } return nil } +func (m *ExtendedCommit) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExtendedCommit: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExtendedCommit: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Height", wireType) + } + m.Height = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Height |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Round", wireType) + } + m.Round = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Round |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockID", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.BlockID.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtendedSignatures", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtendedSignatures = append(m.ExtendedSignatures, ExtendedCommitSig{}) + if err := m.ExtendedSignatures[len(m.ExtendedSignatures)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ExtendedCommitSig) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ExtendedCommitSig: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ExtendedCommitSig: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockIdFlag", wireType) + } + m.BlockIdFlag = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.BlockIdFlag |= BlockIDFlag(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ValidatorAddress", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ValidatorAddress = append(m.ValidatorAddress[:0], dAtA[iNdEx:postIndex]...) + if m.ValidatorAddress == nil { + m.ValidatorAddress = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Timestamp, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = append(m.Signature[:0], dAtA[iNdEx:postIndex]...) + if m.Signature == nil { + m.Signature = []byte{} + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Extension", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Extension = append(m.Extension[:0], dAtA[iNdEx:postIndex]...) + if m.Extension == nil { + m.Extension = []byte{} + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ExtensionSignature", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ExtensionSignature = append(m.ExtensionSignature[:0], dAtA[iNdEx:postIndex]...) + if m.ExtensionSignature == nil { + m.ExtensionSignature = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *Proposal) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/proto/tendermint/types/types.proto b/proto/tendermint/types/types.proto index e2b8a46c8..52668f719 100644 --- a/proto/tendermint/types/types.proto +++ b/proto/tendermint/types/types.proto @@ -142,6 +142,28 @@ message CommitSig { bytes signature = 4; } +message ExtendedCommit { + int64 height = 1; + int32 round = 2; + BlockID block_id = 3 + [(gogoproto.nullable) = false, (gogoproto.customname) = "BlockID"]; + repeated ExtendedCommitSig extended_signatures = 4 [(gogoproto.nullable) = false]; +} + +// ExtendedCommitSig retains all the same fields as CommitSig but adds vote +// extension-related fields. +message ExtendedCommitSig { + BlockIDFlag block_id_flag = 1; + bytes validator_address = 2; + google.protobuf.Timestamp timestamp = 3 + [(gogoproto.nullable) = false, (gogoproto.stdtime) = true]; + bytes signature = 4; + // Vote extension data + bytes extension = 5; + // Vote extension signature + bytes extension_signature = 6; +} + message Proposal { SignedMsgType type = 1; int64 height = 2; diff --git a/scripts/confix/confix.go b/scripts/confix/confix.go index 6677f0b49..b24c3a778 100644 --- a/scripts/confix/confix.go +++ b/scripts/confix/confix.go @@ -17,6 +17,7 @@ import ( "github.com/creachadair/tomledit" "github.com/creachadair/tomledit/transform" "github.com/spf13/viper" + "github.com/tendermint/tendermint/config" ) diff --git a/test/e2e/runner/evidence.go b/test/e2e/runner/evidence.go index 849e4edc3..a71ea14fb 100644 --- a/test/e2e/runner/evidence.go +++ b/test/e2e/runner/evidence.go @@ -167,7 +167,7 @@ func generateLightClientAttackEvidence( blockID := makeBlockID(header.Hash(), 1000, []byte("partshash")) voteSet := types.NewVoteSet(chainID, forgedHeight, 0, tmproto.SignedMsgType(2), conflictingVals) - commit, err := factory.MakeCommit(ctx, blockID, forgedHeight, 0, voteSet, pv, forgedTime) + commit, err := factory.MakeExtendedCommit(ctx, blockID, forgedHeight, 0, voteSet, pv, forgedTime) if err != nil { return nil, err } @@ -176,7 +176,7 @@ func generateLightClientAttackEvidence( ConflictingBlock: &types.LightBlock{ SignedHeader: &types.SignedHeader{ Header: header, - Commit: commit, + Commit: commit.StripExtensions(), }, ValidatorSet: conflictingVals, }, diff --git a/types/block.go b/types/block.go index 17e9812cf..32a4f9a0a 100644 --- a/types/block.go +++ b/types/block.go @@ -608,16 +608,6 @@ type CommitSig struct { Signature []byte `json:"signature"` } -// NewCommitSigForBlock returns new CommitSig with BlockIDFlagCommit. -func NewCommitSigForBlock(signature []byte, valAddr Address, ts time.Time) CommitSig { - return CommitSig{ - BlockIDFlag: BlockIDFlagCommit, - ValidatorAddress: valAddr, - Timestamp: ts, - Signature: signature, - } -} - func MaxCommitBytes(valCount int) int64 { // From the repeated commit sig field var protoEncodingOverhead int64 = 2 @@ -632,16 +622,6 @@ func NewCommitSigAbsent() CommitSig { } } -// ForBlock returns true if CommitSig is for the block. -func (cs CommitSig) ForBlock() bool { - return cs.BlockIDFlag == BlockIDFlagCommit -} - -// Absent returns true if CommitSig is absent. -func (cs CommitSig) Absent() bool { - return cs.BlockIDFlag == BlockIDFlagAbsent -} - // CommitSig returns a string representation of CommitSig. // // 1. first 6 bytes of signature @@ -730,7 +710,6 @@ func (cs *CommitSig) ToProto() *tmproto.CommitSig { // FromProto sets a protobuf CommitSig to the given pointer. // It returns an error if the CommitSig is invalid. func (cs *CommitSig) FromProto(csp tmproto.CommitSig) error { - cs.BlockIDFlag = BlockIDFlag(csp.BlockIdFlag) cs.ValidatorAddress = csp.ValidatorAddress cs.Timestamp = csp.Timestamp @@ -741,6 +720,95 @@ func (cs *CommitSig) FromProto(csp tmproto.CommitSig) error { //------------------------------------- +// ExtendedCommitSig contains a commit signature along with its corresponding +// vote extension and vote extension signature. +type ExtendedCommitSig struct { + CommitSig // Commit signature + Extension []byte // Vote extension + ExtensionSignature []byte // Vote extension signature +} + +// NewExtendedCommitSigAbsent returns new ExtendedCommitSig with +// BlockIDFlagAbsent. Other fields are all empty. +func NewExtendedCommitSigAbsent() ExtendedCommitSig { + return ExtendedCommitSig{CommitSig: NewCommitSigAbsent()} +} + +// String returns a string representation of an ExtendedCommitSig. +// +// 1. commit sig +// 2. first 6 bytes of vote extension +// 3. first 6 bytes of vote extension signature +func (ecs ExtendedCommitSig) String() string { + return fmt.Sprintf("ExtendedCommitSig{%s with %X %X}", + ecs.CommitSig, + tmbytes.Fingerprint(ecs.Extension), + tmbytes.Fingerprint(ecs.ExtensionSignature), + ) +} + +// ValidateBasic checks whether the structure is well-formed. +func (ecs ExtendedCommitSig) ValidateBasic() error { + if err := ecs.CommitSig.ValidateBasic(); err != nil { + return err + } + + if ecs.BlockIDFlag == BlockIDFlagCommit { + if len(ecs.Extension) > MaxVoteExtensionSize { + return fmt.Errorf("vote extension is too big (max: %d)", MaxVoteExtensionSize) + } + if len(ecs.ExtensionSignature) == 0 { + return errors.New("vote extension signature is missing") + } + if len(ecs.ExtensionSignature) > MaxSignatureSize { + return fmt.Errorf("vote extension signature is too big (max: %d)", MaxSignatureSize) + } + return nil + } + + // We expect there to not be any vote extension or vote extension signature + // on nil or absent votes. + if len(ecs.Extension) != 0 { + return fmt.Errorf("vote extension is present for commit sig with block ID flag %v", ecs.BlockIDFlag) + } + if len(ecs.ExtensionSignature) != 0 { + return fmt.Errorf("vote extension signature is present for commit sig with block ID flag %v", ecs.BlockIDFlag) + } + return nil +} + +// ToProto converts the ExtendedCommitSig to its Protobuf representation. +func (ecs *ExtendedCommitSig) ToProto() *tmproto.ExtendedCommitSig { + if ecs == nil { + return nil + } + + return &tmproto.ExtendedCommitSig{ + BlockIdFlag: tmproto.BlockIDFlag(ecs.BlockIDFlag), + ValidatorAddress: ecs.ValidatorAddress, + Timestamp: ecs.Timestamp, + Signature: ecs.Signature, + Extension: ecs.Extension, + ExtensionSignature: ecs.ExtensionSignature, + } +} + +// FromProto populates the ExtendedCommitSig with values from the given +// Protobuf representation. Returns an error if the ExtendedCommitSig is +// invalid. +func (ecs *ExtendedCommitSig) FromProto(ecsp tmproto.ExtendedCommitSig) error { + ecs.BlockIDFlag = BlockIDFlag(ecsp.BlockIdFlag) + ecs.ValidatorAddress = ecsp.ValidatorAddress + ecs.Timestamp = ecsp.Timestamp + ecs.Signature = ecsp.Signature + ecs.Extension = ecsp.Extension + ecs.ExtensionSignature = ecsp.ExtensionSignature + + return ecs.ValidateBasic() +} + +//------------------------------------- + // Commit contains the evidence that a block was committed by a set of validators. // NOTE: Commit is empty for height 1, but never nil. type Commit struct { @@ -756,42 +824,12 @@ type Commit struct { // Memoized in first call to corresponding method. // NOTE: can't memoize in constructor because constructor isn't used for // unmarshaling. - hash tmbytes.HexBytes - bitArray *bits.BitArray + hash tmbytes.HexBytes } -// NewCommit returns a new Commit. -func NewCommit(height int64, round int32, blockID BlockID, commitSigs []CommitSig) *Commit { - return &Commit{ - Height: height, - Round: round, - BlockID: blockID, - Signatures: commitSigs, - } -} - -// CommitToVoteSet constructs a VoteSet from the Commit and validator set. -// Panics if signatures from the commit can't be added to the voteset. -// Inverse of VoteSet.MakeCommit(). -func CommitToVoteSet(chainID string, commit *Commit, vals *ValidatorSet) *VoteSet { - voteSet := NewVoteSet(chainID, commit.Height, commit.Round, tmproto.PrecommitType, vals) - for idx, commitSig := range commit.Signatures { - if commitSig.Absent() { - continue // OK, some precommits can be missing. - } - vote := commit.GetVote(int32(idx)) - if err := vote.ValidateBasic(); err != nil { - panic(fmt.Errorf("failed to validate vote reconstructed from LastCommit: %w", err)) - } - added, err := voteSet.AddVote(vote) - if !added || err != nil { - panic(fmt.Errorf("failed to reconstruct LastCommit: %w", err)) - } - } - return voteSet -} - -// GetVote converts the CommitSig for the given valIdx to a Vote. +// GetVote converts the CommitSig for the given valIdx to a Vote. Commits do +// not contain vote extensions, so the vote extension and vote extension +// signature will not be present in the returned vote. // Returns nil if the precommit at valIdx is nil. // Panics if valIdx >= commit.Size(). func (commit *Commit) GetVote(valIdx int32) *Vote { @@ -822,26 +860,7 @@ func (commit *Commit) VoteSignBytes(chainID string, valIdx int32) []byte { return VoteSignBytes(chainID, v) } -// Type returns the vote type of the commit, which is always VoteTypePrecommit -// Implements VoteSetReader. -func (commit *Commit) Type() byte { - return byte(tmproto.PrecommitType) -} - -// GetHeight returns height of the commit. -// Implements VoteSetReader. -func (commit *Commit) GetHeight() int64 { - return commit.Height -} - -// GetRound returns height of the commit. -// Implements VoteSetReader. -func (commit *Commit) GetRound() int32 { - return commit.Round -} - // Size returns the number of signatures in the commit. -// Implements VoteSetReader. func (commit *Commit) Size() int { if commit == nil { return 0 @@ -849,33 +868,6 @@ func (commit *Commit) Size() int { return len(commit.Signatures) } -// BitArray returns a BitArray of which validators voted for BlockID or nil in this commit. -// Implements VoteSetReader. -func (commit *Commit) BitArray() *bits.BitArray { - if commit.bitArray == nil { - commit.bitArray = bits.NewBitArray(len(commit.Signatures)) - for i, commitSig := range commit.Signatures { - // TODO: need to check the BlockID otherwise we could be counting conflicts, - // not just the one with +2/3 ! - commit.bitArray.SetIndex(i, !commitSig.Absent()) - } - } - return commit.bitArray -} - -// GetByIndex returns the vote corresponding to a given validator index. -// Panics if `index >= commit.Size()`. -// Implements VoteSetReader. -func (commit *Commit) GetByIndex(valIdx int32) *Vote { - return commit.GetVote(valIdx) -} - -// IsCommit returns true if there is at least one signature. -// Implements VoteSetReader. -func (commit *Commit) IsCommit() bool { - return len(commit.Signatures) != 0 -} - // ValidateBasic performs basic validation that doesn't involve state data. // Does not actually check the cryptographic signatures. func (commit *Commit) ValidateBasic() error { @@ -999,7 +991,209 @@ func CommitFromProto(cp *tmproto.Commit) (*Commit, error) { return commit, commit.ValidateBasic() } -//----------------------------------------------------------------------------- +//------------------------------------- + +// ExtendedCommit is similar to Commit, except that its signatures also retain +// their corresponding vote extensions and vote extension signatures. +type ExtendedCommit struct { + Height int64 + Round int32 + BlockID BlockID + ExtendedSignatures []ExtendedCommitSig + + bitArray *bits.BitArray +} + +// Clone creates a deep copy of this extended commit. +func (ec *ExtendedCommit) Clone() *ExtendedCommit { + sigs := make([]ExtendedCommitSig, len(ec.ExtendedSignatures)) + copy(sigs, ec.ExtendedSignatures) + ecc := *ec + ecc.ExtendedSignatures = sigs + return &ecc +} + +// ToVoteSet constructs a VoteSet from the Commit and validator set. +// Panics if signatures from the commit can't be added to the voteset. +// Inverse of VoteSet.MakeExtendedCommit(). +func (ec *ExtendedCommit) ToVoteSet(chainID string, vals *ValidatorSet) *VoteSet { + voteSet := NewVoteSet(chainID, ec.Height, ec.Round, tmproto.PrecommitType, vals) + for idx, ecs := range ec.ExtendedSignatures { + if ecs.BlockIDFlag == BlockIDFlagAbsent { + continue // OK, some precommits can be missing. + } + vote := ec.GetExtendedVote(int32(idx)) + if err := vote.ValidateWithExtension(); err != nil { + panic(fmt.Errorf("failed to validate vote reconstructed from LastCommit: %w", err)) + } + added, err := voteSet.AddVote(vote) + if !added || err != nil { + panic(fmt.Errorf("failed to reconstruct vote set from extended commit: %w", err)) + } + } + return voteSet +} + +// StripExtensions converts an ExtendedCommit to a Commit by removing all vote +// extension-related fields. +func (ec *ExtendedCommit) StripExtensions() *Commit { + cs := make([]CommitSig, len(ec.ExtendedSignatures)) + for idx, ecs := range ec.ExtendedSignatures { + cs[idx] = ecs.CommitSig + } + return &Commit{ + Height: ec.Height, + Round: ec.Round, + BlockID: ec.BlockID, + Signatures: cs, + } +} + +// GetExtendedVote converts the ExtendedCommitSig for the given validator +// index to a Vote with a vote extensions. +// It panics if valIndex is out of range. +func (ec *ExtendedCommit) GetExtendedVote(valIndex int32) *Vote { + ecs := ec.ExtendedSignatures[valIndex] + return &Vote{ + Type: tmproto.PrecommitType, + Height: ec.Height, + Round: ec.Round, + BlockID: ecs.BlockID(ec.BlockID), + Timestamp: ecs.Timestamp, + ValidatorAddress: ecs.ValidatorAddress, + ValidatorIndex: valIndex, + Signature: ecs.Signature, + Extension: ecs.Extension, + ExtensionSignature: ecs.ExtensionSignature, + } +} + +// Type returns the vote type of the extended commit, which is always +// VoteTypePrecommit +// Implements VoteSetReader. +func (ec *ExtendedCommit) Type() byte { return byte(tmproto.PrecommitType) } + +// GetHeight returns height of the extended commit. +// Implements VoteSetReader. +func (ec *ExtendedCommit) GetHeight() int64 { return ec.Height } + +// GetRound returns height of the extended commit. +// Implements VoteSetReader. +func (ec *ExtendedCommit) GetRound() int32 { return ec.Round } + +// Size returns the number of signatures in the extended commit. +// Implements VoteSetReader. +func (ec *ExtendedCommit) Size() int { + if ec == nil { + return 0 + } + return len(ec.ExtendedSignatures) +} + +// BitArray returns a BitArray of which validators voted for BlockID or nil in +// this extended commit. +// Implements VoteSetReader. +func (ec *ExtendedCommit) BitArray() *bits.BitArray { + if ec.bitArray == nil { + ec.bitArray = bits.NewBitArray(len(ec.ExtendedSignatures)) + for i, extCommitSig := range ec.ExtendedSignatures { + // TODO: need to check the BlockID otherwise we could be counting conflicts, + // not just the one with +2/3 ! + ec.bitArray.SetIndex(i, extCommitSig.BlockIDFlag != BlockIDFlagAbsent) + } + } + return ec.bitArray +} + +// GetByIndex returns the vote corresponding to a given validator index. +// Panics if `index >= extCommit.Size()`. +// Implements VoteSetReader. +func (ec *ExtendedCommit) GetByIndex(valIdx int32) *Vote { + return ec.GetExtendedVote(valIdx) +} + +// IsCommit returns true if there is at least one signature. +// Implements VoteSetReader. +func (ec *ExtendedCommit) IsCommit() bool { + return len(ec.ExtendedSignatures) != 0 +} + +// ValidateBasic checks whether the extended commit is well-formed. Does not +// actually check the cryptographic signatures. +func (ec *ExtendedCommit) ValidateBasic() error { + if ec.Height < 0 { + return errors.New("negative Height") + } + if ec.Round < 0 { + return errors.New("negative Round") + } + + if ec.Height >= 1 { + if ec.BlockID.IsNil() { + return errors.New("commit cannot be for nil block") + } + + if len(ec.ExtendedSignatures) == 0 { + return errors.New("no signatures in commit") + } + for i, extCommitSig := range ec.ExtendedSignatures { + if err := extCommitSig.ValidateBasic(); err != nil { + return fmt.Errorf("wrong ExtendedCommitSig #%d: %v", i, err) + } + } + } + return nil +} + +// ToProto converts ExtendedCommit to protobuf +func (ec *ExtendedCommit) ToProto() *tmproto.ExtendedCommit { + if ec == nil { + return nil + } + + c := new(tmproto.ExtendedCommit) + sigs := make([]tmproto.ExtendedCommitSig, len(ec.ExtendedSignatures)) + for i := range ec.ExtendedSignatures { + sigs[i] = *ec.ExtendedSignatures[i].ToProto() + } + c.ExtendedSignatures = sigs + + c.Height = ec.Height + c.Round = ec.Round + c.BlockID = ec.BlockID.ToProto() + + return c +} + +// ExtendedCommitFromProto constructs an ExtendedCommit from the given Protobuf +// representation. It returns an error if the extended commit is invalid. +func ExtendedCommitFromProto(ecp *tmproto.ExtendedCommit) (*ExtendedCommit, error) { + if ecp == nil { + return nil, errors.New("nil ExtendedCommit") + } + + extCommit := new(ExtendedCommit) + + bi, err := BlockIDFromProto(&ecp.BlockID) + if err != nil { + return nil, err + } + + sigs := make([]ExtendedCommitSig, len(ecp.ExtendedSignatures)) + for i := range ecp.ExtendedSignatures { + if err := sigs[i].FromProto(ecp.ExtendedSignatures[i]); err != nil { + return nil, err + } + } + extCommit.ExtendedSignatures = sigs + extCommit.Height = ecp.Height + extCommit.Round = ecp.Round + extCommit.BlockID = *bi + + return extCommit, extCommit.ValidateBasic() +} + +//------------------------------------- // Data contains the set of transactions included in the block type Data struct { @@ -1170,3 +1364,9 @@ func BlockIDFromProto(bID *tmproto.BlockID) (*BlockID, error) { return blockID, blockID.ValidateBasic() } + +// ProtoBlockIDIsNil is similar to the IsNil function on BlockID, but for the +// Protobuf representation. +func ProtoBlockIDIsNil(bID *tmproto.BlockID) bool { + return len(bID.Hash) == 0 && ProtoPartSetHeaderIsZero(&bID.PartSetHeader) +} diff --git a/types/block_test.go b/types/block_test.go index 7f2378505..09a8b602e 100644 --- a/types/block_test.go +++ b/types/block_test.go @@ -42,14 +42,14 @@ func TestBlockAddEvidence(t *testing.T) { h := int64(3) voteSet, _, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1) - commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) + extCommit, err := makeExtCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) require.NoError(t, err) ev, err := NewMockDuplicateVoteEvidenceWithValidator(ctx, h, time.Now(), vals[0], "block-test-chain") require.NoError(t, err) evList := []Evidence{ev} - block := MakeBlock(h, txs, commit, evList) + block := MakeBlock(h, txs, extCommit.StripExtensions(), evList) require.NotNil(t, block) require.Equal(t, 1, len(block.Evidence)) require.NotNil(t, block.EvidenceHash) @@ -66,9 +66,9 @@ func TestBlockValidateBasic(t *testing.T) { h := int64(3) voteSet, valSet, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1) - commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) - + extCommit, err := makeExtCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) require.NoError(t, err) + commit := extCommit.StripExtensions() ev, err := NewMockDuplicateVoteEvidenceWithValidator(ctx, h, time.Now(), vals[0], "block-test-chain") require.NoError(t, err) @@ -104,7 +104,10 @@ func TestBlockValidateBasic(t *testing.T) { blk.LastCommit = nil }, true}, {"Invalid LastCommit", func(blk *Block) { - blk.LastCommit = NewCommit(-1, 0, *voteSet.maj23, nil) + blk.LastCommit = &Commit{ + Height: -1, + BlockID: *voteSet.maj23, + } }, true}, {"Invalid Evidence", func(blk *Block) { emptyEv := &DuplicateVoteEvidence{} @@ -153,15 +156,14 @@ func TestBlockMakePartSetWithEvidence(t *testing.T) { h := int64(3) voteSet, _, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1) - commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) - + extCommit, err := makeExtCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) require.NoError(t, err) ev, err := NewMockDuplicateVoteEvidenceWithValidator(ctx, h, time.Now(), vals[0], "block-test-chain") require.NoError(t, err) evList := []Evidence{ev} - partSet, err := MakeBlock(h, []Tx{Tx("Hello World")}, commit, evList).MakePartSet(512) + partSet, err := MakeBlock(h, []Tx{Tx("Hello World")}, extCommit.StripExtensions(), evList).MakePartSet(512) require.NoError(t, err) assert.NotNil(t, partSet) @@ -178,14 +180,14 @@ func TestBlockHashesTo(t *testing.T) { h := int64(3) voteSet, valSet, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1) - commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) + extCommit, err := makeExtCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) require.NoError(t, err) ev, err := NewMockDuplicateVoteEvidenceWithValidator(ctx, h, time.Now(), vals[0], "block-test-chain") require.NoError(t, err) evList := []Evidence{ev} - block := MakeBlock(h, []Tx{Tx("Hello World")}, commit, evList) + block := MakeBlock(h, []Tx{Tx("Hello World")}, extCommit.StripExtensions(), evList) block.ValidatorsHash = valSet.Hash() assert.False(t, block.HashesTo([]byte{})) assert.False(t, block.HashesTo([]byte("something else"))) @@ -260,7 +262,7 @@ func TestCommit(t *testing.T) { lastID := makeBlockIDRandom() h := int64(3) voteSet, _, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1) - commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) + commit, err := makeExtCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) require.NoError(t, err) assert.Equal(t, h-1, commit.Height) @@ -273,7 +275,7 @@ func TestCommit(t *testing.T) { require.NotNil(t, commit.BitArray()) assert.Equal(t, bits.NewBitArray(10).Size(), commit.BitArray().Size()) - assert.Equal(t, voteWithoutExtension(voteSet.GetByIndex(0)), commit.GetByIndex(0)) + assert.Equal(t, voteSet.GetByIndex(0), commit.GetByIndex(0)) assert.True(t, commit.IsCommit()) } @@ -477,11 +479,11 @@ func randCommit(ctx context.Context, t *testing.T, now time.Time) *Commit { lastID := makeBlockIDRandom() h := int64(3) voteSet, _, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1) - commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, now) + commit, err := makeExtCommit(ctx, lastID, h-1, 1, voteSet, vals, now) require.NoError(t, err) - return commit + return commit.StripExtensions() } func hexBytesFromString(t *testing.T, s string) bytes.HexBytes { @@ -554,7 +556,7 @@ func TestBlockMaxDataBytesNoEvidence(t *testing.T) { } } -func TestCommitToVoteSet(t *testing.T) { +func TestExtendedCommitToVoteSet(t *testing.T) { lastID := makeBlockIDRandom() h := int64(3) @@ -562,17 +564,16 @@ func TestCommitToVoteSet(t *testing.T) { defer cancel() voteSet, valSet, vals := randVoteSet(ctx, t, h-1, 1, tmproto.PrecommitType, 10, 1) - commit, err := makeCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) - + extCommit, err := makeExtCommit(ctx, lastID, h-1, 1, voteSet, vals, time.Now()) assert.NoError(t, err) chainID := voteSet.ChainID() - voteSet2 := CommitToVoteSet(chainID, commit, valSet) + voteSet2 := extCommit.ToVoteSet(chainID, valSet) for i := int32(0); int(i) < len(vals); i++ { - vote1 := voteWithoutExtension(voteSet.GetByIndex(i)) + vote1 := voteSet.GetByIndex(i) vote2 := voteSet2.GetByIndex(i) - vote3 := commit.GetVote(i) + vote3 := extCommit.GetExtendedVote(i) vote1bz, err := vote1.ToProto().Marshal() require.NoError(t, err) @@ -634,12 +635,12 @@ func TestCommitToVoteSetWithVotesForNilBlock(t *testing.T) { } if tc.valid { - commit := voteSet.MakeCommit() // panics without > 2/3 valid votes - assert.NotNil(t, commit) - err := valSet.VerifyCommit(voteSet.ChainID(), blockID, height-1, commit) + extCommit := voteSet.MakeExtendedCommit() // panics without > 2/3 valid votes + assert.NotNil(t, extCommit) + err := valSet.VerifyCommit(voteSet.ChainID(), blockID, height-1, extCommit.StripExtensions()) assert.NoError(t, err) } else { - assert.Panics(t, func() { voteSet.MakeCommit() }) + assert.Panics(t, func() { voteSet.MakeExtendedCommit() }) } } } diff --git a/types/evidence.go b/types/evidence.go index aed954a93..c5b5b6223 100644 --- a/types/evidence.go +++ b/types/evidence.go @@ -309,7 +309,7 @@ func (l *LightClientAttackEvidence) GetByzantineValidators(commonVals *Validator // validators who are in the commonVals and voted for the lunatic header if l.ConflictingHeaderIsInvalid(trusted.Header) { for _, commitSig := range l.ConflictingBlock.Commit.Signatures { - if !commitSig.ForBlock() { + if commitSig.BlockIDFlag != BlockIDFlagCommit { continue } @@ -329,12 +329,12 @@ func (l *LightClientAttackEvidence) GetByzantineValidators(commonVals *Validator // only need a single loop to find the validators that voted twice. for i := 0; i < len(l.ConflictingBlock.Commit.Signatures); i++ { sigA := l.ConflictingBlock.Commit.Signatures[i] - if !sigA.ForBlock() { + if sigA.BlockIDFlag != BlockIDFlagCommit { continue } sigB := trusted.Commit.Signatures[i] - if !sigB.ForBlock() { + if sigB.BlockIDFlag != BlockIDFlagCommit { continue } diff --git a/types/evidence_test.go b/types/evidence_test.go index 27e346343..8b06a6218 100644 --- a/types/evidence_test.go +++ b/types/evidence_test.go @@ -153,8 +153,10 @@ func TestLightClientAttackEvidenceBasic(t *testing.T) { header := makeHeaderRandom() header.Height = height blockID := makeBlockID(crypto.Checksum([]byte("blockhash")), math.MaxInt32, crypto.Checksum([]byte("partshash"))) - commit, err := makeCommit(ctx, blockID, height, 1, voteSet, privVals, defaultVoteTime) + extCommit, err := makeExtCommit(ctx, blockID, height, 1, voteSet, privVals, defaultVoteTime) require.NoError(t, err) + commit := extCommit.StripExtensions() + lcae := &LightClientAttackEvidence{ ConflictingBlock: &LightBlock{ SignedHeader: &SignedHeader{ @@ -217,8 +219,10 @@ func TestLightClientAttackEvidenceValidation(t *testing.T) { header.Height = height header.ValidatorsHash = valSet.Hash() blockID := makeBlockID(header.Hash(), math.MaxInt32, crypto.Checksum([]byte("partshash"))) - commit, err := makeCommit(ctx, blockID, height, 1, voteSet, privVals, time.Now()) + extCommit, err := makeExtCommit(ctx, blockID, height, 1, voteSet, privVals, time.Now()) require.NoError(t, err) + commit := extCommit.StripExtensions() + lcae := &LightClientAttackEvidence{ ConflictingBlock: &LightBlock{ SignedHeader: &SignedHeader{ @@ -424,13 +428,13 @@ func TestEvidenceVectors(t *testing.T) { ProposerAddress: []byte("2915b7b15f979e48ebc61774bb1d86ba3136b7eb"), } blockID3 := makeBlockID(header.Hash(), math.MaxInt32, crypto.Checksum([]byte("partshash"))) - commit, err := makeCommit(ctx, blockID3, height, 1, voteSet, privVals, defaultVoteTime) + extCommit, err := makeExtCommit(ctx, blockID3, height, 1, voteSet, privVals, defaultVoteTime) require.NoError(t, err) lcae := &LightClientAttackEvidence{ ConflictingBlock: &LightBlock{ SignedHeader: &SignedHeader{ Header: header, - Commit: commit, + Commit: extCommit.StripExtensions(), }, ValidatorSet: valSet, }, diff --git a/types/part_set.go b/types/part_set.go index 9bf36279f..d9341b61f 100644 --- a/types/part_set.go +++ b/types/part_set.go @@ -145,6 +145,12 @@ func PartSetHeaderFromProto(ppsh *tmproto.PartSetHeader) (*PartSetHeader, error) return psh, psh.ValidateBasic() } +// ProtoPartSetHeaderIsZero is similar to the IsZero function for +// PartSetHeader, but for the Protobuf representation. +func ProtoPartSetHeaderIsZero(ppsh *tmproto.PartSetHeader) bool { + return ppsh.Total == 0 && len(ppsh.Hash) == 0 +} + //------------------------------------- type PartSet struct { diff --git a/types/priv_validator.go b/types/priv_validator.go index 72027c622..b7f4bd165 100644 --- a/types/priv_validator.go +++ b/types/priv_validator.go @@ -90,7 +90,6 @@ func (pv MockPV) SignVote(ctx context.Context, chainID string, vote *tmproto.Vot } signBytes := VoteSignBytes(useChainID, vote) - extSignBytes := VoteExtensionSignBytes(useChainID, vote) sig, err := pv.PrivKey.Sign(signBytes) if err != nil { return err @@ -98,14 +97,15 @@ func (pv MockPV) SignVote(ctx context.Context, chainID string, vote *tmproto.Vot vote.Signature = sig var extSig []byte - // We only sign vote extensions for precommits - if vote.Type == tmproto.PrecommitType { + // We only sign vote extensions for non-nil precommits + if vote.Type == tmproto.PrecommitType && !ProtoBlockIDIsNil(&vote.BlockID) { + extSignBytes := VoteExtensionSignBytes(useChainID, vote) extSig, err = pv.PrivKey.Sign(extSignBytes) if err != nil { return err } } else if len(vote.Extension) > 0 { - return errors.New("unexpected vote extension - vote extensions are only allowed in precommits") + return errors.New("unexpected vote extension - vote extensions are only allowed in non-nil precommits") } vote.ExtensionSignature = extSig return nil diff --git a/types/test_util.go b/types/test_util.go index 8aea2f02c..11daa69b9 100644 --- a/types/test_util.go +++ b/types/test_util.go @@ -8,8 +8,8 @@ import ( tmproto "github.com/tendermint/tendermint/proto/tendermint/types" ) -func makeCommit(ctx context.Context, blockID BlockID, height int64, round int32, - voteSet *VoteSet, validators []PrivValidator, now time.Time) (*Commit, error) { +func makeExtCommit(ctx context.Context, blockID BlockID, height int64, round int32, + voteSet *VoteSet, validators []PrivValidator, now time.Time) (*ExtendedCommit, error) { // all sign for i := 0; i < len(validators); i++ { @@ -33,7 +33,7 @@ func makeCommit(ctx context.Context, blockID BlockID, height int64, round int32, } } - return voteSet.MakeCommit(), nil + return voteSet.MakeExtendedCommit(), nil } func signAddVote(ctx context.Context, privVal PrivValidator, vote *Vote, voteSet *VoteSet) (signed bool, err error) { @@ -46,13 +46,3 @@ func signAddVote(ctx context.Context, privVal PrivValidator, vote *Vote, voteSet vote.ExtensionSignature = v.ExtensionSignature return voteSet.AddVote(vote) } - -// Votes constructed from commits don't have extensions, because we don't store -// the extensions themselves in the commit. This method is used to construct a -// copy of a vote, but nil its extension and signature. -func voteWithoutExtension(v *Vote) *Vote { - vc := v.Copy() - vc.Extension = nil - vc.ExtensionSignature = nil - return vc -} diff --git a/types/validation.go b/types/validation.go index 21c8730f5..02d1b0b56 100644 --- a/types/validation.go +++ b/types/validation.go @@ -36,10 +36,10 @@ func VerifyCommit(chainID string, vals *ValidatorSet, blockID BlockID, votingPowerNeeded := vals.TotalVotingPower() * 2 / 3 // ignore all absent signatures - ignore := func(c CommitSig) bool { return c.Absent() } + ignore := func(c CommitSig) bool { return c.BlockIDFlag == BlockIDFlagAbsent } // only count the signatures that are for the block - count := func(c CommitSig) bool { return c.ForBlock() } + count := func(c CommitSig) bool { return c.BlockIDFlag == BlockIDFlagCommit } // attempt to batch verify if shouldBatchVerify(vals, commit) { @@ -69,7 +69,7 @@ func VerifyCommitLight(chainID string, vals *ValidatorSet, blockID BlockID, votingPowerNeeded := vals.TotalVotingPower() * 2 / 3 // ignore all commit signatures that are not for the block - ignore := func(c CommitSig) bool { return !c.ForBlock() } + ignore := func(c CommitSig) bool { return c.BlockIDFlag != BlockIDFlagCommit } // count all the remaining signatures count := func(c CommitSig) bool { return true } @@ -113,7 +113,7 @@ func VerifyCommitLightTrusting(chainID string, vals *ValidatorSet, commit *Commi votingPowerNeeded := totalVotingPowerMulByNumerator / int64(trustLevel.Denominator) // ignore all commit signatures that are not for the block - ignore := func(c CommitSig) bool { return !c.ForBlock() } + ignore := func(c CommitSig) bool { return c.BlockIDFlag != BlockIDFlagCommit } // count all the remaining signatures count := func(c CommitSig) bool { return true } diff --git a/types/validation_test.go b/types/validation_test.go index 7900ee5ce..f63c34450 100644 --- a/types/validation_test.go +++ b/types/validation_test.go @@ -99,7 +99,12 @@ func TestValidatorSet_VerifyCommit_All(t *testing.T) { vi++ } - commit := NewCommit(tc.height, round, tc.blockID, sigs) + commit := &Commit{ + Height: tc.height, + Round: round, + BlockID: tc.blockID, + Signatures: sigs, + } err := valSet.VerifyCommit(chainID, blockID, height, commit) if tc.expErr { @@ -146,9 +151,10 @@ func TestValidatorSet_VerifyCommit_CheckAllSignatures(t *testing.T) { defer cancel() voteSet, valSet, vals := randVoteSet(ctx, t, h, 0, tmproto.PrecommitType, 4, 10) - commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) - + extCommit, err := makeExtCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) require.NoError(t, err) + commit := extCommit.StripExtensions() + require.NoError(t, valSet.VerifyCommit(chainID, blockID, h, commit)) // malleate 4th signature @@ -176,9 +182,10 @@ func TestValidatorSet_VerifyCommitLight_ReturnsAsSoonAsMajorityOfVotingPowerSign defer cancel() voteSet, valSet, vals := randVoteSet(ctx, t, h, 0, tmproto.PrecommitType, 4, 10) - commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) - + extCommit, err := makeExtCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) require.NoError(t, err) + commit := extCommit.StripExtensions() + require.NoError(t, valSet.VerifyCommit(chainID, blockID, h, commit)) // malleate 4th signature (3 signatures are enough for 2/3+) @@ -203,9 +210,10 @@ func TestValidatorSet_VerifyCommitLightTrusting_ReturnsAsSoonAsTrustLevelOfVotin defer cancel() voteSet, valSet, vals := randVoteSet(ctx, t, h, 0, tmproto.PrecommitType, 4, 10) - commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) - + extCommit, err := makeExtCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) require.NoError(t, err) + commit := extCommit.StripExtensions() + require.NoError(t, valSet.VerifyCommit(chainID, blockID, h, commit)) // malleate 3rd signature (2 signatures are enough for 1/3+ trust level) @@ -227,10 +235,11 @@ func TestValidatorSet_VerifyCommitLightTrusting(t *testing.T) { var ( blockID = makeBlockIDRandom() voteSet, originalValset, vals = randVoteSet(ctx, t, 1, 1, tmproto.PrecommitType, 6, 1) - commit, err = makeCommit(ctx, blockID, 1, 1, voteSet, vals, time.Now()) + extCommit, err = makeExtCommit(ctx, blockID, 1, 1, voteSet, vals, time.Now()) newValSet, _ = randValidatorPrivValSet(ctx, t, 2, 1) ) require.NoError(t, err) + commit := extCommit.StripExtensions() testCases := []struct { valSet *ValidatorSet @@ -271,11 +280,11 @@ func TestValidatorSet_VerifyCommitLightTrustingErrorsOnOverflow(t *testing.T) { var ( blockID = makeBlockIDRandom() voteSet, valSet, vals = randVoteSet(ctx, t, 1, 1, tmproto.PrecommitType, 1, MaxTotalVotingPower) - commit, err = makeCommit(ctx, blockID, 1, 1, voteSet, vals, time.Now()) + extCommit, err = makeExtCommit(ctx, blockID, 1, 1, voteSet, vals, time.Now()) ) require.NoError(t, err) - err = valSet.VerifyCommitLightTrusting("test_chain_id", commit, + err = valSet.VerifyCommitLightTrusting("test_chain_id", extCommit.StripExtensions(), tmmath.Fraction{Numerator: 25, Denominator: 55}) if assert.Error(t, err) { assert.Contains(t, err.Error(), "int64 overflow") diff --git a/types/validator_set_test.go b/types/validator_set_test.go index 096327626..8b5846da9 100644 --- a/types/validator_set_test.go +++ b/types/validator_set_test.go @@ -1539,8 +1539,9 @@ func BenchmarkValidatorSet_VerifyCommit_Ed25519(b *testing.B) { // nolint // generate n validators voteSet, valSet, vals := randVoteSet(ctx, b, h, 0, tmproto.PrecommitType, n, int64(n*5)) // create a commit with n validators - commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) + extCommit, err := makeExtCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) require.NoError(b, err) + commit := extCommit.StripExtensions() for i := 0; i < b.N/n; i++ { err = valSet.VerifyCommit(chainID, blockID, h, commit) @@ -1567,8 +1568,9 @@ func BenchmarkValidatorSet_VerifyCommitLight_Ed25519(b *testing.B) { // nolint voteSet, valSet, vals := randVoteSet(ctx, b, h, 0, tmproto.PrecommitType, n, int64(n*5)) // create a commit with n validators - commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) + extCommit, err := makeExtCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) require.NoError(b, err) + commit := extCommit.StripExtensions() for i := 0; i < b.N/n; i++ { err = valSet.VerifyCommitLight(chainID, blockID, h, commit) @@ -1594,8 +1596,9 @@ func BenchmarkValidatorSet_VerifyCommitLightTrusting_Ed25519(b *testing.B) { // generate n validators voteSet, valSet, vals := randVoteSet(ctx, b, h, 0, tmproto.PrecommitType, n, int64(n*5)) // create a commit with n validators - commit, err := makeCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) + extCommit, err := makeExtCommit(ctx, blockID, h, 0, voteSet, vals, time.Now()) require.NoError(b, err) + commit := extCommit.StripExtensions() for i := 0; i < b.N/n; i++ { err = valSet.VerifyCommitLightTrusting(chainID, commit, tmmath.Fraction{Numerator: 1, Denominator: 3}) diff --git a/types/vote.go b/types/vote.go index f20ee491e..446de130a 100644 --- a/types/vote.go +++ b/types/vote.go @@ -14,6 +14,9 @@ import ( const ( nilVoteStr string = "nil-Vote" + + // The maximum supported number of bytes in a vote extension. + MaxVoteExtensionSize int = 1024 * 1024 ) var ( @@ -109,6 +112,26 @@ func (vote *Vote) CommitSig() CommitSig { } } +// ExtendedCommitSig attempts to construct an ExtendedCommitSig from this vote. +// Panics if either the vote extension signature is missing or if the block ID +// is not either empty or complete. +func (vote *Vote) ExtendedCommitSig() ExtendedCommitSig { + if vote == nil { + return NewExtendedCommitSigAbsent() + } + + cs := vote.CommitSig() + if vote.BlockID.IsComplete() && len(vote.ExtensionSignature) == 0 { + panic(fmt.Sprintf("Invalid vote %v - BlockID is complete but missing vote extension signature", vote)) + } + + return ExtendedCommitSig{ + CommitSig: cs, + Extension: vote.Extension, + ExtensionSignature: vote.ExtensionSignature, + } +} + // VoteSignBytes returns the proto-encoding of the canonicalized Vote, for // signing. Panics if the marshaling fails. // @@ -216,12 +239,10 @@ func (vote *Vote) VerifyWithExtension(chainID string, pubKey crypto.PubKey) erro if err != nil { return err } - // We only verify vote extension signatures for precommits. - if vote.Type == tmproto.PrecommitType { + // We only verify vote extension signatures for non-nil precommits. + if vote.Type == tmproto.PrecommitType && !ProtoBlockIDIsNil(&v.BlockID) { extSignBytes := VoteExtensionSignBytes(chainID, v) - // TODO: Remove extension signature nil check to enforce vote extension - // signing once we resolve https://github.com/tendermint/tendermint/issues/8272 - if vote.ExtensionSignature != nil && !pubKey.VerifySignature(extSignBytes, vote.ExtensionSignature) { + if !pubKey.VerifySignature(extSignBytes, vote.ExtensionSignature) { return ErrVoteInvalidSignature } } @@ -273,8 +294,10 @@ func (vote *Vote) ValidateBasic() error { return fmt.Errorf("signature is too big (max: %d)", MaxSignatureSize) } - // We should only ever see vote extensions in precommits. - if vote.Type != tmproto.PrecommitType { + // We should only ever see vote extensions in non-nil precommits, otherwise + // this is a violation of the specification. + // https://github.com/tendermint/tendermint/issues/8487 + if vote.Type != tmproto.PrecommitType || (vote.Type == tmproto.PrecommitType && vote.BlockID.IsNil()) { if len(vote.Extension) > 0 { return errors.New("unexpected vote extension") } @@ -294,12 +317,9 @@ func (vote *Vote) ValidateWithExtension() error { return err } - // We should always see vote extension signatures in precommits - if vote.Type == tmproto.PrecommitType { - // TODO(thane): Remove extension length check once - // https://github.com/tendermint/tendermint/issues/8272 is - // resolved. - if len(vote.Extension) > 0 && len(vote.ExtensionSignature) == 0 { + // We should always see vote extension signatures in non-nil precommits + if vote.Type == tmproto.PrecommitType && !vote.BlockID.IsNil() { + if len(vote.ExtensionSignature) == 0 { return errors.New("vote extension signature is missing") } if len(vote.ExtensionSignature) > MaxSignatureSize { diff --git a/types/vote_set.go b/types/vote_set.go index b4d149576..224d4e4f8 100644 --- a/types/vote_set.go +++ b/types/vote_set.go @@ -220,13 +220,6 @@ func (voteSet *VoteSet) getVote(valIndex int32, blockKey string) (vote *Vote, ok return nil, false } -func (voteSet *VoteSet) GetVotes() []*Vote { - if voteSet == nil { - return nil - } - return voteSet.votes -} - // Assumes signature is valid. // If conflicting vote exists, returns it. func (voteSet *VoteSet) addVerifiedVote( @@ -606,36 +599,41 @@ func (voteSet *VoteSet) sumTotalFrac() (int64, int64, float64) { //-------------------------------------------------------------------------------- // Commit -// MakeCommit constructs a Commit from the VoteSet. It only includes precommits -// for the block, which has 2/3+ majority, and nil. +// MakeExtendedCommit constructs a Commit from the VoteSet. It only includes +// precommits for the block, which has 2/3+ majority, and nil. // // Panics if the vote type is not PrecommitType or if there's no +2/3 votes for // a single block. -func (voteSet *VoteSet) MakeCommit() *Commit { +func (voteSet *VoteSet) MakeExtendedCommit() *ExtendedCommit { if voteSet.signedMsgType != tmproto.PrecommitType { - panic("Cannot MakeCommit() unless VoteSet.Type is PrecommitType") + panic("Cannot MakeExtendCommit() unless VoteSet.Type is PrecommitType") } voteSet.mtx.Lock() defer voteSet.mtx.Unlock() // Make sure we have a 2/3 majority if voteSet.maj23 == nil { - panic("Cannot MakeCommit() unless a blockhash has +2/3") + panic("Cannot MakeExtendCommit() unless a blockhash has +2/3") } - // For every validator, get the precommit - commitSigs := make([]CommitSig, len(voteSet.votes)) + // For every validator, get the precommit with extensions + sigs := make([]ExtendedCommitSig, len(voteSet.votes)) for i, v := range voteSet.votes { - commitSig := v.CommitSig() + sig := v.ExtendedCommitSig() // if block ID exists but doesn't match, exclude sig - if commitSig.ForBlock() && !v.BlockID.Equals(*voteSet.maj23) { - commitSig = NewCommitSigAbsent() + if sig.BlockIDFlag == BlockIDFlagCommit && !v.BlockID.Equals(*voteSet.maj23) { + sig = NewExtendedCommitSigAbsent() } - commitSigs[i] = commitSig + sigs[i] = sig } - return NewCommit(voteSet.GetHeight(), voteSet.GetRound(), *voteSet.maj23, commitSigs) + return &ExtendedCommit{ + Height: voteSet.GetHeight(), + Round: voteSet.GetRound(), + BlockID: *voteSet.maj23, + ExtendedSignatures: sigs, + } } //-------------------------------------------------------------------------------- diff --git a/types/vote_set_test.go b/types/vote_set_test.go index 1805b4c3e..8d166d508 100644 --- a/types/vote_set_test.go +++ b/types/vote_set_test.go @@ -450,7 +450,7 @@ func TestVoteSet_MakeCommit(t *testing.T) { } // MakeCommit should fail. - assert.Panics(t, func() { voteSet.MakeCommit() }, "Doesn't have +2/3 majority") + assert.Panics(t, func() { voteSet.MakeExtendedCommit() }, "Doesn't have +2/3 majority") // 7th voted for some other block. { @@ -487,13 +487,13 @@ func TestVoteSet_MakeCommit(t *testing.T) { require.NoError(t, err) } - commit := voteSet.MakeCommit() + extCommit := voteSet.MakeExtendedCommit() // Commit should have 10 elements - assert.Equal(t, 10, len(commit.Signatures)) + assert.Equal(t, 10, len(extCommit.ExtendedSignatures)) // Ensure that Commit is good. - if err := commit.ValidateBasic(); err != nil { + if err := extCommit.ValidateBasic(); err != nil { t.Errorf("error in Commit.ValidateBasic(): %v", err) } } diff --git a/types/vote_test.go b/types/vote_test.go index 5673ccf57..70cd91381 100644 --- a/types/vote_test.go +++ b/types/vote_test.go @@ -223,26 +223,22 @@ func TestVoteExtension(t *testing.T) { includeSignature: true, expectError: false, }, - // TODO(thane): Re-enable once - // https://github.com/tendermint/tendermint/issues/8272 is resolved - //{ - // name: "no extension signature", - // extension: []byte("extension"), - // includeSignature: false, - // expectError: true, - //}, + { + name: "no extension signature", + extension: []byte("extension"), + includeSignature: false, + expectError: true, + }, { name: "empty extension", includeSignature: true, expectError: false, }, - // TODO: Re-enable once - // https://github.com/tendermint/tendermint/issues/8272 is resolved. - //{ - // name: "no extension and no signature", - // includeSignature: false, - // expectError: true, - //}, + { + name: "no extension and no signature", + includeSignature: false, + expectError: true, + }, } for _, tc := range testCases { @@ -497,11 +493,11 @@ func getSampleCommit(ctx context.Context, t testing.TB) *Commit { lastID := makeBlockIDRandom() voteSet, _, vals := randVoteSet(ctx, t, 2, 1, tmproto.PrecommitType, 10, 1) - commit, err := makeCommit(ctx, lastID, 2, 1, voteSet, vals, time.Now()) + commit, err := makeExtCommit(ctx, lastID, 2, 1, voteSet, vals, time.Now()) require.NoError(t, err) - return commit + return commit.StripExtensions() } func BenchmarkVoteSignBytes(b *testing.B) {