Skip to content

Commit

Permalink
offchain - import chainlink-common (#624)
Browse files Browse the repository at this point in the history
Replacing local `cciptypes` package with `ccip/types` from
`chainlink-common`.
This change required:
- Passing `ctx` to all the reader methods.
- Accepting `err` from readers.
- Re-generating mocks.
- Replacing the imports.


Blocked by:
smartcontractkit/chainlink-common#409
(Update chainlink-common tag after 409 gets merged)
  • Loading branch information
dimkouv authored Mar 20, 2024
1 parent a88c392 commit 05a0c86
Show file tree
Hide file tree
Showing 107 changed files with 554 additions and 878 deletions.
12 changes: 5 additions & 7 deletions core/services/ocr2/plugins/ccip/ccipcommit/factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,8 +8,8 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/smartcontractkit/libocr/offchainreporting2plus/types"

cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/ccipcalc"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/ccipdata"
)
Expand Down Expand Up @@ -37,7 +37,7 @@ func NewCommitReportingPluginFactory(config CommitPluginStaticConfig) *CommitRep
}
}

func (rf *CommitReportingPluginFactory) UpdateDynamicReaders(newPriceRegAddr common.Address) error {
func (rf *CommitReportingPluginFactory) UpdateDynamicReaders(ctx context.Context, newPriceRegAddr common.Address) error {
rf.readersMu.Lock()
defer rf.readersMu.Unlock()
// TODO: Investigate use of Close() to cleanup.
Expand Down Expand Up @@ -65,6 +65,8 @@ func (rf *CommitReportingPluginFactory) UpdateDynamicReaders(newPriceRegAddr com

// NewReportingPlugin returns the ccip CommitReportingPlugin and satisfies the ReportingPluginFactory interface.
func (rf *CommitReportingPluginFactory) NewReportingPlugin(config types.ReportingPluginConfig) (types.ReportingPlugin, types.ReportingPluginInfo, error) {
ctx := context.Background() // todo: consider adding some timeout

destPriceReg, err := rf.config.commitStore.ChangeConfig(config.OnchainConfig, config.OffchainConfig)
if err != nil {
return nil, types.ReportingPluginInfo{}, err
Expand All @@ -74,11 +76,7 @@ func (rf *CommitReportingPluginFactory) NewReportingPlugin(config types.Reportin
if err != nil {
return nil, types.ReportingPluginInfo{}, err
}
if err = rf.UpdateDynamicReaders(priceRegEvmAddr); err != nil {
return nil, types.ReportingPluginInfo{}, err
}

if err != nil {
if err = rf.UpdateDynamicReaders(ctx, priceRegEvmAddr); err != nil {
return nil, types.ReportingPluginInfo{}, err
}

Expand Down
2 changes: 1 addition & 1 deletion core/services/ocr2/plugins/ccip/ccipcommit/inflight.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,8 +6,8 @@ import (

"github.com/ethereum/go-ethereum/common/hexutil"

cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
"github.com/smartcontractkit/chainlink/v2/core/logger"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
)

const (
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,9 +10,9 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"

cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
"github.com/smartcontractkit/chainlink/v2/core/chains/evm/utils"
"github.com/smartcontractkit/chainlink/v2/core/logger"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/ccipcalc"
)

Expand Down
4 changes: 2 additions & 2 deletions core/services/ocr2/plugins/ccip/ccipcommit/initializers.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ import (

commonlogger "github.com/smartcontractkit/chainlink-common/pkg/logger"

"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/cache"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/ccipcalc"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/ccipdata/ccipdataprovider"
Expand Down Expand Up @@ -87,7 +87,7 @@ func CommitReportToEthTxMeta(typ ccipconfig.ContractType, ver semver.Version) (f
// https://github.com/smartcontractkit/ccip/blob/68e2197472fb017dd4e5630d21e7878d58bc2a44/core/services/feeds/service.go#L716
// TODO once that transaction is broken up, we should be able to simply rely on oracle.Close() to cleanup the filters.
// Until then we have to deterministically reload the readers from the spec (and thus their filters) and close them.
func UnregisterCommitPluginLpFilters(lggr logger.Logger, jb job.Job, chainSet legacyevm.LegacyChainContainer, qopts ...pg.QOpt) error {
func UnregisterCommitPluginLpFilters(ctx context.Context, lggr logger.Logger, jb job.Job, chainSet legacyevm.LegacyChainContainer, qopts ...pg.QOpt) error {
params, err := extractJobSpecParams(jb, chainSet)
if err != nil {
return err
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import (

"github.com/smartcontractkit/chainlink/v2/core/chains/evm/utils"
legacyEvmORMMocks "github.com/smartcontractkit/chainlink/v2/core/chains/legacyevm/mocks"
"github.com/smartcontractkit/chainlink/v2/core/internal/testutils"
"github.com/smartcontractkit/chainlink/v2/core/logger"
"github.com/smartcontractkit/chainlink/v2/core/services/job"
)
Expand Down Expand Up @@ -53,6 +54,8 @@ func TestGetCommitPluginFilterNamesFromSpec(t *testing.T) {
},
}

ctx := testutils.Context(t)

for _, tc := range testCases {
t.Run(tc.description, func(t *testing.T) {
chainSet := &legacyEvmORMMocks.LegacyChainContainer{}
Expand All @@ -65,7 +68,7 @@ func TestGetCommitPluginFilterNamesFromSpec(t *testing.T) {
}
}

err := UnregisterCommitPluginLpFilters(lggr, job.Job{OCR2OracleSpec: tc.spec}, chainSet)
err := UnregisterCommitPluginLpFilters(ctx, lggr, job.Job{OCR2OracleSpec: tc.spec}, chainSet)
if tc.expectingErr {
assert.Error(t, err)
} else {
Expand Down
2 changes: 1 addition & 1 deletion core/services/ocr2/plugins/ccip/ccipcommit/ocr2.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ import (

"github.com/smartcontractkit/chainlink-common/pkg/utils/mathutil"

"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/cache"

"github.com/smartcontractkit/chainlink/v2/core/logger"
Expand Down
2 changes: 1 addition & 1 deletion core/services/ocr2/plugins/ccip/ccipcommit/ocr2_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (

"github.com/smartcontractkit/chainlink-common/pkg/config"

cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
"github.com/smartcontractkit/chainlink/v2/core/chains/evm/gas/mocks"
mocks2 "github.com/smartcontractkit/chainlink/v2/core/chains/evm/logpoller/mocks"
"github.com/smartcontractkit/chainlink/v2/core/chains/evm/utils"
Expand All @@ -31,7 +32,6 @@ import (
"github.com/smartcontractkit/chainlink/v2/core/logger"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/abihelpers"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
ccipconfig "github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/config"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/cache"
ccipcachemocks "github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/cache/mocks"
Expand Down
2 changes: 1 addition & 1 deletion core/services/ocr2/plugins/ccip/ccipexec/batching.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,9 +6,9 @@ import (

"github.com/pkg/errors"

cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/abihelpers"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/ccipdata"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/hashlib"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/merklemulti"
Expand Down
2 changes: 1 addition & 1 deletion core/services/ocr2/plugins/ccip/ccipexec/batching_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ import (

"github.com/stretchr/testify/assert"

"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
)

func Test_validateSendRequests(t *testing.T) {
Expand Down
33 changes: 25 additions & 8 deletions core/services/ocr2/plugins/ccip/ccipexec/factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -2,12 +2,13 @@ package ccipexec

import (
"context"
"fmt"
"sync"

"github.com/smartcontractkit/libocr/offchainreporting2plus/types"

cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/cache"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/ccipdata"
)
Expand All @@ -32,7 +33,7 @@ func NewExecutionReportingPluginFactory(config ExecutionPluginStaticConfig) *Exe
}
}

func (rf *ExecutionReportingPluginFactory) UpdateDynamicReaders(newPriceRegAddr cciptypes.Address) error {
func (rf *ExecutionReportingPluginFactory) UpdateDynamicReaders(ctx context.Context, newPriceRegAddr cciptypes.Address) error {
rf.readersMu.Lock()
defer rf.readersMu.Unlock()
// TODO: Investigate use of Close() to cleanup.
Expand All @@ -59,35 +60,51 @@ func (rf *ExecutionReportingPluginFactory) UpdateDynamicReaders(newPriceRegAddr
}

func (rf *ExecutionReportingPluginFactory) NewReportingPlugin(config types.ReportingPluginConfig) (types.ReportingPlugin, types.ReportingPluginInfo, error) {
destPriceRegistry, destWrappedNative, err := rf.config.offRampReader.ChangeConfig(config.OnchainConfig, config.OffchainConfig)
ctx := context.Background() // todo: consider setting a timeout

destPriceRegistry, destWrappedNative, err := rf.config.offRampReader.ChangeConfig(ctx, config.OnchainConfig, config.OffchainConfig)
if err != nil {
return nil, types.ReportingPluginInfo{}, err
}
// Open dynamic readers
err = rf.UpdateDynamicReaders(destPriceRegistry)
err = rf.UpdateDynamicReaders(ctx, destPriceRegistry)
if err != nil {
return nil, types.ReportingPluginInfo{}, err
}

offchainConfig := rf.config.offRampReader.OffchainConfig()
offchainConfig, err := rf.config.offRampReader.OffchainConfig(ctx)
if err != nil {
return nil, types.ReportingPluginInfo{}, fmt.Errorf("get offchain config from offramp: %w", err)
}

gasPriceEstimator, err := rf.config.offRampReader.GasPriceEstimator(ctx)
if err != nil {
return nil, types.ReportingPluginInfo{}, fmt.Errorf("get gas price estimator from offramp: %w", err)
}

onchainConfig, err := rf.config.offRampReader.OnchainConfig(ctx)
if err != nil {
return nil, types.ReportingPluginInfo{}, fmt.Errorf("get onchain config from offramp: %w", err)
}

return &ExecutionReportingPlugin{
F: config.F,
lggr: rf.config.lggr.Named("ExecutionReportingPlugin"),
offchainConfig: offchainConfig,
tokenDataWorker: rf.config.tokenDataWorker,
gasPriceEstimator: rf.config.offRampReader.GasPriceEstimator(),
gasPriceEstimator: gasPriceEstimator,
sourcePriceRegistryProvider: rf.config.sourcePriceRegistryProvider,
sourcePriceRegistryLock: sync.RWMutex{},
sourceWrappedNativeToken: rf.config.sourceWrappedNativeToken,
onRampReader: rf.config.onRampReader,
commitStoreReader: rf.config.commitStoreReader,
destPriceRegistry: rf.destPriceRegReader,
destWrappedNative: destWrappedNative,
onchainConfig: rf.config.offRampReader.OnchainConfig(),
onchainConfig: onchainConfig,
offRampReader: rf.config.offRampReader,
tokenPoolBatchedReader: rf.config.tokenPoolBatchedReader,
inflightReports: newInflightExecReportsContainer(offchainConfig.InflightCacheExpiry.Duration()),
snoozedRoots: cache.NewSnoozedRoots(rf.config.offRampReader.OnchainConfig().PermissionLessExecutionThresholdSeconds, offchainConfig.RootSnoozeTime.Duration()),
snoozedRoots: cache.NewSnoozedRoots(onchainConfig.PermissionLessExecutionThresholdSeconds, offchainConfig.RootSnoozeTime.Duration()),
metricsCollector: rf.config.metricsCollector,
chainHealthcheck: rf.config.chainHealthcheck,
}, types.ReportingPluginInfo{
Expand Down
2 changes: 1 addition & 1 deletion core/services/ocr2/plugins/ccip/ccipexec/inflight.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,8 +6,8 @@ import (

"github.com/pkg/errors"

cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
"github.com/smartcontractkit/chainlink/v2/core/logger"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
)

// InflightInternalExecutionReport serves the same purpose as InflightCommitReport
Expand Down
2 changes: 1 addition & 1 deletion core/services/ocr2/plugins/ccip/ccipexec/inflight_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,8 +6,8 @@ import (

"github.com/stretchr/testify/require"

cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
"github.com/smartcontractkit/chainlink/v2/core/logger"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
)

func TestInflightReportsContainer_add(t *testing.T) {
Expand Down
37 changes: 26 additions & 11 deletions core/services/ocr2/plugins/ccip/ccipexec/initializers.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ import (

commonlogger "github.com/smartcontractkit/chainlink-common/pkg/logger"

"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/cciptypes"
cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccip"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/cache"
"github.com/smartcontractkit/chainlink/v2/core/services/ocr2/plugins/ccip/internal/ccipcalc"

Expand Down Expand Up @@ -46,8 +46,8 @@ import (

const numTokenDataWorkers = 5

func NewExecutionServices(lggr logger.Logger, jb job.Job, chainSet legacyevm.LegacyChainContainer, new bool, argsNoPlugin libocr2.OCR2OracleArgs, logError func(string), qopts ...pg.QOpt) ([]job.ServiceCtx, error) {
execPluginConfig, backfillArgs, chainHealthcheck, tokenWorker, err := jobSpecToExecPluginConfig(lggr, jb, chainSet, qopts...)
func NewExecutionServices(ctx context.Context, lggr logger.Logger, jb job.Job, chainSet legacyevm.LegacyChainContainer, new bool, argsNoPlugin libocr2.OCR2OracleArgs, logError func(string), qopts ...pg.QOpt) ([]job.ServiceCtx, error) {
execPluginConfig, backfillArgs, chainHealthcheck, tokenWorker, err := jobSpecToExecPluginConfig(ctx, lggr, jb, chainSet, qopts...)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -87,12 +87,17 @@ func NewExecutionServices(lggr logger.Logger, jb job.Job, chainSet legacyevm.Leg
// UnregisterExecPluginLpFilters unregisters all the registered filters for both source and dest chains.
// See comment in UnregisterCommitPluginLpFilters
// It MUST mirror the filters registered in NewExecutionServices.
func UnregisterExecPluginLpFilters(lggr logger.Logger, jb job.Job, chainSet legacyevm.LegacyChainContainer, qopts ...pg.QOpt) error {
func UnregisterExecPluginLpFilters(ctx context.Context, lggr logger.Logger, jb job.Job, chainSet legacyevm.LegacyChainContainer, qopts ...pg.QOpt) error {
params, err := extractJobSpecParams(lggr, jb, chainSet, false, qopts...)
if err != nil {
return err
}

offRampAddress, err := params.offRampReader.Address(ctx)
if err != nil {
return fmt.Errorf("get offramp reader address: %w", err)
}

versionFinder := factory.NewEvmVersionFinder()
unregisterFuncs := []func() error{
func() error {
Expand All @@ -102,7 +107,7 @@ func UnregisterExecPluginLpFilters(lggr logger.Logger, jb job.Job, chainSet lega
return factory.CloseOnRampReader(lggr, versionFinder, params.offRampConfig.SourceChainSelector, params.offRampConfig.ChainSelector, params.offRampConfig.OnRamp, params.sourceChain.LogPoller(), params.sourceChain.Client(), qopts...)
},
func() error {
return factory.CloseOffRampReader(lggr, versionFinder, params.offRampReader.Address(), params.destChain.Client(), params.destChain.LogPoller(), params.destChain.GasEstimator(), params.destChain.Config().EVM().GasEstimator().PriceMax().ToInt(), qopts...)
return factory.CloseOffRampReader(lggr, versionFinder, offRampAddress, params.destChain.Client(), params.destChain.LogPoller(), params.destChain.GasEstimator(), params.destChain.Config().EVM().GasEstimator().PriceMax().ToInt(), qopts...)
},
func() error { // usdc token data reader
if usdcDisabled := params.pluginConfig.USDCConfig.AttestationAPI == ""; usdcDisabled {
Expand All @@ -123,8 +128,8 @@ func UnregisterExecPluginLpFilters(lggr logger.Logger, jb job.Job, chainSet lega

// ExecReportToEthTxMeta generates a txmgr.EthTxMeta from the given report.
// Only MessageIDs will be populated in the TxMeta.
func ExecReportToEthTxMeta(typ ccipconfig.ContractType, ver semver.Version) (func(report []byte) (*txmgr.TxMeta, error), error) {
return factory.ExecReportToEthTxMeta(typ, ver)
func ExecReportToEthTxMeta(ctx context.Context, typ ccipconfig.ContractType, ver semver.Version) (func(report []byte) (*txmgr.TxMeta, error), error) {
return factory.ExecReportToEthTxMeta(ctx, typ, ver)
}

func initTokenDataProviders(lggr logger.Logger, jobID string, pluginConfig ccipconfig.ExecutionPluginJobSpecConfig, sourceLP logpoller.LogPoller, qopts ...pg.QOpt) (map[cciptypes.Address]tokendata.Reader, error) {
Expand Down Expand Up @@ -160,7 +165,7 @@ func initTokenDataProviders(lggr logger.Logger, jobID string, pluginConfig ccipc
return tokenDataProviders, nil
}

func jobSpecToExecPluginConfig(lggr logger.Logger, jb job.Job, chainSet legacyevm.LegacyChainContainer, qopts ...pg.QOpt) (*ExecutionPluginStaticConfig, *ccipcommon.BackfillArgs, *cache.ObservedChainHealthcheck, *tokendata.BackgroundWorker, error) {
func jobSpecToExecPluginConfig(ctx context.Context, lggr logger.Logger, jb job.Job, chainSet legacyevm.LegacyChainContainer, qopts ...pg.QOpt) (*ExecutionPluginStaticConfig, *ccipcommon.BackfillArgs, *cache.ObservedChainHealthcheck, *tokendata.BackgroundWorker, error) {
params, err := extractJobSpecParams(lggr, jb, chainSet, true, qopts...)
if err != nil {
return nil, nil, nil, nil, err
Expand Down Expand Up @@ -238,7 +243,12 @@ func jobSpecToExecPluginConfig(lggr logger.Logger, jb job.Job, chainSet legacyev

batchCaller := rpclib.NewDynamicLimitedBatchCaller(lggr, params.destChain.Client(), rpclib.DefaultRpcBatchSizeLimit, rpclib.DefaultRpcBatchBackOffMultiplier)

tokenPoolBatchedReader, err := batchreader.NewEVMTokenPoolBatchedReader(execLggr, sourceChainSelector, offRampReader.Address(), batchCaller)
offrampAddress, err := offRampReader.Address(ctx)
if err != nil {
return nil, nil, nil, nil, fmt.Errorf("get offramp reader address: %w", err)
}

tokenPoolBatchedReader, err := batchreader.NewEVMTokenPoolBatchedReader(execLggr, sourceChainSelector, offrampAddress, batchCaller)
if err != nil {
return nil, nil, nil, nil, fmt.Errorf("new token pool batched reader: %w", err)
}
Expand All @@ -250,7 +260,7 @@ func jobSpecToExecPluginConfig(lggr logger.Logger, jb job.Job, chainSet legacyev
lggr.With(
"onramp", params.offRampConfig.OnRamp,
"commitStore", params.offRampConfig.CommitStore,
"offramp", params.offRampReader.Address(),
"offramp", offrampAddress,
),
onRampReader,
commitStoreReader,
Expand All @@ -261,11 +271,16 @@ func jobSpecToExecPluginConfig(lggr logger.Logger, jb job.Job, chainSet legacyev
params.offRampConfig.OnRamp,
)

onchainConfig, err := offRampReader.OnchainConfig(ctx)
if err != nil {
return nil, nil, nil, nil, fmt.Errorf("get onchain config from offramp reader: %w", err)
}

tokenBackgroundWorker := tokendata.NewBackgroundWorker(
tokenDataProviders,
numTokenDataWorkers,
5*time.Second,
offRampReader.OnchainConfig().PermissionLessExecutionThresholdSeconds,
onchainConfig.PermissionLessExecutionThresholdSeconds,
)
return &ExecutionPluginStaticConfig{
lggr: execLggr,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ import (
"github.com/stretchr/testify/assert"

legacyEvmORMMocks "github.com/smartcontractkit/chainlink/v2/core/chains/legacyevm/mocks"
"github.com/smartcontractkit/chainlink/v2/core/internal/testutils"
"github.com/smartcontractkit/chainlink/v2/core/logger"
"github.com/smartcontractkit/chainlink/v2/core/services/job"
)
Expand Down Expand Up @@ -44,10 +45,12 @@ func TestGetExecutionPluginFilterNamesFromSpec(t *testing.T) {
},
}

ctx := testutils.Context(t)

for _, tc := range testCases {
chainSet := &legacyEvmORMMocks.LegacyChainContainer{}
t.Run(tc.description, func(t *testing.T) {
err := UnregisterExecPluginLpFilters(logger.TestLogger(t), job.Job{OCR2OracleSpec: tc.spec}, chainSet)
err := UnregisterExecPluginLpFilters(ctx, logger.TestLogger(t), job.Job{OCR2OracleSpec: tc.spec}, chainSet)
if tc.expectingErr {
assert.Error(t, err)
} else {
Expand Down
Loading

0 comments on commit 05a0c86

Please sign in to comment.