forked from TrueCloudLab/frostfs-node
[#337] morph: Move subscription logic to subscriber
Signed-off-by: Evgenii Stratonikov <e.stratonikov@yadro.com>
This commit is contained in:
parent
a5f118a987
commit
35fdf6f315
5 changed files with 258 additions and 507 deletions
|
@ -11,8 +11,8 @@ import (
|
|||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/block"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/state"
|
||||
"github.com/nspcc-dev/neo-go/pkg/neorpc"
|
||||
"github.com/nspcc-dev/neo-go/pkg/neorpc/result"
|
||||
"github.com/nspcc-dev/neo-go/pkg/rpcclient"
|
||||
"github.com/nspcc-dev/neo-go/pkg/util"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
@ -35,16 +35,27 @@ type (
|
|||
Close()
|
||||
}
|
||||
|
||||
subChannels struct {
|
||||
NotifyChan chan *state.ContainedNotificationEvent
|
||||
BlockChan chan *block.Block
|
||||
NotaryChan chan *result.NotaryRequestEvent
|
||||
}
|
||||
|
||||
subscriber struct {
|
||||
*sync.RWMutex
|
||||
log *logger.Logger
|
||||
client *client.Client
|
||||
|
||||
notifyChan chan *state.ContainedNotificationEvent
|
||||
|
||||
blockChan chan *block.Block
|
||||
|
||||
blockChan chan *block.Block
|
||||
notaryChan chan *result.NotaryRequestEvent
|
||||
|
||||
current subChannels
|
||||
|
||||
// cached subscription information
|
||||
subscribedEvents map[util.Uint160]bool
|
||||
subscribedNotaryEvents map[util.Uint160]bool
|
||||
subscribedToNewBlocks bool
|
||||
}
|
||||
|
||||
// Params is a group of Subscriber constructor parameters.
|
||||
|
@ -75,22 +86,28 @@ func (s *subscriber) SubscribeForNotification(contracts ...util.Uint160) error {
|
|||
s.Lock()
|
||||
defer s.Unlock()
|
||||
|
||||
notifyIDs := make(map[util.Uint160]struct{}, len(contracts))
|
||||
notifyIDs := make([]string, 0, len(contracts))
|
||||
|
||||
for i := range contracts {
|
||||
if s.subscribedEvents[contracts[i]] {
|
||||
continue
|
||||
}
|
||||
// subscribe to contract notifications
|
||||
err := s.client.SubscribeForExecutionNotifications(contracts[i])
|
||||
id, err := s.client.ReceiveExecutionNotifications(contracts[i], s.current.NotifyChan)
|
||||
if err != nil {
|
||||
// if there is some error, undo all subscriptions and return error
|
||||
for hash := range notifyIDs {
|
||||
_ = s.client.UnsubscribeContract(hash)
|
||||
for _, id := range notifyIDs {
|
||||
_ = s.client.Unsubscribe(id)
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
// save notification id
|
||||
notifyIDs[contracts[i]] = struct{}{}
|
||||
notifyIDs = append(notifyIDs, id)
|
||||
}
|
||||
for i := range contracts {
|
||||
s.subscribedEvents[contracts[i]] = true
|
||||
}
|
||||
|
||||
return nil
|
||||
|
@ -101,82 +118,34 @@ func (s *subscriber) Close() {
|
|||
}
|
||||
|
||||
func (s *subscriber) BlockNotifications() error {
|
||||
if err := s.client.SubscribeForNewBlocks(); err != nil {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
if s.subscribedToNewBlocks {
|
||||
return nil
|
||||
}
|
||||
if _, err := s.client.ReceiveBlocks(s.current.BlockChan); err != nil {
|
||||
return fmt.Errorf("could not subscribe for new block events: %w", err)
|
||||
}
|
||||
|
||||
s.subscribedToNewBlocks = true
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *subscriber) SubscribeForNotaryRequests(mainTXSigner util.Uint160) error {
|
||||
if err := s.client.SubscribeForNotaryRequests(mainTXSigner); err != nil {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
if s.subscribedNotaryEvents[mainTXSigner] {
|
||||
return nil
|
||||
}
|
||||
if _, err := s.client.ReceiveNotaryRequests(mainTXSigner, s.current.NotaryChan); err != nil {
|
||||
return fmt.Errorf("could not subscribe for notary request events: %w", err)
|
||||
}
|
||||
|
||||
s.subscribedNotaryEvents[mainTXSigner] = true
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *subscriber) routeNotifications(ctx context.Context) {
|
||||
notificationChan := s.client.NotificationChannel()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case notification, ok := <-notificationChan:
|
||||
if !ok {
|
||||
s.log.Warn(logs.SubscriberRemoteNotificationChannelHasBeenClosed)
|
||||
close(s.notifyChan)
|
||||
close(s.blockChan)
|
||||
close(s.notaryChan)
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
switch notification.Type {
|
||||
case neorpc.NotificationEventID:
|
||||
notifyEvent, ok := notification.Value.(*state.ContainedNotificationEvent)
|
||||
if !ok {
|
||||
s.log.Error(logs.SubscriberCantCastNotifyEventValueToTheNotifyStruct,
|
||||
zap.String("received type", fmt.Sprintf("%T", notification.Value)),
|
||||
)
|
||||
continue
|
||||
}
|
||||
|
||||
s.log.Debug(logs.SubscriberNewNotificationEventFromSidechain,
|
||||
zap.String("name", notifyEvent.Name),
|
||||
)
|
||||
|
||||
s.notifyChan <- notifyEvent
|
||||
case neorpc.BlockEventID:
|
||||
b, ok := notification.Value.(*block.Block)
|
||||
if !ok {
|
||||
s.log.Error(logs.SubscriberCantCastBlockEventValueToBlock,
|
||||
zap.String("received type", fmt.Sprintf("%T", notification.Value)),
|
||||
)
|
||||
continue
|
||||
}
|
||||
|
||||
s.blockChan <- b
|
||||
case neorpc.NotaryRequestEventID:
|
||||
notaryRequest, ok := notification.Value.(*result.NotaryRequestEvent)
|
||||
if !ok {
|
||||
s.log.Error(logs.SubscriberCantCastNotifyEventValueToTheNotaryRequestStruct,
|
||||
zap.String("received type", fmt.Sprintf("%T", notification.Value)),
|
||||
)
|
||||
continue
|
||||
}
|
||||
|
||||
s.notaryChan <- notaryRequest
|
||||
default:
|
||||
s.log.Debug(logs.SubscriberUnsupportedNotificationFromTheChain,
|
||||
zap.Uint8("type", uint8(notification.Type)),
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// New is a constructs Neo:Morph event listener and returns Subscriber interface.
|
||||
func New(ctx context.Context, p *Params) (Subscriber, error) {
|
||||
switch {
|
||||
|
@ -200,16 +169,170 @@ func New(ctx context.Context, p *Params) (Subscriber, error) {
|
|||
notifyChan: make(chan *state.ContainedNotificationEvent),
|
||||
blockChan: make(chan *block.Block),
|
||||
notaryChan: make(chan *result.NotaryRequestEvent),
|
||||
}
|
||||
|
||||
// Worker listens all events from neo-go websocket and puts them
|
||||
// into corresponding channel. It may be notifications, transactions,
|
||||
// new blocks. For now only notifications.
|
||||
current: newSubChannels(),
|
||||
|
||||
subscribedEvents: make(map[util.Uint160]bool),
|
||||
subscribedNotaryEvents: make(map[util.Uint160]bool),
|
||||
}
|
||||
// Worker listens all events from temporary NeoGo channel and puts them
|
||||
// into corresponding permanent channels.
|
||||
go sub.routeNotifications(ctx)
|
||||
|
||||
return sub, nil
|
||||
}
|
||||
|
||||
func (s *subscriber) routeNotifications(ctx context.Context) {
|
||||
var (
|
||||
// TODO: not needed after nspcc-dev/neo-go#2980.
|
||||
cliCh = s.client.NotificationChannel()
|
||||
restoreCh = make(chan bool)
|
||||
restoreInProgress bool
|
||||
)
|
||||
|
||||
routeloop:
|
||||
for {
|
||||
var connLost bool
|
||||
s.RLock()
|
||||
curr := s.current
|
||||
s.RUnlock()
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
break routeloop
|
||||
case ev, ok := <-curr.NotifyChan:
|
||||
if ok {
|
||||
s.notifyChan <- ev
|
||||
} else {
|
||||
connLost = true
|
||||
}
|
||||
case ev, ok := <-curr.BlockChan:
|
||||
if ok {
|
||||
s.blockChan <- ev
|
||||
} else {
|
||||
connLost = true
|
||||
}
|
||||
case ev, ok := <-curr.NotaryChan:
|
||||
if ok {
|
||||
s.notaryChan <- ev
|
||||
} else {
|
||||
connLost = true
|
||||
}
|
||||
case _, ok := <-cliCh:
|
||||
connLost = !ok
|
||||
case ok := <-restoreCh:
|
||||
restoreInProgress = false
|
||||
if !ok {
|
||||
connLost = true
|
||||
}
|
||||
}
|
||||
if connLost {
|
||||
if !restoreInProgress {
|
||||
restoreInProgress, cliCh = s.switchEndpoint(ctx, restoreCh)
|
||||
if !restoreInProgress {
|
||||
break routeloop
|
||||
}
|
||||
curr.drain()
|
||||
} else { // Avoid getting additional !ok events.
|
||||
s.Lock()
|
||||
s.current.NotifyChan = nil
|
||||
s.current.BlockChan = nil
|
||||
s.current.NotaryChan = nil
|
||||
s.Unlock()
|
||||
}
|
||||
}
|
||||
}
|
||||
close(s.notifyChan)
|
||||
close(s.blockChan)
|
||||
close(s.notaryChan)
|
||||
}
|
||||
|
||||
func (s *subscriber) switchEndpoint(ctx context.Context, finishCh chan<- bool) (bool, <-chan rpcclient.Notification) {
|
||||
s.log.Info("RPC connection lost, attempting reconnect")
|
||||
if !s.client.SwitchRPC(ctx) {
|
||||
s.log.Error("can't switch RPC node")
|
||||
return false, nil
|
||||
}
|
||||
|
||||
cliCh := s.client.NotificationChannel()
|
||||
|
||||
s.Lock()
|
||||
chs := newSubChannels()
|
||||
go func() {
|
||||
finishCh <- s.restoreSubscriptions(chs.NotifyChan, chs.BlockChan, chs.NotaryChan)
|
||||
}()
|
||||
s.current = chs
|
||||
s.Unlock()
|
||||
|
||||
return true, cliCh
|
||||
}
|
||||
|
||||
func newSubChannels() subChannels {
|
||||
return subChannels{
|
||||
NotifyChan: make(chan *state.ContainedNotificationEvent),
|
||||
BlockChan: make(chan *block.Block),
|
||||
NotaryChan: make(chan *result.NotaryRequestEvent),
|
||||
}
|
||||
}
|
||||
|
||||
func (s *subChannels) drain() {
|
||||
drainloop:
|
||||
for {
|
||||
select {
|
||||
case _, ok := <-s.NotifyChan:
|
||||
if !ok {
|
||||
s.NotifyChan = nil
|
||||
}
|
||||
case _, ok := <-s.BlockChan:
|
||||
if !ok {
|
||||
s.BlockChan = nil
|
||||
}
|
||||
case _, ok := <-s.NotaryChan:
|
||||
if !ok {
|
||||
s.NotaryChan = nil
|
||||
}
|
||||
default:
|
||||
break drainloop
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// restoreSubscriptions restores subscriptions according to
|
||||
// cached information about them.
|
||||
func (s *subscriber) restoreSubscriptions(notifCh chan<- *state.ContainedNotificationEvent,
|
||||
blCh chan<- *block.Block, notaryCh chan<- *result.NotaryRequestEvent) bool {
|
||||
var err error
|
||||
|
||||
// new block events restoration
|
||||
if s.subscribedToNewBlocks {
|
||||
_, err = s.client.ReceiveBlocks(blCh)
|
||||
if err != nil {
|
||||
s.log.Error(logs.ClientCouldNotRestoreBlockSubscriptionAfterRPCSwitch, zap.Error(err))
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
// notification events restoration
|
||||
for contract := range s.subscribedEvents {
|
||||
contract := contract // See https://github.com/nspcc-dev/neo-go/issues/2890
|
||||
_, err = s.client.ReceiveExecutionNotifications(contract, notifCh)
|
||||
if err != nil {
|
||||
s.log.Error(logs.ClientCouldNotRestoreNotificationSubscriptionAfterRPCSwitch, zap.Error(err))
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
// notary notification events restoration
|
||||
for signer := range s.subscribedNotaryEvents {
|
||||
signer := signer // See https://github.com/nspcc-dev/neo-go/issues/2890
|
||||
_, err = s.client.ReceiveNotaryRequests(signer, notaryCh)
|
||||
if err != nil {
|
||||
s.log.Error(logs.ClientCouldNotRestoreNotaryNotificationSubscriptionAfterRPCSwitch, zap.Error(err))
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
// awaitHeight checks if remote client has least expected block height and
|
||||
// returns error if it is not reached that height after timeout duration.
|
||||
// This function is required to avoid connections to unsynced RPC nodes, because
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue