Merge pull request #1759 from nspcc-dev/config/policy

Move some Policy settings to config
This commit is contained in:
Roman Khimov 2021-03-05 00:22:02 +03:00 committed by GitHub
commit 39817aae46
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
12 changed files with 21 additions and 364 deletions

View file

@ -164,16 +164,6 @@ func (chain *FakeChain) P2PSigExtensionsEnabled() bool {
return true return true
} }
// GetMaxBlockSystemFee implements Policer interface.
func (chain *FakeChain) GetMaxBlockSystemFee() int64 {
panic("TODO")
}
// GetMaxBlockSize implements Policer interface.
func (chain *FakeChain) GetMaxBlockSize() uint32 {
panic("TODO")
}
// AddHeaders implements Blockchainer interface. // AddHeaders implements Blockchainer interface.
func (chain *FakeChain) AddHeaders(...*block.Header) error { func (chain *FakeChain) AddHeaders(...*block.Header) error {
panic("TODO") panic("TODO")

View file

@ -118,16 +118,10 @@ func TestNativeHelpersCompile(t *testing.T) {
{"blockAccount", []string{u160}}, {"blockAccount", []string{u160}},
{"getExecFeeFactor", nil}, {"getExecFeeFactor", nil},
{"getFeePerByte", nil}, {"getFeePerByte", nil},
{"getMaxBlockSize", nil},
{"getMaxBlockSystemFee", nil},
{"getMaxTransactionsPerBlock", nil},
{"getStoragePrice", nil}, {"getStoragePrice", nil},
{"isBlocked", []string{u160}}, {"isBlocked", []string{u160}},
{"setExecFeeFactor", []string{"42"}}, {"setExecFeeFactor", []string{"42"}},
{"setFeePerByte", []string{"42"}}, {"setFeePerByte", []string{"42"}},
{"setMaxBlockSize", []string{"42"}},
{"setMaxBlockSystemFee", []string{"42"}},
{"setMaxTransactionsPerBlock", []string{"42"}},
{"setStoragePrice", []string{"42"}}, {"setStoragePrice", []string{"42"}},
{"unblockAccount", []string{u160}}, {"unblockAccount", []string{u160}},
}) })

View file

