Browse Source

Updates -> ValidatoSetUpdates

pull/972/head
Anton Kaliaev 7 years ago
parent
commit
843e1ed400
No known key found for this signature in database GPG Key ID: 7B6881D965918214
14 changed files with 107 additions and 107 deletions
  1. +1
    -1
      blockchain/reactor.go
  2. +2
    -2
      blockchain/reactor_test.go
  3. +1
    -1
      consensus/state.go
  4. +1
    -1
      consensus/wal.go
  5. +2
    -3
      consensus/wal_generator.go
  6. +10
    -9
      docs/architecture/adr-005-consensus-params.md
  7. +3
    -3
      glide.lock
  8. +3
    -3
      state/execution.go
  9. +21
    -21
      state/state.go
  10. +14
    -14
      state/state_test.go
  11. +9
    -9
      types/block.go
  12. +1
    -1
      types/genesis_test.go
  13. +34
    -34
      types/params.go
  14. +5
    -5
      types/params_test.go

+ 1
- 1
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.BlockSizeParams.MaxBytes + 2
return bcR.state.Params.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.


+ 2
- 2
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.BlockGossipParams.BlockPartSizeBytes)
firstParts := firstBlock.MakePartSet(state.Params.BlockGossip.BlockPartSizeBytes)
blockStore.SaveBlock(firstBlock, firstParts, secondBlock.LastCommit) blockStore.SaveBlock(firstBlock, firstParts, secondBlock.LastCommit)
} }
@ -113,7 +113,7 @@ func makeBlock(height int64, state *sm.State) *types.Block {
state.LastBlockTotalTx, new(types.Commit), state.LastBlockTotalTx, new(types.Commit),
prevBlockID, valHash, state.AppHash, prevBlockID, valHash, state.AppHash,
state.LastConsensusHash, state.LastConsensusHash,
state.Params.BlockGossipParams.BlockPartSizeBytes)
state.Params.BlockGossip.BlockPartSizeBytes)
return block return block
} }


+ 1
- 1
consensus/state.go View File

@ -1307,7 +1307,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.BlockSizeParams.MaxBytes, &n, &err).(*types.Block)
cs.state.Params.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() {


+ 1
- 1
consensus/wal.go View File

@ -18,7 +18,7 @@ import (
) )
const ( const (
// must be greater than params.BlockGossipParams.BlockPartSizeBytes + a few bytes
// must be greater than params.BlockGossip.BlockPartSizeBytes + a few bytes
maxMsgSizeBytes = 1024 * 1024 // 1MB maxMsgSizeBytes = 1024 * 1024 // 1MB
) )


+ 2
- 3
consensus/wal_generator.go View File

@ -35,7 +35,6 @@ func WALWithNBlocks(numBlocks int) (data []byte, err error) {
logger := log.TestingLogger().With("wal_generator", "wal_generator") logger := log.TestingLogger().With("wal_generator", "wal_generator")
logger.Info("generating WAL (last height msg excluded)", "numBlocks", numBlocks) logger.Info("generating WAL (last height msg excluded)", "numBlocks", numBlocks)
///////////////////////////////////////////////////////////////////////////// /////////////////////////////////////////////////////////////////////////////
// COPY PASTE FROM node.go WITH A FEW MODIFICATIONS // COPY PASTE FROM node.go WITH A FEW MODIFICATIONS
// NOTE: we can't import node package because of circular dependency // NOTE: we can't import node package because of circular dependency
@ -95,7 +94,7 @@ func WALWithNBlocks(numBlocks int) (data []byte, err error) {
wr.Flush() wr.Flush()
return b.Bytes(), nil return b.Bytes(), nil
case <-time.After(1 * time.Minute): case <-time.After(1 * time.Minute):
wr.Flush()
wr.Flush()
return b.Bytes(), fmt.Errorf("waited too long for tendermint to produce %d blocks (grep logs for `wal_generator`)", numBlocks) return b.Bytes(), fmt.Errorf("waited too long for tendermint to produce %d blocks (grep logs for `wal_generator`)", numBlocks)
} }
} }
@ -147,7 +146,7 @@ type byteBufferWAL struct {
heightToStop int64 heightToStop int64
signalWhenStopsTo chan<- struct{} signalWhenStopsTo chan<- struct{}
logger log.Logger
logger log.Logger
} }
// needed for determinism // needed for determinism


