Skip to content

Commit

Permalink
Merge pull request #1832 from nspcc-dev/consensus/microfixes
Browse files Browse the repository at this point in the history
consensus: add MaxBlockSize and MaxBlockSysFee settings
  • Loading branch information
roman-khimov authored Mar 16, 2021
2 parents dcc0274 + 20b70ee commit 025330f
Show file tree
Hide file tree
Showing 7 changed files with 192 additions and 30 deletions.
4 changes: 4 additions & 0 deletions pkg/config/protocol_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,10 @@ type (
KeepOnlyLatestState bool `yaml:"KeepOnlyLatestState"`
// RemoveUntraceableBlocks specifies if old blocks should be removed.
RemoveUntraceableBlocks bool `yaml:"RemoveUntraceableBlocks"`
// MaxBlockSize is the maximum block size in bytes.
MaxBlockSize uint32 `yaml:"MaxBlockSize"`
// MaxBlockSystemFee is the maximum overall system fee per block.
MaxBlockSystemFee int64 `yaml:"MaxBlockSystemFee"`
// MaxTraceableBlocks is the length of the chain accessible to smart contracts.
MaxTraceableBlocks uint32 `yaml:"MaxTraceableBlocks"`
// MaxTransactionsPerBlock is the maximum amount of transactions per block.
Expand Down
59 changes: 39 additions & 20 deletions pkg/consensus/consensus.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,9 +73,6 @@ type service struct {
blockEvents chan *coreb.Block
lastProposal []util.Uint256
wallet *wallet.Wallet
network netmode.Magic
// stateRootEnabled specifies if state root should be exchanged and checked during consensus.
stateRootEnabled bool
// started is a flag set with Start method that runs an event handling
// goroutine.
started *atomic.Bool
Expand All @@ -97,6 +94,8 @@ type Config struct {
Broadcast func(p *npayload.Extensible)
// Chain is a core.Blockchainer instance.
Chain blockchainer.Blockchainer
// ProtocolConfiguration contains protocol settings.
ProtocolConfiguration config.ProtocolConfiguration
// RequestTx is a callback to which will be called
// when a node lacks transactions present in a block.
RequestTx func(h ...util.Uint256)
Expand All @@ -123,13 +122,11 @@ func NewService(cfg Config) (Service, error) {
txx: newFIFOCache(cacheMaxCapacity),
messages: make(chan Payload, 100),

transactions: make(chan *transaction.Transaction, 100),
blockEvents: make(chan *coreb.Block, 1),
network: cfg.Chain.GetConfig().Magic,
stateRootEnabled: cfg.Chain.GetConfig().StateRootInHeader,
started: atomic.NewBool(false),
quit: make(chan struct{}),
finished: make(chan struct{}),
transactions: make(chan *transaction.Transaction, 100),
blockEvents: make(chan *coreb.Block, 1),
started: atomic.NewBool(false),
quit: make(chan struct{}),
finished: make(chan struct{}),
}

if cfg.Wallet == nil {
Expand Down Expand Up @@ -182,7 +179,7 @@ func NewService(cfg Config) (Service, error) {
dbft.WithNewCommit(func() payload.Commit { return new(commit) }),
dbft.WithNewRecoveryRequest(func() payload.RecoveryRequest { return new(recoveryRequest) }),
dbft.WithNewRecoveryMessage(func() payload.RecoveryMessage {
return &recoveryMessage{stateRootEnabled: srv.stateRootEnabled}
return &recoveryMessage{stateRootEnabled: srv.ProtocolConfiguration.StateRootInHeader}
}),
dbft.WithVerifyPrepareRequest(srv.verifyRequest),
dbft.WithVerifyPrepareResponse(func(_ payload.ConsensusPayload) error { return nil }),
Expand Down Expand Up @@ -214,7 +211,7 @@ func NewPayload(m netmode.Magic, stateRootEnabled bool) *Payload {
}

func (s *service) newPayload(c *dbft.Context, t payload.MessageType, msg interface{}) payload.ConsensusPayload {
cp := NewPayload(s.network, s.stateRootEnabled)
cp := NewPayload(s.ProtocolConfiguration.Magic, s.ProtocolConfiguration.StateRootInHeader)
cp.SetHeight(c.BlockIndex)
cp.SetValidatorIndex(uint16(c.MyIndex))
cp.SetViewNumber(c.ViewNumber)
Expand All @@ -234,7 +231,7 @@ func (s *service) newPayload(c *dbft.Context, t payload.MessageType, msg interfa

func (s *service) newPrepareRequest() payload.PrepareRequest {
r := new(prepareRequest)
if s.stateRootEnabled {
if s.ProtocolConfiguration.StateRootInHeader {
r.stateRootEnabled = true
if sr, err := s.Chain.GetStateModule().GetStateRoot(s.dbft.BlockIndex - 1); err == nil {
r.stateRoot = sr.Root
Expand Down Expand Up @@ -364,7 +361,7 @@ func (s *service) payloadFromExtensible(ep *npayload.Extensible) *Payload {
return &Payload{
Extensible: *ep,
message: message{
stateRootEnabled: s.stateRootEnabled,
stateRootEnabled: s.ProtocolConfiguration.StateRootInHeader,
},
}
}
Expand Down Expand Up @@ -440,11 +437,21 @@ func (s *service) verifyBlock(b block.Block) bool {
return false
}

size := coreb.GetExpectedBlockSize()
if size > int(s.ProtocolConfiguration.MaxBlockSize) {
s.log.Warn("proposed block size exceeds config MaxBlockSize",
zap.Uint32("max size allowed", s.ProtocolConfiguration.MaxBlockSize),
zap.Int("block size", size))
return false
}

var fee int64
var pool = mempool.New(len(coreb.Transactions), 0, false)
var mainPool = s.Chain.GetMemPool()
for _, tx := range coreb.Transactions {
var err error

fee += tx.SystemFee
if mainPool.ContainsKey(tx.Hash()) {
err = pool.Add(tx, s.Chain)
if err == nil {
Expand All @@ -465,13 +472,22 @@ func (s *service) verifyBlock(b block.Block) bool {
}
}

maxBlockSysFee := s.ProtocolConfiguration.MaxBlockSystemFee
if fee > maxBlockSysFee {
s.log.Warn("proposed block system fee exceeds config MaxBlockSystemFee",
zap.Int("max system fee allowed", int(maxBlockSysFee)),
zap.Int("block system fee", int(fee)))
return false
}

return true
}

var (
errInvalidPrevHash = errors.New("invalid PrevHash")
errInvalidVersion = errors.New("invalid Version")
errInvalidStateRoot = errors.New("state root mismatch")
errInvalidPrevHash = errors.New("invalid PrevHash")
errInvalidVersion = errors.New("invalid Version")
errInvalidStateRoot = errors.New("state root mismatch")
errInvalidTransactionsCount = errors.New("invalid transactions count")
)

func (s *service) verifyRequest(p payload.ConsensusPayload) error {
Expand All @@ -482,14 +498,17 @@ func (s *service) verifyRequest(p payload.ConsensusPayload) error {
if req.version != s.dbft.Version {
return errInvalidVersion
}
if s.stateRootEnabled {
if s.ProtocolConfiguration.StateRootInHeader {
sr, err := s.Chain.GetStateModule().GetStateRoot(s.dbft.BlockIndex - 1)
if err != nil {
return err
} else if sr.Root != req.stateRoot {
return fmt.Errorf("%w: %s != %s", errInvalidStateRoot, sr.Root, req.stateRoot)
}
}
if len(req.TransactionHashes()) > int(s.ProtocolConfiguration.MaxTransactionsPerBlock) {
return fmt.Errorf("%w: max = %d, got %d", errInvalidTransactionsCount, s.ProtocolConfiguration.MaxTransactionsPerBlock, len(req.TransactionHashes()))
}
// Save lastProposal for getVerified().
s.lastProposal = req.transactionHashes

Expand Down Expand Up @@ -633,10 +652,10 @@ func (s *service) newBlockFromContext(ctx *dbft.Context) block.Block {
return nil
}

block.Block.Network = s.network
block.Block.Network = s.ProtocolConfiguration.Magic
block.Block.Timestamp = ctx.Timestamp / nsInMs
block.Block.Index = ctx.BlockIndex
if s.stateRootEnabled {
if s.ProtocolConfiguration.StateRootInHeader {
sr, err := s.Chain.GetStateModule().GetStateRoot(ctx.BlockIndex - 1)
if err != nil {
return nil
Expand Down
39 changes: 34 additions & 5 deletions pkg/consensus/consensus_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -323,6 +323,13 @@ func TestService_PrepareRequest(t *testing.T) {

sr, err := srv.Chain.GetStateModule().GetStateRoot(srv.dbft.BlockIndex - 1)
require.NoError(t, err)

checkRequest(t, errInvalidTransactionsCount, &prepareRequest{stateRootEnabled: true,
prevHash: prevHash,
stateRoot: sr.Root,
transactionHashes: make([]util.Uint256, srv.ProtocolConfiguration.MaxTransactionsPerBlock+1),
})

checkRequest(t, nil, &prepareRequest{
stateRootEnabled: true,
prevHash: prevHash,
Expand Down Expand Up @@ -402,6 +409,16 @@ func TestVerifyBlock(t *testing.T) {
b.Index = srv.Chain.BlockHeight()
require.False(t, srv.verifyBlock(&neoBlock{Block: *b}))
})
t.Run("bad big size", func(t *testing.T) {
script := make([]byte, int(srv.ProtocolConfiguration.MaxBlockSize))
script[0] = byte(opcode.RET)
tx := transaction.New(netmode.UnitTestNet, script, 100000)
tx.ValidUntilBlock = 1
addSender(t, tx)
signTx(t, srv.Chain, tx)
b := testchain.NewBlock(t, srv.Chain, 1, 0, tx)
require.False(t, srv.verifyBlock(&neoBlock{Block: *b}))
})
t.Run("bad timestamp", func(t *testing.T) {
b := testchain.NewBlock(t, srv.Chain, 1, 0)
b.Timestamp = srv.lastTimestamp - 1
Expand All @@ -416,6 +433,17 @@ func TestVerifyBlock(t *testing.T) {
b := testchain.NewBlock(t, srv.Chain, 1, 0, tx)
require.False(t, srv.verifyBlock(&neoBlock{Block: *b}))
})
t.Run("bad big sys fee", func(t *testing.T) {
txes := make([]*transaction.Transaction, 2)
for i := range txes {
txes[i] = transaction.New(netmode.UnitTestNet, []byte{byte(opcode.RET)}, srv.ProtocolConfiguration.MaxBlockSystemFee/2+1)
txes[i].ValidUntilBlock = 1
addSender(t, txes[i])
signTx(t, srv.Chain, txes[i])
}
b := testchain.NewBlock(t, srv.Chain, 1, 0, txes...)
require.False(t, srv.verifyBlock(&neoBlock{Block: *b}))
})
}

func shouldReceive(t *testing.T, ch chan Payload) {
Expand Down Expand Up @@ -444,11 +472,12 @@ func newTestService(t *testing.T) *service {

func newTestServiceWithChain(t *testing.T, bc *core.Blockchain) *service {
srv, err := NewService(Config{
Logger: zaptest.NewLogger(t),
Broadcast: func(*npayload.Extensible) {},
Chain: bc,
RequestTx: func(...util.Uint256) {},
TimePerBlock: time.Duration(bc.GetConfig().SecondsPerBlock) * time.Second,
Logger: zaptest.NewLogger(t),
Broadcast: func(*npayload.Extensible) {},
Chain: bc,
ProtocolConfiguration: bc.GetConfig(),
RequestTx: func(...util.Uint256) {},
TimePerBlock: time.Duration(bc.GetConfig().SecondsPerBlock) * time.Second,
Wallet: &config.Wallet{
Path: "./testdata/wallet1.json",
Password: "one",
Expand Down
26 changes: 26 additions & 0 deletions pkg/core/block/block.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,12 @@ const (
// ErrMaxContentsPerBlock is returned when the maximum number of contents per block is reached.
var ErrMaxContentsPerBlock = errors.New("the number of contents exceeds the maximum number of contents per block")

var expectedHeaderSizeWithEmptyWitness int

func init() {
expectedHeaderSizeWithEmptyWitness = io.GetVarSize(new(Header))
}

// Block represents one block in the chain.
type Block struct {
// The base of the block.
Expand Down Expand Up @@ -211,3 +217,23 @@ func (b *Block) UnmarshalJSON(data []byte) error {
}
return nil
}

// GetExpectedBlockSize returns expected block size which should be equal to io.GetVarSize(b)
func (b *Block) GetExpectedBlockSize() int {
var transactionsSize int
for _, tx := range b.Transactions {
transactionsSize += tx.Size()
}
return b.GetExpectedBlockSizeWithoutTransactions(len(b.Transactions)) + transactionsSize
}

// GetExpectedBlockSizeWithoutTransactions returns expected block size without transactions size.
func (b *Block) GetExpectedBlockSizeWithoutTransactions(txCount int) int {
size := expectedHeaderSizeWithEmptyWitness - 1 - 1 + // 1 is for the zero-length (new(Header)).Script.Invocation/Verification
io.GetVarSize(&b.Script) +
io.GetVarSize(txCount)
if b.StateRootEnabled {
size += util.Uint256Size
}
return size
}
44 changes: 44 additions & 0 deletions pkg/core/block/block_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (
"github.com/nspcc-dev/neo-go/pkg/crypto/hash"
"github.com/nspcc-dev/neo-go/pkg/encoding/address"
"github.com/nspcc-dev/neo-go/pkg/io"
"github.com/nspcc-dev/neo-go/pkg/util"
"github.com/nspcc-dev/neo-go/pkg/vm/opcode"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -234,3 +235,46 @@ func TestBlockEncodeDecode(t *testing.T) {
require.True(t, errors.Is(testserdes.DecodeBinary(data, new(Block)), ErrMaxContentsPerBlock))
})
}

func TestGetExpectedBlockSize(t *testing.T) {
check := func(t *testing.T, stateRootEnabled bool) {

t.Run("without transactions", func(t *testing.T) {
b := newDumbBlock()
b.StateRootEnabled = stateRootEnabled
b.Transactions = []*transaction.Transaction{}
require.Equal(t, io.GetVarSize(b), b.GetExpectedBlockSize())
require.Equal(t, io.GetVarSize(b), b.GetExpectedBlockSizeWithoutTransactions(0))
})
t.Run("with one transaction", func(t *testing.T) {
b := newDumbBlock()
b.StateRootEnabled = stateRootEnabled
expected := io.GetVarSize(b)
require.Equal(t, expected, b.GetExpectedBlockSize())
require.Equal(t, expected-b.Transactions[0].Size(), b.GetExpectedBlockSizeWithoutTransactions(len(b.Transactions)))
})
t.Run("with multiple transactions", func(t *testing.T) {
b := newDumbBlock()
b.StateRootEnabled = stateRootEnabled
b.Transactions = make([]*transaction.Transaction, 123)
for i := range b.Transactions {
tx := transaction.New(netmode.UnitTestNet, []byte{byte(opcode.RET)}, int64(i))
tx.Signers = []transaction.Signer{{Account: util.Uint160{1, 2, 3}}}
tx.Scripts = []transaction.Witness{{}}
b.Transactions[i] = tx
}
expected := io.GetVarSize(b)
require.Equal(t, expected, b.GetExpectedBlockSize())
for _, tx := range b.Transactions {
expected -= tx.Size()
}
require.Equal(t, expected, b.GetExpectedBlockSizeWithoutTransactions(len(b.Transactions)))
})
}
t.Run("StateRoot enabled", func(t *testing.T) {
check(t, true)
})
t.Run("StateRoot disabled", func(t *testing.T) {
check(t, false)
})
}
39 changes: 39 additions & 0 deletions pkg/core/blockchain.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,8 @@ const (

defaultMemPoolSize = 50000
defaultP2PNotaryRequestPayloadPoolSize = 1000
defaultMaxBlockSize = 262144
defaultMaxBlockSystemFee = 900000000000
defaultMaxTraceableBlocks = 2102400 // 1 year of 15s blocks
defaultMaxTransactionsPerBlock = 512
verificationGasLimit = 100000000 // 1 GAS
Expand Down Expand Up @@ -134,6 +136,10 @@ type Blockchain struct {

extensible atomic.Value

// defaultBlockWitness stores transaction.Witness with m out of n multisig,
// where n = ValidatorsCount.
defaultBlockWitness atomic.Value

stateRoot *stateroot.Module

// Notification subsystem.
Expand Down Expand Up @@ -167,6 +173,14 @@ func NewBlockchain(s storage.Store, cfg config.ProtocolConfiguration, log *zap.L
cfg.P2PNotaryRequestPayloadPoolSize = defaultP2PNotaryRequestPayloadPoolSize
log.Info("P2PNotaryRequestPayloadPool size is not set or wrong, setting default value", zap.Int("P2PNotaryRequestPayloadPoolSize", cfg.P2PNotaryRequestPayloadPoolSize))
}
if cfg.MaxBlockSize == 0 {
cfg.MaxBlockSize = defaultMaxBlockSize
log.Info("MaxBlockSize is not set or wrong, setting default value", zap.Uint32("MaxBlockSize", cfg.MaxBlockSize))
}
if cfg.MaxBlockSystemFee <= 0 {
cfg.MaxBlockSystemFee = defaultMaxBlockSystemFee
log.Info("MaxBlockSystemFee is not set or wrong, setting default value", zap.Int64("MaxBlockSystemFee", cfg.MaxBlockSystemFee))
}
if cfg.MaxTraceableBlocks == 0 {
cfg.MaxTraceableBlocks = defaultMaxTraceableBlocks
log.Info("MaxTraceableBlocks is not set or wrong, using default value", zap.Uint32("MaxTraceableBlocks", cfg.MaxTraceableBlocks))
Expand Down Expand Up @@ -1339,6 +1353,31 @@ func (bc *Blockchain) ApplyPolicyToTxSet(txes []*transaction.Transaction) []*tra
if maxTx != 0 && len(txes) > int(maxTx) {
txes = txes[:maxTx]
}
maxBlockSize := bc.GetConfig().MaxBlockSize
maxBlockSysFee := bc.GetConfig().MaxBlockSystemFee
defaultWitness := bc.defaultBlockWitness.Load()
if defaultWitness == nil {
m := smartcontract.GetDefaultHonestNodeCount(bc.config.ValidatorsCount)
verification, _ := smartcontract.CreateDefaultMultiSigRedeemScript(bc.contracts.NEO.GetNextBlockValidatorsInternal())
defaultWitness = transaction.Witness{
InvocationScript: make([]byte, 66*m),
VerificationScript: verification,
}
bc.defaultBlockWitness.Store(defaultWitness)
}
var (
b = &block.Block{Header: block.Header{Script: defaultWitness.(transaction.Witness)}}
blockSize = uint32(b.GetExpectedBlockSizeWithoutTransactions(len(txes)))
blockSysFee int64
)
for i, tx := range txes {
blockSize += uint32(tx.Size())
blockSysFee += tx.SystemFee
if blockSize > maxBlockSize || blockSysFee > maxBlockSysFee {
txes = txes[:i]
break
}
}
return txes
}

Expand Down
Loading

0 comments on commit 025330f

Please sign in to comment.