Browse Source

separate block vs state based validation

pull/994/head
Ethan Buchman 7 years ago
parent
commit
3d00c477fc
12 changed files with 231 additions and 209 deletions
  1. +2
    -2
      blockchain/reactor.go
  2. +2
    -10
      blockchain/reactor_test.go
  3. +4
    -4
      consensus/replay_test.go
  4. +2
    -6
      consensus/state.go
  5. +7
    -7
      consensus/state_test.go
  6. +53
    -9
      state/execution.go
  7. +41
    -14
      state/execution_test.go
  8. +65
    -40
      state/state.go
  9. +2
    -2
      state/state_test.go
  10. +27
    -57
      types/block.go
  11. +26
    -48
      types/block_test.go
  12. +0
    -10
      types/vote_set_test.go

+ 2
- 2
blockchain/reactor.go View File

@ -183,7 +183,7 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
// maxMsgSize returns the maximum allowable size of a // maxMsgSize returns the maximum allowable size of a
// message on the blockchain reactor. // message on the blockchain reactor.
func (bcR *BlockchainReactor) maxMsgSize() int { func (bcR *BlockchainReactor) maxMsgSize() int {
return bcR.state.Params.BlockSize.MaxBytes + 2
return bcR.state.ConsensusParams.BlockSize.MaxBytes + 2
} }
// Handle messages from the poolReactor telling the reactor what to do. // Handle messages from the poolReactor telling the reactor what to do.
@ -251,7 +251,7 @@ FOR_LOOP:
// We need both to sync the first block. // We need both to sync the first block.
break SYNC_LOOP break SYNC_LOOP
} }
firstParts := first.MakePartSet(bcR.state.Params.BlockPartSizeBytes)
firstParts := first.MakePartSet(bcR.state.ConsensusParams.BlockPartSizeBytes)
firstPartsHeader := firstParts.Header() firstPartsHeader := firstParts.Header()
// Finally, verify the first block using the second's commit // Finally, verify the first block using the second's commit
// NOTE: we can probably make this more efficient, but note that calling // NOTE: we can probably make this more efficient, but note that calling


+ 2
- 10
blockchain/reactor_test.go View File

@ -41,7 +41,7 @@ func newBlockchainReactor(logger log.Logger, maxBlockHeight int64) *BlockchainRe
for blockHeight := int64(1); blockHeight <= maxBlockHeight; blockHeight++ { for blockHeight := int64(1); blockHeight <= maxBlockHeight; blockHeight++ {
firstBlock := makeBlock(blockHeight, state) firstBlock := makeBlock(blockHeight, state)
secondBlock := makeBlock(blockHeight+1, state) secondBlock := makeBlock(blockHeight+1, state)
firstParts := firstBlock.MakePartSet(state.Params.BlockGossip.BlockPartSizeBytes)
firstParts := firstBlock.MakePartSet(state.ConsensusParams.BlockGossip.BlockPartSizeBytes)
blockStore.SaveBlock(firstBlock, firstParts, secondBlock.LastCommit) blockStore.SaveBlock(firstBlock, firstParts, secondBlock.LastCommit)
} }
@ -105,15 +105,7 @@ func makeTxs(height int64) (txs []types.Tx) {
} }
func makeBlock(height int64, state *sm.State) *types.Block { func makeBlock(height int64, state *sm.State) *types.Block {
prevHash := state.LastBlockID.Hash
prevParts := types.PartSetHeader{}
valHash := state.Validators.Hash()
prevBlockID := types.BlockID{prevHash, prevParts}
block, _ := types.MakeBlock(height, "test_chain", makeTxs(height),
state.LastBlockTotalTx, new(types.Commit),
prevBlockID, valHash, state.AppHash,
state.LastConsensusHash,
state.Params.BlockGossip.BlockPartSizeBytes)
block, _ := state.MakeBlock(height, makeTxs(height), new(types.Commit))
return block return block
} }


+ 4
- 4
consensus/replay_test.go View File

