From 583bc18668dcaf2640459f273f07bd3ef2e1fe56 Mon Sep 17 00:00:00 2001 From: Cayman Date: Wed, 11 Oct 2023 12:54:38 -0400 Subject: [PATCH 01/23] feat: add historical state regen --- packages/beacon-node/src/chain/chain.ts | 15 +- .../historicalState/getHistoricalState.ts | 56 +++++++ .../src/chain/historicalState/index.ts | 72 ++++++++ .../src/chain/historicalState/types.ts | 34 ++++ .../src/chain/historicalState/worker.ts | 158 ++++++++++++++++++ packages/beacon-node/src/node/nodejs.ts | 10 ++ 6 files changed, 341 insertions(+), 4 deletions(-) create mode 100644 packages/beacon-node/src/chain/historicalState/getHistoricalState.ts create mode 100644 packages/beacon-node/src/chain/historicalState/index.ts create mode 100644 packages/beacon-node/src/chain/historicalState/types.ts create mode 100644 packages/beacon-node/src/chain/historicalState/worker.ts diff --git a/packages/beacon-node/src/chain/chain.ts b/packages/beacon-node/src/chain/chain.ts index 7050d7462f4e..2037dba5977f 100644 --- a/packages/beacon-node/src/chain/chain.ts +++ b/packages/beacon-node/src/chain/chain.ts @@ -75,6 +75,7 @@ import {BlockAttributes, produceBlockBody} from "./produceBlock/produceBlockBody import {computeNewStateRoot} from "./produceBlock/computeNewStateRoot.js"; import {BlockInput} from "./blocks/types.js"; import {SeenAttestationDatas} from "./seenCache/seenAttestationData.js"; +import {HistoricalStateRegen} from "./historicalState/index.js"; /** * Arbitrary constants, blobs should be consumed immediately in the same slot they are produced. @@ -105,6 +106,7 @@ export class BeaconChain implements IBeaconChain { readonly regen: QueuedStateRegenerator; readonly lightClientServer: LightClientServer; readonly reprocessController: ReprocessController; + readonly historicalStateRegen: HistoricalStateRegen; // Ops pool readonly attestationPool: AttestationPool; @@ -162,6 +164,7 @@ export class BeaconChain implements IBeaconChain { eth1, executionEngine, executionBuilder, + historicalStateRegen, }: { config: BeaconConfig; db: IBeaconDb; @@ -174,6 +177,7 @@ export class BeaconChain implements IBeaconChain { eth1: IEth1ForBlockProduction; executionEngine: IExecutionEngine; executionBuilder?: IExecutionBuilder; + historicalStateRegen: HistoricalStateRegen; } ) { this.opts = opts; @@ -188,6 +192,7 @@ export class BeaconChain implements IBeaconChain { this.eth1 = eth1; this.executionEngine = executionEngine; this.executionBuilder = executionBuilder; + this.historicalStateRegen = historicalStateRegen; const signal = this.abortController.signal; const emitter = new ChainEventEmitter(); // by default, verify signatures on both main threads and worker threads @@ -390,11 +395,13 @@ export class BeaconChain implements IBeaconChain { return state && {state, executionOptimistic: isOptimisticBlock(block)}; } } else { - // request for finalized state + // request for finalized state using historical state regen + const stateSerialized = await this.historicalStateRegen.getHistoricalState(slot); + const state = this.config + .getForkTypes(slot) + .BeaconState.deserialize(stateSerialized) as unknown as BeaconStateAllForks; - // do not attempt regen, just check if state is already in DB - const state = await this.db.stateArchive.get(slot); - return state && {state, executionOptimistic: false}; + return {state, executionOptimistic: false}; } } diff --git a/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts b/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts new file mode 100644 index 000000000000..31e83ad53cf1 --- /dev/null +++ b/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts @@ -0,0 +1,56 @@ +import { + BeaconStateTransitionMetrics, + CachedBeaconStateAllForks, + DataAvailableStatus, + ExecutionPayloadStatus, + PubkeyIndexMap, + createCachedBeaconState, + stateTransition, +} from "@lodestar/state-transition"; +import {SignedBeaconBlock} from "@lodestar/types/allForks"; +import {BeaconConfig} from "@lodestar/config"; +import {IBeaconDb} from "../../db/index.js"; + +export async function getClosestState( + slot: number, + config: BeaconConfig, + db: IBeaconDb +): Promise { + const states = await db.stateArchive.values({limit: 1, lte: slot, reverse: true}); + if (!states.length) { + throw new Error("No close state found in the database"); + } + return createCachedBeaconState(states[0], { + config, + pubkey2index: new PubkeyIndexMap(), + index2pubkey: [], + }); +} + +export function getBlocksBetween(from: number, to: number, db: IBeaconDb): AsyncIterable { + return db.blockArchive.valuesStream({gt: from, lte: to}); +} + +export async function getHistoricalState( + slot: number, + config: BeaconConfig, + db: IBeaconDb, + metrics?: BeaconStateTransitionMetrics +): Promise { + let state = await getClosestState(slot, config, db); + for await (const block of getBlocksBetween(state.slot, slot, db)) { + state = stateTransition( + state, + block, + { + verifyProposer: false, + verifySignatures: false, + verifyStateRoot: false, + executionPayloadStatus: ExecutionPayloadStatus.valid, + dataAvailableStatus: DataAvailableStatus.available, + }, + metrics + ); + } + return state.serialize(); +} diff --git a/packages/beacon-node/src/chain/historicalState/index.ts b/packages/beacon-node/src/chain/historicalState/index.ts new file mode 100644 index 000000000000..0f17513ccfc7 --- /dev/null +++ b/packages/beacon-node/src/chain/historicalState/index.ts @@ -0,0 +1,72 @@ +import {ModuleThread, spawn} from "@chainsafe/threads"; +import {chainConfigToJson} from "@lodestar/config"; +import {LoggerNode} from "@lodestar/logger/node"; +import {terminateWorkerThread} from "../../util/workerEvents.js"; +import { + HistoricalStateRegenInitModules, + HistoricalStateRegenModules, + HistoricalStateWorkerApi, + HistoricalStateWorkerData, +} from "./types.js"; + +const HISTORICAL_STATE_WORKER_EXIT_RETRY_COUNT = 3; +const HISTORICAL_STATE_WORKER_EXIT_TIMEOUT_MS = 1000; + +/** + * HistoricalStateRegen limits the damage from recreating historical states + * by running regen in a separate worker thread. + */ +export class HistoricalStateRegen implements HistoricalStateWorkerApi { + private readonly api: ModuleThread; + private readonly logger: LoggerNode; + + constructor(modules: HistoricalStateRegenModules) { + this.api = modules.api; + this.logger = modules.logger; + } + static async init(modules: HistoricalStateRegenInitModules): Promise { + const workerData: HistoricalStateWorkerData = { + chainConfigJson: chainConfigToJson(modules.config), + genesisValidatorsRoot: modules.config.genesisValidatorsRoot, + genesisTime: modules.opts.genesisTime, + maxConcurrency: 1, + maxLength: 50, + dbLocation: modules.opts.dbLocation, + metricsEnabled: Boolean(modules.metrics), + loggerOpts: modules.logger.toOpts(), + }; + + const worker = new Worker("./worker.js", { + workerData, + } as ConstructorParameters[1]); + + // eslint-disable-next-line @typescript-eslint/no-explicit-any + const api = await spawn(worker, { + // A Lodestar Node may do very expensive task at start blocking the event loop and causing + // the initialization to timeout. The number below is big enough to almost disable the timeout + timeout: 5 * 60 * 1000, + }); + + return new HistoricalStateRegen({...modules, api}); + } + + async scrapeMetrics(): Promise { + return this.api.scrapeMetrics(); + } + + async close(): Promise { + await this.api.close(); + this.logger.debug("Terminating historical state worker"); + await terminateWorkerThread({ + worker: this.api, + retryCount: HISTORICAL_STATE_WORKER_EXIT_RETRY_COUNT, + retryMs: HISTORICAL_STATE_WORKER_EXIT_TIMEOUT_MS, + logger: this.logger, + }); + this.logger.debug("Terminated historical state worker"); + } + + async getHistoricalState(slot: number): Promise { + return this.api.getHistoricalState(slot); + } +} diff --git a/packages/beacon-node/src/chain/historicalState/types.ts b/packages/beacon-node/src/chain/historicalState/types.ts new file mode 100644 index 000000000000..fd756074c37b --- /dev/null +++ b/packages/beacon-node/src/chain/historicalState/types.ts @@ -0,0 +1,34 @@ +import {ModuleThread} from "@chainsafe/threads"; +import {BeaconConfig} from "@lodestar/config"; +import {LoggerNode, LoggerNodeOpts} from "@lodestar/logger/node"; +import {Metrics} from "../../metrics/index.js"; + +export type HistoricalStateRegenInitModules = { + opts: { + genesisTime: number; + dbLocation: string; + }; + config: BeaconConfig; + logger: LoggerNode; + metrics: Metrics | null; +}; +export type HistoricalStateRegenModules = HistoricalStateRegenInitModules & { + api: ModuleThread; +}; + +export type HistoricalStateWorkerData = { + chainConfigJson: Record; + genesisValidatorsRoot: Uint8Array; + genesisTime: number; + maxConcurrency: number; + maxLength: number; + dbLocation: string; + metricsEnabled: boolean; + loggerOpts: LoggerNodeOpts; +}; + +export type HistoricalStateWorkerApi = { + close(): Promise; + scrapeMetrics(): Promise; + getHistoricalState(slot: number): Promise; +}; diff --git a/packages/beacon-node/src/chain/historicalState/worker.ts b/packages/beacon-node/src/chain/historicalState/worker.ts new file mode 100644 index 000000000000..eb01e523e37e --- /dev/null +++ b/packages/beacon-node/src/chain/historicalState/worker.ts @@ -0,0 +1,158 @@ +import worker from "node:worker_threads"; +import {expose} from "@chainsafe/threads"; +import {createBeaconConfig, chainConfigFromJson} from "@lodestar/config"; +import {getNodeLogger} from "@lodestar/logger/node"; +import {BeaconStateTransitionMetrics} from "@lodestar/state-transition"; +import {LevelDbController} from "@lodestar/db"; +import {RegistryMetricCreator, collectNodeJSMetrics} from "../../metrics/index.js"; +import {JobFnQueue} from "../../util/queue/fnQueue.js"; +import {QueueMetrics} from "../../util/queue/options.js"; +import {BeaconDb} from "../../db/index.js"; +import {HistoricalStateWorkerApi, HistoricalStateWorkerData} from "./types.js"; +import {getHistoricalState as _getHistoricalState} from "./getHistoricalState.js"; + +// most of this setup copied from networkCoreWorker.ts + +const workerData = worker.workerData as HistoricalStateWorkerData; + +// TODO: Pass options from main thread for logging +// TODO: Logging won't be visible in file loggers +const logger = getNodeLogger(workerData.loggerOpts); + +logger.info("Historical state worker started"); + +const config = createBeaconConfig(chainConfigFromJson(workerData.chainConfigJson), workerData.genesisValidatorsRoot); + +const db = new BeaconDb(config, await LevelDbController.create({name: workerData.dbLocation}, {logger})); + +const abortController = new AbortController(); + +// Set up metrics, nodejs, state transition, queue +const metricsRegister = workerData.metricsEnabled ? new RegistryMetricCreator() : null; +let stateTransitionMetrics: BeaconStateTransitionMetrics | undefined; +let queueMetrics: QueueMetrics | undefined; +if (metricsRegister) { + const closeMetrics = collectNodeJSMetrics(metricsRegister, "lodestar_historical_state_worker_"); + abortController.signal.addEventListener("abort", closeMetrics, {once: true}); + + stateTransitionMetrics = { + epochTransitionTime: metricsRegister.histogram({ + name: "lodestar_historical_state_stfn_epoch_transition_seconds", + help: "Time to process a single epoch transition in seconds", + // Epoch transitions are 100ms on very fast clients, and average 800ms on heavy networks + buckets: [0.01, 0.05, 0.1, 0.2, 0.5, 0.75, 1, 1.25, 1.5, 3, 10], + }), + epochTransitionCommitTime: metricsRegister.histogram({ + name: "lodestar_historical_state_stfn_epoch_transition_commit_seconds", + help: "Time to call commit after process a single epoch transition in seconds", + buckets: [0.01, 0.05, 0.1, 0.2, 0.5, 0.75, 1], + }), + processBlockTime: metricsRegister.histogram({ + name: "lodestar_historical_state_stfn_process_block_seconds", + help: "Time to process a single block in seconds", + // TODO: Add metrics for each step + // Block processing can take 5-40ms, 100ms max + buckets: [0.005, 0.01, 0.02, 0.05, 0.1, 1], + }), + processBlockCommitTime: metricsRegister.histogram({ + name: "lodestar_historical_state_stfn_process_block_commit_seconds", + help: "Time to call commit after process a single block in seconds", + buckets: [0.005, 0.01, 0.02, 0.05, 0.1, 1], + }), + stateHashTreeRootTime: metricsRegister.histogram({ + name: "lodestar_stfn_hash_tree_root_seconds", + help: "Time to compute the hash tree root of a post state in seconds", + buckets: [0.005, 0.01, 0.02, 0.05, 0.1, 1], + }), + preStateBalancesNodesPopulatedMiss: metricsRegister.gauge<"source">({ + name: "lodestar_historical_state_stfn_balances_nodes_populated_miss_total", + help: "Total count state.balances nodesPopulated is false on stfn", + labelNames: ["source"], + }), + preStateBalancesNodesPopulatedHit: metricsRegister.gauge<"source">({ + name: "lodestar_historical_state_stfn_balances_nodes_populated_hit_total", + help: "Total count state.balances nodesPopulated is true on stfn", + labelNames: ["source"], + }), + preStateValidatorsNodesPopulatedMiss: metricsRegister.gauge<"source">({ + name: "lodestar_historical_state_stfn_validators_nodes_populated_miss_total", + help: "Total count state.validators nodesPopulated is false on stfn", + labelNames: ["source"], + }), + preStateValidatorsNodesPopulatedHit: metricsRegister.gauge<"source">({ + name: "lodestar_historical_state_stfn_validators_nodes_populated_hit_total", + help: "Total count state.validators nodesPopulated is true on stfn", + labelNames: ["source"], + }), + preStateClonedCount: metricsRegister.histogram({ + name: "lodestar_historical_state_stfn_state_cloned_count", + help: "Histogram of cloned count per state every time state.clone() is called", + buckets: [1, 2, 5, 10, 50, 250], + }), + postStateBalancesNodesPopulatedHit: metricsRegister.gauge({ + name: "lodestar_historical_state_stfn_post_state_balances_nodes_populated_hit_total", + help: "Total count state.validators nodesPopulated is true on stfn for post state", + }), + postStateBalancesNodesPopulatedMiss: metricsRegister.gauge({ + name: "lodestar_historical_state_stfn_post_state_balances_nodes_populated_miss_total", + help: "Total count state.validators nodesPopulated is false on stfn for post state", + }), + postStateValidatorsNodesPopulatedHit: metricsRegister.gauge({ + name: "lodestar_historical_state_stfn_post_state_validators_nodes_populated_hit_total", + help: "Total count state.validators nodesPopulated is true on stfn for post state", + }), + postStateValidatorsNodesPopulatedMiss: metricsRegister.gauge({ + name: "lodestar_historical_state_stfn_post_state_validators_nodes_populated_miss_total", + help: "Total count state.validators nodesPopulated is false on stfn for post state", + }), + registerValidatorStatuses: () => {}, + }; + + queueMetrics = { + length: metricsRegister.gauge({ + name: "lodestar_historical_state_queue_length", + help: "Count of total regen queue length", + }), + droppedJobs: metricsRegister.gauge({ + name: "lodestar_historical_state_queue_dropped_jobs_total", + help: "Count of total regen queue dropped jobs", + }), + jobTime: metricsRegister.histogram({ + name: "lodestar_historical_state_queue_job_time_seconds", + help: "Time to process regen queue job in seconds", + buckets: [0.01, 0.1, 1, 10, 100], + }), + jobWaitTime: metricsRegister.histogram({ + name: "lodestar_historical_state_queue_job_wait_time_seconds", + help: "Time from job added to the regen queue to starting in seconds", + buckets: [0.01, 0.1, 1, 10, 100], + }), + concurrency: metricsRegister.gauge({ + name: "lodestar_historical_state_queue_concurrency", + help: "Current concurrency of regen queue", + }), + }; +} + +const queue = new JobFnQueue( + { + maxConcurrency: workerData.maxConcurrency, + maxLength: workerData.maxLength, + signal: abortController.signal, + }, + queueMetrics +); + +const api: HistoricalStateWorkerApi = { + async close() { + abortController.abort(); + }, + async scrapeMetrics() { + return metricsRegister?.metrics() ?? ""; + }, + async getHistoricalState(slot) { + return queue.push(() => _getHistoricalState(slot, config, db, stateTransitionMetrics)); + }, +}; + +expose(api); diff --git a/packages/beacon-node/src/node/nodejs.ts b/packages/beacon-node/src/node/nodejs.ts index fdef161001db..8f73632d0a96 100644 --- a/packages/beacon-node/src/node/nodejs.ts +++ b/packages/beacon-node/src/node/nodejs.ts @@ -21,6 +21,7 @@ import {getApi, BeaconRestApiServer} from "../api/index.js"; import {initializeExecutionEngine, initializeExecutionBuilder} from "../execution/index.js"; import {initializeEth1ForBlockProduction} from "../eth1/index.js"; import {initCKZG, loadEthereumTrustedSetup, TrustedFileMode} from "../util/kzg.js"; +import {HistoricalStateRegen} from "../chain/historicalState/index.js"; import {IBeaconNodeOptions} from "./options.js"; import {runNodeNotifier} from "./notifier.js"; @@ -218,6 +219,15 @@ export class BeaconNode { executionBuilder: opts.executionBuilder.enabled ? initializeExecutionBuilder(opts.executionBuilder, config, metrics) : undefined, + historicalStateRegen: await HistoricalStateRegen.init({ + opts: { + genesisTime: anchorState.genesisTime, + dbLocation: opts.db.name, + }, + config, + metrics, + logger: logger.child({module: LoggerModule.chain}), + }), }); // Load persisted data from disk to in-memory caches From 8c1790b89e42897dd2411701c1e83b87a0e25b3b Mon Sep 17 00:00:00 2001 From: Cayman Date: Wed, 11 Oct 2023 13:13:10 -0400 Subject: [PATCH 02/23] chore: wire up metrics --- .../beacon-node/src/metrics/server/http.ts | 7 +++--- packages/beacon-node/src/node/nodejs.ts | 22 ++++++++++--------- 2 files changed, 16 insertions(+), 13 deletions(-) diff --git a/packages/beacon-node/src/metrics/server/http.ts b/packages/beacon-node/src/metrics/server/http.ts index b699471e07d5..e3f6e8440e82 100644 --- a/packages/beacon-node/src/metrics/server/http.ts +++ b/packages/beacon-node/src/metrics/server/http.ts @@ -19,9 +19,9 @@ export async function getHttpMetricsServer( opts: HttpMetricsServerOpts, { register, - getOtherMetrics = async () => "", + getOtherMetrics = async () => [], logger, - }: {register: Registry; getOtherMetrics?: () => Promise; logger: Logger} + }: {register: Registry; getOtherMetrics?: () => Promise; logger: Logger} ): Promise { // New registry to metric the metrics. Using the same registry would deadlock the .metrics promise const httpServerRegister = new RegistryMetricCreator(); @@ -48,7 +48,8 @@ export async function getHttpMetricsServer( } else { // Get scrape time metrics const httpServerMetrics = await httpServerRegister.metrics(); - const metricsStr = `${metricsRes[0].result}\n\n${metricsRes[1]}\n\n${httpServerMetrics}`; + const metrics = [metricsRes[0].result, httpServerMetrics, ...metricsRes[1]]; + const metricsStr = metrics.join("\n\n"); res.writeHead(200, {"content-type": register.contentType}).end(metricsStr); } } else { diff --git a/packages/beacon-node/src/node/nodejs.ts b/packages/beacon-node/src/node/nodejs.ts index 8f73632d0a96..62b380c64493 100644 --- a/packages/beacon-node/src/node/nodejs.ts +++ b/packages/beacon-node/src/node/nodejs.ts @@ -197,6 +197,16 @@ export class BeaconNode { ) : null; + const historicalStateRegen = await HistoricalStateRegen.init({ + opts: { + genesisTime: anchorState.genesisTime, + dbLocation: opts.db.name, + }, + config, + metrics, + logger: logger.child({module: LoggerModule.chain}), + }); + const chain = new BeaconChain(opts.chain, { config, db, @@ -219,15 +229,7 @@ export class BeaconNode { executionBuilder: opts.executionBuilder.enabled ? initializeExecutionBuilder(opts.executionBuilder, config, metrics) : undefined, - historicalStateRegen: await HistoricalStateRegen.init({ - opts: { - genesisTime: anchorState.genesisTime, - dbLocation: opts.db.name, - }, - config, - metrics, - logger: logger.child({module: LoggerModule.chain}), - }), + historicalStateRegen, }); // Load persisted data from disk to in-memory caches @@ -286,7 +288,7 @@ export class BeaconNode { const metricsServer = opts.metrics.enabled ? await getHttpMetricsServer(opts.metrics, { register: (metrics as Metrics).register, - getOtherMetrics: () => network.scrapeMetrics(), + getOtherMetrics: async () => Promise.all([network.scrapeMetrics(), historicalStateRegen.scrapeMetrics()]), logger: logger.child({module: LoggerModule.metrics}), }) : null; From 93c1981653371d661a1279ccb9235c78198becd5 Mon Sep 17 00:00:00 2001 From: Cayman Date: Fri, 13 Oct 2023 16:19:05 -0400 Subject: [PATCH 03/23] chore: make historical state regen module optional --- packages/beacon-node/src/chain/chain.ts | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/packages/beacon-node/src/chain/chain.ts b/packages/beacon-node/src/chain/chain.ts index 2037dba5977f..2aadd74efd14 100644 --- a/packages/beacon-node/src/chain/chain.ts +++ b/packages/beacon-node/src/chain/chain.ts @@ -106,7 +106,7 @@ export class BeaconChain implements IBeaconChain { readonly regen: QueuedStateRegenerator; readonly lightClientServer: LightClientServer; readonly reprocessController: ReprocessController; - readonly historicalStateRegen: HistoricalStateRegen; + readonly historicalStateRegen?: HistoricalStateRegen; // Ops pool readonly attestationPool: AttestationPool; @@ -177,7 +177,7 @@ export class BeaconChain implements IBeaconChain { eth1: IEth1ForBlockProduction; executionEngine: IExecutionEngine; executionBuilder?: IExecutionBuilder; - historicalStateRegen: HistoricalStateRegen; + historicalStateRegen?: HistoricalStateRegen; } ) { this.opts = opts; @@ -396,7 +396,10 @@ export class BeaconChain implements IBeaconChain { } } else { // request for finalized state using historical state regen - const stateSerialized = await this.historicalStateRegen.getHistoricalState(slot); + const stateSerialized = await this.historicalStateRegen?.getHistoricalState(slot); + if (!stateSerialized) { + return null; + } const state = this.config .getForkTypes(slot) .BeaconState.deserialize(stateSerialized) as unknown as BeaconStateAllForks; From 06e58afc40cb5e5c3ab79b36981332b229252748 Mon Sep 17 00:00:00 2001 From: Cayman Date: Fri, 13 Oct 2023 16:19:24 -0400 Subject: [PATCH 04/23] chore: persist pubkey cache across historical state regen runs --- .../historicalState/getHistoricalState.ts | 42 +++++++++++++++---- .../src/chain/historicalState/worker.ts | 6 ++- 2 files changed, 37 insertions(+), 11 deletions(-) diff --git a/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts b/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts index 31e83ad53cf1..841408d971ac 100644 --- a/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts +++ b/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts @@ -1,4 +1,5 @@ import { + BeaconStateAllForks, BeaconStateTransitionMetrics, CachedBeaconStateAllForks, DataAvailableStatus, @@ -11,20 +12,42 @@ import {SignedBeaconBlock} from "@lodestar/types/allForks"; import {BeaconConfig} from "@lodestar/config"; import {IBeaconDb} from "../../db/index.js"; -export async function getClosestState( +export function syncPubkeyCache(state: BeaconStateAllForks, pubkey2index: PubkeyIndexMap): void { + // Get the validators sub tree once for all the loop + const validators = state.validators; + + const newCount = state.validators.length; + for (let i = pubkey2index.size; i < newCount; i++) { + const pubkey = validators.getReadonly(i).pubkey; + pubkey2index.set(pubkey, i); + } +} + +export async function getNearestState( slot: number, config: BeaconConfig, - db: IBeaconDb + db: IBeaconDb, + pubkey2index: PubkeyIndexMap ): Promise { const states = await db.stateArchive.values({limit: 1, lte: slot, reverse: true}); if (!states.length) { - throw new Error("No close state found in the database"); + throw new Error("No near state found in the database"); } - return createCachedBeaconState(states[0], { - config, - pubkey2index: new PubkeyIndexMap(), - index2pubkey: [], - }); + + const state = states[0]; + syncPubkeyCache(state, pubkey2index); + + return createCachedBeaconState( + state, + { + config, + pubkey2index, + index2pubkey: [], + }, + { + skipSyncPubkeys: true, + } + ); } export function getBlocksBetween(from: number, to: number, db: IBeaconDb): AsyncIterable { @@ -35,9 +58,10 @@ export async function getHistoricalState( slot: number, config: BeaconConfig, db: IBeaconDb, + pubkey2index: PubkeyIndexMap, metrics?: BeaconStateTransitionMetrics ): Promise { - let state = await getClosestState(slot, config, db); + let state = await getNearestState(slot, config, db, pubkey2index); for await (const block of getBlocksBetween(state.slot, slot, db)) { state = stateTransition( state, diff --git a/packages/beacon-node/src/chain/historicalState/worker.ts b/packages/beacon-node/src/chain/historicalState/worker.ts index eb01e523e37e..dd5086c99ee8 100644 --- a/packages/beacon-node/src/chain/historicalState/worker.ts +++ b/packages/beacon-node/src/chain/historicalState/worker.ts @@ -2,7 +2,7 @@ import worker from "node:worker_threads"; import {expose} from "@chainsafe/threads"; import {createBeaconConfig, chainConfigFromJson} from "@lodestar/config"; import {getNodeLogger} from "@lodestar/logger/node"; -import {BeaconStateTransitionMetrics} from "@lodestar/state-transition"; +import {BeaconStateTransitionMetrics, PubkeyIndexMap} from "@lodestar/state-transition"; import {LevelDbController} from "@lodestar/db"; import {RegistryMetricCreator, collectNodeJSMetrics} from "../../metrics/index.js"; import {JobFnQueue} from "../../util/queue/fnQueue.js"; @@ -143,6 +143,8 @@ const queue = new JobFnQueue( queueMetrics ); +const pubkey2index = new PubkeyIndexMap(); + const api: HistoricalStateWorkerApi = { async close() { abortController.abort(); @@ -151,7 +153,7 @@ const api: HistoricalStateWorkerApi = { return metricsRegister?.metrics() ?? ""; }, async getHistoricalState(slot) { - return queue.push(() => _getHistoricalState(slot, config, db, stateTransitionMetrics)); + return queue.push(() => _getHistoricalState(slot, config, db, pubkey2index, stateTransitionMetrics)); }, }; From 623b58c17d9726a3ddc67525d8b4ebb2c404ed96 Mon Sep 17 00:00:00 2001 From: Cayman Date: Fri, 13 Oct 2023 16:24:42 -0400 Subject: [PATCH 05/23] chore: cleanup worker termination --- .../beacon-node/src/chain/historicalState/index.ts | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/packages/beacon-node/src/chain/historicalState/index.ts b/packages/beacon-node/src/chain/historicalState/index.ts index 0f17513ccfc7..3dd05c47430e 100644 --- a/packages/beacon-node/src/chain/historicalState/index.ts +++ b/packages/beacon-node/src/chain/historicalState/index.ts @@ -1,7 +1,6 @@ -import {ModuleThread, spawn} from "@chainsafe/threads"; +import {ModuleThread, Thread, spawn} from "@chainsafe/threads"; import {chainConfigToJson} from "@lodestar/config"; import {LoggerNode} from "@lodestar/logger/node"; -import {terminateWorkerThread} from "../../util/workerEvents.js"; import { HistoricalStateRegenInitModules, HistoricalStateRegenModules, @@ -9,9 +8,6 @@ import { HistoricalStateWorkerData, } from "./types.js"; -const HISTORICAL_STATE_WORKER_EXIT_RETRY_COUNT = 3; -const HISTORICAL_STATE_WORKER_EXIT_TIMEOUT_MS = 1000; - /** * HistoricalStateRegen limits the damage from recreating historical states * by running regen in a separate worker thread. @@ -57,12 +53,7 @@ export class HistoricalStateRegen implements HistoricalStateWorkerApi { async close(): Promise { await this.api.close(); this.logger.debug("Terminating historical state worker"); - await terminateWorkerThread({ - worker: this.api, - retryCount: HISTORICAL_STATE_WORKER_EXIT_RETRY_COUNT, - retryMs: HISTORICAL_STATE_WORKER_EXIT_TIMEOUT_MS, - logger: this.logger, - }); + await Thread.terminate(this.api); this.logger.debug("Terminated historical state worker"); } From b8d611302313ce671f3bddb1a791046ad2de7099 Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 23 Oct 2023 11:51:33 -0400 Subject: [PATCH 06/23] chore: fix worker usage --- packages/beacon-node/src/chain/historicalState/index.ts | 2 +- packages/beacon-node/src/chain/historicalState/worker.ts | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/packages/beacon-node/src/chain/historicalState/index.ts b/packages/beacon-node/src/chain/historicalState/index.ts index 3dd05c47430e..dee15324b6ab 100644 --- a/packages/beacon-node/src/chain/historicalState/index.ts +++ b/packages/beacon-node/src/chain/historicalState/index.ts @@ -1,4 +1,4 @@ -import {ModuleThread, Thread, spawn} from "@chainsafe/threads"; +import {ModuleThread, Thread, spawn, Worker} from "@chainsafe/threads"; import {chainConfigToJson} from "@lodestar/config"; import {LoggerNode} from "@lodestar/logger/node"; import { diff --git a/packages/beacon-node/src/chain/historicalState/worker.ts b/packages/beacon-node/src/chain/historicalState/worker.ts index dd5086c99ee8..3bc6fdf29846 100644 --- a/packages/beacon-node/src/chain/historicalState/worker.ts +++ b/packages/beacon-node/src/chain/historicalState/worker.ts @@ -1,5 +1,5 @@ import worker from "node:worker_threads"; -import {expose} from "@chainsafe/threads"; +import {expose} from "@chainsafe/threads/worker"; import {createBeaconConfig, chainConfigFromJson} from "@lodestar/config"; import {getNodeLogger} from "@lodestar/logger/node"; import {BeaconStateTransitionMetrics, PubkeyIndexMap} from "@lodestar/state-transition"; From 0315e1a33425b01a50a8a833966f7bdf4e7c27a5 Mon Sep 17 00:00:00 2001 From: Matthew Keil Date: Mon, 6 Nov 2023 22:04:09 +0300 Subject: [PATCH 07/23] fix: swap Level for ClassicLevel for multithreading --- packages/db/package.json | 2 +- packages/db/src/controller/level.ts | 10 +++++----- yarn.lock | 15 +++++++++++++++ 3 files changed, 21 insertions(+), 6 deletions(-) diff --git a/packages/db/package.json b/packages/db/package.json index a8e48a5eb020..a9207ba80522 100644 --- a/packages/db/package.json +++ b/packages/db/package.json @@ -42,7 +42,7 @@ "@lodestar/utils": "^1.11.3", "@types/levelup": "^4.3.3", "it-all": "^3.0.2", - "level": "^8.0.0" + "classic-level": "matthewkeil/classic-level#temp-dep-install" }, "devDependencies": { "@lodestar/logger": "^1.11.3" diff --git a/packages/db/src/controller/level.ts b/packages/db/src/controller/level.ts index 3eed75958e3e..daec8a1afd3b 100644 --- a/packages/db/src/controller/level.ts +++ b/packages/db/src/controller/level.ts @@ -1,5 +1,4 @@ -import {Level} from "level"; -import type {ClassicLevel} from "classic-level"; +import {ClassicLevel} from "classic-level"; import {Logger} from "@lodestar/utils"; import {DbReqOpts, DatabaseController, DatabaseOptions, FilterOptions, KeyValue} from "./interface.js"; import {LevelDbControllerMetrics} from "./metrics.js"; @@ -12,7 +11,7 @@ enum Status { type LevelNodeJS = ClassicLevel; export interface LevelDBOptions extends DatabaseOptions { - db?: Level; + db?: ClassicLevel; } export type LevelDbControllerModules = { @@ -35,7 +34,7 @@ export class LevelDbController implements DatabaseController, + private readonly db: ClassicLevel, private metrics: LevelDbControllerMetrics | null ) { this.metrics = metrics ?? null; @@ -46,7 +45,8 @@ export class LevelDbController implements DatabaseController { - const db = opts.db || new Level(opts.name || "beaconchain", {keyEncoding: "binary", valueEncoding: "binary"}); + const db = + opts.db || new ClassicLevel(opts.name || "beaconchain", {keyEncoding: "binary", valueEncoding: "binary", allowMultiThreading: true}); try { await db.open(); diff --git a/yarn.lock b/yarn.lock index 60d316df8964..d6e00ac38790 100644 --- a/yarn.lock +++ b/yarn.lock @@ -4933,6 +4933,16 @@ classic-level@^1.2.0: napi-macros "~2.0.0" node-gyp-build "^4.3.0" +classic-level@matthewkeil/classic-level#temp-dep-install: + version "1.3.0" + resolved "https://codeload.github.com/matthewkeil/classic-level/tar.gz/9bd7bfbc3c0ee642cd50970fef0abc21ea71cffb" + dependencies: + abstract-level "^1.0.2" + catering "^2.1.0" + module-error "^1.0.1" + napi-macros "^2.2.2" + node-gyp-build "^4.3.0" + clean-stack@^2.0.0: version "2.2.0" resolved "https://registry.yarnpkg.com/clean-stack/-/clean-stack-2.2.0.tgz#ee8472dbb129e727b31e8a10a427dee9dfe4008b" @@ -10268,6 +10278,11 @@ nanoid@^4.0.0: resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-4.0.2.tgz#140b3c5003959adbebf521c170f282c5e7f9fb9e" integrity sha512-7ZtY5KTCNheRGfEFxnedV5zFiORN1+Y1N6zvPTnHQd8ENUvfaDBeuJDZb2bN/oXwXxu3qkTXDzy57W5vAmDTBw== +napi-macros@^2.2.2: + version "2.2.2" + resolved "https://registry.yarnpkg.com/napi-macros/-/napi-macros-2.2.2.tgz#817fef20c3e0e40a963fbf7b37d1600bd0201044" + integrity sha512-hmEVtAGYzVQpCKdbQea4skABsdXW4RUh5t5mJ2zzqowJS2OyXZTU1KhDVFhx+NlWZ4ap9mqR9TcDO3LTTttd+g== + napi-macros@~2.0.0: version "2.0.0" resolved "https://registry.npmjs.org/napi-macros/-/napi-macros-2.0.0.tgz" From cdf37324b6dbd2c37fad0aa1105e2913f60b5647 Mon Sep 17 00:00:00 2001 From: Matthew Keil Date: Mon, 6 Nov 2023 22:34:55 +0300 Subject: [PATCH 08/23] fix: getStateV2 state handling hack --- .../beacon-node/src/api/impl/debug/index.ts | 17 +++++++++++++++-- packages/db/src/controller/level.ts | 7 ++++++- 2 files changed, 21 insertions(+), 3 deletions(-) diff --git a/packages/beacon-node/src/api/impl/debug/index.ts b/packages/beacon-node/src/api/impl/debug/index.ts index 22ba4e607c6b..3e05fc89c3c2 100644 --- a/packages/beacon-node/src/api/impl/debug/index.ts +++ b/packages/beacon-node/src/api/impl/debug/index.ts @@ -1,4 +1,5 @@ import {routes, ServerApi, ResponseFormat} from "@lodestar/api"; +import {allForks} from "@lodestar/types"; import {resolveStateId} from "../beacon/state/utils.js"; import {ApiModules} from "../types.js"; import {isOptimisticBlock} from "../../../util/forkChoice.js"; @@ -51,10 +52,22 @@ export function getDebugApi({chain, config}: Pick { const db = - opts.db || new ClassicLevel(opts.name || "beaconchain", {keyEncoding: "binary", valueEncoding: "binary", allowMultiThreading: true}); + opts.db || + new ClassicLevel(opts.name || "beaconchain", { + keyEncoding: "binary", + valueEncoding: "binary", + allowMultiThreading: true, + }); try { await db.open(); From 6d9d2562bbdf5af7b6814da43eb1c15f49760c79 Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 22 Jan 2024 10:32:42 -0500 Subject: [PATCH 09/23] chore: update classic-level --- packages/db/package.json | 2 +- yarn.lock | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/packages/db/package.json b/packages/db/package.json index c5bc93dbb920..aeb7e461f1f3 100644 --- a/packages/db/package.json +++ b/packages/db/package.json @@ -42,7 +42,7 @@ "@lodestar/utils": "^1.14.0", "@types/levelup": "^4.3.3", "it-all": "^3.0.4", - "classic-level": "matthewkeil/classic-level#temp-dep-install" + "classic-level": "^1.4.1" }, "devDependencies": { "@lodestar/logger": "^1.14.0" diff --git a/yarn.lock b/yarn.lock index 41f5d72b1eb9..f4793436c96a 100644 --- a/yarn.lock +++ b/yarn.lock @@ -5328,9 +5328,10 @@ classic-level@^1.2.0: napi-macros "~2.0.0" node-gyp-build "^4.3.0" -classic-level@matthewkeil/classic-level#temp-dep-install: - version "1.3.0" - resolved "https://codeload.github.com/matthewkeil/classic-level/tar.gz/9bd7bfbc3c0ee642cd50970fef0abc21ea71cffb" +classic-level@^1.4.1: + version "1.4.1" + resolved "https://registry.yarnpkg.com/classic-level/-/classic-level-1.4.1.tgz#169ecf9f9c6200ad42a98c8576af449c1badbaee" + integrity sha512-qGx/KJl3bvtOHrGau2WklEZuXhS3zme+jf+fsu6Ej7W7IP/C49v7KNlWIsT1jZu0YnfzSIYDGcEWpCa1wKGWXQ== dependencies: abstract-level "^1.0.2" catering "^2.1.0" From 5a812e1ee397c2ef3550464a8971732193da0a9e Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 22 Jan 2024 10:44:04 -0500 Subject: [PATCH 10/23] chore: fix build errors --- .../src/chain/historicalState/worker.ts | 35 +++++++++++++------ packages/db/src/controller/level.ts | 2 +- 2 files changed, 25 insertions(+), 12 deletions(-) diff --git a/packages/beacon-node/src/chain/historicalState/worker.ts b/packages/beacon-node/src/chain/historicalState/worker.ts index 3bc6fdf29846..918277c0ad0b 100644 --- a/packages/beacon-node/src/chain/historicalState/worker.ts +++ b/packages/beacon-node/src/chain/historicalState/worker.ts @@ -2,7 +2,13 @@ import worker from "node:worker_threads"; import {expose} from "@chainsafe/threads/worker"; import {createBeaconConfig, chainConfigFromJson} from "@lodestar/config"; import {getNodeLogger} from "@lodestar/logger/node"; -import {BeaconStateTransitionMetrics, PubkeyIndexMap} from "@lodestar/state-transition"; +import { + BeaconStateTransitionMetrics, + EpochTransitionStep, + PubkeyIndexMap, + StateCloneSource, + StateHashTreeRootSource, +} from "@lodestar/state-transition"; import {LevelDbController} from "@lodestar/db"; import {RegistryMetricCreator, collectNodeJSMetrics} from "../../metrics/index.js"; import {JobFnQueue} from "../../util/queue/fnQueue.js"; @@ -47,6 +53,12 @@ if (metricsRegister) { help: "Time to call commit after process a single epoch transition in seconds", buckets: [0.01, 0.05, 0.1, 0.2, 0.5, 0.75, 1], }), + epochTransitionStepTime: metricsRegister.histogram<{step: EpochTransitionStep}>({ + name: "lodestar_stfn_epoch_transition_step_seconds", + help: "Time to call each step of epoch transition in seconds", + labelNames: ["step"], + buckets: [0.01, 0.05, 0.1, 0.2, 0.5, 0.75, 1], + }), processBlockTime: metricsRegister.histogram({ name: "lodestar_historical_state_stfn_process_block_seconds", help: "Time to process a single block in seconds", @@ -59,28 +71,29 @@ if (metricsRegister) { help: "Time to call commit after process a single block in seconds", buckets: [0.005, 0.01, 0.02, 0.05, 0.1, 1], }), - stateHashTreeRootTime: metricsRegister.histogram({ + stateHashTreeRootTime: metricsRegister.histogram<{source: StateHashTreeRootSource}>({ name: "lodestar_stfn_hash_tree_root_seconds", help: "Time to compute the hash tree root of a post state in seconds", - buckets: [0.005, 0.01, 0.02, 0.05, 0.1, 1], + buckets: [0.05, 0.1, 0.2, 0.5, 1, 1.5], + labelNames: ["source"], }), - preStateBalancesNodesPopulatedMiss: metricsRegister.gauge<"source">({ - name: "lodestar_historical_state_stfn_balances_nodes_populated_miss_total", + preStateBalancesNodesPopulatedMiss: metricsRegister.gauge<{source: StateCloneSource}>({ + name: "lodestar_stfn_balances_nodes_populated_miss_total", help: "Total count state.balances nodesPopulated is false on stfn", labelNames: ["source"], }), - preStateBalancesNodesPopulatedHit: metricsRegister.gauge<"source">({ - name: "lodestar_historical_state_stfn_balances_nodes_populated_hit_total", + preStateBalancesNodesPopulatedHit: metricsRegister.gauge<{source: StateCloneSource}>({ + name: "lodestar_stfn_balances_nodes_populated_hit_total", help: "Total count state.balances nodesPopulated is true on stfn", labelNames: ["source"], }), - preStateValidatorsNodesPopulatedMiss: metricsRegister.gauge<"source">({ - name: "lodestar_historical_state_stfn_validators_nodes_populated_miss_total", + preStateValidatorsNodesPopulatedMiss: metricsRegister.gauge<{source: StateCloneSource}>({ + name: "lodestar_stfn_validators_nodes_populated_miss_total", help: "Total count state.validators nodesPopulated is false on stfn", labelNames: ["source"], }), - preStateValidatorsNodesPopulatedHit: metricsRegister.gauge<"source">({ - name: "lodestar_historical_state_stfn_validators_nodes_populated_hit_total", + preStateValidatorsNodesPopulatedHit: metricsRegister.gauge<{source: StateCloneSource}>({ + name: "lodestar_stfn_validators_nodes_populated_hit_total", help: "Total count state.validators nodesPopulated is true on stfn", labelNames: ["source"], }), diff --git a/packages/db/src/controller/level.ts b/packages/db/src/controller/level.ts index a6a212a8eebb..830c225c304c 100644 --- a/packages/db/src/controller/level.ts +++ b/packages/db/src/controller/level.ts @@ -50,7 +50,7 @@ export class LevelDbController implements DatabaseController Date: Mon, 22 Jan 2024 11:20:59 -0500 Subject: [PATCH 11/23] chore: add comments --- .../src/chain/historicalState/getHistoricalState.ts | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts b/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts index 841408d971ac..3b5a672078dd 100644 --- a/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts +++ b/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts @@ -12,6 +12,9 @@ import {SignedBeaconBlock} from "@lodestar/types/allForks"; import {BeaconConfig} from "@lodestar/config"; import {IBeaconDb} from "../../db/index.js"; +/** + * Populate a PubkeyIndexMap with any new entries based on a BeaconState + */ export function syncPubkeyCache(state: BeaconStateAllForks, pubkey2index: PubkeyIndexMap): void { // Get the validators sub tree once for all the loop const validators = state.validators; @@ -23,6 +26,9 @@ export function syncPubkeyCache(state: BeaconStateAllForks, pubkey2index: Pubkey } } +/** + * Get the nearest BeaconState at or before a slot + */ export async function getNearestState( slot: number, config: BeaconConfig, @@ -50,10 +56,16 @@ export async function getNearestState( ); } +/** + * Get blocks (from, to] + */ export function getBlocksBetween(from: number, to: number, db: IBeaconDb): AsyncIterable { return db.blockArchive.valuesStream({gt: from, lte: to}); } +/** + * Get and regenerate a historical state + */ export async function getHistoricalState( slot: number, config: BeaconConfig, From 51fc29a9d94d0b75f7c489ca34366b7781495431 Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 22 Jan 2024 11:38:48 -0500 Subject: [PATCH 12/23] chore: fix test worker path --- packages/beacon-node/src/chain/historicalState/index.ts | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/packages/beacon-node/src/chain/historicalState/index.ts b/packages/beacon-node/src/chain/historicalState/index.ts index dee15324b6ab..026379a29f0b 100644 --- a/packages/beacon-node/src/chain/historicalState/index.ts +++ b/packages/beacon-node/src/chain/historicalState/index.ts @@ -1,3 +1,4 @@ +import path from "node:path"; import {ModuleThread, Thread, spawn, Worker} from "@chainsafe/threads"; import {chainConfigToJson} from "@lodestar/config"; import {LoggerNode} from "@lodestar/logger/node"; @@ -8,6 +9,9 @@ import { HistoricalStateWorkerData, } from "./types.js"; +// Worker constructor consider the path relative to the current working directory +const WORKER_DIR = process.env.NODE_ENV === "test" ? "../../../lib/chain/historicalState" : "./"; + /** * HistoricalStateRegen limits the damage from recreating historical states * by running regen in a separate worker thread. @@ -32,7 +36,7 @@ export class HistoricalStateRegen implements HistoricalStateWorkerApi { loggerOpts: modules.logger.toOpts(), }; - const worker = new Worker("./worker.js", { + const worker = new Worker(path.join(WORKER_DIR, "worker.js"), { workerData, } as ConstructorParameters[1]); From 21bb96bc0716ea3dfa29892443369d14810b07fd Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 22 Jan 2024 11:39:00 -0500 Subject: [PATCH 13/23] chore: simplify function naming --- packages/beacon-node/src/chain/historicalState/worker.ts | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/packages/beacon-node/src/chain/historicalState/worker.ts b/packages/beacon-node/src/chain/historicalState/worker.ts index 918277c0ad0b..4a26df8ca8a1 100644 --- a/packages/beacon-node/src/chain/historicalState/worker.ts +++ b/packages/beacon-node/src/chain/historicalState/worker.ts @@ -15,7 +15,7 @@ import {JobFnQueue} from "../../util/queue/fnQueue.js"; import {QueueMetrics} from "../../util/queue/options.js"; import {BeaconDb} from "../../db/index.js"; import {HistoricalStateWorkerApi, HistoricalStateWorkerData} from "./types.js"; -import {getHistoricalState as _getHistoricalState} from "./getHistoricalState.js"; +import {getHistoricalState} from "./getHistoricalState.js"; // most of this setup copied from networkCoreWorker.ts @@ -166,7 +166,7 @@ const api: HistoricalStateWorkerApi = { return metricsRegister?.metrics() ?? ""; }, async getHistoricalState(slot) { - return queue.push(() => _getHistoricalState(slot, config, db, pubkey2index, stateTransitionMetrics)); + return queue.push(() => getHistoricalState(slot, config, db, pubkey2index, stateTransitionMetrics)); }, }; From dc45097502368dcd422f7aa61587c5cfb4403af3 Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 22 Jan 2024 13:59:34 -0500 Subject: [PATCH 14/23] chore: optimize getSlotFromOffset --- packages/beacon-node/src/util/sszBytes.ts | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/packages/beacon-node/src/util/sszBytes.ts b/packages/beacon-node/src/util/sszBytes.ts index cd12c4bd9c18..30a7ff476e73 100644 --- a/packages/beacon-node/src/util/sszBytes.ts +++ b/packages/beacon-node/src/util/sszBytes.ts @@ -201,8 +201,6 @@ export function getSlotFromBlobSidecarSerialized(data: Uint8Array): Slot | null } function getSlotFromOffset(data: Uint8Array, offset: number): Slot { - // TODO: Optimize - const dv = new DataView(data.buffer, data.byteOffset, data.byteLength); // Read only the first 4 bytes of Slot, max value is 4,294,967,295 will be reached 1634 years after genesis - return dv.getUint32(offset, true); + return (data[offset] | (data[offset + 1] << 8) | (data[offset + 2] << 16) | (data[offset + 3] << 24)) >>> 0; } From d32f1031c60cdbc1dad99a9a0784af71ff11b401 Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 22 Jan 2024 16:22:45 -0500 Subject: [PATCH 15/23] chore: refactor to avoid needless deserialization --- packages/api/src/beacon/routes/debug.ts | 2 +- .../src/api/impl/beacon/state/index.ts | 14 +-- .../src/api/impl/beacon/state/utils.ts | 85 ++++++++++++------- .../beacon-node/src/api/impl/debug/index.ts | 40 +++++---- .../beacon-node/src/api/impl/proof/index.ts | 4 +- packages/beacon-node/src/chain/chain.ts | 25 ++++-- packages/beacon-node/src/chain/interface.ts | 2 + packages/beacon-node/src/util/sszBytes.ts | 12 +++ 8 files changed, 118 insertions(+), 66 deletions(-) diff --git a/packages/api/src/beacon/routes/debug.ts b/packages/api/src/beacon/routes/debug.ts index 84eed0af04c9..727e6e37d37b 100644 --- a/packages/api/src/beacon/routes/debug.ts +++ b/packages/api/src/beacon/routes/debug.ts @@ -92,7 +92,7 @@ export type Api = { format?: ResponseFormat ): Promise< ApiClientResponse<{ - [HttpStatusCode.OK]: Uint8Array | {data: allForks.BeaconState; executionOptimistic: ExecutionOptimistic}; + [HttpStatusCode.OK]: Uint8Array | {data: allForks.BeaconState}; }> >; diff --git a/packages/beacon-node/src/api/impl/beacon/state/index.ts b/packages/beacon-node/src/api/impl/beacon/state/index.ts index c9f74b45a9f2..39f353ca0164 100644 --- a/packages/beacon-node/src/api/impl/beacon/state/index.ts +++ b/packages/beacon-node/src/api/impl/beacon/state/index.ts @@ -14,7 +14,7 @@ import { filterStateValidatorsByStatus, getStateValidatorIndex, getValidatorStatus, - resolveStateId, + getStateResponse, toValidatorResponse, } from "./utils.js"; @@ -25,7 +25,7 @@ export function getBeaconStateApi({ async function getState( stateId: routes.beacon.StateId ): Promise<{state: BeaconStateAllForks; executionOptimistic: boolean}> { - return resolveStateId(chain, stateId); + return getStateResponse(chain, stateId); } return { @@ -77,7 +77,7 @@ export function getBeaconStateApi({ }, async getStateValidators(stateId, filters) { - const {state, executionOptimistic} = await resolveStateId(chain, stateId); + const {state, executionOptimistic} = await getStateResponse(chain, stateId); const currentEpoch = getCurrentEpoch(state); const {validators, balances} = state; // Get the validators sub tree once for all the loop const {pubkey2index} = chain.getHeadState().epochCtx; @@ -128,7 +128,7 @@ export function getBeaconStateApi({ }, async getStateValidator(stateId, validatorId) { - const {state, executionOptimistic} = await resolveStateId(chain, stateId); + const {state, executionOptimistic} = await getStateResponse(chain, stateId); const {pubkey2index} = chain.getHeadState().epochCtx; const resp = getStateValidatorIndex(validatorId, state, pubkey2index); @@ -149,7 +149,7 @@ export function getBeaconStateApi({ }, async getStateValidatorBalances(stateId, indices) { - const {state, executionOptimistic} = await resolveStateId(chain, stateId); + const {state, executionOptimistic} = await getStateResponse(chain, stateId); if (indices) { const headState = chain.getHeadState(); @@ -186,7 +186,7 @@ export function getBeaconStateApi({ }, async getEpochCommittees(stateId, filters) { - const {state, executionOptimistic} = await resolveStateId(chain, stateId); + const {state, executionOptimistic} = await getStateResponse(chain, stateId); const stateCached = state as CachedBeaconStateAltair; if (stateCached.epochCtx === undefined) { @@ -228,7 +228,7 @@ export function getBeaconStateApi({ */ async getEpochSyncCommittees(stateId, epoch) { // TODO: Should pick a state with the provided epoch too - const {state, executionOptimistic} = await resolveStateId(chain, stateId); + const {state, executionOptimistic} = await getStateResponse(chain, stateId); // TODO: If possible compute the syncCommittees in advance of the fork and expose them here. // So the validators can prepare and potentially attest the first block. Not critical tho, it's very unlikely diff --git a/packages/beacon-node/src/api/impl/beacon/state/utils.ts b/packages/beacon-node/src/api/impl/beacon/state/utils.ts index 3c17fa30ac67..0004b4bd1fba 100644 --- a/packages/beacon-node/src/api/impl/beacon/state/utils.ts +++ b/packages/beacon-node/src/api/impl/beacon/state/utils.ts @@ -1,56 +1,41 @@ import {fromHexString} from "@chainsafe/ssz"; +import {ChainForkConfig} from "@lodestar/config"; import {routes} from "@lodestar/api"; import {FAR_FUTURE_EPOCH, GENESIS_SLOT} from "@lodestar/params"; import {BeaconStateAllForks, PubkeyIndexMap} from "@lodestar/state-transition"; -import {BLSPubkey, phase0} from "@lodestar/types"; +import {BLSPubkey, allForks, phase0} from "@lodestar/types"; import {Epoch, ValidatorIndex} from "@lodestar/types"; -import {IBeaconChain, StateGetOpts} from "../../../../chain/index.js"; +import {IBeaconChain} from "../../../../chain/index.js"; import {ApiError, ValidationError} from "../../errors.js"; -import {isOptimisticBlock} from "../../../../util/forkChoice.js"; +import {getSlotFromBeaconStateSerialized} from "../../../../util/sszBytes.js"; -export async function resolveStateId( - chain: IBeaconChain, - stateId: routes.beacon.StateId, - opts?: StateGetOpts -): Promise<{state: BeaconStateAllForks; executionOptimistic: boolean}> { - const stateRes = await resolveStateIdOrNull(chain, stateId, opts); - if (!stateRes) { - throw new ApiError(404, `No state found for id '${stateId}'`); - } - - return stateRes; +export function deserializeBeaconStateSerialized(config: ChainForkConfig, data: Uint8Array): allForks.BeaconState { + const slot = getSlotFromBeaconStateSerialized(data); + return config.getForkTypes(slot).BeaconState.deserialize(data); } -async function resolveStateIdOrNull( - chain: IBeaconChain, - stateId: routes.beacon.StateId, - opts?: StateGetOpts -): Promise<{state: BeaconStateAllForks; executionOptimistic: boolean} | null> { +export function resolveStateId(chain: IBeaconChain, stateId: routes.beacon.StateId): string | number { if (stateId === "head") { - // TODO: This is not OK, head and headState must be fetched atomically const head = chain.forkChoice.getHead(); - const headState = chain.getHeadState(); - return {state: headState, executionOptimistic: isOptimisticBlock(head)}; + return head.stateRoot; } if (stateId === "genesis") { - return chain.getStateBySlot(GENESIS_SLOT, opts); + return GENESIS_SLOT; } if (stateId === "finalized") { const block = chain.forkChoice.getFinalizedBlock(); - const state = await chain.getStateByStateRoot(block.stateRoot, opts); - return state && {state: state.state, executionOptimistic: isOptimisticBlock(block)}; + return block.stateRoot; } if (stateId === "justified") { const block = chain.forkChoice.getJustifiedBlock(); - const state = await chain.getStateByStateRoot(block.stateRoot, opts); - return state && {state: state.state, executionOptimistic: isOptimisticBlock(block)}; + return block.stateRoot; } if (typeof stateId === "string" && stateId.startsWith("0x")) { - return chain.getStateByStateRoot(stateId, opts); + return stateId as string; } // id must be slot @@ -59,7 +44,49 @@ async function resolveStateIdOrNull( throw new ValidationError(`Invalid block id '${stateId}'`, "blockId"); } - return chain.getStateBySlot(blockSlot, opts); + return blockSlot; +} + +export async function getStateResponse( + chain: IBeaconChain, + stateId: routes.beacon.StateId +): Promise<{state: BeaconStateAllForks; executionOptimistic: boolean}> { + const rootOrSlot = resolveStateId(chain, stateId); + + let state: {state: BeaconStateAllForks; executionOptimistic: boolean} | null = null; + if (typeof rootOrSlot === "string") { + state = await chain.getStateByStateRoot(rootOrSlot); + } else if (typeof rootOrSlot === "number") { + state = await chain.getStateBySlot(rootOrSlot); + } + + if (state == null) { + throw new ApiError(404, `No state found for id '${stateId}'`); + } + return state; +} + +export async function getStateResponseWithRegen( + chain: IBeaconChain, + stateId: routes.beacon.StateId +): Promise<{state: BeaconStateAllForks | Uint8Array; executionOptimistic: boolean}> { + const rootOrSlot = resolveStateId(chain, stateId); + + let state: {state: BeaconStateAllForks | Uint8Array; executionOptimistic: boolean} | null = null; + if (typeof rootOrSlot === "string") { + state = await chain.getStateByStateRoot(rootOrSlot, {allowRegen: true}); + } else if (typeof rootOrSlot === "number") { + if (rootOrSlot >= chain.forkChoice.getFinalizedBlock().slot) { + state = await chain.getStateBySlot(rootOrSlot, {allowRegen: true}); + } else { + state = await chain.getHistoricalStateBySlot(rootOrSlot); + } + } + + if (state == null) { + throw new ApiError(404, `No state found for id '${stateId}'`); + } + return state; } /** diff --git a/packages/beacon-node/src/api/impl/debug/index.ts b/packages/beacon-node/src/api/impl/debug/index.ts index 3e05fc89c3c2..7d711aa9de32 100644 --- a/packages/beacon-node/src/api/impl/debug/index.ts +++ b/packages/beacon-node/src/api/impl/debug/index.ts @@ -1,6 +1,5 @@ import {routes, ServerApi, ResponseFormat} from "@lodestar/api"; -import {allForks} from "@lodestar/types"; -import {resolveStateId} from "../beacon/state/utils.js"; +import {deserializeBeaconStateSerialized, getStateResponseWithRegen} from "../beacon/state/utils.js"; import {ApiModules} from "../types.js"; import {isOptimisticBlock} from "../../../util/forkChoice.js"; @@ -38,35 +37,40 @@ export function getDebugApi({chain, config}: Pick { + const finalizedBlock = this.forkChoice.getFinalizedBlock(); - return {state, executionOptimistic: false}; + if (slot >= finalizedBlock.slot) { + return null; + } + + // request for finalized state using historical state regen + const stateSerialized = await this.historicalStateRegen?.getHistoricalState(slot); + if (!stateSerialized) { + return null; } + + return {state: stateSerialized, executionOptimistic: false}; } async getStateByStateRoot( diff --git a/packages/beacon-node/src/chain/interface.ts b/packages/beacon-node/src/chain/interface.ts index 3939457a8ac3..a672020ade18 100644 --- a/packages/beacon-node/src/chain/interface.ts +++ b/packages/beacon-node/src/chain/interface.ts @@ -131,6 +131,8 @@ export interface IBeaconChain { getHeadStateAtCurrentEpoch(regenCaller: RegenCaller): Promise; getHeadStateAtEpoch(epoch: Epoch, regenCaller: RegenCaller): Promise; + getHistoricalStateBySlot(slot: Slot): Promise<{state: Uint8Array; executionOptimistic: boolean} | null>; + /** Returns a local state canonical at `slot` */ getStateBySlot( slot: Slot, diff --git a/packages/beacon-node/src/util/sszBytes.ts b/packages/beacon-node/src/util/sszBytes.ts index 30a7ff476e73..603840516a62 100644 --- a/packages/beacon-node/src/util/sszBytes.ts +++ b/packages/beacon-node/src/util/sszBytes.ts @@ -6,6 +6,18 @@ import {BYTES_PER_FIELD_ELEMENT, FIELD_ELEMENTS_PER_BLOB} from "@lodestar/params export type BlockRootHex = RootHex; export type AttDataBase64 = string; +// class BeaconState(Container): +// genesis_time: uint64 - 8 +// genesis_validators_root: bytes32 - 32 +// slot: uint64 +// ... + +const STATE_SLOT_OFFSET = 40; + +export function getSlotFromBeaconStateSerialized(data: Uint8Array): Slot { + return getSlotFromOffset(data, STATE_SLOT_OFFSET); +} + // class Attestation(Container): // aggregation_bits: Bitlist[MAX_VALIDATORS_PER_COMMITTEE] - offset 4 // data: AttestationData - target data - 128 From 25b1c9f53eb49598946aef60be48537c6d62bf1c Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 22 Jan 2024 17:41:30 -0500 Subject: [PATCH 16/23] fix: update metrics names --- .../beacon-node/src/chain/historicalState/worker.ts | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/packages/beacon-node/src/chain/historicalState/worker.ts b/packages/beacon-node/src/chain/historicalState/worker.ts index 4a26df8ca8a1..be28246d4acd 100644 --- a/packages/beacon-node/src/chain/historicalState/worker.ts +++ b/packages/beacon-node/src/chain/historicalState/worker.ts @@ -54,7 +54,7 @@ if (metricsRegister) { buckets: [0.01, 0.05, 0.1, 0.2, 0.5, 0.75, 1], }), epochTransitionStepTime: metricsRegister.histogram<{step: EpochTransitionStep}>({ - name: "lodestar_stfn_epoch_transition_step_seconds", + name: "lodestar_historical_state_stfn_epoch_transition_step_seconds", help: "Time to call each step of epoch transition in seconds", labelNames: ["step"], buckets: [0.01, 0.05, 0.1, 0.2, 0.5, 0.75, 1], @@ -72,28 +72,28 @@ if (metricsRegister) { buckets: [0.005, 0.01, 0.02, 0.05, 0.1, 1], }), stateHashTreeRootTime: metricsRegister.histogram<{source: StateHashTreeRootSource}>({ - name: "lodestar_stfn_hash_tree_root_seconds", + name: "lodestar_historical_state_stfn_hash_tree_root_seconds", help: "Time to compute the hash tree root of a post state in seconds", buckets: [0.05, 0.1, 0.2, 0.5, 1, 1.5], labelNames: ["source"], }), preStateBalancesNodesPopulatedMiss: metricsRegister.gauge<{source: StateCloneSource}>({ - name: "lodestar_stfn_balances_nodes_populated_miss_total", + name: "lodestar_historical_state_stfn_balances_nodes_populated_miss_total", help: "Total count state.balances nodesPopulated is false on stfn", labelNames: ["source"], }), preStateBalancesNodesPopulatedHit: metricsRegister.gauge<{source: StateCloneSource}>({ - name: "lodestar_stfn_balances_nodes_populated_hit_total", + name: "lodestar_historical_state_stfn_balances_nodes_populated_hit_total", help: "Total count state.balances nodesPopulated is true on stfn", labelNames: ["source"], }), preStateValidatorsNodesPopulatedMiss: metricsRegister.gauge<{source: StateCloneSource}>({ - name: "lodestar_stfn_validators_nodes_populated_miss_total", + name: "lodestar_historical_state_stfn_validators_nodes_populated_miss_total", help: "Total count state.validators nodesPopulated is false on stfn", labelNames: ["source"], }), preStateValidatorsNodesPopulatedHit: metricsRegister.gauge<{source: StateCloneSource}>({ - name: "lodestar_stfn_validators_nodes_populated_hit_total", + name: "lodestar_historical_state_stfn_validators_nodes_populated_hit_total", help: "Total count state.validators nodesPopulated is true on stfn", labelNames: ["source"], }), From b6fd57750d22b5a0e7e04b65512a31344d56d26c Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 22 Jan 2024 18:03:02 -0500 Subject: [PATCH 17/23] feat: add historical state regen dashboard --- .../lodestar_historical_state_regen.json | 2646 +++++++++++++++++ 1 file changed, 2646 insertions(+) create mode 100644 dashboards/lodestar_historical_state_regen.json diff --git a/dashboards/lodestar_historical_state_regen.json b/dashboards/lodestar_historical_state_regen.json new file mode 100644 index 000000000000..20eddcd1f31e --- /dev/null +++ b/dashboards/lodestar_historical_state_regen.json @@ -0,0 +1,2646 @@ +{ + "__inputs": [ + { + "description": "", + "label": "Prometheus", + "name": "DS_PROMETHEUS", + "pluginId": "prometheus", + "pluginName": "Prometheus", + "type": "datasource" + }, + { + "description": "", + "label": "Beacon node job name", + "name": "VAR_BEACON_JOB", + "type": "constant", + "value": "beacon" + } + ], + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": { + "type": "datasource", + "uid": "grafana" + }, + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "target": { + "limit": 100, + "matchAny": false, + "tags": [], + "type": "dashboard" + }, + "type": "dashboard" + } + ] + }, + "editable": true, + "fiscalYearStartMonth": 0, + "graphTooltip": 1, + "id": null, + "links": [ + { + "asDropdown": true, + "icon": "external link", + "includeVars": true, + "keepTime": true, + "tags": [ + "lodestar" + ], + "targetBlank": false, + "title": "Lodestar dashboards", + "tooltip": "", + "type": "dashboards", + "url": "" + } + ], + "liveNow": false, + "panels": [ + { + "collapsed": false, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 535, + "panels": [], + "title": "VM", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 30, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "decbytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 1 + }, + "id": 536, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "lodestar_historical_state_worker_nodejs_heap_space_size_used_bytes{job=~\"$beacon_job|beacon\"}", + "interval": "", + "legendFormat": "{{space}}", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "lodestar_historical_state_worker_nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"}", + "hide": false, + "interval": "", + "legendFormat": "external_memory", + "range": true, + "refId": "B" + } + ], + "title": "Heap Allocations", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "GC Time", + "axisPlacement": "left", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "opacity", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "percentunit" + }, + "overrides": [ + { + "matcher": { + "id": "byFrameRefID", + "options": "C" + }, + "properties": [ + { + "id": "custom.axisPlacement", + "value": "right" + }, + { + "id": "custom.axisLabel", + "value": "GC Bytes" + }, + { + "id": "unit", + "value": "decbytes" + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 1 + }, + "id": 537, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "8.4.0-beta1", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_worker_nodejs_gc_pause_seconds_total{job=~\"$beacon_job|beacon\"}[$rate_interval])", + "hide": false, + "legendFormat": "{{gctype}}", + "range": true, + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_worker_nodejs_gc_reclaimed_bytes_total{job=~\"$beacon_job|beacon\"}[$rate_interval])", + "format": "time_series", + "hide": false, + "legendFormat": "{{gctype}}", + "range": true, + "refId": "C" + } + ], + "title": "GC pause time rate + reclaimed bytes", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "log": 2, + "type": "log" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "s" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "p50" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "light-blue", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "p90" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "blue", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "p99" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "semi-dark-blue", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "max" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-blue", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "min" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "dark-blue", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "set_immediate" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "yellow", + "mode": "fixed" + } + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "max" + }, + "properties": [ + { + "id": "custom.fillBelowTo", + "value": "min" + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 9 + }, + "id": 538, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg_over_time(lodestar_historical_state_worker_nodejs_eventloop_lag_min_seconds{job=~\"$beacon_job|beacon\"}[$rate_interval])", + "hide": false, + "legendFormat": "min", + "range": true, + "refId": "0" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg_over_time(lodestar_historical_state_worker_nodejs_eventloop_lag_p50_seconds{job=~\"$beacon_job|beacon\"}[$rate_interval])", + "legendFormat": "p50", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg_over_time(lodestar_historical_state_worker_nodejs_eventloop_lag_p90_seconds{job=~\"$beacon_job|beacon\"}[$rate_interval])", + "hide": false, + "legendFormat": "p90", + "range": true, + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg_over_time(lodestar_historical_state_worker_nodejs_eventloop_lag_p99_seconds{job=~\"$beacon_job|beacon\"}[$rate_interval])", + "hide": false, + "legendFormat": "p99", + "range": true, + "refId": "C" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg_over_time(lodestar_historical_state_worker_nodejs_eventloop_lag_max_seconds{job=~\"$beacon_job|beacon\"}[$rate_interval])", + "hide": false, + "legendFormat": "max", + "range": true, + "refId": "D" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "avg_over_time(lodestar_historical_state_worker_nodejs_eventloop_lag_seconds{job=~\"$beacon_job|beacon\"}[$rate_interval])", + "hide": false, + "legendFormat": "set_immediate", + "range": true, + "refId": "E" + } + ], + "title": "Event loop lag", + "type": "timeseries" + }, + { + "collapsed": false, + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 17 + }, + "id": 25, + "panels": [], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "refId": "A" + } + ], + "title": "Job Queue", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 10, + "gradientMode": "opacity", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "percentunit" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 18 + }, + "id": 26, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "8.4.0-beta1", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_queue_job_time_seconds_sum[$rate_interval])", + "instant": false, + "interval": "", + "legendFormat": "block_processor", + "refId": "A" + } + ], + "title": "Utilization rate", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 30, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 18 + }, + "id": 81, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "12*rate(lodestar_historical_state_queue_job_time_seconds_count[6m])", + "instant": false, + "interval": "", + "legendFormat": "block_processor", + "refId": "A" + } + ], + "title": "Jobs / slot", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 30, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 2, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 26 + }, + "id": 100, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_queue_job_time_seconds_sum[$rate_interval])/rate(lodestar_historical_state_queue_job_time_seconds_count[$rate_interval])", + "instant": false, + "interval": "", + "legendFormat": "block_processor", + "refId": "A" + } + ], + "title": "Job time", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 30, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "none" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 26 + }, + "id": 128, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "lodestar_historical_state_queue_length", + "instant": false, + "interval": "", + "legendFormat": "block_processor", + "refId": "A" + } + ], + "title": "Queue length", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 30, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 2, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 34 + }, + "id": 127, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_queue_job_wait_time_seconds_sum[$rate_interval])/rate(lodestar_historical_state_queue_job_wait_time_seconds_count[$rate_interval])", + "instant": false, + "interval": "", + "legendFormat": "block_processor", + "refId": "A" + } + ], + "title": "Job wait time", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 30, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 2, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "percentunit" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 34 + }, + "id": 126, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_queue_dropped_jobs_total[$rate_interval])/(rate(lodestar_historical_state_queue_job_time_seconds_count[$rate_interval])+rate(lodestar_historical_state_queue_dropped_jobs_total[$rate_interval]))", + "instant": false, + "interval": "", + "legendFormat": "block_processor", + "refId": "A" + } + ], + "title": "Dropped jobs %", + "type": "timeseries" + }, + { + "collapsed": false, + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 42 + }, + "id": 108, + "panels": [], + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "refId": "A" + } + ], + "title": "Beacon state transition", + "type": "row" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "custom": { + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "scaleDistribution": { + "type": "linear" + } + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 43 + }, + "id": 528, + "options": { + "calculate": false, + "cellGap": 1, + "color": { + "exponent": 0.5, + "fill": "dark-orange", + "mode": "scheme", + "reverse": false, + "scale": "exponential", + "scheme": "Magma", + "steps": 64 + }, + "exemplars": { + "color": "rgba(255,0,255,0.7)" + }, + "filterValues": { + "le": 1e-9 + }, + "legend": { + "show": true + }, + "rowsFrame": { + "layout": "auto" + }, + "tooltip": { + "show": true, + "yHistogram": false + }, + "yAxis": { + "axisPlacement": "left", + "reverse": false + } + }, + "pluginVersion": "10.1.1", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "32*12*rate(lodestar_historical_state_stfn_epoch_transition_seconds_bucket[$rate_interval])", + "format": "heatmap", + "interval": "", + "legendFormat": "{{le}}", + "range": true, + "refId": "A" + } + ], + "title": "Epoch transition time", + "type": "heatmap" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "custom": { + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "scaleDistribution": { + "type": "linear" + } + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 43 + }, + "id": 529, + "options": { + "calculate": false, + "cellGap": 1, + "color": { + "exponent": 0.5, + "fill": "dark-orange", + "mode": "scheme", + "reverse": false, + "scale": "exponential", + "scheme": "Magma", + "steps": 64 + }, + "exemplars": { + "color": "rgba(255,0,255,0.7)" + }, + "filterValues": { + "le": 1e-9 + }, + "legend": { + "show": true + }, + "rowsFrame": { + "layout": "auto" + }, + "tooltip": { + "show": true, + "yHistogram": false + }, + "yAxis": { + "axisPlacement": "left", + "reverse": false + } + }, + "pluginVersion": "10.1.1", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "12*rate(lodestar_historical_state_stfn_process_block_seconds_bucket[$rate_interval])", + "format": "heatmap", + "interval": "", + "legendFormat": "{{le}}", + "range": true, + "refId": "A" + } + ], + "title": "Process block time", + "type": "heatmap" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 22, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 4, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 51 + }, + "id": 120, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "rate(lodestar_historical_state_stfn_epoch_transition_seconds_sum[$rate_interval])\n/\nrate(lodestar_historical_state_stfn_epoch_transition_seconds_count[$rate_interval])", + "interval": "", + "legendFormat": "epoch transition", + "range": true, + "refId": "A" + } + ], + "title": "Epoch transition avg time", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 22, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 51 + }, + "id": 121, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_process_block_seconds_sum[$rate_interval])\n/\nrate(lodestar_historical_state_stfn_process_block_seconds_count[$rate_interval])", + "interval": "", + "legendFormat": "process block time", + "range": true, + "refId": "A" + } + ], + "title": "Process block avg time", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 59 + }, + "id": 534, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_epoch_transition_step_seconds_sum[$rate_interval])\n/\nrate(lodestar_historical_state_stfn_epoch_transition_step_seconds_count[$rate_interval])", + "instant": false, + "legendFormat": "{{step}}", + "range": true, + "refId": "A" + } + ], + "title": "Epoch Transition By Steps", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 22, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 59 + }, + "id": 525, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_process_block_commit_seconds_sum[$rate_interval])\n/\nrate(lodestar_historical_state_stfn_process_block_commit_seconds_count[$rate_interval])", + "interval": "", + "legendFormat": "process block time", + "range": true, + "refId": "A" + } + ], + "title": "Process block commit step avg time", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 22, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 4, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 67 + }, + "id": 524, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "rate(lodestar_historical_state_stfn_epoch_transition_commit_seconds_sum[$rate_interval])\n/\nrate(lodestar_historical_state_stfn_epoch_transition_commit_seconds_count[$rate_interval])", + "interval": "", + "legendFormat": "epoch transition", + "range": true, + "refId": "A" + } + ], + "title": "Epoch transition commit step avg time", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 22, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "percentunit" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "process block time" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "orange", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 67 + }, + "id": 123, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "rate(lodestar_historical_state_stfn_process_block_seconds_sum[6m])", + "interval": "", + "legendFormat": "process block time", + "range": true, + "refId": "A" + } + ], + "title": "Process block utilization rate", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 22, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "percentunit" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "process block time" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "orange", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 75 + }, + "id": 122, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "rate(lodestar_historical_state_stfn_epoch_transition_seconds_sum[13m])", + "interval": "", + "legendFormat": "process block time", + "range": true, + "refId": "A" + } + ], + "title": "Epoch transition utilization rate", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 22, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "none" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "process block time" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "yellow", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 75 + }, + "id": 125, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "12*rate(lodestar_historical_state_stfn_process_block_seconds_count[6m])", + "interval": "", + "legendFormat": "process block time", + "range": true, + "refId": "A" + } + ], + "title": "process block / slot", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 22, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "min": 0, + "unit": "none" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "number of epoch transition" + }, + "properties": [ + { + "id": "color", + "value": { + "fixedColor": "yellow", + "mode": "fixed" + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 83 + }, + "id": 124, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "384 * rate(lodestar_historical_state_stfn_epoch_transition_seconds_count[13m])", + "interval": "", + "legendFormat": "number of epoch transition", + "range": true, + "refId": "A" + } + ], + "title": "Epoch transitions / epoch", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 22, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "short" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 83 + }, + "id": 527, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_validators_nodes_populated_miss_total{source=\"stateTransition\"} [$rate_interval])\n/on(instance)\nrate(lodestar_historical_state_stfn_state_cloned_count_count[$rate_interval])", + "interval": "", + "legendFormat": "stateTransition-validators", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_balances_nodes_populated_miss_total{source=\"stateTransition\"} [$rate_interval])\n/on(instance)\nrate(lodestar_historical_state_stfn_state_cloned_count_count[$rate_interval])", + "hide": false, + "legendFormat": "stateTransition-balances", + "range": true, + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_validators_nodes_populated_miss_total{source=\"processSlots\"} [$rate_interval])\n/on(instance)\nrate(lodestar_historical_state_stfn_state_cloned_count_count[$rate_interval])", + "hide": false, + "legendFormat": "processSlots-validators", + "range": true, + "refId": "C" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_balances_nodes_populated_miss_total{source=\"processSlots\"} [$rate_interval])\n/on(instance)\nrate(lodestar_historical_state_stfn_state_cloned_count_count[$rate_interval])", + "hide": false, + "legendFormat": "processSlots-balances", + "range": true, + "refId": "D" + } + ], + "title": "State SSZ cache miss rate on preState", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 22, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "s" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 91 + }, + "id": 526, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": false + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_hash_tree_root_seconds_sum[$rate_interval])\n/ on(source)\nrate(lodestar_historical_state_stfn_hash_tree_root_seconds_count[$rate_interval])", + "interval": "", + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "State hash_tree_root avg time", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "custom": { + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "scaleDistribution": { + "type": "linear" + } + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 91 + }, + "id": 523, + "options": { + "calculate": false, + "cellGap": 1, + "color": { + "exponent": 0.5, + "fill": "dark-orange", + "mode": "scheme", + "reverse": false, + "scale": "exponential", + "scheme": "Magma", + "steps": 50 + }, + "exemplars": { + "color": "rgba(255,0,255,0.7)" + }, + "filterValues": { + "le": 1e-9 + }, + "legend": { + "show": true + }, + "rowsFrame": { + "layout": "auto" + }, + "tooltip": { + "show": true, + "yHistogram": false + }, + "yAxis": { + "axisPlacement": "left", + "reverse": false + } + }, + "pluginVersion": "10.1.1", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_state_cloned_count_bucket[$rate_interval])", + "format": "heatmap", + "legendFormat": "__auto", + "range": true, + "refId": "A" + } + ], + "title": "Clone count per state", + "type": "heatmap" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [] + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 99 + }, + "id": 521, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_balances_nodes_populated_miss_total{source=\"processSlots\"} [$rate_interval])\n/\nrate(lodestar_historical_state_stfn_state_clone_total[$rate_interval])", + "legendFormat": "balances-{{source}}", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_balances_nodes_populated_miss_total{source=\"stateTransition\"} [$rate_interval])\n/\nrate(lodestar_historical_state_stfn_state_clone_total[$rate_interval])", + "hide": false, + "legendFormat": "balances-{{source}}", + "range": true, + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_validators_nodes_populated_miss_total{source=\"processSlots\"} [$rate_interval])\n/\nrate(lodestar_historical_state_stfn_state_clone_total[$rate_interval])", + "hide": false, + "legendFormat": "validators-{{source}}", + "range": true, + "refId": "C" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_stfn_validators_nodes_populated_miss_total{source=\"stateTransition\"} [$rate_interval])\n/\nrate(lodestar_historical_state_stfn_state_clone_total[$rate_interval])", + "hide": false, + "legendFormat": "validators-{{source}}", + "range": true, + "refId": "D" + } + ], + "title": "State transition fn nodes populated cache miss", + "type": "timeseries" + } + ], + "refresh": "10s", + "schemaVersion": 38, + "style": "dark", + "tags": [ + "lodestar" + ], + "templating": { + "list": [ + { + "current": { + "selected": false, + "text": "default", + "value": "default" + }, + "hide": 0, + "includeAll": false, + "label": "datasource", + "multi": false, + "name": "DS_PROMETHEUS", + "options": [], + "query": "prometheus", + "queryValue": "", + "refresh": 1, + "regex": "", + "skipUrlSync": false, + "type": "datasource" + }, + { + "auto": true, + "auto_count": 30, + "auto_min": "10s", + "current": { + "selected": false, + "text": "1h", + "value": "1h" + }, + "hide": 0, + "label": "rate() interval", + "name": "rate_interval", + "options": [ + { + "selected": false, + "text": "auto", + "value": "$__auto_interval_rate_interval" + }, + { + "selected": false, + "text": "1m", + "value": "1m" + }, + { + "selected": false, + "text": "10m", + "value": "10m" + }, + { + "selected": false, + "text": "30m", + "value": "30m" + }, + { + "selected": true, + "text": "1h", + "value": "1h" + }, + { + "selected": false, + "text": "6h", + "value": "6h" + }, + { + "selected": false, + "text": "12h", + "value": "12h" + }, + { + "selected": false, + "text": "1d", + "value": "1d" + }, + { + "selected": false, + "text": "7d", + "value": "7d" + }, + { + "selected": false, + "text": "14d", + "value": "14d" + }, + { + "selected": false, + "text": "30d", + "value": "30d" + } + ], + "query": "1m,10m,30m,1h,6h,12h,1d,7d,14d,30d", + "queryValue": "", + "refresh": 2, + "skipUrlSync": false, + "type": "interval" + }, + { + "datasource": { + "type": "prometheus", + "uid": "prometheus_local" + }, + "filters": [ + { + "condition": "", + "key": "instance", + "operator": "=", + "value": "unstable-lg1k-hzax41" + } + ], + "hide": 0, + "name": "Filters", + "skipUrlSync": false, + "type": "adhoc" + }, + { + "description": "Job name used in Prometheus config to scrape Beacon node", + "hide": 2, + "label": "Beacon node job name", + "name": "beacon_job", + "query": "${VAR_BEACON_JOB}", + "skipUrlSync": false, + "type": "constant" + } + ] + }, + "time": { + "from": "now-24h", + "to": "now" + }, + "timepicker": { + "refresh_intervals": [ + "5s", + "10s", + "30s", + "1m", + "5m", + "15m", + "30m", + "1h", + "2h", + "1d" + ] + }, + "timezone": "utc", + "title": "Lodestar - historical state regen", + "uid": "lodestar_historical_state_regen", + "version": 32, + "weekStart": "monday" +} From e8c9d27456ed21c267a9f5933f6d66aa1399f252 Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 22 Jan 2024 18:19:40 -0500 Subject: [PATCH 18/23] fix: update vm dashboards with historical state worker --- dashboards/lodestar_vm_host.json | 318 ++++++++++++++++++++++++++----- 1 file changed, 270 insertions(+), 48 deletions(-) diff --git a/dashboards/lodestar_vm_host.json b/dashboards/lodestar_vm_host.json index 7471defd0e8e..457d4d120fa4 100644 --- a/dashboards/lodestar_vm_host.json +++ b/dashboards/lodestar_vm_host.json @@ -177,7 +177,7 @@ }, "editorMode": "code", "exemplar": false, - "expr": "(sum(nodejs_heap_size_total_bytes) + sum(discv5_worker_nodejs_heap_size_total_bytes) + sum(network_worker_nodejs_heap_size_total_bytes))\nor\n(sum(nodejs_heap_size_total_bytes) + sum(discv5_worker_nodejs_heap_size_total_bytes))", + "expr": "(sum(nodejs_heap_size_total_bytes) + sum(discv5_worker_nodejs_heap_size_total_bytes) + sum(network_worker_nodejs_heap_size_total_bytes) + sum(lodestar_historical_state_worker_nodejs_heap_size_total_bytes))\nor\n(sum(nodejs_heap_size_total_bytes) + sum(discv5_worker_nodejs_heap_size_total_bytes) + sum(lodestar_historical_state_worker_nodejs_heap_size_total_bytes))", "hide": false, "interval": "", "legendFormat": "node allocated heap", @@ -191,7 +191,7 @@ }, "editorMode": "code", "exemplar": false, - "expr": "(sum(nodejs_heap_size_used_bytes) + sum(discv5_worker_nodejs_heap_size_used_bytes) + sum(network_worker_nodejs_heap_size_used_bytes)) \nor\n(sum(nodejs_heap_size_used_bytes) + sum(discv5_worker_nodejs_heap_size_used_bytes)) ", + "expr": "(sum(nodejs_heap_size_used_bytes) + sum(discv5_worker_nodejs_heap_size_used_bytes) + sum(network_worker_nodejs_heap_size_used_bytes) + sum(lodestar_historical_state_worker_nodejs_heap_size_used_bytes)) \nor\n(sum(nodejs_heap_size_used_bytes) + sum(discv5_worker_nodejs_heap_size_used_bytes) + sum(lodestar_historical_state_worker_nodejs_heap_size_used_bytes)) ", "hide": false, "interval": "", "legendFormat": "node used heap", @@ -205,7 +205,7 @@ }, "editorMode": "code", "exemplar": false, - "expr": "(sum(nodejs_external_memory_bytes) + sum(discv5_worker_nodejs_external_memory_bytes) + sum(network_worker_nodejs_external_memory_bytes))\nor\n(sum(nodejs_external_memory_bytes) + sum(discv5_worker_nodejs_external_memory_bytes))", + "expr": "(sum(nodejs_external_memory_bytes) + sum(discv5_worker_nodejs_external_memory_bytes) + sum(network_worker_nodejs_external_memory_bytes) + sum(lodestar_historical_state_worker_nodejs_external_memory_bytes))\nor\n(sum(nodejs_external_memory_bytes) + sum(discv5_worker_nodejs_external_memory_bytes) + sum(lodestar_historical_state_worker_nodejs_external_memory_bytes))", "hide": false, "interval": "", "legendFormat": "node external memory", @@ -291,7 +291,7 @@ "uid": "${DS_PROMETHEUS}" }, "editorMode": "code", - "expr": "sum by (space) ({__name__=~\"nodejs_heap_space_size_used_bytes|network_worker_nodejs_heap_space_size_used_bytes|discv5_worker_nodejs_heap_space_size_used_bytes\")", + "expr": "sum by (space) ({__name__=~\"nodejs_heap_space_size_used_bytes|network_worker_nodejs_heap_space_size_used_bytes|discv5_worker_nodejs_heap_space_size_used_bytes|lodestar_historical_state_worker_nodejs_heap_space_size_used_bytes\")", "hide": false, "interval": "", "legendFormat": "{{space}}", @@ -305,7 +305,7 @@ }, "editorMode": "code", "exemplar": false, - "expr": "(nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"} + discv5_worker_nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"} + network_worker_nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"})\nor\n(nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"} + discv5_worker_nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"})", + "expr": "(nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"} + discv5_worker_nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"} + network_worker_nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"} + lodestar_historical_state_worker_nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"})\nor\n(nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"} + discv5_worker_nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"} + lodestar_historical_state_worker_nodejs_external_memory_bytes{job=~\"$beacon_job|beacon\"})", "hide": false, "interval": "", "legendFormat": "external_memory", @@ -786,6 +786,7 @@ "tooltip": false, "viz": false }, + "insertNulls": false, "lineInterpolation": "linear", "lineWidth": 1, "pointSize": 5, @@ -883,6 +884,7 @@ "tooltip": false, "viz": false }, + "insertNulls": false, "lineInterpolation": "linear", "lineWidth": 1, "pointSize": 5, @@ -1000,6 +1002,7 @@ "tooltip": false, "viz": false }, + "insertNulls": false, "lineInterpolation": "linear", "lineWidth": 1, "pointSize": 5, @@ -1027,6 +1030,222 @@ "x": 0, "y": 34 }, + "id": 562, + "options": { + "graph": {}, + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "7.4.5", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "lodestar_historical_state_worker_nodejs_heap_space_size_used_bytes", + "interval": "", + "legendFormat": "{{space}}", + "range": true, + "refId": "A" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "exemplar": false, + "expr": "lodestar_historical_state_worker_nodejs_external_memory_bytes", + "hide": false, + "interval": "", + "legendFormat": "external_memory", + "range": true, + "refId": "B" + } + ], + "title": "Historical State Worker Thread - Heap Allocations", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "GC Time", + "axisPlacement": "left", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 20, + "gradientMode": "opacity", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": true, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "percentunit" + }, + "overrides": [ + { + "matcher": { + "id": "byFrameRefID", + "options": "C" + }, + "properties": [ + { + "id": "custom.axisPlacement", + "value": "right" + }, + { + "id": "custom.axisLabel", + "value": "GC Bytes" + }, + { + "id": "unit", + "value": "decbytes" + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 34 + }, + "id": 563, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "multi", + "sort": "none" + } + }, + "pluginVersion": "8.4.0-beta1", + "targets": [ + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_worker_nodejs_gc_pause_seconds_total[$rate_interval])", + "hide": false, + "legendFormat": "{{gctype}}", + "range": true, + "refId": "B" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "editorMode": "code", + "expr": "rate(lodestar_historical_state_worker_nodejs_gc_reclaimed_bytes_total[$rate_interval])", + "format": "time_series", + "hide": false, + "legendFormat": "{{gctype}}", + "range": true, + "refId": "C" + } + ], + "title": "HIstorical State Worker Thread - GC pause time rate + reclaimed bytes", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus", + "uid": "${DS_PROMETHEUS}" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 30, + "gradientMode": "opacity", + "hideFrom": { + "graph": false, + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "never", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "unit": "decbytes" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 42 + }, "id": 542, "options": { "graph": {}, @@ -1097,6 +1316,7 @@ "tooltip": false, "viz": false }, + "insertNulls": false, "lineInterpolation": "linear", "lineWidth": 1, "pointSize": 5, @@ -1143,7 +1363,7 @@ "h": 8, "w": 12, "x": 12, - "y": 34 + "y": 42 }, "id": 548, "options": { @@ -1199,7 +1419,7 @@ "h": 1, "w": 24, "x": 0, - "y": 42 + "y": 50 }, "id": 12, "panels": [], @@ -1239,6 +1459,7 @@ "tooltip": false, "viz": false }, + "insertNulls": false, "lineInterpolation": "linear", "lineWidth": 1, "pointSize": 5, @@ -1368,7 +1589,7 @@ "h": 8, "w": 12, "x": 0, - "y": 43 + "y": 51 }, "id": 555, "options": { @@ -1483,6 +1704,7 @@ "tooltip": false, "viz": false }, + "insertNulls": false, "lineInterpolation": "linear", "lineWidth": 1, "pointSize": 5, @@ -1612,7 +1834,7 @@ "h": 8, "w": 12, "x": 12, - "y": 43 + "y": 51 }, "id": 559, "options": { @@ -1856,7 +2078,7 @@ "h": 8, "w": 12, "x": 0, - "y": 51 + "y": 59 }, "id": 560, "options": { @@ -2100,7 +2322,7 @@ "h": 8, "w": 12, "x": 12, - "y": 51 + "y": 59 }, "id": 561, "options": { @@ -2241,7 +2463,7 @@ "h": 8, "w": 12, "x": 0, - "y": 59 + "y": 67 }, "id": 6, "options": { @@ -2364,7 +2586,7 @@ "h": 8, "w": 12, "x": 12, - "y": 59 + "y": 67 }, "id": 42, "options": { @@ -2443,7 +2665,7 @@ "h": 8, "w": 12, "x": 12, - "y": 67 + "y": 75 }, "id": 268, "options": { @@ -2484,7 +2706,7 @@ "h": 1, "w": 24, "x": 0, - "y": 75 + "y": 83 }, "id": 104, "panels": [], @@ -2550,7 +2772,7 @@ "h": 6, "w": 12, "x": 0, - "y": 76 + "y": 84 }, "id": 102, "options": { @@ -2659,7 +2881,7 @@ "h": 6, "w": 12, "x": 12, - "y": 76 + "y": 84 }, "id": 172, "options": { @@ -2744,7 +2966,7 @@ "h": 6, "w": 12, "x": 0, - "y": 82 + "y": 90 }, "id": 171, "options": { @@ -2787,7 +3009,7 @@ "h": 6, "w": 12, "x": 12, - "y": 82 + "y": 90 }, "id": 99, "options": { @@ -2828,7 +3050,7 @@ "h": 1, "w": 24, "x": 0, - "y": 88 + "y": 96 }, "id": 367, "panels": [], @@ -2894,7 +3116,7 @@ "h": 8, "w": 12, "x": 0, - "y": 89 + "y": 97 }, "id": 353, "options": { @@ -2977,7 +3199,7 @@ "h": 8, "w": 12, "x": 12, - "y": 89 + "y": 97 }, "id": 355, "options": { @@ -3059,7 +3281,7 @@ "h": 8, "w": 12, "x": 0, - "y": 97 + "y": 105 }, "id": 357, "options": { @@ -3522,7 +3744,7 @@ "h": 8, "w": 12, "x": 12, - "y": 97 + "y": 105 }, "id": 359, "links": [], @@ -3665,7 +3887,7 @@ "h": 8, "w": 12, "x": 0, - "y": 105 + "y": 113 }, "id": 361, "options": { @@ -3774,7 +3996,7 @@ "h": 8, "w": 12, "x": 12, - "y": 105 + "y": 113 }, "id": 363, "options": { @@ -3866,7 +4088,7 @@ "h": 8, "w": 12, "x": 0, - "y": 113 + "y": 121 }, "id": 365, "options": { @@ -3919,7 +4141,7 @@ "h": 1, "w": 24, "x": 0, - "y": 121 + "y": 129 }, "id": 46, "panels": [], @@ -3985,7 +4207,7 @@ "h": 8, "w": 12, "x": 0, - "y": 122 + "y": 130 }, "id": 48, "options": { @@ -4112,7 +4334,7 @@ "h": 8, "w": 12, "x": 12, - "y": 122 + "y": 130 }, "id": 50, "options": { @@ -4155,7 +4377,7 @@ "h": 1, "w": 24, "x": 0, - "y": 130 + "y": 138 }, "id": 527, "panels": [], @@ -4221,7 +4443,7 @@ "h": 8, "w": 12, "x": 0, - "y": 131 + "y": 139 }, "id": 529, "options": { @@ -4366,7 +4588,7 @@ "h": 8, "w": 12, "x": 12, - "y": 131 + "y": 139 }, "id": 531, "options": { @@ -4410,7 +4632,7 @@ "h": 1, "w": 24, "x": 0, - "y": 139 + "y": 147 }, "id": 86, "panels": [], @@ -4475,7 +4697,7 @@ "h": 11, "w": 12, "x": 0, - "y": 140 + "y": 148 }, "id": 84, "options": { @@ -4558,7 +4780,7 @@ "h": 11, "w": 12, "x": 12, - "y": 140 + "y": 148 }, "id": 87, "options": { @@ -4639,7 +4861,7 @@ "h": 8, "w": 12, "x": 0, - "y": 151 + "y": 159 }, "id": 516, "options": { @@ -4730,7 +4952,7 @@ "h": 8, "w": 12, "x": 12, - "y": 151 + "y": 159 }, "id": 515, "options": { @@ -4809,7 +5031,7 @@ "h": 8, "w": 12, "x": 0, - "y": 159 + "y": 167 }, "id": 513, "options": { @@ -4888,7 +5110,7 @@ "h": 8, "w": 12, "x": 12, - "y": 159 + "y": 167 }, "id": 517, "options": { @@ -4928,7 +5150,7 @@ "h": 1, "w": 24, "x": 0, - "y": 167 + "y": 175 }, "id": 164, "panels": [], @@ -4995,7 +5217,7 @@ "h": 9, "w": 12, "x": 0, - "y": 168 + "y": 176 }, "id": 160, "options": { @@ -5079,7 +5301,7 @@ "h": 9, "w": 12, "x": 12, - "y": 168 + "y": 176 }, "id": 162, "options": { @@ -5163,7 +5385,7 @@ "h": 9, "w": 12, "x": 0, - "y": 177 + "y": 185 }, "id": 481, "options": { @@ -5247,7 +5469,7 @@ "h": 9, "w": 12, "x": 12, - "y": 177 + "y": 185 }, "id": 480, "options": { @@ -5354,7 +5576,7 @@ "h": 8, "w": 12, "x": 0, - "y": 186 + "y": 194 }, "id": 535, "options": { @@ -5459,7 +5681,7 @@ "h": 8, "w": 12, "x": 12, - "y": 186 + "y": 194 }, "id": 537, "options": { @@ -5540,7 +5762,7 @@ "h": 8, "w": 12, "x": 0, - "y": 194 + "y": 202 }, "id": 533, "options": { From 03a241ce3411a771817f4c796e691641a9841f66 Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 22 Jan 2024 18:22:30 -0500 Subject: [PATCH 19/23] chore: fix test data --- packages/api/test/unit/beacon/testData/debug.ts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packages/api/test/unit/beacon/testData/debug.ts b/packages/api/test/unit/beacon/testData/debug.ts index 6b65d610d16f..2f5ac5262f6d 100644 --- a/packages/api/test/unit/beacon/testData/debug.ts +++ b/packages/api/test/unit/beacon/testData/debug.ts @@ -46,7 +46,7 @@ export const testData: GenericServerTestCases = { }, getState: { args: ["head", "json"], - res: {executionOptimistic: true, data: ssz.phase0.BeaconState.defaultValue()}, + res: {data: ssz.phase0.BeaconState.defaultValue()}, }, getStateV2: { args: ["head", "json"], From 6001521eb51a07e8a7dab850d1268268d354cb37 Mon Sep 17 00:00:00 2001 From: Cayman Date: Mon, 22 Jan 2024 18:55:15 -0500 Subject: [PATCH 20/23] feat: transfer state across worker boundary --- packages/beacon-node/src/chain/historicalState/worker.ts | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/packages/beacon-node/src/chain/historicalState/worker.ts b/packages/beacon-node/src/chain/historicalState/worker.ts index be28246d4acd..108dbe95f745 100644 --- a/packages/beacon-node/src/chain/historicalState/worker.ts +++ b/packages/beacon-node/src/chain/historicalState/worker.ts @@ -1,5 +1,5 @@ import worker from "node:worker_threads"; -import {expose} from "@chainsafe/threads/worker"; +import {Transfer, expose} from "@chainsafe/threads/worker"; import {createBeaconConfig, chainConfigFromJson} from "@lodestar/config"; import {getNodeLogger} from "@lodestar/logger/node"; import { @@ -166,7 +166,10 @@ const api: HistoricalStateWorkerApi = { return metricsRegister?.metrics() ?? ""; }, async getHistoricalState(slot) { - return queue.push(() => getHistoricalState(slot, config, db, pubkey2index, stateTransitionMetrics)); + const state = await queue.push(() => + getHistoricalState(slot, config, db, pubkey2index, stateTransitionMetrics) + ); + return Transfer(state, [state.buffer]) as unknown as Uint8Array; }, }; From 7ea9c1aa827bffac5f941144e6b2482325e4ffcc Mon Sep 17 00:00:00 2001 From: Cayman Date: Tue, 9 Jul 2024 16:25:29 -0400 Subject: [PATCH 21/23] chore: address some pr comments --- packages/beacon-node/src/chain/chain.ts | 48 +++++++++---------- .../historicalState/getHistoricalState.ts | 10 +--- .../src/chain/historicalState/index.ts | 1 - packages/beacon-node/src/node/nodejs.ts | 1 + 4 files changed, 26 insertions(+), 34 deletions(-) diff --git a/packages/beacon-node/src/chain/chain.ts b/packages/beacon-node/src/chain/chain.ts index f416fddc5250..c8b56f6c56ad 100644 --- a/packages/beacon-node/src/chain/chain.ts +++ b/packages/beacon-node/src/chain/chain.ts @@ -422,32 +422,32 @@ export class BeaconChain implements IBeaconChain { ): Promise<{state: BeaconStateAllForks; executionOptimistic: boolean; finalized: boolean} | null> { const finalizedBlock = this.forkChoice.getFinalizedBlock(); - if (slot >= finalizedBlock.slot) { - // request for non-finalized state - - if (opts?.allowRegen) { - // Find closest canonical block to slot, then trigger regen - const block = this.forkChoice.getCanonicalBlockClosestLteSlot(slot) ?? finalizedBlock; - const state = await this.regen.getBlockSlotState( - block.blockRoot, - slot, - {dontTransferCache: true}, - RegenCaller.restApi - ); - return {state, executionOptimistic: isOptimisticBlock(block), finalized: slot === finalizedBlock.slot}; - } else { - // Just check if state is already in the cache. If it's not dialed to the correct slot, - // do not bother in advancing the state. restApiCanTriggerRegen == false means do no work - const block = this.forkChoice.getCanonicalBlockAtSlot(slot); - if (!block) { - return null; - } + if (slot < finalizedBlock.slot) { + // request for finalized state not supported in this API + // fall back to caller to look in db or getHistoricalStateBySlot + return null; + } - const state = this.regen.getStateSync(block.stateRoot); - return state && {state, executionOptimistic: isOptimisticBlock(block), finalized: slot === finalizedBlock.slot}; - } + if (opts?.allowRegen) { + // Find closest canonical block to slot, then trigger regen + const block = this.forkChoice.getCanonicalBlockClosestLteSlot(slot) ?? finalizedBlock; + const state = await this.regen.getBlockSlotState( + block.blockRoot, + slot, + {dontTransferCache: true}, + RegenCaller.restApi + ); + return {state, executionOptimistic: isOptimisticBlock(block), finalized: slot === finalizedBlock.slot}; } else { - return null; + // Just check if state is already in the cache. If it's not dialed to the correct slot, + // do not bother in advancing the state. restApiCanTriggerRegen == false means do no work + const block = this.forkChoice.getCanonicalBlockAtSlot(slot); + if (!block) { + return null; + } + + const state = this.regen.getStateSync(block.stateRoot); + return state && {state, executionOptimistic: isOptimisticBlock(block), finalized: slot === finalizedBlock.slot}; } } diff --git a/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts b/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts index 3b5a672078dd..eeae80bfdb39 100644 --- a/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts +++ b/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts @@ -8,7 +8,6 @@ import { createCachedBeaconState, stateTransition, } from "@lodestar/state-transition"; -import {SignedBeaconBlock} from "@lodestar/types/allForks"; import {BeaconConfig} from "@lodestar/config"; import {IBeaconDb} from "../../db/index.js"; @@ -56,13 +55,6 @@ export async function getNearestState( ); } -/** - * Get blocks (from, to] - */ -export function getBlocksBetween(from: number, to: number, db: IBeaconDb): AsyncIterable { - return db.blockArchive.valuesStream({gt: from, lte: to}); -} - /** * Get and regenerate a historical state */ @@ -74,7 +66,7 @@ export async function getHistoricalState( metrics?: BeaconStateTransitionMetrics ): Promise { let state = await getNearestState(slot, config, db, pubkey2index); - for await (const block of getBlocksBetween(state.slot, slot, db)) { + for await (const block of db.blockArchive.valuesStream({gt: state.slot, lte: slot})) { state = stateTransition( state, block, diff --git a/packages/beacon-node/src/chain/historicalState/index.ts b/packages/beacon-node/src/chain/historicalState/index.ts index 026379a29f0b..c2f89e15aeed 100644 --- a/packages/beacon-node/src/chain/historicalState/index.ts +++ b/packages/beacon-node/src/chain/historicalState/index.ts @@ -40,7 +40,6 @@ export class HistoricalStateRegen implements HistoricalStateWorkerApi { workerData, } as ConstructorParameters[1]); - // eslint-disable-next-line @typescript-eslint/no-explicit-any const api = await spawn(worker, { // A Lodestar Node may do very expensive task at start blocking the event loop and causing // the initialization to timeout. The number below is big enough to almost disable the timeout diff --git a/packages/beacon-node/src/node/nodejs.ts b/packages/beacon-node/src/node/nodejs.ts index 5d17b2c2363a..552dc49548aa 100644 --- a/packages/beacon-node/src/node/nodejs.ts +++ b/packages/beacon-node/src/node/nodejs.ts @@ -204,6 +204,7 @@ export class BeaconNode { metrics, logger: logger.child({module: LoggerModule.chain}), }); + signal.addEventListener("abort", () => historicalStateRegen.close(), {once: true}); const chain = new BeaconChain(opts.chain, { config, From 4a297ddb8e047136fc72f58d6e89f1d51316eb16 Mon Sep 17 00:00:00 2001 From: Cayman Date: Tue, 9 Jul 2024 17:47:15 -0400 Subject: [PATCH 22/23] chore: clean module close --- packages/beacon-node/src/chain/historicalState/index.ts | 1 + packages/beacon-node/src/chain/historicalState/types.ts | 1 + packages/beacon-node/src/node/nodejs.ts | 2 +- 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/packages/beacon-node/src/chain/historicalState/index.ts b/packages/beacon-node/src/chain/historicalState/index.ts index c2f89e15aeed..8688bcf9f372 100644 --- a/packages/beacon-node/src/chain/historicalState/index.ts +++ b/packages/beacon-node/src/chain/historicalState/index.ts @@ -23,6 +23,7 @@ export class HistoricalStateRegen implements HistoricalStateWorkerApi { constructor(modules: HistoricalStateRegenModules) { this.api = modules.api; this.logger = modules.logger; + modules.signal?.addEventListener("abort", () => this.close(), {once: true}); } static async init(modules: HistoricalStateRegenInitModules): Promise { const workerData: HistoricalStateWorkerData = { diff --git a/packages/beacon-node/src/chain/historicalState/types.ts b/packages/beacon-node/src/chain/historicalState/types.ts index fd756074c37b..a44cf0ed478a 100644 --- a/packages/beacon-node/src/chain/historicalState/types.ts +++ b/packages/beacon-node/src/chain/historicalState/types.ts @@ -11,6 +11,7 @@ export type HistoricalStateRegenInitModules = { config: BeaconConfig; logger: LoggerNode; metrics: Metrics | null; + signal?: AbortSignal; }; export type HistoricalStateRegenModules = HistoricalStateRegenInitModules & { api: ModuleThread; diff --git a/packages/beacon-node/src/node/nodejs.ts b/packages/beacon-node/src/node/nodejs.ts index 552dc49548aa..088541a6b5d5 100644 --- a/packages/beacon-node/src/node/nodejs.ts +++ b/packages/beacon-node/src/node/nodejs.ts @@ -203,8 +203,8 @@ export class BeaconNode { config, metrics, logger: logger.child({module: LoggerModule.chain}), + signal, }); - signal.addEventListener("abort", () => historicalStateRegen.close(), {once: true}); const chain = new BeaconChain(opts.chain, { config, From e8f4d97cfaf6d438081f7d1483f92d4b77cc86ad Mon Sep 17 00:00:00 2001 From: Tuyen Nguyen Date: Tue, 30 Jul 2024 09:49:56 +0700 Subject: [PATCH 23/23] feat: add metrics --- .../historicalState/getHistoricalState.ts | 59 +++++++++++----- .../src/chain/historicalState/types.ts | 19 +++++ .../src/chain/historicalState/worker.ts | 69 +++++++++++++++++-- 3 files changed, 124 insertions(+), 23 deletions(-) diff --git a/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts b/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts index eeae80bfdb39..ada4f3c284d7 100644 --- a/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts +++ b/packages/beacon-node/src/chain/historicalState/getHistoricalState.ts @@ -1,6 +1,5 @@ import { BeaconStateAllForks, - BeaconStateTransitionMetrics, CachedBeaconStateAllForks, DataAvailableStatus, ExecutionPayloadStatus, @@ -10,6 +9,7 @@ import { } from "@lodestar/state-transition"; import {BeaconConfig} from "@lodestar/config"; import {IBeaconDb} from "../../db/index.js"; +import {HistoricalStateRegenMetrics, RegenErrorType} from "./types.js"; /** * Populate a PubkeyIndexMap with any new entries based on a BeaconState @@ -63,22 +63,49 @@ export async function getHistoricalState( config: BeaconConfig, db: IBeaconDb, pubkey2index: PubkeyIndexMap, - metrics?: BeaconStateTransitionMetrics + metrics?: HistoricalStateRegenMetrics ): Promise { - let state = await getNearestState(slot, config, db, pubkey2index); + const regenTimer = metrics?.regenTime.startTimer(); + + const loadStateTimer = metrics?.loadStateTime.startTimer(); + let state = await getNearestState(slot, config, db, pubkey2index).catch((e) => { + metrics?.regenErrorCount.inc({reason: RegenErrorType.loadState}); + throw e; + }); + loadStateTimer?.(); + + const transitionTimer = metrics?.stateTransitionTime.startTimer(); + let blockCount = 0; for await (const block of db.blockArchive.valuesStream({gt: state.slot, lte: slot})) { - state = stateTransition( - state, - block, - { - verifyProposer: false, - verifySignatures: false, - verifyStateRoot: false, - executionPayloadStatus: ExecutionPayloadStatus.valid, - dataAvailableStatus: DataAvailableStatus.available, - }, - metrics - ); + try { + state = stateTransition( + state, + block, + { + verifyProposer: false, + verifySignatures: false, + verifyStateRoot: false, + executionPayloadStatus: ExecutionPayloadStatus.valid, + dataAvailableStatus: DataAvailableStatus.available, + }, + metrics + ); + } catch (e) { + metrics?.regenErrorCount.inc({reason: RegenErrorType.blockProcessing}); + throw e; + } + blockCount++; + if (Buffer.compare(state.hashTreeRoot(), block.message.stateRoot) !== 0) { + metrics?.regenErrorCount.inc({reason: RegenErrorType.invalidStateRoot}); + } } - return state.serialize(); + metrics?.stateTransitionBlocks.observe(blockCount); + transitionTimer?.(); + + const serializeTimer = metrics?.stateSerializationTime.startTimer(); + const stateBytes = state.serialize(); + serializeTimer?.(); + + regenTimer?.(); + return stateBytes; } diff --git a/packages/beacon-node/src/chain/historicalState/types.ts b/packages/beacon-node/src/chain/historicalState/types.ts index a44cf0ed478a..5bc813141a5d 100644 --- a/packages/beacon-node/src/chain/historicalState/types.ts +++ b/packages/beacon-node/src/chain/historicalState/types.ts @@ -1,6 +1,8 @@ import {ModuleThread} from "@chainsafe/threads"; import {BeaconConfig} from "@lodestar/config"; import {LoggerNode, LoggerNodeOpts} from "@lodestar/logger/node"; +import {BeaconStateTransitionMetrics} from "@lodestar/state-transition"; +import {Gauge, Histogram} from "@lodestar/utils"; import {Metrics} from "../../metrics/index.js"; export type HistoricalStateRegenInitModules = { @@ -33,3 +35,20 @@ export type HistoricalStateWorkerApi = { scrapeMetrics(): Promise; getHistoricalState(slot: number): Promise; }; + +export enum RegenErrorType { + loadState = "load_state", + invalidStateRoot = "invalid_state_root", + blockProcessing = "block_processing", +} + +export type HistoricalStateRegenMetrics = BeaconStateTransitionMetrics & { + regenTime: Histogram; + loadStateTime: Histogram; + stateTransitionTime: Histogram; + stateTransitionBlocks: Histogram; + stateSerializationTime: Histogram; + regenRequestCount: Gauge; + regenSuccessCount: Gauge; + regenErrorCount: Gauge<{reason: RegenErrorType}>; +}; diff --git a/packages/beacon-node/src/chain/historicalState/worker.ts b/packages/beacon-node/src/chain/historicalState/worker.ts index 108dbe95f745..9a9f9cc9cd0e 100644 --- a/packages/beacon-node/src/chain/historicalState/worker.ts +++ b/packages/beacon-node/src/chain/historicalState/worker.ts @@ -3,7 +3,6 @@ import {Transfer, expose} from "@chainsafe/threads/worker"; import {createBeaconConfig, chainConfigFromJson} from "@lodestar/config"; import {getNodeLogger} from "@lodestar/logger/node"; import { - BeaconStateTransitionMetrics, EpochTransitionStep, PubkeyIndexMap, StateCloneSource, @@ -14,7 +13,12 @@ import {RegistryMetricCreator, collectNodeJSMetrics} from "../../metrics/index.j import {JobFnQueue} from "../../util/queue/fnQueue.js"; import {QueueMetrics} from "../../util/queue/options.js"; import {BeaconDb} from "../../db/index.js"; -import {HistoricalStateWorkerApi, HistoricalStateWorkerData} from "./types.js"; +import { + HistoricalStateRegenMetrics, + HistoricalStateWorkerApi, + HistoricalStateWorkerData, + RegenErrorType, +} from "./types.js"; import {getHistoricalState} from "./getHistoricalState.js"; // most of this setup copied from networkCoreWorker.ts @@ -35,13 +39,14 @@ const abortController = new AbortController(); // Set up metrics, nodejs, state transition, queue const metricsRegister = workerData.metricsEnabled ? new RegistryMetricCreator() : null; -let stateTransitionMetrics: BeaconStateTransitionMetrics | undefined; +let historicalStateRegenMetrics: HistoricalStateRegenMetrics | undefined; let queueMetrics: QueueMetrics | undefined; if (metricsRegister) { const closeMetrics = collectNodeJSMetrics(metricsRegister, "lodestar_historical_state_worker_"); abortController.signal.addEventListener("abort", closeMetrics, {once: true}); - stateTransitionMetrics = { + historicalStateRegenMetrics = { + // state transition metrics epochTransitionTime: metricsRegister.histogram({ name: "lodestar_historical_state_stfn_epoch_transition_seconds", help: "Time to process a single epoch transition in seconds", @@ -119,6 +124,51 @@ if (metricsRegister) { help: "Total count state.validators nodesPopulated is false on stfn for post state", }), registerValidatorStatuses: () => {}, + + // historical state regen metrics + regenTime: metricsRegister.histogram({ + name: "lodestar_historical_state_regen_time_seconds", + help: "Time to regenerate a historical state in seconds", + // Historical state regen can take up to 3h as of Aug 2024 + // 5m, 10m, 30m, 1h, 3h + buckets: [5 * 60, 10 * 60, 30 * 60, 60 * 60, 180 * 60], + }), + loadStateTime: metricsRegister.histogram({ + name: "lodestar_historical_state_load_nearest_state_time_seconds", + help: "Time to load a nearest historical state from the database in seconds", + // 30s, 1m, 2m, 4m + buckets: [30, 60, 120, 240], + }), + stateTransitionTime: metricsRegister.histogram({ + name: "lodestar_historical_state_state_transition_time_seconds", + help: "Time to run state transition to regen historical state in seconds", + // 5m, 10m, 30m, 1h, 3h + buckets: [5 * 60, 10 * 60, 30 * 60, 60 * 60, 180 * 60], + }), + stateTransitionBlocks: metricsRegister.histogram({ + name: "lodestar_historical_state_state_transition_blocks", + help: "Count of blocks processed during state transition to regen historical state", + // given archiveStateEpochFrequency=1024, it could process up to 32768 blocks + buckets: [10, 100, 1000, 10000, 30000], + }), + stateSerializationTime: metricsRegister.histogram({ + name: "lodestar_historical_state_serialization_time_seconds", + help: "Time to serialize a historical state in seconds", + buckets: [0.25, 0.5, 1, 2], + }), + regenRequestCount: metricsRegister.gauge({ + name: "lodestar_historical_state_request_count", + help: "Count of total historical state requests", + }), + regenSuccessCount: metricsRegister.gauge({ + name: "lodestar_historical_state_success_count", + help: "Count of successful historical state regen", + }), + regenErrorCount: metricsRegister.gauge<{reason: RegenErrorType}>({ + name: "lodestar_historical_state_error_count", + help: "Count of failed historical state regen", + labelNames: ["reason"], + }), }; queueMetrics = { @@ -166,10 +216,15 @@ const api: HistoricalStateWorkerApi = { return metricsRegister?.metrics() ?? ""; }, async getHistoricalState(slot) { - const state = await queue.push(() => - getHistoricalState(slot, config, db, pubkey2index, stateTransitionMetrics) + historicalStateRegenMetrics?.regenRequestCount.inc(); + + const stateBytes = await queue.push(() => + getHistoricalState(slot, config, db, pubkey2index, historicalStateRegenMetrics) ); - return Transfer(state, [state.buffer]) as unknown as Uint8Array; + const result = Transfer(stateBytes, [stateBytes.buffer]) as unknown as Uint8Array; + + historicalStateRegenMetrics?.regenSuccessCount.inc(); + return result; }, };