mirror of
https://github.com/fluencelabs/tendermint
synced 2025-05-30 14:41:20 +00:00
Merge pull request #650 from tendermint/consensus_params
Consensus params
This commit is contained in:
commit
8ff5b365dd
@ -352,7 +352,7 @@ func (peer *bpPeer) setLogger(l log.Logger) {
|
|||||||
|
|
||||||
func (peer *bpPeer) resetMonitor() {
|
func (peer *bpPeer) resetMonitor() {
|
||||||
peer.recvMonitor = flow.New(time.Second, time.Second*40)
|
peer.recvMonitor = flow.New(time.Second, time.Second*40)
|
||||||
var initialValue = float64(minRecvRate) * math.E
|
initialValue := float64(minRecvRate) * math.E
|
||||||
peer.recvMonitor.SetREMA(initialValue)
|
peer.recvMonitor.SetREMA(initialValue)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -28,7 +28,6 @@ const (
|
|||||||
statusUpdateIntervalSeconds = 10
|
statusUpdateIntervalSeconds = 10
|
||||||
// check if we should switch to consensus reactor
|
// check if we should switch to consensus reactor
|
||||||
switchToConsensusIntervalSeconds = 1
|
switchToConsensusIntervalSeconds = 1
|
||||||
maxBlockchainResponseSize = types.MaxBlockSize + 2
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type consensusReactor interface {
|
type consensusReactor interface {
|
||||||
@ -124,7 +123,7 @@ func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
|
|||||||
|
|
||||||
// Receive implements Reactor by handling 4 types of messages (look below).
|
// Receive implements Reactor by handling 4 types of messages (look below).
|
||||||
func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
|
func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
|
||||||
_, msg, err := DecodeMessage(msgBytes)
|
_, msg, err := DecodeMessage(msgBytes, bcR.maxMsgSize())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
bcR.Logger.Error("Error decoding message", "err", err)
|
bcR.Logger.Error("Error decoding message", "err", err)
|
||||||
return
|
return
|
||||||
@ -163,6 +162,12 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// maxMsgSize returns the maximum allowable size of a
|
||||||
|
// message on the blockchain reactor.
|
||||||
|
func (bcR *BlockchainReactor) maxMsgSize() int {
|
||||||
|
return bcR.state.Params().BlockSizeParams.MaxBytes + 2
|
||||||
|
}
|
||||||
|
|
||||||
// Handle messages from the poolReactor telling the reactor what to do.
|
// Handle messages from the poolReactor telling the reactor what to do.
|
||||||
// NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down!
|
// NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down!
|
||||||
// (Except for the SYNC_LOOP, which is the primary purpose and must be synchronous.)
|
// (Except for the SYNC_LOOP, which is the primary purpose and must be synchronous.)
|
||||||
@ -221,7 +226,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(types.DefaultBlockPartSize)
|
firstParts := first.MakePartSet(bcR.state.Params().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
|
||||||
@ -290,11 +295,11 @@ var _ = wire.RegisterInterface(
|
|||||||
|
|
||||||
// DecodeMessage decodes BlockchainMessage.
|
// DecodeMessage decodes BlockchainMessage.
|
||||||
// TODO: ensure that bz is completely read.
|
// TODO: ensure that bz is completely read.
|
||||||
func DecodeMessage(bz []byte) (msgType byte, msg BlockchainMessage, err error) {
|
func DecodeMessage(bz []byte, maxSize int) (msgType byte, msg BlockchainMessage, err error) {
|
||||||
msgType = bz[0]
|
msgType = bz[0]
|
||||||
n := int(0)
|
n := int(0)
|
||||||
r := bytes.NewReader(bz)
|
r := bytes.NewReader(bz)
|
||||||
msg = wire.ReadBinary(struct{ BlockchainMessage }{}, r, maxBlockchainResponseSize, &n, &err).(struct{ BlockchainMessage }).BlockchainMessage
|
msg = wire.ReadBinary(struct{ BlockchainMessage }{}, r, maxSize, &n, &err).(struct{ BlockchainMessage }).BlockchainMessage
|
||||||
if err != nil && n != len(bz) {
|
if err != nil && n != len(bz) {
|
||||||
err = errors.New("DecodeMessage() had bytes left over")
|
err = errors.New("DecodeMessage() had bytes left over")
|
||||||
}
|
}
|
||||||
|
@ -34,7 +34,7 @@ func initFiles(cmd *cobra.Command, args []string) {
|
|||||||
}
|
}
|
||||||
genDoc.Validators = []types.GenesisValidator{types.GenesisValidator{
|
genDoc.Validators = []types.GenesisValidator{types.GenesisValidator{
|
||||||
PubKey: privValidator.PubKey,
|
PubKey: privValidator.PubKey,
|
||||||
Amount: 10,
|
Power: 10,
|
||||||
}}
|
}}
|
||||||
|
|
||||||
genDoc.SaveAs(genFile)
|
genDoc.SaveAs(genFile)
|
||||||
|
@ -7,8 +7,8 @@ import (
|
|||||||
|
|
||||||
"github.com/spf13/cobra"
|
"github.com/spf13/cobra"
|
||||||
|
|
||||||
cmn "github.com/tendermint/tmlibs/common"
|
|
||||||
"github.com/tendermint/tendermint/types"
|
"github.com/tendermint/tendermint/types"
|
||||||
|
cmn "github.com/tendermint/tmlibs/common"
|
||||||
)
|
)
|
||||||
|
|
||||||
var testnetFilesCmd = &cobra.Command{
|
var testnetFilesCmd = &cobra.Command{
|
||||||
@ -48,7 +48,7 @@ func testnetFiles(cmd *cobra.Command, args []string) {
|
|||||||
privVal := types.LoadPrivValidator(privValFile)
|
privVal := types.LoadPrivValidator(privValFile)
|
||||||
genVals[i] = types.GenesisValidator{
|
genVals[i] = types.GenesisValidator{
|
||||||
PubKey: privVal.PubKey,
|
PubKey: privVal.PubKey,
|
||||||
Amount: 1,
|
Power: 1,
|
||||||
Name: mach,
|
Name: mach,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -4,8 +4,6 @@ import (
|
|||||||
"fmt"
|
"fmt"
|
||||||
"path/filepath"
|
"path/filepath"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/tendermint/tendermint/types" // TODO: remove
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// Config defines the top level configuration for a Tendermint node
|
// Config defines the top level configuration for a Tendermint node
|
||||||
@ -320,10 +318,6 @@ type ConsensusConfig struct {
|
|||||||
CreateEmptyBlocks bool `mapstructure:"create_empty_blocks"`
|
CreateEmptyBlocks bool `mapstructure:"create_empty_blocks"`
|
||||||
CreateEmptyBlocksInterval int `mapstructure:"create_empty_blocks_interval"`
|
CreateEmptyBlocksInterval int `mapstructure:"create_empty_blocks_interval"`
|
||||||
|
|
||||||
// TODO: This probably shouldn't be exposed but it makes it
|
|
||||||
// easy to write tests for the wal/replay
|
|
||||||
BlockPartSize int `mapstructure:"block_part_size"`
|
|
||||||
|
|
||||||
// Reactor sleep duration parameters are in ms
|
// Reactor sleep duration parameters are in ms
|
||||||
PeerGossipSleepDuration int `mapstructure:"peer_gossip_sleep_duration"`
|
PeerGossipSleepDuration int `mapstructure:"peer_gossip_sleep_duration"`
|
||||||
PeerQueryMaj23SleepDuration int `mapstructure:"peer_query_maj23_sleep_duration"`
|
PeerQueryMaj23SleepDuration int `mapstructure:"peer_query_maj23_sleep_duration"`
|
||||||
@ -386,7 +380,6 @@ func DefaultConsensusConfig() *ConsensusConfig {
|
|||||||
MaxBlockSizeBytes: 1, // TODO
|
MaxBlockSizeBytes: 1, // TODO
|
||||||
CreateEmptyBlocks: true,
|
CreateEmptyBlocks: true,
|
||||||
CreateEmptyBlocksInterval: 0,
|
CreateEmptyBlocksInterval: 0,
|
||||||
BlockPartSize: types.DefaultBlockPartSize, // TODO: we shouldnt be importing types
|
|
||||||
PeerGossipSleepDuration: 100,
|
PeerGossipSleepDuration: 100,
|
||||||
PeerQueryMaj23SleepDuration: 2000,
|
PeerQueryMaj23SleepDuration: 2000,
|
||||||
}
|
}
|
||||||
|
@ -119,7 +119,7 @@ var testGenesis = `{
|
|||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
"data":"3B3069C422E19688B45CBFAE7BB009FC0FA1B1EA86593519318B7214853803C8"
|
"data":"3B3069C422E19688B45CBFAE7BB009FC0FA1B1EA86593519318B7214853803C8"
|
||||||
},
|
},
|
||||||
"amount": 10,
|
"power": 10,
|
||||||
"name": ""
|
"name": ""
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
|
@ -400,7 +400,7 @@ func randGenesisDoc(numValidators int, randPower bool, minPower int64) (*types.G
|
|||||||
val, privVal := types.RandValidator(randPower, minPower)
|
val, privVal := types.RandValidator(randPower, minPower)
|
||||||
validators[i] = types.GenesisValidator{
|
validators[i] = types.GenesisValidator{
|
||||||
PubKey: val.PubKey,
|
PubKey: val.PubKey,
|
||||||
Amount: val.VotingPower,
|
Power: val.VotingPower,
|
||||||
}
|
}
|
||||||
privValidators[i] = privVal
|
privValidators[i] = privVal
|
||||||
}
|
}
|
||||||
|
@ -267,8 +267,6 @@ func testReplayCrashBeforeWriteVote(t *testing.T, thisCase *testCase, lineNum in
|
|||||||
var (
|
var (
|
||||||
NUM_BLOCKS = 6 // number of blocks in the test_data/many_blocks.cswal
|
NUM_BLOCKS = 6 // number of blocks in the test_data/many_blocks.cswal
|
||||||
mempool = types.MockMempool{}
|
mempool = types.MockMempool{}
|
||||||
|
|
||||||
testPartSize int
|
|
||||||
)
|
)
|
||||||
|
|
||||||
//---------------------------------------
|
//---------------------------------------
|
||||||
@ -320,7 +318,6 @@ func testHandshakeReplay(t *testing.T, nBlocks int, mode uint) {
|
|||||||
config.Consensus.SetWalFile(walFile)
|
config.Consensus.SetWalFile(walFile)
|
||||||
|
|
||||||
privVal := types.LoadPrivValidator(config.PrivValidatorFile())
|
privVal := types.LoadPrivValidator(config.PrivValidatorFile())
|
||||||
testPartSize = config.Consensus.BlockPartSize
|
|
||||||
|
|
||||||
wal, err := NewWAL(walFile, false)
|
wal, err := NewWAL(walFile, false)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -384,6 +381,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
|
||||||
err := st.ApplyBlock(nil, proxyApp.Consensus(), blk, blk.MakePartSet(testPartSize).Header(), mempool)
|
err := st.ApplyBlock(nil, proxyApp.Consensus(), blk, blk.MakePartSet(testPartSize).Header(), mempool)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err)
|
panic(err)
|
||||||
@ -503,7 +501,7 @@ func makeBlockchainFromWAL(wal *WAL) ([]*types.Block, []*types.Commit, error) {
|
|||||||
// if its not the first one, we have a full block
|
// if its not the first one, we have a full block
|
||||||
if blockParts != nil {
|
if blockParts != nil {
|
||||||
var n int
|
var n int
|
||||||
block := wire.ReadBinary(&types.Block{}, blockParts.GetReader(), types.MaxBlockSize, &n, &err).(*types.Block)
|
block := wire.ReadBinary(&types.Block{}, blockParts.GetReader(), 0, &n, &err).(*types.Block)
|
||||||
blocks = append(blocks, block)
|
blocks = append(blocks, block)
|
||||||
}
|
}
|
||||||
blockParts = types.NewPartSetFromHeader(*p)
|
blockParts = types.NewPartSetFromHeader(*p)
|
||||||
@ -524,7 +522,7 @@ func makeBlockchainFromWAL(wal *WAL) ([]*types.Block, []*types.Commit, error) {
|
|||||||
}
|
}
|
||||||
// grab the last block too
|
// grab the last block too
|
||||||
var n int
|
var n int
|
||||||
block := wire.ReadBinary(&types.Block{}, blockParts.GetReader(), types.MaxBlockSize, &n, &err).(*types.Block)
|
block := wire.ReadBinary(&types.Block{}, blockParts.GetReader(), 0, &n, &err).(*types.Block)
|
||||||
blocks = append(blocks, block)
|
blocks = append(blocks, block)
|
||||||
return blocks, commits, nil
|
return blocks, commits, nil
|
||||||
}
|
}
|
||||||
@ -563,7 +561,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)
|
store := NewMockBlockStore(config, state.Params())
|
||||||
return state, store
|
return state, store
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -572,13 +570,14 @@ func stateAndStore(config *cfg.Config, pubKey crypto.PubKey) (*sm.State, *mockBl
|
|||||||
|
|
||||||
type mockBlockStore struct {
|
type mockBlockStore struct {
|
||||||
config *cfg.Config
|
config *cfg.Config
|
||||||
|
params types.ConsensusParams
|
||||||
chain []*types.Block
|
chain []*types.Block
|
||||||
commits []*types.Commit
|
commits []*types.Commit
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: NewBlockStore(db.NewMemDB) ...
|
// TODO: NewBlockStore(db.NewMemDB) ...
|
||||||
func NewMockBlockStore(config *cfg.Config) *mockBlockStore {
|
func NewMockBlockStore(config *cfg.Config, params types.ConsensusParams) *mockBlockStore {
|
||||||
return &mockBlockStore{config, nil, nil}
|
return &mockBlockStore{config, params, nil, nil}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (bs *mockBlockStore) Height() int { return len(bs.chain) }
|
func (bs *mockBlockStore) Height() int { return len(bs.chain) }
|
||||||
@ -586,7 +585,7 @@ func (bs *mockBlockStore) LoadBlock(height int) *types.Block { return bs.chain[h
|
|||||||
func (bs *mockBlockStore) LoadBlockMeta(height int) *types.BlockMeta {
|
func (bs *mockBlockStore) LoadBlockMeta(height int) *types.BlockMeta {
|
||||||
block := bs.chain[height-1]
|
block := bs.chain[height-1]
|
||||||
return &types.BlockMeta{
|
return &types.BlockMeta{
|
||||||
BlockID: types.BlockID{block.Hash(), block.MakePartSet(bs.config.Consensus.BlockPartSize).Header()},
|
BlockID: types.BlockID{block.Hash(), block.MakePartSet(bs.params.BlockPartSizeBytes).Header()},
|
||||||
Header: block.Header,
|
Header: block.Header,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -983,7 +983,8 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
|
|||||||
txs := cs.mempool.Reap(cs.config.MaxBlockSizeTxs)
|
txs := cs.mempool.Reap(cs.config.MaxBlockSizeTxs)
|
||||||
|
|
||||||
return types.MakeBlock(cs.Height, cs.state.ChainID, txs, commit,
|
return types.MakeBlock(cs.Height, cs.state.ChainID, txs, commit,
|
||||||
cs.state.LastBlockID, cs.state.Validators.Hash(), cs.state.AppHash, cs.config.BlockPartSize)
|
cs.state.LastBlockID, cs.state.Validators.Hash(),
|
||||||
|
cs.state.AppHash, cs.state.Params().BlockPartSizeBytes)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Enter: `timeoutPropose` after entering Propose.
|
// Enter: `timeoutPropose` after entering Propose.
|
||||||
@ -1417,7 +1418,8 @@ func (cs *ConsensusState) addProposalBlockPart(height int, part *types.Part, ver
|
|||||||
// Added and completed!
|
// Added and completed!
|
||||||
var n int
|
var n int
|
||||||
var err error
|
var err error
|
||||||
cs.ProposalBlock = wire.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(), types.MaxBlockSize, &n, &err).(*types.Block)
|
cs.ProposalBlock = wire.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(),
|
||||||
|
cs.state.Params().BlockSizeParams.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 == RoundStepPropose && cs.isProposalComplete() {
|
if cs.Step == RoundStepPropose && cs.isProposalComplete() {
|
||||||
|
@ -180,7 +180,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 := config.Consensus.BlockPartSize
|
partSize := cs1.state.Params().BlockPartSizeBytes
|
||||||
|
|
||||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||||
voteCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringVote(), 1)
|
voteCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringVote(), 1)
|
||||||
@ -327,7 +327,7 @@ func TestLockNoPOL(t *testing.T) {
|
|||||||
vs2 := vss[1]
|
vs2 := vss[1]
|
||||||
height := cs1.Height
|
height := cs1.Height
|
||||||
|
|
||||||
partSize := config.Consensus.BlockPartSize
|
partSize := cs1.state.Params().BlockPartSizeBytes
|
||||||
|
|
||||||
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||||
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
||||||
@ -493,7 +493,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 := config.Consensus.BlockPartSize
|
partSize := cs1.state.Params().BlockPartSizeBytes
|
||||||
|
|
||||||
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||||
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
||||||
@ -608,7 +608,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 := config.Consensus.BlockPartSize
|
partSize := cs1.state.Params().BlockPartSizeBytes
|
||||||
|
|
||||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||||
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||||
@ -703,7 +703,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 := config.Consensus.BlockPartSize
|
partSize := cs1.state.Params().BlockPartSizeBytes
|
||||||
|
|
||||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||||
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||||
@ -824,7 +824,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 := config.Consensus.BlockPartSize
|
partSize := cs1.state.Params().BlockPartSizeBytes
|
||||||
|
|
||||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||||
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
timeoutProposeCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutPropose(), 1)
|
||||||
@ -999,7 +999,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 := config.Consensus.BlockPartSize
|
partSize := cs1.state.Params().BlockPartSizeBytes
|
||||||
|
|
||||||
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
proposalCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringCompleteProposal(), 1)
|
||||||
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
timeoutWaitCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringTimeoutWait(), 1)
|
||||||
|
85
docs/architecture/adr-005-consensus-params.md
Normal file
85
docs/architecture/adr-005-consensus-params.md
Normal file
@ -0,0 +1,85 @@
|
|||||||
|
# ADR 005: Consensus Params
|
||||||
|
|
||||||
|
## Context
|
||||||
|
|
||||||
|
Consensus critical parameters controlling blockchain capacity have until now been hard coded, loaded from a local config, or neglected.
|
||||||
|
Since they may be need to be different in different networks, and potentially to evolve over time within
|
||||||
|
networks, we seek to initialize them in a genesis file, and expose them through the ABCI.
|
||||||
|
|
||||||
|
While we have some specific parameters now, like maximum block and transaction size, we expect to have more in the future,
|
||||||
|
such as a period over which evidence is valid, or the frequency of checkpoints.
|
||||||
|
|
||||||
|
## Decision
|
||||||
|
|
||||||
|
### ConsensusParams
|
||||||
|
|
||||||
|
No consensus critical parameters should ever be found in the `config.toml`.
|
||||||
|
|
||||||
|
A new `ConsensusParams` is optionally included in the `genesis.json` file,
|
||||||
|
and loaded into the `State`. Any items not included are set to their default value.
|
||||||
|
A value of 0 is undefined (see ABCI, below). A value of -1 is used to indicate the parameter does not apply.
|
||||||
|
The parameters are used to determine the validity of a block (and tx) via the union of all relevant parameters.
|
||||||
|
|
||||||
|
```
|
||||||
|
type ConsensusParams struct {
|
||||||
|
BlockSizeParams
|
||||||
|
TxSizeParams
|
||||||
|
BlockGossipParams
|
||||||
|
}
|
||||||
|
|
||||||
|
type BlockSizeParams struct {
|
||||||
|
MaxBytes int
|
||||||
|
MaxTxs int
|
||||||
|
MaxGas int
|
||||||
|
}
|
||||||
|
|
||||||
|
type TxSizeParams struct {
|
||||||
|
MaxBytes int
|
||||||
|
MaxGas int
|
||||||
|
}
|
||||||
|
|
||||||
|
type BlockGossipParams struct {
|
||||||
|
BlockPartSizeBytes int
|
||||||
|
}
|
||||||
|
```
|
||||||
|
|
||||||
|
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 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
|
||||||
|
|
||||||
|
#### InitChain
|
||||||
|
|
||||||
|
InitChain currently takes the initial validator set. It should be extended to also take parts of the ConsensusParams.
|
||||||
|
There is some case to be made for it to take the entire Genesis, except there may be things in the genesis,
|
||||||
|
like the BlockPartSize, that the app shouldn't really know about.
|
||||||
|
|
||||||
|
#### EndBlock
|
||||||
|
|
||||||
|
The EndBlock response includes a `ConsensusParams`, which includes BlockSizeParams and TxSizeParams, but not BlockGossipParams.
|
||||||
|
Other param struct can be added to `ConsensusParams` in the future.
|
||||||
|
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.
|
||||||
|
Tendermint should have hard-coded upper limits as sanity checks.
|
||||||
|
|
||||||
|
## Status
|
||||||
|
|
||||||
|
Proposed.
|
||||||
|
|
||||||
|
## Consequences
|
||||||
|
|
||||||
|
### Positive
|
||||||
|
|
||||||
|
- Alternative capacity limits and consensus parameters can be specified without re-compiling the software.
|
||||||
|
- They can also change over time under the control of the application
|
||||||
|
|
||||||
|
### Negative
|
||||||
|
|
||||||
|
- More exposed parameters is more complexity
|
||||||
|
- Different rules at different heights in the blockchain complicates fast sync
|
||||||
|
|
||||||
|
### Neutral
|
||||||
|
|
||||||
|
- The TxSizeParams, which checks validity, may be in conflict with the config's `max_block_size_tx`, which determines proposal sizes
|
@ -21,7 +21,7 @@ Fields
|
|||||||
- ``validators``:
|
- ``validators``:
|
||||||
- ``pub_key``: The first element specifies the pub\_key type. 1 ==
|
- ``pub_key``: The first element specifies the pub\_key type. 1 ==
|
||||||
Ed25519. The second element are the pubkey bytes.
|
Ed25519. The second element are the pubkey bytes.
|
||||||
- ``amount``: The validator's voting power.
|
- ``power``: The validator's voting power.
|
||||||
- ``name``: Name of the validator (optional).
|
- ``name``: Name of the validator (optional).
|
||||||
- ``app_hash``: The expected application hash (as returned by the
|
- ``app_hash``: The expected application hash (as returned by the
|
||||||
``Commit`` ABCI message) upon genesis. If the app's hash does not
|
``Commit`` ABCI message) upon genesis. If the app's hash does not
|
||||||
@ -41,7 +41,7 @@ Sample genesis.json
|
|||||||
1,
|
1,
|
||||||
"9BC5112CB9614D91CE423FA8744885126CD9D08D9FC9D1F42E552D662BAA411E"
|
"9BC5112CB9614D91CE423FA8744885126CD9D08D9FC9D1F42E552D662BAA411E"
|
||||||
],
|
],
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach1"
|
"name": "mach1"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
@ -49,7 +49,7 @@ Sample genesis.json
|
|||||||
1,
|
1,
|
||||||
"F46A5543D51F31660D9F59653B4F96061A740FF7433E0DC1ECBC30BE8494DE06"
|
"F46A5543D51F31660D9F59653B4F96061A740FF7433E0DC1ECBC30BE8494DE06"
|
||||||
],
|
],
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach2"
|
"name": "mach2"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
@ -57,7 +57,7 @@ Sample genesis.json
|
|||||||
1,
|
1,
|
||||||
"0E7B423C1635FD07C0FC3603B736D5D27953C1C6CA865BB9392CD79DE1A682BB"
|
"0E7B423C1635FD07C0FC3603B736D5D27953C1C6CA865BB9392CD79DE1A682BB"
|
||||||
],
|
],
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach3"
|
"name": "mach3"
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
@ -65,7 +65,7 @@ Sample genesis.json
|
|||||||
1,
|
1,
|
||||||
"4F49237B9A32EB50682EDD83C48CE9CDB1D02A7CFDADCFF6EC8C1FAADB358879"
|
"4F49237B9A32EB50682EDD83C48CE9CDB1D02A7CFDADCFF6EC8C1FAADB358879"
|
||||||
],
|
],
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach4"
|
"name": "mach4"
|
||||||
}
|
}
|
||||||
],
|
],
|
||||||
|
@ -177,7 +177,7 @@ When ``tendermint init`` is run, both a ``genesis.json`` and
|
|||||||
"genesis_time": "0001-01-01T00:00:00.000Z",
|
"genesis_time": "0001-01-01T00:00:00.000Z",
|
||||||
"validators": [
|
"validators": [
|
||||||
{
|
{
|
||||||
"amount": 10,
|
"power": 10,
|
||||||
"name": "",
|
"name": "",
|
||||||
"pub_key": [
|
"pub_key": [
|
||||||
1,
|
1,
|
||||||
@ -310,7 +310,7 @@ then the new ``genesis.json`` will be:
|
|||||||
"genesis_time": "0001-01-01T00:00:00.000Z",
|
"genesis_time": "0001-01-01T00:00:00.000Z",
|
||||||
"validators": [
|
"validators": [
|
||||||
{
|
{
|
||||||
"amount": 10,
|
"power": 10,
|
||||||
"name": "",
|
"name": "",
|
||||||
"pub_key": [
|
"pub_key": [
|
||||||
1,
|
1,
|
||||||
@ -318,7 +318,7 @@ then the new ``genesis.json`` will be:
|
|||||||
]
|
]
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 10,
|
"power": 10,
|
||||||
"name": "",
|
"name": "",
|
||||||
"pub_key": [
|
"pub_key": [
|
||||||
1,
|
1,
|
||||||
|
@ -93,7 +93,7 @@ func TestGenesisAndValidators(t *testing.T) {
|
|||||||
val := vals.Validators[0]
|
val := vals.Validators[0]
|
||||||
|
|
||||||
// make sure the current set is also the genesis set
|
// make sure the current set is also the genesis set
|
||||||
assert.Equal(t, gval.Amount, val.VotingPower)
|
assert.Equal(t, gval.Power, val.VotingPower)
|
||||||
assert.Equal(t, gval.PubKey, val.PubKey)
|
assert.Equal(t, gval.PubKey, val.PubKey)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -90,7 +90,7 @@ func UnsafeDialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) {
|
|||||||
// "validators": [
|
// "validators": [
|
||||||
// {
|
// {
|
||||||
// "name": "",
|
// "name": "",
|
||||||
// "amount": 10,
|
// "power": 10,
|
||||||
// "pub_key": {
|
// "pub_key": {
|
||||||
// "data": "68DFDA7E50F82946E7E8546BED37944A422CD1B831E70DF66BA3B8430593944D",
|
// "data": "68DFDA7E50F82946E7E8546BED37944A422CD1B831E70DF66BA3B8430593944D",
|
||||||
// "type": "ed25519"
|
// "type": "ed25519"
|
||||||
|
@ -65,6 +65,19 @@ type State struct {
|
|||||||
logger log.Logger
|
logger log.Logger
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// GetState loads the most recent state from the database,
|
||||||
|
// or creates a new one from the given genesisFile and persists the result
|
||||||
|
// to the database.
|
||||||
|
func GetState(stateDB dbm.DB, genesisFile string) *State {
|
||||||
|
state := LoadState(stateDB)
|
||||||
|
if state == nil {
|
||||||
|
state = MakeGenesisStateFromFile(stateDB, genesisFile)
|
||||||
|
state.Save()
|
||||||
|
}
|
||||||
|
|
||||||
|
return state
|
||||||
|
}
|
||||||
|
|
||||||
// LoadState loads the State from the database.
|
// LoadState loads the State from the database.
|
||||||
func LoadState(db dbm.DB) *State {
|
func LoadState(db dbm.DB) *State {
|
||||||
return loadState(db, stateKey)
|
return loadState(db, stateKey)
|
||||||
@ -248,17 +261,12 @@ func (s *State) GetValidators() (*types.ValidatorSet, *types.ValidatorSet) {
|
|||||||
return s.LastValidators, s.Validators
|
return s.LastValidators, s.Validators
|
||||||
}
|
}
|
||||||
|
|
||||||
// GetState loads the most recent state from the database,
|
// Params returns the consensus parameters used for
|
||||||
// or creates a new one from the given genesisFile and persists the result
|
// validating blocks
|
||||||
// to the database.
|
func (s *State) Params() types.ConsensusParams {
|
||||||
func GetState(stateDB dbm.DB, genesisFile string) *State {
|
// TODO: this should move into the State proper
|
||||||
state := LoadState(stateDB)
|
// when we allow the app to change it
|
||||||
if state == nil {
|
return *s.GenesisDoc.ConsensusParams
|
||||||
state = MakeGenesisStateFromFile(stateDB, genesisFile)
|
|
||||||
state.Save()
|
|
||||||
}
|
|
||||||
|
|
||||||
return state
|
|
||||||
}
|
}
|
||||||
|
|
||||||
//------------------------------------------------------------------------
|
//------------------------------------------------------------------------
|
||||||
@ -324,12 +332,9 @@ func MakeGenesisStateFromFile(db dbm.DB, genDocFile string) *State {
|
|||||||
//
|
//
|
||||||
// Used in tests.
|
// Used in tests.
|
||||||
func MakeGenesisState(db dbm.DB, genDoc *types.GenesisDoc) *State {
|
func MakeGenesisState(db dbm.DB, genDoc *types.GenesisDoc) *State {
|
||||||
if len(genDoc.Validators) == 0 {
|
err := genDoc.ValidateAndComplete()
|
||||||
cmn.Exit(cmn.Fmt("The genesis file has no validators"))
|
if err != nil {
|
||||||
}
|
cmn.Exit(cmn.Fmt("Error in genesis file: %v", err))
|
||||||
|
|
||||||
if genDoc.GenesisTime.IsZero() {
|
|
||||||
genDoc.GenesisTime = time.Now()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Make validators slice
|
// Make validators slice
|
||||||
@ -342,7 +347,7 @@ func MakeGenesisState(db dbm.DB, genDoc *types.GenesisDoc) *State {
|
|||||||
validators[i] = &types.Validator{
|
validators[i] = &types.Validator{
|
||||||
Address: address,
|
Address: address,
|
||||||
PubKey: pubKey,
|
PubKey: pubKey,
|
||||||
VotingPower: val.Amount,
|
VotingPower: val.Power,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -4,7 +4,7 @@
|
|||||||
"genesis_time": "2016-06-24T20:01:19.322Z",
|
"genesis_time": "2016-06-24T20:01:19.322Z",
|
||||||
"validators": [
|
"validators": [
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach1",
|
"name": "mach1",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -12,7 +12,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach2",
|
"name": "mach2",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -20,7 +20,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach3",
|
"name": "mach3",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -28,7 +28,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach4",
|
"name": "mach4",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
|
@ -4,7 +4,7 @@
|
|||||||
"genesis_time": "2016-06-24T20:01:19.322Z",
|
"genesis_time": "2016-06-24T20:01:19.322Z",
|
||||||
"validators": [
|
"validators": [
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach1",
|
"name": "mach1",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -12,7 +12,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach2",
|
"name": "mach2",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -20,7 +20,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach3",
|
"name": "mach3",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -28,7 +28,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach4",
|
"name": "mach4",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
|
@ -4,7 +4,7 @@
|
|||||||
"genesis_time": "2016-06-24T20:01:19.322Z",
|
"genesis_time": "2016-06-24T20:01:19.322Z",
|
||||||
"validators": [
|
"validators": [
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach1",
|
"name": "mach1",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -12,7 +12,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach2",
|
"name": "mach2",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -20,7 +20,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach3",
|
"name": "mach3",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -28,7 +28,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach4",
|
"name": "mach4",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
|
@ -4,7 +4,7 @@
|
|||||||
"genesis_time": "2016-06-24T20:01:19.322Z",
|
"genesis_time": "2016-06-24T20:01:19.322Z",
|
||||||
"validators": [
|
"validators": [
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach1",
|
"name": "mach1",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -12,7 +12,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach2",
|
"name": "mach2",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -20,7 +20,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach3",
|
"name": "mach3",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
@ -28,7 +28,7 @@
|
|||||||
}
|
}
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"amount": 1,
|
"power": 1,
|
||||||
"name": "mach4",
|
"name": "mach4",
|
||||||
"pub_key": {
|
"pub_key": {
|
||||||
"type": "ed25519",
|
"type": "ed25519",
|
||||||
|
@ -14,11 +14,6 @@ import (
|
|||||||
"github.com/tendermint/tmlibs/merkle"
|
"github.com/tendermint/tmlibs/merkle"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
|
||||||
MaxBlockSize = 22020096 // 21MB TODO make it configurable
|
|
||||||
DefaultBlockPartSize = 65536 // 64kB TODO: put part size in parts header?
|
|
||||||
)
|
|
||||||
|
|
||||||
// Block defines the atomic unit of a Tendermint blockchain
|
// Block defines the atomic unit of a Tendermint blockchain
|
||||||
type Block struct {
|
type Block struct {
|
||||||
*Header `json:"header"`
|
*Header `json:"header"`
|
||||||
|
@ -12,27 +12,23 @@ import (
|
|||||||
cmn "github.com/tendermint/tmlibs/common"
|
cmn "github.com/tendermint/tmlibs/common"
|
||||||
)
|
)
|
||||||
|
|
||||||
//------------------------------------------------------------
|
|
||||||
// we store the gendoc in the db
|
|
||||||
|
|
||||||
var GenDocKey = []byte("GenDocKey")
|
|
||||||
|
|
||||||
//------------------------------------------------------------
|
//------------------------------------------------------------
|
||||||
// core types for a genesis definition
|
// core types for a genesis definition
|
||||||
|
|
||||||
// GenesisValidator is an initial validator.
|
// GenesisValidator is an initial validator.
|
||||||
type GenesisValidator struct {
|
type GenesisValidator struct {
|
||||||
PubKey crypto.PubKey `json:"pub_key"`
|
PubKey crypto.PubKey `json:"pub_key"`
|
||||||
Amount int64 `json:"amount"`
|
Power int64 `json:"power"`
|
||||||
Name string `json:"name"`
|
Name string `json:"name"`
|
||||||
}
|
}
|
||||||
|
|
||||||
// GenesisDoc defines the initial conditions for a tendermint blockchain, in particular its validator set.
|
// GenesisDoc defines the initial conditions for a tendermint blockchain, in particular its validator set.
|
||||||
type GenesisDoc struct {
|
type GenesisDoc struct {
|
||||||
GenesisTime time.Time `json:"genesis_time"`
|
GenesisTime time.Time `json:"genesis_time"`
|
||||||
ChainID string `json:"chain_id"`
|
ChainID string `json:"chain_id"`
|
||||||
Validators []GenesisValidator `json:"validators"`
|
ConsensusParams *ConsensusParams `json:"consensus_params,omitempty"`
|
||||||
AppHash data.Bytes `json:"app_hash"`
|
Validators []GenesisValidator `json:"validators"`
|
||||||
|
AppHash data.Bytes `json:"app_hash"`
|
||||||
}
|
}
|
||||||
|
|
||||||
// SaveAs is a utility method for saving GenensisDoc as a JSON file.
|
// SaveAs is a utility method for saving GenensisDoc as a JSON file.
|
||||||
@ -48,12 +44,45 @@ func (genDoc *GenesisDoc) SaveAs(file string) error {
|
|||||||
func (genDoc *GenesisDoc) ValidatorHash() []byte {
|
func (genDoc *GenesisDoc) ValidatorHash() []byte {
|
||||||
vals := make([]*Validator, len(genDoc.Validators))
|
vals := make([]*Validator, len(genDoc.Validators))
|
||||||
for i, v := range genDoc.Validators {
|
for i, v := range genDoc.Validators {
|
||||||
vals[i] = NewValidator(v.PubKey, v.Amount)
|
vals[i] = NewValidator(v.PubKey, v.Power)
|
||||||
}
|
}
|
||||||
vset := NewValidatorSet(vals)
|
vset := NewValidatorSet(vals)
|
||||||
return vset.Hash()
|
return vset.Hash()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ValidateAndComplete checks that all necessary fields are present
|
||||||
|
// and fills in defaults for optional fields left empty
|
||||||
|
func (genDoc *GenesisDoc) ValidateAndComplete() error {
|
||||||
|
|
||||||
|
if genDoc.ChainID == "" {
|
||||||
|
return errors.Errorf("Genesis doc must include non-empty chain_id")
|
||||||
|
}
|
||||||
|
|
||||||
|
if genDoc.ConsensusParams == nil {
|
||||||
|
genDoc.ConsensusParams = DefaultConsensusParams()
|
||||||
|
} else {
|
||||||
|
if err := genDoc.ConsensusParams.Validate(); err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if len(genDoc.Validators) == 0 {
|
||||||
|
return errors.Errorf("The genesis file must have at least one validator")
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, v := range genDoc.Validators {
|
||||||
|
if v.Power == 0 {
|
||||||
|
return errors.Errorf("The genesis file cannot contain validators with no voting power: %v", v)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if genDoc.GenesisTime.IsZero() {
|
||||||
|
genDoc.GenesisTime = time.Now()
|
||||||
|
}
|
||||||
|
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
//------------------------------------------------------------
|
//------------------------------------------------------------
|
||||||
// Make genesis state from file
|
// Make genesis state from file
|
||||||
|
|
||||||
@ -61,6 +90,14 @@ func (genDoc *GenesisDoc) ValidatorHash() []byte {
|
|||||||
func GenesisDocFromJSON(jsonBlob []byte) (*GenesisDoc, error) {
|
func GenesisDocFromJSON(jsonBlob []byte) (*GenesisDoc, error) {
|
||||||
genDoc := GenesisDoc{}
|
genDoc := GenesisDoc{}
|
||||||
err := json.Unmarshal(jsonBlob, &genDoc)
|
err := json.Unmarshal(jsonBlob, &genDoc)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
if err := genDoc.ValidateAndComplete(); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
return &genDoc, err
|
return &genDoc, err
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -72,10 +109,7 @@ func GenesisDocFromFile(genDocFile string) (*GenesisDoc, error) {
|
|||||||
}
|
}
|
||||||
genDoc, err := GenesisDocFromJSON(jsonBlob)
|
genDoc, err := GenesisDocFromJSON(jsonBlob)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, errors.Wrap(err, "Error reading GenesisDoc")
|
return nil, errors.Wrap(err, cmn.Fmt("Error reading GenesisDoc at %v", genDocFile))
|
||||||
}
|
|
||||||
if genDoc.ChainID == "" {
|
|
||||||
return nil, errors.Errorf("Genesis doc %v must include non-empty chain_id", genDocFile)
|
|
||||||
}
|
}
|
||||||
return genDoc, nil
|
return genDoc, nil
|
||||||
}
|
}
|
||||||
|
89
types/genesis_test.go
Normal file
89
types/genesis_test.go
Normal file
@ -0,0 +1,89 @@
|
|||||||
|
package types
|
||||||
|
|
||||||
|
import (
|
||||||
|
"encoding/json"
|
||||||
|
"testing"
|
||||||
|
|
||||||
|
"github.com/stretchr/testify/assert"
|
||||||
|
crypto "github.com/tendermint/go-crypto"
|
||||||
|
)
|
||||||
|
|
||||||
|
func TestGenesis(t *testing.T) {
|
||||||
|
|
||||||
|
// test some bad ones from raw json
|
||||||
|
testCases := [][]byte{
|
||||||
|
[]byte{}, // empty
|
||||||
|
[]byte{1, 1, 1, 1, 1}, // junk
|
||||||
|
[]byte(`{}`), // empty
|
||||||
|
[]byte(`{"chain_id": "mychain"}`), // missing validators
|
||||||
|
[]byte(`{"chain_id": "mychain", "validators": []`), // missing validators
|
||||||
|
[]byte(`{"chain_id": "mychain", "validators": [{}]`), // missing validators
|
||||||
|
[]byte(`{"validators":[{"pub_key":
|
||||||
|
{"type":"ed25519","data":"961EAB8752E51A03618502F55C2B6E09C38C65635C64CCF3173ED452CF86C957"},
|
||||||
|
"power":10,"name":""}]}`), // missing chain_id
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, testCase := range testCases {
|
||||||
|
_, err := GenesisDocFromJSON(testCase)
|
||||||
|
assert.Error(t, err, "expected error for empty genDoc json")
|
||||||
|
}
|
||||||
|
|
||||||
|
// test a good one by raw json
|
||||||
|
genDocBytes := []byte(`{"genesis_time":"0001-01-01T00:00:00Z","chain_id":"test-chain-QDKdJr","consensus_params":null,"validators":[{"pub_key":{"type":"ed25519","data":"961EAB8752E51A03618502F55C2B6E09C38C65635C64CCF3173ED452CF86C957"},"power":10,"name":""}],"app_hash":""}`)
|
||||||
|
_, err := GenesisDocFromJSON(genDocBytes)
|
||||||
|
assert.NoError(t, err, "expected no error for good genDoc json")
|
||||||
|
|
||||||
|
// create a base gendoc from struct
|
||||||
|
baseGenDoc := &GenesisDoc{
|
||||||
|
ChainID: "abc",
|
||||||
|
Validators: []GenesisValidator{{crypto.GenPrivKeyEd25519().PubKey(), 10, "myval"}},
|
||||||
|
}
|
||||||
|
genDocBytes, err = json.Marshal(baseGenDoc)
|
||||||
|
assert.NoError(t, err, "error marshalling genDoc")
|
||||||
|
|
||||||
|
// test base gendoc and check consensus params were filled
|
||||||
|
genDoc, err := GenesisDocFromJSON(genDocBytes)
|
||||||
|
assert.NoError(t, err, "expected no error for valid genDoc json")
|
||||||
|
assert.NotNil(t, genDoc.ConsensusParams, "expected consensus params to be filled in")
|
||||||
|
|
||||||
|
// create json with consensus params filled
|
||||||
|
genDocBytes, err = json.Marshal(genDoc)
|
||||||
|
assert.NoError(t, err, "error marshalling genDoc")
|
||||||
|
genDoc, err = GenesisDocFromJSON(genDocBytes)
|
||||||
|
assert.NoError(t, err, "expected no error for valid genDoc json")
|
||||||
|
|
||||||
|
// test with invalid consensus params
|
||||||
|
genDoc.ConsensusParams.BlockSizeParams.MaxBytes = 0
|
||||||
|
genDocBytes, err = json.Marshal(genDoc)
|
||||||
|
assert.NoError(t, err, "error marshalling genDoc")
|
||||||
|
genDoc, err = GenesisDocFromJSON(genDocBytes)
|
||||||
|
assert.Error(t, err, "expected error for genDoc json with block size of 0")
|
||||||
|
}
|
||||||
|
|
||||||
|
func newConsensusParams(blockSize, partSize int) ConsensusParams {
|
||||||
|
return ConsensusParams{
|
||||||
|
BlockSizeParams: BlockSizeParams{MaxBytes: blockSize},
|
||||||
|
BlockGossipParams: BlockGossipParams{BlockPartSizeBytes: partSize},
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestConsensusParams(t *testing.T) {
|
||||||
|
|
||||||
|
testCases := []struct {
|
||||||
|
params ConsensusParams
|
||||||
|
valid bool
|
||||||
|
}{
|
||||||
|
{newConsensusParams(1, 1), true},
|
||||||
|
{newConsensusParams(1, 0), false},
|
||||||
|
{newConsensusParams(0, 1), false},
|
||||||
|
{newConsensusParams(0, 0), false},
|
||||||
|
}
|
||||||
|
for _, testCase := range testCases {
|
||||||
|
if testCase.valid {
|
||||||
|
assert.NoError(t, testCase.params.Validate(), "expected no error for valid params")
|
||||||
|
} else {
|
||||||
|
assert.Error(t, testCase.params.Validate(), "expected error for non valid params")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
87
types/params.go
Normal file
87
types/params.go
Normal file
@ -0,0 +1,87 @@
|
|||||||
|
package types
|
||||||
|
|
||||||
|
import (
|
||||||
|
"github.com/pkg/errors"
|
||||||
|
)
|
||||||
|
|
||||||
|
const (
|
||||||
|
maxBlockSizeBytes = 104857600 // 100MB
|
||||||
|
)
|
||||||
|
|
||||||
|
// ConsensusParams contains consensus critical parameters
|
||||||
|
// that determine the validity of blocks.
|
||||||
|
type ConsensusParams struct {
|
||||||
|
BlockSizeParams `json:"block_size_params"`
|
||||||
|
TxSizeParams `json:"tx_size_params"`
|
||||||
|
BlockGossipParams `json:"block_gossip_params"`
|
||||||
|
}
|
||||||
|
|
||||||
|
// BlockSizeParams contain limits on the block size.
|
||||||
|
type BlockSizeParams struct {
|
||||||
|
MaxBytes int `json:"max_bytes"` // NOTE: must not be 0
|
||||||
|
MaxTxs int `json:"max_txs"`
|
||||||
|
MaxGas int `json:"max_gas"`
|
||||||
|
}
|
||||||
|
|
||||||
|
// TxSizeParams contain limits on the tx size.
|
||||||
|
type TxSizeParams struct {
|
||||||
|
MaxBytes int `json:"max_bytes"`
|
||||||
|
MaxGas int `json:"max_gas"`
|
||||||
|
}
|
||||||
|
|
||||||
|
// BlockGossipParams determine consensus critical elements of how blocks are gossiped
|
||||||
|
type BlockGossipParams struct {
|
||||||
|
BlockPartSizeBytes int `json:"block_part_size_bytes"` // NOTE: must not be 0
|
||||||
|
}
|
||||||
|
|
||||||
|
// DefaultConsensusParams returns a default ConsensusParams.
|
||||||
|
func DefaultConsensusParams() *ConsensusParams {
|
||||||
|
return &ConsensusParams{
|
||||||
|
DefaultBlockSizeParams(),
|
||||||
|
DefaultTxSizeParams(),
|
||||||
|
DefaultBlockGossipParams(),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// DefaultBlockSizeParams returns a default BlockSizeParams.
|
||||||
|
func DefaultBlockSizeParams() BlockSizeParams {
|
||||||
|
return BlockSizeParams{
|
||||||
|
MaxBytes: 22020096, // 21MB
|
||||||
|
MaxTxs: 100000,
|
||||||
|
MaxGas: -1,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// DefaultTxSizeParams returns a default TxSizeParams.
|
||||||
|
func DefaultTxSizeParams() TxSizeParams {
|
||||||
|
return TxSizeParams{
|
||||||
|
MaxBytes: 10240, // 10kB
|
||||||
|
MaxGas: -1,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// DefaultBlockGossipParams returns a default BlockGossipParams.
|
||||||
|
func DefaultBlockGossipParams() BlockGossipParams {
|
||||||
|
return BlockGossipParams{
|
||||||
|
BlockPartSizeBytes: 65536, // 64kB,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Validate validates the ConsensusParams to ensure all values
|
||||||
|
// are within their allowed limits, and returns an error if they are not.
|
||||||
|
func (params *ConsensusParams) Validate() error {
|
||||||
|
// 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.BlockGossipParams.BlockPartSizeBytes <= 0 {
|
||||||
|
return errors.Errorf("BlockGossipParams.BlockPartSizeBytes must be greater than 0. Got %d", params.BlockGossipParams.BlockPartSizeBytes)
|
||||||
|
}
|
||||||
|
|
||||||
|
// 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)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
@ -15,6 +15,7 @@ import (
|
|||||||
"github.com/tendermint/tmlibs/log"
|
"github.com/tendermint/tmlibs/log"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
// TODO: type ?
|
||||||
const (
|
const (
|
||||||
stepNone = 0 // Used to distinguish the initial state
|
stepNone = 0 // Used to distinguish the initial state
|
||||||
stepPropose = 1
|
stepPropose = 1
|
||||||
|
Loading…
x
Reference in New Issue
Block a user