diff --git a/codegenerator/cli/npm/envio/src/Batch.res b/codegenerator/cli/npm/envio/src/Batch.res index dc040fe3e..59781e25b 100644 --- a/codegenerator/cli/npm/envio/src/Batch.res +++ b/codegenerator/cli/npm/envio/src/Batch.res @@ -1,17 +1,35 @@ -type progressedChain = { - chainId: int, +open Belt + +@@warning("-44") +open Utils.UnsafeIntOperators + +type chainAfterBatch = { batchSize: int, progressBlockNumber: int, - isProgressAtHead: bool, + totalEventsProcessed: int, + fetchState: FetchState.t, + dcsToStore: option>, + isProgressAtHeadWhenBatchCreated: bool, +} + +type chainBeforeBatch = { + fetchState: FetchState.t, + reorgDetection: ReorgDetection.t, + progressBlockNumber: int, + sourceBlockNumber: int, totalEventsProcessed: int, } type t = { + totalBatchSize: int, items: array, - progressedChains: array, - updatedFetchStates: ChainMap.t, - dcsToStoreByChainId: dict>, - creationTimeMs: int, + progressedChainsById: dict, + // Unnest-like checkpoint fields: + checkpointIds: array, + checkpointChainIds: array, + checkpointBlockNumbers: array, + checkpointBlockHashes: array>, + checkpointEventsProcessed: array, } /** @@ -22,7 +40,7 @@ let getOrderedNextChain = (fetchStates: ChainMap.t, ~batchSizePerC let earliestChainTimestamp = ref(0) let chainKeys = fetchStates->ChainMap.keys for idx in 0 to chainKeys->Array.length - 1 { - let chain = chainKeys->Array.get(idx) + let chain = chainKeys->Array.getUnsafe(idx) let fetchState = fetchStates->ChainMap.get(chain) if fetchState->FetchState.isActivelyIndexing { let timestamp = fetchState->FetchState.getTimestampAt( @@ -75,37 +93,283 @@ let hasMultichainReadyItem = ( } } +let getProgressedChainsById = { + let getChainAfterBatchIfProgressed = ( + ~chainBeforeBatch: chainBeforeBatch, + ~progressBlockNumberAfterBatch, + ~fetchStateAfterBatch, + ~batchSize, + ~dcsToStore, + ) => { + // The check is sufficient, since we guarantee to include a full block in a batch + // Also, this might be true even if batchSize is 0, + // eg when indexing at the head and chain doesn't have items in a block + if chainBeforeBatch.progressBlockNumber < progressBlockNumberAfterBatch { + Some( + ( + { + batchSize, + progressBlockNumber: progressBlockNumberAfterBatch, + totalEventsProcessed: chainBeforeBatch.totalEventsProcessed + batchSize, + dcsToStore, + fetchState: fetchStateAfterBatch, + isProgressAtHeadWhenBatchCreated: progressBlockNumberAfterBatch >= + chainBeforeBatch.sourceBlockNumber, + }: chainAfterBatch + ), + ) + } else { + None + } + } + + ( + ~chainsBeforeBatch: ChainMap.t, + ~batchSizePerChain: dict, + ~progressBlockNumberPerChain: dict, + ) => { + let progressedChainsById = Js.Dict.empty() + + // Needed to: + // - Recalculate the computed queue sizes + // - Accumulate registered dynamic contracts to store in the db + // - Trigger onBlock pointer update + chainsBeforeBatch + ->ChainMap.values + ->Array.forEachU(chainBeforeBatch => { + let fetchState = chainBeforeBatch.fetchState + + let progressBlockNumberAfterBatch = switch progressBlockNumberPerChain->Utils.Dict.dangerouslyGetNonOption( + fetchState.chainId->Int.toString, + ) { + | Some(progressBlockNumber) => progressBlockNumber + | None => chainBeforeBatch.progressBlockNumber + } + + switch switch batchSizePerChain->Utils.Dict.dangerouslyGetNonOption( + fetchState.chainId->Int.toString, + ) { + | Some(batchSize) => + let leftItems = fetchState.buffer->Js.Array2.sliceFrom(batchSize) + switch fetchState.dcsToStore { + | [] => + getChainAfterBatchIfProgressed( + ~chainBeforeBatch, + ~batchSize, + ~dcsToStore=None, + ~fetchStateAfterBatch=fetchState->FetchState.updateInternal(~mutItems=leftItems), + ~progressBlockNumberAfterBatch, + ) + + | dcs => { + let leftDcsToStore = [] + let batchDcs = [] + let fetchStateAfterBatch = + fetchState->FetchState.updateInternal(~mutItems=leftItems, ~dcsToStore=leftDcsToStore) + + dcs->Array.forEach(dc => { + // Important: This should be a registering block number. + // This works for now since dc.startBlock is a registering block number. + if dc.startBlock <= progressBlockNumberAfterBatch { + batchDcs->Array.push(dc) + } else { + // Mutate the array we passed to the updateInternal beforehand + leftDcsToStore->Array.push(dc) + } + }) + + getChainAfterBatchIfProgressed( + ~chainBeforeBatch, + ~batchSize, + ~dcsToStore=Some(batchDcs), + ~fetchStateAfterBatch, + ~progressBlockNumberAfterBatch, + ) + } + } + // Skip not affected chains + | None => + getChainAfterBatchIfProgressed( + ~chainBeforeBatch, + ~batchSize=0, + ~dcsToStore=None, + ~fetchStateAfterBatch=chainBeforeBatch.fetchState, + ~progressBlockNumberAfterBatch, + ) + } { + | Some(progressedChain) => + progressedChainsById->Utils.Dict.setByInt( + chainBeforeBatch.fetchState.chainId, + progressedChain, + ) + | None => () + } + }) + + progressedChainsById + } +} + +@inline +let addReorgCheckpoints = ( + ~prevCheckpointId, + ~reorgDetection: ReorgDetection.t, + ~fromBlockExclusive, + ~toBlockExclusive, + ~chainId, + ~mutCheckpointIds, + ~mutCheckpointChainIds, + ~mutCheckpointBlockNumbers, + ~mutCheckpointBlockHashes, + ~mutCheckpointEventsProcessed, +) => { + if ( + reorgDetection.shouldRollbackOnReorg && !(reorgDetection.dataByBlockNumber->Utils.Dict.isEmpty) + ) { + let prevCheckpointId = ref(prevCheckpointId) + for blockNumber in fromBlockExclusive + 1 to toBlockExclusive - 1 { + switch reorgDetection->ReorgDetection.getHashByBlockNumber(~blockNumber) { + | Js.Null.Value(hash) => + let checkpointId = prevCheckpointId.contents + 1 + prevCheckpointId := checkpointId + + mutCheckpointIds->Js.Array2.push(checkpointId)->ignore + mutCheckpointChainIds->Js.Array2.push(chainId)->ignore + mutCheckpointBlockNumbers->Js.Array2.push(blockNumber)->ignore + mutCheckpointBlockHashes->Js.Array2.push(Js.Null.Value(hash))->ignore + mutCheckpointEventsProcessed->Js.Array2.push(0)->ignore + | Js.Null.Null => () + } + } + prevCheckpointId.contents + } else { + prevCheckpointId + } +} + let prepareOrderedBatch = ( + ~checkpointIdBeforeBatch, + ~chainsBeforeBatch: ChainMap.t, ~batchSizeTarget, - ~fetchStates: ChainMap.t, - ~mutBatchSizePerChain: dict, ) => { - let batchSize = ref(0) + let totalBatchSize = ref(0) let isFinished = ref(false) + let prevCheckpointId = ref(checkpointIdBeforeBatch) + let mutBatchSizePerChain = Js.Dict.empty() + let mutProgressBlockNumberPerChain = Js.Dict.empty() + + let fetchStates = chainsBeforeBatch->ChainMap.map(chainBeforeBatch => chainBeforeBatch.fetchState) + let items = [] + let checkpointIds = [] + let checkpointChainIds = [] + let checkpointBlockNumbers = [] + let checkpointBlockHashes = [] + let checkpointEventsProcessed = [] - while batchSize.contents < batchSizeTarget && !isFinished.contents { + while totalBatchSize.contents < batchSizeTarget && !isFinished.contents { switch fetchStates->getOrderedNextChain(~batchSizePerChain=mutBatchSizePerChain) { | Some(fetchState) => { + let chainBeforeBatch = + chainsBeforeBatch->ChainMap.get(ChainMap.Chain.makeUnsafe(~chainId=fetchState.chainId)) let itemsCountBefore = switch mutBatchSizePerChain->Utils.Dict.dangerouslyGetByIntNonOption( fetchState.chainId, ) { | Some(batchSize) => batchSize | None => 0 } - let newItemsCount = - fetchState->FetchState.getReadyItemsCount(~targetSize=1, ~fromItem=itemsCountBefore) + + let prevBlockNumber = switch mutProgressBlockNumberPerChain->Utils.Dict.dangerouslyGetByIntNonOption( + fetchState.chainId, + ) { + | Some(progressBlockNumber) => progressBlockNumber + | None => chainBeforeBatch.progressBlockNumber + } + + let newItemsCount = fetchState->FetchState.getReadyItemsCount( + // We should get items only for a single block + // Since for the ordered mode next block could be after another chain's block + ~targetSize=1, + ~fromItem=itemsCountBefore, + ) if newItemsCount > 0 { - for idx in itemsCountBefore to itemsCountBefore + newItemsCount - 1 { - items->Js.Array2.push(fetchState.buffer->Belt.Array.getUnsafe(idx))->ignore + let item0 = fetchState.buffer->Array.getUnsafe(itemsCountBefore) + let blockNumber = item0->Internal.getItemBlockNumber + + prevCheckpointId := + addReorgCheckpoints( + ~chainId=fetchState.chainId, + ~reorgDetection=chainBeforeBatch.reorgDetection, + ~prevCheckpointId=prevCheckpointId.contents, + ~fromBlockExclusive=prevBlockNumber, + ~toBlockExclusive=blockNumber, + ~mutCheckpointIds=checkpointIds, + ~mutCheckpointChainIds=checkpointChainIds, + ~mutCheckpointBlockNumbers=checkpointBlockNumbers, + ~mutCheckpointBlockHashes=checkpointBlockHashes, + ~mutCheckpointEventsProcessed=checkpointEventsProcessed, + ) + + let checkpointId = prevCheckpointId.contents + 1 + + items + ->Js.Array2.push(item0) + ->ignore + for idx in 1 to newItemsCount - 1 { + items + ->Js.Array2.push(fetchState.buffer->Belt.Array.getUnsafe(itemsCountBefore + idx)) + ->ignore } - batchSize := batchSize.contents + newItemsCount + + checkpointIds + ->Js.Array2.push(checkpointId) + ->ignore + checkpointChainIds + ->Js.Array2.push(fetchState.chainId) + ->ignore + checkpointBlockNumbers + ->Js.Array2.push(blockNumber) + ->ignore + checkpointBlockHashes + ->Js.Array2.push( + chainBeforeBatch.reorgDetection->ReorgDetection.getHashByBlockNumber(~blockNumber), + ) + ->ignore + checkpointEventsProcessed + ->Js.Array2.push(newItemsCount) + ->ignore + + prevCheckpointId := checkpointId + totalBatchSize := totalBatchSize.contents + newItemsCount mutBatchSizePerChain->Utils.Dict.setByInt( fetchState.chainId, itemsCountBefore + newItemsCount, ) + mutProgressBlockNumberPerChain->Utils.Dict.setByInt(fetchState.chainId, blockNumber) } else { + let blockNumberAfterBatch = fetchState->FetchState.bufferBlockNumber + + prevCheckpointId := + addReorgCheckpoints( + ~chainId=fetchState.chainId, + ~reorgDetection=chainBeforeBatch.reorgDetection, + ~prevCheckpointId=prevCheckpointId.contents, + ~fromBlockExclusive=prevBlockNumber, + ~toBlockExclusive=blockNumberAfterBatch + 1, // Make it inclusive + ~mutCheckpointIds=checkpointIds, + ~mutCheckpointChainIds=checkpointChainIds, + ~mutCheckpointBlockNumbers=checkpointBlockNumbers, + ~mutCheckpointBlockHashes=checkpointBlockHashes, + ~mutCheckpointEventsProcessed=checkpointEventsProcessed, + ) + + // Since the chain was chosen as next + // the fact that it doesn't have new items means that it reached the buffer block number + mutProgressBlockNumberPerChain->Utils.Dict.setByInt( + fetchState.chainId, + blockNumberAfterBatch, + ) isFinished := true } } @@ -114,45 +378,187 @@ let prepareOrderedBatch = ( } } - items + { + totalBatchSize: totalBatchSize.contents, + items, + progressedChainsById: getProgressedChainsById( + ~chainsBeforeBatch, + ~batchSizePerChain=mutBatchSizePerChain, + ~progressBlockNumberPerChain=mutProgressBlockNumberPerChain, + ), + checkpointIds, + checkpointChainIds, + checkpointBlockNumbers, + checkpointBlockHashes, + checkpointEventsProcessed, + } } let prepareUnorderedBatch = ( + ~checkpointIdBeforeBatch, + ~chainsBeforeBatch: ChainMap.t, ~batchSizeTarget, - ~fetchStates: ChainMap.t, - ~mutBatchSizePerChain: dict, ) => { let preparedFetchStates = - fetchStates + chainsBeforeBatch ->ChainMap.values - ->FetchState.filterAndSortForUnorderedBatch(~batchSizeTarget) + ->Js.Array2.map(chainBeforeBatch => chainBeforeBatch.fetchState) + ->FetchState.sortForUnorderedBatch(~batchSizeTarget) let chainIdx = ref(0) let preparedNumber = preparedFetchStates->Array.length - let batchSize = ref(0) + let totalBatchSize = ref(0) + + let prevCheckpointId = ref(checkpointIdBeforeBatch) + let mutBatchSizePerChain = Js.Dict.empty() + let mutProgressBlockNumberPerChain = Js.Dict.empty() let items = [] + let checkpointIds = [] + let checkpointChainIds = [] + let checkpointBlockNumbers = [] + let checkpointBlockHashes = [] + let checkpointEventsProcessed = [] // Accumulate items for all actively indexing chains // the way to group as many items from a single chain as possible // This way the loaders optimisations will hit more often - while batchSize.contents < batchSizeTarget && chainIdx.contents < preparedNumber { + while totalBatchSize.contents < batchSizeTarget && chainIdx.contents < preparedNumber { let fetchState = preparedFetchStates->Js.Array2.unsafe_get(chainIdx.contents) let chainBatchSize = fetchState->FetchState.getReadyItemsCount( - ~targetSize=batchSizeTarget - batchSize.contents, + ~targetSize=batchSizeTarget - totalBatchSize.contents, ~fromItem=0, ) + let chainBeforeBatch = + chainsBeforeBatch->ChainMap.get(ChainMap.Chain.makeUnsafe(~chainId=fetchState.chainId)) + + let prevBlockNumber = ref(chainBeforeBatch.progressBlockNumber) if chainBatchSize > 0 { for idx in 0 to chainBatchSize - 1 { - items->Js.Array2.push(fetchState.buffer->Belt.Array.getUnsafe(idx))->ignore + let item = fetchState.buffer->Belt.Array.getUnsafe(idx) + let blockNumber = item->Internal.getItemBlockNumber + + // Every new block we should create a new checkpoint + if blockNumber !== prevBlockNumber.contents { + prevCheckpointId := + addReorgCheckpoints( + ~chainId=fetchState.chainId, + ~reorgDetection=chainBeforeBatch.reorgDetection, + ~prevCheckpointId=prevCheckpointId.contents, + ~fromBlockExclusive=prevBlockNumber.contents, + ~toBlockExclusive=blockNumber, + ~mutCheckpointIds=checkpointIds, + ~mutCheckpointChainIds=checkpointChainIds, + ~mutCheckpointBlockNumbers=checkpointBlockNumbers, + ~mutCheckpointBlockHashes=checkpointBlockHashes, + ~mutCheckpointEventsProcessed=checkpointEventsProcessed, + ) + + let checkpointId = prevCheckpointId.contents + 1 + + checkpointIds->Js.Array2.push(checkpointId)->ignore + checkpointChainIds->Js.Array2.push(fetchState.chainId)->ignore + checkpointBlockNumbers->Js.Array2.push(blockNumber)->ignore + checkpointBlockHashes + ->Js.Array2.push( + chainBeforeBatch.reorgDetection->ReorgDetection.getHashByBlockNumber(~blockNumber), + ) + ->ignore + checkpointEventsProcessed->Js.Array2.push(1)->ignore + + prevBlockNumber := blockNumber + prevCheckpointId := checkpointId + } else { + let lastIndex = checkpointEventsProcessed->Array.length - 1 + checkpointEventsProcessed + ->Belt.Array.setUnsafe( + lastIndex, + checkpointEventsProcessed->Array.getUnsafe(lastIndex) + 1, + ) + ->ignore + } + + items->Js.Array2.push(item)->ignore } - batchSize := batchSize.contents + chainBatchSize + + totalBatchSize := totalBatchSize.contents + chainBatchSize mutBatchSizePerChain->Utils.Dict.setByInt(fetchState.chainId, chainBatchSize) } + let progressBlockNumberAfterBatch = + fetchState->FetchState.getUnorderedMultichainProgressBlockNumberAt(~index=chainBatchSize) + + prevCheckpointId := + addReorgCheckpoints( + ~chainId=fetchState.chainId, + ~reorgDetection=chainBeforeBatch.reorgDetection, + ~prevCheckpointId=prevCheckpointId.contents, + ~fromBlockExclusive=prevBlockNumber.contents, + ~toBlockExclusive=progressBlockNumberAfterBatch + 1, // Make it inclusive + ~mutCheckpointIds=checkpointIds, + ~mutCheckpointChainIds=checkpointChainIds, + ~mutCheckpointBlockNumbers=checkpointBlockNumbers, + ~mutCheckpointBlockHashes=checkpointBlockHashes, + ~mutCheckpointEventsProcessed=checkpointEventsProcessed, + ) + + mutProgressBlockNumberPerChain->Utils.Dict.setByInt( + fetchState.chainId, + progressBlockNumberAfterBatch, + ) + chainIdx := chainIdx.contents + 1 } - items + { + totalBatchSize: totalBatchSize.contents, + items, + progressedChainsById: getProgressedChainsById( + ~chainsBeforeBatch, + ~batchSizePerChain=mutBatchSizePerChain, + ~progressBlockNumberPerChain=mutProgressBlockNumberPerChain, + ), + checkpointIds, + checkpointChainIds, + checkpointBlockNumbers, + checkpointBlockHashes, + checkpointEventsProcessed, + } +} + +let make = ( + ~checkpointIdBeforeBatch, + ~chainsBeforeBatch: ChainMap.t, + ~multichain: InternalConfig.multichain, + ~batchSizeTarget, +) => { + if ( + switch multichain { + | Unordered => true + | Ordered => chainsBeforeBatch->ChainMap.size === 1 + } + ) { + prepareUnorderedBatch(~checkpointIdBeforeBatch, ~chainsBeforeBatch, ~batchSizeTarget) + } else { + prepareOrderedBatch(~checkpointIdBeforeBatch, ~chainsBeforeBatch, ~batchSizeTarget) + } +} + +let findFirstEventBlockNumber = (batch: t, ~chainId) => { + let idx = ref(0) + let result = ref(None) + let checkpointsLength = batch.checkpointIds->Array.length + while idx.contents < checkpointsLength && result.contents === None { + let checkpointChainId = batch.checkpointChainIds->Array.getUnsafe(idx.contents) + if ( + checkpointChainId === chainId && + batch.checkpointEventsProcessed->Array.getUnsafe(idx.contents) > 0 + ) { + result := Some(batch.checkpointBlockNumbers->Array.getUnsafe(idx.contents)) + } else { + idx := idx.contents + 1 + } + } + result.contents } diff --git a/codegenerator/cli/npm/envio/src/FetchState.res b/codegenerator/cli/npm/envio/src/FetchState.res index 5427904ab..a6c7c621c 100644 --- a/codegenerator/cli/npm/envio/src/FetchState.res +++ b/codegenerator/cli/npm/envio/src/FetchState.res @@ -92,7 +92,7 @@ let mergeIntoPartition = (p: partition, ~target: partition, ~maxAddrInPartition) let allowedAddressesNumber = ref(maxAddrInPartition) - target.addressesByContractName->Utils.Dict.forEachWithKey((contractName, addresses) => { + target.addressesByContractName->Utils.Dict.forEachWithKey((addresses, contractName) => { allowedAddressesNumber := allowedAddressesNumber.contents - addresses->Array.length mergedAddresses->Js.Dict.set(contractName, addresses) }) @@ -100,7 +100,7 @@ let mergeIntoPartition = (p: partition, ~target: partition, ~maxAddrInPartition) // Start with putting all addresses to the merging dict // And if they exceed the limit, start removing from the merging dict // and putting into the rest dict - p.addressesByContractName->Utils.Dict.forEachWithKey((contractName, addresses) => { + p.addressesByContractName->Utils.Dict.forEachWithKey((addresses, contractName) => { allowedAddressesNumber := allowedAddressesNumber.contents - addresses->Array.length switch mergedAddresses->Utils.Dict.dangerouslyGetNonOption(contractName) { | Some(targetAddresses) => @@ -112,7 +112,7 @@ let mergeIntoPartition = (p: partition, ~target: partition, ~maxAddrInPartition) let rest = if allowedAddressesNumber.contents < 0 { let restAddresses = Js.Dict.empty() - mergedAddresses->Utils.Dict.forEachWithKey((contractName, addresses) => { + mergedAddresses->Utils.Dict.forEachWithKey((addresses, contractName) => { if allowedAddressesNumber.contents === 0 { () } else if addresses->Array.length <= -allowedAddressesNumber.contents { @@ -1153,7 +1153,7 @@ let rollbackPartition = ( }) | {addressesByContractName} => let rollbackedAddressesByContractName = Js.Dict.empty() - addressesByContractName->Utils.Dict.forEachWithKey((contractName, addresses) => { + addressesByContractName->Utils.Dict.forEachWithKey((addresses, contractName) => { let keptAddresses = addresses->Array.keep(address => !(addressesToRemove->Utils.Set.has(address))) if keptAddresses->Array.length > 0 { @@ -1252,7 +1252,7 @@ let isReadyToEnterReorgThreshold = ( buffer->Utils.Array.isEmpty } -let filterAndSortForUnorderedBatch = { +let sortForUnorderedBatch = { let hasFullBatch = ({buffer} as fetchState: t, ~batchSizeTarget) => { switch buffer->Belt.Array.get(batchSizeTarget - 1) { | Some(item) => item->Internal.getItemBlockNumber <= fetchState->bufferBlockNumber @@ -1262,20 +1262,24 @@ let filterAndSortForUnorderedBatch = { (fetchStates: array, ~batchSizeTarget: int) => { fetchStates - ->Array.keepU(hasReadyItem) + ->Array.copy ->Js.Array2.sortInPlaceWith((a: t, b: t) => { switch (a->hasFullBatch(~batchSizeTarget), b->hasFullBatch(~batchSizeTarget)) { | (true, true) | (false, false) => - // Use unsafe since we filtered out all queues without batch items - switch (a.buffer->Belt.Array.getUnsafe(0), b.buffer->Belt.Array.getUnsafe(0)) { - | (Event({timestamp: aTimestamp}), Event({timestamp: bTimestamp})) => + switch (a.buffer->Belt.Array.get(0), b.buffer->Belt.Array.get(0)) { + | (Some(Event({timestamp: aTimestamp})), Some(Event({timestamp: bTimestamp}))) => aTimestamp - bTimestamp - | (Block(_), _) - | (_, Block(_)) => + | (Some(Block(_)), _) + | (_, Some(Block(_))) => // Currently block items don't have a timestamp, // so we sort chains with them in a random order Js.Math.random_int(-1, 1) + // We don't care about the order of chains with no items + // Just keep them to increase the progress block number when relevant + | (Some(_), None) => -1 + | (None, Some(_)) => 1 + | (None, None) => 0 } | (true, false) => -1 | (false, true) => 1 @@ -1284,9 +1288,10 @@ let filterAndSortForUnorderedBatch = { } } -let getProgressBlockNumber = ({buffer} as fetchState: t) => { +// Ordered multichain mode can't skip blocks, even if there are no items. +let getUnorderedMultichainProgressBlockNumberAt = ({buffer} as fetchState: t, ~index) => { let bufferBlockNumber = fetchState->bufferBlockNumber - switch buffer->Belt.Array.get(0) { + switch buffer->Belt.Array.get(index) { | Some(item) if bufferBlockNumber >= item->Internal.getItemBlockNumber => item->Internal.getItemBlockNumber - 1 | _ => bufferBlockNumber diff --git a/codegenerator/cli/npm/envio/src/Internal.res b/codegenerator/cli/npm/envio/src/Internal.res index deacc9f5c..855d1bc02 100644 --- a/codegenerator/cli/npm/envio/src/Internal.res +++ b/codegenerator/cli/npm/envio/src/Internal.res @@ -285,3 +285,14 @@ let makeCacheTable = (~effectName) => { @genType.import(("./Types.ts", "Invalid")) type noEventFilters + +type reorgCheckpoint = { + @as("id") + checkpointId: int, + @as("chain_id") + chainId: int, + @as("block_number") + blockNumber: int, + @as("block_hash") + blockHash: string, +} diff --git a/codegenerator/cli/npm/envio/src/InternalConfig.res b/codegenerator/cli/npm/envio/src/InternalConfig.res index 61f3d2ec6..447b999c1 100644 --- a/codegenerator/cli/npm/envio/src/InternalConfig.res +++ b/codegenerator/cli/npm/envio/src/InternalConfig.res @@ -30,7 +30,7 @@ type chain = { id: int, startBlock: int, endBlock?: int, - confirmedBlockThreshold: int, + maxReorgDepth: int, contracts: array, sources: array, } diff --git a/codegenerator/cli/npm/envio/src/Persistence.res b/codegenerator/cli/npm/envio/src/Persistence.res index 8986dd232..bd87052b9 100644 --- a/codegenerator/cli/npm/envio/src/Persistence.res +++ b/codegenerator/cli/npm/envio/src/Persistence.res @@ -17,6 +17,9 @@ type initialState = { cleanRun: bool, cache: dict, chains: array, + checkpointId: int, + // Needed to keep reorg detection logic between restarts + reorgCheckpoints: array, } type operator = [#">" | #"="] diff --git a/codegenerator/cli/npm/envio/src/PgStorage.res b/codegenerator/cli/npm/envio/src/PgStorage.res index 9e7211d1b..2e21fc54a 100644 --- a/codegenerator/cli/npm/envio/src/PgStorage.res +++ b/codegenerator/cli/npm/envio/src/PgStorage.res @@ -65,7 +65,7 @@ let makeInitializeTransaction = ( let generalTables = [ InternalTable.Chains.table, InternalTable.PersistedState.table, - InternalTable.EndOfBlockRangeScannedData.table, + InternalTable.Checkpoints.table, InternalTable.RawEvents.table, ] @@ -325,7 +325,7 @@ let chunkArray = (arr: array<'a>, ~chunkSize) => { let removeInvalidUtf8InPlace = entities => entities->Js.Array2.forEach(item => { let dict = item->(Utils.magic: 'a => dict) - dict->Utils.Dict.forEachWithKey((key, value) => { + dict->Utils.Dict.forEachWithKey((value, key) => { if value->Js.typeof === "string" { let value = value->(Utils.magic: unknown => string) // We mutate here, since we don't care @@ -707,7 +707,9 @@ let make = ( { cleanRun: true, cache, + reorgCheckpoints: [], chains: chainConfigs->Js.Array2.map(InternalTable.Chains.initialFromConfig), + checkpointId: InternalTable.Checkpoints.initialCheckpointId, } } @@ -895,19 +897,27 @@ let make = ( } let resumeInitialState = async (): Persistence.initialState => { - let (cache, chains) = await Promise.all2(( + let (cache, chains, checkpointIdResult, reorgCheckpoints) = await Promise.all4(( restoreEffectCache(~withUpload=false), sql ->Postgres.unsafe( makeLoadAllQuery(~pgSchema, ~tableName=InternalTable.Chains.table.tableName), ) ->(Utils.magic: promise> => promise>), + sql + ->Postgres.unsafe(InternalTable.Checkpoints.makeCommitedCheckpointIdQuery(~pgSchema)) + ->(Utils.magic: promise> => promise>), + sql + ->Postgres.unsafe(InternalTable.Checkpoints.makeGetReorgCheckpointsQuery(~pgSchema)) + ->(Utils.magic: promise> => promise>), )) { cleanRun: false, + reorgCheckpoints, cache, chains, + checkpointId: (checkpointIdResult->Belt.Array.getUnsafe(0))["id"], } } diff --git a/codegenerator/cli/npm/envio/src/ReorgDetection.res b/codegenerator/cli/npm/envio/src/ReorgDetection.res index 58dec963d..9301c862e 100644 --- a/codegenerator/cli/npm/envio/src/ReorgDetection.res +++ b/codegenerator/cli/npm/envio/src/ReorgDetection.res @@ -40,188 +40,157 @@ type reorgResult = NoReorg | ReorgDetected(reorgDetected) type validBlockError = NotFound | AlreadyReorgedHashes type validBlockResult = result -module LastBlockScannedHashes: { - type t - /**Instantiat t with existing data*/ - let makeWithData: ( - array, - ~confirmedBlockThreshold: int, - ~detectedReorgBlock: blockData=?, - ) => t - - /**Instantiat empty t with no block data*/ - let empty: (~confirmedBlockThreshold: int) => t - - /** Registers a new reorg guard, prunes unneeded data, and returns the updated state. - * Resets internal state if shouldRollbackOnReorg is false (detect-only mode) - */ - let registerReorgGuard: ( - t, - ~reorgGuard: reorgGuard, - ~currentBlockHeight: int, - ~shouldRollbackOnReorg: bool, - ) => (t, reorgResult) - - /** - Returns the latest block data which matches block number and hashes in the provided array - If it doesn't exist in the reorg threshold it returns None or the latest scanned block outside of the reorg threshold - */ - let getLatestValidScannedBlock: ( - t, - ~blockNumbersAndHashes: array, - ~currentBlockHeight: int, - ~skipReorgDuplicationCheck: bool=?, - ) => validBlockResult - - let getThresholdBlockNumbers: (t, ~currentBlockHeight: int) => array - - let rollbackToValidBlockNumber: (t, ~blockNumber: int) => t -} = { - type t = { - // Number of blocks behind head, we want to keep track - // as a threshold for reorgs. If for eg. this is 200, - // it means we are accounting for reorgs up to 200 blocks - // behind the head - confirmedBlockThreshold: int, - // A hash map of recent blockdata by block number to make comparison checks - // for reorgs. - dataByBlockNumber: dict, - // The latest block which detected a reorg - // and should never be valid. - // We keep track of this to avoid responses - // with the stale data from other data-source instances. - detectedReorgBlock: option, - } +type t = { + // Whether to rollback on reorg + // Even if it's disabled, we still track reorgs checkpoints in memory + // and log when we detect an unhandled reorg + shouldRollbackOnReorg: bool, + // Number of blocks behind head, we want to keep track + // as a threshold for reorgs. If for eg. this is 200, + // it means we are accounting for reorgs up to 200 blocks + // behind the head + maxReorgDepth: int, + // A hash map of recent blockdata by block number to make comparison checks + // for reorgs. + dataByBlockNumber: dict, + // The latest block which detected a reorg + // and should never be valid. + // We keep track of this to avoid responses + // with the stale data from other data-source instances. + detectedReorgBlock: option, +} - let makeWithData = (blocks, ~confirmedBlockThreshold, ~detectedReorgBlock=?) => { - let dataByBlockNumber = Js.Dict.empty() +let make = (~blocks, ~maxReorgDepth, ~shouldRollbackOnReorg, ~detectedReorgBlock=?) => { + let dataByBlockNumber = Js.Dict.empty() - blocks->Belt.Array.forEach(block => { - dataByBlockNumber->Js.Dict.set(block.blockNumber->Js.Int.toString, block) - }) + blocks->Belt.Array.forEach(block => { + dataByBlockNumber->Js.Dict.set(block.blockNumber->Js.Int.toString, block) + }) - { - confirmedBlockThreshold, - dataByBlockNumber, - detectedReorgBlock, - } - } - //Instantiates empty LastBlockHashes - let empty = (~confirmedBlockThreshold) => { - confirmedBlockThreshold, - dataByBlockNumber: Js.Dict.empty(), - detectedReorgBlock: None, + { + shouldRollbackOnReorg, + maxReorgDepth, + dataByBlockNumber, + detectedReorgBlock, } +} - let getDataByBlockNumberCopyInThreshold = ( - {dataByBlockNumber, confirmedBlockThreshold}: t, - ~currentBlockHeight, - ) => { - // Js engine automatically orders numeric object keys - let ascBlockNumberKeys = dataByBlockNumber->Js.Dict.keys - let thresholdBlockNumber = currentBlockHeight - confirmedBlockThreshold +let getDataByBlockNumberCopyInThreshold = ( + {dataByBlockNumber, maxReorgDepth}: t, + ~currentBlockHeight, +) => { + // Js engine automatically orders numeric object keys + let ascBlockNumberKeys = dataByBlockNumber->Js.Dict.keys + let thresholdBlockNumber = currentBlockHeight - maxReorgDepth + + let copy = Js.Dict.empty() + + for idx in 0 to ascBlockNumberKeys->Array.length - 1 { + let blockNumberKey = ascBlockNumberKeys->Js.Array2.unsafe_get(idx) + let scannedBlock = dataByBlockNumber->Js.Dict.unsafeGet(blockNumberKey) + let isInReorgThreshold = scannedBlock.blockNumber >= thresholdBlockNumber + if isInReorgThreshold { + copy->Js.Dict.set(blockNumberKey, scannedBlock) + } + } - let copy = Js.Dict.empty() + copy +} - for idx in 0 to ascBlockNumberKeys->Array.length - 1 { - let blockNumberKey = ascBlockNumberKeys->Js.Array2.unsafe_get(idx) - let scannedBlock = dataByBlockNumber->Js.Dict.unsafeGet(blockNumberKey) - let isInReorgThreshold = scannedBlock.blockNumber >= thresholdBlockNumber - if isInReorgThreshold { - copy->Js.Dict.set(blockNumberKey, scannedBlock) +/** Registers a new reorg guard, prunes unneeded data, and returns the updated state. + * Resets internal state if shouldRollbackOnReorg is false (detect-only mode) + */ +let registerReorgGuard = ( + {maxReorgDepth, shouldRollbackOnReorg} as self: t, + ~reorgGuard: reorgGuard, + ~currentBlockHeight, +) => { + let dataByBlockNumberCopyInThreshold = + self->getDataByBlockNumberCopyInThreshold(~currentBlockHeight) + + let {rangeLastBlock, prevRangeLastBlock} = reorgGuard + + let maybeReorgDetected = switch dataByBlockNumberCopyInThreshold->Utils.Dict.dangerouslyGetNonOption( + rangeLastBlock.blockNumber->Int.toString, + ) { + | Some(scannedBlock) if scannedBlock.blockHash !== rangeLastBlock.blockHash => + Some({ + receivedBlock: rangeLastBlock, + scannedBlock, + }) + | _ => + switch prevRangeLastBlock { + //If parentHash is None, then it's the genesis block (no reorg) + //Need to check that parentHash matches because of the dynamic contracts + | None => None + | Some(prevRangeLastBlock) => + switch dataByBlockNumberCopyInThreshold->Utils.Dict.dangerouslyGetNonOption( + prevRangeLastBlock.blockNumber->Int.toString, + ) { + | Some(scannedBlock) if scannedBlock.blockHash !== prevRangeLastBlock.blockHash => + Some({ + receivedBlock: prevRangeLastBlock, + scannedBlock, + }) + | _ => None } } - - copy } - let registerReorgGuard = ( - {confirmedBlockThreshold} as self: t, - ~reorgGuard: reorgGuard, - ~currentBlockHeight, - ~shouldRollbackOnReorg, - ) => { - let dataByBlockNumberCopyInThreshold = - self->getDataByBlockNumberCopyInThreshold(~currentBlockHeight) - - let {rangeLastBlock, prevRangeLastBlock} = reorgGuard - - let maybeReorgDetected = switch dataByBlockNumberCopyInThreshold->Utils.Dict.dangerouslyGetNonOption( - rangeLastBlock.blockNumber->Int.toString, - ) { - | Some(scannedBlock) if scannedBlock.blockHash !== rangeLastBlock.blockHash => - Some({ - receivedBlock: rangeLastBlock, - scannedBlock, - }) - | _ => + switch maybeReorgDetected { + | Some(reorgDetected) => ( + shouldRollbackOnReorg + ? { + ...self, + detectedReorgBlock: Some(reorgDetected.scannedBlock), + } + : make(~blocks=[], ~maxReorgDepth, ~shouldRollbackOnReorg), + ReorgDetected(reorgDetected), + ) + | None => { + dataByBlockNumberCopyInThreshold->Js.Dict.set( + rangeLastBlock.blockNumber->Int.toString, + rangeLastBlock, + ) switch prevRangeLastBlock { - //If parentHash is None, then it's the genesis block (no reorg) - //Need to check that parentHash matches because of the dynamic contracts - | None => None + | None => () | Some(prevRangeLastBlock) => - switch dataByBlockNumberCopyInThreshold->Utils.Dict.dangerouslyGetNonOption( + dataByBlockNumberCopyInThreshold->Js.Dict.set( prevRangeLastBlock.blockNumber->Int.toString, - ) { - | Some(scannedBlock) if scannedBlock.blockHash !== prevRangeLastBlock.blockHash => - Some({ - receivedBlock: prevRangeLastBlock, - scannedBlock, - }) - | _ => None - } + prevRangeLastBlock, + ) } - } - switch maybeReorgDetected { - | Some(reorgDetected) => ( - shouldRollbackOnReorg - ? { - ...self, - detectedReorgBlock: Some(reorgDetected.scannedBlock), - } - : empty(~confirmedBlockThreshold), - ReorgDetected(reorgDetected), + ( + { + maxReorgDepth, + dataByBlockNumber: dataByBlockNumberCopyInThreshold, + detectedReorgBlock: None, + shouldRollbackOnReorg, + }, + NoReorg, ) - | None => { - dataByBlockNumberCopyInThreshold->Js.Dict.set( - rangeLastBlock.blockNumber->Int.toString, - rangeLastBlock, - ) - switch prevRangeLastBlock { - | None => () - | Some(prevRangeLastBlock) => - dataByBlockNumberCopyInThreshold->Js.Dict.set( - prevRangeLastBlock.blockNumber->Int.toString, - prevRangeLastBlock, - ) - } - - ( - { - confirmedBlockThreshold, - dataByBlockNumber: dataByBlockNumberCopyInThreshold, - detectedReorgBlock: None, - }, - NoReorg, - ) - } } } +} - let getLatestValidScannedBlock = ( - self: t, - ~blockNumbersAndHashes: array, - ~currentBlockHeight, - ~skipReorgDuplicationCheck=false, - ) => { - let verifiedDataByBlockNumber = Js.Dict.empty() - for idx in 0 to blockNumbersAndHashes->Array.length - 1 { - let blockData = blockNumbersAndHashes->Array.getUnsafe(idx) - verifiedDataByBlockNumber->Js.Dict.set(blockData.blockNumber->Int.toString, blockData) - } +/** +Returns the latest block data which matches block number and hashes in the provided array +If it doesn't exist in the reorg threshold it returns None or the latest scanned block outside of the reorg threshold +*/ +let getLatestValidScannedBlock = ( + self: t, + ~blockNumbersAndHashes: array, + ~currentBlockHeight, + ~skipReorgDuplicationCheck=false, +) => { + let verifiedDataByBlockNumber = Js.Dict.empty() + for idx in 0 to blockNumbersAndHashes->Array.length - 1 { + let blockData = blockNumbersAndHashes->Array.getUnsafe(idx) + verifiedDataByBlockNumber->Js.Dict.set(blockData.blockNumber->Int.toString, blockData) + } - /* + /* Let's say we indexed block X with hash A. The next query we got the block X with hash B. We assume that the hash A is reorged since we received it earlier than B. @@ -234,98 +203,105 @@ module LastBlockScannedHashes: { we can skip the reorg duplication check if we're sure that the block hashes query is not coming from a different instance. (let's say we tried several times) */ - let isAlreadyReorgedResponse = skipReorgDuplicationCheck - ? false - : switch self.detectedReorgBlock { - | Some(detectedReorgBlock) => - switch verifiedDataByBlockNumber->Utils.Dict.dangerouslyGetNonOption( - detectedReorgBlock.blockNumber->Int.toString, - ) { - | Some(verifiedBlockData) => verifiedBlockData.blockHash === detectedReorgBlock.blockHash - | None => false - } + let isAlreadyReorgedResponse = skipReorgDuplicationCheck + ? false + : switch self.detectedReorgBlock { + | Some(detectedReorgBlock) => + switch verifiedDataByBlockNumber->Utils.Dict.dangerouslyGetNonOption( + detectedReorgBlock.blockNumber->Int.toString, + ) { + | Some(verifiedBlockData) => verifiedBlockData.blockHash === detectedReorgBlock.blockHash | None => false } - - if isAlreadyReorgedResponse { - Error(AlreadyReorgedHashes) - } else { - let dataByBlockNumber = self->getDataByBlockNumberCopyInThreshold(~currentBlockHeight) - // Js engine automatically orders numeric object keys - let ascBlockNumberKeys = dataByBlockNumber->Js.Dict.keys - - let getPrevScannedBlock = idx => - switch ascBlockNumberKeys - ->Belt.Array.get(idx - 1) - ->Option.flatMap(key => { - // We should already validate that the block number is verified at the point - verifiedDataByBlockNumber->Utils.Dict.dangerouslyGetNonOption(key) - }) { - | Some(data) => Ok(data) - | None => Error(NotFound) - } - - let rec loop = idx => { - switch ascBlockNumberKeys->Belt.Array.get(idx) { - | Some(blockNumberKey) => - let scannedBlock = dataByBlockNumber->Js.Dict.unsafeGet(blockNumberKey) - switch verifiedDataByBlockNumber->Utils.Dict.dangerouslyGetNonOption(blockNumberKey) { - | None => - Js.Exn.raiseError( - `Unexpected case. Couldn't find verified hash for block number ${blockNumberKey}`, - ) - | Some(verifiedBlockData) if verifiedBlockData.blockHash === scannedBlock.blockHash => - loop(idx + 1) - | Some(_) => getPrevScannedBlock(idx) - } - | None => getPrevScannedBlock(idx) - } + | None => false } - loop(0) - } - } - /** - Return a BlockNumbersAndHashes.t rolled back to where blockData is less - than the provided blockNumber - */ - let rollbackToValidBlockNumber = ( - {dataByBlockNumber, confirmedBlockThreshold}: t, - ~blockNumber: int, - ) => { + if isAlreadyReorgedResponse { + Error(AlreadyReorgedHashes) + } else { + let dataByBlockNumber = self->getDataByBlockNumberCopyInThreshold(~currentBlockHeight) // Js engine automatically orders numeric object keys let ascBlockNumberKeys = dataByBlockNumber->Js.Dict.keys - let newDataByBlockNumber = Js.Dict.empty() + let getPrevScannedBlock = idx => + switch ascBlockNumberKeys + ->Belt.Array.get(idx - 1) + ->Option.flatMap(key => { + // We should already validate that the block number is verified at the point + verifiedDataByBlockNumber->Utils.Dict.dangerouslyGetNonOption(key) + }) { + | Some(data) => Ok(data) + | None => Error(NotFound) + } let rec loop = idx => { switch ascBlockNumberKeys->Belt.Array.get(idx) { - | Some(blockNumberKey) => { - let scannedBlock = dataByBlockNumber->Js.Dict.unsafeGet(blockNumberKey) - let shouldKeep = scannedBlock.blockNumber <= blockNumber - if shouldKeep { - newDataByBlockNumber->Js.Dict.set(blockNumberKey, scannedBlock) - loop(idx + 1) - } else { - () - } + | Some(blockNumberKey) => + let scannedBlock = dataByBlockNumber->Js.Dict.unsafeGet(blockNumberKey) + switch verifiedDataByBlockNumber->Utils.Dict.dangerouslyGetNonOption(blockNumberKey) { + | None => + Js.Exn.raiseError( + `Unexpected case. Couldn't find verified hash for block number ${blockNumberKey}`, + ) + | Some(verifiedBlockData) if verifiedBlockData.blockHash === scannedBlock.blockHash => + loop(idx + 1) + | Some(_) => getPrevScannedBlock(idx) } - | None => () + | None => getPrevScannedBlock(idx) } } loop(0) + } +} - { - confirmedBlockThreshold, - dataByBlockNumber: newDataByBlockNumber, - detectedReorgBlock: None, +/** + Return a BlockNumbersAndHashes.t rolled back to where blockData is less + than the provided blockNumber + */ +let rollbackToValidBlockNumber = ( + {dataByBlockNumber, maxReorgDepth, shouldRollbackOnReorg}: t, + ~blockNumber: int, +) => { + // Js engine automatically orders numeric object keys + let ascBlockNumberKeys = dataByBlockNumber->Js.Dict.keys + + let newDataByBlockNumber = Js.Dict.empty() + + let rec loop = idx => { + switch ascBlockNumberKeys->Belt.Array.get(idx) { + | Some(blockNumberKey) => { + let scannedBlock = dataByBlockNumber->Js.Dict.unsafeGet(blockNumberKey) + let shouldKeep = scannedBlock.blockNumber <= blockNumber + if shouldKeep { + newDataByBlockNumber->Js.Dict.set(blockNumberKey, scannedBlock) + loop(idx + 1) + } else { + () + } + } + | None => () } } + loop(0) + + { + maxReorgDepth, + dataByBlockNumber: newDataByBlockNumber, + detectedReorgBlock: None, + shouldRollbackOnReorg, + } +} - let getThresholdBlockNumbers = (self: t, ~currentBlockHeight) => { - let dataByBlockNumberCopyInThreshold = - self->getDataByBlockNumberCopyInThreshold(~currentBlockHeight) +let getThresholdBlockNumbers = (self: t, ~currentBlockHeight) => { + let dataByBlockNumberCopyInThreshold = + self->getDataByBlockNumberCopyInThreshold(~currentBlockHeight) + + dataByBlockNumberCopyInThreshold->Js.Dict.values->Js.Array2.map(v => v.blockNumber) +} - dataByBlockNumberCopyInThreshold->Js.Dict.values->Js.Array2.map(v => v.blockNumber) +let getHashByBlockNumber = (reorgDetection: t, ~blockNumber) => { + switch reorgDetection.dataByBlockNumber->Utils.Dict.dangerouslyGetByIntNonOption(blockNumber) { + | Some(v) => Js.Null.Value(v.blockHash) + | None => Js.Null.Null } } diff --git a/codegenerator/cli/npm/envio/src/Utils.res b/codegenerator/cli/npm/envio/src/Utils.res index 97372e341..e4c7144e5 100644 --- a/codegenerator/cli/npm/envio/src/Utils.res +++ b/codegenerator/cli/npm/envio/src/Utils.res @@ -97,30 +97,66 @@ module Dict = { @val external mergeInPlace: (dict<'a>, dict<'a>) => dict<'a> = "Object.assign" - let map = (dict, fn) => { - let newDict = Js.Dict.empty() - let keys = dict->Js.Dict.keys - for idx in 0 to keys->Js.Array2.length - 1 { - let key = keys->Js.Array2.unsafe_get(idx) - newDict->Js.Dict.set(key, fn(dict->Js.Dict.unsafeGet(key))) + // Use %raw to support for..in which is a ~10% faster than .forEach + let mapValues: (dict<'a>, 'a => 'b) => dict<'b> = %raw(`(dict, f) => { + var target = {}, i; + for (i in dict) { + target[i] = f(dict[i]); } - newDict - } + return target; + }`) - let forEach = (dict, fn) => { - let keys = dict->Js.Dict.keys - for idx in 0 to keys->Js.Array2.length - 1 { - fn(dict->Js.Dict.unsafeGet(keys->Js.Array2.unsafe_get(idx))) + // Use %raw to support for..in which is a ~10% faster than .forEach + let filterMapValues: (dict<'a>, 'a => option<'b>) => dict<'b> = %raw(`(dict, f) => { + var target = {}, i, v; + for (i in dict) { + v = f(dict[i]); + if (v !== undefined) { + target[i] = v; + } } - } + return target; + }`) - let forEachWithKey = (dict, fn) => { - let keys = dict->Js.Dict.keys - for idx in 0 to keys->Js.Array2.length - 1 { - let key = keys->Js.Array2.unsafe_get(idx) - fn(key, dict->Js.Dict.unsafeGet(key)) + // Use %raw to support for..in which is a ~10% faster than .forEach + let mapValuesToArray: (dict<'a>, 'a => 'b) => array<'b> = %raw(`(dict, f) => { + var target = [], i; + for (i in dict) { + target.push(f(dict[i])); } - } + return target; + }`) + + // Use %raw to support for..in which is a ~10% faster than .forEach + let forEach: (dict<'a>, 'a => unit) => unit = %raw(`(dict, f) => { + for (var i in dict) { + f(dict[i]); + } + }`) + + // Use %raw to support for..in which is a ~10% faster than .forEach + let forEachWithKey: (dict<'a>, ('a, string) => unit) => unit = %raw(`(dict, f) => { + for (var i in dict) { + f(dict[i], i); + } + }`) + + // Use %raw to support for..in which is a ~10% faster than Object.keys + let size: dict<'a> => int = %raw(`(dict) => { + var size = 0, i; + for (i in dict) { + size++; + } + return size; + }`) + + // Use %raw to support for..in which is a 2x faster than Object.keys + let isEmpty: dict<'a> => bool = %raw(`(dict) => { + for (var _ in dict) { + return false + } + return true + }`) let deleteInPlace: (dict<'a>, string) => unit = %raw(`(dict, key) => { delete dict[key]; @@ -135,8 +171,6 @@ module Dict = { let shallowCopy: dict<'a> => dict<'a> = %raw(`(dict) => ({...dict})`) - let size = dict => dict->Js.Dict.keys->Js.Array2.length - @set_index external setByInt: (dict<'a>, int, 'a) => unit = "" @@ -155,6 +189,15 @@ module Math = { } } +// This is a microoptimization to avoid int32 safeguards +module UnsafeIntOperators = { + external \"*": (int, int) => int = "%mulfloat" + + external \"+": (int, int) => int = "%addfloat" + + external \"-": (int, int) => int = "%subfloat" +} + module Array = { @send external forEachAsync: (array<'a>, 'a => promise) => unit = "forEach" diff --git a/codegenerator/cli/npm/envio/src/db/InternalTable.res b/codegenerator/cli/npm/envio/src/db/InternalTable.res index 1dc236fdf..3fc282ebe 100644 --- a/codegenerator/cli/npm/envio/src/db/InternalTable.res +++ b/codegenerator/cli/npm/envio/src/db/InternalTable.res @@ -16,6 +16,7 @@ module Chains = { | #id | #start_block | #end_block + | #max_reorg_depth | #source_block | #first_event_block | #buffer_block @@ -28,6 +29,7 @@ module Chains = { #id, #start_block, #end_block, + #max_reorg_depth, #source_block, #first_event_block, #buffer_block, @@ -52,6 +54,7 @@ module Chains = { @as("id") id: int, @as("start_block") startBlock: int, @as("end_block") endBlock: Js.null, + @as("max_reorg_depth") maxReorgDepth: int, @as("progress_block") progressBlockNumber: int, @as("events_processed") numEventsProcessed: int, ...metaFields, @@ -64,6 +67,7 @@ module Chains = { // Values populated from config mkField((#start_block: field :> string), Integer, ~fieldSchema=S.int), mkField((#end_block: field :> string), Integer, ~fieldSchema=S.null(S.int), ~isNullable), + mkField((#max_reorg_depth: field :> string), Integer, ~fieldSchema=S.int), // Block number of the latest block that was fetched from the source mkField((#buffer_block: field :> string), Integer, ~fieldSchema=S.int), // Block number of the currently active source @@ -98,6 +102,7 @@ module Chains = { id: chainConfig.id, startBlock: chainConfig.startBlock, endBlock: chainConfig.endBlock->Js.Null.fromOption, + maxReorgDepth: chainConfig.maxReorgDepth, blockHeight: 0, firstEventBlockNumber: Js.Null.empty, latestFetchedBlockNumber: -1, @@ -182,7 +187,7 @@ WHERE "id" = $1;` let promises = [] - chainsData->Utils.Dict.forEachWithKey((chainId, data) => { + chainsData->Utils.Dict.forEachWithKey((data, chainId) => { let params = [] // Push id first (for WHERE clause) @@ -201,7 +206,13 @@ WHERE "id" = $1;` Promise.all(promises) } - let setProgressedChains = (sql, ~pgSchema, ~progressedChains: array) => { + type progressedChain = { + chainId: int, + progressBlockNumber: int, + totalEventsProcessed: int, + } + + let setProgressedChains = (sql, ~pgSchema, ~progressedChains: array) => { let query = makeProgressFieldsUpdateQuery(~pgSchema) let promises = [] @@ -254,21 +265,147 @@ module PersistedState = { ) } -module EndOfBlockRangeScannedData = { +module Checkpoints = { + type field = [ + | #id + | #chain_id + | #block_number + | #block_hash + | #events_processed + ] + type t = { - chain_id: int, - block_number: int, - block_hash: string, + id: int, + @as("chain_id") + chainId: int, + @as("block_number") + blockNumber: int, + @as("block_hash") + blockHash: Js.null, + @as("events_processed") + eventsProcessed: int, } + let initialCheckpointId = 0 + let table = mkTable( - "end_of_block_range_scanned_data", + "envio_checkpoints", ~fields=[ - mkField("chain_id", Integer, ~fieldSchema=S.int, ~isPrimaryKey), - mkField("block_number", Integer, ~fieldSchema=S.int, ~isPrimaryKey), - mkField("block_hash", Text, ~fieldSchema=S.string), + mkField((#id: field :> string), Integer, ~fieldSchema=S.int, ~isPrimaryKey), + mkField((#chain_id: field :> string), Integer, ~fieldSchema=S.int), + mkField((#block_number: field :> string), Integer, ~fieldSchema=S.int), + mkField((#block_hash: field :> string), Text, ~fieldSchema=S.null(S.string), ~isNullable), + mkField((#events_processed: field :> string), Integer, ~fieldSchema=S.int), ], ) + + let makeGetReorgCheckpointsQuery = (~pgSchema): string => { + // Use CTE to pre-filter chains and compute safe_block once per chain + // This is faster because: + // 1. Chains table is small, so filtering it first is cheap + // 2. safe_block is computed once per chain, not per checkpoint + // 3. Query planner can materialize the small CTE result before joining + `WITH reorg_chains AS ( + SELECT + "${(#id: Chains.field :> string)}" as id, + "${(#source_block: Chains.field :> string)}" - "${(#max_reorg_depth: Chains.field :> string)}" AS safe_block + FROM "${pgSchema}"."${Chains.table.tableName}" + WHERE "${(#max_reorg_depth: Chains.field :> string)}" > 0 + AND "${(#progress_block: Chains.field :> string)}" > "${(#source_block: Chains.field :> string)}" - "${(#max_reorg_depth: Chains.field :> string)}" +) +SELECT + cp."${(#id: field :> string)}", + cp."${(#chain_id: field :> string)}", + cp."${(#block_number: field :> string)}", + cp."${(#block_hash: field :> string)}" +FROM "${pgSchema}"."${table.tableName}" cp +INNER JOIN reorg_chains rc + ON cp."${(#chain_id: field :> string)}" = rc.id +WHERE cp."${(#block_hash: field :> string)}" IS NOT NULL + AND cp."${(#block_number: field :> string)}" > rc.safe_block;` + } + + let makeCommitedCheckpointIdQuery = (~pgSchema) => { + `SELECT COALESCE(MAX(${(#id: field :> string)}), ${initialCheckpointId->Belt.Int.toString}) AS id FROM "${pgSchema}"."${table.tableName}";` + } + + let makeInsertCheckpointQuery = (~pgSchema) => { + `INSERT INTO "${pgSchema}"."${table.tableName}" ("${(#id: field :> string)}", "${(#chain_id: field :> string)}", "${(#block_number: field :> string)}", "${(#block_hash: field :> string)}", "${(#events_processed: field :> string)}") +SELECT * FROM unnest($1::${(Integer :> string)}[],$2::${(Integer :> string)}[],$3::${(Integer :> string)}[],$4::${(Text :> string)}[],$5::${(Integer :> string)}[]);` + } + + let insert = ( + sql, + ~pgSchema, + ~checkpointIds, + ~checkpointChainIds, + ~checkpointBlockNumbers, + ~checkpointBlockHashes, + ~checkpointEventsProcessed, + ) => { + let query = makeInsertCheckpointQuery(~pgSchema) + + sql + ->Postgres.preparedUnsafe( + query, + ( + checkpointIds, + checkpointChainIds, + checkpointBlockNumbers, + checkpointBlockHashes, + checkpointEventsProcessed, + )->( + Utils.magic: ( + (array, array, array, array>, array) + ) => unknown + ), + ) + ->Promise.ignoreValue + } + + // This is how it used to work before checkpoints + // To make it correct, we need to delete checkpoints of all chains + let deprecated_rollbackReorgedChainCheckpoints = ( + sql, + ~pgSchema, + ~chainId, + ~knownBlockNumber, + ) => { + sql + ->Postgres.preparedUnsafe( + `DELETE FROM "${pgSchema}"."${table.tableName}" WHERE "${(#chain_id: field :> string)}" = $1 AND "${(#block_number: field :> string)}" > $2;`, + (chainId, knownBlockNumber)->(Utils.magic: ((int, int)) => unknown), + ) + ->Promise.ignoreValue + } + + // This is how it used to work before checkpoints + // To make it correct, we need to first find + // a safe checkpoint - which is the min checkpoint outside all chains + // And then delete everything below it - the same for history items + let deprecated_pruneStaleCheckpoints = (sql, ~pgSchema) => { + // Delete checkpoints that are outside the reorg window: + // 1. All checkpoints for chains with max_reorg_depth = 0 (no reorg protection) + // 2. Checkpoints at or below safe_block for chains with max_reorg_depth > 0 + sql + ->Postgres.unsafe( + `WITH chain_safe_blocks AS ( + SELECT + "${(#id: Chains.field :> string)}" as id, + "${(#max_reorg_depth: Chains.field :> string)}" as max_reorg_depth, + "${(#source_block: Chains.field :> string)}" - "${(#max_reorg_depth: Chains.field :> string)}" AS safe_block + FROM "${pgSchema}"."${Chains.table.tableName}" + ) + DELETE FROM "${pgSchema}"."${table.tableName}" cp + USING chain_safe_blocks csb + WHERE cp."${(#chain_id: field :> string)}" = csb.id + AND ( + csb.max_reorg_depth = 0 + OR cp."${(#block_number: field :> string)}" <= csb.safe_block + );`, + ) + ->Promise.ignoreValue + } } module RawEvents = { diff --git a/codegenerator/cli/npm/envio/src/db/Table.res b/codegenerator/cli/npm/envio/src/db/Table.res index 714b0495b..95f483c20 100644 --- a/codegenerator/cli/npm/envio/src/db/Table.res +++ b/codegenerator/cli/npm/envio/src/db/Table.res @@ -5,6 +5,7 @@ type derived @unboxed type fieldType = | @as("INTEGER") Integer + | @as("BIGINT") BigInt | @as("BOOLEAN") Boolean | @as("NUMERIC") Numeric | @as("DOUBLE PRECISION") DoublePrecision diff --git a/codegenerator/cli/templates/dynamic/codegen/src/RegisterHandlers.res.hbs b/codegenerator/cli/templates/dynamic/codegen/src/RegisterHandlers.res.hbs index fc60b33aa..0b3346af2 100644 --- a/codegenerator/cli/templates/dynamic/codegen/src/RegisterHandlers.res.hbs +++ b/codegenerator/cli/templates/dynamic/codegen/src/RegisterHandlers.res.hbs @@ -56,7 +56,7 @@ let registerContractHandlers = ( ] let chain = ChainMap.Chain.makeUnsafe(~chainId={{chain_config.network_config.id}}) { - InternalConfig.confirmedBlockThreshold: {{chain_config.network_config.confirmed_block_threshold}}, + InternalConfig.maxReorgDepth: {{chain_config.network_config.confirmed_block_threshold}}, startBlock: {{chain_config.network_config.start_block}}, {{#if chain_config.network_config.end_block}} endBlock: {{chain_config.network_config.end_block}}, diff --git a/codegenerator/cli/templates/dynamic/codegen/src/TestHelpers_MockDb.res.hbs b/codegenerator/cli/templates/dynamic/codegen/src/TestHelpers_MockDb.res.hbs index c6a1ec195..05dc04c8b 100644 --- a/codegenerator/cli/templates/dynamic/codegen/src/TestHelpers_MockDb.res.hbs +++ b/codegenerator/cli/templates/dynamic/codegen/src/TestHelpers_MockDb.res.hbs @@ -315,121 +315,188 @@ let rec makeWithInMemoryStore: InMemoryStore.t => t = (inMemoryStore: InMemorySt and makeProcessEvents = (mockDb: t, ~chainId=?) => async ( events: array>, ) => { - let itemsWithContractRegister = [] - - let config = { - ...config, - registrations: Some(EventRegister.finishRegistration()), - } - - let processingChainId = ref(chainId) - let items = events->Array.map(event => { - let event = event->Internal.fromGenericEvent - let eventConfig = switch mockEventRegisters->Utils.WeakMap.get(event) { - | Some(register) => register() - | None => - Js.Exn.raiseError( - "Events must be created using the mock API (e.g. createMockEvent) to be processed by mockDb.processEvents", - ) - } - let chainId = switch chainId { - | Some(chainId) => chainId - | None => event.chainId + if events->Utils.Array.isEmpty { + mockDb + } else { + let itemsWithContractRegister = [] + + let config = { + ...config, + registrations: Some(EventRegister.finishRegistration()), } - switch processingChainId.contents { - | Some(chainId) => - if chainId != event.chainId { + let processingChainId = ref(chainId) + let latestFetchedBlockNumber = ref(0) + let newItems = events->Array.map(event => { + let event = event->Internal.fromGenericEvent + let eventConfig = switch mockEventRegisters->Utils.WeakMap.get(event) { + | Some(register) => register() + | None => Js.Exn.raiseError( - `Processing events on multiple chains is not supported yet. Got chainId ${event.chainId->Belt.Int.toString} but expected ${chainId->Belt.Int.toString}`, + "Events must be created using the mock API (e.g. createMockEvent) to be processed by mockDb.processEvents", ) } - | None => processingChainId.contents = Some(chainId) - } + let chainId = switch chainId { + | Some(chainId) => chainId + | None => event.chainId + } - let chain = config->Config.getChain(~chainId) - let item = Internal.Event({ - eventConfig, - event, - chain, - logIndex: event.logIndex, - timestamp: event.block->Types.Block.getTimestamp, - blockNumber: event.block->Types.Block.getNumber, + switch processingChainId.contents { + | Some(chainId) => + if chainId != event.chainId { + Js.Exn.raiseError( + `Processing events on multiple chains is not supported yet. Got chainId ${event.chainId->Belt.Int.toString} but expected ${chainId->Belt.Int.toString}`, + ) + } + | None => processingChainId.contents = Some(chainId) + } + + let chain = config->Config.getChain(~chainId) + let item = Internal.Event({ + eventConfig, + event, + chain, + logIndex: event.logIndex, + timestamp: event.block->Types.Block.getTimestamp, + blockNumber: event.block->Types.Block.getNumber, + }) + latestFetchedBlockNumber.contents = Pervasives.max( + latestFetchedBlockNumber.contents, + event.block->Types.Block.getNumber, + ) + if eventConfig.contractRegister->Option.isSome { + itemsWithContractRegister->Js.Array2.push(item)->ignore + } + item }) - if eventConfig.contractRegister->Option.isSome { - itemsWithContractRegister->Js.Array2.push(item)->ignore + + let processingChainId = switch processingChainId.contents { + | Some(chainId) => chainId + | None => + // Should never happen due to early return at line 318 + Js.Exn.raiseError("Internal error: processingChainId not set despite non-empty events array") } - item - }) - let processingChainId = switch processingChainId.contents { - | Some(chainId) => chainId - | None => - Js.Exn.raiseError("No events provided to processEvents. Please provide at least one event.") - } + //Deep copy the data in mockDb, mutate the clone and return the clone + //So no side effects occur here and state can be compared between process + //steps + let mockDbClone = mockDb->cloneMockDb + + //Construct a new instance of an in memory store to run for the given event + let inMemoryStore = InMemoryStore.make() + let loadManager = LoadManager.make() + let persistence = { + ...config.persistence, + storage: makeMockStorage(mockDb), + storageStatus: Ready({ + cleanRun: false, + cache: Js.Dict.empty(), + chains: [], + reorgCheckpoints: [], + checkpointId: 0, + }), + } + let config = { + ...config, + persistence, + } - //Deep copy the data in mockDb, mutate the clone and return the clone - //So no side effects occur here and state can be compared between process - //steps - let mockDbClone = mockDb->cloneMockDb - - //Construct a new instance of an in memory store to run for the given event - let inMemoryStore = InMemoryStore.make() - let loadManager = LoadManager.make() - let persistence = { - ...config.persistence, - storage: makeMockStorage(mockDb), - storageStatus: Ready({cleanRun: false, cache: Js.Dict.empty(), chains: []}), - } - let config = { - ...config, - persistence, - } + //No need to check contract is registered or return anything. + //The only purpose is to test the registerContract function and to + //add the entity to the in memory store for asserting registrations + if itemsWithContractRegister->Utils.Array.notEmpty { + let dcs = await ChainFetcher.runContractRegistersOrThrow( + ~itemsWithContractRegister, + ~chain=ChainMap.Chain.makeUnsafe(~chainId=processingChainId), + ~config, + ) - //No need to check contract is registered or return anything. - //The only purpose is to test the registerContract function and to - //add the entity to the in memory store for asserting registrations - if itemsWithContractRegister->Utils.Array.notEmpty { - let dcs = await ChainFetcher.runContractRegistersOrThrow( - ~itemsWithContractRegister, - ~chain=ChainMap.Chain.makeUnsafe(~chainId=processingChainId), - ~config, + // TODO: Reuse FetchState logic to clean up duplicate dcs + if dcs->Utils.Array.notEmpty { + inMemoryStore->InMemoryStore.setDcsToStore( + ~chainId=processingChainId, + ~dcs, + ~shouldSaveHistory=false, + ) + } + } + + let batch = Batch.prepareUnorderedBatch( + ~checkpointIdBeforeBatch=0, + ~chainsBeforeBatch=ChainMap.fromArrayUnsafe([ + ( + ChainMap.Chain.makeUnsafe(~chainId=processingChainId), + ( + { + fetchState: FetchState.make( + ~chainId=processingChainId, + ~startBlock=0, + ~endBlock=None, + ~eventConfigs=[ + // We don't care about the actual config here, + // just a reference, so FetchState doesn't complain + // about nothing to fetch + { + "id": "test", + }->Obj.magic, + ], + ~contracts=[], + ~maxAddrInPartition=0, + ~targetBufferSize=0, + ) + ->FetchState.handleQueryResult( + ~latestFetchedBlock={ + blockNumber: latestFetchedBlockNumber.contents, + blockTimestamp: 0, + }, + ~query={ + partitionId: "0", + fromBlock: 0, + selection: {eventConfigs: [], dependsOnAddresses: false}, + addressesByContractName: Js.Dict.empty(), + target: FetchState.Head, + indexingContracts: Js.Dict.empty(), + }, + ~newItems, + ) + ->Result.getExn, + reorgDetection: ReorgDetection.make( + ~blocks=[], + ~maxReorgDepth=0, + ~shouldRollbackOnReorg=false, + ), + progressBlockNumber: 0, + sourceBlockNumber: 0, + totalEventsProcessed: 0, + }: Batch.chainBeforeBatch + ), + ), + ]), + ~batchSizeTarget=newItems->Array.length, ) - // TODO: Reuse FetchState logic to clean up duplicate dcs - if dcs->Utils.Array.notEmpty { - inMemoryStore->InMemoryStore.setDcsToStore( - Js.Dict.fromArray([(processingChainId->Belt.Int.toString, dcs)]), + try { + await batch->EventProcessing.preloadBatchOrThrow(~loadManager, ~persistence, ~inMemoryStore) + await batch->EventProcessing.runBatchHandlersOrThrow( + ~inMemoryStore, + ~loadManager, + ~config, ~shouldSaveHistory=false, + ~shouldBenchmark=false, ) + } catch { + | EventProcessing.ProcessingError({message, exn, item}) => + exn + ->ErrorHandling.make(~msg=message, ~logger=item->Logging.getItemLogger) + ->ErrorHandling.logAndRaise } - } - try { - await items->EventProcessing.preloadBatchOrThrow( - ~loadManager, - ~persistence, - ~inMemoryStore, - ) - await items->EventProcessing.runBatchHandlersOrThrow( - ~inMemoryStore, - ~loadManager, - ~config, - ~shouldSaveHistory=false, - ~shouldBenchmark=false, - ) - } catch { - | EventProcessing.ProcessingError({message, exn, item}) => - exn - ->ErrorHandling.make(~msg=message, ~logger=item->Logging.getItemLogger) - ->ErrorHandling.logAndRaise + //In mem store can still contatin raw events and dynamic contracts for the + //testing framework in cases where either contract register or loaderHandler + //is None + mockDbClone->writeFromMemoryStore(~inMemoryStore) + mockDbClone } - - //In mem store can still contatin raw events and dynamic contracts for the - //testing framework in cases where either contract register or loaderHandler - //is None - mockDbClone->writeFromMemoryStore(~inMemoryStore) - mockDbClone } and makeMockStorage = (mockDb: t): Persistence.storage => { { diff --git a/codegenerator/cli/templates/static/codegen/src/EventProcessing.res b/codegenerator/cli/templates/static/codegen/src/EventProcessing.res index 036186263..a51a04045 100644 --- a/codegenerator/cli/templates/static/codegen/src/EventProcessing.res +++ b/codegenerator/cli/templates/static/codegen/src/EventProcessing.res @@ -190,18 +190,14 @@ let runHandlerOrThrow = async ( } } -let preloadBatchOrThrow = async ( - eventBatch: array, - ~loadManager, - ~persistence, - ~inMemoryStore, -) => { +let preloadBatchOrThrow = async (batch: Batch.t, ~loadManager, ~persistence, ~inMemoryStore) => { // On the first run of loaders, we don't care about the result, // whether it's an error or a return type. // We'll rerun the loader again right before the handler run, // to avoid having a stale data returned from the loader. + let _ = await Promise.all( - eventBatch->Array.keepMap(item => { + batch.items->Array.keepMap(item => { switch item { | Event({eventConfig: {handler}, event}) => switch handler { @@ -255,15 +251,15 @@ let preloadBatchOrThrow = async ( } let runBatchHandlersOrThrow = async ( - eventBatch: array, + {items}: Batch.t, ~inMemoryStore, ~loadManager, ~config, ~shouldSaveHistory, ~shouldBenchmark, ) => { - for i in 0 to eventBatch->Array.length - 1 { - let item = eventBatch->Js.Array2.unsafe_get(i) + for i in 0 to items->Array.length - 1 { + let item = items->Js.Array2.unsafe_get(i) await runHandlerOrThrow( item, ~inMemoryStore, @@ -301,31 +297,28 @@ type logPartitionInfo = { } let processEventBatch = async ( - ~items: array, - ~progressedChains: array, + ~batch: Batch.t, ~inMemoryStore: InMemoryStore.t, ~isInReorgThreshold, ~loadManager, ~config: Config.t, ) => { - let batchSize = items->Array.length - let byChain = Js.Dict.empty() - progressedChains->Js.Array2.forEach(data => { - if data.batchSize > 0 { - byChain->Utils.Dict.setByInt( - data.chainId, - { - "batchSize": data.batchSize, - "toBlockNumber": data.progressBlockNumber, - }, - ) - } - }) + let totalBatchSize = batch.totalBatchSize + let logger = Logging.createChildFrom( ~logger=Logging.getLogger(), ~params={ - "totalBatchSize": batchSize, - "byChain": byChain, + "totalBatchSize": totalBatchSize, + "byChain": batch.progressedChainsById->Utils.Dict.filterMapValues(chainAfterBatch => + if chainAfterBatch.batchSize > 0 { + Some({ + "batchSize": chainAfterBatch.batchSize, + "toBlockNumber": chainAfterBatch.progressBlockNumber, + }) + } else { + None + } + ), }, ) logger->Logging.childTrace("Started processing batch") @@ -333,11 +326,11 @@ let processEventBatch = async ( try { let timeRef = Hrtime.makeTimer() - await items->preloadBatchOrThrow(~loadManager, ~persistence=config.persistence, ~inMemoryStore) + await batch->preloadBatchOrThrow(~loadManager, ~persistence=config.persistence, ~inMemoryStore) let elapsedTimeAfterLoaders = timeRef->Hrtime.timeSince->Hrtime.toMillis->Hrtime.intFromMillis - await items->runBatchHandlersOrThrow( + await batch->runBatchHandlersOrThrow( ~inMemoryStore, ~loadManager, ~config, @@ -350,7 +343,7 @@ let processEventBatch = async ( let rec executeBatch = async (~escapeTables=?) => { switch await Db.sql->IO.executeBatch( - ~progressedChains, + ~batch, ~inMemoryStore, ~isInReorgThreshold, ~config, @@ -383,7 +376,7 @@ let processEventBatch = async ( ) if Env.Benchmark.shouldSaveData { Benchmark.addEventProcessing( - ~batchSize, + ~batchSize=totalBatchSize, ~loadDuration=loaderDuration, ~handlerDuration, ~dbWriteDuration, diff --git a/codegenerator/cli/templates/static/codegen/src/IO.res b/codegenerator/cli/templates/static/codegen/src/IO.res index 521926e88..9c9174421 100644 --- a/codegenerator/cli/templates/static/codegen/src/IO.res +++ b/codegenerator/cli/templates/static/codegen/src/IO.res @@ -50,12 +50,14 @@ let getEntityHistoryItems = (entityUpdates, ~containsRollbackDiffChange) => { let executeBatch = async ( sql, - ~progressedChains: array, + ~batch: Batch.t, ~inMemoryStore: InMemoryStore.t, ~isInReorgThreshold, ~config, ~escapeTables=?, ) => { + let shouldSaveHistory = config->Config.shouldSaveHistory(~isInReorgThreshold) + let specificError = ref(None) let setRawEvents = executeSet( @@ -90,7 +92,7 @@ let executeBatch = async ( } }) - if config->Config.shouldSaveHistory(~isInReorgThreshold) { + if shouldSaveHistory { rows->Js.Array2.forEach(row => { switch row { | Updated({history, containsRollbackDiffChange}) => @@ -232,7 +234,8 @@ let executeBatch = async ( }, ~eventIdentifier, ), - sql->DbFunctions.EndOfBlockRangeScannedData.rollbackEndOfBlockRangeScannedDataForChain( + sql->InternalTable.Checkpoints.deprecated_rollbackReorgedChainCheckpoints( + ~pgSchema=Db.publicSchema, ~chainId=eventIdentifier.chainId, ~knownBlockNumber=eventIdentifier.blockNumber, ), @@ -251,17 +254,35 @@ let executeBatch = async ( | None => () } - await Belt.Array.concatMany([ - [ - sql => - sql->InternalTable.Chains.setProgressedChains( - ~pgSchema=Db.publicSchema, - ~progressedChains, - ), - setRawEvents, - ], - setEntities, - ]) + let setOperations = [ + sql => + sql->InternalTable.Chains.setProgressedChains( + ~pgSchema=Db.publicSchema, + ~progressedChains=batch.progressedChainsById->Utils.Dict.mapValuesToArray(( + chainAfterBatch + ): InternalTable.Chains.progressedChain => { + chainId: chainAfterBatch.fetchState.chainId, + progressBlockNumber: chainAfterBatch.progressBlockNumber, + totalEventsProcessed: chainAfterBatch.totalEventsProcessed, + }), + ), + setRawEvents, + ]->Belt.Array.concat(setEntities) + + if shouldSaveHistory { + setOperations->Array.push(sql => + sql->InternalTable.Checkpoints.insert( + ~pgSchema=Db.publicSchema, + ~checkpointIds=batch.checkpointIds, + ~checkpointChainIds=batch.checkpointChainIds, + ~checkpointBlockNumbers=batch.checkpointBlockNumbers, + ~checkpointBlockHashes=batch.checkpointBlockHashes, + ~checkpointEventsProcessed=batch.checkpointEventsProcessed, + ) + ) + } + + await setOperations ->Belt.Array.map(dbFunc => sql->dbFunc) ->Promise.all }), diff --git a/codegenerator/cli/templates/static/codegen/src/InMemoryStore.res b/codegenerator/cli/templates/static/codegen/src/InMemoryStore.res index c29f63025..ece1c75a5 100644 --- a/codegenerator/cli/templates/static/codegen/src/InMemoryStore.res +++ b/codegenerator/cli/templates/static/codegen/src/InMemoryStore.res @@ -102,46 +102,44 @@ let isRollingBack = (inMemoryStore: t) => inMemoryStore.rollBackEventIdentifier- let setDcsToStore = ( inMemoryStore: t, - dcsToStoreByChainId: dict>, + ~chainId: int, + ~dcs: array, ~shouldSaveHistory, ) => { let inMemTable = inMemoryStore->getInMemTable( ~entityConfig=module(InternalTable.DynamicContractRegistry)->Entities.entityModToInternal, ) - dcsToStoreByChainId->Utils.Dict.forEachWithKey((chainId, dcs) => { - let chainId = chainId->Belt.Int.fromString->Belt.Option.getExn - dcs->Belt.Array.forEach(dc => { - let dcData = switch dc.register { - | Config => Js.Exn.raiseError("Config contract should not be in dcsToStore") - | DC(data) => data - } - let entity: InternalTable.DynamicContractRegistry.t = { - id: InternalTable.DynamicContractRegistry.makeId(~chainId, ~contractAddress=dc.address), - chainId, - contractAddress: dc.address, - contractName: dc.contractName, - registeringEventBlockNumber: dc.startBlock, - registeringEventBlockTimestamp: dcData.registeringEventBlockTimestamp, - registeringEventLogIndex: dcData.registeringEventLogIndex, - registeringEventContractName: dcData.registeringEventContractName, - registeringEventName: dcData.registeringEventName, - registeringEventSrcAddress: dcData.registeringEventSrcAddress, - } + dcs->Belt.Array.forEach(dc => { + let dcData = switch dc.register { + | Config => Js.Exn.raiseError("Config contract should not be in dcsToStore") + | DC(data) => data + } + let entity: InternalTable.DynamicContractRegistry.t = { + id: InternalTable.DynamicContractRegistry.makeId(~chainId, ~contractAddress=dc.address), + chainId, + contractAddress: dc.address, + contractName: dc.contractName, + registeringEventBlockNumber: dc.startBlock, + registeringEventBlockTimestamp: dcData.registeringEventBlockTimestamp, + registeringEventLogIndex: dcData.registeringEventLogIndex, + registeringEventContractName: dcData.registeringEventContractName, + registeringEventName: dcData.registeringEventName, + registeringEventSrcAddress: dcData.registeringEventSrcAddress, + } - let eventIdentifier: Types.eventIdentifier = { - chainId, - blockTimestamp: dcData.registeringEventBlockTimestamp, - blockNumber: dc.startBlock, - logIndex: dcData.registeringEventLogIndex, - } - inMemTable->InMemoryTable.Entity.set( - Set(entity->InternalTable.DynamicContractRegistry.castToInternal)->Types.mkEntityUpdate( - ~eventIdentifier, - ~entityId=entity.id, - ), - ~shouldSaveHistory, - ) - }) + let eventIdentifier: Types.eventIdentifier = { + chainId, + blockTimestamp: dcData.registeringEventBlockTimestamp, + blockNumber: dc.startBlock, + logIndex: dcData.registeringEventLogIndex, + } + inMemTable->InMemoryTable.Entity.set( + Set(entity->InternalTable.DynamicContractRegistry.castToInternal)->Types.mkEntityUpdate( + ~eventIdentifier, + ~entityId=entity.id, + ), + ~shouldSaveHistory, + ) }) } diff --git a/codegenerator/cli/templates/static/codegen/src/db/DbFunctions.res b/codegenerator/cli/templates/static/codegen/src/db/DbFunctions.res index 9773f94fa..5136fb711 100644 --- a/codegenerator/cli/templates/static/codegen/src/db/DbFunctions.res +++ b/codegenerator/cli/templates/static/codegen/src/db/DbFunctions.res @@ -13,43 +13,6 @@ module General = { } } -module EndOfBlockRangeScannedData = { - type endOfBlockRangeScannedData = { - @as("chain_id") chainId: int, - @as("block_number") blockNumber: int, - @as("block_hash") blockHash: string, - } - - @module("./DbFunctionsImplementation.js") - external batchSet: (Postgres.sql, array) => promise = - "batchSetEndOfBlockRangeScannedData" - - let setEndOfBlockRangeScannedData = (sql, endOfBlockRangeScannedData) => - batchSet(sql, [endOfBlockRangeScannedData]) - - @module("./DbFunctionsImplementation.js") - external readEndOfBlockRangeScannedDataForChain: ( - Postgres.sql, - ~chainId: int, - ) => promise> = "readEndOfBlockRangeScannedDataForChain" - - @module("./DbFunctionsImplementation.js") - external deleteStaleEndOfBlockRangeScannedDataForChain: ( - Postgres.sql, - ~chainId: int, - //minimum blockNumber that should be kept in db - ~blockNumberThreshold: int, - ) => promise = "deleteStaleEndOfBlockRangeScannedDataForChain" - - @module("./DbFunctionsImplementation.js") - external rollbackEndOfBlockRangeScannedDataForChain: ( - Postgres.sql, - ~chainId: int, - //The known block number we are rollbacking to - ~knownBlockNumber: int, - ) => promise = "rollbackEndOfBlockRangeScannedDataForChain" -} - module DynamicContractRegistry = { @module("./DbFunctionsImplementation.js") external readAllDynamicContractsRaw: (Postgres.sql, ~chainId: chainId) => promise = diff --git a/codegenerator/cli/templates/static/codegen/src/db/DbFunctionsImplementation.js b/codegenerator/cli/templates/static/codegen/src/db/DbFunctionsImplementation.js index 42878e2b8..b83191a7e 100644 --- a/codegenerator/cli/templates/static/codegen/src/db/DbFunctionsImplementation.js +++ b/codegenerator/cli/templates/static/codegen/src/db/DbFunctionsImplementation.js @@ -1,21 +1,6 @@ const TableModule = require("envio/src/db/Table.res.js"); -const Utils = require("envio/src/Utils.res.js"); const { publicSchema } = require("./Db.res.js"); -// db operations for raw_events: -const MAX_ITEMS_PER_QUERY = 500; - -const chunkBatchQuery = (queryToExecute) => async (sql, entityDataArray) => { - const responses = []; - // Split entityDataArray into chunks of MAX_ITEMS_PER_QUERY - for (let i = 0; i < entityDataArray.length; i += MAX_ITEMS_PER_QUERY) { - const chunk = entityDataArray.slice(i, i + MAX_ITEMS_PER_QUERY); - const pendingRes = queryToExecute(sql, chunk); - responses.push(pendingRes); - } - return Promise.all(responses); -}; - module.exports.batchDeleteItemsInTable = (table, sql, pkArray) => { const primaryKeyFieldNames = TableModule.getPrimaryKeyFieldNames(table); @@ -31,53 +16,6 @@ module.exports.batchDeleteItemsInTable = (table, sql, pkArray) => { } }; -const batchSetEndOfBlockRangeScannedDataCore = (sql, rowDataArray) => { - return sql` - INSERT INTO ${sql(publicSchema)}."end_of_block_range_scanned_data" - ${sql(rowDataArray, "chain_id", "block_number", "block_hash")} - ON CONFLICT(chain_id, block_number) DO UPDATE - SET - "chain_id" = EXCLUDED."chain_id", - "block_number" = EXCLUDED."block_number", - "block_hash" = EXCLUDED."block_hash";`; -}; - -module.exports.batchSetEndOfBlockRangeScannedData = chunkBatchQuery( - batchSetEndOfBlockRangeScannedDataCore -); - -module.exports.readEndOfBlockRangeScannedDataForChain = (sql, chainId) => { - return sql` - SELECT * FROM ${sql(publicSchema)}."end_of_block_range_scanned_data" - WHERE - chain_id = ${chainId} - ORDER BY block_number ASC;`; -}; - -module.exports.deleteStaleEndOfBlockRangeScannedDataForChain = ( - sql, - chainId, - blockNumberThreshold -) => { - return sql` - DELETE - FROM ${sql(publicSchema)}."end_of_block_range_scanned_data" - WHERE chain_id = ${chainId} - AND block_number < ${blockNumberThreshold};`; -}; - -module.exports.rollbackEndOfBlockRangeScannedDataForChain = ( - sql, - chainId, - knownBlockNumber -) => { - return sql` - DELETE - FROM ${sql(publicSchema)}."end_of_block_range_scanned_data" - WHERE chain_id = ${chainId} - AND block_number > ${knownBlockNumber};`; -}; - module.exports.readAllDynamicContracts = (sql, chainId) => sql` SELECT * FROM ${sql(publicSchema)}."dynamic_contract_registry" diff --git a/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainFetcher.res b/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainFetcher.res index 58ded8d68..00d3d089b 100644 --- a/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainFetcher.res +++ b/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainFetcher.res @@ -20,7 +20,7 @@ type t = { firstEventBlockNumber: option, numEventsProcessed: int, numBatchesFetched: int, - lastBlockScannedHashes: ReorgDetection.LastBlockScannedHashes.t, + reorgDetection: ReorgDetection.t, //An optional list of filters to apply on event queries //Used for reorgs and restarts processingFilters: option>, @@ -29,7 +29,6 @@ type t = { //CONSTRUCTION let make = ( ~chainConfig: InternalConfig.chain, - ~lastBlockScannedHashes, ~dynamicContracts: array, ~startBlock, ~endBlock, @@ -42,6 +41,8 @@ let make = ( ~numEventsProcessed, ~numBatchesFetched, ~isInReorgThreshold, + ~reorgCheckpoints: array, + ~maxReorgDepth, ): t => { // We don't need the router itself, but only validation logic, // since now event router is created for selection of events @@ -180,9 +181,7 @@ let make = ( ~targetBufferSize, ~chainId=chainConfig.id, ~blockLag=Pervasives.max( - !(config->Config.shouldRollbackOnReorg) || isInReorgThreshold - ? 0 - : chainConfig.confirmedBlockThreshold, + !(config->Config.shouldRollbackOnReorg) || isInReorgThreshold ? 0 : chainConfig.maxReorgDepth, Env.indexingBlockLag->Option.getWithDefault(0), ), ~onBlockConfigs?, @@ -195,7 +194,20 @@ let make = ( ~sources=chainConfig.sources, ~maxPartitionConcurrency=Env.maxPartitionConcurrency, ), - lastBlockScannedHashes, + reorgDetection: ReorgDetection.make( + ~blocks=reorgCheckpoints->Array.keepMapU(reorgCheckpoint => { + if reorgCheckpoint.chainId === chainConfig.id { + Some({ + ReorgDetection.blockNumber: reorgCheckpoint.blockNumber, + blockHash: reorgCheckpoint.blockHash, + }) + } else { + None + } + }), + ~maxReorgDepth, + ~shouldRollbackOnReorg=config->Config.shouldRollbackOnReorg, + ), currentBlockHeight: 0, isProgressAtHead: false, fetchState, @@ -210,16 +222,14 @@ let make = ( let makeFromConfig = (chainConfig: InternalConfig.chain, ~config, ~targetBufferSize) => { let logger = Logging.createChild(~params={"chainId": chainConfig.id}) - let lastBlockScannedHashes = ReorgDetection.LastBlockScannedHashes.empty( - ~confirmedBlockThreshold=chainConfig.confirmedBlockThreshold, - ) make( ~chainConfig, ~config, ~startBlock=chainConfig.startBlock, ~endBlock=chainConfig.endBlock, - ~lastBlockScannedHashes, + ~reorgCheckpoints=[], + ~maxReorgDepth=chainConfig.maxReorgDepth, ~firstEventBlockNumber=None, ~progressBlockNumber=chainConfig.startBlock - 1, ~timestampCaughtUpToHeadOrEndblock=None, @@ -238,6 +248,7 @@ let makeFromConfig = (chainConfig: InternalConfig.chain, ~config, ~targetBufferS let makeFromDbState = async ( chainConfig: InternalConfig.chain, ~resumedChainState: InternalTable.Chains.t, + ~reorgCheckpoints, ~isInReorgThreshold, ~config, ~targetBufferSize, @@ -251,21 +262,6 @@ let makeFromDbState = async ( let dbRecoveredDynamicContracts = await sql->DbFunctions.DynamicContractRegistry.readAllDynamicContracts(~chainId) - let endOfBlockRangeScannedData = - await sql->DbFunctions.EndOfBlockRangeScannedData.readEndOfBlockRangeScannedDataForChain( - ~chainId, - ) - - let lastBlockScannedHashes = - endOfBlockRangeScannedData - ->Array.map(({blockNumber, blockHash}) => { - ReorgDetection.blockNumber, - blockHash, - }) - ->ReorgDetection.LastBlockScannedHashes.makeWithData( - ~confirmedBlockThreshold=chainConfig.confirmedBlockThreshold, - ) - Prometheus.ProgressEventsCount.set(~processedCount=resumedChainState.numEventsProcessed, ~chainId) let progressBlockNumber = @@ -280,7 +276,8 @@ let makeFromDbState = async ( ~startBlock=resumedChainState.startBlock, ~endBlock=resumedChainState.endBlock->Js.Null.toOption, ~config, - ~lastBlockScannedHashes, + ~reorgCheckpoints, + ~maxReorgDepth=resumedChainState.maxReorgDepth, ~firstEventBlockNumber=resumedChainState.firstEventBlockNumber->Js.Null.toOption, ~progressBlockNumber, ~timestampCaughtUpToHeadOrEndblock=Env.updateSyncTimeOnRestart @@ -475,7 +472,7 @@ let hasNoMoreEventsToProcess = (self: t) => { } let getHighestBlockBelowThreshold = (cf: t): int => { - let highestBlockBelowThreshold = cf.currentBlockHeight - cf.chainConfig.confirmedBlockThreshold + let highestBlockBelowThreshold = cf.currentBlockHeight - cf.chainConfig.maxReorgDepth highestBlockBelowThreshold < 0 ? 0 : highestBlockBelowThreshold } @@ -489,7 +486,7 @@ let getLastKnownValidBlock = async ( ~getBlockHashes=(chainFetcher.sourceManager->SourceManager.getActiveSource).getBlockHashes, ) => { let scannedBlockNumbers = - chainFetcher.lastBlockScannedHashes->ReorgDetection.LastBlockScannedHashes.getThresholdBlockNumbers( + chainFetcher.reorgDetection->ReorgDetection.getThresholdBlockNumbers( ~currentBlockHeight=chainFetcher.currentBlockHeight, ) @@ -524,7 +521,7 @@ let getLastKnownValidBlock = async ( while blockRef.contents->Option.isNone { let blockNumbersAndHashes = await getBlockHashes(scannedBlockNumbers) - switch chainFetcher.lastBlockScannedHashes->ReorgDetection.LastBlockScannedHashes.getLatestValidScannedBlock( + switch chainFetcher.reorgDetection->ReorgDetection.getLatestValidScannedBlock( ~blockNumbersAndHashes, ~currentBlockHeight=chainFetcher.currentBlockHeight, ~skipReorgDuplicationCheck=retryCount.contents > 2, diff --git a/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainManager.res b/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainManager.res index eedbc7042..51ca18910 100644 --- a/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainManager.res +++ b/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainManager.res @@ -1,6 +1,7 @@ open Belt type t = { + commitedCheckpointId: int, chainFetchers: ChainMap.t, multichain: InternalConfig.multichain, isInReorgThreshold: bool, @@ -23,6 +24,7 @@ let makeFromConfig = (~config: Config.t): t => { let chainFetchers = config.chainMap->ChainMap.map(ChainFetcher.makeFromConfig(_, ~config, ~targetBufferSize)) { + commitedCheckpointId: 0, chainFetchers, multichain: config.multichain, isInReorgThreshold: false, @@ -64,6 +66,7 @@ let makeFromDbState = async (~initialState: Persistence.initialState, ~config: C chain, await chainConfig->ChainFetcher.makeFromDbState( ~resumedChainState, + ~reorgCheckpoints=initialState.reorgCheckpoints, ~isInReorgThreshold, ~targetBufferSize, ~config, @@ -75,6 +78,7 @@ let makeFromDbState = async (~initialState: Persistence.initialState, ~config: C let chainFetchers = ChainMap.fromArrayUnsafe(chainFetchersArr) { + commitedCheckpointId: initialState.checkpointId, multichain: config.multichain, chainFetchers, isInReorgThreshold, @@ -95,113 +99,28 @@ let setChainFetcher = (chainManager: t, chainFetcher: ChainFetcher.t) => { } } -let getFetchStates = (chainManager: t): ChainMap.t => { - chainManager.chainFetchers->ChainMap.map(cf => { - cf.fetchState - }) -} - let nextItemIsNone = (chainManager: t): bool => { - !Batch.hasMultichainReadyItem(chainManager->getFetchStates, ~multichain=chainManager.multichain) + !Batch.hasMultichainReadyItem( + chainManager.chainFetchers->ChainMap.map(cf => { + cf.fetchState + }), + ~multichain=chainManager.multichain, + ) } let createBatch = (chainManager: t, ~batchSizeTarget: int): Batch.t => { - let refTime = Hrtime.makeTimer() - let fetchStates = chainManager->getFetchStates - - let mutBatchSizePerChain = Js.Dict.empty() - let items = if ( - switch chainManager.multichain { - | Unordered => true - | Ordered => fetchStates->ChainMap.size === 1 - } - ) { - Batch.prepareUnorderedBatch(~batchSizeTarget, ~fetchStates, ~mutBatchSizePerChain) - } else { - Batch.prepareOrderedBatch(~batchSizeTarget, ~fetchStates, ~mutBatchSizePerChain) - } - let batchSizePerChain = mutBatchSizePerChain - - let dcsToStoreByChainId = Js.Dict.empty() - // Needed to: - // - Recalculate the computed queue sizes - // - Accumulate registered dynamic contracts to store in the db - // - Trigger onBlock pointer update - let updatedFetchStates = fetchStates->ChainMap.map(fetchState => { - switch batchSizePerChain->Utils.Dict.dangerouslyGetNonOption(fetchState.chainId->Int.toString) { - | Some(batchSize) => - let leftItems = fetchState.buffer->Js.Array2.sliceFrom(batchSize) - switch fetchState.dcsToStore { - | [] => fetchState->FetchState.updateInternal(~mutItems=leftItems) - | dcs => { - let leftDcsToStore = [] - let batchDcs = [] - let updatedFetchState = - fetchState->FetchState.updateInternal(~mutItems=leftItems, ~dcsToStore=leftDcsToStore) - let nextProgressBlockNumber = updatedFetchState->FetchState.getProgressBlockNumber - - dcs->Array.forEach(dc => { - // Important: This should be a registering block number. - // This works for now since dc.startBlock is a registering block number. - if dc.startBlock <= nextProgressBlockNumber { - batchDcs->Array.push(dc) - } else { - // Mutate the array we passed to the updateInternal beforehand - leftDcsToStore->Array.push(dc) - } - }) - - dcsToStoreByChainId->Js.Dict.set(fetchState.chainId->Int.toString, batchDcs) - updatedFetchState - } - } - // Skip not affected chains - | None => fetchState - } - }) - - let progressedChains = [] - chainManager.chainFetchers - ->ChainMap.entries - ->Array.forEach(((chain, chainFetcher)) => { - let updatedFetchState = updatedFetchStates->ChainMap.get(chain) - let nextProgressBlockNumber = updatedFetchState->FetchState.getProgressBlockNumber - let maybeItemsCountInBatch = - batchSizePerChain->Utils.Dict.dangerouslyGetNonOption( - chain->ChainMap.Chain.toChainId->Int.toString, - ) - if ( - chainFetcher.committedProgressBlockNumber < nextProgressBlockNumber || - // It should never be 0 - maybeItemsCountInBatch->Option.isSome - ) { - let chainBatchSize = maybeItemsCountInBatch->Option.getWithDefault(0) - progressedChains - ->Js.Array2.push( - ( - { - chainId: chain->ChainMap.Chain.toChainId, - batchSize: chainBatchSize, - progressBlockNumber: nextProgressBlockNumber, - totalEventsProcessed: chainFetcher.numEventsProcessed + chainBatchSize, - // Snapshot the value at the moment of batch creation - // so we don't have a case where we can't catch up the head because of the - // defference between processing and new blocks - isProgressAtHead: nextProgressBlockNumber >= chainFetcher.currentBlockHeight, - }: Batch.progressedChain - ), - ) - ->ignore - } - }) - - { - items, - progressedChains, - updatedFetchStates, - dcsToStoreByChainId, - creationTimeMs: refTime->Hrtime.timeSince->Hrtime.toMillis->Hrtime.intFromMillis, - } + Batch.make( + ~checkpointIdBeforeBatch=chainManager.commitedCheckpointId, + ~chainsBeforeBatch=chainManager.chainFetchers->ChainMap.map((cf): Batch.chainBeforeBatch => { + fetchState: cf.fetchState, + progressBlockNumber: cf.committedProgressBlockNumber, + totalEventsProcessed: cf.numEventsProcessed, + sourceBlockNumber: cf.currentBlockHeight, + reorgDetection: cf.reorgDetection, + }), + ~multichain=chainManager.multichain, + ~batchSizeTarget, + ) } let isProgressAtHead = chainManager => diff --git a/codegenerator/cli/templates/static/codegen/src/globalState/GlobalState.res b/codegenerator/cli/templates/static/codegen/src/globalState/GlobalState.res index 2cf4fd74d..eca628d3d 100644 --- a/codegenerator/cli/templates/static/codegen/src/globalState/GlobalState.res +++ b/codegenerator/cli/templates/static/codegen/src/globalState/GlobalState.res @@ -1,15 +1,15 @@ open Belt type chain = ChainMap.Chain.t -type rollbackState = NoRollback | RollingBack(chain) | RollbackInMemStore(InMemoryStore.t) +type rollbackState = + NoRollback | PreparingRollback(chain) | PreparedRollback({diffInMemoryStore: InMemoryStore.t}) module WriteThrottlers = { type t = { chainMetaData: Throttler.t, - pruneStaleEndBlockData: ChainMap.t, pruneStaleEntityHistory: Throttler.t, } - let make = (~config: Config.t): t => { + let make = (): t => { let chainMetaData = { let intervalMillis = Env.ThrottleWrites.chainMetadataIntervalMillis let logger = Logging.createChild( @@ -21,18 +21,6 @@ module WriteThrottlers = { Throttler.make(~intervalMillis, ~logger) } - let pruneStaleEndBlockData = config.chainMap->ChainMap.map(cfg => { - let intervalMillis = Env.ThrottleWrites.pruneStaleDataIntervalMillis - let logger = Logging.createChild( - ~params={ - "context": "Throttler for pruning stale endblock data", - "intervalMillis": intervalMillis, - "chain": cfg.id, - }, - ) - Throttler.make(~intervalMillis, ~logger) - }) - let pruneStaleEntityHistory = { let intervalMillis = Env.ThrottleWrites.pruneStaleDataIntervalMillis let logger = Logging.createChild( @@ -43,7 +31,7 @@ module WriteThrottlers = { ) Throttler.make(~intervalMillis, ~logger) } - {chainMetaData, pruneStaleEndBlockData, pruneStaleEntityHistory} + {chainMetaData, pruneStaleEntityHistory} } } @@ -70,7 +58,7 @@ let make = (~config: Config.t, ~chainManager: ChainManager.t, ~shouldUseTui=fals chainManager, indexerStartTime: Js.Date.make(), rollbackState: NoRollback, - writeThrottlers: WriteThrottlers.make(~config), + writeThrottlers: WriteThrottlers.make(), loadManager: LoadManager.make(), shouldUseTui, id: 0, @@ -79,7 +67,7 @@ let make = (~config: Config.t, ~chainManager: ChainManager.t, ~shouldUseTui=fals let getId = self => self.id let incrementId = self => {...self, id: self.id + 1} -let setRollingBack = (self, chain) => {...self, rollbackState: RollingBack(chain)} +let setRollingBack = (self, chain) => {...self, rollbackState: PreparingRollback(chain)} let setChainManager = (self, chainManager) => { ...self, chainManager, @@ -87,7 +75,7 @@ let setChainManager = (self, chainManager) => { let isRollingBack = state => switch state.rollbackState { - | RollingBack(_) => true + | PreparingRollback(_) => true | _ => false } @@ -117,14 +105,11 @@ type action = chain: chain, }) | FinishWaitingForNewBlock({chain: chain, currentBlockHeight: int}) - | EventBatchProcessed({ - progressedChains: array, - items: array, - }) + | EventBatchProcessed({batch: Batch.t}) | StartProcessingBatch | EnterReorgThreshold | UpdateQueues({ - updatedFetchStates: ChainMap.t, + progressedChainsById: dict, // Needed to prevent overwriting the blockLag // set by EnterReorgThreshold shouldEnterReorgThreshold: bool, @@ -137,11 +122,6 @@ type action = type queryChain = CheckAllChains | Chain(chain) type task = | NextQuery(queryChain) - | UpdateEndOfBlockRangeScannedData({ - chain: chain, - blockNumberThreshold: int, - nextEndOfBlockRangeScannedData: DbFunctions.EndOfBlockRangeScannedData.endOfBlockRangeScannedData, - }) | ProcessPartitionQueryResponse(partitionQueryResponse) | ProcessEventBatch | UpdateChainMetaDataAndCheckForExit(shouldExit) @@ -195,11 +175,7 @@ let updateChainMetadataTable = (cm: ChainManager.t, ~throttler: Throttler.t) => Takes in a chain manager and sets all chains timestamp caught up to head when valid state lines up and returns an updated chain manager */ -let updateProgressedChains = ( - chainManager: ChainManager.t, - ~progressedChains: array, - ~items: array, -) => { +let updateProgressedChains = (chainManager: ChainManager.t, ~batch: Batch.t) => { let nextQueueItemIsNone = chainManager->ChainManager.nextItemIsNone let allChainsAtHead = chainManager->ChainManager.isProgressAtHead @@ -207,22 +183,22 @@ let updateProgressedChains = ( let chainFetchers = chainManager.chainFetchers->ChainMap.map(cf => { let chain = ChainMap.Chain.makeUnsafe(~chainId=cf.chainConfig.id) - let maybeProgressData = - progressedChains->Js.Array2.find(progressedChain => - progressedChain.chainId === chain->ChainMap.Chain.toChainId + let maybeChainAfterBatch = + batch.progressedChainsById->Utils.Dict.dangerouslyGetByIntNonOption( + chain->ChainMap.Chain.toChainId, ) - let cf = switch maybeProgressData { - | Some(progressData) => { - if cf.committedProgressBlockNumber !== progressData.progressBlockNumber { + let cf = switch maybeChainAfterBatch { + | Some(chainAfterBatch) => { + if cf.committedProgressBlockNumber !== chainAfterBatch.progressBlockNumber { Prometheus.ProgressBlockNumber.set( - ~blockNumber=progressData.progressBlockNumber, + ~blockNumber=chainAfterBatch.progressBlockNumber, ~chainId=chain->ChainMap.Chain.toChainId, ) } - if cf.numEventsProcessed !== progressData.totalEventsProcessed { + if cf.numEventsProcessed !== chainAfterBatch.totalEventsProcessed { Prometheus.ProgressEventsCount.set( - ~processedCount=progressData.totalEventsProcessed, + ~processedCount=chainAfterBatch.totalEventsProcessed, ~chainId=chain->ChainMap.Chain.toChainId, ) } @@ -232,21 +208,11 @@ let updateProgressedChains = ( // we need to calculate it once, by using the first item in a batch firstEventBlockNumber: switch cf.firstEventBlockNumber { | Some(_) => cf.firstEventBlockNumber - | None => - switch items->Js.Array2.find(item => - switch item { - | Internal.Event({chain: eventChain}) => eventChain === chain - | Internal.Block({onBlockConfig: {chainId}}) => - chainId === chain->ChainMap.Chain.toChainId - } - ) { - | Some(item) => Some(item->Internal.getItemBlockNumber) - | None => None - } + | None => batch->Batch.findFirstEventBlockNumber(~chainId=chain->ChainMap.Chain.toChainId) }, - isProgressAtHead: cf.isProgressAtHead || progressData.isProgressAtHead, - committedProgressBlockNumber: progressData.progressBlockNumber, - numEventsProcessed: progressData.totalEventsProcessed, + committedProgressBlockNumber: chainAfterBatch.progressBlockNumber, + numEventsProcessed: chainAfterBatch.totalEventsProcessed, + isProgressAtHead: cf.isProgressAtHead || chainAfterBatch.isProgressAtHeadWhenBatchCreated, } } | None => cf @@ -324,6 +290,10 @@ let updateProgressedChains = ( { ...chainManager, + commitedCheckpointId: switch batch.checkpointIds->Utils.Array.last { + | Some(checkpointId) => checkpointId + | None => chainManager.commitedCheckpointId + }, chainFetchers, } } @@ -341,7 +311,6 @@ let validatePartitionQueryResponse = ( reorgGuard, fromBlockQueried, } = response - let {rangeLastBlock} = reorgGuard if currentBlockHeight > chainFetcher.currentBlockHeight { Prometheus.SourceHeight.set( @@ -372,16 +341,12 @@ let validatePartitionQueryResponse = ( ) } - let (updatedLastBlockScannedHashes, reorgResult) = - chainFetcher.lastBlockScannedHashes->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( - ~reorgGuard, - ~currentBlockHeight, - ~shouldRollbackOnReorg=state.config->Config.shouldRollbackOnReorg, - ) + let (updatedReorgDetection, reorgResult: ReorgDetection.reorgResult) = + chainFetcher.reorgDetection->ReorgDetection.registerReorgGuard(~reorgGuard, ~currentBlockHeight) let updatedChainFetcher = { ...chainFetcher, - lastBlockScannedHashes: updatedLastBlockScannedHashes, + reorgDetection: updatedReorgDetection, } let nextState = { @@ -412,30 +377,7 @@ let validatePartitionQueryResponse = ( if isRollback { (nextState->incrementId->setRollingBack(chain), [Rollback]) } else { - let updateEndOfBlockRangeScannedDataArr = - //Only update endOfBlockRangeScannedData if rollbacks are enabled - state.config->Config.shouldRollbackOnReorg - ? [ - UpdateEndOfBlockRangeScannedData({ - chain, - blockNumberThreshold: rangeLastBlock.blockNumber - - updatedChainFetcher.chainConfig.confirmedBlockThreshold, - nextEndOfBlockRangeScannedData: { - chainId: chain->ChainMap.Chain.toChainId, - blockNumber: rangeLastBlock.blockNumber, - blockHash: rangeLastBlock.blockHash, - }, - }), - ] - : [] - - ( - nextState, - Array.concat( - updateEndOfBlockRangeScannedDataArr, - [ProcessPartitionQueryResponse(partitionQueryResponse)], - ), - ) + (nextState, [ProcessPartitionQueryResponse(partitionQueryResponse)]) } } @@ -461,10 +403,7 @@ let submitPartitionQueryResponse = ( numBatchesFetched: updatedChainFetcher.numBatchesFetched + 1, } - let wasFetchingAtHead = chainFetcher.isProgressAtHead - let isCurrentlyFetchingAtHead = updatedChainFetcher.isProgressAtHead - - if !wasFetchingAtHead && isCurrentlyFetchingAtHead { + if !chainFetcher.isProgressAtHead && updatedChainFetcher.isProgressAtHead { updatedChainFetcher.logger->Logging.childInfo("All events have been fetched") } @@ -614,7 +553,7 @@ let actionReducer = (state: t, action: action) => { ~query, ~chain, ) - | EventBatchProcessed({progressedChains, items}) => + | EventBatchProcessed({batch}) => let maybePruneEntityHistory = state.config->Config.shouldPruneHistory( ~isInReorgThreshold=state.chainManager.isInReorgThreshold, @@ -624,7 +563,7 @@ let actionReducer = (state: t, action: action) => { let state = { ...state, - chainManager: state.chainManager->updateProgressedChains(~progressedChains, ~items), + chainManager: state.chainManager->updateProgressedChains(~batch), currentlyProcessingBatch: false, processedBatches: state.processedBatches + 1, } @@ -678,9 +617,14 @@ let actionReducer = (state: t, action: action) => { }, [NextQuery(CheckAllChains)], ) - | UpdateQueues({updatedFetchStates, shouldEnterReorgThreshold}) => + | UpdateQueues({progressedChainsById, shouldEnterReorgThreshold}) => let chainFetchers = state.chainManager.chainFetchers->ChainMap.mapWithKey((chain, cf) => { - let fs = ChainMap.get(updatedFetchStates, chain) + let fs = switch progressedChainsById->Utils.Dict.dangerouslyGetByIntNonOption( + chain->ChainMap.Chain.toChainId, + ) { + | Some(chainAfterBatch) => chainAfterBatch.fetchState + | None => cf.fetchState + } { ...cf, fetchState: shouldEnterReorgThreshold @@ -702,7 +646,7 @@ let actionReducer = (state: t, action: action) => { [NextQuery(CheckAllChains)], ) | SetRollbackState(inMemoryStore, chainManager) => ( - {...state, rollbackState: RollbackInMemStore(inMemoryStore), chainManager}, + {...state, rollbackState: PreparedRollback({diffInMemoryStore: inMemoryStore}), chainManager}, [NextQuery(CheckAllChains), ProcessEventBatch], ) | ResetRollbackState => ({...state, rollbackState: NoRollback}, []) @@ -720,7 +664,7 @@ let actionReducer = (state: t, action: action) => { let invalidatedActionReducer = (state: t, action: action) => switch (state, action) { - | ({rollbackState: RollingBack(_)}, EventBatchProcessed(_)) => + | ({rollbackState: PreparingRollback(_)}, EventBatchProcessed(_)) => Logging.info("Finished processing batch before rollback, actioning rollback") ( {...state, currentlyProcessingBatch: false, processedBatches: state.processedBatches + 1}, @@ -781,51 +725,15 @@ let injectedTaskReducer = ( switch task { | ProcessPartitionQueryResponse(partitionQueryResponse) => state->processPartitionQueryResponse(partitionQueryResponse, ~dispatchAction)->Promise.done - | UpdateEndOfBlockRangeScannedData({ - chain, - blockNumberThreshold, - nextEndOfBlockRangeScannedData, - }) => - let timeRef = Hrtime.makeTimer() - await Db.sql->DbFunctions.EndOfBlockRangeScannedData.setEndOfBlockRangeScannedData( - nextEndOfBlockRangeScannedData, - ) - - if Env.Benchmark.shouldSaveData { - let elapsedTimeMillis = Hrtime.timeSince(timeRef)->Hrtime.toMillis->Hrtime.intFromMillis - Benchmark.addSummaryData( - ~group="Other", - ~label=`Chain ${chain->ChainMap.Chain.toString} UpdateEndOfBlockRangeScannedData (ms)`, - ~value=elapsedTimeMillis->Belt.Int.toFloat, - ) - } - - //These prune functions can be scheduled and throttled if a more recent prune function gets called - //before the current one is executed - let runPrune = async () => { - let timeRef = Hrtime.makeTimer() - await Db.sql->DbFunctions.EndOfBlockRangeScannedData.deleteStaleEndOfBlockRangeScannedDataForChain( - ~chainId=chain->ChainMap.Chain.toChainId, - ~blockNumberThreshold, - ) - - if Env.Benchmark.shouldSaveData { - let elapsedTimeMillis = Hrtime.timeSince(timeRef)->Hrtime.toMillis->Hrtime.intFromMillis - Benchmark.addSummaryData( - ~group="Other", - ~label=`Chain ${chain->ChainMap.Chain.toString} PruneStaleData (ms)`, - ~value=elapsedTimeMillis->Belt.Int.toFloat, - ) - } - } - - let throttler = state.writeThrottlers.pruneStaleEndBlockData->ChainMap.get(chain) - throttler->Throttler.schedule(runPrune) | PruneStaleEntityHistory => let runPrune = async () => { let safeReorgBlocks = state.chainManager->ChainManager.getSafeReorgBlocks if safeReorgBlocks.chainIds->Utils.Array.notEmpty { + await Db.sql->InternalTable.Checkpoints.deprecated_pruneStaleCheckpoints( + ~pgSchema=Env.Db.publicSchema, + ) + for idx in 0 to Entities.allEntities->Array.length - 1 { if idx !== 0 { // Add some delay between entities @@ -899,68 +807,73 @@ let injectedTaskReducer = ( if !state.currentlyProcessingBatch && !isRollingBack(state) { let batch = state.chainManager->ChainManager.createBatch(~batchSizeTarget=state.config.batchSize) - - let updatedFetchStates = batch.updatedFetchStates + let progressedChainsById = batch.progressedChainsById + let totalBatchSize = batch.totalBatchSize let isInReorgThreshold = state.chainManager.isInReorgThreshold + let shouldSaveHistory = state.config->Config.shouldSaveHistory(~isInReorgThreshold) + let isBelowReorgThreshold = !state.chainManager.isInReorgThreshold && state.config->Config.shouldRollbackOnReorg let shouldEnterReorgThreshold = isBelowReorgThreshold && - updatedFetchStates - ->ChainMap.keys - ->Array.every(chain => { - updatedFetchStates - ->ChainMap.get(chain) - ->FetchState.isReadyToEnterReorgThreshold( - ~currentBlockHeight=( - state.chainManager.chainFetchers->ChainMap.get(chain) - ).currentBlockHeight, + state.chainManager.chainFetchers + ->ChainMap.values + ->Array.every(chainFetcher => { + let fetchState = switch progressedChainsById->Utils.Dict.dangerouslyGetByIntNonOption( + chainFetcher.fetchState.chainId, + ) { + | Some(chainAfterBatch) => chainAfterBatch.fetchState + | None => chainFetcher.fetchState + } + fetchState->FetchState.isReadyToEnterReorgThreshold( + ~currentBlockHeight=chainFetcher.currentBlockHeight, ) }) if shouldEnterReorgThreshold { dispatchAction(EnterReorgThreshold) } - switch batch { - | {progressedChains: []} => () - | {items: [], progressedChains} => + if progressedChainsById->Utils.Dict.isEmpty { + () + } else if totalBatchSize === 0 { dispatchAction(StartProcessingBatch) // For this case there shouldn't be any FetchState changes // so we don't dispatch UpdateQueues - only update the progress for chains without events await Db.sql->InternalTable.Chains.setProgressedChains( ~pgSchema=Db.publicSchema, - ~progressedChains, + ~progressedChains=progressedChainsById->Utils.Dict.mapValuesToArray(( + chainAfterBatch + ): InternalTable.Chains.progressedChain => { + chainId: chainAfterBatch.fetchState.chainId, + progressBlockNumber: chainAfterBatch.progressBlockNumber, + totalEventsProcessed: chainAfterBatch.totalEventsProcessed, + }), ) - // FIXME: When state.rollbackState is RollbackInMemStore + // FIXME: When state.rollbackState is PreparedRollback // If we increase progress in this case (no items) // and then indexer restarts - there's a high chance of missing // the rollback. This should be tested and fixed. - dispatchAction(EventBatchProcessed({progressedChains, items: batch.items})) - | {items, progressedChains, updatedFetchStates, dcsToStoreByChainId} => + dispatchAction(EventBatchProcessed({batch: batch})) + } else { if Env.Benchmark.shouldSaveData { let group = "Other" - Benchmark.addSummaryData( - ~group, - ~label=`Batch Creation Time (ms)`, - ~value=batch.creationTimeMs->Belt.Int.toFloat, - ) Benchmark.addSummaryData( ~group, ~label=`Batch Size`, - ~value=items->Array.length->Belt.Int.toFloat, + ~value=totalBatchSize->Belt.Int.toFloat, ) } dispatchAction(StartProcessingBatch) - dispatchAction(UpdateQueues({updatedFetchStates, shouldEnterReorgThreshold})) + dispatchAction(UpdateQueues({progressedChainsById, shouldEnterReorgThreshold})) //In the case of a rollback, use the provided in memory store //With rolled back values let rollbackInMemStore = switch state.rollbackState { - | RollbackInMemStore(inMemoryStore) => Some(inMemoryStore) + | PreparedRollback({diffInMemoryStore}) => Some(diffInMemoryStore) | NoRollback - | RollingBack( + | PreparingRollback( _, ) /* This is an impossible case due to the surrounding if statement check */ => None @@ -968,22 +881,25 @@ let injectedTaskReducer = ( let inMemoryStore = rollbackInMemStore->Option.getWithDefault(InMemoryStore.make()) - if dcsToStoreByChainId->Utils.Dict.size > 0 { - let shouldSaveHistory = state.config->Config.shouldSaveHistory(~isInReorgThreshold) - inMemoryStore->InMemoryStore.setDcsToStore(dcsToStoreByChainId, ~shouldSaveHistory) - } - state.chainManager.chainFetchers ->ChainMap.keys ->Array.forEach(chain => { let chainId = chain->ChainMap.Chain.toChainId - switch progressedChains->Js.Array2.find(progressedChain => - progressedChain.chainId === chainId - ) { - | Some(progressData) => - Prometheus.ProcessingBatchSize.set(~batchSize=progressData.batchSize, ~chainId) + switch progressedChainsById->Utils.Dict.dangerouslyGetByIntNonOption(chainId) { + | Some(chainAfterBatch) => + switch chainAfterBatch.dcsToStore { + | Some(dcsToStore) => + inMemoryStore->InMemoryStore.setDcsToStore( + ~chainId, + ~dcs=dcsToStore, + ~shouldSaveHistory, + ) + | None => () + } + + Prometheus.ProcessingBatchSize.set(~batchSize=chainAfterBatch.batchSize, ~chainId) Prometheus.ProcessingBlockNumber.set( - ~blockNumber=progressData.progressBlockNumber, + ~blockNumber=chainAfterBatch.progressBlockNumber, ~chainId, ) | None => Prometheus.ProcessingBatchSize.set(~batchSize=0, ~chainId) @@ -991,8 +907,7 @@ let injectedTaskReducer = ( }) switch await EventProcessing.processEventBatch( - ~items, - ~progressedChains, + ~batch, ~inMemoryStore, ~isInReorgThreshold, ~loadManager=state.loadManager, @@ -1011,7 +926,7 @@ let injectedTaskReducer = ( dispatchAction(ResetRollbackState) } switch res { - | Ok() => dispatchAction(EventBatchProcessed({progressedChains, items})) + | Ok() => dispatchAction(EventBatchProcessed({batch: batch})) | Error(errHandler) => dispatchAction(ErrorExit(errHandler)) } } @@ -1020,7 +935,7 @@ let injectedTaskReducer = ( | Rollback => //If it isn't processing a batch currently continue with rollback otherwise wait for current batch to finish processing switch state { - | {currentlyProcessingBatch: false, rollbackState: RollingBack(reorgChain)} => + | {currentlyProcessingBatch: false, rollbackState: PreparingRollback(reorgChain)} => let startTime = Hrtime.makeTimer() let chainFetcher = state.chainManager.chainFetchers->ChainMap.get(reorgChain) @@ -1084,12 +999,16 @@ let injectedTaskReducer = ( let rolledBackCf = { ...cf, - lastBlockScannedHashes: chain == reorgChain - ? cf.lastBlockScannedHashes->ReorgDetection.LastBlockScannedHashes.rollbackToValidBlockNumber( + reorgDetection: chain == reorgChain + ? cf.reorgDetection->ReorgDetection.rollbackToValidBlockNumber( ~blockNumber=lastKnownValidBlockNumber, ) - : cf.lastBlockScannedHashes, + : cf.reorgDetection, fetchState, + committedProgressBlockNumber: PervasivesU.min( + cf.committedProgressBlockNumber, + firstChangeEvent.blockNumber - 1, + ), } //On other chains, filter out evennts based on the first change present on the chain after the reorg rolledBackCf->ChainFetcher.addProcessingFilter( diff --git a/scenarios/erc20_multichain_factory/test/RollbackDynamicContract_test.res b/scenarios/erc20_multichain_factory/test/RollbackDynamicContract_test.res index 2b952b628..dea2b8962 100644 --- a/scenarios/erc20_multichain_factory/test/RollbackDynamicContract_test.res +++ b/scenarios/erc20_multichain_factory/test/RollbackDynamicContract_test.res @@ -100,26 +100,6 @@ ensure that this doesn't trigger a reorg | _ => Js.Exn.raiseError("Unexpected chain") } ) - - let getUpdateEndofBlockRangeScannedData = ( - mcdMap, - ~chain, - ~blockNumber, - ~blockNumberThreshold, - ) => { - let {blockNumber, blockHash} = - mcdMap->ChainMap.get(chain)->MockChainData.getBlock(~blockNumber)->Option.getUnsafe - - GlobalState.UpdateEndOfBlockRangeScannedData({ - blockNumberThreshold, - chain, - nextEndOfBlockRangeScannedData: { - blockNumber, - blockHash, - chainId: chain->ChainMap.Chain.toChainId, - }, - }) - } } module Sql = RollbackMultichain_test.Sql diff --git a/scenarios/erc20_multichain_factory/test/RollbackMultichain_test.res b/scenarios/erc20_multichain_factory/test/RollbackMultichain_test.res index 7aa98f4c3..f28360a2c 100644 --- a/scenarios/erc20_multichain_factory/test/RollbackMultichain_test.res +++ b/scenarios/erc20_multichain_factory/test/RollbackMultichain_test.res @@ -166,26 +166,6 @@ module Mock = { | _ => Js.Exn.raiseError("Unexpected chain") } ) - - let getUpdateEndofBlockRangeScannedData = ( - mcdMap, - ~chain, - ~blockNumber, - ~blockNumberThreshold, - ) => { - let {blockNumber, blockHash} = - mcdMap->ChainMap.get(chain)->MockChainData.getBlock(~blockNumber)->Option.getUnsafe - - GlobalState.UpdateEndOfBlockRangeScannedData({ - blockNumberThreshold, - chain, - nextEndOfBlockRangeScannedData: { - blockNumber, - blockHash, - chainId: chain->ChainMap.Chain.toChainId, - }, - }) - } } module Sql = { diff --git a/scenarios/erc20_multichain_factory/test/TestDeleteEntity.res b/scenarios/erc20_multichain_factory/test/TestDeleteEntity.res index 1a3600e5c..f6fe493e7 100644 --- a/scenarios/erc20_multichain_factory/test/TestDeleteEntity.res +++ b/scenarios/erc20_multichain_factory/test/TestDeleteEntity.res @@ -88,26 +88,6 @@ module Mock = { | _ => Js.Exn.raiseError("Unexpected chain") } ) - - let getUpdateEndofBlockRangeScannedData = ( - mcdMap, - ~chain, - ~blockNumber, - ~blockNumberThreshold, - ) => { - let {blockNumber, blockHash} = - mcdMap->ChainMap.get(chain)->MockChainData.getBlock(~blockNumber)->Option.getUnsafe - - GlobalState.UpdateEndOfBlockRangeScannedData({ - blockNumberThreshold, - chain, - nextEndOfBlockRangeScannedData: { - blockNumber, - blockHash, - chainId: chain->ChainMap.Chain.toChainId, - }, - }) - } } module Sql = RollbackMultichain_test.Sql diff --git a/scenarios/erc20_multichain_factory/test/TestWhereQuery.res b/scenarios/erc20_multichain_factory/test/TestWhereQuery.res index 823db7b66..957a5332f 100644 --- a/scenarios/erc20_multichain_factory/test/TestWhereQuery.res +++ b/scenarios/erc20_multichain_factory/test/TestWhereQuery.res @@ -82,26 +82,6 @@ module Mock = { | _ => Js.Exn.raiseError("Unexpected chain") } ) - - let getUpdateEndofBlockRangeScannedData = ( - mcdMap, - ~chain, - ~blockNumber, - ~blockNumberThreshold, - ) => { - let {blockNumber, blockHash} = - mcdMap->ChainMap.get(chain)->MockChainData.getBlock(~blockNumber)->Option.getUnsafe - - GlobalState.UpdateEndOfBlockRangeScannedData({ - blockNumberThreshold, - chain, - nextEndOfBlockRangeScannedData: { - blockNumber, - blockHash, - chainId: chain->ChainMap.Chain.toChainId, - }, - }) - } } module Sql = RollbackMultichain_test.Sql diff --git a/scenarios/test_codegen/test/ChainManager_test.res b/scenarios/test_codegen/test/ChainManager_test.res index 6ec3530ea..f67b66b52 100644 --- a/scenarios/test_codegen/test/ChainManager_test.res +++ b/scenarios/test_codegen/test/ChainManager_test.res @@ -31,7 +31,7 @@ let populateChainQueuesWithRandomEvents = (~runTime=1000, ~maxBlockTime=15, ()) ~contracts=[], ~startBlock=0, ~targetBufferSize=5000, - ~chainId=0, + ~chainId=1, ) let fetchState = ref(fetcherStateInit) @@ -108,8 +108,10 @@ let populateChainQueuesWithRandomEvents = (~runTime=1000, ~maxBlockTime=15, ()) ), chainConfig, // This is quite a hack - but it works! - lastBlockScannedHashes: ReorgDetection.LastBlockScannedHashes.empty( - ~confirmedBlockThreshold=200, + reorgDetection: ReorgDetection.make( + ~blocks=[], + ~maxReorgDepth=200, + ~shouldRollbackOnReorg=false, ), isProgressAtHead: false, currentBlockHeight: 0, @@ -123,6 +125,7 @@ let populateChainQueuesWithRandomEvents = (~runTime=1000, ~maxBlockTime=15, ()) { ChainManager.chainFetchers, multichain: Ordered, + commitedCheckpointId: 0, isInReorgThreshold: false, }, numberOfMockEventsCreated.contents, @@ -154,20 +157,22 @@ describe("ChainManager", () => { let numberOfMockEventsReadFromQueues = ref(0) let allEventsRead = [] let rec testThatCreatedEventsAreOrderedCorrectly = (chainManager, lastEvent) => { - let eventsInBlock = ChainManager.createBatch(chainManager, ~batchSizeTarget=10000) + let {items, totalBatchSize, progressedChainsById} = ChainManager.createBatch( + chainManager, + ~batchSizeTarget=10000, + ) // ensure that the events are ordered correctly - switch eventsInBlock { - | {items: []} => chainManager - | {items, updatedFetchStates} => - items->Belt.Array.forEach( - i => { - let _ = allEventsRead->Js.Array2.push(i) + if totalBatchSize === 0 { + chainManager + } else { + items->Array.forEach( + item => { + allEventsRead->Js.Array2.push(item)->ignore }, ) - let batchSize = items->Array.length numberOfMockEventsReadFromQueues := - numberOfMockEventsReadFromQueues.contents + batchSize + numberOfMockEventsReadFromQueues.contents + totalBatchSize let firstEventInBlock = items[0]->Option.getExn @@ -180,7 +185,12 @@ describe("ChainManager", () => { let nextChainFetchers = chainManager.chainFetchers->ChainMap.mapWithKey( (chain, fetcher) => { - let fetchState = updatedFetchStates->ChainMap.get(chain) + let fetchState = switch progressedChainsById->Utils.Dict.dangerouslyGetByIntNonOption( + chain->ChainMap.Chain.toChainId, + ) { + | Some(chainAfterBatch) => chainAfterBatch.fetchState + | None => fetcher.fetchState + } { ...fetcher, fetchState, diff --git a/scenarios/test_codegen/test/E2EEthNode_test.res b/scenarios/test_codegen/test/E2EEthNode_test.res index 334df9f74..a6a941826 100644 --- a/scenarios/test_codegen/test/E2EEthNode_test.res +++ b/scenarios/test_codegen/test/E2EEthNode_test.res @@ -39,7 +39,7 @@ describe("E2E Integration Test", () => { ) let chain = MockConfig.chain1337 { - confirmedBlockThreshold: 200, + maxReorgDepth: 200, startBlock: 0, id: 1337, contracts, diff --git a/scenarios/test_codegen/test/Integration_ts_helpers.res b/scenarios/test_codegen/test/Integration_ts_helpers.res index 6d41b246b..d30a51c5f 100644 --- a/scenarios/test_codegen/test/Integration_ts_helpers.res +++ b/scenarios/test_codegen/test/Integration_ts_helpers.res @@ -28,7 +28,7 @@ let getLocalChainConfig = (nftFactoryContractAddress): chainConfig => { }) let chain = MockConfig.chain1337 { - confirmedBlockThreshold: 200, + maxReorgDepth: 200, startBlock: 1, id: 1337, contracts, diff --git a/scenarios/test_codegen/test/Mock_test.res b/scenarios/test_codegen/test/Mock_test.res deleted file mode 100644 index dace6599d..000000000 --- a/scenarios/test_codegen/test/Mock_test.res +++ /dev/null @@ -1,95 +0,0 @@ -open RescriptMocha - -let inMemoryStore = InMemoryStore.make() - -describe("E2E Mock Event Batch", () => { - Async.before(async () => { - DbStub.setGravatarDb(~gravatar=MockEntities.gravatarEntity1) - DbStub.setGravatarDb(~gravatar=MockEntities.gravatarEntity2) - // EventProcessing.processEventBatch(MockEvents.eventBatch) - - let loadManager = LoadManager.make() - - try { - await MockEvents.eventBatchItems->EventProcessing.runBatchHandlersOrThrow( - ~inMemoryStore, - ~loadManager, - ~config=RegisterHandlers.getConfig(), - ~shouldSaveHistory=false, - ~shouldBenchmark=false, - ) - } catch { - | EventProcessing.ProcessingError({message, exn, item}) => - exn - ->ErrorHandling.make(~msg=message, ~logger=item->Logging.getItemLogger) - ->ErrorHandling.logAndRaise - } - }) -}) - -// NOTE: skipping this test for now since there seems to be some invalid DB state. Need to investigate again. -// TODO: add a similar kind of test back again. -// describe_skip("E2E Db check", () => { -// Async.before(async () => { -// await DbHelpers.runUpDownMigration() - -// let config = RegisterHandlers.registerAllHandlers() -// let loadLayer = LoadLayer.makeWithDbConnection() - -// let _ = await DbFunctionsEntities.batchSet(~entityMod=module(Entities.Gravatar))( -// Migrations.sql, -// [MockEntities.gravatarEntity1, MockEntities.gravatarEntity2], -// ) - -// let _ = await EventProcessing.processEventBatch( -// ~inMemoryStore, -// ~eventBatch=MockEvents.eventBatchItems, -// ~latestProcessedBlocks=EventProcessing.EventsProcessed.makeEmpty(~config), -// ~loadLayer, -// ~config, -// ~isInReorgThreshold=false, -// ) - -// //// TODO: write code (maybe via dependency injection) to allow us to use the stub rather than the actual database here. -// // DbStub.setGravatarDb(~gravatar=MockEntities.gravatarEntity1) -// // DbStub.setGravatarDb(~gravatar=MockEntities.gravatarEntity2) -// // await EventProcessing.processEventBatch(MockEvents.eventBatch, ~context=Context.getContext()) -// }) - -// it("Validate inmemory store state", () => { -// let gravatars = -// inMemoryStore.entities -// ->InMemoryStore.EntityTables.get(module(Entities.Gravatar)) -// ->InMemoryTable.Entity.values - -// Assert.deepEqual( -// gravatars, -// [ -// { -// id: "1001", -// owner_id: "0x1230000000000000000000000000000000000000", -// displayName: "update1", -// imageUrl: "https://gravatar1.com", -// updatesCount: BigInt.fromInt(2), -// size: MEDIUM, -// }, -// { -// id: "1002", -// owner_id: "0x4560000000000000000000000000000000000000", -// displayName: "update2", -// imageUrl: "https://gravatar2.com", -// updatesCount: BigInt.fromInt(2), -// size: MEDIUM, -// }, -// { -// id: "1003", -// owner_id: "0x7890000000000000000000000000000000000000", -// displayName: "update3", -// imageUrl: "https://gravatar3.com", -// updatesCount: BigInt.fromInt(2), -// size: MEDIUM, -// }, -// ], -// ) -// }) -// }) diff --git a/scenarios/test_codegen/test/ReorgDetection_test.res b/scenarios/test_codegen/test/ReorgDetection_test.res index 9dd2ff88c..3c99502da 100644 --- a/scenarios/test_codegen/test/ReorgDetection_test.res +++ b/scenarios/test_codegen/test/ReorgDetection_test.res @@ -1,76 +1,72 @@ open RescriptMocha open Belt -open ReorgDetection describe("Validate reorg detection functions", () => { let scannedHashesFixture = [(1, "0x123"), (50, "0x456"), (300, "0x789"), (500, "0x5432")] - let shouldRollbackOnReorg = true let pipeNoReorg = ((updated, reorgResult)) => { switch reorgResult { - | ReorgDetected(_) => Js.Exn.raiseError("Unexpected reorg detected") + | ReorgDetection.ReorgDetected(_) => Js.Exn.raiseError("Unexpected reorg detected") | NoReorg => updated } } - let mock = (arr, ~confirmedBlockThreshold=200, ~detectedReorgBlock=?) => { - arr - ->Array.map(((blockNumber, blockHash)) => { - blockNumber, - blockHash, - }) - ->LastBlockScannedHashes.makeWithData(~confirmedBlockThreshold, ~detectedReorgBlock?) + let mock = (arr, ~maxReorgDepth=200, ~shouldRollbackOnReorg=true, ~detectedReorgBlock=?) => { + ReorgDetection.make( + ~blocks=arr->Array.map(((blockNumber, blockHash)) => { + ReorgDetection.blockNumber, + blockHash, + }), + ~maxReorgDepth, + ~detectedReorgBlock?, + ~shouldRollbackOnReorg, + ) } it("getThresholdBlockNumbers works as expected", () => { Assert.deepEqual( - mock( - scannedHashesFixture, - ~confirmedBlockThreshold=200, - )->ReorgDetection.LastBlockScannedHashes.getThresholdBlockNumbers(~currentBlockHeight=500), + mock(scannedHashesFixture, ~maxReorgDepth=200)->ReorgDetection.getThresholdBlockNumbers( + ~currentBlockHeight=500, + ), [300, 500], ~message="Both 300 and 500 should be included in the threshold", ) Assert.deepEqual( - mock( - scannedHashesFixture, - ~confirmedBlockThreshold=200, - )->ReorgDetection.LastBlockScannedHashes.getThresholdBlockNumbers(~currentBlockHeight=501), + mock(scannedHashesFixture, ~maxReorgDepth=200)->ReorgDetection.getThresholdBlockNumbers( + ~currentBlockHeight=501, + ), [500], ~message="If chain progresses one more block, 300 is not included in the threshold anymore", ) Assert.deepEqual( - mock( - scannedHashesFixture, - ~confirmedBlockThreshold=200, - )->ReorgDetection.LastBlockScannedHashes.getThresholdBlockNumbers(~currentBlockHeight=499), + mock(scannedHashesFixture, ~maxReorgDepth=200)->ReorgDetection.getThresholdBlockNumbers( + ~currentBlockHeight=499, + ), [300, 500], ~message="We don't prevent blocks higher than currentBlockHeight from being included in the threshold, since the case is not possible", ) Assert.deepEqual( mock( [(300, "0x789"), (50, "0x456"), (500, "0x5432"), (1, "0x123")], - ~confirmedBlockThreshold=200, - )->ReorgDetection.LastBlockScannedHashes.getThresholdBlockNumbers(~currentBlockHeight=500), + ~maxReorgDepth=200, + )->ReorgDetection.getThresholdBlockNumbers(~currentBlockHeight=500), [300, 500], ~message="The order of blocks doesn't matter when we create reorg detection object", ) Assert.deepEqual( - mock( - scannedHashesFixture, - ~confirmedBlockThreshold=199, - )->ReorgDetection.LastBlockScannedHashes.getThresholdBlockNumbers(~currentBlockHeight=500), + mock(scannedHashesFixture, ~maxReorgDepth=199)->ReorgDetection.getThresholdBlockNumbers( + ~currentBlockHeight=500, + ), [500], - ~message="Possible to shrink confirmedBlockThreshold", + ~message="Possible to shrink maxReorgDepth", ) Assert.deepEqual( - mock( - scannedHashesFixture, - ~confirmedBlockThreshold=450, - )->ReorgDetection.LastBlockScannedHashes.getThresholdBlockNumbers(~currentBlockHeight=500), + mock(scannedHashesFixture, ~maxReorgDepth=450)->ReorgDetection.getThresholdBlockNumbers( + ~currentBlockHeight=500, + ), [50, 300, 500], - ~message="Possible to increase confirmedBlockThreshold", + ~message="Possible to increase maxReorgDepth", ) }) @@ -78,8 +74,8 @@ describe("Validate reorg detection functions", () => { let currentBlockHeight = 500 let reorgDetection = - ReorgDetection.LastBlockScannedHashes.empty(~confirmedBlockThreshold=500) - ->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( + mock([], ~maxReorgDepth=500) + ->ReorgDetection.registerReorgGuard( ~reorgGuard={ rangeLastBlock: { blockNumber: 1, @@ -88,10 +84,9 @@ describe("Validate reorg detection functions", () => { prevRangeLastBlock: None, }, ~currentBlockHeight, - ~shouldRollbackOnReorg, ) ->pipeNoReorg - ->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( + ->ReorgDetection.registerReorgGuard( ~reorgGuard={ rangeLastBlock: { blockNumber: 50, @@ -103,10 +98,9 @@ describe("Validate reorg detection functions", () => { }), }, ~currentBlockHeight, - ~shouldRollbackOnReorg, ) ->pipeNoReorg - ->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( + ->ReorgDetection.registerReorgGuard( ~reorgGuard={ rangeLastBlock: { blockNumber: 300, @@ -118,10 +112,9 @@ describe("Validate reorg detection functions", () => { }), }, ~currentBlockHeight, - ~shouldRollbackOnReorg, ) ->pipeNoReorg - ->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( + ->ReorgDetection.registerReorgGuard( ~reorgGuard={ rangeLastBlock: { blockNumber: 500, @@ -133,13 +126,12 @@ describe("Validate reorg detection functions", () => { }), }, ~currentBlockHeight, - ~shouldRollbackOnReorg, ) ->pipeNoReorg Assert.deepEqual( reorgDetection, - mock(scannedHashesFixture, ~confirmedBlockThreshold=500), + mock(scannedHashesFixture, ~maxReorgDepth=500), ~message="Should have the same data as the mock", ) }) @@ -149,8 +141,8 @@ describe("Validate reorg detection functions", () => { () => { let currentBlockHeight = 500 let reorgDetection = - ReorgDetection.LastBlockScannedHashes.empty(~confirmedBlockThreshold=200) - ->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( + mock([], ~maxReorgDepth=200) + ->ReorgDetection.registerReorgGuard( ~reorgGuard={ rangeLastBlock: { blockNumber: 50, @@ -162,13 +154,12 @@ describe("Validate reorg detection functions", () => { }), }, ~currentBlockHeight, - ~shouldRollbackOnReorg, ) ->pipeNoReorg Assert.deepEqual( reorgDetection, - mock([(1, "0x123"), (50, "0x456")], ~confirmedBlockThreshold=200), + mock([(1, "0x123"), (50, "0x456")], ~maxReorgDepth=200), ~message="Should add two records. One for rangeLastBlock and one for prevRangeLastBlock", ) }, @@ -176,8 +167,8 @@ describe("Validate reorg detection functions", () => { it("Should prune records outside of the reorg threshold on registering new data", () => { let reorgDetection = - mock([(1, "0x1"), (2, "0x2"), (3, "0x3")], ~confirmedBlockThreshold=2) - ->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( + mock([(1, "0x1"), (2, "0x2"), (3, "0x3")], ~maxReorgDepth=2) + ->ReorgDetection.registerReorgGuard( ~reorgGuard={ rangeLastBlock: { blockNumber: 4, @@ -189,21 +180,20 @@ describe("Validate reorg detection functions", () => { }), }, ~currentBlockHeight=4, - ~shouldRollbackOnReorg, ) ->pipeNoReorg Assert.deepEqual( reorgDetection, - mock([(2, "0x2"), (3, "0x3"), (4, "0x4")], ~confirmedBlockThreshold=2), + mock([(2, "0x2"), (3, "0x3"), (4, "0x4")], ~maxReorgDepth=2), ~message="Should prune 1 since it's outside of reorg threshold", // Keeping block n 2 is questionable ) }) it("Shouldn't validate reorg detection if it's outside of the reorg threshold", () => { let reorgDetection = - mock(scannedHashesFixture, ~confirmedBlockThreshold=200) - ->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( + mock(scannedHashesFixture, ~maxReorgDepth=200) + ->ReorgDetection.registerReorgGuard( ~reorgGuard={ rangeLastBlock: { blockNumber: 50, @@ -215,7 +205,6 @@ describe("Validate reorg detection functions", () => { }), }, ~currentBlockHeight=500, - ~shouldRollbackOnReorg, ) ->pipeNoReorg @@ -223,7 +212,7 @@ describe("Validate reorg detection functions", () => { reorgDetection, mock( [(20, "0x20-invalid"), (50, "0x50-invalid"), (300, "0x789"), (500, "0x5432")], - ~confirmedBlockThreshold=200, + ~maxReorgDepth=200, ), ~message="Prunes original blocks at 1 and 50. It writes invalid data for block 20 and 50, but they are outside of the reorg thershold, so we don't care", ) @@ -233,7 +222,7 @@ describe("Validate reorg detection functions", () => { "Correctly getLatestValidScannedBlock when returned invalid block from another instance", () => { let reorgGuard = { - rangeLastBlock: { + ReorgDetection.rangeLastBlock: { blockNumber: 10, blockHash: "0x10", }, @@ -242,11 +231,7 @@ describe("Validate reorg detection functions", () => { let hashes = mock([(9, "0x9"), (10, "0x10-invalid")]) let (updatedHashes, reorgResult) = - hashes->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( - ~reorgGuard, - ~currentBlockHeight=10, - ~shouldRollbackOnReorg, - ) + hashes->ReorgDetection.registerReorgGuard(~reorgGuard, ~currentBlockHeight=10) Assert.deepEqual( updatedHashes, @@ -270,12 +255,12 @@ describe("Validate reorg detection functions", () => { }), ) Assert.deepEqual( - updatedHashes->LastBlockScannedHashes.getThresholdBlockNumbers(~currentBlockHeight=10), + updatedHashes->ReorgDetection.getThresholdBlockNumbers(~currentBlockHeight=10), [9, 10], ~message="Returns block numbers in hashes together with the invalid one", ) Assert.deepEqual( - updatedHashes->LastBlockScannedHashes.getLatestValidScannedBlock( + updatedHashes->ReorgDetection.getLatestValidScannedBlock( ~blockNumbersAndHashes=[ { blockNumber: 9, @@ -296,7 +281,7 @@ describe("Validate reorg detection functions", () => { `, ) Assert.deepEqual( - updatedHashes->LastBlockScannedHashes.getLatestValidScannedBlock( + updatedHashes->ReorgDetection.getLatestValidScannedBlock( ~blockNumbersAndHashes=[ { blockNumber: 9, @@ -320,7 +305,7 @@ describe("Validate reorg detection functions", () => { ) Assert.deepEqual( - updatedHashes->LastBlockScannedHashes.rollbackToValidBlockNumber(~blockNumber=9), + updatedHashes->ReorgDetection.rollbackToValidBlockNumber(~blockNumber=9), mock([(9, "0x9")]), ~message=`Should clean up the invalid block during rollback`, ) @@ -329,24 +314,21 @@ describe("Validate reorg detection functions", () => { it("Should detect reorg when rangeLastBlock hash doesn't match the scanned block", () => { let reorgGuard = { - rangeLastBlock: { + ReorgDetection.rangeLastBlock: { blockNumber: 10, blockHash: "0x10", }, prevRangeLastBlock: None, } let scannedBlock = { - blockNumber: 10, + ReorgDetection.blockNumber: 10, blockHash: "0x10-invalid", } - let hashes = mock([(10, "0x10-invalid")]) - Assert.deepEqual( - hashes->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( + mock([(10, "0x10-invalid")], ~shouldRollbackOnReorg=true)->ReorgDetection.registerReorgGuard( ~reorgGuard, ~currentBlockHeight=10, - ~shouldRollbackOnReorg, ), ( mock([(10, "0x10-invalid")], ~detectedReorgBlock=scannedBlock), @@ -358,13 +340,12 @@ describe("Validate reorg detection functions", () => { ) Assert.deepEqual( - hashes->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( + mock([(10, "0x10-invalid")], ~shouldRollbackOnReorg=false)->ReorgDetection.registerReorgGuard( ~reorgGuard, ~currentBlockHeight=10, - ~shouldRollbackOnReorg=false, ), ( - mock([]), + mock([], ~shouldRollbackOnReorg=false), ReorgDetected({ scannedBlock, receivedBlock: reorgGuard.rangeLastBlock, @@ -377,7 +358,7 @@ describe("Validate reorg detection functions", () => { it("Should detect reorg when prevRangeLastBlock hash doesn't match the scanned block", () => { let reorgGuard = { - rangeLastBlock: { + ReorgDetection.rangeLastBlock: { blockNumber: 11, blockHash: "0x11", }, @@ -387,21 +368,17 @@ describe("Validate reorg detection functions", () => { }), } - let hashes = mock([(10, "0x10-invalid")], ~confirmedBlockThreshold=2) + let hashes = mock([(10, "0x10-invalid")], ~maxReorgDepth=2) let reorgDetectionResult = - hashes->ReorgDetection.LastBlockScannedHashes.registerReorgGuard( - ~reorgGuard, - ~currentBlockHeight=11, - ~shouldRollbackOnReorg, - ) + hashes->ReorgDetection.registerReorgGuard(~reorgGuard, ~currentBlockHeight=11) Assert.deepEqual( reorgDetectionResult, ( mock( [(10, "0x10-invalid")], - ~confirmedBlockThreshold=2, + ~maxReorgDepth=2, ~detectedReorgBlock={ blockNumber: 10, blockHash: "0x10-invalid", @@ -422,16 +399,16 @@ describe("Validate reorg detection functions", () => { }) it("rollbackToValidBlockNumber works as expected", () => { - let reorgDetection = mock(scannedHashesFixture, ~confirmedBlockThreshold=200) + let reorgDetection = mock(scannedHashesFixture, ~maxReorgDepth=200) Assert.deepEqual( - reorgDetection->LastBlockScannedHashes.rollbackToValidBlockNumber(~blockNumber=500), + reorgDetection->ReorgDetection.rollbackToValidBlockNumber(~blockNumber=500), reorgDetection, ~message="Shouldn't prune anything when the latest block number is the valid one", ) Assert.deepEqual( - reorgDetection->LastBlockScannedHashes.rollbackToValidBlockNumber(~blockNumber=499), - mock([(1, "0x123"), (50, "0x456"), (300, "0x789")], ~confirmedBlockThreshold=200), + reorgDetection->ReorgDetection.rollbackToValidBlockNumber(~blockNumber=499), + mock([(1, "0x123"), (50, "0x456"), (300, "0x789")], ~maxReorgDepth=200), ~message="Shouldn't prune blocks outside of the threshold. Would be nice, but it doesn't matter", ) }) @@ -452,10 +429,7 @@ describe("Validate reorg detection functions", () => { ) Assert.deepEqual( - mock( - scannedHashesFixture, - ~confirmedBlockThreshold=500, - )->LastBlockScannedHashes.getLatestValidScannedBlock( + mock(scannedHashesFixture, ~maxReorgDepth=500)->ReorgDetection.getLatestValidScannedBlock( ~blockNumbersAndHashes, ~currentBlockHeight=500, ), @@ -467,10 +441,7 @@ describe("Validate reorg detection functions", () => { ~message="Should return the latest non-different block if we assume that all blocks are in the threshold", ) Assert.deepEqual( - mock( - scannedHashesFixture, - ~confirmedBlockThreshold=200, - )->LastBlockScannedHashes.getLatestValidScannedBlock( + mock(scannedHashesFixture, ~maxReorgDepth=200)->ReorgDetection.getLatestValidScannedBlock( ~blockNumbersAndHashes, ~currentBlockHeight=500, ), @@ -491,10 +462,7 @@ describe("Validate reorg detection functions", () => { }, ) Assert.deepEqual( - mock( - scannedHashesFixture, - ~confirmedBlockThreshold=500, - )->LastBlockScannedHashes.getLatestValidScannedBlock( + mock(scannedHashesFixture, ~maxReorgDepth=500)->ReorgDetection.getLatestValidScannedBlock( ~blockNumbersAndHashes, ~currentBlockHeight=500, ), @@ -506,10 +474,7 @@ describe("Validate reorg detection functions", () => { ~message="Case when the different block is in between of valid ones", ) Assert.deepEqual( - mock( - scannedHashesFixture, - ~confirmedBlockThreshold=200, - )->LastBlockScannedHashes.getLatestValidScannedBlock( + mock(scannedHashesFixture, ~maxReorgDepth=200)->ReorgDetection.getLatestValidScannedBlock( ~blockNumbersAndHashes, ~currentBlockHeight=500, ), @@ -517,10 +482,7 @@ describe("Validate reorg detection functions", () => { ~message="Returns Error(NotFound) if the different block is the last one in the threshold", ) Assert.deepEqual( - mock( - scannedHashesFixture, - ~confirmedBlockThreshold=200, - )->LastBlockScannedHashes.getLatestValidScannedBlock( + mock(scannedHashesFixture, ~maxReorgDepth=200)->ReorgDetection.getLatestValidScannedBlock( ~blockNumbersAndHashes, ~currentBlockHeight=501, ), diff --git a/scenarios/test_codegen/test/__mocks__/DbStub.res b/scenarios/test_codegen/test/__mocks__/DbStub.res deleted file mode 100644 index 7d67621c5..000000000 --- a/scenarios/test_codegen/test/__mocks__/DbStub.res +++ /dev/null @@ -1,35 +0,0 @@ -let deleteDictKey = (_dict: dict<'a>, _key: string) => %raw(`delete _dict[_key]`) - -let databaseDict: dict = Js.Dict.empty() - -let getGravatarDb = (~id: string) => { - Js.Dict.get(databaseDict, id) -} - -let setGravatarDb = (~gravatar: Entities.Gravatar.t) => { - Js.Dict.set(databaseDict, gravatar.id, gravatar) -} - -let batchSetGravatar = (batch: array) => { - batch - ->Belt.Array.forEach(entity => { - setGravatarDb(~gravatar=entity) - }) - ->Promise.resolve -} - -let batchDeleteGravatar = (batch: array) => { - batch - ->Belt.Array.forEach(entity => { - deleteDictKey(databaseDict, entity.id) - }) - ->Promise.resolve -} - -let readGravatarEntities = (entityReads: array): promise> => { - entityReads - ->Belt.Array.keepMap(id => { - getGravatarDb(~id)->Belt.Option.map(gravatar => gravatar) - }) - ->Promise.resolve -} diff --git a/scenarios/test_codegen/test/__mocks__/MockConfig.res b/scenarios/test_codegen/test/__mocks__/MockConfig.res index b1316c884..9f583ed8b 100644 --- a/scenarios/test_codegen/test/__mocks__/MockConfig.res +++ b/scenarios/test_codegen/test/__mocks__/MockConfig.res @@ -40,7 +40,7 @@ let evmContracts = contracts->Js.Array2.map((contract): Internal.evmContractConf let mockChainConfig: InternalConfig.chain = { id: 1337, - confirmedBlockThreshold: 200, + maxReorgDepth: 200, startBlock: 1, contracts, sources: [ diff --git a/scenarios/test_codegen/test/helpers/Mock.res b/scenarios/test_codegen/test/helpers/Mock.res index 9aa7c7fc4..05205c633 100644 --- a/scenarios/test_codegen/test/helpers/Mock.res +++ b/scenarios/test_codegen/test/helpers/Mock.res @@ -200,6 +200,8 @@ module Storage = { cleanRun: false, cache: Js.Dict.empty(), chains: [], + reorgCheckpoints: [], + checkpointId: 0, }), } } @@ -222,7 +224,11 @@ module Indexer = { type chainConfig = {chain: Types.chain, sources: array, startBlock?: int} - let make = async (~chains: array, ~multichain=InternalConfig.Unordered) => { + let make = async ( + ~chains: array, + ~multichain=InternalConfig.Unordered, + ~reset=true, + ) => { DbHelpers.resetPostgresClient() // TODO: Should stop using global client PromClient.defaultRegister->PromClient.resetMetrics @@ -279,7 +285,7 @@ module Indexer = { await config.persistence->Persistence.init( ~chainConfigs=config.chainMap->ChainMap.values, - ~reset=true, + ~reset, ) let chainManager = await ChainManager.makeFromDbState( @@ -311,8 +317,8 @@ module Indexer = { Promise.makeAsync(async (resolve, _reject) => { while ( switch (gsManager->GlobalStateManager.getState).rollbackState { - | RollbackInMemStore(_) => false - | RollingBack(_) + | PreparedRollback(_) => false + | PreparingRollback(_) | NoRollback => true } ) { @@ -389,6 +395,7 @@ module Source = { resolveGetItemsOrThrow: ( array, ~latestFetchedBlockNumber: int=?, + ~currentBlockHeight: int=?, ~prevRangeLastBlock: ReorgDetection.blockData=?, ) => unit, rejectGetItemsOrThrow: 'exn. 'exn => unit, @@ -426,12 +433,18 @@ module Source = { getHeightOrThrowRejectFns->Array.forEach(reject => reject(exn->Obj.magic)) }, getItemsOrThrowCalls, - resolveGetItemsOrThrow: (items, ~latestFetchedBlockNumber=?, ~prevRangeLastBlock=?) => { + resolveGetItemsOrThrow: ( + items, + ~latestFetchedBlockNumber=?, + ~currentBlockHeight=?, + ~prevRangeLastBlock=?, + ) => { getItemsOrThrowResolveFns->Array.forEach(resolve => resolve({ "items": items, "latestFetchedBlockNumber": latestFetchedBlockNumber, "prevRangeLastBlock": prevRangeLastBlock, + "currentBlockHeight": currentBlockHeight, }) ) }, @@ -490,7 +503,9 @@ module Source = { ) resolve({ - Source.currentBlockHeight, + Source.currentBlockHeight: data["currentBlockHeight"]->Option.getWithDefault( + currentBlockHeight, + ), reorgGuard: { rangeLastBlock: { blockNumber: latestFetchedBlockNumber, diff --git a/scenarios/test_codegen/test/lib_tests/FetchState_test.res b/scenarios/test_codegen/test/lib_tests/FetchState_test.res index 3bbf797c9..000d2da0e 100644 --- a/scenarios/test_codegen/test/lib_tests/FetchState_test.res +++ b/scenarios/test_codegen/test/lib_tests/FetchState_test.res @@ -2362,53 +2362,51 @@ describe("FetchState unit tests for specific cases", () => { ) }) -describe("FetchState.filterAndSortForUnorderedBatch", () => { - it( - "Filters out states without eligible items and sorts by earliest timestamp (public API)", - () => { - let mk = () => makeInitial() - let mkQuery = (fetchState: FetchState.t) => { - { - FetchState.partitionId: "0", - target: Head, - selection: fetchState.normalSelection, - addressesByContractName: Js.Dict.empty(), - fromBlock: 0, - indexingContracts: fetchState.indexingContracts, - } +describe("FetchState.sortForUnorderedBatch", () => { + it("Sorts by earliest timestamp. Chains without eligible items should go last", () => { + let mk = () => makeInitial() + let mkQuery = (fetchState: FetchState.t) => { + { + FetchState.partitionId: "0", + target: Head, + selection: fetchState.normalSelection, + addressesByContractName: Js.Dict.empty(), + fromBlock: 0, + indexingContracts: fetchState.indexingContracts, } + } - // Helper: create a fetch state with desired latestFetchedBlock and queue items via public API - let makeFsWith = (~latestBlock: int, ~queueBlocks: array): FetchState.t => { - let fs0 = mk() - let query = mkQuery(fs0) - fs0 - ->FetchState.handleQueryResult( - ~query, - ~latestFetchedBlock={blockNumber: latestBlock, blockTimestamp: latestBlock}, - ~newItems=queueBlocks->Array.map(b => mockEvent(~blockNumber=b)), - ) - ->Result.getExn - } + // Helper: create a fetch state with desired latestFetchedBlock and queue items via public API + let makeFsWith = (~latestBlock: int, ~queueBlocks: array): FetchState.t => { + let fs0 = mk() + let query = mkQuery(fs0) + fs0 + ->FetchState.handleQueryResult( + ~query, + ~latestFetchedBlock={blockNumber: latestBlock, blockTimestamp: latestBlock}, + ~newItems=queueBlocks->Array.map(b => mockEvent(~blockNumber=b)), + ) + ->Result.getExn + } - // Included: last queue item at block 1, latestFullyFetchedBlock = 10 - let fsEarly = makeFsWith(~latestBlock=10, ~queueBlocks=[2, 1]) - // Included: last queue item at block 5, latestFullyFetchedBlock = 10 - let fsLate = makeFsWith(~latestBlock=10, ~queueBlocks=[5]) - // Excluded: last queue item at block 11 (> latestFullyFetchedBlock = 10) - let fsExcluded = makeFsWith(~latestBlock=10, ~queueBlocks=[11]) + // Included: last queue item at block 1, latestFullyFetchedBlock = 10 + let fsEarly = makeFsWith(~latestBlock=10, ~queueBlocks=[2, 1]) + // Included: last queue item at block 5, latestFullyFetchedBlock = 10 + let fsLate = makeFsWith(~latestBlock=10, ~queueBlocks=[5]) + // Excluded: last queue item at block 11 (> latestFullyFetchedBlock = 10) + // UPD: Starting from 2.30.1+ it should go last instead of filtered + let fsExcluded = makeFsWith(~latestBlock=10, ~queueBlocks=[11]) - let prepared = FetchState.filterAndSortForUnorderedBatch( - [fsLate, fsExcluded, fsEarly], - ~batchSizeTarget=3, - ) + let prepared = FetchState.sortForUnorderedBatch( + [fsLate, fsExcluded, fsEarly], + ~batchSizeTarget=3, + ) - Assert.deepEqual( - prepared->Array.map(fs => fs.buffer->Belt.Array.getUnsafe(0)->Internal.getItemBlockNumber), - [1, 5], - ) - }, - ) + Assert.deepEqual( + prepared->Array.map(fs => fs.buffer->Belt.Array.getUnsafe(0)->Internal.getItemBlockNumber), + [1, 5, 11], + ) + }) it("Prioritizes full batches over half full ones", () => { let mk = () => makeInitial() @@ -2440,7 +2438,7 @@ describe("FetchState.filterAndSortForUnorderedBatch", () => { // Half-full batch (1 item) but earlier earliest item (block 1) let fsHalfEarlier = makeFsWith(~latestBlock=10, ~queueBlocks=[1]) - let prepared = FetchState.filterAndSortForUnorderedBatch( + let prepared = FetchState.sortForUnorderedBatch( [fsHalfEarlier, fsFullLater], ~batchSizeTarget=2, ) @@ -2481,7 +2479,7 @@ describe("FetchState.filterAndSortForUnorderedBatch", () => { // Half-full (1 item) but earlier earliest item let fsHalfEarlier = makeFsWith(~latestBlock=10, ~queueBlocks=[1]) - let prepared = FetchState.filterAndSortForUnorderedBatch( + let prepared = FetchState.sortForUnorderedBatch( [fsHalfEarlier, fsExactFull], ~batchSizeTarget=2, ) @@ -2771,7 +2769,7 @@ describe("FetchState progress tracking", () => { let fetchStateEmpty = makeFetchStateWith(~latestBlock=100, ~queueBlocks=[]) Assert.equal( - fetchStateEmpty->FetchState.getProgressBlockNumber, + fetchStateEmpty->FetchState.getUnorderedMultichainProgressBlockNumberAt(~index=0), 100, ~message="Should return latestFullyFetchedBlock.blockNumber when queue is empty", ) @@ -2781,7 +2779,7 @@ describe("FetchState progress tracking", () => { let fetchStateSingleItem = makeFetchStateWith(~latestBlock=55, ~queueBlocks=[(55, 0)]) Assert.equal( - fetchStateSingleItem->FetchState.getProgressBlockNumber, + fetchStateSingleItem->FetchState.getUnorderedMultichainProgressBlockNumberAt(~index=0), 54, ~message="Should return single queue item blockNumber - 1", ) @@ -2791,7 +2789,7 @@ describe("FetchState progress tracking", () => { let fetchStateSingleItem = makeFetchStateWith(~latestBlock=55, ~queueBlocks=[(55, 5)]) Assert.equal( - fetchStateSingleItem->FetchState.getProgressBlockNumber, + fetchStateSingleItem->FetchState.getUnorderedMultichainProgressBlockNumberAt(~index=0), 54, ~message="Should return single queue item blockNumber - 1", ) @@ -2804,7 +2802,7 @@ describe("FetchState progress tracking", () => { ) Assert.equal( - fetchStateWithQueue->FetchState.getProgressBlockNumber, + fetchStateWithQueue->FetchState.getUnorderedMultichainProgressBlockNumberAt(~index=0), 90, ~message="Should return latest fetched block number", ) diff --git a/scenarios/test_codegen/test/lib_tests/Persistence_test.res b/scenarios/test_codegen/test/lib_tests/Persistence_test.res index 6159565b1..d698036b8 100644 --- a/scenarios/test_codegen/test/lib_tests/Persistence_test.res +++ b/scenarios/test_codegen/test/lib_tests/Persistence_test.res @@ -78,6 +78,8 @@ describe("Test Persistence layer init", () => { cleanRun: true, chains: [], cache: Js.Dict.empty(), + reorgCheckpoints: [], + checkpointId: 0, } storageMock.resolveInitialize(initialState) let _ = await Promise.resolve() @@ -142,6 +144,8 @@ describe("Test Persistence layer init", () => { cleanRun: false, chains: [], cache: Js.Dict.empty(), + reorgCheckpoints: [], + checkpointId: 0, } storageMock.resolveLoadInitialState(initialState) let _ = await Promise.resolve() diff --git a/scenarios/test_codegen/test/lib_tests/PgStorage_test.res b/scenarios/test_codegen/test/lib_tests/PgStorage_test.res index 4fddda1c3..ec74b29c3 100644 --- a/scenarios/test_codegen/test/lib_tests/PgStorage_test.res +++ b/scenarios/test_codegen/test/lib_tests/PgStorage_test.res @@ -115,14 +115,14 @@ describe("Test PgStorage SQL generation functions", () => { id: 1, startBlock: 100, endBlock: 200, - confirmedBlockThreshold: 10, + maxReorgDepth: 10, contracts: [], sources: [], }, { id: 137, startBlock: 0, - confirmedBlockThreshold: 200, + maxReorgDepth: 200, contracts: [], sources: [], }, @@ -144,9 +144,9 @@ CREATE SCHEMA "test_schema"; GRANT ALL ON SCHEMA "test_schema" TO "postgres"; GRANT ALL ON SCHEMA "test_schema" TO public; CREATE TYPE "test_schema".ENTITY_TYPE AS ENUM('A', 'B', 'C', 'CustomSelectionTestPass', 'D', 'EntityWithAllNonArrayTypes', 'EntityWithAllTypes', 'EntityWithBigDecimal', 'EntityWithTimestamp', 'Gravatar', 'NftCollection', 'PostgresNumericPrecisionEntityTester', 'SimpleEntity', 'Token', 'User', 'dynamic_contract_registry'); -CREATE TABLE IF NOT EXISTS "test_schema"."envio_chains"("id" INTEGER NOT NULL, "start_block" INTEGER NOT NULL, "end_block" INTEGER, "buffer_block" INTEGER NOT NULL, "source_block" INTEGER NOT NULL, "first_event_block" INTEGER, "ready_at" TIMESTAMP WITH TIME ZONE NULL, "events_processed" INTEGER NOT NULL, "_is_hyper_sync" BOOLEAN NOT NULL, "progress_block" INTEGER NOT NULL, "_num_batches_fetched" INTEGER NOT NULL, PRIMARY KEY("id")); +CREATE TABLE IF NOT EXISTS "test_schema"."envio_chains"("id" INTEGER NOT NULL, "start_block" INTEGER NOT NULL, "end_block" INTEGER, "max_reorg_depth" INTEGER NOT NULL, "buffer_block" INTEGER NOT NULL, "source_block" INTEGER NOT NULL, "first_event_block" INTEGER, "ready_at" TIMESTAMP WITH TIME ZONE NULL, "events_processed" INTEGER NOT NULL, "_is_hyper_sync" BOOLEAN NOT NULL, "progress_block" INTEGER NOT NULL, "_num_batches_fetched" INTEGER NOT NULL, PRIMARY KEY("id")); CREATE TABLE IF NOT EXISTS "test_schema"."persisted_state"("id" SERIAL NOT NULL, "envio_version" TEXT NOT NULL, "config_hash" TEXT NOT NULL, "schema_hash" TEXT NOT NULL, "handler_files_hash" TEXT NOT NULL, "abi_files_hash" TEXT NOT NULL, PRIMARY KEY("id")); -CREATE TABLE IF NOT EXISTS "test_schema"."end_of_block_range_scanned_data"("chain_id" INTEGER NOT NULL, "block_number" INTEGER NOT NULL, "block_hash" TEXT NOT NULL, PRIMARY KEY("chain_id", "block_number")); +CREATE TABLE IF NOT EXISTS "test_schema"."envio_checkpoints"("id" INTEGER NOT NULL, "chain_id" INTEGER NOT NULL, "block_number" INTEGER NOT NULL, "block_hash" TEXT, "events_processed" INTEGER NOT NULL, PRIMARY KEY("id")); CREATE TABLE IF NOT EXISTS "test_schema"."raw_events"("chain_id" INTEGER NOT NULL, "event_id" NUMERIC NOT NULL, "event_name" TEXT NOT NULL, "contract_name" TEXT NOT NULL, "block_number" INTEGER NOT NULL, "log_index" INTEGER NOT NULL, "src_address" TEXT NOT NULL, "block_hash" TEXT NOT NULL, "block_timestamp" INTEGER NOT NULL, "block_fields" JSONB NOT NULL, "transaction_fields" JSONB NOT NULL, "params" JSONB NOT NULL, "serial" SERIAL, PRIMARY KEY("serial")); CREATE TABLE IF NOT EXISTS "test_schema"."A"("b_id" TEXT NOT NULL, "id" TEXT NOT NULL, "optionalStringToTestLinkedEntities" TEXT, PRIMARY KEY("id")); CREATE TABLE IF NOT EXISTS "test_schema"."A_history"("entity_history_block_timestamp" INTEGER NOT NULL, "entity_history_chain_id" INTEGER NOT NULL, "entity_history_block_number" INTEGER NOT NULL, "entity_history_log_index" INTEGER NOT NULL, "previous_entity_history_block_timestamp" INTEGER, "previous_entity_history_chain_id" INTEGER, "previous_entity_history_block_number" INTEGER, "previous_entity_history_log_index" INTEGER, "b_id" TEXT, "id" TEXT NOT NULL, "optionalStringToTestLinkedEntities" TEXT, "action" "test_schema".ENTITY_HISTORY_ROW_ACTION NOT NULL, "serial" SERIAL, PRIMARY KEY("entity_history_block_timestamp", "entity_history_chain_id", "entity_history_block_number", "entity_history_log_index", "id")); @@ -184,9 +184,9 @@ CREATE VIEW "test_schema"."chain_metadata" AS "start_block" AS "start_block", "ready_at" AS "timestamp_caught_up_to_head_or_endblock" FROM "test_schema"."envio_chains"; -INSERT INTO "test_schema"."envio_chains" ("id", "start_block", "end_block", "source_block", "first_event_block", "buffer_block", "progress_block", "ready_at", "events_processed", "_is_hyper_sync", "_num_batches_fetched") -VALUES (1, 100, 200, 0, NULL, -1, -1, NULL, 0, false, 0), - (137, 0, NULL, 0, NULL, -1, -1, NULL, 0, false, 0);` +INSERT INTO "test_schema"."envio_chains" ("id", "start_block", "end_block", "max_reorg_depth", "source_block", "first_event_block", "buffer_block", "progress_block", "ready_at", "events_processed", "_is_hyper_sync", "_num_batches_fetched") +VALUES (1, 100, 200, 10, 0, NULL, -1, -1, NULL, 0, false, 0), + (137, 0, NULL, 200, 0, NULL, -1, -1, NULL, 0, false, 0);` Assert.equal( mainQuery, @@ -229,9 +229,9 @@ VALUES (1, 100, 200, 0, NULL, -1, -1, NULL, 0, false, 0), CREATE SCHEMA "test_schema"; GRANT ALL ON SCHEMA "test_schema" TO "postgres"; GRANT ALL ON SCHEMA "test_schema" TO public; -CREATE TABLE IF NOT EXISTS "test_schema"."envio_chains"("id" INTEGER NOT NULL, "start_block" INTEGER NOT NULL, "end_block" INTEGER, "buffer_block" INTEGER NOT NULL, "source_block" INTEGER NOT NULL, "first_event_block" INTEGER, "ready_at" TIMESTAMP WITH TIME ZONE NULL, "events_processed" INTEGER NOT NULL, "_is_hyper_sync" BOOLEAN NOT NULL, "progress_block" INTEGER NOT NULL, "_num_batches_fetched" INTEGER NOT NULL, PRIMARY KEY("id")); +CREATE TABLE IF NOT EXISTS "test_schema"."envio_chains"("id" INTEGER NOT NULL, "start_block" INTEGER NOT NULL, "end_block" INTEGER, "max_reorg_depth" INTEGER NOT NULL, "buffer_block" INTEGER NOT NULL, "source_block" INTEGER NOT NULL, "first_event_block" INTEGER, "ready_at" TIMESTAMP WITH TIME ZONE NULL, "events_processed" INTEGER NOT NULL, "_is_hyper_sync" BOOLEAN NOT NULL, "progress_block" INTEGER NOT NULL, "_num_batches_fetched" INTEGER NOT NULL, PRIMARY KEY("id")); CREATE TABLE IF NOT EXISTS "test_schema"."persisted_state"("id" SERIAL NOT NULL, "envio_version" TEXT NOT NULL, "config_hash" TEXT NOT NULL, "schema_hash" TEXT NOT NULL, "handler_files_hash" TEXT NOT NULL, "abi_files_hash" TEXT NOT NULL, PRIMARY KEY("id")); -CREATE TABLE IF NOT EXISTS "test_schema"."end_of_block_range_scanned_data"("chain_id" INTEGER NOT NULL, "block_number" INTEGER NOT NULL, "block_hash" TEXT NOT NULL, PRIMARY KEY("chain_id", "block_number")); +CREATE TABLE IF NOT EXISTS "test_schema"."envio_checkpoints"("id" INTEGER NOT NULL, "chain_id" INTEGER NOT NULL, "block_number" INTEGER NOT NULL, "block_hash" TEXT, "events_processed" INTEGER NOT NULL, PRIMARY KEY("id")); CREATE TABLE IF NOT EXISTS "test_schema"."raw_events"("chain_id" INTEGER NOT NULL, "event_id" NUMERIC NOT NULL, "event_name" TEXT NOT NULL, "contract_name" TEXT NOT NULL, "block_number" INTEGER NOT NULL, "log_index" INTEGER NOT NULL, "src_address" TEXT NOT NULL, "block_hash" TEXT NOT NULL, "block_timestamp" INTEGER NOT NULL, "block_fields" JSONB NOT NULL, "transaction_fields" JSONB NOT NULL, "params" JSONB NOT NULL, "serial" SERIAL, PRIMARY KEY("serial")); CREATE VIEW "test_schema"."_meta" AS SELECT @@ -311,9 +311,9 @@ $$ LANGUAGE plpgsql;`, CREATE SCHEMA "public"; GRANT ALL ON SCHEMA "public" TO "postgres"; GRANT ALL ON SCHEMA "public" TO public; -CREATE TABLE IF NOT EXISTS "public"."envio_chains"("id" INTEGER NOT NULL, "start_block" INTEGER NOT NULL, "end_block" INTEGER, "buffer_block" INTEGER NOT NULL, "source_block" INTEGER NOT NULL, "first_event_block" INTEGER, "ready_at" TIMESTAMP WITH TIME ZONE NULL, "events_processed" INTEGER NOT NULL, "_is_hyper_sync" BOOLEAN NOT NULL, "progress_block" INTEGER NOT NULL, "_num_batches_fetched" INTEGER NOT NULL, PRIMARY KEY("id")); +CREATE TABLE IF NOT EXISTS "public"."envio_chains"("id" INTEGER NOT NULL, "start_block" INTEGER NOT NULL, "end_block" INTEGER, "max_reorg_depth" INTEGER NOT NULL, "buffer_block" INTEGER NOT NULL, "source_block" INTEGER NOT NULL, "first_event_block" INTEGER, "ready_at" TIMESTAMP WITH TIME ZONE NULL, "events_processed" INTEGER NOT NULL, "_is_hyper_sync" BOOLEAN NOT NULL, "progress_block" INTEGER NOT NULL, "_num_batches_fetched" INTEGER NOT NULL, PRIMARY KEY("id")); CREATE TABLE IF NOT EXISTS "public"."persisted_state"("id" SERIAL NOT NULL, "envio_version" TEXT NOT NULL, "config_hash" TEXT NOT NULL, "schema_hash" TEXT NOT NULL, "handler_files_hash" TEXT NOT NULL, "abi_files_hash" TEXT NOT NULL, PRIMARY KEY("id")); -CREATE TABLE IF NOT EXISTS "public"."end_of_block_range_scanned_data"("chain_id" INTEGER NOT NULL, "block_number" INTEGER NOT NULL, "block_hash" TEXT NOT NULL, PRIMARY KEY("chain_id", "block_number")); +CREATE TABLE IF NOT EXISTS "public"."envio_checkpoints"("id" INTEGER NOT NULL, "chain_id" INTEGER NOT NULL, "block_number" INTEGER NOT NULL, "block_hash" TEXT, "events_processed" INTEGER NOT NULL, PRIMARY KEY("id")); CREATE TABLE IF NOT EXISTS "public"."raw_events"("chain_id" INTEGER NOT NULL, "event_id" NUMERIC NOT NULL, "event_name" TEXT NOT NULL, "contract_name" TEXT NOT NULL, "block_number" INTEGER NOT NULL, "log_index" INTEGER NOT NULL, "src_address" TEXT NOT NULL, "block_hash" TEXT NOT NULL, "block_timestamp" INTEGER NOT NULL, "block_fields" JSONB NOT NULL, "transaction_fields" JSONB NOT NULL, "params" JSONB NOT NULL, "serial" SERIAL, PRIMARY KEY("serial")); CREATE TABLE IF NOT EXISTS "public"."A"("b_id" TEXT NOT NULL, "id" TEXT NOT NULL, "optionalStringToTestLinkedEntities" TEXT, PRIMARY KEY("id")); CREATE TABLE IF NOT EXISTS "public"."A_history"("entity_history_block_timestamp" INTEGER NOT NULL, "entity_history_chain_id" INTEGER NOT NULL, "entity_history_block_number" INTEGER NOT NULL, "entity_history_log_index" INTEGER NOT NULL, "previous_entity_history_block_timestamp" INTEGER, "previous_entity_history_chain_id" INTEGER, "previous_entity_history_block_number" INTEGER, "previous_entity_history_log_index" INTEGER, "b_id" TEXT, "id" TEXT NOT NULL, "optionalStringToTestLinkedEntities" TEXT, "action" "public".ENTITY_HISTORY_ROW_ACTION NOT NULL, "serial" SERIAL, PRIMARY KEY("entity_history_block_timestamp", "entity_history_chain_id", "entity_history_block_number", "entity_history_log_index", "id")); @@ -544,6 +544,41 @@ WHERE "id" = $1;` ) }) + describe("InternalTable.Checkpoints.makeGetReorgCheckpointsQuery", () => { + Async.it( + "Should generate optimized SQL query with CTE", + async () => { + let query = InternalTable.Checkpoints.makeGetReorgCheckpointsQuery(~pgSchema="test_schema") + + // The query should use a CTE to pre-filter chains and compute safe_block + let expectedQuery = `WITH reorg_chains AS ( + SELECT + "id" as id, + "source_block" - "max_reorg_depth" AS safe_block + FROM "test_schema"."envio_chains" + WHERE "max_reorg_depth" > 0 + AND "progress_block" > "source_block" - "max_reorg_depth" +) +SELECT + cp."id", + cp."chain_id", + cp."block_number", + cp."block_hash" +FROM "test_schema"."envio_checkpoints" cp +INNER JOIN reorg_chains rc + ON cp."chain_id" = rc.id +WHERE cp."block_hash" IS NOT NULL + AND cp."block_number" > rc.safe_block;` + + Assert.equal( + query, + expectedQuery, + ~message="Should generate optimized CTE query filtering chains outside reorg threshold", + ) + }, + ) + }) + describe("InternalTable.Chains.makeInitialValuesQuery", () => { Async.it( "Should return empty string for empty chain configs", @@ -568,7 +603,7 @@ WHERE "id" = $1;` id: 1, startBlock: 100, endBlock: 200, - confirmedBlockThreshold: 5, + maxReorgDepth: 5, contracts: [], sources: [], } @@ -578,8 +613,8 @@ WHERE "id" = $1;` ~chainConfigs=[chainConfig], ) - let expectedQuery = `INSERT INTO "test_schema"."envio_chains" ("id", "start_block", "end_block", "source_block", "first_event_block", "buffer_block", "progress_block", "ready_at", "events_processed", "_is_hyper_sync", "_num_batches_fetched") -VALUES (1, 100, 200, 0, NULL, -1, -1, NULL, 0, false, 0);` + let expectedQuery = `INSERT INTO "test_schema"."envio_chains" ("id", "start_block", "end_block", "max_reorg_depth", "source_block", "first_event_block", "buffer_block", "progress_block", "ready_at", "events_processed", "_is_hyper_sync", "_num_batches_fetched") +VALUES (1, 100, 200, 5, 0, NULL, -1, -1, NULL, 0, false, 0);` Assert.equal( query, @@ -595,7 +630,7 @@ VALUES (1, 100, 200, 0, NULL, -1, -1, NULL, 0, false, 0);` let chainConfig: InternalConfig.chain = { id: 1, startBlock: 100, - confirmedBlockThreshold: 5, + maxReorgDepth: 5, contracts: [], sources: [], } @@ -605,8 +640,8 @@ VALUES (1, 100, 200, 0, NULL, -1, -1, NULL, 0, false, 0);` ~chainConfigs=[chainConfig], ) - let expectedQuery = `INSERT INTO "public"."envio_chains" ("id", "start_block", "end_block", "source_block", "first_event_block", "buffer_block", "progress_block", "ready_at", "events_processed", "_is_hyper_sync", "_num_batches_fetched") -VALUES (1, 100, NULL, 0, NULL, -1, -1, NULL, 0, false, 0);` + let expectedQuery = `INSERT INTO "public"."envio_chains" ("id", "start_block", "end_block", "max_reorg_depth", "source_block", "first_event_block", "buffer_block", "progress_block", "ready_at", "events_processed", "_is_hyper_sync", "_num_batches_fetched") +VALUES (1, 100, NULL, 5, 0, NULL, -1, -1, NULL, 0, false, 0);` Assert.equal( query, @@ -623,7 +658,7 @@ VALUES (1, 100, NULL, 0, NULL, -1, -1, NULL, 0, false, 0);` id: 1, startBlock: 100, endBlock: 200, - confirmedBlockThreshold: 5, + maxReorgDepth: 5, contracts: [], sources: [], } @@ -631,7 +666,7 @@ VALUES (1, 100, NULL, 0, NULL, -1, -1, NULL, 0, false, 0);` let chainConfig2: InternalConfig.chain = { id: 42, startBlock: 500, - confirmedBlockThreshold: 0, + maxReorgDepth: 0, contracts: [], sources: [], } @@ -641,9 +676,9 @@ VALUES (1, 100, NULL, 0, NULL, -1, -1, NULL, 0, false, 0);` ~chainConfigs=[chainConfig1, chainConfig2], ) - let expectedQuery = `INSERT INTO "production"."envio_chains" ("id", "start_block", "end_block", "source_block", "first_event_block", "buffer_block", "progress_block", "ready_at", "events_processed", "_is_hyper_sync", "_num_batches_fetched") -VALUES (1, 100, 200, 0, NULL, -1, -1, NULL, 0, false, 0), - (42, 500, NULL, 0, NULL, -1, -1, NULL, 0, false, 0);` + let expectedQuery = `INSERT INTO "production"."envio_chains" ("id", "start_block", "end_block", "max_reorg_depth", "source_block", "first_event_block", "buffer_block", "progress_block", "ready_at", "events_processed", "_is_hyper_sync", "_num_batches_fetched") +VALUES (1, 100, 200, 5, 0, NULL, -1, -1, NULL, 0, false, 0), + (42, 500, NULL, 0, 0, NULL, -1, -1, NULL, 0, false, 0);` Assert.equal( query, diff --git a/scenarios/test_codegen/test/rollback/Rollback_test.res b/scenarios/test_codegen/test/rollback/Rollback_test.res index 08d049694..8e274eaf8 100644 --- a/scenarios/test_codegen/test/rollback/Rollback_test.res +++ b/scenarios/test_codegen/test/rollback/Rollback_test.res @@ -165,24 +165,20 @@ describe("Single Chain Simple Rollback", () => { ) await dispatchAllTasksInitalChain() - let block2 = Mock.mockChainData->MockChainData.getBlock(~blockNumber=2)->Option.getUnsafe - Assert.deepEqual( - tasks.contents->Utils.getVariantsTags, - ["UpdateEndOfBlockRangeScannedData", "ProcessPartitionQueryResponse"], - ) - Assert.deepEqual( - tasks.contents->Js.Array2.unsafe_get(0), - UpdateEndOfBlockRangeScannedData({ - blockNumberThreshold: -198, - chain: MockConfig.chain1337, - nextEndOfBlockRangeScannedData: { - blockHash: block2.blockHash, - blockNumber: block2.blockNumber, - chainId: 1337, - }, - }), - ) + Assert.deepEqual(tasks.contents->Utils.getVariantsTags, ["ProcessPartitionQueryResponse"]) + // Assert.deepEqual( + // tasks.contents->Js.Array2.unsafe_get(0), + // UpdateEndOfBlockRangeScannedData({ + // blockNumberThreshold: -198, + // chain: MockConfig.chain1337, + // nextEndOfBlockRangeScannedData: { + // blockHash: block2.blockHash, + // blockNumber: block2.blockNumber, + // chainId: 1337, + // }, + // }), + // ) await dispatchAllTasksInitalChain() @@ -232,23 +228,19 @@ describe("Single Chain Simple Rollback", () => { await dispatchAllTasksInitalChain() - let block2 = Mock.mockChainData->MockChainData.getBlock(~blockNumber=2)->Option.getUnsafe - Assert.deepEqual( - tasks.contents->Utils.getVariantsTags, - ["UpdateEndOfBlockRangeScannedData", "ProcessPartitionQueryResponse"], - ) - Assert.deepEqual( - tasks.contents->Js.Array2.unsafe_get(0), - UpdateEndOfBlockRangeScannedData({ - blockNumberThreshold: -198, - chain: MockConfig.chain1337, - nextEndOfBlockRangeScannedData: { - blockHash: block2.blockHash, - blockNumber: block2.blockNumber, - chainId: 1337, - }, - }), - ) + Assert.deepEqual(tasks.contents->Utils.getVariantsTags, ["ProcessPartitionQueryResponse"]) + // Assert.deepEqual( + // tasks.contents->Js.Array2.unsafe_get(0), + // UpdateEndOfBlockRangeScannedData({ + // blockNumberThreshold: -198, + // chain: MockConfig.chain1337, + // nextEndOfBlockRangeScannedData: { + // blockHash: block2.blockHash, + // blockNumber: block2.blockNumber, + // chainId: 1337, + // }, + // }), + // ) await dispatchAllTasksInitalChain() @@ -323,27 +315,19 @@ describe("Single Chain Simple Rollback", () => { await dispatchAllTasksReorgChain() - let block2 = - Mock.mockChainDataReorg - ->MockChainData.getBlock(~blockNumber=2) - ->Option.getUnsafe - - Assert.deepEqual( - tasks.contents->Utils.getVariantsTags, - ["UpdateEndOfBlockRangeScannedData", "ProcessPartitionQueryResponse"], - ) - Assert.deepEqual( - tasks.contents->Js.Array2.unsafe_get(0), - GlobalState.UpdateEndOfBlockRangeScannedData({ - blockNumberThreshold: -198, - chain: MockConfig.chain1337, - nextEndOfBlockRangeScannedData: { - blockHash: block2.blockHash, - blockNumber: block2.blockNumber, - chainId: 1337, - }, - }), - ) + Assert.deepEqual(tasks.contents->Utils.getVariantsTags, ["ProcessPartitionQueryResponse"]) + // Assert.deepEqual( + // tasks.contents->Js.Array2.unsafe_get(0), + // GlobalState.UpdateEndOfBlockRangeScannedData({ + // blockNumberThreshold: -198, + // chain: MockConfig.chain1337, + // nextEndOfBlockRangeScannedData: { + // blockHash: block2.blockHash, + // blockNumber: block2.blockNumber, + // chainId: 1337, + // }, + // }), + // ) await dispatchAllTasksReorgChain() @@ -355,34 +339,28 @@ describe("Single Chain Simple Rollback", () => { await dispatchAllTasksReorgChain() - let block4 = - Mock.mockChainDataReorg - ->MockChainData.getBlock(~blockNumber=4) - ->Option.getUnsafe - Assert.deepEqual( tasks.contents->Utils.getVariantsTags, [ "NextQuery", - "UpdateEndOfBlockRangeScannedData", "ProcessPartitionQueryResponse", "UpdateChainMetaDataAndCheckForExit", "ProcessEventBatch", "PruneStaleEntityHistory", ], ) - Assert.deepEqual( - tasks.contents->Js.Array2.unsafe_get(1), - GlobalState.UpdateEndOfBlockRangeScannedData({ - blockNumberThreshold: -196, - chain: MockConfig.chain1337, - nextEndOfBlockRangeScannedData: { - blockHash: block4.blockHash, - blockNumber: block4.blockNumber, - chainId: 1337, - }, - }), - ) + // Assert.deepEqual( + // tasks.contents->Js.Array2.unsafe_get(1), + // GlobalState.UpdateEndOfBlockRangeScannedData({ + // blockNumberThreshold: -196, + // chain: MockConfig.chain1337, + // nextEndOfBlockRangeScannedData: { + // blockHash: block4.blockHash, + // blockNumber: block4.blockNumber, + // chainId: 1337, + // }, + // }), + // ) let expectedGravatars: array = [ { @@ -698,6 +676,103 @@ describe("E2E rollback tests", () => { ) } + Async.it("Should re-enter reorg threshold on restart", async () => { + let sourceMock1337 = M.Source.make( + [#getHeightOrThrow, #getItemsOrThrow, #getBlockHashes], + ~chain=#1337, + ) + let sourceMock100 = M.Source.make( + [#getHeightOrThrow, #getItemsOrThrow, #getBlockHashes], + ~chain=#100, + ) + let chains = [ + { + M.Indexer.chain: #1337, + sources: [sourceMock1337.source], + }, + { + M.Indexer.chain: #100, + sources: [sourceMock100.source], + }, + ] + let indexerMock = await M.Indexer.make(~chains) + await Utils.delay(0) + + let _ = await Promise.all2(( + M.Helper.initialEnterReorgThreshold(~sourceMock=sourceMock1337), + M.Helper.initialEnterReorgThreshold(~sourceMock=sourceMock100), + )) + + Assert.deepEqual( + sourceMock1337.getItemsOrThrowCalls->Utils.Array.last, + Some({ + "fromBlock": 101, + "toBlock": None, + "retry": 0, + }), + ~message="Should enter reorg threshold and request now to the latest block", + ) + sourceMock1337.resolveGetItemsOrThrow([], ~latestFetchedBlockNumber=110) + await indexerMock.getBatchWritePromise() + + Assert.deepEqual( + await indexerMock.metric("envio_reorg_threshold"), + [{value: "1", labels: Js.Dict.empty()}], + ) + + let indexerMock = await M.Indexer.make(~chains, ~reset=false) + + sourceMock1337.getHeightOrThrowCalls->Utils.Array.clearInPlace + sourceMock1337.getItemsOrThrowCalls->Utils.Array.clearInPlace + + await Utils.delay(0) + + Assert.deepEqual( + await indexerMock.metric("envio_reorg_threshold"), + [{value: "0", labels: Js.Dict.empty()}], + ) + + Assert.deepEqual( + sourceMock1337.getHeightOrThrowCalls->Array.length, + 1, + ~message="should have called getHeightOrThrow on restart", + ) + sourceMock1337.resolveGetHeightOrThrow(300) + await Utils.delay(0) + await Utils.delay(0) + + Assert.deepEqual( + sourceMock1337.getItemsOrThrowCalls->Utils.Array.last, + Some({ + "fromBlock": 111, + "toBlock": None, + "retry": 0, + }), + ~message="Should enter reorg threshold for the second time and request now to the latest block", + ) + sourceMock1337.resolveGetItemsOrThrow( + [], + ~latestFetchedBlockNumber=200, + ~currentBlockHeight=320, + ) + await indexerMock.getBatchWritePromise() + + Assert.deepEqual( + sourceMock1337.getItemsOrThrowCalls->Utils.Array.last, + Some({ + "fromBlock": 201, + "toBlock": None, + "retry": 0, + }), + ~message="Should enter reorg threshold for the second time and request now to the latest block", + ) + + Assert.deepEqual( + await indexerMock.metric("envio_reorg_threshold"), + [{value: "0", labels: Js.Dict.empty()}], + ) + }) + Async.it("Rollback of a single chain indexer", async () => { let sourceMock = M.Source.make( [#getHeightOrThrow, #getItemsOrThrow, #getBlockHashes],