diff --git a/hive_integration/nodocker/engine/clmock.nim b/hive_integration/nodocker/engine/clmock.nim index 574154049..731bcf437 100644 --- a/hive_integration/nodocker/engine/clmock.nim +++ b/hive_integration/nodocker/engine/clmock.nim @@ -75,11 +75,6 @@ type onSafeBlockChange * : proc(): bool {.gcsafe.} onFinalizedBlockChange* : proc(): bool {.gcsafe.} - GetPayloadResponse = object - executionPayload: ExecutionPayload - blockValue: Option[UInt256] - blobsBundle: Option[BlobsBundleV1] - func latestPayloadNumber*(h: Table[uint64, ExecutionPayload]): uint64 = result = 0'u64 for n, _ in h: @@ -99,17 +94,21 @@ func latestWithdrawalsIndex*(h: Table[uint64, ExecutionPayload]): uint64 = func client(cl: CLMocker): RpcClient = cl.clients.first.client -proc init(cl: CLMocker, clients: ClientPool, com: CommonRef) = - cl.clients = clients +proc init(cl: CLMocker, eng: EngineEnv, com: CommonRef) = + cl.clients = ClientPool() + cl.clients.add eng cl.com = com cl.slotsToSafe = 1 cl.slotsToFinalized = 2 cl.payloadProductionClientDelay = 1 cl.headerHistory[0] = com.genesisHeader() -proc newClMocker*(clients: ClientPool, com: CommonRef): CLMocker = +proc newClMocker*(eng: EngineEnv, com: CommonRef): CLMocker = new result - result.init(clients, com) + result.init(eng, com) + +proc addEngine*(cl: CLMocker, eng: EngineEnv) = + cl.clients.add eng proc waitForTTD*(cl: CLMocker): Future[bool] {.async.} = let ttd = cl.com.ttd() @@ -193,32 +192,6 @@ func isCancun(cl: CLMocker, timestamp: Quantity): bool = let ts = fromUnix(timestamp.int64) cl.com.isCancunOrLater(ts) -func V1(attr: Option[PayloadAttributes]): Option[PayloadAttributesV1] = - if attr.isNone: - return none(PayloadAttributesV1) - some(attr.get.V1) - -when false: - func V2(attr: Option[PayloadAttributes]): Option[PayloadAttributesV2] = - if attr.isNone: - return none(PayloadAttributesV2) - some(attr.get.V2) - - func V3(attr: Option[PayloadAttributes]): Option[PayloadAttributesV3] = - if attr.isNone: - return none(PayloadAttributesV3) - some(attr.get.V3) - -proc fcu(cl: CLMocker, version: Version, - update: ForkchoiceStateV1, - attr: Option[PayloadAttributes]): - Result[ForkchoiceUpdatedResponse, string] = - let client = cl.nextBlockProducer.client - case version - of Version.V1: client.forkchoiceUpdatedV1(update, attr.V1) - of Version.V2: client.forkchoiceUpdatedV2(update, attr) - of Version.V3: client.forkchoiceUpdatedV3(update, attr) - # Picks the next payload producer from the set of clients registered proc pickNextPayloadProducer(cl: CLMocker): bool = doAssert cl.clients.len != 0 @@ -274,7 +247,8 @@ proc requestNextPayload(cl: CLMocker): bool = cl.prevRandaoHistory[number] = nextPrevRandao let version = cl.latestPayloadAttributes.version - let res = cl.fcu(version, cl.latestForkchoice, some(cl.latestPayloadAttributes)) + let client = cl.nextBlockProducer.client + let res = client.forkchoiceUpdated(version, cl.latestForkchoice, some(cl.latestPayloadAttributes)) if res.isErr: error "CLMocker: Could not send forkchoiceUpdated", version=version, msg=res.error return false @@ -299,32 +273,11 @@ proc getPayload(cl: CLMocker, payloadId: PayloadID): Result[GetPayloadResponse, let ts = cl.latestPayloadAttributes.timestamp let client = cl.nextBlockProducer.client if cl.isCancun(ts): - let res = client.getPayloadV3(payloadId) - if res.isErr: - return err(res.error) - let x = res.get - return ok(GetPayloadResponse( - executionPayload: executionPayload(x.executionPayload), - blockValue: some(x.blockValue), - blobsBundle: some(x.blobsBundle) - )) - - if cl.isShanghai(ts): - let res = client.getPayloadV2(payloadId) - if res.isErr: - return err(res.error) - let x = res.get - return ok(GetPayloadResponse( - executionPayload: executionPayload(x.executionPayload), - blockValue: some(x.blockValue) - )) - - let res = client.getPayloadV1(payloadId) - if res.isErr: - return err(res.error) - return ok(GetPayloadResponse( - executionPayload: executionPayload(res.get), - )) + client.getPayload(payloadId, Version.V3) + elif cl.isShanghai(ts): + client.getPayload(payloadId, Version.V2) + else: + client.getPayload(payloadId, Version.V1) proc getNextPayload(cl: CLMocker): bool = let res = cl.getPayload(cl.nextPayloadID) @@ -451,7 +404,8 @@ proc broadcastNextNewPayload(cl: CLMocker): bool = proc broadcastForkchoiceUpdated(cl: CLMocker, update: ForkchoiceStateV1): Result[ForkchoiceUpdatedResponse, string] = let version = cl.latestExecutedPayload.version - cl.fcu(version, update, none(PayloadAttributes)) + let client = cl.nextBlockProducer.client + client.forkchoiceUpdated(version, update, none(PayloadAttributes)) proc broadcastLatestForkchoice(cl: CLMocker): bool = let res = cl.broadcastForkchoiceUpdated(cl.latestForkchoice) @@ -480,7 +434,6 @@ proc broadcastLatestForkchoice(cl: CLMocker): bool = return true - proc produceSingleBlock*(cl: CLMocker, cb: BlockProcessCallbacks): bool {.gcsafe.} = doAssert(cl.ttdReached) diff --git a/hive_integration/nodocker/engine/engine/engine_spec.nim b/hive_integration/nodocker/engine/engine/engine_spec.nim index b9a60be85..1d113a37c 100644 --- a/hive_integration/nodocker/engine/engine/engine_spec.nim +++ b/hive_integration/nodocker/engine/engine/engine_spec.nim @@ -19,9 +19,6 @@ type slotsToFinalized*: int slotsToSafe*: int -const - prevRandaoContractAddr = hexToByteArray[20]("0000000000000000000000000000000000000316") - template testNP(res, cond: untyped, validHash = none(common.Hash256)) = testCond res.isOk let s = res.get() diff --git a/hive_integration/nodocker/engine/engine_client.nim b/hive_integration/nodocker/engine/engine_client.nim index 47f28e104..1314f0826 100644 --- a/hive_integration/nodocker/engine/engine_client.nim +++ b/hive_integration/nodocker/engine/engine_client.nim @@ -11,7 +11,9 @@ import import web3/engine_api as web3_engine_api -export execution_types +export + execution_types, + rpcclient type Hash256 = eth_types.Hash256 @@ -43,20 +45,6 @@ proc forkchoiceUpdatedV1*(client: RpcClient, wrapTrySimpleRes: client.engine_forkchoiceUpdatedV1(update, payloadAttributes) -#proc forkchoiceUpdatedV2*(client: RpcClient, -# update: ForkchoiceStateV1, -# payloadAttributes = none(PayloadAttributesV2)): -# Result[ForkchoiceUpdatedResponse, string] = -# wrapTrySimpleRes: -# client.engine_forkchoiceUpdatedV2(update, payloadAttributes) - -#proc forkchoiceUpdatedV3*(client: RpcClient, -# update: ForkchoiceStateV1, -# payloadAttributes = none(PayloadAttributesV3)): -# Result[ForkchoiceUpdatedResponse, string] = -# wrapTrySimpleRes: -# client.engine_forkchoiceUpdatedV3(update, payloadAttributes) - proc forkchoiceUpdatedV2*(client: RpcClient, update: ForkchoiceStateV1, payloadAttributes = none(PayloadAttributes)): @@ -83,6 +71,50 @@ proc getPayloadV3*(client: RpcClient, payloadId: PayloadID): Result[GetPayloadV3 wrapTrySimpleRes: client.engine_getPayloadV3(payloadId) +proc getPayload*(client: RpcClient, + payloadId: PayloadID, + version: Version): Result[GetPayloadResponse, string] = + if version == Version.V3: + let x = client.getPayloadV3(payloadId).valueOr: + return err(error) + ok(GetPayloadResponse( + executionPayload: executionPayload(x.executionPayload), + blockValue: some(x.blockValue), + blobsBundle: some(x.blobsBundle) + )) + elif version == Version.V2: + let x = client.getPayloadV2(payloadId).valueOr: + return err(error) + ok(GetPayloadResponse( + executionPayload: executionPayload(x.executionPayload), + blockValue: some(x.blockValue) + )) + else: + let x = client.getPayloadV1(payloadId).valueOr: + return err(error) + ok(GetPayloadResponse( + executionPayload: executionPayload(x), + )) + +proc forkchoiceUpdated*(client: RpcClient, + update: ForkchoiceStateV1, + attr: PayloadAttributes): + Result[ForkchoiceUpdatedResponse, string] = + case attr.version + of Version.V1: client.forkchoiceUpdatedV1(update, some attr.V1) + of Version.V2: client.forkchoiceUpdatedV2(update, some attr) + of Version.V3: client.forkchoiceUpdatedV3(update, some attr) + +proc forkchoiceUpdated*(client: RpcClient, + version: Version, + update: ForkchoiceStateV1, + attr = none(PayloadAttributes)): + Result[ForkchoiceUpdatedResponse, string] = + case version + of Version.V1: client.forkchoiceUpdatedV1(update, attr.V1) + of Version.V2: client.forkchoiceUpdatedV2(update, attr) + of Version.V3: client.forkchoiceUpdatedV3(update, attr) + proc newPayloadV1*(client: RpcClient, payload: ExecutionPayloadV1): Result[PayloadStatusV1, string] = @@ -110,6 +142,15 @@ proc newPayloadV3*(client: RpcClient, wrapTrySimpleRes: client.engine_newPayloadV3(payload, versionedHashes, parentBeaconBlockRoot) +proc newPayload*(client: RpcClient, + payload: ExecutionPayload, + version: Version): + Result[PayloadStatusV1, string] = + if version == Version.V1: + client.newPayloadV1(payload.V1) + else: + client.newPayloadV2(payload.V2) + proc exchangeCapabilities*(client: RpcClient, methods: seq[string]): Result[seq[string], string] = diff --git a/hive_integration/nodocker/engine/engine_env.nim b/hive_integration/nodocker/engine/engine_env.nim index 827232cd7..481139789 100644 --- a/hive_integration/nodocker/engine/engine_env.nim +++ b/hive_integration/nodocker/engine/engine_env.nim @@ -1,14 +1,13 @@ import - std/[os, math], + std/os, eth/keys, eth/p2p as eth_p2p, chronos, json_rpc/[rpcserver, rpcclient], - stew/[results, byteutils], + stew/[results], ../../../nimbus/[ config, constants, - transaction, core/sealer, core/chain, core/tx_pool, @@ -20,25 +19,12 @@ import beacon/beacon_engine, common ], - ../../../tests/test_helpers, - ./engine_client + ../../../tests/test_helpers export results type - BaseTx* = object of RootObj - recipient*: Option[EthAddress] - gasLimit* : GasInt - amount* : UInt256 - payload* : seq[byte] - txType* : Option[TxType] - - BigInitcodeTx* = object of BaseTx - initcodeLength*: int - padByte* : uint8 - initcode* : seq[byte] - EngineEnv* = ref object conf : NimbusConf com : CommonRef @@ -46,8 +32,6 @@ type server : RpcHttpServer sealer : SealingEngineRef ttd : DifficultyInt - tx : Transaction - nonce : uint64 client : RpcHttpClient sync : BeaconSyncRef @@ -56,12 +40,8 @@ const genesisFile = baseFolder & "/init/genesis.json" sealerKey = baseFolder & "/init/sealer.key" chainFolder = baseFolder & "/chains" - - # This is the account that sends vault funding transactions. - vaultAddr* = hexToByteArray[20]("0xcf49fda3be353c69b41ed96333cd24302da4556f") jwtSecret = "0x7365637265747365637265747365637265747365637265747365637265747365" - proc makeCom*(conf: NimbusConf): CommonRef = CommonRef.new( newCoreDbRef LegacyDbMemory, @@ -189,98 +169,3 @@ func node*(env: EngineEnv): ENode = proc connect*(env: EngineEnv, node: ENode) = waitFor env.node.connectToNode(node) - -func gwei(n: int64): GasInt {.compileTime.} = - GasInt(n * (10 ^ 9)) - -proc getTxType(tc: BaseTx, nonce: uint64): TxType = - if tc.txType.isNone: - if nonce mod 2 == 0: - TxLegacy - else: - TxEIP1559 - else: - tc.txType.get - -proc makeTx*(env: EngineEnv, vaultKey: PrivateKey, tc: BaseTx, nonce: AccountNonce): Transaction = - const - gasPrice = 30.gwei - gasTipPrice = 1.gwei - - gasFeeCap = gasPrice - gasTipCap = gasTipPrice - - let chainId = env.conf.networkParams.config.chainId - let txType = tc.getTxType(nonce) - - # Build the transaction depending on the specified type - let tx = if txType == TxLegacy: - Transaction( - txType : TxLegacy, - nonce : nonce, - to : tc.recipient, - value : tc.amount, - gasLimit: tc.gasLimit, - gasPrice: gasPrice, - payload : tc.payload - ) - else: - Transaction( - txType : TxEIP1559, - nonce : nonce, - gasLimit: tc.gasLimit, - maxFee : gasFeeCap, - maxPriorityFee: gasTipCap, - to : tc.recipient, - value : tc.amount, - payload : tc.payload, - chainId : chainId - ) - - signTransaction(tx, vaultKey, chainId, eip155 = true) - -proc makeTx*(env: EngineEnv, vaultKey: PrivateKey, tc: var BigInitcodeTx, nonce: AccountNonce): Transaction = - if tc.payload.len == 0: - # Prepare initcode payload - if tc.initcode.len != 0: - doAssert(tc.initcode.len <= tc.initcodeLength, "invalid initcode (too big)") - tc.payload = tc.initcode - - while tc.payload.len < tc.initcodeLength: - tc.payload.add tc.padByte - - doAssert(tc.recipient.isNone, "invalid configuration for big contract tx creator") - env.makeTx(vaultKey, tc.BaseTx, nonce) - -proc sendNextTx*(env: EngineEnv, vaultKey: PrivateKey, tc: BaseTx): bool = - env.tx = env.makeTx(vaultKey, tc, env.nonce) - inc env.nonce - let rr = env.client.sendTransaction(env.tx) - if rr.isErr: - error "Unable to send transaction", msg=rr.error - return false - return true - -proc sendTx*(env: EngineEnv, vaultKey: PrivateKey, tc: BaseTx, nonce: AccountNonce): bool = - env.tx = env.makeTx(vaultKey, tc, nonce) - let rr = env.client.sendTransaction(env.tx) - if rr.isErr: - error "Unable to send transaction", msg=rr.error - return false - return true - -proc sendTx*(env: EngineEnv, vaultKey: PrivateKey, tc: BigInitcodeTx, nonce: AccountNonce): bool = - env.tx = env.makeTx(vaultKey, tc, nonce) - let rr = env.client.sendTransaction(env.tx) - if rr.isErr: - error "Unable to send transaction", msg=rr.error - return false - return true - -proc sendTx*(env: EngineEnv, tx: Transaction): bool = - env.tx = tx - let rr = env.client.sendTransaction(env.tx) - if rr.isErr: - error "Unable to send transaction", msg=rr.error - return false - return true diff --git a/hive_integration/nodocker/engine/test_env.nim b/hive_integration/nodocker/engine/test_env.nim index 9bd88aa7d..a83e59c4c 100644 --- a/hive_integration/nodocker/engine/test_env.nim +++ b/hive_integration/nodocker/engine/test_env.nim @@ -7,13 +7,15 @@ import ./clmock, ./engine_client, ./client_pool, - ./engine_env + ./engine_env, + ./tx_sender export clmock, engine_client, client_pool, - engine_env + engine_env, + tx_sender type TestEnv* = ref object @@ -23,26 +25,18 @@ type port : int rpcPort : int clients : ClientPool + sender : TxSender clMock* : CLMocker - vaultKey : PrivateKey - -const - vaultKeyHex = "63b508a03c3b5937ceb903af8b1b0c191012ef6eb7e9c3fb7afa94e5d214d376" proc makeEnv(conf: NimbusConf): TestEnv = - let env = TestEnv( + TestEnv( conf : conf, port : 30303, rpcPort: 8545, clients: ClientPool(), + sender : TxSender.new(conf.networkParams), ) - env.vaultKey = PrivateKey.fromHex(vaultKeyHex).valueOr: - echo error - quit(QuitFailure) - - env - proc addEngine(env: TestEnv, conf: var NimbusConf): EngineEnv = conf.tcpPort = Port env.port conf.udpPort = Port env.port @@ -85,47 +79,58 @@ func engine*(env: TestEnv): EngineEnv = env.clients.first proc setupCLMock*(env: TestEnv) = - env.clmock = newCLMocker(env.clients, env.engine.com) + env.clmock = newCLMocker(env.engine, env.engine.com) -proc addEngine*(env: TestEnv): EngineEnv = +proc addEngine*(env: TestEnv, addToCL: bool = true): EngineEnv = + doAssert(env.clMock.isNil.not) var conf = env.conf # clone the conf let eng = env.addEngine(conf) eng.connect(env.engine.node) + if addToCL: + env.clMock.addEngine(eng) eng -proc makeTx*(env: TestEnv, eng: EngineEnv, tc: BaseTx, nonce: AccountNonce): Transaction = - eng.makeTx(env.vaultKey, tc, nonce) +proc makeTx*(env: TestEnv, tc: BaseTx, nonce: AccountNonce): Transaction = + env.sender.makeTx(tc, nonce) -proc makeTx*(env: TestEnv, eng: EngineEnv, tc: var BigInitcodeTx, nonce: AccountNonce): Transaction = - eng.makeTx(env.vaultKey, tc, nonce) +proc makeTx*(env: TestEnv, tc: BigInitcodeTx, nonce: AccountNonce): Transaction = + env.sender.makeTx(tc, nonce) + +proc makeTxs*(env: TestEnv, tc: BaseTx, num: int): seq[Transaction] = + result = newSeqOfCap[Transaction](num) + for _ in 0.. ws.getTotalPayloadCount() { - panic("invalid payload/re-org configuration") - return ws.getTotalPayloadCount() + 1 - ws.ReOrgBlockCount + Sidechain = ref object + startAccount: UInt256 + nextIndex : int + wdHistory : WDHistory + sidechain : Table[uint64, ExecutionPayload] + payloadId : PayloadID + height : uint64 + attr : Option[PayloadAttributes] -func (ws *WithdrawalsReorgSpec) GetSidechainBlockTimeIncrements() uint64 { - if ws.SidechainTimeIncrements == 0 { + Canonical = ref object + startAccount: UInt256 + nextIndex : int + +proc getSidechainSplitHeight(ws: ReorgSpec): int = + doAssert(ws.reOrgBlockCount <= ws.getTotalPayloadCount()) + return ws.getTotalPayloadCount() + 1 - ws.reOrgBlockCount + +proc getSidechainBlockTimeIncrements(ws: ReorgSpec): int= + if ws.sidechainTimeIncrements == 0: return ws.getBlockTimeIncrements() + ws.sidechainTimeIncrements - return ws.SidechainTimeIncrements +proc getSidechainWdForkHeight(ws: ReorgSpec): int = + if ws.getSidechainBlockTimeIncrements() != ws.getBlockTimeIncrements(): + # Block timestamp increments in both chains are different so need to + # calculate different heights, only if split happens before fork. + # We cannot split by having two different genesis blocks. + doAssert(ws.getSidechainSplitHeight() != 0, "invalid sidechain split height") -func (ws *WithdrawalsReorgSpec) GetSidechainWithdrawalsForkHeight() uint64 { - if ws.getSidechainBlockTimeIncrements() != ws.getBlockTimeIncrements() { - # Block timestamp increments in both chains are different so need to calculate different heights, only if split happens before fork - if ws.getSidechainSplitHeight() == 0 { - # We cannot split by having two different genesis blocks. - panic("invalid sidechain split height") - - if ws.getSidechainSplitHeight() <= ws.WithdrawalsForkHeight { + if ws.getSidechainSplitHeight() <= ws.wdForkHeight: # We need to calculate the height of the fork on the sidechain - sidechainSplitBlockTimestamp := ((ws.getSidechainSplitHeight() - 1) * ws.getBlockTimeIncrements()) - remainingTime := (ws.getWithdrawalsGenesisTimeDelta() - sidechainSplitBlockTimestamp) - if remainingTime == 0 { + let sidechainSplitBlocktimestamp = (ws.getSidechainSplitHeight() - 1) * ws.getBlockTimeIncrements() + let remainingTime = ws.getWithdrawalsGenesisTimeDelta() - sidechainSplitBlocktimestamp + if remainingTime == 0 : return ws.getSidechainSplitHeight() - return ((remainingTime - 1) / ws.SidechainTimeIncrements) + ws.getSidechainSplitHeight() + return ((remainingTime - 1) div ws.sidechainTimeIncrements) + ws.getSidechainSplitHeight() - return ws.WithdrawalsForkHeight -]# + return ws.wdForkHeight -proc execute*(ws: ReorgSpec, t: TestEnv): bool = - testCond waitFor t.clMock.waitForTTD() +proc execute*(ws: ReorgSpec, env: TestEnv): bool = + result = true + + testCond waitFor env.clMock.waitForTTD() - return true -#[ # Spawn a secondary client which will produce the sidechain - secondaryEngine, err := hive_rpc.HiveRPCEngineStarter{}.StartClient(t.T, t.TestContext, t.Genesis, t.ClientParams, t.ClientFiles, t.Engine) - if err != nil { - error "Unable to spawn a secondary client: %v", t.TestName, err) - } - secondaryEngineTest := test.NewTestEngineClient(t, secondaryEngine) - # t.clMock.AddEngineClient(secondaryEngine) + let sec = env.addEngine(addToCL = false) - var ( - canonicalStartAccount = big.NewInt(0x1000) - canonicalNextIndex = uint64(0) - sidechainStartAccount = new(big.Int).SetBit(common.Big0, 160, 1) - sidechainNextIndex = uint64(0) - sidechainwdHistory = make(wdHistory) - sidechain = make(map[uint64]*typ.ExecutableData) - sidechainPayloadId *beacon.PayloadID - ) + var + canonical = Canonical( + startAccount: u256(0x1000), + nextIndex : 0, + ) + sidechain = Sidechain( + startAccount: 1.u256 shl 160, + nextIndex : 0, + wdHistory : WDHistory(), + sidechain : initTable[uint64, ExecutionPayload]() + ) # Sidechain withdraws on the max account value range 0xffffffffffffffffffffffffffffffffffffffff - sidechainStartAccount.Sub(sidechainStartAccount, big.NewInt(int64(ws.getWithdrawableAccountCount())+1)) + sidechain.startAccount -= u256(ws.getWithdrawableAccountCount()+1) - t.clMock.ProduceBlocks(int(ws.getPreWithdrawalsBlockCount()+ws.WithdrawalsBlockCount), clmock.BlockProcessCallbacks{ - OnPayloadProducerSelected: proc(): bool = - t.clMock.NextWithdrawals = nil + let numBlocks = ws.getPreWithdrawalsBlockCount()+ws.wdBlockCount + let pbRes = env.clMock.produceBlocks(numBlocks, BlockProcessCallbacks( + onPayloadProducerSelected: proc(): bool = + env.clMock.nextWithdrawals = none(seq[WithdrawalV1]) - if t.clMock.CurrentPayloadNumber >= ws.WithdrawalsForkHeight { + if env.clMock.currentPayloadNumber >= ws.wdForkHeight.uint64: # Prepare some withdrawals - t.clMock.NextWithdrawals, canonicalNextIndex = ws.GenerateWithdrawalsForBlock(canonicalNextIndex, canonicalStartAccount) - ws.wdHistory[t.clMock.CurrentPayloadNumber] = t.clMock.NextWithdrawals - } + let wfb = ws.generateWithdrawalsForBlock(canonical.nextIndex, canonical.startAccount) + env.clMock.nextWithdrawals = some(w3Withdrawals wfb.wds) + canonical.nextIndex = wfb.nextIndex + ws.wdHistory.put(env.clMock.currentPayloadNumber, wfb.wds) - if t.clMock.CurrentPayloadNumber >= ws.getSidechainSplitHeight() { + if env.clMock.currentPayloadNumber >= ws.getSidechainSplitHeight().uint64: # We have split - if t.clMock.CurrentPayloadNumber >= ws.getSidechainWithdrawalsForkHeight() { + if env.clMock.currentPayloadNumber >= ws.getSidechainWdForkHeight().uint64: # And we are past the withdrawals fork on the sidechain - sidechainwdHistory[t.clMock.CurrentPayloadNumber], sidechainNextIndex = ws.GenerateWithdrawalsForBlock(sidechainNextIndex, sidechainStartAccount) - } # else nothing to do - } else { - # We have not split - sidechainwdHistory[t.clMock.CurrentPayloadNumber] = t.clMock.NextWithdrawals - sidechainNextIndex = canonicalNextIndex - } + let wfb = ws.generateWithdrawalsForBlock(sidechain.nextIndex, sidechain.startAccount) + sidechain.wdHistory.put(env.clMock.currentPayloadNumber, wfb.wds) + sidechain.nextIndex = wfb.nextIndex + else: + if env.clMock.nextWithdrawals.isSome: + let wds = ethWithdrawals env.clMock.nextWithdrawals.get() + sidechain.wdHistory.put(env.clMock.currentPayloadNumber, wds) + sidechain.nextIndex = canonical.nextIndex - }, - OnRequestNextPayload: proc(): bool = + return true + , + onRequestNextPayload: proc(): bool = # Send transactions to be included in the payload - txs, err := helper.SendNextTransactions( - t.TestContext, - t.clMock.NextBlockProducer, - &helper.BaseTransactionCreator{ - Recipient: &globals.PrevRandaoContractAddr, - Amount: common.Big1, - Payload: nil, - TxType: t.TestTransactionType, - GasLimit: 75000, - }, - ws.getTransactionCountPerPayload(), + let txs = env.makeTxs( + BaseTx( + recipient: some(prevRandaoContractAddr), + amount: 1.u256, + txType: ws.txType, + gasLimit: 75000.GasInt, + ), + ws.getTransactionCountPerPayload() ) - if err != nil { - error "Error trying to send transactions: %v", t.TestName, err) - } + + testCond env.sendTxs(env.clMock.nextBlockProducer, txs): + error "Error trying to send transaction" # Error will be ignored here since the tx could have been already relayed - secondaryEngine.SendTransactions(t.TestContext, txs...) + discard env.sendTxs(sec, txs) - if t.clMock.CurrentPayloadNumber >= ws.getSidechainSplitHeight() { + if env.clMock.currentPayloadNumber >= ws.getSidechainSplitHeight().uint64: # Also request a payload from the sidechain - fcU := beacon.ForkchoiceStateV1{ - HeadBlockHash: t.clMock.latestForkchoice.HeadBlockHash, - } + var fcState = ForkchoiceStateV1( + headBlockHash: env.clMock.latestForkchoice.headBlockHash, + ) - if t.clMock.CurrentPayloadNumber > ws.getSidechainSplitHeight() { - if lastSidePayload, ok := sidechain[t.clMock.CurrentPayloadNumber-1]; !ok { - panic("sidechain payload not found") - } else { - fcU.HeadBlockHash = lastSidePayload.BlockHash - } - } + if env.clMock.currentPayloadNumber > ws.getSidechainSplitHeight().uint64: + let lastSidePayload = sidechain.sidechain[env.clMock.currentPayloadNumber-1] + fcState.headBlockHash = lastSidePayload.blockHash - var version int - pAttributes := typ.PayloadAttributes{ - Random: t.clMock.latestPayloadAttributes.Random, - SuggestedFeeRecipient: t.clMock.latestPayloadAttributes.SuggestedFeeRecipient, - } - if t.clMock.CurrentPayloadNumber > ws.getSidechainSplitHeight() { - pAttributes.Timestamp = sidechain[t.clMock.CurrentPayloadNumber-1].Timestamp + uint64(ws.getSidechainBlockTimeIncrements()) - } else if t.clMock.CurrentPayloadNumber == ws.getSidechainSplitHeight() { - pAttributes.Timestamp = t.clMock.latestHeader.Time + uint64(ws.getSidechainBlockTimeIncrements()) - } else { - pAttributes.Timestamp = t.clMock.latestPayloadAttributes.Timestamp - } - if t.clMock.CurrentPayloadNumber >= ws.getSidechainWithdrawalsForkHeight() { + var attr = PayloadAttributes( + prevRandao: env.clMock.latestPayloadAttributes.prevRandao, + suggestedFeeRecipient: env.clMock.latestPayloadAttributes.suggestedFeeRecipient, + ) + + if env.clMock.currentPayloadNumber > ws.getSidechainSplitHeight().uint64: + attr.timestamp = w3Qty(sidechain.sidechain[env.clMock.currentPayloadNumber-1].timestamp, ws.getSidechainBlockTimeIncrements()) + elif env.clMock.currentPayloadNumber == ws.getSidechainSplitHeight().uint64: + attr.timestamp = w3Qty(env.clMock.latestHeader.timestamp, ws.getSidechainBlockTimeIncrements()) + else: + attr.timestamp = env.clMock.latestPayloadAttributes.timestamp + + if env.clMock.currentPayloadNumber >= ws.getSidechainwdForkHeight().uint64: # Withdrawals - version = 2 - pAttributes.Withdrawals = sidechainwdHistory[t.clMock.CurrentPayloadNumber] - } else { - # No withdrawals - version = 1 - } + let rr = sidechain.wdHistory.get(env.clMock.currentPayloadNumber) + testCond rr.isOk: + error "sidechain wd", msg=rr.error - info "Requesting sidechain payload %d: %v", t.TestName, t.clMock.CurrentPayloadNumber, pAttributes) + attr.withdrawals = some(w3Withdrawals rr.get) - r := secondaryEngineTest.forkchoiceUpdated(&fcU, &pAttributes, version) + info "Requesting sidechain payload", + number=env.clMock.currentPayloadNumber + + sidechain.attr = some(attr) + let r = sec.client.forkchoiceUpdated(fcState, attr) r.expectNoError() - r.expectPayloadStatus(test.Valid) - if r.Response.PayloadID == nil { - error "Unable to get a payload ID on the sidechain", t.TestName) - } - sidechainPayloadId = r.Response.PayloadID - } - }, - OnGetPayload: proc(): bool = - var ( - version int - payload *typ.ExecutableData - ) - if t.clMock.CurrentPayloadNumber >= ws.getSidechainWithdrawalsForkHeight() { - version = 2 - } else { - version = 1 - } - if t.clMock.latestPayloadBuilt.Number >= ws.getSidechainSplitHeight() { + r.testFCU(valid) + testCond r.get().payloadID.isSome: + error "Unable to get a payload ID on the sidechain" + sidechain.payloadId = r.get().payloadID.get() + + return true + , + onGetPayload: proc(): bool = + var + payload: ExecutionPayload + + if env.clMock.latestPayloadBuilt.blockNumber.uint64 >= ws.getSidechainSplitHeight().uint64: # This payload is built by the secondary client, hence need to manually fetch it here - r := secondaryEngineTest.getPayload(sidechainPayloadId, version) + doAssert(sidechain.attr.isSome) + let version = sidechain.attr.get().version + let r = sec.client.getPayload(sidechain.payloadId, version) r.expectNoError() - payload = &r.Payload - sidechain[payload.Number] = payload - } else { + payload = r.get().executionPayload + sidechain.sidechain[payload.blockNumber.uint64] = payload + else: # This block is part of both chains, simply forward it to the secondary client - payload = &t.clMock.latestPayloadBuilt - } - r := secondaryEngineTest.newPayload(payload, nil, nil, version) - r.expectStatus(test.Valid) - p := secondaryEngineTest.forkchoiceUpdated( - &beacon.ForkchoiceStateV1{ - HeadBlockHash: payload.BlockHash, - }, - nil, - version, + payload = env.clMock.latestPayloadBuilt + + let r = sec.client.newPayload(payload, payload.version) + r.expectStatus(valid) + + let fcState = ForkchoiceStateV1( + headBlockHash: payload.blockHash, ) - p.expectPayloadStatus(test.Valid) - }, - }) + let p = sec.client.forkchoiceUpdated(payload.version, fcState) + p.testFCU(valid) + return true + )) + testCond pbRes - sidechainHeight := t.clMock.latestExecutedPayload.Number + sidechain.height = env.clMock.latestExecutedPayload.blockNumber.uint64 - if ws.WithdrawalsForkHeight < ws.getSidechainWithdrawalsForkHeight() { + if ws.wdForkHeight < ws.getSidechainwdForkHeight(): # This means the canonical chain forked before the sidechain. # Therefore we need to produce more sidechain payloads to reach # at least`ws.WithdrawalsBlockCount` withdrawals payloads produced on # the sidechain. - for i := uint64(0); i < ws.getSidechainWithdrawalsForkHeight()-ws.WithdrawalsForkHeight; i++ { - sidechainwdHistory[sidechainHeight+1], sidechainNextIndex = ws.GenerateWithdrawalsForBlock(sidechainNextIndex, sidechainStartAccount) - pAttributes := typ.PayloadAttributes{ - Timestamp: sidechain[sidechainHeight].Timestamp + ws.getSidechainBlockTimeIncrements(), - Random: t.clMock.latestPayloadAttributes.Random, - SuggestedFeeRecipient: t.clMock.latestPayloadAttributes.SuggestedFeeRecipient, - Withdrawals: sidechainwdHistory[sidechainHeight+1], - } - r := secondaryEngineTest.forkchoiceUpdatedV2(&beacon.ForkchoiceStateV1{ - HeadBlockHash: sidechain[sidechainHeight].BlockHash, - }, &pAttributes) - r.expectPayloadStatus(test.Valid) - time.Sleep(time.Second) - p := secondaryEngineTest.getPayloadV2(r.Response.PayloadID) + let height = ws.getSidechainwdForkHeight()-ws.wdForkHeight + for i in 0..= ws.getSidechainWithdrawalsForkHeight() { - version = 2 - } else { - version = 1 - } - info "Sending sidechain payload %d, hash=%s, parent=%s", t.TestName, payloadNumber, payload.BlockHash, payload.ParentHash) - r := t.rpcClient.newPayload(payload, nil, nil, version) - r.expectStatusEither(test.Valid, test.Accepted) - p := t.rpcClient.forkchoiceUpdated( - &beacon.ForkchoiceStateV1{ - HeadBlockHash: payload.BlockHash, - }, - nil, - version, - ) - p.expectPayloadStatus(test.Valid) - } - } + var payloadNumber = ws.getSidechainSplitHeight() + while payloadNumber.uint64 <= sidechain.height: + let payload = sidechain.sidechain[payloadNumber.uint64] + var version = Version.V1 + if payloadNumber >= ws.getSidechainwdForkHeight(): + version = Version.V2 + + info "Sending sidechain", + payloadNumber, + hash=payload.blockHash.short, + parentHash=payload.parentHash.short + + let r = env.client.newPayload(payload, version) + r.expectStatusEither(valid, accepted) + + let fcState = ForkchoiceStateV1(headBlockHash: payload.blockHash) + let p = env.client.forkchoiceUpdated(version, fcState) + p.testFCU(valid) + inc payloadNumber + # Verify withdrawals changed - sidechainwdHistory.VerifyWithdrawals( - sidechainHeight, - nil, - t.TestEngine, - ) + let r2 = sidechain.wdHistory.verifyWithdrawals(sidechain.height, none(UInt256), env.client) + testCond r2.isOk + # Verify all balances of accounts in the original chain didn't increase # after the fork. # We are using different accounts credited between the canonical chain # and the fork. # We check on `latest`. - ws.wdHistory.VerifyWithdrawals( - ws.WithdrawalsForkHeight-1, - nil, - t.TestEngine, - ) + let r3 = ws.wdHistory.verifyWithdrawals(uint64(ws.wdForkHeight-1), none(UInt256), env.client) + testCond r3.isOk # Re-Org back to the canonical chain - r := t.rpcClient.forkchoiceUpdatedV2(&beacon.ForkchoiceStateV1{ - HeadBlockHash: t.clMock.latestPayloadBuilt.BlockHash, - }, nil) - r.expectPayloadStatus(test.Valid) -]# + let fcState = ForkchoiceStateV1(headBlockHash: env.clMock.latestPayloadBuilt.blockHash) + let r = env.client.forkchoiceUpdatedV2(fcState) + r.testFCU(valid) diff --git a/nimbus/beacon/execution_types.nim b/nimbus/beacon/execution_types.nim index 3fe5c73c9..a591ace22 100644 --- a/nimbus/beacon/execution_types.nim +++ b/nimbus/beacon/execution_types.nim @@ -52,6 +52,11 @@ type Option[PayloadAttributesV2] | Option[PayloadAttributesV3] + GetPayloadResponse* = object + executionPayload*: ExecutionPayload + blockValue*: Option[UInt256] + blobsBundle*: Option[BlobsBundleV1] + Version* {.pure.} = enum V1 V2 @@ -59,21 +64,27 @@ type func version*(payload: ExecutionPayload): Version = if payload.blobGasUsed.isSome and payload.excessBlobGas.isSome: - return Version.V3 - - if payload.withdrawals.isSome: - return Version.V2 - - Version.V1 + Version.V3 + elif payload.withdrawals.isSome: + Version.V2 + else: + Version.V1 func version*(attr: PayloadAttributes): Version = if attr.parentBeaconBlockRoot.isSome: - return Version.V3 + Version.V3 + elif attr.withdrawals.isSome: + Version.V2 + else: + Version.V1 - if attr.withdrawals.isSome: - return Version.V2 - - Version.V1 +func version*(res: GetPayloadResponse): Version = + if res.blobsBundle.isSome: + Version.V3 + elif res.blockValue.isSome: + Version.V2 + else: + Version.V1 func V1V2*(attr: PayloadAttributes): PayloadAttributesV1OrV2 = PayloadAttributesV1OrV2( @@ -107,6 +118,22 @@ func V3*(attr: PayloadAttributes): PayloadAttributesV3 = parentBeaconBlockRoot: attr.parentBeaconBlockRoot.get ) +func V1*(attr: Option[PayloadAttributes]): Option[PayloadAttributesV1] = + if attr.isNone: + return none(PayloadAttributesV1) + some(attr.get.V1) + +when false: + func V2*(attr: Option[PayloadAttributes]): Option[PayloadAttributesV2] = + if attr.isNone: + return none(PayloadAttributesV2) + some(attr.get.V2) + + func V3*(attr: Option[PayloadAttributes]): Option[PayloadAttributesV3] = + if attr.isNone: + return none(PayloadAttributesV3) + some(attr.get.V3) + func payloadAttributes*(attr: PayloadAttributesV1): PayloadAttributes = PayloadAttributes( timestamp: attr.timestamp, @@ -308,3 +335,19 @@ func executionPayload*(p: ExecutionPayloadV1OrV2): ExecutionPayload = transactions: p.transactions, withdrawals: p.withdrawals ) + +func V1*(res: GetPayloadResponse): ExecutionPayloadV1 = + res.executionPayload.V1 + +func V2*(res: GetPayloadResponse): GetPayloadV2Response = + GetPayloadV2Response( + executionPayload: res.executionPayload.V1V2, + blockValue: res.blockValue.get + ) + +func V3*(res: GetPayloadResponse): GetPayloadV3Response = + GetPayloadV3Response( + executionPayload: res.executionPayload.V3, + blockValue: res.blockValue.get, + blobsBundle: res.blobsBundle.get + ) diff --git a/nimbus/beacon/payload_queue.nim b/nimbus/beacon/payload_queue.nim index 2f08c1809..523e52b27 100644 --- a/nimbus/beacon/payload_queue.nim +++ b/nimbus/beacon/payload_queue.nim @@ -21,7 +21,7 @@ const # maxTrackedHeaders is the maximum number of executed payloads the execution # engine tracks before evicting old ones. Ideally we should only ever track # the latest one; but have a slight wiggle room for non-ideal conditions. - MaxTrackedHeaders = 10 + MaxTrackedHeaders = 96 type QueueItem[T] = object diff --git a/nimbus/beacon/web3_eth_conv.nim b/nimbus/beacon/web3_eth_conv.nim index 104f4dcc6..c597dfe94 100644 --- a/nimbus/beacon/web3_eth_conv.nim +++ b/nimbus/beacon/web3_eth_conv.nim @@ -56,7 +56,7 @@ proc `$`*(x: Web3Quantity): string = proc `$`*(x: Web3Address): string = distinctBase(x).toHex - + proc short*(x: Web3Hash): string = let z = common.Hash256(data: distinctBase x) short(z) @@ -168,6 +168,9 @@ func w3Qty*(x: common.EthTime): Web3Quantity = func w3Qty*(x: common.EthTime, y: int): Web3Quantity = Web3Quantity(x.toUnix + y.int64) +func w3Qty*(x: Web3Quantity, y: int): Web3Quantity = + Web3Quantity(x.uint64 + y.uint64) + func w3Qty*(x: Option[uint64]): Option[Web3Quantity] = if x.isNone: none(Web3Quantity) else: some(Web3Quantity x.get) diff --git a/nimbus/core/tx_pool/tx_tasks/tx_classify.nim b/nimbus/core/tx_pool/tx_tasks/tx_classify.nim index 958b70ce2..b9bc6d7aa 100644 --- a/nimbus/core/tx_pool/tx_tasks/tx_classify.nim +++ b/nimbus/core/tx_pool/tx_tasks/tx_classify.nim @@ -114,12 +114,17 @@ proc txFeesCovered(xp: TxPoolRef; item: TxItemRef): bool = return false true +import stew/byteutils +import ../../../utils/debug + proc txCostInBudget(xp: TxPoolRef; item: TxItemRef): bool = ## Check whether the worst case expense is covered by the price budget, let balance = xp.chain.getBalance(item.sender) gasCost = item.tx.gasCost if balance < gasCost: + debugEcho "nonce: ", item.tx.nonce, " ", balance, " ", gasCost, " ", item.sender.toHex + debugEcho debug(item.tx) debug "invalid tx: not enough cash for gas", available = balance, require = gasCost diff --git a/nimbus/sync/beacon/worker.nim b/nimbus/sync/beacon/worker.nim index 1f2a1f884..ab633a342 100644 --- a/nimbus/sync/beacon/worker.nim +++ b/nimbus/sync/beacon/worker.nim @@ -18,7 +18,6 @@ import ".."/[protocol, sync_desc], ./worker_desc, ./skeleton_main, - ./skeleton_utils, ./beacon_impl logScope: