Browse Source

Remove ConsensusParams.TxSize and ConsensusParams.BlockGossip (#2364)

* remove ConsensusParams.TxSize and ConsensusParams.BlockGossip

Refs #2347

* block part size is now fixed

Refs #2347

* use max data size, not max bytes for tx limit

Refs #2347
pull/2410/head
Anton Kaliaev 6 years ago
committed by Ethan Buchman
parent
commit
0e1cd88863
18 changed files with 585 additions and 914 deletions
  1. +430
    -553
      abci/types/types.pb.go
  2. +7
    -13
      abci/types/types.proto
  3. +17
    -141
      abci/types/typespb_test.go
  4. +1
    -1
      blockchain/reactor.go
  5. +1
    -1
      blockchain/reactor_test.go
  6. +2
    -10
      consensus/replay_test.go
  7. +2
    -9
      consensus/state.go
  8. +7
    -7
      consensus/state_test.go
  9. +1
    -1
      consensus/wal.go
  10. +5
    -3
      node/node.go
  11. +5
    -2
      state/execution.go
  12. +1
    -1
      state/state.go
  13. +10
    -27
      state/state_test.go
  14. +22
    -0
      types/block.go
  15. +5
    -0
      types/evidence.go
  16. +31
    -69
      types/params.go
  17. +30
    -57
      types/params_test.go
  18. +8
    -19
      types/protobuf.go

+ 430
- 553
abci/types/types.pb.go
File diff suppressed because it is too large
View File


+ 7
- 13
abci/types/types.proto View File

@ -200,27 +200,21 @@ message ResponseCommit {
// that can be adjusted by the abci app
message ConsensusParams {
BlockSize block_size = 1;
TxSize tx_size = 2;
BlockGossip block_gossip = 3;
EvidenceParams evidence_params = 2;
}
// BlockSize contains limits on the block size.
message BlockSize {
// Note: must be greater than 0
int32 max_bytes = 1;
// Note: must be greater or equal to -1
int64 max_gas = 2;
}
// TxSize contains limits on the tx size.
message TxSize {
int32 max_bytes = 1;
int64 max_gas = 2;
}
// BlockGossip determine consensus critical
// elements of how blocks are gossiped
message BlockGossip {
// Note: must not be 0
int32 block_part_size_bytes = 1;
// EvidenceParams contains limits on the evidence.
message EvidenceParams {
// Note: must be greater than 0
int64 max_age = 1;
}
message LastCommitInfo {


+ 17
- 141
abci/types/typespb_test.go View File

@ -1534,15 +1534,15 @@ func TestBlockSizeMarshalTo(t *testing.T) {
}
}
func TestTxSizeProto(t *testing.T) {
func TestEvidenceParamsProto(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedTxSize(popr, false)
p := NewPopulatedEvidenceParams(popr, false)
dAtA, err := github_com_gogo_protobuf_proto.Marshal(p)
if err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
msg := &TxSize{}
msg := &EvidenceParams{}
if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
@ -1565,10 +1565,10 @@ func TestTxSizeProto(t *testing.T) {
}
}
func TestTxSizeMarshalTo(t *testing.T) {
func TestEvidenceParamsMarshalTo(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedTxSize(popr, false)
p := NewPopulatedEvidenceParams(popr, false)
size := p.Size()
dAtA := make([]byte, size)
for i := range dAtA {
@ -1578,63 +1578,7 @@ func TestTxSizeMarshalTo(t *testing.T) {
if err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
msg := &TxSize{}
if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
for i := range dAtA {
dAtA[i] = byte(popr.Intn(256))
}
if !p.Equal(msg) {
t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p)
}
}
func TestBlockGossipProto(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedBlockGossip(popr, false)
dAtA, err := github_com_gogo_protobuf_proto.Marshal(p)
if err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
msg := &BlockGossip{}
if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
littlefuzz := make([]byte, len(dAtA))
copy(littlefuzz, dAtA)
for i := range dAtA {
dAtA[i] = byte(popr.Intn(256))
}
if !p.Equal(msg) {
t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p)
}
if len(littlefuzz) > 0 {
fuzzamount := 100
for i := 0; i < fuzzamount; i++ {
littlefuzz[popr.Intn(len(littlefuzz))] = byte(popr.Intn(256))
littlefuzz = append(littlefuzz, byte(popr.Intn(256)))
}
// shouldn't panic
_ = github_com_gogo_protobuf_proto.Unmarshal(littlefuzz, msg)
}
}
func TestBlockGossipMarshalTo(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedBlockGossip(popr, false)
size := p.Size()
dAtA := make([]byte, size)
for i := range dAtA {
dAtA[i] = byte(popr.Intn(256))
}
_, err := p.MarshalTo(dAtA)
if err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
msg := &BlockGossip{}
msg := &EvidenceParams{}
if err := github_com_gogo_protobuf_proto.Unmarshal(dAtA, msg); err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
@ -2636,34 +2580,16 @@ func TestBlockSizeJSON(t *testing.T) {
t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p)
}
}
func TestTxSizeJSON(t *testing.T) {
func TestEvidenceParamsJSON(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedTxSize(popr, true)
p := NewPopulatedEvidenceParams(popr, true)
marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{}
jsondata, err := marshaler.MarshalToString(p)
if err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
msg := &TxSize{}
err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg)
if err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
if !p.Equal(msg) {
t.Fatalf("seed = %d, %#v !Json Equal %#v", seed, msg, p)
}
}
func TestBlockGossipJSON(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedBlockGossip(popr, true)
marshaler := github_com_gogo_protobuf_jsonpb.Marshaler{}
jsondata, err := marshaler.MarshalToString(p)
if err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
msg := &BlockGossip{}
msg := &EvidenceParams{}
err = github_com_gogo_protobuf_jsonpb.UnmarshalString(jsondata, msg)
if err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
@ -3590,12 +3516,12 @@ func TestBlockSizeProtoCompactText(t *testing.T) {
}
}
func TestTxSizeProtoText(t *testing.T) {
func TestEvidenceParamsProtoText(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedTxSize(popr, true)
p := NewPopulatedEvidenceParams(popr, true)
dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p)
msg := &TxSize{}
msg := &EvidenceParams{}
if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
@ -3604,40 +3530,12 @@ func TestTxSizeProtoText(t *testing.T) {
}
}
func TestTxSizeProtoCompactText(t *testing.T) {
func TestEvidenceParamsProtoCompactText(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedTxSize(popr, true)
p := NewPopulatedEvidenceParams(popr, true)
dAtA := github_com_gogo_protobuf_proto.CompactTextString(p)
msg := &TxSize{}
if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
if !p.Equal(msg) {
t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p)
}
}
func TestBlockGossipProtoText(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedBlockGossip(popr, true)
dAtA := github_com_gogo_protobuf_proto.MarshalTextString(p)
msg := &BlockGossip{}
if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
if !p.Equal(msg) {
t.Fatalf("seed = %d, %#v !Proto %#v", seed, msg, p)
}
}
func TestBlockGossipProtoCompactText(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedBlockGossip(popr, true)
dAtA := github_com_gogo_protobuf_proto.CompactTextString(p)
msg := &BlockGossip{}
msg := &EvidenceParams{}
if err := github_com_gogo_protobuf_proto.UnmarshalText(dAtA, msg); err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
@ -4492,32 +4390,10 @@ func TestBlockSizeSize(t *testing.T) {
}
}
func TestTxSizeSize(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedTxSize(popr, true)
size2 := github_com_gogo_protobuf_proto.Size(p)
dAtA, err := github_com_gogo_protobuf_proto.Marshal(p)
if err != nil {
t.Fatalf("seed = %d, err = %v", seed, err)
}
size := p.Size()
if len(dAtA) != size {
t.Errorf("seed = %d, size %v != marshalled size %v", seed, size, len(dAtA))
}
if size2 != size {
t.Errorf("seed = %d, size %v != before marshal proto.Size %v", seed, size, size2)
}
size3 := github_com_gogo_protobuf_proto.Size(p)
if size3 != size {
t.Errorf("seed = %d, size %v != after marshal proto.Size %v", seed, size, size3)
}
}
func TestBlockGossipSize(t *testing.T) {
func TestEvidenceParamsSize(t *testing.T) {
seed := time.Now().UnixNano()
popr := math_rand.New(math_rand.NewSource(seed))
p := NewPopulatedBlockGossip(popr, true)
p := NewPopulatedEvidenceParams(popr, true)
size2 := github_com_gogo_protobuf_proto.Size(p)
dAtA, err := github_com_gogo_protobuf_proto.Marshal(p)
if err != nil {


+ 1
- 1
blockchain/reactor.go View File

@ -290,7 +290,7 @@ FOR_LOOP:
didProcessCh <- struct{}{}
}
firstParts := first.MakePartSet(state.ConsensusParams.BlockPartSizeBytes)
firstParts := first.MakePartSet(types.BlockPartSizeBytes)
firstPartsHeader := firstParts.Header()
firstID := types.BlockID{first.Hash(), firstPartsHeader}
// Finally, verify the first block using the second's commit


+ 1
- 1
blockchain/reactor_test.go View File

@ -48,7 +48,7 @@ func newBlockchainReactor(logger log.Logger, maxBlockHeight int64) *BlockchainRe
for blockHeight := int64(1); blockHeight <= maxBlockHeight; blockHeight++ {
firstBlock := makeBlock(blockHeight, state)
secondBlock := makeBlock(blockHeight+1, state)
firstParts := firstBlock.MakePartSet(state.ConsensusParams.BlockGossip.BlockPartSizeBytes)
firstParts := firstBlock.MakePartSet(types.BlockPartSizeBytes)
blockStore.SaveBlock(firstBlock, firstParts, secondBlock.LastCommit)
}


+ 2
- 10
consensus/replay_test.go View File

@ -102,14 +102,6 @@ func TestWALCrash(t *testing.T) {
{"empty block",
func(stateDB dbm.DB, cs *ConsensusState, ctx context.Context) {},
1},
{"block with a smaller part size",
func(stateDB dbm.DB, cs *ConsensusState, ctx context.Context) {
// XXX: is there a better way to change BlockPartSizeBytes?
cs.state.ConsensusParams.BlockPartSizeBytes = 512
sm.SaveState(stateDB, cs.state)
go sendTxs(cs, ctx)
},
1},
{"many non-empty blocks",
func(stateDB dbm.DB, cs *ConsensusState, ctx context.Context) {
go sendTxs(cs, ctx)
@ -397,7 +389,7 @@ func testHandshakeReplay(t *testing.T, nBlocks int, mode uint) {
}
func applyBlock(stateDB dbm.DB, st sm.State, blk *types.Block, proxyApp proxy.AppConns) sm.State {
testPartSize := st.ConsensusParams.BlockPartSizeBytes
testPartSize := types.BlockPartSizeBytes
blockExec := sm.NewBlockExecutor(stateDB, log.TestingLogger(), proxyApp.Consensus(), mempool, evpool)
blkID := types.BlockID{blk.Hash(), blk.MakePartSet(testPartSize).Header()}
@ -620,7 +612,7 @@ func (bs *mockBlockStore) LoadBlock(height int64) *types.Block { return bs.chain
func (bs *mockBlockStore) LoadBlockMeta(height int64) *types.BlockMeta {
block := bs.chain[height-1]
return &types.BlockMeta{
BlockID: types.BlockID{block.Hash(), block.MakePartSet(bs.params.BlockPartSizeBytes).Header()},
BlockID: types.BlockID{block.Hash(), block.MakePartSet(types.BlockPartSizeBytes).Header()},
Header: block.Header,
}
}


+ 2
- 9
consensus/state.go View File

@ -950,22 +950,15 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
maxBytes := cs.state.ConsensusParams.BlockSize.MaxBytes
// bound evidence to 1/10th of the block
evidence := cs.evpool.PendingEvidence(maxBytes / 10)
evidence := cs.evpool.PendingEvidence(types.MaxEvidenceBytesPerBlock(maxBytes))
// Mempool validated transactions
txs := cs.mempool.ReapMaxBytes(maxDataBytes(maxBytes, cs.state.Validators.Size(), len(evidence)))
txs := cs.mempool.ReapMaxBytes(types.MaxDataBytes(maxBytes, cs.state.Validators.Size(), len(evidence)))
proposerAddr := cs.privValidator.GetAddress()
block, parts := cs.state.MakeBlock(cs.Height, txs, commit, evidence, proposerAddr)
return block, parts
}
func maxDataBytes(maxBytes, valsCount, evidenceCount int) int {
return maxBytes -
types.MaxAminoOverheadForBlock -
types.MaxHeaderBytes -
(valsCount * types.MaxVoteBytes) -
(evidenceCount * types.MaxEvidenceBytes)
}
// Enter: `timeoutPropose` after entering Propose.
// Enter: proposal block and POL is ready.


+ 7
- 7
consensus/state_test.go View File

@ -184,7 +184,7 @@ func TestStateBadProposal(t *testing.T) {
height, round := cs1.Height, cs1.Round
vs2 := vss[1]
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
voteCh := subscribe(cs1.eventBus, types.EventQueryVote)
@ -339,7 +339,7 @@ func TestStateLockNoPOL(t *testing.T) {
vs2 := vss[1]
height := cs1.Height
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
partSize := types.BlockPartSizeBytes
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
@ -507,7 +507,7 @@ func TestStateLockPOLRelock(t *testing.T) {
cs1, vss := randConsensusState(4)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
partSize := types.BlockPartSizeBytes
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)
@ -622,7 +622,7 @@ func TestStateLockPOLUnlock(t *testing.T) {
cs1, vss := randConsensusState(4)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
@ -719,7 +719,7 @@ func TestStateLockPOLSafety1(t *testing.T) {
cs1, vss := randConsensusState(4)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
@ -842,7 +842,7 @@ func TestStateLockPOLSafety2(t *testing.T) {
cs1, vss := randConsensusState(4)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutProposeCh := subscribe(cs1.eventBus, types.EventQueryTimeoutPropose)
@ -1021,7 +1021,7 @@ func TestStateHalt1(t *testing.T) {
cs1, vss := randConsensusState(4)
vs2, vs3, vs4 := vss[1], vss[2], vss[3]
partSize := cs1.state.ConsensusParams.BlockPartSizeBytes
partSize := types.BlockPartSizeBytes
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
timeoutWaitCh := subscribe(cs1.eventBus, types.EventQueryTimeoutWait)


+ 1
- 1
consensus/wal.go View File

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


+ 5
- 3
node/node.go View File

@ -241,13 +241,16 @@ func NewNode(config *cfg.Config,
csMetrics, p2pMetrics, memplMetrics := metricsProvider()
// Make MempoolReactor
maxBytes := state.ConsensusParams.TxSize.MaxBytes
maxDataBytes := types.MaxDataBytesUnknownEvidence(
state.ConsensusParams.BlockSize.MaxBytes,
state.Validators.Size(),
)
mempool := mempl.NewMempool(
config.Mempool,
proxyApp.Mempool(),
state.LastBlockHeight,
mempl.WithMetrics(memplMetrics),
mempl.WithFilter(func(tx types.Tx) bool { return len(tx) <= maxBytes }),
mempl.WithFilter(func(tx types.Tx) bool { return len(tx) <= maxDataBytes }),
)
mempoolLogger := logger.With("module", "mempool")
mempool.SetLogger(mempoolLogger)
@ -751,7 +754,6 @@ func saveGenesisDoc(db dbm.DB, genDoc *types.GenesisDoc) {
db.SetSync(genesisDocKey, bytes)
}
// splitAndTrimEmpty slices s into all subslices separated by sep and returns a
// slice of the string s with all leading and trailing Unicode code points
// contained in cutset removed. If sep is empty, SplitAndTrim splits after each


+ 5
- 2
state/execution.go View File

@ -145,8 +145,11 @@ func (blockExec *BlockExecutor) Commit(state State, block *types.Block) ([]byte,
"appHash", fmt.Sprintf("%X", res.Data))
// Update mempool.
maxBytes := state.ConsensusParams.TxSize.MaxBytes
filter := func(tx types.Tx) bool { return len(tx) <= maxBytes }
maxDataBytes := types.MaxDataBytesUnknownEvidence(
state.ConsensusParams.BlockSize.MaxBytes,
state.Validators.Size(),
)
filter := func(tx types.Tx) bool { return len(tx) <= maxDataBytes }
if err := blockExec.mempool.Update(block.Height, block.Txs, filter); err != nil {
return nil, err
}


+ 1
- 1
state/state.go View File

@ -139,7 +139,7 @@ func (state State) MakeBlock(
// IncrementAccum for rounds there.
block.ProposerAddress = proposerAddress
return block, block.MakePartSet(state.ConsensusParams.BlockGossip.BlockPartSizeBytes)
return block, block.MakePartSet(types.BlockPartSizeBytes)
}
// MedianTime computes a median time for a given Commit (based on Timestamp field of votes messages) and the


+ 10
- 27
state/state_test.go View File

@ -373,18 +373,14 @@ func TestConsensusParamsChangesSaveLoad(t *testing.T) {
}
}
func makeParams(txsBytes, blockGas, txBytes, txGas, partSize int) types.ConsensusParams {
func makeParams(txsBytes, blockGas, evidenceAge int) types.ConsensusParams {
return types.ConsensusParams{
BlockSize: types.BlockSize{
MaxBytes: txsBytes,
MaxGas: int64(blockGas),
},
TxSize: types.TxSize{
MaxBytes: txBytes,
MaxGas: int64(txGas),
},
BlockGossip: types.BlockGossip{
BlockPartSizeBytes: partSize,
EvidenceParams: types.EvidenceParams{
MaxAge: int64(evidenceAge),
},
}
}
@ -394,7 +390,7 @@ func pk() []byte {
}
func TestApplyUpdates(t *testing.T) {
initParams := makeParams(1, 2, 3, 4, 5)
initParams := makeParams(1, 2, 3)
cases := [...]struct {
init types.ConsensusParams
@ -404,33 +400,20 @@ func TestApplyUpdates(t *testing.T) {
0: {initParams, abci.ConsensusParams{}, initParams},
1: {initParams, abci.ConsensusParams{}, initParams},
2: {initParams,
abci.ConsensusParams{
TxSize: &abci.TxSize{
MaxBytes: 123,
},
},
makeParams(1, 2, 123, 4, 5)},
3: {initParams,
abci.ConsensusParams{
BlockSize: &abci.BlockSize{
MaxBytes: 1,
MaxBytes: 44,
MaxGas: 55,
},
},
makeParams(1, 55, 3, 4, 5)},
4: {initParams,
makeParams(44, 55, 3)},
3: {initParams,
abci.ConsensusParams{
BlockSize: &abci.BlockSize{
MaxBytes: 1,
},
TxSize: &abci.TxSize{
MaxGas: 888,
},
BlockGossip: &abci.BlockGossip{
BlockPartSizeBytes: 2002,
EvidenceParams: &abci.EvidenceParams{
MaxAge: 66,
},
},
makeParams(1, 2, 3, 888, 2002)},
makeParams(1, 2, 66)},
}
for i, tc := range cases {


+ 22
- 0
types/block.go View File

@ -204,6 +204,28 @@ func (b *Block) StringShort() string {
//-----------------------------------------------------------------------------
// MaxDataBytes returns the maximum size of block's data.
func MaxDataBytes(maxBytes, valsCount, evidenceCount int) int {
return maxBytes -
MaxAminoOverheadForBlock -
MaxHeaderBytes -
(valsCount * MaxVoteBytes) -
(evidenceCount * MaxEvidenceBytes)
}
// MaxDataBytesUnknownEvidence returns the maximum size of block's data when
// evidence count is unknown. MaxEvidenceBytesPerBlock will be used as the size
// of evidence.
func MaxDataBytesUnknownEvidence(maxBytes, valsCount int) int {
return maxBytes -
MaxAminoOverheadForBlock -
MaxHeaderBytes -
(valsCount * MaxVoteBytes) -
MaxEvidenceBytesPerBlock(maxBytes)
}
//-----------------------------------------------------------------------------
// Header defines the structure of a Tendermint block header
// TODO: limit header size
// NOTE: changes to the Header should be duplicated in the abci Header


+ 5
- 0
types/evidence.go View File

@ -52,6 +52,11 @@ func RegisterEvidences(cdc *amino.Codec) {
cdc.RegisterConcrete(MockBadEvidence{}, "tendermint/MockBadEvidence", nil)
}
// MaxEvidenceBytesPerBlock returns the maximum evidence size per block.
func MaxEvidenceBytesPerBlock(blockMaxBytes int) int {
return blockMaxBytes / 10
}
//-------------------------------------------
// DuplicateVoteEvidence contains evidence a validator signed two conflicting votes.


+ 31
- 69
types/params.go View File

@ -9,34 +9,24 @@ import (
const (
// MaxBlockSizeBytes is the maximum permitted size of the blocks.
MaxBlockSizeBytes = 104857600 // 100MB
// BlockPartSizeBytes is the size of one block part.
BlockPartSizeBytes = 65536 // 64kB
)
// ConsensusParams contains consensus critical parameters
// that determine the validity of blocks.
// ConsensusParams contains consensus critical parameters that determine the
// validity of blocks.
type ConsensusParams struct {
BlockSize `json:"block_size_params"`
TxSize `json:"tx_size_params"`
BlockGossip `json:"block_gossip_params"`
EvidenceParams `json:"evidence_params"`
}
// BlockSize contain limits on the block size.
type BlockSize struct {
MaxBytes int `json:"max_txs_bytes"` // NOTE: must not be 0 nor greater than 100MB
MaxGas int64 `json:"max_gas"`
}
// TxSize contain limits on the tx size.
type TxSize struct {
MaxBytes int `json:"max_bytes"`
MaxBytes int `json:"max_txs_bytes"`
MaxGas int64 `json:"max_gas"`
}
// 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
}
// EvidenceParams determine how we handle evidence of malfeasance
type EvidenceParams struct {
MaxAge int64 `json:"max_age"` // only accept new evidence more recent than this
@ -46,8 +36,6 @@ type EvidenceParams struct {
func DefaultConsensusParams() *ConsensusParams {
return &ConsensusParams{
DefaultBlockSize(),
DefaultTxSize(),
DefaultBlockGossip(),
DefaultEvidenceParams(),
}
}
@ -55,61 +43,49 @@ func DefaultConsensusParams() *ConsensusParams {
// DefaultBlockSize returns a default BlockSize.
func DefaultBlockSize() BlockSize {
return BlockSize{
MaxBytes: 22020096, // 21MB
MaxGas: -1,
}
}
// DefaultTxSize returns a default TxSize.
func DefaultTxSize() TxSize {
return TxSize{
MaxBytes: 10240, // 10kB
MaxBytes: 22020096, // 21MB
MaxGas: -1,
}
}
// DefaultBlockGossip returns a default BlockGossip.
func DefaultBlockGossip() BlockGossip {
return BlockGossip{
BlockPartSizeBytes: 65536, // 64kB,
}
}
// DefaultEvidence Params returns a default EvidenceParams.
// DefaultEvidenceParams Params returns a default EvidenceParams.
func DefaultEvidenceParams() EvidenceParams {
return EvidenceParams{
MaxAge: 100000, // 27.8 hrs at 1block/s
}
}
// Validate validates the ConsensusParams to ensure all values
// are within their allowed limits, and returns an error if they are not.
// 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.BlockSize.MaxBytes <= 0 {
return cmn.NewError("BlockSize.MaxBytes must be greater than 0. Got %d", params.BlockSize.MaxBytes)
}
if params.BlockGossip.BlockPartSizeBytes <= 0 {
return cmn.NewError("BlockGossip.BlockPartSizeBytes must be greater than 0. Got %d", params.BlockGossip.BlockPartSizeBytes)
return cmn.NewError("BlockSize.MaxBytes must be greater than 0. Got %d",
params.BlockSize.MaxBytes)
}
// ensure blocks aren't too big
if params.BlockSize.MaxBytes > MaxBlockSizeBytes {
return cmn.NewError("BlockSize.MaxBytes is too big. %d > %d",
params.BlockSize.MaxBytes, MaxBlockSizeBytes)
}
if params.BlockSize.MaxGas < -1 {
return cmn.NewError("BlockSize.MaxGas must be greater or equal to -1. Got %d",
params.BlockSize.MaxGas)
}
if params.EvidenceParams.MaxAge <= 0 {
return cmn.NewError("EvidenceParams.MaxAge must be greater than 0. Got %d",
params.EvidenceParams.MaxAge)
}
return nil
}
// Hash returns a merkle hash of the parameters to store
// in the block header
// Hash returns a merkle hash of the parameters to store in the block header
func (params *ConsensusParams) Hash() []byte {
return merkle.SimpleHashFromMap(map[string]merkle.Hasher{
"block_gossip_part_size_bytes": aminoHasher(params.BlockGossip.BlockPartSizeBytes),
"block_size_max_bytes": aminoHasher(params.BlockSize.MaxBytes),
"block_size_max_gas": aminoHasher(params.BlockSize.MaxGas),
"tx_size_max_bytes": aminoHasher(params.TxSize.MaxBytes),
"tx_size_max_gas": aminoHasher(params.TxSize.MaxGas),
"block_size_max_bytes": aminoHasher(params.BlockSize.MaxBytes),
"block_size_max_gas": aminoHasher(params.BlockSize.MaxGas),
"evidence_params_max_age": aminoHasher(params.EvidenceParams.MaxAge),
})
}
@ -126,25 +102,11 @@ func (params ConsensusParams) Update(params2 *abci.ConsensusParams) ConsensusPar
// XXX: it's cast city over here. It's ok because we only do int32->int
// but still, watch it champ.
if params2.BlockSize != nil {
if params2.BlockSize.MaxBytes > 0 {
res.BlockSize.MaxBytes = int(params2.BlockSize.MaxBytes)
}
if params2.BlockSize.MaxGas > 0 {
res.BlockSize.MaxGas = params2.BlockSize.MaxGas
}
}
if params2.TxSize != nil {
if params2.TxSize.MaxBytes > 0 {
res.TxSize.MaxBytes = int(params2.TxSize.MaxBytes)
}
if params2.TxSize.MaxGas > 0 {
res.TxSize.MaxGas = params2.TxSize.MaxGas
}
res.BlockSize.MaxBytes = int(params2.BlockSize.MaxBytes)
res.BlockSize.MaxGas = params2.BlockSize.MaxGas
}
if params2.BlockGossip != nil {
if params2.BlockGossip.BlockPartSizeBytes > 0 {
res.BlockGossip.BlockPartSizeBytes = int(params2.BlockGossip.BlockPartSizeBytes)
}
if params2.EvidenceParams != nil {
res.EvidenceParams.MaxAge = params2.EvidenceParams.MaxAge
}
return res
}

+ 30
- 57
types/params_test.go View File

@ -9,10 +9,10 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
)
func newConsensusParams(txsBytes, partSize int) ConsensusParams {
func newConsensusParams(txsBytes, evidenceAge int) ConsensusParams {
return ConsensusParams{
BlockSize: BlockSize{MaxBytes: txsBytes},
BlockGossip: BlockGossip{BlockPartSizeBytes: partSize},
BlockSize: BlockSize{MaxBytes: txsBytes},
EvidenceParams: EvidenceParams{MaxAge: int64(evidenceAge)},
}
}
@ -21,50 +21,45 @@ func TestConsensusParamsValidation(t *testing.T) {
params ConsensusParams
valid bool
}{
{newConsensusParams(1, 1), true},
{newConsensusParams(1, 0), false},
{newConsensusParams(0, 1), false},
{newConsensusParams(0, 0), false},
{newConsensusParams(0, 10), false},
{newConsensusParams(10, -1), false},
{newConsensusParams(47*1024*1024, 400), true},
{newConsensusParams(10, 400), true},
{newConsensusParams(100*1024*1024, 400), true},
{newConsensusParams(101*1024*1024, 400), false},
{newConsensusParams(1024*1024*1024, 400), false},
// test block size
0: {newConsensusParams(1, 1), true},
1: {newConsensusParams(0, 1), false},
2: {newConsensusParams(47*1024*1024, 1), true},
3: {newConsensusParams(10, 1), true},
4: {newConsensusParams(100*1024*1024, 1), true},
5: {newConsensusParams(101*1024*1024, 1), false},
6: {newConsensusParams(1024*1024*1024, 1), false},
7: {newConsensusParams(1024*1024*1024, -1), false},
// test evidence age
8: {newConsensusParams(1, 0), false},
9: {newConsensusParams(1, -1), false},
}
for _, tc := range testCases {
for i, tc := range testCases {
if tc.valid {
assert.NoError(t, tc.params.Validate(), "expected no error for valid params")
assert.NoErrorf(t, tc.params.Validate(), "expected no error for valid params (#%d)", i)
} else {
assert.Error(t, tc.params.Validate(), "expected error for non valid params")
assert.Errorf(t, tc.params.Validate(), "expected error for non valid params (#%d)", i)
}
}
}
func makeParams(txsBytes, blockGas, txBytes, txGas, partSize int) ConsensusParams {
func makeParams(txsBytes, blockGas, evidenceAge int) ConsensusParams {
return ConsensusParams{
BlockSize: BlockSize{
MaxBytes: txsBytes,
MaxGas: int64(blockGas),
},
TxSize: TxSize{
MaxBytes: txBytes,
MaxGas: int64(txGas),
},
BlockGossip: BlockGossip{
BlockPartSizeBytes: partSize,
EvidenceParams: EvidenceParams{
MaxAge: int64(evidenceAge),
},
}
}
func TestConsensusParamsHash(t *testing.T) {
params := []ConsensusParams{
makeParams(6, 2, 3, 4, 5),
makeParams(1, 6, 3, 4, 5),
makeParams(1, 2, 6, 4, 5),
makeParams(1, 2, 3, 6, 5),
makeParams(1, 2, 3, 4, 6),
makeParams(4, 2, 3),
makeParams(1, 4, 3),
makeParams(1, 2, 4),
}
hashes := make([][]byte, len(params))
@ -90,45 +85,23 @@ func TestConsensusParamsUpdate(t *testing.T) {
}{
// empty updates
{
makeParams(1, 2, 3, 4, 5),
makeParams(1, 2, 3),
&abci.ConsensusParams{},
makeParams(1, 2, 3, 4, 5),
},
// negative BlockPartSizeBytes
{
makeParams(1, 2, 3, 4, 5),
&abci.ConsensusParams{
BlockSize: &abci.BlockSize{
MaxBytes: -100,
MaxGas: -200,
},
TxSize: &abci.TxSize{
MaxBytes: -400,
MaxGas: -500,
},
BlockGossip: &abci.BlockGossip{
BlockPartSizeBytes: -600,
},
},
makeParams(1, 2, 3, 4, 5),
makeParams(1, 2, 3),
},
// fine updates
{
makeParams(1, 2, 3, 4, 5),
makeParams(1, 2, 3),
&abci.ConsensusParams{
BlockSize: &abci.BlockSize{
MaxBytes: 100,
MaxGas: 200,
},
TxSize: &abci.TxSize{
MaxBytes: 300,
MaxGas: 400,
},
BlockGossip: &abci.BlockGossip{
BlockPartSizeBytes: 500,
EvidenceParams: &abci.EvidenceParams{
MaxAge: 300,
},
},
makeParams(100, 200, 300, 400, 500),
makeParams(100, 200, 300),
},
}
for _, tc := range testCases {


+ 8
- 19
types/protobuf.go View File

@ -115,15 +115,11 @@ func (tm2pb) ValidatorUpdates(vals *ValidatorSet) []abci.ValidatorUpdate {
func (tm2pb) ConsensusParams(params *ConsensusParams) *abci.ConsensusParams {
return &abci.ConsensusParams{
BlockSize: &abci.BlockSize{
MaxBytes: int32(params.BlockSize.MaxBytes),
MaxGas: params.BlockSize.MaxGas,
MaxBytes: int32(params.BlockSize.MaxBytes),
MaxGas: params.BlockSize.MaxGas,
},
TxSize: &abci.TxSize{
MaxBytes: int32(params.TxSize.MaxBytes),
MaxGas: params.TxSize.MaxGas,
},
BlockGossip: &abci.BlockGossip{
BlockPartSizeBytes: int32(params.BlockGossip.BlockPartSizeBytes),
EvidenceParams: &abci.EvidenceParams{
MaxAge: params.EvidenceParams.MaxAge,
},
}
}
@ -215,18 +211,11 @@ func (pb2tm) ValidatorUpdates(vals []abci.ValidatorUpdate) ([]*Validator, error)
func (pb2tm) ConsensusParams(csp *abci.ConsensusParams) ConsensusParams {
return ConsensusParams{
BlockSize: BlockSize{
MaxBytes: int(csp.BlockSize.MaxBytes), // XXX
MaxGas: csp.BlockSize.MaxGas,
},
TxSize: TxSize{
MaxBytes: int(csp.TxSize.MaxBytes), // XXX
MaxGas: csp.TxSize.MaxGas,
MaxBytes: int(csp.BlockSize.MaxBytes), // XXX
MaxGas: csp.BlockSize.MaxGas,
},
BlockGossip: BlockGossip{
BlockPartSizeBytes: int(csp.BlockGossip.BlockPartSizeBytes), // XXX
EvidenceParams: EvidenceParams{
MaxAge: csp.EvidenceParams.MaxAge, // XXX
},
// TODO: EvidenceParams: EvidenceParams{
// MaxAge: int(csp.Evidence.MaxAge), // XXX
// },
}
}

Loading…
Cancel
Save