diff --git a/codegenerator/cli/npm/envio/src/Batch.res b/codegenerator/cli/npm/envio/src/Batch.res index 59781e25b..85752f6ea 100644 --- a/codegenerator/cli/npm/envio/src/Batch.res +++ b/codegenerator/cli/npm/envio/src/Batch.res @@ -8,7 +8,6 @@ type chainAfterBatch = { progressBlockNumber: int, totalEventsProcessed: int, fetchState: FetchState.t, - dcsToStore: option>, isProgressAtHeadWhenBatchCreated: bool, } @@ -99,7 +98,6 @@ let getProgressedChainsById = { ~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, @@ -111,7 +109,6 @@ let getProgressedChainsById = { batchSize, progressBlockNumber: progressBlockNumberAfterBatch, totalEventsProcessed: chainBeforeBatch.totalEventsProcessed + batchSize, - dcsToStore, fetchState: fetchStateAfterBatch, isProgressAtHeadWhenBatchCreated: progressBlockNumberAfterBatch >= chainBeforeBatch.sourceBlockNumber, @@ -151,48 +148,17 @@ let getProgressedChainsById = { ) { | 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, - ) - } - } + getChainAfterBatchIfProgressed( + ~chainBeforeBatch, + ~batchSize, + ~fetchStateAfterBatch=fetchState->FetchState.updateInternal(~mutItems=leftItems), + ~progressBlockNumberAfterBatch, + ) // Skip not affected chains | None => getChainAfterBatchIfProgressed( ~chainBeforeBatch, ~batchSize=0, - ~dcsToStore=None, ~fetchStateAfterBatch=chainBeforeBatch.fetchState, ~progressBlockNumberAfterBatch, ) diff --git a/codegenerator/cli/npm/envio/src/EventRegister.res b/codegenerator/cli/npm/envio/src/EventRegister.res index 02fb4894d..78407c98b 100644 --- a/codegenerator/cli/npm/envio/src/EventRegister.res +++ b/codegenerator/cli/npm/envio/src/EventRegister.res @@ -1,4 +1,7 @@ -type registrations = {onBlockByChainId: dict>} +type registrations = { + onBlockByChainId: dict>, + mutable hasEvents: bool, +} type activeRegistration = { ecosystem: InternalConfig.ecosystem, @@ -40,6 +43,7 @@ let startRegistration = (~ecosystem, ~multichain, ~preloadHandlers) => { preloadHandlers, registrations: { onBlockByChainId: Js.Dict.empty(), + hasEvents: false, }, finished: false, } @@ -205,7 +209,8 @@ let setEventOptions = (t: t, ~eventOptions, ~logger=Logging.getLogger()) => { } let setHandler = (t: t, handler, ~eventOptions, ~logger=Logging.getLogger()) => { - withRegistration(_ => { + withRegistration(registration => { + registration.registrations.hasEvents = true switch t.handler { | None => t.handler = @@ -225,7 +230,8 @@ let setHandler = (t: t, handler, ~eventOptions, ~logger=Logging.getLogger()) => } let setContractRegister = (t: t, contractRegister, ~eventOptions, ~logger=Logging.getLogger()) => { - withRegistration(_ => { + withRegistration(registration => { + registration.registrations.hasEvents = true switch t.contractRegister { | None => t.contractRegister = Some( diff --git a/codegenerator/cli/npm/envio/src/EventRegister.resi b/codegenerator/cli/npm/envio/src/EventRegister.resi index 877c74bd8..330fa057e 100644 --- a/codegenerator/cli/npm/envio/src/EventRegister.resi +++ b/codegenerator/cli/npm/envio/src/EventRegister.resi @@ -1,4 +1,7 @@ -type registrations = {onBlockByChainId: dict>} +type registrations = { + onBlockByChainId: dict>, + mutable hasEvents: bool, +} let startRegistration: ( ~ecosystem: InternalConfig.ecosystem, diff --git a/codegenerator/cli/npm/envio/src/FetchState.res b/codegenerator/cli/npm/envio/src/FetchState.res index a6c7c621c..1b2d6ce52 100644 --- a/codegenerator/cli/npm/envio/src/FetchState.res +++ b/codegenerator/cli/npm/envio/src/FetchState.res @@ -1,24 +1,5 @@ open Belt -type dcData = { - registeringEventBlockTimestamp: int, - registeringEventLogIndex: int, - registeringEventContractName: string, - registeringEventName: string, - registeringEventSrcAddress: Address.t, -} - -@unboxed -type contractRegister = - | Config - | DC(dcData) -type indexingContract = { - address: Address.t, - contractName: string, - startBlock: int, - register: contractRegister, -} - type contractConfig = {filterByAddresses: bool} type blockNumberAndTimestamp = { @@ -56,12 +37,9 @@ type t = { maxAddrInPartition: int, normalSelection: selection, // By address - indexingContracts: dict, + indexingContracts: dict, // By contract name contractConfigs: dict, - // Registered dynamic contracts that need to be stored in the db - // Should read them at the same time when getting items for the batch - dcsToStore: array, // Not used for logic - only metadata chainId: int, // The block number of the latest block fetched @@ -204,7 +182,6 @@ let updateInternal = ( ~partitions=fetchState.partitions, ~nextPartitionIndex=fetchState.nextPartitionIndex, ~indexingContracts=fetchState.indexingContracts, - ~dcsToStore=fetchState.dcsToStore, ~mutItems=?, ~blockLag=fetchState.blockLag, ): t => { @@ -304,7 +281,6 @@ let updateInternal = ( latestOnBlockBlockNumber, latestFullyFetchedBlock, indexingContracts, - dcsToStore, blockLag, buffer: switch mutItemsRef.contents { // Theoretically it could be faster to asume that @@ -333,7 +309,11 @@ let updateInternal = ( let numAddresses = fetchState => fetchState.indexingContracts->Js.Dict.keys->Array.length -let warnDifferentContractType = (fetchState, ~existingContract, ~dc: indexingContract) => { +let warnDifferentContractType = ( + fetchState, + ~existingContract: Internal.indexingContract, + ~dc: Internal.indexingContract, +) => { let logger = Logging.createChild( ~params={ "chainId": fetchState.chainId, @@ -347,9 +327,9 @@ let warnDifferentContractType = (fetchState, ~existingContract, ~dc: indexingCon let registerDynamicContracts = ( fetchState: t, - // These are raw dynamic contracts received from contractRegister call. + // These are raw items which might have dynamic contracts received from contractRegister call. // Might contain duplicates which we should filter out - dynamicContracts: array, + items: array, ) => { if fetchState.normalSelection.eventConfigs->Utils.Array.isEmpty { // Can the normalSelection be empty? @@ -361,79 +341,83 @@ let registerDynamicContracts = ( } let indexingContracts = fetchState.indexingContracts - let registeringContracts = Js.Dict.empty() + let registeringContracts: dict = Js.Dict.empty() let addressesByContractName = Js.Dict.empty() let earliestRegisteringEventBlockNumber = ref(%raw(`Infinity`)) let hasDCWithFilterByAddresses = ref(false) - for idx in 0 to dynamicContracts->Array.length - 1 { - let dc = dynamicContracts->Js.Array2.unsafe_get(idx) - switch fetchState.contractConfigs->Utils.Dict.dangerouslyGetNonOption(dc.contractName) { - | Some({filterByAddresses}) => - // Prevent registering already indexing contracts - switch indexingContracts->Utils.Dict.dangerouslyGetNonOption(dc.address->Address.toString) { - | Some(existingContract) => - // FIXME: Instead of filtering out duplicates, - // we should check the block number first. - // If new registration with earlier block number - // we should register it for the missing block range - if existingContract.contractName != dc.contractName { - fetchState->warnDifferentContractType(~existingContract, ~dc) - } else if existingContract.startBlock > dc.startBlock { - let logger = Logging.createChild( - ~params={ - "chainId": fetchState.chainId, - "contractAddress": dc.address->Address.toString, - "existingBlockNumber": existingContract.startBlock, - "newBlockNumber": dc.startBlock, - }, - ) - logger->Logging.childWarn(`Skipping contract registration: Contract address is already registered at a later block number. Currently registration of the same contract address is not supported by Envio. Reach out to us if it's a problem for you.`) - } - () - | None => - let shouldUpdate = switch registeringContracts->Utils.Dict.dangerouslyGetNonOption( - dc.address->Address.toString, - ) { - | Some(registeringContract) if registeringContract.contractName != dc.contractName => - fetchState->warnDifferentContractType(~existingContract=registeringContract, ~dc) - false - | Some(registeringContract) => - switch (registeringContract.register, dc.register) { - | ( - DC({registeringEventLogIndex}), - DC({registeringEventLogIndex: newRegisteringEventLogIndex}), - ) => - // Update DC registration if the new one from the batch has an earlier registration log - registeringContract.startBlock > dc.startBlock || - (registeringContract.startBlock === dc.startBlock && - registeringEventLogIndex > newRegisteringEventLogIndex) - | (Config, _) | (_, Config) => - Js.Exn.raiseError( - "Unexpected case: Config registration should be handled in a different function", + for itemIdx in 0 to items->Array.length - 1 { + let item = items->Js.Array2.unsafe_get(itemIdx) + switch item->Internal.getItemDcs { + | None => () + | Some(dcs) => + for idx in 0 to dcs->Array.length - 1 { + let dc = dcs->Js.Array2.unsafe_get(idx) + + switch fetchState.contractConfigs->Utils.Dict.dangerouslyGetNonOption(dc.contractName) { + | Some({filterByAddresses}) => + // Prevent registering already indexing contracts + switch indexingContracts->Utils.Dict.dangerouslyGetNonOption( + dc.address->Address.toString, + ) { + | Some(existingContract) => + // FIXME: Instead of filtering out duplicates, + // we should check the block number first. + // If new registration with earlier block number + // we should register it for the missing block range + if existingContract.contractName != dc.contractName { + fetchState->warnDifferentContractType(~existingContract, ~dc) + } else if existingContract.startBlock > dc.startBlock { + let logger = Logging.createChild( + ~params={ + "chainId": fetchState.chainId, + "contractAddress": dc.address->Address.toString, + "existingBlockNumber": existingContract.startBlock, + "newBlockNumber": dc.startBlock, + }, + ) + logger->Logging.childWarn(`Skipping contract registration: Contract address is already registered at a later block number. Currently registration of the same contract address is not supported by Envio. Reach out to us if it's a problem for you.`) + } + // Remove the DC from item to prevent it from saving to the db + let _ = dcs->Js.Array2.removeCountInPlace(~count=1, ~pos=idx) + | None => + let shouldUpdate = switch registeringContracts->Utils.Dict.dangerouslyGetNonOption( + dc.address->Address.toString, + ) { + | Some(registeringContract) if registeringContract.contractName != dc.contractName => + fetchState->warnDifferentContractType(~existingContract=registeringContract, ~dc) + false + | Some(_) => // Since the DC is registered by an earlier item in the query + // FIXME: This unsafely relies on the asc order of the items + // which is 99% true, but there were cases when the source ordering was wrong + false + | None => + hasDCWithFilterByAddresses := hasDCWithFilterByAddresses.contents || filterByAddresses + addressesByContractName->Utils.Dict.push(dc.contractName, dc.address) + true + } + if shouldUpdate { + earliestRegisteringEventBlockNumber := + Pervasives.min(earliestRegisteringEventBlockNumber.contents, dc.startBlock) + registeringContracts->Js.Dict.set(dc.address->Address.toString, dc) + } else { + // Remove the DC from item to prevent it from saving to the db + let _ = dcs->Js.Array2.removeCountInPlace(~count=1, ~pos=idx) + } + } + | None => { + let logger = Logging.createChild( + ~params={ + "chainId": fetchState.chainId, + "contractAddress": dc.address->Address.toString, + "contractName": dc.contractName, + }, ) + logger->Logging.childWarn(`Skipping contract registration: Contract doesn't have any events to fetch.`) + let _ = dcs->Js.Array2.removeCountInPlace(~count=1, ~pos=idx) } - | None => - hasDCWithFilterByAddresses := hasDCWithFilterByAddresses.contents || filterByAddresses - addressesByContractName->Utils.Dict.push(dc.contractName, dc.address) - true - } - if shouldUpdate { - earliestRegisteringEventBlockNumber := - Pervasives.min(earliestRegisteringEventBlockNumber.contents, dc.startBlock) - registeringContracts->Js.Dict.set(dc.address->Address.toString, dc) } } - | None => { - let logger = Logging.createChild( - ~params={ - "chainId": fetchState.chainId, - "contractAddress": dc.address->Address.toString, - "contractName": dc.contractName, - }, - ) - logger->Logging.childWarn(`Skipping contract registration: Contract doesn't have any events to fetch.`) - } } } @@ -568,10 +552,6 @@ let registerDynamicContracts = ( fetchState->updateInternal( ~partitions=fetchState.partitions->Js.Array2.concat(newPartitions), - ~dcsToStore=switch fetchState.dcsToStore { - | [] => dcsToStore - | existingDcs => Array.concat(existingDcs, dcsToStore) - }, ~indexingContracts=// We don't need registeringContracts anymore, // so we can safely mixin indexingContracts in it // The original indexingContracts won't be mutated @@ -598,7 +578,7 @@ type query = { selection: selection, addressesByContractName: dict>, target: queryTarget, - indexingContracts: dict, + indexingContracts: dict, } exception UnexpectedPartitionNotFound({partitionId: string}) @@ -975,7 +955,7 @@ let make = ( ~startBlock, ~endBlock, ~eventConfigs: array, - ~contracts: array, + ~contracts: array, ~maxAddrInPartition, ~chainId, ~targetBufferSize, @@ -1104,7 +1084,6 @@ let make = ( latestOnBlockBlockNumber: progressBlockNumber, normalSelection, indexingContracts, - dcsToStore: [], blockLag, onBlockConfigs, targetBufferSize, @@ -1114,31 +1093,14 @@ let make = ( let bufferSize = ({buffer}: t) => buffer->Array.length -let pruneQueueFromFirstChangeEvent = ( - buffer: array, - ~firstChangeEvent: blockNumberAndLogIndex, -) => { - buffer->Array.keep(item => - switch item { - | Event({blockNumber, logIndex}) - | Block({blockNumber, logIndex}) => (blockNumber, logIndex) - } < - (firstChangeEvent.blockNumber, firstChangeEvent.logIndex) - ) -} - /** Rolls back partitions to the given valid block */ -let rollbackPartition = ( - p: partition, - ~firstChangeEvent: blockNumberAndLogIndex, - ~addressesToRemove, -) => { - let shouldRollbackFetched = p.latestFetchedBlock.blockNumber >= firstChangeEvent.blockNumber +let rollbackPartition = (p: partition, ~targetBlockNumber, ~addressesToRemove) => { + let shouldRollbackFetched = p.latestFetchedBlock.blockNumber > targetBlockNumber let latestFetchedBlock = shouldRollbackFetched ? { - blockNumber: firstChangeEvent.blockNumber - 1, + blockNumber: targetBlockNumber, blockTimestamp: 0, } : p.latestFetchedBlock @@ -1177,7 +1139,7 @@ let rollbackPartition = ( } } -let rollback = (fetchState: t, ~firstChangeEvent) => { +let rollback = (fetchState: t, ~targetBlockNumber) => { let addressesToRemove = Utils.Set.make() let indexingContracts = Js.Dict.empty() @@ -1185,40 +1147,34 @@ let rollback = (fetchState: t, ~firstChangeEvent) => { ->Js.Dict.keys ->Array.forEach(address => { let indexingContract = fetchState.indexingContracts->Js.Dict.unsafeGet(address) - if ( - switch indexingContract { - | {register: Config} => true - | {register: DC(dc)} => - indexingContract.startBlock < firstChangeEvent.blockNumber || - (indexingContract.startBlock === firstChangeEvent.blockNumber && - dc.registeringEventLogIndex < firstChangeEvent.logIndex) + switch indexingContract.registrationBlock { + | Some(registrationBlock) if registrationBlock > targetBlockNumber => { + //If the registration block is later than the first change event, + //Do not keep it and add to the removed addresses + let _ = addressesToRemove->Utils.Set.add(address->Address.unsafeFromString) } - ) { - indexingContracts->Js.Dict.set(address, indexingContract) - } else { - //If the registration block is later than the first change event, - //Do not keep it and add to the removed addresses - let _ = addressesToRemove->Utils.Set.add(address->Address.unsafeFromString) + | _ => indexingContracts->Js.Dict.set(address, indexingContract) } }) let partitions = fetchState.partitions->Array.keepMap(p => - p->rollbackPartition(~firstChangeEvent, ~addressesToRemove) + p->rollbackPartition(~targetBlockNumber, ~addressesToRemove) ) { ...fetchState, - latestOnBlockBlockNumber: firstChangeEvent.blockNumber - 1, // TODO: This is not tested + latestOnBlockBlockNumber: targetBlockNumber, // TODO: This is not tested. I assume there might be a possible issue of it skipping some blocks }->updateInternal( ~partitions, ~indexingContracts, - ~mutItems=fetchState.buffer->pruneQueueFromFirstChangeEvent(~firstChangeEvent), - ~dcsToStore=switch fetchState.dcsToStore { - | [] as empty => empty - | dcsToStore => - dcsToStore->Js.Array2.filter(dc => !(addressesToRemove->Utils.Set.has(dc.address))) - }, + ~mutItems=fetchState.buffer->Array.keep(item => + switch item { + | Event({blockNumber}) + | Block({blockNumber}) => blockNumber + } <= + targetBlockNumber + ), ) } diff --git a/codegenerator/cli/npm/envio/src/Internal.res b/codegenerator/cli/npm/envio/src/Internal.res index 855d1bc02..e47b27633 100644 --- a/codegenerator/cli/npm/envio/src/Internal.res +++ b/codegenerator/cli/npm/envio/src/Internal.res @@ -133,6 +133,18 @@ type evmContractConfig = { events: array, } +type indexingContract = { + address: Address.t, + contractName: string, + startBlock: int, + // Needed for rollback + // If not set, assume the contract comes from config + // and shouldn't be rolled back + registrationBlock: option, +} + +type dcs = array + // Duplicate the type from item // to make item properly unboxed type eventItem = private { @@ -188,6 +200,11 @@ external getItemBlockNumber: item => int = "blockNumber" @get external getItemLogIndex: item => int = "logIndex" +@get +external getItemDcs: item => option = "dcs" +@set +external setItemDcs: (item, dcs) => unit = "dcs" + @genType type eventOptions<'eventFilters> = { wildcard?: bool, @@ -296,3 +313,24 @@ type reorgCheckpoint = { @as("block_hash") blockHash: string, } + +type entityValueAtStartOfBatch<'entityType> = + | NotSet // The entity isn't in the DB yet + | AlreadySet('entityType) + +type updatedValue<'entityType> = { + latest: EntityHistory.entityUpdate<'entityType>, + history: array>, + // In the event of a rollback, some entity updates may have been + // been affected by a rollback diff. If there was no rollback diff + // this will always be false. + // If there was a rollback diff, this will be false in the case of a + // new entity update (where entity affected is not present in the diff) b + // but true if the update is related to an entity that is + // currently present in the diff + containsRollbackDiffChange: bool, +} + +type inMemoryStoreRowEntity<'entityType> = + | Updated(updatedValue<'entityType>) + | InitialReadFromDb(entityValueAtStartOfBatch<'entityType>) // This means there is no change from the db. diff --git a/codegenerator/cli/npm/envio/src/Persistence.res b/codegenerator/cli/npm/envio/src/Persistence.res index bd87052b9..91c070228 100644 --- a/codegenerator/cli/npm/envio/src/Persistence.res +++ b/codegenerator/cli/npm/envio/src/Persistence.res @@ -13,10 +13,22 @@ type effectCacheRecord = { mutable count: int, } +type initialChainState = { + id: int, + startBlock: int, + endBlock: option, + maxReorgDepth: int, + progressBlockNumber: int, + numEventsProcessed: int, + firstEventBlockNumber: option, + timestampCaughtUpToHeadOrEndblock: option, + dynamicContracts: array, +} + type initialState = { cleanRun: bool, cache: dict, - chains: array, + chains: array, checkpointId: int, // Needed to keep reorg detection logic between restarts reorgCheckpoints: array, diff --git a/codegenerator/cli/npm/envio/src/PgStorage.res b/codegenerator/cli/npm/envio/src/PgStorage.res index 2e21fc54a..8738b7ce7 100644 --- a/codegenerator/cli/npm/envio/src/PgStorage.res +++ b/codegenerator/cli/npm/envio/src/PgStorage.res @@ -116,13 +116,8 @@ GRANT ALL ON SCHEMA "${pgSchema}" TO public;`, } }) - let functionsQuery = ref("") - // Add derived indices entities->Js.Array2.forEach((entity: Internal.entityConfig) => { - functionsQuery := - functionsQuery.contents ++ "\n" ++ entity.entityHistory.makeInsertFnQuery(~pgSchema) - entity.table ->Table.getDerivedFromFields ->Js.Array2.forEach(derivedFromField => { @@ -149,10 +144,8 @@ GRANT ALL ON SCHEMA "${pgSchema}" TO public;`, | None => () } - // Add cache row count function - functionsQuery := - functionsQuery.contents ++ - "\n" ++ + [ + query.contents, `CREATE OR REPLACE FUNCTION ${getCacheRowCountFnName}(table_name text) RETURNS integer AS $$ DECLARE @@ -161,11 +154,8 @@ BEGIN EXECUTE format('SELECT COUNT(*) FROM "${pgSchema}".%I', table_name) INTO result; RETURN result; END; -$$ LANGUAGE plpgsql;` - - [query.contents]->Js.Array2.concat( - functionsQuery.contents !== "" ? [functionsQuery.contents] : [], - ) +$$ LANGUAGE plpgsql;`, + ] } let makeLoadByIdQuery = (~pgSchema, ~tableName) => { @@ -270,6 +260,11 @@ let makeTableBatchSetQuery = (~pgSchema, ~table: Table.table, ~itemSchema: S.t<' // FIXME what about Fuel params? let isRawEvents = table.tableName === InternalTable.RawEvents.table.tableName + // Currently history update table uses S.object with transformation for schema, + // which is being lossed during conversion to dbSchema. + // So use simple insert values for now. + let isHistoryUpdate = table.tableName->Js.String2.startsWith("envio_history_") + // Should experiment how much it'll affect performance // Although, it should be fine not to perform the validation check, // since the values are validated by type system. @@ -277,7 +272,7 @@ let makeTableBatchSetQuery = (~pgSchema, ~table: Table.table, ~itemSchema: S.t<' // db write fails to show a better user error. let typeValidation = false - if isRawEvents || !hasArrayField { + if (isRawEvents || !hasArrayField) && !isHistoryUpdate { { "query": makeInsertUnnestSetQuery(~pgSchema, ~table, ~itemSchema, ~isRawEvents), "convertOrThrow": S.compile( @@ -414,44 +409,6 @@ let setOrThrow = async (sql, ~items, ~table: Table.table, ~itemSchema, ~pgSchema } } -let setEntityHistoryOrThrow = ( - sql, - ~entityHistory: EntityHistory.t<'entity>, - ~rows: array>, - ~shouldCopyCurrentEntity=?, - ~shouldRemoveInvalidUtf8=false, -) => { - rows->Belt.Array.map(historyRow => { - let row = historyRow->S.reverseConvertToJsonOrThrow(entityHistory.schema) - if shouldRemoveInvalidUtf8 { - [row]->removeInvalidUtf8InPlace - } - entityHistory.insertFn( - sql, - row, - ~shouldCopyCurrentEntity=switch shouldCopyCurrentEntity { - | Some(v) => v - | None => { - let containsRollbackDiffChange = - historyRow.containsRollbackDiffChange->Belt.Option.getWithDefault(false) - !containsRollbackDiffChange - } - }, - )->Promise.catch(exn => { - let reason = exn->Utils.prettifyExn - let detail = %raw(`reason?.detail || ""`) - raise( - Persistence.StorageError({ - message: `Failed to insert history item into table "${entityHistory.table.tableName}".${detail !== "" - ? ` Details: ${detail}` - : ""}`, - reason, - }), - ) - }) - }) -} - type schemaTableName = { @as("table_name") tableName: string, @@ -708,7 +665,17 @@ let make = ( cleanRun: true, cache, reorgCheckpoints: [], - chains: chainConfigs->Js.Array2.map(InternalTable.Chains.initialFromConfig), + chains: chainConfigs->Js.Array2.map((chainConfig): Persistence.initialChainState => { + id: chainConfig.id, + startBlock: chainConfig.startBlock, + endBlock: chainConfig.endBlock, + maxReorgDepth: chainConfig.maxReorgDepth, + progressBlockNumber: -1, + numEventsProcessed: 0, + firstEventBlockNumber: None, + timestampCaughtUpToHeadOrEndblock: None, + dynamicContracts: [], + }), checkpointId: InternalTable.Checkpoints.initialCheckpointId, } } @@ -899,11 +866,22 @@ let make = ( let resumeInitialState = async (): Persistence.initialState => { 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>), + InternalTable.Chains.getInitialState( + sql, + ~pgSchema, + )->Promise.thenResolve(rawInitialStates => { + rawInitialStates->Belt.Array.map((rawInitialState): Persistence.initialChainState => { + id: rawInitialState.id, + startBlock: rawInitialState.startBlock, + endBlock: rawInitialState.endBlock->Js.Null.toOption, + maxReorgDepth: rawInitialState.maxReorgDepth, + firstEventBlockNumber: rawInitialState.firstEventBlockNumber->Js.Null.toOption, + timestampCaughtUpToHeadOrEndblock: rawInitialState.timestampCaughtUpToHeadOrEndblock->Js.Null.toOption, + numEventsProcessed: rawInitialState.numEventsProcessed, + progressBlockNumber: rawInitialState.progressBlockNumber, + dynamicContracts: rawInitialState.dynamicContracts, + }) + }), sql ->Postgres.unsafe(InternalTable.Checkpoints.makeCommitedCheckpointIdQuery(~pgSchema)) ->(Utils.magic: promise> => promise>), diff --git a/codegenerator/cli/npm/envio/src/Prometheus.res b/codegenerator/cli/npm/envio/src/Prometheus.res index dfaac0a71..6831830ef 100644 --- a/codegenerator/cli/npm/envio/src/Prometheus.res +++ b/codegenerator/cli/npm/envio/src/Prometheus.res @@ -477,9 +477,15 @@ module RollbackSuccess = { "help": "Number of successful rollbacks on reorg", }) - let increment = (~timeMillis: Hrtime.milliseconds) => { + let eventsCounter = PromClient.Counter.makeCounter({ + "name": "envio_rollback_events_count", + "help": "Number of events rollbacked on reorg", + }) + + let increment = (~timeMillis: Hrtime.milliseconds, ~rollbackedProcessedEvents) => { timeCounter->PromClient.Counter.incMany(timeMillis->Hrtime.intFromMillis) counter->PromClient.Counter.inc + eventsCounter->PromClient.Counter.incMany(rollbackedProcessedEvents) } } diff --git a/codegenerator/cli/npm/envio/src/ReorgDetection.res b/codegenerator/cli/npm/envio/src/ReorgDetection.res index 9301c862e..1ce384188 100644 --- a/codegenerator/cli/npm/envio/src/ReorgDetection.res +++ b/codegenerator/cli/npm/envio/src/ReorgDetection.res @@ -60,11 +60,22 @@ type t = { detectedReorgBlock: option, } -let make = (~blocks, ~maxReorgDepth, ~shouldRollbackOnReorg, ~detectedReorgBlock=?) => { +let make = ( + ~chainReorgCheckpoints: array, + ~maxReorgDepth, + ~shouldRollbackOnReorg, + ~detectedReorgBlock=?, +) => { let dataByBlockNumber = Js.Dict.empty() - blocks->Belt.Array.forEach(block => { - dataByBlockNumber->Js.Dict.set(block.blockNumber->Js.Int.toString, block) + chainReorgCheckpoints->Belt.Array.forEach(block => { + dataByBlockNumber->Utils.Dict.setByInt( + block.blockNumber, + { + blockHash: block.blockHash, + blockNumber: block.blockNumber, + }, + ) }) { @@ -144,7 +155,7 @@ let registerReorgGuard = ( ...self, detectedReorgBlock: Some(reorgDetected.scannedBlock), } - : make(~blocks=[], ~maxReorgDepth, ~shouldRollbackOnReorg), + : make(~chainReorgCheckpoints=[], ~maxReorgDepth, ~shouldRollbackOnReorg), ReorgDetected(reorgDetected), ) | None => { diff --git a/codegenerator/cli/npm/envio/src/SafeCheckpointTracking.res b/codegenerator/cli/npm/envio/src/SafeCheckpointTracking.res new file mode 100644 index 000000000..59a7eaab8 --- /dev/null +++ b/codegenerator/cli/npm/envio/src/SafeCheckpointTracking.res @@ -0,0 +1,132 @@ +// We need this module to effectively track safe checkpoint id +// this is very cheap to do in memory, while requires a lot of work on a db +// especially when save_full_history is enabled. +// The safe checkpoint id can be used to optimize checkpoints traverse logic and +// make pruning operation super cheap. +type t = { + checkpointIds: array, + checkpointBlockNumbers: array, + maxReorgDepth: int, +} + +let make = ( + ~maxReorgDepth, + ~shouldRollbackOnReorg, + ~chainReorgCheckpoints: array, +) => { + if maxReorgDepth > 0 && shouldRollbackOnReorg { + let checkpointIds = Belt.Array.makeUninitializedUnsafe(chainReorgCheckpoints->Array.length) + let checkpointBlockNumbers = Belt.Array.makeUninitializedUnsafe( + chainReorgCheckpoints->Array.length, + ) + chainReorgCheckpoints->Js.Array2.forEachi((checkpoint, idx) => { + checkpointIds->Belt.Array.setUnsafe(idx, checkpoint.checkpointId) + checkpointBlockNumbers->Belt.Array.setUnsafe(idx, checkpoint.blockNumber) + }) + Some({ + checkpointIds, + checkpointBlockNumbers, + maxReorgDepth, + }) + } else { + None + } +} + +let getSafeCheckpointId = (safeCheckpointTracking: t, ~sourceBlockNumber: int) => { + let safeBlockNumber = sourceBlockNumber - safeCheckpointTracking.maxReorgDepth + + if safeCheckpointTracking.checkpointBlockNumbers->Belt.Array.getUnsafe(0) > safeBlockNumber { + 0 + } else { + let trackingCheckpointsCount = safeCheckpointTracking.checkpointBlockNumbers->Array.length + switch trackingCheckpointsCount { + | 1 => safeCheckpointTracking.checkpointIds->Belt.Array.getUnsafe(0) + | _ => { + let result = ref(None) + let idx = ref(1) + + while idx.contents < trackingCheckpointsCount && result.contents === None { + if ( + safeCheckpointTracking.checkpointBlockNumbers->Belt.Array.getUnsafe(idx.contents) > + safeBlockNumber + ) { + result := + Some(safeCheckpointTracking.checkpointIds->Belt.Array.getUnsafe(idx.contents - 1)) + } + idx := idx.contents + 1 + } + + switch result.contents { + | Some(checkpointId) => checkpointId + | None => + safeCheckpointTracking.checkpointIds->Belt.Array.getUnsafe(trackingCheckpointsCount - 1) + } + } + } + } +} + +let updateOnNewBatch = ( + safeCheckpointTracking: t, + ~sourceBlockNumber: int, + ~chainId: int, + ~batchCheckpointIds: array, + ~batchCheckpointBlockNumbers: array, + ~batchCheckpointChainIds: array, +) => { + let safeCheckpointId = getSafeCheckpointId(safeCheckpointTracking, ~sourceBlockNumber) + + let mutCheckpointIds = [] + let mutCheckpointBlockNumbers = [] + + // Copy + Clean up old checkpoints + for idx in 0 to safeCheckpointTracking.checkpointIds->Array.length - 1 { + let checkpointId = safeCheckpointTracking.checkpointIds->Belt.Array.getUnsafe(idx) + if checkpointId >= safeCheckpointId { + mutCheckpointIds->Js.Array2.push(checkpointId)->ignore + mutCheckpointBlockNumbers + ->Js.Array2.push(safeCheckpointTracking.checkpointBlockNumbers->Belt.Array.getUnsafe(idx)) + ->ignore + } + } + + // Append new checkpoints + for idx in 0 to batchCheckpointIds->Array.length - 1 { + if batchCheckpointChainIds->Belt.Array.getUnsafe(idx) === chainId { + mutCheckpointIds->Js.Array2.push(batchCheckpointIds->Belt.Array.getUnsafe(idx))->ignore + mutCheckpointBlockNumbers + ->Js.Array2.push(batchCheckpointBlockNumbers->Belt.Array.getUnsafe(idx)) + ->ignore + } + } + + { + checkpointIds: mutCheckpointIds, + checkpointBlockNumbers: mutCheckpointBlockNumbers, + maxReorgDepth: safeCheckpointTracking.maxReorgDepth, + } +} + +let rollback = (safeCheckpointTracking: t, ~targetBlockNumber: int) => { + let mutCheckpointIds = [] + let mutCheckpointBlockNumbers = [] + + for idx in 0 to safeCheckpointTracking.checkpointIds->Array.length - 1 { + let blockNumber = safeCheckpointTracking.checkpointBlockNumbers->Belt.Array.getUnsafe(idx) + if blockNumber <= targetBlockNumber { + mutCheckpointIds + ->Js.Array2.push(safeCheckpointTracking.checkpointIds->Belt.Array.getUnsafe(idx)) + ->ignore + mutCheckpointBlockNumbers + ->Js.Array2.push(safeCheckpointTracking.checkpointBlockNumbers->Belt.Array.getUnsafe(idx)) + ->ignore + } + } + + { + checkpointIds: mutCheckpointIds, + checkpointBlockNumbers: mutCheckpointBlockNumbers, + maxReorgDepth: safeCheckpointTracking.maxReorgDepth, + } +} diff --git a/codegenerator/cli/npm/envio/src/db/EntityHistory.res b/codegenerator/cli/npm/envio/src/db/EntityHistory.res index 87308f5f1..2789b7197 100644 --- a/codegenerator/cli/npm/envio/src/db/EntityHistory.res +++ b/codegenerator/cli/npm/envio/src/db/EntityHistory.res @@ -3,183 +3,48 @@ open Table module RowAction = { type t = SET | DELETE let variants = [SET, DELETE] - let name = "ENTITY_HISTORY_ROW_ACTION" + let name = "ENVIO_HISTORY_CHANGE" let schema = S.enum(variants) } -type historyFieldsGeneral<'a> = { - chain_id: 'a, - block_timestamp: 'a, - block_number: 'a, - log_index: 'a, -} - -type historyFields = historyFieldsGeneral +type entityUpdateAction<'entityType> = + | Set('entityType) + | Delete -type entityIdOnly = {id: string} -let entityIdOnlySchema = S.schema(s => {id: s.matches(S.string)}) -type entityData<'entity> = Delete(entityIdOnly) | Set('entity) - -type historyRow<'entity> = { - current: historyFields, - previous: option, - entityData: entityData<'entity>, - // In the event of a rollback, some entity updates may have been - // been affected by a rollback diff. If there was no rollback diff - // this will always be false. - // If there was a rollback diff, this will be false in the case of a - // new entity update (where entity affected is not present in the diff) b - // but true if the update is related to an entity that is - // currently present in the diff - // Optional since it's discarded during parsing/serialization - containsRollbackDiffChange?: bool, +type entityUpdate<'entityType> = { + entityId: string, + entityUpdateAction: entityUpdateAction<'entityType>, + checkpointId: int, } -type previousHistoryFields = historyFieldsGeneral> - -//For flattening the optional previous fields into their own individual nullable fields -let previousHistoryFieldsSchema = S.object(s => { - chain_id: s.field("previous_entity_history_chain_id", S.null(S.int)), - block_timestamp: s.field("previous_entity_history_block_timestamp", S.null(S.int)), - block_number: s.field("previous_entity_history_block_number", S.null(S.int)), - log_index: s.field("previous_entity_history_log_index", S.null(S.int)), -}) - -let currentHistoryFieldsSchema = S.object(s => { - chain_id: s.field("entity_history_chain_id", S.int), - block_timestamp: s.field("entity_history_block_timestamp", S.int), - block_number: s.field("entity_history_block_number", S.int), - log_index: s.field("entity_history_log_index", S.int), -}) - -let makeHistoryRowSchema: S.t<'entity> => S.t> = entitySchema => { - //Maps a schema object for the given entity with all fields nullable except for the id field - //Keeps any original nullable fields - let nullableEntitySchema: S.t> = S.schema(s => - switch entitySchema->S.classify { - | Object({items}) => - let nulldict = Js.Dict.empty() - items->Belt.Array.forEach(({location, schema}) => { - let nullableFieldSchema = switch (location, schema->S.classify) { - | ("id", _) - | (_, Null(_)) => schema //TODO double check this works for array types - | _ => S.null(schema)->S.toUnknown - } - - nulldict->Js.Dict.set(location, s.matches(nullableFieldSchema)) - }) - nulldict - | _ => - Js.Exn.raiseError( - "Failed creating nullableEntitySchema. Expected an object schema for entity", - ) - } - ) - - let previousWithNullFields = { - chain_id: None, - block_timestamp: None, - block_number: None, - log_index: None, - } +// Prefix with envio_ to avoid colleasions +let changeFieldName = "envio_change" +let checkpointIdFieldName = "checkpoint_id" +let makeSetUpdateSchema: S.t<'entity> => S.t> = entitySchema => { S.object(s => { + s.tag(changeFieldName, RowAction.SET) { - "current": s.flatten(currentHistoryFieldsSchema), - "previous": s.flatten(previousHistoryFieldsSchema), - "entityData": s.flatten(nullableEntitySchema), - "action": s.field("action", RowAction.schema), + checkpointId: s.field(checkpointIdFieldName, S.int), + entityId: s.field("id", S.string), + entityUpdateAction: Set(s.flatten(entitySchema)), } - })->S.transform(s => { - parser: v => { - current: v["current"], - previous: switch v["previous"] { - | { - chain_id: Some(chain_id), - block_timestamp: Some(block_timestamp), - block_number: Some(block_number), - log_index: Some(log_index), - } => - Some({ - chain_id, - block_timestamp, - block_number, - log_index, - }) - | {chain_id: None, block_timestamp: None, block_number: None, log_index: None} => None - | _ => s.fail("Unexpected mix of null and non-null values in previous history fields") - }, - entityData: switch v["action"] { - | SET => v["entityData"]->(Utils.magic: Js.Dict.t => 'entity)->Set - | DELETE => - let {id} = v["entityData"]->(Utils.magic: Js.Dict.t => entityIdOnly) - Delete({id: id}) - }, - }, - serializer: v => { - let (entityData, action) = switch v.entityData { - | Set(entityData) => (entityData->(Utils.magic: 'entity => Js.Dict.t), RowAction.SET) - | Delete(entityIdOnly) => ( - entityIdOnly->(Utils.magic: entityIdOnly => Js.Dict.t), - DELETE, - ) - } - - { - "current": v.current, - "entityData": entityData, - "action": action, - "previous": switch v.previous { - | Some(historyFields) => - historyFields->(Utils.magic: historyFields => previousHistoryFields) //Cast to previousHistoryFields (with "Some" field values) - | None => previousWithNullFields - }, - } - }, }) } type t<'entity> = { table: table, - makeInsertFnQuery: (~pgSchema: string) => string, - schema: S.t>, + setUpdateSchema: S.t>, // Used for parsing - schemaRows: S.t>>, - insertFn: (Postgres.sql, Js.Json.t, ~shouldCopyCurrentEntity: bool) => promise, + setUpdateSchemaRows: S.t>>, + makeInsertDeleteUpdatesQuery: (~pgSchema: string) => string, + makeGetRollbackRemovedIdsQuery: (~pgSchema: string) => string, + makeGetRollbackRestoredEntitiesQuery: (~pgSchema: string) => string, } -type entityInternal - -external castInternal: t<'entity> => t = "%identity" -external eval: string => 'a = "eval" +let historyTableName = (~entityName) => "envio_history_" ++ entityName let fromTable = (table: table, ~schema: S.t<'entity>): t<'entity> => { - let entity_history_block_timestamp = "entity_history_block_timestamp" - let entity_history_chain_id = "entity_history_chain_id" - let entity_history_block_number = "entity_history_block_number" - let entity_history_log_index = "entity_history_log_index" - - //NB: Ordered by hirarchy of event ordering - let currentChangeFieldNames = [ - entity_history_block_timestamp, - entity_history_chain_id, - entity_history_block_number, - entity_history_log_index, - ] - - let currentHistoryFields = - currentChangeFieldNames->Belt.Array.map(fieldName => - mkField(fieldName, Integer, ~fieldSchema=S.never, ~isPrimaryKey=true) - ) - - let previousChangeFieldNames = - currentChangeFieldNames->Belt.Array.map(fieldName => "previous_" ++ fieldName) - - let previousHistoryFields = - previousChangeFieldNames->Belt.Array.map(fieldName => - mkField(fieldName, Integer, ~fieldSchema=S.never, ~isNullable=true) - ) - let id = "id" let dataFields = table.fields->Belt.Array.keepMap(field => @@ -202,118 +67,97 @@ let fromTable = (table: table, ~schema: S.t<'entity>): t<'entity> => { } ) - let actionFieldName = "action" - - let actionField = mkField(actionFieldName, Custom(RowAction.name), ~fieldSchema=S.never) + let actionField = mkField(changeFieldName, Custom(RowAction.name), ~fieldSchema=S.never) - let serialField = mkField("serial", Serial, ~fieldSchema=S.never, ~isNullable=true, ~isIndex=true) + let checkpointIdField = mkField( + checkpointIdFieldName, + Integer, + ~fieldSchema=S.int, + ~isPrimaryKey=true, + ) - let dataFieldNames = dataFields->Belt.Array.map(field => field->getFieldName) + // let dataFieldNames = dataFields->Belt.Array.map(field => field->getFieldName) - let originTableName = table.tableName - let historyTableName = originTableName ++ "_history" + let entityTableName = table.tableName + let historyTableName = historyTableName(~entityName=entityTableName) //ignore composite indices let table = mkTable( historyTableName, - ~fields=Belt.Array.concatMany([ - currentHistoryFields, - previousHistoryFields, - dataFields, - [actionField, serialField], - ]), + ~fields=dataFields->Belt.Array.concat([checkpointIdField, actionField]), ) - let insertFnName = `"insert_${table.tableName}"` - - let allFieldNamesDoubleQuoted = - Belt.Array.concatMany([ - currentChangeFieldNames, - previousChangeFieldNames, - dataFieldNames, - [actionFieldName], - ])->Belt.Array.map(fieldName => `"${fieldName}"`) - - let makeInsertFnQuery = (~pgSchema) => { - let historyRowArg = "history_row" - let historyTablePath = `"${pgSchema}"."${historyTableName}"` - let originTablePath = `"${pgSchema}"."${originTableName}"` - - let previousHistoryFieldsAreNullStr = - previousChangeFieldNames - ->Belt.Array.map(fieldName => `${historyRowArg}.${fieldName} IS NULL`) - ->Js.Array2.joinWith(" OR ") - - let currentChangeFieldNamesCommaSeparated = currentChangeFieldNames->Js.Array2.joinWith(", ") - - let dataFieldNamesDoubleQuoted = dataFieldNames->Belt.Array.map(fieldName => `"${fieldName}"`) - let dataFieldNamesCommaSeparated = dataFieldNamesDoubleQuoted->Js.Array2.joinWith(", ") - - `CREATE OR REPLACE FUNCTION ${insertFnName}(${historyRowArg} ${historyTablePath}, should_copy_current_entity BOOLEAN) -RETURNS void AS $$ -DECLARE - v_previous_record RECORD; - v_origin_record RECORD; -BEGIN - -- Check if previous values are not provided - IF ${previousHistoryFieldsAreNullStr} THEN - -- Find the most recent record for the same id - SELECT ${currentChangeFieldNamesCommaSeparated} INTO v_previous_record - FROM ${historyTablePath} - WHERE ${id} = ${historyRowArg}.${id} - ORDER BY ${currentChangeFieldNames - ->Belt.Array.map(fieldName => fieldName ++ " DESC") - ->Js.Array2.joinWith(", ")} - LIMIT 1; - - -- If a previous record exists, use its values - IF FOUND THEN - ${Belt.Array.zip(currentChangeFieldNames, previousChangeFieldNames) - ->Belt.Array.map(((currentFieldName, previousFieldName)) => { - `${historyRowArg}.${previousFieldName} := v_previous_record.${currentFieldName};` - }) - ->Js.Array2.joinWith(" ")} - ElSIF should_copy_current_entity THEN - -- Check if a value for the id exists in the origin table and if so, insert a history row for it. - SELECT ${dataFieldNamesCommaSeparated} FROM ${originTablePath} WHERE id = ${historyRowArg}.${id} INTO v_origin_record; - IF FOUND THEN - INSERT INTO ${historyTablePath} (${currentChangeFieldNamesCommaSeparated}, ${dataFieldNamesCommaSeparated}, "${actionFieldName}") - -- SET the current change data fields to 0 since we don't know what they were - -- and it doesn't matter provided they are less than any new values - VALUES (${currentChangeFieldNames - ->Belt.Array.map(_ => "0") - ->Js.Array2.joinWith(", ")}, ${dataFieldNames - ->Belt.Array.map(fieldName => `v_origin_record."${fieldName}"`) - ->Js.Array2.joinWith(", ")}, 'SET'); - - ${previousChangeFieldNames - ->Belt.Array.map(previousFieldName => { - `${historyRowArg}.${previousFieldName} := 0;` - }) - ->Js.Array2.joinWith(" ")} - END IF; - END IF; - END IF; - - INSERT INTO ${historyTablePath} (${allFieldNamesDoubleQuoted->Js.Array2.joinWith(", ")}) - VALUES (${allFieldNamesDoubleQuoted - ->Belt.Array.map(fieldName => `${historyRowArg}.${fieldName}`) - ->Js.Array2.joinWith(", ")}); -END; -$$ LANGUAGE plpgsql;` + let setUpdateSchema = makeSetUpdateSchema(schema) + + let makeInsertDeleteUpdatesQuery = { + // Get all field names for the INSERT statement + let allFieldNames = table.fields->Belt.Array.map(field => field->getFieldName) + let allFieldNamesStr = + allFieldNames->Belt.Array.map(name => `"${name}"`)->Js.Array2.joinWith(", ") + + // Build the SELECT part: id from unnest, checkpoint_id from unnest, 'DELETE' for action, NULL for all other fields + let selectParts = allFieldNames->Belt.Array.map(fieldName => { + switch fieldName { + | "id" => "u.id" + | field if field == checkpointIdFieldName => "u.checkpoint_id" + | field if field == changeFieldName => "'DELETE'" + | _ => "NULL" + } + }) + let selectPartsStr = selectParts->Js.Array2.joinWith(", ") + (~pgSchema) => { + `INSERT INTO "${pgSchema}"."${historyTableName}" (${allFieldNamesStr}) +SELECT ${selectPartsStr} +FROM UNNEST($1::text[], $2::int[]) AS u(id, checkpoint_id)` + } } - let insertFnString = `(sql, rowArgs, shouldCopyCurrentEntity) => - sql\`select ${insertFnName}(ROW(${allFieldNamesDoubleQuoted - ->Belt.Array.map(fieldNameDoubleQuoted => `\${rowArgs[${fieldNameDoubleQuoted}]\}`) - ->Js.Array2.joinWith(", ")}, NULL), --NULL argument for SERIAL field - \${shouldCopyCurrentEntity});\`` - - let insertFn: (Postgres.sql, Js.Json.t, ~shouldCopyCurrentEntity: bool) => promise = - insertFnString->eval + // Get data field names for rollback queries (exclude changeFieldName and checkpointIdFieldName) + let dataFieldNames = + table.fields + ->Belt.Array.map(field => field->getFieldName) + ->Belt.Array.keep(fieldName => + fieldName != changeFieldName && fieldName != checkpointIdFieldName + ) + let dataFieldsCommaSeparated = + dataFieldNames->Belt.Array.map(name => `"${name}"`)->Js.Array2.joinWith(", ") + + // Returns entity IDs that were created after the rollback target and have no history before it. + // These entities should be deleted during rollback. + let makeGetRollbackRemovedIdsQuery = (~pgSchema) => { + `SELECT DISTINCT id +FROM "${pgSchema}"."${historyTableName}" +WHERE "${checkpointIdFieldName}" > $1 + AND NOT EXISTS ( + SELECT 1 + FROM "${pgSchema}"."${historyTableName}" h + WHERE h.id = "${historyTableName}".id + AND h."${checkpointIdFieldName}" <= $1 + )` + } - let schema = makeHistoryRowSchema(schema) + // Returns the most recent entity state for IDs that need to be restored during rollback. + // For each ID modified after the rollback target, retrieves its latest state at or before the target. + let makeGetRollbackRestoredEntitiesQuery = (~pgSchema) => { + `SELECT DISTINCT ON (id) ${dataFieldsCommaSeparated} +FROM "${pgSchema}"."${historyTableName}" +WHERE "${checkpointIdFieldName}" <= $1 + AND EXISTS ( + SELECT 1 + FROM "${pgSchema}"."${historyTableName}" h + WHERE h.id = "${historyTableName}".id + AND h."${checkpointIdFieldName}" > $1 + ) +ORDER BY id, "${checkpointIdFieldName}" DESC` + } - {table, makeInsertFnQuery, schema, schemaRows: S.array(schema), insertFn} + { + table, + setUpdateSchema, + setUpdateSchemaRows: S.array(setUpdateSchema), + makeInsertDeleteUpdatesQuery, + makeGetRollbackRemovedIdsQuery, + makeGetRollbackRestoredEntitiesQuery, + } } type safeReorgBlocks = { @@ -323,60 +167,94 @@ type safeReorgBlocks = { // We want to keep only the minimum history needed to survive chain reorgs and delete everything older. // Each chain gives us a "safe block": we assume reorgs will never happen at that block. +// The latest checkpoint belonging to safe blocks of all chains is the safe checkpoint id. // // What we keep per entity id: -// - The latest history row at or before the safe block (the "anchor"). This is the last state that could -// ever be relevant during a rollback. // - If there are history rows in reorg threshold (after the safe block), we keep the anchor and delete all older rows. // - If there are no history rows in reorg threshold (after the safe block), even the anchor is redundant, so we delete it too. +// Anchor is the latest history row at or before the safe checkpoint id. +// This is the last state that could ever be relevant during a rollback. // // Why this is safe: -// - Rollbacks will not cross the safe block, so rows older than the anchor can never be referenced again. -// - If nothing changed in reorg threshold (after the safe block), the current state for that id can be reconstructed from the +// - Rollbacks will not cross the safe checkpoint id, so rows older than the anchor can never be referenced again. +// - If nothing changed in reorg threshold (after the safe checkpoint), the current state for that id can be reconstructed from the // origin table; we do not need a pre-safe anchor for it. -// -// Performance notes: -// - Multi-chain batching: inputs are expanded with unnest, letting one prepared statement prune many chains and -// enabling the planner to use indexes per chain_id efficiently. -// - Minimal row touches: we only compute keep_serial per id and delete strictly older rows; this reduces write -// amplification and vacuum pressure compared to broad time-based purges. -// - Contention-awareness: the DELETE joins on ids first, narrowing target rows early to limit locking and buffer churn. let makePruneStaleEntityHistoryQuery = (~entityName, ~pgSchema) => { - let historyTableName = entityName ++ "_history" - let historyTableRef = `"${pgSchema}"."${historyTableName}"` + let historyTableRef = `"${pgSchema}"."${historyTableName(~entityName)}"` - `WITH safe AS ( - SELECT s.chain_id, s.block_number - FROM unnest($1::int[], $2::bigint[]) AS s(chain_id, block_number) -), -max_before_safe AS ( - SELECT t.id, MAX(t.serial) AS keep_serial - FROM ${historyTableRef} t - JOIN safe s - ON s.chain_id = t.entity_history_chain_id - AND t.entity_history_block_number <= s.block_number + `WITH anchors AS ( + SELECT t.id, MAX(t.${checkpointIdFieldName}) AS keep_checkpoint_id + FROM ${historyTableRef} t WHERE t.${checkpointIdFieldName} <= $1 GROUP BY t.id -), -post_safe AS ( - SELECT DISTINCT t.id - FROM ${historyTableRef} t - JOIN safe s - ON s.chain_id = t.entity_history_chain_id - AND t.entity_history_block_number > s.block_number ) DELETE FROM ${historyTableRef} d -USING max_before_safe m -LEFT JOIN post_safe p ON p.id = m.id -WHERE d.id = m.id +USING anchors a +WHERE d.id = a.id AND ( - d.serial < m.keep_serial - OR (p.id IS NULL AND d.serial = m.keep_serial) + d.${checkpointIdFieldName} < a.keep_checkpoint_id + OR ( + d.${checkpointIdFieldName} = a.keep_checkpoint_id AND + NOT EXISTS ( + SELECT 1 FROM ${historyTableRef} ps + WHERE ps.id = d.id AND ps.${checkpointIdFieldName} > $1 + ) + ) );` } -let pruneStaleEntityHistory = (sql, ~entityName, ~pgSchema, ~safeReorgBlocks): promise => { +let pruneStaleEntityHistory = (sql, ~entityName, ~pgSchema, ~safeCheckpointId): promise => { sql->Postgres.preparedUnsafe( makePruneStaleEntityHistoryQuery(~entityName, ~pgSchema), - (safeReorgBlocks.chainIds, safeReorgBlocks.blockNumbers)->Utils.magic, + [safeCheckpointId]->Utils.magic, + ) +} + +// If an entity doesn't have a history before the update +// we create it automatically with checkpoint_id 0 +let makeBackfillHistoryQuery = (~pgSchema, ~entityName) => { + `WITH target_ids AS ( + SELECT UNNEST($1::${(Text: Table.fieldType :> string)}[]) AS id +), +missing_history AS ( + SELECT e.* + FROM "${pgSchema}"."${entityName}" e + JOIN target_ids t ON e.id = t.id + LEFT JOIN "${pgSchema}"."${historyTableName(~entityName)}" h ON h.id = e.id + WHERE h.id IS NULL +) +INSERT INTO "${pgSchema}"."${historyTableName(~entityName)}" +SELECT *, 0 AS ${checkpointIdFieldName}, '${(RowAction.SET :> string)}' as ${changeFieldName} +FROM missing_history;` +} + +let backfillHistory = (sql, ~pgSchema, ~entityName, ~ids: array) => { + sql + ->Postgres.preparedUnsafe(makeBackfillHistoryQuery(~entityName, ~pgSchema), [ids]->Obj.magic) + ->Promise.ignoreValue +} + +let insertDeleteUpdates = ( + sql, + ~pgSchema, + ~entityHistory, + ~batchDeleteEntityIds, + ~batchDeleteCheckpointIds, +) => { + sql + ->Postgres.preparedUnsafe( + entityHistory.makeInsertDeleteUpdatesQuery(~pgSchema), + (batchDeleteEntityIds, batchDeleteCheckpointIds)->Obj.magic, + ) + ->Promise.ignoreValue +} + +let rollback = (sql, ~pgSchema, ~entityName, ~rollbackTargetCheckpointId: int) => { + sql + ->Postgres.preparedUnsafe( + `DELETE FROM "${pgSchema}"."${historyTableName( + ~entityName, + )}" WHERE "${checkpointIdFieldName}" > $1;`, + [rollbackTargetCheckpointId]->Utils.magic, ) + ->Promise.ignoreValue } diff --git a/codegenerator/cli/npm/envio/src/db/InternalTable.gen.ts b/codegenerator/cli/npm/envio/src/db/InternalTable.gen.ts index 69ffd044e..f59727739 100644 --- a/codegenerator/cli/npm/envio/src/db/InternalTable.gen.ts +++ b/codegenerator/cli/npm/envio/src/db/InternalTable.gen.ts @@ -7,6 +7,19 @@ import type {Json_t as Js_Json_t} from '../../src/Js.shim'; import type {t as Address_t} from '../../src/Address.gen'; +export type DynamicContractRegistry_t = { + readonly id: string; + readonly chain_id: number; + readonly registering_event_block_number: number; + readonly registering_event_log_index: number; + readonly registering_event_block_timestamp: number; + readonly registering_event_contract_name: string; + readonly registering_event_name: string; + readonly registering_event_src_address: Address_t; + readonly contract_address: Address_t; + readonly contract_name: string +}; + export type RawEvents_t = { readonly chain_id: number; readonly event_id: bigint; @@ -21,16 +34,3 @@ export type RawEvents_t = { readonly transaction_fields: Js_Json_t; readonly params: Js_Json_t }; - -export type DynamicContractRegistry_t = { - readonly id: string; - readonly chain_id: number; - readonly registering_event_block_number: number; - readonly registering_event_log_index: number; - readonly registering_event_block_timestamp: number; - readonly registering_event_contract_name: string; - readonly registering_event_name: string; - readonly registering_event_src_address: Address_t; - readonly contract_address: Address_t; - readonly contract_name: string -}; diff --git a/codegenerator/cli/npm/envio/src/db/InternalTable.res b/codegenerator/cli/npm/envio/src/db/InternalTable.res index 3fc282ebe..a2f4e79d3 100644 --- a/codegenerator/cli/npm/envio/src/db/InternalTable.res +++ b/codegenerator/cli/npm/envio/src/db/InternalTable.res @@ -5,6 +5,72 @@ let isPrimaryKey = true let isNullable = true let isIndex = true +module DynamicContractRegistry = { + let name = "dynamic_contract_registry" + + let makeId = (~chainId, ~contractAddress) => { + chainId->Belt.Int.toString ++ "-" ++ contractAddress->Address.toString + } + + // @genType Used for Test DB + @genType + type t = { + id: string, + @as("chain_id") chainId: int, + @as("registering_event_block_number") registeringEventBlockNumber: int, + @as("registering_event_log_index") registeringEventLogIndex: int, + @as("registering_event_block_timestamp") registeringEventBlockTimestamp: int, + @as("registering_event_contract_name") registeringEventContractName: string, + @as("registering_event_name") registeringEventName: string, + @as("registering_event_src_address") registeringEventSrcAddress: Address.t, + @as("contract_address") contractAddress: Address.t, + @as("contract_name") contractName: string, + } + + let schema = S.schema(s => { + id: s.matches(S.string), + chainId: s.matches(S.int), + registeringEventBlockNumber: s.matches(S.int), + registeringEventLogIndex: s.matches(S.int), + registeringEventContractName: s.matches(S.string), + registeringEventName: s.matches(S.string), + registeringEventSrcAddress: s.matches(Address.schema), + registeringEventBlockTimestamp: s.matches(S.int), + contractAddress: s.matches(Address.schema), + contractName: s.matches(S.string), + }) + + let rowsSchema = S.array(schema) + + let table = mkTable( + name, + ~fields=[ + mkField("id", Text, ~isPrimaryKey, ~fieldSchema=S.string), + mkField("chain_id", Integer, ~fieldSchema=S.int), + mkField("registering_event_block_number", Integer, ~fieldSchema=S.int), + mkField("registering_event_log_index", Integer, ~fieldSchema=S.int), + mkField("registering_event_block_timestamp", Integer, ~fieldSchema=S.int), + mkField("registering_event_contract_name", Text, ~fieldSchema=S.string), + mkField("registering_event_name", Text, ~fieldSchema=S.string), + mkField("registering_event_src_address", Text, ~fieldSchema=Address.schema), + mkField("contract_address", Text, ~fieldSchema=Address.schema), + mkField("contract_name", Text, ~fieldSchema=S.string), + ], + ) + + let entityHistory = table->EntityHistory.fromTable(~schema) + + external castToInternal: t => Internal.entity = "%identity" + + let config = { + name, + schema, + rowsSchema, + table, + entityHistory, + }->Internal.fromGenericEntityConfig +} + module Chains = { type progressFields = [ | #progress_block @@ -165,7 +231,51 @@ VALUES ${valuesRows->Js.Array2.joinWith(",\n ")};`, `UPDATE "${pgSchema}"."${table.tableName}" SET ${setClauses->Js.Array2.joinWith(",\n ")} -WHERE "id" = $1;` +WHERE "${(#id: field :> string)}" = $1;` + } + + type rawInitialState = { + id: int, + startBlock: int, + endBlock: Js.Null.t, + maxReorgDepth: int, + firstEventBlockNumber: Js.Null.t, + timestampCaughtUpToHeadOrEndblock: Js.Null.t, + numEventsProcessed: int, + progressBlockNumber: int, + dynamicContracts: array, + } + + // FIXME: Using registering_event_block_number for startBlock + // seems incorrect, since there might be a custom start block + // for the contract. + // TODO: Write a repro test where it might break something and fix + let makeGetInitialStateQuery = (~pgSchema) => { + `SELECT "${(#id: field :> string)}" as "id", +"${(#start_block: field :> string)}" as "startBlock", +"${(#end_block: field :> string)}" as "endBlock", +"${(#max_reorg_depth: field :> string)}" as "maxReorgDepth", +"${(#first_event_block: field :> string)}" as "firstEventBlockNumber", +"${(#ready_at: field :> string)}" as "timestampCaughtUpToHeadOrEndblock", +"${(#events_processed: field :> string)}" as "numEventsProcessed", +"${(#progress_block: field :> string)}" as "progressBlockNumber", +( + SELECT COALESCE(json_agg(json_build_object( + 'address', "contract_address", + 'contractName', "contract_name", + 'startBlock', "registering_event_block_number", + 'registrationBlock', "registering_event_block_number" + )), '[]'::json) + FROM "${pgSchema}"."${DynamicContractRegistry.table.tableName}" + WHERE "chain_id" = chains."${(#id: field :> string)}" +) as "dynamicContracts" +FROM "${pgSchema}"."${table.tableName}" as chains;` + } + + let getInitialState = (sql, ~pgSchema) => { + sql + ->Postgres.unsafe(makeGetInitialStateQuery(~pgSchema)) + ->(Utils.magic: promise> => promise>) } let progressFields: array = [#progress_block, #events_processed] @@ -322,7 +432,7 @@ 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;` + AND cp."${(#block_number: field :> string)}" >= rc.safe_block;` // Include safe_block checkpoint to use it for safe checkpoint tracking } let makeCommitedCheckpointIdQuery = (~pgSchema) => { @@ -363,48 +473,76 @@ SELECT * FROM unnest($1::${(Integer :> string)}[],$2::${(Integer :> string)}[],$ ->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 = ( + let rollback = (sql, ~pgSchema, ~rollbackTargetCheckpointId: int) => { + sql + ->Postgres.preparedUnsafe( + `DELETE FROM "${pgSchema}"."${table.tableName}" WHERE "${(#id: field :> string)}" > $1;`, + [rollbackTargetCheckpointId]->Utils.magic, + ) + ->Promise.ignoreValue + } + + let makePruneStaleCheckpointsQuery = (~pgSchema) => { + `DELETE FROM "${pgSchema}"."${table.tableName}" WHERE "${(#id: field :> string)}" < $1;` + } + + let pruneStaleCheckpoints = (sql, ~pgSchema, ~safeCheckpointId: int) => { + sql + ->Postgres.preparedUnsafe( + makePruneStaleCheckpointsQuery(~pgSchema), + [safeCheckpointId]->Obj.magic, + ) + ->Promise.ignoreValue + } + + let makeGetRollbackTargetCheckpointQuery = (~pgSchema) => { + `SELECT "${(#id: field :> string)}" FROM "${pgSchema}"."${table.tableName}" +WHERE + "${(#chain_id: field :> string)}" = $1 AND + "${(#block_number: field :> string)}" <= $2 +ORDER BY "${(#id: field :> string)}" DESC +LIMIT 1;` + } + + let getRollbackTargetCheckpoint = ( sql, ~pgSchema, - ~chainId, - ~knownBlockNumber, + ~reorgChainId: int, + ~lastKnownValidBlockNumber: int, ) => { 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), + makeGetRollbackTargetCheckpointQuery(~pgSchema), + (reorgChainId, lastKnownValidBlockNumber)->Obj.magic, ) - ->Promise.ignoreValue + ->(Utils.magic: promise => promise>) } - // 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 + let makeGetRollbackProgressDiffQuery = (~pgSchema) => { + `SELECT + "${(#chain_id: field :> string)}", + SUM("${(#events_processed: field :> string)}") as events_processed_diff, + MIN("${(#block_number: field :> string)}") - 1 as new_progress_block_number +FROM "${pgSchema}"."${table.tableName}" +WHERE "${(#id: field :> string)}" > $1 +GROUP BY "${(#chain_id: field :> string)}";` + } + + let getRollbackProgressDiff = (sql, ~pgSchema, ~rollbackTargetCheckpointId: int) => { 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 - );`, + ->Postgres.preparedUnsafe( + makeGetRollbackProgressDiffQuery(~pgSchema), + [rollbackTargetCheckpointId]->Obj.magic, + ) + ->( + Utils.magic: promise => promise< + array<{ + "chain_id": int, + "events_processed_diff": string, + "new_progress_block_number": int, + }>, + > ) - ->Promise.ignoreValue } } @@ -500,69 +638,3 @@ module Views = { FROM "${pgSchema}"."${Chains.table.tableName}";` } } - -module DynamicContractRegistry = { - let name = "dynamic_contract_registry" - - let makeId = (~chainId, ~contractAddress) => { - chainId->Belt.Int.toString ++ "-" ++ contractAddress->Address.toString - } - - // @genType Used for Test DB - @genType - type t = { - id: string, - @as("chain_id") chainId: int, - @as("registering_event_block_number") registeringEventBlockNumber: int, - @as("registering_event_log_index") registeringEventLogIndex: int, - @as("registering_event_block_timestamp") registeringEventBlockTimestamp: int, - @as("registering_event_contract_name") registeringEventContractName: string, - @as("registering_event_name") registeringEventName: string, - @as("registering_event_src_address") registeringEventSrcAddress: Address.t, - @as("contract_address") contractAddress: Address.t, - @as("contract_name") contractName: string, - } - - let schema = S.schema(s => { - id: s.matches(S.string), - chainId: s.matches(S.int), - registeringEventBlockNumber: s.matches(S.int), - registeringEventLogIndex: s.matches(S.int), - registeringEventContractName: s.matches(S.string), - registeringEventName: s.matches(S.string), - registeringEventSrcAddress: s.matches(Address.schema), - registeringEventBlockTimestamp: s.matches(S.int), - contractAddress: s.matches(Address.schema), - contractName: s.matches(S.string), - }) - - let rowsSchema = S.array(schema) - - let table = mkTable( - name, - ~fields=[ - mkField("id", Text, ~isPrimaryKey, ~fieldSchema=S.string), - mkField("chain_id", Integer, ~fieldSchema=S.int), - mkField("registering_event_block_number", Integer, ~fieldSchema=S.int), - mkField("registering_event_log_index", Integer, ~fieldSchema=S.int), - mkField("registering_event_block_timestamp", Integer, ~fieldSchema=S.int), - mkField("registering_event_contract_name", Text, ~fieldSchema=S.string), - mkField("registering_event_name", Text, ~fieldSchema=S.string), - mkField("registering_event_src_address", Text, ~fieldSchema=Address.schema), - mkField("contract_address", Text, ~fieldSchema=Address.schema), - mkField("contract_name", Text, ~fieldSchema=S.string), - ], - ) - - let entityHistory = table->EntityHistory.fromTable(~schema) - - external castToInternal: t => Internal.entity = "%identity" - - let config = { - name, - schema, - rowsSchema, - table, - entityHistory, - }->Internal.fromGenericEntityConfig -} diff --git a/codegenerator/cli/npm/envio/src/sources/EventRouter.res b/codegenerator/cli/npm/envio/src/sources/EventRouter.res index 22fef72ac..3a62fc178 100644 --- a/codegenerator/cli/npm/envio/src/sources/EventRouter.res +++ b/codegenerator/cli/npm/envio/src/sources/EventRouter.res @@ -34,7 +34,7 @@ module Group = { group: t<'a>, ~contractAddress, ~blockNumber, - ~indexingContracts: dict, + ~indexingContracts: dict, ) => switch group { | {wildcard, byContractName} => diff --git a/codegenerator/cli/npm/envio/src/sources/Source.res b/codegenerator/cli/npm/envio/src/sources/Source.res index e7f491a64..5cbdb30b2 100644 --- a/codegenerator/cli/npm/envio/src/sources/Source.res +++ b/codegenerator/cli/npm/envio/src/sources/Source.res @@ -49,7 +49,7 @@ type t = { ~fromBlock: int, ~toBlock: option, ~addressesByContractName: dict>, - ~indexingContracts: dict, + ~indexingContracts: dict, ~currentBlockHeight: int, ~partitionId: string, ~selection: FetchState.selection, 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 05dc04c8b..42d8b8688 100644 --- a/codegenerator/cli/templates/dynamic/codegen/src/TestHelpers_MockDb.res.hbs +++ b/codegenerator/cli/templates/dynamic/codegen/src/TestHelpers_MockDb.res.hbs @@ -118,10 +118,11 @@ let makeStoreOperatorEntity = ( let table = cloned->getStore table->set( - Delete->Types.mkEntityUpdate( - ~entityId, - ~eventIdentifier={chainId: -1, blockNumber: -1, blockTimestamp: 0, logIndex: -1}, - ), + { + entityId, + entityUpdateAction: Delete, + checkpointId: 0, + }, ~shouldSaveHistory=false, ) @@ -134,10 +135,11 @@ let makeStoreOperatorEntity = ( let entityId = entity->getKey table->set( - Set(entity)->Types.mkEntityUpdate( - ~entityId, - ~eventIdentifier={chainId: -1, blockNumber: -1, blockTimestamp: 0, logIndex: -1}, - ), + { + entityId, + entityUpdateAction: Set(entity), + checkpointId: 0, + }, ~shouldSaveHistory=false, ) @@ -320,9 +322,17 @@ and makeProcessEvents = (mockDb: t, ~chainId=?) => async ( } else { let itemsWithContractRegister = [] - let config = { - ...config, - registrations: Some(EventRegister.finishRegistration()), + let registrations = EventRegister.finishRegistration() + + let config = if ( + registrations.hasEvents || !(registrations.onBlockByChainId->Utils.Dict.isEmpty) + ) { + { + ...config, + registrations: Some(registrations), + } + } else { + RegisterHandlers.registerAllHandlers() } let processingChainId = ref(chainId) @@ -342,8 +352,8 @@ and makeProcessEvents = (mockDb: t, ~chainId=?) => async ( } switch processingChainId.contents { - | Some(chainId) => - if chainId != event.chainId { + | Some(prevItemChainId) => + if prevItemChainId !== 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}`, ) @@ -373,9 +383,15 @@ and makeProcessEvents = (mockDb: t, ~chainId=?) => async ( 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") + Js.Exn.raiseError("No events provided to processEvents. Please provide at least one event.") } + let processingChain = config->Config.getChain(~chainId=processingChainId) + + let chainFetcher = ChainFetcher.makeFromConfig( + config.chainMap->ChainMap.get(processingChain), + ~config, + ~targetBufferSize=5000, + ) //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 @@ -401,73 +417,80 @@ and makeProcessEvents = (mockDb: t, ~chainId=?) => async ( 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( + let newItemsWithDcs = if itemsWithContractRegister->Utils.Array.notEmpty { + await ChainFetcher.runContractRegistersOrThrow( ~itemsWithContractRegister, - ~chain=ChainMap.Chain.makeUnsafe(~chainId=processingChainId), + ~chain=processingChain, ~config, ) + } else { + itemsWithContractRegister + } + + let updatedFetchState = ref(chainFetcher.fetchState) + + switch newItemsWithDcs { + | [] => () + | _ => + updatedFetchState := + updatedFetchState.contents->FetchState.registerDynamicContracts(newItemsWithDcs) + } - // TODO: Reuse FetchState logic to clean up duplicate dcs - if dcs->Utils.Array.notEmpty { - inMemoryStore->InMemoryStore.setDcsToStore( - ~chainId=processingChainId, - ~dcs, - ~shouldSaveHistory=false, + updatedFetchState := + updatedFetchState.contents + ->FetchState.handleQueryResult( + ~latestFetchedBlock={ + blockNumber: latestFetchedBlockNumber.contents, + blockTimestamp: 0, + }, + ~query={ + partitionId: (updatedFetchState.contents.partitions->Array.getUnsafe(0)).id, + fromBlock: 0, + selection: {eventConfigs: [], dependsOnAddresses: false}, + addressesByContractName: Js.Dict.empty(), + target: FetchState.Head, + indexingContracts: Js.Dict.empty(), + }, + ~newItems, + ) + ->Result.getExn + + // Handle query for the rest partitions without items + // to catch up the latest fully fetched block + for idx in 1 to updatedFetchState.contents.partitions->Array.length - 1 { + let partition = updatedFetchState.contents.partitions->Array.getUnsafe(idx) + updatedFetchState := + updatedFetchState.contents + ->FetchState.handleQueryResult( + ~latestFetchedBlock={ + blockNumber: latestFetchedBlockNumber.contents, + blockTimestamp: 0, + }, + ~query={ + partitionId: partition.id, + fromBlock: 0, + selection: {eventConfigs: [], dependsOnAddresses: false}, + addressesByContractName: Js.Dict.empty(), + target: FetchState.Head, + indexingContracts: Js.Dict.empty(), + }, + ~newItems=[], ) - } + ->Result.getExn } let batch = Batch.prepareUnorderedBatch( ~checkpointIdBeforeBatch=0, ~chainsBeforeBatch=ChainMap.fromArrayUnsafe([ ( - ChainMap.Chain.makeUnsafe(~chainId=processingChainId), + processingChain, ( { - 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, + fetchState: updatedFetchState.contents, + reorgDetection: chainFetcher.reorgDetection, + progressBlockNumber: chainFetcher.committedProgressBlockNumber, + sourceBlockNumber: chainFetcher.currentBlockHeight, + totalEventsProcessed: chainFetcher.numEventsProcessed, }: Batch.chainBeforeBatch ), ), @@ -475,6 +498,8 @@ and makeProcessEvents = (mockDb: t, ~chainId=?) => async ( ~batchSizeTarget=newItems->Array.length, ) + inMemoryStore->InMemoryStore.setBatchDcs(~batch, ~shouldSaveHistory=false) + try { await batch->EventProcessing.preloadBatchOrThrow(~loadManager, ~persistence, ~inMemoryStore) await batch->EventProcessing.runBatchHandlersOrThrow( diff --git a/codegenerator/cli/templates/dynamic/codegen/src/Types.res.hbs b/codegenerator/cli/templates/dynamic/codegen/src/Types.res.hbs index de829d8ff..d385ca0ca 100644 --- a/codegenerator/cli/templates/dynamic/codegen/src/Types.res.hbs +++ b/codegenerator/cli/templates/dynamic/codegen/src/Types.res.hbs @@ -73,51 +73,6 @@ type handlerContext = { type {{entity.name.uncapitalized}} = Entities.{{entity.name.capitalized}}.t {{/each}} -type eventIdentifier = { - chainId: int, - blockTimestamp: int, - blockNumber: int, - logIndex: int, -} - -type entityUpdateAction<'entityType> = - | Set('entityType) - | Delete - -type entityUpdate<'entityType> = { - eventIdentifier: eventIdentifier, - entityId: id, - entityUpdateAction: entityUpdateAction<'entityType>, -} - -let mkEntityUpdate = (~eventIdentifier, ~entityId, entityUpdateAction) => { - entityId, - eventIdentifier, - entityUpdateAction, -} - -type entityValueAtStartOfBatch<'entityType> = - | NotSet // The entity isn't in the DB yet - | AlreadySet('entityType) - -type updatedValue<'entityType> = { - latest: entityUpdate<'entityType>, - history: array>, - // In the event of a rollback, some entity updates may have been - // been affected by a rollback diff. If there was no rollback diff - // this will always be false. - // If there was a rollback diff, this will be false in the case of a - // new entity update (where entity affected is not present in the diff) b - // but true if the update is related to an entity that is - // currently present in the diff - containsRollbackDiffChange: bool, -} - -@genType -type inMemoryStoreRowEntity<'entityType> = - | Updated(updatedValue<'entityType>) - | InitialReadFromDb(entityValueAtStartOfBatch<'entityType>) // This means there is no change from the db. - //************* //**CONTRACTS** //************* diff --git a/codegenerator/cli/templates/static/codegen/src/EventProcessing.res b/codegenerator/cli/templates/static/codegen/src/EventProcessing.res index a51a04045..92380a1c2 100644 --- a/codegenerator/cli/templates/static/codegen/src/EventProcessing.res +++ b/codegenerator/cli/templates/static/codegen/src/EventProcessing.res @@ -81,6 +81,7 @@ exception ProcessingError({message: string, exn: exn, item: Internal.item}) let runEventHandlerOrThrow = async ( item: Internal.item, + ~checkpointId, ~handler, ~inMemoryStore, ~loadManager, @@ -100,6 +101,7 @@ let runEventHandlerOrThrow = async ( event: eventItem.event, context: UserContext.getHandlerContext({ item, + checkpointId, inMemoryStore, loadManager, persistence, @@ -132,6 +134,7 @@ let runEventHandlerOrThrow = async ( let runHandlerOrThrow = async ( item: Internal.item, + ~checkpointId, ~inMemoryStore, ~loadManager, ~config: Config.t, @@ -154,6 +157,7 @@ let runHandlerOrThrow = async ( loadManager, persistence: config.persistence, shouldSaveHistory, + checkpointId, isPreload: false, }), }: Internal.onBlockArgs @@ -163,7 +167,7 @@ let runHandlerOrThrow = async ( | exn => raise( ProcessingError({ - message: "Unexpected error in the event handler. Please handle the error to keep the indexer running smoothly.", + message: "Unexpected error in the block handler. Please handle the error to keep the indexer running smoothly.", item, exn, }), @@ -174,6 +178,7 @@ let runHandlerOrThrow = async ( | Some(handler) => await item->runEventHandlerOrThrow( ~handler, + ~checkpointId, ~inMemoryStore, ~loadManager, ~persistence=config.persistence, @@ -196,15 +201,23 @@ let preloadBatchOrThrow = async (batch: Batch.t, ~loadManager, ~persistence, ~in // 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( - batch.items->Array.keepMap(item => { + let promises = [] + let itemIdx = ref(0) + + for checkpointIdx in 0 to batch.checkpointIds->Array.length - 1 { + let checkpointId = batch.checkpointIds->Js.Array2.unsafe_get(checkpointIdx) + let checkpointEventsProcessed = + batch.checkpointEventsProcessed->Js.Array2.unsafe_get(checkpointIdx) + + for idx in 0 to checkpointEventsProcessed - 1 { + let item = batch.items->Js.Array2.unsafe_get(itemIdx.contents + idx) switch item { | Event({eventConfig: {handler}, event}) => switch handler { - | None => None + | None => () | Some(handler) => try { - Some( + promises->Array.push( handler({ event, context: UserContext.getHandlerContext({ @@ -212,6 +225,7 @@ let preloadBatchOrThrow = async (batch: Batch.t, ~loadManager, ~persistence, ~in inMemoryStore, loadManager, persistence, + checkpointId, isPreload: true, shouldSaveHistory: false, }), @@ -221,12 +235,12 @@ let preloadBatchOrThrow = async (batch: Batch.t, ~loadManager, ~persistence, ~in // it won't create a rejected promise ) } catch { - | _ => None + | _ => () } } | Block({onBlockConfig: {handler, chainId}, blockNumber}) => try { - Some( + promises->Array.push( handler({ block: { number: blockNumber, @@ -237,37 +251,53 @@ let preloadBatchOrThrow = async (batch: Batch.t, ~loadManager, ~persistence, ~in inMemoryStore, loadManager, persistence, + checkpointId, isPreload: true, shouldSaveHistory: false, }), })->Promise.silentCatch, ) } catch { - | _ => None + | _ => () } } - }), - ) + } + + itemIdx := itemIdx.contents + checkpointEventsProcessed + } + + let _ = await Promise.all(promises) } let runBatchHandlersOrThrow = async ( - {items}: Batch.t, + batch: Batch.t, ~inMemoryStore, ~loadManager, ~config, ~shouldSaveHistory, ~shouldBenchmark, ) => { - for i in 0 to items->Array.length - 1 { - let item = items->Js.Array2.unsafe_get(i) - await runHandlerOrThrow( - item, - ~inMemoryStore, - ~loadManager, - ~config, - ~shouldSaveHistory, - ~shouldBenchmark, - ) + let itemIdx = ref(0) + + for checkpointIdx in 0 to batch.checkpointIds->Array.length - 1 { + let checkpointId = batch.checkpointIds->Js.Array2.unsafe_get(checkpointIdx) + let checkpointEventsProcessed = + batch.checkpointEventsProcessed->Js.Array2.unsafe_get(checkpointIdx) + + for idx in 0 to checkpointEventsProcessed - 1 { + let item = batch.items->Js.Array2.unsafe_get(itemIdx.contents + idx) + + await runHandlerOrThrow( + item, + ~checkpointId, + ~inMemoryStore, + ~loadManager, + ~config, + ~shouldSaveHistory, + ~shouldBenchmark, + ) + } + itemIdx := itemIdx.contents + checkpointEventsProcessed } } diff --git a/codegenerator/cli/templates/static/codegen/src/IO.res b/codegenerator/cli/templates/static/codegen/src/IO.res index 9c9174421..e7f0b95b4 100644 --- a/codegenerator/cli/templates/static/codegen/src/IO.res +++ b/codegenerator/cli/templates/static/codegen/src/IO.res @@ -12,42 +12,6 @@ let executeSet = ( } } -let getEntityHistoryItems = (entityUpdates, ~containsRollbackDiffChange) => { - let (_, entityHistoryItems) = entityUpdates->Belt.Array.reduce((None, []), ( - prev: (option, array>), - entityUpdate: Types.entityUpdate<'a>, - ) => { - let (optPreviousEventIdentifier, entityHistoryItems) = prev - - let {eventIdentifier, entityUpdateAction, entityId} = entityUpdate - let entityHistoryItems = { - let historyItem: EntityHistory.historyRow<_> = { - current: { - chain_id: eventIdentifier.chainId, - block_timestamp: eventIdentifier.blockTimestamp, - block_number: eventIdentifier.blockNumber, - log_index: eventIdentifier.logIndex, - }, - previous: optPreviousEventIdentifier->Belt.Option.map(prev => { - EntityHistory.chain_id: prev.chainId, - block_timestamp: prev.blockTimestamp, - block_number: prev.blockNumber, - log_index: prev.logIndex, - }), - entityData: switch entityUpdateAction { - | Set(entity) => Set(entity) - | Delete => Delete({id: entityId}) - }, - containsRollbackDiffChange, - } - entityHistoryItems->Belt.Array.concat([historyItem]) - } - (Some(eventIdentifier), entityHistoryItems) - }) - - entityHistoryItems -} - let executeBatch = async ( sql, ~batch: Batch.t, @@ -76,7 +40,6 @@ let executeBatch = async ( let setEntities = Entities.allEntities->Belt.Array.map(entityConfig => { let entitiesToSet = [] let idsToDelete = [] - let entityHistoryItemsToSet = [] let rows = inMemoryStore @@ -92,156 +55,180 @@ let executeBatch = async ( } }) - if shouldSaveHistory { - rows->Js.Array2.forEach(row => { - switch row { - | Updated({history, containsRollbackDiffChange}) => - let entityHistoryItems = history->getEntityHistoryItems(~containsRollbackDiffChange) - entityHistoryItemsToSet->Js.Array2.pushMany(entityHistoryItems)->ignore - | _ => () - } - }) - - // Keep history items in the order of the events. Without sorting, - // they will only be in order per row, but not across the whole entity - // table. - - switch config.multichain { - | Ordered => - let _ = entityHistoryItemsToSet->Js.Array2.sortInPlaceWith((a, b) => { - EventUtils.isEarlier( - ( - a.current.block_timestamp, - a.current.chain_id, - a.current.block_number, - a.current.log_index, - ), - ( - b.current.block_timestamp, - b.current.chain_id, - b.current.block_number, - b.current.log_index, - ), - ) - ? -1 - : 1 - }) - | Unordered => - let _ = entityHistoryItemsToSet->Js.Array2.sortInPlaceWith((a, b) => { - EventUtils.isEarlierUnordered( - (a.current.chain_id, a.current.block_number, a.current.log_index), - (b.current.chain_id, b.current.block_number, b.current.log_index), - ) - ? -1 - : 1 - }) - } - } - let shouldRemoveInvalidUtf8 = switch escapeTables { | Some(tables) if tables->Utils.Set.has(entityConfig.table) => true | _ => false } - sql => { - let promises = [] - if entityHistoryItemsToSet->Utils.Array.notEmpty { - promises - ->Js.Array2.pushMany( - sql->PgStorage.setEntityHistoryOrThrow( - ~entityHistory=entityConfig.entityHistory, - ~rows=entityHistoryItemsToSet, - ~shouldRemoveInvalidUtf8, - ), - ) - ->ignore - } - if entitiesToSet->Utils.Array.notEmpty { - if shouldRemoveInvalidUtf8 { - entitiesToSet->PgStorage.removeInvalidUtf8InPlace + async sql => { + try { + let promises = [] + + if shouldSaveHistory { + let backfillHistoryIds = Utils.Set.make() + let batchSetUpdates = [] + // Use unnest approach + let batchDeleteCheckpointIds = [] + let batchDeleteEntityIds = [] + + rows->Js.Array2.forEach(row => { + switch row { + | Updated({history, containsRollbackDiffChange}) => + history->Js.Array2.forEach( + (entityUpdate: EntityHistory.entityUpdate<'a>) => { + if !containsRollbackDiffChange { + // For every update we want to make sure that there's an existing history item + // with the current entity state. So we backfill history with checkpoint id 0, + // before writing updates. Don't do this if the update has a rollback diff change. + backfillHistoryIds->Utils.Set.add(entityUpdate.entityId)->ignore + } + switch entityUpdate.entityUpdateAction { + | Delete => { + batchDeleteEntityIds->Array.push(entityUpdate.entityId)->ignore + batchDeleteCheckpointIds->Array.push(entityUpdate.checkpointId)->ignore + } + | Set(_) => batchSetUpdates->Js.Array2.push(entityUpdate)->ignore + } + }, + ) + | _ => () + } + }) + + if backfillHistoryIds->Utils.Set.size !== 0 { + // This must run before updating entity or entity history tables + await EntityHistory.backfillHistory( + sql, + ~pgSchema=Db.publicSchema, + ~entityName=entityConfig.name, + ~ids=backfillHistoryIds->Utils.Set.toArray, + ) + } + + if batchDeleteCheckpointIds->Utils.Array.notEmpty { + promises->Array.push( + sql->EntityHistory.insertDeleteUpdates( + ~pgSchema=Db.publicSchema, + ~entityHistory=entityConfig.entityHistory, + ~batchDeleteEntityIds, + ~batchDeleteCheckpointIds, + ), + ) + } + + if batchSetUpdates->Utils.Array.notEmpty { + if shouldRemoveInvalidUtf8 { + let entities = batchSetUpdates->Js.Array2.map(batchSetUpdate => { + switch batchSetUpdate.entityUpdateAction { + | Set(entity) => entity + | _ => Js.Exn.raiseError("Expected Set action") + } + }) + entities->PgStorage.removeInvalidUtf8InPlace + } + + promises + ->Js.Array2.push( + sql->PgStorage.setOrThrow( + ~items=batchSetUpdates, + ~itemSchema=entityConfig.entityHistory.setUpdateSchema, + ~table=entityConfig.entityHistory.table, + ~pgSchema=Db.publicSchema, + ), + ) + ->ignore + } } - promises->Array.push( - sql->PgStorage.setOrThrow( - ~items=entitiesToSet, - ~table=entityConfig.table, - ~itemSchema=entityConfig.schema, - ~pgSchema=Config.storagePgSchema, - ), - ) - } - if idsToDelete->Utils.Array.notEmpty { - promises->Array.push(sql->DbFunctionsEntities.batchDelete(~entityConfig)(idsToDelete)) - } - // This should have await, to properly propagate errors to the caller. - promises - ->Promise.all + + if entitiesToSet->Utils.Array.notEmpty { + if shouldRemoveInvalidUtf8 { + entitiesToSet->PgStorage.removeInvalidUtf8InPlace + } + promises->Array.push( + sql->PgStorage.setOrThrow( + ~items=entitiesToSet, + ~table=entityConfig.table, + ~itemSchema=entityConfig.schema, + ~pgSchema=Config.storagePgSchema, + ), + ) + } + if idsToDelete->Utils.Array.notEmpty { + promises->Array.push(sql->DbFunctionsEntities.batchDelete(~entityConfig)(idsToDelete)) + } + + let _ = await promises->Promise.all + } catch { // There's a race condition that sql->Postgres.beginSql // might throw PG error, earlier, than the handled error // from setOrThrow will be passed through. // This is needed for the utf8 encoding fix. - ->Promise.catch(exn => { - /* Note: Entity History doesn't return StorageError yet, and directly throws JsError */ - let normalizedExn = switch exn { - | JsError(_) => exn - | Persistence.StorageError({reason: exn}) => exn - | _ => exn - }->Js.Exn.anyToExnInternal - - switch normalizedExn { - | JsError(error) => - // Workaround for https://github.com/enviodev/hyperindex/issues/446 - // We do escaping only when we actually got an error writing for the first time. - // This is not perfect, but an optimization to avoid escaping for every single item. - - switch error->S.parseOrThrow(PgStorage.pgErrorMessageSchema) { - | `current transaction is aborted, commands ignored until end of transaction block` => () - | `invalid byte sequence for encoding "UTF8": 0x00` => - // Since the transaction is aborted at this point, - // we can't simply retry the function with escaped items, - // so propagate the error, to restart the whole batch write. - // Also, pass the failing table, to escape only its items. - // TODO: Ideally all this should be done in the file, - // so it'll be easier to work on PG specific logic. - specificError.contents = Some(PgStorage.PgEncodingError({table: entityConfig.table})) - | _ => specificError.contents = Some(exn->Utils.prettifyExn) - | exception _ => () + | exn => { + /* Note: Entity History doesn't return StorageError yet, and directly throws JsError */ + let normalizedExn = switch exn { + | JsError(_) => exn + | Persistence.StorageError({reason: exn}) => exn + | _ => exn + }->Js.Exn.anyToExnInternal + + switch normalizedExn { + | JsError(error) => + // Workaround for https://github.com/enviodev/hyperindex/issues/446 + // We do escaping only when we actually got an error writing for the first time. + // This is not perfect, but an optimization to avoid escaping for every single item. + + switch error->S.parseOrThrow(PgStorage.pgErrorMessageSchema) { + | `current transaction is aborted, commands ignored until end of transaction block` => () + | `invalid byte sequence for encoding "UTF8": 0x00` => + // Since the transaction is aborted at this point, + // we can't simply retry the function with escaped items, + // so propagate the error, to restart the whole batch write. + // Also, pass the failing table, to escape only its items. + // TODO: Ideally all this should be done in the file, + // so it'll be easier to work on PG specific logic. + specificError.contents = Some(PgStorage.PgEncodingError({table: entityConfig.table})) + | _ => specificError.contents = Some(exn->Utils.prettifyExn) + | exception _ => () + } + | S.Raised(_) => raise(normalizedExn) // But rethrow this one, since it's not a PG error + | _ => () } - | _ => () - } - // Improtant: Don't rethrow here, since it'll result in - // an unhandled rejected promise error. - // That's fine not to throw, since sql->Postgres.beginSql - // will fail anyways. - Promise.resolve([]) - }) - ->(Utils.magic: promise> => promise) + // Improtant: Don't rethrow here, since it'll result in + // an unhandled rejected promise error. + // That's fine not to throw, since sql->Postgres.beginSql + // will fail anyways. + } + } } }) //In the event of a rollback, rollback all meta tables based on the given //valid event identifier, where all rows created after this eventIdentifier should //be deleted - let rollbackTables = switch inMemoryStore.rollBackEventIdentifier { - | Some(eventIdentifier) => + let rollbackTables = switch inMemoryStore { + | {rollbackTargetCheckpointId: Some(rollbackTargetCheckpointId)} => Some( - sql => - Promise.all2(( - sql->DbFunctions.EntityHistory.deleteAllEntityHistoryAfterEventIdentifier( - ~isUnorderedMultichainMode=switch config.multichain { - | Unordered => true - | Ordered => false - }, - ~eventIdentifier, - ), - sql->InternalTable.Checkpoints.deprecated_rollbackReorgedChainCheckpoints( + sql => { + let promises = Entities.allEntities->Js.Array2.map(entityConfig => { + sql->EntityHistory.rollback( + ~pgSchema=Db.publicSchema, + ~entityName=entityConfig.name, + ~rollbackTargetCheckpointId, + ) + }) + promises + ->Js.Array2.push( + sql->InternalTable.Checkpoints.rollback( ~pgSchema=Db.publicSchema, - ~chainId=eventIdentifier.chainId, - ~knownBlockNumber=eventIdentifier.blockNumber, + ~rollbackTargetCheckpointId, ), - )), + ) + ->ignore + Promise.all(promises) + }, ) - | None => None + | _ => None } try { @@ -331,83 +318,71 @@ let executeBatch = async ( } } -module RollBack = { - exception DecodeError(S.error) - let rollBack = async ( - ~chainId, - ~blockTimestamp, - ~blockNumber, - ~logIndex, - ~isUnorderedMultichainMode, - ) => { - let rollBackEventIdentifier: Types.eventIdentifier = { - chainId, - blockTimestamp, - blockNumber, - logIndex, - } - - let inMemStore = InMemoryStore.make(~rollBackEventIdentifier) +let prepareRollbackDiff = async (~rollbackTargetCheckpointId) => { + let inMemStore = InMemoryStore.make(~rollbackTargetCheckpointId) - let deletedEntities = Js.Dict.empty() - let setEntities = Js.Dict.empty() + let deletedEntities = Js.Dict.empty() + let setEntities = Js.Dict.empty() - let fullDiff: dict>> = Js.Dict.empty() + let _ = + await Entities.allEntities + ->Belt.Array.map(async entityConfig => { + let entityTable = inMemStore->InMemoryStore.getInMemTable(~entityConfig) - let _ = - await Entities.allEntities - ->Belt.Array.map(async entityConfig => { - let diff = await Db.sql->DbFunctions.EntityHistory.getRollbackDiff( - isUnorderedMultichainMode - ? UnorderedMultichain({ - reorgChainId: chainId, - safeBlockNumber: blockNumber, - }) - : OrderedMultichain({ - safeBlockTimestamp: blockTimestamp, - reorgChainId: chainId, - safeBlockNumber: blockNumber, - }), - ~entityConfig, + let (removedIdsResult, restoredEntitiesResult) = await Promise.all2(( + // Get IDs of entities that should be deleted (created after rollback target with no prior history) + Db.sql + ->Postgres.preparedUnsafe( + entityConfig.entityHistory.makeGetRollbackRemovedIdsQuery(~pgSchema=Db.publicSchema), + [rollbackTargetCheckpointId]->Utils.magic, ) - if diff->Utils.Array.notEmpty { - fullDiff->Js.Dict.set(entityConfig.name, diff) - } - - let entityTable = inMemStore->InMemoryStore.getInMemTable(~entityConfig) + ->(Utils.magic: promise => promise>), + // Get entities that should be restored to their state at or before rollback target + Db.sql + ->Postgres.preparedUnsafe( + entityConfig.entityHistory.makeGetRollbackRestoredEntitiesQuery( + ~pgSchema=Db.publicSchema, + ), + [rollbackTargetCheckpointId]->Utils.magic, + ) + ->(Utils.magic: promise => promise>), + )) + + // Process removed IDs + removedIdsResult->Js.Array2.forEach(data => { + deletedEntities->Utils.Dict.push(entityConfig.name, data["id"]) + entityTable->InMemoryTable.Entity.set( + { + entityId: data["id"], + checkpointId: 0, + entityUpdateAction: Delete, + }, + ~shouldSaveHistory=false, + ~containsRollbackDiffChange=true, + ) + }) - diff->Belt.Array.forEach(historyRow => { - let eventIdentifier: Types.eventIdentifier = { - chainId: historyRow.current.chain_id, - blockNumber: historyRow.current.block_number, - logIndex: historyRow.current.log_index, - blockTimestamp: historyRow.current.block_timestamp, - } - switch historyRow.entityData { - | Set(entity: Entities.internalEntity) => - setEntities->Utils.Dict.push(entityConfig.name, entity.id) - entityTable->InMemoryTable.Entity.set( - Set(entity)->Types.mkEntityUpdate(~eventIdentifier, ~entityId=entity.id), - ~shouldSaveHistory=false, - ~containsRollbackDiffChange=true, - ) - | Delete({id}) => - deletedEntities->Utils.Dict.push(entityConfig.name, id) - entityTable->InMemoryTable.Entity.set( - Delete->Types.mkEntityUpdate(~eventIdentifier, ~entityId=id), - ~shouldSaveHistory=false, - ~containsRollbackDiffChange=true, - ) - } - }) + let restoredEntities = restoredEntitiesResult->S.parseOrThrow(entityConfig.rowsSchema) + + // Process restored entities + restoredEntities->Belt.Array.forEach((entity: Entities.internalEntity) => { + setEntities->Utils.Dict.push(entityConfig.name, entity.id) + entityTable->InMemoryTable.Entity.set( + { + entityId: entity.id, + checkpointId: 0, + entityUpdateAction: Set(entity), + }, + ~shouldSaveHistory=false, + ~containsRollbackDiffChange=true, + ) }) - ->Promise.all + }) + ->Promise.all - { - "inMemStore": inMemStore, - "deletedEntities": deletedEntities, - "setEntities": setEntities, - "fullDiff": fullDiff, - } + { + "inMemStore": inMemStore, + "deletedEntities": deletedEntities, + "setEntities": setEntities, } } diff --git a/codegenerator/cli/templates/static/codegen/src/InMemoryStore.res b/codegenerator/cli/templates/static/codegen/src/InMemoryStore.res index ece1c75a5..9ebb198b0 100644 --- a/codegenerator/cli/templates/static/codegen/src/InMemoryStore.res +++ b/codegenerator/cli/templates/static/codegen/src/InMemoryStore.res @@ -52,17 +52,17 @@ type t = { rawEvents: InMemoryTable.t, entities: dict>, effects: dict, - rollBackEventIdentifier: option, + rollbackTargetCheckpointId: option, } let make = ( ~entities: array=Entities.allEntities, - ~rollBackEventIdentifier=?, + ~rollbackTargetCheckpointId=?, ): t => { rawEvents: InMemoryTable.make(~hash=hashRawEventsKey), entities: EntityTables.make(entities), effects: Js.Dict.empty(), - rollBackEventIdentifier, + rollbackTargetCheckpointId, } let clone = (self: t) => { @@ -73,7 +73,7 @@ let clone = (self: t) => { dict: table.dict->Utils.Dict.shallowCopy, effect: table.effect, }, self.effects), - rollBackEventIdentifier: self.rollBackEventIdentifier->Lodash.cloneDeep, + rollbackTargetCheckpointId: self.rollbackTargetCheckpointId, } let getEffectInMemTable = (inMemoryStore: t, ~effect: Internal.effect) => { @@ -98,48 +98,56 @@ let getInMemTable = ( inMemoryStore.entities->EntityTables.get(~entityName=entityConfig.name) } -let isRollingBack = (inMemoryStore: t) => inMemoryStore.rollBackEventIdentifier->Belt.Option.isSome +let isRollingBack = (inMemoryStore: t) => inMemoryStore.rollbackTargetCheckpointId !== None -let setDcsToStore = ( - inMemoryStore: t, - ~chainId: int, - ~dcs: array, - ~shouldSaveHistory, -) => { +let setBatchDcs = (inMemoryStore: t, ~batch: Batch.t, ~shouldSaveHistory) => { let inMemTable = inMemoryStore->getInMemTable( ~entityConfig=module(InternalTable.DynamicContractRegistry)->Entities.entityModToInternal, ) - 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, + let itemIdx = ref(0) + + for checkpoint in 0 to batch.checkpointIds->Array.length - 1 { + let checkpointId = batch.checkpointIds->Js.Array2.unsafe_get(checkpoint) + let chainId = batch.checkpointChainIds->Js.Array2.unsafe_get(checkpoint) + let checkpointEventsProcessed = + batch.checkpointEventsProcessed->Js.Array2.unsafe_get(checkpoint) + + for idx in 0 to checkpointEventsProcessed - 1 { + let item = batch.items->Js.Array2.unsafe_get(itemIdx.contents + idx) + switch item->Internal.getItemDcs { + | None => () + | Some(dcs) => + // Currently only events support contract registration, so we can cast to event item + let eventItem = item->Internal.castUnsafeEventItem + for dcIdx in 0 to dcs->Array.length - 1 { + let dc = dcs->Js.Array2.unsafe_get(dcIdx) + let entity: InternalTable.DynamicContractRegistry.t = { + id: InternalTable.DynamicContractRegistry.makeId(~chainId, ~contractAddress=dc.address), + chainId, + contractAddress: dc.address, + contractName: dc.contractName, + registeringEventBlockNumber: eventItem.blockNumber, + registeringEventLogIndex: eventItem.logIndex, + registeringEventBlockTimestamp: eventItem.timestamp, + registeringEventContractName: eventItem.eventConfig.contractName, + registeringEventName: eventItem.eventConfig.name, + registeringEventSrcAddress: eventItem.event.srcAddress, + } + + inMemTable->InMemoryTable.Entity.set( + { + entityId: entity.id, + checkpointId, + entityUpdateAction: Set(entity->InternalTable.DynamicContractRegistry.castToInternal), + }, + ~shouldSaveHistory, + ) + } + } } - inMemTable->InMemoryTable.Entity.set( - Set(entity->InternalTable.DynamicContractRegistry.castToInternal)->Types.mkEntityUpdate( - ~eventIdentifier, - ~entityId=entity.id, - ), - ~shouldSaveHistory, - ) - }) + + itemIdx := itemIdx.contents + checkpointEventsProcessed + } } diff --git a/codegenerator/cli/templates/static/codegen/src/InMemoryTable.res b/codegenerator/cli/templates/static/codegen/src/InMemoryTable.res index d3b4e7886..1d27e5ce4 100644 --- a/codegenerator/cli/templates/static/codegen/src/InMemoryTable.res +++ b/codegenerator/cli/templates/static/codegen/src/InMemoryTable.res @@ -39,7 +39,7 @@ module Entity = { type indexFieldNameToIndices = t type entityWithIndices<'entity> = { - entityRow: Types.inMemoryStoreRowEntity<'entity>, + entityRow: Internal.inMemoryStoreRowEntity<'entity>, entityIndices: Utils.Set.t, } type t<'entity> = { @@ -136,7 +136,7 @@ module Entity = { //or if allowOverWriteEntity is true (used for mockDb in test helpers) if shouldWriteEntity { let entityIndices = Utils.Set.make() - let initialStoreRow: Types.inMemoryStoreRowEntity<'entity> = switch entity { + let initialStoreRow: Internal.inMemoryStoreRowEntity<'entity> = switch entity { | Some(entity) => //update table indices in the case where there //is an already set entity @@ -155,16 +155,15 @@ module Entity = { let setRow = set let set = ( inMemTable: t<'entity>, - entityUpdate: Types.entityUpdate<'entity>, + entityUpdate: EntityHistory.entityUpdate<'entity>, ~shouldSaveHistory, ~containsRollbackDiffChange=false, ) => { //New entity row with only the latest update @inline - let newEntityRow = () => Types.Updated({ + let newEntityRow = () => Internal.Updated({ latest: entityUpdate, history: shouldSaveHistory ? [entityUpdate] : [], - // For new entities, apply "containsRollbackDiffChange" from param containsRollbackDiffChange, }) @@ -176,27 +175,22 @@ module Entity = { } | Some({entityRow: Updated(previous_values), entityIndices}) // This prevents two db actions in the same event on the same entity from being recorded to the history table. - if shouldSaveHistory && - previous_values.latest.eventIdentifier == entityUpdate.eventIdentifier => - let entityRow = Types.Updated({ + if shouldSaveHistory && previous_values.latest.checkpointId === entityUpdate.checkpointId => + let entityRow = Internal.Updated({ latest: entityUpdate, history: previous_values.history->Utils.Array.setIndexImmutable( previous_values.history->Array.length - 1, entityUpdate, ), - // For updated entities, apply "containsRollbackDiffChange" from previous values - // (so that the first change if from a rollback diff applies throughout the batch) containsRollbackDiffChange: previous_values.containsRollbackDiffChange, }) {entityRow, entityIndices} | Some({entityRow: Updated(previous_values), entityIndices}) => - let entityRow = Types.Updated({ + let entityRow = Internal.Updated({ latest: entityUpdate, history: shouldSaveHistory ? [...previous_values.history, entityUpdate] : previous_values.history, - // For updated entities, apply "containsRollbackDiffChange" from previous values - // (so that the first change if from a rollback diff applies throughout the batch) containsRollbackDiffChange: previous_values.containsRollbackDiffChange, }) {entityRow, entityIndices} @@ -211,7 +205,7 @@ module Entity = { let rowToEntity = row => switch row.entityRow { - | Types.Updated({latest: {entityUpdateAction: Set(entity)}}) => Some(entity) + | Internal.Updated({latest: {entityUpdateAction: Set(entity)}}) => Some(entity) | Updated({latest: {entityUpdateAction: Delete}}) => None | InitialReadFromDb(AlreadySet(entity)) => Some(entity) | InitialReadFromDb(NotSet) => None diff --git a/codegenerator/cli/templates/static/codegen/src/UserContext.res b/codegenerator/cli/templates/static/codegen/src/UserContext.res index 792486352..d1daf9107 100644 --- a/codegenerator/cli/templates/static/codegen/src/UserContext.res +++ b/codegenerator/cli/templates/static/codegen/src/UserContext.res @@ -1,24 +1,8 @@ let codegenHelpMessage = `Rerun 'pnpm dev' to update generated code after schema.graphql changes.` -let makeEventIdentifier = (item: Internal.item): Types.eventIdentifier => { - switch item { - | Internal.Event({chain, blockNumber, logIndex, timestamp}) => { - chainId: chain->ChainMap.Chain.toChainId, - blockTimestamp: timestamp, - blockNumber, - logIndex, - } - | Internal.Block({onBlockConfig: {chainId}, blockNumber, logIndex}) => { - chainId, - blockTimestamp: 0, - blockNumber, - logIndex, - } - } -} - type contextParams = { item: Internal.item, + checkpointId: int, inMemoryStore: InMemoryStore.t, loadManager: LoadManager.t, persistence: Persistence.t, @@ -133,10 +117,11 @@ let entityTraps: Utils.Proxy.traps = { params.inMemoryStore ->InMemoryStore.getInMemTable(~entityConfig=params.entityConfig) ->InMemoryTable.Entity.set( - Set(entity)->Types.mkEntityUpdate( - ~eventIdentifier=params.item->makeEventIdentifier, - ~entityId=entity.id, - ), + { + entityId: entity.id, + checkpointId: params.checkpointId, + entityUpdateAction: Set(entity), + }, ~shouldSaveHistory=params.shouldSaveHistory, ) } @@ -209,10 +194,11 @@ let entityTraps: Utils.Proxy.traps = { params.inMemoryStore ->InMemoryStore.getInMemTable(~entityConfig=params.entityConfig) ->InMemoryTable.Entity.set( - Delete->Types.mkEntityUpdate( - ~eventIdentifier=params.item->makeEventIdentifier, - ~entityId, - ), + { + entityId, + checkpointId: params.checkpointId, + entityUpdateAction: Delete, + }, ~shouldSaveHistory=params.shouldSaveHistory, ) } @@ -246,6 +232,7 @@ let handlerTraps: Utils.Proxy.traps = { loadManager: params.loadManager, persistence: params.persistence, shouldSaveHistory: params.shouldSaveHistory, + checkpointId: params.checkpointId, entityConfig, } ->Utils.Proxy.make(entityTraps) diff --git a/codegenerator/cli/templates/static/codegen/src/db/DbFunctions.res b/codegenerator/cli/templates/static/codegen/src/db/DbFunctions.res index 5136fb711..f4d6ab967 100644 --- a/codegenerator/cli/templates/static/codegen/src/db/DbFunctions.res +++ b/codegenerator/cli/templates/static/codegen/src/db/DbFunctions.res @@ -1,6 +1,3 @@ -type chainId = int -type eventId = string - module General = { type existsRes = {exists: bool} @@ -13,286 +10,7 @@ module General = { } } -module DynamicContractRegistry = { - @module("./DbFunctionsImplementation.js") - external readAllDynamicContractsRaw: (Postgres.sql, ~chainId: chainId) => promise = - "readAllDynamicContracts" - - let readAllDynamicContracts = async (sql: Postgres.sql, ~chainId: chainId) => { - let json = await sql->readAllDynamicContractsRaw(~chainId) - json->S.parseJsonOrThrow(InternalTable.DynamicContractRegistry.rowsSchema) - } -} - module EntityHistory = { - type dynamicSqlQuery - module UnorderedMultichain = { - @module("./DbFunctionsImplementation.js") - external getFirstChangeSerial: ( - Postgres.sql, - ~reorgChainId: int, - ~safeBlockNumber: int, - ~entityName: string, - ) => dynamicSqlQuery = "getFirstChangeSerial_UnorderedMultichain" - } - - module OrderedMultichain = { - @module("./DbFunctionsImplementation.js") - external getFirstChangeSerial: ( - Postgres.sql, - ~safeBlockTimestamp: int, - ~reorgChainId: int, - ~safeBlockNumber: int, - ~entityName: string, - ) => dynamicSqlQuery = "getFirstChangeSerial_OrderedMultichain" - } - - @module("./DbFunctionsImplementation.js") - external getFirstChangeEntityHistoryPerChain: ( - Postgres.sql, - ~entityName: string, - ~getFirstChangeSerial: Postgres.sql => dynamicSqlQuery, - ) => promise = "getFirstChangeEntityHistoryPerChain" - - @module("./DbFunctionsImplementation.js") - external getRollbackDiffInternal: ( - Postgres.sql, - ~entityName: string, - ~getFirstChangeSerial: Postgres.sql => dynamicSqlQuery, - ) => //Returns an array of entity history rows - promise = "getRollbackDiff" - - @module("./DbFunctionsImplementation.js") - external deleteRolledBackEntityHistory: ( - Postgres.sql, - ~entityName: string, - ~getFirstChangeSerial: Postgres.sql => dynamicSqlQuery, - ) => promise = "deleteRolledBackEntityHistory" - - let rollbacksGroup = "Rollbacks" - - module Args = { - type t = - | OrderedMultichain({safeBlockTimestamp: int, reorgChainId: int, safeBlockNumber: int}) - | UnorderedMultichain({reorgChainId: int, safeBlockNumber: int}) - - /** - Uses two different methods for determining the first change event after rollback block - - This is needed since unordered multichain mode only cares about any changes that - occurred after the first change on the reorg chain. To prevent skipping or double processing events - on the other chains. If for instance there are no entity changes based on the reorg chain, the other - chains do not need to be rolled back, and if the reorg chain has new included events, it does not matter - that if those events are processed out of order from other chains since this is "unordered_multichain_mode" - - Ordered multichain mode needs to ensure that all chains rollback to any event that occurred after the reorg chain - block number. Regardless of whether the reorg chain incurred any changes or not to entities. - */ - let makeGetFirstChangeSerial = (self: t, ~entityName) => - switch self { - | OrderedMultichain({safeBlockTimestamp, reorgChainId, safeBlockNumber}) => - sql => - OrderedMultichain.getFirstChangeSerial( - sql, - ~safeBlockTimestamp, - ~reorgChainId, - ~safeBlockNumber, - ~entityName, - ) - | UnorderedMultichain({reorgChainId, safeBlockNumber}) => - sql => - UnorderedMultichain.getFirstChangeSerial( - sql, - ~reorgChainId, - ~safeBlockNumber, - ~entityName, - ) - } - - let getLogger = (self: t, ~entityName) => { - switch self { - | OrderedMultichain({safeBlockTimestamp, reorgChainId, safeBlockNumber}) => - Logging.createChild( - ~params={ - "type": "OrderedMultichain", - "safeBlockTimestamp": safeBlockTimestamp, - "reorgChainId": reorgChainId, - "safeBlockNumber": safeBlockNumber, - "entityName": entityName, - }, - ) - | UnorderedMultichain({reorgChainId, safeBlockNumber}) => - Logging.createChild( - ~params={ - "type": "UnorderedMultichain", - "reorgChainId": reorgChainId, - "safeBlockNumber": safeBlockNumber, - "entityName": entityName, - }, - ) - } - } - } - - let deleteAllEntityHistoryAfterEventIdentifier = async ( - sql, - ~isUnorderedMultichainMode, - ~eventIdentifier: Types.eventIdentifier, - ~allEntities=Entities.allEntities, - ): unit => { - let startTime = Hrtime.makeTimer() - - let {chainId, blockNumber, blockTimestamp} = eventIdentifier - let args: Args.t = isUnorderedMultichainMode - ? UnorderedMultichain({reorgChainId: chainId, safeBlockNumber: blockNumber}) - : OrderedMultichain({ - reorgChainId: chainId, - safeBlockNumber: blockNumber, - safeBlockTimestamp: blockTimestamp, - }) - - let _ = - await allEntities - ->Belt.Array.map(async entityConfig => { - try await deleteRolledBackEntityHistory( - sql, - ~entityName=entityConfig.name, - ~getFirstChangeSerial=args->Args.makeGetFirstChangeSerial(~entityName=entityConfig.name), - ) catch { - | exn => - exn->ErrorHandling.mkLogAndRaise( - ~msg=`Failed to delete rolled back entity history`, - ~logger=args->Args.getLogger(~entityName=entityConfig.name), - ) - } - }) - ->Promise.all - - if Env.Benchmark.shouldSaveData { - let elapsedTimeMillis = Hrtime.timeSince(startTime)->Hrtime.toMillis->Hrtime.floatFromMillis - - Benchmark.addSummaryData( - ~group=rollbacksGroup, - ~label=`Delete Rolled Back History Time (ms)`, - ~value=elapsedTimeMillis, - ) - } - } - - let getRollbackDiff = async (sql, args: Args.t, ~entityConfig: Internal.entityConfig) => { - let startTime = Hrtime.makeTimer() - - let diffRes = switch await getRollbackDiffInternal( - sql, - ~getFirstChangeSerial=args->Args.makeGetFirstChangeSerial(~entityName=entityConfig.name), - ~entityName=entityConfig.name, - ) { - | exception exn => - exn->ErrorHandling.mkLogAndRaise( - ~msg="Failed to get rollback diff from entity history", - ~logger=args->Args.getLogger(~entityName=entityConfig.name), - ) - | res => res - } - - if Env.Benchmark.shouldSaveData { - let elapsedTimeMillis = Hrtime.timeSince(startTime)->Hrtime.toMillis->Hrtime.floatFromMillis - - Benchmark.addSummaryData( - ~group=rollbacksGroup, - ~label=`Diff Creation Time (ms)`, - ~value=elapsedTimeMillis, - ) - } - - switch diffRes->S.parseOrThrow(entityConfig.entityHistory.schemaRows) { - | exception exn => - exn->ErrorHandling.mkLogAndRaise( - ~msg="Failed to parse rollback diff from entity history", - ~logger=args->Args.getLogger(~entityName=entityConfig.name), - ) - | diffRows => diffRows - } - } - - module FirstChangeEventPerChain = { - type t = Js.Dict.t - let getKey = chainId => chainId->Belt.Int.toString - let make = () => Js.Dict.empty() - let get = (self: t, ~chainId) => self->Utils.Dict.dangerouslyGetNonOption(getKey(chainId)) - - let setIfEarlier = (self: t, ~chainId, ~event: FetchState.blockNumberAndLogIndex) => { - let chainKey = chainId->Belt.Int.toString - switch self->Utils.Dict.dangerouslyGetNonOption(chainKey) { - | Some(existingEvent) => - if ( - (event.blockNumber, event.logIndex) < (existingEvent.blockNumber, existingEvent.logIndex) - ) { - self->Js.Dict.set(chainKey, event) - } - | None => self->Js.Dict.set(chainKey, event) - } - } - } - - let getFirstChangeEventPerChain = async ( - sql, - args: Args.t, - ~allEntities=Entities.allEntities, - ) => { - let startTime = Hrtime.makeTimer() - let firstChangeEventPerChain = FirstChangeEventPerChain.make() - - let _ = - await allEntities - ->Belt.Array.map(async entityConfig => { - let res = try await getFirstChangeEntityHistoryPerChain( - sql, - ~entityName=entityConfig.name, - ~getFirstChangeSerial=args->Args.makeGetFirstChangeSerial(~entityName=entityConfig.name), - ) catch { - | exn => - exn->ErrorHandling.mkLogAndRaise( - ~msg=`Failed to get first change entity history per chain for entity`, - ~logger=args->Args.getLogger(~entityName=entityConfig.name), - ) - } - - let chainHistoryRows = try res->S.parseOrThrow( - entityConfig.entityHistory.schemaRows, - ) catch { - | exn => - exn->ErrorHandling.mkLogAndRaise( - ~msg=`Failed to parse entity history rows from db on getFirstChangeEntityHistoryPerChain`, - ~logger=args->Args.getLogger(~entityName=entityConfig.name), - ) - } - - chainHistoryRows->Belt.Array.forEach(chainHistoryRow => { - firstChangeEventPerChain->FirstChangeEventPerChain.setIfEarlier( - ~chainId=chainHistoryRow.current.chain_id, - ~event={ - blockNumber: chainHistoryRow.current.block_number, - logIndex: chainHistoryRow.current.log_index, - }, - ) - }) - }) - ->Promise.all - - if Env.Benchmark.shouldSaveData { - let elapsedTimeMillis = Hrtime.timeSince(startTime)->Hrtime.toMillis->Hrtime.floatFromMillis - - Benchmark.addSummaryData( - ~group=rollbacksGroup, - ~label=`Get First Change Event Per Chain Time (ms)`, - ~value=elapsedTimeMillis, - ) - } - - firstChangeEventPerChain - } - let hasRows = async sql => { let all = await Entities.allEntities diff --git a/codegenerator/cli/templates/static/codegen/src/db/DbFunctionsImplementation.js b/codegenerator/cli/templates/static/codegen/src/db/DbFunctionsImplementation.js index b83191a7e..d36a66914 100644 --- a/codegenerator/cli/templates/static/codegen/src/db/DbFunctionsImplementation.js +++ b/codegenerator/cli/templates/static/codegen/src/db/DbFunctionsImplementation.js @@ -15,174 +15,3 @@ module.exports.batchDeleteItemsInTable = (table, sql, pkArray) => { //May be best to make pkArray an array of objects with fieldName -> value } }; - -module.exports.readAllDynamicContracts = (sql, chainId) => sql` - SELECT * - FROM ${sql(publicSchema)}."dynamic_contract_registry" - WHERE chain_id = ${chainId};`; - -const makeHistoryTableName = (entityName) => entityName + "_history"; - -/** - Find the "first change" serial originating from the reorg chain above the safe block number - (Using serial to account for unordered multi chain reorgs, where an earier event on another chain could be rolled back) - - If for instance there are no entity changes based on the reorg chain, the other - chains do not need to be rolled back, and if the reorg chain has new included events, it does not matter - that if those events are processed out of order from other chains since this is "unordered_multichain_mode" -*/ -module.exports.getFirstChangeSerial_UnorderedMultichain = ( - sql, - reorgChainId, - safeBlockNumber, - entityName -) => - sql` - SELECT - MIN(serial) AS first_change_serial - FROM - ${sql(publicSchema)}.${sql(makeHistoryTableName(entityName))} - WHERE - entity_history_chain_id = ${reorgChainId} - AND entity_history_block_number > ${safeBlockNumber} - `; - -/** - Find the "first change" serial originating from any chain above the provided safe block - - Ordered multichain mode needs to ensure that all chains rollback to any event that occurred after the reorg chain - block number. Regardless of whether the reorg chain incurred any changes or not to entities. There could be no changes - on the orphaned blocks, but new changes on the reorged blocks where other chains need to be processed in order after this - fact. -*/ -module.exports.getFirstChangeSerial_OrderedMultichain = ( - sql, - safeBlockTimestamp, - reorgChainId, - safeBlockNumber, - entityName -) => - sql` - SELECT - MIN(serial) AS first_change_serial - FROM - ${sql(publicSchema)}.${sql(makeHistoryTableName(entityName))} - WHERE - entity_history_block_timestamp > ${safeBlockTimestamp} - OR - (entity_history_block_timestamp = ${safeBlockTimestamp} AND entity_history_chain_id > ${reorgChainId}) - OR - (entity_history_block_timestamp = ${safeBlockTimestamp} AND entity_history_chain_id = ${reorgChainId} AND entity_history_block_number > ${safeBlockNumber}) - `; - -module.exports.getFirstChangeEntityHistoryPerChain = ( - sql, - entityName, - getFirstChangeSerial -) => sql` - WITH - first_change AS ( - -- Step 1: Find the "first change" serial originating from the reorg chain above the safe block number - -- (Using serial to account for unordered multi chain reorgs, where an earier event on another chain could be rolled back) - ${getFirstChangeSerial(sql)} - ) - -- Step 2: Distinct on entity_history_chain_id, get the entity_history_block_number of the row with the - -- lowest serial >= the first change serial - SELECT DISTINCT - ON (entity_history_chain_id) * - FROM - ${sql(publicSchema)}.${sql(makeHistoryTableName(entityName))} - WHERE - serial >= ( - SELECT - first_change_serial - FROM - first_change - ) - ORDER BY - entity_history_chain_id, - serial - ASC; -- Select the row with the lowest serial per id -`; - -module.exports.deleteRolledBackEntityHistory = ( - sql, - entityName, - getFirstChangeSerial -) => sql` - WITH - first_change AS ( - -- Step 1: Find the "first change" serial originating from the reorg chain above the safe block number - -- (Using serial to account for unordered multi chain reorgs, where an earier event on another chain could be rolled back) - ${getFirstChangeSerial(sql)} - ) - -- Step 2: Delete all rows that have a serial >= the first change serial - DELETE FROM - ${sql(publicSchema)}.${sql(makeHistoryTableName(entityName))} - WHERE - serial >= ( - SELECT - first_change_serial - FROM - first_change - ) - -- Filter out rows with a chain_id of 0 since they are the copied history rows - -- check timestamp as well in case a future chain is added with id of 0 - AND NOT ( - entity_history_chain_id = 0 AND - entity_history_block_timestamp = 0 - ); - `; - -module.exports.getRollbackDiff = (sql, entityName, getFirstChangeSerial) => sql` - WITH - first_change AS ( - -- Step 1: Find the "first change" serial originating from the reorg chain above the safe block number - -- (Using serial to account for unordered multi chain reorgs, where an earier event on another chain could be rolled back) - ${getFirstChangeSerial(sql)} - ), - rollback_ids AS ( - -- Step 2: Get all unique entity ids of rows that require rollbacks where the row's serial is above the first change serial - SELECT DISTINCT - ON (id) after.* - FROM - ${sql(publicSchema)}.${sql(makeHistoryTableName(entityName))} after - WHERE - after.serial >= ( - SELECT - first_change_serial - FROM - first_change - ) - -- Filter out rows with a chain_id of 0 since they are the copied history rows - -- check timestamp as well in case a future chain is added with id of 0 - AND NOT ( - after.entity_history_chain_id = 0 AND - after.entity_history_block_timestamp = 0 - ) - ORDER BY - after.id, - after.serial ASC -- Select the row with the lowest serial per id - ) - -- Step 3: For each relevant id, join to the row on the "previous_entity_history" fields - SELECT - -- Select all before fields, overriding the needed values with defaults - before.*, - -- In the case where no previous row exists, coalesce the needed values since this new entity - -- will need to be deleted - COALESCE(before.id, after.id) AS id, - COALESCE(before.action, 'DELETE') AS action, - -- Deleting at 0 values will work fine for future rollbacks - COALESCE(before.entity_history_block_number, 0) AS entity_history_block_number, - COALESCE(before.entity_history_block_timestamp, 0) AS entity_history_block_timestamp, - COALESCE(before.entity_history_chain_id, 0) AS entity_history_chain_id, - COALESCE(before.entity_history_log_index, 0) AS entity_history_log_index - FROM - -- Use a RIGHT JOIN, to ensure that nulls get returned if there is no "before" row - ${sql(publicSchema)}.${sql(makeHistoryTableName(entityName))} before - RIGHT JOIN rollback_ids after ON before.id = after.id - AND before.entity_history_block_timestamp = after.previous_entity_history_block_timestamp - AND before.entity_history_chain_id = after.previous_entity_history_chain_id - AND before.entity_history_block_number = after.previous_entity_history_block_number - AND before.entity_history_log_index = after.previous_entity_history_log_index; -`; diff --git a/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainFetcher.res b/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainFetcher.res index 00d3d089b..3cf590642 100644 --- a/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainFetcher.res +++ b/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainFetcher.res @@ -21,15 +21,13 @@ type t = { numEventsProcessed: int, numBatchesFetched: int, reorgDetection: ReorgDetection.t, - //An optional list of filters to apply on event queries - //Used for reorgs and restarts - processingFilters: option>, + safeCheckpointTracking: option, } //CONSTRUCTION let make = ( ~chainConfig: InternalConfig.chain, - ~dynamicContracts: array, + ~dynamicContracts: array, ~startBlock, ~endBlock, ~firstEventBlockNumber, @@ -100,31 +98,18 @@ let make = ( contract.addresses->Array.forEach(address => { contracts->Array.push({ - FetchState.address, + Internal.address, contractName: contract.name, startBlock: switch contract.startBlock { | Some(startBlock) => startBlock | None => chainConfig.startBlock }, - register: Config, + registrationBlock: None, }) }) }) - dynamicContracts->Array.forEach(dc => - contracts->Array.push({ - FetchState.address: dc.contractAddress, - contractName: dc.contractName, - startBlock: dc.registeringEventBlockNumber, - register: DC({ - registeringEventLogIndex: dc.registeringEventLogIndex, - registeringEventBlockTimestamp: dc.registeringEventBlockTimestamp, - registeringEventContractName: dc.registeringEventContractName, - registeringEventName: dc.registeringEventName, - registeringEventSrcAddress: dc.registeringEventSrcAddress, - }), - }) - ) + dynamicContracts->Array.forEach(dc => contracts->Array.push(dc)) if notRegisteredEvents->Utils.Array.notEmpty { logger->Logging.childInfo( @@ -187,6 +172,14 @@ let make = ( ~onBlockConfigs?, ) + let chainReorgCheckpoints = reorgCheckpoints->Array.keepMapU(reorgCheckpoint => { + if reorgCheckpoint.chainId === chainConfig.id { + Some(reorgCheckpoint) + } else { + None + } + }) + { logger, chainConfig, @@ -195,19 +188,15 @@ let make = ( ~maxPartitionConcurrency=Env.maxPartitionConcurrency, ), reorgDetection: ReorgDetection.make( - ~blocks=reorgCheckpoints->Array.keepMapU(reorgCheckpoint => { - if reorgCheckpoint.chainId === chainConfig.id { - Some({ - ReorgDetection.blockNumber: reorgCheckpoint.blockNumber, - blockHash: reorgCheckpoint.blockHash, - }) - } else { - None - } - }), + ~chainReorgCheckpoints, ~maxReorgDepth, ~shouldRollbackOnReorg=config->Config.shouldRollbackOnReorg, ), + safeCheckpointTracking: SafeCheckpointTracking.make( + ~maxReorgDepth, + ~shouldRollbackOnReorg=config->Config.shouldRollbackOnReorg, + ~chainReorgCheckpoints, + ), currentBlockHeight: 0, isProgressAtHead: false, fetchState, @@ -216,7 +205,6 @@ let make = ( timestampCaughtUpToHeadOrEndblock, numEventsProcessed, numBatchesFetched, - processingFilters: None, } } @@ -231,7 +219,7 @@ let makeFromConfig = (chainConfig: InternalConfig.chain, ~config, ~targetBufferS ~reorgCheckpoints=[], ~maxReorgDepth=chainConfig.maxReorgDepth, ~firstEventBlockNumber=None, - ~progressBlockNumber=chainConfig.startBlock - 1, + ~progressBlockNumber=-1, ~timestampCaughtUpToHeadOrEndblock=None, ~numEventsProcessed=0, ~numBatchesFetched=0, @@ -247,21 +235,15 @@ let makeFromConfig = (chainConfig: InternalConfig.chain, ~config, ~targetBufferS */ let makeFromDbState = async ( chainConfig: InternalConfig.chain, - ~resumedChainState: InternalTable.Chains.t, + ~resumedChainState: Persistence.initialChainState, ~reorgCheckpoints, ~isInReorgThreshold, ~config, ~targetBufferSize, - ~sql=Db.sql, ) => { let chainId = chainConfig.id let logger = Logging.createChild(~params={"chainId": chainId}) - // Since we deleted all contracts after the restart point, - // we can simply query all dcs we have in db - let dbRecoveredDynamicContracts = - await sql->DbFunctions.DynamicContractRegistry.readAllDynamicContracts(~chainId) - Prometheus.ProgressEventsCount.set(~processedCount=resumedChainState.numEventsProcessed, ~chainId) let progressBlockNumber = @@ -271,18 +253,18 @@ let makeFromDbState = async ( : resumedChainState.startBlock - 1 make( - ~dynamicContracts=dbRecoveredDynamicContracts, + ~dynamicContracts=resumedChainState.dynamicContracts, ~chainConfig, ~startBlock=resumedChainState.startBlock, - ~endBlock=resumedChainState.endBlock->Js.Null.toOption, + ~endBlock=resumedChainState.endBlock, ~config, ~reorgCheckpoints, ~maxReorgDepth=resumedChainState.maxReorgDepth, - ~firstEventBlockNumber=resumedChainState.firstEventBlockNumber->Js.Null.toOption, + ~firstEventBlockNumber=resumedChainState.firstEventBlockNumber, ~progressBlockNumber, ~timestampCaughtUpToHeadOrEndblock=Env.updateSyncTimeOnRestart ? None - : resumedChainState.timestampCaughtUpToHeadOrEndblock->Js.Null.toOption, + : resumedChainState.timestampCaughtUpToHeadOrEndblock, ~numEventsProcessed=resumedChainState.numEventsProcessed, ~numBatchesFetched=0, ~logger, @@ -291,34 +273,6 @@ let makeFromDbState = async ( ) } -/** -Adds an event filter that will be passed to worker on query -isValid is a function that determines when the filter -should be cleaned up -*/ -let addProcessingFilter = (self: t, ~filter, ~isValid) => { - let processingFilter: processingFilter = {filter, isValid} - { - ...self, - processingFilters: switch self.processingFilters { - | Some(processingFilters) => Some(processingFilters->Array.concat([processingFilter])) - | None => Some([processingFilter]) - }, - } -} - -//Run the clean up condition "isNoLongerValid" against fetchState on each eventFilter and remove -//any that meet the cleanup condition -let cleanUpProcessingFilters = ( - processingFilters: array, - ~fetchState: FetchState.t, -) => { - switch processingFilters->Array.keep(processingFilter => processingFilter.isValid(~fetchState)) { - | [] => None - | filters => Some(filters) - } -} - /** * Helper function to get the configured start block for a contract from config */ @@ -338,7 +292,7 @@ let runContractRegistersOrThrow = async ( ~chain: ChainMap.Chain.t, ~config: Config.t, ) => { - let dynamicContracts = [] + let itemsWithDcs = [] let isDone = ref(false) let onRegister = (~item: Internal.item, ~contractAddress, ~contractName) => { @@ -349,7 +303,7 @@ let runContractRegistersOrThrow = async ( `Skipping contract registration: The context.add${(contractName: Enums.ContractType.t :> string)} was called after the contract register resolved. Use await or return a promise from the contract register handler to avoid this error.`, ) } else { - let {timestamp, blockNumber, logIndex, eventConfig, event} = eventItem + let {blockNumber} = eventItem // Use contract-specific start block if configured, otherwise fall back to registration block let contractStartBlock = switch getContractStartBlock( @@ -361,20 +315,20 @@ let runContractRegistersOrThrow = async ( | None => blockNumber } - let dc: FetchState.indexingContract = { + let dc: Internal.indexingContract = { address: contractAddress, contractName: (contractName: Enums.ContractType.t :> string), startBlock: contractStartBlock, - register: DC({ - registeringEventBlockTimestamp: timestamp, - registeringEventLogIndex: logIndex, - registeringEventName: eventConfig.name, - registeringEventContractName: eventConfig.contractName, - registeringEventSrcAddress: event.srcAddress, - }), + registrationBlock: Some(blockNumber), } - dynamicContracts->Array.push(dc) + switch item->Internal.getItemDcs { + | None => { + item->Internal.setItemDcs([dc]) + itemsWithDcs->Array.push(item) + } + | Some(dcs) => dcs->Array.push(dc) + } } } @@ -417,42 +371,26 @@ let runContractRegistersOrThrow = async ( } isDone.contents = true - dynamicContracts -} - -@inline -let applyProcessingFilters = (~item: Internal.item, ~processingFilters) => { - processingFilters->Js.Array2.every(processingFilter => processingFilter.filter(item)) + itemsWithDcs } -/** -Updates of fetchState and cleans up event filters. Should be used whenever updating fetchState -to ensure processingFilters are always valid. -Returns Error if the node with given id cannot be found (unexpected) -*/ let handleQueryResult = ( chainFetcher: t, ~query: FetchState.query, ~newItems, - ~dynamicContracts, + ~newItemsWithDcs, ~latestFetchedBlock, ) => { - let fs = switch dynamicContracts { + let fs = switch newItemsWithDcs { | [] => chainFetcher.fetchState - | _ => chainFetcher.fetchState->FetchState.registerDynamicContracts(dynamicContracts) + | _ => chainFetcher.fetchState->FetchState.registerDynamicContracts(newItemsWithDcs) } fs ->FetchState.handleQueryResult(~query, ~latestFetchedBlock, ~newItems) - ->Result.map(fetchState => { - { - ...chainFetcher, - fetchState, - processingFilters: switch chainFetcher.processingFilters { - | Some(processingFilters) => processingFilters->cleanUpProcessingFilters(~fetchState) - | None => None - }, - } + ->Result.map(fs => { + ...chainFetcher, + fetchState: fs, }) } diff --git a/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainManager.res b/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainManager.res index 51ca18910..fcc5342eb 100644 --- a/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainManager.res +++ b/codegenerator/cli/templates/static/codegen/src/eventFetching/ChainManager.res @@ -58,7 +58,7 @@ let makeFromDbState = async (~initialState: Persistence.initialState, ~config: C let chainFetchersArr = await initialState.chains - ->Array.map(async (resumedChainState: InternalTable.Chains.t) => { + ->Array.map(async (resumedChainState: Persistence.initialChainState) => { let chain = Config.getChain(config, ~chainId=resumedChainState.id) let chainConfig = config.chainMap->ChainMap.get(chain) @@ -133,17 +133,31 @@ let isActivelyIndexing = chainManager => ->ChainMap.values ->Js.Array2.every(ChainFetcher.isActivelyIndexing) -let getSafeReorgBlocks = (chainManager: t): EntityHistory.safeReorgBlocks => { - let chainIds = [] - let blockNumbers = [] - chainManager.chainFetchers - ->ChainMap.values - ->Array.forEach(cf => { - chainIds->Js.Array2.push(cf.chainConfig.id)->ignore - blockNumbers->Js.Array2.push(cf->ChainFetcher.getHighestBlockBelowThreshold)->ignore - }) - { - chainIds, - blockNumbers, +let getSafeCheckpointId = (chainManager: t) => { + let chainFetchers = chainManager.chainFetchers->ChainMap.values + + let infinity = (%raw(`Infinity`): int) + let result = ref(infinity) + + for idx in 0 to chainFetchers->Array.length - 1 { + let chainFetcher = chainFetchers->Array.getUnsafe(idx) + switch chainFetcher.safeCheckpointTracking { + | None => () // Skip chains with maxReorgDepth = 0 + | Some(safeCheckpointTracking) => { + let safeCheckpointId = + safeCheckpointTracking->SafeCheckpointTracking.getSafeCheckpointId( + ~sourceBlockNumber=chainFetcher.currentBlockHeight, + ) + if safeCheckpointId < result.contents { + result := safeCheckpointId + } + } + } + } + + if result.contents === infinity || result.contents === 0 { + None // No safe checkpoint found + } else { + Some(result.contents) } } diff --git a/codegenerator/cli/templates/static/codegen/src/globalState/GlobalState.res b/codegenerator/cli/templates/static/codegen/src/globalState/GlobalState.res index 289e3d8d3..b33489622 100644 --- a/codegenerator/cli/templates/static/codegen/src/globalState/GlobalState.res +++ b/codegenerator/cli/templates/static/codegen/src/globalState/GlobalState.res @@ -110,7 +110,7 @@ type action = // So after it's finished we dispatch the submit action to get the latest fetch state. | SubmitPartitionQueryResponse({ newItems: array, - dynamicContracts: array, + newItemsWithDcs: array, currentBlockHeight: int, latestFetchedBlock: FetchState.blockNumberAndTimestamp, query: FetchState.query, @@ -134,7 +134,7 @@ type action = }) | SuccessExit | ErrorExit(ErrorHandling.t) - | SetRollbackState(InMemoryStore.t, ChainManager.t) + | SetRollbackState({diffInMemoryStore: InMemoryStore.t, rollbackedChainManager: ChainManager.t}) type queryChain = CheckAllChains | Chain(chain) type task = @@ -230,6 +230,19 @@ let updateProgressedChains = (chainManager: ChainManager.t, ~batch: Batch.t) => committedProgressBlockNumber: chainAfterBatch.progressBlockNumber, numEventsProcessed: chainAfterBatch.totalEventsProcessed, isProgressAtHead: cf.isProgressAtHead || chainAfterBatch.isProgressAtHeadWhenBatchCreated, + safeCheckpointTracking: switch cf.safeCheckpointTracking { + | Some(safeCheckpointTracking) => + Some( + safeCheckpointTracking->SafeCheckpointTracking.updateOnNewBatch( + ~sourceBlockNumber=cf.currentBlockHeight, + ~chainId=chain->ChainMap.Chain.toChainId, + ~batchCheckpointIds=batch.checkpointIds, + ~batchCheckpointBlockNumbers=batch.checkpointBlockNumbers, + ~batchCheckpointChainIds=batch.checkpointChainIds, + ), + ) + | None => None + }, } } | None => cf @@ -407,7 +420,7 @@ let validatePartitionQueryResponse = ( let submitPartitionQueryResponse = ( state, ~newItems, - ~dynamicContracts, + ~newItemsWithDcs, ~currentBlockHeight, ~latestFetchedBlock, ~query, @@ -417,7 +430,7 @@ let submitPartitionQueryResponse = ( let updatedChainFetcher = chainFetcher - ->ChainFetcher.handleQueryResult(~query, ~latestFetchedBlock, ~newItems, ~dynamicContracts) + ->ChainFetcher.handleQueryResult(~query, ~latestFetchedBlock, ~newItems, ~newItemsWithDcs) ->Utils.unwrapResultExn ->updateChainFetcherCurrentBlockHeight(~currentBlockHeight) @@ -449,7 +462,6 @@ let processPartitionQueryResponse = async ( {chain, response, query}: partitionQueryResponse, ~dispatchAction, ) => { - let chainFetcher = state.chainManager.chainFetchers->ChainMap.get(chain) let { parsedQueueItems, latestFetchedBlockNumber, @@ -463,26 +475,17 @@ let processPartitionQueryResponse = async ( for idx in 0 to parsedQueueItems->Array.length - 1 { let item = parsedQueueItems->Array.getUnsafe(idx) let eventItem = item->Internal.castUnsafeEventItem - if ( - switch chainFetcher.processingFilters { - | None => true - | Some(processingFilters) => ChainFetcher.applyProcessingFilters(~item, ~processingFilters) - } - ) { - if eventItem.eventConfig.contractRegister !== None { - itemsWithContractRegister->Array.push(item) - } - - // TODO: Don't really need to keep it in the queue - // when there's no handler (besides raw_events, processed counter, and dcsToStore consuming) - newItems->Array.push(item) + if eventItem.eventConfig.contractRegister !== None { + itemsWithContractRegister->Array.push(item) } + + // TODO: Don't really need to keep it in the queue + // when there's no handler (besides raw_events, processed counter, and dcsToStore consuming) + newItems->Array.push(item) } - let dynamicContracts = switch itemsWithContractRegister { - | [] as empty => - // A small optimisation to not recreate an empty array - empty->(Utils.magic: array => array) + let newItemsWithDcs = switch itemsWithContractRegister { + | [] as empty => empty | _ => await ChainFetcher.runContractRegistersOrThrow( ~itemsWithContractRegister, @@ -494,7 +497,7 @@ let processPartitionQueryResponse = async ( dispatchAction( SubmitPartitionQueryResponse({ newItems, - dynamicContracts, + newItemsWithDcs, currentBlockHeight, latestFetchedBlock: { blockNumber: latestFetchedBlockNumber, @@ -562,7 +565,7 @@ let actionReducer = (state: t, action: action) => { state->validatePartitionQueryResponse(partitionQueryResponse) | SubmitPartitionQueryResponse({ newItems, - dynamicContracts, + newItemsWithDcs, currentBlockHeight, latestFetchedBlock, query, @@ -570,7 +573,7 @@ let actionReducer = (state: t, action: action) => { }) => state->submitPartitionQueryResponse( ~newItems, - ~dynamicContracts, + ~newItemsWithDcs, ~currentBlockHeight, ~latestFetchedBlock, ~query, @@ -683,8 +686,14 @@ let actionReducer = (state: t, action: action) => { }, [NextQuery(CheckAllChains)], ) - | SetRollbackState(inMemoryStore, chainManager) => ( - {...state, rollbackState: RollbackReady({diffInMemoryStore: inMemoryStore}), chainManager}, + | SetRollbackState({diffInMemoryStore, rollbackedChainManager}) => ( + { + ...state, + rollbackState: RollbackReady({ + diffInMemoryStore: diffInMemoryStore, + }), + chainManager: rollbackedChainManager, + }, [NextQuery(CheckAllChains), ProcessEventBatch], ) | SuccessExit => { @@ -701,11 +710,12 @@ let actionReducer = (state: t, action: action) => { let invalidatedActionReducer = (state: t, action: action) => switch action { - | EventBatchProcessed(_) if state->isPreparingRollback => + | EventBatchProcessed({batch}) if state->isPreparingRollback => Logging.info("Finished processing batch before rollback, actioning rollback") ( { ...state, + chainManager: state.chainManager->updateProgressedChains(~batch), currentlyProcessingBatch: false, processedBatches: state.processedBatches + 1, }, @@ -768,11 +778,12 @@ let injectedTaskReducer = ( state->processPartitionQueryResponse(partitionQueryResponse, ~dispatchAction)->Promise.done | PruneStaleEntityHistory => let runPrune = async () => { - let safeReorgBlocks = state.chainManager->ChainManager.getSafeReorgBlocks - - if safeReorgBlocks.chainIds->Utils.Array.notEmpty { - await Db.sql->InternalTable.Checkpoints.deprecated_pruneStaleCheckpoints( + switch state.chainManager->ChainManager.getSafeCheckpointId { + | None => () + | Some(safeCheckpointId) => + await Db.sql->InternalTable.Checkpoints.pruneStaleCheckpoints( ~pgSchema=Env.Db.publicSchema, + ~safeCheckpointId, ) for idx in 0 to Entities.allEntities->Array.length - 1 { @@ -788,7 +799,7 @@ let injectedTaskReducer = ( await Db.sql->EntityHistory.pruneStaleEntityHistory( ~entityName=entityConfig.name, ~pgSchema=Env.Db.publicSchema, - ~safeReorgBlocks, + ~safeCheckpointId, ) } catch { | exn => @@ -797,12 +808,7 @@ let injectedTaskReducer = ( ~logger=Logging.createChild( ~params={ "entityName": entityConfig.name, - "safeBlockNumbers": safeReorgBlocks.chainIds - ->Js.Array2.mapi((chainId, idx) => ( - chainId->Belt.Int.toString, - safeReorgBlocks.blockNumbers->Js.Array2.unsafe_get(idx), - )) - ->Js.Dict.fromArray, + "safeCheckpointId": safeCheckpointId, }, ), ) @@ -919,22 +925,14 @@ let injectedTaskReducer = ( let inMemoryStore = rollbackInMemStore->Option.getWithDefault(InMemoryStore.make()) + inMemoryStore->InMemoryStore.setBatchDcs(~batch, ~shouldSaveHistory) + state.chainManager.chainFetchers ->ChainMap.keys ->Array.forEach(chain => { let chainId = chain->ChainMap.Chain.toChainId 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=chainAfterBatch.progressBlockNumber, @@ -1018,106 +1016,132 @@ let injectedTaskReducer = ( let reorgChainId = reorgChain->ChainMap.Chain.toChainId - //Get the first change event that occurred on each chain after the last known valid block - //Uses a different method depending on if the reorg chain is ordered or unordered - let firstChangeEventIdentifierPerChain = - await Db.sql->DbFunctions.EntityHistory.getFirstChangeEventPerChain( - switch state.config.multichain { - | Unordered => - UnorderedMultichain({ - reorgChainId, - safeBlockNumber: lastKnownValidBlockNumber, - }) - | Ordered => - OrderedMultichain({ - safeBlockTimestamp: lastKnownValidBlockTimestamp, - reorgChainId, - safeBlockNumber: lastKnownValidBlockNumber, - }) - }, - ) + let rollbackTargetCheckpointId = { + switch await Db.sql->InternalTable.Checkpoints.getRollbackTargetCheckpoint( + ~pgSchema=Env.Db.publicSchema, + ~reorgChainId, + ~lastKnownValidBlockNumber, + ) { + | [checkpoint] => checkpoint["id"] + | _ => 0 + } + } - firstChangeEventIdentifierPerChain->DbFunctions.EntityHistory.FirstChangeEventPerChain.setIfEarlier( - ~chainId=reorgChainId, - ~event={ - blockNumber: lastKnownValidBlockNumber + 1, - logIndex: 0, - }, - ) + let eventsProcessedDiffPerChain = Js.Dict.empty() + let newProgressBlockNumberPerChain = Js.Dict.empty() + let rollbackedProcessedEvents = ref(0) + + { + let rollbackProgressDiff = + await Db.sql->InternalTable.Checkpoints.getRollbackProgressDiff( + ~pgSchema=Env.Db.publicSchema, + ~rollbackTargetCheckpointId, + ) + for idx in 0 to rollbackProgressDiff->Js.Array2.length - 1 { + let diff = rollbackProgressDiff->Js.Array2.unsafe_get(idx) + eventsProcessedDiffPerChain->Utils.Dict.setByInt( + diff["chain_id"], + switch diff["events_processed_diff"]->Int.fromString { + | Some(eventsProcessedDiff) => { + rollbackedProcessedEvents := + rollbackedProcessedEvents.contents + eventsProcessedDiff + eventsProcessedDiff + } + | None => + Js.Exn.raiseError( + `Unexpedted case: Invalid events processed diff ${diff["events_processed_diff"]}`, + ) + }, + ) + newProgressBlockNumberPerChain->Utils.Dict.setByInt( + diff["chain_id"], + diff["new_progress_block_number"], + ) + } + } let chainFetchers = state.chainManager.chainFetchers->ChainMap.mapWithKey((chain, cf) => { - switch firstChangeEventIdentifierPerChain->DbFunctions.EntityHistory.FirstChangeEventPerChain.get( - ~chainId=chain->ChainMap.Chain.toChainId, + switch newProgressBlockNumberPerChain->Utils.Dict.dangerouslyGetByIntNonOption( + chain->ChainMap.Chain.toChainId, ) { - | Some(firstChangeEvent) => - let fetchState = cf.fetchState->FetchState.rollback(~firstChangeEvent) + | Some(newProgressBlockNumber) => + let fetchState = + cf.fetchState->FetchState.rollback(~targetBlockNumber=newProgressBlockNumber) + let newTotalEventsProcessed = + cf.numEventsProcessed - + eventsProcessedDiffPerChain + ->Utils.Dict.dangerouslyGetByIntNonOption(chain->ChainMap.Chain.toChainId) + ->Option.getUnsafe + + if cf.committedProgressBlockNumber !== newProgressBlockNumber { + Prometheus.ProgressBlockNumber.set( + ~blockNumber=newProgressBlockNumber, + ~chainId=chain->ChainMap.Chain.toChainId, + ) + } + if cf.numEventsProcessed !== newTotalEventsProcessed { + Prometheus.ProgressEventsCount.set( + ~processedCount=newTotalEventsProcessed, + ~chainId=chain->ChainMap.Chain.toChainId, + ) + } - let rolledBackCf = { + { ...cf, reorgDetection: chain == reorgChain ? cf.reorgDetection->ReorgDetection.rollbackToValidBlockNumber( ~blockNumber=lastKnownValidBlockNumber, ) : cf.reorgDetection, + safeCheckpointTracking: switch cf.safeCheckpointTracking { + | Some(safeCheckpointTracking) => + Some( + safeCheckpointTracking->SafeCheckpointTracking.rollback( + ~targetBlockNumber=newProgressBlockNumber, + ), + ) + | None => None + }, fetchState, - committedProgressBlockNumber: PervasivesU.min( - cf.committedProgressBlockNumber, - firstChangeEvent.blockNumber - 1, - ), + committedProgressBlockNumber: newProgressBlockNumber, + numEventsProcessed: newTotalEventsProcessed, } - //On other chains, filter out evennts based on the first change present on the chain after the reorg - rolledBackCf->ChainFetcher.addProcessingFilter( - ~filter=item => { - switch item { - | Internal.Event({blockNumber, logIndex}) - | Internal.Block({blockNumber, logIndex}) => - //Filter out events that occur passed the block where the query starts but - //are lower than the timestamp where we rolled back to - (blockNumber, logIndex) >= (firstChangeEvent.blockNumber, firstChangeEvent.logIndex) - } - }, - ~isValid=(~fetchState) => { - //Remove the event filter once the fetchState has fetched passed the - //blockNumber of the valid first change event - fetchState->FetchState.bufferBlockNumber <= firstChangeEvent.blockNumber - }, - ) + | None => //If no change was produced on the given chain after the reorged chain, no need to rollback anything cf } }) - //Construct a rolledback in Memory store - let rollbackResult = await IO.RollBack.rollBack( - ~chainId=reorgChain->ChainMap.Chain.toChainId, - ~blockTimestamp=lastKnownValidBlockTimestamp, - ~blockNumber=lastKnownValidBlockNumber, - ~logIndex=0, - ~isUnorderedMultichainMode=switch state.config.multichain { - | Unordered => true - | Ordered => false - }, - ) + // Construct in Memory store with rollback diff + let diff = await IO.prepareRollbackDiff(~rollbackTargetCheckpointId) let chainManager = { ...state.chainManager, + commitedCheckpointId: rollbackTargetCheckpointId, chainFetchers, } logger->Logging.childTrace({ "msg": "Finished rollback on reorg", "entityChanges": { - "deleted": rollbackResult["deletedEntities"], - "upserted": rollbackResult["setEntities"], + "deleted": diff["deletedEntities"], + "upserted": diff["setEntities"], }, + "rollbackedEvents": rollbackedProcessedEvents.contents, + "beforeCheckpointId": state.chainManager.commitedCheckpointId, + "targetCheckpointId": rollbackTargetCheckpointId, }) - logger->Logging.childTrace({ - "msg": "Initial diff of rollback entity history", - "diff": rollbackResult["fullDiff"], - }) - Prometheus.RollbackSuccess.increment(~timeMillis=Hrtime.timeSince(startTime)->Hrtime.toMillis) + Prometheus.RollbackSuccess.increment( + ~timeMillis=Hrtime.timeSince(startTime)->Hrtime.toMillis, + ~rollbackedProcessedEvents=rollbackedProcessedEvents.contents, + ) - dispatchAction(SetRollbackState(rollbackResult["inMemStore"], chainManager)) + dispatchAction( + SetRollbackState({ + diffInMemoryStore: diff["inMemStore"], + rollbackedChainManager: chainManager, + }), + ) } } } diff --git a/codegenerator/cli/templates/static/codegen/src/globalState/GlobalStateManager.res b/codegenerator/cli/templates/static/codegen/src/globalState/GlobalStateManager.res index d50879ad9..33d585a96 100644 --- a/codegenerator/cli/templates/static/codegen/src/globalState/GlobalStateManager.res +++ b/codegenerator/cli/templates/static/codegen/src/globalState/GlobalStateManager.res @@ -10,6 +10,11 @@ module type State = { let getId: t => int } +let handleFatalError = e => { + e->ErrorHandling.make(~msg="Indexer has failed with an unxpected error")->ErrorHandling.log + NodeJs.process->NodeJs.exitWithCode(Failure) +} + module MakeManager = (S: State) => { type t = {mutable state: S.t, stateUpdatedHook: option unit>} @@ -32,9 +37,7 @@ module MakeManager = (S: State) => { self.state = nextState nextTasks->Array.forEach(task => dispatchTask(self, task)) } catch { - | e => - e->ErrorHandling.make(~msg="Indexer has failed with an unxpected error")->ErrorHandling.log - NodeJs.process->NodeJs.exitWithCode(Failure) + | e => e->handleFatalError } } and dispatchTask = (self, task: S.task) => { @@ -43,9 +46,18 @@ module MakeManager = (S: State) => { if stateId !== self.state->S.getId { Logging.info("Invalidated task discarded") } else { - S.taskReducer(self.state, task, ~dispatchAction=action => - dispatchAction(~stateId, self, action) - )->ignore + try { + S.taskReducer(self.state, task, ~dispatchAction=action => + dispatchAction(~stateId, self, action) + ) + ->Promise.catch(e => { + e->handleFatalError + Promise.resolve() + }) + ->ignore + } catch { + | e => e->handleFatalError + } } }, 0)->ignore } diff --git a/scenarios/erc20_multichain_factory/test/MockChainData.res b/scenarios/erc20_multichain_factory/test/MockChainData.res index 0c7fddf83..8c7e16288 100644 --- a/scenarios/erc20_multichain_factory/test/MockChainData.res +++ b/scenarios/erc20_multichain_factory/test/MockChainData.res @@ -1,9 +1 @@ -module Indexer = { - module ErrorHandling = ErrorHandling - module Types = Types - module Config = Config - module Source = Source - module FetchState = FetchState -} - -include Helpers.ChainMocking.Make(Indexer) +include Helpers.ChainMocking.Make() diff --git a/scenarios/helpers/src/ChainMocking.res b/scenarios/helpers/src/ChainMocking.res index 63c7e7a71..699334fe0 100644 --- a/scenarios/helpers/src/ChainMocking.res +++ b/scenarios/helpers/src/ChainMocking.res @@ -34,8 +34,7 @@ module Crypto = { input->hashKeccak256(~toString=v => anyToString(v) ++ previousHash) } -module Make = (Indexer: Indexer.S) => { - open Indexer +module Make = () => { type log = { item: Internal.item, srcAddress: Address.t, diff --git a/scenarios/helpers/src/Indexer.res b/scenarios/helpers/src/Indexer.res deleted file mode 100644 index f59269862..000000000 --- a/scenarios/helpers/src/Indexer.res +++ /dev/null @@ -1,75 +0,0 @@ -module type S = { - module ErrorHandling: { - type t - } - - module FetchState: { - type indexingContract - - type selection = { - eventConfigs: array, - dependsOnAddresses: bool, - } - - type queryTarget = - | Head - | EndBlock({toBlock: int}) - | Merge({ - // The partition we are going to merge into - // It shouldn't be fetching during the query - intoPartitionId: string, - toBlock: int, - }) - - type query = { - partitionId: string, - fromBlock: int, - selection: selection, - addressesByContractName: dict>, - target: queryTarget, - indexingContracts: dict, - } - } - - module Source: { - type blockRangeFetchStats = { - @as("total time elapsed (ms)") totalTimeElapsed: int, - @as("parsing time (ms)") parsingTimeElapsed?: int, - @as("page fetch time (ms)") pageFetchTime?: int, - } - type blockRangeFetchResponse = { - currentBlockHeight: int, - reorgGuard: ReorgDetection.reorgGuard, - parsedQueueItems: array, - fromBlockQueried: int, - latestFetchedBlockNumber: int, - latestFetchedBlockTimestamp: int, - stats: blockRangeFetchStats, - } - type sourceFor = Sync | Fallback - type t = { - name: string, - sourceFor: sourceFor, - chain: ChainMap.Chain.t, - poweredByHyperSync: bool, - /* Frequency (in ms) used when polling for new events on this network. */ - pollingInterval: int, - getBlockHashes: ( - ~blockNumbers: array, - ~logger: Pino.t, - ) => promise, exn>>, - getHeightOrThrow: unit => promise, - getItemsOrThrow: ( - ~fromBlock: int, - ~toBlock: option, - ~addressesByContractName: dict>, - ~indexingContracts: dict, - ~currentBlockHeight: int, - ~partitionId: string, - ~selection: FetchState.selection, - ~retry: int, - ~logger: Pino.t, - ) => promise, - } - } -} diff --git a/scenarios/test_codegen/test/ChainFetcher_test.res b/scenarios/test_codegen/test/ChainFetcher_test.res deleted file mode 100644 index 77f743af8..000000000 --- a/scenarios/test_codegen/test/ChainFetcher_test.res +++ /dev/null @@ -1,75 +0,0 @@ -open RescriptMocha -open Belt - -describe("Test Processing Filters", () => { - // Assert.deepEqual doesn't work, because of deeply nested rescript-schema objects - // Assert.equal doesn't work because the array is always recreated on filter - // So I added the helper - let assertEqualItems = (items1, items2) => { - Assert.equal( - items1->Array.length, - items2->Array.length, - ~message="Length of the items doesn't match", - ) - items1->Array.forEachWithIndex((i, item1) => { - let item2 = items2->Js.Array2.unsafe_get(i) - Assert.equal(item1, item2) - }) - } - - it("Keeps items when there are not filters", () => { - let items = MockEvents.eventBatchItems - assertEqualItems( - items, - items->Js.Array2.filter( - item => ChainFetcher.applyProcessingFilters(~item, ~processingFilters=[]), - ), - ) - }) - - it("Keeps items when all filters return true", () => { - let items = MockEvents.eventBatchItems - assertEqualItems( - items, - items->Js.Array2.filter( - item => - ChainFetcher.applyProcessingFilters( - ~item, - ~processingFilters=[ - { - filter: _ => true, - isValid: (~fetchState as _) => true, - }, - { - filter: _ => true, - isValid: (~fetchState as _) => true, - }, - ], - ), - ), - ) - }) - - it("Removes all items when there is one filter returning false", () => { - let items = MockEvents.eventBatchItems - assertEqualItems( - [], - items->Js.Array2.filter( - item => - ChainFetcher.applyProcessingFilters( - ~item, - ~processingFilters=[ - { - filter: _ => false, - isValid: (~fetchState as _) => true, - }, - { - filter: _ => true, - isValid: (~fetchState as _) => true, - }, - ], - ), - ), - ) - }) -}) diff --git a/scenarios/test_codegen/test/ChainManager_test.res b/scenarios/test_codegen/test/ChainManager_test.res index f67b66b52..7a420dec7 100644 --- a/scenarios/test_codegen/test/ChainManager_test.res +++ b/scenarios/test_codegen/test/ChainManager_test.res @@ -109,13 +109,13 @@ let populateChainQueuesWithRandomEvents = (~runTime=1000, ~maxBlockTime=15, ()) chainConfig, // This is quite a hack - but it works! reorgDetection: ReorgDetection.make( - ~blocks=[], + ~chainReorgCheckpoints=[], ~maxReorgDepth=200, ~shouldRollbackOnReorg=false, ), + safeCheckpointTracking: None, isProgressAtHead: false, currentBlockHeight: 0, - processingFilters: None, } mockChainFetcher diff --git a/scenarios/test_codegen/test/E2E_test.res b/scenarios/test_codegen/test/E2E_test.res index f58366e5c..80efe9db6 100644 --- a/scenarios/test_codegen/test/E2E_test.res +++ b/scenarios/test_codegen/test/E2E_test.res @@ -83,7 +83,7 @@ describe("E2E tests", () => { ) }) - // A regression test for bug introduced in 2.30.0 + // A regression test for a bug introduced in 2.30.0 Async.it("Correct event ordering for ordered multichain indexer", async () => { let sourceMock1337 = Mock.Source.make( [#getHeightOrThrow, #getItemsOrThrow, #getBlockHashes], @@ -158,58 +158,75 @@ describe("E2E tests", () => { await indexerMock.getBatchWritePromise() Assert.deepEqual( - await indexerMock.queryHistory(module(Entities.SimpleEntity)), - [ - { - current: { - chain_id: 100, - block_timestamp: 150, - block_number: 150, - log_index: 0, + await Promise.all2(( + indexerMock.queryCheckpoints(), + indexerMock.queryHistory(module(Entities.SimpleEntity)), + )), + ( + [ + { + id: 2, + chainId: 100, + blockNumber: 150, + blockHash: Js.Null.Null, + eventsProcessed: 1, }, - previous: undefined, - entityData: Set({ - Entities.SimpleEntity.id: "1", - value: "call-0", - }), - }, - { - current: { - chain_id: 1337, - block_timestamp: 150, - block_number: 150, - log_index: 2, + { + id: 3, + chainId: 1337, + blockNumber: 100, + blockHash: Js.Null.Value("0x100"), + eventsProcessed: 0, }, - previous: Some({ - chain_id: 100, - block_timestamp: 150, - block_number: 150, - log_index: 0, - }), - entityData: Set({ - Entities.SimpleEntity.id: "1", - value: "call-1", - }), - }, - { - current: { - chain_id: 100, - block_timestamp: 151, - block_number: 151, - log_index: 0, + { + id: 4, + chainId: 1337, + blockNumber: 150, + blockHash: Js.Null.Null, + eventsProcessed: 1, }, - previous: Some({ - chain_id: 1337, - block_timestamp: 150, - block_number: 150, - log_index: 2, - }), - entityData: Set({ - Entities.SimpleEntity.id: "1", - value: "call-2", - }), - }, - ], + { + id: 5, + chainId: 100, + blockNumber: 151, + blockHash: Js.Null.Null, + eventsProcessed: 1, + }, + { + id: 6, + chainId: 100, + blockNumber: 160, + blockHash: Js.Null.Value("0x160"), + eventsProcessed: 0, + }, + ], + [ + { + checkpointId: 2, + entityId: "1", + entityUpdateAction: Set({ + Entities.SimpleEntity.id: "1", + value: "call-0", + }), + }, + { + checkpointId: 4, + entityId: "1", + entityUpdateAction: Set({ + Entities.SimpleEntity.id: "1", + value: "call-1", + }), + }, + { + checkpointId: 5, + entityId: "1", + entityUpdateAction: Set({ + Entities.SimpleEntity.id: "1", + value: "call-2", + }), + }, + ], + ), ) }) }) diff --git a/scenarios/test_codegen/test/EventHandler_test.ts b/scenarios/test_codegen/test/EventHandler_test.ts index 0a27812a5..d535cb2aa 100644 --- a/scenarios/test_codegen/test/EventHandler_test.ts +++ b/scenarios/test_codegen/test/EventHandler_test.ts @@ -13,6 +13,12 @@ describe("Use Envio test framework to test event handlers", () => { const event = Gravatar.FactoryEvent.createMockEvent({ contract: dcAddress, testCase: "syncRegistration", + mockEventData: { + chainId: 1337, + block: { + number: 2, + }, + }, }); const updatedMockDb = await mockDbInitial.processEvents([event]); @@ -20,11 +26,11 @@ describe("Use Envio test framework to test event handlers", () => { const registeredDcs = updatedMockDb.dynamicContractRegistry.getAll(); assert.deepEqual(registeredDcs, [ { - id: `1-${dcAddress}`, + id: `1337-${dcAddress}`, contract_name: "SimpleNft", contract_address: dcAddress, - chain_id: 1, - registering_event_block_number: 0, + chain_id: 1337, + registering_event_block_number: 2, registering_event_log_index: 0, registering_event_name: "FactoryEvent", registering_event_src_address: `0xf39Fd6e51aad88F6F4ce6aB8827279cffFb92266`, @@ -43,6 +49,9 @@ describe("Use Envio test framework to test event handlers", () => { const event = Gravatar.FactoryEvent.createMockEvent({ contract: dcAddress, testCase: "asyncRegistration", + mockEventData: { + chainId: 1337, + }, }); const updatedMockDb = await mockDbInitial.processEvents([event]); @@ -50,10 +59,10 @@ describe("Use Envio test framework to test event handlers", () => { const registeredDcs = updatedMockDb.dynamicContractRegistry.getAll(); assert.deepEqual(registeredDcs, [ { - id: `1-${dcAddress}`, + id: `1337-${dcAddress}`, contract_name: "SimpleNft", contract_address: dcAddress, - chain_id: 1, + chain_id: 1337, registering_event_block_number: 0, registering_event_log_index: 0, registering_event_name: "FactoryEvent", @@ -304,6 +313,7 @@ describe("Use Envio test framework to test event handlers", () => { testCase: "checksumsAddress", mockEventData: { srcAddress: eventAddress, + chainId: 1337, }, }); @@ -312,10 +322,10 @@ describe("Use Envio test framework to test event handlers", () => { const registeredDcs = updatedMockDb.dynamicContractRegistry.getAll(); assert.deepEqual(registeredDcs, [ { - id: `1-${expectedChecksummedAddress}`, + id: `1337-${expectedChecksummedAddress}`, contract_name: "SimpleNft", contract_address: expectedChecksummedAddress, - chain_id: 1, + chain_id: 1337, registering_event_block_number: 0, registering_event_log_index: 0, registering_event_name: "FactoryEvent", diff --git a/scenarios/test_codegen/test/Integration_ts_helpers.res b/scenarios/test_codegen/test/Integration_ts_helpers.res index d30a51c5f..7263e6bf1 100644 --- a/scenarios/test_codegen/test/Integration_ts_helpers.res +++ b/scenarios/test_codegen/test/Integration_ts_helpers.res @@ -68,7 +68,7 @@ let makeChainManager = (cfg: chainConfig): chainManager => { ~config=Config.make( ~isUnorderedMultichainMode=true, ~chains=[cfg], - ~registrations={onBlockByChainId: Js.Dict.empty()}, + ~registrations={onBlockByChainId: Js.Dict.empty(), hasEvents: false}, ), ) } diff --git a/scenarios/test_codegen/test/ReorgDetection_test.res b/scenarios/test_codegen/test/ReorgDetection_test.res index 3c99502da..4d1f34ca5 100644 --- a/scenarios/test_codegen/test/ReorgDetection_test.res +++ b/scenarios/test_codegen/test/ReorgDetection_test.res @@ -14,8 +14,13 @@ describe("Validate reorg detection functions", () => { let mock = (arr, ~maxReorgDepth=200, ~shouldRollbackOnReorg=true, ~detectedReorgBlock=?) => { ReorgDetection.make( - ~blocks=arr->Array.map(((blockNumber, blockHash)) => { - ReorgDetection.blockNumber, + ~chainReorgCheckpoints=arr->Array.map((( + blockNumber, + blockHash, + )): Internal.reorgCheckpoint => { + chainId: 0, // It's not used + checkpointId: 0, // It's not used + blockNumber, blockHash, }), ~maxReorgDepth, diff --git a/scenarios/test_codegen/test/SerDe_Test.res b/scenarios/test_codegen/test/SerDe_Test.res index 003295154..5e40a95d4 100644 --- a/scenarios/test_codegen/test/SerDe_Test.res +++ b/scenarios/test_codegen/test/SerDe_Test.res @@ -8,14 +8,24 @@ let mockDate = (~year=2024, ~month=1, ~day=1) => { } describe("SerDe Test", () => { - Async.before(async () => { - await DbHelpers.runUpDownMigration() - }) - Async.it("All type entity", async () => { - let storage = Config.codegenPersistence->Persistence.getInitializedStorageOrThrow + let sourceMock = Mock.Source.make(~chain=#1337, [#getHeightOrThrow, #getItemsOrThrow]) + let indexerMock = await Mock.Indexer.make( + ~chains=[{chain: #1337, sources: [sourceMock.source]}], + ~saveFullHistory=true, + ) + await Utils.delay(0) - let entity: Entities.EntityWithAllTypes.t = { + Assert.deepEqual( + sourceMock.getHeightOrThrowCalls->Array.length, + 1, + ~message="should have called getHeightOrThrow to get initial height", + ) + sourceMock.resolveGetHeightOrThrow(300) + await Utils.delay(0) + await Utils.delay(0) + + let entityWithAllTypes: Entities.EntityWithAllTypes.t = { id: "1", string: "string", optString: Some("optString"), @@ -46,106 +56,7 @@ describe("SerDe Test", () => { enumField: ADMIN, optEnumField: Some(ADMIN), } - - let entityHistoryItem: EntityHistory.historyRow<_> = { - current: { - chain_id: 1, - block_timestamp: 1, - block_number: 1, - log_index: 1, - }, - previous: None, - entityData: Set(entity), - } - - //Fails if serialziation does not work - let set = (sql, items) => - sql->PgStorage.setOrThrow( - ~items, - ~table=Entities.EntityWithAllTypes.table, - ~itemSchema=Entities.EntityWithAllTypes.schema, - ~pgSchema=Config.storagePgSchema, - ) - - //Fails if parsing does not work - let read = ids => - storage.loadByIdsOrThrow( - ~ids, - ~table=Entities.EntityWithAllTypes.table, - ~rowsSchema=Entities.EntityWithAllTypes.rowsSchema, - ) - - let setHistory = (sql, row) => - Promise.all( - sql->PgStorage.setEntityHistoryOrThrow( - ~entityHistory=Entities.EntityWithAllTypes.entityHistory, - ~rows=[row], - ), - )->Promise.ignoreValue - - try await Db.sql->setHistory(entityHistoryItem) catch { - | exn => - Js.log2("setHistory exn", exn) - Assert.fail("Failed to set entity history in table") - } - - //set the entity - try await Db.sql->set([entity->Entities.EntityWithAllTypes.castToInternal]) catch { - | exn => - Js.log(exn) - Assert.fail("Failed to set entity in table") - } - - switch await read([entity.id]) { - | exception exn => - Js.log(exn) - Assert.fail("Failed to read entity from table") - | [_entity] => Assert.deepEqual(_entity, entity) - | _ => Assert.fail("Should have returned a row on batch read fn") - } - - //The copy function will do it's custom postgres serialization of the entity - // await Db.sql->DbFunctions.EntityHistory.copyAllEntitiesToEntityHistory - - let res = await Db.sql->Postgres.unsafe(`SELECT * FROM public."EntityWithAllTypes_history";`) - - switch res { - | [row] => - let parsed = row->S.parseJsonOrThrow(Entities.EntityWithAllTypes.entityHistory.schema) - Assert.deepEqual( - parsed.entityData, - Set(entity), - ~message="Postgres json serialization should be compatable with our schema", - ) - | _ => Assert.fail("Should have returned a row") - } - }) - - it("contains correct query for unnest entity", () => { - let createQuery = - Entities.EntityWithAllNonArrayTypes.table->PgStorage.makeCreateTableQuery(~pgSchema="public") - Assert.equal( - createQuery, - `CREATE TABLE IF NOT EXISTS "public"."EntityWithAllNonArrayTypes"("bigDecimal" NUMERIC NOT NULL, "bigDecimalWithConfig" NUMERIC(10, 8) NOT NULL, "bigInt" NUMERIC NOT NULL, "bool" BOOLEAN NOT NULL, "enumField" "public".AccountType NOT NULL, "float_" DOUBLE PRECISION NOT NULL, "id" TEXT NOT NULL, "int_" INTEGER NOT NULL, "optBigDecimal" NUMERIC, "optBigInt" NUMERIC, "optBool" BOOLEAN, "optEnumField" "public".AccountType, "optFloat" DOUBLE PRECISION, "optInt" INTEGER, "optString" TEXT, "string" TEXT NOT NULL, PRIMARY KEY("id"));`, - ) - let query = PgStorage.makeInsertUnnestSetQuery( - ~table=Entities.EntityWithAllNonArrayTypes.table, - ~itemSchema=Entities.EntityWithAllNonArrayTypes.schema, - ~isRawEvents=false, - ~pgSchema="public", - ) - - Assert.equal( - query, - `INSERT INTO "public"."EntityWithAllNonArrayTypes" ("bigDecimal", "bigDecimalWithConfig", "bigInt", "bool", "enumField", "float_", "id", "int_", "optBigDecimal", "optBigInt", "optBool", "optEnumField", "optFloat", "optInt", "optString", "string") -SELECT * FROM unnest($1::NUMERIC[],$2::NUMERIC(10, 8)[],$3::NUMERIC[],$4::INTEGER[]::BOOLEAN[],$5::TEXT[]::"public".AccountType[],$6::DOUBLE PRECISION[],$7::TEXT[],$8::INTEGER[],$9::NUMERIC[],$10::NUMERIC[],$11::INTEGER[]::BOOLEAN[],$12::TEXT[]::"public".AccountType[],$13::DOUBLE PRECISION[],$14::INTEGER[],$15::TEXT[],$16::TEXT[])ON CONFLICT("id") DO UPDATE SET "bigDecimal" = EXCLUDED."bigDecimal","bigDecimalWithConfig" = EXCLUDED."bigDecimalWithConfig","bigInt" = EXCLUDED."bigInt","bool" = EXCLUDED."bool","enumField" = EXCLUDED."enumField","float_" = EXCLUDED."float_","int_" = EXCLUDED."int_","optBigDecimal" = EXCLUDED."optBigDecimal","optBigInt" = EXCLUDED."optBigInt","optBool" = EXCLUDED."optBool","optEnumField" = EXCLUDED."optEnumField","optFloat" = EXCLUDED."optFloat","optInt" = EXCLUDED."optInt","optString" = EXCLUDED."optString","string" = EXCLUDED."string";`, - ) - }) - - Async.it("All type entity without array types for unnest case", async () => { - let storage = Config.codegenPersistence->Persistence.getInitializedStorageOrThrow - - let entity: Entities.EntityWithAllNonArrayTypes.t = { + let entityWithAllNonArrayTypes: Entities.EntityWithAllNonArrayTypes.t = { id: "1", string: "string", optString: Some("optString"), @@ -164,79 +75,45 @@ SELECT * FROM unnest($1::NUMERIC[],$2::NUMERIC(10, 8)[],$3::NUMERIC[],$4::INTEGE optEnumField: Some(ADMIN), } - let entityHistoryItem: EntityHistory.historyRow<_> = { - current: { - chain_id: 1, - block_timestamp: 1, - block_number: 1, - log_index: 1, + sourceMock.resolveGetItemsOrThrow([ + { + blockNumber: 50, + logIndex: 1, + handler: async ({context}) => { + context.entityWithAllTypes.set(entityWithAllTypes) + context.entityWithAllNonArrayTypes.set(entityWithAllNonArrayTypes) + }, }, - previous: None, - entityData: Set(entity), - } - - //Fails if serialziation does not work - let set = (sql, items) => { - sql->PgStorage.setOrThrow( - ~items, - ~table=Entities.EntityWithAllNonArrayTypes.table, - ~itemSchema=Entities.EntityWithAllNonArrayTypes.schema, - ~pgSchema="public", - ) - } - - //Fails if parsing does not work - let read = ids => - storage.loadByIdsOrThrow( - ~ids, - ~table=Entities.EntityWithAllNonArrayTypes.table, - ~rowsSchema=Entities.EntityWithAllNonArrayTypes.rowsSchema, - ) - - let setHistory = (sql, row) => - Promise.all( - sql->PgStorage.setEntityHistoryOrThrow( - ~entityHistory=Entities.EntityWithAllNonArrayTypes.entityHistory, - ~rows=[row], - ), - )->Promise.ignoreValue - - try await Db.sql->setHistory(entityHistoryItem) catch { - | exn => - Js.log2("setHistory exn", exn) - Assert.fail("Failed to set entity history in table") - } - - //set the entity - try await Db.sql->set([entity->Entities.EntityWithAllNonArrayTypes.castToInternal]) catch { - | exn => - Js.log(exn) - Assert.fail("Failed to set entity in table") - } - - switch await read([entity.id]) { - | exception exn => - Js.log(exn) - Assert.fail("Failed to read entity from table") - | [_entity] => Assert.deepEqual(_entity, entity) - | _ => Assert.fail("Should have returned a row on batch read fn") - } - - //The copy function will do it's custom postgres serialization of the entity - // await Db.sql->DbFunctions.EntityHistory.copyAllEntitiesToEntityHistory + ]) + await indexerMock.getBatchWritePromise() - let res = - await Db.sql->Postgres.unsafe(`SELECT * FROM public."EntityWithAllNonArrayTypes_history";`) - - switch res { - | [row] => - let parsed = row->S.parseJsonOrThrow(Entities.EntityWithAllNonArrayTypes.entityHistory.schema) - Assert.deepEqual( - parsed.entityData, - Set(entity), - ~message="Postgres json serialization should be compatable with our schema", - ) - | _ => Assert.fail("Should have returned a row") - } + Assert.deepEqual( + await indexerMock.query(module(Entities.EntityWithAllTypes)), + [entityWithAllTypes], + ) + Assert.deepEqual( + await indexerMock.queryHistory(module(Entities.EntityWithAllTypes)), + [ + { + checkpointId: 1, + entityId: "1", + entityUpdateAction: Set(entityWithAllTypes), + }, + ], + ) + Assert.deepEqual( + await indexerMock.query(module(Entities.EntityWithAllNonArrayTypes)), + [entityWithAllNonArrayTypes], + ) + Assert.deepEqual( + await indexerMock.queryHistory(module(Entities.EntityWithAllNonArrayTypes)), + [ + { + checkpointId: 1, + entityId: "1", + entityUpdateAction: Set(entityWithAllNonArrayTypes), + }, + ], + ) }) }) diff --git a/scenarios/test_codegen/test/helpers/Mock.res b/scenarios/test_codegen/test/helpers/Mock.res index 7f3cda32f..2d5c1e234 100644 --- a/scenarios/test_codegen/test/helpers/Mock.res +++ b/scenarios/test_codegen/test/helpers/Mock.res @@ -8,15 +8,11 @@ module InMemoryStore = { ) let entity = entity->(Utils.magic: 'a => Entities.internalEntity) inMemTable->InMemoryTable.Entity.set( - Set(entity)->Types.mkEntityUpdate( - ~eventIdentifier={ - chainId: 0, - blockTimestamp: 0, - blockNumber: 0, - logIndex: 0, - }, - ~entityId=entity->Entities.getEntityId, - ), + { + entityId: entity->Entities.getEntityId, + checkpointId: 0, + entityUpdateAction: Set(entity), + }, ~shouldSaveHistory=RegisterHandlers.getConfig()->Config.shouldSaveHistory( ~isInReorgThreshold=false, ), @@ -217,8 +213,9 @@ module Indexer = { getRollbackReadyPromise: unit => promise, query: 'entity. module(Entities.Entity with type t = 'entity) => promise>, queryHistory: 'entity. module(Entities.Entity with type t = 'entity) => promise< - array>, + array>, >, + queryCheckpoints: unit => promise>, metric: string => promise>, } @@ -227,6 +224,7 @@ module Indexer = { let make = async ( ~chains: array, ~multichain=InternalConfig.Unordered, + ~saveFullHistory=false, ~reset=true, ) => { DbHelpers.resetPostgresClient() @@ -275,6 +273,10 @@ module Indexer = { } let config: Config.t = { ...config, + historyConfig: { + rollbackFlag: RollbackOnReorg, + historyFlag: saveFullHistory ? FullHistory : MinHistory, + }, persistence, enableRawEvents: false, chainMap, @@ -349,14 +351,38 @@ module Indexer = { ), ) ->Promise.thenResolve(items => { - items->S.parseOrThrow(S.array(entityConfig.entityHistory.schema)) + items->S.parseOrThrow( + S.array( + S.union([ + entityConfig.entityHistory.setUpdateSchema, + S.object((s): EntityHistory.entityUpdate<'entity> => { + s.tag(EntityHistory.changeFieldName, EntityHistory.RowAction.DELETE) + { + entityId: s.field("id", S.string), + checkpointId: s.field(EntityHistory.checkpointIdFieldName, S.int), + entityUpdateAction: Delete, + } + }), + ]), + ), + ) }) ->( - Utils.magic: promise>> => promise< - array>, + Utils.magic: promise>> => promise< + array>, > ) }, + queryCheckpoints: () => { + Db.sql + ->Postgres.unsafe( + PgStorage.makeLoadAllQuery( + ~pgSchema, + ~tableName=InternalTable.Checkpoints.table.tableName, + ), + ) + ->(Utils.magic: promise => promise>) + }, metric: async name => { switch PromClient.defaultRegister->PromClient.getSingleMetric(name) { | Some(m) => diff --git a/scenarios/test_codegen/test/lib_tests/EntityHistory_test.res b/scenarios/test_codegen/test/lib_tests/EntityHistory_test.res index 7384ac50b..4c7ebe68d 100644 --- a/scenarios/test_codegen/test/lib_tests/EntityHistory_test.res +++ b/scenarios/test_codegen/test/lib_tests/EntityHistory_test.res @@ -1,1213 +1,1053 @@ -open RescriptMocha - -//unsafe polymorphic toString binding for any type -@send external toStringUnsafe: 'a => string = "toString" - -let stripUndefinedFieldsInPlace = (val: 'a): 'a => { - let json = val->(Utils.magic: 'a => Js.Json.t) - //Hot fix for rescript equality check that removes optional fields - let rec strip = (json: Js.Json.t) => { - switch json { - | Object(obj) => - obj - ->Js.Dict.keys - ->Belt.Array.forEach(key => { - let value = obj->Utils.Dict.dangerouslyGetNonOption(key) - if value === %raw(`undefined`) { - obj->Utils.Dict.deleteInPlace(key) - } else { - strip(value->Belt.Option.getExn) - } - }) - | Array(arr) => arr->Belt.Array.forEach(value => strip(value)) - | _ => () - } - } - - json->strip - json->(Utils.magic: Js.Json.t => 'a) -} - -module TestEntity = { - type t = { - id: string, - fieldA: int, - fieldB: option, - } - - let name = "TestEntity" - let schema = S.schema(s => { - id: s.matches(S.string), - fieldA: s.matches(S.int), - fieldB: s.matches(S.null(S.string)), - }) - - let rowsSchema = S.array(schema) - let table = Table.mkTable( - "TestEntity", - ~fields=[ - Table.mkField("id", Text, ~fieldSchema=S.string, ~isPrimaryKey=true), - Table.mkField("fieldA", Integer, ~fieldSchema=S.int), - Table.mkField("fieldB", Text, ~fieldSchema=S.null(S.string), ~isNullable=true), - ], - ) - - let entityHistory = table->EntityHistory.fromTable(~schema) - - external castToInternal: t => Internal.entity = "%identity" -} - -type testEntityHistory = EntityHistory.historyRow -let testEntityHistorySchema = EntityHistory.makeHistoryRowSchema(TestEntity.schema) - -let batchSetMockEntity = (sql, items) => - PgStorage.setOrThrow( - sql, - ~items, - ~pgSchema="public", - ~table=TestEntity.table, - ~itemSchema=TestEntity.schema, - ) - -let getAllMockEntity = sql => - sql - ->Postgres.unsafe(`SELECT * FROM "public"."${TestEntity.table.tableName}"`) - ->Promise.thenResolve(json => json->S.parseJsonOrThrow(TestEntity.rowsSchema)) - -let getAllMockEntityHistory = sql => - sql->Postgres.unsafe(`SELECT * FROM "public"."${TestEntity.entityHistory.table.tableName}"`) - -describe("Entity history serde", () => { - it("serializes and deserializes correctly", () => { - let history: testEntityHistory = { - current: { - chain_id: 1, - block_number: 2, - block_timestamp: 3, - log_index: 4, - }, - previous: None, - entityData: Set({id: "1", fieldA: 1, fieldB: Some("test")}), - } - - let serializedHistory = history->S.reverseConvertToJsonOrThrow(testEntityHistorySchema) - let expected = %raw(`{ - "entity_history_block_timestamp": 3, - "entity_history_chain_id": 1, - "entity_history_block_number": 2, - "entity_history_log_index": 4, - "previous_entity_history_block_timestamp": null, - "previous_entity_history_chain_id": null, - "previous_entity_history_block_number": null, - "previous_entity_history_log_index": null, - "id": "1", - "fieldA": 1, - "fieldB": "test", - "action": "SET" - }`) - - Assert.deepEqual(serializedHistory, expected) - let deserializedHistory = serializedHistory->S.parseJsonOrThrow(testEntityHistorySchema) - Assert.deepEqual(deserializedHistory->stripUndefinedFieldsInPlace, history) - }) - - it("serializes and deserializes correctly with previous history", () => { - let history: testEntityHistory = { - current: { - chain_id: 1, - block_number: 2, - block_timestamp: 3, - log_index: 4, - }, - previous: Some({ - chain_id: 5, - block_number: 6, - block_timestamp: 7, - log_index: 8, - }), //previous - entityData: Set({id: "1", fieldA: 1, fieldB: Some("test")}), - } - let serializedHistory = history->S.reverseConvertToJsonOrThrow(testEntityHistorySchema) - let expected = %raw(`{ - "entity_history_block_timestamp": 3, - "entity_history_chain_id": 1, - "entity_history_block_number": 2, - "entity_history_log_index": 4, - "previous_entity_history_block_timestamp": 7, - "previous_entity_history_chain_id": 5, - "previous_entity_history_block_number": 6, - "previous_entity_history_log_index": 8, - "id": "1", - "fieldA": 1, - "fieldB": "test", - "action": "SET" - }`) - - Assert.deepEqual(serializedHistory, expected) - let deserializedHistory = serializedHistory->S.parseJsonOrThrow(testEntityHistorySchema) - Assert.deepEqual(deserializedHistory, history) - }) - - it("serializes and deserializes correctly with deleted entity", () => { - let history: testEntityHistory = { - current: { - chain_id: 1, - block_number: 2, - block_timestamp: 3, - log_index: 4, - }, - previous: None, - entityData: Delete({id: "1"}), - } - let serializedHistory = history->S.reverseConvertToJsonOrThrow(testEntityHistorySchema) - let expected = %raw(`{ - "entity_history_block_timestamp": 3, - "entity_history_chain_id": 1, - "entity_history_block_number": 2, - "entity_history_log_index": 4, - "previous_entity_history_block_timestamp": null, - "previous_entity_history_chain_id": null, - "previous_entity_history_block_number": null, - "previous_entity_history_log_index": null, - "id": "1", - "fieldA": null, - "fieldB":null, - "action": "DELETE" - }`) - - Assert.deepEqual(serializedHistory, expected) - }) -}) - -describe("Entity History Codegen", () => { - it("Creates a postgres insert function", () => { - let expected = `CREATE OR REPLACE FUNCTION "insert_TestEntity_history"(history_row "public"."TestEntity_history", should_copy_current_entity BOOLEAN) -RETURNS void AS $$ -DECLARE - v_previous_record RECORD; - v_origin_record RECORD; -BEGIN - -- Check if previous values are not provided - IF history_row.previous_entity_history_block_timestamp IS NULL OR history_row.previous_entity_history_chain_id IS NULL OR history_row.previous_entity_history_block_number IS NULL OR history_row.previous_entity_history_log_index IS NULL THEN - -- Find the most recent record for the same id - SELECT entity_history_block_timestamp, entity_history_chain_id, entity_history_block_number, entity_history_log_index INTO v_previous_record - FROM "public"."TestEntity_history" - WHERE id = history_row.id - ORDER BY entity_history_block_timestamp DESC, entity_history_chain_id DESC, entity_history_block_number DESC, entity_history_log_index DESC - LIMIT 1; - - -- If a previous record exists, use its values - IF FOUND THEN - history_row.previous_entity_history_block_timestamp := v_previous_record.entity_history_block_timestamp; history_row.previous_entity_history_chain_id := v_previous_record.entity_history_chain_id; history_row.previous_entity_history_block_number := v_previous_record.entity_history_block_number; history_row.previous_entity_history_log_index := v_previous_record.entity_history_log_index; - ElSIF should_copy_current_entity THEN - -- Check if a value for the id exists in the origin table and if so, insert a history row for it. - SELECT "id", "fieldA", "fieldB" FROM "public"."TestEntity" WHERE id = history_row.id INTO v_origin_record; - IF FOUND THEN - INSERT INTO "public"."TestEntity_history" (entity_history_block_timestamp, entity_history_chain_id, entity_history_block_number, entity_history_log_index, "id", "fieldA", "fieldB", "action") - -- SET the current change data fields to 0 since we don't know what they were - -- and it doesn't matter provided they are less than any new values - VALUES (0, 0, 0, 0, v_origin_record."id", v_origin_record."fieldA", v_origin_record."fieldB", 'SET'); - - history_row.previous_entity_history_block_timestamp := 0; history_row.previous_entity_history_chain_id := 0; history_row.previous_entity_history_block_number := 0; history_row.previous_entity_history_log_index := 0; - END IF; - END IF; - END IF; - - INSERT INTO "public"."TestEntity_history" ("entity_history_block_timestamp", "entity_history_chain_id", "entity_history_block_number", "entity_history_log_index", "previous_entity_history_block_timestamp", "previous_entity_history_chain_id", "previous_entity_history_block_number", "previous_entity_history_log_index", "id", "fieldA", "fieldB", "action") - VALUES (history_row."entity_history_block_timestamp", history_row."entity_history_chain_id", history_row."entity_history_block_number", history_row."entity_history_log_index", history_row."previous_entity_history_block_timestamp", history_row."previous_entity_history_chain_id", history_row."previous_entity_history_block_number", history_row."previous_entity_history_log_index", history_row."id", history_row."fieldA", history_row."fieldB", history_row."action"); -END; -$$ LANGUAGE plpgsql;` - - Assert.equal(expected, TestEntity.entityHistory.makeInsertFnQuery(~pgSchema="public")) - }) - it("Creates an entity history table", () => { - let createQuery = - TestEntity.entityHistory.table->PgStorage.makeCreateTableQuery(~pgSchema="public") - Assert.equal( - `CREATE TABLE IF NOT EXISTS "public"."TestEntity_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, "id" TEXT NOT NULL, "fieldA" INTEGER, "fieldB" 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"));`, - createQuery, - ) - }) - - it("Creates a js insert function", () => { - let insertFnString = TestEntity.entityHistory.insertFn->toStringUnsafe - - let expected = `(sql, rowArgs, shouldCopyCurrentEntity) => - sql\`select "insert_TestEntity_history"(ROW(\${rowArgs["entity_history_block_timestamp"]}, \${rowArgs["entity_history_chain_id"]}, \${rowArgs["entity_history_block_number"]}, \${rowArgs["entity_history_log_index"]}, \${rowArgs["previous_entity_history_block_timestamp"]}, \${rowArgs["previous_entity_history_chain_id"]}, \${rowArgs["previous_entity_history_block_number"]}, \${rowArgs["previous_entity_history_log_index"]}, \${rowArgs["id"]}, \${rowArgs["fieldA"]}, \${rowArgs["fieldB"]}, \${rowArgs["action"]}, NULL), --NULL argument for SERIAL field - \${shouldCopyCurrentEntity});\`` - - Assert.equal(insertFnString, expected) - }) - - Async.it("Creating tables and functions works", async () => { - let storage = PgStorage.make( - ~sql=Db.sql, - ~pgSchema=Env.Db.publicSchema, - ~pgUser=Env.Db.user, - ~pgDatabase=Env.Db.database, - ~pgPassword=Env.Db.password, - ~pgHost=Env.Db.host, - ~pgPort=Env.Db.port, - ) - try { - let _ = await storage.initialize( - ~chainConfigs=[], - ~entities=[module(TestEntity)->Entities.entityModToInternal], - ~enums=[Persistence.entityHistoryActionEnumConfig->Internal.fromGenericEnumConfig], - ) - } catch { - | exn => - Js.log2("Setup exn", exn) - Assert.fail("Failed setting up tables") - } - - switch await Db.sql->Postgres.unsafe( - TestEntity.entityHistory.makeInsertFnQuery(~pgSchema="public"), - ) { - | exception exn => - Js.log2("createInsertFnQuery exn", exn) - Assert.fail("Failed creating insert function") - | _ => () - } - - let mockEntity: TestEntity.t = {id: "1", fieldA: 1, fieldB: Some("test")} - switch await Db.sql->batchSetMockEntity([mockEntity]) { - | exception exn => - Js.log2("batchSetMockEntity exn", exn) - Assert.fail("Failed to set mock entity in table") - | _ => () - } - let afterInsert = switch await Db.sql->getAllMockEntity { - | exception exn => - Js.log2("getAllMockEntity exn", exn) - Assert.fail("Failed to get mock entity from table")->Utils.magic - | entities => entities - } - - Assert.deepEqual(afterInsert, [mockEntity], ~message="Should have inserted mock entity") - - let chainId = 137 - let blockNumber = 123456 - let blockTimestamp = blockNumber * 15 - let logIndex = 1 - - let entityHistoryItem: testEntityHistory = { - current: { - chain_id: chainId, - block_timestamp: blockTimestamp, - block_number: blockNumber, - log_index: logIndex, - }, - previous: None, - entityData: Set({ - id: "1", - fieldA: 2, - fieldB: Some("test2"), - }), - } - - switch { - await Promise.all( - Db.sql->PgStorage.setEntityHistoryOrThrow( - ~entityHistory=TestEntity.entityHistory, - ~rows=[entityHistoryItem], - ~shouldCopyCurrentEntity=true, - ), - ) - } { - | exception exn => - Js.log2("insertRow exn", exn) - Assert.fail("Failed to insert mock entity history") - | _ => () - } - - let expectedResult = [ - { - "entity_history_block_timestamp": 0, - "entity_history_chain_id": 0, - "entity_history_block_number": 0, - "entity_history_log_index": 0, - "previous_entity_history_block_timestamp": Js.Nullable.Null, - "previous_entity_history_chain_id": Js.Nullable.Null, - "previous_entity_history_block_number": Js.Nullable.Null, - "previous_entity_history_log_index": Js.Nullable.Null, - "id": "1", - "fieldA": 1, - "fieldB": "test", - "action": "SET", - "serial": 1, - }, - { - "entity_history_block_timestamp": blockTimestamp, - "entity_history_chain_id": chainId, - "entity_history_block_number": blockNumber, - "entity_history_log_index": logIndex, - "previous_entity_history_block_timestamp": Js.Nullable.Value(0), - "previous_entity_history_chain_id": Js.Nullable.Value(0), - "previous_entity_history_block_number": Js.Nullable.Value(0), - "previous_entity_history_log_index": Js.Nullable.Value(0), - "id": "1", - "fieldA": 2, - "fieldB": "test2", - "action": "SET", - "serial": 2, - }, - ] - - let currentHistoryItems = await Db.sql->getAllMockEntityHistory - Assert.deepEqual(currentHistoryItems, expectedResult) - - switch await Promise.all( - Db.sql->PgStorage.setEntityHistoryOrThrow( - ~entityHistory=TestEntity.entityHistory, - ~rows=[ - { - entityData: Set({id: "2", fieldA: 1, fieldB: None}), - previous: None, - current: { - chain_id: 1, - block_timestamp: 4, - block_number: 4, - log_index: 6, - }, - }, - ], - ~shouldCopyCurrentEntity=true, - ), - ) { - | exception exn => - Js.log2("insertRow exn", exn) - Assert.fail("Failed to insert mock entity history") - | _ => () - } - switch await Promise.all( - Db.sql->PgStorage.setEntityHistoryOrThrow( - ~entityHistory=TestEntity.entityHistory, - ~rows=[ - { - entityData: Set({id: "2", fieldA: 3, fieldB: None}), - previous: None, - current: { - chain_id: 1, - block_timestamp: 4, - block_number: 10, - log_index: 6, - }, - }, - ], - ~shouldCopyCurrentEntity=true, - ), - ) { - | exception exn => - Js.log2("insertRow exn", exn) - Assert.fail("Failed to insert mock entity history") - | _ => () - } - - let _ = await Promise.all( - Db.sql->PgStorage.setEntityHistoryOrThrow( - ~entityHistory=TestEntity.entityHistory, - ~rows=[ - { - entityData: Set({id: "3", fieldA: 4, fieldB: None}), - previous: None, - current: { - chain_id: 137, - block_timestamp: 4, - block_number: 7, - log_index: 6, - }, - }, - ], - ~shouldCopyCurrentEntity=true, - ), - ) - }) - - it("Creates prune stale entity history query", () => { - let query = EntityHistory.makePruneStaleEntityHistoryQuery( - ~entityName="TestEntity", - ~pgSchema="foo", - ) - Assert.equal( - query, - `WITH safe AS ( - SELECT s.chain_id, s.block_number - FROM unnest($1::int[], $2::bigint[]) AS s(chain_id, block_number) -), -max_before_safe AS ( - SELECT t.id, MAX(t.serial) AS keep_serial - FROM "foo"."TestEntity_history" t - JOIN safe s - ON s.chain_id = t.entity_history_chain_id - AND t.entity_history_block_number <= s.block_number - GROUP BY t.id -), -post_safe AS ( - SELECT DISTINCT t.id - FROM "foo"."TestEntity_history" t - JOIN safe s - ON s.chain_id = t.entity_history_chain_id - AND t.entity_history_block_number > s.block_number -) -DELETE FROM "foo"."TestEntity_history" d -USING max_before_safe m -LEFT JOIN post_safe p ON p.id = m.id -WHERE d.id = m.id - AND ( - d.serial < m.keep_serial - OR (p.id IS NULL AND d.serial = m.keep_serial) - );`, - ) - }) -}) - -module Mocks = { - module Entity = { - open TestEntity - let entityId1 = "1" - let mockEntity1 = {id: entityId1, fieldA: 1, fieldB: Some("test")} - let mockEntity2 = {id: entityId1, fieldA: 2, fieldB: Some("test2")} - let mockEntity3 = {id: entityId1, fieldA: 3, fieldB: Some("test3")} - let mockEntity4 = {id: entityId1, fieldA: 4, fieldB: Some("test4")} - - let entityId2 = "2" - let mockEntity5 = {id: entityId2, fieldA: 5, fieldB: None} - let mockEntity6 = {id: entityId2, fieldA: 6, fieldB: None} - - let entityId3 = "3" - let mockEntity7 = {id: entityId3, fieldA: 7, fieldB: None} - let mockEntity8 = {id: entityId3, fieldA: 8, fieldB: None} - } - - module GnosisBug = { - let chain_id = 1 - - let event1: EntityHistory.historyFields = { - chain_id, - block_timestamp: 10 * 5, - block_number: 10, - log_index: 0, - } - - let event2: EntityHistory.historyFields = { - chain_id, - block_timestamp: 10 * 5, - block_number: 10, - log_index: 1, - } - - let historyRow1: testEntityHistory = { - current: event1, - previous: None, - entityData: Set(Entity.mockEntity2), - } - - let historyRow2: testEntityHistory = { - current: event2, - previous: None, - entityData: Set(Entity.mockEntity6), - } - - let historyRows = [historyRow1, historyRow2] - - // For setting a different entity and testing pruning - let event3: EntityHistory.historyFields = { - chain_id, - block_timestamp: 12 * 5, - block_number: 12, - log_index: 0, - } - - let historyRow3: testEntityHistory = { - current: event3, - previous: None, - entityData: Set(Entity.mockEntity3), - } - - let historyRow4: testEntityHistory = { - current: event3, - previous: None, - entityData: Set(Entity.mockEntity8), - } - - let historyRowsForPrune = [historyRow3, historyRow4] - } - - module Chain1 = { - let chain_id = 1 - - let event1: EntityHistory.historyFields = { - chain_id, - block_timestamp: 1, - block_number: 1, - log_index: 0, - } - - let event2: EntityHistory.historyFields = { - chain_id, - block_timestamp: 5, - block_number: 2, - log_index: 1, - } - - let event3: EntityHistory.historyFields = { - chain_id, - block_timestamp: 15, - block_number: 4, - log_index: 2, - } - - let historyRow1: testEntityHistory = { - current: event1, - previous: None, - entityData: Set(Entity.mockEntity1), - } - - let historyRow2: testEntityHistory = { - current: event2, - previous: Some(event1), - entityData: Set(Entity.mockEntity2), - } - - let historyRow3: testEntityHistory = { - current: event3, - previous: Some(event2), - entityData: Set(Entity.mockEntity3), - } - - let historyRows = [historyRow1, historyRow2, historyRow3] - - //Shows a case where no event exists on this block - let rollbackEventIdentifier: Types.eventIdentifier = { - blockTimestamp: 10, - chainId: chain_id, - blockNumber: 3, - logIndex: 0, - } - - let orderedMultichainArg = DbFunctions.EntityHistory.Args.OrderedMultichain({ - safeBlockTimestamp: rollbackEventIdentifier.blockTimestamp, - reorgChainId: chain_id, - safeBlockNumber: rollbackEventIdentifier.blockNumber, - }) - - let unorderedMultichainArg = DbFunctions.EntityHistory.Args.UnorderedMultichain({ - reorgChainId: chain_id, - safeBlockNumber: rollbackEventIdentifier.blockNumber, - }) - } - - module Chain2 = { - let chain_id = 2 - - let event1: EntityHistory.historyFields = { - chain_id, - block_timestamp: 3, - block_number: 1, - log_index: 0, - } - - let event2: EntityHistory.historyFields = { - chain_id, - block_timestamp: 8, - block_number: 2, - log_index: 1, - } - - let event3: EntityHistory.historyFields = { - chain_id, - block_timestamp: 13, - block_number: 3, - log_index: 2, - } - - let historyRow1: testEntityHistory = { - current: event1, - previous: None, - entityData: Set(Entity.mockEntity5), - } - - let historyRow2: testEntityHistory = { - current: event2, - previous: Some(event1), - entityData: Delete({id: Entity.entityId2}), - } - let historyRow3: testEntityHistory = { - current: event3, - previous: Some(event2), - entityData: Set(Entity.mockEntity6), - } - - let historyRows = [historyRow1, historyRow2, historyRow3] - } - - let historyRows = Utils.Array.mergeSorted( - (a, b) => a.EntityHistory.current.block_timestamp < b.current.block_timestamp, - Chain1.historyRows, - Chain2.historyRows, - ) -} - -describe("Entity history rollbacks", () => { - Async.beforeEach(async () => { - try { - let _ = DbHelpers.resetPostgresClient() - let storage = PgStorage.make( - ~sql=Db.sql, - ~pgSchema=Env.Db.publicSchema, - ~pgUser=Env.Db.user, - ~pgDatabase=Env.Db.database, - ~pgPassword=Env.Db.password, - ~pgHost=Env.Db.host, - ~pgPort=Env.Db.port, - ) - let _ = await storage.initialize( - ~chainConfigs=[], - ~entities=[module(TestEntity)->Entities.entityModToInternal], - ~enums=[Persistence.entityHistoryActionEnumConfig->Internal.fromGenericEnumConfig], - ) - - let _ = - await Db.sql->Postgres.unsafe( - TestEntity.entityHistory.makeInsertFnQuery(~pgSchema="public"), - ) - - try await Db.sql->PgStorage.setOrThrow( - ~items=[ - Mocks.Entity.mockEntity1->TestEntity.castToInternal, - Mocks.Entity.mockEntity5->TestEntity.castToInternal, - ], - ~table=TestEntity.table, - ~itemSchema=TestEntity.schema, - ~pgSchema=Config.storagePgSchema, - ) catch { - | exn => - Js.log2("batchSet mock entity exn", exn) - Assert.fail("Failed to set mock entity in table") - } - - try await Db.sql->Postgres.beginSql( - sql => - sql - ->PgStorage.setEntityHistoryOrThrow( - ~entityHistory=TestEntity.entityHistory, - ~rows=Mocks.GnosisBug.historyRows, - ) - ->Promise.all - ->Promise.ignoreValue, - ) catch { - | exn => - Js.log2("insert mock rows exn", exn) - Assert.fail("Failed to insert mock rows") - } - - let historyItems = { - let items = await Db.sql->getAllMockEntityHistory - items->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) - } - Assert.equal(historyItems->Js.Array2.length, 4, ~message="Should have 4 history items") - Assert.ok( - historyItems->Belt.Array.some(item => item.current.chain_id == 0), - ~message="Should contain 2 copied items", - ) - } catch { - | exn => - Js.log2(" Entity history setup exn", exn) - Assert.fail("Failed setting up tables") - } - }) - - Async.it("Rollback ignores copied entities as an item in reorg threshold", async () => { - let rollbackDiff = await Db.sql->DbFunctions.EntityHistory.getRollbackDiff( - OrderedMultichain({ - reorgChainId: Mocks.GnosisBug.chain_id, - safeBlockNumber: 9, - safeBlockTimestamp: 9 * 5, - }), - ~entityConfig=module(TestEntity)->Entities.entityModToInternal, - ) - - let expectedDiff: array> = [ - { - current: {chain_id: 0, block_timestamp: 0, block_number: 0, log_index: 0}, - previous: %raw(`undefined`), - entityData: Set(Mocks.Entity.mockEntity1->TestEntity.castToInternal), - }, - { - current: {chain_id: 0, block_timestamp: 0, block_number: 0, log_index: 0}, - previous: %raw(`undefined`), - entityData: Set(Mocks.Entity.mockEntity5->TestEntity.castToInternal), - }, - ] - - Assert.deepStrictEqual( - rollbackDiff, - expectedDiff, - ~message="Should rollback to the copied entity", - ) - }) - - Async.it( - "Deleting items after reorg event should not remove the copied history item", - async () => { - await Db.sql->DbFunctions.EntityHistory.deleteAllEntityHistoryAfterEventIdentifier( - ~isUnorderedMultichainMode=false, - ~eventIdentifier={ - chainId: Mocks.GnosisBug.chain_id, - blockTimestamp: 9 * 5, - blockNumber: 9, - logIndex: 0, - }, - ~allEntities=[module(TestEntity)->Entities.entityModToInternal], - ) - - let historyItems = { - let items = await Db.sql->getAllMockEntityHistory - items->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) - } - - Assert.equal(historyItems->Js.Array2.length, 2, ~message="Should have the 2 copied items") - - let allItemsAreZeroChainId = - historyItems->Belt.Array.every(item => item.current.chain_id == 0) - - Assert.ok( - allItemsAreZeroChainId, - ~message="Should have all items in the zero chain id since they are copied", - ) - }, - ) - - Async.it("Prunes history correctly with items in reorg threshold", async () => { - // set the current entity of id 3 - await Db.sql->PgStorage.setOrThrow( - ~items=[Mocks.Entity.mockEntity7->TestEntity.castToInternal], - ~table=TestEntity.table, - ~itemSchema=TestEntity.schema, - ~pgSchema=Config.storagePgSchema, - ) - - // set an updated version of its row to get a copied entity history - try await Db.sql->Postgres.beginSql( - sql => - sql - ->PgStorage.setEntityHistoryOrThrow( - ~entityHistory=TestEntity.entityHistory, - ~rows=Mocks.GnosisBug.historyRowsForPrune, - ) - ->Promise.all - ->Promise.ignoreValue, - ) catch { - | exn => - Js.log2("insert mock rows exn", exn) - Assert.fail("Failed to insert mock rows") - } - - // let historyItemsBefore = { - // let items = await Db.sql->getAllMockEntityHistory - // Js.log2("history items before prune", items) - // items->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) - // } - - let () = await Db.sql->EntityHistory.pruneStaleEntityHistory( - ~entityName=(module(TestEntity)->Entities.entityModToInternal).name, - ~pgSchema=Env.Db.publicSchema, - ~safeReorgBlocks={ - chainIds: [Mocks.GnosisBug.chain_id], - blockNumbers: [11], - }, - ) - - let historyItemsAfter = { - let items = await Db.sql->getAllMockEntityHistory - // Js.log2("history items after prune", items) - items->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) - } - - Assert.equal( - historyItemsAfter->Js.Array2.length, - 4, - ~message="Should have 4 history items for entity id 1 and 3 before and after block 11", - ) - }) -}) - -describe("Entity history rollbacks", () => { - Async.beforeEach(async () => { - try { - let _ = DbHelpers.resetPostgresClient() - let storage = PgStorage.make( - ~sql=Db.sql, - ~pgSchema=Env.Db.publicSchema, - ~pgUser=Env.Db.user, - ~pgDatabase=Env.Db.database, - ~pgPassword=Env.Db.password, - ~pgHost=Env.Db.host, - ~pgPort=Env.Db.port, - ) - let _ = await storage.initialize( - ~chainConfigs=[], - ~entities=[module(TestEntity)->Entities.entityModToInternal], - ~enums=[Persistence.entityHistoryActionEnumConfig->Internal.fromGenericEnumConfig], - ) - - let _ = - await Db.sql->Postgres.unsafe( - TestEntity.entityHistory.makeInsertFnQuery(~pgSchema="public"), - ) - - try await Db.sql->Postgres.beginSql( - sql => - sql - ->PgStorage.setEntityHistoryOrThrow( - ~entityHistory=TestEntity.entityHistory, - ~rows=Mocks.historyRows, - ) - ->Promise.all - ->Promise.ignoreValue, - ) catch { - | exn => - Js.log2("insert mock rows exn", exn) - Assert.fail("Failed to insert mock rows") - } - - let historyItems = { - let items = await Db.sql->getAllMockEntityHistory - items->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) - } - Assert.equal(historyItems->Js.Array2.length, 6, ~message="Should have 6 history items") - Assert.ok( - !(historyItems->Belt.Array.some(item => item.current.chain_id == 0)), - ~message="No defaulted/copied values should exist in history", - ) - } catch { - | exn => - Js.log2(" Entity history setup exn", exn) - Assert.fail("Failed setting up tables") - } - }) - - Async.it("Returns expected diff for ordered multichain mode", async () => { - let orderdMultichainRollbackDiff = try await Db.sql->DbFunctions.EntityHistory.getRollbackDiff( - Mocks.Chain1.orderedMultichainArg, - ~entityConfig=module(TestEntity)->Entities.entityModToInternal, - ) catch { - | exn => - Js.log2("getRollbackDiff exn", exn) - Assert.fail("Failed to get rollback diff") - } - - switch orderdMultichainRollbackDiff { - | [ - {current: currentA, entityData: Set(entitySetA)}, - {current: currentB, entityData: Delete({id: entityDeleteB})}, - ] => - Assert.deepEqual( - currentA, - Mocks.Chain1.event2, - ~message="First history item should haved diffed to event2", - ) - Assert.deepEqual( - entitySetA, - Mocks.Entity.mockEntity2->TestEntity.castToInternal, - ~message="First history item should haved diffed to mockEntity2", - ) - Assert.deepEqual( - currentB, - Mocks.Chain2.event2, - ~message="Second history item should haved diffed to event3", - ) - Assert.deepEqual( - entityDeleteB, - Mocks.Entity.entityId2, - ~message="Second history item should haved diffed a delete of entityId2", - ) - | _ => Assert.fail("Should have a set and delete history item in diff") - } - }) - - Async.it("Returns expected diff for unordered multichain mode", async () => { - let unorderedMultichainRollbackDiff = try await Db.sql->DbFunctions.EntityHistory.getRollbackDiff( - Mocks.Chain1.unorderedMultichainArg, - ~entityConfig=module(TestEntity)->Entities.entityModToInternal, - ) catch { - | exn => - Js.log2("getRollbackDiff exn", exn) - Assert.fail("Failed to get rollback diff") - } - - switch unorderedMultichainRollbackDiff { - | [{current: currentA, entityData: Set(entitySetA)}] => - Assert.deepEqual( - currentA, - Mocks.Chain1.event2, - ~message="First history item should haved diffed to event2", - ) - Assert.deepEqual( - entitySetA, - Mocks.Entity.mockEntity2->TestEntity.castToInternal, - ~message="First history item should haved diffed to mockEntity2", - ) - | _ => Assert.fail("Should have only chain 1 item in diff") - } - }) - - Async.it("Gets first event change per chain ordered mode", async () => { - let firstChangeEventPerChain = try await Db.sql->DbFunctions.EntityHistory.getFirstChangeEventPerChain( - Mocks.Chain1.orderedMultichainArg, - ~allEntities=[module(TestEntity)->Entities.entityModToInternal], - ) catch { - | exn => - Js.log2("getFirstChangeEventPerChain exn", exn) - Assert.fail("Failed to get rollback diff") - } - - let expected = DbFunctions.EntityHistory.FirstChangeEventPerChain.make() - expected->DbFunctions.EntityHistory.FirstChangeEventPerChain.setIfEarlier( - ~chainId=Mocks.Chain1.chain_id, - ~event={ - blockNumber: Mocks.Chain1.event3.block_number, - logIndex: Mocks.Chain1.event3.log_index, - }, - ) - expected->DbFunctions.EntityHistory.FirstChangeEventPerChain.setIfEarlier( - ~chainId=Mocks.Chain2.chain_id, - ~event={ - blockNumber: Mocks.Chain2.event3.block_number, - logIndex: Mocks.Chain2.event3.log_index, - }, - ) - - Assert.deepEqual( - firstChangeEventPerChain, - expected, - ~message="Should have chain 1 and 2 first change events", - ) - }) - - Async.it("Gets first event change per chain unordered mode", async () => { - let firstChangeEventPerChain = try await Db.sql->DbFunctions.EntityHistory.getFirstChangeEventPerChain( - Mocks.Chain1.unorderedMultichainArg, - ~allEntities=[module(TestEntity)->Entities.entityModToInternal], - ) catch { - | exn => - Js.log2("getFirstChangeEventPerChain exn", exn) - Assert.fail("Failed to get rollback diff") - } - - let expected = DbFunctions.EntityHistory.FirstChangeEventPerChain.make() - expected->DbFunctions.EntityHistory.FirstChangeEventPerChain.setIfEarlier( - ~chainId=Mocks.Chain1.chain_id, - ~event={ - blockNumber: Mocks.Chain1.event3.block_number, - logIndex: Mocks.Chain1.event3.log_index, - }, - ) - - Assert.deepEqual( - firstChangeEventPerChain, - expected, - ~message="Should only have chain 1 first change event", - ) - }) - - Async.it("Deletes current history after rollback ordered", async () => { - let _ = - await Db.sql->DbFunctions.EntityHistory.deleteAllEntityHistoryAfterEventIdentifier( - ~isUnorderedMultichainMode=false, - ~eventIdentifier=Mocks.Chain1.rollbackEventIdentifier, - ~allEntities=[module(TestEntity)->Entities.entityModToInternal], - ) - - let currentHistoryItems = await Db.sql->getAllMockEntityHistory - let parsedHistoryItems = - currentHistoryItems->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) - - let expectedHistoryItems = Mocks.historyRows->Belt.Array.slice(~offset=0, ~len=4) - - Assert.deepEqual( - parsedHistoryItems->stripUndefinedFieldsInPlace, - expectedHistoryItems->stripUndefinedFieldsInPlace, - ~message="Should have deleted last 2 items in history", - ) - }) - - Async.it("Deletes current history after rollback unordered", async () => { - let _ = - await Db.sql->DbFunctions.EntityHistory.deleteAllEntityHistoryAfterEventIdentifier( - ~isUnorderedMultichainMode=true, - ~eventIdentifier=Mocks.Chain1.rollbackEventIdentifier, - ~allEntities=[module(TestEntity)->Entities.entityModToInternal], - ) - - let currentHistoryItems = await Db.sql->getAllMockEntityHistory - let parsedHistoryItems = - currentHistoryItems->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) - - let expectedHistoryItems = Mocks.historyRows->Belt.Array.slice(~offset=0, ~len=5) - - Assert.deepEqual( - parsedHistoryItems->stripUndefinedFieldsInPlace, - expectedHistoryItems->stripUndefinedFieldsInPlace, - ~message="Should have deleted just the last item in history", - ) - }) - - Async.it("Prunes history correctly with items in reorg threshold", async () => { - let () = await Db.sql->EntityHistory.pruneStaleEntityHistory( - ~entityName=(module(TestEntity)->Entities.entityModToInternal).name, - ~pgSchema=Env.Db.publicSchema, - ~safeReorgBlocks={ - chainIds: [1, 2], - blockNumbers: [3, 2], - }, - ) - let currentHistoryItems = await Db.sql->getAllMockEntityHistory - - let parsedHistoryItems = - currentHistoryItems->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) - - let expectedHistoryItems = [ - Mocks.Chain1.historyRow2, - Mocks.Chain1.historyRow3, - Mocks.Chain2.historyRow2, - Mocks.Chain2.historyRow3, - ] - - let sort = arr => - arr->Js.Array2.sortInPlaceWith( - (a, b) => a.EntityHistory.current.block_number - b.current.block_number, - ) - - Assert.deepEqual( - parsedHistoryItems->sort->stripUndefinedFieldsInPlace, - expectedHistoryItems->sort->stripUndefinedFieldsInPlace, - ~message="Should have deleted the unneeded first items in history", - ) - }) - - Async.it("Prunes history correctly with items in reorg threshold", async () => { - let () = await Db.sql->EntityHistory.pruneStaleEntityHistory( - ~entityName=(module(TestEntity)->Entities.entityModToInternal).name, - ~pgSchema=Env.Db.publicSchema, - ~safeReorgBlocks={ - chainIds: [1, 2], - blockNumbers: [3, 2], - }, - ) - let currentHistoryItems = await Db.sql->getAllMockEntityHistory - - let parsedHistoryItems = - currentHistoryItems->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) - - let sort = arr => - arr->Js.Array2.sortInPlaceWith( - (a, b) => a.EntityHistory.current.block_number - b.current.block_number, - ) - - Assert.deepEqual( - parsedHistoryItems->sort->stripUndefinedFieldsInPlace, - [ - Mocks.Chain1.historyRow2, - Mocks.Chain2.historyRow2, - Mocks.Chain2.historyRow3, - Mocks.Chain1.historyRow3, - ]->stripUndefinedFieldsInPlace, - ~message="Should have deleted the unneeded first items in history", - ) - }) - - Async.it("Prunes history correctly with no items in reorg threshold", async () => { - let () = await Db.sql->EntityHistory.pruneStaleEntityHistory( - ~entityName=(module(TestEntity)->Entities.entityModToInternal).name, - ~pgSchema=Env.Db.publicSchema, - ~safeReorgBlocks={ - chainIds: [1, 2], - blockNumbers: [4, 3], - }, - ) - let currentHistoryItems = await Db.sql->getAllMockEntityHistory - - Assert.ok( - currentHistoryItems->Array.length == 0, - ~message="Should have deleted all items in history", - ) - }) -}) - -describe_skip("Prune performance test", () => { - Async.it("Print benchmark of prune function", async () => { - let _ = DbHelpers.resetPostgresClient() - let storage = PgStorage.make( - ~sql=Db.sql, - ~pgSchema=Env.Db.publicSchema, - ~pgUser=Env.Db.user, - ~pgDatabase=Env.Db.database, - ~pgPassword=Env.Db.password, - ~pgHost=Env.Db.host, - ~pgPort=Env.Db.port, - ) - let _ = await storage.initialize( - ~entities=[module(TestEntity)->Entities.entityModToInternal], - ~chainConfigs=[], - ~enums=[Persistence.entityHistoryActionEnumConfig->Internal.fromGenericEnumConfig], - ) - - let _ = - await Db.sql->Postgres.unsafe(TestEntity.entityHistory.makeInsertFnQuery(~pgSchema="public")) - - let rows: array = [] - for i in 0 to 1000 { - let mockEntity: TestEntity.t = { - id: i->mod(10)->Belt.Int.toString, - fieldA: i, - fieldB: None, - } - - let historyRow: testEntityHistory = { - current: { - chain_id: 1, - block_timestamp: i * 5, - block_number: i, - log_index: 0, - }, - previous: None, - entityData: Set(mockEntity), - } - rows->Js.Array2.push(historyRow)->ignore - } - - try await Db.sql->Postgres.beginSql( - sql => - sql - ->PgStorage.setEntityHistoryOrThrow(~entityHistory=TestEntity.entityHistory, ~rows) - ->Promise.all - ->Promise.ignoreValue, - ) catch { - | exn => - Js.log2("insert mock rows exn", exn) - Assert.fail("Failed to insert mock rows") - } - - let startTime = Hrtime.makeTimer() - - try { - let () = await Db.sql->EntityHistory.pruneStaleEntityHistory( - ~entityName=(module(TestEntity)->Entities.entityModToInternal).name, - ~pgSchema=Env.Db.publicSchema, - ~safeReorgBlocks={ - chainIds: [1], - blockNumbers: [500], - }, - ) - } catch { - | exn => - Js.log2("prune stale entity history exn", exn) - Assert.fail("Failed to prune stale entity history") - } - - let elapsedTime = Hrtime.timeSince(startTime)->Hrtime.toMillis->Hrtime.intFromMillis - Js.log2("Elapsed time", elapsedTime) - }) -}) +// open RescriptMocha + +// //unsafe polymorphic toString binding for any type +// @send external toStringUnsafe: 'a => string = "toString" + +// let stripUndefinedFieldsInPlace = (val: 'a): 'a => { +// let json = val->(Utils.magic: 'a => Js.Json.t) +// //Hot fix for rescript equality check that removes optional fields +// let rec strip = (json: Js.Json.t) => { +// switch json { +// | Object(obj) => +// obj +// ->Js.Dict.keys +// ->Belt.Array.forEach(key => { +// let value = obj->Utils.Dict.dangerouslyGetNonOption(key) +// if value === %raw(`undefined`) { +// obj->Utils.Dict.deleteInPlace(key) +// } else { +// strip(value->Belt.Option.getExn) +// } +// }) +// | Array(arr) => arr->Belt.Array.forEach(value => strip(value)) +// | _ => () +// } +// } + +// json->strip +// json->(Utils.magic: Js.Json.t => 'a) +// } + +// module TestEntity = { +// type t = { +// id: string, +// fieldA: int, +// fieldB: option, +// } + +// let name = "TestEntity" +// let schema = S.schema(s => { +// id: s.matches(S.string), +// fieldA: s.matches(S.int), +// fieldB: s.matches(S.null(S.string)), +// }) + +// let rowsSchema = S.array(schema) +// let table = Table.mkTable( +// "TestEntity", +// ~fields=[ +// Table.mkField("id", Text, ~fieldSchema=S.string, ~isPrimaryKey=true), +// Table.mkField("fieldA", Integer, ~fieldSchema=S.int), +// Table.mkField("fieldB", Text, ~fieldSchema=S.null(S.string), ~isNullable=true), +// ], +// ) + +// let entityHistory = table->EntityHistory.fromTable(~schema) + +// external castToInternal: t => Internal.entity = "%identity" +// } + +// type testEntityHistory = EntityHistory.entityUpdate +// let testEntitySetUpdateSchema = EntityHistory.makeSetUpdateSchema(TestEntity.schema) + +// let batchSetMockEntity = (sql, items) => +// PgStorage.setOrThrow( +// sql, +// ~items, +// ~pgSchema="public", +// ~table=TestEntity.table, +// ~itemSchema=TestEntity.schema, +// ) + +// let getAllMockEntity = sql => +// sql +// ->Postgres.unsafe(`SELECT * FROM "public"."${TestEntity.table.tableName}"`) +// ->Promise.thenResolve(json => json->S.parseJsonOrThrow(TestEntity.rowsSchema)) + +// let getAllMockEntityHistory = sql => +// sql->Postgres.unsafe(`SELECT * FROM "public"."${TestEntity.entityHistory.table.tableName}"`) + +// describe("Entity History Codegen", () => { +// it("Creates a postgres insert function", () => { +// let expected = `CREATE OR REPLACE FUNCTION "insert_TestEntity_history"(history_row "public"."TestEntity_history", should_copy_current_entity BOOLEAN) +// RETURNS void AS $$ +// DECLARE +// v_previous_record RECORD; +// v_origin_record RECORD; +// BEGIN +// -- Check if previous values are not provided +// IF history_row.previous_entity_history_block_timestamp IS NULL OR history_row.previous_entity_history_chain_id IS NULL OR history_row.previous_entity_history_block_number IS NULL OR history_row.previous_entity_history_log_index IS NULL THEN +// -- Find the most recent record for the same id +// SELECT entity_history_block_timestamp, entity_history_chain_id, entity_history_block_number, entity_history_log_index INTO v_previous_record +// FROM "public"."TestEntity_history" +// WHERE id = history_row.id +// ORDER BY entity_history_block_timestamp DESC, entity_history_chain_id DESC, entity_history_block_number DESC, entity_history_log_index DESC +// LIMIT 1; + +// -- If a previous record exists, use its values +// IF FOUND THEN +// history_row.previous_entity_history_block_timestamp := v_previous_record.entity_history_block_timestamp; history_row.previous_entity_history_chain_id := v_previous_record.entity_history_chain_id; history_row.previous_entity_history_block_number := v_previous_record.entity_history_block_number; history_row.previous_entity_history_log_index := v_previous_record.entity_history_log_index; +// ElSIF should_copy_current_entity THEN +// -- Check if a value for the id exists in the origin table and if so, insert a history row for it. +// SELECT "id", "fieldA", "fieldB" FROM "public"."TestEntity" WHERE id = history_row.id INTO v_origin_record; +// IF FOUND THEN +// INSERT INTO "public"."TestEntity_history" (entity_history_block_timestamp, entity_history_chain_id, entity_history_block_number, entity_history_log_index, "id", "fieldA", "fieldB", "action") +// -- SET the current change data fields to 0 since we don't know what they were +// -- and it doesn't matter provided they are less than any new values +// VALUES (0, 0, 0, 0, v_origin_record."id", v_origin_record."fieldA", v_origin_record."fieldB", 'SET'); + +// history_row.previous_entity_history_block_timestamp := 0; history_row.previous_entity_history_chain_id := 0; history_row.previous_entity_history_block_number := 0; history_row.previous_entity_history_log_index := 0; +// END IF; +// END IF; +// END IF; + +// INSERT INTO "public"."TestEntity_history" ("entity_history_block_timestamp", "entity_history_chain_id", "entity_history_block_number", "entity_history_log_index", "previous_entity_history_block_timestamp", "previous_entity_history_chain_id", "previous_entity_history_block_number", "previous_entity_history_log_index", "id", "fieldA", "fieldB", "action") +// VALUES (history_row."entity_history_block_timestamp", history_row."entity_history_chain_id", history_row."entity_history_block_number", history_row."entity_history_log_index", history_row."previous_entity_history_block_timestamp", history_row."previous_entity_history_chain_id", history_row."previous_entity_history_block_number", history_row."previous_entity_history_log_index", history_row."id", history_row."fieldA", history_row."fieldB", history_row."action"); +// END; +// $$ LANGUAGE plpgsql;` + +// Assert.equal(expected, TestEntity.entityHistory.makeInsertFnQuery(~pgSchema="public")) +// }) +// it("Creates an entity history table", () => { +// let createQuery = +// TestEntity.entityHistory.table->PgStorage.makeCreateTableQuery(~pgSchema="public") +// Assert.equal( +// `CREATE TABLE IF NOT EXISTS "public"."TestEntity_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, "id" TEXT NOT NULL, "fieldA" INTEGER, "fieldB" 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"));`, +// createQuery, +// ) +// }) + +// it("Creates a js insert function", () => { +// let insertFnString = TestEntity.entityHistory.insertFn->toStringUnsafe + +// let expected = `(sql, rowArgs, shouldCopyCurrentEntity) => +// sql\`select "insert_TestEntity_history"(ROW(\${rowArgs["entity_history_block_timestamp"]}, \${rowArgs["entity_history_chain_id"]}, \${rowArgs["entity_history_block_number"]}, \${rowArgs["entity_history_log_index"]}, \${rowArgs["previous_entity_history_block_timestamp"]}, \${rowArgs["previous_entity_history_chain_id"]}, \${rowArgs["previous_entity_history_block_number"]}, \${rowArgs["previous_entity_history_log_index"]}, \${rowArgs["id"]}, \${rowArgs["fieldA"]}, \${rowArgs["fieldB"]}, \${rowArgs["action"]}, NULL), --NULL argument for SERIAL field +// \${shouldCopyCurrentEntity});\`` + +// Assert.equal(insertFnString, expected) +// }) + +// Async.it("Creating tables and functions works", async () => { +// let storage = PgStorage.make( +// ~sql=Db.sql, +// ~pgSchema=Env.Db.publicSchema, +// ~pgUser=Env.Db.user, +// ~pgDatabase=Env.Db.database, +// ~pgPassword=Env.Db.password, +// ~pgHost=Env.Db.host, +// ~pgPort=Env.Db.port, +// ) +// try { +// let _ = await storage.initialize( +// ~chainConfigs=[], +// ~entities=[module(TestEntity)->Entities.entityModToInternal], +// ~enums=[Persistence.entityHistoryActionEnumConfig->Internal.fromGenericEnumConfig], +// ) +// } catch { +// | exn => +// Js.log2("Setup exn", exn) +// Assert.fail("Failed setting up tables") +// } + +// switch await Db.sql->Postgres.unsafe( +// TestEntity.entityHistory.makeInsertFnQuery(~pgSchema="public"), +// ) { +// | exception exn => +// Js.log2("createInsertFnQuery exn", exn) +// Assert.fail("Failed creating insert function") +// | _ => () +// } + +// let mockEntity: TestEntity.t = {id: "1", fieldA: 1, fieldB: Some("test")} +// switch await Db.sql->batchSetMockEntity([mockEntity]) { +// | exception exn => +// Js.log2("batchSetMockEntity exn", exn) +// Assert.fail("Failed to set mock entity in table") +// | _ => () +// } +// let afterInsert = switch await Db.sql->getAllMockEntity { +// | exception exn => +// Js.log2("getAllMockEntity exn", exn) +// Assert.fail("Failed to get mock entity from table")->Utils.magic +// | entities => entities +// } + +// Assert.deepEqual(afterInsert, [mockEntity], ~message="Should have inserted mock entity") + +// let chainId = 137 +// let blockNumber = 123456 +// let blockTimestamp = blockNumber * 15 +// let logIndex = 1 + +// let entityHistoryItem: testEntityHistory = { +// current: { +// chain_id: chainId, +// block_timestamp: blockTimestamp, +// block_number: blockNumber, +// log_index: logIndex, +// }, +// previous: None, +// entityData: Set({ +// id: "1", +// fieldA: 2, +// fieldB: Some("test2"), +// }), +// } + +// switch { +// await Promise.all( +// Db.sql->PgStorage.setEntityHistoryOrThrow( +// ~entityHistory=TestEntity.entityHistory, +// ~rows=[entityHistoryItem], +// ~shouldCopyCurrentEntity=true, +// ), +// ) +// } { +// | exception exn => +// Js.log2("insertRow exn", exn) +// Assert.fail("Failed to insert mock entity history") +// | _ => () +// } + +// let expectedResult = [ +// { +// "entity_history_block_timestamp": 0, +// "entity_history_chain_id": 0, +// "entity_history_block_number": 0, +// "entity_history_log_index": 0, +// "previous_entity_history_block_timestamp": Js.Nullable.Null, +// "previous_entity_history_chain_id": Js.Nullable.Null, +// "previous_entity_history_block_number": Js.Nullable.Null, +// "previous_entity_history_log_index": Js.Nullable.Null, +// "id": "1", +// "fieldA": 1, +// "fieldB": "test", +// "action": "SET", +// "serial": 1, +// }, +// { +// "entity_history_block_timestamp": blockTimestamp, +// "entity_history_chain_id": chainId, +// "entity_history_block_number": blockNumber, +// "entity_history_log_index": logIndex, +// "previous_entity_history_block_timestamp": Js.Nullable.Value(0), +// "previous_entity_history_chain_id": Js.Nullable.Value(0), +// "previous_entity_history_block_number": Js.Nullable.Value(0), +// "previous_entity_history_log_index": Js.Nullable.Value(0), +// "id": "1", +// "fieldA": 2, +// "fieldB": "test2", +// "action": "SET", +// "serial": 2, +// }, +// ] + +// let currentHistoryItems = await Db.sql->getAllMockEntityHistory +// Assert.deepEqual(currentHistoryItems, expectedResult) + +// switch await Promise.all( +// Db.sql->PgStorage.setEntityHistoryOrThrow( +// ~entityHistory=TestEntity.entityHistory, +// ~rows=[ +// { +// entityData: Set({id: "2", fieldA: 1, fieldB: None}), +// previous: None, +// current: { +// chain_id: 1, +// block_timestamp: 4, +// block_number: 4, +// log_index: 6, +// }, +// }, +// ], +// ~shouldCopyCurrentEntity=true, +// ), +// ) { +// | exception exn => +// Js.log2("insertRow exn", exn) +// Assert.fail("Failed to insert mock entity history") +// | _ => () +// } +// switch await Promise.all( +// Db.sql->PgStorage.setEntityHistoryOrThrow( +// ~entityHistory=TestEntity.entityHistory, +// ~rows=[ +// { +// entityData: Set({id: "2", fieldA: 3, fieldB: None}), +// previous: None, +// current: { +// chain_id: 1, +// block_timestamp: 4, +// block_number: 10, +// log_index: 6, +// }, +// }, +// ], +// ~shouldCopyCurrentEntity=true, +// ), +// ) { +// | exception exn => +// Js.log2("insertRow exn", exn) +// Assert.fail("Failed to insert mock entity history") +// | _ => () +// } + +// let _ = await Promise.all( +// Db.sql->PgStorage.setEntityHistoryOrThrow( +// ~entityHistory=TestEntity.entityHistory, +// ~rows=[ +// { +// entityData: Set({id: "3", fieldA: 4, fieldB: None}), +// previous: None, +// current: { +// chain_id: 137, +// block_timestamp: 4, +// block_number: 7, +// log_index: 6, +// }, +// }, +// ], +// ~shouldCopyCurrentEntity=true, +// ), +// ) +// }) + +// it("Creates prune stale entity history query", () => { +// let query = EntityHistory.makePruneStaleEntityHistoryQuery( +// ~entityName="TestEntity", +// ~pgSchema="foo", +// ) +// Assert.equal( +// query, +// `WITH safe AS ( +// SELECT s.chain_id, s.block_number +// FROM unnest($1::int[], $2::bigint[]) AS s(chain_id, block_number) +// ), +// max_before_safe AS ( +// SELECT t.id, MAX(t.serial) AS keep_serial +// FROM "foo"."TestEntity_history" t +// JOIN safe s +// ON s.chain_id = t.entity_history_chain_id +// AND t.entity_history_block_number <= s.block_number +// GROUP BY t.id +// ), +// post_safe AS ( +// SELECT DISTINCT t.id +// FROM "foo"."TestEntity_history" t +// JOIN safe s +// ON s.chain_id = t.entity_history_chain_id +// AND t.entity_history_block_number > s.block_number +// ) +// DELETE FROM "foo"."TestEntity_history" d +// USING max_before_safe m +// LEFT JOIN post_safe p ON p.id = m.id +// WHERE d.id = m.id +// AND ( +// d.serial < m.keep_serial +// OR (p.id IS NULL AND d.serial = m.keep_serial) +// );`, +// ) +// }) +// }) + +// module Mocks = { +// module Entity = { +// open TestEntity +// let entityId1 = "1" +// let mockEntity1 = {id: entityId1, fieldA: 1, fieldB: Some("test")} +// let mockEntity2 = {id: entityId1, fieldA: 2, fieldB: Some("test2")} +// let mockEntity3 = {id: entityId1, fieldA: 3, fieldB: Some("test3")} +// let mockEntity4 = {id: entityId1, fieldA: 4, fieldB: Some("test4")} + +// let entityId2 = "2" +// let mockEntity5 = {id: entityId2, fieldA: 5, fieldB: None} +// let mockEntity6 = {id: entityId2, fieldA: 6, fieldB: None} + +// let entityId3 = "3" +// let mockEntity7 = {id: entityId3, fieldA: 7, fieldB: None} +// let mockEntity8 = {id: entityId3, fieldA: 8, fieldB: None} +// } + +// module GnosisBug = { +// let chain_id = 1 + +// let event1: EntityHistory.historyFields = { +// chain_id, +// block_timestamp: 10 * 5, +// block_number: 10, +// log_index: 0, +// } + +// let event2: EntityHistory.historyFields = { +// chain_id, +// block_timestamp: 10 * 5, +// block_number: 10, +// log_index: 1, +// } + +// let historyRow1: testEntityHistory = { +// current: event1, +// previous: None, +// entityData: Set(Entity.mockEntity2), +// } + +// let historyRow2: testEntityHistory = { +// current: event2, +// previous: None, +// entityData: Set(Entity.mockEntity6), +// } + +// let historyRows = [historyRow1, historyRow2] + +// // For setting a different entity and testing pruning +// let event3: EntityHistory.historyFields = { +// chain_id, +// block_timestamp: 12 * 5, +// block_number: 12, +// log_index: 0, +// } + +// let historyRow3: testEntityHistory = { +// current: event3, +// previous: None, +// entityData: Set(Entity.mockEntity3), +// } + +// let historyRow4: testEntityHistory = { +// current: event3, +// previous: None, +// entityData: Set(Entity.mockEntity8), +// } + +// let historyRowsForPrune = [historyRow3, historyRow4] +// } + +// module Chain1 = { +// let chain_id = 1 + +// let event1: EntityHistory.historyFields = { +// chain_id, +// block_timestamp: 1, +// block_number: 1, +// log_index: 0, +// } + +// let event2: EntityHistory.historyFields = { +// chain_id, +// block_timestamp: 5, +// block_number: 2, +// log_index: 1, +// } + +// let event3: EntityHistory.historyFields = { +// chain_id, +// block_timestamp: 15, +// block_number: 4, +// log_index: 2, +// } + +// let historyRow1: testEntityHistory = { +// current: event1, +// previous: None, +// entityData: Set(Entity.mockEntity1), +// } + +// let historyRow2: testEntityHistory = { +// current: event2, +// previous: Some(event1), +// entityData: Set(Entity.mockEntity2), +// } + +// let historyRow3: testEntityHistory = { +// current: event3, +// previous: Some(event2), +// entityData: Set(Entity.mockEntity3), +// } + +// let historyRows = [historyRow1, historyRow2, historyRow3] + +// //Shows a case where no event exists on this block +// let rollbackEventIdentifier: Types.eventIdentifier = { +// blockTimestamp: 10, +// chainId: chain_id, +// blockNumber: 3, +// logIndex: 0, +// } + +// let orderedMultichainArg = DbFunctions.EntityHistory.Args.OrderedMultichain({ +// safeBlockTimestamp: rollbackEventIdentifier.blockTimestamp, +// reorgChainId: chain_id, +// safeBlockNumber: rollbackEventIdentifier.blockNumber, +// }) + +// let unorderedMultichainArg = DbFunctions.EntityHistory.Args.UnorderedMultichain({ +// reorgChainId: chain_id, +// safeBlockNumber: rollbackEventIdentifier.blockNumber, +// }) +// } + +// module Chain2 = { +// let chain_id = 2 + +// let event1: EntityHistory.historyFields = { +// chain_id, +// block_timestamp: 3, +// block_number: 1, +// log_index: 0, +// } + +// let event2: EntityHistory.historyFields = { +// chain_id, +// block_timestamp: 8, +// block_number: 2, +// log_index: 1, +// } + +// let event3: EntityHistory.historyFields = { +// chain_id, +// block_timestamp: 13, +// block_number: 3, +// log_index: 2, +// } + +// let historyRow1: testEntityHistory = { +// current: event1, +// previous: None, +// entityData: Set(Entity.mockEntity5), +// } + +// let historyRow2: testEntityHistory = { +// current: event2, +// previous: Some(event1), +// entityData: Delete({id: Entity.entityId2}), +// } +// let historyRow3: testEntityHistory = { +// current: event3, +// previous: Some(event2), +// entityData: Set(Entity.mockEntity6), +// } + +// let historyRows = [historyRow1, historyRow2, historyRow3] +// } + +// let historyRows = Utils.Array.mergeSorted( +// (a, b) => a.EntityHistory.current.block_timestamp < b.current.block_timestamp, +// Chain1.historyRows, +// Chain2.historyRows, +// ) +// } + +// describe("Entity history rollbacks", () => { +// Async.beforeEach(async () => { +// try { +// let _ = DbHelpers.resetPostgresClient() +// let storage = PgStorage.make( +// ~sql=Db.sql, +// ~pgSchema=Env.Db.publicSchema, +// ~pgUser=Env.Db.user, +// ~pgDatabase=Env.Db.database, +// ~pgPassword=Env.Db.password, +// ~pgHost=Env.Db.host, +// ~pgPort=Env.Db.port, +// ) +// let _ = await storage.initialize( +// ~chainConfigs=[], +// ~entities=[module(TestEntity)->Entities.entityModToInternal], +// ~enums=[Persistence.entityHistoryActionEnumConfig->Internal.fromGenericEnumConfig], +// ) + +// let _ = +// await Db.sql->Postgres.unsafe( +// TestEntity.entityHistory.makeInsertFnQuery(~pgSchema="public"), +// ) + +// try await Db.sql->PgStorage.setOrThrow( +// ~items=[ +// Mocks.Entity.mockEntity1->TestEntity.castToInternal, +// Mocks.Entity.mockEntity5->TestEntity.castToInternal, +// ], +// ~table=TestEntity.table, +// ~itemSchema=TestEntity.schema, +// ~pgSchema=Config.storagePgSchema, +// ) catch { +// | exn => +// Js.log2("batchSet mock entity exn", exn) +// Assert.fail("Failed to set mock entity in table") +// } + +// try await Db.sql->Postgres.beginSql( +// sql => +// sql +// ->PgStorage.setEntityHistoryOrThrow( +// ~entityHistory=TestEntity.entityHistory, +// ~rows=Mocks.GnosisBug.historyRows, +// ) +// ->Promise.all +// ->Promise.ignoreValue, +// ) catch { +// | exn => +// Js.log2("insert mock rows exn", exn) +// Assert.fail("Failed to insert mock rows") +// } + +// let historyItems = { +// let items = await Db.sql->getAllMockEntityHistory +// items->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) +// } +// Assert.equal(historyItems->Js.Array2.length, 4, ~message="Should have 4 history items") +// Assert.ok( +// historyItems->Belt.Array.some(item => item.current.chain_id == 0), +// ~message="Should contain 2 copied items", +// ) +// } catch { +// | exn => +// Js.log2(" Entity history setup exn", exn) +// Assert.fail("Failed setting up tables") +// } +// }) + +// Async.it("Rollback ignores copied entities as an item in reorg threshold", async () => { +// let rollbackDiff = await Db.sql->DbFunctions.EntityHistory.getRollbackDiff( +// OrderedMultichain({ +// reorgChainId: Mocks.GnosisBug.chain_id, +// safeBlockNumber: 9, +// safeBlockTimestamp: 9 * 5, +// }), +// ~entityConfig=module(TestEntity)->Entities.entityModToInternal, +// ) + +// let expectedDiff: array> = [ +// { +// current: {chain_id: 0, block_timestamp: 0, block_number: 0, log_index: 0}, +// previous: %raw(`undefined`), +// entityData: Set(Mocks.Entity.mockEntity1->TestEntity.castToInternal), +// }, +// { +// current: {chain_id: 0, block_timestamp: 0, block_number: 0, log_index: 0}, +// previous: %raw(`undefined`), +// entityData: Set(Mocks.Entity.mockEntity5->TestEntity.castToInternal), +// }, +// ] + +// Assert.deepStrictEqual( +// rollbackDiff, +// expectedDiff, +// ~message="Should rollback to the copied entity", +// ) +// }) + +// Async.it( +// "Deleting items after reorg event should not remove the copied history item", +// async () => { +// await Db.sql->DbFunctions.EntityHistory.deleteAllEntityHistoryAfterEventIdentifier( +// ~isUnorderedMultichainMode=false, +// ~eventIdentifier={ +// chainId: Mocks.GnosisBug.chain_id, +// blockTimestamp: 9 * 5, +// blockNumber: 9, +// logIndex: 0, +// }, +// ~allEntities=[module(TestEntity)->Entities.entityModToInternal], +// ) + +// let historyItems = { +// let items = await Db.sql->getAllMockEntityHistory +// items->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) +// } + +// Assert.equal(historyItems->Js.Array2.length, 2, ~message="Should have the 2 copied items") + +// let allItemsAreZeroChainId = +// historyItems->Belt.Array.every(item => item.current.chain_id == 0) + +// Assert.ok( +// allItemsAreZeroChainId, +// ~message="Should have all items in the zero chain id since they are copied", +// ) +// }, +// ) + +// Async.it("Prunes history correctly with items in reorg threshold", async () => { +// // set the current entity of id 3 +// await Db.sql->PgStorage.setOrThrow( +// ~items=[Mocks.Entity.mockEntity7->TestEntity.castToInternal], +// ~table=TestEntity.table, +// ~itemSchema=TestEntity.schema, +// ~pgSchema=Config.storagePgSchema, +// ) + +// // set an updated version of its row to get a copied entity history +// try await Db.sql->Postgres.beginSql( +// sql => +// sql +// ->PgStorage.setEntityHistoryOrThrow( +// ~entityHistory=TestEntity.entityHistory, +// ~rows=Mocks.GnosisBug.historyRowsForPrune, +// ) +// ->Promise.all +// ->Promise.ignoreValue, +// ) catch { +// | exn => +// Js.log2("insert mock rows exn", exn) +// Assert.fail("Failed to insert mock rows") +// } + +// // let historyItemsBefore = { +// // let items = await Db.sql->getAllMockEntityHistory +// // Js.log2("history items before prune", items) +// // items->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) +// // } + +// let () = await Db.sql->EntityHistory.pruneStaleEntityHistory( +// ~entityName=(module(TestEntity)->Entities.entityModToInternal).name, +// ~pgSchema=Env.Db.publicSchema, +// ~safeReorgBlocks={ +// chainIds: [Mocks.GnosisBug.chain_id], +// blockNumbers: [11], +// }, +// ) + +// let historyItemsAfter = { +// let items = await Db.sql->getAllMockEntityHistory +// // Js.log2("history items after prune", items) +// items->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) +// } + +// Assert.equal( +// historyItemsAfter->Js.Array2.length, +// 4, +// ~message="Should have 4 history items for entity id 1 and 3 before and after block 11", +// ) +// }) +// }) + +// describe("Entity history rollbacks", () => { +// Async.beforeEach(async () => { +// try { +// let _ = DbHelpers.resetPostgresClient() +// let storage = PgStorage.make( +// ~sql=Db.sql, +// ~pgSchema=Env.Db.publicSchema, +// ~pgUser=Env.Db.user, +// ~pgDatabase=Env.Db.database, +// ~pgPassword=Env.Db.password, +// ~pgHost=Env.Db.host, +// ~pgPort=Env.Db.port, +// ) +// let _ = await storage.initialize( +// ~chainConfigs=[], +// ~entities=[module(TestEntity)->Entities.entityModToInternal], +// ~enums=[Persistence.entityHistoryActionEnumConfig->Internal.fromGenericEnumConfig], +// ) + +// let _ = +// await Db.sql->Postgres.unsafe( +// TestEntity.entityHistory.makeInsertFnQuery(~pgSchema="public"), +// ) + +// try await Db.sql->Postgres.beginSql( +// sql => +// sql +// ->PgStorage.setEntityHistoryOrThrow( +// ~entityHistory=TestEntity.entityHistory, +// ~rows=Mocks.historyRows, +// ) +// ->Promise.all +// ->Promise.ignoreValue, +// ) catch { +// | exn => +// Js.log2("insert mock rows exn", exn) +// Assert.fail("Failed to insert mock rows") +// } + +// let historyItems = { +// let items = await Db.sql->getAllMockEntityHistory +// items->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) +// } +// Assert.equal(historyItems->Js.Array2.length, 6, ~message="Should have 6 history items") +// Assert.ok( +// !(historyItems->Belt.Array.some(item => item.current.chain_id == 0)), +// ~message="No defaulted/copied values should exist in history", +// ) +// } catch { +// | exn => +// Js.log2(" Entity history setup exn", exn) +// Assert.fail("Failed setting up tables") +// } +// }) + +// Async.it("Returns expected diff for ordered multichain mode", async () => { +// let orderdMultichainRollbackDiff = try await Db.sql->DbFunctions.EntityHistory.getRollbackDiff( +// Mocks.Chain1.orderedMultichainArg, +// ~entityConfig=module(TestEntity)->Entities.entityModToInternal, +// ) catch { +// | exn => +// Js.log2("getRollbackDiff exn", exn) +// Assert.fail("Failed to get rollback diff") +// } + +// switch orderdMultichainRollbackDiff { +// | [ +// {current: currentA, entityData: Set(entitySetA)}, +// {current: currentB, entityData: Delete({id: entityDeleteB})}, +// ] => +// Assert.deepEqual( +// currentA, +// Mocks.Chain1.event2, +// ~message="First history item should haved diffed to event2", +// ) +// Assert.deepEqual( +// entitySetA, +// Mocks.Entity.mockEntity2->TestEntity.castToInternal, +// ~message="First history item should haved diffed to mockEntity2", +// ) +// Assert.deepEqual( +// currentB, +// Mocks.Chain2.event2, +// ~message="Second history item should haved diffed to event3", +// ) +// Assert.deepEqual( +// entityDeleteB, +// Mocks.Entity.entityId2, +// ~message="Second history item should haved diffed a delete of entityId2", +// ) +// | _ => Assert.fail("Should have a set and delete history item in diff") +// } +// }) + +// Async.it("Returns expected diff for unordered multichain mode", async () => { +// let unorderedMultichainRollbackDiff = try await Db.sql->DbFunctions.EntityHistory.getRollbackDiff( +// Mocks.Chain1.unorderedMultichainArg, +// ~entityConfig=module(TestEntity)->Entities.entityModToInternal, +// ) catch { +// | exn => +// Js.log2("getRollbackDiff exn", exn) +// Assert.fail("Failed to get rollback diff") +// } + +// switch unorderedMultichainRollbackDiff { +// | [{current: currentA, entityData: Set(entitySetA)}] => +// Assert.deepEqual( +// currentA, +// Mocks.Chain1.event2, +// ~message="First history item should haved diffed to event2", +// ) +// Assert.deepEqual( +// entitySetA, +// Mocks.Entity.mockEntity2->TestEntity.castToInternal, +// ~message="First history item should haved diffed to mockEntity2", +// ) +// | _ => Assert.fail("Should have only chain 1 item in diff") +// } +// }) + +// Async.it("Deletes current history after rollback ordered", async () => { +// let _ = +// await Db.sql->DbFunctions.EntityHistory.deleteAllEntityHistoryAfterEventIdentifier( +// ~isUnorderedMultichainMode=false, +// ~eventIdentifier=Mocks.Chain1.rollbackEventIdentifier, +// ~allEntities=[module(TestEntity)->Entities.entityModToInternal], +// ) + +// let currentHistoryItems = await Db.sql->getAllMockEntityHistory +// let parsedHistoryItems = +// currentHistoryItems->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) + +// let expectedHistoryItems = Mocks.historyRows->Belt.Array.slice(~offset=0, ~len=4) + +// Assert.deepEqual( +// parsedHistoryItems->stripUndefinedFieldsInPlace, +// expectedHistoryItems->stripUndefinedFieldsInPlace, +// ~message="Should have deleted last 2 items in history", +// ) +// }) + +// Async.it("Deletes current history after rollback unordered", async () => { +// let _ = +// await Db.sql->DbFunctions.EntityHistory.deleteAllEntityHistoryAfterEventIdentifier( +// ~isUnorderedMultichainMode=true, +// ~eventIdentifier=Mocks.Chain1.rollbackEventIdentifier, +// ~allEntities=[module(TestEntity)->Entities.entityModToInternal], +// ) + +// let currentHistoryItems = await Db.sql->getAllMockEntityHistory +// let parsedHistoryItems = +// currentHistoryItems->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) + +// let expectedHistoryItems = Mocks.historyRows->Belt.Array.slice(~offset=0, ~len=5) + +// Assert.deepEqual( +// parsedHistoryItems->stripUndefinedFieldsInPlace, +// expectedHistoryItems->stripUndefinedFieldsInPlace, +// ~message="Should have deleted just the last item in history", +// ) +// }) + +// Async.it("Prunes history correctly with items in reorg threshold", async () => { +// let () = await Db.sql->EntityHistory.pruneStaleEntityHistory( +// ~entityName=(module(TestEntity)->Entities.entityModToInternal).name, +// ~pgSchema=Env.Db.publicSchema, +// ~safeReorgBlocks={ +// chainIds: [1, 2], +// blockNumbers: [3, 2], +// }, +// ) +// let currentHistoryItems = await Db.sql->getAllMockEntityHistory + +// let parsedHistoryItems = +// currentHistoryItems->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) + +// let expectedHistoryItems = [ +// Mocks.Chain1.historyRow2, +// Mocks.Chain1.historyRow3, +// Mocks.Chain2.historyRow2, +// Mocks.Chain2.historyRow3, +// ] + +// let sort = arr => +// arr->Js.Array2.sortInPlaceWith( +// (a, b) => a.EntityHistory.current.block_number - b.current.block_number, +// ) + +// Assert.deepEqual( +// parsedHistoryItems->sort->stripUndefinedFieldsInPlace, +// expectedHistoryItems->sort->stripUndefinedFieldsInPlace, +// ~message="Should have deleted the unneeded first items in history", +// ) +// }) + +// Async.it("Prunes history correctly with items in reorg threshold", async () => { +// let () = await Db.sql->EntityHistory.pruneStaleEntityHistory( +// ~entityName=(module(TestEntity)->Entities.entityModToInternal).name, +// ~pgSchema=Env.Db.publicSchema, +// ~safeReorgBlocks={ +// chainIds: [1, 2], +// blockNumbers: [3, 2], +// }, +// ) +// let currentHistoryItems = await Db.sql->getAllMockEntityHistory + +// let parsedHistoryItems = +// currentHistoryItems->S.parseJsonOrThrow(TestEntity.entityHistory.schemaRows) + +// let sort = arr => +// arr->Js.Array2.sortInPlaceWith( +// (a, b) => a.EntityHistory.current.block_number - b.current.block_number, +// ) + +// Assert.deepEqual( +// parsedHistoryItems->sort->stripUndefinedFieldsInPlace, +// [ +// Mocks.Chain1.historyRow2, +// Mocks.Chain2.historyRow2, +// Mocks.Chain2.historyRow3, +// Mocks.Chain1.historyRow3, +// ]->stripUndefinedFieldsInPlace, +// ~message="Should have deleted the unneeded first items in history", +// ) +// }) + +// Async.it("Prunes history correctly with no items in reorg threshold", async () => { +// let () = await Db.sql->EntityHistory.pruneStaleEntityHistory( +// ~entityName=(module(TestEntity)->Entities.entityModToInternal).name, +// ~pgSchema=Env.Db.publicSchema, +// ~safeReorgBlocks={ +// chainIds: [1, 2], +// blockNumbers: [4, 3], +// }, +// ) +// let currentHistoryItems = await Db.sql->getAllMockEntityHistory + +// Assert.ok( +// currentHistoryItems->Array.length == 0, +// ~message="Should have deleted all items in history", +// ) +// }) +// }) + +// describe_skip("Prune performance test", () => { +// Async.it("Print benchmark of prune function", async () => { +// let _ = DbHelpers.resetPostgresClient() +// let storage = PgStorage.make( +// ~sql=Db.sql, +// ~pgSchema=Env.Db.publicSchema, +// ~pgUser=Env.Db.user, +// ~pgDatabase=Env.Db.database, +// ~pgPassword=Env.Db.password, +// ~pgHost=Env.Db.host, +// ~pgPort=Env.Db.port, +// ) +// let _ = await storage.initialize( +// ~entities=[module(TestEntity)->Entities.entityModToInternal], +// ~chainConfigs=[], +// ~enums=[Persistence.entityHistoryActionEnumConfig->Internal.fromGenericEnumConfig], +// ) + +// let _ = +// await Db.sql->Postgres.unsafe(TestEntity.entityHistory.makeInsertFnQuery(~pgSchema="public")) + +// let rows: array = [] +// for i in 0 to 1000 { +// let mockEntity: TestEntity.t = { +// id: i->mod(10)->Belt.Int.toString, +// fieldA: i, +// fieldB: None, +// } + +// let historyRow: testEntityHistory = { +// current: { +// chain_id: 1, +// block_timestamp: i * 5, +// block_number: i, +// log_index: 0, +// }, +// previous: None, +// entityData: Set(mockEntity), +// } +// rows->Js.Array2.push(historyRow)->ignore +// } + +// try await Db.sql->Postgres.beginSql( +// sql => +// sql +// ->PgStorage.setEntityHistoryOrThrow(~entityHistory=TestEntity.entityHistory, ~rows) +// ->Promise.all +// ->Promise.ignoreValue, +// ) catch { +// | exn => +// Js.log2("insert mock rows exn", exn) +// Assert.fail("Failed to insert mock rows") +// } + +// let startTime = Hrtime.makeTimer() + +// try { +// let () = await Db.sql->EntityHistory.pruneStaleEntityHistory( +// ~entityName=(module(TestEntity)->Entities.entityModToInternal).name, +// ~pgSchema=Env.Db.publicSchema, +// ~safeReorgBlocks={ +// chainIds: [1], +// blockNumbers: [500], +// }, +// ) +// } catch { +// | exn => +// Js.log2("prune stale entity history exn", exn) +// Assert.fail("Failed to prune stale entity history") +// } + +// let elapsedTime = Hrtime.timeSince(startTime)->Hrtime.toMillis->Hrtime.intFromMillis +// Js.log2("Elapsed time", elapsedTime) +// }) +// }) + diff --git a/scenarios/test_codegen/test/lib_tests/EventRouter_test.res b/scenarios/test_codegen/test/lib_tests/EventRouter_test.res index a073def7f..998ac2c4a 100644 --- a/scenarios/test_codegen/test/lib_tests/EventRouter_test.res +++ b/scenarios/test_codegen/test/lib_tests/EventRouter_test.res @@ -155,10 +155,10 @@ describe("EventRouter", () => { indexingContracts->Js.Dict.set( nonWildcardContractAddress->Address.toString, { - FetchState.startBlock: 0, + Internal.startBlock: 0, contractName: nonWildcardContractName, address: nonWildcardContractAddress, - register: Config, + registrationBlock: None, }, ) diff --git a/scenarios/test_codegen/test/lib_tests/FetchState_onBlock_test.res b/scenarios/test_codegen/test/lib_tests/FetchState_onBlock_test.res index ff596a289..d8aa7f139 100644 --- a/scenarios/test_codegen/test/lib_tests/FetchState_onBlock_test.res +++ b/scenarios/test_codegen/test/lib_tests/FetchState_onBlock_test.res @@ -38,10 +38,10 @@ let makeInitialWithOnBlock = (~startBlock=0, ~onBlockConfigs) => { ~eventConfigs=[baseEventConfig], ~contracts=[ { - FetchState.address: mockAddress0, + Internal.address: mockAddress0, contractName: "Gravatar", startBlock, - register: Config, + registrationBlock: None, }, ], ~startBlock, diff --git a/scenarios/test_codegen/test/lib_tests/FetchState_test.res b/scenarios/test_codegen/test/lib_tests/FetchState_test.res index 000d2da0e..b7e29f939 100644 --- a/scenarios/test_codegen/test/lib_tests/FetchState_test.res +++ b/scenarios/test_codegen/test/lib_tests/FetchState_test.res @@ -45,32 +45,22 @@ let getBlockData = (~blockNumber): FetchState.blockNumberAndTimestamp => { let makeDynContractRegistration = ( ~contractAddress, ~blockNumber, - ~logIndex=0, ~contractType=Gravatar, - ~registeringEventContractName="MockGravatarFactory", - ~registeringEventName="MockCreateGravatar", - ~registeringEventSrcAddress=mockFactoryAddress, -): FetchState.indexingContract => { +): Internal.indexingContract => { { address: contractAddress, contractName: (contractType :> string), startBlock: blockNumber, - register: DC({ - registeringEventLogIndex: logIndex, - registeringEventBlockTimestamp: getTimestamp(~blockNumber), - registeringEventContractName, - registeringEventName, - registeringEventSrcAddress, - }), + registrationBlock: Some(blockNumber), } } -let makeConfigContract = (contractName, address): FetchState.indexingContract => { +let makeConfigContract = (contractName, address): Internal.indexingContract => { { address, contractName, startBlock: 0, - register: Config, + registrationBlock: None, } } @@ -83,6 +73,12 @@ let mockEvent = (~blockNumber, ~logIndex=0, ~chainId=1): Internal.item => Intern event: Utils.magic("Mock event in fetchstate test"), }) +let dcToItem = (dc: Internal.indexingContract) => { + let item = mockEvent(~blockNumber=dc.startBlock) + item->Internal.setItemDcs([dc]) + item +} + let baseEventConfig = (Mock.evmEventConfig( ~id="0", ~contractName="Gravatar", @@ -103,10 +99,10 @@ let makeInitial = ( ~eventConfigs=[baseEventConfig, baseEventConfig2], ~contracts=[ { - FetchState.address: mockAddress0, + Internal.address: mockAddress0, contractName: "Gravatar", startBlock, - register: Config, + registrationBlock: None, }, ], ~startBlock, @@ -120,7 +116,7 @@ let makeInitial = ( // Helper to build indexingContracts dict for test expectations // Note: dynamic contract info is now only tracked by the register field (DC variant) -let makeIndexingContractsWithDynamics = (dcs: array, ~static=[]) => { +let makeIndexingContractsWithDynamics = (dcs: array, ~static=[]) => { let dict = Js.Dict.empty() dcs->Array.forEach(dc => { dict->Js.Dict.set(dc.address->Address.toString, dc) @@ -132,7 +128,7 @@ let makeIndexingContractsWithDynamics = (dcs: array address, contractName: (Gravatar :> string), startBlock: 0, - register: Config, + registrationBlock: None, }, ) }) @@ -177,7 +173,6 @@ describe("FetchState.make", () => { chainId: 0, indexingContracts: fetchState.indexingContracts, contractConfigs: fetchState.contractConfigs, - dcsToStore: [], blockLag: 0, onBlockConfigs: [], }, @@ -250,7 +245,6 @@ describe("FetchState.make", () => { chainId, indexingContracts: fetchState.indexingContracts, contractConfigs: fetchState.contractConfigs, - dcsToStore: [], blockLag: 0, onBlockConfigs: [], }, @@ -316,7 +310,6 @@ describe("FetchState.make", () => { chainId, indexingContracts: fetchState.indexingContracts, contractConfigs: fetchState.contractConfigs, - dcsToStore: [], blockLag: 0, onBlockConfigs: [], }, @@ -414,7 +407,6 @@ describe("FetchState.make", () => { chainId, indexingContracts: fetchState.indexingContracts, contractConfigs: fetchState.contractConfigs, - dcsToStore: [], blockLag: 0, onBlockConfigs: [], }, @@ -440,7 +432,7 @@ describe("FetchState.registerDynamicContracts", () => { Assert.equal( fetchState->FetchState.registerDynamicContracts([ - makeDynContractRegistration(~blockNumber=0, ~contractAddress=mockAddress0), + makeDynContractRegistration(~blockNumber=0, ~contractAddress=mockAddress0)->dcToItem, ]), fetchState, ~message="Should return fetchState without updating it", @@ -454,7 +446,7 @@ describe("FetchState.registerDynamicContracts", () => { let dc1 = makeDynContractRegistration(~blockNumber=2, ~contractAddress=mockAddress1) - let fetchStateWithDc1 = fetchState->FetchState.registerDynamicContracts([dc1]) + let fetchStateWithDc1 = fetchState->FetchState.registerDynamicContracts([dc1->dcToItem]) Assert.deepEqual( (fetchState.partitions->Array.length, fetchStateWithDc1.partitions->Array.length), @@ -463,14 +455,14 @@ describe("FetchState.registerDynamicContracts", () => { ) Assert.equal( - fetchStateWithDc1->FetchState.registerDynamicContracts([dc1]), + fetchStateWithDc1->FetchState.registerDynamicContracts([dc1->dcToItem]), fetchStateWithDc1, ~message="Calling it with the same dc for the second time shouldn't change anything", ) Assert.equal( fetchStateWithDc1->FetchState.registerDynamicContracts([ - makeDynContractRegistration(~blockNumber=0, ~contractAddress=mockAddress1), + makeDynContractRegistration(~blockNumber=0, ~contractAddress=mockAddress1)->dcToItem, ]), fetchStateWithDc1, ~message=`BROKEN: Calling it with the same dc @@ -490,7 +482,13 @@ describe("FetchState.registerDynamicContracts", () => { let dc3 = makeDynContractRegistration(~blockNumber=2, ~contractAddress=mockAddress3) let dc4 = makeDynContractRegistration(~blockNumber=2, ~contractAddress=mockAddress4) - let updatedFetchState = fetchState->FetchState.registerDynamicContracts([dc1, dc2, dc3, dc4]) + let updatedFetchState = + fetchState->FetchState.registerDynamicContracts([ + dc1->dcToItem, + dc2->dcToItem, + dc3->dcToItem, + dc4->dcToItem, + ]) Assert.deepEqual( updatedFetchState.partitions, @@ -533,10 +531,10 @@ describe("FetchState.registerDynamicContracts", () => { ) let updatedFetchState = fetchState->FetchState.registerDynamicContracts([ - dc1FromAnotherContract, - dc2, - dc3, - dc4FromAnotherContract, + dc1FromAnotherContract->dcToItem, + dc2->dcToItem, + dc3->dcToItem, + dc4FromAnotherContract->dcToItem, ]) Assert.deepEqual( @@ -634,7 +632,13 @@ describe("FetchState.registerDynamicContracts", () => { ) let updatedFetchState = - fetchState->FetchState.registerDynamicContracts([dc1, dc2, dc3, dc4, dc5]) + fetchState->FetchState.registerDynamicContracts([ + dc1->dcToItem, + dc2->dcToItem, + dc3->dcToItem, + dc4->dcToItem, + dc5->dcToItem, + ]) Assert.deepEqual( updatedFetchState.partitions, @@ -715,13 +719,16 @@ describe("FetchState.registerDynamicContracts", () => { let dc1 = makeDynContractRegistration(~blockNumber=20, ~contractAddress=mockAddress1) let dc2 = makeDynContractRegistration(~blockNumber=10, ~contractAddress=mockAddress1) + let dcItem1 = dc1->dcToItem + let dcItem2 = dc2->dcToItem - let updatedFetchState = fetchState->FetchState.registerDynamicContracts([dc1, dc2]) + let updatedFetchState = fetchState->FetchState.registerDynamicContracts([dcItem2, dcItem1]) Assert.deepEqual( - updatedFetchState.dcsToStore, - [dc2], - ~message="Should choose the earliest dc from the batch", + (dcItem1->Internal.getItemDcs, dcItem2->Internal.getItemDcs), + (Some([]), Some([dc2])), + ~message=`Should choose the earliest dc from the batch +End remove the dc from the later one, so they are not duplicated in the db`, ) Assert.deepEqual( updatedFetchState.indexingContracts, @@ -764,13 +771,12 @@ describe("FetchState.registerDynamicContracts", () => { let updatedFetchState = fetchState->FetchState.registerDynamicContracts(// Order of dcs doesn't matter // but they are not sorted in fetch state - [dc1, dc3, dc2]) + [dc1->dcToItem, dc3->dcToItem, dc2->dcToItem]) Assert.equal(updatedFetchState.indexingContracts->Utils.Dict.size, 4) Assert.deepEqual( updatedFetchState, { ...fetchState, - dcsToStore: [dc1, dc3, dc2], indexingContracts: updatedFetchState.indexingContracts, nextPartitionIndex: 2, partitions: fetchState.partitions->Array.concat([ @@ -890,7 +896,6 @@ describe("FetchState.registerDynamicContracts", () => { chainId, indexingContracts: fetchState.indexingContracts, contractConfigs: fetchState.contractConfigs, - dcsToStore: [], blockLag: 0, onBlockConfigs: [], }, @@ -931,7 +936,6 @@ describe("FetchState.getNextQuery & integration", () => { buffer: [mockEvent(~blockNumber=1), mockEvent(~blockNumber=2)], startBlock: 0, endBlock: None, - dcsToStore: [], blockLag: 0, normalSelection, chainId, @@ -939,10 +943,10 @@ describe("FetchState.getNextQuery & integration", () => { ( mockAddress0->Address.toString, { - FetchState.contractName: (Gravatar :> string), + Internal.contractName: (Gravatar :> string), startBlock: 0, address: mockAddress0, - register: Config, + registrationBlock: None, }, ), ]), @@ -954,7 +958,6 @@ describe("FetchState.getNextQuery & integration", () => { let makeIntermidiateDcMerge = (): FetchState.t => { let normalSelection = makeInitial().normalSelection { - dcsToStore: [dc2, dc1, dc3], partitions: [ { id: "0", @@ -1231,14 +1234,13 @@ describe("FetchState.getNextQuery & integration", () => { let fetchStateWithDcs = fetchState - ->FetchState.registerDynamicContracts([dc2, dc1]) - ->FetchState.registerDynamicContracts([dc3]) + ->FetchState.registerDynamicContracts([dc2->dcToItem, dc1->dcToItem]) + ->FetchState.registerDynamicContracts([dc3->dcToItem]) Assert.deepEqual( fetchStateWithDcs, { ...fetchState, - dcsToStore: [dc2, dc1, dc3], indexingContracts: makeIndexingContractsWithDynamics( [dc2, dc1, dc3], ~static=[mockAddress0], @@ -1626,7 +1628,7 @@ describe("FetchState.getNextQuery & integration", () => { ~targetBufferSize=10, ~chainId, )->FetchState.registerDynamicContracts([ - makeDynContractRegistration(~blockNumber=2, ~contractAddress=mockAddress2), + makeDynContractRegistration(~blockNumber=2, ~contractAddress=mockAddress2)->dcToItem, ]) Assert.deepEqual(fetchState.partitions->Array.length, 3) @@ -1667,14 +1669,12 @@ describe("FetchState.getNextQuery & integration", () => { it("Correctly rollbacks fetch state", () => { let fetchState = makeIntermidiateDcMerge() - let fetchStateAfterRollback1 = - fetchState->FetchState.rollback(~firstChangeEvent={blockNumber: 2, logIndex: 0}) + let fetchStateAfterRollback1 = fetchState->FetchState.rollback(~targetBlockNumber=1) Assert.deepEqual( fetchStateAfterRollback1, { ...fetchState, - dcsToStore: [dc1], indexingContracts: makeIndexingContractsWithDynamics([dc1], ~static=[mockAddress0]), partitions: [ { @@ -1708,13 +1708,12 @@ describe("FetchState.getNextQuery & integration", () => { // Rollback even more to see the removal of partition "2" let fetchStateAfterRollback2 = - fetchStateAfterRollback1->FetchState.rollback(~firstChangeEvent={blockNumber: 0, logIndex: 0}) + fetchStateAfterRollback1->FetchState.rollback(~targetBlockNumber=-1) Assert.deepEqual( fetchStateAfterRollback2, { ...fetchStateAfterRollback1, - dcsToStore: [], indexingContracts: makeIndexingContractsWithDynamics([], ~static=[mockAddress0]), partitions: [ { @@ -1761,7 +1760,7 @@ describe("FetchState.getNextQuery & integration", () => { ~targetBufferSize=10, ~chainId, )->FetchState.registerDynamicContracts([ - makeDynContractRegistration(~blockNumber=2, ~contractAddress=mockAddress2), + makeDynContractRegistration(~blockNumber=2, ~contractAddress=mockAddress2)->dcToItem, ]) // Additionally test that state being reset @@ -1788,14 +1787,12 @@ describe("FetchState.getNextQuery & integration", () => { ~message=`Should have 2 partitions before rollback`, ) - let fetchStateAfterRollback = - fetchState->FetchState.rollback(~firstChangeEvent={blockNumber: 2, logIndex: 0}) + let fetchStateAfterRollback = fetchState->FetchState.rollback(~targetBlockNumber=1) Assert.deepEqual( fetchStateAfterRollback, { ...fetchState, - dcsToStore: [], indexingContracts: Js.Dict.empty(), partitions: [ { @@ -1882,7 +1879,6 @@ describe("FetchState unit tests for specific cases", () => { updatedFetchState, { ...fetchState, - dcsToStore: [], partitions: [ { id: "0", @@ -2080,7 +2076,7 @@ describe("FetchState unit tests for specific cases", () => { makeDynContractRegistration( ~contractAddress=mockAddress1, ~blockNumber=registeringBlockNumber, - ), + )->dcToItem, ]) Assert.deepEqual( @@ -2182,7 +2178,7 @@ describe("FetchState unit tests for specific cases", () => { Assert.deepEqual( fetchState ->FetchState.registerDynamicContracts([ - makeDynContractRegistration(~contractAddress=mockAddress1, ~blockNumber=2), + makeDynContractRegistration(~contractAddress=mockAddress1, ~blockNumber=2)->dcToItem, ]) ->getEarliestEvent, NoItem({ @@ -2277,7 +2273,7 @@ describe("FetchState unit tests for specific cases", () => { //Dynamic contract A registered at block 100 let dcA = makeDynContractRegistration(~contractAddress=mockAddress2, ~blockNumber=100) - let fetchStateWithDcA = fetchState->FetchState.registerDynamicContracts([dcA]) + let fetchStateWithDcA = fetchState->FetchState.registerDynamicContracts([dcA->dcToItem]) let queryA = switch fetchStateWithDcA->FetchState.getNextQuery( ~concurrencyLimit=10, @@ -2314,7 +2310,7 @@ describe("FetchState unit tests for specific cases", () => { //Next registration happens at block 200, between the first register and the upperbound of it's query let fetchStateWithDcB = fetchStateWithDcA->FetchState.registerDynamicContracts([ - makeDynContractRegistration(~contractAddress=mockAddress3, ~blockNumber=200), + makeDynContractRegistration(~contractAddress=mockAddress3, ~blockNumber=200)->dcToItem, ]) Assert.deepEqual( @@ -2515,10 +2511,10 @@ describe("FetchState.isReadyToEnterReorgThreshold", () => { ~eventConfigs=[baseEventConfig, baseEventConfig2], ~contracts=[ { - FetchState.address: mockAddress0, + Internal.address: mockAddress0, contractName: "Gravatar", startBlock: 6, - register: Config, + registrationBlock: None, }, ], ~startBlock=6, @@ -2537,10 +2533,10 @@ describe("FetchState.isReadyToEnterReorgThreshold", () => { ~eventConfigs=[baseEventConfig, baseEventConfig2], ~contracts=[ { - FetchState.address: mockAddress0, + Internal.address: mockAddress0, contractName: "Gravatar", startBlock: 50, - register: Config, + registrationBlock: None, }, ], ~startBlock=50, @@ -2559,10 +2555,10 @@ describe("FetchState.isReadyToEnterReorgThreshold", () => { ~eventConfigs=[baseEventConfig, baseEventConfig2], ~contracts=[ { - FetchState.address: mockAddress0, + Internal.address: mockAddress0, contractName: "Gravatar", startBlock: 50, - register: Config, + registrationBlock: None, }, ], ~startBlock=50, @@ -2581,10 +2577,10 @@ describe("FetchState.isReadyToEnterReorgThreshold", () => { ~eventConfigs=[baseEventConfig, baseEventConfig2], ~contracts=[ { - FetchState.address: mockAddress0, + Internal.address: mockAddress0, contractName: "Gravatar", startBlock: 51, - register: Config, + registrationBlock: None, }, ], ~startBlock=51, @@ -2603,10 +2599,10 @@ describe("FetchState.isReadyToEnterReorgThreshold", () => { ~eventConfigs=[baseEventConfig, baseEventConfig2], ~contracts=[ { - FetchState.address: mockAddress0, + Internal.address: mockAddress0, contractName: "Gravatar", startBlock: 50, - register: Config, + registrationBlock: None, }, ], ~startBlock=50, @@ -2625,10 +2621,10 @@ describe("FetchState.isReadyToEnterReorgThreshold", () => { ~eventConfigs=[baseEventConfig, baseEventConfig2], ~contracts=[ { - FetchState.address: mockAddress0, + Internal.address: mockAddress0, contractName: "Gravatar", startBlock: 6, - register: Config, + registrationBlock: None, }, ], ~startBlock=6, @@ -2650,10 +2646,10 @@ describe("FetchState.isReadyToEnterReorgThreshold", () => { ~eventConfigs=[baseEventConfig, baseEventConfig2], ~contracts=[ { - FetchState.address: mockAddress0, + Internal.address: mockAddress0, contractName: "Gravatar", startBlock: 6, - register: Config, + registrationBlock: None, }, ], ~startBlock=6, @@ -2679,7 +2675,8 @@ describe("Dynamic contracts with start blocks", () => { ) // Register the contract at block 100 (before its startBlock) - let updatedFetchState = fetchState->FetchState.registerDynamicContracts([dynamicContract]) + let updatedFetchState = + fetchState->FetchState.registerDynamicContracts([dynamicContract->dcToItem]) // The contract should be registered in indexingContracts Assert.ok( @@ -2719,7 +2716,8 @@ describe("Dynamic contracts with start blocks", () => { ~contractType=Gravatar, ) - let updatedFetchState = fetchState->FetchState.registerDynamicContracts([contract1, contract2]) + let updatedFetchState = + fetchState->FetchState.registerDynamicContracts([contract1->dcToItem, contract2->dcToItem]) // Verify both contracts are registered with correct startBlocks let contract1Registered = @@ -2818,7 +2816,8 @@ describe("FetchState buffer overflow prevention", () => { // Create a second partition to ensure buffer limiting logic is exercised across partitions // Register at a later block, so partition "0" remains the earliest and is selected let dc = makeDynContractRegistration(~blockNumber=0, ~contractAddress=mockAddress1) - let fetchStateWithTwoPartitions = fetchState->FetchState.registerDynamicContracts([dc]) + let fetchStateWithTwoPartitions = + fetchState->FetchState.registerDynamicContracts([dc->dcToItem]) // Build up a large queue using public API (handleQueryResult) // queue.length = 15, targetBufferSize = 10 diff --git a/scenarios/test_codegen/test/lib_tests/PgStorage_test.res b/scenarios/test_codegen/test/lib_tests/PgStorage_test.res index ec74b29c3..c673440de 100644 --- a/scenarios/test_codegen/test/lib_tests/PgStorage_test.res +++ b/scenarios/test_codegen/test/lib_tests/PgStorage_test.res @@ -137,7 +137,6 @@ describe("Test PgStorage SQL generation functions", () => { ) let mainQuery = queries->Belt.Array.get(0)->Belt.Option.getExn - let functionsQuery = queries->Belt.Array.get(1)->Belt.Option.getExn let expectedMainQuery = `DROP SCHEMA IF EXISTS "test_schema" CASCADE; CREATE SCHEMA "test_schema"; @@ -149,12 +148,10 @@ CREATE TABLE IF NOT EXISTS "test_schema"."persisted_state"("id" SERIAL NOT NULL, 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")); +CREATE TABLE IF NOT EXISTS "test_schema"."envio_history_A"("b_id" TEXT, "id" TEXT NOT NULL, "optionalStringToTestLinkedEntities" TEXT, "checkpoint_id" INTEGER NOT NULL, "envio_change" "test_schema".ENVIO_HISTORY_CHANGE NOT NULL, PRIMARY KEY("id", "checkpoint_id")); CREATE TABLE IF NOT EXISTS "test_schema"."B"("c_id" TEXT, "id" TEXT NOT NULL, PRIMARY KEY("id")); -CREATE TABLE IF NOT EXISTS "test_schema"."B_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, "c_id" TEXT, "id" TEXT NOT NULL, "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")); +CREATE TABLE IF NOT EXISTS "test_schema"."envio_history_B"("c_id" TEXT, "id" TEXT NOT NULL, "checkpoint_id" INTEGER NOT NULL, "envio_change" "test_schema".ENVIO_HISTORY_CHANGE NOT NULL, PRIMARY KEY("id", "checkpoint_id")); CREATE INDEX IF NOT EXISTS "A_b_id" ON "test_schema"."A"("b_id"); -CREATE INDEX IF NOT EXISTS "A_history_serial" ON "test_schema"."A_history"("serial"); -CREATE INDEX IF NOT EXISTS "B_history_serial" ON "test_schema"."B_history"("serial"); CREATE INDEX IF NOT EXISTS "A_b_id" ON "test_schema"."A"("b_id"); CREATE VIEW "test_schema"."_meta" AS SELECT @@ -193,17 +190,6 @@ VALUES (1, 100, 200, 10, 0, NULL, -1, -1, NULL, 0, false, 0), expectedMainQuery, ~message="Main query should match expected SQL exactly", ) - - // Functions query should contain both A and B history functions - Assert.ok( - functionsQuery->Js.String2.includes(`CREATE OR REPLACE FUNCTION "insert_A_history"`), - ~message="Should contain A history function", - ) - - Assert.ok( - functionsQuery->Js.String2.includes(`CREATE OR REPLACE FUNCTION "insert_B_history"`), - ~message="Should contain B history function", - ) }, ) @@ -270,8 +256,7 @@ CREATE VIEW "test_schema"."chain_metadata" AS Assert.equal( queries->Belt.Array.get(1)->Belt.Option.getExn, - ` -CREATE OR REPLACE FUNCTION get_cache_row_count(table_name text) + `CREATE OR REPLACE FUNCTION get_cache_row_count(table_name text) RETURNS integer AS $$ DECLARE result integer; @@ -316,9 +301,8 @@ CREATE TABLE IF NOT EXISTS "public"."persisted_state"("id" SERIAL NOT NULL, "env 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")); +CREATE TABLE IF NOT EXISTS "public"."envio_history_A"("b_id" TEXT, "id" TEXT NOT NULL, "optionalStringToTestLinkedEntities" TEXT, "checkpoint_id" INTEGER NOT NULL, "envio_change" "public".ENVIO_HISTORY_CHANGE NOT NULL, PRIMARY KEY("id", "checkpoint_id")); CREATE INDEX IF NOT EXISTS "A_b_id" ON "public"."A"("b_id"); -CREATE INDEX IF NOT EXISTS "A_history_serial" ON "public"."A_history"("serial"); CREATE VIEW "public"."_meta" AS SELECT "id" AS "chainId", @@ -355,9 +339,18 @@ CREATE VIEW "public"."chain_metadata" AS ) // Verify functions query contains the A history function - Assert.ok( - functionsQuery->Js.String2.includes(`CREATE OR REPLACE FUNCTION "insert_A_history"`), - ~message="Should contain A history function definition", + Assert.equal( + functionsQuery, + `CREATE OR REPLACE FUNCTION get_cache_row_count(table_name text) +RETURNS integer AS $$ +DECLARE + result integer; +BEGIN + EXECUTE format('SELECT COUNT(*) FROM "public".%I', table_name) INTO result; + RETURN result; +END; +$$ LANGUAGE plpgsql;`, + ~message="Should contain cache row count function definition", ) }, ) @@ -568,7 +561,7 @@ 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;` + AND cp."block_number" >= rc.safe_block;` Assert.equal( query, diff --git a/scenarios/test_codegen/test/lib_tests/SourceManager_test.res b/scenarios/test_codegen/test/lib_tests/SourceManager_test.res index aa781affa..f0be6ac5a 100644 --- a/scenarios/test_codegen/test/lib_tests/SourceManager_test.res +++ b/scenarios/test_codegen/test/lib_tests/SourceManager_test.res @@ -168,10 +168,10 @@ describe("SourceManager fetchNext", () => { indexingContracts->Js.Dict.set( address->Address.toString, { - FetchState.contractName, + Internal.contractName, startBlock: 0, address, - register: Config, + registrationBlock: None, }, ) }, @@ -194,7 +194,6 @@ describe("SourceManager fetchNext", () => { chainId: 0, indexingContracts, contractConfigs: Js.Dict.empty(), - dcsToStore: [], blockLag: 0, onBlockConfigs: [], // All the null values should be computed during updateInternal diff --git a/scenarios/test_codegen/test/rollback/MockChainData.res b/scenarios/test_codegen/test/rollback/MockChainData.res index 0c7fddf83..8c7e16288 100644 --- a/scenarios/test_codegen/test/rollback/MockChainData.res +++ b/scenarios/test_codegen/test/rollback/MockChainData.res @@ -1,9 +1 @@ -module Indexer = { - module ErrorHandling = ErrorHandling - module Types = Types - module Config = Config - module Source = Source - module FetchState = FetchState -} - -include Helpers.ChainMocking.Make(Indexer) +include Helpers.ChainMocking.Make() diff --git a/scenarios/test_codegen/test/rollback/Rollback_test.res b/scenarios/test_codegen/test/rollback/Rollback_test.res index 06d4ca1b3..d9a230e47 100644 --- a/scenarios/test_codegen/test/rollback/Rollback_test.res +++ b/scenarios/test_codegen/test/rollback/Rollback_test.res @@ -403,7 +403,11 @@ describe("Single Chain Simple Rollback", () => { let undefined = (%raw(`undefined`): option<'a>) describe("E2E rollback tests", () => { - let testSingleChainRollback = async (~sourceMock: M.Source.t, ~indexerMock: M.Indexer.t) => { + let testSingleChainRollback = async ( + ~sourceMock: M.Source.t, + ~indexerMock: M.Indexer.t, + ~firstHistoryCheckpointId=2, + ) => { Assert.deepEqual( sourceMock.getItemsOrThrowCalls->Utils.Array.last, Some({ @@ -478,11 +482,28 @@ describe("E2E rollback tests", () => { await indexerMock.getBatchWritePromise() Assert.deepEqual( - await Promise.all2(( + await Promise.all3(( + indexerMock.queryCheckpoints(), indexerMock.query(module(Entities.SimpleEntity)), indexerMock.queryHistory(module(Entities.SimpleEntity)), )), ( + [ + { + id: firstHistoryCheckpointId, + blockHash: Js.Null.empty, + blockNumber: 101, + chainId: 1337, + eventsProcessed: 2, + }, + { + id: firstHistoryCheckpointId + 1, + blockHash: Js.Null.Value("0x102"), + blockNumber: 102, + chainId: 1337, + eventsProcessed: 1, + }, + ], [ { Entities.SimpleEntity.id: "1", @@ -499,58 +520,25 @@ describe("E2E rollback tests", () => { ], [ { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 0, - }, - previous: undefined, - entityData: Set({ + checkpointId: firstHistoryCheckpointId, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "value-2", }), }, { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 0, - }, - previous: undefined, - entityData: Set({ - Entities.SimpleEntity.id: "2", - value: "value-1", - }), - }, - { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 1, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 0, - }), - entityData: Set({ + checkpointId: firstHistoryCheckpointId, + entityId: "2", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "2", value: "value-2", }), }, { - current: { - chain_id: 1337, - block_timestamp: 102, - block_number: 102, - log_index: 0, - }, - previous: undefined, - entityData: Set({ + checkpointId: firstHistoryCheckpointId + 1, + entityId: "3", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "3", value: "value-1", }), @@ -637,11 +625,21 @@ describe("E2E rollback tests", () => { await indexerMock.getBatchWritePromise() Assert.deepEqual( - await Promise.all2(( + await Promise.all3(( + indexerMock.queryCheckpoints(), indexerMock.query(module(Entities.SimpleEntity)), indexerMock.queryHistory(module(Entities.SimpleEntity)), )), ( + [ + { + id: 1, + blockHash: Js.Null.Value("0x101"), + blockNumber: 101, + chainId: 1337, + eventsProcessed: 1, + }, + ], [ { Entities.SimpleEntity.id: "1", @@ -654,27 +652,17 @@ describe("E2E rollback tests", () => { ], [ { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 1, - }, - previous: undefined, - entityData: Set({ + checkpointId: 1, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "value-1", }), }, { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 1, - }, - previous: undefined, - entityData: Set({ + checkpointId: 1, + entityId: "2", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "2", value: "value-2", }), @@ -831,7 +819,11 @@ describe("E2E rollback tests", () => { M.Helper.initialEnterReorgThreshold(~indexerMock, ~sourceMock=sourceMock2), )) - await testSingleChainRollback(~sourceMock=sourceMock1, ~indexerMock) + await testSingleChainRollback( + ~sourceMock=sourceMock1, + ~indexerMock, + ~firstHistoryCheckpointId=3, + ) }, ) @@ -1167,21 +1159,71 @@ This might be wrong after we start exposing a block hash for progress block.`, }, ]) await indexerMock.getBatchWritePromise() - sourceMock1337.resolveGetItemsOrThrow([ - { - blockNumber: 102, - logIndex: 4, - handler, - }, - ]) + sourceMock1337.resolveGetItemsOrThrow( + [ + { + blockNumber: 103, + logIndex: 4, + handler, + }, + ], + ~latestFetchedBlockNumber=105, + ) await indexerMock.getBatchWritePromise() Assert.deepEqual( - await Promise.all2(( + await Promise.all3(( + indexerMock.queryCheckpoints(), indexerMock.query(module(Entities.SimpleEntity)), indexerMock.queryHistory(module(Entities.SimpleEntity)), )), ( + [ + { + id: 3, + eventsProcessed: 1, + chainId: 100, + blockNumber: 101, + blockHash: Js.Null.Value("0x101"), + }, + { + id: 4, + eventsProcessed: 2, + chainId: 1337, + blockNumber: 101, + blockHash: Js.Null.Value("0x101"), + }, + { + id: 5, + eventsProcessed: 1, + chainId: 1337, + blockNumber: 102, + blockHash: Js.Null.Value("0x102"), + }, + { + id: 6, + eventsProcessed: 1, + chainId: 100, + blockNumber: 102, + blockHash: Js.Null.Value("0x102"), + }, + { + id: 7, + eventsProcessed: 1, + chainId: 1337, + blockNumber: 103, + blockHash: Js.Null.Null, + }, + // Block 104 is skipped, since we don't have + // ether events processed or block hash for it + { + id: 8, + eventsProcessed: 0, + chainId: 1337, + blockNumber: 105, + blockHash: Js.Null.Value("0x105"), + }, + ], [ { Entities.SimpleEntity.id: "1", @@ -1190,114 +1232,75 @@ This might be wrong after we start exposing a block hash for progress block.`, ], [ { - current: { - chain_id: 100, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }, - previous: undefined, - entityData: Set({ + checkpointId: 3, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-0", }), }, { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 1, - }, - previous: Some({ - chain_id: 100, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }), - entityData: Set({ - Entities.SimpleEntity.id: "1", - value: "call-1", - }), - }, - { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 1, - }), - entityData: Set({ + checkpointId: 4, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-2", }), }, { - current: { - chain_id: 1337, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }), - entityData: Set({ + checkpointId: 5, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-3", }), }, { - current: { - chain_id: 100, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }), - entityData: Set({ + checkpointId: 6, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-4", }), }, { - current: { - chain_id: 1337, - block_timestamp: 102, - block_number: 102, - log_index: 4, - }, - // FIXME: This looks wrong - previous: Some({ - chain_id: 1337, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }), - entityData: Set({ + checkpointId: 7, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-5", }), }, ], ), - ~message=`Should create multiple history rows: -Sorted for the batch for block number 101 -Different batches for block number 102`, + ~message=`Should create history rows and checkpoints`, + ) + + Assert.deepEqual( + await indexerMock.metric("envio_progress_events_count"), + [ + {value: "2", labels: Js.Dict.fromArray([("chainId", "100")])}, + {value: "4", labels: Js.Dict.fromArray([("chainId", "1337")])}, + ], + ~message="Events count before rollback", + ) + Assert.deepEqual( + await indexerMock.metric("envio_progress_block_number"), + [ + {value: "102", labels: Js.Dict.fromArray([("chainId", "100")])}, + {value: "105", labels: Js.Dict.fromArray([("chainId", "1337")])}, + ], + ~message="Progress block number before rollback", + ) + Assert.deepEqual( + await indexerMock.metric("envio_rollback_events_count"), + [{value: "0", labels: Js.Dict.empty()}], + ~message="Rollbacked events count before rollback", + ) + Assert.deepEqual( + await indexerMock.metric("envio_rollback_count"), + [{value: "0", labels: Js.Dict.empty()}], + ~message="Rollbacks count before rollback", ) // Should trigger rollback @@ -1313,7 +1316,7 @@ Different batches for block number 102`, Assert.deepEqual( sourceMock1337.getBlockHashesCalls, - [[100, 101, 102, 103]], + [[100, 101, 102, 105]], ~message="Should have called getBlockHashes to find rollback depth", ) sourceMock1337.resolveGetBlockHashes([ @@ -1321,11 +1324,38 @@ Different batches for block number 102`, {blockNumber: 100, blockHash: "0x100", blockTimestamp: 100}, {blockNumber: 101, blockHash: "0x101", blockTimestamp: 101}, {blockNumber: 102, blockHash: "0x102-reorged", blockTimestamp: 102}, - {blockNumber: 103, blockHash: "0x103-reorged", blockTimestamp: 103}, + {blockNumber: 105, blockHash: "0x105-reorged", blockTimestamp: 105}, ]) await indexerMock.getRollbackReadyPromise() + Assert.deepEqual( + await indexerMock.metric("envio_progress_events_count"), + [ + {value: "1", labels: Js.Dict.fromArray([("chainId", "100")])}, + {value: "2", labels: Js.Dict.fromArray([("chainId", "1337")])}, + ], + ~message="Events count after rollback", + ) + Assert.deepEqual( + await indexerMock.metric("envio_progress_block_number"), + [ + {value: "101", labels: Js.Dict.fromArray([("chainId", "100")])}, + {value: "101", labels: Js.Dict.fromArray([("chainId", "1337")])}, + ], + ~message="Progress block number after rollback", + ) + Assert.deepEqual( + await indexerMock.metric("envio_rollback_events_count"), + [{value: "3", labels: Js.Dict.empty()}], + ~message="Rollbacked events count after rollback", + ) + Assert.deepEqual( + await indexerMock.metric("envio_rollback_count"), + [{value: "1", labels: Js.Dict.empty()}], + ~message="Rollbacks count after rollback", + ) + Assert.deepEqual( ( sourceMock1337.getItemsOrThrowCalls->Utils.Array.last, @@ -1373,11 +1403,38 @@ Different batches for block number 102`, await indexerMock.getBatchWritePromise() Assert.deepEqual( - await Promise.all2(( + await Promise.all3(( + indexerMock.queryCheckpoints(), indexerMock.query(module(Entities.SimpleEntity)), indexerMock.queryHistory(module(Entities.SimpleEntity)), )), ( + [ + { + id: 3, + eventsProcessed: 1, + chainId: 100, + blockNumber: 101, + blockHash: Js.Null.Value("0x101"), + }, + { + id: 4, + eventsProcessed: 2, + chainId: 1337, + blockNumber: 101, + blockHash: Js.Null.Value("0x101"), + }, + // Reorg checkpoint id was checkpoint id 5 + // for chain 1337. After rollback it was removed + // and replaced with chain id 100 + { + id: 5, + eventsProcessed: 2, + chainId: 100, + blockNumber: 102, + blockHash: Js.Null.Value("0x102"), + }, + ], [ { Entities.SimpleEntity.id: "1", @@ -1386,68 +1443,25 @@ Different batches for block number 102`, ], [ { - current: { - chain_id: 100, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }, - previous: undefined, - entityData: Set({ + checkpointId: 3, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-0", }), }, { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 1, - }, - previous: Some({ - chain_id: 100, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }), - entityData: Set({ - Entities.SimpleEntity.id: "1", - value: "call-1", - }), - }, - { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 1, - }), - entityData: Set({ + checkpointId: 4, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-2", }), }, { - current: { - chain_id: 100, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }), - entityData: Set({ + checkpointId: 5, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-4", }), @@ -1457,8 +1471,8 @@ Different batches for block number 102`, ) }) - // Fixes duplicate history bug before 2.29.3 - Async.it_skip( + // Fixes duplicate history bug before 2.31 + Async.it( "Rollback of unordered multichain indexer (single entity id change + another entity on non-reorg chain)", async () => { let sourceMock1337 = M.Source.make( @@ -1550,21 +1564,71 @@ Different batches for block number 102`, }, ]) await indexerMock.getBatchWritePromise() - sourceMock1337.resolveGetItemsOrThrow([ - { - blockNumber: 102, - logIndex: 4, - handler, - }, - ]) + sourceMock1337.resolveGetItemsOrThrow( + [ + { + blockNumber: 103, + logIndex: 4, + handler, + }, + ], + ~latestFetchedBlockNumber=105, + ) await indexerMock.getBatchWritePromise() Assert.deepEqual( - await Promise.all2(( + await Promise.all3(( + indexerMock.queryCheckpoints(), indexerMock.query(module(Entities.SimpleEntity)), indexerMock.queryHistory(module(Entities.SimpleEntity)), )), ( + [ + { + id: 3, + eventsProcessed: 1, + chainId: 100, + blockNumber: 101, + blockHash: Js.Null.Value("0x101"), + }, + { + id: 4, + eventsProcessed: 2, + chainId: 1337, + blockNumber: 101, + blockHash: Js.Null.Value("0x101"), + }, + { + id: 5, + eventsProcessed: 1, + chainId: 1337, + blockNumber: 102, + blockHash: Js.Null.Value("0x102"), + }, + { + id: 6, + eventsProcessed: 2, + chainId: 100, + blockNumber: 102, + blockHash: Js.Null.Value("0x102"), + }, + { + id: 7, + eventsProcessed: 1, + chainId: 1337, + blockNumber: 103, + blockHash: Js.Null.Null, + }, + // Block 104 is skipped, since we don't have + // ether events processed or block hash for it + { + id: 8, + eventsProcessed: 0, + chainId: 1337, + blockNumber: 105, + blockHash: Js.Null.Value("0x105"), + }, + ], [ { Entities.SimpleEntity.id: "1", @@ -1573,114 +1637,48 @@ Different batches for block number 102`, ], [ { - current: { - chain_id: 100, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }, - previous: undefined, - entityData: Set({ + checkpointId: 3, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-0", }), }, { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 1, - }, - previous: Some({ - chain_id: 100, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }), - entityData: Set({ - Entities.SimpleEntity.id: "1", - value: "call-1", - }), - }, - { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 1, - }), - entityData: Set({ + checkpointId: 4, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-2", }), }, { - current: { - chain_id: 1337, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }), - entityData: Set({ + checkpointId: 5, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-3", }), }, { - current: { - chain_id: 100, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }), - entityData: Set({ + checkpointId: 6, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-4", }), }, { - current: { - chain_id: 1337, - block_timestamp: 102, - block_number: 102, - log_index: 4, - }, - // FIXME: This looks wrong - previous: Some({ - chain_id: 1337, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }), - entityData: Set({ + checkpointId: 7, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-5", }), }, ], ), - ~message=`Should create multiple history rows: -Sorted for the batch for block number 101 -Different batches for block number 102`, + ~message=`Should create history rows and checkpoints`, ) Assert.deepEqual( await Promise.all2(( @@ -1696,21 +1694,16 @@ Different batches for block number 102`, ], [ { - current: { - chain_id: 100, - block_timestamp: 102, - block_number: 102, - log_index: 3, - }, - previous: undefined, - entityData: Set({ + checkpointId: 6, + entityId: "foo", + entityUpdateAction: Set({ Entities.EntityWithBigDecimal.id: "foo", bigDecimal: BigDecimal.fromFloat(0.), }), }, ], ), - ~message="Should also add another entity for a non-reorg chain, which should also be rollbacked (theoretically)", + ~message="Should also add another entity for a non-reorg chain, which should also be rollbacked", ) // Should trigger rollback @@ -1726,7 +1719,7 @@ Different batches for block number 102`, Assert.deepEqual( sourceMock1337.getBlockHashesCalls, - [[100, 101, 102, 103]], + [[100, 101, 102, 105]], ~message="Should have called getBlockHashes to find rollback depth", ) sourceMock1337.resolveGetBlockHashes([ @@ -1734,7 +1727,7 @@ Different batches for block number 102`, {blockNumber: 100, blockHash: "0x100", blockTimestamp: 100}, {blockNumber: 101, blockHash: "0x101", blockTimestamp: 101}, {blockNumber: 102, blockHash: "0x102-reorged", blockTimestamp: 102}, - {blockNumber: 103, blockHash: "0x103-reorged", blockTimestamp: 103}, + {blockNumber: 105, blockHash: "0x105-reorged", blockTimestamp: 105}, ]) await indexerMock.getRollbackReadyPromise() @@ -1786,11 +1779,38 @@ Different batches for block number 102`, await indexerMock.getBatchWritePromise() Assert.deepEqual( - await Promise.all2(( + await Promise.all3(( + indexerMock.queryCheckpoints(), indexerMock.query(module(Entities.SimpleEntity)), indexerMock.queryHistory(module(Entities.SimpleEntity)), )), ( + [ + { + id: 3, + eventsProcessed: 1, + chainId: 100, + blockNumber: 101, + blockHash: Js.Null.Value("0x101"), + }, + { + id: 4, + eventsProcessed: 2, + chainId: 1337, + blockNumber: 101, + blockHash: Js.Null.Value("0x101"), + }, + // Reorg checkpoint id was checkpoint id 5 + // for chain 1337. After rollback it was removed + // and replaced with chain id 100 + { + id: 5, + eventsProcessed: 2, + chainId: 100, + blockNumber: 102, + blockHash: Js.Null.Value("0x102"), + }, + ], [ { Entities.SimpleEntity.id: "1", @@ -1799,68 +1819,25 @@ Different batches for block number 102`, ], [ { - current: { - chain_id: 100, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }, - previous: undefined, - entityData: Set({ + checkpointId: 3, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-0", }), }, { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 1, - }, - previous: Some({ - chain_id: 100, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }), - entityData: Set({ - Entities.SimpleEntity.id: "1", - value: "call-1", - }), - }, - { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 1, - }), - entityData: Set({ + checkpointId: 4, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-2", }), }, { - current: { - chain_id: 100, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }), - entityData: Set({ + checkpointId: 5, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-4", }), @@ -1882,14 +1859,9 @@ Different batches for block number 102`, ], [ { - current: { - chain_id: 100, - block_timestamp: 102, - block_number: 102, - log_index: 3, - }, - previous: undefined, - entityData: Set({ + checkpointId: 5, + entityId: "foo", + entityUpdateAction: Set({ Entities.EntityWithBigDecimal.id: "foo", bigDecimal: BigDecimal.fromFloat(0.), }), @@ -1989,11 +1961,49 @@ Different batches for block number 102`, await indexerMock.getBatchWritePromise() Assert.deepEqual( - await Promise.all2(( + await Promise.all3(( + indexerMock.queryCheckpoints(), indexerMock.query(module(Entities.SimpleEntity)), indexerMock.queryHistory(module(Entities.SimpleEntity)), )), ( + [ + { + id: 3, + eventsProcessed: 0, + chainId: 1337, + blockNumber: 100, + blockHash: Js.Null.Value("0x100"), + }, + { + id: 4, + eventsProcessed: 1, + chainId: 1337, + blockNumber: 101, + blockHash: Js.Null.Value("0x101"), + }, + { + id: 5, + eventsProcessed: 1, + chainId: 100, + blockNumber: 102, + blockHash: Js.Null.Null, + }, + { + id: 6, + eventsProcessed: 1, + chainId: 1337, + blockNumber: 102, + blockHash: Js.Null.Value("0x102"), + }, + { + id: 7, + eventsProcessed: 1, + chainId: 100, + blockNumber: 103, + blockHash: Js.Null.Value("0x103"), + }, + ], [ { Entities.SimpleEntity.id: "1", @@ -2002,50 +2012,25 @@ Different batches for block number 102`, ], [ { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }, - previous: undefined, - entityData: Set({ + checkpointId: 4, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-0", }), }, { - current: { - chain_id: 1337, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }), - entityData: Set({ + checkpointId: 6, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-1", }), }, { - current: { - chain_id: 100, - block_timestamp: 103, - block_number: 103, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }), - entityData: Set({ + checkpointId: 7, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-2", }), @@ -2069,14 +2054,9 @@ Sorted by timestamp and chain id`, ], [ { - current: { - chain_id: 100, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }, - previous: undefined, - entityData: Set({ + checkpointId: 5, + entityId: "foo", + entityUpdateAction: Set({ Entities.EntityWithBigDecimal.id: "foo", bigDecimal: BigDecimal.fromFloat(0.), }), @@ -2086,6 +2066,23 @@ Sorted by timestamp and chain id`, ~message="Should also add another entity for a non-reorg chain, which should also be rollbacked (theoretically)", ) + Assert.deepEqual( + await indexerMock.metric("envio_progress_events_count"), + [ + {value: "2", labels: Js.Dict.fromArray([("chainId", "100")])}, + {value: "2", labels: Js.Dict.fromArray([("chainId", "1337")])}, + ], + ~message="Events count before rollback", + ) + Assert.deepEqual( + await indexerMock.metric("envio_progress_block_number"), + [ + {value: "103", labels: Js.Dict.fromArray([("chainId", "100")])}, + {value: "102", labels: Js.Dict.fromArray([("chainId", "1337")])}, + ], + ~message="Progress block number before rollback", + ) + // Should trigger rollback sourceMock1337.resolveGetItemsOrThrow( [], @@ -2112,6 +2109,23 @@ Sorted by timestamp and chain id`, await indexerMock.getRollbackReadyPromise() + Assert.deepEqual( + await indexerMock.metric("envio_progress_events_count"), + [ + {value: "0", labels: Js.Dict.fromArray([("chainId", "100")])}, + {value: "1", labels: Js.Dict.fromArray([("chainId", "1337")])}, + ], + ~message="Events count after rollback", + ) + Assert.deepEqual( + await indexerMock.metric("envio_progress_block_number"), + [ + {value: "101", labels: Js.Dict.fromArray([("chainId", "100")])}, + {value: "101", labels: Js.Dict.fromArray([("chainId", "1337")])}, + ], + ~message="Progress block number after rollback", + ) + Assert.deepEqual( ( sourceMock1337.getItemsOrThrowCalls->Utils.Array.last, @@ -2159,11 +2173,44 @@ Sorted by timestamp and chain id`, await indexerMock.getBatchWritePromise() Assert.deepEqual( - await Promise.all2(( + await Promise.all3(( + indexerMock.queryCheckpoints(), indexerMock.query(module(Entities.SimpleEntity)), indexerMock.queryHistory(module(Entities.SimpleEntity)), )), ( + [ + { + id: 3, + eventsProcessed: 0, + chainId: 1337, + blockNumber: 100, + blockHash: Js.Null.Value("0x100"), + }, + { + id: 4, + eventsProcessed: 1, + chainId: 1337, + blockNumber: 101, + blockHash: Js.Null.Value("0x101"), + }, + // Block 101 for chain 100 is skipped, + // since it doesn't have events processed or block hash + { + id: 5, + eventsProcessed: 1, + chainId: 100, + blockNumber: 102, + blockHash: Js.Null.Null, + }, + { + id: 6, + eventsProcessed: 1, + chainId: 100, + blockNumber: 103, + blockHash: Js.Null.Value("0x103"), + }, + ], [ { Entities.SimpleEntity.id: "1", @@ -2172,32 +2219,17 @@ Sorted by timestamp and chain id`, ], [ { - current: { - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }, - previous: undefined, - entityData: Set({ + checkpointId: 4, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-0", }), }, { - current: { - chain_id: 100, - block_timestamp: 103, - block_number: 103, - log_index: 2, - }, - previous: Some({ - chain_id: 1337, - block_timestamp: 101, - block_number: 101, - log_index: 2, - }), - entityData: Set({ + checkpointId: 6, + entityId: "1", + entityUpdateAction: Set({ Entities.SimpleEntity.id: "1", value: "call-3", }), @@ -2219,14 +2251,9 @@ Sorted by timestamp and chain id`, ], [ { - current: { - chain_id: 100, - block_timestamp: 102, - block_number: 102, - log_index: 2, - }, - previous: undefined, - entityData: Set({ + checkpointId: 5, + entityId: "foo", + entityUpdateAction: Set({ Entities.EntityWithBigDecimal.id: "foo", bigDecimal: BigDecimal.fromFloat(0.), }), diff --git a/scenarios/test_codegen/test/schema_types/BigDecimal_test.res b/scenarios/test_codegen/test/schema_types/BigDecimal_test.res index 9a2ebefed..d74e405ec 100644 --- a/scenarios/test_codegen/test/schema_types/BigDecimal_test.res +++ b/scenarios/test_codegen/test/schema_types/BigDecimal_test.res @@ -46,6 +46,7 @@ describe("Load and save an entity with a BigDecimal from DB", () => { inMemoryStore, shouldSaveHistory: false, isPreload: false, + checkpointId: 0, })->(Utils.magic: Internal.handlerContext => Types.loaderContext) let _ = handlerContext.entityWithBigDecimal.get(testEntity1.id) diff --git a/scenarios/test_codegen/test/schema_types/Timestamp_test.res b/scenarios/test_codegen/test/schema_types/Timestamp_test.res index 786552e2e..177d93365 100644 --- a/scenarios/test_codegen/test/schema_types/Timestamp_test.res +++ b/scenarios/test_codegen/test/schema_types/Timestamp_test.res @@ -38,6 +38,7 @@ describe("Load and save an entity with a Timestamp from DB", () => { inMemoryStore, shouldSaveHistory: false, isPreload: false, + checkpointId: 0, })->(Utils.magic: Internal.handlerContext => Types.loaderContext) let _ = handlerContext.entityWithTimestamp.get(testEntity.id)