@ -107,9 +107,9 @@ func TestWALCrash(t *testing.T) {
{"block with a smaller part size", {"block with a smaller part size",
func(cs *ConsensusState, ctx context.Context) { func(cs *ConsensusState, ctx context.Context) {
// XXX: is there a better way to change BlockPartSizeBytes? // XXX: is there a better way to change BlockPartSizeBytes?
params := cs.state.Params
params := cs.state.ConsensusParams
params.BlockPartSizeBytes = 512 params.BlockPartSizeBytes = 512
cs.state.Params = params
cs.state.ConsensusParams = params
sendTxs(cs, ctx) sendTxs(cs, ctx)
}, },
1}, 1},
@ -392,7 +392,7 @@ func testHandshakeReplay(t *testing.T, nBlocks int, mode uint) {
} }
func applyBlock(st *sm.State, blk *types.Block, proxyApp proxy.AppConns) { func applyBlock(st *sm.State, blk *types.Block, proxyApp proxy.AppConns) {
testPartSize := st.Params.BlockPartSizeBytes
testPartSize := st.ConsensusParams.BlockPartSizeBytes
err := st.ApplyBlock(types.NopEventBus{}, proxyApp.Consensus(), blk, blk.MakePartSet(testPartSize).Header(), mempool) err := st.ApplyBlock(types.NopEventBus{}, proxyApp.Consensus(), blk, blk.MakePartSet(testPartSize).Header(), mempool)
if err != nil { if err != nil {
panic(err) panic(err)
@ -590,7 +590,7 @@ func stateAndStore(config *cfg.Config, pubKey crypto.PubKey) (*sm.State, *mockBl
state, _ := sm.MakeGenesisStateFromFile(stateDB, config.GenesisFile()) state, _ := sm.MakeGenesisStateFromFile(stateDB, config.GenesisFile())
state.SetLogger(log.TestingLogger().With("module", "state")) state.SetLogger(log.TestingLogger().With("module", "state"))
store := NewMockBlockStore(config, state.Params)
store := NewMockBlockStore(config, state.ConsensusParams)
return state, store return state, store
} }


+ 2
- 6
consensus/state.go View File

@ -863,11 +863,7 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
// Mempool validated transactions // Mempool validated transactions
txs := cs.mempool.Reap(cs.config.MaxBlockSizeTxs) txs := cs.mempool.Reap(cs.config.MaxBlockSizeTxs)
return types.MakeBlock(cs.Height, cs.state.ChainID, txs,
cs.state.LastBlockTotalTx, commit,
cs.state.LastBlockID, cs.state.Validators.Hash(),
cs.state.AppHash, cs.state.LastConsensusHash,
cs.state.Params.BlockPartSizeBytes)
return cs.state.MakeBlock(cs.Height, txs, commit)
} }
// Enter: `timeoutPropose` after entering Propose. // Enter: `timeoutPropose` after entering Propose.
@ -1307,7 +1303,7 @@ func (cs *ConsensusState) addProposalBlockPart(height int64, part *types.Part, v
var n int var n int
var err error var err error
cs.ProposalBlock = wire.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(), cs.ProposalBlock = wire.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(),
cs.state.Params.BlockSize.MaxBytes, &n, &err).(*types.Block)
cs.state.ConsensusParams.BlockSize.MaxBytes, &n, &err).(*types.Block)
// NOTE: it's possible to receive complete proposal blocks for future rounds without having the proposal // NOTE: it's possible to receive complete proposal blocks for future rounds without having the proposal
cs.Logger.Info("Received complete proposal block", "height", cs.ProposalBlock.Height, "hash", cs.ProposalBlock.Hash()) cs.Logger.Info("Received complete proposal block", "height", cs.ProposalBlock.Height, "hash", cs.ProposalBlock.Hash())
if cs.Step == cstypes.RoundStepPropose && cs.isProposalComplete() { if cs.Step == cstypes.RoundStepPropose && cs.isProposalComplete() {


+ 7
- 7
consensus/state_test.go View File

@ -184,7 +184,7 @@ func TestBadProposal(t *testing.T) {
height, round := cs1.Height, cs1.Round height, round := cs1.Height, cs1.Round
vs2 := vss[1] vs2 := vss[1]
partSize := cs1.state.Params.BlockPartSizeBytes
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal) proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
voteCh := subscribe(cs1.eventBus, types.EventQueryVote) voteCh := subscribe(cs1.eventBus, types.EventQueryVote)
@ -339,7 +339,7 @@ func TestLockNoPOL(t *testing.T) {
vs2 := vss[1] vs2 := vss[1]
height := cs1.Height height := cs1.Height
partSize := cs1.state.Params.BlockPartSizeBytes
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose) timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait) timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
@ -507,7 +507,7 @@ func TestLockPOLRelock(t *testing.T) {
cs1, vss := randConsensusState(4) cs1, vss := randConsensusState(4)
vs2, vs3, vs4 := vss[1], vss[2], vss[3] vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := cs1.state.Params.BlockPartSizeBytes
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose) timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait) timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
@ -622,7 +622,7 @@ func TestLockPOLUnlock(t *testing.T) {
cs1, vss := randConsensusState(4) cs1, vss := randConsensusState(4)
vs2, vs3, vs4 := vss[1], vss[2], vss[3] vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := cs1.state.Params.BlockPartSizeBytes
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal) proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose) timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
@ -719,7 +719,7 @@ func TestLockPOLSafety1(t *testing.T) {
cs1, vss := randConsensusState(4) cs1, vss := randConsensusState(4)
vs2, vs3, vs4 := vss[1], vss[2], vss[3] vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := cs1.state.Params.BlockPartSizeBytes
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal) proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose) timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
@ -842,7 +842,7 @@ func TestLockPOLSafety2(t *testing.T) {
cs1, vss := randConsensusState(4) cs1, vss := randConsensusState(4)
vs2, vs3, vs4 := vss[1], vss[2], vss[3] vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := cs1.state.Params.BlockPartSizeBytes
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal) proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose) timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
@ -1021,7 +1021,7 @@ func TestHalt1(t *testing.T) {
cs1, vss := randConsensusState(4) cs1, vss := randConsensusState(4)
vs2, vs3, vs4 := vss[1], vss[2], vss[3] vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := cs1.state.Params.BlockPartSizeBytes
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal) proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait) timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)