+ 10
- 9
docs/architecture/adr-005-consensus-params.md View File

@ -22,30 +22,30 @@ The parameters are used to determine the validity of a block (and tx) via the un
``` ```
type ConsensusParams struct { type ConsensusParams struct {
BlockSizeParams
TxSizeParams
BlockGossipParams
BlockSize
TxSize
BlockGossip
} }
type BlockSizeParams struct {
type BlockSize struct {
MaxBytes int MaxBytes int
MaxTxs int MaxTxs int
MaxGas int MaxGas int
} }
type TxSizeParams struct {
type TxSize struct {
MaxBytes int MaxBytes int
MaxGas int MaxGas int
} }
type BlockGossipParams struct {
type BlockGossip struct {
BlockPartSizeBytes int BlockPartSizeBytes int
} }
``` ```
The `ConsensusParams` can evolve over time by adding new structs that cover different aspects of the consensus rules. The `ConsensusParams` can evolve over time by adding new structs that cover different aspects of the consensus rules.
The `BlockPartSizeBytes` and the `BlockSizeParams.MaxBytes` are enforced to be greater than 0.
The `BlockPartSizeBytes` and the `BlockSize.MaxBytes` are enforced to be greater than 0.
The former because we need a part size, the latter so that we always have at least some sanity check over the size of blocks. The former because we need a part size, the latter so that we always have at least some sanity check over the size of blocks.
### ABCI ### ABCI
@ -58,7 +58,7 @@ like the BlockPartSize, that the app shouldn't really know about.
#### EndBlock #### EndBlock
The EndBlock response includes a `ConsensusParams`, which includes BlockSizeParams and TxSizeParams, but not BlockGossipParams.
The EndBlock response includes a `ConsensusParams`, which includes BlockSize and TxSize, but not BlockGossip.
Other param struct can be added to `ConsensusParams` in the future. Other param struct can be added to `ConsensusParams` in the future.
The `0` value is used to denote no change. The `0` value is used to denote no change.
Any other value will update that parameter in the `State.ConsensusParams`, to be applied for the next block. Any other value will update that parameter in the `State.ConsensusParams`, to be applied for the next block.
@ -82,4 +82,5 @@ Proposed.
### Neutral ### Neutral
- The TxSizeParams, which checks validity, may be in conflict with the config's `max_block_size_tx`, which determines proposal sizes
- The TxSize, which checks validity, may be in conflict with the config's `max_block_size_tx`, which determines proposal sizes

+ 3
- 3
glide.lock View File

@ -1,5 +1,5 @@
hash: ff6e6786ec24ffac91df45d4a1cdcefae5280700667ca76b8b9b96f343d78c95 hash: ff6e6786ec24ffac91df45d4a1cdcefae5280700667ca76b8b9b96f343d78c95
updated: 2017-12-13T18:04:10.05914801+01:00
updated: 2017-12-19T18:45:37.28268356Z
imports: imports:
- name: github.com/btcsuite/btcd - name: github.com/btcsuite/btcd
version: 2e60448ffcc6bf78332d1fe590260095f554dd78 version: 2e60448ffcc6bf78332d1fe590260095f554dd78
@ -103,7 +103,7 @@ imports:
- leveldb/table - leveldb/table
- leveldb/util - leveldb/util
- name: github.com/tendermint/abci - name: github.com/tendermint/abci
version: 895e14d6bd9cdad98eab4a051decbaad46f7eebd
version: 66296fe11aaa518381c6cd7160bc6318e28c4e02
subpackages: subpackages:
- client - client
- example/code - example/code
@ -129,7 +129,7 @@ imports:
subpackages: subpackages:
- iavl - iavl
- name: github.com/tendermint/tmlibs - name: github.com/tendermint/tmlibs
version: a483e1ff486b577ba94e6a20f08bf52fbb7bff14
version: e4ef2835f0081c2ece83b9c1f777cf071f956e81
subpackages: subpackages:
- autofile - autofile
- cli - cli


