morph/event: Remove more code #1546

Merged
fyrchik merged 4 commits from fyrchik/frostfs-node:fix-events into master 2024-12-11 07:39:50 +00:00
13 changed files with 83 additions and 318 deletions

View file

@ -223,27 +223,17 @@ func registerNotificationHandlers(scHash util.Uint160, lis event.Listener, parse
subs map[event.Type][]event.Handler, subs map[event.Type][]event.Handler,
) { ) {
for typ, handlers := range subs { for typ, handlers := range subs {
pi := event.NotificationParserInfo{}
pi.SetType(typ)
pi.SetScriptHash(scHash)
p, ok := parsers[typ] p, ok := parsers[typ]
if !ok { if !ok {
panic(fmt.Sprintf("missing parser for event %s", typ)) panic(fmt.Sprintf("missing parser for event %s", typ))
} }
pi.SetParser(p) lis.RegisterNotificationHandler(event.NotificationHandlerInfo{
Contract: scHash,
lis.SetNotificationParser(pi) Type: typ,
Parser: p,
for _, h := range handlers { Handlers: handlers,
hi := event.NotificationHandlerInfo{} })
hi.SetType(typ)
hi.SetScriptHash(scHash)
hi.SetHandler(h)
lis.RegisterNotificationHandler(hi)
}
} }
} }

View file

