Add new job queue for historical blocks processing (#442)

* Add TODOs for historical blocks processing

* Add new job for historical blocks processing

* Handle historical job completion

* Fetch latest block in chain and start historical block processing

* Fix starting realtime block processing from latest canonical block

* Refactor historical block processing method and add logs

* Add dummy indexer methods in graph-node to pass test

* Changes in codegen for historical processing in generated watcher
This commit is contained in:
Nabarun Gogoi 2023-11-01 10:38:59 +05:30 committed by GitHub
parent 15ee523e71
commit 6c17662cad
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 306 additions and 51 deletions

View File

@ -253,6 +253,12 @@ export class Database implements DatabaseInterface {
return this._baseDatabase.updateSyncStatusChainHead(repo, blockHash, blockNumber, force); return this._baseDatabase.updateSyncStatusChainHead(repo, blockHash, blockNumber, force);
} }
async forceUpdateSyncStatus (queryRunner: QueryRunner, blockHash: string, blockNumber: number): Promise<SyncStatus> {
const repo = queryRunner.manager.getRepository(SyncStatus);
return this._baseDatabase.forceUpdateSyncStatus(repo, blockHash, blockNumber);
}
async getSyncStatus (queryRunner: QueryRunner): Promise<SyncStatus | undefined> { async getSyncStatus (queryRunner: QueryRunner): Promise<SyncStatus | undefined> {
const repo = queryRunner.manager.getRepository(SyncStatus); const repo = queryRunner.manager.getRepository(SyncStatus);
@ -271,6 +277,12 @@ export class Database implements DatabaseInterface {
return this._baseDatabase.getBlocksAtHeight(repo, height, isPruned); return this._baseDatabase.getBlocksAtHeight(repo, height, isPruned);
} }
async getLatestProcessedBlockProgress (isPruned: boolean): Promise<BlockProgress | undefined> {
const repo = this._conn.getRepository(BlockProgress);
return this._baseDatabase.getLatestProcessedBlockProgress(repo, isPruned);
}
async markBlocksAsPruned (queryRunner: QueryRunner, blocks: BlockProgress[]): Promise<void> { async markBlocksAsPruned (queryRunner: QueryRunner, blocks: BlockProgress[]): Promise<void> {
const repo = queryRunner.manager.getRepository(BlockProgress); const repo = queryRunner.manager.getRepository(BlockProgress);

View File

@ -509,7 +509,7 @@ export class Indexer implements IndexerInterface {
if (!this._serverConfig.enableState) { if (!this._serverConfig.enableState) {
return; return;
} }
const dbTx = await this._db.createTransactionRunner(); const dbTx = await this._db.createTransactionRunner();
let res; let res;
@ -637,6 +637,10 @@ export class Indexer implements IndexerInterface {
return syncStatus; return syncStatus;
} }
async forceUpdateSyncStatus (blockHash: string, blockNumber: number): Promise<SyncStatus> {
return this._baseIndexer.forceUpdateSyncStatus(blockHash, blockNumber);
}
async getEvent (id: string): Promise<Event | undefined> { async getEvent (id: string): Promise<Event | undefined> {
return this._baseIndexer.getEvent(id); return this._baseIndexer.getEvent(id);
} }
@ -653,6 +657,10 @@ export class Indexer implements IndexerInterface {
return this._baseIndexer.getBlocksAtHeight(height, isPruned); return this._baseIndexer.getBlocksAtHeight(height, isPruned);
} }
async getLatestProcessedBlockProgress (isPruned: boolean): Promise<BlockProgress | undefined> {
return this._db.getLatestProcessedBlockProgress(isPruned);
}
async fetchEventsAndSaveBlocks (blocks: DeepPartial<BlockProgress>[]): Promise<{ blockProgress: BlockProgress, events: DeepPartial<Event>[] }[]> { async fetchEventsAndSaveBlocks (blocks: DeepPartial<BlockProgress>[]): Promise<{ blockProgress: BlockProgress, events: DeepPartial<Event>[] }[]> {
return this._baseIndexer.fetchEventsAndSaveBlocks(blocks, this._eventSignaturesMap, this.parseEventNameAndArgs.bind(this)); return this._baseIndexer.fetchEventsAndSaveBlocks(blocks, this._eventSignaturesMap, this.parseEventNameAndArgs.bind(this));
} }

View File

@ -34,6 +34,7 @@ export const main = async (): Promise<any> => {
await jobRunnerCmd.exec(async (jobRunner: JobRunner): Promise<void> => { await jobRunnerCmd.exec(async (jobRunner: JobRunner): Promise<void> => {
await jobRunner.subscribeBlockProcessingQueue(); await jobRunner.subscribeBlockProcessingQueue();
await jobRunner.subscribeHistoricalProcessingQueue();
await jobRunner.subscribeEventProcessingQueue(); await jobRunner.subscribeEventProcessingQueue();
await jobRunner.subscribeBlockCheckpointQueue(); await jobRunner.subscribeBlockCheckpointQueue();
await jobRunner.subscribeHooksQueue(); await jobRunner.subscribeHooksQueue();

View File

@ -93,6 +93,12 @@ export class Indexer implements IndexerInterface {
return []; return [];
} }
async getLatestProcessedBlockProgress (isPruned: boolean): Promise<BlockProgressInterface | undefined> {
assert(isPruned);
return undefined;
}
async getBlockEvents (blockHash: string): Promise<Array<EventInterface>> { async getBlockEvents (blockHash: string): Promise<Array<EventInterface>> {
assert(blockHash); assert(blockHash);
@ -150,6 +156,13 @@ export class Indexer implements IndexerInterface {
return {} as SyncStatusInterface; return {} as SyncStatusInterface;
} }
async forceUpdateSyncStatus (blockHash: string, blockNumber: number): Promise<SyncStatusInterface> {
assert(blockNumber);
assert(blockHash);
return {} as SyncStatusInterface;
}
async markBlocksAsPruned (blocks: BlockProgressInterface[]): Promise<void> { async markBlocksAsPruned (blocks: BlockProgressInterface[]): Promise<void> {
assert(blocks); assert(blocks);

View File

@ -129,7 +129,7 @@ export const fetchBlocksAtHeight = async (
cid, cid,
blockHash, blockHash,
parentHash, parentHash,
blockTimestamp: timestamp blockTimestamp: Number(timestamp)
}); });
} }
@ -182,6 +182,9 @@ export const _fetchBatchBlocks = async (
while (true) { while (true) {
console.time('time:common#fetchBatchBlocks-getBlocks'); console.time('time:common#fetchBatchBlocks-getBlocks');
// TODO: Fetch logs by filter before fetching blocks
// TODO: Fetch only blocks needed for returned logs
// TODO: Save blocks and logs to DB
const blockPromises = blockNumbers.map(async blockNumber => indexer.getBlocks({ blockNumber })); const blockPromises = blockNumbers.map(async blockNumber => indexer.getBlocks({ blockNumber }));
const settledResults = await Promise.allSettled(blockPromises); const settledResults = await Promise.allSettled(blockPromises);
@ -238,7 +241,7 @@ export const _fetchBatchBlocks = async (
} }
blocks.forEach(block => { blocks.forEach(block => {
block.blockTimestamp = block.timestamp; block.blockTimestamp = Number(block.timestamp);
block.blockNumber = Number(block.blockNumber); block.blockNumber = Number(block.blockNumber);
}); });
@ -265,7 +268,7 @@ export const processBatchEvents = async (indexer: IndexerInterface, block: Block
if (indexer.processBlockAfterEvents) { if (indexer.processBlockAfterEvents) {
if (!dbBlock.isComplete) { if (!dbBlock.isComplete) {
await indexer.processBlockAfterEvents(block.blockHash, block.blockNumber); await indexer.processBlockAfterEvents(dbBlock.blockHash, dbBlock.blockNumber);
} }
} }

View File

@ -6,6 +6,7 @@ export const MAX_REORG_DEPTH = 16;
export const DIFF_MERGE_BATCH_SIZE = 10000; export const DIFF_MERGE_BATCH_SIZE = 10000;
export const QUEUE_BLOCK_PROCESSING = 'block-processing'; export const QUEUE_BLOCK_PROCESSING = 'block-processing';
export const QUEUE_HISTORICAL_PROCESSING = 'historical-processing';
export const QUEUE_EVENT_PROCESSING = 'event-processing'; export const QUEUE_EVENT_PROCESSING = 'event-processing';
export const QUEUE_CHAIN_PRUNING = 'chain-pruning'; export const QUEUE_CHAIN_PRUNING = 'chain-pruning';
export const QUEUE_BLOCK_CHECKPOINT = 'block-checkpoint'; export const QUEUE_BLOCK_CHECKPOINT = 'block-checkpoint';

View File

@ -166,6 +166,26 @@ export class Database {
return await repo.save(entity); return await repo.save(entity);
} }
async forceUpdateSyncStatus (repo: Repository<SyncStatusInterface>, blockHash: string, blockNumber: number): Promise<SyncStatusInterface> {
let entity = await repo.findOne();
if (!entity) {
entity = repo.create({
initialIndexedBlockHash: blockHash,
initialIndexedBlockNumber: blockNumber
});
}
entity.chainHeadBlockHash = blockHash;
entity.chainHeadBlockNumber = blockNumber;
entity.latestCanonicalBlockHash = blockHash;
entity.latestCanonicalBlockNumber = blockNumber;
entity.latestIndexedBlockHash = blockHash;
entity.latestIndexedBlockNumber = blockNumber;
return await repo.save(entity);
}
async getBlockProgress (repo: Repository<BlockProgressInterface>, blockHash: string): Promise<BlockProgressInterface | undefined> { async getBlockProgress (repo: Repository<BlockProgressInterface>, blockHash: string): Promise<BlockProgressInterface | undefined> {
return repo.findOne({ where: { blockHash } }); return repo.findOne({ where: { blockHash } });
} }
@ -182,6 +202,14 @@ export class Database {
.getMany(); .getMany();
} }
async getLatestProcessedBlockProgress (repo: Repository<BlockProgressInterface>, isPruned: boolean): Promise<BlockProgressInterface | undefined> {
return repo.createQueryBuilder('block_progress')
.where('is_pruned = :isPruned AND is_complete = :isComplete', { isPruned, isComplete: true })
.orderBy('block_number', 'DESC')
.limit(1)
.getOne();
}
async saveBlockProgress (repo: Repository<BlockProgressInterface>, block: DeepPartial<BlockProgressInterface>): Promise<BlockProgressInterface> { async saveBlockProgress (repo: Repository<BlockProgressInterface>, block: DeepPartial<BlockProgressInterface>): Promise<BlockProgressInterface> {
blockProgressCount.inc(1); blockProgressCount.inc(1);

View File

@ -5,12 +5,15 @@
import assert from 'assert'; import assert from 'assert';
import debug from 'debug'; import debug from 'debug';
import { PubSub } from 'graphql-subscriptions'; import { PubSub } from 'graphql-subscriptions';
import PgBoss from 'pg-boss';
import { constants } from 'ethers';
import { JobQueue } from './job-queue'; import { JobQueue } from './job-queue';
import { BlockProgressInterface, EventInterface, IndexerInterface, EthClient } from './types'; import { BlockProgressInterface, EventInterface, IndexerInterface, EthClient } from './types';
import { MAX_REORG_DEPTH, JOB_KIND_PRUNE, JOB_KIND_INDEX, UNKNOWN_EVENT_NAME, JOB_KIND_EVENTS, QUEUE_BLOCK_PROCESSING, QUEUE_EVENT_PROCESSING } from './constants'; import { MAX_REORG_DEPTH, JOB_KIND_PRUNE, JOB_KIND_INDEX, UNKNOWN_EVENT_NAME, JOB_KIND_EVENTS, QUEUE_BLOCK_PROCESSING, QUEUE_EVENT_PROCESSING, QUEUE_HISTORICAL_PROCESSING } from './constants';
import { createPruningJob, processBlockByNumber } from './common'; import { createPruningJob, processBlockByNumber } from './common';
import { OrderDirection } from './database'; import { OrderDirection } from './database';
import { HISTORICAL_BLOCKS_BATCH_SIZE, HistoricalJobData } from './job-runner';
const EVENT = 'event'; const EVENT = 'event';
@ -26,6 +29,7 @@ export class EventWatcher {
_shutDown = false; _shutDown = false;
_signalCount = 0; _signalCount = 0;
_historicalProcessingEndBlockNumber = 0;
constructor (ethClient: EthClient, indexer: IndexerInterface, pubsub: PubSub, jobQueue: JobQueue) { constructor (ethClient: EthClient, indexer: IndexerInterface, pubsub: PubSub, jobQueue: JobQueue) {
this._ethClient = ethClient; this._ethClient = ethClient;
@ -44,6 +48,7 @@ export class EventWatcher {
async start (): Promise<void> { async start (): Promise<void> {
await this.initBlockProcessingOnCompleteHandler(); await this.initBlockProcessingOnCompleteHandler();
await this.initHistoricalProcessingOnCompleteHandler();
await this.initEventProcessingOnCompleteHandler(); await this.initEventProcessingOnCompleteHandler();
this.startBlockProcessing(); this.startBlockProcessing();
@ -57,6 +62,12 @@ export class EventWatcher {
}); });
} }
async initHistoricalProcessingOnCompleteHandler (): Promise<void> {
this._jobQueue.onComplete(QUEUE_HISTORICAL_PROCESSING, async (job) => {
await this.historicalProcessingCompleteHandler(job);
});
}
async initEventProcessingOnCompleteHandler (): Promise<void> { async initEventProcessingOnCompleteHandler (): Promise<void> {
await this._jobQueue.onComplete(QUEUE_EVENT_PROCESSING, async (job) => { await this._jobQueue.onComplete(QUEUE_EVENT_PROCESSING, async (job) => {
await this.eventProcessingCompleteHandler(job); await this.eventProcessingCompleteHandler(job);
@ -64,17 +75,44 @@ export class EventWatcher {
} }
async startBlockProcessing (): Promise<void> { async startBlockProcessing (): Promise<void> {
const syncStatus = await this._indexer.getSyncStatus(); // Get latest block in chain and sync status from DB.
let startBlockNumber: number; const [{ block: latestBlock }, syncStatus] = await Promise.all([
this._ethClient.getBlockByHash(),
this._indexer.getSyncStatus()
]);
if (!syncStatus) { const latestCanonicalBlockNumber = latestBlock.number - MAX_REORG_DEPTH;
// Get latest block in chain. let startBlockNumber = latestBlock.number;
const { block: currentBlock } = await this._ethClient.getBlockByHash();
startBlockNumber = currentBlock.number; if (syncStatus) {
} else {
startBlockNumber = syncStatus.chainHeadBlockNumber + 1; startBlockNumber = syncStatus.chainHeadBlockNumber + 1;
} }
// Check if starting block for watcher is before latest canonical block
if (startBlockNumber < latestCanonicalBlockNumber) {
await this.startHistoricalBlockProcessing(startBlockNumber, latestCanonicalBlockNumber);
return;
}
await this.startRealtimeBlockProcessing(startBlockNumber);
}
async startHistoricalBlockProcessing (startBlockNumber: number, endBlockNumber: number): Promise<void> {
this._historicalProcessingEndBlockNumber = endBlockNumber;
log(`Starting historical block processing up to block ${this._historicalProcessingEndBlockNumber}`);
// Push job for historical block processing
await this._jobQueue.pushJob(
QUEUE_HISTORICAL_PROCESSING,
{
blockNumber: startBlockNumber
}
);
}
async startRealtimeBlockProcessing (startBlockNumber: number): Promise<void> {
log(`Starting realtime block processing from block ${startBlockNumber}`);
await processBlockByNumber(this._jobQueue, startBlockNumber); await processBlockByNumber(this._jobQueue, startBlockNumber);
// Creating an AsyncIterable from AsyncIterator to iterate over the values. // Creating an AsyncIterable from AsyncIterator to iterate over the values.
@ -139,6 +177,67 @@ export class EventWatcher {
} }
} }
async historicalProcessingCompleteHandler (job: PgBoss.Job<any>): Promise<void> {
const { id, data: { failed, request: { data } } } = job;
const { blockNumber }: HistoricalJobData = data;
if (failed) {
log(`Job ${id} for queue ${QUEUE_HISTORICAL_PROCESSING} failed`);
return;
}
// TODO: Get batch size from config
const nextBatchStartBlockNumber = blockNumber + HISTORICAL_BLOCKS_BATCH_SIZE + 1;
log(`Historical block processing completed for block range: ${blockNumber} to ${nextBatchStartBlockNumber}`);
// Check if historical processing endBlock / latest canonical block is reached
if (nextBatchStartBlockNumber > this._historicalProcessingEndBlockNumber) {
let newSyncStatusBlock: {
blockNumber: number;
blockHash: string;
} | undefined;
// Fetch latest processed block from DB
const latestProcessedBlock = await this._indexer.getLatestProcessedBlockProgress(false);
if (latestProcessedBlock) {
if (latestProcessedBlock.blockNumber > this._historicalProcessingEndBlockNumber) {
// Set new sync status to latest processed block
newSyncStatusBlock = {
blockHash: latestProcessedBlock.blockHash,
blockNumber: latestProcessedBlock.blockNumber
};
}
}
if (!newSyncStatusBlock) {
const [block] = await this._indexer.getBlocks({ blockNumber: this._historicalProcessingEndBlockNumber });
newSyncStatusBlock = {
// At latestCanonicalBlockNumber height null block might be returned in case of FEVM
blockHash: block ? block.blockHash : constants.AddressZero,
blockNumber: this._historicalProcessingEndBlockNumber
};
}
// Update sync status to max of latest processed block or latest canonical block
const syncStatus = await this._indexer.forceUpdateSyncStatus(newSyncStatusBlock.blockHash, newSyncStatusBlock.blockNumber);
log(`Sync status canonical block updated to ${syncStatus.latestCanonicalBlockNumber}`);
// Start realtime processing
this.startBlockProcessing();
return;
}
// Push job for next batch of blocks
await this._jobQueue.pushJob(
QUEUE_HISTORICAL_PROCESSING,
{
blockNumber: nextBatchStartBlockNumber
}
);
}
async eventProcessingCompleteHandler (job: any): Promise<void> { async eventProcessingCompleteHandler (job: any): Promise<void> {
const { id, data: { request, failed, state, createdOn } } = job; const { id, data: { request, failed, state, createdOn } } = job;

View File

@ -130,7 +130,13 @@ const prefetchBlocks = async (
const blockProgress = await indexer.getBlockProgress(blockHash); const blockProgress = await indexer.getBlockProgress(blockHash);
if (!blockProgress) { if (!blockProgress) {
await indexer.saveBlockAndFetchEvents({ cid, blockHash, blockNumber, parentHash, blockTimestamp: timestamp }); await indexer.saveBlockAndFetchEvents({
cid,
blockHash,
blockNumber: Number(blockNumber),
parentHash,
blockTimestamp: Number(timestamp)
});
} }
}); });

View File

@ -22,7 +22,8 @@ export const indexBlock = async (
const blocks = await indexer.getBlocks({ blockNumber: argv.block }); const blocks = await indexer.getBlocks({ blockNumber: argv.block });
blockProgressEntities = blocks.map((block: any): Partial<BlockProgressInterface> => { blockProgressEntities = blocks.map((block: any): Partial<BlockProgressInterface> => {
block.blockTimestamp = block.timestamp; block.blockTimestamp = Number(block.timestamp);
block.blockNumber = Number(block.blockNumber);
return block; return block;
}); });

View File

@ -194,6 +194,23 @@ export class Indexer {
return res; return res;
} }
async forceUpdateSyncStatus (blockHash: string, blockNumber: number): Promise<SyncStatusInterface> {
const dbTx = await this._db.createTransactionRunner();
let res;
try {
res = await this._db.forceUpdateSyncStatus(dbTx, blockHash, blockNumber);
await dbTx.commitTransaction();
} catch (error) {
await dbTx.rollbackTransaction();
throw error;
} finally {
await dbTx.release();
}
return res;
}
async getBlocks (blockFilter: { blockNumber?: number, blockHash?: string }): Promise<any> { async getBlocks (blockFilter: { blockNumber?: number, blockHash?: string }): Promise<any> {
assert(blockFilter.blockHash || blockFilter.blockNumber); assert(blockFilter.blockHash || blockFilter.blockNumber);
const result = await this._ethClient.getBlocks(blockFilter); const result = await this._ethClient.getBlocks(blockFilter);

View File

@ -13,7 +13,7 @@ interface Config {
maxCompletionLag: number maxCompletionLag: number
} }
type JobCallback = (job: any) => Promise<void>; type JobCallback = (job: PgBoss.JobWithDoneCallback<any, any>) => Promise<void>;
const JOBS_PER_INTERVAL = 5; const JOBS_PER_INTERVAL = 5;
@ -93,7 +93,7 @@ export class JobQueue {
teamSize: JOBS_PER_INTERVAL, teamSize: JOBS_PER_INTERVAL,
teamConcurrency: 1 teamConcurrency: 1
}, },
async (job: any) => { async (job) => {
try { try {
log(`Processing queue ${queue} job ${job.id}...`); log(`Processing queue ${queue} job ${job.id}...`);
await callback(job); await callback(job);
@ -114,7 +114,7 @@ export class JobQueue {
teamSize: JOBS_PER_INTERVAL, teamSize: JOBS_PER_INTERVAL,
teamConcurrency: 1 teamConcurrency: 1
}, },
async (job: any) => { async (job: PgBoss.JobWithDoneCallback<any, any>) => {
try { try {
const { id, data: { failed, createdOn } } = job; const { id, data: { failed, createdOn } } = job;
log(`Job onComplete for queue ${queue} job ${id} created ${createdOn} success ${!failed}`); log(`Job onComplete for queue ${queue} job ${id} created ${createdOn} success ${!failed}`);
@ -128,7 +128,7 @@ export class JobQueue {
); );
} }
async markComplete (job: any): Promise<void> { async markComplete (job: PgBoss.Job): Promise<void> {
this._boss.complete(job.id); this._boss.complete(job.id);
} }

View File

@ -6,6 +6,7 @@ import assert from 'assert';
import debug from 'debug'; import debug from 'debug';
import { ethers } from 'ethers'; import { ethers } from 'ethers';
import { DeepPartial, In } from 'typeorm'; import { DeepPartial, In } from 'typeorm';
import PgBoss from 'pg-boss';
import { JobQueueConfig } from './config'; import { JobQueueConfig } from './config';
import { import {
@ -17,7 +18,8 @@ import {
QUEUE_BLOCK_PROCESSING, QUEUE_BLOCK_PROCESSING,
QUEUE_EVENT_PROCESSING, QUEUE_EVENT_PROCESSING,
QUEUE_BLOCK_CHECKPOINT, QUEUE_BLOCK_CHECKPOINT,
QUEUE_HOOKS QUEUE_HOOKS,
QUEUE_HISTORICAL_PROCESSING
} from './constants'; } from './constants';
import { JobQueue } from './job-queue'; import { JobQueue } from './job-queue';
import { BlockProgressInterface, EventInterface, IndexerInterface } from './types'; import { BlockProgressInterface, EventInterface, IndexerInterface } from './types';
@ -34,6 +36,16 @@ import { lastBlockNumEvents, lastBlockProcessDuration, lastProcessedBlockNumber
const log = debug('vulcanize:job-runner'); const log = debug('vulcanize:job-runner');
// Wait time for retrying events processing on error (in ms)
const EVENTS_PROCESSING_RETRY_WAIT = 2000;
// TODO: Get batch size from config
export const HISTORICAL_BLOCKS_BATCH_SIZE = 100;
export interface HistoricalJobData {
blockNumber: number;
}
export class JobRunner { export class JobRunner {
jobQueue: JobQueue; jobQueue: JobQueue;
_indexer: IndexerInterface; _indexer: IndexerInterface;
@ -57,6 +69,12 @@ export class JobRunner {
}); });
} }
async subscribeHistoricalProcessingQueue (): Promise<void> {
await this.jobQueue.subscribe(QUEUE_HISTORICAL_PROCESSING, async (job) => {
await this.processHistoricalBlocks(job);
});
}
async subscribeEventProcessingQueue (): Promise<void> { async subscribeEventProcessingQueue (): Promise<void> {
await this.jobQueue.subscribe(QUEUE_EVENT_PROCESSING, async (job) => { await this.jobQueue.subscribe(QUEUE_EVENT_PROCESSING, async (job) => {
await this.processEvent(job); await this.processEvent(job);
@ -129,6 +147,28 @@ export class JobRunner {
await this.jobQueue.markComplete(job); await this.jobQueue.markComplete(job);
} }
async processHistoricalBlocks (job: PgBoss.JobWithDoneCallback<HistoricalJobData, HistoricalJobData>): Promise<void> {
const { data: { blockNumber: startBlock } } = job;
const endBlock = startBlock + HISTORICAL_BLOCKS_BATCH_SIZE;
log(`Processing historical blocks from ${startBlock} to ${endBlock}`);
// TODO: Use method from common.ts to fetch and save filtered logs and blocks
const blocks: BlockProgressInterface[] = [];
// Push event processing job for each block
const pushJobForBlockPromises = blocks.map(async block => this.jobQueue.pushJob(
QUEUE_EVENT_PROCESSING,
{
kind: JOB_KIND_EVENTS,
blockHash: block.blockHash,
publish: true
}
));
await Promise.all(pushJobForBlockPromises);
await this.jobQueue.markComplete(job);
}
async processEvent (job: any): Promise<EventInterface | void> { async processEvent (job: any): Promise<EventInterface | void> {
const { data: { kind } } = job; const { data: { kind } } = job;
@ -360,8 +400,9 @@ export class JobRunner {
console.timeEnd('time:job-runner#_indexBlock-get-block-progress-entities'); console.timeEnd('time:job-runner#_indexBlock-get-block-progress-entities');
// Check if parent block has been processed yet, if not, push a high priority job to process that first and abort. // Check if parent block has been processed yet, if not, push a high priority job to process that first and abort.
// However, don't go beyond the `latestCanonicalBlockHash` from SyncStatus as we have to assume the reorg can't be that deep. // However, don't go beyond the `latestCanonicalBlockNumber` from SyncStatus as we have to assume the reorg can't be that deep.
if (blockHash !== syncStatus.latestCanonicalBlockHash) { // latestCanonicalBlockNumber is used to handle null blocks in case of FEVM.
if (blockNumber > syncStatus.latestCanonicalBlockNumber) {
// Create a higher priority job to index parent block and then abort. // Create a higher priority job to index parent block and then abort.
// We don't have to worry about aborting as this job will get retried later. // We don't have to worry about aborting as this job will get retried later.
const newPriority = (priority || 0) + 1; const newPriority = (priority || 0) + 1;
@ -382,9 +423,9 @@ export class JobRunner {
kind: JOB_KIND_INDEX, kind: JOB_KIND_INDEX,
cid: parentCid, cid: parentCid,
blockHash: parentHash, blockHash: parentHash,
blockNumber: parentBlockNumber, blockNumber: Number(parentBlockNumber),
parentHash: grandparentHash, parentHash: grandparentHash,
timestamp: parentTimestamp, timestamp: Number(parentTimestamp),
priority: newPriority priority: newPriority
}, { priority: newPriority }); }, { priority: newPriority });
@ -418,8 +459,6 @@ export class JobRunner {
await this._indexer.removeUnknownEvents(parentBlock); await this._indexer.removeUnknownEvents(parentBlock);
console.timeEnd('time:job-runner#_indexBlock-remove-unknown-events'); console.timeEnd('time:job-runner#_indexBlock-remove-unknown-events');
} }
} else {
blockProgress = parentBlock;
} }
if (!blockProgress) { if (!blockProgress) {
@ -442,7 +481,9 @@ export class JobRunner {
} }
} }
await this._indexer.processBlock(blockProgress); if (!blockProgress.isComplete) {
await this._indexer.processBlock(blockProgress);
}
// Push job to event processing queue. // Push job to event processing queue.
// Block with all events processed or no events will not be processed again due to check in _processEvents. // Block with all events processed or no events will not be processed again due to check in _processEvents.
@ -455,40 +496,62 @@ export class JobRunner {
async _processEvents (job: any): Promise<void> { async _processEvents (job: any): Promise<void> {
const { blockHash } = job.data; const { blockHash } = job.data;
if (!this._blockAndEventsMap.has(blockHash)) { try {
console.time('time:job-runner#_processEvents-get-block-progress'); if (!this._blockAndEventsMap.has(blockHash)) {
const block = await this._indexer.getBlockProgress(blockHash); console.time('time:job-runner#_processEvents-get-block-progress');
console.timeEnd('time:job-runner#_processEvents-get-block-progress'); const block = await this._indexer.getBlockProgress(blockHash);
console.timeEnd('time:job-runner#_processEvents-get-block-progress');
assert(block); assert(block);
this._blockAndEventsMap.set(blockHash, { block, events: [] }); this._blockAndEventsMap.set(blockHash, { block, events: [] });
} }
const prefetchedBlock = this._blockAndEventsMap.get(blockHash); const prefetchedBlock = this._blockAndEventsMap.get(blockHash);
assert(prefetchedBlock); assert(prefetchedBlock);
const { block } = prefetchedBlock; const { block } = prefetchedBlock;
console.time('time:job-runner#_processEvents-events'); console.time('time:job-runner#_processEvents-events');
await processBatchEvents(this._indexer, block, this._jobQueueConfig.eventsInBatch, this._jobQueueConfig.subgraphEventsOrder); await processBatchEvents(this._indexer, block, this._jobQueueConfig.eventsInBatch, this._jobQueueConfig.subgraphEventsOrder);
console.timeEnd('time:job-runner#_processEvents-events'); console.timeEnd('time:job-runner#_processEvents-events');
// Update metrics // Update metrics
lastProcessedBlockNumber.set(block.blockNumber); lastProcessedBlockNumber.set(block.blockNumber);
lastBlockNumEvents.set(block.numEvents); lastBlockNumEvents.set(block.numEvents);
this._blockAndEventsMap.delete(block.blockHash); this._blockAndEventsMap.delete(block.blockHash);
if (this._endBlockProcessTimer) { if (this._endBlockProcessTimer) {
this._endBlockProcessTimer(); this._endBlockProcessTimer();
} }
this._endBlockProcessTimer = lastBlockProcessDuration.startTimer(); this._endBlockProcessTimer = lastBlockProcessDuration.startTimer();
if (this._shutDown) { if (this._shutDown) {
log(`Graceful shutdown after processing block ${block.blockNumber}`); log(`Graceful shutdown after processing block ${block.blockNumber}`);
this.jobQueue.stop(); this.jobQueue.stop();
process.exit(0); process.exit(0);
}
} catch (error) {
log(`Error in processing events for block ${blockHash}`);
log(error);
// TODO: Remove processed entities for current block to avoid reprocessing of events
// Catch event processing error and push to job queue after some time with higher priority
log(`Retrying event processing after ${EVENTS_PROCESSING_RETRY_WAIT} ms`);
await wait(EVENTS_PROCESSING_RETRY_WAIT);
await this.jobQueue.pushJob(
QUEUE_EVENT_PROCESSING,
{
kind: JOB_KIND_EVENTS,
blockHash: blockHash,
publish: true
},
{
priority: 1
}
);
} }
} }

View File

@ -91,6 +91,7 @@ export interface IndexerInterface {
getStateSyncStatus (): Promise<StateSyncStatusInterface | undefined> getStateSyncStatus (): Promise<StateSyncStatusInterface | undefined>
getBlocks (blockFilter: { blockHash?: string, blockNumber?: number }): Promise<any> getBlocks (blockFilter: { blockHash?: string, blockNumber?: number }): Promise<any>
getBlocksAtHeight (height: number, isPruned: boolean): Promise<BlockProgressInterface[]> getBlocksAtHeight (height: number, isPruned: boolean): Promise<BlockProgressInterface[]>
getLatestProcessedBlockProgress (isPruned: boolean): Promise<BlockProgressInterface | undefined>
getLatestCanonicalBlock (): Promise<BlockProgressInterface | undefined> getLatestCanonicalBlock (): Promise<BlockProgressInterface | undefined>
getLatestStateIndexedBlock (): Promise<BlockProgressInterface> getLatestStateIndexedBlock (): Promise<BlockProgressInterface>
getBlockEvents (blockHash: string, where: Where, queryOptions: QueryOptions): Promise<Array<EventInterface>> getBlockEvents (blockHash: string, where: Where, queryOptions: QueryOptions): Promise<Array<EventInterface>>
@ -102,6 +103,7 @@ export interface IndexerInterface {
updateSyncStatusChainHead (blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface> updateSyncStatusChainHead (blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface>
updateSyncStatusIndexedBlock (blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface> updateSyncStatusIndexedBlock (blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface>
updateSyncStatusCanonicalBlock (blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface> updateSyncStatusCanonicalBlock (blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface>
forceUpdateSyncStatus (blockHash: string, blockNumber: number): Promise<SyncStatusInterface>
updateStateSyncStatusIndexedBlock (blockNumber: number, force?: boolean): Promise<StateSyncStatusInterface | undefined> updateStateSyncStatusIndexedBlock (blockNumber: number, force?: boolean): Promise<StateSyncStatusInterface | undefined>
updateStateSyncStatusCheckpointBlock (blockNumber: number, force?: boolean): Promise<StateSyncStatusInterface> updateStateSyncStatusCheckpointBlock (blockNumber: number, force?: boolean): Promise<StateSyncStatusInterface>
markBlocksAsPruned (blocks: BlockProgressInterface[]): Promise<void> markBlocksAsPruned (blocks: BlockProgressInterface[]): Promise<void>
@ -164,6 +166,7 @@ export interface DatabaseInterface {
updateSyncStatusIndexedBlock (queryRunner: QueryRunner, blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface>; updateSyncStatusIndexedBlock (queryRunner: QueryRunner, blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface>;
updateSyncStatusChainHead (queryRunner: QueryRunner, blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface>; updateSyncStatusChainHead (queryRunner: QueryRunner, blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface>;
updateSyncStatusCanonicalBlock (queryRunner: QueryRunner, blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface>; updateSyncStatusCanonicalBlock (queryRunner: QueryRunner, blockHash: string, blockNumber: number, force?: boolean): Promise<SyncStatusInterface>;
forceUpdateSyncStatus (queryRunner: QueryRunner, blockHash: string, blockNumber: number): Promise<SyncStatusInterface>;
saveEvents (queryRunner: QueryRunner, events: DeepPartial<EventInterface>[]): Promise<void>; saveEvents (queryRunner: QueryRunner, events: DeepPartial<EventInterface>[]): Promise<void>;
saveBlockWithEvents (queryRunner: QueryRunner, block: DeepPartial<BlockProgressInterface>, events: DeepPartial<EventInterface>[]): Promise<BlockProgressInterface>; saveBlockWithEvents (queryRunner: QueryRunner, block: DeepPartial<BlockProgressInterface>, events: DeepPartial<EventInterface>[]): Promise<BlockProgressInterface>;
saveEventEntity (queryRunner: QueryRunner, entity: EventInterface): Promise<EventInterface>; saveEventEntity (queryRunner: QueryRunner, entity: EventInterface): Promise<EventInterface>;