+ 3
- 3
state/execution.go View File

@ -111,11 +111,11 @@ func execBlockOnProxyApp(txEventPublisher types.TxEventPublisher, proxyAppConn p
return nil, err return nil, err
} }
valChanges := abciResponses.EndBlock.Changes
valSetUpdates := abciResponses.EndBlock.ValidatorSetUpdates
logger.Info("Executed block", "height", block.Height, "validTxs", validTxs, "invalidTxs", invalidTxs) logger.Info("Executed block", "height", block.Height, "validTxs", validTxs, "invalidTxs", invalidTxs)
if len(valChanges) > 0 {
logger.Info("Update to validator set", "updates", abci.ValidatorsString(valChanges))
if len(valSetUpdates) > 0 {
logger.Info("Updates to validator set", "updates", abci.ValidatorsString(valSetUpdates))
} }
return abciResponses, nil return abciResponses, nil


+ 21
- 21
state/state.go View File

@ -241,8 +241,8 @@ func (s *State) SetBlockAndValidators(header *types.Header, blockPartsHeader typ
nextValSet := prevValSet.Copy() nextValSet := prevValSet.Copy()
// update the validator set with the latest abciResponses // update the validator set with the latest abciResponses
if len(abciResponses.EndBlock.Changes) > 0 {
err := updateValidators(nextValSet, abciResponses.EndBlock.Changes)
if len(abciResponses.EndBlock.ValidatorSetUpdates) > 0 {
err := updateValidators(nextValSet, abciResponses.EndBlock.ValidatorSetUpdates)
if err != nil { if err != nil {
s.logger.Error("Error changing validator set", "err", err) s.logger.Error("Error changing validator set", "err", err)
// TODO: err or carry on? // TODO: err or carry on?
@ -254,8 +254,8 @@ 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 := applyChanges(s.Params,
abciResponses.EndBlock.ConsensusParamChanges)
nextParams := applyUpdates(s.Params,
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)
@ -272,10 +272,10 @@ func (s *State) SetBlockAndValidators(header *types.Header, blockPartsHeader typ
} }
// applyChanges returns new ConsensusParams
// applyUpdates returns new ConsensusParams
// whose fields are set to any non-zero fields of c. // whose fields are set to any non-zero fields of c.
// If c is nil, it returns p unmodified, as it was passed in. // If c is nil, it returns p unmodified, as it was passed in.
func applyChanges(p types.ConsensusParams,
func applyUpdates(p types.ConsensusParams,
c *abci.ConsensusParams) types.ConsensusParams { c *abci.ConsensusParams) types.ConsensusParams {
if c == nil { if c == nil {
@ -283,29 +283,29 @@ func applyChanges(p types.ConsensusParams,
} }
res := p res := p
// we must defensively consider any structs may be nil // we must defensively consider any structs may be nil
if c.BlockSizeParams != nil {
if c.BlockSize != nil {
if c.BlockSizeParams.MaxBytes != 0 {
res.BlockSizeParams.MaxBytes = int(c.BlockSizeParams.MaxBytes)
if c.BlockSize.MaxBytes != 0 {
res.BlockSize.MaxBytes = int(c.BlockSize.MaxBytes)
} }
if c.BlockSizeParams.MaxTxs != 0 {
res.BlockSizeParams.MaxTxs = int(c.BlockSizeParams.MaxTxs)
if c.BlockSize.MaxTxs != 0 {
res.BlockSize.MaxTxs = int(c.BlockSize.MaxTxs)
} }
if c.BlockSizeParams.MaxGas != 0 {
res.BlockSizeParams.MaxGas = int(c.BlockSizeParams.MaxGas)
if c.BlockSize.MaxGas != 0 {
res.BlockSize.MaxGas = int(c.BlockSize.MaxGas)
} }
} }
if c.TxSizeParams != nil {
if c.TxSizeParams.MaxBytes != 0 {
res.TxSizeParams.MaxBytes = int(c.TxSizeParams.MaxBytes)
if c.TxSize != nil {
if c.TxSize.MaxBytes != 0 {
res.TxSize.MaxBytes = int(c.TxSize.MaxBytes)
} }
if c.TxSizeParams.MaxGas != 0 {
res.TxSizeParams.MaxGas = int(c.TxSizeParams.MaxGas)
if c.TxSize.MaxGas != 0 {
res.TxSize.MaxGas = int(c.TxSize.MaxGas)
} }
} }
if c.BlockGossipParams != nil {
if c.BlockGossipParams.BlockPartSizeBytes != 0 {
res.BlockGossipParams.BlockPartSizeBytes = int(c.BlockGossipParams.BlockPartSizeBytes)
if c.BlockGossip != nil {
if c.BlockGossip.BlockPartSizeBytes != 0 {
res.BlockGossip.BlockPartSizeBytes = int(c.BlockGossip.BlockPartSizeBytes)
} }
} }
return res return res


+ 14
- 14
state/state_test.go View File

@ -80,7 +80,7 @@ func TestABCIResponsesSaveLoad(t *testing.T) {
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{}}
abciResponses.EndBlock = &abci.ResponseEndBlock{Changes: []*abci.Validator{
abciResponses.EndBlock = &abci.ResponseEndBlock{ValidatorSetUpdates: []*abci.Validator{
{ {
PubKey: crypto.GenPrivKeyEd25519().PubKey().Bytes(), PubKey: crypto.GenPrivKeyEd25519().PubKey().Bytes(),
Power: 10, Power: 10,
@ -196,41 +196,41 @@ func makeParams(blockBytes, blockTx, blockGas, txBytes,
txGas, partSize int) types.ConsensusParams { txGas, partSize int) types.ConsensusParams {
return types.ConsensusParams{ return types.ConsensusParams{
BlockSizeParams: types.BlockSizeParams{
BlockSize: types.BlockSize{
MaxBytes: blockBytes, MaxBytes: blockBytes,
MaxTxs: blockTx, MaxTxs: blockTx,
MaxGas: blockGas, MaxGas: blockGas,
}, },
TxSizeParams: types.TxSizeParams{
TxSize: types.TxSize{
MaxBytes: txBytes, MaxBytes: txBytes,
MaxGas: txGas, MaxGas: txGas,
}, },
BlockGossipParams: types.BlockGossipParams{
BlockGossip: types.BlockGossip{
BlockPartSizeBytes: partSize, BlockPartSizeBytes: partSize,
}, },
} }
} }
func TestApplyChanges(t *testing.T) {
func TestApplyUpdates(t *testing.T) {
initParams := makeParams(1, 2, 3, 4, 5, 6) initParams := makeParams(1, 2, 3, 4, 5, 6)
cases := [...]struct { cases := [...]struct {
init types.ConsensusParams init types.ConsensusParams
changes *abci.ConsensusParams
updates *abci.ConsensusParams
expected types.ConsensusParams expected types.ConsensusParams
}{ }{
0: {initParams, nil, initParams}, 0: {initParams, nil, initParams},
1: {initParams, &abci.ConsensusParams{}, initParams}, 1: {initParams, &abci.ConsensusParams{}, initParams},
2: {initParams, 2: {initParams,
&abci.ConsensusParams{ &abci.ConsensusParams{
TxSizeParams: &abci.TxSizeParams{
TxSize: &abci.TxSize{
MaxBytes: 123, MaxBytes: 123,
}, },
}, },
makeParams(1, 2, 3, 123, 5, 6)}, makeParams(1, 2, 3, 123, 5, 6)},
3: {initParams, 3: {initParams,
&abci.ConsensusParams{ &abci.ConsensusParams{
BlockSizeParams: &abci.BlockSizeParams{
BlockSize: &abci.BlockSize{
MaxTxs: 44, MaxTxs: 44,
MaxGas: 55, MaxGas: 55,
}, },
@ -238,13 +238,13 @@ func TestApplyChanges(t *testing.T) {
makeParams(1, 44, 55, 4, 5, 6)}, makeParams(1, 44, 55, 4, 5, 6)},
4: {initParams, 4: {initParams,
&abci.ConsensusParams{ &abci.ConsensusParams{
BlockSizeParams: &abci.BlockSizeParams{
BlockSize: &abci.BlockSize{
MaxTxs: 789, MaxTxs: 789,
}, },
TxSizeParams: &abci.TxSizeParams{
TxSize: &abci.TxSize{
MaxGas: 888, MaxGas: 888,
}, },
BlockGossipParams: &abci.BlockGossipParams{
BlockGossip: &abci.BlockGossip{
BlockPartSizeBytes: 2002, BlockPartSizeBytes: 2002,
}, },
}, },
@ -252,7 +252,7 @@ func TestApplyChanges(t *testing.T) {
} }
for i, tc := range cases { for i, tc := range cases {
res := applyChanges(tc.init, tc.changes)
res := applyUpdates(tc.init, tc.updates)
assert.Equal(t, tc.expected, res, "case %d", i) assert.Equal(t, tc.expected, res, "case %d", i)
} }
} }
@ -264,13 +264,13 @@ func makeHeaderPartsResponses(state *State, height int64,
_, val := state.Validators.GetByIndex(0) _, val := state.Validators.GetByIndex(0)
abciResponses := &ABCIResponses{ abciResponses := &ABCIResponses{
Height: height, Height: height,
EndBlock: &abci.ResponseEndBlock{Changes: []*abci.Validator{}},
EndBlock: &abci.ResponseEndBlock{ValidatorSetUpdates: []*abci.Validator{}},
} }
// if the pubkey is new, remove the old and add the new // if the pubkey is new, remove the old and add the new
if !bytes.Equal(pubkey.Bytes(), val.PubKey.Bytes()) { if !bytes.Equal(pubkey.Bytes(), val.PubKey.Bytes()) {
abciResponses.EndBlock = &abci.ResponseEndBlock{ abciResponses.EndBlock = &abci.ResponseEndBlock{
Changes: []*abci.Validator{
ValidatorSetUpdates: []*abci.Validator{
{val.PubKey.Bytes(), 0}, {val.PubKey.Bytes(), 0},
{pubkey.Bytes(), 10}, {pubkey.Bytes(), 10},
}, },


+ 9
- 9
types/block.go View File

@ -56,10 +56,10 @@ func (b *Block) ValidateBasic(chainID string, lastBlockHeight int64,
lastBlockTime time.Time, appHash, consensusHash []byte) error { lastBlockTime time.Time, appHash, consensusHash []byte) error {
if b.ChainID != chainID { if b.ChainID != chainID {
return errors.New(cmn.Fmt("Wrong Block.Header.ChainID. Expected %v, got %v", chainID, b.ChainID))
return fmt.Errorf("Wrong Block.Header.ChainID. Expected %v, got %v", chainID, b.ChainID)
} }
if b.Height != lastBlockHeight+1 { if b.Height != lastBlockHeight+1 {
return errors.New(cmn.Fmt("Wrong Block.Header.Height. Expected %v, got %v", lastBlockHeight+1, b.Height))
return fmt.Errorf("Wrong Block.Header.Height. Expected %v, got %v", lastBlockHeight+1, b.Height)
} }
/* TODO: Determine bounds for Time /* TODO: Determine bounds for Time
See blockchain/reactor "stopSyncingDurationMinutes" See blockchain/reactor "stopSyncingDurationMinutes"
@ -70,16 +70,16 @@ func (b *Block) ValidateBasic(chainID string, lastBlockHeight int64,
*/ */
newTxs := int64(len(b.Data.Txs)) newTxs := int64(len(b.Data.Txs))
if b.NumTxs != newTxs { if b.NumTxs != newTxs {
return errors.New(cmn.Fmt("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 { if b.TotalTxs != lastBlockTotalTx+newTxs {
return errors.New(cmn.Fmt("Wrong Block.Header.TotalTxs. Expected %v, got %v", lastBlockTotalTx+newTxs, b.TotalTxs))
return fmt.Errorf("Wrong Block.Header.TotalTxs. Expected %v, got %v", lastBlockTotalTx+newTxs, b.TotalTxs)
} }
if !b.LastBlockID.Equals(lastBlockID) { if !b.LastBlockID.Equals(lastBlockID) {
return errors.New(cmn.Fmt("Wrong Block.Header.LastBlockID. Expected %v, got %v", lastBlockID, b.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 errors.New(cmn.Fmt("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())
} }
if b.Header.Height != 1 { if b.Header.Height != 1 {
if err := b.LastCommit.ValidateBasic(); err != nil { if err := b.LastCommit.ValidateBasic(); err != nil {
@ -87,13 +87,13 @@ 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 errors.New(cmn.Fmt("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) { if !bytes.Equal(b.AppHash, appHash) {
return errors.New(cmn.Fmt("Wrong Block.Header.AppHash. Expected %X, got %v", appHash, b.AppHash))
return fmt.Errorf("Wrong Block.Header.AppHash. Expected %X, got %v", appHash, b.AppHash)
} }
if !bytes.Equal(b.ConsensusHash, consensusHash) { if !bytes.Equal(b.ConsensusHash, consensusHash) {
return errors.New(cmn.Fmt("Wrong Block.Header.ConsensusHash. Expected %X, got %v", consensusHash, b.ConsensusHash))
return fmt.Errorf("Wrong Block.Header.ConsensusHash. Expected %X, got %v", consensusHash, b.ConsensusHash)
} }
// NOTE: the AppHash and ValidatorsHash are validated later. // NOTE: the AppHash and ValidatorsHash are validated later.
return nil return nil


+ 1
- 1
types/genesis_test.go View File

@ -54,7 +54,7 @@ func TestGenesis(t *testing.T) {
assert.NoError(t, err, "expected no error for valid genDoc json") assert.NoError(t, err, "expected no error for valid genDoc json")
// test with invalid consensus params // test with invalid consensus params
genDoc.ConsensusParams.BlockSizeParams.MaxBytes = 0
genDoc.ConsensusParams.BlockSize.MaxBytes = 0
genDocBytes, err = json.Marshal(genDoc) genDocBytes, err = json.Marshal(genDoc)
assert.NoError(t, err, "error marshalling genDoc") assert.NoError(t, err, "error marshalling genDoc")
genDoc, err = GenesisDocFromJSON(genDocBytes) genDoc, err = GenesisDocFromJSON(genDocBytes)


+ 34
- 34
types/params.go View File

@ -13,58 +13,58 @@ const (
// ConsensusParams contains consensus critical parameters // ConsensusParams contains consensus critical parameters
// that determine the validity of blocks. // that determine the validity of blocks.
type ConsensusParams struct { type ConsensusParams struct {
BlockSizeParams `json:"block_size_params"`
TxSizeParams `json:"tx_size_params"`
BlockGossipParams `json:"block_gossip_params"`
BlockSize `json:"block_size_params"`
TxSize `json:"tx_size_params"`
BlockGossip `json:"block_gossip_params"`
} }
// BlockSizeParams contain limits on the block size.
type BlockSizeParams struct {
// BlockSize contain limits on the block size.
type BlockSize struct {
MaxBytes int `json:"max_bytes"` // NOTE: must not be 0 nor greater than 100MB MaxBytes int `json:"max_bytes"` // NOTE: must not be 0 nor greater than 100MB
MaxTxs int `json:"max_txs"` MaxTxs int `json:"max_txs"`
MaxGas int `json:"max_gas"` MaxGas int `json:"max_gas"`
} }
// TxSizeParams contain limits on the tx size.
type TxSizeParams struct {
// TxSize contain limits on the tx size.
type TxSize struct {
MaxBytes int `json:"max_bytes"` MaxBytes int `json:"max_bytes"`
MaxGas int `json:"max_gas"` MaxGas int `json:"max_gas"`
} }
// BlockGossipParams determine consensus critical elements of how blocks are gossiped
type BlockGossipParams struct {
// BlockGossip determine consensus critical elements of how blocks are gossiped
type BlockGossip struct {
BlockPartSizeBytes int `json:"block_part_size_bytes"` // NOTE: must not be 0 BlockPartSizeBytes int `json:"block_part_size_bytes"` // NOTE: must not be 0
} }
// DefaultConsensusParams returns a default ConsensusParams. // DefaultConsensusParams returns a default ConsensusParams.
func DefaultConsensusParams() *ConsensusParams { func DefaultConsensusParams() *ConsensusParams {
return &ConsensusParams{ return &ConsensusParams{
DefaultBlockSizeParams(),
DefaultTxSizeParams(),
DefaultBlockGossipParams(),
DefaultBlockSize(),
DefaultTxSize(),
DefaultBlockGossip(),
} }
} }
// DefaultBlockSizeParams returns a default BlockSizeParams.
func DefaultBlockSizeParams() BlockSizeParams {
return BlockSizeParams{
// DefaultBlockSize returns a default BlockSize.
func DefaultBlockSize() BlockSize {
return BlockSize{
MaxBytes: 22020096, // 21MB MaxBytes: 22020096, // 21MB
MaxTxs: 100000, MaxTxs: 100000,
MaxGas: -1, MaxGas: -1,
} }
} }
// DefaultTxSizeParams returns a default TxSizeParams.
func DefaultTxSizeParams() TxSizeParams {
return TxSizeParams{
// DefaultTxSize returns a default TxSize.
func DefaultTxSize() TxSize {
return TxSize{
MaxBytes: 10240, // 10kB MaxBytes: 10240, // 10kB
MaxGas: -1, MaxGas: -1,
} }
} }
// DefaultBlockGossipParams returns a default BlockGossipParams.
func DefaultBlockGossipParams() BlockGossipParams {
return BlockGossipParams{
// DefaultBlockGossip returns a default BlockGossip.
func DefaultBlockGossip() BlockGossip {
return BlockGossip{
BlockPartSizeBytes: 65536, // 64kB, BlockPartSizeBytes: 65536, // 64kB,
} }
} }
@ -73,17 +73,17 @@ func DefaultBlockGossipParams() BlockGossipParams {
// are within their allowed limits, and returns an error if they are not. // are within their allowed limits, and returns an error if they are not.
func (params *ConsensusParams) Validate() error { func (params *ConsensusParams) Validate() error {
// ensure some values are greater than 0 // ensure some values are greater than 0
if params.BlockSizeParams.MaxBytes <= 0 {
return errors.Errorf("BlockSizeParams.MaxBytes must be greater than 0. Got %d", params.BlockSizeParams.MaxBytes)
if params.BlockSize.MaxBytes <= 0 {
return errors.Errorf("BlockSize.MaxBytes must be greater than 0. Got %d", params.BlockSize.MaxBytes)
} }
if params.BlockGossipParams.BlockPartSizeBytes <= 0 {
return errors.Errorf("BlockGossipParams.BlockPartSizeBytes must be greater than 0. Got %d", params.BlockGossipParams.BlockPartSizeBytes)
if params.BlockGossip.BlockPartSizeBytes <= 0 {
return errors.Errorf("BlockGossip.BlockPartSizeBytes must be greater than 0. Got %d", params.BlockGossip.BlockPartSizeBytes)
} }
// ensure blocks aren't too big // ensure blocks aren't too big
if params.BlockSizeParams.MaxBytes > maxBlockSizeBytes {
return errors.Errorf("BlockSizeParams.MaxBytes is too big. %d > %d",
params.BlockSizeParams.MaxBytes, maxBlockSizeBytes)
if params.BlockSize.MaxBytes > maxBlockSizeBytes {
return errors.Errorf("BlockSize.MaxBytes is too big. %d > %d",
params.BlockSize.MaxBytes, maxBlockSizeBytes)
} }
return nil return nil
} }
@ -92,11 +92,11 @@ func (params *ConsensusParams) Validate() error {
// in the block header // in the block header
func (params *ConsensusParams) Hash() []byte { func (params *ConsensusParams) Hash() []byte {
return merkle.SimpleHashFromMap(map[string]interface{}{ return merkle.SimpleHashFromMap(map[string]interface{}{
"block_gossip_part_size_bytes": params.BlockGossipParams.BlockPartSizeBytes,
"block_size_max_bytes": params.BlockSizeParams.MaxBytes,
"block_size_max_gas": params.BlockSizeParams.MaxGas,
"block_size_max_txs": params.BlockSizeParams.MaxTxs,
"tx_size_max_bytes": params.TxSizeParams.MaxBytes,
"tx_size_max_gas": params.TxSizeParams.MaxGas,
"block_gossip_part_size_bytes": params.BlockGossip.BlockPartSizeBytes,
"block_size_max_bytes": params.BlockSize.MaxBytes,
"block_size_max_gas": params.BlockSize.MaxGas,
"block_size_max_txs": params.BlockSize.MaxTxs,
"tx_size_max_bytes": params.TxSize.MaxBytes,
"tx_size_max_gas": params.TxSize.MaxGas,
}) })
} }

+ 5
- 5
types/params_test.go View File

@ -10,8 +10,8 @@ import (
func newConsensusParams(blockSize, partSize int) ConsensusParams { func newConsensusParams(blockSize, partSize int) ConsensusParams {
return ConsensusParams{ return ConsensusParams{
BlockSizeParams: BlockSizeParams{MaxBytes: blockSize},
BlockGossipParams: BlockGossipParams{BlockPartSizeBytes: partSize},
BlockSize: BlockSize{MaxBytes: blockSize},
BlockGossip: BlockGossip{BlockPartSizeBytes: partSize},
} }
} }
@ -45,16 +45,16 @@ func makeParams(blockBytes, blockTx, blockGas, txBytes,
txGas, partSize int) ConsensusParams { txGas, partSize int) ConsensusParams {
return ConsensusParams{ return ConsensusParams{
BlockSizeParams: BlockSizeParams{
BlockSize: BlockSize{
MaxBytes: blockBytes, MaxBytes: blockBytes,
MaxTxs: blockTx, MaxTxs: blockTx,
MaxGas: blockGas, MaxGas: blockGas,
}, },
TxSizeParams: TxSizeParams{
TxSize: TxSize{
MaxBytes: txBytes, MaxBytes: txBytes,
MaxGas: txGas, MaxGas: txGas,
}, },
BlockGossipParams: BlockGossipParams{
BlockGossip: BlockGossip{
BlockPartSizeBytes: partSize, BlockPartSizeBytes: partSize,
}, },
} }


Loading…
Cancel
Save