Implement System Calls (#7263)

Signed-off-by: Gabriel-Trintinalia <gabriel.trintinalia@consensys.net>
pull/7295/head
Gabriel-Trintinalia 5 months ago committed by GitHub
parent 8a9a84ad58
commit 4ee14eca1d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 16
      ethereum/blockcreation/src/main/java/org/hyperledger/besu/ethereum/blockcreation/AbstractBlockCreator.java
  2. 4
      ethereum/blockcreation/src/test/java/org/hyperledger/besu/ethereum/blockcreation/AbstractBlockCreatorTest.java
  3. 14
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/AbstractBlockProcessor.java
  4. 2
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/MainnetTransactionProcessor.java
  5. 141
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/SystemCallProcessor.java
  6. 197
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestContractHelper.java
  7. 102
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/AbstractSystemCallRequestProcessor.java
  8. 7
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/DepositRequestProcessor.java
  9. 32
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/ProcessRequestContext.java
  10. 6
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/RequestProcessor.java
  11. 7
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/RequestProcessorCoordinator.java
  12. 63
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/WithdrawalRequestProcessor.java
  13. 6
      ethereum/core/src/main/java/org/hyperledger/besu/ethereum/mainnet/requests/WithdrawalRequestValidator.java
  14. 113
      ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/SystemCallProcessorTest.java
  15. 205
      ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestContractHelperTest.java
  16. 2
      ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/WithdrawalRequestValidatorTestFixtures.java
  17. 12
      ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/T8nExecutor.java

@ -50,7 +50,9 @@ import org.hyperledger.besu.ethereum.mainnet.WithdrawalsProcessor;
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 @@ public abstract class AbstractBlockCreator implements AsyncBlockCreator {
// EIP-7685: process EL requests
final Optional<RequestProcessorCoordinator> requestProcessor =
newProtocolSpec.getRequestProcessorCoordinator();
ProcessRequestContext context =
new ProcessRequestContext(
processableBlockHeader,
disposableWorldState,
newProtocolSpec,
transactionResults.getReceipts(),
new CachingBlockHashLookup(processableBlockHeader, protocolContext.getBlockchain()),
operationTracer);
Optional<List<Request>> maybeRequests =
requestProcessor.flatMap(
processor ->
processor.process(disposableWorldState, transactionResults.getReceipts()));
requestProcessor.flatMap(processor -> processor.process(context));
throwIfStopped();

@ -77,6 +77,7 @@ import org.hyperledger.besu.ethereum.mainnet.WithdrawalsProcessor;
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 @@ abstract class AbstractBlockCreatorTest {
final List<DepositRequest> 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);
}

@ -29,6 +29,7 @@ import org.hyperledger.besu.ethereum.core.Request;
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 abstract class AbstractBlockProcessor implements BlockProcessor {
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 abstract class AbstractBlockProcessor implements BlockProcessor {
final WorldUpdater worldStateUpdater = worldState.updater();
final BlockHashLookup blockHashLookup = new CachingBlockHashLookup(blockHeader, blockchain);
final Address miningBeneficiary =
miningBeneficiaryCalculator.calculateBeneficiary(blockHeader);
@ -197,7 +198,16 @@ public abstract class AbstractBlockProcessor implements BlockProcessor {
protocolSpec.getRequestProcessorCoordinator();
Optional<List<Request>> 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)) {

@ -563,7 +563,7 @@ public class MainnetTransactionProcessor {
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;

@ -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<MessageFrame> 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<Account> 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();
}
}

@ -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)
*
* <p>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<WithdrawalRequest> 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<WithdrawalRequest> withdrawalRequests = dequeueWithdrawalRequests(account);
updateExcessWithdrawalRequests(account);
resetWithdrawalRequestsCount(account);
worldUpdater.commit();
return withdrawalRequests;
}
private static List<WithdrawalRequest> 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<WithdrawalRequest> 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<WithdrawalRequest> 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<WithdrawalRequest> 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));
}
}

@ -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 <T> The type of request to be processed.
*/
public abstract class AbstractSystemCallRequestProcessor<T extends Request>
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<List<? extends Request>> 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<T> 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<T> parseRequests(final Bytes bytes) {
if (bytes == null) {
return null;
}
final List<T> 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();
}

@ -16,7 +16,6 @@ package org.hyperledger.besu.ethereum.mainnet.requests;
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 class DepositRequestProcessor implements RequestProcessor {
}
@Override
public Optional<List<? extends Request>> process(
final MutableWorldState ignored, final List<TransactionReceipt> transactionReceipts) {
public Optional<List<? extends Request>> process(final ProcessRequestContext context) {
if (depositContractAddress.isEmpty()) {
return Optional.empty();
}
List<DepositRequest> depositRequests = findDepositRequestsFromReceipts(transactionReceipts);
List<DepositRequest> depositRequests =
findDepositRequestsFromReceipts(context.transactionReceipts());
return Optional.of(depositRequests);
}

@ -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<TransactionReceipt> transactionReceipts,
BlockHashLookup blockHashLookup,
OperationTracer operationTracer) {}

@ -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<List<? extends Request>> process(
final MutableWorldState mutableWorldState,
final List<TransactionReceipt> transactionReceipts);
Optional<List<? extends Request>> process(final ProcessRequestContext context);
}

@ -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 @@ public class RequestProcessorCoordinator {
this.processors = processors;
}
public Optional<List<Request>> process(
final MutableWorldState mutableWorldState, final List<TransactionReceipt> receipts) {
public Optional<List<Request>> process(final ProcessRequestContext context) {
List<Request> 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<>();

@ -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<WithdrawalRequest> {
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<List<? extends Request>> process(
final MutableWorldState mutableWorldState,
final List<TransactionReceipt> transactionReceipts) {
protected Address getCallAddress() {
return WITHDRAWAL_REQUEST_PREDEPLOY_ADDRESS;
}
List<WithdrawalRequest> 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));
}
}

@ -21,7 +21,6 @@ import org.hyperledger.besu.ethereum.core.Block;
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 @@ import org.slf4j.LoggerFactory;
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 @@ public class WithdrawalRequestValidator implements RequestValidator {
.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;

@ -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;
}
}

@ -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<WithdrawalRequest> validatorWithdrawalRequests =
List.of(createExit(), createExit(), createExit());
loadContractStorage(worldState, validatorWithdrawalRequests);
final List<WithdrawalRequest> 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<WithdrawalRequest> 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<WithdrawalRequest> 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<WithdrawalRequest> 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<WithdrawalRequest> 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<WithdrawalRequest> 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<WithdrawalRequest> 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);
}
}

@ -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;

@ -42,6 +42,7 @@ import org.hyperledger.besu.ethereum.mainnet.MainnetTransactionProcessor;
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.BytesValueRLPInput;
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 @@ public class T8nExecutor {
var requestProcessorCoordinator = protocolSpec.getRequestProcessorCoordinator();
if (requestProcessorCoordinator.isPresent()) {
var rpc = requestProcessorCoordinator.get();
Optional<List<Request>> maybeRequests = rpc.process(worldState, receipts);
ProcessRequestContext context =
new ProcessRequestContext(
blockHeader,
worldState,
protocolSpec,
receipts,
new CachingBlockHashLookup(blockHeader, blockchain),
OperationTracer.NO_TRACING);
Optional<List<Request>> maybeRequests = rpc.process(context);
Hash requestRoot = BodyValidation.requestsRoot(maybeRequests.orElse(List.of()));
resultObject.put("requestsRoot", requestRoot.toHexString());

Loading…
Cancel
Save