From 4ee14eca1de3857af218311cf466970999cf0d40 Mon Sep 17 00:00:00 2001 From: Gabriel-Trintinalia Date: Thu, 4 Jul 2024 12:21:25 +1000 Subject: [PATCH] Implement System Calls (#7263) Signed-off-by: Gabriel-Trintinalia --- .../blockcreation/AbstractBlockCreator.java | 16 +- .../AbstractBlockCreatorTest.java | 4 +- .../mainnet/AbstractBlockProcessor.java | 14 +- .../mainnet/MainnetTransactionProcessor.java | 2 +- .../ethereum/mainnet/SystemCallProcessor.java | 141 ++++++++++++ .../WithdrawalRequestContractHelper.java | 197 ----------------- .../AbstractSystemCallRequestProcessor.java | 102 +++++++++ .../requests/DepositRequestProcessor.java | 7 +- .../requests/ProcessRequestContext.java | 32 +++ .../mainnet/requests/RequestProcessor.java | 6 +- .../requests/RequestProcessorCoordinator.java | 7 +- .../requests/WithdrawalRequestProcessor.java | 63 ++++-- .../requests/WithdrawalRequestValidator.java | 6 +- .../mainnet/SystemCallProcessorTest.java | 113 ++++++++++ .../WithdrawalRequestContractHelperTest.java | 205 ------------------ ...ithdrawalRequestValidatorTestFixtures.java | 2 +- .../hyperledger/besu/evmtool/T8nExecutor.java | 12 +- 17 files changed, 487 insertions(+), 442 deletions(-) create mode 100644 ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/SystemCallProcessor.java delete mode 100644 ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestContractHelper.java create mode 100644 ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/AbstractSystemCallRequestProcessor.java create mode 100644 ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/ProcessRequestContext.java create mode 100644 ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/SystemCallProcessorTest.java delete mode 100644 ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestContractHelperTest.java diff --git a/ethereum/blockcreation/src/main/java/org/hyperledger/besu/ethereum/blockcreation/AbstractBlockCreator.java b/ethereum/blockcreation/src/main/java/org/hyperledger/besu/ethereum/blockcreation/AbstractBlockCreator.java index 15f3b16d89a..78c32842ff7 100644 --- a/ethereum/blockcreation/src/main/java/org/hyperledger/besu/ethereum/blockcreation/AbstractBlockCreator.java +++ b/ethereum/blockcreation/src/main/java/org/hyperledger/besu/ethereum/blockcreation/AbstractBlockCreator.java @@ -50,7 +50,9 @@ import org.hyperledger.besu.ethereum.mainnet.feemarket.BaseFeeMarket; import org.hyperledger.besu.ethereum.mainnet.feemarket.ExcessBlobGasCalculator; import org.hyperledger.besu.ethereum.mainnet.feemarket.FeeMarket; +import org.hyperledger.besu.ethereum.mainnet.requests.ProcessRequestContext; import org.hyperledger.besu.ethereum.mainnet.requests.RequestProcessorCoordinator; +import org.hyperledger.besu.ethereum.vm.CachingBlockHashLookup; import org.hyperledger.besu.evm.account.MutableAccount; import org.hyperledger.besu.evm.worldstate.WorldUpdater; import org.hyperledger.besu.plugin.services.exception.StorageException; @@ -241,10 +243,18 @@ protected BlockCreationResult createBlock( // EIP-7685: process EL requests final Optional requestProcessor = newProtocolSpec.getRequestProcessorCoordinator(); + + ProcessRequestContext context = + new ProcessRequestContext( + processableBlockHeader, + disposableWorldState, + newProtocolSpec, + transactionResults.getReceipts(), + new CachingBlockHashLookup(processableBlockHeader, protocolContext.getBlockchain()), + operationTracer); + Optional> maybeRequests = - requestProcessor.flatMap( - processor -> - processor.process(disposableWorldState, transactionResults.getReceipts())); + requestProcessor.flatMap(processor -> processor.process(context)); throwIfStopped(); diff --git a/ethereum/blockcreation/src/test/java/org/hyperledger/besu/ethereum/blockcreation/AbstractBlockCreatorTest.java b/ethereum/blockcreation/src/test/java/org/hyperledger/besu/ethereum/blockcreation/AbstractBlockCreatorTest.java index 856bf874b20..68d13987bb3 100644 --- a/ethereum/blockcreation/src/test/java/org/hyperledger/besu/ethereum/blockcreation/AbstractBlockCreatorTest.java +++ b/ethereum/blockcreation/src/test/java/org/hyperledger/besu/ethereum/blockcreation/AbstractBlockCreatorTest.java @@ -77,6 +77,7 @@ import org.hyperledger.besu.ethereum.mainnet.feemarket.CancunFeeMarket; import org.hyperledger.besu.ethereum.mainnet.requests.DepositRequestProcessor; import org.hyperledger.besu.ethereum.mainnet.requests.DepositRequestValidator; +import org.hyperledger.besu.ethereum.mainnet.requests.ProcessRequestContext; import org.hyperledger.besu.ethereum.mainnet.requests.RequestProcessorCoordinator; import org.hyperledger.besu.ethereum.mainnet.requests.RequestsValidatorCoordinator; import org.hyperledger.besu.evm.internal.EvmConfiguration; @@ -135,7 +136,8 @@ void findDepositRequestsFromReceipts() { final List expectedDepositRequests = List.of(expectedDepositRequest); var depositRequestsFromReceipts = - new DepositRequestProcessor(DEFAULT_DEPOSIT_CONTRACT_ADDRESS).process(null, receipts); + new DepositRequestProcessor(DEFAULT_DEPOSIT_CONTRACT_ADDRESS) + .process(new ProcessRequestContext(null, null, null, receipts, null, null)); assertThat(depositRequestsFromReceipts.get()).isEqualTo(expectedDepositRequests); } diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/AbstractBlockProcessor.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/AbstractBlockProcessor.java index 888bc848f15..8ecfa453d9f 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/AbstractBlockProcessor.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/AbstractBlockProcessor.java @@ -29,6 +29,7 @@ import org.hyperledger.besu.ethereum.core.Transaction; import org.hyperledger.besu.ethereum.core.TransactionReceipt; import org.hyperledger.besu.ethereum.core.Withdrawal; +import org.hyperledger.besu.ethereum.mainnet.requests.ProcessRequestContext; import org.hyperledger.besu.ethereum.mainnet.requests.RequestProcessorCoordinator; import org.hyperledger.besu.ethereum.privacy.storage.PrivateMetadataUpdater; import org.hyperledger.besu.ethereum.processing.TransactionProcessingResult; @@ -107,6 +108,7 @@ public BlockProcessingResult processBlock( final ProtocolSpec protocolSpec = protocolSchedule.getByBlockHeader(blockHeader); protocolSpec.getBlockHashProcessor().processBlockHashes(blockchain, worldState, blockHeader); + final BlockHashLookup blockHashLookup = new CachingBlockHashLookup(blockHeader, blockchain); for (final Transaction transaction : transactions) { if (!hasAvailableBlockBudget(blockHeader, transaction, currentGasUsed)) { @@ -115,7 +117,6 @@ public BlockProcessingResult processBlock( final WorldUpdater worldStateUpdater = worldState.updater(); - final BlockHashLookup blockHashLookup = new CachingBlockHashLookup(blockHeader, blockchain); final Address miningBeneficiary = miningBeneficiaryCalculator.calculateBeneficiary(blockHeader); @@ -197,7 +198,16 @@ public BlockProcessingResult processBlock( protocolSpec.getRequestProcessorCoordinator(); Optional> maybeRequests = Optional.empty(); if (requestProcessor.isPresent()) { - maybeRequests = requestProcessor.get().process(worldState, receipts); + ProcessRequestContext context = + new ProcessRequestContext( + blockHeader, + worldState, + protocolSpec, + receipts, + blockHashLookup, + OperationTracer.NO_TRACING); + + maybeRequests = requestProcessor.get().process(context); } if (!rewardCoinbase(worldState, blockHeader, ommers, skipZeroBlockRewards)) { diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/MainnetTransactionProcessor.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/MainnetTransactionProcessor.java index d982265f242..53801d14cd3 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/MainnetTransactionProcessor.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/MainnetTransactionProcessor.java @@ -563,7 +563,7 @@ public void process(final MessageFrame frame, final OperationTracer operationTra executor.process(frame, operationTracer); } - private AbstractMessageProcessor getMessageProcessor(final MessageFrame.Type type) { + public AbstractMessageProcessor getMessageProcessor(final MessageFrame.Type type) { return switch (type) { case MESSAGE_CALL -> messageCallProcessor; case CONTRACT_CREATION -> contractCreationProcessor; diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/SystemCallProcessor.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/SystemCallProcessor.java new file mode 100644 index 00000000000..f74de79442f --- /dev/null +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/SystemCallProcessor.java @@ -0,0 +1,141 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + * + * SPDX-License-Identifier: Apache-2.0 + */ +package org.hyperledger.besu.ethereum.mainnet; + +import static org.hyperledger.besu.evm.frame.MessageFrame.DEFAULT_MAX_STACK_SIZE; + +import org.hyperledger.besu.datatypes.Address; +import org.hyperledger.besu.datatypes.Wei; +import org.hyperledger.besu.ethereum.core.ProcessableBlockHeader; +import org.hyperledger.besu.evm.account.Account; +import org.hyperledger.besu.evm.code.CodeV0; +import org.hyperledger.besu.evm.frame.MessageFrame; +import org.hyperledger.besu.evm.operation.BlockHashOperation; +import org.hyperledger.besu.evm.processor.AbstractMessageProcessor; +import org.hyperledger.besu.evm.tracing.OperationTracer; +import org.hyperledger.besu.evm.worldstate.WorldUpdater; + +import java.util.Deque; +import java.util.Optional; + +import org.apache.tuweni.bytes.Bytes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SystemCallProcessor { + private static final Logger LOG = LoggerFactory.getLogger(SystemCallProcessor.class); + + /** The system address */ + static final Address SYSTEM_ADDRESS = + Address.fromHexString("0xfffffffffffffffffffffffffffffffffffffffe"); + + private final MainnetTransactionProcessor mainnetTransactionProcessor; + + public SystemCallProcessor(final MainnetTransactionProcessor mainnetTransactionProcessor) { + this.mainnetTransactionProcessor = mainnetTransactionProcessor; + } + + /** + * Processes a system call to a specified address, using the provided world state, block header, + * operation tracer, and block hash lookup. + * + * @param callAddress the address to call. + * @param worldState the current world state. + * @param blockHeader the current block header. + * @param operationTracer the operation tracer for tracing EVM operations. + * @param blockHashLookup the block hash lookup function. + * @return the output data from the call + */ + public Bytes process( + final Address callAddress, + final WorldUpdater worldState, + final ProcessableBlockHeader blockHeader, + final OperationTracer operationTracer, + final BlockHashOperation.BlockHashLookup blockHashLookup) { + + // if no code exists at CALL_ADDRESS, the call must fail silently + final Account maybeContract = worldState.get(callAddress); + if (maybeContract == null) { + LOG.trace("System call address not found {}", callAddress); + return null; + } + + final AbstractMessageProcessor messageProcessor = + mainnetTransactionProcessor.getMessageProcessor(MessageFrame.Type.MESSAGE_CALL); + final MessageFrame initialFrame = + createCallFrame(callAddress, worldState, blockHeader, blockHashLookup); + + return processFrame(initialFrame, messageProcessor, operationTracer, worldState); + } + + private Bytes processFrame( + final MessageFrame frame, + final AbstractMessageProcessor processor, + final OperationTracer tracer, + final WorldUpdater updater) { + + if (!frame.getCode().isValid()) { + throw new RuntimeException("System call did not execute to completion - opcode invalid"); + } + + Deque stack = frame.getMessageFrameStack(); + while (!stack.isEmpty()) { + processor.process(stack.peekFirst(), tracer); + } + + if (frame.getState() == MessageFrame.State.COMPLETED_SUCCESS) { + updater.commit(); + return frame.getOutputData(); + } + + // the call must execute to completion + throw new RuntimeException("System call did not execute to completion"); + } + + private MessageFrame createCallFrame( + final Address callAddress, + final WorldUpdater worldUpdater, + final ProcessableBlockHeader blockHeader, + final BlockHashOperation.BlockHashLookup blockHashLookup) { + + final Optional maybeContract = Optional.ofNullable(worldUpdater.get(callAddress)); + final AbstractMessageProcessor processor = + mainnetTransactionProcessor.getMessageProcessor(MessageFrame.Type.MESSAGE_CALL); + + return MessageFrame.builder() + .maxStackSize(DEFAULT_MAX_STACK_SIZE) + .worldUpdater(worldUpdater) + .initialGas(30_000_000L) + .originator(SYSTEM_ADDRESS) + .gasPrice(Wei.ZERO) + .blobGasPrice(Wei.ZERO) + .value(Wei.ZERO) + .apparentValue(Wei.ZERO) + .blockValues(blockHeader) + .completer(__ -> {}) + .miningBeneficiary(Address.ZERO) // Confirm this + .type(MessageFrame.Type.MESSAGE_CALL) + .address(callAddress) + .contract(callAddress) + .inputData(Bytes.EMPTY) + .sender(SYSTEM_ADDRESS) + .blockHashLookup(blockHashLookup) + .code( + maybeContract + .map(c -> processor.getCodeFromEVM(c.getCodeHash(), c.getCode())) + .orElse(CodeV0.EMPTY_CODE)) + .build(); + } +} diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestContractHelper.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestContractHelper.java deleted file mode 100644 index c0b7302e867..00000000000 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestContractHelper.java +++ /dev/null @@ -1,197 +0,0 @@ -/* - * Copyright contributors to Hyperledger Besu. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - * - * SPDX-License-Identifier: Apache-2.0 - */ -package org.hyperledger.besu.ethereum.mainnet; - -import org.hyperledger.besu.datatypes.Address; -import org.hyperledger.besu.datatypes.BLSPublicKey; -import org.hyperledger.besu.datatypes.GWei; -import org.hyperledger.besu.datatypes.Hash; -import org.hyperledger.besu.ethereum.core.MutableWorldState; -import org.hyperledger.besu.ethereum.core.WithdrawalRequest; -import org.hyperledger.besu.evm.account.Account; -import org.hyperledger.besu.evm.account.MutableAccount; -import org.hyperledger.besu.evm.worldstate.WorldUpdater; - -import java.util.ArrayList; -import java.util.List; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.tuweni.bytes.Bytes; -import org.apache.tuweni.units.bigints.UInt256; -import org.apache.tuweni.units.bigints.UInt64; - -/** - * Helper for interacting with the Validator Withdrawal Request Contract - * (https://eips.ethereum.org/EIPS/eip-7002) - * - *

TODO: Please note that this is not the spec-way of interacting with the Validator Withdrawal - * Request contract. See https://github.com/hyperledger/besu/issues/6918 for more information. - */ -public class WithdrawalRequestContractHelper { - - public static final Address WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS = - Address.fromHexString("0x00A3ca265EBcb825B45F985A16CEFB49958cE017"); - - /** private constructor to prevent instantiations */ - private WithdrawalRequestContractHelper() {} - - @VisibleForTesting - // Storage slot to store the difference between number of withdrawal requests since last block and - // target withdrawal requests - // per block - static final UInt256 EXCESS_WITHDRAWAL_REQUESTS_STORAGE_SLOT = UInt256.valueOf(0L); - - @VisibleForTesting - // Storage slot to store the number of withdrawal requests added since last block - static final UInt256 WITHDRAWAL_REQUEST_COUNT_STORAGE_SLOT = UInt256.valueOf(1L); - - @VisibleForTesting - static final UInt256 WITHDRAWAL_REQUEST_QUEUE_HEAD_STORAGE_SLOT = UInt256.valueOf(2L); - - @VisibleForTesting - static final UInt256 WITHDRAWAL_REQUEST_QUEUE_TAIL_STORAGE_SLOT = UInt256.valueOf(3L); - - private static final UInt256 WITHDRAWAL_REQUEST_QUEUE_STORAGE_OFFSET = UInt256.valueOf(4L); - - // How many slots each withdrawal request occupies in the account state - private static final int WITHDRAWAL_REQUEST_STORAGE_SLOT_SIZE = 3; - - public static final int MAX_WITHDRAWAL_REQUESTS_PER_BLOCK = 16; - - private static final int TARGET_WITHDRAWAL_REQUESTS_PER_BLOCK = 2; - - private static final UInt256 INITIAL_EXCESS_WITHDRAWAL_REQUESTS_STORAGE_SLOT = - UInt256.valueOf(1181); - - // TODO-lucas Add MIN_WITHDRAWAL_REQUEST_FEE and WITHDRAWAL_REQUEST_FEE_UPDATE_FRACTION - - /* - Pop the expected list of withdrawal requests from the smart contract, updating the queue pointers and other - control variables in the contract state. - */ - public static List popWithdrawalRequestsFromQueue( - final MutableWorldState mutableWorldState) { - final WorldUpdater worldUpdater = mutableWorldState.updater(); - final MutableAccount account = worldUpdater.getAccount(WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS); - if (account == null || Hash.EMPTY.equals(account.getCodeHash())) { - return List.of(); - } - - final List withdrawalRequests = dequeueWithdrawalRequests(account); - updateExcessWithdrawalRequests(account); - resetWithdrawalRequestsCount(account); - - worldUpdater.commit(); - - return withdrawalRequests; - } - - private static List dequeueWithdrawalRequests(final MutableAccount account) { - final UInt256 queueHeadIndex = - account.getStorageValue(WITHDRAWAL_REQUEST_QUEUE_HEAD_STORAGE_SLOT); - final UInt256 queueTailIndex = - account.getStorageValue(WITHDRAWAL_REQUEST_QUEUE_TAIL_STORAGE_SLOT); - - final List withdrawalRequests = - peekExpectedWithdrawalRequests(account, queueHeadIndex, queueTailIndex); - - final UInt256 newQueueHeadIndex = queueHeadIndex.plus(withdrawalRequests.size()); - if (newQueueHeadIndex.equals(queueTailIndex)) { - // Queue is empty, reset queue pointers - account.setStorageValue(WITHDRAWAL_REQUEST_QUEUE_HEAD_STORAGE_SLOT, UInt256.valueOf(0L)); - account.setStorageValue(WITHDRAWAL_REQUEST_QUEUE_TAIL_STORAGE_SLOT, UInt256.valueOf(0L)); - } else { - account.setStorageValue(WITHDRAWAL_REQUEST_QUEUE_HEAD_STORAGE_SLOT, newQueueHeadIndex); - } - - return withdrawalRequests; - } - - /* - ;; Each stack element has the following layout: - ;; - ;; A: addr - ;; 0x00 | 00 00 00 00 00 00 00 00 00 00 00 00 aa aa aa aa - ;; 0x10 | aa aa aa aa aa aa aa aa aa aa aa aa aa aa aa aa - ;; - ;; B: pk[0:32] - ;; 0x00 | bb bb bb bb bb bb bb bb bb bb bb bb bb bb bb bb - ;; 0x10 | bb bb bb bb bb bb bb bb bb bb bb bb bb bb bb bb - ;; - ;; C: pk[32:48] ++ am[0:8] -> pk2_am - ;; 0x00 | cc cc cc cc cc cc cc cc cc cc cc cc cc cc cc cc - ;; 0x10 | dd dd dd dd dd dd dd dd 00 00 00 00 00 00 00 00 - ;; - ;; To get these three stack elements into the correct contiguous format, it is - ;; necessary to combine them in the follow form: - ;; - ;; (A[12:32] ++ B[0:12], B[12:32] ++ C[0:12], C[12:24]) - */ - private static List peekExpectedWithdrawalRequests( - final Account account, final UInt256 queueHeadIndex, final UInt256 queueTailIndex) { - final long numRequestsInQueue = queueTailIndex.subtract(queueHeadIndex).toLong(); - final long numRequestsDequeued = - Long.min(numRequestsInQueue, MAX_WITHDRAWAL_REQUESTS_PER_BLOCK); - - final List withdrawalRequests = new ArrayList<>(); - - for (int i = 0; i < numRequestsDequeued; i++) { - final UInt256 queueStorageSlot = - WITHDRAWAL_REQUEST_QUEUE_STORAGE_OFFSET.plus( - queueHeadIndex.plus(i).multiply(WITHDRAWAL_REQUEST_STORAGE_SLOT_SIZE)); - final Address sourceAddress = - Address.wrap(account.getStorageValue(queueStorageSlot).toBytes().slice(12, 20)); - final BLSPublicKey validatorPubkey = - BLSPublicKey.wrap( - Bytes.concatenate( - account - .getStorageValue(queueStorageSlot.plus(1)) - .toBytes() - .slice(0, 32), // no need to slice - account.getStorageValue(queueStorageSlot.plus(2)).toBytes().slice(0, 16))); - final UInt64 amount = - UInt64.fromBytes(account.getStorageValue(queueStorageSlot.plus(2)).slice(16, 8)); - - withdrawalRequests.add( - new WithdrawalRequest(sourceAddress, validatorPubkey, GWei.of(amount))); - } - - return withdrawalRequests; - } - - private static void updateExcessWithdrawalRequests(final MutableAccount account) { - UInt256 previousExcessRequests = - account.getStorageValue(EXCESS_WITHDRAWAL_REQUESTS_STORAGE_SLOT); - - if (previousExcessRequests.equals(INITIAL_EXCESS_WITHDRAWAL_REQUESTS_STORAGE_SLOT)) { - previousExcessRequests = UInt256.ZERO; - } - - final UInt256 requestsCount = account.getStorageValue(WITHDRAWAL_REQUEST_COUNT_STORAGE_SLOT); - - UInt256 newExcessRequests = UInt256.valueOf(0L); - if (previousExcessRequests.plus(requestsCount).toLong() - > TARGET_WITHDRAWAL_REQUESTS_PER_BLOCK) { - newExcessRequests = - previousExcessRequests.plus(requestsCount).subtract(TARGET_WITHDRAWAL_REQUESTS_PER_BLOCK); - } - - account.setStorageValue(EXCESS_WITHDRAWAL_REQUESTS_STORAGE_SLOT, newExcessRequests); - } - - private static void resetWithdrawalRequestsCount(final MutableAccount account) { - account.setStorageValue(WITHDRAWAL_REQUEST_COUNT_STORAGE_SLOT, UInt256.valueOf(0L)); - } -} diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/AbstractSystemCallRequestProcessor.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/AbstractSystemCallRequestProcessor.java new file mode 100644 index 00000000000..a7d959f4b98 --- /dev/null +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/AbstractSystemCallRequestProcessor.java @@ -0,0 +1,102 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + * + * SPDX-License-Identifier: Apache-2.0 + */ +package org.hyperledger.besu.ethereum.mainnet.requests; + +import org.hyperledger.besu.datatypes.Address; +import org.hyperledger.besu.ethereum.core.Request; +import org.hyperledger.besu.ethereum.mainnet.SystemCallProcessor; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import org.apache.tuweni.bytes.Bytes; + +/** + * Abstract base class for processing system call requests. + * + * @param The type of request to be processed. + */ +public abstract class AbstractSystemCallRequestProcessor + implements RequestProcessor { + + /** + * Processes a system call and converts the result into requests of type T. + * + * @param context The request context being processed. + * @return An {@link Optional} containing a list of {@link T} objects if any are found + */ + @Override + public Optional> process(final ProcessRequestContext context) { + + SystemCallProcessor systemCallProcessor = + new SystemCallProcessor(context.protocolSpec().getTransactionProcessor()); + + Bytes systemCallOutput = + systemCallProcessor.process( + getCallAddress(), + context.mutableWorldState().updater(), + context.blockHeader(), + context.operationTracer(), + context.blockHashLookup()); + + List requests = parseRequests(systemCallOutput); + return Optional.ofNullable(requests); + } + + /** + * Parses the provided bytes into a list of {@link T} objects. + * + * @param bytes The bytes representing requests. + * @return A list of parsed {@link T} objects. + */ + protected List parseRequests(final Bytes bytes) { + if (bytes == null) { + return null; + } + final List requests = new ArrayList<>(); + if (bytes.isEmpty()) { + return requests; + } + int count = bytes.size() / getRequestBytesSize(); + for (int i = 0; i < count; i++) { + Bytes requestBytes = bytes.slice(i * getRequestBytesSize(), getRequestBytesSize()); + requests.add(parseRequest(requestBytes)); + } + return requests; + } + + /** + * Parses a single request from the provided bytes. + * + * @param requestBytes The bytes representing a single request. + * @return A parsed {@link T} object. + */ + protected abstract T parseRequest(final Bytes requestBytes); + + /** + * Gets the call address for the specific request type. + * + * @return The call address. + */ + protected abstract Address getCallAddress(); + + /** + * Gets the size of the bytes representing a single request. + * + * @return The size of the bytes representing a single request. + */ + protected abstract int getRequestBytesSize(); +} diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/DepositRequestProcessor.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/DepositRequestProcessor.java index ce3ed6a5f65..8902ecc510a 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/DepositRequestProcessor.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/DepositRequestProcessor.java @@ -16,7 +16,6 @@ import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.ethereum.core.DepositRequest; -import org.hyperledger.besu.ethereum.core.MutableWorldState; import org.hyperledger.besu.ethereum.core.Request; import org.hyperledger.besu.ethereum.core.TransactionReceipt; import org.hyperledger.besu.ethereum.core.encoding.DepositRequestDecoder; @@ -39,12 +38,12 @@ public DepositRequestProcessor(final Address depositContractAddress) { } @Override - public Optional> process( - final MutableWorldState ignored, final List transactionReceipts) { + public Optional> process(final ProcessRequestContext context) { if (depositContractAddress.isEmpty()) { return Optional.empty(); } - List depositRequests = findDepositRequestsFromReceipts(transactionReceipts); + List depositRequests = + findDepositRequestsFromReceipts(context.transactionReceipts()); return Optional.of(depositRequests); } diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/ProcessRequestContext.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/ProcessRequestContext.java new file mode 100644 index 00000000000..63f4a8d5144 --- /dev/null +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/ProcessRequestContext.java @@ -0,0 +1,32 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + * + * SPDX-License-Identifier: Apache-2.0 + */ +package org.hyperledger.besu.ethereum.mainnet.requests; + +import org.hyperledger.besu.ethereum.core.MutableWorldState; +import org.hyperledger.besu.ethereum.core.ProcessableBlockHeader; +import org.hyperledger.besu.ethereum.core.TransactionReceipt; +import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec; +import org.hyperledger.besu.evm.operation.BlockHashOperation.BlockHashLookup; +import org.hyperledger.besu.evm.tracing.OperationTracer; + +import java.util.List; + +public record ProcessRequestContext( + ProcessableBlockHeader blockHeader, + MutableWorldState mutableWorldState, + ProtocolSpec protocolSpec, + List transactionReceipts, + BlockHashLookup blockHashLookup, + OperationTracer operationTracer) {} diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/RequestProcessor.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/RequestProcessor.java index d09b3c47d18..55f3cd41788 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/RequestProcessor.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/RequestProcessor.java @@ -14,15 +14,11 @@ */ package org.hyperledger.besu.ethereum.mainnet.requests; -import org.hyperledger.besu.ethereum.core.MutableWorldState; import org.hyperledger.besu.ethereum.core.Request; -import org.hyperledger.besu.ethereum.core.TransactionReceipt; import java.util.List; import java.util.Optional; public interface RequestProcessor { - Optional> process( - final MutableWorldState mutableWorldState, - final List transactionReceipts); + Optional> process(final ProcessRequestContext context); } diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/RequestProcessorCoordinator.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/RequestProcessorCoordinator.java index b72674b4d24..b98274729d5 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/RequestProcessorCoordinator.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/RequestProcessorCoordinator.java @@ -15,9 +15,7 @@ package org.hyperledger.besu.ethereum.mainnet.requests; import org.hyperledger.besu.datatypes.RequestType; -import org.hyperledger.besu.ethereum.core.MutableWorldState; import org.hyperledger.besu.ethereum.core.Request; -import org.hyperledger.besu.ethereum.core.TransactionReceipt; import java.util.ArrayList; import java.util.List; @@ -39,11 +37,10 @@ private RequestProcessorCoordinator( this.processors = processors; } - public Optional> process( - final MutableWorldState mutableWorldState, final List receipts) { + public Optional> process(final ProcessRequestContext context) { List requests = null; for (RequestProcessor requestProcessor : processors.values()) { - var r = requestProcessor.process(mutableWorldState, receipts); + var r = requestProcessor.process(context); if (r.isPresent()) { if (requests == null) { requests = new ArrayList<>(); diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/WithdrawalRequestProcessor.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/WithdrawalRequestProcessor.java index 9803f23f3f9..b230a6d6103 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/WithdrawalRequestProcessor.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/WithdrawalRequestProcessor.java @@ -14,25 +14,60 @@ */ package org.hyperledger.besu.ethereum.mainnet.requests; -import org.hyperledger.besu.ethereum.core.MutableWorldState; -import org.hyperledger.besu.ethereum.core.Request; -import org.hyperledger.besu.ethereum.core.TransactionReceipt; +import org.hyperledger.besu.datatypes.Address; +import org.hyperledger.besu.datatypes.BLSPublicKey; +import org.hyperledger.besu.datatypes.GWei; import org.hyperledger.besu.ethereum.core.WithdrawalRequest; -import org.hyperledger.besu.ethereum.mainnet.WithdrawalRequestContractHelper; -import java.util.List; -import java.util.Optional; +import org.apache.tuweni.bytes.Bytes; +import org.apache.tuweni.units.bigints.UInt64; -public class WithdrawalRequestProcessor implements RequestProcessor { +/** Processor for handling withdrawal requests. */ +public class WithdrawalRequestProcessor + extends AbstractSystemCallRequestProcessor { + + public static final Address WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS = + Address.fromHexString("0x00A3ca265EBcb825B45F985A16CEFB49958cE017"); + + private static final int ADDRESS_BYTES = 20; + private static final int PUBLIC_KEY_BYTES = 48; + private static final int AMOUNT_BYTES = 8; + private static final int WITHDRAWAL_REQUEST_BYTES_SIZE = + ADDRESS_BYTES + PUBLIC_KEY_BYTES + AMOUNT_BYTES; + + /** + * Gets the call address for withdrawal requests. + * + * @return The call address. + */ @Override - public Optional> process( - final MutableWorldState mutableWorldState, - final List transactionReceipts) { + protected Address getCallAddress() { + return WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS; + } - List withdrawalRequests = - WithdrawalRequestContractHelper.popWithdrawalRequestsFromQueue(mutableWorldState).stream() - .toList(); + /** + * Gets the size of the bytes representing a single withdrawal request. + * + * @return The size of the bytes representing a single withdrawal request. + */ + @Override + protected int getRequestBytesSize() { + return WITHDRAWAL_REQUEST_BYTES_SIZE; + } - return Optional.of(withdrawalRequests); + /** + * Parses a single withdrawal request from the provided bytes. + * + * @param requestBytes The bytes representing a single withdrawal request. + * @return A parsed {@link WithdrawalRequest} object. + */ + @Override + protected WithdrawalRequest parseRequest(final Bytes requestBytes) { + final Address sourceAddress = Address.wrap(requestBytes.slice(0, ADDRESS_BYTES)); + final BLSPublicKey validatorPublicKey = + BLSPublicKey.wrap(requestBytes.slice(ADDRESS_BYTES, PUBLIC_KEY_BYTES)); + final UInt64 amount = + UInt64.fromBytes(requestBytes.slice(ADDRESS_BYTES + PUBLIC_KEY_BYTES, AMOUNT_BYTES)); + return new WithdrawalRequest(sourceAddress, validatorPublicKey, GWei.of(amount)); } } diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/WithdrawalRequestValidator.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/WithdrawalRequestValidator.java index d5f04e6ef74..fc108b798f3 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/WithdrawalRequestValidator.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/WithdrawalRequestValidator.java @@ -21,7 +21,6 @@ import org.hyperledger.besu.ethereum.core.Request; import org.hyperledger.besu.ethereum.core.TransactionReceipt; import org.hyperledger.besu.ethereum.core.WithdrawalRequest; -import org.hyperledger.besu.ethereum.mainnet.WithdrawalRequestContractHelper; import java.util.Collections; import java.util.List; @@ -32,6 +31,8 @@ public class WithdrawalRequestValidator implements RequestValidator { + public static final int MAX_WITHDRAWAL_REQUESTS_PER_BLOCK = 16; + private static final Logger LOG = LoggerFactory.getLogger(WithdrawalRequestValidator.class); private boolean validateWithdrawalRequestParameter( @@ -51,8 +52,7 @@ private boolean validateWithdrawalRequestsInBlock( .orElse(Collections.emptyList()); // TODO Do we need to allow for customization? (e.g. if the value changes in the next fork) - if (withdrawalRequestsInBlock.size() - > WithdrawalRequestContractHelper.MAX_WITHDRAWAL_REQUESTS_PER_BLOCK) { + if (withdrawalRequestsInBlock.size() > MAX_WITHDRAWAL_REQUESTS_PER_BLOCK) { LOG.warn( "Block {} has more than the allowed maximum number of withdrawal requests", blockHash); return false; diff --git a/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/SystemCallProcessorTest.java b/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/SystemCallProcessorTest.java new file mode 100644 index 00000000000..e1d3906e734 --- /dev/null +++ b/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/SystemCallProcessorTest.java @@ -0,0 +1,113 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + * + * SPDX-License-Identifier: Apache-2.0 + */ +package org.hyperledger.besu.ethereum.mainnet; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import org.hyperledger.besu.datatypes.Address; +import org.hyperledger.besu.ethereum.core.InMemoryKeyValueStorageProvider; +import org.hyperledger.besu.ethereum.core.MutableWorldState; +import org.hyperledger.besu.ethereum.core.ProcessableBlockHeader; +import org.hyperledger.besu.evm.frame.MessageFrame; +import org.hyperledger.besu.evm.operation.BlockHashOperation; +import org.hyperledger.besu.evm.processor.AbstractMessageProcessor; +import org.hyperledger.besu.evm.processor.MessageCallProcessor; +import org.hyperledger.besu.evm.tracing.OperationTracer; +import org.hyperledger.besu.evm.worldstate.WorldUpdater; + +import org.apache.tuweni.bytes.Bytes; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class SystemCallProcessorTest { + private static final Address CALL_ADDRESS = Address.fromHexString("0x1"); + private static final Bytes EXPECTED_OUTPUT = Bytes.fromHexString("0x01"); + private ProcessableBlockHeader mockBlockHeader; + private MainnetTransactionProcessor mockTransactionProcessor; + private BlockHashOperation.BlockHashLookup mockBlockHashLookup; + private AbstractMessageProcessor mockMessageCallProcessor; + + @BeforeEach + public void setUp() { + mockBlockHeader = mock(ProcessableBlockHeader.class); + mockTransactionProcessor = mock(MainnetTransactionProcessor.class); + mockMessageCallProcessor = mock(MessageCallProcessor.class); + mockBlockHashLookup = mock(BlockHashOperation.BlockHashLookup.class); + when(mockTransactionProcessor.getMessageProcessor(any())).thenReturn(mockMessageCallProcessor); + } + + @Test + void shouldProcessSuccessfully() { + doAnswer( + invocation -> { + MessageFrame messageFrame = invocation.getArgument(0); + messageFrame.setOutputData(EXPECTED_OUTPUT); + messageFrame.getMessageFrameStack().pop(); + messageFrame.setState(MessageFrame.State.COMPLETED_SUCCESS); + return null; + }) + .when(mockMessageCallProcessor) + .process(any(), any()); + final MutableWorldState worldState = createWorldState(CALL_ADDRESS); + Bytes actualOutput = processSystemCall(worldState); + assertThat(actualOutput).isEqualTo(EXPECTED_OUTPUT); + } + + @Test + void shouldThrowExceptionOnFailedExecution() { + doAnswer( + invocation -> { + MessageFrame messageFrame = invocation.getArgument(0); + messageFrame.getMessageFrameStack().pop(); + messageFrame.setState(MessageFrame.State.COMPLETED_FAILED); + return null; + }) + .when(mockMessageCallProcessor) + .process(any(), any()); + final MutableWorldState worldState = createWorldState(CALL_ADDRESS); + var exception = assertThrows(RuntimeException.class, () -> processSystemCall(worldState)); + assertThat(exception.getMessage()).isEqualTo("System call did not execute to completion"); + } + + @Test + void shouldReturnNullWhenContractDoesNotExist() { + final MutableWorldState worldState = InMemoryKeyValueStorageProvider.createInMemoryWorldState(); + Bytes actualOutput = processSystemCall(worldState); + assertThat(actualOutput).isNull(); + } + + Bytes processSystemCall(final MutableWorldState worldState) { + SystemCallProcessor systemCallProcessor = new SystemCallProcessor(mockTransactionProcessor); + return systemCallProcessor.process( + CALL_ADDRESS, + worldState.updater(), + mockBlockHeader, + OperationTracer.NO_TRACING, + mockBlockHashLookup); + } + + private MutableWorldState createWorldState(final Address address) { + final MutableWorldState worldState = InMemoryKeyValueStorageProvider.createInMemoryWorldState(); + final WorldUpdater updater = worldState.updater(); + updater.getOrCreate(address); + updater.commit(); + return worldState; + } +} diff --git a/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestContractHelperTest.java b/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestContractHelperTest.java deleted file mode 100644 index 7e1e8571350..00000000000 --- a/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestContractHelperTest.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * Copyright contributors to Hyperledger Besu. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - * - * SPDX-License-Identifier: Apache-2.0 - */ -package org.hyperledger.besu.ethereum.mainnet; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.hyperledger.besu.ethereum.core.InMemoryKeyValueStorageProvider.createInMemoryWorldStateArchive; -import static org.hyperledger.besu.ethereum.mainnet.WithdrawalRequestContractHelper.EXCESS_WITHDRAWAL_REQUESTS_STORAGE_SLOT; -import static org.hyperledger.besu.ethereum.mainnet.WithdrawalRequestContractHelper.WITHDRAWAL_REQUEST_COUNT_STORAGE_SLOT; -import static org.hyperledger.besu.ethereum.mainnet.WithdrawalRequestContractHelper.WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS; -import static org.hyperledger.besu.ethereum.mainnet.WithdrawalRequestContractHelper.WITHDRAWAL_REQUEST_QUEUE_HEAD_STORAGE_SLOT; -import static org.hyperledger.besu.ethereum.mainnet.WithdrawalRequestContractHelper.WITHDRAWAL_REQUEST_QUEUE_TAIL_STORAGE_SLOT; - -import org.hyperledger.besu.datatypes.Address; -import org.hyperledger.besu.datatypes.BLSPublicKey; -import org.hyperledger.besu.datatypes.GWei; -import org.hyperledger.besu.ethereum.core.MutableWorldState; -import org.hyperledger.besu.ethereum.core.WithdrawalRequest; -import org.hyperledger.besu.evm.account.MutableAccount; -import org.hyperledger.besu.evm.worldstate.WorldUpdater; - -import java.util.List; -import java.util.stream.IntStream; - -import org.apache.tuweni.bytes.Bytes; -import org.apache.tuweni.bytes.Bytes32; -import org.apache.tuweni.bytes.Bytes48; -import org.apache.tuweni.units.bigints.UInt256; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -class WithdrawalRequestContractHelperTest { - - private MutableWorldState worldState; - private MutableAccount contract; - - @BeforeEach - void setUp() { - worldState = createInMemoryWorldStateArchive().getMutable(); - } - - @Test - void popWithdrawalRequestsFromQueue_ReadWithdrawalRequestsCorrectly() { - final List validatorWithdrawalRequests = - List.of(createExit(), createExit(), createExit()); - loadContractStorage(worldState, validatorWithdrawalRequests); - - final List poppedWithdrawalRequests = - WithdrawalRequestContractHelper.popWithdrawalRequestsFromQueue(worldState); - - assertThat(poppedWithdrawalRequests).isEqualTo(validatorWithdrawalRequests); - } - - @Test - void - popWithdrawalRequestsFromQueue_whenContractCodeIsEmpty_ReturnsEmptyListOfWithdrawalRequests() { - // Create account with empty code - final WorldUpdater updater = worldState.updater(); - updater.createAccount(WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS); - updater.commit(); - - assertThat(WithdrawalRequestContractHelper.popWithdrawalRequestsFromQueue(worldState)) - .isEmpty(); - } - - @Test - void popWithdrawalRequestsFromQueue_WhenMoreWithdrawalRequests_UpdatesQueuePointers() { - // Loading contract with more than 16 WithdrawalRequests - final List validatorWithdrawalRequests = - IntStream.range(0, 30).mapToObj(__ -> createExit()).toList(); - loadContractStorage(worldState, validatorWithdrawalRequests); - // After loading the contract, the WithdrawalRequests count since last block should match the - // size of the list - assertContractStorageValue( - WITHDRAWAL_REQUEST_COUNT_STORAGE_SLOT, validatorWithdrawalRequests.size()); - - final List poppedWithdrawalRequests = - WithdrawalRequestContractHelper.popWithdrawalRequestsFromQueue(worldState); - assertThat(poppedWithdrawalRequests).hasSize(16); - - // Check that queue pointers were updated successfully (head advanced to index 16) - assertContractStorageValue(WITHDRAWAL_REQUEST_QUEUE_HEAD_STORAGE_SLOT, 16); - assertContractStorageValue(WITHDRAWAL_REQUEST_QUEUE_TAIL_STORAGE_SLOT, 30); - - // We had 30 WithdrawalRequests in the queue, and target per block is 2, so we have 28 excess - assertContractStorageValue(EXCESS_WITHDRAWAL_REQUESTS_STORAGE_SLOT, 28); - - // We always reset the WithdrawalRequests count after processing the queue - assertContractStorageValue(WITHDRAWAL_REQUEST_COUNT_STORAGE_SLOT, 0); - } - - @Test - void popWithdrawalRequestsFromQueue_WhenNoMoreWithdrawalRequests_ZeroQueuePointers() { - final List withdrawalRequests = - List.of(createExit(), createExit(), createExit()); - loadContractStorage(worldState, withdrawalRequests); - // After loading the contract, the exit count since last block should match the size of the list - assertContractStorageValue(WITHDRAWAL_REQUEST_COUNT_STORAGE_SLOT, withdrawalRequests.size()); - - final List poppedWithdrawalRequests = - WithdrawalRequestContractHelper.popWithdrawalRequestsFromQueue(worldState); - assertThat(poppedWithdrawalRequests).hasSize(3); - - // Check that queue pointers were updated successfully (head and tail zero because queue is - // empty) - assertContractStorageValue(WITHDRAWAL_REQUEST_QUEUE_HEAD_STORAGE_SLOT, 0); - assertContractStorageValue(WITHDRAWAL_REQUEST_QUEUE_TAIL_STORAGE_SLOT, 0); - - // We had 3 WithdrawalRequests in the queue, target per block is 2, so we have 1 excess - assertContractStorageValue(EXCESS_WITHDRAWAL_REQUESTS_STORAGE_SLOT, 1); - - // We always reset the WithdrawalRequests count after processing the queue - assertContractStorageValue(WITHDRAWAL_REQUEST_COUNT_STORAGE_SLOT, 0); - } - - @Test - void popWithdrawalRequestsFromQueue_WhenNoWithdrawalRequests_DoesNothing() { - // Loading contract with 0 WithdrawalRequests - loadContractStorage(worldState, List.of()); - // After loading storage, we have the WithdrawalRequests count as zero because no - // WithdrawalRequests were added - assertContractStorageValue(WITHDRAWAL_REQUEST_COUNT_STORAGE_SLOT, 0); - - final List poppedWithdrawalRequests = - WithdrawalRequestContractHelper.popWithdrawalRequestsFromQueue(worldState); - assertThat(poppedWithdrawalRequests).isEmpty(); - - // Check that queue pointers are correct (head and tail are zero) - assertContractStorageValue(WITHDRAWAL_REQUEST_QUEUE_HEAD_STORAGE_SLOT, 0); - assertContractStorageValue(WITHDRAWAL_REQUEST_QUEUE_TAIL_STORAGE_SLOT, 0); - - // We had 0 WithdrawalRequests in the queue, and target per block is 2, so we have 0 excess - assertContractStorageValue(EXCESS_WITHDRAWAL_REQUESTS_STORAGE_SLOT, 0); - - // We always reset the exit count after processing the queue - assertContractStorageValue(WITHDRAWAL_REQUEST_COUNT_STORAGE_SLOT, 0); - } - - private void assertContractStorageValue(final UInt256 slot, final int expectedValue) { - assertContractStorageValue(slot, UInt256.valueOf(expectedValue)); - } - - private void assertContractStorageValue(final UInt256 slot, final UInt256 expectedValue) { - assertThat(worldState.get(WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS).getStorageValue(slot)) - .isEqualTo(expectedValue); - } - - private void loadContractStorage( - final MutableWorldState worldState, final List withdrawalRequests) { - final WorldUpdater updater = worldState.updater(); - contract = updater.getOrCreate(WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS); - - contract.setCode( - Bytes.fromHexString( - "0x61013680600a5f395ff33373fffffffffffffffffffffffffffffffffffffffe146090573615156028575f545f5260205ff35b36603014156101325760115f54600182026001905f5b5f82111560595781019083028483029004916001019190603e565b90939004341061013257600154600101600155600354806003026004013381556001015f3581556001016020359055600101600355005b6003546002548082038060101160a4575060105b5f5b81811460ed5780604402838201600302600401805490600101805490600101549160601b8160a01c17835260601b8160a01c17826020015260601b906040015260010160a6565b910180921460fe5790600255610109565b90505f6002555f6003555b5f546001546002828201116101205750505f610126565b01600290035b5f555f6001556044025ff35b5f5ffd")); - // excess requests - contract.setStorageValue(UInt256.valueOf(0), UInt256.valueOf(0)); - // requests count - contract.setStorageValue(UInt256.valueOf(1), UInt256.valueOf(withdrawalRequests.size())); - // requests queue head pointer - contract.setStorageValue(UInt256.valueOf(2), UInt256.valueOf(0)); - // requests queue tail pointer - contract.setStorageValue(UInt256.valueOf(3), UInt256.valueOf(withdrawalRequests.size())); - - int offset = 4; - for (int i = 0; i < withdrawalRequests.size(); i++) { - final WithdrawalRequest request = withdrawalRequests.get(i); - // source_account - contract.setStorageValue( - // set account to slot, with 12 bytes padding on the left - UInt256.valueOf(offset++), - UInt256.fromBytes( - Bytes.concatenate( - Bytes.fromHexString("0x000000000000000000000000"), request.getSourceAddress()))); - // validator_pubkey - contract.setStorageValue( - UInt256.valueOf(offset++), UInt256.fromBytes(request.getValidatorPubkey().slice(0, 32))); - contract.setStorageValue( - // set public key to slot, with 16 bytes padding on the right - UInt256.valueOf(offset++), - UInt256.fromBytes( - Bytes.concatenate( - request.getValidatorPubkey().slice(32, 16), - request.getAmount().toBytes(), // 8 bytes for amount - Bytes.fromHexString("0x0000000000000000")))); - } - updater.commit(); - } - - private WithdrawalRequest createExit() { - return new WithdrawalRequest( - Address.extract(Bytes32.random()), BLSPublicKey.wrap(Bytes48.random()), GWei.ONE); - } -} diff --git a/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestValidatorTestFixtures.java b/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestValidatorTestFixtures.java index 7b8b440bb64..e719810c282 100644 --- a/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestValidatorTestFixtures.java +++ b/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestValidatorTestFixtures.java @@ -14,7 +14,7 @@ */ package org.hyperledger.besu.ethereum.mainnet; -import static org.hyperledger.besu.ethereum.mainnet.WithdrawalRequestContractHelper.MAX_WITHDRAWAL_REQUESTS_PER_BLOCK; +import static org.hyperledger.besu.ethereum.mainnet.requests.WithdrawalRequestValidator.MAX_WITHDRAWAL_REQUESTS_PER_BLOCK; import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.BLSPublicKey; diff --git a/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/T8nExecutor.java b/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/T8nExecutor.java index d5239848821..32ffbe10934 100644 --- a/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/T8nExecutor.java +++ b/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/T8nExecutor.java @@ -42,6 +42,7 @@ import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec; import org.hyperledger.besu.ethereum.mainnet.TransactionValidationParams; +import org.hyperledger.besu.ethereum.mainnet.requests.ProcessRequestContext; import org.hyperledger.besu.ethereum.mainnet.requests.RequestUtil; import org.hyperledger.besu.ethereum.processing.TransactionProcessingResult; import org.hyperledger.besu.ethereum.referencetests.BonsaiReferenceTestWorldState; @@ -52,6 +53,7 @@ import org.hyperledger.besu.ethereum.rlp.BytesValueRLPOutput; import org.hyperledger.besu.ethereum.rlp.RLP; import org.hyperledger.besu.ethereum.trie.diffbased.common.DiffBasedAccount; +import org.hyperledger.besu.ethereum.vm.CachingBlockHashLookup; import org.hyperledger.besu.evm.account.Account; import org.hyperledger.besu.evm.gascalculator.GasCalculator; import org.hyperledger.besu.evm.log.Log; @@ -511,7 +513,15 @@ static T8nResult runTest( var requestProcessorCoordinator = protocolSpec.getRequestProcessorCoordinator(); if (requestProcessorCoordinator.isPresent()) { var rpc = requestProcessorCoordinator.get(); - Optional> maybeRequests = rpc.process(worldState, receipts); + ProcessRequestContext context = + new ProcessRequestContext( + blockHeader, + worldState, + protocolSpec, + receipts, + new CachingBlockHashLookup(blockHeader, blockchain), + OperationTracer.NO_TRACING); + Optional> maybeRequests = rpc.process(context); Hash requestRoot = BodyValidation.requestsRoot(maybeRequests.orElse(List.of())); resultObject.put("requestsRoot", requestRoot.toHexString());