@ -20,6 +20,8 @@ type (
RemoveUntraceableBlocks bool `yaml:"RemoveUntraceableBlocks"` RemoveUntraceableBlocks bool `yaml:"RemoveUntraceableBlocks"`
// MaxTraceableBlocks is the length of the chain accessible to smart contracts. // MaxTraceableBlocks is the length of the chain accessible to smart contracts.
MaxTraceableBlocks uint32 `yaml:"MaxTraceableBlocks"` MaxTraceableBlocks uint32 `yaml:"MaxTraceableBlocks"`
// MaxTransactionsPerBlock is the maximum amount of transactions per block.
MaxTransactionsPerBlock uint16 `yaml:"MaxTransactionsPerBlock"`
// P2PSigExtensions enables additional signature-related logic. // P2PSigExtensions enables additional signature-related logic.
P2PSigExtensions bool `yaml:"P2PSigExtensions"` P2PSigExtensions bool `yaml:"P2PSigExtensions"`
// ReservedAttributes allows to have reserved attributes range for experimental or private purposes. // ReservedAttributes allows to have reserved attributes range for experimental or private purposes.

View file

@ -439,22 +439,12 @@ func (s *service) verifyBlock(b block.Block) bool {
zap.Uint64("last", s.lastTimestamp)) zap.Uint64("last", s.lastTimestamp))
return false return false
} }
maxBlockSize := int(s.Chain.GetPolicer().GetMaxBlockSize())
size := io.GetVarSize(coreb)
if size > maxBlockSize {
s.log.Warn("proposed block size exceeds policy max block size",
zap.Int("max size allowed", maxBlockSize),
zap.Int("block size", size))
return false
}
var fee int64
var pool = mempool.New(len(coreb.Transactions), 0, false) var pool = mempool.New(len(coreb.Transactions), 0, false)
var mainPool = s.Chain.GetMemPool() var mainPool = s.Chain.GetMemPool()
for _, tx := range coreb.Transactions { for _, tx := range coreb.Transactions {
var err error var err error
fee += tx.SystemFee
if mainPool.ContainsKey(tx.Hash()) { if mainPool.ContainsKey(tx.Hash()) {
err = pool.Add(tx, s.Chain) err = pool.Add(tx, s.Chain)
if err == nil { if err == nil {
@ -475,14 +465,6 @@ func (s *service) verifyBlock(b block.Block) bool {
} }
} }
maxBlockSysFee := s.Chain.GetPolicer().GetMaxBlockSystemFee()
if fee > maxBlockSysFee {
s.log.Warn("proposed block system fee exceeds policy max block system fee",
zap.Int("max system fee allowed", int(maxBlockSysFee)),
zap.Int("block system fee", int(fee)))
return false
}
return true return true
} }

View file

@ -407,16 +407,6 @@ func TestVerifyBlock(t *testing.T) {
b.Timestamp = srv.lastTimestamp - 1 b.Timestamp = srv.lastTimestamp - 1
require.False(t, srv.verifyBlock(&neoBlock{Block: *b})) require.False(t, srv.verifyBlock(&neoBlock{Block: *b}))
}) })
t.Run("bad big size", func(t *testing.T) {
script := make([]byte, int(srv.Chain.GetPolicer().GetMaxBlockSize()))
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 tx", func(t *testing.T) { t.Run("bad tx", func(t *testing.T) {
tx := transaction.New(netmode.UnitTestNet, []byte{byte(opcode.RET)}, 100000) tx := transaction.New(netmode.UnitTestNet, []byte{byte(opcode.RET)}, 100000)
tx.ValidUntilBlock = 1 tx.ValidUntilBlock = 1
@ -426,17 +416,6 @@ func TestVerifyBlock(t *testing.T) {
b := testchain.NewBlock(t, srv.Chain, 1, 0, tx) b := testchain.NewBlock(t, srv.Chain, 1, 0, tx)
require.False(t, srv.verifyBlock(&neoBlock{Block: *b})) 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.Chain.GetPolicer().GetMaxBlockSystemFee()/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) { func shouldReceive(t *testing.T, ch chan Payload) {

View file

@ -47,6 +47,7 @@ const (
defaultMemPoolSize = 50000 defaultMemPoolSize = 50000
defaultP2PNotaryRequestPayloadPoolSize = 1000 defaultP2PNotaryRequestPayloadPoolSize = 1000
defaultMaxTraceableBlocks = 2102400 // 1 year of 15s blocks defaultMaxTraceableBlocks = 2102400 // 1 year of 15s blocks
defaultMaxTransactionsPerBlock = 512
verificationGasLimit = 100000000 // 1 GAS verificationGasLimit = 100000000 // 1 GAS
) )
@ -168,6 +169,11 @@ func NewBlockchain(s storage.Store, cfg config.ProtocolConfiguration, log *zap.L
cfg.MaxTraceableBlocks = defaultMaxTraceableBlocks cfg.MaxTraceableBlocks = defaultMaxTraceableBlocks
log.Info("MaxTraceableBlocks is not set or wrong, using default value", zap.Uint32("MaxTraceableBlocks", cfg.MaxTraceableBlocks)) log.Info("MaxTraceableBlocks is not set or wrong, using default value", zap.Uint32("MaxTraceableBlocks", cfg.MaxTraceableBlocks))
} }
if cfg.MaxTransactionsPerBlock == 0 {
cfg.MaxTransactionsPerBlock = defaultMaxTransactionsPerBlock
log.Info("MaxTransactionsPerBlock is not set or wrong, using default value",
zap.Uint16("MaxTransactionsPerBlock", cfg.MaxTransactionsPerBlock))
}
committee, err := committeeFromConfig(cfg) committee, err := committeeFromConfig(cfg)
if err != nil { if err != nil {
return nil, err return nil, err
@ -1347,25 +1353,10 @@ func (bc *Blockchain) GetMemPool() *mempool.Pool {
// ApplyPolicyToTxSet applies configured policies to given transaction set. It // ApplyPolicyToTxSet applies configured policies to given transaction set. It
// expects slice to be ordered by fee and returns a subslice of it. // expects slice to be ordered by fee and returns a subslice of it.
func (bc *Blockchain) ApplyPolicyToTxSet(txes []*transaction.Transaction) []*transaction.Transaction { func (bc *Blockchain) ApplyPolicyToTxSet(txes []*transaction.Transaction) []*transaction.Transaction {
maxTx := bc.contracts.Policy.GetMaxTransactionsPerBlockInternal(bc.dao) maxTx := bc.config.MaxTransactionsPerBlock
if maxTx != 0 && len(txes) > int(maxTx) { if maxTx != 0 && len(txes) > int(maxTx) {
txes = txes[:maxTx] txes = txes[:maxTx]
} }
maxBlockSize := bc.contracts.Policy.GetMaxBlockSizeInternal(bc.dao)
maxBlockSysFee := bc.contracts.Policy.GetMaxBlockSystemFeeInternal(bc.dao)
var (
blockSize uint32
sysFee int64
)
blockSize = uint32(io.GetVarSize(new(block.Block)) + io.GetVarSize(len(txes)+1))
for i, tx := range txes {
blockSize += uint32(tx.Size())
sysFee += tx.SystemFee
if blockSize > maxBlockSize || sysFee > maxBlockSysFee {
txes = txes[:i]
break
}
}
return txes return txes
} }
@ -1943,16 +1934,6 @@ func (bc *Blockchain) GetBaseExecFee() int64 {
return bc.contracts.Policy.GetExecFeeFactorInternal(bc.dao) return bc.contracts.Policy.GetExecFeeFactorInternal(bc.dao)
} }
// GetMaxBlockSize returns maximum allowed block size from native Policy contract.
func (bc *Blockchain) GetMaxBlockSize() uint32 {
return bc.contracts.Policy.GetMaxBlockSizeInternal(bc.dao)
}
// GetMaxBlockSystemFee returns maximum block system fee from native Policy contract.
func (bc *Blockchain) GetMaxBlockSystemFee() int64 {
return bc.contracts.Policy.GetMaxBlockSystemFeeInternal(bc.dao)
}
// GetMaxVerificationGAS returns maximum verification GAS Policy limit. // GetMaxVerificationGAS returns maximum verification GAS Policy limit.
func (bc *Blockchain) GetMaxVerificationGAS() int64 { func (bc *Blockchain) GetMaxVerificationGAS() int64 {
return bc.contracts.Policy.GetMaxVerificationGas(bc.dao) return bc.contracts.Policy.GetMaxVerificationGas(bc.dao)

View file

@ -3,8 +3,6 @@ package blockchainer
// Policer is an interface that abstracts the implementation of policy methods. // Policer is an interface that abstracts the implementation of policy methods.
type Policer interface { type Policer interface {
GetBaseExecFee() int64 GetBaseExecFee() int64
GetMaxBlockSize() uint32
GetMaxBlockSystemFee() int64
GetMaxVerificationGAS() int64 GetMaxVerificationGAS() int64
GetStoragePrice() int64 GetStoragePrice() int64
} }

View file

@ -6,13 +6,11 @@ import (
"sort" "sort"
"sync" "sync"
"github.com/nspcc-dev/neo-go/pkg/core/block"
"github.com/nspcc-dev/neo-go/pkg/core/dao" "github.com/nspcc-dev/neo-go/pkg/core/dao"
"github.com/nspcc-dev/neo-go/pkg/core/interop" "github.com/nspcc-dev/neo-go/pkg/core/interop"
"github.com/nspcc-dev/neo-go/pkg/core/native/nativenames" "github.com/nspcc-dev/neo-go/pkg/core/native/nativenames"
"github.com/nspcc-dev/neo-go/pkg/core/state" "github.com/nspcc-dev/neo-go/pkg/core/state"
"github.com/nspcc-dev/neo-go/pkg/core/transaction" "github.com/nspcc-dev/neo-go/pkg/core/transaction"
"github.com/nspcc-dev/neo-go/pkg/network/payload"
"github.com/nspcc-dev/neo-go/pkg/smartcontract" "github.com/nspcc-dev/neo-go/pkg/smartcontract"
"github.com/nspcc-dev/neo-go/pkg/smartcontract/callflag" "github.com/nspcc-dev/neo-go/pkg/smartcontract/callflag"
"github.com/nspcc-dev/neo-go/pkg/smartcontract/manifest" "github.com/nspcc-dev/neo-go/pkg/smartcontract/manifest"
@ -23,17 +21,12 @@ import (
const ( const (
policyContractID = -5 policyContractID = -5
defaultMaxBlockSize = 1024 * 256
defaultMaxTransactionsPerBlock = 512
defaultExecFeeFactor = interop.DefaultBaseExecFee defaultExecFeeFactor = interop.DefaultBaseExecFee
defaultFeePerByte = 1000 defaultFeePerByte = 1000
defaultMaxVerificationGas = 50000000 defaultMaxVerificationGas = 50000000
defaultMaxBlockSystemFee = 9000 * GASFactor
// DefaultStoragePrice is the price to pay for 1 byte of storage. // DefaultStoragePrice is the price to pay for 1 byte of storage.
DefaultStoragePrice = 100000 DefaultStoragePrice = 100000
// minBlockSystemFee is the minimum allowed system fee per block.
minBlockSystemFee = 4007600
// maxExecFeeFactor is the maximum allowed execution fee factor. // maxExecFeeFactor is the maximum allowed execution fee factor.
maxExecFeeFactor = 1000 maxExecFeeFactor = 1000
// maxFeePerByte is the maximum allowed fee per byte value. // maxFeePerByte is the maximum allowed fee per byte value.
@ -46,18 +39,11 @@ const (
) )
var ( var (
// maxTransactionsPerBlockKey is a key used to store the maximum number of
// transactions allowed in block.
maxTransactionsPerBlockKey = []byte{23}
// execFeeFactorKey is a key used to store execution fee factor. // execFeeFactorKey is a key used to store execution fee factor.
execFeeFactorKey = []byte{18} execFeeFactorKey = []byte{18}
// feePerByteKey is a key used to store the minimum fee per byte for // feePerByteKey is a key used to store the minimum fee per byte for
// transaction. // transaction.
feePerByteKey = []byte{10} feePerByteKey = []byte{10}
// maxBlockSizeKey is a key used to store the maximum block size value.
maxBlockSizeKey = []byte{12}
// maxBlockSystemFeeKey is a key used to store the maximum block system fee value.
maxBlockSystemFeeKey = []byte{17}
// storagePriceKey is a key used to store storage price. // storagePriceKey is a key used to store storage price.
storagePriceKey = []byte{19} storagePriceKey = []byte{19}
) )
@ -71,11 +57,8 @@ type Policy struct {
// consensus iteration. If false, these values will be updated after // consensus iteration. If false, these values will be updated after
// blockchain DAO persisting. If true, we can safely use cached values. // blockchain DAO persisting. If true, we can safely use cached values.
isValid bool isValid bool
maxTransactionsPerBlock uint32
maxBlockSize uint32
execFeeFactor uint32 execFeeFactor uint32
feePerByte int64 feePerByte int64
maxBlockSystemFee int64
maxVerificationGas int64 maxVerificationGas int64
storagePrice uint32 storagePrice uint32
blockedAccounts []util.Uint160 blockedAccounts []util.Uint160
@ -88,16 +71,8 @@ func newPolicy() *Policy {
p := &Policy{ContractMD: *interop.NewContractMD(nativenames.Policy, policyContractID)} p := &Policy{ContractMD: *interop.NewContractMD(nativenames.Policy, policyContractID)}
defer p.UpdateHash() defer p.UpdateHash()
desc := newDescriptor("getMaxTransactionsPerBlock", smartcontract.IntegerType) desc := newDescriptor("getFeePerByte", smartcontract.IntegerType)
md := newMethodAndPrice(p.getMaxTransactionsPerBlock, 1000000, callflag.ReadStates) md := newMethodAndPrice(p.getFeePerByte, 1000000, callflag.ReadStates)
p.AddMethod(md, desc)
desc = newDescriptor("getMaxBlockSize", smartcontract.IntegerType)
md = newMethodAndPrice(p.getMaxBlockSize, 1000000, callflag.ReadStates)
p.AddMethod(md, desc)
desc = newDescriptor("getFeePerByte", smartcontract.IntegerType)
md = newMethodAndPrice(p.getFeePerByte, 1000000, callflag.ReadStates)
p.AddMethod(md, desc) p.AddMethod(md, desc)
desc = newDescriptor("isBlocked", smartcontract.BoolType, desc = newDescriptor("isBlocked", smartcontract.BoolType,
@ -105,10 +80,6 @@ func newPolicy() *Policy {
md = newMethodAndPrice(p.isBlocked, 1000000, callflag.ReadStates) md = newMethodAndPrice(p.isBlocked, 1000000, callflag.ReadStates)
p.AddMethod(md, desc) p.AddMethod(md, desc)
desc = newDescriptor("getMaxBlockSystemFee", smartcontract.IntegerType)
md = newMethodAndPrice(p.getMaxBlockSystemFee, 1000000, callflag.ReadStates)
p.AddMethod(md, desc)
desc = newDescriptor("getExecFeeFactor", smartcontract.IntegerType) desc = newDescriptor("getExecFeeFactor", smartcontract.IntegerType)
md = newMethodAndPrice(p.getExecFeeFactor, 1000000, callflag.ReadStates) md = newMethodAndPrice(p.getExecFeeFactor, 1000000, callflag.ReadStates)
p.AddMethod(md, desc) p.AddMethod(md, desc)
@ -127,26 +98,11 @@ func newPolicy() *Policy {
md = newMethodAndPrice(p.setStoragePrice, 3000000, callflag.States) md = newMethodAndPrice(p.setStoragePrice, 3000000, callflag.States)
p.AddMethod(md, desc) p.AddMethod(md, desc)
desc = newDescriptor("setMaxBlockSize", smartcontract.VoidType,
manifest.NewParameter("value", smartcontract.IntegerType))
md = newMethodAndPrice(p.setMaxBlockSize, 3000000, callflag.States)
p.AddMethod(md, desc)
desc = newDescriptor("setMaxTransactionsPerBlock", smartcontract.VoidType,
manifest.NewParameter("value", smartcontract.IntegerType))
md = newMethodAndPrice(p.setMaxTransactionsPerBlock, 3000000, callflag.States)
p.AddMethod(md, desc)
desc = newDescriptor("setFeePerByte", smartcontract.VoidType, desc = newDescriptor("setFeePerByte", smartcontract.VoidType,
manifest.NewParameter("value", smartcontract.IntegerType)) manifest.NewParameter("value", smartcontract.IntegerType))
md = newMethodAndPrice(p.setFeePerByte, 3000000, callflag.States) md = newMethodAndPrice(p.setFeePerByte, 3000000, callflag.States)
p.AddMethod(md, desc) p.AddMethod(md, desc)
desc = newDescriptor("setMaxBlockSystemFee", smartcontract.VoidType,
manifest.NewParameter("value", smartcontract.IntegerType))
md = newMethodAndPrice(p.setMaxBlockSystemFee, 3000000, callflag.States)
p.AddMethod(md, desc)
desc = newDescriptor("blockAccount", smartcontract.BoolType, desc = newDescriptor("blockAccount", smartcontract.BoolType,
manifest.NewParameter("account", smartcontract.Hash160Type)) manifest.NewParameter("account", smartcontract.Hash160Type))
md = newMethodAndPrice(p.blockAccount, 3000000, callflag.States) md = newMethodAndPrice(p.blockAccount, 3000000, callflag.States)
@ -167,18 +123,9 @@ func (p *Policy) Metadata() *interop.ContractMD {
// Initialize initializes Policy native contract and implements Contract interface. // Initialize initializes Policy native contract and implements Contract interface.
func (p *Policy) Initialize(ic *interop.Context) error { func (p *Policy) Initialize(ic *interop.Context) error {
if err := setIntWithKey(p.ID, ic.DAO, maxTransactionsPerBlockKey, defaultMaxTransactionsPerBlock); err != nil {
return err
}
if err := setIntWithKey(p.ID, ic.DAO, feePerByteKey, defaultFeePerByte); err != nil { if err := setIntWithKey(p.ID, ic.DAO, feePerByteKey, defaultFeePerByte); err != nil {
return err return err
} }
if err := setIntWithKey(p.ID, ic.DAO, maxBlockSizeKey, defaultMaxBlockSize); err != nil {
return err
}
if err := setIntWithKey(p.ID, ic.DAO, maxBlockSystemFeeKey, defaultMaxBlockSystemFee); err != nil {
return err
}
if err := setIntWithKey(p.ID, ic.DAO, execFeeFactorKey, defaultExecFeeFactor); err != nil { if err := setIntWithKey(p.ID, ic.DAO, execFeeFactorKey, defaultExecFeeFactor); err != nil {
return err return err
} }
@ -187,11 +134,8 @@ func (p *Policy) Initialize(ic *interop.Context) error {
} }
p.isValid = true p.isValid = true
p.maxTransactionsPerBlock = defaultMaxTransactionsPerBlock
p.maxBlockSize = defaultMaxBlockSize
p.execFeeFactor = defaultExecFeeFactor p.execFeeFactor = defaultExecFeeFactor
p.feePerByte = defaultFeePerByte p.feePerByte = defaultFeePerByte
p.maxBlockSystemFee = defaultMaxBlockSystemFee
p.maxVerificationGas = defaultMaxVerificationGas p.maxVerificationGas = defaultMaxVerificationGas
p.storagePrice = DefaultStoragePrice p.storagePrice = DefaultStoragePrice
p.blockedAccounts = make([]util.Uint160, 0) p.blockedAccounts = make([]util.Uint160, 0)
@ -212,11 +156,8 @@ func (p *Policy) PostPersist(ic *interop.Context) error {
return nil return nil
} }
p.maxTransactionsPerBlock = uint32(getIntWithKey(p.ID, ic.DAO, maxTransactionsPerBlockKey))
p.maxBlockSize = uint32(getIntWithKey(p.ID, ic.DAO, maxBlockSizeKey))
p.execFeeFactor = uint32(getIntWithKey(p.ID, ic.DAO, execFeeFactorKey)) p.execFeeFactor = uint32(getIntWithKey(p.ID, ic.DAO, execFeeFactorKey))
p.feePerByte = getIntWithKey(p.ID, ic.DAO, feePerByteKey) p.feePerByte = getIntWithKey(p.ID, ic.DAO, feePerByteKey)
p.maxBlockSystemFee = getIntWithKey(p.ID, ic.DAO, maxBlockSystemFeeKey)
p.maxVerificationGas = defaultMaxVerificationGas p.maxVerificationGas = defaultMaxVerificationGas
p.storagePrice = uint32(getIntWithKey(p.ID, ic.DAO, storagePriceKey)) p.storagePrice = uint32(getIntWithKey(p.ID, ic.DAO, storagePriceKey))
@ -240,38 +181,6 @@ func (p *Policy) PostPersist(ic *interop.Context) error {
return nil return nil
} }
// getMaxTransactionsPerBlock is Policy contract method and returns the upper
// limit of transactions per block.
func (p *Policy) getMaxTransactionsPerBlock(ic *interop.Context, _ []stackitem.Item) stackitem.Item {
return stackitem.NewBigInteger(big.NewInt(int64(p.GetMaxTransactionsPerBlockInternal(ic.DAO))))
}
// GetMaxTransactionsPerBlockInternal returns the upper limit of transactions per
// block.
func (p *Policy) GetMaxTransactionsPerBlockInternal(dao dao.DAO) uint32 {
p.lock.RLock()
defer p.lock.RUnlock()
if p.isValid {
return p.maxTransactionsPerBlock
}
return uint32(getIntWithKey(p.ID, dao, maxTransactionsPerBlockKey))
}
// getMaxBlockSize is Policy contract method and returns maximum block size.
func (p *Policy) getMaxBlockSize(ic *interop.Context, _ []stackitem.Item) stackitem.Item {
return stackitem.NewBigInteger(big.NewInt(int64(p.GetMaxBlockSizeInternal(ic.DAO))))
}
// GetMaxBlockSizeInternal returns maximum block size.
func (p *Policy) GetMaxBlockSizeInternal(dao dao.DAO) uint32 {
p.lock.RLock()
defer p.lock.RUnlock()
if p.isValid {
return p.maxBlockSize
}
return uint32(getIntWithKey(p.ID, dao, maxBlockSizeKey))
}
// getFeePerByte is Policy contract method and returns required transaction's fee // getFeePerByte is Policy contract method and returns required transaction's fee
// per byte. // per byte.
func (p *Policy) getFeePerByte(ic *interop.Context, _ []stackitem.Item) stackitem.Item { func (p *Policy) getFeePerByte(ic *interop.Context, _ []stackitem.Item) stackitem.Item {
@ -296,22 +205,6 @@ func (p *Policy) GetMaxVerificationGas(_ dao.DAO) int64 {
return defaultMaxVerificationGas return defaultMaxVerificationGas
} }
// getMaxBlockSystemFee is Policy contract method and returns the maximum overall
// system fee per block.
func (p *Policy) getMaxBlockSystemFee(ic *interop.Context, _ []stackitem.Item) stackitem.Item {
return stackitem.NewBigInteger(big.NewInt(p.GetMaxBlockSystemFeeInternal(ic.DAO)))
}
// GetMaxBlockSystemFeeInternal the maximum overall system fee per block.
func (p *Policy) GetMaxBlockSystemFeeInternal(dao dao.DAO) int64 {
p.lock.RLock()
defer p.lock.RUnlock()
if p.isValid {
return p.maxBlockSystemFee
}
return getIntWithKey(p.ID, dao, maxBlockSystemFeeKey)
}
func (p *Policy) getExecFeeFactor(ic *interop.Context, _ []stackitem.Item) stackitem.Item { func (p *Policy) getExecFeeFactor(ic *interop.Context, _ []stackitem.Item) stackitem.Item {
return stackitem.NewBigInteger(big.NewInt(int64(p.GetExecFeeFactorInternal(ic.DAO)))) return stackitem.NewBigInteger(big.NewInt(int64(p.GetExecFeeFactorInternal(ic.DAO))))
} }
@ -400,45 +293,6 @@ func (p *Policy) setStoragePrice(ic *interop.Context, args []stackitem.Item) sta
return stackitem.Null{} return stackitem.Null{}
} }
// setMaxTransactionsPerBlock is Policy contract method and sets the upper limit
// of transactions per block.
func (p *Policy) setMaxTransactionsPerBlock(ic *interop.Context, args []stackitem.Item) stackitem.Item {
value := uint32(toBigInt(args[0]).Int64())
if value > block.MaxTransactionsPerBlock {
panic(fmt.Errorf("MaxTransactionsPerBlock cannot exceed the maximum allowed transactions per block = %d", block.MaxTransactionsPerBlock))
}
if !p.NEO.checkCommittee(ic) {
panic("invalid committee signature")
}
p.lock.Lock()
defer p.lock.Unlock()
err := setIntWithKey(p.ID, ic.DAO, maxTransactionsPerBlockKey, int64(value))
if err != nil {
panic(err)
}
p.isValid = false
return stackitem.Null{}
}
// setMaxBlockSize is Policy contract method and sets maximum block size.
func (p *Policy) setMaxBlockSize(ic *interop.Context, args []stackitem.Item) stackitem.Item {
value := uint32(toBigInt(args[0]).Int64())
if value > payload.MaxSize {
panic(fmt.Errorf("MaxBlockSize cannot be more than the maximum payload size = %d", payload.MaxSize))
}
if !p.NEO.checkCommittee(ic) {
panic("invalid committee signature")
}
p.lock.Lock()
defer p.lock.Unlock()
err := setIntWithKey(p.ID, ic.DAO, maxBlockSizeKey, int64(value))
if err != nil {
panic(err)
}
p.isValid = false
return stackitem.Null{}
}
// setFeePerByte is Policy contract method and sets transaction's fee per byte. // setFeePerByte is Policy contract method and sets transaction's fee per byte.
func (p *Policy) setFeePerByte(ic *interop.Context, args []stackitem.Item) stackitem.Item { func (p *Policy) setFeePerByte(ic *interop.Context, args []stackitem.Item) stackitem.Item {
value := toBigInt(args[0]).Int64() value := toBigInt(args[0]).Int64()
@ -458,25 +312,6 @@ func (p *Policy) setFeePerByte(ic *interop.Context, args []stackitem.Item) stack
return stackitem.Null{} return stackitem.Null{}
} }
// setMaxBlockSystemFee is Policy contract method and sets the maximum system fee per block.
func (p *Policy) setMaxBlockSystemFee(ic *interop.Context, args []stackitem.Item) stackitem.Item {
value := toBigInt(args[0]).Int64()
if value <= minBlockSystemFee {
panic(fmt.Errorf("MaxBlockSystemFee cannot be less then %d", minBlockSystemFee))
}
if !p.NEO.checkCommittee(ic) {
panic("invalid committee signature")
}
p.lock.Lock()
defer p.lock.Unlock()
err := setIntWithKey(p.ID, ic.DAO, maxBlockSystemFeeKey, value)
if err != nil {
panic(err)
}
p.isValid = false
return stackitem.Null{}
}
// blockAccount is Policy contract method and adds given account hash to the list // blockAccount is Policy contract method and adds given account hash to the list
// of blocked accounts. // of blocked accounts.
func (p *Policy) blockAccount(ic *interop.Context, args []stackitem.Item) stackitem.Item { func (p *Policy) blockAccount(ic *interop.Context, args []stackitem.Item) stackitem.Item {
@ -530,9 +365,5 @@ func (p *Policy) CheckPolicy(d dao.DAO, tx *transaction.Transaction) error {
return fmt.Errorf("account %s is blocked", signer.Account.StringLE()) return fmt.Errorf("account %s is blocked", signer.Account.StringLE())
} }
} }
maxBlockSystemFee := p.GetMaxBlockSystemFeeInternal(d)
if maxBlockSystemFee < tx.SystemFee {
return fmt.Errorf("transaction's fee can't exceed maximum block system fee %d", maxBlockSystemFee)
}
return nil return nil
} }

View file

@ -5,10 +5,8 @@ import (
"github.com/nspcc-dev/neo-go/internal/random" "github.com/nspcc-dev/neo-go/internal/random"
"github.com/nspcc-dev/neo-go/internal/testchain" "github.com/nspcc-dev/neo-go/internal/testchain"
"github.com/nspcc-dev/neo-go/pkg/core/block"
"github.com/nspcc-dev/neo-go/pkg/core/interop" "github.com/nspcc-dev/neo-go/pkg/core/interop"
"github.com/nspcc-dev/neo-go/pkg/core/native" "github.com/nspcc-dev/neo-go/pkg/core/native"
"github.com/nspcc-dev/neo-go/pkg/network/payload"
"github.com/nspcc-dev/neo-go/pkg/util" "github.com/nspcc-dev/neo-go/pkg/util"
"github.com/nspcc-dev/neo-go/pkg/vm/stackitem" "github.com/nspcc-dev/neo-go/pkg/vm/stackitem"
"github.com/nspcc-dev/neo-go/pkg/wallet" "github.com/nspcc-dev/neo-go/pkg/wallet"
@ -76,28 +74,6 @@ func testGetSet(t *testing.T, chain *Blockchain, hash util.Uint160, name string,
}) })
} }
func TestMaxTransactionsPerBlock(t *testing.T) {
chain := newTestChain(t)
t.Run("get, internal method", func(t *testing.T) {
n := chain.contracts.Policy.GetMaxTransactionsPerBlockInternal(chain.dao)
require.Equal(t, 512, int(n))
})
testGetSet(t, chain, chain.contracts.Policy.Hash, "MaxTransactionsPerBlock", 512, 0, block.MaxTransactionsPerBlock)
}
func TestMaxBlockSize(t *testing.T) {
chain := newTestChain(t)
t.Run("get, internal method", func(t *testing.T) {
n := chain.contracts.Policy.GetMaxBlockSizeInternal(chain.dao)
require.Equal(t, 1024*256, int(n))
})
testGetSet(t, chain, chain.contracts.Policy.Hash, "MaxBlockSize", 1024*256, 0, payload.MaxSize)
}
func TestFeePerByte(t *testing.T) { func TestFeePerByte(t *testing.T) {
chain := newTestChain(t) chain := newTestChain(t)
@ -120,17 +96,6 @@ func TestExecFeeFactor(t *testing.T) {
testGetSet(t, chain, chain.contracts.Policy.Hash, "ExecFeeFactor", interop.DefaultBaseExecFee, 1, 1000) testGetSet(t, chain, chain.contracts.Policy.Hash, "ExecFeeFactor", interop.DefaultBaseExecFee, 1, 1000)
} }
func TestBlockSystemFee(t *testing.T) {
chain := newTestChain(t)
t.Run("get, internal method", func(t *testing.T) {
n := chain.contracts.Policy.GetMaxBlockSystemFeeInternal(chain.dao)
require.Equal(t, 9000*native.GASFactor, int(n))
})
testGetSet(t, chain, chain.contracts.Policy.Hash, "MaxBlockSystemFee", 9000*native.GASFactor, 4007600, 0)
}
func TestStoragePrice(t *testing.T) { func TestStoragePrice(t *testing.T) {
chain := newTestChain(t) chain := newTestChain(t)

View file

@ -8,26 +8,6 @@ import (
// Hash represents Policy contract hash. // Hash represents Policy contract hash.
const Hash = "\x7b\xc6\x81\xc0\xa1\xf7\x1d\x54\x34\x57\xb6\x8b\xba\x8d\x5f\x9f\xdd\x4e\x5e\xcc" const Hash = "\x7b\xc6\x81\xc0\xa1\xf7\x1d\x54\x34\x57\xb6\x8b\xba\x8d\x5f\x9f\xdd\x4e\x5e\xcc"
// GetMaxTransactionsPerBlock represents `getMaxTransactionsPerBlock` method of Policy native contract.
func GetMaxTransactionsPerBlock() int {
return contract.Call(interop.Hash160(Hash), "getMaxTransactionsPerBlock", contract.ReadStates).(int)
}
// SetMaxTransactionsPerBlock represents `setMaxTransactionsPerBlock` method of Policy native contract.
func SetMaxTransactionsPerBlock(value int) {
contract.Call(interop.Hash160(Hash), "setMaxTransactionsPerBlock", contract.States, value)
}
// GetMaxBlockSize represents `getMaxBlockSize` method of Policy native contract.
func GetMaxBlockSize() int {
return contract.Call(interop.Hash160(Hash), "getMaxBlockSize", contract.ReadStates).(int)
}
// SetMaxBlockSize represents `setMaxBlockSize` method of Policy native contract.
func SetMaxBlockSize(value int) {
contract.Call(interop.Hash160(Hash), "setMaxBlockSize", contract.States, value)
}
// GetFeePerByte represents `getFeePerByte` method of Policy native contract. // GetFeePerByte represents `getFeePerByte` method of Policy native contract.
func GetFeePerByte() int { func GetFeePerByte() int {
return contract.Call(interop.Hash160(Hash), "getFeePerByte", contract.ReadStates).(int) return contract.Call(interop.Hash160(Hash), "getFeePerByte", contract.ReadStates).(int)
@ -38,16 +18,6 @@ func SetFeePerByte(value int) {
contract.Call(interop.Hash160(Hash), "setFeePerByte", contract.States, value) contract.Call(interop.Hash160(Hash), "setFeePerByte", contract.States, value)
} }
// GetMaxBlockSystemFee represents `getMaxBlockSystemFee` method of Policy native contract.
func GetMaxBlockSystemFee() int {
return contract.Call(interop.Hash160(Hash), "getMaxBlockSystemFee", contract.ReadStates).(int)
}
// SetMaxBlockSystemFee represents `setMaxBlockSystemFee` method of Policy native contract.
func SetMaxBlockSystemFee(value int) {
contract.Call(interop.Hash160(Hash), "setMaxBlockSystemFee", contract.States, value)
}
// GetExecFeeFactor represents `getExecFeeFactor` method of Policy native contract. // GetExecFeeFactor represents `getExecFeeFactor` method of Policy native contract.
func GetExecFeeFactor() int { func GetExecFeeFactor() int {
return contract.Call(interop.Hash160(Hash), "getExecFeeFactor", contract.ReadStates).(int) return contract.Call(interop.Hash160(Hash), "getExecFeeFactor", contract.ReadStates).(int)

View file

@ -9,17 +9,6 @@ import (
"github.com/nspcc-dev/neo-go/pkg/vm/stackitem" "github.com/nspcc-dev/neo-go/pkg/vm/stackitem"
) )
// GetMaxTransactionsPerBlock invokes `getMaxTransactionsPerBlock` method on a
// native Policy contract.
func (c *Client) GetMaxTransactionsPerBlock() (int64, error) {
return c.invokeNativePolicyMethod("getMaxTransactionsPerBlock")
}
// GetMaxBlockSize invokes `getMaxBlockSize` method on a native Policy contract.
func (c *Client) GetMaxBlockSize() (int64, error) {
return c.invokeNativePolicyMethod("getMaxBlockSize")
}
// GetFeePerByte invokes `getFeePerByte` method on a native Policy contract. // GetFeePerByte invokes `getFeePerByte` method on a native Policy contract.
func (c *Client) GetFeePerByte() (int64, error) { func (c *Client) GetFeePerByte() (int64, error) {
if !c.initDone { if !c.initDone {

View file

@ -431,30 +431,6 @@ var rpcClientTestCases = map[string][]rpcClientTestCase{
}, },
}, },
}, },
"getMaxTransacctionsPerBlock": {
{
name: "positive",
invoke: func(c *Client) (interface{}, error) {
return c.GetMaxTransactionsPerBlock()
},
serverResponse: `{"id":1,"jsonrpc":"2.0","result":{"state":"HALT","gasconsumed":"2007390","script":"EMAMGmdldE1heFRyYW5zYWN0aW9uc1BlckJsb2NrDBSaYaRu7Je4kwbXzoHxW0YgkdAJMkFifVtS","stack":[{"type":"Integer","value":"512"}],"tx":null}}`,
result: func(c *Client) interface{} {
return int64(512)
},
},
},
"getMaxBlockSize": {
{
name: "positive",
invoke: func(c *Client) (interface{}, error) {
return c.GetMaxBlockSize()
},
serverResponse: `{"id":1,"jsonrpc":"2.0","result":{"state":"HALT","gasconsumed":"2007390","script":"EMAMD2dldE1heEJsb2NrU2l6ZQwUmmGkbuyXuJMG186B8VtGIJHQCTJBYn1bUg==","stack":[{"type":"Integer","value":"262144"}],"tx":null}}`,
result: func(c *Client) interface{} {
return int64(262144)
},
},
},
"getMaxNotValidBeforeDelta": { "getMaxNotValidBeforeDelta": {
{ {
name: "positive", name: "positive",