From 96d2fe13b8510631bbc92ffd20a4d923b93002e6 Mon Sep 17 00:00:00 2001 From: Dmytro Haidashenko <34754799+dhaidashenko@users.noreply.github.com> Date: Tue, 26 Mar 2024 21:42:09 +0100 Subject: [PATCH] latest finalized block metrics (#12339) * Add LatestFinalizedBlock to HeadTracker * Added LatestFinalizedHead to Head * remove unused func * fix flakey nil pointer * improve logs & address lint issue * nitpicks * fixed copy on heads on MarkFinalized * error instead of panic * return error instead of panic * nitpicks * Finalized block based history depth * simplify trimming * nit fixes * fix build issues caused by merge * regen * FIx rpc client mock generation * nit fixes * nit fixes * update comments * ensure that we trim redundant blocks both in slice and in chain in Heads handle corner case for multiple uncle blocks at the end of the slice * nit fix * Update common/headtracker/head_tracker.go Co-authored-by: Dimitris Grigoriou * HeadTracker backfill test with 0 finality depth * latest finalized block metrics * changelog & go generate fix * move nodeConfig back into the test pkg * rollback fields renaming * nit * changeset * removed unused func * Set default value for FinalizedBlockPollInterval * updated docs --------- Co-authored-by: Dimitris Grigoriou --- .changeset/poor-melons-vanish.md | 5 + common/client/mock_head_test.go | 18 ++ common/client/mock_node_client_test.go | 28 +++ common/client/mocks/config.go | 30 +++ common/client/node.go | 34 ++-- common/client/node_lifecycle.go | 45 ++++- common/client/node_lifecycle_test.go | 178 ++++++++++++++++-- common/client/node_test.go | 40 ++-- common/client/types.go | 2 + common/types/head.go | 2 + common/types/mocks/head.go | 18 ++ core/chains/evm/client/config_builder.go | 29 ++- core/chains/evm/client/config_builder_test.go | 38 ++-- core/chains/evm/client/evm_client.go | 10 +- core/chains/evm/client/evm_client_test.go | 9 +- core/chains/evm/client/helpers_test.go | 26 +-- core/chains/evm/config/chain_scoped.go | 142 +++++++------- .../evm/config/chain_scoped_node_pool.go | 4 + core/chains/evm/config/config.go | 1 + core/chains/evm/config/toml/config.go | 17 +- .../evm/config/toml/defaults/fallback.toml | 1 + core/chains/legacyevm/chain.go | 4 +- core/config/docs/chains-evm.toml | 9 +- core/services/chainlink/config_test.go | 14 +- .../chainlink/testdata/config-full.toml | 1 + .../config-multi-chain-effective.toml | 3 + core/web/resolver/testdata/config-full.toml | 1 + .../config-multi-chain-effective.toml | 3 + docs/CONFIG.md | 67 +++++++ .../disk-based-logging-disabled.txtar | 1 + .../validate/disk-based-logging-no-dir.txtar | 1 + .../node/validate/disk-based-logging.txtar | 1 + testdata/scripts/node/validate/invalid.txtar | 1 + testdata/scripts/node/validate/valid.txtar | 1 + 34 files changed, 607 insertions(+), 177 deletions(-) create mode 100644 .changeset/poor-melons-vanish.md create mode 100644 common/client/mocks/config.go diff --git a/.changeset/poor-melons-vanish.md b/.changeset/poor-melons-vanish.md new file mode 100644 index 00000000000..3b6d901b157 --- /dev/null +++ b/.changeset/poor-melons-vanish.md @@ -0,0 +1,5 @@ +--- +"chainlink": minor +--- + +Add the `pool_rpc_node_highest_finalized_block` metric that tracks the highest finalized block seen per RPC. If `FinalityTagEnabled = true`, a positive `NodePool.FinalizedBlockPollInterval` is needed to collect the metric. If the finality tag is not enabled, the metric is populated with a calculated latest finalized block based on the latest head and finality depth. diff --git a/common/client/mock_head_test.go b/common/client/mock_head_test.go index 747770480f5..e68a047e078 100644 --- a/common/client/mock_head_test.go +++ b/common/client/mock_head_test.go @@ -51,6 +51,24 @@ func (_m *mockHead) BlockNumber() int64 { return r0 } +// IsValid provides a mock function with given fields: +func (_m *mockHead) IsValid() bool { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for IsValid") + } + + var r0 bool + if rf, ok := ret.Get(0).(func() bool); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + // newMockHead creates a new instance of mockHead. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. // The first argument is typically a *testing.T value. func newMockHead(t interface { diff --git a/common/client/mock_node_client_test.go b/common/client/mock_node_client_test.go index d143ebb88a5..dfe9f32664a 100644 --- a/common/client/mock_node_client_test.go +++ b/common/client/mock_node_client_test.go @@ -144,6 +144,34 @@ func (_m *mockNodeClient[CHAIN_ID, HEAD]) IsSyncing(ctx context.Context) (bool, return r0, r1 } +// LatestFinalizedBlock provides a mock function with given fields: ctx +func (_m *mockNodeClient[CHAIN_ID, HEAD]) LatestFinalizedBlock(ctx context.Context) (HEAD, error) { + ret := _m.Called(ctx) + + if len(ret) == 0 { + panic("no return value specified for LatestFinalizedBlock") + } + + var r0 HEAD + var r1 error + if rf, ok := ret.Get(0).(func(context.Context) (HEAD, error)); ok { + return rf(ctx) + } + if rf, ok := ret.Get(0).(func(context.Context) HEAD); ok { + r0 = rf(ctx) + } else { + r0 = ret.Get(0).(HEAD) + } + + if rf, ok := ret.Get(1).(func(context.Context) error); ok { + r1 = rf(ctx) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // SetAliveLoopSub provides a mock function with given fields: _a0 func (_m *mockNodeClient[CHAIN_ID, HEAD]) SetAliveLoopSub(_a0 types.Subscription) { _m.Called(_a0) diff --git a/common/client/mocks/config.go b/common/client/mocks/config.go new file mode 100644 index 00000000000..27b717f61a0 --- /dev/null +++ b/common/client/mocks/config.go @@ -0,0 +1,30 @@ +package mocks + +import ( + "time" + + commonconfig "github.com/smartcontractkit/chainlink/v2/common/config" +) + +type ChainConfig struct { + IsFinalityTagEnabled bool + FinalityDepthVal uint32 + NoNewHeadsThresholdVal time.Duration + ChainTypeVal commonconfig.ChainType +} + +func (t ChainConfig) ChainType() commonconfig.ChainType { + return t.ChainTypeVal +} + +func (t ChainConfig) NodeNoNewHeadsThreshold() time.Duration { + return t.NoNewHeadsThresholdVal +} + +func (t ChainConfig) FinalityDepth() uint32 { + return t.FinalityDepthVal +} + +func (t ChainConfig) FinalityTagEnabled() bool { + return t.IsFinalityTagEnabled +} diff --git a/common/client/node.go b/common/client/node.go index 082b1b45f99..61816be21da 100644 --- a/common/client/node.go +++ b/common/client/node.go @@ -15,6 +15,7 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/logger" "github.com/smartcontractkit/chainlink-common/pkg/services" + commonconfig "github.com/smartcontractkit/chainlink/v2/common/config" "github.com/smartcontractkit/chainlink/v2/common/types" ) @@ -43,6 +44,14 @@ type NodeConfig interface { SelectionMode() string SyncThreshold() uint32 NodeIsSyncingEnabled() bool + FinalizedBlockPollInterval() time.Duration +} + +type ChainConfig interface { + NodeNoNewHeadsThreshold() time.Duration + FinalityDepth() uint32 + FinalityTagEnabled() bool + ChainType() commonconfig.ChainType } //go:generate mockery --quiet --name Node --structname mockNode --filename "mock_node_test.go" --inpackage --case=underscore @@ -73,14 +82,14 @@ type node[ RPC NodeClient[CHAIN_ID, HEAD], ] struct { services.StateMachine - lfcLog logger.Logger - name string - id int32 - chainID CHAIN_ID - nodePoolCfg NodeConfig - noNewHeadsThreshold time.Duration - order int32 - chainFamily string + lfcLog logger.Logger + name string + id int32 + chainID CHAIN_ID + nodePoolCfg NodeConfig + chainCfg ChainConfig + order int32 + chainFamily string ws url.URL http *url.URL @@ -90,8 +99,9 @@ type node[ stateMu sync.RWMutex // protects state* fields state nodeState // Each node is tracking the last received head number and total difficulty - stateLatestBlockNumber int64 - stateLatestTotalDifficulty *big.Int + stateLatestBlockNumber int64 + stateLatestTotalDifficulty *big.Int + stateLatestFinalizedBlockNumber int64 // nodeCtx is the node lifetime's context nodeCtx context.Context @@ -113,7 +123,7 @@ func NewNode[ RPC NodeClient[CHAIN_ID, HEAD], ]( nodeCfg NodeConfig, - noNewHeadsThreshold time.Duration, + chainCfg ChainConfig, lggr logger.Logger, wsuri url.URL, httpuri *url.URL, @@ -129,7 +139,7 @@ func NewNode[ n.id = id n.chainID = chainID n.nodePoolCfg = nodeCfg - n.noNewHeadsThreshold = noNewHeadsThreshold + n.chainCfg = chainCfg n.ws = wsuri n.order = nodeOrder if httpuri != nil { diff --git a/common/client/node_lifecycle.go b/common/client/node_lifecycle.go index 20902277480..4707a60426f 100644 --- a/common/client/node_lifecycle.go +++ b/common/client/node_lifecycle.go @@ -22,6 +22,10 @@ var ( Name: "pool_rpc_node_highest_seen_block", Help: "The highest seen block for the given RPC node", }, []string{"chainID", "nodeName"}) + promPoolRPCNodeHighestFinalizedBlock = promauto.NewGaugeVec(prometheus.GaugeOpts{ + Name: "pool_rpc_node_highest_finalized_block", + Help: "The highest seen finalized block for the given RPC node", + }, []string{"chainID", "nodeName"}) promPoolRPCNodeNumSeenBlocks = promauto.NewCounterVec(prometheus.CounterOpts{ Name: "pool_rpc_node_num_seen_blocks", Help: "The total number of new blocks seen by the given RPC node", @@ -88,7 +92,7 @@ func (n *node[CHAIN_ID, HEAD, RPC]) aliveLoop() { } } - noNewHeadsTimeoutThreshold := n.noNewHeadsThreshold + noNewHeadsTimeoutThreshold := n.chainCfg.NodeNoNewHeadsThreshold() pollFailureThreshold := n.nodePoolCfg.PollFailureThreshold() pollInterval := n.nodePoolCfg.PollInterval() @@ -134,6 +138,14 @@ func (n *node[CHAIN_ID, HEAD, RPC]) aliveLoop() { lggr.Debug("Polling disabled") } + var pollFinalizedHeadCh <-chan time.Time + if n.chainCfg.FinalityTagEnabled() && n.nodePoolCfg.FinalizedBlockPollInterval() > 0 { + lggr.Debugw("Finalized block polling enabled") + pollT := time.NewTicker(n.nodePoolCfg.FinalizedBlockPollInterval()) + defer pollT.Stop() + pollFinalizedHeadCh = pollT.C + } + _, highestReceivedBlockNumber, _ := n.StateAndLatest() var pollFailures uint32 @@ -201,6 +213,13 @@ func (n *node[CHAIN_ID, HEAD, RPC]) aliveLoop() { outOfSyncT.Reset(noNewHeadsTimeoutThreshold) } n.setLatestReceived(bh.BlockNumber(), bh.BlockDifficulty()) + if !n.chainCfg.FinalityTagEnabled() { + latestFinalizedBN := max(bh.BlockNumber()-int64(n.chainCfg.FinalityDepth()), 0) + if latestFinalizedBN > n.stateLatestFinalizedBlockNumber { + promPoolRPCNodeHighestFinalizedBlock.WithLabelValues(n.chainID.String(), n.name).Set(float64(latestFinalizedBN)) + n.stateLatestFinalizedBlockNumber = latestFinalizedBN + } + } case err := <-sub.Err(): lggr.Errorw("Subscription was terminated", "err", err, "nodeState", n.State()) n.declareUnreachable() @@ -214,13 +233,33 @@ func (n *node[CHAIN_ID, HEAD, RPC]) aliveLoop() { lggr.Criticalf("RPC endpoint detected out of sync; %s %s", msgCannotDisable, msgDegradedState) // We don't necessarily want to wait the full timeout to check again, we should // check regularly and log noisily in this state - outOfSyncT.Reset(zombieNodeCheckInterval(n.noNewHeadsThreshold)) + outOfSyncT.Reset(zombieNodeCheckInterval(noNewHeadsTimeoutThreshold)) continue } } n.declareOutOfSync(func(num int64, td *big.Int) bool { return num < highestReceivedBlockNumber }) return + case <-pollFinalizedHeadCh: + ctx, cancel := context.WithTimeout(n.nodeCtx, n.nodePoolCfg.FinalizedBlockPollInterval()) + latestFinalized, err := n.RPC().LatestFinalizedBlock(ctx) + cancel() + if err != nil { + lggr.Warnw("Failed to fetch latest finalized block", "err", err) + continue + } + + if !latestFinalized.IsValid() { + lggr.Warn("Latest finalized block is not valid") + continue + } + + latestFinalizedBN := latestFinalized.BlockNumber() + if latestFinalizedBN > n.stateLatestFinalizedBlockNumber { + promPoolRPCNodeHighestFinalizedBlock.WithLabelValues(n.chainID.String(), n.name).Set(float64(latestFinalizedBN)) + n.stateLatestFinalizedBlockNumber = latestFinalizedBN + } } + } } @@ -316,7 +355,7 @@ func (n *node[CHAIN_ID, HEAD, RPC]) outOfSyncLoop(isOutOfSync func(num int64, td return } lggr.Debugw(msgReceivedBlock, "blockNumber", head.BlockNumber(), "blockDifficulty", head.BlockDifficulty(), "nodeState", n.State()) - case <-time.After(zombieNodeCheckInterval(n.noNewHeadsThreshold)): + case <-time.After(zombieNodeCheckInterval(n.chainCfg.NodeNoNewHeadsThreshold())): if n.nLiveNodes != nil { if l, _, _ := n.nLiveNodes(); l < 1 { lggr.Critical("RPC endpoint is still out of sync, but there are no other available nodes. This RPC node will be forcibly moved back into the live pool in a degraded state") diff --git a/common/client/node_lifecycle_test.go b/common/client/node_lifecycle_test.go index 437bc4a655b..b3c09b35000 100644 --- a/common/client/node_lifecycle_test.go +++ b/common/client/node_lifecycle_test.go @@ -8,6 +8,7 @@ import ( "testing" "github.com/cometbft/cometbft/libs/rand" + prom "github.com/prometheus/client_model/go" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -17,6 +18,7 @@ import ( bigmath "github.com/smartcontractkit/chainlink-common/pkg/utils/big_math" "github.com/smartcontractkit/chainlink-common/pkg/utils/tests" + clientMocks "github.com/smartcontractkit/chainlink/v2/common/client/mocks" "github.com/smartcontractkit/chainlink/v2/common/types" "github.com/smartcontractkit/chainlink/v2/common/types/mocks" ) @@ -283,9 +285,11 @@ func TestUnit_NodeLifecycle_aliveLoop(t *testing.T) { t.Parallel() rpc := newMockNodeClient[types.ID, Head](t) node := newSubscribedNode(t, testNodeOpts{ - config: testNodeConfig{}, - noNewHeadsThreshold: tests.TestInterval, - rpc: rpc, + config: testNodeConfig{}, + chainConfig: clientMocks.ChainConfig{ + NoNewHeadsThresholdVal: tests.TestInterval, + }, + rpc: rpc, }) defer func() { assert.NoError(t, node.close()) }() // tries to redial in outOfSync @@ -308,10 +312,12 @@ func TestUnit_NodeLifecycle_aliveLoop(t *testing.T) { rpc := newMockNodeClient[types.ID, Head](t) lggr, observedLogs := logger.TestObserved(t, zap.DebugLevel) node := newSubscribedNode(t, testNodeOpts{ - config: testNodeConfig{}, - lggr: lggr, - noNewHeadsThreshold: tests.TestInterval, - rpc: rpc, + config: testNodeConfig{}, + lggr: lggr, + chainConfig: clientMocks.ChainConfig{ + NoNewHeadsThresholdVal: tests.TestInterval, + }, + rpc: rpc, }) defer func() { assert.NoError(t, node.close()) }() node.nLiveNodes = func() (count int, blockNumber int64, totalDifficulty *big.Int) { @@ -335,10 +341,12 @@ func TestUnit_NodeLifecycle_aliveLoop(t *testing.T) { rpc.On("SetAliveLoopSub", sub).Once() lggr, observedLogs := logger.TestObserved(t, zap.ErrorLevel) node := newDialedNode(t, testNodeOpts{ - lggr: lggr, - config: testNodeConfig{}, - noNewHeadsThreshold: tests.TestInterval, - rpc: rpc, + lggr: lggr, + config: testNodeConfig{}, + chainConfig: clientMocks.ChainConfig{ + NoNewHeadsThresholdVal: tests.TestInterval, + }, + rpc: rpc, }) defer func() { assert.NoError(t, node.close()) }() // disconnects all on transfer to unreachable or outOfSync @@ -374,6 +382,128 @@ func TestUnit_NodeLifecycle_aliveLoop(t *testing.T) { return state == nodeStateAlive && block == expectedBlockNumber == bigmath.Equal(diff, expectedDiff) }) }) + t.Run("If finality tag is not enabled updates finalized block metric using finality depth and latest head", func(t *testing.T) { + t.Parallel() + rpc := newMockNodeClient[types.ID, Head](t) + sub := mocks.NewSubscription(t) + sub.On("Err").Return((<-chan error)(nil)) + sub.On("Unsubscribe").Once() + const blockNumber = 1000 + const finalityDepth = 10 + const expectedBlock = 990 + rpc.On("Subscribe", mock.Anything, mock.Anything, rpcSubscriptionMethodNewHeads).Run(func(args mock.Arguments) { + ch := args.Get(1).(chan<- Head) + go writeHeads(t, ch, head{BlockNumber: blockNumber - 1}, head{BlockNumber: blockNumber}, head{BlockNumber: blockNumber - 1}) + }).Return(sub, nil).Once() + rpc.On("SetAliveLoopSub", sub).Once() + name := "node-" + rand.Str(5) + node := newDialedNode(t, testNodeOpts{ + config: testNodeConfig{}, + chainConfig: clientMocks.ChainConfig{FinalityDepthVal: finalityDepth}, + rpc: rpc, + name: name, + chainID: big.NewInt(1), + }) + defer func() { assert.NoError(t, node.close()) }() + node.declareAlive() + tests.AssertEventually(t, func() bool { + metric, err := promPoolRPCNodeHighestFinalizedBlock.GetMetricWithLabelValues(big.NewInt(1).String(), name) + require.NoError(t, err) + var m = &prom.Metric{} + require.NoError(t, metric.Write(m)) + return float64(expectedBlock) == m.Gauge.GetValue() + }) + }) + t.Run("Logs warning if failed to get finalized block", func(t *testing.T) { + t.Parallel() + rpc := newMockNodeClient[types.ID, Head](t) + rpc.On("LatestFinalizedBlock", mock.Anything).Return(newMockHead(t), errors.New("failed to get finalized block")) + sub := mocks.NewSubscription(t) + sub.On("Err").Return((<-chan error)(nil)) + sub.On("Unsubscribe").Once() + rpc.On("Subscribe", mock.Anything, mock.Anything, rpcSubscriptionMethodNewHeads).Return(sub, nil).Once() + rpc.On("SetAliveLoopSub", sub).Once() + lggr, observedLogs := logger.TestObserved(t, zap.DebugLevel) + node := newDialedNode(t, testNodeOpts{ + config: testNodeConfig{ + finalizedBlockPollInterval: tests.TestInterval, + }, + chainConfig: clientMocks.ChainConfig{ + IsFinalityTagEnabled: true, + }, + rpc: rpc, + lggr: lggr, + }) + defer func() { assert.NoError(t, node.close()) }() + node.declareAlive() + tests.AssertLogEventually(t, observedLogs, "Failed to fetch latest finalized block") + }) + t.Run("Logs warning if latest finalized block is not valid", func(t *testing.T) { + t.Parallel() + rpc := newMockNodeClient[types.ID, Head](t) + head := newMockHead(t) + head.On("IsValid").Return(false) + rpc.On("LatestFinalizedBlock", mock.Anything).Return(head, nil) + sub := mocks.NewSubscription(t) + sub.On("Err").Return((<-chan error)(nil)) + sub.On("Unsubscribe").Once() + rpc.On("Subscribe", mock.Anything, mock.Anything, rpcSubscriptionMethodNewHeads).Return(sub, nil).Once() + rpc.On("SetAliveLoopSub", sub).Once() + lggr, observedLogs := logger.TestObserved(t, zap.DebugLevel) + node := newDialedNode(t, testNodeOpts{ + config: testNodeConfig{ + finalizedBlockPollInterval: tests.TestInterval, + }, + chainConfig: clientMocks.ChainConfig{ + IsFinalityTagEnabled: true, + }, + rpc: rpc, + lggr: lggr, + }) + defer func() { assert.NoError(t, node.close()) }() + node.declareAlive() + tests.AssertLogEventually(t, observedLogs, "Latest finalized block is not valid") + }) + t.Run("If finality tag and finalized block polling are enabled updates latest finalized block metric", func(t *testing.T) { + t.Parallel() + rpc := newMockNodeClient[types.ID, Head](t) + const expectedBlock = 1101 + const finalityDepth = 10 + rpc.On("LatestFinalizedBlock", mock.Anything).Return(head{BlockNumber: expectedBlock - 1}.ToMockHead(t), nil).Once() + rpc.On("LatestFinalizedBlock", mock.Anything).Return(head{BlockNumber: expectedBlock}.ToMockHead(t), nil) + sub := mocks.NewSubscription(t) + sub.On("Err").Return((<-chan error)(nil)) + sub.On("Unsubscribe").Once() + rpc.On("Subscribe", mock.Anything, mock.Anything, rpcSubscriptionMethodNewHeads).Run(func(args mock.Arguments) { + ch := args.Get(1).(chan<- Head) + // ensure that "calculated" finalized head is larger than actual, to ensure we are correctly setting + // the metric + go writeHeads(t, ch, head{BlockNumber: expectedBlock*2 + finalityDepth}) + }).Return(sub, nil).Once() + rpc.On("SetAliveLoopSub", sub).Once() + name := "node-" + rand.Str(5) + node := newDialedNode(t, testNodeOpts{ + config: testNodeConfig{ + finalizedBlockPollInterval: tests.TestInterval, + }, + chainConfig: clientMocks.ChainConfig{ + FinalityDepthVal: finalityDepth, + IsFinalityTagEnabled: true, + }, + rpc: rpc, + name: name, + chainID: big.NewInt(1), + }) + defer func() { assert.NoError(t, node.close()) }() + node.declareAlive() + tests.AssertEventually(t, func() bool { + metric, err := promPoolRPCNodeHighestFinalizedBlock.GetMetricWithLabelValues(big.NewInt(1).String(), name) + require.NoError(t, err) + var m = &prom.Metric{} + require.NoError(t, metric.Write(m)) + return float64(expectedBlock) == m.Gauge.GetValue() + }) + }) } type head struct { @@ -381,11 +511,17 @@ type head struct { BlockDifficulty *big.Int } +func (h head) ToMockHead(t *testing.T) *mockHead { + m := newMockHead(t) + m.On("BlockNumber").Return(h.BlockNumber).Maybe() + m.On("BlockDifficulty").Return(h.BlockDifficulty).Maybe() + m.On("IsValid").Return(true).Maybe() + return m +} + func writeHeads(t *testing.T, ch chan<- Head, heads ...head) { for _, head := range heads { - h := newMockHead(t) - h.On("BlockNumber").Return(head.BlockNumber) - h.On("BlockDifficulty").Return(head.BlockDifficulty) + h := head.ToMockHead(t) select { case ch <- h: case <-tests.Context(t).Done(): @@ -675,10 +811,12 @@ func TestUnit_NodeLifecycle_outOfSyncLoop(t *testing.T) { nodeChainID := types.RandomID() lggr, observedLogs := logger.TestObserved(t, zap.DebugLevel) node := newAliveNode(t, testNodeOpts{ - noNewHeadsThreshold: tests.TestInterval, - rpc: rpc, - chainID: nodeChainID, - lggr: lggr, + chainConfig: clientMocks.ChainConfig{ + NoNewHeadsThresholdVal: tests.TestInterval, + }, + rpc: rpc, + chainID: nodeChainID, + lggr: lggr, }) defer func() { assert.NoError(t, node.close()) }() node.nLiveNodes = func() (count int, blockNumber int64, totalDifficulty *big.Int) { @@ -1234,7 +1372,7 @@ func TestUnit_NodeLifecycle_syncStatus(t *testing.T) { } for _, td := range []int64{totalDifficulty - syncThreshold - 1, totalDifficulty - syncThreshold, totalDifficulty, totalDifficulty + 1} { for _, testCase := range testCases { - t.Run(fmt.Sprintf("%s: selectionMode: %s: total difficulty: %d", testCase.name, selectionMode, td), func(t *testing.T) { + t.Run(fmt.Sprintf("%s: SelectionModeVal: %s: total difficulty: %d", testCase.name, selectionMode, td), func(t *testing.T) { outOfSync, liveNodes := node.syncStatus(testCase.blockNumber, big.NewInt(td)) assert.Equal(t, nodesNum, liveNodes) assert.Equal(t, testCase.outOfSync, outOfSync) @@ -1287,7 +1425,7 @@ func TestUnit_NodeLifecycle_syncStatus(t *testing.T) { } for _, hb := range []int64{highestBlock - syncThreshold - 1, highestBlock - syncThreshold, highestBlock, highestBlock + 1} { for _, testCase := range testCases { - t.Run(fmt.Sprintf("%s: selectionMode: %s: highest block: %d", testCase.name, NodeSelectionModeTotalDifficulty, hb), func(t *testing.T) { + t.Run(fmt.Sprintf("%s: SelectionModeVal: %s: highest block: %d", testCase.name, NodeSelectionModeTotalDifficulty, hb), func(t *testing.T) { outOfSync, liveNodes := node.syncStatus(hb, big.NewInt(testCase.totalDifficulty)) assert.Equal(t, nodesNum, liveNodes) assert.Equal(t, testCase.outOfSync, outOfSync) diff --git a/common/client/node_test.go b/common/client/node_test.go index c7a7a710d8f..a97f26555a9 100644 --- a/common/client/node_test.go +++ b/common/client/node_test.go @@ -7,15 +7,17 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/logger" + clientMocks "github.com/smartcontractkit/chainlink/v2/common/client/mocks" "github.com/smartcontractkit/chainlink/v2/common/types" ) type testNodeConfig struct { - pollFailureThreshold uint32 - pollInterval time.Duration - selectionMode string - syncThreshold uint32 - nodeIsSyncingEnabled bool + pollFailureThreshold uint32 + pollInterval time.Duration + selectionMode string + syncThreshold uint32 + nodeIsSyncingEnabled bool + finalizedBlockPollInterval time.Duration } func (n testNodeConfig) PollFailureThreshold() uint32 { @@ -38,22 +40,26 @@ func (n testNodeConfig) NodeIsSyncingEnabled() bool { return n.nodeIsSyncingEnabled } +func (n testNodeConfig) FinalizedBlockPollInterval() time.Duration { + return n.finalizedBlockPollInterval +} + type testNode struct { *node[types.ID, Head, NodeClient[types.ID, Head]] } type testNodeOpts struct { - config testNodeConfig - noNewHeadsThreshold time.Duration - lggr logger.Logger - wsuri url.URL - httpuri *url.URL - name string - id int32 - chainID types.ID - nodeOrder int32 - rpc *mockNodeClient[types.ID, Head] - chainFamily string + config testNodeConfig + chainConfig clientMocks.ChainConfig + lggr logger.Logger + wsuri url.URL + httpuri *url.URL + name string + id int32 + chainID types.ID + nodeOrder int32 + rpc *mockNodeClient[types.ID, Head] + chainFamily string } func newTestNode(t *testing.T, opts testNodeOpts) testNode { @@ -77,7 +83,7 @@ func newTestNode(t *testing.T, opts testNodeOpts) testNode { opts.id = 42 } - nodeI := NewNode[types.ID, Head, NodeClient[types.ID, Head]](opts.config, opts.noNewHeadsThreshold, opts.lggr, + nodeI := NewNode[types.ID, Head, NodeClient[types.ID, Head]](opts.config, opts.chainConfig, opts.lggr, opts.wsuri, opts.httpuri, opts.name, opts.id, opts.chainID, opts.nodeOrder, opts.rpc, opts.chainFamily) return testNode{ diff --git a/common/client/types.go b/common/client/types.go index 8d7b5b71b83..a27e6a50b73 100644 --- a/common/client/types.go +++ b/common/client/types.go @@ -53,6 +53,7 @@ type RPC[ type Head interface { BlockNumber() int64 BlockDifficulty() *big.Int + IsValid() bool } // NodeClient includes all the necessary RPC methods required by a node. @@ -72,6 +73,7 @@ type NodeClient[ SetAliveLoopSub(types.Subscription) UnsubscribeAllExceptAliveLoop() IsSyncing(ctx context.Context) (bool, error) + LatestFinalizedBlock(ctx context.Context) (HEAD, error) } // clientAPI includes all the direct RPC methods required by the generalized common client to implement its own. diff --git a/common/types/head.go b/common/types/head.go index c363fd5d0f2..4ecdb981c78 100644 --- a/common/types/head.go +++ b/common/types/head.go @@ -36,4 +36,6 @@ type Head[BLOCK_HASH Hashable] interface { // Returns the total difficulty of the block. For chains who do not have a concept of block // difficulty, return 0. BlockDifficulty() *big.Int + // IsValid returns true if the head is valid. + IsValid() bool } diff --git a/common/types/mocks/head.go b/common/types/mocks/head.go index 29b6d073656..fd5c95d472f 100644 --- a/common/types/mocks/head.go +++ b/common/types/mocks/head.go @@ -184,6 +184,24 @@ func (_m *Head[BLOCK_HASH]) HashAtHeight(blockNum int64) BLOCK_HASH { return r0 } +// IsValid provides a mock function with given fields: +func (_m *Head[BLOCK_HASH]) IsValid() bool { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for IsValid") + } + + var r0 bool + if rf, ok := ret.Get(0).(func() bool); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + // NewHead creates a new instance of Head. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. // The first argument is typically a *testing.T value. func NewHead[BLOCK_HASH types.Hashable](t interface { diff --git a/core/chains/evm/client/config_builder.go b/core/chains/evm/client/config_builder.go index c004bc4e9c6..d78a981b881 100644 --- a/core/chains/evm/client/config_builder.go +++ b/core/chains/evm/client/config_builder.go @@ -8,12 +8,13 @@ import ( "go.uber.org/multierr" commonconfig "github.com/smartcontractkit/chainlink-common/pkg/config" - "github.com/smartcontractkit/chainlink/v2/common/config" + + commonclient "github.com/smartcontractkit/chainlink/v2/common/client" evmconfig "github.com/smartcontractkit/chainlink/v2/core/chains/evm/config" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/config/toml" ) -type nodeConfig struct { +type NodeConfig struct { Name *string WSURL *string HTTPURL *string @@ -28,15 +29,19 @@ func NewClientConfigs( selectionMode *string, leaseDuration time.Duration, chainType string, - nodeCfgs []nodeConfig, + nodeCfgs []NodeConfig, pollFailureThreshold *uint32, pollInterval time.Duration, syncThreshold *uint32, nodeIsSyncingEnabled *bool, -) (evmconfig.NodePool, []*toml.Node, config.ChainType, error) { + noNewHeadsThreshold time.Duration, + finalityDepth *uint32, + finalityTagEnabled *bool, + +) (commonclient.ChainConfig, evmconfig.NodePool, []*toml.Node, error) { nodes, err := parseNodeConfigs(nodeCfgs) if err != nil { - return nil, nil, "", err + return nil, nil, nil, err } nodePool := toml.NodePool{ SelectionMode: selectionMode, @@ -47,10 +52,20 @@ func NewClientConfigs( NodeIsSyncingEnabled: nodeIsSyncingEnabled, } nodePoolCfg := &evmconfig.NodePoolConfig{C: nodePool} - return nodePoolCfg, nodes, config.ChainType(chainType), nil + chainConfig := &evmconfig.EVMConfig{ + C: &toml.EVMConfig{ + Chain: toml.Chain{ + ChainType: &chainType, + FinalityDepth: finalityDepth, + FinalityTagEnabled: finalityTagEnabled, + NoNewHeadsThreshold: commonconfig.MustNewDuration(noNewHeadsThreshold), + }, + }, + } + return chainConfig, nodePoolCfg, nodes, nil } -func parseNodeConfigs(nodeCfgs []nodeConfig) ([]*toml.Node, error) { +func parseNodeConfigs(nodeCfgs []NodeConfig) ([]*toml.Node, error) { nodes := make([]*toml.Node, len(nodeCfgs)) for i, nodeCfg := range nodeCfgs { if nodeCfg.WSURL == nil || nodeCfg.HTTPURL == nil { diff --git a/core/chains/evm/client/config_builder_test.go b/core/chains/evm/client/config_builder_test.go index cc00029d270..43a33859615 100644 --- a/core/chains/evm/client/config_builder_test.go +++ b/core/chains/evm/client/config_builder_test.go @@ -1,12 +1,14 @@ package client_test import ( + "math/big" "testing" "time" "github.com/stretchr/testify/require" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/client" + "github.com/smartcontractkit/chainlink/v2/core/logger" ) func TestClientConfigBuilder(t *testing.T) { @@ -19,14 +21,18 @@ func TestClientConfigBuilder(t *testing.T) { syncThreshold := ptr(uint32(5)) nodeIsSyncingEnabled := ptr(false) chainTypeStr := "" - nodeConfigs := []client.TestNodeConfig{ + nodeConfigs := []client.NodeConfig{ { Name: ptr("foo"), WSURL: ptr("ws://foo.test"), HTTPURL: ptr("http://foo.test"), }, } - nodePool, nodes, chainType, err := client.NewClientConfigs(selectionMode, leaseDuration, chainTypeStr, nodeConfigs, pollFailureThreshold, pollInterval, syncThreshold, nodeIsSyncingEnabled) + finalityDepth := ptr(uint32(10)) + finalityTagEnabled := ptr(true) + noNewHeadsThreshold := time.Second + chainCfg, nodePool, nodes, err := client.NewClientConfigs(selectionMode, leaseDuration, chainTypeStr, nodeConfigs, + pollFailureThreshold, pollInterval, syncThreshold, nodeIsSyncingEnabled, noNewHeadsThreshold, finalityDepth, finalityTagEnabled) require.NoError(t, err) // Validate node pool configs @@ -42,15 +48,21 @@ func TestClientConfigBuilder(t *testing.T) { require.Equal(t, *nodeConfigs[0].WSURL, (*nodes[0].WSURL).String()) require.Equal(t, *nodeConfigs[0].HTTPURL, (*nodes[0].HTTPURL).String()) - // Validate chain type - require.Equal(t, chainTypeStr, string(chainType)) + // Validate chain config + require.Equal(t, chainTypeStr, string(chainCfg.ChainType())) + require.Equal(t, noNewHeadsThreshold, chainCfg.NodeNoNewHeadsThreshold()) + require.Equal(t, *finalityDepth, chainCfg.FinalityDepth()) + require.Equal(t, *finalityTagEnabled, chainCfg.FinalityTagEnabled()) + + // let combiler tell us, when we do not have sufficient data to create evm client + _ = client.NewEvmClient(nodePool, chainCfg, logger.TestLogger(t), big.NewInt(10), nodes) } func TestNodeConfigs(t *testing.T) { t.Parallel() t.Run("parsing unique node configs succeeds", func(t *testing.T) { - nodeConfigs := []client.TestNodeConfig{ + nodeConfigs := []client.NodeConfig{ { Name: ptr("foo1"), WSURL: ptr("ws://foo1.test"), @@ -68,7 +80,7 @@ func TestNodeConfigs(t *testing.T) { }) t.Run("parsing missing ws url fails", func(t *testing.T) { - nodeConfigs := []client.TestNodeConfig{ + nodeConfigs := []client.NodeConfig{ { Name: ptr("foo1"), HTTPURL: ptr("http://foo1.test"), @@ -79,7 +91,7 @@ func TestNodeConfigs(t *testing.T) { }) t.Run("parsing missing http url fails", func(t *testing.T) { - nodeConfigs := []client.TestNodeConfig{ + nodeConfigs := []client.NodeConfig{ { Name: ptr("foo1"), WSURL: ptr("ws://foo1.test"), @@ -90,7 +102,7 @@ func TestNodeConfigs(t *testing.T) { }) t.Run("parsing invalid ws url fails", func(t *testing.T) { - nodeConfigs := []client.TestNodeConfig{ + nodeConfigs := []client.NodeConfig{ { Name: ptr("foo1"), WSURL: ptr("http://foo1.test"), @@ -102,7 +114,7 @@ func TestNodeConfigs(t *testing.T) { }) t.Run("parsing duplicate http url fails", func(t *testing.T) { - nodeConfigs := []client.TestNodeConfig{ + nodeConfigs := []client.NodeConfig{ { Name: ptr("foo1"), WSURL: ptr("ws://foo1.test"), @@ -114,7 +126,7 @@ func TestNodeConfigs(t *testing.T) { }) t.Run("parsing duplicate node names fails", func(t *testing.T) { - nodeConfigs := []client.TestNodeConfig{ + nodeConfigs := []client.NodeConfig{ { Name: ptr("foo1"), WSURL: ptr("ws://foo1.test"), @@ -131,7 +143,7 @@ func TestNodeConfigs(t *testing.T) { }) t.Run("parsing duplicate node ws urls fails", func(t *testing.T) { - nodeConfigs := []client.TestNodeConfig{ + nodeConfigs := []client.NodeConfig{ { Name: ptr("foo1"), WSURL: ptr("ws://foo1.test"), @@ -148,7 +160,7 @@ func TestNodeConfigs(t *testing.T) { }) t.Run("parsing duplicate node http urls fails", func(t *testing.T) { - nodeConfigs := []client.TestNodeConfig{ + nodeConfigs := []client.NodeConfig{ { Name: ptr("foo1"), WSURL: ptr("ws://foo1.test"), @@ -165,7 +177,7 @@ func TestNodeConfigs(t *testing.T) { }) t.Run("parsing order too large fails", func(t *testing.T) { - nodeConfigs := []client.TestNodeConfig{ + nodeConfigs := []client.NodeConfig{ { Name: ptr("foo1"), WSURL: ptr("ws://foo1.test"), diff --git a/core/chains/evm/client/evm_client.go b/core/chains/evm/client/evm_client.go index cd7d4a74b80..f60605bc88e 100644 --- a/core/chains/evm/client/evm_client.go +++ b/core/chains/evm/client/evm_client.go @@ -3,17 +3,16 @@ package client import ( "math/big" "net/url" - "time" "github.com/smartcontractkit/chainlink-common/pkg/logger" + commonclient "github.com/smartcontractkit/chainlink/v2/common/client" - "github.com/smartcontractkit/chainlink/v2/common/config" evmconfig "github.com/smartcontractkit/chainlink/v2/core/chains/evm/config" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/config/toml" evmtypes "github.com/smartcontractkit/chainlink/v2/core/chains/evm/types" ) -func NewEvmClient(cfg evmconfig.NodePool, noNewHeadsThreshold time.Duration, lggr logger.Logger, chainID *big.Int, chainType config.ChainType, nodes []*toml.Node) Client { +func NewEvmClient(cfg evmconfig.NodePool, chainCfg commonclient.ChainConfig, lggr logger.Logger, chainID *big.Int, nodes []*toml.Node) Client { var empty url.URL var primaries []commonclient.Node[*big.Int, *evmtypes.Head, RPCClient] var sendonlys []commonclient.SendOnlyNode[*big.Int, RPCClient] @@ -27,11 +26,12 @@ func NewEvmClient(cfg evmconfig.NodePool, noNewHeadsThreshold time.Duration, lgg } else { rpc := NewRPCClient(lggr, (url.URL)(*node.WSURL), (*url.URL)(node.HTTPURL), *node.Name, int32(i), chainID, commonclient.Primary) - primaryNode := commonclient.NewNode(cfg, noNewHeadsThreshold, + primaryNode := commonclient.NewNode(cfg, chainCfg, lggr, (url.URL)(*node.WSURL), (*url.URL)(node.HTTPURL), *node.Name, int32(i), chainID, *node.Order, rpc, "EVM") primaries = append(primaries, primaryNode) } } - return NewChainClient(lggr, cfg.SelectionMode(), cfg.LeaseDuration(), noNewHeadsThreshold, primaries, sendonlys, chainID, chainType) + return NewChainClient(lggr, cfg.SelectionMode(), cfg.LeaseDuration(), chainCfg.NodeNoNewHeadsThreshold(), + primaries, sendonlys, chainID, chainCfg.ChainType()) } diff --git a/core/chains/evm/client/evm_client_test.go b/core/chains/evm/client/evm_client_test.go index 2764a2b3611..1542e41265b 100644 --- a/core/chains/evm/client/evm_client_test.go +++ b/core/chains/evm/client/evm_client_test.go @@ -22,16 +22,19 @@ func TestNewEvmClient(t *testing.T) { syncThreshold := ptr(uint32(5)) nodeIsSyncingEnabled := ptr(false) chainTypeStr := "" - nodeConfigs := []client.TestNodeConfig{ + nodeConfigs := []client.NodeConfig{ { Name: ptr("foo"), WSURL: ptr("ws://foo.test"), HTTPURL: ptr("http://foo.test"), }, } - nodePool, nodes, chainType, err := client.NewClientConfigs(selectionMode, leaseDuration, chainTypeStr, nodeConfigs, pollFailureThreshold, pollInterval, syncThreshold, nodeIsSyncingEnabled) + finalityDepth := ptr(uint32(10)) + finalityTagEnabled := ptr(true) + chainCfg, nodePool, nodes, err := client.NewClientConfigs(selectionMode, leaseDuration, chainTypeStr, nodeConfigs, + pollFailureThreshold, pollInterval, syncThreshold, nodeIsSyncingEnabled, noNewHeadsThreshold, finalityDepth, finalityTagEnabled) require.NoError(t, err) - client := client.NewEvmClient(nodePool, noNewHeadsThreshold, logger.TestLogger(t), testutils.FixtureChainID, chainType, nodes) + client := client.NewEvmClient(nodePool, chainCfg, logger.TestLogger(t), testutils.FixtureChainID, nodes) require.NotNil(t, client) } diff --git a/core/chains/evm/client/helpers_test.go b/core/chains/evm/client/helpers_test.go index 1decf3ed89d..2a360219046 100644 --- a/core/chains/evm/client/helpers_test.go +++ b/core/chains/evm/client/helpers_test.go @@ -12,6 +12,7 @@ import ( "github.com/smartcontractkit/chainlink-common/pkg/logger" commonclient "github.com/smartcontractkit/chainlink/v2/common/client" + clientMocks "github.com/smartcontractkit/chainlink/v2/common/client/mocks" commonconfig "github.com/smartcontractkit/chainlink/v2/common/config" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/config" "github.com/smartcontractkit/chainlink/v2/core/chains/evm/config/toml" @@ -19,12 +20,13 @@ import ( ) type TestNodePoolConfig struct { - NodePollFailureThreshold uint32 - NodePollInterval time.Duration - NodeSelectionMode string - NodeSyncThreshold uint32 - NodeLeaseDuration time.Duration - NodeIsSyncingEnabledVal bool + NodePollFailureThreshold uint32 + NodePollInterval time.Duration + NodeSelectionMode string + NodeSyncThreshold uint32 + NodeLeaseDuration time.Duration + NodeIsSyncingEnabledVal bool + NodeFinalizedBlockPollInterval time.Duration } func (tc TestNodePoolConfig) PollFailureThreshold() uint32 { return tc.NodePollFailureThreshold } @@ -39,6 +41,10 @@ func (tc TestNodePoolConfig) NodeIsSyncingEnabled() bool { return tc.NodeIsSyncingEnabledVal } +func (tc TestNodePoolConfig) FinalizedBlockPollInterval() time.Duration { + return tc.NodeFinalizedBlockPollInterval +} + func NewClientWithTestNode(t *testing.T, nodePoolCfg config.NodePool, noNewHeadsThreshold time.Duration, rpcUrl string, rpcHTTPURL *url.URL, sendonlyRPCURLs []url.URL, id int32, chainID *big.Int) (*client, error) { parsed, err := url.ParseRequestURI(rpcUrl) if err != nil { @@ -97,7 +103,7 @@ func NewChainClientWithTestNode( rpc := NewRPCClient(lggr, *parsed, rpcHTTPURL, "eth-primary-rpc-0", id, chainID, commonclient.Primary) n := commonclient.NewNode[*big.Int, *evmtypes.Head, RPCClient]( - nodeCfg, noNewHeadsThreshold, lggr, *parsed, rpcHTTPURL, "eth-primary-node-0", id, chainID, 1, rpc, "EVM") + nodeCfg, clientMocks.ChainConfig{NoNewHeadsThresholdVal: noNewHeadsThreshold}, lggr, *parsed, rpcHTTPURL, "eth-primary-node-0", id, chainID, 1, rpc, "EVM") primaries := []commonclient.Node[*big.Int, *evmtypes.Head, RPCClient]{n} var sendonlys []commonclient.SendOnlyNode[*big.Int, RPCClient] @@ -153,7 +159,7 @@ func NewChainClientWithMockedRpc( parsed, _ := url.ParseRequestURI("ws://test") n := commonclient.NewNode[*big.Int, *evmtypes.Head, RPCClient]( - cfg, noNewHeadsThreshold, lggr, *parsed, nil, "eth-primary-node-0", 1, chainID, 1, rpc, "EVM") + cfg, clientMocks.ChainConfig{NoNewHeadsThresholdVal: noNewHeadsThreshold}, lggr, *parsed, nil, "eth-primary-node-0", 1, chainID, 1, rpc, "EVM") primaries := []commonclient.Node[*big.Int, *evmtypes.Head, RPCClient]{n} c := NewChainClient(lggr, selectionMode, leaseDuration, noNewHeadsThreshold, primaries, nil, chainID, chainType) t.Cleanup(c.Close) @@ -187,8 +193,6 @@ func (mes *mockSubscription) Unsubscribe() { close(mes.Errors) } -type TestNodeConfig = nodeConfig - -func ParseTestNodeConfigs(nodes []TestNodeConfig) ([]*toml.Node, error) { +func ParseTestNodeConfigs(nodes []NodeConfig) ([]*toml.Node, error) { return parseNodeConfigs(nodes) } diff --git a/core/chains/evm/config/chain_scoped.go b/core/chains/evm/config/chain_scoped.go index aa13b9a2282..2201831feaf 100644 --- a/core/chains/evm/config/chain_scoped.go +++ b/core/chains/evm/config/chain_scoped.go @@ -20,7 +20,7 @@ import ( func NewTOMLChainScopedConfig(appCfg config.AppConfig, tomlConfig *toml.EVMConfig, lggr logger.Logger) *ChainScoped { return &ChainScoped{ AppConfig: appCfg, - evmConfig: &evmConfig{c: tomlConfig}, + evmConfig: &EVMConfig{C: tomlConfig}, lggr: lggr} } @@ -29,7 +29,7 @@ type ChainScoped struct { config.AppConfig lggr logger.Logger - evmConfig *evmConfig + evmConfig *EVMConfig } func (c *ChainScoped) EVM() EVM { @@ -37,7 +37,7 @@ func (c *ChainScoped) EVM() EVM { } func (c *ChainScoped) Nodes() toml.EVMNodes { - return c.evmConfig.c.Nodes + return c.evmConfig.C.Nodes } func (c *ChainScoped) BlockEmissionIdleWarningThreshold() time.Duration { @@ -62,142 +62,142 @@ func (c *ChainScoped) Validate() (err error) { return } -type evmConfig struct { - c *toml.EVMConfig +type EVMConfig struct { + C *toml.EVMConfig } -func (e *evmConfig) IsEnabled() bool { - return e.c.IsEnabled() +func (e *EVMConfig) IsEnabled() bool { + return e.C.IsEnabled() } -func (e *evmConfig) TOMLString() (string, error) { - return e.c.TOMLString() +func (e *EVMConfig) TOMLString() (string, error) { + return e.C.TOMLString() } -func (e *evmConfig) BalanceMonitor() BalanceMonitor { - return &balanceMonitorConfig{c: e.c.BalanceMonitor} +func (e *EVMConfig) BalanceMonitor() BalanceMonitor { + return &balanceMonitorConfig{c: e.C.BalanceMonitor} } -func (e *evmConfig) Transactions() Transactions { - return &transactionsConfig{c: e.c.Transactions} +func (e *EVMConfig) Transactions() Transactions { + return &transactionsConfig{c: e.C.Transactions} } -func (e *evmConfig) HeadTracker() HeadTracker { - return &headTrackerConfig{c: e.c.HeadTracker} +func (e *EVMConfig) HeadTracker() HeadTracker { + return &headTrackerConfig{c: e.C.HeadTracker} } -func (e *evmConfig) OCR() OCR { - return &ocrConfig{c: e.c.OCR} +func (e *EVMConfig) OCR() OCR { + return &ocrConfig{c: e.C.OCR} } -func (e *evmConfig) OCR2() OCR2 { - return &ocr2Config{c: e.c.OCR2} +func (e *EVMConfig) OCR2() OCR2 { + return &ocr2Config{c: e.C.OCR2} } -func (e *evmConfig) ChainWriter() ChainWriter { - return &chainWriterConfig{c: e.c.ChainWriter} +func (e *EVMConfig) ChainWriter() ChainWriter { + return &chainWriterConfig{c: e.C.ChainWriter} } -func (e *evmConfig) GasEstimator() GasEstimator { - return &gasEstimatorConfig{c: e.c.GasEstimator, blockDelay: e.c.RPCBlockQueryDelay, transactionsMaxInFlight: e.c.Transactions.MaxInFlight, k: e.c.KeySpecific} +func (e *EVMConfig) GasEstimator() GasEstimator { + return &gasEstimatorConfig{c: e.C.GasEstimator, blockDelay: e.C.RPCBlockQueryDelay, transactionsMaxInFlight: e.C.Transactions.MaxInFlight, k: e.C.KeySpecific} } -func (e *evmConfig) AutoCreateKey() bool { - return *e.c.AutoCreateKey +func (e *EVMConfig) AutoCreateKey() bool { + return *e.C.AutoCreateKey } -func (e *evmConfig) BlockBackfillDepth() uint64 { - return uint64(*e.c.BlockBackfillDepth) +func (e *EVMConfig) BlockBackfillDepth() uint64 { + return uint64(*e.C.BlockBackfillDepth) } -func (e *evmConfig) BlockBackfillSkip() bool { - return *e.c.BlockBackfillSkip +func (e *EVMConfig) BlockBackfillSkip() bool { + return *e.C.BlockBackfillSkip } -func (e *evmConfig) LogBackfillBatchSize() uint32 { - return *e.c.LogBackfillBatchSize +func (e *EVMConfig) LogBackfillBatchSize() uint32 { + return *e.C.LogBackfillBatchSize } -func (e *evmConfig) LogPollInterval() time.Duration { - return e.c.LogPollInterval.Duration() +func (e *EVMConfig) LogPollInterval() time.Duration { + return e.C.LogPollInterval.Duration() } -func (e *evmConfig) FinalityDepth() uint32 { - return *e.c.FinalityDepth +func (e *EVMConfig) FinalityDepth() uint32 { + return *e.C.FinalityDepth } -func (e *evmConfig) FinalityTagEnabled() bool { - return *e.c.FinalityTagEnabled +func (e *EVMConfig) FinalityTagEnabled() bool { + return *e.C.FinalityTagEnabled } -func (e *evmConfig) LogKeepBlocksDepth() uint32 { - return *e.c.LogKeepBlocksDepth +func (e *EVMConfig) LogKeepBlocksDepth() uint32 { + return *e.C.LogKeepBlocksDepth } -func (e *evmConfig) BackupLogPollerBlockDelay() uint64 { - return *e.c.BackupLogPollerBlockDelay +func (e *EVMConfig) BackupLogPollerBlockDelay() uint64 { + return *e.C.BackupLogPollerBlockDelay } -func (e *evmConfig) NonceAutoSync() bool { - return *e.c.NonceAutoSync +func (e *EVMConfig) NonceAutoSync() bool { + return *e.C.NonceAutoSync } -func (e *evmConfig) RPCDefaultBatchSize() uint32 { - return *e.c.RPCDefaultBatchSize +func (e *EVMConfig) RPCDefaultBatchSize() uint32 { + return *e.C.RPCDefaultBatchSize } -func (e *evmConfig) BlockEmissionIdleWarningThreshold() time.Duration { - return e.c.NoNewHeadsThreshold.Duration() +func (e *EVMConfig) BlockEmissionIdleWarningThreshold() time.Duration { + return e.C.NoNewHeadsThreshold.Duration() } -func (e *evmConfig) ChainType() commonconfig.ChainType { - if e.c.ChainType == nil { +func (e *EVMConfig) ChainType() commonconfig.ChainType { + if e.C.ChainType == nil { return "" } - return commonconfig.ChainType(*e.c.ChainType) + return commonconfig.ChainType(*e.C.ChainType) } -func (e *evmConfig) ChainID() *big.Int { - return e.c.ChainID.ToInt() +func (e *EVMConfig) ChainID() *big.Int { + return e.C.ChainID.ToInt() } -func (e *evmConfig) MinIncomingConfirmations() uint32 { - return *e.c.MinIncomingConfirmations +func (e *EVMConfig) MinIncomingConfirmations() uint32 { + return *e.C.MinIncomingConfirmations } -func (e *evmConfig) NodePool() NodePool { - return &NodePoolConfig{C: e.c.NodePool} +func (e *EVMConfig) NodePool() NodePool { + return &NodePoolConfig{C: e.C.NodePool} } -func (e *evmConfig) NodeNoNewHeadsThreshold() time.Duration { - return e.c.NoNewHeadsThreshold.Duration() +func (e *EVMConfig) NodeNoNewHeadsThreshold() time.Duration { + return e.C.NoNewHeadsThreshold.Duration() } -func (e *evmConfig) MinContractPayment() *assets.Link { - return e.c.MinContractPayment +func (e *EVMConfig) MinContractPayment() *assets.Link { + return e.C.MinContractPayment } -func (e *evmConfig) FlagsContractAddress() string { - if e.c.FlagsContractAddress == nil { +func (e *EVMConfig) FlagsContractAddress() string { + if e.C.FlagsContractAddress == nil { return "" } - return e.c.FlagsContractAddress.String() + return e.C.FlagsContractAddress.String() } -func (e *evmConfig) LinkContractAddress() string { - if e.c.LinkContractAddress == nil { +func (e *EVMConfig) LinkContractAddress() string { + if e.C.LinkContractAddress == nil { return "" } - return e.c.LinkContractAddress.String() + return e.C.LinkContractAddress.String() } -func (e *evmConfig) OperatorFactoryAddress() string { - if e.c.OperatorFactoryAddress == nil { +func (e *EVMConfig) OperatorFactoryAddress() string { + if e.C.OperatorFactoryAddress == nil { return "" } - return e.c.OperatorFactoryAddress.String() + return e.C.OperatorFactoryAddress.String() } -func (e *evmConfig) LogPrunePageSize() uint32 { - return *e.c.LogPrunePageSize +func (e *EVMConfig) LogPrunePageSize() uint32 { + return *e.C.LogPrunePageSize } diff --git a/core/chains/evm/config/chain_scoped_node_pool.go b/core/chains/evm/config/chain_scoped_node_pool.go index 0796d004cae..6874b412f7f 100644 --- a/core/chains/evm/config/chain_scoped_node_pool.go +++ b/core/chains/evm/config/chain_scoped_node_pool.go @@ -33,3 +33,7 @@ func (n *NodePoolConfig) LeaseDuration() time.Duration { func (n *NodePoolConfig) NodeIsSyncingEnabled() bool { return *n.C.NodeIsSyncingEnabled } + +func (n *NodePoolConfig) FinalizedBlockPollInterval() time.Duration { + return n.C.FinalizedBlockPollInterval.Duration() +} diff --git a/core/chains/evm/config/config.go b/core/chains/evm/config/config.go index ee9c46c14e4..b9ff9ea9f8e 100644 --- a/core/chains/evm/config/config.go +++ b/core/chains/evm/config/config.go @@ -141,6 +141,7 @@ type NodePool interface { SyncThreshold() uint32 LeaseDuration() time.Duration NodeIsSyncingEnabled() bool + FinalizedBlockPollInterval() time.Duration } // TODO BCF-2509 does the chainscopedconfig really need the entire app config? diff --git a/core/chains/evm/config/toml/config.go b/core/chains/evm/config/toml/config.go index 4ad0e6569bb..0f647034162 100644 --- a/core/chains/evm/config/toml/config.go +++ b/core/chains/evm/config/toml/config.go @@ -400,6 +400,7 @@ func (c *Chain) ValidateConfig() (err error) { err = multierr.Append(err, commonconfig.ErrInvalid{Name: "MinIncomingConfirmations", Value: *c.MinIncomingConfirmations, Msg: "must be greater than or equal to 1"}) } + return } @@ -702,12 +703,13 @@ func (t *HeadTracker) setFrom(f *HeadTracker) { } type NodePool struct { - PollFailureThreshold *uint32 - PollInterval *commonconfig.Duration - SelectionMode *string - SyncThreshold *uint32 - LeaseDuration *commonconfig.Duration - NodeIsSyncingEnabled *bool + PollFailureThreshold *uint32 + PollInterval *commonconfig.Duration + SelectionMode *string + SyncThreshold *uint32 + LeaseDuration *commonconfig.Duration + NodeIsSyncingEnabled *bool + FinalizedBlockPollInterval *commonconfig.Duration } func (p *NodePool) setFrom(f *NodePool) { @@ -729,6 +731,9 @@ func (p *NodePool) setFrom(f *NodePool) { if v := f.NodeIsSyncingEnabled; v != nil { p.NodeIsSyncingEnabled = v } + if v := f.FinalizedBlockPollInterval; v != nil { + p.FinalizedBlockPollInterval = v + } } type OCR struct { diff --git a/core/chains/evm/config/toml/defaults/fallback.toml b/core/chains/evm/config/toml/defaults/fallback.toml index 1a1d9b69439..d65d0a1b0c1 100644 --- a/core/chains/evm/config/toml/defaults/fallback.toml +++ b/core/chains/evm/config/toml/defaults/fallback.toml @@ -62,6 +62,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 diff --git a/core/chains/legacyevm/chain.go b/core/chains/legacyevm/chain.go index 830a1198258..1066eeecbea 100644 --- a/core/chains/legacyevm/chain.go +++ b/core/chains/legacyevm/chain.go @@ -212,13 +212,13 @@ func NewTOMLChain(ctx context.Context, chain *toml.EVMConfig, opts ChainRelayExt } func newChain(ctx context.Context, cfg *evmconfig.ChainScoped, nodes []*toml.Node, opts ChainRelayExtenderConfig) (*chain, error) { - chainID, chainType := cfg.EVM().ChainID(), cfg.EVM().ChainType() + chainID := cfg.EVM().ChainID() l := opts.Logger var client evmclient.Client if !cfg.EVMRPCEnabled() { client = evmclient.NewNullClient(chainID, l) } else if opts.GenEthClient == nil { - client = evmclient.NewEvmClient(cfg.EVM().NodePool(), cfg.EVM().NodeNoNewHeadsThreshold(), l, chainID, chainType, nodes) + client = evmclient.NewEvmClient(cfg.EVM().NodePool(), cfg.EVM(), l, chainID, nodes) } else { client = opts.GenEthClient(chainID) } diff --git a/core/config/docs/chains-evm.toml b/core/config/docs/chains-evm.toml index dcf2f6e688e..dd63edaf694 100644 --- a/core/config/docs/chains-evm.toml +++ b/core/config/docs/chains-evm.toml @@ -344,7 +344,14 @@ LeaseDuration = '0s' # Default # # Set true to enable this check NodeIsSyncingEnabled = false # Default - +# FinalizedBlockPollInterval controls how often to poll RPC for new finalized blocks. +# The finalized block is only used to report to the `pool_rpc_node_highest_finalized_block` metric. We plan to use it +# in RPCs health assessment in the future. +# If `FinalityTagEnabled = false`, poll is not performed and `pool_rpc_node_highest_finalized_block` is +# reported based on latest block and finality depth. +# +# Set to 0 to disable. +FinalizedBlockPollInterval = '5s' # Default [EVM.OCR] # ContractConfirmations sets `OCR.ContractConfirmations` for this EVM chain. ContractConfirmations = 4 # Default diff --git a/core/services/chainlink/config_test.go b/core/services/chainlink/config_test.go index b01a8da50d2..ad5c7e8c33d 100644 --- a/core/services/chainlink/config_test.go +++ b/core/services/chainlink/config_test.go @@ -575,12 +575,13 @@ func TestConfig_Marshal(t *testing.T) { }, NodePool: evmcfg.NodePool{ - PollFailureThreshold: ptr[uint32](5), - PollInterval: &minute, - SelectionMode: &selectionMode, - SyncThreshold: ptr[uint32](13), - LeaseDuration: &zeroSeconds, - NodeIsSyncingEnabled: ptr(true), + PollFailureThreshold: ptr[uint32](5), + PollInterval: &minute, + SelectionMode: &selectionMode, + SyncThreshold: ptr[uint32](13), + LeaseDuration: &zeroSeconds, + NodeIsSyncingEnabled: ptr(true), + FinalizedBlockPollInterval: &second, }, OCR: evmcfg.OCR{ ContractConfirmations: ptr[uint16](11), @@ -1020,6 +1021,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 13 LeaseDuration = '0s' NodeIsSyncingEnabled = true +FinalizedBlockPollInterval = '1s' [EVM.OCR] ContractConfirmations = 11 diff --git a/core/services/chainlink/testdata/config-full.toml b/core/services/chainlink/testdata/config-full.toml index 736c746b138..087a7e971fc 100644 --- a/core/services/chainlink/testdata/config-full.toml +++ b/core/services/chainlink/testdata/config-full.toml @@ -342,6 +342,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 13 LeaseDuration = '0s' NodeIsSyncingEnabled = true +FinalizedBlockPollInterval = '1s' [EVM.OCR] ContractConfirmations = 11 diff --git a/core/services/chainlink/testdata/config-multi-chain-effective.toml b/core/services/chainlink/testdata/config-multi-chain-effective.toml index 7e4e6e44d2e..046ff28be22 100644 --- a/core/services/chainlink/testdata/config-multi-chain-effective.toml +++ b/core/services/chainlink/testdata/config-multi-chain-effective.toml @@ -313,6 +313,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 4 @@ -403,6 +404,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 4 @@ -487,6 +489,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 4 diff --git a/core/web/resolver/testdata/config-full.toml b/core/web/resolver/testdata/config-full.toml index 3e4f2539943..068931099ca 100644 --- a/core/web/resolver/testdata/config-full.toml +++ b/core/web/resolver/testdata/config-full.toml @@ -341,6 +341,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 13 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 11 diff --git a/core/web/resolver/testdata/config-multi-chain-effective.toml b/core/web/resolver/testdata/config-multi-chain-effective.toml index 7e4e6e44d2e..046ff28be22 100644 --- a/core/web/resolver/testdata/config-multi-chain-effective.toml +++ b/core/web/resolver/testdata/config-multi-chain-effective.toml @@ -313,6 +313,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 4 @@ -403,6 +404,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 4 @@ -487,6 +489,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 4 diff --git a/docs/CONFIG.md b/docs/CONFIG.md index 2b7620a3c7c..84f60207b37 100644 --- a/docs/CONFIG.md +++ b/docs/CONFIG.md @@ -1770,6 +1770,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -1854,6 +1855,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -1938,6 +1940,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -2022,6 +2025,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -2107,6 +2111,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -2191,6 +2196,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -2275,6 +2281,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -2360,6 +2367,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -2444,6 +2452,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -2527,6 +2536,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -2610,6 +2620,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -2694,6 +2705,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -2779,6 +2791,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -2863,6 +2876,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -2947,6 +2961,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -3031,6 +3046,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -3115,6 +3131,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -3199,6 +3216,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -3283,6 +3301,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -3367,6 +3386,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -3452,6 +3472,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -3536,6 +3557,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -3619,6 +3641,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -3703,6 +3726,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -3786,6 +3810,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -3870,6 +3895,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -3954,6 +3980,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -4037,6 +4064,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -4120,6 +4148,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -4204,6 +4233,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -4287,6 +4317,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -4371,6 +4402,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -4454,6 +4486,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -4538,6 +4571,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -4622,6 +4656,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -4707,6 +4742,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -4791,6 +4827,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -4875,6 +4912,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -4959,6 +4997,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -5043,6 +5082,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -5126,6 +5166,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -5209,6 +5250,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -5293,6 +5335,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -5377,6 +5420,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -5461,6 +5505,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -5546,6 +5591,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -5631,6 +5677,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -5715,6 +5762,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -5799,6 +5847,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -5883,6 +5932,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -5967,6 +6017,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -6051,6 +6102,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 10 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 1 @@ -6135,6 +6187,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -6219,6 +6272,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [OCR] ContractConfirmations = 4 @@ -6863,6 +6917,7 @@ SelectionMode = 'HighestHead' # Default SyncThreshold = 5 # Default LeaseDuration = '0s' # Default NodeIsSyncingEnabled = false # Default +FinalizedBlockPollInterval = '5s' # Default ``` The node pool manages multiple RPC endpoints. @@ -6924,6 +6979,18 @@ All of the requests to node in state `Syncing` are rejected. Set true to enable this check +### FinalizedBlockPollInterval +```toml +FinalizedBlockPollInterval = '5s' # Default +``` +FinalizedBlockPollInterval controls how often to poll RPC for new finalized blocks. +The finalized block is only used to report to the `pool_rpc_node_highest_finalized_block` metric. We plan to use it +in RPCs health assessment in the future. +If `FinalityTagEnabled = false`, poll is not performed and `pool_rpc_node_highest_finalized_block` is +reported based on latest block and finality depth. + +Set to 0 to disable. + ## EVM.OCR ```toml [EVM.OCR] diff --git a/testdata/scripts/node/validate/disk-based-logging-disabled.txtar b/testdata/scripts/node/validate/disk-based-logging-disabled.txtar index c3f9b61243b..cb8fba9d59e 100644 --- a/testdata/scripts/node/validate/disk-based-logging-disabled.txtar +++ b/testdata/scripts/node/validate/disk-based-logging-disabled.txtar @@ -369,6 +369,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 4 diff --git a/testdata/scripts/node/validate/disk-based-logging-no-dir.txtar b/testdata/scripts/node/validate/disk-based-logging-no-dir.txtar index f22bfc0a28f..2b9cc653d6d 100644 --- a/testdata/scripts/node/validate/disk-based-logging-no-dir.txtar +++ b/testdata/scripts/node/validate/disk-based-logging-no-dir.txtar @@ -369,6 +369,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 4 diff --git a/testdata/scripts/node/validate/disk-based-logging.txtar b/testdata/scripts/node/validate/disk-based-logging.txtar index 82663ae7f95..dc679ad7dbb 100644 --- a/testdata/scripts/node/validate/disk-based-logging.txtar +++ b/testdata/scripts/node/validate/disk-based-logging.txtar @@ -369,6 +369,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 4 diff --git a/testdata/scripts/node/validate/invalid.txtar b/testdata/scripts/node/validate/invalid.txtar index e8726baee47..1631ac88cfb 100644 --- a/testdata/scripts/node/validate/invalid.txtar +++ b/testdata/scripts/node/validate/invalid.txtar @@ -359,6 +359,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 4 diff --git a/testdata/scripts/node/validate/valid.txtar b/testdata/scripts/node/validate/valid.txtar index 657451d42d6..85ac9640363 100644 --- a/testdata/scripts/node/validate/valid.txtar +++ b/testdata/scripts/node/validate/valid.txtar @@ -366,6 +366,7 @@ SelectionMode = 'HighestHead' SyncThreshold = 5 LeaseDuration = '0s' NodeIsSyncingEnabled = false +FinalizedBlockPollInterval = '5s' [EVM.OCR] ContractConfirmations = 4