Merge pull request #650 from tendermint/consensus_params

Consensus params
This commit is contained in:
Ethan Buchman 2017-09-21 15:46:57 -04:00 committed by GitHub
commit 8ff5b365dd
25 changed files with 395 additions and 100 deletions

View File

@ -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)
} }

View File

@ -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")
} }

View File

@ -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)

View File

@ -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,
} }
} }

View File

@ -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,
} }

View File

@ -119,7 +119,7 @@ var testGenesis = `{
"type": "ed25519", "type": "ed25519",
"data":"3B3069C422E19688B45CBFAE7BB009FC0FA1B1EA86593519318B7214853803C8" "data":"3B3069C422E19688B45CBFAE7BB009FC0FA1B1EA86593519318B7214853803C8"
}, },
"amount": 10, "power": 10,
"name": "" "name": ""
} }
], ],

View File

@ -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
} }

View File

@ -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,
} }
} }

View File

@ -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() {

View File

@ -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)

View 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

View File

@ -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"
} }
], ],

View File

@ -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,

View File

@ -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)
} }
} }

View File

@ -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"

View File

@ -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,
} }
} }

View File

@ -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",

View File

@ -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",

View File

@ -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",

View File

@ -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",

View File

@ -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"`

View File

@ -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
View 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
View 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
}

View File

@ -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