rpc: allow to track notary requests via Notification subsystem

This commit is contained in:
Anna Shaleva 2021-05-28 14:55:06 +03:00
parent 133b600c8c
commit 1dbf1d4310
8 changed files with 276 additions and 55 deletions

View file

@ -13,8 +13,8 @@ import (
// P2PNotaryRequest contains main and fallback transactions for the Notary service.
type P2PNotaryRequest struct {
MainTransaction *transaction.Transaction
FallbackTransaction *transaction.Transaction
MainTransaction *transaction.Transaction `json:"maintx"`
FallbackTransaction *transaction.Transaction `json:"fallbacktx"`
Witness transaction.Witness

View file

@ -16,6 +16,7 @@ import (
"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/mempoolevent"
"github.com/nspcc-dev/neo-go/pkg/core/transaction"
"github.com/nspcc-dev/neo-go/pkg/network/capability"
"github.com/nspcc-dev/neo-go/pkg/network/extpool"
@ -143,7 +144,7 @@ func newServerFromConstructors(config ServerConfig, chain blockchainer.Blockchai
}
if chain.P2PSigExtensionsEnabled() {
s.notaryFeer = NewNotaryFeer(chain)
s.notaryRequestPool = mempool.New(chain.GetConfig().P2PNotaryRequestPayloadPoolSize, 1, config.P2PNotaryCfg.Enabled)
s.notaryRequestPool = mempool.New(chain.GetConfig().P2PNotaryRequestPayloadPoolSize, 1, true)
chain.RegisterPostBlock(func(bc blockchainer.Blockchainer, txpool *mempool.Pool, _ *block.Block) {
s.notaryRequestPool.RemoveStale(func(t *transaction.Transaction) bool {
return bc.IsTxStillRelevant(t, txpool, true)
@ -295,6 +296,8 @@ func (s *Server) Shutdown() {
}
if s.notaryModule != nil {
s.notaryModule.Stop()
}
if s.chain.P2PSigExtensionsEnabled() {
s.notaryRequestPool.StopSubscriptions()
}
close(s.quit)
@ -449,13 +452,39 @@ func (s *Server) tryStartServices() {
if s.oracle != nil {
go s.oracle.Run()
}
if s.chain.P2PSigExtensionsEnabled() {
s.notaryRequestPool.RunSubscriptions() // WSClient is also a subscriber.
}
if s.notaryModule != nil {
s.notaryRequestPool.RunSubscriptions()
go s.notaryModule.Run()
}
}
}
// SubscribeForNotaryRequests adds given channel to a notary request event
// broadcasting, so when a new P2PNotaryRequest is received or an existing
// P2PNotaryRequest is removed from pool you'll receive it via this channel.
// Make sure it's read from regularly as not reading these events might affect
// other Server functions.
// Ensure that P2PSigExtensions are enabled before calling this method.
func (s *Server) SubscribeForNotaryRequests(ch chan<- mempoolevent.Event) {
if !s.chain.P2PSigExtensionsEnabled() {
panic("P2PSigExtensions are disabled")
}
s.notaryRequestPool.SubscribeForTransactions(ch)
}
// UnsubscribeFromNotaryRequests unsubscribes given channel from notary request
// notifications, you can close it afterwards. Passing non-subscribed channel
// is a no-op.
// Ensure that P2PSigExtensions are enabled before calling this method.
func (s *Server) UnsubscribeFromNotaryRequests(ch chan<- mempoolevent.Event) {
if !s.chain.P2PSigExtensionsEnabled() {
panic("P2PSigExtensions are disabled")
}
s.notaryRequestPool.UnsubscribeFromTransactions(ch)
}
// Peers returns the current list of peers connected to
// the server.
func (s *Server) Peers() map[Peer]bool {

View file

@ -149,6 +149,8 @@ readloop:
val = new(state.NotificationEvent)
case response.ExecutionEventID:
val = new(state.AppExecResult)
case response.NotaryRequestEventID:
val = new(response.NotaryRequestEvent)
case response.MissedEventID:
// No value.
default:
@ -300,6 +302,18 @@ func (c *WSClient) SubscribeForTransactionExecutions(state *string) (string, err
return c.performSubscription(params)
}
// SubscribeForNotaryRequests adds subscription for notary request payloads
// addition or removal events to this instance of client. It can be filtered by
// request sender's hash, or main tx signer's hash, nil value puts no such
// restrictions.
func (c *WSClient) SubscribeForNotaryRequests(sender *util.Uint160, mainSigner *util.Uint160) (string, error) {
params := request.NewRawParams("notary_request_event")
if sender != nil {
params.Values = append(params.Values, request.TxFilter{Sender: sender, Signer: mainSigner})
}
return c.performSubscription(params)
}
// Unsubscribe removes subscription for given event stream.
func (c *WSClient) Unsubscribe(id string) error {
return c.performUnsubscription(id)

View file

@ -3,11 +3,20 @@ package response
import (
"encoding/json"
"errors"
"github.com/nspcc-dev/neo-go/pkg/core/mempoolevent"
"github.com/nspcc-dev/neo-go/pkg/network/payload"
)
type (
// EventID represents an event type happening on the chain.
EventID byte
// NotaryRequestEvent represents P2PNotaryRequest event either added or removed
// from notary payload pool.
NotaryRequestEvent struct {
Type mempoolevent.Type `json:"type"`
NotaryRequest *payload.P2PNotaryRequest `json:"notaryrequest"`
}
)
const (
@ -22,6 +31,8 @@ const (
NotificationEventID
// ExecutionEventID is used for `transaction_executed` events.
ExecutionEventID
// NotaryRequestEventID is used for `notary_request_event` event.
NotaryRequestEventID
// MissedEventID notifies user of missed events.
MissedEventID EventID = 255
)
@ -37,6 +48,8 @@ func (e EventID) String() string {
return "notification_from_execution"
case ExecutionEventID:
return "transaction_executed"
case NotaryRequestEventID:
return "notary_request_event"
case MissedEventID:
return "event_missed"
default:
@ -55,6 +68,8 @@ func GetEventIDFromString(s string) (EventID, error) {
return NotificationEventID, nil
case "transaction_executed":
return ExecutionEventID, nil
case "notary_request_event":
return NotaryRequestEventID, nil
case "event_missed":
return MissedEventID, nil
default:

View file

@ -21,6 +21,7 @@ import (
"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/fee"
"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/state"
"github.com/nspcc-dev/neo-go/pkg/core/transaction"
@ -57,16 +58,18 @@ type (
https *http.Server
shutdown chan struct{}
subsLock sync.RWMutex
subscribers map[*subscriber]bool
blockSubs int
executionSubs int
notificationSubs int
transactionSubs int
blockCh chan *block.Block
executionCh chan *state.AppExecResult
notificationCh chan *state.NotificationEvent
transactionCh chan *transaction.Transaction
subsLock sync.RWMutex
subscribers map[*subscriber]bool
blockSubs int
executionSubs int
notificationSubs int
transactionSubs int
notaryRequestSubs int
blockCh chan *block.Block
executionCh chan *state.AppExecResult
notificationCh chan *state.NotificationEvent
transactionCh chan *transaction.Transaction
notaryRequestCh chan mempoolevent.Event
}
)
@ -174,10 +177,11 @@ func New(chain blockchainer.Blockchainer, conf rpc.Config, coreServer *network.S
subscribers: make(map[*subscriber]bool),
// These are NOT buffered to preserve original order of events.
blockCh: make(chan *block.Block),
executionCh: make(chan *state.AppExecResult),
notificationCh: make(chan *state.NotificationEvent),
transactionCh: make(chan *transaction.Transaction),
blockCh: make(chan *block.Block),
executionCh: make(chan *state.AppExecResult),
notificationCh: make(chan *state.NotificationEvent),
transactionCh: make(chan *transaction.Transaction),
notaryRequestCh: make(chan mempoolevent.Event),
}
}
@ -1448,6 +1452,9 @@ func (s *Server) subscribe(reqParams request.Params, sub *subscriber) (interface
if err != nil || event == response.MissedEventID {
return nil, response.ErrInvalidParams
}
if event == response.NotaryRequestEventID && !s.chain.P2PSigExtensionsEnabled() {
return nil, response.WrapErrorWithData(response.ErrInvalidParams, errors.New("P2PSigExtensions are disabled"))
}
// Optional filter.
var filter interface{}
if p := reqParams.Value(1); p != nil {
@ -1468,6 +1475,10 @@ func (s *Server) subscribe(reqParams request.Params, sub *subscriber) (interface
if p.Type != request.ExecutionFilterT {
return nil, response.ErrInvalidParams
}
case response.NotaryRequestEventID:
if p.Type != request.TxFilterT {
return nil, response.ErrInvalidParams
}
}
filter = p.Value
}
@ -1519,6 +1530,11 @@ func (s *Server) subscribeToChannel(event response.EventID) {
s.chain.SubscribeForExecutions(s.executionCh)
}
s.executionSubs++
case response.NotaryRequestEventID:
if s.notaryRequestSubs == 0 {
s.coreServer.SubscribeForNotaryRequests(s.notaryRequestCh)
}
s.notaryRequestSubs++
}
}
@ -1565,6 +1581,11 @@ func (s *Server) unsubscribeFromChannel(event response.EventID) {
if s.executionSubs == 0 {
s.chain.UnsubscribeFromExecutions(s.executionCh)
}
case response.NotaryRequestEventID:
s.notaryRequestSubs--
if s.notaryRequestSubs == 0 {
s.coreServer.UnsubscribeFromNotaryRequests(s.notaryRequestCh)
}
}
}
@ -1605,6 +1626,12 @@ chloop:
case tx := <-s.transactionCh:
resp.Event = response.TransactionEventID
resp.Payload[0] = tx
case e := <-s.notaryRequestCh:
resp.Event = response.NotaryRequestEventID
resp.Payload[0] = &response.NotaryRequestEvent{
Type: e.Type,
NotaryRequest: e.Data.(*payload.P2PNotaryRequest),
}
}
s.subsLock.RLock()
subloop:
@ -1657,6 +1684,9 @@ chloop:
s.chain.UnsubscribeFromTransactions(s.transactionCh)
s.chain.UnsubscribeFromNotifications(s.notificationCh)
s.chain.UnsubscribeFromExecutions(s.executionCh)
if s.chain.P2PSigExtensionsEnabled() {
s.coreServer.UnsubscribeFromNotaryRequests(s.notaryRequestCh)
}
s.subsLock.Unlock()
drainloop:
for {
@ -1665,6 +1695,7 @@ drainloop:
case <-s.executionCh:
case <-s.notificationCh:
case <-s.transactionCh:
case <-s.notaryRequestCh:
default:
break drainloop
}
@ -1675,6 +1706,7 @@ drainloop:
close(s.transactionCh)
close(s.notificationCh)
close(s.executionCh)
close(s.notaryRequestCh)
}
func (s *Server) blockHeightFromParam(param *request.Param) (int, *response.Error) {

View file

@ -1107,42 +1107,7 @@ func TestSubmitNotaryRequest(t *testing.T) {
})
t.Run("valid request", func(t *testing.T) {
sender := testchain.PrivateKeyByID(0) // owner of the deposit in testchain
mainTx := &transaction.Transaction{
Attributes: []transaction.Attribute{{Type: transaction.NotaryAssistedT, Value: &transaction.NotaryAssisted{NKeys: 1}}},
Script: []byte{byte(opcode.RET)},
ValidUntilBlock: 123,
Signers: []transaction.Signer{{Account: util.Uint160{1, 5, 9}}},
Scripts: []transaction.Witness{{
InvocationScript: []byte{1, 4, 7},
VerificationScript: []byte{3, 6, 9},
}},
}
fallbackTx := &transaction.Transaction{
Script: []byte{byte(opcode.RET)},
ValidUntilBlock: 123,
Attributes: []transaction.Attribute{
{Type: transaction.NotValidBeforeT, Value: &transaction.NotValidBefore{Height: 123}},
{Type: transaction.ConflictsT, Value: &transaction.Conflicts{Hash: mainTx.Hash()}},
{Type: transaction.NotaryAssistedT, Value: &transaction.NotaryAssisted{NKeys: 0}},
},
Signers: []transaction.Signer{{Account: chain.GetNotaryContractScriptHash()}, {Account: sender.GetScriptHash()}},
Scripts: []transaction.Witness{
{InvocationScript: append([]byte{byte(opcode.PUSHDATA1), 64}, make([]byte, 64)...), VerificationScript: []byte{}},
},
NetworkFee: 2_0000_0000,
}
fallbackTx.Scripts = append(fallbackTx.Scripts, transaction.Witness{
InvocationScript: append([]byte{byte(opcode.PUSHDATA1), 64}, sender.SignHashable(uint32(testchain.Network()), fallbackTx)...),
VerificationScript: sender.PublicKey().GetVerificationScript(),
})
p := &payload.P2PNotaryRequest{
MainTransaction: mainTx,
FallbackTransaction: fallbackTx,
}
p.Witness = transaction.Witness{
InvocationScript: append([]byte{byte(opcode.PUSHDATA1), 64}, sender.SignHashable(uint32(testchain.Network()), p)...),
VerificationScript: sender.PublicKey().GetVerificationScript(),
}
p := createValidNotaryRequest(chain, sender, 1)
bytes, err := p.Bytes()
require.NoError(t, err)
str := fmt.Sprintf(`"%s"`, base64.StdEncoding.EncodeToString(bytes))
@ -1150,6 +1115,50 @@ func TestSubmitNotaryRequest(t *testing.T) {
})
}
// createValidNotaryRequest creates and signs P2PNotaryRequest payload which can
// pass verification.
func createValidNotaryRequest(chain *core.Blockchain, sender *keys.PrivateKey, nonce uint32) *payload.P2PNotaryRequest {
h := chain.BlockHeight()
mainTx := &transaction.Transaction{
Nonce: nonce,
Attributes: []transaction.Attribute{{Type: transaction.NotaryAssistedT, Value: &transaction.NotaryAssisted{NKeys: 1}}},
Script: []byte{byte(opcode.RET)},
ValidUntilBlock: h + 100,
Signers: []transaction.Signer{{Account: sender.GetScriptHash()}},
Scripts: []transaction.Witness{{
InvocationScript: []byte{1, 4, 7},
VerificationScript: []byte{3, 6, 9},
}},
}
fallbackTx := &transaction.Transaction{
Script: []byte{byte(opcode.RET)},
ValidUntilBlock: h + 100,
Attributes: []transaction.Attribute{
{Type: transaction.NotValidBeforeT, Value: &transaction.NotValidBefore{Height: h + 50}},
{Type: transaction.ConflictsT, Value: &transaction.Conflicts{Hash: mainTx.Hash()}},
{Type: transaction.NotaryAssistedT, Value: &transaction.NotaryAssisted{NKeys: 0}},
},
Signers: []transaction.Signer{{Account: chain.GetNotaryContractScriptHash()}, {Account: sender.GetScriptHash()}},
Scripts: []transaction.Witness{
{InvocationScript: append([]byte{byte(opcode.PUSHDATA1), 64}, make([]byte, 64)...), VerificationScript: []byte{}},
},
NetworkFee: 2_0000_0000,
}
fallbackTx.Scripts = append(fallbackTx.Scripts, transaction.Witness{
InvocationScript: append([]byte{byte(opcode.PUSHDATA1), 64}, sender.SignHashable(uint32(testchain.Network()), fallbackTx)...),
VerificationScript: sender.PublicKey().GetVerificationScript(),
})
p := &payload.P2PNotaryRequest{
MainTransaction: mainTx,
FallbackTransaction: fallbackTx,
}
p.Witness = transaction.Witness{
InvocationScript: append([]byte{byte(opcode.PUSHDATA1), 64}, sender.SignHashable(uint32(testchain.Network()), p)...),
VerificationScript: sender.PublicKey().GetVerificationScript(),
}
return p
}
// testRPCProtocol runs a full set of tests using given callback to make actual
// calls. Some tests change the chain state, thus we reinitialize the chain from
// scratch here.

View file

@ -80,6 +80,21 @@ func (f *feed) Matches(r *response.Notification) bool {
filt := f.filter.(request.ExecutionFilter)
applog := r.Payload[0].(*state.AppExecResult)
return applog.VMState.String() == filt.State
case response.NotaryRequestEventID:
filt := f.filter.(request.TxFilter)
req := r.Payload[0].(*response.NotaryRequestEvent)
senderOk := filt.Sender == nil || req.NotaryRequest.FallbackTransaction.Signers[1].Account == *filt.Sender
signerOK := true
if filt.Signer != nil {
signerOK = false
for _, signer := range req.NotaryRequest.MainTransaction.Signers {
if signer.Account.Equals(*filt.Signer) {
signerOK = true
break
}
}
}
return senderOk && signerOK
}
return false
}

View file

@ -86,10 +86,12 @@ func callUnsubscribe(t *testing.T, ws *websocket.Conn, msgs <-chan []byte, id st
func TestSubscriptions(t *testing.T) {
var subIDs = make([]string, 0)
var subFeeds = []string{"block_added", "transaction_added", "notification_from_execution", "transaction_executed"}
var subFeeds = []string{"block_added", "transaction_added", "notification_from_execution", "transaction_executed", "notary_request_event"}
chain, rpcSrv, c, respMsgs, finishedFlag := initCleanServerAndWSClient(t)
go rpcSrv.coreServer.Start(make(chan error))
defer rpcSrv.coreServer.Shutdown()
defer chain.Close()
defer func() { _ = rpcSrv.Shutdown() }()
@ -133,6 +135,17 @@ func TestSubscriptions(t *testing.T) {
require.Equal(t, response.BlockEventID, resp.Event)
}
// We should manually add NotaryRequest to test notification.
sender := testchain.PrivateKeyByID(0)
err := rpcSrv.coreServer.RelayP2PNotaryRequest(createValidNotaryRequest(chain, sender, 1))
require.NoError(t, err)
for {
resp := getNotification(t, respMsgs)
if resp.Event == response.NotaryRequestEventID {
break
}
}
for _, id := range subIDs {
callUnsubscribe(t, c, respMsgs, id)
}
@ -277,6 +290,100 @@ func TestFilteredSubscriptions(t *testing.T) {
}
}
func TestFilteredNotaryRequestSubscriptions(t *testing.T) {
// We can't fit this into TestFilteredSubscriptions, because notary requests
// event doesn't depend on blocks events.
priv0 := testchain.PrivateKeyByID(0)
var goodSender = priv0.GetScriptHash()
var cases = map[string]struct {
params string
check func(*testing.T, *response.Notification)
}{
"matching sender": {
params: `["notary_request_event", {"sender":"` + goodSender.StringLE() + `"}]`,
check: func(t *testing.T, resp *response.Notification) {
rmap := resp.Payload[0].(map[string]interface{})
require.Equal(t, response.NotaryRequestEventID, resp.Event)
require.Equal(t, "added", rmap["type"].(string))
req := rmap["notaryrequest"].(map[string]interface{})
fbTx := req["fallbacktx"].(map[string]interface{})
sender := fbTx["signers"].([]interface{})[1].(map[string]interface{})["account"].(string)
require.Equal(t, "0x"+goodSender.StringLE(), sender)
},
},
"matching signer": {
params: `["notary_request_event", {"signer":"` + goodSender.StringLE() + `"}]`,
check: func(t *testing.T, resp *response.Notification) {
rmap := resp.Payload[0].(map[string]interface{})
require.Equal(t, response.NotaryRequestEventID, resp.Event)
require.Equal(t, "added", rmap["type"].(string))
req := rmap["notaryrequest"].(map[string]interface{})
mainTx := req["maintx"].(map[string]interface{})
signers := mainTx["signers"].([]interface{})
signer0 := signers[0].(map[string]interface{})
signer0acc := signer0["account"].(string)
require.Equal(t, "0x"+goodSender.StringLE(), signer0acc)
},
},
"matching sender and signer": {
params: `["notary_request_event", {"sender":"` + goodSender.StringLE() + `", "signer":"` + goodSender.StringLE() + `"}]`,
check: func(t *testing.T, resp *response.Notification) {
rmap := resp.Payload[0].(map[string]interface{})
require.Equal(t, response.NotaryRequestEventID, resp.Event)
require.Equal(t, "added", rmap["type"].(string))
req := rmap["notaryrequest"].(map[string]interface{})
mainTx := req["maintx"].(map[string]interface{})
fbTx := req["fallbacktx"].(map[string]interface{})
sender := fbTx["signers"].([]interface{})[1].(map[string]interface{})["account"].(string)
require.Equal(t, "0x"+goodSender.StringLE(), sender)
signers := mainTx["signers"].([]interface{})
signer0 := signers[0].(map[string]interface{})
signer0acc := signer0["account"].(string)
require.Equal(t, "0x"+goodSender.StringLE(), signer0acc)
},
},
}
chain, rpcSrv, c, respMsgs, finishedFlag := initCleanServerAndWSClient(t)
go rpcSrv.coreServer.Start(make(chan error, 1))
defer chain.Close()
defer func() { _ = rpcSrv.Shutdown() }()
// blocks are needed to make GAS deposit for priv0
blocks := getTestBlocks(t)
for _, b := range blocks {
require.NoError(t, chain.AddBlock(b))
}
var nonce uint32 = 100
for name, this := range cases {
t.Run(name, func(t *testing.T) {
subID := callSubscribe(t, c, respMsgs, this.params)
err := rpcSrv.coreServer.RelayP2PNotaryRequest(createValidNotaryRequest(chain, priv0, nonce))
require.NoError(t, err)
nonce++
var resp = new(response.Notification)
select {
case body := <-respMsgs:
require.NoError(t, json.Unmarshal(body, resp))
case <-time.After(time.Second):
t.Fatal("timeout waiting for event")
}
require.Equal(t, response.NotaryRequestEventID, resp.Event)
this.check(t, resp)
callUnsubscribe(t, c, respMsgs, subID)
})
}
finishedFlag.CAS(false, true)
c.Close()
}
func TestFilteredBlockSubscriptions(t *testing.T) {
// We can't fit this into TestFilteredSubscriptions, because it uses
// blocks as EOF events to wait for.