mirror of
https://github.com/vacp2p/linea-besu.git
synced 2026-01-09 15:37:54 -05:00
EIP2935 - Use system call instead of direct storage modification (#8243)
Signed-off-by: Gabriel-Trintinalia <gabriel.trintinalia@consensys.net>
This commit is contained in:
committed by
GitHub
parent
1e5fb0051e
commit
630a8f7d0f
@@ -48,8 +48,9 @@ import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ScheduleBasedBlockHeaderFunctions;
|
||||
import org.hyperledger.besu.ethereum.mainnet.WithdrawalsProcessor;
|
||||
import org.hyperledger.besu.ethereum.mainnet.feemarket.ExcessBlobGasCalculator;
|
||||
import org.hyperledger.besu.ethereum.mainnet.requests.ProcessRequestContext;
|
||||
import org.hyperledger.besu.ethereum.mainnet.requests.RequestProcessingContext;
|
||||
import org.hyperledger.besu.ethereum.mainnet.requests.RequestProcessorCoordinator;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.BlockProcessingContext;
|
||||
import org.hyperledger.besu.evm.account.MutableAccount;
|
||||
import org.hyperledger.besu.evm.worldstate.WorldUpdater;
|
||||
import org.hyperledger.besu.plugin.services.exception.StorageException;
|
||||
@@ -210,10 +211,6 @@ public abstract class AbstractBlockCreator implements AsyncBlockCreator {
|
||||
|
||||
final List<BlockHeader> ommers = maybeOmmers.orElse(selectOmmers());
|
||||
|
||||
newProtocolSpec
|
||||
.getBlockHashProcessor()
|
||||
.processBlockHashes(disposableWorldState, processableBlockHeader);
|
||||
|
||||
throwIfStopped();
|
||||
|
||||
final PluginTransactionSelector pluginTransactionSelector =
|
||||
@@ -223,6 +220,17 @@ public abstract class AbstractBlockCreator implements AsyncBlockCreator {
|
||||
pluginTransactionSelector.getOperationTracer();
|
||||
|
||||
operationTracer.traceStartBlock(processableBlockHeader, miningBeneficiary);
|
||||
BlockProcessingContext blockProcessingContext =
|
||||
new BlockProcessingContext(
|
||||
processableBlockHeader,
|
||||
disposableWorldState,
|
||||
newProtocolSpec,
|
||||
newProtocolSpec
|
||||
.getBlockHashProcessor()
|
||||
.createBlockHashLookup(protocolContext.getBlockchain(), processableBlockHeader),
|
||||
operationTracer);
|
||||
newProtocolSpec.getBlockHashProcessor().process(blockProcessingContext);
|
||||
|
||||
timings.register("preTxsSelection");
|
||||
final TransactionSelectionResults transactionResults =
|
||||
selectTransactions(
|
||||
@@ -252,20 +260,11 @@ 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(),
|
||||
newProtocolSpec
|
||||
.getBlockHashProcessor()
|
||||
.createBlockHashLookup(protocolContext.getBlockchain(), processableBlockHeader),
|
||||
operationTracer);
|
||||
RequestProcessingContext requestProcessingContext =
|
||||
new RequestProcessingContext(blockProcessingContext, transactionResults.getReceipts());
|
||||
|
||||
Optional<List<Request>> maybeRequests =
|
||||
requestProcessor.map(processor -> processor.process(context));
|
||||
requestProcessor.map(processor -> processor.process(requestProcessingContext));
|
||||
|
||||
throwIfStopped();
|
||||
|
||||
|
||||
@@ -77,7 +77,8 @@ import org.hyperledger.besu.ethereum.mainnet.TransactionValidatorFactory;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ValidationResult;
|
||||
import org.hyperledger.besu.ethereum.mainnet.WithdrawalsProcessor;
|
||||
import org.hyperledger.besu.ethereum.mainnet.requests.DepositRequestProcessor;
|
||||
import org.hyperledger.besu.ethereum.mainnet.requests.ProcessRequestContext;
|
||||
import org.hyperledger.besu.ethereum.mainnet.requests.RequestProcessingContext;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.BlockProcessingContext;
|
||||
import org.hyperledger.besu.ethereum.transaction.TransactionInvalidReason;
|
||||
import org.hyperledger.besu.evm.account.Account;
|
||||
import org.hyperledger.besu.evm.internal.EvmConfiguration;
|
||||
@@ -142,7 +143,9 @@ abstract class AbstractBlockCreatorTest {
|
||||
|
||||
var depositRequestsFromReceipts =
|
||||
new DepositRequestProcessor(DEFAULT_DEPOSIT_CONTRACT_ADDRESS)
|
||||
.process(new ProcessRequestContext(null, null, null, receipts, null, null));
|
||||
.process(
|
||||
new RequestProcessingContext(
|
||||
new BlockProcessingContext(null, null, null, null, null), receipts));
|
||||
assertThat(depositRequestsFromReceipts).isEqualTo(expectedDepositRequest);
|
||||
}
|
||||
|
||||
|
||||
@@ -30,8 +30,9 @@ 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.AbstractBlockProcessor.PreprocessingFunction.NoPreprocessing;
|
||||
import org.hyperledger.besu.ethereum.mainnet.requests.ProcessRequestContext;
|
||||
import org.hyperledger.besu.ethereum.mainnet.requests.RequestProcessingContext;
|
||||
import org.hyperledger.besu.ethereum.mainnet.requests.RequestProcessorCoordinator;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.BlockProcessingContext;
|
||||
import org.hyperledger.besu.ethereum.privacy.storage.PrivateMetadataUpdater;
|
||||
import org.hyperledger.besu.ethereum.processing.TransactionProcessingResult;
|
||||
import org.hyperledger.besu.ethereum.trie.MerkleTrieException;
|
||||
@@ -126,10 +127,12 @@ public abstract class AbstractBlockProcessor implements BlockProcessor {
|
||||
long currentBlobGasUsed = 0;
|
||||
|
||||
final ProtocolSpec protocolSpec = protocolSchedule.getByBlockHeader(blockHeader);
|
||||
|
||||
protocolSpec.getBlockHashProcessor().processBlockHashes(worldState, blockHeader);
|
||||
final BlockHashLookup blockHashLookup =
|
||||
protocolSpec.getBlockHashProcessor().createBlockHashLookup(blockchain, blockHeader);
|
||||
final BlockProcessingContext blockProcessingContext =
|
||||
new BlockProcessingContext(
|
||||
blockHeader, worldState, protocolSpec, blockHashLookup, OperationTracer.NO_TRACING);
|
||||
protocolSpec.getBlockHashProcessor().process(blockProcessingContext);
|
||||
|
||||
final Address miningBeneficiary = miningBeneficiaryCalculator.calculateBeneficiary(blockHeader);
|
||||
|
||||
@@ -240,16 +243,9 @@ public abstract class AbstractBlockProcessor implements BlockProcessor {
|
||||
protocolSpec.getRequestProcessorCoordinator();
|
||||
Optional<List<Request>> maybeRequests = Optional.empty();
|
||||
if (requestProcessor.isPresent()) {
|
||||
ProcessRequestContext context =
|
||||
new ProcessRequestContext(
|
||||
blockHeader,
|
||||
worldState,
|
||||
protocolSpec,
|
||||
receipts,
|
||||
blockHashLookup,
|
||||
OperationTracer.NO_TRACING);
|
||||
|
||||
maybeRequests = Optional.of(requestProcessor.get().process(context));
|
||||
RequestProcessingContext requestProcessingContext =
|
||||
new RequestProcessingContext(blockProcessingContext, receipts);
|
||||
maybeRequests = Optional.of(requestProcessor.get().process(requestProcessingContext));
|
||||
}
|
||||
|
||||
if (maybeRequests.isPresent() && blockHeader.getRequestsHash().isPresent()) {
|
||||
|
||||
@@ -15,13 +15,12 @@
|
||||
package org.hyperledger.besu.ethereum.mainnet.blockhash;
|
||||
|
||||
import org.hyperledger.besu.ethereum.chain.Blockchain;
|
||||
import org.hyperledger.besu.ethereum.core.MutableWorldState;
|
||||
import org.hyperledger.besu.ethereum.core.ProcessableBlockHeader;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.BlockContextProcessor;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.BlockProcessingContext;
|
||||
import org.hyperledger.besu.evm.blockhash.BlockHashLookup;
|
||||
|
||||
public interface BlockHashProcessor {
|
||||
|
||||
void processBlockHashes(MutableWorldState worldState, ProcessableBlockHeader currentBlockHeader);
|
||||
public interface BlockHashProcessor extends BlockContextProcessor<Void, BlockProcessingContext> {
|
||||
|
||||
BlockHashLookup createBlockHashLookup(Blockchain blockchain, ProcessableBlockHeader blockHeader);
|
||||
}
|
||||
|
||||
@@ -14,27 +14,28 @@
|
||||
*/
|
||||
package org.hyperledger.besu.ethereum.mainnet.blockhash;
|
||||
|
||||
import org.hyperledger.besu.ethereum.core.MutableWorldState;
|
||||
import org.hyperledger.besu.ethereum.core.ProcessableBlockHeader;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ParentBeaconBlockRootHelper;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.BlockProcessingContext;
|
||||
import org.hyperledger.besu.evm.worldstate.WorldUpdater;
|
||||
|
||||
/** Processes the beacon block storage if it is present in the block header. */
|
||||
public class CancunBlockHashProcessor extends FrontierBlockHashProcessor {
|
||||
|
||||
@Override
|
||||
public void processBlockHashes(
|
||||
final MutableWorldState mutableWorldState, final ProcessableBlockHeader currentBlockHeader) {
|
||||
public Void process(final BlockProcessingContext context) {
|
||||
ProcessableBlockHeader currentBlockHeader = context.getBlockHeader();
|
||||
currentBlockHeader
|
||||
.getParentBeaconBlockRoot()
|
||||
.ifPresent(
|
||||
beaconBlockRoot -> {
|
||||
if (!beaconBlockRoot.isEmpty()) {
|
||||
WorldUpdater worldUpdater = mutableWorldState.updater();
|
||||
WorldUpdater worldUpdater = context.getWorldState().updater();
|
||||
ParentBeaconBlockRootHelper.storeParentBeaconBlockRoot(
|
||||
worldUpdater, currentBlockHeader.getTimestamp(), beaconBlockRoot);
|
||||
worldUpdater.commit();
|
||||
}
|
||||
});
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -29,6 +29,6 @@ public class Eip7709BlockHashProcessor extends PragueBlockHashProcessor {
|
||||
@Override
|
||||
public BlockHashLookup createBlockHashLookup(
|
||||
final Blockchain blockchain, final ProcessableBlockHeader blockHeader) {
|
||||
return new Eip7709BlockHashLookup(historyStorageAddress, historyServeWindow);
|
||||
return new Eip7709BlockHashLookup(historyStorageAddress);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -15,8 +15,8 @@
|
||||
package org.hyperledger.besu.ethereum.mainnet.blockhash;
|
||||
|
||||
import org.hyperledger.besu.ethereum.chain.Blockchain;
|
||||
import org.hyperledger.besu.ethereum.core.MutableWorldState;
|
||||
import org.hyperledger.besu.ethereum.core.ProcessableBlockHeader;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.BlockProcessingContext;
|
||||
import org.hyperledger.besu.ethereum.vm.BlockchainBasedBlockHashLookup;
|
||||
import org.hyperledger.besu.evm.blockhash.BlockHashLookup;
|
||||
import org.hyperledger.besu.evm.operation.BlockHashOperation;
|
||||
@@ -24,9 +24,9 @@ import org.hyperledger.besu.evm.operation.BlockHashOperation;
|
||||
public class FrontierBlockHashProcessor implements BlockHashProcessor {
|
||||
|
||||
@Override
|
||||
public void processBlockHashes(
|
||||
final MutableWorldState mutableWorldState, final ProcessableBlockHeader currentBlockHeader) {
|
||||
public Void process(final BlockProcessingContext context) {
|
||||
// do nothing
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -15,16 +15,11 @@
|
||||
package org.hyperledger.besu.ethereum.mainnet.blockhash;
|
||||
|
||||
import org.hyperledger.besu.datatypes.Address;
|
||||
import org.hyperledger.besu.datatypes.Hash;
|
||||
import org.hyperledger.besu.ethereum.core.MutableWorldState;
|
||||
import org.hyperledger.besu.ethereum.core.ProcessableBlockHeader;
|
||||
import org.hyperledger.besu.evm.account.MutableAccount;
|
||||
import org.hyperledger.besu.evm.worldstate.WorldUpdater;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.BlockProcessingContext;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.SystemCallProcessor;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.tuweni.units.bigints.UInt256;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.tuweni.bytes.Bytes;
|
||||
|
||||
/**
|
||||
* Processes and stores historical block hashes in accordance with EIP-2935. This class is
|
||||
@@ -32,20 +27,14 @@ import org.slf4j.LoggerFactory;
|
||||
* historical block hash access in smart contracts.
|
||||
*/
|
||||
public class PragueBlockHashProcessor extends CancunBlockHashProcessor {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(PragueBlockHashProcessor.class);
|
||||
|
||||
public static final Address HISTORY_STORAGE_ADDRESS =
|
||||
private static final Address HISTORY_STORAGE_ADDRESS =
|
||||
Address.fromHexString("0x0000f90827f1c53a10cb7a02335b175320002935");
|
||||
|
||||
/** The HISTORY_SERVE_WINDOW */
|
||||
private static final long HISTORY_SERVE_WINDOW = 8191;
|
||||
|
||||
protected final long historyServeWindow;
|
||||
protected final Address historyStorageAddress;
|
||||
|
||||
/** Constructs a BlockHashProcessor. */
|
||||
public PragueBlockHashProcessor() {
|
||||
this(HISTORY_STORAGE_ADDRESS, HISTORY_SERVE_WINDOW);
|
||||
this(HISTORY_STORAGE_ADDRESS);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -53,53 +42,20 @@ public class PragueBlockHashProcessor extends CancunBlockHashProcessor {
|
||||
* primarily used for testing.
|
||||
*
|
||||
* @param historyStorageAddress the address of the contract storing the history
|
||||
* @param historyServeWindow The number of blocks for which history should be saved.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public PragueBlockHashProcessor(
|
||||
final Address historyStorageAddress, final long historyServeWindow) {
|
||||
public PragueBlockHashProcessor(final Address historyStorageAddress) {
|
||||
this.historyStorageAddress = historyStorageAddress;
|
||||
this.historyServeWindow = historyServeWindow;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processBlockHashes(
|
||||
final MutableWorldState mutableWorldState, final ProcessableBlockHeader currentBlockHeader) {
|
||||
super.processBlockHashes(mutableWorldState, currentBlockHeader);
|
||||
public Void process(final BlockProcessingContext context) {
|
||||
super.process(context);
|
||||
SystemCallProcessor processor =
|
||||
new SystemCallProcessor(context.getProtocolSpec().getTransactionProcessor());
|
||||
|
||||
WorldUpdater worldUpdater = mutableWorldState.updater();
|
||||
final MutableAccount historyStorageAccount = worldUpdater.getAccount(historyStorageAddress);
|
||||
|
||||
if (historyStorageAccount != null
|
||||
&& historyStorageAccount.getNonce() > 0
|
||||
&& currentBlockHeader.getNumber() > 0) {
|
||||
storeParentHash(historyStorageAccount, currentBlockHeader);
|
||||
}
|
||||
worldUpdater.commit();
|
||||
}
|
||||
|
||||
/**
|
||||
* Stores the hash of the parent block in the world state.
|
||||
*
|
||||
* @param account The account associated with the historical block hash storage.
|
||||
* @param header The current block header being processed.
|
||||
*/
|
||||
private void storeParentHash(final MutableAccount account, final ProcessableBlockHeader header) {
|
||||
storeHash(account, header.getNumber() - 1, header.getParentHash());
|
||||
}
|
||||
|
||||
/**
|
||||
* Stores the hash in the world state.
|
||||
*
|
||||
* @param account The account associated with the historical block hash storage.
|
||||
* @param number The slot to store.
|
||||
* @param hash The hash to be stored.
|
||||
*/
|
||||
private void storeHash(final MutableAccount account, final long number, final Hash hash) {
|
||||
UInt256 slot = UInt256.valueOf(number % historyServeWindow);
|
||||
UInt256 value = UInt256.fromBytes(hash);
|
||||
LOG.trace(
|
||||
"Writing to {} {}=%{}", account.getAddress(), slot.toDecimalString(), value.toHexString());
|
||||
account.setStorageValue(slot, value);
|
||||
Bytes inputData = context.getBlockHeader().getParentHash();
|
||||
processor.process(historyStorageAddress, context, inputData);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -35,7 +35,7 @@ public class DepositRequestProcessor implements RequestProcessor {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Request process(final ProcessRequestContext context) {
|
||||
public Request process(final RequestProcessingContext context) {
|
||||
if (depositContractAddress.isEmpty()) {
|
||||
return new Request(RequestType.DEPOSIT, Bytes.EMPTY);
|
||||
}
|
||||
|
||||
@@ -14,19 +14,26 @@
|
||||
*/
|
||||
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.blockhash.BlockHashLookup;
|
||||
import org.hyperledger.besu.evm.tracing.OperationTracer;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.BlockProcessingContext;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public record ProcessRequestContext(
|
||||
ProcessableBlockHeader blockHeader,
|
||||
MutableWorldState mutableWorldState,
|
||||
ProtocolSpec protocolSpec,
|
||||
List<TransactionReceipt> transactionReceipts,
|
||||
BlockHashLookup blockHashLookup,
|
||||
OperationTracer operationTracer) {}
|
||||
public final class RequestProcessingContext extends BlockProcessingContext {
|
||||
private final List<TransactionReceipt> transactionReceipts;
|
||||
|
||||
public RequestProcessingContext(
|
||||
final BlockProcessingContext context, final List<TransactionReceipt> transactionReceipts) {
|
||||
super(
|
||||
context.getBlockHeader(),
|
||||
context.getWorldState(),
|
||||
context.getProtocolSpec(),
|
||||
context.getBlockHashLookup(),
|
||||
context.getOperationTracer());
|
||||
this.transactionReceipts = transactionReceipts;
|
||||
}
|
||||
|
||||
public List<TransactionReceipt> transactionReceipts() {
|
||||
return transactionReceipts;
|
||||
}
|
||||
}
|
||||
@@ -17,5 +17,5 @@ package org.hyperledger.besu.ethereum.mainnet.requests;
|
||||
import org.hyperledger.besu.ethereum.core.Request;
|
||||
|
||||
public interface RequestProcessor {
|
||||
Request process(final ProcessRequestContext context);
|
||||
Request process(final RequestProcessingContext context);
|
||||
}
|
||||
|
||||
@@ -35,7 +35,7 @@ public class RequestProcessorCoordinator {
|
||||
this.processors = processors;
|
||||
}
|
||||
|
||||
public List<Request> process(final ProcessRequestContext context) {
|
||||
public List<Request> process(final RequestProcessingContext context) {
|
||||
return processors.values().stream()
|
||||
.map(requestProcessor -> requestProcessor.process(context))
|
||||
.toList();
|
||||
|
||||
@@ -17,12 +17,14 @@ package org.hyperledger.besu.ethereum.mainnet.requests;
|
||||
import org.hyperledger.besu.datatypes.Address;
|
||||
import org.hyperledger.besu.datatypes.RequestType;
|
||||
import org.hyperledger.besu.ethereum.core.Request;
|
||||
import org.hyperledger.besu.ethereum.mainnet.SystemCallProcessor;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.BlockContextProcessor;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.SystemCallProcessor;
|
||||
|
||||
import org.apache.tuweni.bytes.Bytes;
|
||||
|
||||
/** Processes system call requests. */
|
||||
public class SystemCallRequestProcessor implements RequestProcessor {
|
||||
public class SystemCallRequestProcessor
|
||||
implements RequestProcessor, BlockContextProcessor<Request, RequestProcessingContext> {
|
||||
|
||||
private final Address callAddress;
|
||||
private final RequestType requestType;
|
||||
@@ -39,18 +41,12 @@ public class SystemCallRequestProcessor implements RequestProcessor {
|
||||
* @return A {@link Request} request
|
||||
*/
|
||||
@Override
|
||||
public Request process(final ProcessRequestContext context) {
|
||||
public Request process(final RequestProcessingContext context) {
|
||||
|
||||
SystemCallProcessor systemCallProcessor =
|
||||
new SystemCallProcessor(context.protocolSpec().getTransactionProcessor());
|
||||
final SystemCallProcessor systemCallProcessor =
|
||||
new SystemCallProcessor(context.getProtocolSpec().getTransactionProcessor());
|
||||
|
||||
Bytes systemCallOutput =
|
||||
systemCallProcessor.process(
|
||||
callAddress,
|
||||
context.mutableWorldState().updater(),
|
||||
context.blockHeader(),
|
||||
context.operationTracer(),
|
||||
context.blockHashLookup());
|
||||
Bytes systemCallOutput = systemCallProcessor.process(callAddress, context, Bytes.EMPTY);
|
||||
|
||||
return new Request(requestType, systemCallOutput);
|
||||
}
|
||||
|
||||
@@ -0,0 +1,19 @@
|
||||
/*
|
||||
* Copyright contributors to 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.systemcall;
|
||||
|
||||
public interface BlockContextProcessor<T, C extends BlockProcessingContext> {
|
||||
T process(final C context);
|
||||
}
|
||||
@@ -0,0 +1,63 @@
|
||||
/*
|
||||
* Copyright contributors to 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.systemcall;
|
||||
|
||||
import org.hyperledger.besu.ethereum.core.MutableWorldState;
|
||||
import org.hyperledger.besu.ethereum.core.ProcessableBlockHeader;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
|
||||
import org.hyperledger.besu.evm.blockhash.BlockHashLookup;
|
||||
import org.hyperledger.besu.evm.tracing.OperationTracer;
|
||||
|
||||
public class BlockProcessingContext {
|
||||
|
||||
private final MutableWorldState worldState;
|
||||
private final ProcessableBlockHeader blockHeader;
|
||||
private final OperationTracer operationTracer;
|
||||
private final BlockHashLookup blockHashLookup;
|
||||
private final ProtocolSpec protocolSpec;
|
||||
|
||||
public BlockProcessingContext(
|
||||
final ProcessableBlockHeader blockHeader,
|
||||
final MutableWorldState worldState,
|
||||
final ProtocolSpec protocolSpec,
|
||||
final BlockHashLookup blockHashLookup,
|
||||
final OperationTracer operationTracer) {
|
||||
this.blockHeader = blockHeader;
|
||||
this.worldState = worldState;
|
||||
this.protocolSpec = protocolSpec;
|
||||
this.blockHashLookup = blockHashLookup;
|
||||
this.operationTracer = operationTracer;
|
||||
}
|
||||
|
||||
public MutableWorldState getWorldState() {
|
||||
return worldState;
|
||||
}
|
||||
|
||||
public ProcessableBlockHeader getBlockHeader() {
|
||||
return blockHeader;
|
||||
}
|
||||
|
||||
public OperationTracer getOperationTracer() {
|
||||
return operationTracer;
|
||||
}
|
||||
|
||||
public BlockHashLookup getBlockHashLookup() {
|
||||
return blockHashLookup;
|
||||
}
|
||||
|
||||
public ProtocolSpec getProtocolSpec() {
|
||||
return protocolSpec;
|
||||
}
|
||||
}
|
||||
@@ -12,19 +12,19 @@
|
||||
*
|
||||
* SPDX-License-Identifier: Apache-2.0
|
||||
*/
|
||||
package org.hyperledger.besu.ethereum.mainnet;
|
||||
package org.hyperledger.besu.ethereum.mainnet.systemcall;
|
||||
|
||||
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.ethereum.mainnet.MainnetTransactionProcessor;
|
||||
import org.hyperledger.besu.evm.account.Account;
|
||||
import org.hyperledger.besu.evm.blockhash.BlockHashLookup;
|
||||
import org.hyperledger.besu.evm.code.CodeV0;
|
||||
import org.hyperledger.besu.evm.frame.MessageFrame;
|
||||
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;
|
||||
@@ -48,44 +48,33 @@ public class SystemCallProcessor {
|
||||
}
|
||||
|
||||
/**
|
||||
* Processes a system call to a specified address, using the provided world state, block header,
|
||||
* operation tracer, and block hash lookup.
|
||||
* Processes a system call.
|
||||
*
|
||||
* @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. If no code exists at the callAddress then an empty Bytes
|
||||
* is returned.
|
||||
* @param callAddress The address to call.
|
||||
* @param context The system call context. The input data to the system call.
|
||||
* @param inputData The input data to the system call.
|
||||
* @return The output of the system call.
|
||||
*/
|
||||
public Bytes process(
|
||||
final Address callAddress,
|
||||
final WorldUpdater worldState,
|
||||
final ProcessableBlockHeader blockHeader,
|
||||
final OperationTracer operationTracer,
|
||||
final BlockHashLookup blockHashLookup) {
|
||||
final Address callAddress, final BlockProcessingContext context, final Bytes inputData) {
|
||||
WorldUpdater updater = context.getWorldState().updater();
|
||||
|
||||
// if no code exists at CALL_ADDRESS, the call must fail silently
|
||||
final Account maybeContract = worldState.get(callAddress);
|
||||
final Account maybeContract = updater.get(callAddress);
|
||||
if (maybeContract == null) {
|
||||
LOG.trace("System call address not found {}", callAddress);
|
||||
return Bytes.EMPTY;
|
||||
}
|
||||
|
||||
final AbstractMessageProcessor messageProcessor =
|
||||
final AbstractMessageProcessor processor =
|
||||
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) {
|
||||
final MessageFrame frame =
|
||||
createMessageFrame(
|
||||
callAddress,
|
||||
updater,
|
||||
context.getBlockHeader(),
|
||||
context.getBlockHashLookup(),
|
||||
inputData);
|
||||
|
||||
if (!frame.getCode().isValid()) {
|
||||
throw new RuntimeException("System call did not execute to completion - opcode invalid");
|
||||
@@ -93,7 +82,7 @@ public class SystemCallProcessor {
|
||||
|
||||
Deque<MessageFrame> stack = frame.getMessageFrameStack();
|
||||
while (!stack.isEmpty()) {
|
||||
processor.process(stack.peekFirst(), tracer);
|
||||
processor.process(stack.peekFirst(), context.getOperationTracer());
|
||||
}
|
||||
|
||||
if (frame.getState() == MessageFrame.State.COMPLETED_SUCCESS) {
|
||||
@@ -105,11 +94,12 @@ public class SystemCallProcessor {
|
||||
throw new RuntimeException("System call did not execute to completion");
|
||||
}
|
||||
|
||||
private MessageFrame createCallFrame(
|
||||
private MessageFrame createMessageFrame(
|
||||
final Address callAddress,
|
||||
final WorldUpdater worldUpdater,
|
||||
final ProcessableBlockHeader blockHeader,
|
||||
final BlockHashLookup blockHashLookup) {
|
||||
final BlockHashLookup blockHashLookup,
|
||||
final Bytes inputData) {
|
||||
|
||||
final Optional<Account> maybeContract = Optional.ofNullable(worldUpdater.get(callAddress));
|
||||
final AbstractMessageProcessor processor =
|
||||
@@ -130,7 +120,7 @@ public class SystemCallProcessor {
|
||||
.type(MessageFrame.Type.MESSAGE_CALL)
|
||||
.address(callAddress)
|
||||
.contract(callAddress)
|
||||
.inputData(Bytes.EMPTY)
|
||||
.inputData(inputData)
|
||||
.sender(SYSTEM_ADDRESS)
|
||||
.blockHashLookup(blockHashLookup)
|
||||
.code(
|
||||
@@ -37,6 +37,7 @@ import org.slf4j.LoggerFactory;
|
||||
public class Eip7709BlockHashLookup implements BlockHashLookup {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(Eip7709BlockHashLookup.class);
|
||||
private static final long BLOCKHASH_SERVE_WINDOW = 256L;
|
||||
private static final long HISTORY_SERVE_WINDOW = 8191;
|
||||
|
||||
private final Address contractAddress;
|
||||
private final long historyServeWindow;
|
||||
@@ -47,10 +48,9 @@ public class Eip7709BlockHashLookup implements BlockHashLookup {
|
||||
* Constructs a Eip7709BlockHashLookup.
|
||||
*
|
||||
* @param contractAddress the address of the contract storing the history.
|
||||
* @param historyServeWindow the number of blocks for which history should be saved.
|
||||
*/
|
||||
public Eip7709BlockHashLookup(final Address contractAddress, final long historyServeWindow) {
|
||||
this(contractAddress, historyServeWindow, BLOCKHASH_SERVE_WINDOW);
|
||||
public Eip7709BlockHashLookup(final Address contractAddress) {
|
||||
this(contractAddress, HISTORY_SERVE_WINDOW, BLOCKHASH_SERVE_WINDOW);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -22,9 +22,12 @@ import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import org.hyperledger.besu.datatypes.Address;
|
||||
import org.hyperledger.besu.datatypes.Hash;
|
||||
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.ethereum.mainnet.systemcall.BlockProcessingContext;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.SystemCallProcessor;
|
||||
import org.hyperledger.besu.evm.blockhash.BlockHashLookup;
|
||||
import org.hyperledger.besu.evm.frame.MessageFrame;
|
||||
import org.hyperledger.besu.evm.processor.AbstractMessageProcessor;
|
||||
@@ -36,7 +39,7 @@ import org.apache.tuweni.bytes.Bytes;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public class SystemCallProcessorTest {
|
||||
public class MainnetBlockContextProcessorTest {
|
||||
private static final Address CALL_ADDRESS = Address.fromHexString("0x1");
|
||||
private static final Bytes EXPECTED_OUTPUT = Bytes.fromHexString("0x01");
|
||||
private ProcessableBlockHeader mockBlockHeader;
|
||||
@@ -95,12 +98,17 @@ public class SystemCallProcessorTest {
|
||||
|
||||
Bytes processSystemCall(final MutableWorldState worldState) {
|
||||
SystemCallProcessor systemCallProcessor = new SystemCallProcessor(mockTransactionProcessor);
|
||||
return systemCallProcessor.process(
|
||||
CALL_ADDRESS,
|
||||
worldState.updater(),
|
||||
mockBlockHeader,
|
||||
OperationTracer.NO_TRACING,
|
||||
mockBlockHashLookup);
|
||||
|
||||
BlockProcessingContext blockProcessingContext =
|
||||
new BlockProcessingContext(
|
||||
mockBlockHeader,
|
||||
worldState,
|
||||
mock(ProtocolSpec.class),
|
||||
mockBlockHashLookup,
|
||||
OperationTracer.NO_TRACING);
|
||||
|
||||
when(mockBlockHashLookup.apply(any(), any())).thenReturn(Hash.EMPTY);
|
||||
return systemCallProcessor.process(CALL_ADDRESS, blockProcessingContext, Bytes.EMPTY);
|
||||
}
|
||||
|
||||
private MutableWorldState createWorldState(final Address address) {
|
||||
@@ -1,141 +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.blockhash;
|
||||
|
||||
import static org.hyperledger.besu.datatypes.Hash.fromHexStringLenient;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.verifyNoInteractions;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import org.hyperledger.besu.datatypes.Hash;
|
||||
import org.hyperledger.besu.ethereum.core.BlockHeader;
|
||||
import org.hyperledger.besu.ethereum.core.MutableWorldState;
|
||||
import org.hyperledger.besu.evm.account.MutableAccount;
|
||||
import org.hyperledger.besu.evm.worldstate.WorldUpdater;
|
||||
|
||||
import org.apache.tuweni.units.bigints.UInt256;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
class BlockHashProcessorTest {
|
||||
private WorldUpdater worldUpdater;
|
||||
private MutableWorldState mutableWorldState;
|
||||
private MutableAccount account;
|
||||
private BlockHashProcessor processor;
|
||||
|
||||
private final long historicalWindow = 255;
|
||||
|
||||
@BeforeEach
|
||||
void setUp() {
|
||||
mutableWorldState = mock(MutableWorldState.class);
|
||||
worldUpdater = mock(WorldUpdater.class);
|
||||
account = mock(MutableAccount.class);
|
||||
when(account.getNonce()).thenReturn(1L);
|
||||
when(mutableWorldState.updater()).thenReturn(worldUpdater);
|
||||
when(worldUpdater.getAccount(PragueBlockHashProcessor.HISTORY_STORAGE_ADDRESS))
|
||||
.thenReturn(account);
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldStoreParentBlockHash() {
|
||||
long currentBlock = 3;
|
||||
processor = new PragueBlockHashProcessor();
|
||||
BlockHeader currentBlockHeader = mockBlockHeader(currentBlock);
|
||||
mockAncestorHeaders(currentBlockHeader, 3);
|
||||
processor.processBlockHashes(mutableWorldState, currentBlockHeader);
|
||||
// only parent slot number must be set
|
||||
verify(account, times(1)).setStorageValue(any(), any());
|
||||
verifyAccount(currentBlock - 1, historicalWindow);
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldNotStoreBlockHashForGenesisBlock() {
|
||||
// For the fork to be activated at genesis, no history is written to the genesis state, and at
|
||||
// the start of block 1, genesis hash will be written as a normal operation to slot 0.
|
||||
long currentBlock = 0;
|
||||
processor = new PragueBlockHashProcessor();
|
||||
BlockHeader currentBlockHeader = mockBlockHeader(currentBlock);
|
||||
mockAncestorHeaders(currentBlockHeader, 0);
|
||||
|
||||
processor.processBlockHashes(mutableWorldState, currentBlockHeader);
|
||||
verify(account, times(0)).setStorageValue(any(), any());
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldStoreAncestorBlockHashesAtForkCorrectlyParentIsGenesis() {
|
||||
// for activation at block 1, only genesis hash will be written at slot 0 as there is no
|
||||
// additional history that needs to be persisted.
|
||||
long currentBlock = 1;
|
||||
processor = new PragueBlockHashProcessor();
|
||||
BlockHeader currentBlockHeader = mockBlockHeader(currentBlock);
|
||||
mockAncestorHeaders(currentBlockHeader, 10);
|
||||
|
||||
processor.processBlockHashes(mutableWorldState, currentBlockHeader);
|
||||
verify(account, times(1)).setStorageValue(any(), any());
|
||||
verifyAccount(0, historicalWindow);
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldNotStoreBlockHashIfContractIsNotDeployed() {
|
||||
when(worldUpdater.getAccount(PragueBlockHashProcessor.HISTORY_STORAGE_ADDRESS))
|
||||
.thenReturn(null);
|
||||
|
||||
long currentBlock = 1;
|
||||
processor = new PragueBlockHashProcessor();
|
||||
BlockHeader currentBlockHeader = mockBlockHeader(currentBlock);
|
||||
mockAncestorHeaders(currentBlockHeader, 0);
|
||||
|
||||
processor.processBlockHashes(mutableWorldState, currentBlockHeader);
|
||||
verifyNoInteractions(account);
|
||||
}
|
||||
|
||||
@Test
|
||||
void shouldWriteGenesisHashAtSlot0() {
|
||||
processor = new PragueBlockHashProcessor();
|
||||
BlockHeader header = mockBlockHeader(1);
|
||||
mockAncestorHeaders(header, 1);
|
||||
processor.processBlockHashes(mutableWorldState, header);
|
||||
verify(account)
|
||||
.setStorageValue(UInt256.valueOf(0), UInt256.fromHexString(Hash.ZERO.toHexString()));
|
||||
}
|
||||
|
||||
private void verifyAccount(final long number, final long historicalWindow) {
|
||||
verify(account)
|
||||
.setStorageValue(UInt256.valueOf(number % historicalWindow), UInt256.valueOf(number));
|
||||
}
|
||||
|
||||
private void mockAncestorHeaders(final BlockHeader blockHeader, final int count) {
|
||||
long firstAncestor = Math.max(blockHeader.getNumber() - count, 0);
|
||||
var block = blockHeader;
|
||||
for (long i = blockHeader.getNumber(); i > firstAncestor; i--) {
|
||||
long parentNumber = block.getNumber() - 1;
|
||||
block = mockBlockHeader(parentNumber);
|
||||
}
|
||||
}
|
||||
|
||||
private BlockHeader mockBlockHeader(final long currentNumber) {
|
||||
BlockHeader blockHeader = mock(BlockHeader.class);
|
||||
when(blockHeader.getNumber()).thenReturn(currentNumber);
|
||||
Hash hash = fromHexStringLenient("0x" + Long.toHexString(currentNumber));
|
||||
Hash parentHash = fromHexStringLenient("0x" + Long.toHexString(currentNumber - 1));
|
||||
when(blockHeader.getHash()).thenReturn(hash);
|
||||
when(blockHeader.getTimestamp()).thenReturn(currentNumber);
|
||||
when(blockHeader.getParentHash()).thenReturn(parentHash);
|
||||
return blockHeader;
|
||||
}
|
||||
}
|
||||
@@ -43,7 +43,8 @@ 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.RequestProcessingContext;
|
||||
import org.hyperledger.besu.ethereum.mainnet.systemcall.BlockProcessingContext;
|
||||
import org.hyperledger.besu.ethereum.processing.TransactionProcessingResult;
|
||||
import org.hyperledger.besu.ethereum.referencetests.BonsaiReferenceTestWorldState;
|
||||
import org.hyperledger.besu.ethereum.referencetests.ReferenceTestEnv;
|
||||
@@ -356,9 +357,18 @@ public class T8nExecutor {
|
||||
.orElse(blockHeader.getExcessBlobGas().orElse(BlobGas.ZERO)); // state-test
|
||||
final Wei blobGasPrice = protocolSpec.getFeeMarket().blobGasPricePerGas(excessBlobGas);
|
||||
long blobGasLimit = protocolSpec.getGasLimitCalculator().currentBlobGasLimit();
|
||||
BlockProcessingContext blockProcessingContext =
|
||||
new BlockProcessingContext(
|
||||
referenceTestEnv,
|
||||
worldState,
|
||||
protocolSpec,
|
||||
protocolSpec
|
||||
.getBlockHashProcessor()
|
||||
.createBlockHashLookup(blockchain, referenceTestEnv),
|
||||
OperationTracer.NO_TRACING);
|
||||
|
||||
if (!referenceTestEnv.isStateTest()) {
|
||||
protocolSpec.getBlockHashProcessor().processBlockHashes(worldState, referenceTestEnv);
|
||||
protocolSpec.getBlockHashProcessor().process(blockProcessingContext);
|
||||
}
|
||||
|
||||
final WorldUpdater rootWorldStateUpdater = worldState.updater();
|
||||
@@ -535,15 +545,10 @@ public class T8nExecutor {
|
||||
var requestProcessorCoordinator = protocolSpec.getRequestProcessorCoordinator();
|
||||
if (requestProcessorCoordinator.isPresent()) {
|
||||
var rpc = requestProcessorCoordinator.get();
|
||||
ProcessRequestContext context =
|
||||
new ProcessRequestContext(
|
||||
blockHeader,
|
||||
worldState,
|
||||
protocolSpec,
|
||||
receipts,
|
||||
protocolSpec.getBlockHashProcessor().createBlockHashLookup(blockchain, blockHeader),
|
||||
OperationTracer.NO_TRACING);
|
||||
Optional<List<Request>> maybeRequests = Optional.of(rpc.process(context));
|
||||
|
||||
RequestProcessingContext requestContext =
|
||||
new RequestProcessingContext(blockProcessingContext, receipts);
|
||||
Optional<List<Request>> maybeRequests = Optional.of(rpc.process(requestContext));
|
||||
Hash requestsHash = BodyValidation.requestsHash(maybeRequests.orElse(List.of()));
|
||||
|
||||
resultObject.put("requestsHash", requestsHash.toHexString());
|
||||
|
||||
Reference in New Issue
Block a user