@ -164,17 +164,9 @@ const (
EventNotaryParserNotSet = "notary parser not set" EventNotaryParserNotSet = "notary parser not set"
EventCouldNotParseNotaryEvent = "could not parse notary event" EventCouldNotParseNotaryEvent = "could not parse notary event"
EventNotaryHandlersForParsedNotificationEventWereNotRegistered = "notary handlers for parsed notification event were not registered" EventNotaryHandlersForParsedNotificationEventWereNotRegistered = "notary handlers for parsed notification event were not registered"
EventIgnoreNilEventParser = "ignore nil event parser"
EventListenerHasBeenAlreadyStartedIgnoreParser = "listener has been already started, ignore parser"
EventRegisteredNewEventParser = "registered new event parser" EventRegisteredNewEventParser = "registered new event parser"
EventIgnoreNilEventHandler = "ignore nil event handler"
EventIgnoreHandlerOfEventWoParser = "ignore handler of event w/o parser"
EventRegisteredNewEventHandler = "registered new event handler" EventRegisteredNewEventHandler = "registered new event handler"
EventIgnoreNilNotaryEventParser = "ignore nil notary event parser"
EventListenerHasBeenAlreadyStartedIgnoreNotaryParser = "listener has been already started, ignore notary parser"
EventIgnoreNilNotaryEventHandler = "ignore nil notary event handler"
EventIgnoreHandlerOfNotaryEventWoParser = "ignore handler of notary event w/o parser" EventIgnoreHandlerOfNotaryEventWoParser = "ignore handler of notary event w/o parser"
EventIgnoreNilBlockHandler = "ignore nil block handler"
StorageOperation = "local object storage operation" StorageOperation = "local object storage operation"
BlobovniczaCreatingDirectoryForBoltDB = "creating directory for BoltDB" BlobovniczaCreatingDirectoryForBoltDB = "creating directory for BoltDB"
BlobovniczaOpeningBoltDB = "opening BoltDB" BlobovniczaOpeningBoltDB = "opening BoltDB"

View file

@ -8,7 +8,6 @@ type (
// ContractProcessor interface defines functions for binding event producers // ContractProcessor interface defines functions for binding event producers
// such as event.Listener and Timers with contract processor. // such as event.Listener and Timers with contract processor.
ContractProcessor interface { ContractProcessor interface {
ListenerNotificationParsers() []event.NotificationParserInfo
ListenerNotificationHandlers() []event.NotificationHandlerInfo ListenerNotificationHandlers() []event.NotificationHandlerInfo
ListenerNotaryParsers() []event.NotaryParserInfo ListenerNotaryParsers() []event.NotaryParserInfo
ListenerNotaryHandlers() []event.NotaryHandlerInfo ListenerNotaryHandlers() []event.NotaryHandlerInfo
@ -16,11 +15,6 @@ type (
) )
func connectListenerWithProcessor(l event.Listener, p ContractProcessor) { func connectListenerWithProcessor(l event.Listener, p ContractProcessor) {
// register notification parsers
for _, parser := range p.ListenerNotificationParsers() {
l.SetNotificationParser(parser)
}
// register notification handlers // register notification handlers
for _, handler := range p.ListenerNotificationHandlers() { for _, handler := range p.ListenerNotificationHandlers() {
l.RegisterNotificationHandler(handler) l.RegisterNotificationHandler(handler)

View file

@ -114,11 +114,6 @@ func (ap *Processor) SetParsedWallets(parsedWallets []util.Uint160) {
ap.pwLock.Unlock() ap.pwLock.Unlock()
} }
// ListenerNotificationParsers for the 'event.Listener' event producer.
func (ap *Processor) ListenerNotificationParsers() []event.NotificationParserInfo {
return nil
}
// ListenerNotificationHandlers for the 'event.Listener' event producer. // ListenerNotificationHandlers for the 'event.Listener' event producer.
func (ap *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo { func (ap *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo {
return nil return nil

View file

@ -88,32 +88,16 @@ func New(p *Params) (*Processor, error) {
}, nil }, nil
} }
// ListenerNotificationParsers for the 'event.Listener' event producer.
func (bp *Processor) ListenerNotificationParsers() []event.NotificationParserInfo {
var parsers []event.NotificationParserInfo
// new lock event
lock := event.NotificationParserInfo{}
lock.SetType(lockNotification)
lock.SetScriptHash(bp.balanceSC)
lock.SetParser(balanceEvent.ParseLock)
parsers = append(parsers, lock)
return parsers
}
// ListenerNotificationHandlers for the 'event.Listener' event producer. // ListenerNotificationHandlers for the 'event.Listener' event producer.
func (bp *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo { func (bp *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo {
var handlers []event.NotificationHandlerInfo return []event.NotificationHandlerInfo{
{
// lock handler Contract: bp.balanceSC,
lock := event.NotificationHandlerInfo{} Type: lockNotification,
lock.SetType(lockNotification) Parser: balanceEvent.ParseLock,
lock.SetScriptHash(bp.balanceSC) Handlers: []event.Handler{bp.handleLock},
lock.SetHandler(bp.handleLock) },
handlers = append(handlers, lock) }
return handlers
} }
// ListenerNotaryParsers for the 'event.Listener' event producer. // ListenerNotaryParsers for the 'event.Listener' event producer.

View file

@ -118,11 +118,6 @@ func New(p *Params) (*Processor, error) {
}, nil }, nil
} }
// ListenerNotificationParsers for the 'event.Listener' event producer.
func (cp *Processor) ListenerNotificationParsers() []event.NotificationParserInfo {
return nil
}
// ListenerNotificationHandlers for the 'event.Listener' event producer. // ListenerNotificationHandlers for the 'event.Listener' event producer.
func (cp *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo { func (cp *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo {
return nil return nil

View file

@ -142,70 +142,34 @@ func New(p *Params) (*Processor, error) {
}, nil }, nil
} }
// ListenerNotificationParsers for the 'event.Listener' event producer.
func (np *Processor) ListenerNotificationParsers() []event.NotificationParserInfo {
var (
parsers = make([]event.NotificationParserInfo, 0, 6)
p event.NotificationParserInfo
)
p.SetScriptHash(np.frostfsContract)
// deposit event
p.SetType(event.TypeFromString(depositNotification))
p.SetParser(frostfsEvent.ParseDeposit)
parsers = append(parsers, p)
// withdraw event
p.SetType(event.TypeFromString(withdrawNotification))
p.SetParser(frostfsEvent.ParseWithdraw)
parsers = append(parsers, p)
// cheque event
p.SetType(event.TypeFromString(chequeNotification))
p.SetParser(frostfsEvent.ParseCheque)
parsers = append(parsers, p)
// config event
p.SetType(event.TypeFromString(configNotification))
p.SetParser(frostfsEvent.ParseConfig)
parsers = append(parsers, p)
return parsers
}
// ListenerNotificationHandlers for the 'event.Listener' event producer. // ListenerNotificationHandlers for the 'event.Listener' event producer.
func (np *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo { func (np *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo {
var ( return []event.NotificationHandlerInfo{
handlers = make([]event.NotificationHandlerInfo, 0, 6) {
Contract: np.frostfsContract,
h event.NotificationHandlerInfo Type: event.TypeFromString(depositNotification),
) Parser: frostfsEvent.ParseDeposit,
Handlers: []event.Handler{np.handleDeposit},
h.SetScriptHash(np.frostfsContract) },
{
// deposit handler Contract: np.frostfsContract,
h.SetType(event.TypeFromString(depositNotification)) Type: event.TypeFromString(withdrawNotification),
h.SetHandler(np.handleDeposit) Parser: frostfsEvent.ParseWithdraw,
handlers = append(handlers, h) Handlers: []event.Handler{np.handleWithdraw},
},
// withdraw handler {
h.SetType(event.TypeFromString(withdrawNotification)) Contract: np.frostfsContract,
h.SetHandler(np.handleWithdraw) Type: event.TypeFromString(chequeNotification),
handlers = append(handlers, h) Parser: frostfsEvent.ParseCheque,
Handlers: []event.Handler{np.handleCheque},
// cheque handler },
h.SetType(event.TypeFromString(chequeNotification)) {
h.SetHandler(np.handleCheque) Contract: np.frostfsContract,
handlers = append(handlers, h) Type: event.TypeFromString(configNotification),
Parser: frostfsEvent.ParseConfig,
// config handler Handlers: []event.Handler{np.handleConfig},
h.SetType(event.TypeFromString(configNotification)) },
h.SetHandler(np.handleConfig) }
handlers = append(handlers, h)
return handlers
} }
// ListenerNotaryParsers for the 'event.Listener' event producer. // ListenerNotaryParsers for the 'event.Listener' event producer.

View file

@ -155,22 +155,16 @@ func New(p *Params) (*Processor, error) {
}, nil }, nil
} }
// ListenerNotificationParsers for the 'event.Listener' event producer.
func (gp *Processor) ListenerNotificationParsers() []event.NotificationParserInfo {
var pi event.NotificationParserInfo
pi.SetScriptHash(gp.designate)
pi.SetType(event.TypeFromString(native.DesignationEventName))
pi.SetParser(rolemanagement.ParseDesignate)
return []event.NotificationParserInfo{pi}
}
// ListenerNotificationHandlers for the 'event.Listener' event producer. // ListenerNotificationHandlers for the 'event.Listener' event producer.
func (gp *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo { func (gp *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo {
var hi event.NotificationHandlerInfo return []event.NotificationHandlerInfo{
hi.SetScriptHash(gp.designate) {
hi.SetType(event.TypeFromString(native.DesignationEventName)) Contract: gp.designate,
hi.SetHandler(gp.HandleAlphabetSync) Type: event.TypeFromString(native.DesignationEventName),
return []event.NotificationHandlerInfo{hi} Parser: rolemanagement.ParseDesignate,
Handlers: []event.Handler{gp.HandleAlphabetSync},
},
}
} }
// ListenerNotaryParsers for the 'event.Listener' event producer. // ListenerNotaryParsers for the 'event.Listener' event producer.

View file

@ -161,36 +161,16 @@ func New(p *Params) (*Processor, error) {
}, nil }, nil
} }
// ListenerNotificationParsers for the 'event.Listener' event producer.
func (np *Processor) ListenerNotificationParsers() []event.NotificationParserInfo {
parsers := make([]event.NotificationParserInfo, 0, 3)
var p event.NotificationParserInfo
p.SetScriptHash(np.netmapClient.ContractAddress())
// new epoch event
p.SetType(newEpochNotification)
p.SetParser(netmapEvent.ParseNewEpoch)
parsers = append(parsers, p)
return parsers
}
// ListenerNotificationHandlers for the 'event.Listener' event producer. // ListenerNotificationHandlers for the 'event.Listener' event producer.
func (np *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo { func (np *Processor) ListenerNotificationHandlers() []event.NotificationHandlerInfo {
handlers := make([]event.NotificationHandlerInfo, 0, 3) return []event.NotificationHandlerInfo{
{
var i event.NotificationHandlerInfo Contract: np.netmapClient.ContractAddress(),
Type: newEpochNotification,
i.SetScriptHash(np.netmapClient.ContractAddress()) Parser: netmapEvent.ParseNewEpoch,
Handlers: []event.Handler{np.handleNewEpoch},
// new epoch handler },
i.SetType(newEpochNotification) }
i.SetHandler(np.handleNewEpoch)
handlers = append(handlers, i)
return handlers
} }
// ListenerNotaryParsers for the 'event.Listener' event producer. // ListenerNotaryParsers for the 'event.Listener' event producer.

View file

@ -4,6 +4,7 @@ import (
"context" "context"
"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/util"
) )
// Handler is an Event processing function. // Handler is an Event processing function.
@ -16,19 +17,10 @@ type BlockHandler func(context.Context, *block.Block)
// the parameters of the handler of particular // the parameters of the handler of particular
// contract event. // contract event.
type NotificationHandlerInfo struct { type NotificationHandlerInfo struct {
scriptHashWithType Contract util.Uint160
Type Type
h Handler Parser NotificationParser
} Handlers []Handler
// SetHandler is an event handler setter.
func (s *NotificationHandlerInfo) SetHandler(v Handler) {
s.h = v
}
// Handler returns an event handler.
func (s NotificationHandlerInfo) Handler() Handler {
return s.h
} }
// NotaryHandlerInfo is a structure that groups // NotaryHandlerInfo is a structure that groups

View file

@ -33,13 +33,6 @@ type Listener interface {
// it could not be started. // it could not be started.
ListenWithError(context.Context, chan<- error) ListenWithError(context.Context, chan<- error)
// SetNotificationParser must set the parser of particular contract event.
//
// Parser of each event must be set once. All parsers must be set before Listen call.
//
// Must ignore nil parsers and all calls after listener has been started.
SetNotificationParser(NotificationParserInfo)
// RegisterNotificationHandler must register the event handler for particular notification event of contract. // RegisterNotificationHandler must register the event handler for particular notification event of contract.
// //
// The specified handler must be called after each capture and parsing of the event. // The specified handler must be called after each capture and parsing of the event.
@ -100,8 +93,6 @@ type listener struct {
startOnce, stopOnce sync.Once startOnce, stopOnce sync.Once
started bool
notificationParsers map[scriptHashWithType]NotificationParser notificationParsers map[scriptHashWithType]NotificationParser
notificationHandlers map[scriptHashWithType][]Handler notificationHandlers map[scriptHashWithType][]Handler
@ -171,9 +162,6 @@ func (l *listener) ListenWithError(ctx context.Context, intError chan<- error) {
} }
func (l *listener) listen(ctx context.Context, intError chan<- error) error { func (l *listener) listen(ctx context.Context, intError chan<- error) error {
// mark listener as started
l.started = true
subErrCh := make(chan error) subErrCh := make(chan error)
go l.subscribe(subErrCh) go l.subscribe(subErrCh)
@ -449,72 +437,29 @@ func (l *listener) parseAndHandleNotary(ctx context.Context, nr *result.NotaryRe
handler(ctx, event) handler(ctx, event)
} }
// SetNotificationParser sets the parser of particular contract event.
//
// Ignores nil and already set parsers.
// Ignores the parser if listener is started.
func (l *listener) SetNotificationParser(pi NotificationParserInfo) {
log := l.log.With(
zap.String("contract", pi.ScriptHash().StringLE()),
zap.Stringer("event_type", pi.getType()),
)
parser := pi.parser()
if parser == nil {
log.Info(context.Background(), logs.EventIgnoreNilEventParser)
return
}
l.mtx.Lock()
defer l.mtx.Unlock()
// check if the listener was started
if l.started {
log.Warn(context.Background(), logs.EventListenerHasBeenAlreadyStartedIgnoreParser)
return
}
// add event parser
if _, ok := l.notificationParsers[pi.scriptHashWithType]; !ok {
l.notificationParsers[pi.scriptHashWithType] = pi.parser()
}
log.Debug(context.Background(), logs.EventRegisteredNewEventParser)
}
// RegisterNotificationHandler registers the handler for particular notification event of contract. // RegisterNotificationHandler registers the handler for particular notification event of contract.
// //
// Ignores nil handlers. // Ignores nil handlers.
// Ignores handlers of event without parser. // Ignores handlers of event without parser.
func (l *listener) RegisterNotificationHandler(hi NotificationHandlerInfo) { func (l *listener) RegisterNotificationHandler(hi NotificationHandlerInfo) {
log := l.log.With( log := l.log.With(
zap.String("contract", hi.ScriptHash().StringLE()), zap.String("contract", hi.Contract.StringLE()),
zap.Stringer("event_type", hi.GetType()), zap.Stringer("event_type", hi.Type),
) )
handler := hi.Handler()
if handler == nil {
log.Warn(context.Background(), logs.EventIgnoreNilEventHandler)
return
}
// check if parser was set // check if parser was set
l.mtx.RLock()
_, ok := l.notificationParsers[hi.scriptHashWithType]
l.mtx.RUnlock()
if !ok {
log.Warn(context.Background(), logs.EventIgnoreHandlerOfEventWoParser)
return
}
// add event handler
l.mtx.Lock() l.mtx.Lock()
l.notificationHandlers[hi.scriptHashWithType] = append( defer l.mtx.Unlock()
l.notificationHandlers[hi.scriptHashWithType],
hi.Handler(), var k scriptHashWithType
k.hash = hi.Contract
k.typ = hi.Type
l.notificationParsers[k] = hi.Parser
l.notificationHandlers[k] = append(
l.notificationHandlers[k],
hi.Handlers...,
) )
l.mtx.Unlock()
log.Debug(context.Background(), logs.EventRegisteredNewEventHandler) log.Debug(context.Background(), logs.EventRegisteredNewEventHandler)
} }
@ -555,21 +500,9 @@ func (l *listener) SetNotaryParser(pi NotaryParserInfo) {
zap.Stringer("notary_type", pi.RequestType()), zap.Stringer("notary_type", pi.RequestType()),
) )
parser := pi.parser()
if parser == nil {
log.Info(context.Background(), logs.EventIgnoreNilNotaryEventParser)
return
}
l.mtx.Lock() l.mtx.Lock()
defer l.mtx.Unlock() defer l.mtx.Unlock()
// check if the listener was started
if l.started {
log.Warn(context.Background(), logs.EventListenerHasBeenAlreadyStartedIgnoreNotaryParser)
return
}
// add event parser // add event parser
if _, ok := l.notaryParsers[pi.notaryRequestTypes]; !ok { if _, ok := l.notaryParsers[pi.notaryRequestTypes]; !ok {
l.notaryParsers[pi.notaryRequestTypes] = pi.parser() l.notaryParsers[pi.notaryRequestTypes] = pi.parser()
@ -593,12 +526,6 @@ func (l *listener) RegisterNotaryHandler(hi NotaryHandlerInfo) {
zap.Stringer("notary type", hi.RequestType()), zap.Stringer("notary type", hi.RequestType()),
) )
handler := hi.Handler()
if handler == nil {
log.Warn(context.Background(), logs.EventIgnoreNilNotaryEventHandler)
return
}
// check if parser was set // check if parser was set
l.mtx.RLock() l.mtx.RLock()
_, ok := l.notaryParsers[hi.notaryRequestTypes] _, ok := l.notaryParsers[hi.notaryRequestTypes]
@ -627,11 +554,6 @@ func (l *listener) Stop() {
} }
func (l *listener) RegisterBlockHandler(handler BlockHandler) { func (l *listener) RegisterBlockHandler(handler BlockHandler) {
if handler == nil {
l.log.Warn(context.Background(), logs.EventIgnoreNilBlockHandler)
return
}
l.blockHandlers = append(l.blockHandlers, handler) l.blockHandlers = append(l.blockHandlers, handler)
} }

View file

@ -39,29 +39,19 @@ func TestEventHandling(t *testing.T) {
blockHandled <- true blockHandled <- true
}) })
key := scriptHashWithType{
scriptHashValue: scriptHashValue{
hash: util.Uint160{100},
},
typeValue: typeValue{
typ: TypeFromString("notification type"),
},
}
l.SetNotificationParser(NotificationParserInfo{
scriptHashWithType: key,
p: func(cne *state.ContainedNotificationEvent) (Event, error) {
return testNotificationEvent{source: cne}, nil
},
})
notificationHandled := make(chan bool) notificationHandled := make(chan bool)
handledNotifications := make([]Event, 0) handledNotifications := make([]Event, 0)
l.RegisterNotificationHandler(NotificationHandlerInfo{ l.RegisterNotificationHandler(NotificationHandlerInfo{
scriptHashWithType: key, Contract: util.Uint160{100},
h: func(_ context.Context, e Event) { Type: TypeFromString("notification type"),
handledNotifications = append(handledNotifications, e) Parser: func(cne *state.ContainedNotificationEvent) (Event, error) {
notificationHandled <- true return testNotificationEvent{source: cne}, nil
},
Handlers: []Handler{
func(_ context.Context, e Event) {
handledNotifications = append(handledNotifications, e)
notificationHandled <- true
},
}, },
}) })

View file

@ -11,15 +11,6 @@ import (
// from the StackItem list. // from the StackItem list.
type NotificationParser func(*state.ContainedNotificationEvent) (Event, error) type NotificationParser func(*state.ContainedNotificationEvent) (Event, error)
// NotificationParserInfo is a structure that groups
// the parameters of particular contract
// notification event parser.
type NotificationParserInfo struct {
scriptHashWithType
p NotificationParser
}
// NotaryPreparator constructs NotaryEvent // NotaryPreparator constructs NotaryEvent
// from the NotaryRequest event. // from the NotaryRequest event.
type NotaryPreparator interface { type NotaryPreparator interface {
@ -47,24 +38,6 @@ func (n *NotaryParserInfo) SetParser(p NotaryParser) {
n.p = p n.p = p
} }
// SetParser is an event parser setter.
func (s *NotificationParserInfo) SetParser(v NotificationParser) {
s.p = v
}
func (s NotificationParserInfo) parser() NotificationParser {
return s.p
}
// SetType is an event type setter.
func (s *NotificationParserInfo) SetType(v Type) {
s.typ = v
}
func (s NotificationParserInfo) getType() Type {
return s.typ
}
type wrongPrmNumber struct { type wrongPrmNumber struct {
exp, act int exp, act int
} }