+ 53
- 9
state/execution.go View File

@ -1,6 +1,7 @@
package state package state
import ( import (
"bytes"
"errors" "errors"
"fmt" "fmt"
@ -184,26 +185,69 @@ func (s *State) ValidateBlock(block *types.Block) error {
return s.validateBlock(block) return s.validateBlock(block)
} }
func (s *State) validateBlock(block *types.Block) error {
// MakeBlock builds a block with the given txs and commit from the current state.
func (s *State) MakeBlock(height int64, txs []types.Tx, commit *types.Commit) (*types.Block, *types.PartSet) {
// build base block
block := types.MakeBlock(height, txs, commit)
// fill header with state data
block.ChainID = s.ChainID
block.TotalTxs = s.LastBlockTotalTx + block.NumTxs
block.LastBlockID = s.LastBlockID
block.ValidatorsHash = s.Validators.Hash()
block.AppHash = s.AppHash
block.ConsensusHash = s.LastConsensusParams.Hash()
return block, block.MakePartSet(s.ConsensusParams.BlockGossip.BlockPartSizeBytes)
}
func (s *State) validateBlock(b *types.Block) error {
// Basic block validation. // Basic block validation.
err := block.ValidateBasic(s.ChainID, s.LastBlockHeight,
s.LastBlockTotalTx, s.LastBlockID, s.LastBlockTime, s.AppHash, s.LastConsensusHash)
if err != nil {
if err := b.ValidateBasic(); err != nil {
return err return err
} }
if b.ChainID != s.ChainID {
return fmt.Errorf("Wrong Block.Header.ChainID. Expected %v, got %v", s.ChainID, b.ChainID)
}
if b.Height != s.LastBlockHeight+1 {
return fmt.Errorf("Wrong Block.Header.Height. Expected %v, got %v", s.LastBlockHeight+1, b.Height)
}
/* TODO: Determine bounds for Time
See blockchain/reactor "stopSyncingDurationMinutes"
if !b.Time.After(lastBlockTime) {
return errors.New("Invalid Block.Header.Time")
}
*/
newTxs := int64(len(b.Data.Txs))
if b.TotalTxs != s.LastBlockTotalTx+newTxs {
return fmt.Errorf("Wrong Block.Header.TotalTxs. Expected %v, got %v", s.LastBlockTotalTx+newTxs, b.TotalTxs)
}
if !b.LastBlockID.Equals(s.LastBlockID) {
return fmt.Errorf("Wrong Block.Header.LastBlockID. Expected %v, got %v", s.LastBlockID, b.LastBlockID)
}
if !bytes.Equal(b.AppHash, s.AppHash) {
return fmt.Errorf("Wrong Block.Header.AppHash. Expected %X, got %v", s.AppHash, b.AppHash)
}
if !bytes.Equal(b.ConsensusHash, s.LastConsensusParams.Hash()) {
return fmt.Errorf("Wrong Block.Header.ConsensusHash. Expected %X, got %v", s.LastConsensusParams.Hash(), b.ConsensusHash)
}
// Validate block LastCommit. // Validate block LastCommit.
if block.Height == 1 {
if len(block.LastCommit.Precommits) != 0 {
if b.Height == 1 {
if len(b.LastCommit.Precommits) != 0 {
return errors.New("Block at height 1 (first block) should have no LastCommit precommits") return errors.New("Block at height 1 (first block) should have no LastCommit precommits")
} }
} else { } else {
if len(block.LastCommit.Precommits) != s.LastValidators.Size() {
if len(b.LastCommit.Precommits) != s.LastValidators.Size() {
return errors.New(cmn.Fmt("Invalid block commit size. Expected %v, got %v", return errors.New(cmn.Fmt("Invalid block commit size. Expected %v, got %v",
s.LastValidators.Size(), len(block.LastCommit.Precommits)))
s.LastValidators.Size(), len(b.LastCommit.Precommits)))
} }
err := s.LastValidators.VerifyCommit( err := s.LastValidators.VerifyCommit(
s.ChainID, s.LastBlockID, block.Height-1, block.LastCommit)
s.ChainID, s.LastBlockID, b.Height-1, b.LastCommit)
if err != nil { if err != nil {
return err return err
} }


+ 41
- 14
state/execution_test.go View File

@ -23,6 +23,43 @@ var (
nTxsPerBlock = 10 nTxsPerBlock = 10
) )
/*
func TestValidateBlock(t *testing.T) {
state := state()
state.SetLogger(log.TestingLogger())
block := makeBlock(1, state)
// proper block must pass
err = block.ValidateBasic("hello", h-1, 10, lastID, block.Time, appHash, consensusHash)
require.NoError(t, err)
// wrong chain fails
err = block.ValidateBasic("other", h-1, 10, lastID, block.Time, appHash, consensusHash)
require.Error(t, err)
// wrong height fails
err = block.ValidateBasic("hello", h+4, 10, lastID, block.Time, appHash, consensusHash)
require.Error(t, err)
// wrong total tx fails
err = block.ValidateBasic("hello", h-1, 15, lastID, block.Time, appHash, consensusHash)
require.Error(t, err)
// wrong blockid fails
err = block.ValidateBasic("hello", h-1, 10, makeBlockID(), block.Time, appHash, consensusHash)
require.Error(t, err)
// wrong app hash fails
err = block.ValidateBasic("hello", h-1, 10, lastID, block.Time, []byte("bad-hash"), consensusHash)
require.Error(t, err)
// wrong consensus hash fails
err = block.ValidateBasic("hello", h-1, 10, lastID, block.Time, appHash, []byte("wrong-params"))
require.Error(t, err)
}
*/
func TestApplyBlock(t *testing.T) { func TestApplyBlock(t *testing.T) {
cc := proxy.NewLocalClientCreator(dummy.NewDummyApplication()) cc := proxy.NewLocalClientCreator(dummy.NewDummyApplication())
proxyApp := proxy.NewAppConns(cc, nil) proxyApp := proxy.NewAppConns(cc, nil)
@ -33,7 +70,7 @@ func TestApplyBlock(t *testing.T) {
state := state() state := state()
state.SetLogger(log.TestingLogger()) state.SetLogger(log.TestingLogger())
block := makeBlock(1, state)
block := makeBlock(state, 1)
err = state.ApplyBlock(types.NopEventBus{}, proxyApp.Consensus(), block, block.MakePartSet(testPartSize).Header(), types.MockMempool{}) err = state.ApplyBlock(types.NopEventBus{}, proxyApp.Consensus(), block, block.MakePartSet(testPartSize).Header(), types.MockMempool{})
require.Nil(t, err) require.Nil(t, err)
@ -79,10 +116,7 @@ func TestBeginBlockAbsentValidators(t *testing.T) {
for _, tc := range testCases { for _, tc := range testCases {
lastCommit := &types.Commit{BlockID: prevBlockID, Precommits: tc.lastCommitPrecommits} lastCommit := &types.Commit{BlockID: prevBlockID, Precommits: tc.lastCommitPrecommits}
valHash := state.Validators.Hash()
block, _ := types.MakeBlock(2, chainID, makeTxs(2), state.LastBlockTotalTx, lastCommit,
prevBlockID, valHash, state.AppHash, state.LastConsensusHash, testPartSize)
block, _ := state.MakeBlock(2, makeTxs(2), lastCommit)
_, err = ExecCommitBlock(proxyApp.Consensus(), block, log.TestingLogger(), lastValidators) _, err = ExecCommitBlock(proxyApp.Consensus(), block, log.TestingLogger(), lastValidators)
require.Nil(t, err, tc.desc) require.Nil(t, err, tc.desc)
@ -112,15 +146,8 @@ func state() *State {
return s return s
} }
func makeBlock(height int64, state *State) *types.Block {
prevHash := state.LastBlockID.Hash
prevParts := types.PartSetHeader{}
valHash := state.Validators.Hash()
prevBlockID := types.BlockID{prevHash, prevParts}
block, _ := types.MakeBlock(height, chainID,
makeTxs(height), state.LastBlockTotalTx,
new(types.Commit), prevBlockID, valHash,
state.AppHash, state.LastConsensusHash, testPartSize)
func makeBlock(state *State, height int64) *types.Block {
block, _ := state.MakeBlock(height, makeTxs(state.LastBlockHeight), new(types.Commit))
return block return block
} }


+ 65
- 40
state/state.go View File

@ -18,6 +18,7 @@ import (
"github.com/tendermint/tendermint/types" "github.com/tendermint/tendermint/types"
) )
// database keys
var ( var (
stateKey = []byte("stateKey") stateKey = []byte("stateKey")
abciResponsesKey = []byte("abciResponsesKey") abciResponsesKey = []byte("abciResponsesKey")
@ -27,40 +28,51 @@ func calcValidatorsKey(height int64) []byte {
return []byte(cmn.Fmt("validatorsKey:%v", height)) return []byte(cmn.Fmt("validatorsKey:%v", height))
} }
/*func calcConsensusParamsKey(height int64) []byte {
return []byte(cmn.Fmt("consensusParamsKey:%v", height))
}*/
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------
// State represents the latest committed state of the Tendermint consensus,
// including the last committed block and validator set.
// Newly committed blocks are validated and executed against the State.
// State is a short description of the latest committed block of the Tendermint consensus.
// It keeps all information necessary to validate new blocks,
// including the last validator set and the consensus params.
// All fields are exposed so the struct can be easily serialized,
// but the fields should only be changed by calling state.SetBlockAndValidators.
// NOTE: not goroutine-safe. // NOTE: not goroutine-safe.
type State struct { type State struct {
// mtx for writing to db // mtx for writing to db
mtx sync.Mutex mtx sync.Mutex
db dbm.DB db dbm.DB
// Immutable
ChainID string ChainID string
// Consensus parameters used for validating blocks
Params types.ConsensusParams
// These fields are updated by SetBlockAndValidators.
// Exposed fields are updated by SetBlockAndValidators.
// LastBlockHeight=0 at genesis (ie. block(H=0) does not exist) // LastBlockHeight=0 at genesis (ie. block(H=0) does not exist)
// LastValidators is used to validate block.LastCommit.
LastBlockHeight int64 LastBlockHeight int64
LastBlockTotalTx int64 LastBlockTotalTx int64
LastBlockID types.BlockID LastBlockID types.BlockID
LastBlockTime time.Time LastBlockTime time.Time
Validators *types.ValidatorSet
LastValidators *types.ValidatorSet
// When a block returns a validator set change via EndBlock,
// the change only applies to the next block.
// So, if s.LastBlockHeight causes a valset change,
// LastValidators is used to validate block.LastCommit.
// Validators are persisted to the database separately every time they change,
// so we can query for historical validator sets.
// Note that if s.LastBlockHeight causes a valset change,
// we set s.LastHeightValidatorsChanged = s.LastBlockHeight + 1 // we set s.LastHeightValidatorsChanged = s.LastBlockHeight + 1
Validators *types.ValidatorSet
LastValidators *types.ValidatorSet
LastHeightValidatorsChanged int64 LastHeightValidatorsChanged int64
// AppHash is updated after Commit
// Consensus parameters used for validating blocks.
// Changes returned by EndBlock and updated after Commit.
ConsensusParams types.ConsensusParams
LastConsensusParams types.ConsensusParams
LastHeightConsensusParamsChanged int64
// The latest AppHash we've received from calling abci.Commit()
AppHash []byte AppHash []byte
// LastConsensusHash is updated after Commit
LastConsensusHash []byte
logger log.Logger logger log.Logger
} }
@ -114,19 +126,26 @@ func (s *State) SetLogger(l log.Logger) {
// Copy makes a copy of the State for mutating. // Copy makes a copy of the State for mutating.
func (s *State) Copy() *State { func (s *State) Copy() *State {
return &State{ return &State{
db: s.db,
LastBlockHeight: s.LastBlockHeight,
LastBlockTotalTx: s.LastBlockTotalTx,
LastBlockID: s.LastBlockID,
LastBlockTime: s.LastBlockTime,
db: s.db,
ChainID: s.ChainID,
LastBlockHeight: s.LastBlockHeight,
LastBlockTotalTx: s.LastBlockTotalTx,
LastBlockID: s.LastBlockID,
LastBlockTime: s.LastBlockTime,
Validators: s.Validators.Copy(), Validators: s.Validators.Copy(),
LastValidators: s.LastValidators.Copy(), LastValidators: s.LastValidators.Copy(),
AppHash: s.AppHash,
LastConsensusHash: s.LastConsensusHash,
LastHeightValidatorsChanged: s.LastHeightValidatorsChanged, LastHeightValidatorsChanged: s.LastHeightValidatorsChanged,
logger: s.logger,
ChainID: s.ChainID,
Params: s.Params,
ConsensusParams: s.ConsensusParams,
LastConsensusParams: s.LastConsensusParams,
LastHeightConsensusParamsChanged: s.LastHeightConsensusParamsChanged,
AppHash: s.AppHash,
logger: s.logger,
} }
} }
@ -254,20 +273,20 @@ func (s *State) SetBlockAndValidators(header *types.Header, blockPartsHeader typ
// Update validator accums and set state variables // Update validator accums and set state variables
nextValSet.IncrementAccum(1) nextValSet.IncrementAccum(1)
nextParams := applyUpdates(s.Params,
nextParams := applyUpdates(s.ConsensusParams,
abciResponses.EndBlock.ConsensusParamUpdates) abciResponses.EndBlock.ConsensusParamUpdates)
err := nextParams.Validate() err := nextParams.Validate()
if err != nil { if err != nil {
s.logger.Error("Error updating consensus params", "err", err) s.logger.Error("Error updating consensus params", "err", err)
// TODO: err or carry on? // TODO: err or carry on?
nextParams = s.Params
nextParams = s.ConsensusParams
} }
s.setBlockAndValidators(header.Height, s.setBlockAndValidators(header.Height,
header.NumTxs, header.NumTxs,
types.BlockID{header.Hash(), blockPartsHeader}, types.BlockID{header.Hash(), blockPartsHeader},
header.Time, header.Time,
prevValSet, nextValSet,
nextValSet,
nextParams) nextParams)
} }
@ -313,17 +332,19 @@ func applyUpdates(p types.ConsensusParams,
func (s *State) setBlockAndValidators(height int64, func (s *State) setBlockAndValidators(height int64,
newTxs int64, blockID types.BlockID, blockTime time.Time, newTxs int64, blockID types.BlockID, blockTime time.Time,
prevValSet, nextValSet *types.ValidatorSet,
nextParams types.ConsensusParams) {
valSet *types.ValidatorSet,
params types.ConsensusParams) {
s.LastBlockHeight = height s.LastBlockHeight = height
s.LastBlockTotalTx += newTxs s.LastBlockTotalTx += newTxs
s.LastBlockID = blockID s.LastBlockID = blockID
s.LastBlockTime = blockTime s.LastBlockTime = blockTime
s.Validators = nextValSet
s.LastValidators = prevValSet
s.LastConsensusHash = s.Params.Hash()
s.Params = nextParams
s.LastValidators = s.Validators.Copy()
s.Validators = valSet
s.LastConsensusParams = s.ConsensusParams
s.ConsensusParams = params
} }
// GetValidators returns the last and current validator sets. // GetValidators returns the last and current validator sets.
@ -424,15 +445,19 @@ func MakeGenesisState(db dbm.DB, genDoc *types.GenesisDoc) (*State, error) {
db: db, db: db,
ChainID: genDoc.ChainID, ChainID: genDoc.ChainID,
Params: *genDoc.ConsensusParams,
LastBlockHeight: 0,
LastBlockID: types.BlockID{},
LastBlockTime: genDoc.GenesisTime,
LastBlockHeight: 0,
LastBlockID: types.BlockID{},
LastBlockTime: genDoc.GenesisTime,
Validators: types.NewValidatorSet(validators), Validators: types.NewValidatorSet(validators),
LastValidators: types.NewValidatorSet(nil), LastValidators: types.NewValidatorSet(nil),
AppHash: genDoc.AppHash,
LastConsensusHash: genDoc.ConsensusParams.Hash(),
LastHeightValidatorsChanged: 1, LastHeightValidatorsChanged: 1,
ConsensusParams: *genDoc.ConsensusParams,
LastConsensusParams: types.ConsensusParams{},
LastHeightConsensusParamsChanged: 1,
AppHash: genDoc.AppHash,
}, nil }, nil
} }

+ 2
- 2
state/state_test.go View File

@ -76,7 +76,7 @@ func TestABCIResponsesSaveLoad(t *testing.T) {
state.LastBlockHeight++ state.LastBlockHeight++
// build mock responses // build mock responses
block := makeBlock(2, state)
block := makeBlock(state, 2)
abciResponses := NewABCIResponses(block) abciResponses := NewABCIResponses(block)
abciResponses.DeliverTx[0] = &abci.ResponseDeliverTx{Data: []byte("foo"), Tags: []*abci.KVPair{}} abciResponses.DeliverTx[0] = &abci.ResponseDeliverTx{Data: []byte("foo"), Tags: []*abci.KVPair{}}
abciResponses.DeliverTx[1] = &abci.ResponseDeliverTx{Data: []byte("bar"), Log: "ok", Tags: []*abci.KVPair{}} abciResponses.DeliverTx[1] = &abci.ResponseDeliverTx{Data: []byte("bar"), Log: "ok", Tags: []*abci.KVPair{}}
@ -260,7 +260,7 @@ func TestApplyUpdates(t *testing.T) {
func makeHeaderPartsResponses(state *State, height int64, func makeHeaderPartsResponses(state *State, height int64,
pubkey crypto.PubKey) (*types.Header, types.PartSetHeader, *ABCIResponses) { pubkey crypto.PubKey) (*types.Header, types.PartSetHeader, *ABCIResponses) {
block := makeBlock(height, state)
block := makeBlock(state, height)
_, val := state.Validators.GetByIndex(0) _, val := state.Validators.GetByIndex(0)
abciResponses := &ABCIResponses{ abciResponses := &ABCIResponses{
Height: height, Height: height,


+ 27
- 57
types/block.go View File

@ -15,31 +15,21 @@ import (
) )
// Block defines the atomic unit of a Tendermint blockchain. // Block defines the atomic unit of a Tendermint blockchain.
// TODO: add Version byte
type Block struct { type Block struct {
*Header `json:"header"` *Header `json:"header"`
*Data `json:"data"` *Data `json:"data"`
LastCommit *Commit `json:"last_commit"` LastCommit *Commit `json:"last_commit"`
} }
// MakeBlock returns a new block and corresponding partset from the given information.
// TODO: Add version information to the Block struct.
func MakeBlock(height int64, chainID string, txs []Tx,
totalTxs int64, commit *Commit,
prevBlockID BlockID, valHash, appHash, consensusHash []byte,
partSize int) (*Block, *PartSet) {
newTxs := int64(len(txs))
// MakeBlock returns a new block with an empty header, except what can be computed from itself.
// It populates the same set of fields validated by ValidateBasic
func MakeBlock(height int64, txs []Tx, commit *Commit) *Block {
block := &Block{ block := &Block{
Header: &Header{ Header: &Header{
ChainID: chainID,
Height: height,
Time: time.Now(),
NumTxs: newTxs,
TotalTxs: totalTxs + newTxs,
LastBlockID: prevBlockID,
ValidatorsHash: valHash,
AppHash: appHash, // state merkle root of txs from the previous block.
ConsensusHash: consensusHash,
Height: height,
Time: time.Now(),
NumTxs: int64(len(txs)),
}, },
LastCommit: commit, LastCommit: commit,
Data: &Data{ Data: &Data{
@ -47,37 +37,16 @@ func MakeBlock(height int64, chainID string, txs []Tx,
}, },
} }
block.FillHeader() block.FillHeader()
return block, block.MakePartSet(partSize)
return block
} }
// ValidateBasic performs basic validation that doesn't involve state data. // ValidateBasic performs basic validation that doesn't involve state data.
func (b *Block) ValidateBasic(chainID string, lastBlockHeight int64,
lastBlockTotalTx int64, lastBlockID BlockID,
lastBlockTime time.Time, appHash, consensusHash []byte) error {
if b.ChainID != chainID {
return fmt.Errorf("Wrong Block.Header.ChainID. Expected %v, got %v", chainID, b.ChainID)
}
if b.Height != lastBlockHeight+1 {
return fmt.Errorf("Wrong Block.Header.Height. Expected %v, got %v", lastBlockHeight+1, b.Height)
}
/* TODO: Determine bounds for Time
See blockchain/reactor "stopSyncingDurationMinutes"
if !b.Time.After(lastBlockTime) {
return errors.New("Invalid Block.Header.Time")
}
*/
// It checks the internal consistency of the block.
func (b *Block) ValidateBasic() error {
newTxs := int64(len(b.Data.Txs)) newTxs := int64(len(b.Data.Txs))
if b.NumTxs != newTxs { if b.NumTxs != newTxs {
return fmt.Errorf("Wrong Block.Header.NumTxs. Expected %v, got %v", newTxs, b.NumTxs) return fmt.Errorf("Wrong Block.Header.NumTxs. Expected %v, got %v", newTxs, b.NumTxs)
} }
if b.TotalTxs != lastBlockTotalTx+newTxs {
return fmt.Errorf("Wrong Block.Header.TotalTxs. Expected %v, got %v", lastBlockTotalTx+newTxs, b.TotalTxs)
}
if !b.LastBlockID.Equals(lastBlockID) {
return fmt.Errorf("Wrong Block.Header.LastBlockID. Expected %v, got %v", lastBlockID, b.LastBlockID)
}
if !bytes.Equal(b.LastCommitHash, b.LastCommit.Hash()) { if !bytes.Equal(b.LastCommitHash, b.LastCommit.Hash()) {
return fmt.Errorf("Wrong Block.Header.LastCommitHash. Expected %v, got %v", b.LastCommitHash, b.LastCommit.Hash()) return fmt.Errorf("Wrong Block.Header.LastCommitHash. Expected %v, got %v", b.LastCommitHash, b.LastCommit.Hash())
} }
@ -89,13 +58,6 @@ func (b *Block) ValidateBasic(chainID string, lastBlockHeight int64,
if !bytes.Equal(b.DataHash, b.Data.Hash()) { if !bytes.Equal(b.DataHash, b.Data.Hash()) {
return fmt.Errorf("Wrong Block.Header.DataHash. Expected %v, got %v", b.DataHash, b.Data.Hash()) return fmt.Errorf("Wrong Block.Header.DataHash. Expected %v, got %v", b.DataHash, b.Data.Hash())
} }
if !bytes.Equal(b.AppHash, appHash) {
return fmt.Errorf("Wrong Block.Header.AppHash. Expected %X, got %v", appHash, b.AppHash)
}
if !bytes.Equal(b.ConsensusHash, consensusHash) {
return fmt.Errorf("Wrong Block.Header.ConsensusHash. Expected %X, got %v", consensusHash, b.ConsensusHash)
}
// NOTE: the AppHash and ValidatorsHash are validated later.
return nil return nil
} }
@ -171,18 +133,26 @@ func (b *Block) StringShort() string {
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------
// Header defines the structure of a Tendermint block header // Header defines the structure of a Tendermint block header
// TODO: limit header size
type Header struct { type Header struct {
ChainID string `json:"chain_id"`
Height int64 `json:"height"`
Time time.Time `json:"time"`
NumTxs int64 `json:"num_txs"` // XXX: Can we get rid of this?
TotalTxs int64 `json:"total_txs"`
LastBlockID BlockID `json:"last_block_id"`
// basic block info
ChainID string `json:"chain_id"`
Height int64 `json:"height"`
Time time.Time `json:"time"`
NumTxs int64 `json:"num_txs"`
// prev block info
LastBlockID BlockID `json:"last_block_id"`
TotalTxs int64 `json:"total_txs"`
// hashes of block data
LastCommitHash data.Bytes `json:"last_commit_hash"` // commit from validators from the last block LastCommitHash data.Bytes `json:"last_commit_hash"` // commit from validators from the last block
DataHash data.Bytes `json:"data_hash"` // transactions DataHash data.Bytes `json:"data_hash"` // transactions
ValidatorsHash data.Bytes `json:"validators_hash"` // validators for the current block
AppHash data.Bytes `json:"app_hash"` // state after txs from the previous block
ConsensusHash data.Bytes `json:"consensus_hash"` // consensus params for current block
// hashes from the app
ValidatorsHash data.Bytes `json:"validators_hash"` // validators for the current block
ConsensusHash data.Bytes `json:"consensus_hash"` // consensus params for current block
AppHash data.Bytes `json:"app_hash"` // state after txs from the previous block
} }
// Hash returns the hash of the header. // Hash returns the hash of the header.


+ 26
- 48
types/block_test.go View File

@ -2,55 +2,59 @@ package types
import ( import (
"testing" "testing"
"time"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
crypto "github.com/tendermint/go-crypto" crypto "github.com/tendermint/go-crypto"
cmn "github.com/tendermint/tmlibs/common"
) )
func TestValidateBlock(t *testing.T) { func TestValidateBlock(t *testing.T) {
txs := []Tx{Tx("foo"), Tx("bar")} txs := []Tx{Tx("foo"), Tx("bar")}
lastID := makeBlockID() lastID := makeBlockID()
valHash := []byte("val")
appHash := []byte("app")
consensusHash := []byte("consensus-params")
h := int64(3) h := int64(3)
voteSet, _, vals := randVoteSet(h-1, 1, VoteTypePrecommit, voteSet, _, vals := randVoteSet(h-1, 1, VoteTypePrecommit,
10, 1) 10, 1)
commit, err := makeCommit(lastID, h-1, 1, voteSet, vals)
commit, err := MakeCommit(lastID, h-1, 1, voteSet, vals)
require.NoError(t, err) require.NoError(t, err)
block, _ := MakeBlock(h, "hello", txs, 10, commit,
lastID, valHash, appHash, consensusHash, 2)
block := MakeBlock(h, txs, commit)
require.NotNil(t, block) require.NotNil(t, block)
// proper block must pass // proper block must pass
err = block.ValidateBasic("hello", h-1, 10, lastID, block.Time, appHash, consensusHash)
err = block.ValidateBasic()
require.NoError(t, err) require.NoError(t, err)
// wrong chain fails
err = block.ValidateBasic("other", h-1, 10, lastID, block.Time, appHash, consensusHash)
// tamper with NumTxs
block = MakeBlock(h, txs, commit)
block.NumTxs += 1
err = block.ValidateBasic()
require.Error(t, err) require.Error(t, err)
// wrong height fails
err = block.ValidateBasic("hello", h+4, 10, lastID, block.Time, appHash, consensusHash)
// remove 1/2 the commits
block = MakeBlock(h, txs, commit)
block.LastCommit.Precommits = commit.Precommits[:commit.Size()/2]
block.LastCommit.hash = nil // clear hash or change wont be noticed
err = block.ValidateBasic()
require.Error(t, err) require.Error(t, err)
// wrong total tx fails
err = block.ValidateBasic("hello", h-1, 15, lastID, block.Time, appHash, consensusHash)
// tamper with LastCommitHash
block = MakeBlock(h, txs, commit)
block.LastCommitHash = []byte("something else")
err = block.ValidateBasic()
require.Error(t, err) require.Error(t, err)
// wrong blockid fails
err = block.ValidateBasic("hello", h-1, 10, makeBlockID(), block.Time, appHash, consensusHash)
// tamper with data
block = MakeBlock(h, txs, commit)
block.Data.Txs[0] = Tx("something else")
block.Data.hash = nil // clear hash or change wont be noticed
err = block.ValidateBasic()
require.Error(t, err) require.Error(t, err)
// wrong app hash fails
err = block.ValidateBasic("hello", h-1, 10, lastID, block.Time, []byte("bad-hash"), consensusHash)
require.Error(t, err)
// wrong consensus hash fails
err = block.ValidateBasic("hello", h-1, 10, lastID, block.Time, appHash, []byte("wrong-params"))
// tamper with DataHash
block = MakeBlock(h, txs, commit)
block.DataHash = cmn.RandBytes(len(block.DataHash))
err = block.ValidateBasic()
require.Error(t, err) require.Error(t, err)
} }
@ -58,29 +62,3 @@ func makeBlockID() BlockID {
blockHash, blockPartsHeader := crypto.CRandBytes(32), PartSetHeader{123, crypto.CRandBytes(32)} blockHash, blockPartsHeader := crypto.CRandBytes(32), PartSetHeader{123, crypto.CRandBytes(32)}
return BlockID{blockHash, blockPartsHeader} return BlockID{blockHash, blockPartsHeader}
} }
func makeCommit(blockID BlockID, height int64, round int,
voteSet *VoteSet,
validators []*PrivValidatorFS) (*Commit, error) {
voteProto := &Vote{
ValidatorAddress: nil,
ValidatorIndex: -1,
Height: height,
Round: round,
Type: VoteTypePrecommit,
BlockID: blockID,
Timestamp: time.Now().UTC(),
}
// all sign
for i := 0; i < len(validators); i++ {
vote := withValidator(voteProto, validators[i].GetAddress(), i)
_, err := signAddVote(validators[i], vote, voteSet)
if err != nil {
return nil, err
}
}
return voteSet.MakeCommit(), nil
}

+ 0
- 10
types/vote_set_test.go View File

@ -59,16 +59,6 @@ func withBlockPartsHeader(vote *Vote, blockPartsHeader PartSetHeader) *Vote {
return vote return vote
} }
func signAddVote(privVal *PrivValidatorFS, vote *Vote, voteSet *VoteSet) (bool, error) {
var err error
vote.Signature, err = privVal.Signer.Sign(SignBytes(voteSet.ChainID(), vote))
if err != nil {
return false, err
}
added, err := voteSet.AddVote(vote)
return added, err
}
func TestAddVote(t *testing.T) { func TestAddVote(t *testing.T) {
height, round := int64(1), 0 height, round := int64(1), 0
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 10, 1) voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 10, 1)


Loading…
Cancel
Save