From 69f7bd68199b91c4ef4be65b5701e4d45a250350 Mon Sep 17 00:00:00 2001 From: Jordan Krage Date: Tue, 6 Aug 2024 23:55:49 +0200 Subject: [PATCH 1/6] use services.Config.NewService/Engine (#13851) * use services.Config.NewService/Engine * feedback --- common/headtracker/head_broadcaster.go | 70 +++---- common/headtracker/head_listener.go | 73 ++++--- common/headtracker/head_tracker.go | 105 ++++------ core/bridges/cache.go | 76 ++----- core/chains/evm/headtracker/head_listener.go | 28 --- .../evm/headtracker/head_listener_test.go | 188 ++++++++--------- core/chains/evm/headtracker/head_tracker.go | 4 +- core/chains/evm/monitor/balance.go | 76 +++---- core/recovery/recover.go | 42 ++-- core/services/chainlink/application.go | 13 +- .../fluxmonitorv2/deviation_checker.go | 4 +- core/services/fluxmonitorv2/flux_monitor.go | 95 ++++----- .../fluxmonitorv2/flux_monitor_test.go | 16 +- core/services/fluxmonitorv2/helpers_test.go | 12 +- core/services/fluxmonitorv2/poll_manager.go | 10 +- core/services/nurse.go | 192 ++++++++---------- core/services/nurse_test.go | 3 +- .../relay/evm/functions/logpoller_wrapper.go | 116 +++++------ core/services/synchronization/helpers_test.go | 6 +- .../telemetry_ingress_batch_client.go | 133 ++++++------ .../telemetry_ingress_batch_worker.go | 64 ++---- .../telemetry_ingress_batch_worker_test.go | 4 - .../telemetry_ingress_client.go | 90 +++----- core/services/telemetry/manager.go | 92 ++++----- core/services/telemetry/manager_test.go | 2 +- 25 files changed, 608 insertions(+), 906 deletions(-) delete mode 100644 core/chains/evm/headtracker/head_listener.go diff --git a/common/headtracker/head_broadcaster.go b/common/headtracker/head_broadcaster.go index 7edcccfccbd..c81c61141f2 100644 --- a/common/headtracker/head_broadcaster.go +++ b/common/headtracker/head_broadcaster.go @@ -42,13 +42,12 @@ type HeadBroadcaster[H types.Head[BLOCK_HASH], BLOCK_HASH types.Hashable] interf } type headBroadcaster[H types.Head[BLOCK_HASH], BLOCK_HASH types.Hashable] struct { - services.StateMachine - logger logger.Logger + services.Service + eng *services.Engine + callbacks callbackSet[H, BLOCK_HASH] mailbox *mailbox.Mailbox[H] mutex sync.Mutex - chClose services.StopChan - wgDone sync.WaitGroup latest H lastCallbackID int } @@ -60,41 +59,29 @@ func NewHeadBroadcaster[ ]( lggr logger.Logger, ) HeadBroadcaster[H, BLOCK_HASH] { - return &headBroadcaster[H, BLOCK_HASH]{ - logger: logger.Named(lggr, "HeadBroadcaster"), + hb := &headBroadcaster[H, BLOCK_HASH]{ callbacks: make(callbackSet[H, BLOCK_HASH]), mailbox: mailbox.NewSingle[H](), - chClose: make(chan struct{}), } + hb.Service, hb.eng = services.Config{ + Name: "HeadBroadcaster", + Start: hb.start, + Close: hb.close, + }.NewServiceEngine(lggr) + return hb } -func (hb *headBroadcaster[H, BLOCK_HASH]) Start(context.Context) error { - return hb.StartOnce("HeadBroadcaster", func() error { - hb.wgDone.Add(1) - go hb.run() - return nil - }) -} - -func (hb *headBroadcaster[H, BLOCK_HASH]) Close() error { - return hb.StopOnce("HeadBroadcaster", func() error { - hb.mutex.Lock() - // clear all callbacks - hb.callbacks = make(callbackSet[H, BLOCK_HASH]) - hb.mutex.Unlock() - - close(hb.chClose) - hb.wgDone.Wait() - return nil - }) +func (hb *headBroadcaster[H, BLOCK_HASH]) start(context.Context) error { + hb.eng.Go(hb.run) + return nil } -func (hb *headBroadcaster[H, BLOCK_HASH]) Name() string { - return hb.logger.Name() -} - -func (hb *headBroadcaster[H, BLOCK_HASH]) HealthReport() map[string]error { - return map[string]error{hb.Name(): hb.Healthy()} +func (hb *headBroadcaster[H, BLOCK_HASH]) close() error { + hb.mutex.Lock() + // clear all callbacks + hb.callbacks = make(callbackSet[H, BLOCK_HASH]) + hb.mutex.Unlock() + return nil } func (hb *headBroadcaster[H, BLOCK_HASH]) BroadcastNewLongestChain(head H) { @@ -121,15 +108,13 @@ func (hb *headBroadcaster[H, BLOCK_HASH]) Subscribe(callback HeadTrackable[H, BL return } -func (hb *headBroadcaster[H, BLOCK_HASH]) run() { - defer hb.wgDone.Done() - +func (hb *headBroadcaster[H, BLOCK_HASH]) run(ctx context.Context) { for { select { - case <-hb.chClose: + case <-ctx.Done(): return case <-hb.mailbox.Notify(): - hb.executeCallbacks() + hb.executeCallbacks(ctx) } } } @@ -137,10 +122,10 @@ func (hb *headBroadcaster[H, BLOCK_HASH]) run() { // DEV: the head relayer makes no promises about head delivery! Subscribing // Jobs should expect to the relayer to skip heads if there is a large number of listeners // and all callbacks cannot be completed in the allotted time. -func (hb *headBroadcaster[H, BLOCK_HASH]) executeCallbacks() { +func (hb *headBroadcaster[H, BLOCK_HASH]) executeCallbacks(ctx context.Context) { head, exists := hb.mailbox.Retrieve() if !exists { - hb.logger.Info("No head to retrieve. It might have been skipped") + hb.eng.Info("No head to retrieve. It might have been skipped") return } @@ -149,7 +134,7 @@ func (hb *headBroadcaster[H, BLOCK_HASH]) executeCallbacks() { hb.latest = head hb.mutex.Unlock() - hb.logger.Debugw("Initiating callbacks", + hb.eng.Debugw("Initiating callbacks", "headNum", head.BlockNumber(), "numCallbacks", len(callbacks), ) @@ -157,9 +142,6 @@ func (hb *headBroadcaster[H, BLOCK_HASH]) executeCallbacks() { wg := sync.WaitGroup{} wg.Add(len(callbacks)) - ctx, cancel := hb.chClose.NewCtx() - defer cancel() - for _, callback := range callbacks { go func(trackable HeadTrackable[H, BLOCK_HASH]) { defer wg.Done() @@ -168,7 +150,7 @@ func (hb *headBroadcaster[H, BLOCK_HASH]) executeCallbacks() { defer cancel() trackable.OnNewLongestChain(cctx, head) elapsed := time.Since(start) - hb.logger.Debugw(fmt.Sprintf("Finished callback in %s", elapsed), + hb.eng.Debugw(fmt.Sprintf("Finished callback in %s", elapsed), "callbackType", reflect.TypeOf(trackable), "blockNumber", head.BlockNumber(), "time", elapsed) }(callback) } diff --git a/common/headtracker/head_listener.go b/common/headtracker/head_listener.go index 25715b35280..d240caab3c3 100644 --- a/common/headtracker/head_listener.go +++ b/common/headtracker/head_listener.go @@ -29,14 +29,15 @@ var ( }, []string{"ChainID"}) ) -// headHandler is a callback that handles incoming heads -type headHandler[H types.Head[BLOCK_HASH], BLOCK_HASH types.Hashable] func(ctx context.Context, header H) error +// HeadHandler is a callback that handles incoming heads +type HeadHandler[H types.Head[BLOCK_HASH], BLOCK_HASH types.Hashable] func(ctx context.Context, header H) error // HeadListener is a chain agnostic interface that manages connection of Client that receives heads from the blockchain node type HeadListener[H types.Head[BLOCK_HASH], BLOCK_HASH types.Hashable] interface { - // ListenForNewHeads kicks off the listen loop (not thread safe) - // done() must be executed upon leaving ListenForNewHeads() - ListenForNewHeads(onSubscribe func(), handleNewHead headHandler[H, BLOCK_HASH], done func()) + services.Service + + // ListenForNewHeads runs the listen loop (not thread safe) + ListenForNewHeads(ctx context.Context) // ReceivingHeads returns true if the listener is receiving heads (thread safe) ReceivingHeads() bool @@ -54,10 +55,13 @@ type headListener[ ID types.ID, BLOCK_HASH types.Hashable, ] struct { + services.Service + eng *services.Engine + config htrktypes.Config client htrktypes.Client[HTH, S, ID, BLOCK_HASH] - logger logger.Logger - chStop services.StopChan + onSubscription func(context.Context) + handleNewHead HeadHandler[HTH, BLOCK_HASH] chHeaders chan HTH headSubscription types.Subscription connected atomic.Bool @@ -74,38 +78,43 @@ func NewHeadListener[ lggr logger.Logger, client CLIENT, config htrktypes.Config, - chStop chan struct{}, + onSubscription func(context.Context), + handleNewHead HeadHandler[HTH, BLOCK_HASH], ) HeadListener[HTH, BLOCK_HASH] { - return &headListener[HTH, S, ID, BLOCK_HASH]{ - config: config, - client: client, - logger: logger.Named(lggr, "HeadListener"), - chStop: chStop, + hl := &headListener[HTH, S, ID, BLOCK_HASH]{ + config: config, + client: client, + onSubscription: onSubscription, + handleNewHead: handleNewHead, } + hl.Service, hl.eng = services.Config{ + Name: "HeadListener", + Start: hl.start, + }.NewServiceEngine(lggr) + return hl } -func (hl *headListener[HTH, S, ID, BLOCK_HASH]) Name() string { - return hl.logger.Name() +func (hl *headListener[HTH, S, ID, BLOCK_HASH]) start(context.Context) error { + hl.eng.Go(hl.ListenForNewHeads) + return nil } -func (hl *headListener[HTH, S, ID, BLOCK_HASH]) ListenForNewHeads(onSubscription func(), handleNewHead headHandler[HTH, BLOCK_HASH], done func()) { - defer done() +func (hl *headListener[HTH, S, ID, BLOCK_HASH]) ListenForNewHeads(ctx context.Context) { defer hl.unsubscribe() - ctx, cancel := hl.chStop.NewCtx() - defer cancel() - for { if !hl.subscribe(ctx) { break } - onSubscription() - err := hl.receiveHeaders(ctx, handleNewHead) + if hl.onSubscription != nil { + hl.onSubscription(ctx) + } + err := hl.receiveHeaders(ctx, hl.handleNewHead) if ctx.Err() != nil { break } else if err != nil { - hl.logger.Errorw("Error in new head subscription, unsubscribed", "err", err) + hl.eng.Errorw("Error in new head subscription, unsubscribed", "err", err) continue } break @@ -131,7 +140,7 @@ func (hl *headListener[HTH, S, ID, BLOCK_HASH]) HealthReport() map[string]error return map[string]error{hl.Name(): err} } -func (hl *headListener[HTH, S, ID, BLOCK_HASH]) receiveHeaders(ctx context.Context, handleNewHead headHandler[HTH, BLOCK_HASH]) error { +func (hl *headListener[HTH, S, ID, BLOCK_HASH]) receiveHeaders(ctx context.Context, handleNewHead HeadHandler[HTH, BLOCK_HASH]) error { var noHeadsAlarmC <-chan time.Time var noHeadsAlarmT *time.Ticker noHeadsAlarmDuration := hl.config.BlockEmissionIdleWarningThreshold() @@ -142,7 +151,7 @@ func (hl *headListener[HTH, S, ID, BLOCK_HASH]) receiveHeaders(ctx context.Conte for { select { - case <-hl.chStop: + case <-ctx.Done(): return nil case blockHeader, open := <-hl.chHeaders: @@ -158,13 +167,13 @@ func (hl *headListener[HTH, S, ID, BLOCK_HASH]) receiveHeaders(ctx context.Conte return errors.New("head listener: chHeaders prematurely closed") } if !blockHeader.IsValid() { - hl.logger.Error("got nil block header") + hl.eng.Error("got nil block header") continue } // Compare the chain ID of the block header to the chain ID of the client if !blockHeader.HasChainID() || blockHeader.ChainID().String() != chainId.String() { - hl.logger.Panicf("head listener for %s received block header for %s", chainId, blockHeader.ChainID()) + hl.eng.Panicf("head listener for %s received block header for %s", chainId, blockHeader.ChainID()) } promNumHeadsReceived.WithLabelValues(chainId.String()).Inc() @@ -184,7 +193,7 @@ func (hl *headListener[HTH, S, ID, BLOCK_HASH]) receiveHeaders(ctx context.Conte case <-noHeadsAlarmC: // We haven't received a head on the channel for a long time, log a warning - hl.logger.Warnf("have not received a head for %v", noHeadsAlarmDuration) + hl.eng.Warnf("have not received a head for %v", noHeadsAlarmDuration) hl.receivingHeads.Store(false) } } @@ -198,19 +207,19 @@ func (hl *headListener[HTH, S, ID, BLOCK_HASH]) subscribe(ctx context.Context) b for { hl.unsubscribe() - hl.logger.Debugf("Subscribing to new heads on chain %s", chainId.String()) + hl.eng.Debugf("Subscribing to new heads on chain %s", chainId.String()) select { - case <-hl.chStop: + case <-ctx.Done(): return false case <-time.After(subscribeRetryBackoff.Duration()): err := hl.subscribeToHead(ctx) if err != nil { promEthConnectionErrors.WithLabelValues(chainId.String()).Inc() - hl.logger.Warnw("Failed to subscribe to heads on chain", "chainID", chainId.String(), "err", err) + hl.eng.Warnw("Failed to subscribe to heads on chain", "chainID", chainId.String(), "err", err) } else { - hl.logger.Debugf("Subscribed to heads on chain %s", chainId.String()) + hl.eng.Debugf("Subscribed to heads on chain %s", chainId.String()) return true } } diff --git a/common/headtracker/head_tracker.go b/common/headtracker/head_tracker.go index 851458591b8..8546d856b67 100644 --- a/common/headtracker/head_tracker.go +++ b/common/headtracker/head_tracker.go @@ -5,7 +5,6 @@ import ( "errors" "fmt" "math/big" - "sync" "time" "github.com/prometheus/client_golang/prometheus" @@ -51,7 +50,9 @@ type headTracker[ ID types.ID, BLOCK_HASH types.Hashable, ] struct { - services.StateMachine + services.Service + eng *services.Engine + log logger.SugaredLogger headBroadcaster HeadBroadcaster[HTH, BLOCK_HASH] headSaver HeadSaver[HTH, BLOCK_HASH] @@ -64,8 +65,6 @@ type headTracker[ backfillMB *mailbox.Mailbox[HTH] broadcastMB *mailbox.Mailbox[HTH] headListener HeadListener[HTH, BLOCK_HASH] - chStop services.StopChan - wgDone sync.WaitGroup getNilHead func() HTH } @@ -85,52 +84,52 @@ func NewHeadTracker[ mailMon *mailbox.Monitor, getNilHead func() HTH, ) HeadTracker[HTH, BLOCK_HASH] { - chStop := make(chan struct{}) - lggr = logger.Named(lggr, "HeadTracker") - return &headTracker[HTH, S, ID, BLOCK_HASH]{ + ht := &headTracker[HTH, S, ID, BLOCK_HASH]{ headBroadcaster: headBroadcaster, client: client, chainID: client.ConfiguredChainID(), config: config, htConfig: htConfig, - log: logger.Sugared(lggr), backfillMB: mailbox.NewSingle[HTH](), broadcastMB: mailbox.New[HTH](HeadsBufferSize), - chStop: chStop, - headListener: NewHeadListener[HTH, S, ID, BLOCK_HASH](lggr, client, config, chStop), headSaver: headSaver, mailMon: mailMon, getNilHead: getNilHead, } + ht.Service, ht.eng = services.Config{ + Name: "HeadTracker", + NewSubServices: func(lggr logger.Logger) []services.Service { + ht.headListener = NewHeadListener[HTH, S, ID, BLOCK_HASH](lggr, client, config, + // NOTE: Always try to start the head tracker off with whatever the + // latest head is, without waiting for the subscription to send us one. + // + // In some cases the subscription will send us the most recent head + // anyway when we connect (but we should not rely on this because it is + // not specced). If it happens this is fine, and the head will be + // ignored as a duplicate. + func(ctx context.Context) { + err := ht.handleInitialHead(ctx) + if err != nil { + ht.log.Errorw("Error handling initial head", "err", err.Error()) + } + }, ht.handleNewHead) + return []services.Service{ht.headListener} + }, + Start: ht.start, + Close: ht.close, + }.NewServiceEngine(lggr) + ht.log = logger.Sugared(ht.eng) + return ht } // Start starts HeadTracker service. -func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) Start(ctx context.Context) error { - return ht.StartOnce("HeadTracker", func() error { - ht.log.Debugw("Starting HeadTracker", "chainID", ht.chainID) - // NOTE: Always try to start the head tracker off with whatever the - // latest head is, without waiting for the subscription to send us one. - // - // In some cases the subscription will send us the most recent head - // anyway when we connect (but we should not rely on this because it is - // not specced). If it happens this is fine, and the head will be - // ignored as a duplicate. - onSubscribe := func() { - err := ht.handleInitialHead(ctx) - if err != nil { - ht.log.Errorw("Error handling initial head", "err", err.Error()) - } - } +func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) start(context.Context) error { + ht.eng.Go(ht.backfillLoop) + ht.eng.Go(ht.broadcastLoop) - ht.wgDone.Add(3) - go ht.headListener.ListenForNewHeads(onSubscribe, ht.handleNewHead, ht.wgDone.Done) - go ht.backfillLoop() - go ht.broadcastLoop() + ht.mailMon.Monitor(ht.broadcastMB, "HeadTracker", "Broadcast", ht.chainID.String()) - ht.mailMon.Monitor(ht.broadcastMB, "HeadTracker", "Broadcast", ht.chainID.String()) - - return nil - }) + return nil } func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) handleInitialHead(ctx context.Context) error { @@ -176,23 +175,8 @@ func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) handleInitialHead(ctx context.Con return nil } -// Close stops HeadTracker service. -func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) Close() error { - return ht.StopOnce("HeadTracker", func() error { - close(ht.chStop) - ht.wgDone.Wait() - return ht.broadcastMB.Close() - }) -} - -func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) Name() string { - return ht.log.Name() -} - -func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) HealthReport() map[string]error { - report := map[string]error{ht.Name(): ht.Healthy()} - services.CopyHealth(report, ht.headListener.HealthReport()) - return report +func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) close() error { + return ht.broadcastMB.Close() } func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) Backfill(ctx context.Context, headWithChain HTH) (err error) { @@ -265,15 +249,13 @@ func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) handleNewHead(ctx context.Context promOldHead.WithLabelValues(ht.chainID.String()).Inc() err := fmt.Errorf("got very old block with number %d (highest seen was %d)", head.BlockNumber(), prevHead.BlockNumber()) ht.log.Critical("Got very old block. Either a very deep re-org occurred, one of the RPC nodes has gotten far out of sync, or the chain went backwards in block numbers. This node may not function correctly without manual intervention.", "err", err) - ht.SvcErrBuffer.Append(err) + ht.eng.EmitHealthErr(err) } } return nil } -func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) broadcastLoop() { - defer ht.wgDone.Done() - +func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) broadcastLoop(ctx context.Context) { samplingInterval := ht.htConfig.SamplingInterval() if samplingInterval > 0 { ht.log.Debugf("Head sampling is enabled - sampling interval is set to: %v", samplingInterval) @@ -281,7 +263,7 @@ func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) broadcastLoop() { defer debounceHead.Stop() for { select { - case <-ht.chStop: + case <-ctx.Done(): return case <-debounceHead.C: item := ht.broadcastMB.RetrieveLatestAndClear() @@ -295,7 +277,7 @@ func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) broadcastLoop() { ht.log.Info("Head sampling is disabled - callback will be called on every head") for { select { - case <-ht.chStop: + case <-ctx.Done(): return case <-ht.broadcastMB.Notify(): for { @@ -310,15 +292,10 @@ func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) broadcastLoop() { } } -func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) backfillLoop() { - defer ht.wgDone.Done() - - ctx, cancel := ht.chStop.NewCtx() - defer cancel() - +func (ht *headTracker[HTH, S, ID, BLOCK_HASH]) backfillLoop(ctx context.Context) { for { select { - case <-ht.chStop: + case <-ctx.Done(): return case <-ht.backfillMB.Notify(): for { diff --git a/core/bridges/cache.go b/core/bridges/cache.go index 4b5a6552447..e97874a35e5 100644 --- a/core/bridges/cache.go +++ b/core/bridges/cache.go @@ -10,11 +10,9 @@ import ( "golang.org/x/exp/maps" + "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/services" "github.com/smartcontractkit/chainlink-common/pkg/sqlutil" - - "github.com/smartcontractkit/chainlink/v2/core/logger" - "github.com/smartcontractkit/chainlink/v2/core/utils" ) const ( @@ -25,13 +23,11 @@ const ( type Cache struct { // dependencies and configurations ORM - lggr logger.Logger interval time.Duration // service state - services.StateMachine - wg sync.WaitGroup - chStop services.StopChan + services.Service + eng *services.Engine // data state bridgeTypesCache sync.Map @@ -43,17 +39,20 @@ var _ ORM = (*Cache)(nil) var _ services.Service = (*Cache)(nil) func NewCache(base ORM, lggr logger.Logger, upsertInterval time.Duration) *Cache { - return &Cache{ + c := &Cache{ ORM: base, - lggr: lggr.Named(CacheServiceName), interval: upsertInterval, - chStop: make(chan struct{}), bridgeLastValueCache: make(map[string]BridgeResponse), } + c.Service, c.eng = services.Config{ + Name: CacheServiceName, + Start: c.start, + }.NewServiceEngine(lggr) + return c } func (c *Cache) WithDataSource(ds sqlutil.DataSource) ORM { - return NewCache(NewORM(ds), c.lggr, c.interval) + return NewCache(NewORM(ds), c.eng, c.interval) } func (c *Cache) FindBridge(ctx context.Context, name BridgeName) (BridgeType, error) { @@ -190,51 +189,17 @@ func (c *Cache) UpsertBridgeResponse(ctx context.Context, dotId string, specId i return nil } -func (c *Cache) Start(_ context.Context) error { - return c.StartOnce(CacheServiceName, func() error { - c.wg.Add(1) - - go c.run() - - return nil - }) -} - -func (c *Cache) Close() error { - return c.StopOnce(CacheServiceName, func() error { - close(c.chStop) - c.wg.Wait() - - return nil - }) -} - -func (c *Cache) HealthReport() map[string]error { - return map[string]error{c.Name(): c.Healthy()} -} - -func (c *Cache) Name() string { - return c.lggr.Name() -} - -func (c *Cache) run() { - defer c.wg.Done() - - for { - timer := time.NewTimer(utils.WithJitter(c.interval)) +func (c *Cache) start(_ context.Context) error { + ticker := services.TickerConfig{ + Initial: c.interval, + JitterPct: services.DefaultJitter, + }.NewTicker(c.interval) + c.eng.GoTick(ticker, c.doBulkUpsert) - select { - case <-timer.C: - c.doBulkUpsert() - case <-c.chStop: - timer.Stop() - - return - } - } + return nil } -func (c *Cache) doBulkUpsert() { +func (c *Cache) doBulkUpsert(ctx context.Context) { c.mu.RLock() values := maps.Values(c.bridgeLastValueCache) c.mu.RUnlock() @@ -243,11 +208,8 @@ func (c *Cache) doBulkUpsert() { return } - ctx, cancel := c.chStop.NewCtx() - defer cancel() - if err := c.ORM.BulkUpsertBridgeResponse(ctx, values); err != nil { - c.lggr.Warnf("bulk upsert of bridge responses failed: %s", err.Error()) + c.eng.Warnf("bulk upsert of bridge responses failed: %s", err.Error()) } } diff --git a/core/chains/evm/headtracker/head_listener.go b/core/chains/evm/headtracker/head_listener.go deleted file mode 100644 index 04535a34868..00000000000 --- a/core/chains/evm/headtracker/head_listener.go +++ /dev/null @@ -1,28 +0,0 @@ -package headtracker - -import ( - "math/big" - - "github.com/ethereum/go-ethereum" - "github.com/ethereum/go-ethereum/common" - - "github.com/smartcontractkit/chainlink-common/pkg/logger" - "github.com/smartcontractkit/chainlink/v2/common/headtracker" - - htrktypes "github.com/smartcontractkit/chainlink/v2/common/headtracker/types" - evmclient "github.com/smartcontractkit/chainlink/v2/core/chains/evm/client" - evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" -) - -type headListener = headtracker.HeadListener[*evmtypes.Head, common.Hash] - -func NewHeadListener( - lggr logger.Logger, - ethClient evmclient.Client, - config htrktypes.Config, chStop chan struct{}, -) headListener { - return headtracker.NewHeadListener[ - *evmtypes.Head, - ethereum.Subscription, *big.Int, common.Hash, - ](lggr, ethClient, config, chStop) -} diff --git a/core/chains/evm/headtracker/head_listener_test.go b/core/chains/evm/headtracker/head_listener_test.go index 29b090bbffe..2e459af2a2b 100644 --- a/core/chains/evm/headtracker/head_listener_test.go +++ b/core/chains/evm/headtracker/head_listener_test.go @@ -16,9 +16,9 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" + "github.com/smartcontractkit/chainlink/v2/common/headtracker" commonmocks "github.com/smartcontractkit/chainlink/v2/common/types/mocks" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/config/toml" - "github.com/smartcontractkit/chainlink/v2/core/chains/evm/headtracker" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/testutils" evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" ) @@ -40,17 +40,10 @@ func Test_HeadListener_HappyPath(t *testing.T) { evmcfg := testutils.NewTestChainScopedConfig(t, func(c *toml.EVMConfig) { c.NoNewHeadsThreshold = &commonconfig.Duration{} }) - chStop := make(chan struct{}) - hl := headtracker.NewHeadListener(lggr, ethClient, evmcfg.EVM(), chStop) var headCount atomic.Int32 - handler := func(context.Context, *evmtypes.Head) error { - headCount.Add(1) - return nil - } - - subscribeAwaiter := testutils.NewAwaiter() unsubscribeAwaiter := testutils.NewAwaiter() + subscribeAwaiter := testutils.NewAwaiter() var chHeads chan<- *evmtypes.Head var chErr = make(chan error) var chSubErr <-chan error = chErr @@ -66,23 +59,23 @@ func Test_HeadListener_HappyPath(t *testing.T) { close(chErr) }) - doneAwaiter := testutils.NewAwaiter() - done := func() { - doneAwaiter.ItHappened() - } - go hl.ListenForNewHeads(func() {}, handler, done) - - subscribeAwaiter.AwaitOrFail(t, tests.WaitTimeout(t)) - require.Eventually(t, hl.Connected, tests.WaitTimeout(t), tests.TestInterval) + func() { + hl := headtracker.NewHeadListener(lggr, ethClient, evmcfg.EVM(), nil, func(context.Context, *evmtypes.Head) error { + headCount.Add(1) + return nil + }) + require.NoError(t, hl.Start(tests.Context(t))) + defer func() { assert.NoError(t, hl.Close()) }() - chHeads <- testutils.Head(0) - chHeads <- testutils.Head(1) - chHeads <- testutils.Head(2) + subscribeAwaiter.AwaitOrFail(t, tests.WaitTimeout(t)) + require.Eventually(t, hl.Connected, tests.WaitTimeout(t), tests.TestInterval) - require.True(t, hl.ReceivingHeads()) + chHeads <- testutils.Head(0) + chHeads <- testutils.Head(1) + chHeads <- testutils.Head(2) - close(chStop) - doneAwaiter.AwaitOrFail(t) + require.True(t, hl.ReceivingHeads()) + }() unsubscribeAwaiter.AwaitOrFail(t) require.Equal(t, int32(3), headCount.Load()) @@ -101,14 +94,8 @@ func Test_HeadListener_NotReceivingHeads(t *testing.T) { evmcfg := testutils.NewTestChainScopedConfig(t, func(c *toml.EVMConfig) { c.NoNewHeadsThreshold = commonconfig.MustNewDuration(time.Second) }) - chStop := make(chan struct{}) - hl := headtracker.NewHeadListener(lggr, ethClient, evmcfg.EVM(), chStop) firstHeadAwaiter := testutils.NewAwaiter() - handler := func(context.Context, *evmtypes.Head) error { - firstHeadAwaiter.ItHappened() - return nil - } subscribeAwaiter := testutils.NewAwaiter() var chHeads chan<- *evmtypes.Head @@ -125,25 +112,25 @@ func Test_HeadListener_NotReceivingHeads(t *testing.T) { close(chErr) }) - doneAwaiter := testutils.NewAwaiter() - done := func() { - doneAwaiter.ItHappened() - } - go hl.ListenForNewHeads(func() {}, handler, done) - - subscribeAwaiter.AwaitOrFail(t, tests.WaitTimeout(t)) + func() { + hl := headtracker.NewHeadListener(lggr, ethClient, evmcfg.EVM(), nil, func(context.Context, *evmtypes.Head) error { + firstHeadAwaiter.ItHappened() + return nil + }) + require.NoError(t, hl.Start(tests.Context(t))) + defer func() { assert.NoError(t, hl.Close()) }() - chHeads <- testutils.Head(0) - firstHeadAwaiter.AwaitOrFail(t) + subscribeAwaiter.AwaitOrFail(t, tests.WaitTimeout(t)) - require.True(t, hl.ReceivingHeads()) + chHeads <- testutils.Head(0) + firstHeadAwaiter.AwaitOrFail(t) - time.Sleep(time.Second * 2) + require.True(t, hl.ReceivingHeads()) - require.False(t, hl.ReceivingHeads()) + time.Sleep(time.Second * 2) - close(chStop) - doneAwaiter.AwaitOrFail(t) + require.False(t, hl.ReceivingHeads()) + }() } func Test_HeadListener_SubscriptionErr(t *testing.T) { @@ -161,19 +148,11 @@ func Test_HeadListener_SubscriptionErr(t *testing.T) { for _, test := range cases { test := test t.Run(test.name, func(t *testing.T) { - l := logger.Test(t) + lggr := logger.Test(t) ethClient := testutils.NewEthClientMockWithDefaultChain(t) evmcfg := testutils.NewTestChainScopedConfig(t, nil) - chStop := make(chan struct{}) - hl := headtracker.NewHeadListener(l, ethClient, evmcfg.EVM(), chStop) hnhCalled := make(chan *evmtypes.Head) - hnh := func(_ context.Context, header *evmtypes.Head) error { - hnhCalled <- header - return nil - } - doneAwaiter := testutils.NewAwaiter() - done := doneAwaiter.ItHappened chSubErrTest := make(chan error) var chSubErr <-chan error = chSubErrTest @@ -189,63 +168,66 @@ func Test_HeadListener_SubscriptionErr(t *testing.T) { headsCh = args.Get(1).(chan<- *evmtypes.Head) subscribeAwaiter.ItHappened() }) - go func() { - hl.ListenForNewHeads(func() {}, hnh, done) - }() - - // Put a head on the channel to ensure we test all code paths - subscribeAwaiter.AwaitOrFail(t, tests.WaitTimeout(t)) - head := testutils.Head(0) - headsCh <- head - - h := <-hnhCalled - assert.Equal(t, head, h) - - // Expect a call to unsubscribe on error - sub.On("Unsubscribe").Once().Run(func(_ mock.Arguments) { - close(headsCh) - // geth guarantees that Unsubscribe closes the errors channel - if !test.closeErr { + func() { + hl := headtracker.NewHeadListener(lggr, ethClient, evmcfg.EVM(), nil, func(_ context.Context, header *evmtypes.Head) error { + hnhCalled <- header + return nil + }) + require.NoError(t, hl.Start(tests.Context(t))) + defer func() { assert.NoError(t, hl.Close()) }() + + // Put a head on the channel to ensure we test all code paths + subscribeAwaiter.AwaitOrFail(t, tests.WaitTimeout(t)) + head := testutils.Head(0) + headsCh <- head + + h := <-hnhCalled + assert.Equal(t, head, h) + + // Expect a call to unsubscribe on error + sub.On("Unsubscribe").Once().Run(func(_ mock.Arguments) { + close(headsCh) + // geth guarantees that Unsubscribe closes the errors channel + if !test.closeErr { + close(chSubErrTest) + } + }) + // Expect a resubscribe + chSubErrTest2 := make(chan error) + var chSubErr2 <-chan error = chSubErrTest2 + sub2 := commonmocks.NewSubscription(t) + sub2.On("Err").Return(chSubErr2) + subscribeAwaiter2 := testutils.NewAwaiter() + + var headsCh2 chan<- *evmtypes.Head + ethClient.On("SubscribeNewHead", mock.Anything, mock.AnythingOfType("chan<- *types.Head")).Return(sub2, nil).Once().Run(func(args mock.Arguments) { + headsCh2 = args.Get(1).(chan<- *evmtypes.Head) + subscribeAwaiter2.ItHappened() + }) + + // Sending test error + if test.closeErr { close(chSubErrTest) + } else { + chSubErrTest <- test.err } - }) - // Expect a resubscribe - chSubErrTest2 := make(chan error) - var chSubErr2 <-chan error = chSubErrTest2 - sub2 := commonmocks.NewSubscription(t) - sub2.On("Err").Return(chSubErr2) - subscribeAwaiter2 := testutils.NewAwaiter() - - var headsCh2 chan<- *evmtypes.Head - ethClient.On("SubscribeNewHead", mock.Anything, mock.AnythingOfType("chan<- *types.Head")).Return(sub2, nil).Once().Run(func(args mock.Arguments) { - headsCh2 = args.Get(1).(chan<- *evmtypes.Head) - subscribeAwaiter2.ItHappened() - }) - - // Sending test error - if test.closeErr { - close(chSubErrTest) - } else { - chSubErrTest <- test.err - } - // Wait for it to resubscribe - subscribeAwaiter2.AwaitOrFail(t, tests.WaitTimeout(t)) + // Wait for it to resubscribe + subscribeAwaiter2.AwaitOrFail(t, tests.WaitTimeout(t)) - head2 := testutils.Head(1) - headsCh2 <- head2 + head2 := testutils.Head(1) + headsCh2 <- head2 - h2 := <-hnhCalled - assert.Equal(t, head2, h2) + h2 := <-hnhCalled + assert.Equal(t, head2, h2) - // Second call to unsubscribe on close - sub2.On("Unsubscribe").Once().Run(func(_ mock.Arguments) { - close(headsCh2) - // geth guarantees that Unsubscribe closes the errors channel - close(chSubErrTest2) - }) - close(chStop) - doneAwaiter.AwaitOrFail(t) + // Second call to unsubscribe on close + sub2.On("Unsubscribe").Once().Run(func(_ mock.Arguments) { + close(headsCh2) + // geth guarantees that Unsubscribe closes the errors channel + close(chSubErrTest2) + }) + }() }) } } diff --git a/core/chains/evm/headtracker/head_tracker.go b/core/chains/evm/headtracker/head_tracker.go index d6c2cdc64e7..f7607189f7e 100644 --- a/core/chains/evm/headtracker/head_tracker.go +++ b/core/chains/evm/headtracker/head_tracker.go @@ -2,10 +2,8 @@ package headtracker import ( "context" - "math/big" "github.com/ethereum/go-ethereum" - "github.com/ethereum/go-ethereum/common" "go.uber.org/zap/zapcore" "github.com/smartcontractkit/chainlink-common/pkg/logger" @@ -27,7 +25,7 @@ func NewHeadTracker( headSaver httypes.HeadSaver, mailMon *mailbox.Monitor, ) httypes.HeadTracker { - return headtracker.NewHeadTracker[*evmtypes.Head, ethereum.Subscription, *big.Int, common.Hash]( + return headtracker.NewHeadTracker[*evmtypes.Head, ethereum.Subscription]( lggr, ethClient, config, diff --git a/core/chains/evm/monitor/balance.go b/core/chains/evm/monitor/balance.go index b8194a38af9..3e28d5c436a 100644 --- a/core/chains/evm/monitor/balance.go +++ b/core/chains/evm/monitor/balance.go @@ -33,14 +33,15 @@ type ( } balanceMonitor struct { - services.StateMachine - logger logger.Logger + services.Service + eng *services.Engine + ethClient evmclient.Client chainID *big.Int chainIDStr string ethKeyStore keystore.Eth ethBalances map[gethCommon.Address]*assets.Eth - ethBalancesMtx *sync.RWMutex + ethBalancesMtx sync.RWMutex sleeperTask *utils.SleeperTask } @@ -53,59 +54,42 @@ var _ BalanceMonitor = (*balanceMonitor)(nil) func NewBalanceMonitor(ethClient evmclient.Client, ethKeyStore keystore.Eth, lggr logger.Logger) *balanceMonitor { chainId := ethClient.ConfiguredChainID() bm := &balanceMonitor{ - services.StateMachine{}, - logger.Named(lggr, "BalanceMonitor"), - ethClient, - chainId, - chainId.String(), - ethKeyStore, - make(map[gethCommon.Address]*assets.Eth), - new(sync.RWMutex), - nil, + ethClient: ethClient, + chainID: chainId, + chainIDStr: chainId.String(), + ethKeyStore: ethKeyStore, + ethBalances: make(map[gethCommon.Address]*assets.Eth), } + bm.Service, bm.eng = services.Config{ + Name: "BalanceMonitor", + Start: bm.start, + Close: bm.close, + }.NewServiceEngine(lggr) bm.sleeperTask = utils.NewSleeperTask(&worker{bm: bm}) return bm } -func (bm *balanceMonitor) Start(ctx context.Context) error { - return bm.StartOnce("BalanceMonitor", func() error { - // Always query latest balance on start - (&worker{bm}).WorkCtx(ctx) - return nil - }) -} - -// Close shuts down the BalanceMonitor, should not be used after this -func (bm *balanceMonitor) Close() error { - return bm.StopOnce("BalanceMonitor", func() error { - return bm.sleeperTask.Stop() - }) -} - -func (bm *balanceMonitor) Ready() error { +func (bm *balanceMonitor) start(ctx context.Context) error { + // Always query latest balance on start + (&worker{bm}).WorkCtx(ctx) return nil } -func (bm *balanceMonitor) Name() string { - return bm.logger.Name() -} - -func (bm *balanceMonitor) HealthReport() map[string]error { - return map[string]error{bm.Name(): bm.Healthy()} +// Close shuts down the BalanceMonitor, should not be used after this +func (bm *balanceMonitor) close() error { + return bm.sleeperTask.Stop() } // OnNewLongestChain checks the balance for each key -func (bm *balanceMonitor) OnNewLongestChain(_ context.Context, head *evmtypes.Head) { - ok := bm.IfStarted(func() { - bm.checkBalance(head) - }) +func (bm *balanceMonitor) OnNewLongestChain(_ context.Context, _ *evmtypes.Head) { + ok := bm.sleeperTask.IfStarted(bm.checkBalances) if !ok { - bm.logger.Debugw("BalanceMonitor: ignoring OnNewLongestChain call, balance monitor is not started", "state", bm.State()) + bm.eng.Debugw("BalanceMonitor: ignoring OnNewLongestChain call, balance monitor is not started", "state", bm.sleeperTask.State()) } } -func (bm *balanceMonitor) checkBalance(head *evmtypes.Head) { - bm.logger.Debugw("BalanceMonitor: signalling balance worker") +func (bm *balanceMonitor) checkBalances() { + bm.eng.Debugw("BalanceMonitor: signalling balance worker") bm.sleeperTask.WakeUp() } @@ -117,7 +101,7 @@ func (bm *balanceMonitor) updateBalance(ethBal assets.Eth, address gethCommon.Ad bm.ethBalances[address] = ðBal bm.ethBalancesMtx.Unlock() - lgr := logger.Named(bm.logger, "BalanceLog") + lgr := logger.Named(bm.eng, "BalanceLog") lgr = logger.With(lgr, "address", address.Hex(), "ethBalance", ethBal.String(), @@ -151,7 +135,7 @@ func (bm *balanceMonitor) promUpdateEthBalance(balance *assets.Eth, from gethCom balanceFloat, err := ApproximateFloat64(balance) if err != nil { - bm.logger.Error(fmt.Errorf("updatePrometheusEthBalance: %v", err)) + bm.eng.Error(fmt.Errorf("updatePrometheusEthBalance: %v", err)) return } @@ -174,7 +158,7 @@ func (w *worker) Work() { func (w *worker) WorkCtx(ctx context.Context) { enabledAddresses, err := w.bm.ethKeyStore.EnabledAddressesForChain(ctx, w.bm.chainID) if err != nil { - w.bm.logger.Error("BalanceMonitor: error getting keys", err) + w.bm.eng.Error("BalanceMonitor: error getting keys", err) } var wg sync.WaitGroup @@ -198,12 +182,12 @@ func (w *worker) checkAccountBalance(ctx context.Context, address gethCommon.Add bal, err := w.bm.ethClient.BalanceAt(ctx, address, nil) if err != nil { - w.bm.logger.Errorw(fmt.Sprintf("BalanceMonitor: error getting balance for key %s", address.Hex()), + w.bm.eng.Errorw(fmt.Sprintf("BalanceMonitor: error getting balance for key %s", address.Hex()), "err", err, "address", address, ) } else if bal == nil { - w.bm.logger.Errorw(fmt.Sprintf("BalanceMonitor: error getting balance for key %s: invariant violation, bal may not be nil", address.Hex()), + w.bm.eng.Errorw(fmt.Sprintf("BalanceMonitor: error getting balance for key %s: invariant violation, bal may not be nil", address.Hex()), "err", err, "address", address, ) diff --git a/core/recovery/recover.go b/core/recovery/recover.go index 8e485abc556..61315defa9a 100644 --- a/core/recovery/recover.go +++ b/core/recovery/recover.go @@ -3,38 +3,38 @@ package recovery import ( "github.com/getsentry/sentry-go" - "github.com/smartcontractkit/chainlink/v2/core/logger" + "github.com/smartcontractkit/chainlink-common/pkg/logger" + + corelogger "github.com/smartcontractkit/chainlink/v2/core/logger" ) func ReportPanics(fn func()) { - defer func() { - if err := recover(); err != nil { - sentry.CurrentHub().Recover(err) - sentry.Flush(logger.SentryFlushDeadline) + HandleFn(fn, func(err any) { + sentry.CurrentHub().Recover(err) + sentry.Flush(corelogger.SentryFlushDeadline) - panic(err) - } - }() - fn() + panic(err) + }) } func WrapRecover(lggr logger.Logger, fn func()) { - defer func() { - if err := recover(); err != nil { - lggr.Recover(err) + WrapRecoverHandle(lggr, fn, nil) +} + +func WrapRecoverHandle(lggr logger.Logger, fn func(), onPanic func(recovered any)) { + HandleFn(fn, func(recovered any) { + logger.Sugared(lggr).Criticalw("Recovered goroutine panic", "panic", recovered) + + if onPanic != nil { + onPanic(recovered) } - }() - fn() + }) } -func WrapRecoverHandle(lggr logger.Logger, fn func(), onPanic func(interface{})) { +func HandleFn(fn func(), onPanic func(recovered any)) { defer func() { - if err := recover(); err != nil { - lggr.Recover(err) - - if onPanic != nil { - onPanic(err) - } + if recovered := recover(); recovered != nil { + onPanic(recovered) } }() fn() diff --git a/core/services/chainlink/application.go b/core/services/chainlink/application.go index 138ca25ed3b..c23ec08a692 100644 --- a/core/services/chainlink/application.go +++ b/core/services/chainlink/application.go @@ -148,7 +148,6 @@ type ChainlinkApplication struct { shutdownOnce sync.Once srvcs []services.ServiceCtx HealthChecker services.Checker - Nurse *services.Nurse logger logger.SugaredLogger AuditLogger audit.AuditLogger closeLogger func() error @@ -277,14 +276,9 @@ func NewApplication(opts ApplicationOpts) (Application, error) { } ap := cfg.AutoPprof() - var nurse *services.Nurse if ap.Enabled() { globalLogger.Info("Nurse service (automatic pprof profiling) is enabled") - nurse = services.NewNurse(ap, globalLogger) - err := nurse.Start() - if err != nil { - return nil, err - } + srvcs = append(srvcs, services.NewNurse(ap, globalLogger)) } else { globalLogger.Info("Nurse service (automatic pprof profiling) is disabled") } @@ -588,7 +582,6 @@ func NewApplication(opts ApplicationOpts) (Application, error) { SessionReaper: sessionReaper, ExternalInitiatorManager: externalInitiatorManager, HealthChecker: healthChecker, - Nurse: nurse, logger: globalLogger, AuditLogger: auditLogger, closeLogger: opts.CloseLogger, @@ -708,10 +701,6 @@ func (app *ChainlinkApplication) stop() (err error) { err = multierr.Append(err, app.FeedsService.Close()) } - if app.Nurse != nil { - err = multierr.Append(err, app.Nurse.Close()) - } - if app.profiler != nil { err = multierr.Append(err, app.profiler.Stop()) } diff --git a/core/services/fluxmonitorv2/deviation_checker.go b/core/services/fluxmonitorv2/deviation_checker.go index 51e85de371e..9dc399b09f9 100644 --- a/core/services/fluxmonitorv2/deviation_checker.go +++ b/core/services/fluxmonitorv2/deviation_checker.go @@ -3,7 +3,7 @@ package fluxmonitorv2 import ( "github.com/shopspring/decimal" - "github.com/smartcontractkit/chainlink/v2/core/logger" + "github.com/smartcontractkit/chainlink-common/pkg/logger" ) // DeviationThresholds carries parameters used by the threshold-trigger logic @@ -26,7 +26,7 @@ func NewDeviationChecker(rel, abs float64, lggr logger.Logger) *DeviationChecker Rel: rel, Abs: abs, }, - lggr: lggr.Named("DeviationChecker").With("threshold", rel, "absoluteThreshold", abs), + lggr: logger.Sugared(lggr).Named("DeviationChecker").With("threshold", rel, "absoluteThreshold", abs), } } diff --git a/core/services/fluxmonitorv2/flux_monitor.go b/core/services/fluxmonitorv2/flux_monitor.go index 9175feb1a68..b8154ab6797 100644 --- a/core/services/fluxmonitorv2/flux_monitor.go +++ b/core/services/fluxmonitorv2/flux_monitor.go @@ -13,6 +13,7 @@ import ( "github.com/pkg/errors" "github.com/shopspring/decimal" + "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/services" "github.com/smartcontractkit/chainlink-common/pkg/sqlutil" @@ -22,7 +23,6 @@ import ( evmutils "github.com/smartcontractkit/chainlink/v2/core/chains/evm/utils" "github.com/smartcontractkit/chainlink/v2/core/gethwrappers/generated/flags_wrapper" "github.com/smartcontractkit/chainlink/v2/core/gethwrappers/generated/flux_aggregator_wrapper" - "github.com/smartcontractkit/chainlink/v2/core/logger" "github.com/smartcontractkit/chainlink/v2/core/recovery" "github.com/smartcontractkit/chainlink/v2/core/services/fluxmonitorv2/promfm" "github.com/smartcontractkit/chainlink/v2/core/services/job" @@ -56,7 +56,10 @@ const DefaultHibernationPollPeriod = 24 * time.Hour // FluxMonitor polls external price adapters via HTTP to check for price swings. type FluxMonitor struct { - services.StateMachine + services.Service + eng *services.Engine + logger logger.SugaredLogger + contractAddress common.Address oracleAddress common.Address jobSpec job.Job @@ -77,13 +80,8 @@ type FluxMonitor struct { logBroadcaster log.Broadcaster chainID *big.Int - logger logger.SugaredLogger - backlog *utils.BoundedPriorityQueue[log.Broadcast] chProcessLogs chan struct{} - - chStop services.StopChan - waitOnStop chan struct{} } // NewFluxMonitor returns a new instance of PollingDeviationChecker. @@ -105,7 +103,7 @@ func NewFluxMonitor( flags Flags, fluxAggregator flux_aggregator_wrapper.FluxAggregatorInterface, logBroadcaster log.Broadcaster, - fmLogger logger.Logger, + lggr logger.Logger, chainID *big.Int, ) (*FluxMonitor, error) { fm := &FluxMonitor{ @@ -126,7 +124,6 @@ func NewFluxMonitor( flags: flags, logBroadcaster: logBroadcaster, fluxAggregator: fluxAggregator, - logger: logger.Sugared(fmLogger), chainID: chainID, backlog: utils.NewBoundedPriorityQueue[log.Broadcast](map[uint]int{ // We want reconnecting nodes to be able to submit to a round @@ -136,9 +133,13 @@ func NewFluxMonitor( PriorityFlagChangedLog: 2, }), chProcessLogs: make(chan struct{}, 1), - chStop: make(services.StopChan), - waitOnStop: make(chan struct{}), } + fm.Service, fm.eng = services.Config{ + Name: "FluxMonitor", + Start: fm.start, + Close: fm.close, + }.NewServiceEngine(lggr) + fm.logger = logger.Sugared(fm.eng) return fm, nil } @@ -220,7 +221,7 @@ func NewFromJobSpec( return nil, err } - fmLogger := lggr.With( + fmLogger := logger.With(lggr, "jobID", jobSpec.ID, "contract", fmSpec.ContractAddress.Hex(), ) @@ -279,14 +280,9 @@ const ( // Start implements the job.Service interface. It begins the CSP consumer in a // single goroutine to poll the price adapters and listen to NewRound events. -func (fm *FluxMonitor) Start(context.Context) error { - return fm.StartOnce("FluxMonitor", func() error { - fm.logger.Debug("Starting Flux Monitor for job") - - go fm.consume() - - return nil - }) +func (fm *FluxMonitor) start(context.Context) error { + fm.eng.Go(fm.consume) + return nil } func (fm *FluxMonitor) IsHibernating() bool { @@ -304,16 +300,12 @@ func (fm *FluxMonitor) IsHibernating() bool { return !isFlagLowered } -// Close implements the job.Service interface. It stops this instance from +// close stops this instance from // polling, cleaning up resources. -func (fm *FluxMonitor) Close() error { - return fm.StopOnce("FluxMonitor", func() error { - fm.pollManager.Stop() - close(fm.chStop) - <-fm.waitOnStop +func (fm *FluxMonitor) close() error { + fm.pollManager.Stop() - return nil - }) + return nil } // JobID implements the listener.Listener interface. @@ -354,10 +346,8 @@ func (fm *FluxMonitor) HandleLog(ctx context.Context, broadcast log.Broadcast) { } } -func (fm *FluxMonitor) consume() { - defer close(fm.waitOnStop) - - if err := fm.SetOracleAddress(); err != nil { +func (fm *FluxMonitor) consume(ctx context.Context) { + if err := fm.SetOracleAddress(ctx); err != nil { fm.logger.Warnw( "unable to set oracle address, this flux monitor job may not work correctly", "err", err, @@ -398,46 +388,46 @@ func (fm *FluxMonitor) consume() { for { select { - case <-fm.chStop: + case <-ctx.Done(): return case <-fm.chProcessLogs: - recovery.WrapRecover(fm.logger, fm.processLogs) + recovery.WrapRecover(fm.logger, func() { fm.processLogs(ctx) }) case at := <-fm.pollManager.PollTickerTicks(): tickLogger.Debugf("Poll ticker fired on %v", formatTime(at)) recovery.WrapRecover(fm.logger, func() { - fm.pollIfEligible(PollRequestTypePoll, fm.deviationChecker, nil) + fm.pollIfEligible(ctx, PollRequestTypePoll, fm.deviationChecker, nil) }) case at := <-fm.pollManager.IdleTimerTicks(): tickLogger.Debugf("Idle timer fired on %v", formatTime(at)) recovery.WrapRecover(fm.logger, func() { - fm.pollIfEligible(PollRequestTypeIdle, NewZeroDeviationChecker(fm.logger), nil) + fm.pollIfEligible(ctx, PollRequestTypeIdle, NewZeroDeviationChecker(fm.logger), nil) }) case at := <-fm.pollManager.RoundTimerTicks(): tickLogger.Debugf("Round timer fired on %v", formatTime(at)) recovery.WrapRecover(fm.logger, func() { - fm.pollIfEligible(PollRequestTypeRound, fm.deviationChecker, nil) + fm.pollIfEligible(ctx, PollRequestTypeRound, fm.deviationChecker, nil) }) case at := <-fm.pollManager.HibernationTimerTicks(): tickLogger.Debugf("Hibernation timer fired on %v", formatTime(at)) recovery.WrapRecover(fm.logger, func() { - fm.pollIfEligible(PollRequestTypeHibernation, NewZeroDeviationChecker(fm.logger), nil) + fm.pollIfEligible(ctx, PollRequestTypeHibernation, NewZeroDeviationChecker(fm.logger), nil) }) case at := <-fm.pollManager.RetryTickerTicks(): tickLogger.Debugf("Retry ticker fired on %v", formatTime(at)) recovery.WrapRecover(fm.logger, func() { - fm.pollIfEligible(PollRequestTypeRetry, NewZeroDeviationChecker(fm.logger), nil) + fm.pollIfEligible(ctx, PollRequestTypeRetry, NewZeroDeviationChecker(fm.logger), nil) }) case at := <-fm.pollManager.DrumbeatTicks(): tickLogger.Debugf("Drumbeat ticker fired on %v", formatTime(at)) recovery.WrapRecover(fm.logger, func() { - fm.pollIfEligible(PollRequestTypeDrumbeat, NewZeroDeviationChecker(fm.logger), nil) + fm.pollIfEligible(ctx, PollRequestTypeDrumbeat, NewZeroDeviationChecker(fm.logger), nil) }) case request := <-fm.pollManager.Poll(): @@ -446,7 +436,7 @@ func (fm *FluxMonitor) consume() { break default: recovery.WrapRecover(fm.logger, func() { - fm.pollIfEligible(request.Type, fm.deviationChecker, nil) + fm.pollIfEligible(ctx, request.Type, fm.deviationChecker, nil) }) } } @@ -460,11 +450,7 @@ func formatTime(at time.Time) string { // SetOracleAddress sets the oracle address which matches the node's keys. // If none match, it uses the first available key -func (fm *FluxMonitor) SetOracleAddress() error { - // fm on deprecation path, using dangling context - ctx, cancel := fm.chStop.NewCtx() - defer cancel() - +func (fm *FluxMonitor) SetOracleAddress(ctx context.Context) error { oracleAddrs, err := fm.fluxAggregator.GetOracles(nil) if err != nil { fm.logger.Error("failed to get list of oracles from FluxAggregator contract") @@ -502,10 +488,7 @@ func (fm *FluxMonitor) SetOracleAddress() error { return errors.New("No keys found") } -func (fm *FluxMonitor) processLogs() { - ctx, cancel := fm.chStop.NewCtx() - defer cancel() - +func (fm *FluxMonitor) processLogs(ctx context.Context) { for ctx.Err() == nil && !fm.backlog.Empty() { broadcast := fm.backlog.Take() fm.processBroadcast(ctx, broadcast) @@ -529,7 +512,7 @@ func (fm *FluxMonitor) processBroadcast(ctx context.Context, broadcast log.Broad decodedLog := broadcast.DecodedLog() switch log := decodedLog.(type) { case *flux_aggregator_wrapper.FluxAggregatorNewRound: - fm.respondToNewRoundLog(*log, broadcast) + fm.respondToNewRoundLog(ctx, *log, broadcast) case *flux_aggregator_wrapper.FluxAggregatorAnswerUpdated: fm.respondToAnswerUpdatedLog(*log) fm.markLogAsConsumed(ctx, broadcast, decodedLog, started) @@ -540,7 +523,7 @@ func (fm *FluxMonitor) processBroadcast(ctx context.Context, broadcast log.Broad // Only reactivate if it is hibernating if fm.pollManager.isHibernating.Load() { fm.pollManager.Awaken(fm.initialRoundState()) - fm.pollIfEligible(PollRequestTypeAwaken, NewZeroDeviationChecker(fm.logger), broadcast) + fm.pollIfEligible(ctx, PollRequestTypeAwaken, NewZeroDeviationChecker(fm.logger), broadcast) } default: fm.logger.Errorf("unknown log %v of type %T", log, log) @@ -589,10 +572,8 @@ func (fm *FluxMonitor) respondToAnswerUpdatedLog(log flux_aggregator_wrapper.Flu // The NewRound log tells us that an oracle has initiated a new round. This tells us that we // need to poll and submit an answer to the contract regardless of the deviation. -func (fm *FluxMonitor) respondToNewRoundLog(log flux_aggregator_wrapper.FluxAggregatorNewRound, lb log.Broadcast) { +func (fm *FluxMonitor) respondToNewRoundLog(ctx context.Context, log flux_aggregator_wrapper.FluxAggregatorNewRound, lb log.Broadcast) { started := time.Now() - ctx, cancel := fm.chStop.NewCtx() - defer cancel() newRoundLogger := fm.logger.With( "round", log.RoundId, @@ -819,10 +800,8 @@ func (fm *FluxMonitor) checkEligibilityAndAggregatorFunding(roundState flux_aggr return nil } -func (fm *FluxMonitor) pollIfEligible(pollReq PollRequestType, deviationChecker *DeviationChecker, broadcast log.Broadcast) { +func (fm *FluxMonitor) pollIfEligible(ctx context.Context, pollReq PollRequestType, deviationChecker *DeviationChecker, broadcast log.Broadcast) { started := time.Now() - ctx, cancel := fm.chStop.NewCtx() - defer cancel() l := fm.logger.With( "threshold", deviationChecker.Thresholds.Rel, diff --git a/core/services/fluxmonitorv2/flux_monitor_test.go b/core/services/fluxmonitorv2/flux_monitor_test.go index b3a5bcee6b9..1d1ed676e48 100644 --- a/core/services/fluxmonitorv2/flux_monitor_test.go +++ b/core/services/fluxmonitorv2/flux_monitor_test.go @@ -10,6 +10,7 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" "github.com/google/uuid" + "github.com/jmoiron/sqlx" "github.com/onsi/gomega" "github.com/pkg/errors" "github.com/shopspring/decimal" @@ -18,11 +19,10 @@ import ( "github.com/stretchr/testify/require" "gopkg.in/guregu/null.v4" - "github.com/jmoiron/sqlx" - "github.com/smartcontractkit/chainlink-common/pkg/assets" "github.com/smartcontractkit/chainlink-common/pkg/services/servicetest" "github.com/smartcontractkit/chainlink-common/pkg/sqlutil" + "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" txmgrcommon "github.com/smartcontractkit/chainlink/v2/common/txmgr" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/log" logmocks "github.com/smartcontractkit/chainlink/v2/core/chains/evm/log/mocks" @@ -491,7 +491,7 @@ func TestFluxMonitor_PollIfEligible(t *testing.T) { oracles := []common.Address{nodeAddr, testutils.NewAddress()} tm.fluxAggregator.On("GetOracles", nilOpts).Return(oracles, nil) - require.NoError(t, fm.SetOracleAddress()) + require.NoError(t, fm.SetOracleAddress(tests.Context(t))) fm.ExportedPollIfEligible(thresholds.rel, thresholds.abs) }) } @@ -526,7 +526,7 @@ func TestFluxMonitor_PollIfEligible_Creates_JobErr(t *testing.T) { Once() tm.fluxAggregator.On("GetOracles", nilOpts).Return(oracles, nil) - require.NoError(t, fm.SetOracleAddress()) + require.NoError(t, fm.SetOracleAddress(tests.Context(t))) fm.ExportedPollIfEligible(1, 1) } @@ -1171,7 +1171,7 @@ func TestFluxMonitor_RoundTimeoutCausesPoll_timesOutAtZero(t *testing.T) { tm.fluxAggregator.On("Address").Return(common.Address{}) tm.fluxAggregator.On("GetOracles", nilOpts).Return(oracles, nil) - require.NoError(t, fm.SetOracleAddress()) + require.NoError(t, fm.SetOracleAddress(tests.Context(t))) fm.ExportedRoundState(t) servicetest.Run(t, fm) @@ -1506,7 +1506,7 @@ func TestFluxMonitor_DoesNotDoubleSubmit(t *testing.T) { Return(nil) tm.fluxAggregator.On("GetOracles", nilOpts).Return(oracles, nil) - require.NoError(t, fm.SetOracleAddress()) + require.NoError(t, fm.SetOracleAddress(tests.Context(t))) tm.fluxAggregator.On("LatestRoundData", nilOpts).Return(flux_aggregator_wrapper.LatestRoundData{ Answer: big.NewInt(10), @@ -1635,7 +1635,7 @@ func TestFluxMonitor_DoesNotDoubleSubmit(t *testing.T) { Once() tm.fluxAggregator.On("GetOracles", nilOpts).Return(oracles, nil) - require.NoError(t, fm.SetOracleAddress()) + require.NoError(t, fm.SetOracleAddress(tests.Context(t))) fm.ExportedPollIfEligible(0, 0) // Now fire off the NewRound log and ensure it does not respond this time @@ -1732,7 +1732,7 @@ func TestFluxMonitor_DoesNotDoubleSubmit(t *testing.T) { Once() tm.fluxAggregator.On("GetOracles", nilOpts).Return(oracles, nil) - require.NoError(t, fm.SetOracleAddress()) + require.NoError(t, fm.SetOracleAddress(tests.Context(t))) fm.ExportedPollIfEligible(0, 0) // Now fire off the NewRound log and ensure it does not respond this time diff --git a/core/services/fluxmonitorv2/helpers_test.go b/core/services/fluxmonitorv2/helpers_test.go index d321ddc35c3..80db82351c7 100644 --- a/core/services/fluxmonitorv2/helpers_test.go +++ b/core/services/fluxmonitorv2/helpers_test.go @@ -19,11 +19,15 @@ func (fm *FluxMonitor) Format(f fmt.State, verb rune) { } func (fm *FluxMonitor) ExportedPollIfEligible(threshold, absoluteThreshold float64) { - fm.pollIfEligible(PollRequestTypePoll, NewDeviationChecker(threshold, absoluteThreshold, fm.logger), nil) + ctx, cancel := fm.eng.NewCtx() + defer cancel() + fm.pollIfEligible(ctx, PollRequestTypePoll, NewDeviationChecker(threshold, absoluteThreshold, fm.logger), nil) } func (fm *FluxMonitor) ExportedProcessLogs() { - fm.processLogs() + ctx, cancel := fm.eng.NewCtx() + defer cancel() + fm.processLogs(ctx) } func (fm *FluxMonitor) ExportedBacklog() *utils.BoundedPriorityQueue[log.Broadcast] { @@ -36,7 +40,9 @@ func (fm *FluxMonitor) ExportedRoundState(t *testing.T) { } func (fm *FluxMonitor) ExportedRespondToNewRoundLog(log *flux_aggregator_wrapper.FluxAggregatorNewRound, broadcast log.Broadcast) { - fm.respondToNewRoundLog(*log, broadcast) + ctx, cancel := fm.eng.NewCtx() + defer cancel() + fm.respondToNewRoundLog(ctx, *log, broadcast) } func (fm *FluxMonitor) ExportedRespondToFlagsRaisedLog() { diff --git a/core/services/fluxmonitorv2/poll_manager.go b/core/services/fluxmonitorv2/poll_manager.go index 78b99aec4d5..aca6c75a311 100644 --- a/core/services/fluxmonitorv2/poll_manager.go +++ b/core/services/fluxmonitorv2/poll_manager.go @@ -5,8 +5,8 @@ import ( "sync/atomic" "time" + "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink/v2/core/gethwrappers/generated/flux_aggregator_wrapper" - "github.com/smartcontractkit/chainlink/v2/core/logger" "github.com/smartcontractkit/chainlink/v2/core/utils" ) @@ -64,7 +64,7 @@ type PollManager struct { } // NewPollManager initializes a new PollManager -func NewPollManager(cfg PollManagerConfig, logger logger.Logger) (*PollManager, error) { +func NewPollManager(cfg PollManagerConfig, lggr logger.Logger) (*PollManager, error) { minBackoffDuration := cfg.MinRetryBackoffDuration if cfg.IdleTimerPeriod < minBackoffDuration { minBackoffDuration = cfg.IdleTimerPeriod @@ -82,7 +82,7 @@ func NewPollManager(cfg PollManagerConfig, logger logger.Logger) (*PollManager, p := &PollManager{ cfg: cfg, - logger: logger.Named("PollManager"), + logger: logger.Named(lggr, "PollManager"), hibernationTimer: utils.NewResettableTimer(), pollTicker: utils.NewPausableTicker(cfg.PollTickerInterval), @@ -277,7 +277,7 @@ func (pm *PollManager) startIdleTimer(roundStartedAtUTC uint64) { deadline := startedAt.Add(pm.cfg.IdleTimerPeriod) deadlineDuration := time.Until(deadline) - log := pm.logger.With( + log := logger.With(pm.logger, "pollFrequency", pm.cfg.PollTickerInterval, "idleDuration", pm.cfg.IdleTimerPeriod, "startedAt", roundStartedAtUTC, @@ -300,7 +300,7 @@ func (pm *PollManager) startIdleTimer(roundStartedAtUTC uint64) { // startRoundTimer starts the round timer func (pm *PollManager) startRoundTimer(roundTimesOutAt uint64) { - log := pm.logger.With( + log := logger.With(pm.logger, "pollFrequency", pm.cfg.PollTickerInterval, "idleDuration", pm.cfg.IdleTimerPeriod, "timesOutAt", roundTimesOutAt, diff --git a/core/services/nurse.go b/core/services/nurse.go index a9069b5181d..7f3cad13e71 100644 --- a/core/services/nurse.go +++ b/core/services/nurse.go @@ -3,6 +3,7 @@ package services import ( "bytes" "compress/gzip" + "context" "fmt" "io/fs" "os" @@ -19,22 +20,21 @@ import ( commonconfig "github.com/smartcontractkit/chainlink-common/pkg/config" "github.com/smartcontractkit/chainlink-common/pkg/services" + "github.com/smartcontractkit/chainlink-common/pkg/timeutil" "github.com/smartcontractkit/chainlink/v2/core/logger" "github.com/smartcontractkit/chainlink/v2/core/utils" ) type Nurse struct { - services.StateMachine + services.Service + eng *services.Engine cfg Config - log logger.Logger checks map[string]CheckFunc checksMu sync.RWMutex chGather chan gatherRequest - chStop chan struct{} - wgDone sync.WaitGroup } type Config interface { @@ -66,85 +66,63 @@ const ( ) func NewNurse(cfg Config, log logger.Logger) *Nurse { - return &Nurse{ + n := &Nurse{ cfg: cfg, - log: log.Named("Nurse"), checks: make(map[string]CheckFunc), chGather: make(chan gatherRequest, 1), - chStop: make(chan struct{}), } + n.Service, n.eng = services.Config{ + Name: "Nurse", + Start: n.start, + }.NewServiceEngine(log) + + return n } -func (n *Nurse) Start() error { - return n.StartOnce("Nurse", func() error { - // This must be set *once*, and it must occur as early as possible - if n.cfg.MemProfileRate() != runtime.MemProfileRate { - runtime.MemProfileRate = n.cfg.BlockProfileRate() - } +func (n *Nurse) start(_ context.Context) error { + // This must be set *once*, and it must occur as early as possible + if n.cfg.MemProfileRate() != runtime.MemProfileRate { + runtime.MemProfileRate = n.cfg.BlockProfileRate() + } - n.log.Debugf("Starting nurse with config %+v", n.cfg) - runtime.SetCPUProfileRate(n.cfg.CPUProfileRate()) - runtime.SetBlockProfileRate(n.cfg.BlockProfileRate()) - runtime.SetMutexProfileFraction(n.cfg.MutexProfileFraction()) + n.eng.Debugf("Starting nurse with config %+v", n.cfg) + runtime.SetCPUProfileRate(n.cfg.CPUProfileRate()) + runtime.SetBlockProfileRate(n.cfg.BlockProfileRate()) + runtime.SetMutexProfileFraction(n.cfg.MutexProfileFraction()) - err := utils.EnsureDirAndMaxPerms(n.cfg.ProfileRoot(), 0744) - if err != nil { - return err - } + err := utils.EnsureDirAndMaxPerms(n.cfg.ProfileRoot(), 0744) + if err != nil { + return err + } - n.AddCheck("mem", n.checkMem) - n.AddCheck("goroutines", n.checkGoroutines) - - n.wgDone.Add(1) - // Checker - go func() { - defer n.wgDone.Done() - for { - select { - case <-n.chStop: - return - case <-time.After(n.cfg.PollInterval().Duration()): - } - - func() { - n.checksMu.RLock() - defer n.checksMu.RUnlock() - for reason, checkFunc := range n.checks { - if unwell, meta := checkFunc(); unwell { - n.GatherVitals(reason, meta) - break - } - } - }() - } - }() - - n.wgDone.Add(1) - // Responder - go func() { - defer n.wgDone.Done() - for { - select { - case <-n.chStop: - return - case req := <-n.chGather: - n.gatherVitals(req.reason, req.meta) - } - } - }() + n.AddCheck("mem", n.checkMem) + n.AddCheck("goroutines", n.checkGoroutines) - return nil + // Checker + n.eng.GoTick(timeutil.NewTicker(n.cfg.PollInterval().Duration), func(ctx context.Context) { + n.checksMu.RLock() + defer n.checksMu.RUnlock() + for reason, checkFunc := range n.checks { + if unwell, meta := checkFunc(); unwell { + n.GatherVitals(ctx, reason, meta) + break + } + } }) -} -func (n *Nurse) Close() error { - return n.StopOnce("Nurse", func() error { - n.log.Debug("Nurse closing...") - defer n.log.Debug("Nurse closed") - close(n.chStop) - n.wgDone.Wait() - return nil + // Responder + n.eng.Go(func(ctx context.Context) { + for { + select { + case <-ctx.Done(): + return + case req := <-n.chGather: + n.gatherVitals(req.reason, req.meta) + } + } }) + + return nil } func (n *Nurse) AddCheck(reason string, checkFunc CheckFunc) { @@ -153,9 +131,9 @@ func (n *Nurse) AddCheck(reason string, checkFunc CheckFunc) { n.checks[reason] = checkFunc } -func (n *Nurse) GatherVitals(reason string, meta Meta) { +func (n *Nurse) GatherVitals(ctx context.Context, reason string, meta Meta) { select { - case <-n.chStop: + case <-ctx.Done(): case n.chGather <- gatherRequest{reason, meta}: default: } @@ -189,14 +167,14 @@ func (n *Nurse) checkGoroutines() (bool, Meta) { func (n *Nurse) gatherVitals(reason string, meta Meta) { loggerFields := (logger.Fields{"reason": reason}).Merge(logger.Fields(meta)) - n.log.Debugw("Nurse is gathering vitals", loggerFields.Slice()...) + n.eng.Debugw("Nurse is gathering vitals", loggerFields.Slice()...) size, err := n.totalProfileBytes() if err != nil { - n.log.Errorw("could not fetch total profile bytes", loggerFields.With("err", err).Slice()...) + n.eng.Errorw("could not fetch total profile bytes", loggerFields.With("err", err).Slice()...) return } else if size >= uint64(n.cfg.MaxProfileSize()) { - n.log.Warnw("cannot write pprof profile, total profile size exceeds configured PPROF_MAX_PROFILE_SIZE", + n.eng.Warnw("cannot write pprof profile, total profile size exceeds configured PPROF_MAX_PROFILE_SIZE", loggerFields.With("total", size, "max", n.cfg.MaxProfileSize()).Slice()..., ) return @@ -206,7 +184,7 @@ func (n *Nurse) gatherVitals(reason string, meta Meta) { err = n.appendLog(now, reason, meta) if err != nil { - n.log.Warnw("cannot write pprof profile", loggerFields.With("err", err).Slice()...) + n.eng.Warnw("cannot write pprof profile", loggerFields.With("err", err).Slice()...) return } var wg sync.WaitGroup @@ -227,7 +205,7 @@ func (n *Nurse) gatherVitals(reason string, meta Meta) { wg.Add(1) go n.gather("heap", now, &wg) } else { - n.log.Info("skipping heap collection because runtime.MemProfileRate = 0") + n.eng.Info("skipping heap collection because runtime.MemProfileRate = 0") } wg.Add(1) @@ -236,15 +214,13 @@ func (n *Nurse) gatherVitals(reason string, meta Meta) { go n.gather("threadcreate", now, &wg) ch := make(chan struct{}) - n.wgDone.Add(1) - go func() { - defer n.wgDone.Done() + n.eng.Go(func(ctx context.Context) { defer close(ch) wg.Wait() - }() + }) select { - case <-n.chStop: + case <-n.eng.StopChan: case <-ch: } } @@ -252,7 +228,7 @@ func (n *Nurse) gatherVitals(reason string, meta Meta) { func (n *Nurse) appendLog(now time.Time, reason string, meta Meta) error { filename := filepath.Join(n.cfg.ProfileRoot(), "nurse.log") - n.log.Debugf("creating nurse log %s", filename) + n.eng.Debugf("creating nurse log %s", filename) file, err := os.Create(filename) if err != nil { @@ -288,34 +264,34 @@ func (n *Nurse) appendLog(now time.Time, reason string, meta Meta) error { func (n *Nurse) gatherCPU(now time.Time, wg *sync.WaitGroup) { defer wg.Done() - n.log.Debugf("gather cpu %d ...", now.UnixMicro()) - defer n.log.Debugf("gather cpu %d done", now.UnixMicro()) + n.eng.Debugf("gather cpu %d ...", now.UnixMicro()) + defer n.eng.Debugf("gather cpu %d done", now.UnixMicro()) wc, err := n.createFile(now, cpuProfName, false) if err != nil { - n.log.Errorw("could not write cpu profile", "err", err) + n.eng.Errorw("could not write cpu profile", "err", err) return } defer wc.Close() err = pprof.StartCPUProfile(wc) if err != nil { - n.log.Errorw("could not start cpu profile", "err", err) + n.eng.Errorw("could not start cpu profile", "err", err) return } select { - case <-n.chStop: - n.log.Debug("gather cpu received stop") + case <-n.eng.StopChan: + n.eng.Debug("gather cpu received stop") case <-time.After(n.cfg.GatherDuration().Duration()): - n.log.Debugf("gather cpu duration elapsed %s. stoping profiling.", n.cfg.GatherDuration().Duration().String()) + n.eng.Debugf("gather cpu duration elapsed %s. stoping profiling.", n.cfg.GatherDuration().Duration().String()) } pprof.StopCPUProfile() err = wc.Close() if err != nil { - n.log.Errorw("could not close cpu profile", "err", err) + n.eng.Errorw("could not close cpu profile", "err", err) return } } @@ -323,23 +299,23 @@ func (n *Nurse) gatherCPU(now time.Time, wg *sync.WaitGroup) { func (n *Nurse) gatherTrace(now time.Time, wg *sync.WaitGroup) { defer wg.Done() - n.log.Debugf("gather trace %d ...", now.UnixMicro()) - defer n.log.Debugf("gather trace %d done", now.UnixMicro()) + n.eng.Debugf("gather trace %d ...", now.UnixMicro()) + defer n.eng.Debugf("gather trace %d done", now.UnixMicro()) wc, err := n.createFile(now, traceProfName, true) if err != nil { - n.log.Errorw("could not write trace profile", "err", err) + n.eng.Errorw("could not write trace profile", "err", err) return } defer wc.Close() err = trace.Start(wc) if err != nil { - n.log.Errorw("could not start trace profile", "err", err) + n.eng.Errorw("could not start trace profile", "err", err) return } select { - case <-n.chStop: + case <-n.eng.StopChan: case <-time.After(n.cfg.GatherTraceDuration().Duration()): } @@ -347,7 +323,7 @@ func (n *Nurse) gatherTrace(now time.Time, wg *sync.WaitGroup) { err = wc.Close() if err != nil { - n.log.Errorw("could not close trace profile", "err", err) + n.eng.Errorw("could not close trace profile", "err", err) return } } @@ -355,18 +331,18 @@ func (n *Nurse) gatherTrace(now time.Time, wg *sync.WaitGroup) { func (n *Nurse) gather(typ string, now time.Time, wg *sync.WaitGroup) { defer wg.Done() - n.log.Debugf("gather %s %d ...", typ, now.UnixMicro()) - n.log.Debugf("gather %s %d done", typ, now.UnixMicro()) + n.eng.Debugf("gather %s %d ...", typ, now.UnixMicro()) + n.eng.Debugf("gather %s %d done", typ, now.UnixMicro()) p := pprof.Lookup(typ) if p == nil { - n.log.Errorf("Invariant violation: pprof type '%v' does not exist", typ) + n.eng.Errorf("Invariant violation: pprof type '%v' does not exist", typ) return } p0, err := collectProfile(p) if err != nil { - n.log.Errorw(fmt.Sprintf("could not collect %v profile", typ), "err", err) + n.eng.Errorw(fmt.Sprintf("could not collect %v profile", typ), "err", err) return } @@ -374,14 +350,14 @@ func (n *Nurse) gather(typ string, now time.Time, wg *sync.WaitGroup) { defer t.Stop() select { - case <-n.chStop: + case <-n.eng.StopChan: return case <-t.C: } p1, err := collectProfile(p) if err != nil { - n.log.Errorw(fmt.Sprintf("could not collect %v profile", typ), "err", err) + n.eng.Errorw(fmt.Sprintf("could not collect %v profile", typ), "err", err) return } ts := p1.TimeNanos @@ -391,7 +367,7 @@ func (n *Nurse) gather(typ string, now time.Time, wg *sync.WaitGroup) { p1, err = profile.Merge([]*profile.Profile{p0, p1}) if err != nil { - n.log.Errorw(fmt.Sprintf("could not compute delta for %v profile", typ), "err", err) + n.eng.Errorw(fmt.Sprintf("could not compute delta for %v profile", typ), "err", err) return } @@ -400,19 +376,19 @@ func (n *Nurse) gather(typ string, now time.Time, wg *sync.WaitGroup) { wc, err := n.createFile(now, typ, false) if err != nil { - n.log.Errorw(fmt.Sprintf("could not write %v profile", typ), "err", err) + n.eng.Errorw(fmt.Sprintf("could not write %v profile", typ), "err", err) return } defer wc.Close() err = p1.Write(wc) if err != nil { - n.log.Errorw(fmt.Sprintf("could not write %v profile", typ), "err", err) + n.eng.Errorw(fmt.Sprintf("could not write %v profile", typ), "err", err) return } err = wc.Close() if err != nil { - n.log.Errorw(fmt.Sprintf("could not close file for %v profile", typ), "err", err) + n.eng.Errorw(fmt.Sprintf("could not close file for %v profile", typ), "err", err) return } } @@ -437,7 +413,7 @@ func (n *Nurse) createFile(now time.Time, typ string, shouldGzip bool) (*utils.D filename += ".gz" } fullpath := filepath.Join(n.cfg.ProfileRoot(), filename) - n.log.Debugf("creating file %s", fullpath) + n.eng.Debugf("creating file %s", fullpath) file, err := os.Create(fullpath) if err != nil { diff --git a/core/services/nurse_test.go b/core/services/nurse_test.go index 4597eeb456b..ed6f6872dc9 100644 --- a/core/services/nurse_test.go +++ b/core/services/nurse_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/require" commonconfig "github.com/smartcontractkit/chainlink-common/pkg/config" + "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" "github.com/smartcontractkit/chainlink/v2/core/internal/testutils" "github.com/smartcontractkit/chainlink/v2/core/logger" "github.com/smartcontractkit/chainlink/v2/core/utils" @@ -102,7 +103,7 @@ func TestNurse(t *testing.T) { nrse := NewNurse(newMockConfig(t), l) nrse.AddCheck("test", func() (bool, Meta) { return true, Meta{} }) - require.NoError(t, nrse.Start()) + require.NoError(t, nrse.Start(tests.Context(t))) defer func() { require.NoError(t, nrse.Close()) }() require.NoError(t, nrse.appendLog(time.Now(), "test", Meta{})) diff --git a/core/services/relay/evm/functions/logpoller_wrapper.go b/core/services/relay/evm/functions/logpoller_wrapper.go index 559b1ec33f5..b0d04b11871 100644 --- a/core/services/relay/evm/functions/logpoller_wrapper.go +++ b/core/services/relay/evm/functions/logpoller_wrapper.go @@ -22,7 +22,8 @@ import ( ) type logPollerWrapper struct { - services.StateMachine + services.Service + eng *services.Engine routerContract *functions_router.FunctionsRouter pluginConfig config.PluginConfig @@ -38,9 +39,6 @@ type logPollerWrapper struct { detectedRequests detectedEvents detectedResponses detectedEvents mu sync.Mutex - closeWait sync.WaitGroup - stopCh services.StopChan - lggr logger.Logger } type detectedEvent struct { @@ -94,7 +92,7 @@ func NewLogPollerWrapper(routerContractAddress common.Address, pluginConfig conf return nil, errors.Errorf("invalid config: number of required confirmation blocks >= pastBlocksToPoll") } - return &logPollerWrapper{ + w := &logPollerWrapper{ routerContract: routerContract, pluginConfig: pluginConfig, requestBlockOffset: requestBlockOffset, @@ -106,40 +104,25 @@ func NewLogPollerWrapper(routerContractAddress common.Address, pluginConfig conf logPoller: logPoller, client: client, subscribers: make(map[string]evmRelayTypes.RouteUpdateSubscriber), - stopCh: make(services.StopChan), - lggr: lggr.Named("LogPollerWrapper"), - }, nil -} - -func (l *logPollerWrapper) Start(context.Context) error { - return l.StartOnce("LogPollerWrapper", func() error { - l.lggr.Infow("starting LogPollerWrapper", "routerContract", l.routerContract.Address().Hex(), "contractVersion", l.pluginConfig.ContractVersion) - l.mu.Lock() - defer l.mu.Unlock() - if l.pluginConfig.ContractVersion != 1 { - return errors.New("only contract version 1 is supported") - } - l.closeWait.Add(1) - go l.checkForRouteUpdates() - return nil - }) -} - -func (l *logPollerWrapper) Close() error { - return l.StopOnce("LogPollerWrapper", func() (err error) { - l.lggr.Info("closing LogPollerWrapper") - close(l.stopCh) - l.closeWait.Wait() - return nil - }) + } + w.Service, w.eng = services.Config{ + Name: "LoggPollerWrapper", + Start: w.start, + }.NewServiceEngine(lggr) + return w, nil } -func (l *logPollerWrapper) HealthReport() map[string]error { - return map[string]error{l.Name(): l.Ready()} +func (l *logPollerWrapper) start(context.Context) error { + l.eng.Infow("starting LogPollerWrapper", "routerContract", l.routerContract.Address().Hex(), "contractVersion", l.pluginConfig.ContractVersion) + l.mu.Lock() + defer l.mu.Unlock() + if l.pluginConfig.ContractVersion != 1 { + return errors.New("only contract version 1 is supported") + } + l.eng.Go(l.checkForRouteUpdates) + return nil } -func (l *logPollerWrapper) Name() string { return l.lggr.Name() } - // methods of LogPollerWrapper func (l *logPollerWrapper) LatestEvents(ctx context.Context) ([]evmRelayTypes.OracleRequest, []evmRelayTypes.OracleResponse, error) { l.mu.Lock() @@ -166,7 +149,7 @@ func (l *logPollerWrapper) LatestEvents(ctx context.Context) ([]evmRelayTypes.Or resultsReq := []evmRelayTypes.OracleRequest{} resultsResp := []evmRelayTypes.OracleResponse{} if len(coordinators) == 0 { - l.lggr.Debug("LatestEvents: no non-zero coordinators to check") + l.eng.Debug("LatestEvents: no non-zero coordinators to check") return resultsReq, resultsResp, errors.New("no non-zero coordinators to check") } @@ -174,32 +157,32 @@ func (l *logPollerWrapper) LatestEvents(ctx context.Context) ([]evmRelayTypes.Or requestEndBlock := latestBlockNum - l.requestBlockOffset requestLogs, err := l.logPoller.Logs(ctx, startBlockNum, requestEndBlock, functions_coordinator.FunctionsCoordinatorOracleRequest{}.Topic(), coordinator) if err != nil { - l.lggr.Errorw("LatestEvents: fetching request logs from LogPoller failed", "startBlock", startBlockNum, "endBlock", requestEndBlock) + l.eng.Errorw("LatestEvents: fetching request logs from LogPoller failed", "startBlock", startBlockNum, "endBlock", requestEndBlock) return nil, nil, err } - l.lggr.Debugw("LatestEvents: fetched request logs", "nRequestLogs", len(requestLogs), "latestBlock", latest, "startBlock", startBlockNum, "endBlock", requestEndBlock) + l.eng.Debugw("LatestEvents: fetched request logs", "nRequestLogs", len(requestLogs), "latestBlock", latest, "startBlock", startBlockNum, "endBlock", requestEndBlock) requestLogs = l.filterPreviouslyDetectedEvents(requestLogs, &l.detectedRequests, "requests") responseEndBlock := latestBlockNum - l.responseBlockOffset responseLogs, err := l.logPoller.Logs(ctx, startBlockNum, responseEndBlock, functions_coordinator.FunctionsCoordinatorOracleResponse{}.Topic(), coordinator) if err != nil { - l.lggr.Errorw("LatestEvents: fetching response logs from LogPoller failed", "startBlock", startBlockNum, "endBlock", responseEndBlock) + l.eng.Errorw("LatestEvents: fetching response logs from LogPoller failed", "startBlock", startBlockNum, "endBlock", responseEndBlock) return nil, nil, err } - l.lggr.Debugw("LatestEvents: fetched request logs", "nResponseLogs", len(responseLogs), "latestBlock", latest, "startBlock", startBlockNum, "endBlock", responseEndBlock) + l.eng.Debugw("LatestEvents: fetched request logs", "nResponseLogs", len(responseLogs), "latestBlock", latest, "startBlock", startBlockNum, "endBlock", responseEndBlock) responseLogs = l.filterPreviouslyDetectedEvents(responseLogs, &l.detectedResponses, "responses") parsingContract, err := functions_coordinator.NewFunctionsCoordinator(coordinator, l.client) if err != nil { - l.lggr.Error("LatestEvents: creating a contract instance for parsing failed") + l.eng.Error("LatestEvents: creating a contract instance for parsing failed") return nil, nil, err } - l.lggr.Debugw("LatestEvents: parsing logs", "nRequestLogs", len(requestLogs), "nResponseLogs", len(responseLogs), "coordinatorAddress", coordinator.Hex()) + l.eng.Debugw("LatestEvents: parsing logs", "nRequestLogs", len(requestLogs), "nResponseLogs", len(responseLogs), "coordinatorAddress", coordinator.Hex()) for _, log := range requestLogs { gethLog := log.ToGethLog() oracleRequest, err := parsingContract.ParseOracleRequest(gethLog) if err != nil { - l.lggr.Errorw("LatestEvents: failed to parse a request log, skipping", "err", err) + l.eng.Errorw("LatestEvents: failed to parse a request log, skipping", "err", err) continue } @@ -212,7 +195,7 @@ func (l *logPollerWrapper) LatestEvents(ctx context.Context) ([]evmRelayTypes.Or bytes32Type, errType7 := abi.NewType("bytes32", "bytes32", nil) if errType1 != nil || errType2 != nil || errType3 != nil || errType4 != nil || errType5 != nil || errType6 != nil || errType7 != nil { - l.lggr.Errorw("LatestEvents: failed to initialize types", "errType1", errType1, + l.eng.Errorw("LatestEvents: failed to initialize types", "errType1", errType1, "errType2", errType2, "errType3", errType3, "errType4", errType4, "errType5", errType5, "errType6", errType6, "errType7", errType7, ) continue @@ -244,7 +227,7 @@ func (l *logPollerWrapper) LatestEvents(ctx context.Context) ([]evmRelayTypes.Or oracleRequest.Commitment.TimeoutTimestamp, ) if err != nil { - l.lggr.Errorw("LatestEvents: failed to pack commitment bytes, skipping", "err", err) + l.eng.Errorw("LatestEvents: failed to pack commitment bytes, skipping", "err", err) } resultsReq = append(resultsReq, evmRelayTypes.OracleRequest{ @@ -266,7 +249,7 @@ func (l *logPollerWrapper) LatestEvents(ctx context.Context) ([]evmRelayTypes.Or gethLog := log.ToGethLog() oracleResponse, err := parsingContract.ParseOracleResponse(gethLog) if err != nil { - l.lggr.Errorw("LatestEvents: failed to parse a response log, skipping") + l.eng.Errorw("LatestEvents: failed to parse a response log, skipping") continue } resultsResp = append(resultsResp, evmRelayTypes.OracleResponse{ @@ -275,13 +258,13 @@ func (l *logPollerWrapper) LatestEvents(ctx context.Context) ([]evmRelayTypes.Or } } - l.lggr.Debugw("LatestEvents: done", "nRequestLogs", len(resultsReq), "nResponseLogs", len(resultsResp), "startBlock", startBlockNum, "endBlock", latestBlockNum) + l.eng.Debugw("LatestEvents: done", "nRequestLogs", len(resultsReq), "nResponseLogs", len(resultsResp), "startBlock", startBlockNum, "endBlock", latestBlockNum) return resultsReq, resultsResp, nil } func (l *logPollerWrapper) filterPreviouslyDetectedEvents(logs []logpoller.Log, detectedEvents *detectedEvents, filterType string) []logpoller.Log { if len(logs) > maxLogsToProcess { - l.lggr.Errorw("filterPreviouslyDetectedEvents: too many logs to process, only processing latest maxLogsToProcess logs", "filterType", filterType, "nLogs", len(logs), "maxLogsToProcess", maxLogsToProcess) + l.eng.Errorw("filterPreviouslyDetectedEvents: too many logs to process, only processing latest maxLogsToProcess logs", "filterType", filterType, "nLogs", len(logs), "maxLogsToProcess", maxLogsToProcess) logs = logs[len(logs)-maxLogsToProcess:] } l.mu.Lock() @@ -290,7 +273,7 @@ func (l *logPollerWrapper) filterPreviouslyDetectedEvents(logs []logpoller.Log, for _, log := range logs { var requestId [32]byte if len(log.Topics) < 2 || len(log.Topics[1]) != 32 { - l.lggr.Errorw("filterPreviouslyDetectedEvents: invalid log, skipping", "filterType", filterType, "log", log) + l.eng.Errorw("filterPreviouslyDetectedEvents: invalid log, skipping", "filterType", filterType, "log", log) continue } copy(requestId[:], log.Topics[1]) // requestId is the second topic (1st topic is the event signature) @@ -310,7 +293,7 @@ func (l *logPollerWrapper) filterPreviouslyDetectedEvents(logs []logpoller.Log, expiredRequests++ } detectedEvents.detectedEventsOrdered = detectedEvents.detectedEventsOrdered[expiredRequests:] - l.lggr.Debugw("filterPreviouslyDetectedEvents: done", "filterType", filterType, "nLogs", len(logs), "nFilteredLogs", len(filteredLogs), "nExpiredRequests", expiredRequests, "previouslyDetectedCacheSize", len(detectedEvents.detectedEventsOrdered)) + l.eng.Debugw("filterPreviouslyDetectedEvents: done", "filterType", filterType, "nLogs", len(logs), "nFilteredLogs", len(filteredLogs), "nExpiredRequests", expiredRequests, "previouslyDetectedCacheSize", len(detectedEvents.detectedEventsOrdered)) return filteredLogs } @@ -319,7 +302,7 @@ func (l *logPollerWrapper) SubscribeToUpdates(ctx context.Context, subscriberNam if l.pluginConfig.ContractVersion == 0 { // in V0, immediately set contract address to Oracle contract and never update again if err := subscriber.UpdateRoutes(ctx, l.routerContract.Address(), l.routerContract.Address()); err != nil { - l.lggr.Errorw("LogPollerWrapper: Failed to update routes", "subscriberName", subscriberName, "err", err) + l.eng.Errorw("LogPollerWrapper: Failed to update routes", "subscriberName", subscriberName, "err", err) } } else if l.pluginConfig.ContractVersion == 1 { l.mu.Lock() @@ -328,37 +311,36 @@ func (l *logPollerWrapper) SubscribeToUpdates(ctx context.Context, subscriberNam } } -func (l *logPollerWrapper) checkForRouteUpdates() { - defer l.closeWait.Done() +func (l *logPollerWrapper) checkForRouteUpdates(ctx context.Context) { freqSec := l.pluginConfig.ContractUpdateCheckFrequencySec if freqSec == 0 { - l.lggr.Errorw("LogPollerWrapper: ContractUpdateCheckFrequencySec is zero - route update checks disabled") + l.eng.Errorw("LogPollerWrapper: ContractUpdateCheckFrequencySec is zero - route update checks disabled") return } - updateOnce := func() { + updateOnce := func(ctx context.Context) { // NOTE: timeout == frequency here, could be changed to a separate config value timeout := time.Duration(l.pluginConfig.ContractUpdateCheckFrequencySec) * time.Second - ctx, cancel := l.stopCh.CtxCancel(context.WithTimeout(context.Background(), timeout)) + ctx, cancel := context.WithTimeout(ctx, timeout) defer cancel() active, proposed, err := l.getCurrentCoordinators(ctx) if err != nil { - l.lggr.Errorw("LogPollerWrapper: error calling getCurrentCoordinators", "err", err) + l.eng.Errorw("LogPollerWrapper: error calling getCurrentCoordinators", "err", err) return } l.handleRouteUpdate(ctx, active, proposed) } - updateOnce() // update once right away + updateOnce(ctx) // update once right away ticker := time.NewTicker(time.Duration(freqSec) * time.Second) defer ticker.Stop() for { select { - case <-l.stopCh: + case <-ctx.Done(): return case <-ticker.C: - updateOnce() + updateOnce(ctx) } } } @@ -394,22 +376,22 @@ func (l *logPollerWrapper) handleRouteUpdate(ctx context.Context, activeCoordina defer l.mu.Unlock() if activeCoordinator == (common.Address{}) { - l.lggr.Error("LogPollerWrapper: cannot update activeCoordinator to zero address") + l.eng.Error("LogPollerWrapper: cannot update activeCoordinator to zero address") return } if activeCoordinator == l.activeCoordinator && proposedCoordinator == l.proposedCoordinator { - l.lggr.Debug("LogPollerWrapper: no changes to routes") + l.eng.Debug("LogPollerWrapper: no changes to routes") return } errActive := l.registerFilters(ctx, activeCoordinator) errProposed := l.registerFilters(ctx, proposedCoordinator) if errActive != nil || errProposed != nil { - l.lggr.Errorw("LogPollerWrapper: Failed to register filters", "errorActive", errActive, "errorProposed", errProposed) + l.eng.Errorw("LogPollerWrapper: Failed to register filters", "errorActive", errActive, "errorProposed", errProposed) return } - l.lggr.Debugw("LogPollerWrapper: new routes", "activeCoordinator", activeCoordinator.Hex(), "proposedCoordinator", proposedCoordinator.Hex()) + l.eng.Debugw("LogPollerWrapper: new routes", "activeCoordinator", activeCoordinator.Hex(), "proposedCoordinator", proposedCoordinator.Hex()) l.activeCoordinator = activeCoordinator l.proposedCoordinator = proposedCoordinator @@ -417,7 +399,7 @@ func (l *logPollerWrapper) handleRouteUpdate(ctx context.Context, activeCoordina for _, subscriber := range l.subscribers { err := subscriber.UpdateRoutes(ctx, activeCoordinator, proposedCoordinator) if err != nil { - l.lggr.Errorw("LogPollerWrapper: Failed to update routes", "err", err) + l.eng.Errorw("LogPollerWrapper: Failed to update routes", "err", err) } } @@ -430,9 +412,9 @@ func (l *logPollerWrapper) handleRouteUpdate(ctx context.Context, activeCoordina continue } if err := l.logPoller.UnregisterFilter(ctx, filter.Name); err != nil { - l.lggr.Errorw("LogPollerWrapper: Failed to unregister filter", "filterName", filter.Name, "err", err) + l.eng.Errorw("LogPollerWrapper: Failed to unregister filter", "filterName", filter.Name, "err", err) } - l.lggr.Debugw("LogPollerWrapper: Successfully unregistered filter", "filterName", filter.Name) + l.eng.Debugw("LogPollerWrapper: Successfully unregistered filter", "filterName", filter.Name) } } diff --git a/core/services/synchronization/helpers_test.go b/core/services/synchronization/helpers_test.go index 7bb2dde7633..aea9bf77f49 100644 --- a/core/services/synchronization/helpers_test.go +++ b/core/services/synchronization/helpers_test.go @@ -12,15 +12,15 @@ import ( // NewTestTelemetryIngressClient calls NewTelemetryIngressClient and injects telemClient. func NewTestTelemetryIngressClient(t *testing.T, url *url.URL, serverPubKeyHex string, ks keystore.CSA, logging bool, telemClient telemPb.TelemClient) TelemetryService { - tc := NewTelemetryIngressClient(url, serverPubKeyHex, ks, logging, logger.TestLogger(t), 100, "test", "test") + tc := NewTelemetryIngressClient(url, serverPubKeyHex, ks, logging, logger.TestLogger(t), 100) tc.(*telemetryIngressClient).telemClient = telemClient return tc } // NewTestTelemetryIngressBatchClient calls NewTelemetryIngressBatchClient and injects telemClient. func NewTestTelemetryIngressBatchClient(t *testing.T, url *url.URL, serverPubKeyHex string, ks keystore.CSA, logging bool, telemClient telemPb.TelemClient, sendInterval time.Duration, uniconn bool) TelemetryService { - tc := NewTelemetryIngressBatchClient(url, serverPubKeyHex, ks, logging, logger.TestLogger(t), 100, 50, sendInterval, time.Second, uniconn, "test", "test") - tc.(*telemetryIngressBatchClient).close = func() error { return nil } + tc := NewTelemetryIngressBatchClient(url, serverPubKeyHex, ks, logging, logger.TestLogger(t), 100, 50, sendInterval, time.Second, uniconn) + tc.(*telemetryIngressBatchClient).closeFn = func() error { return nil } tc.(*telemetryIngressBatchClient).telemClient = telemClient return tc } diff --git a/core/services/synchronization/telemetry_ingress_batch_client.go b/core/services/synchronization/telemetry_ingress_batch_client.go index cade98cf606..26ce1e3066a 100644 --- a/core/services/synchronization/telemetry_ingress_batch_client.go +++ b/core/services/synchronization/telemetry_ingress_batch_client.go @@ -12,8 +12,9 @@ import ( "github.com/smartcontractkit/wsrpc" "github.com/smartcontractkit/wsrpc/examples/simple/keys" + "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/services" - "github.com/smartcontractkit/chainlink/v2/core/logger" + "github.com/smartcontractkit/chainlink-common/pkg/timeutil" "github.com/smartcontractkit/chainlink/v2/core/services/keystore" telemPb "github.com/smartcontractkit/chainlink/v2/core/services/synchronization/telem" ) @@ -37,21 +38,18 @@ func (NoopTelemetryIngressBatchClient) Name() string { return func (NoopTelemetryIngressBatchClient) Ready() error { return nil } type telemetryIngressBatchClient struct { - services.StateMachine + services.Service + eng *services.Engine + url *url.URL ks keystore.CSA serverPubKeyHex string connected atomic.Bool telemClient telemPb.TelemClient - close func() error - - globalLogger logger.Logger - logging bool - lggr logger.Logger + closeFn func() error - wgDone sync.WaitGroup - chDone services.StopChan + logging bool telemBufferSize uint telemMaxBatchSize uint @@ -66,8 +64,8 @@ type telemetryIngressBatchClient struct { // NewTelemetryIngressBatchClient returns a client backed by wsrpc that // can send telemetry to the telemetry ingress server -func NewTelemetryIngressBatchClient(url *url.URL, serverPubKeyHex string, ks keystore.CSA, logging bool, lggr logger.Logger, telemBufferSize uint, telemMaxBatchSize uint, telemSendInterval time.Duration, telemSendTimeout time.Duration, useUniconn bool, network string, chainID string) TelemetryService { - return &telemetryIngressBatchClient{ +func NewTelemetryIngressBatchClient(url *url.URL, serverPubKeyHex string, ks keystore.CSA, logging bool, lggr logger.Logger, telemBufferSize uint, telemMaxBatchSize uint, telemSendInterval time.Duration, telemSendTimeout time.Duration, useUniconn bool) TelemetryService { + c := &telemetryIngressBatchClient{ telemBufferSize: telemBufferSize, telemMaxBatchSize: telemMaxBatchSize, telemSendInterval: telemSendInterval, @@ -75,13 +73,17 @@ func NewTelemetryIngressBatchClient(url *url.URL, serverPubKeyHex string, ks key url: url, ks: ks, serverPubKeyHex: serverPubKeyHex, - globalLogger: lggr, logging: logging, - lggr: lggr.Named("TelemetryIngressBatchClient").Named(network).Named(chainID), - chDone: make(services.StopChan), workers: make(map[string]*telemetryIngressBatchWorker), useUniConn: useUniconn, } + c.Service, c.eng = services.Config{ + Name: "TelemetryIngressBatchClient", + Start: c.start, + Close: c.close, + }.NewServiceEngine(lggr) + + return c } // Start connects the wsrpc client to the telemetry ingress server @@ -90,71 +92,53 @@ func NewTelemetryIngressBatchClient(url *url.URL, serverPubKeyHex string, ks key // an error and wsrpc will continue to retry the connection. Eventually when the ingress // server does come back up, wsrpc will establish the connection without any interaction // on behalf of the node operator. -func (tc *telemetryIngressBatchClient) Start(ctx context.Context) error { - return tc.StartOnce("TelemetryIngressBatchClient", func() error { - clientPrivKey, err := tc.getCSAPrivateKey() - if err != nil { - return err - } +func (tc *telemetryIngressBatchClient) start(ctx context.Context) error { + clientPrivKey, err := tc.getCSAPrivateKey() + if err != nil { + return err + } - serverPubKey := keys.FromHex(tc.serverPubKeyHex) - - // Initialize a new wsrpc client caller - // This is used to call RPC methods on the server - if tc.telemClient == nil { // only preset for tests - if tc.useUniConn { - tc.wgDone.Add(1) - go func() { - defer tc.wgDone.Done() - ctx2, cancel := tc.chDone.NewCtx() - defer cancel() - conn, err := wsrpc.DialUniWithContext(ctx2, tc.lggr, tc.url.String(), clientPrivKey, serverPubKey) - if err != nil { - if ctx2.Err() != nil { - tc.lggr.Warnw("gave up connecting to telemetry endpoint", "err", err) - } else { - tc.lggr.Criticalw("telemetry endpoint dial errored unexpectedly", "err", err, "server pubkey", tc.serverPubKeyHex) - tc.SvcErrBuffer.Append(err) - } - return - } - tc.telemClient = telemPb.NewTelemClient(conn) - tc.close = conn.Close - tc.connected.Store(true) - }() - } else { - // Spawns a goroutine that will eventually connect - conn, err := wsrpc.DialWithContext(ctx, tc.url.String(), wsrpc.WithTransportCreds(clientPrivKey, serverPubKey), wsrpc.WithLogger(tc.lggr)) + serverPubKey := keys.FromHex(tc.serverPubKeyHex) + + // Initialize a new wsrpc client caller + // This is used to call RPC methods on the server + if tc.telemClient == nil { // only preset for tests + if tc.useUniConn { + tc.eng.Go(func(ctx context.Context) { + conn, err := wsrpc.DialUniWithContext(ctx, tc.eng, tc.url.String(), clientPrivKey, serverPubKey) if err != nil { - return fmt.Errorf("could not start TelemIngressBatchClient, Dial returned error: %v", err) + if ctx.Err() != nil { + tc.eng.Warnw("gave up connecting to telemetry endpoint", "err", err) + } else { + tc.eng.Criticalw("telemetry endpoint dial errored unexpectedly", "err", err, "server pubkey", tc.serverPubKeyHex) + tc.eng.EmitHealthErr(err) + } + return } tc.telemClient = telemPb.NewTelemClient(conn) - tc.close = func() error { conn.Close(); return nil } + tc.closeFn = conn.Close + tc.connected.Store(true) + }) + } else { + // Spawns a goroutine that will eventually connect + conn, err := wsrpc.DialWithContext(ctx, tc.url.String(), wsrpc.WithTransportCreds(clientPrivKey, serverPubKey), wsrpc.WithLogger(tc.eng)) + if err != nil { + return fmt.Errorf("could not start TelemIngressBatchClient, Dial returned error: %v", err) } + tc.telemClient = telemPb.NewTelemClient(conn) + tc.closeFn = func() error { conn.Close(); return nil } } + } - return nil - }) + return nil } // Close disconnects the wsrpc client from the ingress server and waits for all workers to exit -func (tc *telemetryIngressBatchClient) Close() error { - return tc.StopOnce("TelemetryIngressBatchClient", func() error { - close(tc.chDone) - tc.wgDone.Wait() - if (tc.useUniConn && tc.connected.Load()) || !tc.useUniConn { - return tc.close() - } - return nil - }) -} - -func (tc *telemetryIngressBatchClient) Name() string { - return tc.lggr.Name() -} - -func (tc *telemetryIngressBatchClient) HealthReport() map[string]error { - return map[string]error{tc.Name(): tc.Healthy()} +func (tc *telemetryIngressBatchClient) close() error { + if (tc.useUniConn && tc.connected.Load()) || !tc.useUniConn { + return tc.closeFn() + } + return nil } // getCSAPrivateKey gets the client's CSA private key @@ -175,7 +159,7 @@ func (tc *telemetryIngressBatchClient) getCSAPrivateKey() (privkey []byte, err e // and a warning is logged. func (tc *telemetryIngressBatchClient) Send(ctx context.Context, telemData []byte, contractID string, telemType TelemetryType) { if tc.useUniConn && !tc.connected.Load() { - tc.lggr.Warnw("not connected to telemetry endpoint", "endpoint", tc.url.String()) + tc.eng.Warnw("not connected to telemetry endpoint", "endpoint", tc.url.String()) return } payload := TelemPayload{ @@ -206,18 +190,17 @@ func (tc *telemetryIngressBatchClient) findOrCreateWorker(payload TelemPayload) if !found { worker = NewTelemetryIngressBatchWorker( tc.telemMaxBatchSize, - tc.telemSendInterval, tc.telemSendTimeout, tc.telemClient, - &tc.wgDone, - tc.chDone, make(chan TelemPayload, tc.telemBufferSize), payload.ContractID, payload.TelemType, - tc.globalLogger, + tc.eng, tc.logging, ) - worker.Start() + tc.eng.GoTick(timeutil.NewTicker(func() time.Duration { + return tc.telemSendInterval + }), worker.Send) tc.workers[workerKey] = worker } diff --git a/core/services/synchronization/telemetry_ingress_batch_worker.go b/core/services/synchronization/telemetry_ingress_batch_worker.go index e7ea6595811..7eca26f02c9 100644 --- a/core/services/synchronization/telemetry_ingress_batch_worker.go +++ b/core/services/synchronization/telemetry_ingress_batch_worker.go @@ -2,13 +2,12 @@ package synchronization import ( "context" - "sync" "sync/atomic" "time" "github.com/smartcontractkit/chainlink-common/pkg/services" - "github.com/smartcontractkit/chainlink/v2/core/logger" + "github.com/smartcontractkit/chainlink-common/pkg/logger" telemPb "github.com/smartcontractkit/chainlink/v2/core/services/synchronization/telem" ) @@ -18,11 +17,8 @@ type telemetryIngressBatchWorker struct { services.Service telemMaxBatchSize uint - telemSendInterval time.Duration telemSendTimeout time.Duration telemClient telemPb.TelemClient - wgDone *sync.WaitGroup - chDone services.StopChan chTelemetry chan TelemPayload contractID string telemType TelemetryType @@ -35,65 +31,45 @@ type telemetryIngressBatchWorker struct { // telemetry to the ingress server via WSRPC func NewTelemetryIngressBatchWorker( telemMaxBatchSize uint, - telemSendInterval time.Duration, telemSendTimeout time.Duration, telemClient telemPb.TelemClient, - wgDone *sync.WaitGroup, - chDone chan struct{}, chTelemetry chan TelemPayload, contractID string, telemType TelemetryType, - globalLogger logger.Logger, + lggr logger.Logger, logging bool, ) *telemetryIngressBatchWorker { return &telemetryIngressBatchWorker{ - telemSendInterval: telemSendInterval, telemSendTimeout: telemSendTimeout, telemMaxBatchSize: telemMaxBatchSize, telemClient: telemClient, - wgDone: wgDone, - chDone: chDone, chTelemetry: chTelemetry, contractID: contractID, telemType: telemType, logging: logging, - lggr: globalLogger.Named("TelemetryIngressBatchWorker"), + lggr: logger.Named(lggr, "TelemetryIngressBatchWorker"), } } -// Start sends batched telemetry to the ingress server on an interval -func (tw *telemetryIngressBatchWorker) Start() { - tw.wgDone.Add(1) - sendTicker := time.NewTicker(tw.telemSendInterval) - - go func() { - defer tw.wgDone.Done() - - for { - select { - case <-sendTicker.C: - if len(tw.chTelemetry) == 0 { - continue - } +// Send sends batched telemetry to the ingress server on an interval +func (tw *telemetryIngressBatchWorker) Send(ctx context.Context) { + if len(tw.chTelemetry) == 0 { + return + } - // Send batched telemetry to the ingress server, log any errors - telemBatchReq := tw.BuildTelemBatchReq() - ctx, cancel := tw.chDone.CtxCancel(context.WithTimeout(context.Background(), tw.telemSendTimeout)) - _, err := tw.telemClient.TelemBatch(ctx, telemBatchReq) - cancel() + // Send batched telemetry to the ingress server, log any errors + telemBatchReq := tw.BuildTelemBatchReq() + ctx, cancel := context.WithTimeout(ctx, tw.telemSendTimeout) + _, err := tw.telemClient.TelemBatch(ctx, telemBatchReq) + cancel() - if err != nil { - tw.lggr.Warnf("Could not send telemetry: %v", err) - continue - } - if tw.logging { - tw.lggr.Debugw("Successfully sent telemetry to ingress server", "contractID", telemBatchReq.ContractId, "telemType", telemBatchReq.TelemetryType, "telemetry", telemBatchReq.Telemetry) - } - case <-tw.chDone: - return - } - } - }() + if err != nil { + tw.lggr.Warnf("Could not send telemetry: %v", err) + return + } + if tw.logging { + tw.lggr.Debugw("Successfully sent telemetry to ingress server", "contractID", telemBatchReq.ContractId, "telemType", telemBatchReq.TelemetryType, "telemetry", telemBatchReq.Telemetry) + } } // logBufferFullWithExpBackoff logs messages at diff --git a/core/services/synchronization/telemetry_ingress_batch_worker_test.go b/core/services/synchronization/telemetry_ingress_batch_worker_test.go index 109022c7135..bf44ee9195a 100644 --- a/core/services/synchronization/telemetry_ingress_batch_worker_test.go +++ b/core/services/synchronization/telemetry_ingress_batch_worker_test.go @@ -1,7 +1,6 @@ package synchronization_test import ( - "sync" "testing" "time" @@ -22,11 +21,8 @@ func TestTelemetryIngressWorker_BuildTelemBatchReq(t *testing.T) { chTelemetry := make(chan synchronization.TelemPayload, 10) worker := synchronization.NewTelemetryIngressBatchWorker( uint(maxTelemBatchSize), - time.Millisecond*1, time.Second, mocks.NewTelemClient(t), - &sync.WaitGroup{}, - make(chan struct{}), chTelemetry, "0xa", synchronization.OCR, diff --git a/core/services/synchronization/telemetry_ingress_client.go b/core/services/synchronization/telemetry_ingress_client.go index dc4ced31d09..1ed55bb5468 100644 --- a/core/services/synchronization/telemetry_ingress_client.go +++ b/core/services/synchronization/telemetry_ingress_client.go @@ -4,15 +4,14 @@ import ( "context" "errors" "net/url" - "sync" "sync/atomic" "time" "github.com/smartcontractkit/wsrpc" "github.com/smartcontractkit/wsrpc/examples/simple/keys" + "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/services" - "github.com/smartcontractkit/chainlink/v2/core/logger" "github.com/smartcontractkit/chainlink/v2/core/services/keystore" telemPb "github.com/smartcontractkit/chainlink/v2/core/services/synchronization/telem" ) @@ -35,82 +34,59 @@ func (NoopTelemetryIngressClient) Name() string { return "Noop func (NoopTelemetryIngressClient) Ready() error { return nil } type telemetryIngressClient struct { - services.StateMachine + services.Service + eng *services.Engine + url *url.URL ks keystore.CSA serverPubKeyHex string telemClient telemPb.TelemClient logging bool - lggr logger.Logger - wgDone sync.WaitGroup - chDone services.StopChan dropMessageCount atomic.Uint32 chTelemetry chan TelemPayload } // NewTelemetryIngressClient returns a client backed by wsrpc that // can send telemetry to the telemetry ingress server -func NewTelemetryIngressClient(url *url.URL, serverPubKeyHex string, ks keystore.CSA, logging bool, lggr logger.Logger, telemBufferSize uint, network string, chainID string) TelemetryService { - return &telemetryIngressClient{ +func NewTelemetryIngressClient(url *url.URL, serverPubKeyHex string, ks keystore.CSA, logging bool, lggr logger.Logger, telemBufferSize uint) TelemetryService { + c := &telemetryIngressClient{ url: url, ks: ks, serverPubKeyHex: serverPubKeyHex, logging: logging, - lggr: lggr.Named("TelemetryIngressClient").Named(network).Named(chainID), chTelemetry: make(chan TelemPayload, telemBufferSize), - chDone: make(services.StopChan), } + c.Service, c.eng = services.Config{ + Name: "TelemetryIngressClient", + Start: c.start, + }.NewServiceEngine(lggr) + return c } // Start connects the wsrpc client to the telemetry ingress server -func (tc *telemetryIngressClient) Start(context.Context) error { - return tc.StartOnce("TelemetryIngressClient", func() error { - privkey, err := tc.getCSAPrivateKey() - if err != nil { - return err - } - - tc.connect(privkey) - - return nil - }) -} - -// Close disconnects the wsrpc client from the ingress server -func (tc *telemetryIngressClient) Close() error { - return tc.StopOnce("TelemetryIngressClient", func() error { - close(tc.chDone) - tc.wgDone.Wait() - return nil - }) -} +func (tc *telemetryIngressClient) start(context.Context) error { + privkey, err := tc.getCSAPrivateKey() + if err != nil { + return err + } -func (tc *telemetryIngressClient) Name() string { - return tc.lggr.Name() -} + tc.connect(privkey) -func (tc *telemetryIngressClient) HealthReport() map[string]error { - return map[string]error{tc.Name(): tc.Healthy()} + return nil } func (tc *telemetryIngressClient) connect(clientPrivKey []byte) { - tc.wgDone.Add(1) - - go func() { - defer tc.wgDone.Done() - ctx, cancel := tc.chDone.NewCtx() - defer cancel() - + tc.eng.Go(func(ctx context.Context) { serverPubKey := keys.FromHex(tc.serverPubKeyHex) - conn, err := wsrpc.DialWithContext(ctx, tc.url.String(), wsrpc.WithTransportCreds(clientPrivKey, serverPubKey), wsrpc.WithLogger(tc.lggr)) + conn, err := wsrpc.DialWithContext(ctx, tc.url.String(), wsrpc.WithTransportCreds(clientPrivKey, serverPubKey), wsrpc.WithLogger(tc.eng)) if err != nil { if ctx.Err() != nil { - tc.lggr.Warnw("gave up connecting to telemetry endpoint", "err", err) + tc.eng.Warnw("gave up connecting to telemetry endpoint", "err", err) } else { - tc.lggr.Criticalw("telemetry endpoint dial errored unexpectedly", "err", err) - tc.SvcErrBuffer.Append(err) + tc.eng.Criticalw("telemetry endpoint dial errored unexpectedly", "err", err) + tc.eng.EmitHealthErr(err) } return } @@ -126,16 +102,12 @@ func (tc *telemetryIngressClient) connect(clientPrivKey []byte) { tc.handleTelemetry() // Wait for close - <-tc.chDone - }() + <-ctx.Done() + }) } func (tc *telemetryIngressClient) handleTelemetry() { - tc.wgDone.Add(1) - go func() { - defer tc.wgDone.Done() - ctx, cancel := tc.chDone.NewCtx() - defer cancel() + tc.eng.Go(func(ctx context.Context) { for { select { case p := <-tc.chTelemetry: @@ -148,17 +120,17 @@ func (tc *telemetryIngressClient) handleTelemetry() { } _, err := tc.telemClient.Telem(ctx, telemReq) if err != nil { - tc.lggr.Errorf("Could not send telemetry: %v", err) + tc.eng.Errorf("Could not send telemetry: %v", err) continue } if tc.logging { - tc.lggr.Debugw("successfully sent telemetry to ingress server", "contractID", p.ContractID, "telemetry", p.Telemetry) + tc.eng.Debugw("successfully sent telemetry to ingress server", "contractID", p.ContractID, "telemetry", p.Telemetry) } - case <-tc.chDone: + case <-ctx.Done(): return } } - }() + }) } // logBufferFullWithExpBackoff logs messages at @@ -176,7 +148,7 @@ func (tc *telemetryIngressClient) handleTelemetry() { func (tc *telemetryIngressClient) logBufferFullWithExpBackoff(payload TelemPayload) { count := tc.dropMessageCount.Add(1) if count > 0 && (count%100 == 0 || count&(count-1) == 0) { - tc.lggr.Warnw("telemetry ingress client buffer full, dropping message", "telemetry", payload.Telemetry, "droppedCount", count) + tc.eng.Warnw("telemetry ingress client buffer full, dropping message", "telemetry", payload.Telemetry, "droppedCount", count) } } diff --git a/core/services/telemetry/manager.go b/core/services/telemetry/manager.go index a65759a5c62..73a94b4b127 100644 --- a/core/services/telemetry/manager.go +++ b/core/services/telemetry/manager.go @@ -1,29 +1,29 @@ package telemetry import ( - "context" "net/url" "strings" "time" "github.com/pkg/errors" - "go.uber.org/multierr" - "github.com/smartcontractkit/libocr/commontypes" + "github.com/smartcontractkit/chainlink-common/pkg/logger" + common "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/services" "github.com/smartcontractkit/chainlink/v2/core/config" - "github.com/smartcontractkit/chainlink/v2/core/logger" "github.com/smartcontractkit/chainlink/v2/core/services/keystore" "github.com/smartcontractkit/chainlink/v2/core/services/synchronization" ) type Manager struct { - services.StateMachine - bufferSize uint - endpoints []*telemetryEndpoint - ks keystore.CSA - lggr logger.Logger + services.Service + eng *services.Engine + + bufferSize uint + endpoints []*telemetryEndpoint + ks keystore.CSA + logging bool maxBatchSize uint sendInterval time.Duration @@ -45,9 +45,7 @@ type telemetryEndpoint struct { func NewManager(cfg config.TelemetryIngress, csaKeyStore keystore.CSA, lggr logger.Logger) *Manager { m := &Manager{ bufferSize: cfg.BufferSize(), - endpoints: nil, ks: csaKeyStore, - lggr: lggr.Named("TelemetryManager"), logging: cfg.Logging(), maxBatchSize: cfg.MaxBatchSize(), sendInterval: cfg.SendInterval(), @@ -55,44 +53,21 @@ func NewManager(cfg config.TelemetryIngress, csaKeyStore keystore.CSA, lggr logg uniConn: cfg.UniConn(), useBatchSend: cfg.UseBatchSend(), } - for _, e := range cfg.Endpoints() { - if err := m.addEndpoint(e); err != nil { - m.lggr.Error(err) - } - } - return m -} - -func (m *Manager) Start(ctx context.Context) error { - return m.StartOnce("TelemetryManager", func() error { - var err error - for _, e := range m.endpoints { - err = multierr.Append(err, e.client.Start(ctx)) - } - return err - }) -} -func (m *Manager) Close() error { - return m.StopOnce("TelemetryManager", func() error { - var err error - for _, e := range m.endpoints { - err = multierr.Append(err, e.client.Close()) - } - return err - }) -} - -func (m *Manager) Name() string { - return m.lggr.Name() -} + m.Service, m.eng = services.Config{ + Name: "TelemetryManager", + NewSubServices: func(lggr common.Logger) (subs []services.Service) { + for _, e := range cfg.Endpoints() { + if sub, err := m.newEndpoint(e, lggr, cfg); err != nil { + lggr.Error(err) + } else { + subs = append(subs, sub) + } + } + return + }, + }.NewServiceEngine(lggr) -func (m *Manager) HealthReport() map[string]error { - hr := map[string]error{m.Name(): m.Healthy()} - - for _, e := range m.endpoints { - services.CopyHealth(hr, e.client.HealthReport()) - } - return hr + return m } // GenMonitoringEndpoint creates a new monitoring endpoints based on the existing available endpoints defined in the core config TOML, if no endpoint for the network and chainID exists, a NOOP agent will be used and the telemetry will not be sent @@ -100,7 +75,7 @@ func (m *Manager) GenMonitoringEndpoint(network string, chainID string, contract e, found := m.getEndpoint(network, chainID) if !found { - m.lggr.Warnf("no telemetry endpoint found for network %q chainID %q, telemetry %q for contactID %q will NOT be sent", network, chainID, telemType, contractID) + m.eng.Warnf("no telemetry endpoint found for network %q chainID %q, telemetry %q for contactID %q will NOT be sent", network, chainID, telemType, contractID) return &NoopAgent{} } @@ -111,32 +86,33 @@ func (m *Manager) GenMonitoringEndpoint(network string, chainID string, contract return NewIngressAgent(e.client, network, chainID, contractID, telemType) } -func (m *Manager) addEndpoint(e config.TelemetryIngressEndpoint) error { +func (m *Manager) newEndpoint(e config.TelemetryIngressEndpoint, lggr logger.Logger, cfg config.TelemetryIngress) (services.Service, error) { if e.Network() == "" { - return errors.New("cannot add telemetry endpoint, network cannot be empty") + return nil, errors.New("cannot add telemetry endpoint, network cannot be empty") } if e.ChainID() == "" { - return errors.New("cannot add telemetry endpoint, chainID cannot be empty") + return nil, errors.New("cannot add telemetry endpoint, chainID cannot be empty") } if e.URL() == nil { - return errors.New("cannot add telemetry endpoint, URL cannot be empty") + return nil, errors.New("cannot add telemetry endpoint, URL cannot be empty") } if e.ServerPubKey() == "" { - return errors.New("cannot add telemetry endpoint, ServerPubKey cannot be empty") + return nil, errors.New("cannot add telemetry endpoint, ServerPubKey cannot be empty") } if _, found := m.getEndpoint(e.Network(), e.ChainID()); found { - return errors.Errorf("cannot add telemetry endpoint for network %q and chainID %q, endpoint already exists", e.Network(), e.ChainID()) + return nil, errors.Errorf("cannot add telemetry endpoint for network %q and chainID %q, endpoint already exists", e.Network(), e.ChainID()) } + lggr = logger.Sugared(lggr).Named(e.Network()).Named(e.ChainID()) var tClient synchronization.TelemetryService if m.useBatchSend { - tClient = synchronization.NewTelemetryIngressBatchClient(e.URL(), e.ServerPubKey(), m.ks, m.logging, m.lggr, m.bufferSize, m.maxBatchSize, m.sendInterval, m.sendTimeout, m.uniConn, e.Network(), e.ChainID()) + tClient = synchronization.NewTelemetryIngressBatchClient(e.URL(), e.ServerPubKey(), m.ks, cfg.Logging(), lggr, cfg.BufferSize(), cfg.MaxBatchSize(), cfg.SendInterval(), cfg.SendTimeout(), cfg.UniConn()) } else { - tClient = synchronization.NewTelemetryIngressClient(e.URL(), e.ServerPubKey(), m.ks, m.logging, m.lggr, m.bufferSize, e.Network(), e.ChainID()) + tClient = synchronization.NewTelemetryIngressClient(e.URL(), e.ServerPubKey(), m.ks, cfg.Logging(), lggr, cfg.BufferSize()) } te := telemetryEndpoint{ @@ -148,7 +124,7 @@ func (m *Manager) addEndpoint(e config.TelemetryIngressEndpoint) error { } m.endpoints = append(m.endpoints, &te) - return nil + return te.client, nil } func (m *Manager) getEndpoint(network string, chainID string) (*telemetryEndpoint, bool) { diff --git a/core/services/telemetry/manager_test.go b/core/services/telemetry/manager_test.go index 4e55cb75752..fef065b572c 100644 --- a/core/services/telemetry/manager_test.go +++ b/core/services/telemetry/manager_test.go @@ -156,7 +156,7 @@ func TestNewManager(t *testing.T) { require.Equal(t, uint(123), m.bufferSize) require.Equal(t, ks, m.ks) - require.Equal(t, "TelemetryManager", m.lggr.Name()) + require.Equal(t, "TelemetryManager", m.Name()) require.Equal(t, true, m.logging) require.Equal(t, uint(51), m.maxBatchSize) require.Equal(t, time.Millisecond*512, m.sendInterval) From 4843d84c260c0300eecfad413cca60af481280bf Mon Sep 17 00:00:00 2001 From: Lukasz <120112546+lukaszcl@users.noreply.github.com> Date: Wed, 7 Aug 2024 15:39:35 +0200 Subject: [PATCH 2/6] Update e2e tests definition for CI and automation workflow (#13908) * Update e2e tests definition for CI * Update test --- .github/e2e-tests.yml | 32 ++++++++++++++++--- .../run-automation-ondemand-e2e-tests.yml | 10 +++++- 2 files changed, 37 insertions(+), 5 deletions(-) diff --git a/.github/e2e-tests.yml b/.github/e2e-tests.yml index 0d92d1900dc..b2c9f12fcaf 100644 --- a/.github/e2e-tests.yml +++ b/.github/e2e-tests.yml @@ -47,6 +47,8 @@ runner-test-matrix: test_env_type: k8s-remote-runner runs_on: ubuntu-latest test_cmd: cd integration-tests/ && go test soak/ocr_test.go -v -test.run ^TestOCRv1Soak$ -test.parallel=1 -timeout 30m -count=1 -json + test_config_override_required: true + test_secrets_required: true test_inputs: test_suite: soak @@ -543,15 +545,37 @@ runner-test-matrix: chainlink_upgrade_version: develop pyroscope_env: ci-smoke-automation-upgrade-tests - - id: integration-tests/reorg/automation_reorg_test.go + - id: integration-tests/reorg/automation_reorg_test.go^TestAutomationReorg/registry_2_0 path: integration-tests/reorg/automation_reorg_test.go runs_on: ubuntu-latest - test_env_type: k8s-remote-runner + test_env_type: docker + test_inputs: + test_suite: reorg + workflows: + - Run Automation On Demand Tests (TEST WORKFLOW) + test_cmd: cd integration-tests/reorg && DETACH_RUNNER=false go test -v -test.run ^TestAutomationReorg/registry_2_0 -test.parallel=1 -timeout 30m -count=1 -json + pyroscope_env: ci-automation-on-demand-reorg + + - id: integration-tests/reorg/automation_reorg_test.go^TestAutomationReorg/registry_2_1 + path: integration-tests/reorg/automation_reorg_test.go + runs_on: ubuntu-latest + test_env_type: docker + test_inputs: + test_suite: reorg + workflows: + - Run Automation On Demand Tests (TEST WORKFLOW) + test_cmd: cd integration-tests/reorg && DETACH_RUNNER=false go test -v -test.run ^TestAutomationReorg/registry_2_1 -test.parallel=2 -timeout 30m -count=1 -json + pyroscope_env: ci-automation-on-demand-reorg + + - id: integration-tests/reorg/automation_reorg_test.go^TestAutomationReorg/registry_2_2 + path: integration-tests/reorg/automation_reorg_test.go + runs_on: ubuntu-latest + test_env_type: docker test_inputs: test_suite: reorg workflows: - Run Automation On Demand Tests (TEST WORKFLOW) - test_cmd: cd integration-tests/reorg && DETACH_RUNNER=false go test -v -test.run ^TestAutomationReorg$ -test.parallel=7 -timeout 60m -count=1 -json + test_cmd: cd integration-tests/reorg && DETACH_RUNNER=false go test -v -test.run ^TestAutomationReorg/registry_2_2 -test.parallel=2 -timeout 30m -count=1 -json pyroscope_env: ci-automation-on-demand-reorg - id: integration-tests/chaos/automation_chaos_test.go @@ -560,7 +584,7 @@ runner-test-matrix: runs_on: ubuntu-latest workflows: - Run Automation On Demand Tests (TEST WORKFLOW) - test_cmd: cd integration-tests/chaos && DETACH_RUNNER=false go test -v -test.run ^TestAutomationChaos$ -test.parallel=15 -timeout 60m -count=1 -json + test_cmd: cd integration-tests/chaos && DETACH_RUNNER=false go test -v -test.run ^TestAutomationChaos$ -test.parallel=20 -timeout 60m -count=1 -json pyroscope_env: ci-automation-on-demand-chaos test_inputs: test_suite: chaos diff --git a/.github/workflows/run-automation-ondemand-e2e-tests.yml b/.github/workflows/run-automation-ondemand-e2e-tests.yml index 7bf4691ecc5..8dac3c56994 100644 --- a/.github/workflows/run-automation-ondemand-e2e-tests.yml +++ b/.github/workflows/run-automation-ondemand-e2e-tests.yml @@ -116,10 +116,18 @@ jobs: # Run reorg tests if enabled if [[ "${{ github.event.inputs.enableReorg }}" == 'true' ]]; then cat >> test_list.yaml < Date: Wed, 7 Aug 2024 07:42:01 -0600 Subject: [PATCH 3/6] bump solana commit (#14062) --- core/scripts/go.mod | 2 +- core/scripts/go.sum | 4 ++-- go.mod | 2 +- go.sum | 4 ++-- integration-tests/go.mod | 2 +- integration-tests/go.sum | 4 ++-- integration-tests/load/go.mod | 2 +- integration-tests/load/go.sum | 4 ++-- 8 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/scripts/go.mod b/core/scripts/go.mod index 45b5ee59059..94504897ab0 100644 --- a/core/scripts/go.mod +++ b/core/scripts/go.mod @@ -273,7 +273,7 @@ require ( github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240710121324-3ed288aa9b45 // indirect github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761c982f // indirect github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827 // indirect - github.com/smartcontractkit/chainlink-solana v1.1.0 // indirect + github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564 // indirect github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799 // indirect github.com/smartcontractkit/tdh2/go/ocr2/decryptionplugin v0.0.0-20230906073235-9e478e5e19f1 // indirect github.com/smartcontractkit/tdh2/go/tdh2 v0.0.0-20230906073235-9e478e5e19f1 // indirect diff --git a/core/scripts/go.sum b/core/scripts/go.sum index dff6f3f356a..f770498cff8 100644 --- a/core/scripts/go.sum +++ b/core/scripts/go.sum @@ -1192,8 +1192,8 @@ github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761 github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761c982f/go.mod h1:V/86loaFSH0dqqUEHqyXVbyNqDRSjvcf9BRomWFTljU= github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827 h1:BCHu4pNP6arrcHLEWx61XjLaonOd2coQNyL0NTUcaMc= github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827/go.mod h1:OPX+wC2TWQsyLNpR7daMt2vMpmsNcoBxbZyGTHr6tiA= -github.com/smartcontractkit/chainlink-solana v1.1.0 h1:+xBeVqx2x0Sx3CBbF8RLSblczsxJDYTkta8h7i8+23I= -github.com/smartcontractkit/chainlink-solana v1.1.0/go.mod h1:Ml88TJTwZCj6yHDkAEN/EhxVutzSlk+kDZgfibRIqF0= +github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564 h1:8ZzsGNhqYxmQ/QMO1fuXO7u9Vpl9YUvPJK+td/ZaBJA= +github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564/go.mod h1:Ml88TJTwZCj6yHDkAEN/EhxVutzSlk+kDZgfibRIqF0= github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799 h1:HyLTySm7BR+oNfZqDTkVJ25wnmcTtxBBD31UkFL+kEM= github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799/go.mod h1:UVFRacRkP7O7TQAzFmR52v5mUlxf+G1ovMlCQAB/cHU= github.com/smartcontractkit/go-plugin v0.0.0-20240208201424-b3b91517de16 h1:TFe+FvzxClblt6qRfqEhUfa4kFQx5UobuoFGO2W4mMo= diff --git a/go.mod b/go.mod index 78ec7d29ee1..2179ffc2d21 100644 --- a/go.mod +++ b/go.mod @@ -78,7 +78,7 @@ require ( github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240710121324-3ed288aa9b45 github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761c982f github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827 - github.com/smartcontractkit/chainlink-solana v1.1.0 + github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564 github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799 github.com/smartcontractkit/libocr v0.0.0-20240717100443-f6226e09bee7 github.com/smartcontractkit/tdh2/go/ocr2/decryptionplugin v0.0.0-20230906073235-9e478e5e19f1 diff --git a/go.sum b/go.sum index f5ef0f91e70..b953f315e92 100644 --- a/go.sum +++ b/go.sum @@ -1147,8 +1147,8 @@ github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761 github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761c982f/go.mod h1:V/86loaFSH0dqqUEHqyXVbyNqDRSjvcf9BRomWFTljU= github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827 h1:BCHu4pNP6arrcHLEWx61XjLaonOd2coQNyL0NTUcaMc= github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827/go.mod h1:OPX+wC2TWQsyLNpR7daMt2vMpmsNcoBxbZyGTHr6tiA= -github.com/smartcontractkit/chainlink-solana v1.1.0 h1:+xBeVqx2x0Sx3CBbF8RLSblczsxJDYTkta8h7i8+23I= -github.com/smartcontractkit/chainlink-solana v1.1.0/go.mod h1:Ml88TJTwZCj6yHDkAEN/EhxVutzSlk+kDZgfibRIqF0= +github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564 h1:8ZzsGNhqYxmQ/QMO1fuXO7u9Vpl9YUvPJK+td/ZaBJA= +github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564/go.mod h1:Ml88TJTwZCj6yHDkAEN/EhxVutzSlk+kDZgfibRIqF0= github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799 h1:HyLTySm7BR+oNfZqDTkVJ25wnmcTtxBBD31UkFL+kEM= github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799/go.mod h1:UVFRacRkP7O7TQAzFmR52v5mUlxf+G1ovMlCQAB/cHU= github.com/smartcontractkit/go-plugin v0.0.0-20240208201424-b3b91517de16 h1:TFe+FvzxClblt6qRfqEhUfa4kFQx5UobuoFGO2W4mMo= diff --git a/integration-tests/go.mod b/integration-tests/go.mod index a648e46e9f0..ff60a8f78b3 100644 --- a/integration-tests/go.mod +++ b/integration-tests/go.mod @@ -380,7 +380,7 @@ require ( github.com/smartcontractkit/chainlink-cosmos v0.4.1-0.20240710121324-3ed288aa9b45 // indirect github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761c982f // indirect github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827 // indirect - github.com/smartcontractkit/chainlink-solana v1.1.0 // indirect + github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564 // indirect github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799 // indirect github.com/smartcontractkit/tdh2/go/ocr2/decryptionplugin v0.0.0-20230906073235-9e478e5e19f1 // indirect github.com/smartcontractkit/tdh2/go/tdh2 v0.0.0-20230906073235-9e478e5e19f1 // indirect diff --git a/integration-tests/go.sum b/integration-tests/go.sum index 03e4a9082ff..5d15dfd92f6 100644 --- a/integration-tests/go.sum +++ b/integration-tests/go.sum @@ -1496,8 +1496,8 @@ github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761 github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761c982f/go.mod h1:V/86loaFSH0dqqUEHqyXVbyNqDRSjvcf9BRomWFTljU= github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827 h1:BCHu4pNP6arrcHLEWx61XjLaonOd2coQNyL0NTUcaMc= github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827/go.mod h1:OPX+wC2TWQsyLNpR7daMt2vMpmsNcoBxbZyGTHr6tiA= -github.com/smartcontractkit/chainlink-solana v1.1.0 h1:+xBeVqx2x0Sx3CBbF8RLSblczsxJDYTkta8h7i8+23I= -github.com/smartcontractkit/chainlink-solana v1.1.0/go.mod h1:Ml88TJTwZCj6yHDkAEN/EhxVutzSlk+kDZgfibRIqF0= +github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564 h1:8ZzsGNhqYxmQ/QMO1fuXO7u9Vpl9YUvPJK+td/ZaBJA= +github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564/go.mod h1:Ml88TJTwZCj6yHDkAEN/EhxVutzSlk+kDZgfibRIqF0= github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799 h1:HyLTySm7BR+oNfZqDTkVJ25wnmcTtxBBD31UkFL+kEM= github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799/go.mod h1:UVFRacRkP7O7TQAzFmR52v5mUlxf+G1ovMlCQAB/cHU= github.com/smartcontractkit/chainlink-testing-framework v1.34.2 h1:YL3ft7KJB7SAopdmJeyeR4/kv0j4jOdagNihXq8OZ38= diff --git a/integration-tests/load/go.mod b/integration-tests/load/go.mod index 1aa754f8cfa..c464231c745 100644 --- a/integration-tests/load/go.mod +++ b/integration-tests/load/go.mod @@ -372,7 +372,7 @@ require ( github.com/smartcontractkit/chain-selectors v1.0.10 // indirect github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761c982f // indirect github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827 // indirect - github.com/smartcontractkit/chainlink-solana v1.1.0 // indirect + github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564 // indirect github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799 // indirect github.com/smartcontractkit/chainlink-testing-framework/grafana v0.0.0-20240405215812-5a72bc9af239 // indirect github.com/smartcontractkit/havoc/k8schaos v0.0.0-20240409145249-e78d20847e37 // indirect diff --git a/integration-tests/load/go.sum b/integration-tests/load/go.sum index 698623c50f1..d1d6f3a4d52 100644 --- a/integration-tests/load/go.sum +++ b/integration-tests/load/go.sum @@ -1478,8 +1478,8 @@ github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761 github.com/smartcontractkit/chainlink-data-streams v0.0.0-20240801131703-fd75761c982f/go.mod h1:V/86loaFSH0dqqUEHqyXVbyNqDRSjvcf9BRomWFTljU= github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827 h1:BCHu4pNP6arrcHLEWx61XjLaonOd2coQNyL0NTUcaMc= github.com/smartcontractkit/chainlink-feeds v0.0.0-20240710170203-5b41615da827/go.mod h1:OPX+wC2TWQsyLNpR7daMt2vMpmsNcoBxbZyGTHr6tiA= -github.com/smartcontractkit/chainlink-solana v1.1.0 h1:+xBeVqx2x0Sx3CBbF8RLSblczsxJDYTkta8h7i8+23I= -github.com/smartcontractkit/chainlink-solana v1.1.0/go.mod h1:Ml88TJTwZCj6yHDkAEN/EhxVutzSlk+kDZgfibRIqF0= +github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564 h1:8ZzsGNhqYxmQ/QMO1fuXO7u9Vpl9YUvPJK+td/ZaBJA= +github.com/smartcontractkit/chainlink-solana v1.1.1-0.20240806154405-8e5684f98564/go.mod h1:Ml88TJTwZCj6yHDkAEN/EhxVutzSlk+kDZgfibRIqF0= github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799 h1:HyLTySm7BR+oNfZqDTkVJ25wnmcTtxBBD31UkFL+kEM= github.com/smartcontractkit/chainlink-starknet/relayer v0.0.1-beta-test.0.20240709043547-03612098f799/go.mod h1:UVFRacRkP7O7TQAzFmR52v5mUlxf+G1ovMlCQAB/cHU= github.com/smartcontractkit/chainlink-testing-framework v1.34.2 h1:YL3ft7KJB7SAopdmJeyeR4/kv0j4jOdagNihXq8OZ38= From 5a1dd1f74007f365bc52e323e5d6a3a638e6d7ed Mon Sep 17 00:00:00 2001 From: Akshay Aggarwal Date: Wed, 7 Aug 2024 15:49:50 +0100 Subject: [PATCH 4/6] Update log trigger default values (#14051) --- .../ocr2keeper/evmregistry/v21/logprovider/factory.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/services/ocr2/plugins/ocr2keeper/evmregistry/v21/logprovider/factory.go b/core/services/ocr2/plugins/ocr2keeper/evmregistry/v21/logprovider/factory.go index 7ec65ff4740..25cc5e939ba 100644 --- a/core/services/ocr2/plugins/ocr2keeper/evmregistry/v21/logprovider/factory.go +++ b/core/services/ocr2/plugins/ocr2keeper/evmregistry/v21/logprovider/factory.go @@ -74,7 +74,7 @@ func (o *LogTriggersOptions) Defaults(finalityDepth int64) { func (o *LogTriggersOptions) defaultBlockRate() uint32 { switch o.chainID.Int64() { - case 42161, 421613, 421614: // Arbitrum + case 42161, 421613, 421614: // Arbitrum, Arb Goerli, Arb Sepolia return 2 default: return 1 @@ -83,10 +83,10 @@ func (o *LogTriggersOptions) defaultBlockRate() uint32 { func (o *LogTriggersOptions) defaultLogLimit() uint32 { switch o.chainID.Int64() { - case 1, 4, 5, 42, 11155111: // Eth + case 1, 4, 5, 42, 11155111: // Eth, Rinkeby, Goerli, Kovan, Sepolia return 20 - case 10, 420, 56, 97, 137, 80001, 43113, 43114, 8453, 84531: // Optimism, BSC, Polygon, Avax, Base - return 5 + case 10, 420, 11155420, 56, 97, 137, 80001, 80002, 43114, 43113, 8453, 84531, 84532: // Optimism, OP Goerli, OP Sepolia, BSC, BSC Test, Polygon, Mumbai, Amoy, Avax, Avax Fuji, Base, Base Goerli, Base Sepolia + return 4 default: return 1 } From e500c1a471c4e9bb66a89ff27a763a83824f767c Mon Sep 17 00:00:00 2001 From: frank zhu Date: Wed, 7 Aug 2024 10:02:24 -0500 Subject: [PATCH 5/6] chore: update dependabot config gomod (#14063) --- .github/dependabot.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 19e008c8ce4..cea4f07b90d 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -4,7 +4,7 @@ updates: directory: "/" schedule: interval: monthly - open-pull-requests-limit: 10 + open-pull-requests-limit: 0 ignore: # Old versions are pinned for libocr. - dependency-name: github.com/libp2p/go-libp2p-core From 215277f9e041d18dc5686c697e6959d5edaaf346 Mon Sep 17 00:00:00 2001 From: FelixFan1992 Date: Wed, 7 Aug 2024 11:21:31 -0400 Subject: [PATCH 6/6] auto-10161: replicate v2_3 to v2_3_zksync (#14035) * auto-10161: replicate v2_3 to v2_3_zksync * update * small fixes * add an zksync automation forwarder * fix linter * update * update * lint --- contracts/.changeset/loud-lobsters-guess.md | 5 + contracts/.solhintignore | 1 + .../native_solc_compile_all_automation | 2 +- .../automation/ZKSyncAutomationForwarder.sol | 92 ++ .../v0.8/automation/test/{v2_3 => }/WETH9.sol | 0 .../v0.8/automation/test/v2_3/BaseTest.t.sol | 4 +- .../ZKSyncAutomationRegistry2_3.sol | 391 ++++++ .../ZKSyncAutomationRegistryBase2_3.sol | 1216 +++++++++++++++++ .../ZKSyncAutomationRegistryLogicA2_3.sol | 283 ++++ .../ZKSyncAutomationRegistryLogicB2_3.sol | 449 ++++++ .../ZKSyncAutomationRegistryLogicC2_3.sol | 638 +++++++++ .../automation/AutomationRegistry2_3.test.ts | 1 - contracts/test/v0.8/automation/helpers.ts | 4 +- 13 files changed, 3080 insertions(+), 6 deletions(-) create mode 100644 contracts/.changeset/loud-lobsters-guess.md create mode 100644 contracts/src/v0.8/automation/ZKSyncAutomationForwarder.sol rename contracts/src/v0.8/automation/test/{v2_3 => }/WETH9.sol (100%) create mode 100644 contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistry2_3.sol create mode 100644 contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryBase2_3.sol create mode 100644 contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicA2_3.sol create mode 100644 contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicB2_3.sol create mode 100644 contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicC2_3.sol diff --git a/contracts/.changeset/loud-lobsters-guess.md b/contracts/.changeset/loud-lobsters-guess.md new file mode 100644 index 00000000000..e470267e4e4 --- /dev/null +++ b/contracts/.changeset/loud-lobsters-guess.md @@ -0,0 +1,5 @@ +--- +'@chainlink/contracts': patch +--- + +auto: create a replication from v2_3 to v2_3_zksync diff --git a/contracts/.solhintignore b/contracts/.solhintignore index bad1935442b..55d195c3059 100644 --- a/contracts/.solhintignore +++ b/contracts/.solhintignore @@ -18,6 +18,7 @@ ./src/v0.8/automation/libraries/internal/Cron.sol ./src/v0.8/automation/AutomationForwarder.sol ./src/v0.8/automation/AutomationForwarderLogic.sol +./src/v0.8/automation/ZKSyncAutomationForwarder.sol ./src/v0.8/automation/interfaces/v2_2/IAutomationRegistryMaster.sol ./src/v0.8/automation/interfaces/v2_3/IAutomationRegistryMaster2_3.sol diff --git a/contracts/scripts/native_solc_compile_all_automation b/contracts/scripts/native_solc_compile_all_automation index f144e4f7dc8..29326a15c05 100755 --- a/contracts/scripts/native_solc_compile_all_automation +++ b/contracts/scripts/native_solc_compile_all_automation @@ -108,4 +108,4 @@ compileContract automation/v2_3/AutomationUtils2_3.sol compileContract automation/interfaces/v2_3/IAutomationRegistryMaster2_3.sol compileContract automation/testhelpers/MockETHUSDAggregator.sol -compileContract automation/test/v2_3/WETH9.sol +compileContract automation/test/WETH9.sol diff --git a/contracts/src/v0.8/automation/ZKSyncAutomationForwarder.sol b/contracts/src/v0.8/automation/ZKSyncAutomationForwarder.sol new file mode 100644 index 00000000000..cfbff1365e1 --- /dev/null +++ b/contracts/src/v0.8/automation/ZKSyncAutomationForwarder.sol @@ -0,0 +1,92 @@ +// SPDX-License-Identifier: BUSL-1.1 +pragma solidity ^0.8.16; + +import {IAutomationRegistryConsumer} from "./interfaces/IAutomationRegistryConsumer.sol"; + +uint256 constant PERFORM_GAS_CUSHION = 5_000; + +/** + * @title AutomationForwarder is a relayer that sits between the registry and the customer's target contract + * @dev The purpose of the forwarder is to give customers a consistent address to authorize against, + * which stays consistent between migrations. The Forwarder also exposes the registry address, so that users who + * want to programmatically interact with the registry (ie top up funds) can do so. + */ +contract ZKSyncAutomationForwarder { + /// @notice the user's target contract address + address private immutable i_target; + + /// @notice the shared logic address + address private immutable i_logic; + + IAutomationRegistryConsumer private s_registry; + + constructor(address target, address registry, address logic) { + s_registry = IAutomationRegistryConsumer(registry); + i_target = target; + i_logic = logic; + } + + /** + * @notice forward is called by the registry and forwards the call to the target + * @param gasAmount is the amount of gas to use in the call + * @param data is the 4 bytes function selector + arbitrary function data + * @return success indicating whether the target call succeeded or failed + */ + function forward(uint256 gasAmount, bytes memory data) external returns (bool success, uint256 gasUsed) { + if (msg.sender != address(s_registry)) revert(); + address target = i_target; + gasUsed = gasleft(); + assembly { + let g := gas() + // Compute g -= PERFORM_GAS_CUSHION and check for underflow + if lt(g, PERFORM_GAS_CUSHION) { + revert(0, 0) + } + g := sub(g, PERFORM_GAS_CUSHION) + // if g - g//64 <= gasAmount, revert + // (we subtract g//64 because of EIP-150) + if iszero(gt(sub(g, div(g, 64)), gasAmount)) { + revert(0, 0) + } + // solidity calls check that a contract actually exists at the destination, so we do the same + if iszero(extcodesize(target)) { + revert(0, 0) + } + // call with exact gas + success := call(gasAmount, target, 0, add(data, 0x20), mload(data), 0, 0) + } + gasUsed = gasUsed - gasleft(); + return (success, gasUsed); + } + + function getTarget() external view returns (address) { + return i_target; + } + + fallback() external { + // copy to memory for assembly access + address logic = i_logic; + // copied directly from OZ's Proxy contract + assembly { + // Copy msg.data. We take full control of memory in this inline assembly + // block because it will not return to Solidity code. We overwrite the + // Solidity scratch pad at memory position 0. + calldatacopy(0, 0, calldatasize()) + + // out and outsize are 0 because we don't know the size yet. + let result := delegatecall(gas(), logic, 0, calldatasize(), 0, 0) + + // Copy the returned data. + returndatacopy(0, 0, returndatasize()) + + switch result + // delegatecall returns 0 on error. + case 0 { + revert(0, returndatasize()) + } + default { + return(0, returndatasize()) + } + } + } +} diff --git a/contracts/src/v0.8/automation/test/v2_3/WETH9.sol b/contracts/src/v0.8/automation/test/WETH9.sol similarity index 100% rename from contracts/src/v0.8/automation/test/v2_3/WETH9.sol rename to contracts/src/v0.8/automation/test/WETH9.sol diff --git a/contracts/src/v0.8/automation/test/v2_3/BaseTest.t.sol b/contracts/src/v0.8/automation/test/v2_3/BaseTest.t.sol index 9016f52c55d..9e46e7bb40d 100644 --- a/contracts/src/v0.8/automation/test/v2_3/BaseTest.t.sol +++ b/contracts/src/v0.8/automation/test/v2_3/BaseTest.t.sol @@ -20,14 +20,14 @@ import {ChainModuleBase} from "../../chains/ChainModuleBase.sol"; import {IERC20Metadata as IERC20} from "../../../vendor/openzeppelin-solidity/v4.8.3/contracts/token/ERC20/extensions/IERC20Metadata.sol"; import {MockUpkeep} from "../../mocks/MockUpkeep.sol"; import {IWrappedNative} from "../../interfaces/v2_3/IWrappedNative.sol"; -import {WETH9} from "./WETH9.sol"; +import {WETH9} from "../WETH9.sol"; /** * @title BaseTest provides basic test setup procedures and dependencies for use by other * unit tests */ contract BaseTest is Test { - // test state (not exposed to derrived tests) + // test state (not exposed to derived tests) uint256 private nonce; // constants diff --git a/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistry2_3.sol b/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistry2_3.sol new file mode 100644 index 00000000000..027fe59aca7 --- /dev/null +++ b/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistry2_3.sol @@ -0,0 +1,391 @@ +// SPDX-License-Identifier: BUSL-1.1 +pragma solidity 0.8.19; + +import {EnumerableSet} from "../../vendor/openzeppelin-solidity/v4.7.3/contracts/utils/structs/EnumerableSet.sol"; +import {Address} from "../../vendor/openzeppelin-solidity/v4.7.3/contracts/utils/Address.sol"; +import {ZKSyncAutomationRegistryBase2_3} from "./ZKSyncAutomationRegistryBase2_3.sol"; +import {ZKSyncAutomationRegistryLogicA2_3} from "./ZKSyncAutomationRegistryLogicA2_3.sol"; +import {ZKSyncAutomationRegistryLogicC2_3} from "./ZKSyncAutomationRegistryLogicC2_3.sol"; +import {Chainable} from "../Chainable.sol"; +import {OCR2Abstract} from "../../shared/ocr2/OCR2Abstract.sol"; +import {IERC20Metadata as IERC20} from "../../vendor/openzeppelin-solidity/v4.8.3/contracts/token/ERC20/extensions/IERC20Metadata.sol"; + +/** + * @notice Registry for adding work for Chainlink nodes to perform on client + * contracts. Clients must support the AutomationCompatibleInterface interface. + */ +contract ZKSyncAutomationRegistry2_3 is ZKSyncAutomationRegistryBase2_3, OCR2Abstract, Chainable { + using Address for address; + using EnumerableSet for EnumerableSet.UintSet; + using EnumerableSet for EnumerableSet.AddressSet; + + /** + * @notice versions: + * AutomationRegistry 2.3.0: supports native and ERC20 billing + * changes flat fee to USD-denominated + * adds support for custom billing overrides + * AutomationRegistry 2.2.0: moves chain-specific integration code into a separate module + * KeeperRegistry 2.1.0: introduces support for log triggers + * removes the need for "wrapped perform data" + * KeeperRegistry 2.0.2: pass revert bytes as performData when target contract reverts + * fixes issue with arbitrum block number + * does an early return in case of stale report instead of revert + * KeeperRegistry 2.0.1: implements workaround for buggy migrate function in 1.X + * KeeperRegistry 2.0.0: implement OCR interface + * KeeperRegistry 1.3.0: split contract into Proxy and Logic + * account for Arbitrum and Optimism L1 gas fee + * allow users to configure upkeeps + * KeeperRegistry 1.2.0: allow funding within performUpkeep + * allow configurable registry maxPerformGas + * add function to let admin change upkeep gas limit + * add minUpkeepSpend requirement + * upgrade to solidity v0.8 + * KeeperRegistry 1.1.0: added flatFeeMicroLink + * KeeperRegistry 1.0.0: initial release + */ + string public constant override typeAndVersion = "AutomationRegistry 2.3.0"; + + /** + * @param logicA the address of the first logic contract + * @dev we cast the contract to logicC in order to call logicC functions (via fallback) + */ + constructor( + ZKSyncAutomationRegistryLogicA2_3 logicA + ) + ZKSyncAutomationRegistryBase2_3( + ZKSyncAutomationRegistryLogicC2_3(address(logicA)).getLinkAddress(), + ZKSyncAutomationRegistryLogicC2_3(address(logicA)).getLinkUSDFeedAddress(), + ZKSyncAutomationRegistryLogicC2_3(address(logicA)).getNativeUSDFeedAddress(), + ZKSyncAutomationRegistryLogicC2_3(address(logicA)).getFastGasFeedAddress(), + ZKSyncAutomationRegistryLogicC2_3(address(logicA)).getAutomationForwarderLogic(), + ZKSyncAutomationRegistryLogicC2_3(address(logicA)).getAllowedReadOnlyAddress(), + ZKSyncAutomationRegistryLogicC2_3(address(logicA)).getPayoutMode(), + ZKSyncAutomationRegistryLogicC2_3(address(logicA)).getWrappedNativeTokenAddress() + ) + Chainable(address(logicA)) + {} + + /** + * @notice holds the variables used in the transmit function, necessary to avoid stack too deep errors + */ + struct TransmitVars { + uint16 numUpkeepsPassedChecks; + uint96 totalReimbursement; + uint96 totalPremium; + uint256 totalCalldataWeight; + } + + // ================================================================ + // | HOT PATH ACTIONS | + // ================================================================ + + /** + * @inheritdoc OCR2Abstract + */ + function transmit( + bytes32[3] calldata reportContext, + bytes calldata rawReport, + bytes32[] calldata rs, + bytes32[] calldata ss, + bytes32 rawVs + ) external override { + uint256 gasOverhead = gasleft(); + // use this msg.data length check to ensure no extra data is included in the call + // 4 is first 4 bytes of the keccak-256 hash of the function signature. ss.length == rs.length so use one of them + // 4 + (32 * 3) + (rawReport.length + 32 + 32) + (32 * rs.length + 32 + 32) + (32 * ss.length + 32 + 32) + 32 + uint256 requiredLength = 324 + rawReport.length + 64 * rs.length; + if (msg.data.length != requiredLength) revert InvalidDataLength(); + HotVars memory hotVars = s_hotVars; + + if (hotVars.paused) revert RegistryPaused(); + if (!s_transmitters[msg.sender].active) revert OnlyActiveTransmitters(); + + // Verify signatures + if (s_latestConfigDigest != reportContext[0]) revert ConfigDigestMismatch(); + if (rs.length != hotVars.f + 1 || rs.length != ss.length) revert IncorrectNumberOfSignatures(); + _verifyReportSignature(reportContext, rawReport, rs, ss, rawVs); + + Report memory report = _decodeReport(rawReport); + + uint40 epochAndRound = uint40(uint256(reportContext[1])); + uint32 epoch = uint32(epochAndRound >> 8); + + _handleReport(hotVars, report, gasOverhead); + + if (epoch > hotVars.latestEpoch) { + s_hotVars.latestEpoch = epoch; + } + } + + /** + * @notice handles the report by performing the upkeeps and updating the state + * @param hotVars the hot variables of the registry + * @param report the report to be handled (already verified and decoded) + * @param gasOverhead the running tally of gas overhead to be split across the upkeeps + * @dev had to split this function from transmit() to avoid stack too deep errors + * @dev all other internal / private functions are generally defined in the Base contract + * we leave this here because it is essentially a continuation of the transmit() function, + */ + function _handleReport(HotVars memory hotVars, Report memory report, uint256 gasOverhead) private { + UpkeepTransmitInfo[] memory upkeepTransmitInfo = new UpkeepTransmitInfo[](report.upkeepIds.length); + TransmitVars memory transmitVars = TransmitVars({ + numUpkeepsPassedChecks: 0, + totalCalldataWeight: 0, + totalReimbursement: 0, + totalPremium: 0 + }); + + uint256 blocknumber = hotVars.chainModule.blockNumber(); + uint256 l1Fee = hotVars.chainModule.getCurrentL1Fee(); + + for (uint256 i = 0; i < report.upkeepIds.length; i++) { + upkeepTransmitInfo[i].upkeep = s_upkeep[report.upkeepIds[i]]; + upkeepTransmitInfo[i].triggerType = _getTriggerType(report.upkeepIds[i]); + + (upkeepTransmitInfo[i].earlyChecksPassed, upkeepTransmitInfo[i].dedupID) = _prePerformChecks( + report.upkeepIds[i], + blocknumber, + report.triggers[i], + upkeepTransmitInfo[i], + hotVars + ); + + if (upkeepTransmitInfo[i].earlyChecksPassed) { + transmitVars.numUpkeepsPassedChecks += 1; + } else { + continue; + } + + // Actually perform the target upkeep + (upkeepTransmitInfo[i].performSuccess, upkeepTransmitInfo[i].gasUsed) = _performUpkeep( + upkeepTransmitInfo[i].upkeep.forwarder, + report.gasLimits[i], + report.performDatas[i] + ); + + // To split L1 fee across the upkeeps, assign a weight to this upkeep based on the length + // of the perform data and calldata overhead + upkeepTransmitInfo[i].calldataWeight = + report.performDatas[i].length + + TRANSMIT_CALLDATA_FIXED_BYTES_OVERHEAD + + (TRANSMIT_CALLDATA_PER_SIGNER_BYTES_OVERHEAD * (hotVars.f + 1)); + transmitVars.totalCalldataWeight += upkeepTransmitInfo[i].calldataWeight; + + // Deduct the gasUsed by upkeep from the overhead tally - upkeeps pay for their own gas individually + gasOverhead -= upkeepTransmitInfo[i].gasUsed; + + // Store last perform block number / deduping key for upkeep + _updateTriggerMarker(report.upkeepIds[i], blocknumber, upkeepTransmitInfo[i]); + } + // No upkeeps to be performed in this report + if (transmitVars.numUpkeepsPassedChecks == 0) { + return; + } + + // This is the overall gas overhead that will be split across performed upkeeps + // Take upper bound of 16 gas per callData bytes + gasOverhead = (gasOverhead - gasleft()) + (16 * msg.data.length) + ACCOUNTING_FIXED_GAS_OVERHEAD; + gasOverhead = gasOverhead / transmitVars.numUpkeepsPassedChecks + ACCOUNTING_PER_UPKEEP_GAS_OVERHEAD; + + { + BillingTokenPaymentParams memory billingTokenParams; + uint256 nativeUSD = _getNativeUSD(hotVars); + for (uint256 i = 0; i < report.upkeepIds.length; i++) { + if (upkeepTransmitInfo[i].earlyChecksPassed) { + if (i == 0 || upkeepTransmitInfo[i].upkeep.billingToken != upkeepTransmitInfo[i - 1].upkeep.billingToken) { + billingTokenParams = _getBillingTokenPaymentParams(hotVars, upkeepTransmitInfo[i].upkeep.billingToken); + } + PaymentReceipt memory receipt = _handlePayment( + hotVars, + PaymentParams({ + gasLimit: upkeepTransmitInfo[i].gasUsed, + gasOverhead: gasOverhead, + l1CostWei: (l1Fee * upkeepTransmitInfo[i].calldataWeight) / transmitVars.totalCalldataWeight, + fastGasWei: report.fastGasWei, + linkUSD: report.linkUSD, + nativeUSD: nativeUSD, + billingToken: upkeepTransmitInfo[i].upkeep.billingToken, + billingTokenParams: billingTokenParams, + isTransaction: true + }), + report.upkeepIds[i], + upkeepTransmitInfo[i].upkeep + ); + transmitVars.totalPremium += receipt.premiumInJuels; + transmitVars.totalReimbursement += receipt.gasReimbursementInJuels; + + emit UpkeepPerformed( + report.upkeepIds[i], + upkeepTransmitInfo[i].performSuccess, + receipt.gasChargeInBillingToken + receipt.premiumInBillingToken, + upkeepTransmitInfo[i].gasUsed, + gasOverhead, + report.triggers[i] + ); + } + } + } + // record payments to NOPs, all in LINK + s_transmitters[msg.sender].balance += transmitVars.totalReimbursement; + s_hotVars.totalPremium += transmitVars.totalPremium; + s_reserveAmounts[IERC20(address(i_link))] += transmitVars.totalReimbursement + transmitVars.totalPremium; + } + + // ================================================================ + // | OCR2ABSTRACT | + // ================================================================ + + /** + * @inheritdoc OCR2Abstract + * @dev prefer the type-safe version of setConfig (below) whenever possible. The OnchainConfig could differ between registry versions + * @dev this function takes up precious space on the root contract, but must be implemented to conform to the OCR2Abstract interface + */ + function setConfig( + address[] memory signers, + address[] memory transmitters, + uint8 f, + bytes memory onchainConfigBytes, + uint64 offchainConfigVersion, + bytes memory offchainConfig + ) external override { + (OnchainConfig memory config, IERC20[] memory billingTokens, BillingConfig[] memory billingConfigs) = abi.decode( + onchainConfigBytes, + (OnchainConfig, IERC20[], BillingConfig[]) + ); + + setConfigTypeSafe( + signers, + transmitters, + f, + config, + offchainConfigVersion, + offchainConfig, + billingTokens, + billingConfigs + ); + } + + /** + * @notice sets the configuration for the registry + * @param signers the list of permitted signers + * @param transmitters the list of permitted transmitters + * @param f the maximum tolerance for faulty nodes + * @param onchainConfig configuration values that are used on-chain + * @param offchainConfigVersion the version of the offchainConfig + * @param offchainConfig configuration values that are used off-chain + * @param billingTokens the list of valid billing tokens + * @param billingConfigs the configurations for each billing token + */ + function setConfigTypeSafe( + address[] memory signers, + address[] memory transmitters, + uint8 f, + OnchainConfig memory onchainConfig, + uint64 offchainConfigVersion, + bytes memory offchainConfig, + IERC20[] memory billingTokens, + BillingConfig[] memory billingConfigs + ) public onlyOwner { + if (signers.length > MAX_NUM_ORACLES) revert TooManyOracles(); + if (f == 0) revert IncorrectNumberOfFaultyOracles(); + if (signers.length != transmitters.length || signers.length <= 3 * f) revert IncorrectNumberOfSigners(); + if (billingTokens.length != billingConfigs.length) revert ParameterLengthError(); + // set billing config for tokens + _setBillingConfig(billingTokens, billingConfigs); + + _updateTransmitters(signers, transmitters); + + s_hotVars = HotVars({ + f: f, + stalenessSeconds: onchainConfig.stalenessSeconds, + gasCeilingMultiplier: onchainConfig.gasCeilingMultiplier, + paused: s_hotVars.paused, + reentrancyGuard: s_hotVars.reentrancyGuard, + totalPremium: s_hotVars.totalPremium, + latestEpoch: 0, // DON restarts epoch + reorgProtectionEnabled: onchainConfig.reorgProtectionEnabled, + chainModule: onchainConfig.chainModule + }); + + uint32 previousConfigBlockNumber = s_storage.latestConfigBlockNumber; + uint32 newLatestConfigBlockNumber = uint32(onchainConfig.chainModule.blockNumber()); + uint32 newConfigCount = s_storage.configCount + 1; + + s_storage = Storage({ + checkGasLimit: onchainConfig.checkGasLimit, + maxPerformGas: onchainConfig.maxPerformGas, + transcoder: onchainConfig.transcoder, + maxCheckDataSize: onchainConfig.maxCheckDataSize, + maxPerformDataSize: onchainConfig.maxPerformDataSize, + maxRevertDataSize: onchainConfig.maxRevertDataSize, + upkeepPrivilegeManager: onchainConfig.upkeepPrivilegeManager, + financeAdmin: onchainConfig.financeAdmin, + nonce: s_storage.nonce, + configCount: newConfigCount, + latestConfigBlockNumber: newLatestConfigBlockNumber + }); + s_fallbackGasPrice = onchainConfig.fallbackGasPrice; + s_fallbackLinkPrice = onchainConfig.fallbackLinkPrice; + s_fallbackNativePrice = onchainConfig.fallbackNativePrice; + + bytes memory onchainConfigBytes = abi.encode(onchainConfig); + + s_latestConfigDigest = _configDigestFromConfigData( + block.chainid, + address(this), + s_storage.configCount, + signers, + transmitters, + f, + onchainConfigBytes, + offchainConfigVersion, + offchainConfig + ); + + for (uint256 idx = s_registrars.length(); idx > 0; idx--) { + s_registrars.remove(s_registrars.at(idx - 1)); + } + + for (uint256 idx = 0; idx < onchainConfig.registrars.length; idx++) { + s_registrars.add(onchainConfig.registrars[idx]); + } + + emit ConfigSet( + previousConfigBlockNumber, + s_latestConfigDigest, + s_storage.configCount, + signers, + transmitters, + f, + onchainConfigBytes, + offchainConfigVersion, + offchainConfig + ); + } + + /** + * @inheritdoc OCR2Abstract + * @dev this function takes up precious space on the root contract, but must be implemented to conform to the OCR2Abstract interface + */ + function latestConfigDetails() + external + view + override + returns (uint32 configCount, uint32 blockNumber, bytes32 configDigest) + { + return (s_storage.configCount, s_storage.latestConfigBlockNumber, s_latestConfigDigest); + } + + /** + * @inheritdoc OCR2Abstract + * @dev this function takes up precious space on the root contract, but must be implemented to conform to the OCR2Abstract interface + */ + function latestConfigDigestAndEpoch() + external + view + override + returns (bool scanLogs, bytes32 configDigest, uint32 epoch) + { + return (false, s_latestConfigDigest, s_hotVars.latestEpoch); + } +} diff --git a/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryBase2_3.sol b/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryBase2_3.sol new file mode 100644 index 00000000000..524ecacc826 --- /dev/null +++ b/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryBase2_3.sol @@ -0,0 +1,1216 @@ +// SPDX-License-Identifier: BUSL-1.1 +pragma solidity 0.8.19; + +import {EnumerableSet} from "../../vendor/openzeppelin-solidity/v4.7.3/contracts/utils/structs/EnumerableSet.sol"; +import {Address} from "../../vendor/openzeppelin-solidity/v4.7.3/contracts/utils/Address.sol"; +import {StreamsLookupCompatibleInterface} from "../interfaces/StreamsLookupCompatibleInterface.sol"; +import {ILogAutomation, Log} from "../interfaces/ILogAutomation.sol"; +import {IAutomationForwarder} from "../interfaces/IAutomationForwarder.sol"; +import {ConfirmedOwner} from "../../shared/access/ConfirmedOwner.sol"; +import {AggregatorV3Interface} from "../../shared/interfaces/AggregatorV3Interface.sol"; +import {LinkTokenInterface} from "../../shared/interfaces/LinkTokenInterface.sol"; +import {KeeperCompatibleInterface} from "../interfaces/KeeperCompatibleInterface.sol"; +import {IChainModule} from "../interfaces/IChainModule.sol"; +import {IERC20Metadata as IERC20} from "../../vendor/openzeppelin-solidity/v4.8.3/contracts/token/ERC20/extensions/IERC20Metadata.sol"; +import {SafeCast} from "../../vendor/openzeppelin-solidity/v4.8.3/contracts/utils/math/SafeCast.sol"; +import {IWrappedNative} from "../interfaces/v2_3/IWrappedNative.sol"; + +/** + * @notice Base Keeper Registry contract, contains shared logic between + * AutomationRegistry and AutomationRegistryLogic + * @dev all errors, events, and internal functions should live here + */ +// solhint-disable-next-line max-states-count +abstract contract ZKSyncAutomationRegistryBase2_3 is ConfirmedOwner { + using Address for address; + using EnumerableSet for EnumerableSet.UintSet; + using EnumerableSet for EnumerableSet.AddressSet; + + address internal constant ZERO_ADDRESS = address(0); + address internal constant IGNORE_ADDRESS = 0xFFfFfFffFFfffFFfFFfFFFFFffFFFffffFfFFFfF; + bytes4 internal constant CHECK_SELECTOR = KeeperCompatibleInterface.checkUpkeep.selector; + bytes4 internal constant PERFORM_SELECTOR = KeeperCompatibleInterface.performUpkeep.selector; + bytes4 internal constant CHECK_CALLBACK_SELECTOR = StreamsLookupCompatibleInterface.checkCallback.selector; + bytes4 internal constant CHECK_LOG_SELECTOR = ILogAutomation.checkLog.selector; + uint256 internal constant PERFORM_GAS_MIN = 2_300; + uint256 internal constant CANCELLATION_DELAY = 50; + uint256 internal constant PERFORM_GAS_CUSHION = 5_000; + uint256 internal constant PPB_BASE = 1_000_000_000; + uint32 internal constant UINT32_MAX = type(uint32).max; + // The first byte of the mask can be 0, because we only ever have 31 oracles + uint256 internal constant ORACLE_MASK = 0x0001010101010101010101010101010101010101010101010101010101010101; + uint8 internal constant UPKEEP_VERSION_BASE = 4; + + // Next block of constants are only used in maxPayment estimation during checkUpkeep simulation + // These values are calibrated using hardhat tests which simulate various cases and verify that + // the variables result in accurate estimation + uint256 internal constant REGISTRY_CONDITIONAL_OVERHEAD = 98_200; // Fixed gas overhead for conditional upkeeps + uint256 internal constant REGISTRY_LOG_OVERHEAD = 122_500; // Fixed gas overhead for log upkeeps + uint256 internal constant REGISTRY_PER_SIGNER_GAS_OVERHEAD = 5_600; // Value scales with f + uint256 internal constant REGISTRY_PER_PERFORM_BYTE_GAS_OVERHEAD = 24; // Per perform data byte overhead + + // The overhead (in bytes) in addition to perform data for upkeep sent in calldata + // This includes overhead for all struct encoding as well as report signatures + // There is a fixed component and a per signer component. This is calculated exactly by doing abi encoding + uint256 internal constant TRANSMIT_CALLDATA_FIXED_BYTES_OVERHEAD = 932; + uint256 internal constant TRANSMIT_CALLDATA_PER_SIGNER_BYTES_OVERHEAD = 64; + + // Next block of constants are used in actual payment calculation. We calculate the exact gas used within the + // tx itself, but since payment processing itself takes gas, and it needs the overhead as input, we use fixed constants + // to account for gas used in payment processing. These values are calibrated using hardhat tests which simulates various cases and verifies that + // the variables result in accurate estimation + uint256 internal constant ACCOUNTING_FIXED_GAS_OVERHEAD = 51_200; // Fixed overhead per tx + uint256 internal constant ACCOUNTING_PER_UPKEEP_GAS_OVERHEAD = 14_200; // Overhead per upkeep performed in batch + + LinkTokenInterface internal immutable i_link; + AggregatorV3Interface internal immutable i_linkUSDFeed; + AggregatorV3Interface internal immutable i_nativeUSDFeed; + AggregatorV3Interface internal immutable i_fastGasFeed; + address internal immutable i_automationForwarderLogic; + address internal immutable i_allowedReadOnlyAddress; + IWrappedNative internal immutable i_wrappedNativeToken; + + /** + * @dev - The storage is gas optimised for one and only one function - transmit. All the storage accessed in transmit + * is stored compactly. Rest of the storage layout is not of much concern as transmit is the only hot path + */ + + // Upkeep storage + EnumerableSet.UintSet internal s_upkeepIDs; + mapping(uint256 => Upkeep) internal s_upkeep; // accessed during transmit + mapping(uint256 => address) internal s_upkeepAdmin; + mapping(uint256 => address) internal s_proposedAdmin; + mapping(uint256 => bytes) internal s_checkData; + mapping(bytes32 => bool) internal s_dedupKeys; + // Registry config and state + EnumerableSet.AddressSet internal s_registrars; + mapping(address => Transmitter) internal s_transmitters; + mapping(address => Signer) internal s_signers; + address[] internal s_signersList; // s_signersList contains the signing address of each oracle + address[] internal s_transmittersList; // s_transmittersList contains the transmission address of each oracle + EnumerableSet.AddressSet internal s_deactivatedTransmitters; + mapping(address => address) internal s_transmitterPayees; // s_payees contains the mapping from transmitter to payee. + mapping(address => address) internal s_proposedPayee; // proposed payee for a transmitter + bytes32 internal s_latestConfigDigest; // Read on transmit path in case of signature verification + HotVars internal s_hotVars; // Mixture of config and state, used in transmit + Storage internal s_storage; // Mixture of config and state, not used in transmit + uint256 internal s_fallbackGasPrice; + uint256 internal s_fallbackLinkPrice; + uint256 internal s_fallbackNativePrice; + mapping(address => MigrationPermission) internal s_peerRegistryMigrationPermission; // Permissions for migration to and fro + mapping(uint256 => bytes) internal s_upkeepTriggerConfig; // upkeep triggers + mapping(uint256 => bytes) internal s_upkeepOffchainConfig; // general config set by users for each upkeep + mapping(uint256 => bytes) internal s_upkeepPrivilegeConfig; // general config set by an administrative role for an upkeep + mapping(address => bytes) internal s_adminPrivilegeConfig; // general config set by an administrative role for an admin + // billing + mapping(IERC20 billingToken => uint256 reserveAmount) internal s_reserveAmounts; // unspent user deposits + unwithdrawn NOP payments + mapping(IERC20 billingToken => BillingConfig billingConfig) internal s_billingConfigs; // billing configurations for different tokens + mapping(uint256 upkeepID => BillingOverrides billingOverrides) internal s_billingOverrides; // billing overrides for specific upkeeps + IERC20[] internal s_billingTokens; // list of billing tokens + PayoutMode internal s_payoutMode; + + error ArrayHasNoEntries(); + error CannotCancel(); + error CheckDataExceedsLimit(); + error ConfigDigestMismatch(); + error DuplicateEntry(); + error DuplicateSigners(); + error GasLimitCanOnlyIncrease(); + error GasLimitOutsideRange(); + error IncorrectNumberOfFaultyOracles(); + error IncorrectNumberOfSignatures(); + error IncorrectNumberOfSigners(); + error IndexOutOfRange(); + error InsufficientBalance(uint256 available, uint256 requested); + error InsufficientLinkLiquidity(); + error InvalidDataLength(); + error InvalidFeed(); + error InvalidTrigger(); + error InvalidPayee(); + error InvalidRecipient(); + error InvalidReport(); + error InvalidSigner(); + error InvalidToken(); + error InvalidTransmitter(); + error InvalidTriggerType(); + error MigrationNotPermitted(); + error MustSettleOffchain(); + error MustSettleOnchain(); + error NotAContract(); + error OnlyActiveSigners(); + error OnlyActiveTransmitters(); + error OnlyCallableByAdmin(); + error OnlyCallableByLINKToken(); + error OnlyCallableByOwnerOrAdmin(); + error OnlyCallableByOwnerOrRegistrar(); + error OnlyCallableByPayee(); + error OnlyCallableByProposedAdmin(); + error OnlyCallableByProposedPayee(); + error OnlyCallableByUpkeepPrivilegeManager(); + error OnlyFinanceAdmin(); + error OnlyPausedUpkeep(); + error OnlySimulatedBackend(); + error OnlyUnpausedUpkeep(); + error ParameterLengthError(); + error ReentrantCall(); + error RegistryPaused(); + error RepeatedSigner(); + error RepeatedTransmitter(); + error TargetCheckReverted(bytes reason); + error TooManyOracles(); + error TranscoderNotSet(); + error TransferFailed(); + error UpkeepAlreadyExists(); + error UpkeepCancelled(); + error UpkeepNotCanceled(); + error UpkeepNotNeeded(); + error ValueNotChanged(); + error ZeroAddressNotAllowed(); + + enum MigrationPermission { + NONE, + OUTGOING, + INCOMING, + BIDIRECTIONAL + } + + enum Trigger { + CONDITION, + LOG + } + + enum UpkeepFailureReason { + NONE, + UPKEEP_CANCELLED, + UPKEEP_PAUSED, + TARGET_CHECK_REVERTED, + UPKEEP_NOT_NEEDED, + PERFORM_DATA_EXCEEDS_LIMIT, + INSUFFICIENT_BALANCE, + CALLBACK_REVERTED, + REVERT_DATA_EXCEEDS_LIMIT, + REGISTRY_PAUSED + } + + enum PayoutMode { + ON_CHAIN, + OFF_CHAIN + } + + /** + * @notice OnchainConfig of the registry + * @dev used only in setConfig() + * @member checkGasLimit gas limit when checking for upkeep + * @member stalenessSeconds number of seconds that is allowed for feed data to + * be stale before switching to the fallback pricing + * @member gasCeilingMultiplier multiplier to apply to the fast gas feed price + * when calculating the payment ceiling for keepers + * @member maxPerformGas max performGas allowed for an upkeep on this registry + * @member maxCheckDataSize max length of checkData bytes + * @member maxPerformDataSize max length of performData bytes + * @member maxRevertDataSize max length of revertData bytes + * @member fallbackGasPrice gas price used if the gas price feed is stale + * @member fallbackLinkPrice LINK price used if the LINK price feed is stale + * @member transcoder address of the transcoder contract + * @member registrars addresses of the registrar contracts + * @member upkeepPrivilegeManager address which can set privilege for upkeeps + * @member reorgProtectionEnabled if this registry enables re-org protection checks + * @member chainModule the chain specific module + */ + struct OnchainConfig { + uint32 checkGasLimit; + uint32 maxPerformGas; + uint32 maxCheckDataSize; + address transcoder; + // 1 word full + bool reorgProtectionEnabled; + uint24 stalenessSeconds; + uint32 maxPerformDataSize; + uint32 maxRevertDataSize; + address upkeepPrivilegeManager; + // 2 words full + uint16 gasCeilingMultiplier; + address financeAdmin; + // 3 words + uint256 fallbackGasPrice; + uint256 fallbackLinkPrice; + uint256 fallbackNativePrice; + address[] registrars; + IChainModule chainModule; + } + + /** + * @notice relevant state of an upkeep which is used in transmit function + * @member paused if this upkeep has been paused + * @member overridesEnabled if this upkeep has overrides enabled + * @member performGas the gas limit of upkeep execution + * @member maxValidBlocknumber until which block this upkeep is valid + * @member forwarder the forwarder contract to use for this upkeep + * @member amountSpent the amount this upkeep has spent, in the upkeep's billing token + * @member balance the balance of this upkeep + * @member lastPerformedBlockNumber the last block number when this upkeep was performed + */ + struct Upkeep { + bool paused; + bool overridesEnabled; + uint32 performGas; + uint32 maxValidBlocknumber; + IAutomationForwarder forwarder; + // 2 bytes left in 1st EVM word - read in transmit path + uint128 amountSpent; + uint96 balance; + uint32 lastPerformedBlockNumber; + // 0 bytes left in 2nd EVM word - written in transmit path + IERC20 billingToken; + // 12 bytes left in 3rd EVM word - read in transmit path + } + + /// @dev Config + State storage struct which is on hot transmit path + struct HotVars { + uint96 totalPremium; // ─────────╮ total historical payment to oracles for premium + uint32 latestEpoch; // │ latest epoch for which a report was transmitted + uint24 stalenessSeconds; // │ Staleness tolerance for feeds + uint16 gasCeilingMultiplier; // │ multiplier on top of fast gas feed for upper bound + uint8 f; // │ maximum number of faulty oracles + bool paused; // │ pause switch for all upkeeps in the registry + bool reentrancyGuard; // | guard against reentrancy + bool reorgProtectionEnabled; // ─╯ if this registry should enable the re-org protection mechanism + IChainModule chainModule; // the interface of chain specific module + } + + /// @dev Config + State storage struct which is not on hot transmit path + struct Storage { + address transcoder; // Address of transcoder contract used in migrations + uint32 checkGasLimit; // Gas limit allowed in checkUpkeep + uint32 maxPerformGas; // Max gas an upkeep can use on this registry + uint32 nonce; // Nonce for each upkeep created + // 1 EVM word full + address upkeepPrivilegeManager; // address which can set privilege for upkeeps + uint32 configCount; // incremented each time a new config is posted, The count is incorporated into the config digest to prevent replay attacks. + uint32 latestConfigBlockNumber; // makes it easier for offchain systems to extract config from logs + uint32 maxCheckDataSize; // max length of checkData bytes + // 2 EVM word full + address financeAdmin; // address which can withdraw funds from the contract + uint32 maxPerformDataSize; // max length of performData bytes + uint32 maxRevertDataSize; // max length of revertData bytes + // 4 bytes left in 3rd EVM word + } + + /// @dev Report transmitted by OCR to transmit function + struct Report { + uint256 fastGasWei; + uint256 linkUSD; + uint256[] upkeepIds; + uint256[] gasLimits; + bytes[] triggers; + bytes[] performDatas; + } + + /** + * @dev This struct is used to maintain run time information about an upkeep in transmit function + * @member upkeep the upkeep struct + * @member earlyChecksPassed whether the upkeep passed early checks before perform + * @member performSuccess whether the perform was successful + * @member triggerType the type of trigger + * @member gasUsed gasUsed by this upkeep in perform + * @member calldataWeight weight assigned to this upkeep for its contribution to calldata. It is used to split L1 fee + * @member dedupID unique ID used to dedup an upkeep/trigger combo + */ + struct UpkeepTransmitInfo { + Upkeep upkeep; + bool earlyChecksPassed; + bool performSuccess; + Trigger triggerType; + uint256 gasUsed; + uint256 calldataWeight; + bytes32 dedupID; + } + + /** + * @notice holds information about a transmiter / node in the DON + * @member active can this transmitter submit reports + * @member index of oracle in s_signersList/s_transmittersList + * @member balance a node's balance in LINK + * @member lastCollected the total balance at which the node last withdrew + * @dev uint96 is safe for balance / last collected because transmitters are only ever paid in LINK + */ + struct Transmitter { + bool active; + uint8 index; + uint96 balance; + uint96 lastCollected; + } + + struct TransmitterPayeeInfo { + address transmitterAddress; + address payeeAddress; + } + + struct Signer { + bool active; + // Index of oracle in s_signersList/s_transmittersList + uint8 index; + } + + /** + * @notice the trigger structure conditional trigger type + */ + struct ConditionalTrigger { + uint32 blockNum; + bytes32 blockHash; + } + + /** + * @notice the trigger structure of log upkeeps + * @dev NOTE that blockNum / blockHash describe the block used for the callback, + * not necessarily the block number that the log was emitted in!!!! + */ + struct LogTrigger { + bytes32 logBlockHash; + bytes32 txHash; + uint32 logIndex; + uint32 blockNum; + bytes32 blockHash; + } + + /** + * @notice the billing config of a token + * @dev this is a storage struct + */ + // solhint-disable-next-line gas-struct-packing + struct BillingConfig { + uint32 gasFeePPB; + uint24 flatFeeMilliCents; // min fee is $0.00001, max fee is $167 + AggregatorV3Interface priceFeed; + uint8 decimals; + // 1st word, read in calculating BillingTokenPaymentParams + uint256 fallbackPrice; + // 2nd word only read if stale + uint96 minSpend; + // 3rd word only read during cancellation + } + + /** + * @notice override-able billing params of a billing token + */ + struct BillingOverrides { + uint32 gasFeePPB; + uint24 flatFeeMilliCents; + } + + /** + * @notice pricing params for a billing token + * @dev this is a memory-only struct, so struct packing is less important + */ + struct BillingTokenPaymentParams { + uint8 decimals; + uint32 gasFeePPB; + uint24 flatFeeMilliCents; + uint256 priceUSD; + } + + /** + * @notice struct containing price & payment information used in calculating payment amount + * @member gasLimit the amount of gas used + * @member gasOverhead the amount of gas overhead + * @member l1CostWei the amount to be charged for L1 fee in wei + * @member fastGasWei the fast gas price + * @member linkUSD the exchange ratio between LINK and USD + * @member nativeUSD the exchange ratio between the chain's native token and USD + * @member billingToken the billing token + * @member billingTokenParams the payment params specific to a particular payment token + * @member isTransaction is this an eth_call or a transaction + */ + struct PaymentParams { + uint256 gasLimit; + uint256 gasOverhead; + uint256 l1CostWei; + uint256 fastGasWei; + uint256 linkUSD; + uint256 nativeUSD; + IERC20 billingToken; + BillingTokenPaymentParams billingTokenParams; + bool isTransaction; + } + + /** + * @notice struct containing receipt information about a payment or cost estimation + * @member gasChargeInBillingToken the amount to charge a user for gas spent using the billing token's native decimals + * @member premiumInBillingToken the premium charged to the user, shared between all nodes, using the billing token's native decimals + * @member gasReimbursementInJuels the amount to reimburse a node for gas spent + * @member premiumInJuels the premium paid to NOPs, shared between all nodes + */ + // solhint-disable-next-line gas-struct-packing + struct PaymentReceipt { + uint96 gasChargeInBillingToken; + uint96 premiumInBillingToken; + // one word ends + uint96 gasReimbursementInJuels; + uint96 premiumInJuels; + // second word ends + IERC20 billingToken; + uint96 linkUSD; + // third word ends + uint96 nativeUSD; + uint96 billingUSD; + // fourth word ends + } + + event AdminPrivilegeConfigSet(address indexed admin, bytes privilegeConfig); + event BillingConfigOverridden(uint256 indexed id, BillingOverrides overrides); + event BillingConfigOverrideRemoved(uint256 indexed id); + event BillingConfigSet(IERC20 indexed token, BillingConfig config); + event CancelledUpkeepReport(uint256 indexed id, bytes trigger); + event ChainSpecificModuleUpdated(address newModule); + event DedupKeyAdded(bytes32 indexed dedupKey); + event FeesWithdrawn(address indexed assetAddress, address indexed recipient, uint256 amount); + event FundsAdded(uint256 indexed id, address indexed from, uint96 amount); + event FundsWithdrawn(uint256 indexed id, uint256 amount, address to); + event InsufficientFundsUpkeepReport(uint256 indexed id, bytes trigger); + event NOPsSettledOffchain(address[] payees, uint256[] payments); + event Paused(address account); + event PayeesUpdated(address[] transmitters, address[] payees); + event PayeeshipTransferRequested(address indexed transmitter, address indexed from, address indexed to); + event PayeeshipTransferred(address indexed transmitter, address indexed from, address indexed to); + event PaymentWithdrawn(address indexed transmitter, uint256 indexed amount, address indexed to, address payee); + event ReorgedUpkeepReport(uint256 indexed id, bytes trigger); + event StaleUpkeepReport(uint256 indexed id, bytes trigger); + event UpkeepAdminTransferred(uint256 indexed id, address indexed from, address indexed to); + event UpkeepAdminTransferRequested(uint256 indexed id, address indexed from, address indexed to); + event UpkeepCanceled(uint256 indexed id, uint64 indexed atBlockHeight); + event UpkeepCheckDataSet(uint256 indexed id, bytes newCheckData); + event UpkeepGasLimitSet(uint256 indexed id, uint96 gasLimit); + event UpkeepMigrated(uint256 indexed id, uint256 remainingBalance, address destination); + event UpkeepOffchainConfigSet(uint256 indexed id, bytes offchainConfig); + event UpkeepPaused(uint256 indexed id); + event UpkeepPerformed( + uint256 indexed id, + bool indexed success, + uint96 totalPayment, + uint256 gasUsed, + uint256 gasOverhead, + bytes trigger + ); + event UpkeepCharged(uint256 indexed id, PaymentReceipt receipt); + event UpkeepPrivilegeConfigSet(uint256 indexed id, bytes privilegeConfig); + event UpkeepReceived(uint256 indexed id, uint256 startingBalance, address importedFrom); + event UpkeepRegistered(uint256 indexed id, uint32 performGas, address admin); + event UpkeepTriggerConfigSet(uint256 indexed id, bytes triggerConfig); + event UpkeepUnpaused(uint256 indexed id); + event Unpaused(address account); + + /** + * @param link address of the LINK Token + * @param linkUSDFeed address of the LINK/USD price feed + * @param nativeUSDFeed address of the Native/USD price feed + * @param fastGasFeed address of the Fast Gas price feed + * @param automationForwarderLogic the address of automation forwarder logic + * @param allowedReadOnlyAddress the address of the allowed read only address + * @param payoutMode the payout mode + */ + constructor( + address link, + address linkUSDFeed, + address nativeUSDFeed, + address fastGasFeed, + address automationForwarderLogic, + address allowedReadOnlyAddress, + PayoutMode payoutMode, + address wrappedNativeTokenAddress + ) ConfirmedOwner(msg.sender) { + i_link = LinkTokenInterface(link); + i_linkUSDFeed = AggregatorV3Interface(linkUSDFeed); + i_nativeUSDFeed = AggregatorV3Interface(nativeUSDFeed); + i_fastGasFeed = AggregatorV3Interface(fastGasFeed); + i_automationForwarderLogic = automationForwarderLogic; + i_allowedReadOnlyAddress = allowedReadOnlyAddress; + s_payoutMode = payoutMode; + i_wrappedNativeToken = IWrappedNative(wrappedNativeTokenAddress); + if (i_linkUSDFeed.decimals() != i_nativeUSDFeed.decimals()) { + revert InvalidFeed(); + } + } + + // ================================================================ + // | INTERNAL FUNCTIONS ONLY | + // ================================================================ + + /** + * @dev creates a new upkeep with the given fields + * @param id the id of the upkeep + * @param upkeep the upkeep to create + * @param admin address to cancel upkeep and withdraw remaining funds + * @param checkData data which is passed to user's checkUpkeep + * @param triggerConfig the trigger config for this upkeep + * @param offchainConfig the off-chain config of this upkeep + */ + function _createUpkeep( + uint256 id, + Upkeep memory upkeep, + address admin, + bytes memory checkData, + bytes memory triggerConfig, + bytes memory offchainConfig + ) internal { + if (s_hotVars.paused) revert RegistryPaused(); + if (checkData.length > s_storage.maxCheckDataSize) revert CheckDataExceedsLimit(); + if (upkeep.performGas < PERFORM_GAS_MIN || upkeep.performGas > s_storage.maxPerformGas) + revert GasLimitOutsideRange(); + if (address(s_upkeep[id].forwarder) != address(0)) revert UpkeepAlreadyExists(); + if (address(s_billingConfigs[upkeep.billingToken].priceFeed) == address(0)) revert InvalidToken(); + s_upkeep[id] = upkeep; + s_upkeepAdmin[id] = admin; + s_checkData[id] = checkData; + s_reserveAmounts[upkeep.billingToken] = s_reserveAmounts[upkeep.billingToken] + upkeep.balance; + s_upkeepTriggerConfig[id] = triggerConfig; + s_upkeepOffchainConfig[id] = offchainConfig; + s_upkeepIDs.add(id); + } + + /** + * @dev creates an ID for the upkeep based on the upkeep's type + * @dev the format of the ID looks like this: + * ****00000000000X**************** + * 4 bytes of entropy + * 11 bytes of zeros + * 1 identifying byte for the trigger type + * 16 bytes of entropy + * @dev this maintains the same level of entropy as eth addresses, so IDs will still be unique + * @dev we add the "identifying" part in the middle so that it is mostly hidden from users who usually only + * see the first 4 and last 4 hex values ex 0x1234...ABCD + */ + function _createID(Trigger triggerType) internal view returns (uint256) { + bytes1 empty; + IChainModule chainModule = s_hotVars.chainModule; + bytes memory idBytes = abi.encodePacked( + keccak256(abi.encode(chainModule.blockHash((chainModule.blockNumber() - 1)), address(this), s_storage.nonce)) + ); + for (uint256 idx = 4; idx < 15; idx++) { + idBytes[idx] = empty; + } + idBytes[15] = bytes1(uint8(triggerType)); + return uint256(bytes32(idBytes)); + } + + /** + * @dev retrieves feed data for fast gas/native and link/native prices. if the feed + * data is stale it uses the configured fallback price. Once a price is picked + * for gas it takes the min of gas price in the transaction or the fast gas + * price in order to reduce costs for the upkeep clients. + */ + function _getFeedData( + HotVars memory hotVars + ) internal view returns (uint256 gasWei, uint256 linkUSD, uint256 nativeUSD) { + uint32 stalenessSeconds = hotVars.stalenessSeconds; + bool staleFallback = stalenessSeconds > 0; + uint256 timestamp; + int256 feedValue; + (, feedValue, , timestamp, ) = i_fastGasFeed.latestRoundData(); + if ( + feedValue <= 0 || block.timestamp < timestamp || (staleFallback && stalenessSeconds < block.timestamp - timestamp) + ) { + gasWei = s_fallbackGasPrice; + } else { + gasWei = uint256(feedValue); + } + (, feedValue, , timestamp, ) = i_linkUSDFeed.latestRoundData(); + if ( + feedValue <= 0 || block.timestamp < timestamp || (staleFallback && stalenessSeconds < block.timestamp - timestamp) + ) { + linkUSD = s_fallbackLinkPrice; + } else { + linkUSD = uint256(feedValue); + } + return (gasWei, linkUSD, _getNativeUSD(hotVars)); + } + + /** + * @dev this price has it's own getter for use in the transmit() hot path + * in the future, all price data should be included in the report instead of + * getting read during execution + */ + function _getNativeUSD(HotVars memory hotVars) internal view returns (uint256) { + (, int256 feedValue, , uint256 timestamp, ) = i_nativeUSDFeed.latestRoundData(); + if ( + feedValue <= 0 || + block.timestamp < timestamp || + (hotVars.stalenessSeconds > 0 && hotVars.stalenessSeconds < block.timestamp - timestamp) + ) { + return s_fallbackNativePrice; + } else { + return uint256(feedValue); + } + } + + /** + * @dev gets the price and billing params for a specific billing token + */ + function _getBillingTokenPaymentParams( + HotVars memory hotVars, + IERC20 billingToken + ) internal view returns (BillingTokenPaymentParams memory paymentParams) { + BillingConfig storage config = s_billingConfigs[billingToken]; + paymentParams.flatFeeMilliCents = config.flatFeeMilliCents; + paymentParams.gasFeePPB = config.gasFeePPB; + paymentParams.decimals = config.decimals; + (, int256 feedValue, , uint256 timestamp, ) = config.priceFeed.latestRoundData(); + if ( + feedValue <= 0 || + block.timestamp < timestamp || + (hotVars.stalenessSeconds > 0 && hotVars.stalenessSeconds < block.timestamp - timestamp) + ) { + paymentParams.priceUSD = config.fallbackPrice; + } else { + paymentParams.priceUSD = uint256(feedValue); + } + return paymentParams; + } + + /** + * @param hotVars the hot path variables + * @param paymentParams the pricing data and gas usage data + * @return receipt the receipt of payment with pricing breakdown + * @dev use of PaymentParams struct is necessary to avoid stack too deep errors + * @dev calculates LINK paid for gas spent plus a configure premium percentage + * @dev 1 USD = 1e18 attoUSD + * @dev 1 USD = 1e26 hexaicosaUSD (had to borrow this prefix from geometry because there is no metric prefix for 1e-26) + * @dev 1 millicent = 1e-5 USD = 1e13 attoUSD + */ + function _calculatePaymentAmount( + HotVars memory hotVars, + PaymentParams memory paymentParams + ) internal view returns (PaymentReceipt memory receipt) { + uint256 decimals = paymentParams.billingTokenParams.decimals; + uint256 gasWei = paymentParams.fastGasWei * hotVars.gasCeilingMultiplier; + // in case it's actual execution use actual gas price, capped by fastGasWei * gasCeilingMultiplier + if (paymentParams.isTransaction && tx.gasprice < gasWei) { + gasWei = tx.gasprice; + } + + // scaling factor is based on decimals of billing token, and applies to premium and gasCharge + uint256 numeratorScalingFactor = decimals > 18 ? 10 ** (decimals - 18) : 1; + uint256 denominatorScalingFactor = decimals < 18 ? 10 ** (18 - decimals) : 1; + + // gas calculation + uint256 gasPaymentHexaicosaUSD = (gasWei * + (paymentParams.gasLimit + paymentParams.gasOverhead) + + paymentParams.l1CostWei) * paymentParams.nativeUSD; // gasPaymentHexaicosaUSD has an extra 8 zeros because of decimals on nativeUSD feed + // gasChargeInBillingToken is scaled by the billing token's decimals. Round up to ensure a minimum billing token is charged for gas + receipt.gasChargeInBillingToken = SafeCast.toUint96( + ((gasPaymentHexaicosaUSD * numeratorScalingFactor) + + (paymentParams.billingTokenParams.priceUSD * denominatorScalingFactor - 1)) / + (paymentParams.billingTokenParams.priceUSD * denominatorScalingFactor) + ); + // 18 decimals: 26 decimals / 8 decimals + receipt.gasReimbursementInJuels = SafeCast.toUint96(gasPaymentHexaicosaUSD / paymentParams.linkUSD); + + // premium calculation + uint256 flatFeeHexaicosaUSD = uint256(paymentParams.billingTokenParams.flatFeeMilliCents) * 1e21; // 1e13 for milliCents to attoUSD and 1e8 for attoUSD to hexaicosaUSD + uint256 premiumHexaicosaUSD = ((((gasWei * paymentParams.gasLimit) + paymentParams.l1CostWei) * + paymentParams.billingTokenParams.gasFeePPB * + paymentParams.nativeUSD) / 1e9) + flatFeeHexaicosaUSD; + // premium is scaled by the billing token's decimals. Round up to ensure at least minimum charge + receipt.premiumInBillingToken = SafeCast.toUint96( + ((premiumHexaicosaUSD * numeratorScalingFactor) + + (paymentParams.billingTokenParams.priceUSD * denominatorScalingFactor - 1)) / + (paymentParams.billingTokenParams.priceUSD * denominatorScalingFactor) + ); + receipt.premiumInJuels = SafeCast.toUint96(premiumHexaicosaUSD / paymentParams.linkUSD); + + receipt.billingToken = paymentParams.billingToken; + receipt.linkUSD = SafeCast.toUint96(paymentParams.linkUSD); + receipt.nativeUSD = SafeCast.toUint96(paymentParams.nativeUSD); + receipt.billingUSD = SafeCast.toUint96(paymentParams.billingTokenParams.priceUSD); + + return receipt; + } + + /** + * @dev calculates the max payment for an upkeep. Called during checkUpkeep simulation and assumes + * maximum gas overhead, L1 fee + */ + function _getMaxPayment( + uint256 upkeepId, + HotVars memory hotVars, + Trigger triggerType, + uint32 performGas, + uint256 fastGasWei, + uint256 linkUSD, + uint256 nativeUSD, + IERC20 billingToken + ) internal view returns (uint96) { + uint256 maxL1Fee; + uint256 maxGasOverhead; + + { + if (triggerType == Trigger.CONDITION) { + maxGasOverhead = REGISTRY_CONDITIONAL_OVERHEAD; + } else if (triggerType == Trigger.LOG) { + maxGasOverhead = REGISTRY_LOG_OVERHEAD; + } else { + revert InvalidTriggerType(); + } + uint256 maxCalldataSize = s_storage.maxPerformDataSize + + TRANSMIT_CALLDATA_FIXED_BYTES_OVERHEAD + + (TRANSMIT_CALLDATA_PER_SIGNER_BYTES_OVERHEAD * (hotVars.f + 1)); + (uint256 chainModuleFixedOverhead, uint256 chainModulePerByteOverhead) = s_hotVars.chainModule.getGasOverhead(); + maxGasOverhead += + (REGISTRY_PER_SIGNER_GAS_OVERHEAD * (hotVars.f + 1)) + + ((REGISTRY_PER_PERFORM_BYTE_GAS_OVERHEAD + chainModulePerByteOverhead) * maxCalldataSize) + + chainModuleFixedOverhead; + maxL1Fee = hotVars.gasCeilingMultiplier * hotVars.chainModule.getMaxL1Fee(maxCalldataSize); + } + + BillingTokenPaymentParams memory paymentParams = _getBillingTokenPaymentParams(hotVars, billingToken); + if (s_upkeep[upkeepId].overridesEnabled) { + BillingOverrides memory billingOverrides = s_billingOverrides[upkeepId]; + // use the overridden configs + paymentParams.gasFeePPB = billingOverrides.gasFeePPB; + paymentParams.flatFeeMilliCents = billingOverrides.flatFeeMilliCents; + } + + PaymentReceipt memory receipt = _calculatePaymentAmount( + hotVars, + PaymentParams({ + gasLimit: performGas, + gasOverhead: maxGasOverhead, + l1CostWei: maxL1Fee, + fastGasWei: fastGasWei, + linkUSD: linkUSD, + nativeUSD: nativeUSD, + billingToken: billingToken, + billingTokenParams: paymentParams, + isTransaction: false + }) + ); + + return receipt.gasChargeInBillingToken + receipt.premiumInBillingToken; + } + + /** + * @dev move a transmitter's balance from total pool to withdrawable balance + */ + function _updateTransmitterBalanceFromPool( + address transmitterAddress, + uint96 totalPremium, + uint96 payeeCount + ) internal returns (uint96) { + Transmitter memory transmitter = s_transmitters[transmitterAddress]; + + if (transmitter.active) { + uint96 uncollected = totalPremium - transmitter.lastCollected; + uint96 due = uncollected / payeeCount; + transmitter.balance += due; + transmitter.lastCollected += due * payeeCount; + s_transmitters[transmitterAddress] = transmitter; + } + + return transmitter.balance; + } + + /** + * @dev gets the trigger type from an upkeepID (trigger type is encoded in the middle of the ID) + */ + function _getTriggerType(uint256 upkeepId) internal pure returns (Trigger) { + bytes32 rawID = bytes32(upkeepId); + bytes1 empty = bytes1(0); + for (uint256 idx = 4; idx < 15; idx++) { + if (rawID[idx] != empty) { + // old IDs that were created before this standard and migrated to this registry + return Trigger.CONDITION; + } + } + return Trigger(uint8(rawID[15])); + } + + function _checkPayload( + uint256 upkeepId, + Trigger triggerType, + bytes memory triggerData + ) internal view returns (bytes memory) { + if (triggerType == Trigger.CONDITION) { + return abi.encodeWithSelector(CHECK_SELECTOR, s_checkData[upkeepId]); + } else if (triggerType == Trigger.LOG) { + Log memory log = abi.decode(triggerData, (Log)); + return abi.encodeWithSelector(CHECK_LOG_SELECTOR, log, s_checkData[upkeepId]); + } + revert InvalidTriggerType(); + } + + /** + * @dev _decodeReport decodes a serialized report into a Report struct + */ + function _decodeReport(bytes calldata rawReport) internal pure returns (Report memory) { + Report memory report = abi.decode(rawReport, (Report)); + uint256 expectedLength = report.upkeepIds.length; + if ( + report.gasLimits.length != expectedLength || + report.triggers.length != expectedLength || + report.performDatas.length != expectedLength + ) { + revert InvalidReport(); + } + return report; + } + + /** + * @dev Does some early sanity checks before actually performing an upkeep + * @return bool whether the upkeep should be performed + * @return bytes32 dedupID for preventing duplicate performances of this trigger + */ + function _prePerformChecks( + uint256 upkeepId, + uint256 blocknumber, + bytes memory rawTrigger, + UpkeepTransmitInfo memory transmitInfo, + HotVars memory hotVars + ) internal returns (bool, bytes32) { + bytes32 dedupID; + if (transmitInfo.triggerType == Trigger.CONDITION) { + if (!_validateConditionalTrigger(upkeepId, blocknumber, rawTrigger, transmitInfo, hotVars)) + return (false, dedupID); + } else if (transmitInfo.triggerType == Trigger.LOG) { + bool valid; + (valid, dedupID) = _validateLogTrigger(upkeepId, blocknumber, rawTrigger, hotVars); + if (!valid) return (false, dedupID); + } else { + revert InvalidTriggerType(); + } + if (transmitInfo.upkeep.maxValidBlocknumber <= blocknumber) { + // Can happen when an upkeep got cancelled after report was generated. + // However we have a CANCELLATION_DELAY of 50 blocks so shouldn't happen in practice + emit CancelledUpkeepReport(upkeepId, rawTrigger); + return (false, dedupID); + } + return (true, dedupID); + } + + /** + * @dev Does some early sanity checks before actually performing an upkeep + */ + function _validateConditionalTrigger( + uint256 upkeepId, + uint256 blocknumber, + bytes memory rawTrigger, + UpkeepTransmitInfo memory transmitInfo, + HotVars memory hotVars + ) internal returns (bool) { + ConditionalTrigger memory trigger = abi.decode(rawTrigger, (ConditionalTrigger)); + if (trigger.blockNum < transmitInfo.upkeep.lastPerformedBlockNumber) { + // Can happen when another report performed this upkeep after this report was generated + emit StaleUpkeepReport(upkeepId, rawTrigger); + return false; + } + if ( + (hotVars.reorgProtectionEnabled && + (trigger.blockHash != bytes32("") && hotVars.chainModule.blockHash(trigger.blockNum) != trigger.blockHash)) || + trigger.blockNum >= blocknumber + ) { + // There are two cases of reorged report + // 1. trigger block number is in future: this is an edge case during extreme deep reorgs of chain + // which is always protected against + // 2. blockHash at trigger block number was same as trigger time. This is an optional check which is + // applied if DON sends non empty trigger.blockHash. Note: It only works for last 256 blocks on chain + // when it is sent + emit ReorgedUpkeepReport(upkeepId, rawTrigger); + return false; + } + return true; + } + + function _validateLogTrigger( + uint256 upkeepId, + uint256 blocknumber, + bytes memory rawTrigger, + HotVars memory hotVars + ) internal returns (bool, bytes32) { + LogTrigger memory trigger = abi.decode(rawTrigger, (LogTrigger)); + bytes32 dedupID = keccak256(abi.encodePacked(upkeepId, trigger.logBlockHash, trigger.txHash, trigger.logIndex)); + if ( + (hotVars.reorgProtectionEnabled && + (trigger.blockHash != bytes32("") && hotVars.chainModule.blockHash(trigger.blockNum) != trigger.blockHash)) || + trigger.blockNum >= blocknumber + ) { + // Reorg protection is same as conditional trigger upkeeps + emit ReorgedUpkeepReport(upkeepId, rawTrigger); + return (false, dedupID); + } + if (s_dedupKeys[dedupID]) { + emit StaleUpkeepReport(upkeepId, rawTrigger); + return (false, dedupID); + } + return (true, dedupID); + } + + /** + * @dev Verify signatures attached to report + */ + function _verifyReportSignature( + bytes32[3] calldata reportContext, + bytes calldata report, + bytes32[] calldata rs, + bytes32[] calldata ss, + bytes32 rawVs + ) internal view { + bytes32 h = keccak256(abi.encode(keccak256(report), reportContext)); + // i-th byte counts number of sigs made by i-th signer + uint256 signedCount = 0; + + Signer memory signer; + address signerAddress; + for (uint256 i = 0; i < rs.length; i++) { + signerAddress = ecrecover(h, uint8(rawVs[i]) + 27, rs[i], ss[i]); + signer = s_signers[signerAddress]; + if (!signer.active) revert OnlyActiveSigners(); + unchecked { + signedCount += 1 << (8 * signer.index); + } + } + + if (signedCount & ORACLE_MASK != signedCount) revert DuplicateSigners(); + } + + /** + * @dev updates a storage marker for this upkeep to prevent duplicate and out of order performances + * @dev for conditional triggers we set the latest block number, for log triggers we store a dedupID + */ + function _updateTriggerMarker( + uint256 upkeepID, + uint256 blocknumber, + UpkeepTransmitInfo memory upkeepTransmitInfo + ) internal { + if (upkeepTransmitInfo.triggerType == Trigger.CONDITION) { + s_upkeep[upkeepID].lastPerformedBlockNumber = uint32(blocknumber); + } else if (upkeepTransmitInfo.triggerType == Trigger.LOG) { + s_dedupKeys[upkeepTransmitInfo.dedupID] = true; + emit DedupKeyAdded(upkeepTransmitInfo.dedupID); + } + } + + /** + * @dev calls the Upkeep target with the performData param passed in by the + * transmitter and the exact gas required by the Upkeep + */ + function _performUpkeep( + IAutomationForwarder forwarder, + uint256 performGas, + bytes memory performData + ) internal nonReentrant returns (bool success, uint256 gasUsed) { + performData = abi.encodeWithSelector(PERFORM_SELECTOR, performData); + return forwarder.forward(performGas, performData); + } + + /** + * @dev handles the payment processing after an upkeep has been performed. + * Deducts an upkeep's balance and increases the amount spent. + */ + function _handlePayment( + HotVars memory hotVars, + PaymentParams memory paymentParams, + uint256 upkeepId, + Upkeep memory upkeep + ) internal returns (PaymentReceipt memory) { + if (upkeep.overridesEnabled) { + BillingOverrides memory billingOverrides = s_billingOverrides[upkeepId]; + // use the overridden configs + paymentParams.billingTokenParams.gasFeePPB = billingOverrides.gasFeePPB; + paymentParams.billingTokenParams.flatFeeMilliCents = billingOverrides.flatFeeMilliCents; + } + + PaymentReceipt memory receipt = _calculatePaymentAmount(hotVars, paymentParams); + + // balance is in the token's native decimals + uint96 balance = upkeep.balance; + // payment is in the token's native decimals + uint96 payment = receipt.gasChargeInBillingToken + receipt.premiumInBillingToken; + + // scaling factors to adjust decimals between billing token and LINK + uint256 decimals = paymentParams.billingTokenParams.decimals; + uint256 scalingFactor1 = decimals < 18 ? 10 ** (18 - decimals) : 1; + uint256 scalingFactor2 = decimals > 18 ? 10 ** (decimals - 18) : 1; + + // this shouldn't happen, but in rare edge cases, we charge the full balance in case the user + // can't cover the amount owed + if (balance < receipt.gasChargeInBillingToken) { + // if the user can't cover the gas fee, then direct all of the payment to the transmitter and distribute no premium to the DON + payment = balance; + receipt.gasReimbursementInJuels = SafeCast.toUint96( + (balance * paymentParams.billingTokenParams.priceUSD * scalingFactor1) / + (paymentParams.linkUSD * scalingFactor2) + ); + receipt.premiumInJuels = 0; + receipt.premiumInBillingToken = 0; + receipt.gasChargeInBillingToken = balance; + } else if (balance < payment) { + // if the user can cover the gas fee, but not the premium, then reduce the premium + payment = balance; + receipt.premiumInJuels = SafeCast.toUint96( + ((balance * paymentParams.billingTokenParams.priceUSD * scalingFactor1) / + (paymentParams.linkUSD * scalingFactor2)) - receipt.gasReimbursementInJuels + ); + // round up + receipt.premiumInBillingToken = SafeCast.toUint96( + ((receipt.premiumInJuels * paymentParams.linkUSD * scalingFactor2) + + (paymentParams.billingTokenParams.priceUSD * scalingFactor1 - 1)) / + (paymentParams.billingTokenParams.priceUSD * scalingFactor1) + ); + } + + s_upkeep[upkeepId].balance -= payment; + s_upkeep[upkeepId].amountSpent += payment; + s_reserveAmounts[paymentParams.billingToken] -= payment; + + emit UpkeepCharged(upkeepId, receipt); + return receipt; + } + + /** + * @dev ensures the upkeep is not cancelled and the caller is the upkeep admin + */ + function _requireAdminAndNotCancelled(uint256 upkeepId) internal view { + if (msg.sender != s_upkeepAdmin[upkeepId]) revert OnlyCallableByAdmin(); + if (s_upkeep[upkeepId].maxValidBlocknumber != UINT32_MAX) revert UpkeepCancelled(); + } + + /** + * @dev replicates Open Zeppelin's ReentrancyGuard but optimized to fit our storage + */ + modifier nonReentrant() { + if (s_hotVars.reentrancyGuard) revert ReentrantCall(); + s_hotVars.reentrancyGuard = true; + _; + s_hotVars.reentrancyGuard = false; + } + + /** + * @notice only allows a pre-configured address to initiate offchain read + */ + function _preventExecution() internal view { + // solhint-disable-next-line avoid-tx-origin + if (tx.origin != i_allowedReadOnlyAddress) { + revert OnlySimulatedBackend(); + } + } + + /** + * @notice only allows finance admin to call the function + */ + function _onlyFinanceAdminAllowed() internal view { + if (msg.sender != s_storage.financeAdmin) { + revert OnlyFinanceAdmin(); + } + } + + /** + * @notice only allows privilege manager to call the function + */ + function _onlyPrivilegeManagerAllowed() internal view { + if (msg.sender != s_storage.upkeepPrivilegeManager) { + revert OnlyCallableByUpkeepPrivilegeManager(); + } + } + + /** + * @notice sets billing configuration for a token + * @param billingTokens the addresses of tokens + * @param billingConfigs the configs for tokens + */ + function _setBillingConfig(IERC20[] memory billingTokens, BillingConfig[] memory billingConfigs) internal { + // Clear existing data + for (uint256 i = 0; i < s_billingTokens.length; i++) { + delete s_billingConfigs[s_billingTokens[i]]; + } + delete s_billingTokens; + + PayoutMode mode = s_payoutMode; + for (uint256 i = 0; i < billingTokens.length; i++) { + IERC20 token = billingTokens[i]; + BillingConfig memory config = billingConfigs[i]; + + // most ERC20 tokens are 18 decimals, priceFeed must be 8 decimals + if (config.decimals != token.decimals() || config.priceFeed.decimals() != 8) { + revert InvalidToken(); + } + + // if LINK is a billing option, payout mode must be ON_CHAIN + if (address(token) == address(i_link) && mode == PayoutMode.OFF_CHAIN) { + revert InvalidToken(); + } + if (address(token) == ZERO_ADDRESS || address(config.priceFeed) == ZERO_ADDRESS) { + revert ZeroAddressNotAllowed(); + } + + // if this is a new token, add it to tokens list. Otherwise revert + if (address(s_billingConfigs[token].priceFeed) != ZERO_ADDRESS) { + revert DuplicateEntry(); + } + s_billingTokens.push(token); + + // update the billing config for an existing token or add a new one + s_billingConfigs[token] = config; + + emit BillingConfigSet(token, config); + } + } + + /** + * @notice updates the signers and transmitters lists + */ + function _updateTransmitters(address[] memory signers, address[] memory transmitters) internal { + uint96 transmittersListLength = uint96(s_transmittersList.length); + uint96 totalPremium = s_hotVars.totalPremium; + + // move all pooled payments out of the pool to each transmitter's balance + for (uint256 i = 0; i < s_transmittersList.length; i++) { + _updateTransmitterBalanceFromPool(s_transmittersList[i], totalPremium, transmittersListLength); + } + + // remove any old signer/transmitter addresses + address transmitterAddress; + PayoutMode mode = s_payoutMode; + for (uint256 i = 0; i < s_transmittersList.length; i++) { + transmitterAddress = s_transmittersList[i]; + delete s_signers[s_signersList[i]]; + // Do not delete the whole transmitter struct as it has balance information stored + s_transmitters[transmitterAddress].active = false; + if (mode == PayoutMode.OFF_CHAIN && s_transmitters[transmitterAddress].balance > 0) { + s_deactivatedTransmitters.add(transmitterAddress); + } + } + delete s_signersList; + delete s_transmittersList; + + // add new signer/transmitter addresses + Transmitter memory transmitter; + for (uint256 i = 0; i < signers.length; i++) { + if (s_signers[signers[i]].active) revert RepeatedSigner(); + if (signers[i] == ZERO_ADDRESS) revert InvalidSigner(); + s_signers[signers[i]] = Signer({active: true, index: uint8(i)}); + + transmitterAddress = transmitters[i]; + if (transmitterAddress == ZERO_ADDRESS) revert InvalidTransmitter(); + transmitter = s_transmitters[transmitterAddress]; + if (transmitter.active) revert RepeatedTransmitter(); + transmitter.active = true; + transmitter.index = uint8(i); + // new transmitters start afresh from current totalPremium + // some spare change of premium from previous pool will be forfeited + transmitter.lastCollected = s_hotVars.totalPremium; + s_transmitters[transmitterAddress] = transmitter; + if (mode == PayoutMode.OFF_CHAIN) { + s_deactivatedTransmitters.remove(transmitterAddress); + } + } + + s_signersList = signers; + s_transmittersList = transmitters; + } + + /** + * @notice returns the size of the LINK liquidity pool + # @dev LINK max supply < 2^96, so casting to int256 is safe + */ + function _linkAvailableForPayment() internal view returns (int256) { + return int256(i_link.balanceOf(address(this))) - int256(s_reserveAmounts[IERC20(address(i_link))]); + } +} diff --git a/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicA2_3.sol b/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicA2_3.sol new file mode 100644 index 00000000000..64d697c70f9 --- /dev/null +++ b/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicA2_3.sol @@ -0,0 +1,283 @@ +// SPDX-License-Identifier: BUSL-1.1 +pragma solidity 0.8.19; + +import {EnumerableSet} from "../../vendor/openzeppelin-solidity/v4.7.3/contracts/utils/structs/EnumerableSet.sol"; +import {Address} from "../../vendor/openzeppelin-solidity/v4.7.3/contracts/utils/Address.sol"; +import {ZKSyncAutomationRegistryBase2_3} from "./ZKSyncAutomationRegistryBase2_3.sol"; +import {ZKSyncAutomationRegistryLogicC2_3} from "./ZKSyncAutomationRegistryLogicC2_3.sol"; +import {ZKSyncAutomationRegistryLogicB2_3} from "./ZKSyncAutomationRegistryLogicB2_3.sol"; +import {Chainable} from "../Chainable.sol"; +import {ZKSyncAutomationForwarder} from "../ZKSyncAutomationForwarder.sol"; +import {IAutomationForwarder} from "../interfaces/IAutomationForwarder.sol"; +import {UpkeepTranscoderInterfaceV2} from "../interfaces/UpkeepTranscoderInterfaceV2.sol"; +import {MigratableKeeperRegistryInterfaceV2} from "../interfaces/MigratableKeeperRegistryInterfaceV2.sol"; +import {IERC20Metadata as IERC20} from "../../vendor/openzeppelin-solidity/v4.8.3/contracts/token/ERC20/extensions/IERC20Metadata.sol"; +import {SafeERC20} from "../../vendor/openzeppelin-solidity/v4.8.3/contracts/token/ERC20/utils/SafeERC20.sol"; +import {IERC677Receiver} from "../../shared/interfaces/IERC677Receiver.sol"; + +/** + * @notice Logic contract, works in tandem with AutomationRegistry as a proxy + */ +contract ZKSyncAutomationRegistryLogicA2_3 is ZKSyncAutomationRegistryBase2_3, Chainable, IERC677Receiver { + using Address for address; + using EnumerableSet for EnumerableSet.UintSet; + using EnumerableSet for EnumerableSet.AddressSet; + using SafeERC20 for IERC20; + + /** + * @param logicB the address of the second logic contract + * @dev we cast the contract to logicC in order to call logicC functions (via fallback) + */ + constructor( + ZKSyncAutomationRegistryLogicB2_3 logicB + ) + ZKSyncAutomationRegistryBase2_3( + ZKSyncAutomationRegistryLogicC2_3(address(logicB)).getLinkAddress(), + ZKSyncAutomationRegistryLogicC2_3(address(logicB)).getLinkUSDFeedAddress(), + ZKSyncAutomationRegistryLogicC2_3(address(logicB)).getNativeUSDFeedAddress(), + ZKSyncAutomationRegistryLogicC2_3(address(logicB)).getFastGasFeedAddress(), + ZKSyncAutomationRegistryLogicC2_3(address(logicB)).getAutomationForwarderLogic(), + ZKSyncAutomationRegistryLogicC2_3(address(logicB)).getAllowedReadOnlyAddress(), + ZKSyncAutomationRegistryLogicC2_3(address(logicB)).getPayoutMode(), + ZKSyncAutomationRegistryLogicC2_3(address(logicB)).getWrappedNativeTokenAddress() + ) + Chainable(address(logicB)) + {} + + /** + * @notice uses LINK's transferAndCall to LINK and add funding to an upkeep + * @dev safe to cast uint256 to uint96 as total LINK supply is under UINT96MAX + * @param sender the account which transferred the funds + * @param amount number of LINK transfer + */ + function onTokenTransfer(address sender, uint256 amount, bytes calldata data) external override { + if (msg.sender != address(i_link)) revert OnlyCallableByLINKToken(); + if (data.length != 32) revert InvalidDataLength(); + uint256 id = abi.decode(data, (uint256)); + if (s_upkeep[id].maxValidBlocknumber != UINT32_MAX) revert UpkeepCancelled(); + if (address(s_upkeep[id].billingToken) != address(i_link)) revert InvalidToken(); + s_upkeep[id].balance = s_upkeep[id].balance + uint96(amount); + s_reserveAmounts[IERC20(address(i_link))] = s_reserveAmounts[IERC20(address(i_link))] + amount; + emit FundsAdded(id, sender, uint96(amount)); + } + + // ================================================================ + // | UPKEEP MANAGEMENT | + // ================================================================ + + /** + * @notice adds a new upkeep + * @param target address to perform upkeep on + * @param gasLimit amount of gas to provide the target contract when + * performing upkeep + * @param admin address to cancel upkeep and withdraw remaining funds + * @param triggerType the trigger for the upkeep + * @param billingToken the billing token for the upkeep + * @param checkData data passed to the contract when checking for upkeep + * @param triggerConfig the config for the trigger + * @param offchainConfig arbitrary offchain config for the upkeep + */ + function registerUpkeep( + address target, + uint32 gasLimit, + address admin, + Trigger triggerType, + IERC20 billingToken, + bytes calldata checkData, + bytes memory triggerConfig, + bytes memory offchainConfig + ) public returns (uint256 id) { + if (msg.sender != owner() && !s_registrars.contains(msg.sender)) revert OnlyCallableByOwnerOrRegistrar(); + if (!target.isContract()) revert NotAContract(); + id = _createID(triggerType); + IAutomationForwarder forwarder = IAutomationForwarder( + address(new ZKSyncAutomationForwarder(target, address(this), i_automationForwarderLogic)) + ); + _createUpkeep( + id, + Upkeep({ + overridesEnabled: false, + performGas: gasLimit, + balance: 0, + maxValidBlocknumber: UINT32_MAX, + lastPerformedBlockNumber: 0, + amountSpent: 0, + paused: false, + forwarder: forwarder, + billingToken: billingToken + }), + admin, + checkData, + triggerConfig, + offchainConfig + ); + s_storage.nonce++; + emit UpkeepRegistered(id, gasLimit, admin); + emit UpkeepCheckDataSet(id, checkData); + emit UpkeepTriggerConfigSet(id, triggerConfig); + emit UpkeepOffchainConfigSet(id, offchainConfig); + return (id); + } + + /** + * @notice cancels an upkeep + * @param id the upkeepID to cancel + * @dev if a user cancels an upkeep, their funds are locked for CANCELLATION_DELAY blocks to + * allow any pending performUpkeep txs time to get confirmed + */ + function cancelUpkeep(uint256 id) external { + Upkeep memory upkeep = s_upkeep[id]; + bool isOwner = msg.sender == owner(); + uint96 minSpend = s_billingConfigs[upkeep.billingToken].minSpend; + + uint256 height = s_hotVars.chainModule.blockNumber(); + if (upkeep.maxValidBlocknumber == 0) revert CannotCancel(); + if (upkeep.maxValidBlocknumber != UINT32_MAX) revert UpkeepCancelled(); + if (!isOwner && msg.sender != s_upkeepAdmin[id]) revert OnlyCallableByOwnerOrAdmin(); + + if (!isOwner) { + height = height + CANCELLATION_DELAY; + } + s_upkeep[id].maxValidBlocknumber = uint32(height); + s_upkeepIDs.remove(id); + + // charge the cancellation fee if the minSpend is not met + uint96 cancellationFee = 0; + // cancellationFee is min(max(minSpend - amountSpent, 0), amountLeft) + if (upkeep.amountSpent < minSpend) { + cancellationFee = minSpend - uint96(upkeep.amountSpent); + if (cancellationFee > upkeep.balance) { + cancellationFee = upkeep.balance; + } + } + s_upkeep[id].balance = upkeep.balance - cancellationFee; + s_reserveAmounts[upkeep.billingToken] = s_reserveAmounts[upkeep.billingToken] - cancellationFee; + + emit UpkeepCanceled(id, uint64(height)); + } + + /** + * @notice migrates upkeeps from one registry to another. + * @param ids the upkeepIDs to migrate + * @param destination the destination registry address + * @dev a transcoder must be set in order to enable migration + * @dev migration permissions must be set on *both* sending and receiving registries + * @dev only an upkeep admin can migrate their upkeeps + * @dev this function is most gas-efficient if upkeepIDs are sorted by billing token + * @dev s_billingOverrides and s_upkeepPrivilegeConfig are not migrated in this function + */ + function migrateUpkeeps(uint256[] calldata ids, address destination) external { + if ( + s_peerRegistryMigrationPermission[destination] != MigrationPermission.OUTGOING && + s_peerRegistryMigrationPermission[destination] != MigrationPermission.BIDIRECTIONAL + ) revert MigrationNotPermitted(); + if (s_storage.transcoder == ZERO_ADDRESS) revert TranscoderNotSet(); + if (ids.length == 0) revert ArrayHasNoEntries(); + + IERC20 billingToken; + uint256 balanceToTransfer; + uint256 id; + Upkeep memory upkeep; + address[] memory admins = new address[](ids.length); + Upkeep[] memory upkeeps = new Upkeep[](ids.length); + bytes[] memory checkDatas = new bytes[](ids.length); + bytes[] memory triggerConfigs = new bytes[](ids.length); + bytes[] memory offchainConfigs = new bytes[](ids.length); + + for (uint256 idx = 0; idx < ids.length; idx++) { + id = ids[idx]; + upkeep = s_upkeep[id]; + + if (idx == 0) { + billingToken = upkeep.billingToken; + balanceToTransfer = upkeep.balance; + } + + // if we encounter a new billing token, send the sum from the last billing token to the destination registry + if (upkeep.billingToken != billingToken) { + s_reserveAmounts[billingToken] = s_reserveAmounts[billingToken] - balanceToTransfer; + billingToken.safeTransfer(destination, balanceToTransfer); + billingToken = upkeep.billingToken; + balanceToTransfer = upkeep.balance; + } else if (idx != 0) { + balanceToTransfer += upkeep.balance; + } + + _requireAdminAndNotCancelled(id); + upkeep.forwarder.updateRegistry(destination); + + upkeeps[idx] = upkeep; + admins[idx] = s_upkeepAdmin[id]; + checkDatas[idx] = s_checkData[id]; + triggerConfigs[idx] = s_upkeepTriggerConfig[id]; + offchainConfigs[idx] = s_upkeepOffchainConfig[id]; + delete s_upkeep[id]; + delete s_checkData[id]; + delete s_upkeepTriggerConfig[id]; + delete s_upkeepOffchainConfig[id]; + // nullify existing proposed admin change if an upkeep is being migrated + delete s_proposedAdmin[id]; + delete s_upkeepAdmin[id]; + s_upkeepIDs.remove(id); + emit UpkeepMigrated(id, upkeep.balance, destination); + } + // always transfer the rolling sum in the end + s_reserveAmounts[billingToken] = s_reserveAmounts[billingToken] - balanceToTransfer; + billingToken.safeTransfer(destination, balanceToTransfer); + + bytes memory encodedUpkeeps = abi.encode( + ids, + upkeeps, + new address[](ids.length), + admins, + checkDatas, + triggerConfigs, + offchainConfigs + ); + MigratableKeeperRegistryInterfaceV2(destination).receiveUpkeeps( + UpkeepTranscoderInterfaceV2(s_storage.transcoder).transcodeUpkeeps( + UPKEEP_VERSION_BASE, + MigratableKeeperRegistryInterfaceV2(destination).upkeepVersion(), + encodedUpkeeps + ) + ); + } + + /** + * @notice received upkeeps migrated from another registry + * @param encodedUpkeeps the raw upkeep data to import + * @dev this function is never called directly, it is only called by another registry's migrate function + * @dev s_billingOverrides and s_upkeepPrivilegeConfig are not handled in this function + */ + function receiveUpkeeps(bytes calldata encodedUpkeeps) external { + if ( + s_peerRegistryMigrationPermission[msg.sender] != MigrationPermission.INCOMING && + s_peerRegistryMigrationPermission[msg.sender] != MigrationPermission.BIDIRECTIONAL + ) revert MigrationNotPermitted(); + ( + uint256[] memory ids, + Upkeep[] memory upkeeps, + address[] memory targets, + address[] memory upkeepAdmins, + bytes[] memory checkDatas, + bytes[] memory triggerConfigs, + bytes[] memory offchainConfigs + ) = abi.decode(encodedUpkeeps, (uint256[], Upkeep[], address[], address[], bytes[], bytes[], bytes[])); + for (uint256 idx = 0; idx < ids.length; idx++) { + if (address(upkeeps[idx].forwarder) == ZERO_ADDRESS) { + upkeeps[idx].forwarder = IAutomationForwarder( + address(new ZKSyncAutomationForwarder(targets[idx], address(this), i_automationForwarderLogic)) + ); + } + _createUpkeep( + ids[idx], + upkeeps[idx], + upkeepAdmins[idx], + checkDatas[idx], + triggerConfigs[idx], + offchainConfigs[idx] + ); + emit UpkeepReceived(ids[idx], upkeeps[idx].balance, msg.sender); + } + } +} diff --git a/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicB2_3.sol b/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicB2_3.sol new file mode 100644 index 00000000000..55af99fde87 --- /dev/null +++ b/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicB2_3.sol @@ -0,0 +1,449 @@ +// SPDX-License-Identifier: BUSL-1.1 +pragma solidity 0.8.19; + +import {ZKSyncAutomationRegistryBase2_3} from "./ZKSyncAutomationRegistryBase2_3.sol"; +import {EnumerableSet} from "../../vendor/openzeppelin-solidity/v4.7.3/contracts/utils/structs/EnumerableSet.sol"; +import {Address} from "../../vendor/openzeppelin-solidity/v4.7.3/contracts/utils/Address.sol"; +import {ZKSyncAutomationRegistryLogicC2_3} from "./ZKSyncAutomationRegistryLogicC2_3.sol"; +import {Chainable} from "../Chainable.sol"; +import {IERC20Metadata as IERC20} from "../../vendor/openzeppelin-solidity/v4.8.3/contracts/token/ERC20/extensions/IERC20Metadata.sol"; +import {SafeERC20} from "../../vendor/openzeppelin-solidity/v4.8.3/contracts/token/ERC20/utils/SafeERC20.sol"; +import {SafeCast} from "../../vendor/openzeppelin-solidity/v4.8.3/contracts/utils/math/SafeCast.sol"; + +contract ZKSyncAutomationRegistryLogicB2_3 is ZKSyncAutomationRegistryBase2_3, Chainable { + using Address for address; + using EnumerableSet for EnumerableSet.UintSet; + using EnumerableSet for EnumerableSet.AddressSet; + using SafeERC20 for IERC20; + + /** + * @param logicC the address of the third logic contract + */ + constructor( + ZKSyncAutomationRegistryLogicC2_3 logicC + ) + ZKSyncAutomationRegistryBase2_3( + logicC.getLinkAddress(), + logicC.getLinkUSDFeedAddress(), + logicC.getNativeUSDFeedAddress(), + logicC.getFastGasFeedAddress(), + logicC.getAutomationForwarderLogic(), + logicC.getAllowedReadOnlyAddress(), + logicC.getPayoutMode(), + logicC.getWrappedNativeTokenAddress() + ) + Chainable(address(logicC)) + {} + + // ================================================================ + // | PIPELINE FUNCTIONS | + // ================================================================ + + /** + * @notice called by the automation DON to check if work is needed + * @param id the upkeep ID to check for work needed + * @param triggerData extra contextual data about the trigger (not used in all code paths) + * @dev this one of the core functions called in the hot path + * @dev there is a 2nd checkUpkeep function (below) that is being maintained for backwards compatibility + * @dev there is an incongruency on what gets returned during failure modes + * ex sometimes we include price data, sometimes we omit it depending on the failure + */ + function checkUpkeep( + uint256 id, + bytes memory triggerData + ) + public + returns ( + bool upkeepNeeded, + bytes memory performData, + UpkeepFailureReason upkeepFailureReason, + uint256 gasUsed, + uint256 gasLimit, + uint256 fastGasWei, + uint256 linkUSD + ) + { + _preventExecution(); + + Trigger triggerType = _getTriggerType(id); + HotVars memory hotVars = s_hotVars; + Upkeep memory upkeep = s_upkeep[id]; + + { + uint256 nativeUSD; + uint96 maxPayment; + if (hotVars.paused) return (false, bytes(""), UpkeepFailureReason.REGISTRY_PAUSED, 0, upkeep.performGas, 0, 0); + if (upkeep.maxValidBlocknumber != UINT32_MAX) + return (false, bytes(""), UpkeepFailureReason.UPKEEP_CANCELLED, 0, upkeep.performGas, 0, 0); + if (upkeep.paused) return (false, bytes(""), UpkeepFailureReason.UPKEEP_PAUSED, 0, upkeep.performGas, 0, 0); + (fastGasWei, linkUSD, nativeUSD) = _getFeedData(hotVars); + maxPayment = _getMaxPayment( + id, + hotVars, + triggerType, + upkeep.performGas, + fastGasWei, + linkUSD, + nativeUSD, + upkeep.billingToken + ); + if (upkeep.balance < maxPayment) { + return (false, bytes(""), UpkeepFailureReason.INSUFFICIENT_BALANCE, 0, upkeep.performGas, 0, 0); + } + } + + bytes memory callData = _checkPayload(id, triggerType, triggerData); + + gasUsed = gasleft(); + // solhint-disable-next-line avoid-low-level-calls + (bool success, bytes memory result) = upkeep.forwarder.getTarget().call{gas: s_storage.checkGasLimit}(callData); + gasUsed = gasUsed - gasleft(); + + if (!success) { + // User's target check reverted. We capture the revert data here and pass it within performData + if (result.length > s_storage.maxRevertDataSize) { + return ( + false, + bytes(""), + UpkeepFailureReason.REVERT_DATA_EXCEEDS_LIMIT, + gasUsed, + upkeep.performGas, + fastGasWei, + linkUSD + ); + } + return ( + upkeepNeeded, + result, + UpkeepFailureReason.TARGET_CHECK_REVERTED, + gasUsed, + upkeep.performGas, + fastGasWei, + linkUSD + ); + } + + (upkeepNeeded, performData) = abi.decode(result, (bool, bytes)); + if (!upkeepNeeded) + return (false, bytes(""), UpkeepFailureReason.UPKEEP_NOT_NEEDED, gasUsed, upkeep.performGas, fastGasWei, linkUSD); + + if (performData.length > s_storage.maxPerformDataSize) + return ( + false, + bytes(""), + UpkeepFailureReason.PERFORM_DATA_EXCEEDS_LIMIT, + gasUsed, + upkeep.performGas, + fastGasWei, + linkUSD + ); + + return (upkeepNeeded, performData, upkeepFailureReason, gasUsed, upkeep.performGas, fastGasWei, linkUSD); + } + + /** + * @notice see other checkUpkeep function for description + * @dev this function may be deprecated in a future version of chainlink automation + */ + function checkUpkeep( + uint256 id + ) + external + returns ( + bool upkeepNeeded, + bytes memory performData, + UpkeepFailureReason upkeepFailureReason, + uint256 gasUsed, + uint256 gasLimit, + uint256 fastGasWei, + uint256 linkUSD + ) + { + return checkUpkeep(id, bytes("")); + } + + /** + * @dev checkCallback is used specifically for automation data streams lookups (see StreamsLookupCompatibleInterface.sol) + * @param id the upkeepID to execute a callback for + * @param values the values returned from the data streams lookup + * @param extraData the user-provided extra context data + */ + function checkCallback( + uint256 id, + bytes[] memory values, + bytes calldata extraData + ) + external + returns (bool upkeepNeeded, bytes memory performData, UpkeepFailureReason upkeepFailureReason, uint256 gasUsed) + { + bytes memory payload = abi.encodeWithSelector(CHECK_CALLBACK_SELECTOR, values, extraData); + return executeCallback(id, payload); + } + + /** + * @notice this is a generic callback executor that forwards a call to a user's contract with the configured + * gas limit + * @param id the upkeepID to execute a callback for + * @param payload the data (including function selector) to call on the upkeep target contract + */ + function executeCallback( + uint256 id, + bytes memory payload + ) + public + returns (bool upkeepNeeded, bytes memory performData, UpkeepFailureReason upkeepFailureReason, uint256 gasUsed) + { + _preventExecution(); + + Upkeep memory upkeep = s_upkeep[id]; + gasUsed = gasleft(); + // solhint-disable-next-line avoid-low-level-calls + (bool success, bytes memory result) = upkeep.forwarder.getTarget().call{gas: s_storage.checkGasLimit}(payload); + gasUsed = gasUsed - gasleft(); + if (!success) { + return (false, bytes(""), UpkeepFailureReason.CALLBACK_REVERTED, gasUsed); + } + (upkeepNeeded, performData) = abi.decode(result, (bool, bytes)); + if (!upkeepNeeded) { + return (false, bytes(""), UpkeepFailureReason.UPKEEP_NOT_NEEDED, gasUsed); + } + if (performData.length > s_storage.maxPerformDataSize) { + return (false, bytes(""), UpkeepFailureReason.PERFORM_DATA_EXCEEDS_LIMIT, gasUsed); + } + return (upkeepNeeded, performData, upkeepFailureReason, gasUsed); + } + + /** + * @notice simulates the upkeep with the perform data returned from checkUpkeep + * @param id identifier of the upkeep to execute the data with. + * @param performData calldata parameter to be passed to the target upkeep. + * @return success whether the call reverted or not + * @return gasUsed the amount of gas the target contract consumed + */ + function simulatePerformUpkeep( + uint256 id, + bytes calldata performData + ) external returns (bool success, uint256 gasUsed) { + _preventExecution(); + + if (s_hotVars.paused) revert RegistryPaused(); + Upkeep memory upkeep = s_upkeep[id]; + (success, gasUsed) = _performUpkeep(upkeep.forwarder, upkeep.performGas, performData); + return (success, gasUsed); + } + + // ================================================================ + // | UPKEEP MANAGEMENT | + // ================================================================ + + /** + * @notice adds fund to an upkeep + * @param id the upkeepID + * @param amount the amount of funds to add, in the upkeep's billing token + */ + function addFunds(uint256 id, uint96 amount) external payable { + Upkeep memory upkeep = s_upkeep[id]; + if (upkeep.maxValidBlocknumber != UINT32_MAX) revert UpkeepCancelled(); + + if (msg.value != 0) { + if (upkeep.billingToken != IERC20(i_wrappedNativeToken)) { + revert InvalidToken(); + } + amount = SafeCast.toUint96(msg.value); + } + + s_upkeep[id].balance = upkeep.balance + amount; + s_reserveAmounts[upkeep.billingToken] = s_reserveAmounts[upkeep.billingToken] + amount; + + if (msg.value == 0) { + // ERC20 payment + upkeep.billingToken.safeTransferFrom(msg.sender, address(this), amount); + } else { + // native payment + i_wrappedNativeToken.deposit{value: amount}(); + } + + emit FundsAdded(id, msg.sender, amount); + } + + /** + * @notice overrides the billing config for an upkeep + * @param id the upkeepID + * @param billingOverrides the override-able billing config + */ + function setBillingOverrides(uint256 id, BillingOverrides calldata billingOverrides) external { + _onlyPrivilegeManagerAllowed(); + if (s_upkeep[id].maxValidBlocknumber != UINT32_MAX) revert UpkeepCancelled(); + + s_upkeep[id].overridesEnabled = true; + s_billingOverrides[id] = billingOverrides; + emit BillingConfigOverridden(id, billingOverrides); + } + + /** + * @notice remove the overridden billing config for an upkeep + * @param id the upkeepID + */ + function removeBillingOverrides(uint256 id) external { + _onlyPrivilegeManagerAllowed(); + + s_upkeep[id].overridesEnabled = false; + delete s_billingOverrides[id]; + emit BillingConfigOverrideRemoved(id); + } + + /** + * @notice transfers the address of an admin for an upkeep + */ + function transferUpkeepAdmin(uint256 id, address proposed) external { + _requireAdminAndNotCancelled(id); + if (proposed == msg.sender) revert ValueNotChanged(); + + if (s_proposedAdmin[id] != proposed) { + s_proposedAdmin[id] = proposed; + emit UpkeepAdminTransferRequested(id, msg.sender, proposed); + } + } + + /** + * @notice accepts the transfer of an upkeep admin + */ + function acceptUpkeepAdmin(uint256 id) external { + Upkeep memory upkeep = s_upkeep[id]; + if (upkeep.maxValidBlocknumber != UINT32_MAX) revert UpkeepCancelled(); + if (s_proposedAdmin[id] != msg.sender) revert OnlyCallableByProposedAdmin(); + address past = s_upkeepAdmin[id]; + s_upkeepAdmin[id] = msg.sender; + s_proposedAdmin[id] = ZERO_ADDRESS; + + emit UpkeepAdminTransferred(id, past, msg.sender); + } + + /** + * @notice pauses an upkeep - an upkeep will be neither checked nor performed while paused + */ + function pauseUpkeep(uint256 id) external { + _requireAdminAndNotCancelled(id); + Upkeep memory upkeep = s_upkeep[id]; + if (upkeep.paused) revert OnlyUnpausedUpkeep(); + s_upkeep[id].paused = true; + s_upkeepIDs.remove(id); + emit UpkeepPaused(id); + } + + /** + * @notice unpauses an upkeep + */ + function unpauseUpkeep(uint256 id) external { + _requireAdminAndNotCancelled(id); + Upkeep memory upkeep = s_upkeep[id]; + if (!upkeep.paused) revert OnlyPausedUpkeep(); + s_upkeep[id].paused = false; + s_upkeepIDs.add(id); + emit UpkeepUnpaused(id); + } + + /** + * @notice updates the checkData for an upkeep + */ + function setUpkeepCheckData(uint256 id, bytes calldata newCheckData) external { + _requireAdminAndNotCancelled(id); + if (newCheckData.length > s_storage.maxCheckDataSize) revert CheckDataExceedsLimit(); + s_checkData[id] = newCheckData; + emit UpkeepCheckDataSet(id, newCheckData); + } + + /** + * @notice updates the gas limit for an upkeep + */ + function setUpkeepGasLimit(uint256 id, uint32 gasLimit) external { + if (gasLimit < PERFORM_GAS_MIN || gasLimit > s_storage.maxPerformGas) revert GasLimitOutsideRange(); + _requireAdminAndNotCancelled(id); + s_upkeep[id].performGas = gasLimit; + + emit UpkeepGasLimitSet(id, gasLimit); + } + + /** + * @notice updates the offchain config for an upkeep + */ + function setUpkeepOffchainConfig(uint256 id, bytes calldata config) external { + _requireAdminAndNotCancelled(id); + s_upkeepOffchainConfig[id] = config; + emit UpkeepOffchainConfigSet(id, config); + } + + /** + * @notice sets the upkeep trigger config + * @param id the upkeepID to change the trigger for + * @param triggerConfig the new trigger config + */ + function setUpkeepTriggerConfig(uint256 id, bytes calldata triggerConfig) external { + _requireAdminAndNotCancelled(id); + s_upkeepTriggerConfig[id] = triggerConfig; + emit UpkeepTriggerConfigSet(id, triggerConfig); + } + + /** + * @notice withdraws an upkeep's funds from an upkeep + * @dev note that an upkeep must be cancelled first!! + */ + function withdrawFunds(uint256 id, address to) external nonReentrant { + if (to == ZERO_ADDRESS) revert InvalidRecipient(); + Upkeep memory upkeep = s_upkeep[id]; + if (s_upkeepAdmin[id] != msg.sender) revert OnlyCallableByAdmin(); + if (upkeep.maxValidBlocknumber > s_hotVars.chainModule.blockNumber()) revert UpkeepNotCanceled(); + uint96 amountToWithdraw = s_upkeep[id].balance; + s_reserveAmounts[upkeep.billingToken] = s_reserveAmounts[upkeep.billingToken] - amountToWithdraw; + s_upkeep[id].balance = 0; + upkeep.billingToken.safeTransfer(to, amountToWithdraw); + emit FundsWithdrawn(id, amountToWithdraw, to); + } + + // ================================================================ + // | FINANCE ACTIONS | + // ================================================================ + + /** + * @notice withdraws excess LINK from the liquidity pool + * @param to the address to send the fees to + * @param amount the amount to withdraw + */ + function withdrawLink(address to, uint256 amount) external { + _onlyFinanceAdminAllowed(); + if (to == ZERO_ADDRESS) revert InvalidRecipient(); + + int256 available = _linkAvailableForPayment(); + if (available < 0) { + revert InsufficientBalance(0, amount); + } else if (amount > uint256(available)) { + revert InsufficientBalance(uint256(available), amount); + } + + bool transferStatus = i_link.transfer(to, amount); + if (!transferStatus) { + revert TransferFailed(); + } + emit FeesWithdrawn(address(i_link), to, amount); + } + + /** + * @notice withdraws non-LINK fees earned by the contract + * @param asset the asset to withdraw + * @param to the address to send the fees to + * @param amount the amount to withdraw + * @dev in ON_CHAIN mode, we prevent withdrawing non-LINK fees unless there is sufficient LINK liquidity + * to cover all outstanding debts on the registry + */ + function withdrawERC20Fees(IERC20 asset, address to, uint256 amount) external { + _onlyFinanceAdminAllowed(); + if (to == ZERO_ADDRESS) revert InvalidRecipient(); + if (address(asset) == address(i_link)) revert InvalidToken(); + if (_linkAvailableForPayment() < 0 && s_payoutMode == PayoutMode.ON_CHAIN) revert InsufficientLinkLiquidity(); + uint256 available = asset.balanceOf(address(this)) - s_reserveAmounts[asset]; + if (amount > available) revert InsufficientBalance(available, amount); + + asset.safeTransfer(to, amount); + emit FeesWithdrawn(address(asset), to, amount); + } +} diff --git a/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicC2_3.sol b/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicC2_3.sol new file mode 100644 index 00000000000..61d0eecfbaf --- /dev/null +++ b/contracts/src/v0.8/automation/v2_3_zksync/ZKSyncAutomationRegistryLogicC2_3.sol @@ -0,0 +1,638 @@ +// SPDX-License-Identifier: BUSL-1.1 +pragma solidity 0.8.19; + +import {ZKSyncAutomationRegistryBase2_3} from "./ZKSyncAutomationRegistryBase2_3.sol"; +import {EnumerableSet} from "../../vendor/openzeppelin-solidity/v4.7.3/contracts/utils/structs/EnumerableSet.sol"; +import {Address} from "../../vendor/openzeppelin-solidity/v4.7.3/contracts/utils/Address.sol"; +import {IAutomationForwarder} from "../interfaces/IAutomationForwarder.sol"; +import {IChainModule} from "../interfaces/IChainModule.sol"; +import {IERC20Metadata as IERC20} from "../../vendor/openzeppelin-solidity/v4.8.3/contracts/token/ERC20/extensions/IERC20Metadata.sol"; +import {IAutomationV21PlusCommon} from "../interfaces/IAutomationV21PlusCommon.sol"; + +contract ZKSyncAutomationRegistryLogicC2_3 is ZKSyncAutomationRegistryBase2_3 { + using Address for address; + using EnumerableSet for EnumerableSet.UintSet; + using EnumerableSet for EnumerableSet.AddressSet; + + /** + * @dev see AutomationRegistry master contract for constructor description + */ + constructor( + address link, + address linkUSDFeed, + address nativeUSDFeed, + address fastGasFeed, + address automationForwarderLogic, + address allowedReadOnlyAddress, + PayoutMode payoutMode, + address wrappedNativeTokenAddress + ) + ZKSyncAutomationRegistryBase2_3( + link, + linkUSDFeed, + nativeUSDFeed, + fastGasFeed, + automationForwarderLogic, + allowedReadOnlyAddress, + payoutMode, + wrappedNativeTokenAddress + ) + {} + + // ================================================================ + // | NODE ACTIONS | + // ================================================================ + + /** + * @notice transfers the address of payee for a transmitter + */ + function transferPayeeship(address transmitter, address proposed) external { + if (s_transmitterPayees[transmitter] != msg.sender) revert OnlyCallableByPayee(); + if (proposed == msg.sender) revert ValueNotChanged(); + + if (s_proposedPayee[transmitter] != proposed) { + s_proposedPayee[transmitter] = proposed; + emit PayeeshipTransferRequested(transmitter, msg.sender, proposed); + } + } + + /** + * @notice accepts the transfer of the payee + */ + function acceptPayeeship(address transmitter) external { + if (s_proposedPayee[transmitter] != msg.sender) revert OnlyCallableByProposedPayee(); + address past = s_transmitterPayees[transmitter]; + s_transmitterPayees[transmitter] = msg.sender; + s_proposedPayee[transmitter] = ZERO_ADDRESS; + + emit PayeeshipTransferred(transmitter, past, msg.sender); + } + + /** + * @notice this is for NOPs to withdraw LINK received as payment for work performed + */ + function withdrawPayment(address from, address to) external { + if (to == ZERO_ADDRESS) revert InvalidRecipient(); + if (s_payoutMode == PayoutMode.OFF_CHAIN) revert MustSettleOffchain(); + if (s_transmitterPayees[from] != msg.sender) revert OnlyCallableByPayee(); + uint96 balance = _updateTransmitterBalanceFromPool(from, s_hotVars.totalPremium, uint96(s_transmittersList.length)); + s_transmitters[from].balance = 0; + s_reserveAmounts[IERC20(address(i_link))] = s_reserveAmounts[IERC20(address(i_link))] - balance; + bool transferStatus = i_link.transfer(to, balance); + if (!transferStatus) { + revert TransferFailed(); + } + emit PaymentWithdrawn(from, balance, to, msg.sender); + } + + // ================================================================ + // | OWNER / MANAGER ACTIONS | + // ================================================================ + + /** + * @notice sets the privilege config for an upkeep + */ + function setUpkeepPrivilegeConfig(uint256 upkeepId, bytes calldata newPrivilegeConfig) external { + _onlyPrivilegeManagerAllowed(); + s_upkeepPrivilegeConfig[upkeepId] = newPrivilegeConfig; + emit UpkeepPrivilegeConfigSet(upkeepId, newPrivilegeConfig); + } + + /** + * @notice this is used by the owner to set the initial payees for newly added transmitters. The owner is not allowed to change payees for existing transmitters. + * @dev the IGNORE_ADDRESS is a "helper" that makes it easier to construct a list of payees when you only care about setting the payee for a small number of transmitters. + */ + function setPayees(address[] calldata payees) external onlyOwner { + if (s_transmittersList.length != payees.length) revert ParameterLengthError(); + for (uint256 i = 0; i < s_transmittersList.length; i++) { + address transmitter = s_transmittersList[i]; + address oldPayee = s_transmitterPayees[transmitter]; + address newPayee = payees[i]; + + if ( + (newPayee == ZERO_ADDRESS) || (oldPayee != ZERO_ADDRESS && oldPayee != newPayee && newPayee != IGNORE_ADDRESS) + ) { + revert InvalidPayee(); + } + + if (newPayee != IGNORE_ADDRESS) { + s_transmitterPayees[transmitter] = newPayee; + } + } + emit PayeesUpdated(s_transmittersList, payees); + } + + /** + * @notice sets the migration permission for a peer registry + * @dev this must be done before upkeeps can be migrated to/from another registry + */ + function setPeerRegistryMigrationPermission(address peer, MigrationPermission permission) external onlyOwner { + s_peerRegistryMigrationPermission[peer] = permission; + } + + /** + * @notice pauses the entire registry + */ + function pause() external onlyOwner { + s_hotVars.paused = true; + emit Paused(msg.sender); + } + + /** + * @notice unpauses the entire registry + */ + function unpause() external onlyOwner { + s_hotVars.paused = false; + emit Unpaused(msg.sender); + } + + /** + * @notice sets a generic bytes field used to indicate the privilege that this admin address had + * @param admin the address to set privilege for + * @param newPrivilegeConfig the privileges that this admin has + */ + function setAdminPrivilegeConfig(address admin, bytes calldata newPrivilegeConfig) external { + _onlyPrivilegeManagerAllowed(); + s_adminPrivilegeConfig[admin] = newPrivilegeConfig; + emit AdminPrivilegeConfigSet(admin, newPrivilegeConfig); + } + + /** + * @notice settles NOPs' LINK payment offchain + */ + function settleNOPsOffchain() external { + _onlyFinanceAdminAllowed(); + if (s_payoutMode == PayoutMode.ON_CHAIN) revert MustSettleOnchain(); + + uint96 totalPremium = s_hotVars.totalPremium; + uint256 activeTransmittersLength = s_transmittersList.length; + uint256 deactivatedTransmittersLength = s_deactivatedTransmitters.length(); + uint256 length = activeTransmittersLength + deactivatedTransmittersLength; + uint256[] memory payments = new uint256[](length); + address[] memory payees = new address[](length); + + for (uint256 i = 0; i < activeTransmittersLength; i++) { + address transmitterAddr = s_transmittersList[i]; + uint96 balance = _updateTransmitterBalanceFromPool( + transmitterAddr, + totalPremium, + uint96(activeTransmittersLength) + ); + + payments[i] = balance; + payees[i] = s_transmitterPayees[transmitterAddr]; + s_transmitters[transmitterAddr].balance = 0; + } + + for (uint256 i = 0; i < deactivatedTransmittersLength; i++) { + address deactivatedAddr = s_deactivatedTransmitters.at(i); + Transmitter memory transmitter = s_transmitters[deactivatedAddr]; + + payees[i + activeTransmittersLength] = s_transmitterPayees[deactivatedAddr]; + payments[i + activeTransmittersLength] = transmitter.balance; + s_transmitters[deactivatedAddr].balance = 0; + } + + // reserve amount of LINK is reset to 0 since no user deposits of LINK are expected in offchain mode + s_reserveAmounts[IERC20(address(i_link))] = 0; + + for (uint256 idx = s_deactivatedTransmitters.length(); idx > 0; idx--) { + s_deactivatedTransmitters.remove(s_deactivatedTransmitters.at(idx - 1)); + } + + emit NOPsSettledOffchain(payees, payments); + } + + /** + * @notice disables offchain payment for NOPs + */ + function disableOffchainPayments() external onlyOwner { + s_payoutMode = PayoutMode.ON_CHAIN; + } + + // ================================================================ + // | GETTERS | + // ================================================================ + + function getConditionalGasOverhead() external pure returns (uint256) { + return REGISTRY_CONDITIONAL_OVERHEAD; + } + + function getLogGasOverhead() external pure returns (uint256) { + return REGISTRY_LOG_OVERHEAD; + } + + function getPerPerformByteGasOverhead() external pure returns (uint256) { + return REGISTRY_PER_PERFORM_BYTE_GAS_OVERHEAD; + } + + function getPerSignerGasOverhead() external pure returns (uint256) { + return REGISTRY_PER_SIGNER_GAS_OVERHEAD; + } + + function getTransmitCalldataFixedBytesOverhead() external pure returns (uint256) { + return TRANSMIT_CALLDATA_FIXED_BYTES_OVERHEAD; + } + + function getTransmitCalldataPerSignerBytesOverhead() external pure returns (uint256) { + return TRANSMIT_CALLDATA_PER_SIGNER_BYTES_OVERHEAD; + } + + function getCancellationDelay() external pure returns (uint256) { + return CANCELLATION_DELAY; + } + + function getLinkAddress() external view returns (address) { + return address(i_link); + } + + function getLinkUSDFeedAddress() external view returns (address) { + return address(i_linkUSDFeed); + } + + function getNativeUSDFeedAddress() external view returns (address) { + return address(i_nativeUSDFeed); + } + + function getFastGasFeedAddress() external view returns (address) { + return address(i_fastGasFeed); + } + + function getAutomationForwarderLogic() external view returns (address) { + return i_automationForwarderLogic; + } + + function getAllowedReadOnlyAddress() external view returns (address) { + return i_allowedReadOnlyAddress; + } + + function getWrappedNativeTokenAddress() external view returns (address) { + return address(i_wrappedNativeToken); + } + + function getBillingToken(uint256 upkeepID) external view returns (IERC20) { + return s_upkeep[upkeepID].billingToken; + } + + function getBillingTokens() external view returns (IERC20[] memory) { + return s_billingTokens; + } + + function supportsBillingToken(IERC20 token) external view returns (bool) { + return address(s_billingConfigs[token].priceFeed) != address(0); + } + + function getBillingTokenConfig(IERC20 token) external view returns (BillingConfig memory) { + return s_billingConfigs[token]; + } + + function getBillingOverridesEnabled(uint256 upkeepID) external view returns (bool) { + return s_upkeep[upkeepID].overridesEnabled; + } + + function getPayoutMode() external view returns (PayoutMode) { + return s_payoutMode; + } + + function upkeepVersion() public pure returns (uint8) { + return UPKEEP_VERSION_BASE; + } + + /** + * @notice gets the number of upkeeps on the registry + */ + function getNumUpkeeps() external view returns (uint256) { + return s_upkeepIDs.length(); + } + + /** + * @notice read all of the details about an upkeep + * @dev this function may be deprecated in a future version of automation in favor of individual + * getters for each field + */ + function getUpkeep(uint256 id) external view returns (IAutomationV21PlusCommon.UpkeepInfoLegacy memory upkeepInfo) { + Upkeep memory reg = s_upkeep[id]; + address target = address(reg.forwarder) == address(0) ? address(0) : reg.forwarder.getTarget(); + upkeepInfo = IAutomationV21PlusCommon.UpkeepInfoLegacy({ + target: target, + performGas: reg.performGas, + checkData: s_checkData[id], + balance: reg.balance, + admin: s_upkeepAdmin[id], + maxValidBlocknumber: reg.maxValidBlocknumber, + lastPerformedBlockNumber: reg.lastPerformedBlockNumber, + amountSpent: uint96(reg.amountSpent), // force casting to uint96 for backwards compatibility. Not an issue if it overflows. + paused: reg.paused, + offchainConfig: s_upkeepOffchainConfig[id] + }); + return upkeepInfo; + } + + /** + * @notice retrieve active upkeep IDs. Active upkeep is defined as an upkeep which is not paused and not canceled. + * @param startIndex starting index in list + * @param maxCount max count to retrieve (0 = unlimited) + * @dev the order of IDs in the list is **not guaranteed**, therefore, if making successive calls, one + * should consider keeping the blockheight constant to ensure a holistic picture of the contract state + */ + function getActiveUpkeepIDs(uint256 startIndex, uint256 maxCount) external view returns (uint256[] memory) { + uint256 numUpkeeps = s_upkeepIDs.length(); + if (startIndex >= numUpkeeps) revert IndexOutOfRange(); + uint256 endIndex = startIndex + maxCount; + endIndex = endIndex > numUpkeeps || maxCount == 0 ? numUpkeeps : endIndex; + uint256[] memory ids = new uint256[](endIndex - startIndex); + for (uint256 idx = 0; idx < ids.length; idx++) { + ids[idx] = s_upkeepIDs.at(idx + startIndex); + } + return ids; + } + + /** + * @notice returns the upkeep's trigger type + */ + function getTriggerType(uint256 upkeepId) external pure returns (Trigger) { + return _getTriggerType(upkeepId); + } + + /** + * @notice returns the trigger config for an upkeeep + */ + function getUpkeepTriggerConfig(uint256 upkeepId) public view returns (bytes memory) { + return s_upkeepTriggerConfig[upkeepId]; + } + + /** + * @notice read the current info about any transmitter address + */ + function getTransmitterInfo( + address query + ) external view returns (bool active, uint8 index, uint96 balance, uint96 lastCollected, address payee) { + Transmitter memory transmitter = s_transmitters[query]; + + uint96 pooledShare = 0; + if (transmitter.active) { + uint96 totalDifference = s_hotVars.totalPremium - transmitter.lastCollected; + pooledShare = totalDifference / uint96(s_transmittersList.length); + } + + return ( + transmitter.active, + transmitter.index, + (transmitter.balance + pooledShare), + transmitter.lastCollected, + s_transmitterPayees[query] + ); + } + + /** + * @notice read the current info about any signer address + */ + function getSignerInfo(address query) external view returns (bool active, uint8 index) { + Signer memory signer = s_signers[query]; + return (signer.active, signer.index); + } + + /** + * @notice read the current on-chain config of the registry + * @dev this function will change between versions, it should never be used where + * backwards compatibility matters! + */ + function getConfig() external view returns (OnchainConfig memory) { + return + OnchainConfig({ + checkGasLimit: s_storage.checkGasLimit, + stalenessSeconds: s_hotVars.stalenessSeconds, + gasCeilingMultiplier: s_hotVars.gasCeilingMultiplier, + maxPerformGas: s_storage.maxPerformGas, + maxCheckDataSize: s_storage.maxCheckDataSize, + maxPerformDataSize: s_storage.maxPerformDataSize, + maxRevertDataSize: s_storage.maxRevertDataSize, + fallbackGasPrice: s_fallbackGasPrice, + fallbackLinkPrice: s_fallbackLinkPrice, + fallbackNativePrice: s_fallbackNativePrice, + transcoder: s_storage.transcoder, + registrars: s_registrars.values(), + upkeepPrivilegeManager: s_storage.upkeepPrivilegeManager, + chainModule: s_hotVars.chainModule, + reorgProtectionEnabled: s_hotVars.reorgProtectionEnabled, + financeAdmin: s_storage.financeAdmin + }); + } + + /** + * @notice read the current state of the registry + * @dev this function is deprecated + */ + function getState() + external + view + returns ( + IAutomationV21PlusCommon.StateLegacy memory state, + IAutomationV21PlusCommon.OnchainConfigLegacy memory config, + address[] memory signers, + address[] memory transmitters, + uint8 f + ) + { + state = IAutomationV21PlusCommon.StateLegacy({ + nonce: s_storage.nonce, + ownerLinkBalance: 0, // deprecated + expectedLinkBalance: 0, // deprecated + totalPremium: s_hotVars.totalPremium, + numUpkeeps: s_upkeepIDs.length(), + configCount: s_storage.configCount, + latestConfigBlockNumber: s_storage.latestConfigBlockNumber, + latestConfigDigest: s_latestConfigDigest, + latestEpoch: s_hotVars.latestEpoch, + paused: s_hotVars.paused + }); + + config = IAutomationV21PlusCommon.OnchainConfigLegacy({ + paymentPremiumPPB: 0, // deprecated + flatFeeMicroLink: 0, // deprecated + checkGasLimit: s_storage.checkGasLimit, + stalenessSeconds: s_hotVars.stalenessSeconds, + gasCeilingMultiplier: s_hotVars.gasCeilingMultiplier, + minUpkeepSpend: 0, // deprecated + maxPerformGas: s_storage.maxPerformGas, + maxCheckDataSize: s_storage.maxCheckDataSize, + maxPerformDataSize: s_storage.maxPerformDataSize, + maxRevertDataSize: s_storage.maxRevertDataSize, + fallbackGasPrice: s_fallbackGasPrice, + fallbackLinkPrice: s_fallbackLinkPrice, + transcoder: s_storage.transcoder, + registrars: s_registrars.values(), + upkeepPrivilegeManager: s_storage.upkeepPrivilegeManager + }); + + return (state, config, s_signersList, s_transmittersList, s_hotVars.f); + } + + /** + * @notice read the Storage data + * @dev this function signature will change with each version of automation + * this should not be treated as a stable function + */ + function getStorage() external view returns (Storage memory) { + return s_storage; + } + + /** + * @notice read the HotVars data + * @dev this function signature will change with each version of automation + * this should not be treated as a stable function + */ + function getHotVars() external view returns (HotVars memory) { + return s_hotVars; + } + + /** + * @notice get the chain module + */ + function getChainModule() external view returns (IChainModule chainModule) { + return s_hotVars.chainModule; + } + + /** + * @notice if this registry has reorg protection enabled + */ + function getReorgProtectionEnabled() external view returns (bool reorgProtectionEnabled) { + return s_hotVars.reorgProtectionEnabled; + } + + /** + * @notice calculates the minimum balance required for an upkeep to remain eligible + * @param id the upkeep id to calculate minimum balance for + */ + function getBalance(uint256 id) external view returns (uint96 balance) { + return s_upkeep[id].balance; + } + + /** + * @notice calculates the minimum balance required for an upkeep to remain eligible + * @param id the upkeep id to calculate minimum balance for + */ + function getMinBalance(uint256 id) external view returns (uint96) { + return getMinBalanceForUpkeep(id); + } + + /** + * @notice calculates the minimum balance required for an upkeep to remain eligible + * @param id the upkeep id to calculate minimum balance for + * @dev this will be deprecated in a future version in favor of getMinBalance + */ + function getMinBalanceForUpkeep(uint256 id) public view returns (uint96 minBalance) { + Upkeep memory upkeep = s_upkeep[id]; + return getMaxPaymentForGas(id, _getTriggerType(id), upkeep.performGas, upkeep.billingToken); + } + + /** + * @notice calculates the maximum payment for a given gas limit + * @param gasLimit the gas to calculate payment for + */ + function getMaxPaymentForGas( + uint256 id, + Trigger triggerType, + uint32 gasLimit, + IERC20 billingToken + ) public view returns (uint96 maxPayment) { + HotVars memory hotVars = s_hotVars; + (uint256 fastGasWei, uint256 linkUSD, uint256 nativeUSD) = _getFeedData(hotVars); + return _getMaxPayment(id, hotVars, triggerType, gasLimit, fastGasWei, linkUSD, nativeUSD, billingToken); + } + + /** + * @notice retrieves the migration permission for a peer registry + */ + function getPeerRegistryMigrationPermission(address peer) external view returns (MigrationPermission) { + return s_peerRegistryMigrationPermission[peer]; + } + + /** + * @notice returns the upkeep privilege config + */ + function getUpkeepPrivilegeConfig(uint256 upkeepId) external view returns (bytes memory) { + return s_upkeepPrivilegeConfig[upkeepId]; + } + + /** + * @notice returns the admin's privilege config + */ + function getAdminPrivilegeConfig(address admin) external view returns (bytes memory) { + return s_adminPrivilegeConfig[admin]; + } + + /** + * @notice returns the upkeep's forwarder contract + */ + function getForwarder(uint256 upkeepID) external view returns (IAutomationForwarder) { + return s_upkeep[upkeepID].forwarder; + } + + /** + * @notice returns if the dedupKey exists or not + */ + function hasDedupKey(bytes32 dedupKey) external view returns (bool) { + return s_dedupKeys[dedupKey]; + } + + /** + * @notice returns the fallback native price + */ + function getFallbackNativePrice() external view returns (uint256) { + return s_fallbackNativePrice; + } + + /** + * @notice returns the amount of a particular token that is reserved as + * user deposits / NOP payments + */ + function getReserveAmount(IERC20 billingToken) external view returns (uint256) { + return s_reserveAmounts[billingToken]; + } + + /** + * @notice returns the amount of a particular token that is withdraw-able by finance admin + */ + function getAvailableERC20ForPayment(IERC20 billingToken) external view returns (uint256) { + return billingToken.balanceOf(address(this)) - s_reserveAmounts[IERC20(address(billingToken))]; + } + + /** + * @notice returns the size of the LINK liquidity pool + */ + function linkAvailableForPayment() public view returns (int256) { + return _linkAvailableForPayment(); + } + + /** + * @notice returns the BillingOverrides config for a given upkeep + */ + function getBillingOverrides(uint256 upkeepID) external view returns (BillingOverrides memory) { + return s_billingOverrides[upkeepID]; + } + + /** + * @notice returns the BillingConfig for a given billing token, this includes decimals and price feed etc + */ + function getBillingConfig(IERC20 billingToken) external view returns (BillingConfig memory) { + return s_billingConfigs[billingToken]; + } + + /** + * @notice returns all active transmitters with their associated payees + */ + function getTransmittersWithPayees() external view returns (TransmitterPayeeInfo[] memory) { + uint256 transmitterCount = s_transmittersList.length; + TransmitterPayeeInfo[] memory transmitters = new TransmitterPayeeInfo[](transmitterCount); + + for (uint256 i = 0; i < transmitterCount; i++) { + address transmitterAddress = s_transmittersList[i]; + address payeeAddress = s_transmitterPayees[transmitterAddress]; + + transmitters[i] = TransmitterPayeeInfo(transmitterAddress, payeeAddress); + } + + return transmitters; + } +} diff --git a/contracts/test/v0.8/automation/AutomationRegistry2_3.test.ts b/contracts/test/v0.8/automation/AutomationRegistry2_3.test.ts index 9a572269695..f993271fbbc 100644 --- a/contracts/test/v0.8/automation/AutomationRegistry2_3.test.ts +++ b/contracts/test/v0.8/automation/AutomationRegistry2_3.test.ts @@ -25,7 +25,6 @@ import { ChainModuleBase__factory as ChainModuleBaseFactory } from '../../../typ import { ArbitrumModule__factory as ArbitrumModuleFactory } from '../../../typechain/factories/ArbitrumModule__factory' import { OptimismModule__factory as OptimismModuleFactory } from '../../../typechain/factories/OptimismModule__factory' import { ILogAutomation__factory as ILogAutomationactory } from '../../../typechain/factories/ILogAutomation__factory' -import { IAutomationForwarder__factory as IAutomationForwarderFactory } from '../../../typechain/factories/IAutomationForwarder__factory' import { MockArbSys__factory as MockArbSysFactory } from '../../../typechain/factories/MockArbSys__factory' import { AutomationCompatibleUtils } from '../../../typechain/AutomationCompatibleUtils' import { MockArbGasInfo } from '../../../typechain/MockArbGasInfo' diff --git a/contracts/test/v0.8/automation/helpers.ts b/contracts/test/v0.8/automation/helpers.ts index 5a95fb482cd..b2cdfb4efd9 100644 --- a/contracts/test/v0.8/automation/helpers.ts +++ b/contracts/test/v0.8/automation/helpers.ts @@ -170,10 +170,10 @@ export const deployRegistry23 = async ( link: Parameters[0], linkUSD: Parameters[1], nativeUSD: Parameters[2], - fastgas: Parameters[2], + fastgas: Parameters[3], allowedReadOnlyAddress: Parameters< AutomationRegistryLogicC2_3Factory['deploy'] - >[3], + >[5], payoutMode: Parameters[6], wrappedNativeTokenAddress: Parameters< AutomationRegistryLogicC2_3Factory['deploy']