network: decouple it from blockchainer.Blockchainer
We don't need all of it.
This commit is contained in:
parent
ddb23ccf78
commit
746644a4eb
13 changed files with 85 additions and 55 deletions
|
@ -40,7 +40,7 @@ type FakeChain struct {
|
||||||
MaxVerificationGAS int64
|
MaxVerificationGAS int64
|
||||||
NotaryContractScriptHash util.Uint160
|
NotaryContractScriptHash util.Uint160
|
||||||
NotaryDepositExpiration uint32
|
NotaryDepositExpiration uint32
|
||||||
PostBlock []func(blockchainer.Blockchainer, *mempool.Pool, *block.Block)
|
PostBlock []func(func(*transaction.Transaction, *mempool.Pool, bool) bool, *mempool.Pool, *block.Block)
|
||||||
UtilityTokenBalance *big.Int
|
UtilityTokenBalance *big.Int
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -158,12 +158,12 @@ func (chain *FakeChain) GetMaxVerificationGAS() int64 {
|
||||||
}
|
}
|
||||||
|
|
||||||
// PoolTxWithData implements Blockchainer interface.
|
// PoolTxWithData implements Blockchainer interface.
|
||||||
func (chain *FakeChain) PoolTxWithData(t *transaction.Transaction, data interface{}, mp *mempool.Pool, feer mempool.Feer, verificationFunction func(bc blockchainer.Blockchainer, t *transaction.Transaction, data interface{}) error) error {
|
func (chain *FakeChain) PoolTxWithData(t *transaction.Transaction, data interface{}, mp *mempool.Pool, feer mempool.Feer, verificationFunction func(t *transaction.Transaction, data interface{}) error) error {
|
||||||
return chain.poolTxWithData(t, data, mp)
|
return chain.poolTxWithData(t, data, mp)
|
||||||
}
|
}
|
||||||
|
|
||||||
// RegisterPostBlock implements Blockchainer interface.
|
// RegisterPostBlock implements Blockchainer interface.
|
||||||
func (chain *FakeChain) RegisterPostBlock(f func(blockchainer.Blockchainer, *mempool.Pool, *block.Block)) {
|
func (chain *FakeChain) RegisterPostBlock(f func(func(*transaction.Transaction, *mempool.Pool, bool) bool, *mempool.Pool, *block.Block)) {
|
||||||
chain.PostBlock = append(chain.PostBlock, f)
|
chain.PostBlock = append(chain.PostBlock, f)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -150,7 +150,7 @@ type Blockchain struct {
|
||||||
|
|
||||||
// postBlock is a set of callback methods which should be run under the Blockchain lock after new block is persisted.
|
// postBlock is a set of callback methods which should be run under the Blockchain lock after new block is persisted.
|
||||||
// Block's transactions are passed via mempool.
|
// Block's transactions are passed via mempool.
|
||||||
postBlock []func(blockchainer.Blockchainer, *mempool.Pool, *block.Block)
|
postBlock []func(func(*transaction.Transaction, *mempool.Pool, bool) bool, *mempool.Pool, *block.Block)
|
||||||
|
|
||||||
sbCommittee keys.PublicKeys
|
sbCommittee keys.PublicKeys
|
||||||
|
|
||||||
|
@ -1155,7 +1155,7 @@ func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error
|
||||||
atomic.StoreUint32(&bc.blockHeight, block.Index)
|
atomic.StoreUint32(&bc.blockHeight, block.Index)
|
||||||
bc.memPool.RemoveStale(func(tx *transaction.Transaction) bool { return bc.IsTxStillRelevant(tx, txpool, false) }, bc)
|
bc.memPool.RemoveStale(func(tx *transaction.Transaction) bool { return bc.IsTxStillRelevant(tx, txpool, false) }, bc)
|
||||||
for _, f := range bc.postBlock {
|
for _, f := range bc.postBlock {
|
||||||
f(bc, txpool, block)
|
f(bc.IsTxStillRelevant, txpool, block)
|
||||||
}
|
}
|
||||||
if err := bc.updateExtensibleWhitelist(block.Index); err != nil {
|
if err := bc.updateExtensibleWhitelist(block.Index); err != nil {
|
||||||
bc.lock.Unlock()
|
bc.lock.Unlock()
|
||||||
|
@ -2093,12 +2093,12 @@ func (bc *Blockchain) PoolTx(t *transaction.Transaction, pools ...*mempool.Pool)
|
||||||
}
|
}
|
||||||
|
|
||||||
// PoolTxWithData verifies and tries to add given transaction with additional data into the mempool.
|
// PoolTxWithData verifies and tries to add given transaction with additional data into the mempool.
|
||||||
func (bc *Blockchain) PoolTxWithData(t *transaction.Transaction, data interface{}, mp *mempool.Pool, feer mempool.Feer, verificationFunction func(bc blockchainer.Blockchainer, tx *transaction.Transaction, data interface{}) error) error {
|
func (bc *Blockchain) PoolTxWithData(t *transaction.Transaction, data interface{}, mp *mempool.Pool, feer mempool.Feer, verificationFunction func(tx *transaction.Transaction, data interface{}) error) error {
|
||||||
bc.lock.RLock()
|
bc.lock.RLock()
|
||||||
defer bc.lock.RUnlock()
|
defer bc.lock.RUnlock()
|
||||||
|
|
||||||
if verificationFunction != nil {
|
if verificationFunction != nil {
|
||||||
err := verificationFunction(bc, t, data)
|
err := verificationFunction(t, data)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -2330,7 +2330,7 @@ func (bc *Blockchain) P2PSigExtensionsEnabled() bool {
|
||||||
|
|
||||||
// RegisterPostBlock appends provided function to the list of functions which should be run after new block
|
// RegisterPostBlock appends provided function to the list of functions which should be run after new block
|
||||||
// is stored.
|
// is stored.
|
||||||
func (bc *Blockchain) RegisterPostBlock(f func(blockchainer.Blockchainer, *mempool.Pool, *block.Block)) {
|
func (bc *Blockchain) RegisterPostBlock(f func(func(*transaction.Transaction, *mempool.Pool, bool) bool, *mempool.Pool, *block.Block)) {
|
||||||
bc.postBlock = append(bc.postBlock, f)
|
bc.postBlock = append(bc.postBlock, f)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -16,7 +16,6 @@ import (
|
||||||
"github.com/nspcc-dev/neo-go/pkg/config"
|
"github.com/nspcc-dev/neo-go/pkg/config"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/config/netmode"
|
"github.com/nspcc-dev/neo-go/pkg/config/netmode"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/block"
|
"github.com/nspcc-dev/neo-go/pkg/core/block"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/blockchainer"
|
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/chaindump"
|
"github.com/nspcc-dev/neo-go/pkg/core/chaindump"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/fee"
|
"github.com/nspcc-dev/neo-go/pkg/core/fee"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/interop/interopnames"
|
"github.com/nspcc-dev/neo-go/pkg/core/interop/interopnames"
|
||||||
|
@ -1073,7 +1072,7 @@ func TestVerifyTx(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
mp := mempool.New(10, 1, false)
|
mp := mempool.New(10, 1, false)
|
||||||
verificationF := func(bc blockchainer.Blockchainer, tx *transaction.Transaction, data interface{}) error {
|
verificationF := func(tx *transaction.Transaction, data interface{}) error {
|
||||||
if data.(int) > 5 {
|
if data.(int) > 5 {
|
||||||
return errors.New("bad data")
|
return errors.New("bad data")
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,8 +21,10 @@ import (
|
||||||
// of the blockchain.
|
// of the blockchain.
|
||||||
type Blockchainer interface {
|
type Blockchainer interface {
|
||||||
ApplyPolicyToTxSet([]*transaction.Transaction) []*transaction.Transaction
|
ApplyPolicyToTxSet([]*transaction.Transaction) []*transaction.Transaction
|
||||||
|
AddBlock(block *block.Block) error
|
||||||
|
AddHeaders(...*block.Header) error
|
||||||
|
BlockHeight() uint32
|
||||||
GetConfig() config.ProtocolConfiguration
|
GetConfig() config.ProtocolConfiguration
|
||||||
Blockqueuer // Blockqueuer interface
|
|
||||||
CalculateClaimable(h util.Uint160, endHeight uint32) (*big.Int, error)
|
CalculateClaimable(h util.Uint160, endHeight uint32) (*big.Int, error)
|
||||||
Close()
|
Close()
|
||||||
InitVerificationContext(ic *interop.Context, hash util.Uint160, witness *transaction.Witness) error
|
InitVerificationContext(ic *interop.Context, hash util.Uint160, witness *transaction.Witness) error
|
||||||
|
@ -65,8 +67,7 @@ type Blockchainer interface {
|
||||||
mempool.Feer // fee interface
|
mempool.Feer // fee interface
|
||||||
ManagementContractHash() util.Uint160
|
ManagementContractHash() util.Uint160
|
||||||
PoolTx(t *transaction.Transaction, pools ...*mempool.Pool) error
|
PoolTx(t *transaction.Transaction, pools ...*mempool.Pool) error
|
||||||
PoolTxWithData(t *transaction.Transaction, data interface{}, mp *mempool.Pool, feer mempool.Feer, verificationFunction func(bc Blockchainer, t *transaction.Transaction, data interface{}) error) error
|
PoolTxWithData(t *transaction.Transaction, data interface{}, mp *mempool.Pool, feer mempool.Feer, verificationFunction func(t *transaction.Transaction, data interface{}) error) error
|
||||||
RegisterPostBlock(f func(Blockchainer, *mempool.Pool, *block.Block))
|
|
||||||
SetNotary(mod services.Notary)
|
SetNotary(mod services.Notary)
|
||||||
SubscribeForBlocks(ch chan<- *block.Block)
|
SubscribeForBlocks(ch chan<- *block.Block)
|
||||||
SubscribeForExecutions(ch chan<- *state.AppExecResult)
|
SubscribeForExecutions(ch chan<- *state.AppExecResult)
|
||||||
|
|
|
@ -1,10 +0,0 @@
|
||||||
package blockchainer
|
|
||||||
|
|
||||||
import "github.com/nspcc-dev/neo-go/pkg/core/block"
|
|
||||||
|
|
||||||
// Blockqueuer is an interface for blockqueue.
|
|
||||||
type Blockqueuer interface {
|
|
||||||
AddBlock(block *block.Block) error
|
|
||||||
AddHeaders(...*block.Header) error
|
|
||||||
BlockHeight() uint32
|
|
||||||
}
|
|
|
@ -13,7 +13,6 @@ import (
|
||||||
"github.com/nspcc-dev/neo-go/internal/testchain"
|
"github.com/nspcc-dev/neo-go/internal/testchain"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/config"
|
"github.com/nspcc-dev/neo-go/pkg/config"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/block"
|
"github.com/nspcc-dev/neo-go/pkg/core/block"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/blockchainer"
|
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/mempool"
|
"github.com/nspcc-dev/neo-go/pkg/core/mempool"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/native/noderoles"
|
"github.com/nspcc-dev/neo-go/pkg/core/native/noderoles"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/transaction"
|
"github.com/nspcc-dev/neo-go/pkg/core/transaction"
|
||||||
|
@ -135,7 +134,7 @@ func TestNotary(t *testing.T) {
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
bc.SetNotary(ntr1)
|
bc.SetNotary(ntr1)
|
||||||
bc.RegisterPostBlock(func(bc blockchainer.Blockchainer, pool *mempool.Pool, b *block.Block) {
|
bc.RegisterPostBlock(func(f func(*transaction.Transaction, *mempool.Pool, bool) bool, pool *mempool.Pool, b *block.Block) {
|
||||||
ntr1.PostPersist()
|
ntr1.PostPersist()
|
||||||
})
|
})
|
||||||
|
|
||||||
|
|
|
@ -4,17 +4,23 @@ import (
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/block"
|
"github.com/nspcc-dev/neo-go/pkg/core/block"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/blockchainer"
|
|
||||||
"go.uber.org/atomic"
|
"go.uber.org/atomic"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
// Blockqueuer is the interface for block queue.
|
||||||
|
type Blockqueuer interface {
|
||||||
|
AddBlock(block *block.Block) error
|
||||||
|
AddHeaders(...*block.Header) error
|
||||||
|
BlockHeight() uint32
|
||||||
|
}
|
||||||
|
|
||||||
type blockQueue struct {
|
type blockQueue struct {
|
||||||
log *zap.Logger
|
log *zap.Logger
|
||||||
queueLock sync.Mutex
|
queueLock sync.Mutex
|
||||||
queue []*block.Block
|
queue []*block.Block
|
||||||
checkBlocks chan struct{}
|
checkBlocks chan struct{}
|
||||||
chain blockchainer.Blockqueuer
|
chain Blockqueuer
|
||||||
relayF func(*block.Block)
|
relayF func(*block.Block)
|
||||||
discarded *atomic.Bool
|
discarded *atomic.Bool
|
||||||
len int
|
len int
|
||||||
|
@ -26,7 +32,7 @@ const (
|
||||||
blockCacheSize = 2000
|
blockCacheSize = 2000
|
||||||
)
|
)
|
||||||
|
|
||||||
func newBlockQueue(capacity int, bc blockchainer.Blockqueuer, log *zap.Logger, relayer func(*block.Block)) *blockQueue {
|
func newBlockQueue(capacity int, bc Blockqueuer, log *zap.Logger, relayer func(*block.Block)) *blockQueue {
|
||||||
if log == nil {
|
if log == nil {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
|
@ -5,11 +5,19 @@ import (
|
||||||
"errors"
|
"errors"
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/blockchainer"
|
"github.com/nspcc-dev/neo-go/pkg/core/transaction"
|
||||||
|
"github.com/nspcc-dev/neo-go/pkg/crypto/hash"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/network/payload"
|
"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"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
// Ledger is enough of Blockchain to satisfy Pool.
|
||||||
|
type Ledger interface {
|
||||||
|
BlockHeight() uint32
|
||||||
|
IsExtensibleAllowed(util.Uint160) bool
|
||||||
|
VerifyWitness(util.Uint160, hash.Hashable, *transaction.Witness, int64) (int64, error)
|
||||||
|
}
|
||||||
|
|
||||||
// Pool represents pool of extensible payloads.
|
// Pool represents pool of extensible payloads.
|
||||||
type Pool struct {
|
type Pool struct {
|
||||||
lock sync.RWMutex
|
lock sync.RWMutex
|
||||||
|
@ -17,11 +25,11 @@ type Pool struct {
|
||||||
senders map[util.Uint160]*list.List
|
senders map[util.Uint160]*list.List
|
||||||
// singleCap represents maximum number of payloads from the single sender.
|
// singleCap represents maximum number of payloads from the single sender.
|
||||||
singleCap int
|
singleCap int
|
||||||
chain blockchainer.Blockchainer
|
chain Ledger
|
||||||
}
|
}
|
||||||
|
|
||||||
// New returns new payload pool using provided chain.
|
// New returns new payload pool using provided chain.
|
||||||
func New(bc blockchainer.Blockchainer, capacity int) *Pool {
|
func New(bc Ledger, capacity int) *Pool {
|
||||||
if capacity <= 0 {
|
if capacity <= 0 {
|
||||||
panic("invalid capacity")
|
panic("invalid capacity")
|
||||||
}
|
}
|
||||||
|
|
|
@ -4,7 +4,6 @@ import (
|
||||||
"errors"
|
"errors"
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/blockchainer"
|
|
||||||
"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/crypto/hash"
|
"github.com/nspcc-dev/neo-go/pkg/crypto/hash"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/network/payload"
|
"github.com/nspcc-dev/neo-go/pkg/network/payload"
|
||||||
|
@ -116,7 +115,7 @@ func (p *Pool) testAdd(t *testing.T, expectedOk bool, expectedErr error, ep *pay
|
||||||
}
|
}
|
||||||
|
|
||||||
type testChain struct {
|
type testChain struct {
|
||||||
blockchainer.Blockchainer
|
Ledger
|
||||||
height uint32
|
height uint32
|
||||||
verifyWitness func(util.Uint160) bool
|
verifyWitness func(util.Uint160) bool
|
||||||
isAllowed func(util.Uint160) bool
|
isAllowed func(util.Uint160) bool
|
||||||
|
|
|
@ -3,13 +3,12 @@ package network
|
||||||
import (
|
import (
|
||||||
"math/big"
|
"math/big"
|
||||||
|
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/blockchainer"
|
|
||||||
"github.com/nspcc-dev/neo-go/pkg/util"
|
"github.com/nspcc-dev/neo-go/pkg/util"
|
||||||
)
|
)
|
||||||
|
|
||||||
// NotaryFeer implements mempool.Feer interface for Notary balance handling.
|
// NotaryFeer implements mempool.Feer interface for Notary balance handling.
|
||||||
type NotaryFeer struct {
|
type NotaryFeer struct {
|
||||||
bc blockchainer.Blockchainer
|
bc Ledger
|
||||||
}
|
}
|
||||||
|
|
||||||
// FeePerByte implements mempool.Feer interface.
|
// FeePerByte implements mempool.Feer interface.
|
||||||
|
@ -33,7 +32,7 @@ func (f NotaryFeer) P2PSigExtensionsEnabled() bool {
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewNotaryFeer returns new NotaryFeer instance.
|
// NewNotaryFeer returns new NotaryFeer instance.
|
||||||
func NewNotaryFeer(bc blockchainer.Blockchainer) NotaryFeer {
|
func NewNotaryFeer(bc Ledger) NotaryFeer {
|
||||||
return NotaryFeer{
|
return NotaryFeer{
|
||||||
bc: bc,
|
bc: bc,
|
||||||
}
|
}
|
||||||
|
|
|
@ -5,6 +5,7 @@ import (
|
||||||
"encoding/binary"
|
"encoding/binary"
|
||||||
"errors"
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"math/big"
|
||||||
mrand "math/rand"
|
mrand "math/rand"
|
||||||
"net"
|
"net"
|
||||||
"sort"
|
"sort"
|
||||||
|
@ -12,9 +13,9 @@ import (
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/nspcc-dev/neo-go/pkg/config"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/config/netmode"
|
"github.com/nspcc-dev/neo-go/pkg/config/netmode"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/block"
|
"github.com/nspcc-dev/neo-go/pkg/core/block"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/blockchainer"
|
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/mempool"
|
"github.com/nspcc-dev/neo-go/pkg/core/mempool"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/mempoolevent"
|
"github.com/nspcc-dev/neo-go/pkg/core/mempoolevent"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/mpt"
|
"github.com/nspcc-dev/neo-go/pkg/core/mpt"
|
||||||
|
@ -48,6 +49,31 @@ var (
|
||||||
)
|
)
|
||||||
|
|
||||||
type (
|
type (
|
||||||
|
// Ledger is everything Server needs from the blockchain.
|
||||||
|
Ledger interface {
|
||||||
|
extpool.Ledger
|
||||||
|
mempool.Feer
|
||||||
|
Blockqueuer
|
||||||
|
GetBlock(hash util.Uint256) (*block.Block, error)
|
||||||
|
GetConfig() config.ProtocolConfiguration
|
||||||
|
GetHeader(hash util.Uint256) (*block.Header, error)
|
||||||
|
GetHeaderHash(int) util.Uint256
|
||||||
|
GetMaxVerificationGAS() int64
|
||||||
|
GetMemPool() *mempool.Pool
|
||||||
|
GetNotaryBalance(acc util.Uint160) *big.Int
|
||||||
|
GetNotaryContractScriptHash() util.Uint160
|
||||||
|
GetNotaryDepositExpiration(acc util.Uint160) uint32
|
||||||
|
GetTransaction(util.Uint256) (*transaction.Transaction, uint32, error)
|
||||||
|
HasBlock(util.Uint256) bool
|
||||||
|
HeaderHeight() uint32
|
||||||
|
P2PSigExtensionsEnabled() bool
|
||||||
|
PoolTx(t *transaction.Transaction, pools ...*mempool.Pool) error
|
||||||
|
PoolTxWithData(t *transaction.Transaction, data interface{}, mp *mempool.Pool, feer mempool.Feer, verificationFunction func(t *transaction.Transaction, data interface{}) error) error
|
||||||
|
RegisterPostBlock(f func(func(*transaction.Transaction, *mempool.Pool, bool) bool, *mempool.Pool, *block.Block))
|
||||||
|
SubscribeForBlocks(ch chan<- *block.Block)
|
||||||
|
UnsubscribeFromBlocks(ch chan<- *block.Block)
|
||||||
|
}
|
||||||
|
|
||||||
// Service is a service abstraction (oracle, state root, consensus, etc).
|
// Service is a service abstraction (oracle, state root, consensus, etc).
|
||||||
Service interface {
|
Service interface {
|
||||||
Start()
|
Start()
|
||||||
|
@ -70,7 +96,7 @@ type (
|
||||||
|
|
||||||
transport Transporter
|
transport Transporter
|
||||||
discovery Discoverer
|
discovery Discoverer
|
||||||
chain blockchainer.Blockchainer
|
chain Ledger
|
||||||
bQueue *blockQueue
|
bQueue *blockQueue
|
||||||
bSyncQueue *blockQueue
|
bSyncQueue *blockQueue
|
||||||
mempool *mempool.Pool
|
mempool *mempool.Pool
|
||||||
|
@ -119,13 +145,13 @@ func randomID() uint32 {
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewServer returns a new Server, initialized with the given configuration.
|
// NewServer returns a new Server, initialized with the given configuration.
|
||||||
func NewServer(config ServerConfig, chain blockchainer.Blockchainer, stSync StateSync, log *zap.Logger) (*Server, error) {
|
func NewServer(config ServerConfig, chain Ledger, stSync StateSync, log *zap.Logger) (*Server, error) {
|
||||||
return newServerFromConstructors(config, chain, stSync, log, func(s *Server) Transporter {
|
return newServerFromConstructors(config, chain, stSync, log, func(s *Server) Transporter {
|
||||||
return NewTCPTransport(s, net.JoinHostPort(s.ServerConfig.Address, strconv.Itoa(int(s.ServerConfig.Port))), s.log)
|
return NewTCPTransport(s, net.JoinHostPort(s.ServerConfig.Address, strconv.Itoa(int(s.ServerConfig.Port))), s.log)
|
||||||
}, newDefaultDiscovery)
|
}, newDefaultDiscovery)
|
||||||
}
|
}
|
||||||
|
|
||||||
func newServerFromConstructors(config ServerConfig, chain blockchainer.Blockchainer, stSync StateSync, log *zap.Logger,
|
func newServerFromConstructors(config ServerConfig, chain Ledger, stSync StateSync, log *zap.Logger,
|
||||||
newTransport func(*Server) Transporter,
|
newTransport func(*Server) Transporter,
|
||||||
newDiscovery func([]string, time.Duration, Transporter) Discoverer,
|
newDiscovery func([]string, time.Duration, Transporter) Discoverer,
|
||||||
) (*Server, error) {
|
) (*Server, error) {
|
||||||
|
@ -161,9 +187,9 @@ func newServerFromConstructors(config ServerConfig, chain blockchainer.Blockchai
|
||||||
if chain.P2PSigExtensionsEnabled() {
|
if chain.P2PSigExtensionsEnabled() {
|
||||||
s.notaryFeer = NewNotaryFeer(chain)
|
s.notaryFeer = NewNotaryFeer(chain)
|
||||||
s.notaryRequestPool = mempool.New(chain.GetConfig().P2PNotaryRequestPayloadPoolSize, 1, true)
|
s.notaryRequestPool = mempool.New(chain.GetConfig().P2PNotaryRequestPayloadPoolSize, 1, true)
|
||||||
chain.RegisterPostBlock(func(bc blockchainer.Blockchainer, txpool *mempool.Pool, _ *block.Block) {
|
chain.RegisterPostBlock(func(isRelevant func(*transaction.Transaction, *mempool.Pool, bool) bool, txpool *mempool.Pool, _ *block.Block) {
|
||||||
s.notaryRequestPool.RemoveStale(func(t *transaction.Transaction) bool {
|
s.notaryRequestPool.RemoveStale(func(t *transaction.Transaction) bool {
|
||||||
return bc.IsTxStillRelevant(t, txpool, true)
|
return isRelevant(t, txpool, true)
|
||||||
}, s.notaryFeer)
|
}, s.notaryFeer)
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
@ -604,7 +630,7 @@ func (s *Server) requestBlocksOrHeaders(p Peer) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
var (
|
var (
|
||||||
bq blockchainer.Blockqueuer = s.chain
|
bq Blockqueuer = s.chain
|
||||||
requestMPTNodes bool
|
requestMPTNodes bool
|
||||||
)
|
)
|
||||||
if s.stateSync.IsActive() {
|
if s.stateSync.IsActive() {
|
||||||
|
@ -974,21 +1000,21 @@ func (s *Server) RelayP2PNotaryRequest(r *payload.P2PNotaryRequest) error {
|
||||||
|
|
||||||
// verifyAndPoolNotaryRequest verifies NotaryRequest payload and adds it to the payload mempool.
|
// verifyAndPoolNotaryRequest verifies NotaryRequest payload and adds it to the payload mempool.
|
||||||
func (s *Server) verifyAndPoolNotaryRequest(r *payload.P2PNotaryRequest) error {
|
func (s *Server) verifyAndPoolNotaryRequest(r *payload.P2PNotaryRequest) error {
|
||||||
return s.chain.PoolTxWithData(r.FallbackTransaction, r, s.notaryRequestPool, s.notaryFeer, verifyNotaryRequest)
|
return s.chain.PoolTxWithData(r.FallbackTransaction, r, s.notaryRequestPool, s.notaryFeer, s.verifyNotaryRequest)
|
||||||
}
|
}
|
||||||
|
|
||||||
// verifyNotaryRequest is a function for state-dependant P2PNotaryRequest payload verification which is executed before ordinary blockchain's verification.
|
// verifyNotaryRequest is a function for state-dependant P2PNotaryRequest payload verification which is executed before ordinary blockchain's verification.
|
||||||
func verifyNotaryRequest(bc blockchainer.Blockchainer, _ *transaction.Transaction, data interface{}) error {
|
func (s *Server) verifyNotaryRequest(_ *transaction.Transaction, data interface{}) error {
|
||||||
r := data.(*payload.P2PNotaryRequest)
|
r := data.(*payload.P2PNotaryRequest)
|
||||||
payer := r.FallbackTransaction.Signers[1].Account
|
payer := r.FallbackTransaction.Signers[1].Account
|
||||||
if _, err := bc.VerifyWitness(payer, r, &r.Witness, bc.GetMaxVerificationGAS()); err != nil {
|
if _, err := s.chain.VerifyWitness(payer, r, &r.Witness, s.chain.GetMaxVerificationGAS()); err != nil {
|
||||||
return fmt.Errorf("bad P2PNotaryRequest payload witness: %w", err)
|
return fmt.Errorf("bad P2PNotaryRequest payload witness: %w", err)
|
||||||
}
|
}
|
||||||
notaryHash := bc.GetNotaryContractScriptHash()
|
notaryHash := s.chain.GetNotaryContractScriptHash()
|
||||||
if r.FallbackTransaction.Sender() != notaryHash {
|
if r.FallbackTransaction.Sender() != notaryHash {
|
||||||
return errors.New("P2PNotary contract should be a sender of the fallback transaction")
|
return errors.New("P2PNotary contract should be a sender of the fallback transaction")
|
||||||
}
|
}
|
||||||
depositExpiration := bc.GetNotaryDepositExpiration(payer)
|
depositExpiration := s.chain.GetNotaryDepositExpiration(payer)
|
||||||
if r.FallbackTransaction.ValidUntilBlock >= depositExpiration {
|
if r.FallbackTransaction.ValidUntilBlock >= depositExpiration {
|
||||||
return fmt.Errorf("fallback transaction is valid after deposit is unlocked: ValidUntilBlock is %d, deposit lock expires at %d", r.FallbackTransaction.ValidUntilBlock, depositExpiration)
|
return fmt.Errorf("fallback transaction is valid after deposit is unlocked: ValidUntilBlock is %d, deposit lock expires at %d", r.FallbackTransaction.ValidUntilBlock, depositExpiration)
|
||||||
}
|
}
|
||||||
|
@ -1043,7 +1069,7 @@ func (s *Server) handleGetAddrCmd(p Peer) error {
|
||||||
// 1. Block range is divided into chunks of payload.MaxHashesCount.
|
// 1. Block range is divided into chunks of payload.MaxHashesCount.
|
||||||
// 2. Send requests for chunk in increasing order.
|
// 2. Send requests for chunk in increasing order.
|
||||||
// 3. After all requests were sent, request random height.
|
// 3. After all requests were sent, request random height.
|
||||||
func (s *Server) requestBlocks(bq blockchainer.Blockqueuer, p Peer) error {
|
func (s *Server) requestBlocks(bq Blockqueuer, p Peer) error {
|
||||||
pl := getRequestBlocksPayload(p, bq.BlockHeight(), &s.lastRequestedBlock)
|
pl := getRequestBlocksPayload(p, bq.BlockHeight(), &s.lastRequestedBlock)
|
||||||
return p.EnqueueP2PMessage(NewMessage(CMDGetBlockByIndex, pl))
|
return p.EnqueueP2PMessage(NewMessage(CMDGetBlockByIndex, pl))
|
||||||
}
|
}
|
||||||
|
|
|
@ -26,6 +26,7 @@ import (
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
"go.uber.org/atomic"
|
"go.uber.org/atomic"
|
||||||
|
"go.uber.org/zap/zaptest"
|
||||||
)
|
)
|
||||||
|
|
||||||
type fakeConsensus struct {
|
type fakeConsensus struct {
|
||||||
|
@ -1008,6 +1009,9 @@ func TestVerifyNotaryRequest(t *testing.T) {
|
||||||
bc := fakechain.NewFakeChain()
|
bc := fakechain.NewFakeChain()
|
||||||
bc.MaxVerificationGAS = 10
|
bc.MaxVerificationGAS = 10
|
||||||
bc.NotaryContractScriptHash = util.Uint160{1, 2, 3}
|
bc.NotaryContractScriptHash = util.Uint160{1, 2, 3}
|
||||||
|
s, err := newServerFromConstructors(ServerConfig{}, bc, new(fakechain.FakeStateSync), zaptest.NewLogger(t), newFakeTransp, newTestDiscovery)
|
||||||
|
require.NoError(t, err)
|
||||||
|
t.Cleanup(s.Shutdown)
|
||||||
newNotaryRequest := func() *payload.P2PNotaryRequest {
|
newNotaryRequest := func() *payload.P2PNotaryRequest {
|
||||||
return &payload.P2PNotaryRequest{
|
return &payload.P2PNotaryRequest{
|
||||||
MainTransaction: &transaction.Transaction{Script: []byte{0, 1, 2}},
|
MainTransaction: &transaction.Transaction{Script: []byte{0, 1, 2}},
|
||||||
|
@ -1021,26 +1025,26 @@ func TestVerifyNotaryRequest(t *testing.T) {
|
||||||
|
|
||||||
t.Run("bad payload witness", func(t *testing.T) {
|
t.Run("bad payload witness", func(t *testing.T) {
|
||||||
bc.VerifyWitnessF = func() (int64, error) { return 0, errors.New("bad witness") }
|
bc.VerifyWitnessF = func() (int64, error) { return 0, errors.New("bad witness") }
|
||||||
require.Error(t, verifyNotaryRequest(bc, nil, newNotaryRequest()))
|
require.Error(t, s.verifyNotaryRequest(nil, newNotaryRequest()))
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("bad fallback sender", func(t *testing.T) {
|
t.Run("bad fallback sender", func(t *testing.T) {
|
||||||
bc.VerifyWitnessF = func() (int64, error) { return 0, nil }
|
bc.VerifyWitnessF = func() (int64, error) { return 0, nil }
|
||||||
r := newNotaryRequest()
|
r := newNotaryRequest()
|
||||||
r.FallbackTransaction.Signers[0] = transaction.Signer{Account: util.Uint160{7, 8, 9}}
|
r.FallbackTransaction.Signers[0] = transaction.Signer{Account: util.Uint160{7, 8, 9}}
|
||||||
require.Error(t, verifyNotaryRequest(bc, nil, r))
|
require.Error(t, s.verifyNotaryRequest(nil, r))
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("expired deposit", func(t *testing.T) {
|
t.Run("expired deposit", func(t *testing.T) {
|
||||||
r := newNotaryRequest()
|
r := newNotaryRequest()
|
||||||
bc.NotaryDepositExpiration = r.FallbackTransaction.ValidUntilBlock
|
bc.NotaryDepositExpiration = r.FallbackTransaction.ValidUntilBlock
|
||||||
require.Error(t, verifyNotaryRequest(bc, nil, r))
|
require.Error(t, s.verifyNotaryRequest(nil, r))
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("good", func(t *testing.T) {
|
t.Run("good", func(t *testing.T) {
|
||||||
r := newNotaryRequest()
|
r := newNotaryRequest()
|
||||||
bc.NotaryDepositExpiration = r.FallbackTransaction.ValidUntilBlock + 1
|
bc.NotaryDepositExpiration = r.FallbackTransaction.ValidUntilBlock + 1
|
||||||
require.NoError(t, verifyNotaryRequest(bc, nil, r))
|
require.NoError(t, s.verifyNotaryRequest(nil, r))
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1,7 +1,6 @@
|
||||||
package network
|
package network
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/blockchainer"
|
|
||||||
"github.com/nspcc-dev/neo-go/pkg/core/mpt"
|
"github.com/nspcc-dev/neo-go/pkg/core/mpt"
|
||||||
"github.com/nspcc-dev/neo-go/pkg/util"
|
"github.com/nspcc-dev/neo-go/pkg/util"
|
||||||
)
|
)
|
||||||
|
@ -9,7 +8,7 @@ import (
|
||||||
// StateSync represents state sync module.
|
// StateSync represents state sync module.
|
||||||
type StateSync interface {
|
type StateSync interface {
|
||||||
AddMPTNodes([][]byte) error
|
AddMPTNodes([][]byte) error
|
||||||
blockchainer.Blockqueuer // Blockqueuer interface
|
Blockqueuer
|
||||||
Init(currChainHeight uint32) error
|
Init(currChainHeight uint32) error
|
||||||
IsActive() bool
|
IsActive() bool
|
||||||
IsInitialized() bool
|
IsInitialized() bool
|
||||||
|
|
Loading…
Reference in a new issue