Refactor the methods for getting mutable states in the WorldState provider (#8113)

Refactor the methods for retrieving mutable states in the WorldState provider and make additional improvements to clean up the code.

---------

Signed-off-by: Karim Taam <karim.t2am@gmail.com>
This commit is contained in:
Karim Taam
2025-01-27 17:59:54 +01:00
committed by GitHub
parent 846e4834b8
commit b3192a5960
84 changed files with 872 additions and 413 deletions

View File

@@ -621,7 +621,7 @@ public abstract class BesuControllerBuilder implements MiningParameterOverrides
worldStateHealerSupplier::get);
if (maybeStoredGenesisBlockHash.isEmpty()) {
genesisState.writeStateTo(worldStateArchive.getMutable());
genesisState.writeStateTo(worldStateArchive.getWorldState());
}
transactionSimulator =

View File

@@ -26,6 +26,7 @@ import org.hyperledger.besu.ethereum.transaction.BlockSimulator;
import org.hyperledger.besu.ethereum.transaction.BlockStateCall;
import org.hyperledger.besu.ethereum.transaction.CallParameter;
import org.hyperledger.besu.ethereum.transaction.TransactionSimulator;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.plugin.Unstable;
import org.hyperledger.besu.plugin.data.BlockOverrides;
@@ -135,8 +136,13 @@ public class BlockSimulatorServiceImpl implements BlockSimulationService {
}
private MutableWorldState getWorldState(final BlockHeader header, final boolean isPersisting) {
final WorldStateQueryParams worldStateQueryParams =
WorldStateQueryParams.newBuilder()
.withBlockHeader(header)
.withShouldWorldStateUpdateHead(isPersisting)
.build();
return worldStateArchive
.getMutable(header, isPersisting)
.getWorldState(worldStateQueryParams)
.orElseThrow(
() ->
new IllegalArgumentException(

View File

@@ -14,6 +14,8 @@
*/
package org.hyperledger.besu.services;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndUpdateNodeHead;
import org.hyperledger.besu.consensus.merge.MergeContext;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext;
@@ -25,7 +27,7 @@ import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.common.DiffBasedWorldStateProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.DiffBasedWorldStateProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.common.storage.DiffBasedWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.plugin.data.BlockBody;
@@ -109,7 +111,9 @@ public class SynchronizationServiceImpl implements SynchronizationService {
final MutableBlockchain blockchain = protocolContext.getBlockchain();
if (worldStateArchive.flatMap(archive -> archive.getMutable(coreHeader, true)).isPresent()) {
if (worldStateArchive
.flatMap(archive -> archive.getWorldState(withBlockHeaderAndUpdateNodeHead(coreHeader)))
.isPresent()) {
if (coreHeader.getParentHash().equals(blockchain.getChainHeadHash())) {
LOG.atDebug()
.setMessage(
@@ -142,7 +146,7 @@ public class SynchronizationServiceImpl implements SynchronizationService {
// TODO maybe find a best way in the future to delete and disable trie
worldStateArchive.ifPresent(
archive -> {
archive.getDefaultWorldStateConfig().setTrieDisabled(true);
archive.getWorldStateSharedSpec().setTrieDisabled(true);
final DiffBasedWorldStateKeyValueStorage worldStateStorage =
archive.getWorldStateKeyValueStorage();
final Optional<Hash> worldStateBlockHash = worldStateStorage.getWorldStateBlockHash();

View File

@@ -16,7 +16,6 @@ package org.hyperledger.besu.services;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.Mockito.lenient;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -59,6 +58,7 @@ import org.hyperledger.besu.ethereum.mainnet.ValidationResult;
import org.hyperledger.besu.ethereum.mainnet.feemarket.FeeMarket;
import org.hyperledger.besu.ethereum.storage.keyvalue.KeyValueStoragePrefixedKeyBlockchainStorage;
import org.hyperledger.besu.ethereum.storage.keyvalue.VariablesKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
import org.hyperledger.besu.plugin.data.AddedBlockContext;
@@ -152,7 +152,7 @@ public class BesuEventsImplTest {
.when(mockTransactionValidatorFactory.get().validateForSender(any(), any(), any()))
.thenReturn(ValidationResult.valid());
lenient()
.when(mockWorldStateArchive.getMutable(any(), anyBoolean()))
.when(mockWorldStateArchive.getWorldState(any(WorldStateQueryParams.class)))
.thenReturn(Optional.of(mockWorldState));
blockBroadcaster = new BlockBroadcaster(mockEthContext, 10 * ByteUnits.MEGABYTE);

View File

@@ -92,7 +92,7 @@ class TraceServiceImplTest {
Address.fromHexString("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b");
final long persistedNonceForAccount =
worldStateArchive.getMutable().get(addressToVerify).getNonce();
worldStateArchive.getWorldState().get(addressToVerify).getNonce();
final long blockNumber = 2;
@@ -109,7 +109,7 @@ class TraceServiceImplTest {
},
opTracer);
assertThat(worldStateArchive.getMutable().get(addressToVerify).getNonce())
assertThat(worldStateArchive.getWorldState().get(addressToVerify).getNonce())
.isEqualTo(persistedNonceForAccount);
final Block tracedBlock = blockchain.getBlockByNumber(blockNumber).get();
@@ -139,7 +139,7 @@ class TraceServiceImplTest {
Address.fromHexString("0xa94f5374fce5edbc8e2a8697c15331677e6ebf0b");
final long persistedNonceForAccount =
worldStateArchive.getMutable().get(addressToVerify).getNonce();
worldStateArchive.getWorldState().get(addressToVerify).getNonce();
final long startBlock = 1;
final long endBlock = 32;
@@ -157,7 +157,7 @@ class TraceServiceImplTest {
},
opTracer);
assertThat(worldStateArchive.getMutable().get(addressToVerify).getNonce())
assertThat(worldStateArchive.getWorldState().get(addressToVerify).getNonce())
.isEqualTo(persistedNonceForAccount);
LongStream.rangeClosed(startBlock, endBlock)

View File

@@ -16,6 +16,7 @@ package org.hyperledger.besu.consensus.merge.blockcreation;
import static java.util.stream.Collectors.joining;
import static org.hyperledger.besu.consensus.merge.blockcreation.MergeMiningCoordinator.ForkchoiceResult.Status.INVALID;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndUpdateNodeHead;
import org.hyperledger.besu.consensus.merge.MergeContext;
import org.hyperledger.besu.consensus.merge.PayloadWrapper;
@@ -679,7 +680,7 @@ public class MergeCoordinator implements MergeMiningCoordinator, BadChainListene
Optional<MutableWorldState> newWorldState =
protocolContext
.getWorldStateArchive()
.getMutable(newHead.getStateRoot(), newHead.getHash());
.getWorldState(withBlockHeaderAndUpdateNodeHead(newHead));
newWorldState.ifPresentOrElse(
mutableWorldState ->

View File

@@ -189,7 +189,7 @@ public class MergeCoordinatorTest implements MergeGenesisConfigHelper {
protocolContext =
new ProtocolContext(blockchain, worldStateArchive, mergeContext, badBlockManager);
var mutable = worldStateArchive.getMutable();
var mutable = worldStateArchive.getWorldState();
genesisState.writeStateTo(mutable);
mutable.persist(null);

View File

@@ -85,7 +85,7 @@ public class MergeReorgTest implements MergeGenesisConfigHelper {
@BeforeEach
public void setUp() {
var mutable = worldStateArchive.getMutable();
var mutable = worldStateArchive.getWorldState();
genesisState.writeStateTo(mutable);
mutable.persist(null);
mergeContext.setTerminalTotalDifficulty(Difficulty.of(1001));

View File

@@ -264,7 +264,7 @@ public class TestContextBuilder {
createInMemoryBlockchain(
genesisState.getBlock(),
BftBlockHeaderFunctions.forOnchainBlock(BFT_EXTRA_DATA_ENCODER));
genesisState.writeStateTo(worldStateArchive.getMutable());
genesisState.writeStateTo(worldStateArchive.getWorldState());
} catch (IOException e) {
throw new IllegalStateException(e);
}

View File

@@ -85,7 +85,7 @@ public class JsonRpcTestMethodsFactory {
this.importer = importer;
this.blockchain = createInMemoryBlockchain(importer.getGenesisBlock());
this.stateArchive = createInMemoryWorldStateArchive();
this.importer.getGenesisState().writeStateTo(stateArchive.getMutable());
this.importer.getGenesisState().writeStateTo(stateArchive.getWorldState());
this.context =
new ProtocolContext(
blockchain, stateArchive, mock(ConsensusContext.class), new BadBlockManager());

View File

@@ -67,7 +67,7 @@ public class EthGetBlockByNumberLatestDesyncIntegrationTest {
MutableBlockchain chain =
InMemoryKeyValueStorageProvider.createInMemoryBlockchain(importer.getGenesisBlock());
WorldStateArchive state = InMemoryKeyValueStorageProvider.createInMemoryWorldStateArchive();
importer.getGenesisState().writeStateTo(state.getMutable());
importer.getGenesisState().writeStateTo(state.getWorldState());
ProtocolContext context =
new ProtocolContext(chain, state, mock(ConsensusContext.class), new BadBlockManager());

View File

@@ -15,6 +15,7 @@
package org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods;
import static org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.RpcErrorType.UNKNOWN_BLOCK;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndUpdateNodeHead;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext;
@@ -29,7 +30,7 @@ import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.RpcErrorType;
import org.hyperledger.besu.ethereum.api.query.BlockchainQueries;
import org.hyperledger.besu.ethereum.chain.MutableBlockchain;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.trie.diffbased.common.DiffBasedWorldStateProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.DiffBasedWorldStateProvider;
import java.util.Optional;
@@ -137,14 +138,14 @@ public class DebugSetHead extends AbstractBlockParameterOrBlockHashMethod {
interimHead.ifPresent(
it -> {
blockchain.rewindToBlock(it.getBlockHash());
archive.getMutable(it.getStateRoot(), it.getBlockHash());
archive.getWorldState(withBlockHeaderAndUpdateNodeHead(it));
LOG.info("incrementally rolled worldstate to {}", it.toLogString());
});
currentHead = interimHead;
} else {
blockchain.rewindToBlock(target.getBlockHash());
archive.getMutable(target.getStateRoot(), target.getBlockHash());
archive.getWorldState(withBlockHeaderAndUpdateNodeHead(target));
currentHead = Optional.of(target);
LOG.info("finished rolling worldstate to {}", target.toLogString());
}

View File

@@ -14,6 +14,8 @@
*/
package org.hyperledger.besu.ethereum.api.jsonrpc.internal.processor.privateProcessor;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withStateRootAndBlockHashAndUpdateNodeHead;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.api.query.BlockchainQueries;
@@ -49,11 +51,12 @@ public class PrivateTracer {
final MutableWorldState disposablePrivateState =
privacyParameters
.getPrivateWorldStateArchive()
.getMutable(
privacyController
.getStateRootByBlockNumber(privacyGroupId, enclaveKey, blockNumber)
.get(),
parentHash)
.getWorldState(
withStateRootAndBlockHashAndUpdateNodeHead(
privacyController
.getStateRootByBlockNumber(privacyGroupId, enclaveKey, blockNumber)
.get(),
parentHash))
.get();
return blockchainQueries.getAndMapWorldState(

View File

@@ -17,6 +17,7 @@ package org.hyperledger.besu.ethereum.api.query;
import static com.google.common.base.Preconditions.checkArgument;
import static org.hyperledger.besu.ethereum.api.query.cache.TransactionLogBloomCacher.BLOCKS_PER_BLOOM_CACHE;
import static org.hyperledger.besu.ethereum.mainnet.feemarket.ExcessBlobGasCalculator.calculateExcessBlobGasForParent;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndNoUpdateNodeHead;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
@@ -1002,7 +1003,10 @@ public class BlockchainQueries {
.getBlockHeader(blockHash)
.flatMap(
blockHeader -> {
try (var ws = worldStateArchive.getMutable(blockHeader, false).orElse(null)) {
try (var ws =
worldStateArchive
.getWorldState(withBlockHeaderAndNoUpdateNodeHead(blockHeader))
.orElse(null)) {
if (ws != null) {
return mapper.apply(ws);
}

View File

@@ -228,7 +228,6 @@ public class StateBackupService {
header.get().getStateRoot(),
Function.identity(),
Function.identity());
accountTrie.visitLeafs(this::visitAccount);
backupStatus.currentAccount = null;
}

View File

@@ -90,7 +90,7 @@ public class DebugSetHeadTest extends AbstractJsonRpcHttpServiceTest {
// assert the chain moved, and the worldstate did not
assertThat(newChainTip).isEqualTo(blockOne);
assertThat(archive.getMutable().rootHash()).isEqualTo(chainTip.getStateRoot());
assertThat(archive.getWorldState().rootHash()).isEqualTo(chainTip.getStateRoot());
}
@ParameterizedTest
@@ -116,7 +116,7 @@ public class DebugSetHeadTest extends AbstractJsonRpcHttpServiceTest {
// assert the chain moved, and the worldstate did not
assertThat(newChainTip).isEqualTo(blockOne);
assertThat(archive.getMutable().rootHash()).isEqualTo(chainTip.getStateRoot());
assertThat(archive.getWorldState().rootHash()).isEqualTo(chainTip.getStateRoot());
}
@ParameterizedTest
@@ -142,7 +142,7 @@ public class DebugSetHeadTest extends AbstractJsonRpcHttpServiceTest {
// assert both the chain and worldstate moved to block one
assertThat(newChainTip).isEqualTo(blockOne);
assertThat(archive.getMutable().rootHash()).isEqualTo(blockOne.getStateRoot());
assertThat(archive.getWorldState().rootHash()).isEqualTo(blockOne.getStateRoot());
}
@Test
@@ -167,7 +167,7 @@ public class DebugSetHeadTest extends AbstractJsonRpcHttpServiceTest {
// assert neither the chain nor the worldstate moved
assertThat(newChainTip).isEqualTo(chainTip);
assertThat(archive.getMutable().rootHash()).isEqualTo(chainTip.getStateRoot());
assertThat(archive.getWorldState().rootHash()).isEqualTo(chainTip.getStateRoot());
}
private JsonRpcRequestContext debugSetHead(

View File

@@ -16,6 +16,7 @@ package org.hyperledger.besu.ethereum.blockcreation;
import static org.hyperledger.besu.ethereum.core.BlockHeaderBuilder.createPending;
import static org.hyperledger.besu.ethereum.mainnet.feemarket.ExcessBlobGasCalculator.calculateExcessBlobGasForParent;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndNoUpdateNodeHead;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.BlobGas;
@@ -404,7 +405,7 @@ public abstract class AbstractBlockCreator implements AsyncBlockCreator {
final Hash parentStateRoot = parentHeader.getStateRoot();
return protocolContext
.getWorldStateArchive()
.getMutable(parentHeader, false)
.getWorldState(withBlockHeaderAndNoUpdateNodeHead(parentHeader))
.orElseThrow(
() -> {
LOG.info("Unable to create block because world state is not available");

View File

@@ -76,6 +76,7 @@ import org.hyperledger.besu.ethereum.processing.TransactionProcessingResult;
import org.hyperledger.besu.ethereum.storage.keyvalue.KeyValueStoragePrefixedKeyBlockchainStorage;
import org.hyperledger.besu.ethereum.storage.keyvalue.VariablesKeyValueStorage;
import org.hyperledger.besu.ethereum.transaction.TransactionInvalidReason;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams;
import org.hyperledger.besu.evm.gascalculator.GasCalculator;
import org.hyperledger.besu.evm.gascalculator.LondonGasCalculator;
import org.hyperledger.besu.evm.internal.EvmConfiguration;
@@ -182,7 +183,7 @@ public abstract class AbstractBlockTransactionSelectorTest {
worldStateUpdater.createAccount(sender, 0, Wei.of(1_000_000_000L));
worldStateUpdater.commit();
when(protocolContext.getWorldStateArchive().getMutable(any(), anyBoolean()))
when(protocolContext.getWorldStateArchive().getWorldState(any(WorldStateQueryParams.class)))
.thenReturn(Optional.of(worldState));
when(ethContext.getEthPeers().subscribeConnect(any())).thenReturn(1L);
when(ethScheduler.scheduleBlockCreationTask(any(Runnable.class)))

View File

@@ -250,7 +250,7 @@ class PoWBlockCreatorTest extends AbstractBlockCreatorTest {
ethScheduler);
final MutableWorldState mutableWorldState =
executionContextTestFixture.getStateArchive().getMutable();
executionContextTestFixture.getStateArchive().getWorldState();
assertThat(mutableWorldState.get(BLOCK_1_COINBASE)).isNull();
final ProcessableBlockHeader header =
@@ -327,7 +327,7 @@ class PoWBlockCreatorTest extends AbstractBlockCreatorTest {
ethScheduler);
final MutableWorldState mutableWorldState =
executionContextTestFixture.getStateArchive().getMutable();
executionContextTestFixture.getStateArchive().getWorldState();
assertThat(mutableWorldState.get(BLOCK_1_COINBASE)).isNull();
final ProcessableBlockHeader header =

View File

@@ -37,7 +37,7 @@ public class EntriesFromIntegrationTest {
@Test
@SuppressWarnings("MathAbsoluteRandom")
public void shouldCollectStateEntries() {
final MutableWorldState worldState = createInMemoryWorldStateArchive().getMutable();
final MutableWorldState worldState = createInMemoryWorldStateArchive().getWorldState();
final WorldUpdater updater = worldState.updater();
MutableAccount account = updater.getOrCreate(Address.fromHexString("0x56"));
final Map<Bytes32, AccountStorageEntry> expectedValues = new TreeMap<>();

View File

@@ -16,6 +16,7 @@ package org.hyperledger.besu.ethereum.vm;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.entry;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withStateRootAndBlockHashAndUpdateNodeHead;
import org.hyperledger.besu.config.GenesisConfig;
import org.hyperledger.besu.crypto.KeyPair;
@@ -102,7 +103,9 @@ public class TraceTransactionIntegrationTest {
final BlockHeader genesisBlockHeader = genesisBlock.getHeader();
final MutableWorldState worldState =
worldStateArchive
.getMutable(genesisBlockHeader.getStateRoot(), genesisBlockHeader.getHash())
.getWorldState(
withStateRootAndBlockHashAndUpdateNodeHead(
genesisBlockHeader.getStateRoot(), genesisBlockHeader.getHash()))
.get();
final WorldUpdater createTransactionUpdater = worldState.updater();
TransactionProcessingResult result =
@@ -177,7 +180,9 @@ public class TraceTransactionIntegrationTest {
final BlockHeader genesisBlockHeader = genesisBlock.getHeader();
transactionProcessor.processTransaction(
worldStateArchive
.getMutable(genesisBlockHeader.getStateRoot(), genesisBlockHeader.getHash())
.getWorldState(
withStateRootAndBlockHashAndUpdateNodeHead(
genesisBlockHeader.getStateRoot(), genesisBlockHeader.getHash()))
.get()
.updater(),
genesisBlockHeader,

View File

@@ -50,7 +50,7 @@ public class TransientStorageOperationBenchmark {
final Blockchain blockchain = mock(Blockchain.class);
final WorldStateArchive worldStateArchive = createInMemoryWorldStateArchive();
final WorldUpdater worldStateUpdater = worldStateArchive.getMutable().updater();
final WorldUpdater worldStateUpdater = worldStateArchive.getWorldState().updater();
final BlockHeader blockHeader = new BlockHeaderTestFixture().buildHeader();
final MessageFrame benchmarkFrame =
new MessageFrameTestFixture()

View File

@@ -28,6 +28,7 @@ import org.hyperledger.besu.ethereum.mainnet.BlockProcessor;
import org.hyperledger.besu.ethereum.mainnet.BodyValidationMode;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import org.hyperledger.besu.ethereum.trie.MerkleTrieException;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams;
import org.hyperledger.besu.plugin.services.exception.StorageException;
import java.util.ArrayList;
@@ -115,7 +116,7 @@ public class MainnetBlockValidator implements BlockValidator {
final Block block,
final HeaderValidationMode headerValidationMode,
final HeaderValidationMode ommerValidationMode,
final boolean shouldPersist,
final boolean shouldUpdateHead,
final boolean shouldRecordBadBlock) {
final BlockHeader header = block.getHeader();
@@ -148,8 +149,14 @@ public class MainnetBlockValidator implements BlockValidator {
handleFailedBlockProcessing(block, retval, false);
return retval;
}
final WorldStateQueryParams worldStateQueryParams =
WorldStateQueryParams.newBuilder()
.withBlockHeader(parentHeader)
.withShouldWorldStateUpdateHead(shouldUpdateHead)
.build();
try (final var worldState =
context.getWorldStateArchive().getMutable(parentHeader, shouldPersist).orElse(null)) {
context.getWorldStateArchive().getWorldState(worldStateQueryParams).orElse(null)) {
if (worldState == null) {
var retval =

View File

@@ -28,7 +28,7 @@ public interface MutableWorldState extends WorldState, MutableWorldView {
*/
void persist(BlockHeader blockHeader);
default MutableWorldState freeze() {
default MutableWorldState freezeStorage() {
// no op
throw new UnsupportedOperationException("cannot freeze");
}

View File

@@ -139,7 +139,7 @@ public class ParallelizedConcurrentTransactionProcessor {
try (final DiffBasedWorldState roundWorldState =
new BonsaiWorldState(
(BonsaiWorldState) worldState, new NoopBonsaiCachedMerkleTrieLoader())) {
roundWorldState.freeze(); // make the clone frozen
roundWorldState.freezeStorage(); // make the clone frozen
final ParallelizedTransactionContext.Builder contextBuilder =
new ParallelizedTransactionContext.Builder();
final DiffBasedWorldStateUpdateAccumulator<?> roundWorldStateUpdater =

View File

@@ -18,6 +18,7 @@ import static org.hyperledger.besu.ethereum.core.PrivacyParameters.FLEXIBLE_PRIV
import static org.hyperledger.besu.ethereum.mainnet.PrivateStateUtils.KEY_IS_PERSISTING_PRIVATE_STATE;
import static org.hyperledger.besu.ethereum.mainnet.PrivateStateUtils.KEY_PRIVATE_METADATA_UPDATER;
import static org.hyperledger.besu.ethereum.mainnet.PrivateStateUtils.KEY_TRANSACTION_HASH;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withStateRootAndUpdateNodeHead;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.enclave.Enclave;
@@ -156,7 +157,7 @@ public class FlexiblePrivacyPrecompiledContract extends PrivacyPrecompiledContra
privateStateRootResolver.resolveLastStateRoot(privacyGroupId, privateMetadataUpdater);
final MutableWorldState disposablePrivateState =
privateWorldStateArchive.getMutable(lastRootHash, null).get();
privateWorldStateArchive.getWorldState(withStateRootAndUpdateNodeHead(lastRootHash)).get();
final WorldUpdater privateWorldStateUpdater = disposablePrivateState.updater();

View File

@@ -14,6 +14,8 @@
*/
package org.hyperledger.besu.ethereum.mainnet.precompiles.privacy;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withStateRootAndUpdateNodeHead;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.MutableWorldState;
import org.hyperledger.besu.ethereum.core.PrivacyParameters;
@@ -81,7 +83,7 @@ public class PrivacyPluginPrecompiledContract extends PrivacyPrecompiledContract
privateStateRootResolver.resolveLastStateRoot(privacyGroupId, privateMetadataUpdater);
final MutableWorldState disposablePrivateState =
privateWorldStateArchive.getMutable(lastRootHash, null).get();
privateWorldStateArchive.getWorldState(withStateRootAndUpdateNodeHead(lastRootHash)).get();
final WorldUpdater privateWorldStateUpdater = disposablePrivateState.updater();

View File

@@ -18,6 +18,7 @@ import static org.hyperledger.besu.ethereum.mainnet.PrivateStateUtils.KEY_IS_PER
import static org.hyperledger.besu.ethereum.mainnet.PrivateStateUtils.KEY_PRIVATE_METADATA_UPDATER;
import static org.hyperledger.besu.ethereum.mainnet.PrivateStateUtils.KEY_TRANSACTION_HASH;
import static org.hyperledger.besu.ethereum.privacy.PrivateStateRootResolver.EMPTY_ROOT_HASH;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withStateRootAndUpdateNodeHead;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
@@ -172,7 +173,7 @@ public class PrivacyPrecompiledContract extends AbstractPrecompiledContract {
privateStateRootResolver.resolveLastStateRoot(privacyGroupId, privateMetadataUpdater);
final MutableWorldState disposablePrivateState =
privateWorldStateArchive.getMutable(lastRootHash, null).get();
privateWorldStateArchive.getWorldState(withStateRootAndUpdateNodeHead(lastRootHash)).get();
final WorldUpdater privateWorldStateUpdater = disposablePrivateState.updater();

View File

@@ -18,6 +18,7 @@ import static org.hyperledger.besu.ethereum.core.PrivacyParameters.FLEXIBLE_PRIV
import static org.hyperledger.besu.ethereum.privacy.group.FlexibleGroupManagement.CAN_EXECUTE_METHOD_SIGNATURE;
import static org.hyperledger.besu.ethereum.privacy.group.FlexibleGroupManagement.GET_PARTICIPANTS_METHOD_SIGNATURE;
import static org.hyperledger.besu.ethereum.privacy.group.FlexibleGroupManagement.GET_VERSION_METHOD_SIGNATURE;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withStateRootAndBlockHashAndUpdateNodeHead;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.crypto.SignatureAlgorithm;
@@ -107,7 +108,11 @@ public class FlexiblePrivacyGroupContract {
final Bytes privacyGroupId = Bytes.fromBase64String(base64privacyGroupId);
final MutableWorldState localMutableState =
privateWorldStateArchive.getMutable(disposablePrivateState.rootHash(), null).get();
privateWorldStateArchive
.getWorldState(
withStateRootAndBlockHashAndUpdateNodeHead(
disposablePrivateState.rootHash(), null))
.get();
final WorldUpdater updater = localMutableState.updater();
final PrivateTransaction privateTransaction =
buildTransaction(privacyGroupId, privateWorldStateUpdater, callData);

View File

@@ -15,6 +15,7 @@
package org.hyperledger.besu.ethereum.privacy;
import static org.hyperledger.besu.ethereum.privacy.PrivateStateRootResolver.EMPTY_ROOT_HASH;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withStateRootAndUpdateNodeHead;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
@@ -125,7 +126,9 @@ public class PrivateGroupRehydrationBlockProcessor {
privateStateRootResolver.resolveLastStateRoot(privacyGroupId, metadataUpdater);
final MutableWorldState disposablePrivateState =
privateWorldStateArchive.getMutable(lastRootHash, null).get();
privateWorldStateArchive
.getWorldState(withStateRootAndUpdateNodeHead(lastRootHash))
.get();
final WorldUpdater privateWorldStateUpdater = disposablePrivateState.updater();
if (lastRootHash.equals(EMPTY_ROOT_HASH)) {

View File

@@ -14,6 +14,8 @@
*/
package org.hyperledger.besu.ethereum.privacy;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndUpdateNodeHead;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.chain.Blockchain;
import org.hyperledger.besu.ethereum.chain.TransactionLocation;
@@ -156,7 +158,8 @@ public class PrivateStateRehydration {
.getBlockHeader(blockHeader.getParentHash())
.flatMap(
header ->
publicWorldStateArchive.getMutable(header.getStateRoot(), header.getHash()))
publicWorldStateArchive.getWorldState(
withBlockHeaderAndUpdateNodeHead(header)))
.orElseThrow(RuntimeException::new);
privateGroupRehydrationBlockProcessor.processBlock(

View File

@@ -14,6 +14,9 @@
*/
package org.hyperledger.besu.ethereum.privacy;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndUpdateNodeHead;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withStateRootAndBlockHashAndUpdateNodeHead;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.crypto.SignatureAlgorithm;
import org.hyperledger.besu.crypto.SignatureAlgorithmFactory;
@@ -105,7 +108,7 @@ public class PrivateTransactionSimulator {
}
final MutableWorldState publicWorldState =
worldStateArchive.getMutable(header.getStateRoot(), header.getHash()).orElse(null);
worldStateArchive.getWorldState(withBlockHeaderAndUpdateNodeHead(header)).orElse(null);
if (publicWorldState == null) {
return Optional.empty();
}
@@ -118,7 +121,8 @@ public class PrivateTransactionSimulator {
final MutableWorldState disposablePrivateState =
privacyParameters
.getPrivateWorldStateArchive()
.getMutable(lastRootHash, header.getHash())
.getWorldState(
withStateRootAndBlockHashAndUpdateNodeHead(lastRootHash, header.getHash()))
.get();
final PrivateTransaction transaction =

View File

@@ -15,6 +15,7 @@
package org.hyperledger.besu.ethereum.privacy.storage.migration;
import static org.hyperledger.besu.ethereum.privacy.storage.PrivateStateKeyValueStorage.SCHEMA_VERSION_1_4_0;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndUpdateNodeHead;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
@@ -102,7 +103,8 @@ public class PrivateStorageMigration {
.getBlockHeader(blockHeader.getParentHash())
.flatMap(
header ->
publicWorldStateArchive.getMutable(header.getStateRoot(), header.getHash()))
publicWorldStateArchive.getWorldState(
withBlockHeaderAndUpdateNodeHead(header)))
.orElseThrow(PrivateStorageMigrationException::new);
final List<Transaction> transactionsToProcess =

View File

@@ -14,6 +14,8 @@
*/
package org.hyperledger.besu.ethereum.transaction;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndNoUpdateNodeHead;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.datatypes.StateOverride;
@@ -91,7 +93,7 @@ public class BlockSimulator {
final BlockHeader header, final List<? extends BlockStateCall> blockStateCalls) {
try (final MutableWorldState ws =
worldStateArchive
.getMutable(header, false)
.getWorldState(withBlockHeaderAndNoUpdateNodeHead(header))
.orElseThrow(
() ->
new IllegalArgumentException(

View File

@@ -16,6 +16,7 @@ package org.hyperledger.besu.ethereum.transaction;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.hyperledger.besu.ethereum.mainnet.feemarket.ExcessBlobGasCalculator.calculateExcessBlobGasForParent;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndNoUpdateNodeHead;
import org.hyperledger.besu.crypto.SECPSignature;
import org.hyperledger.besu.crypto.SignatureAlgorithm;
@@ -210,7 +211,7 @@ public class TransactionSimulator {
final Hash parentStateRoot = parentHeader.getStateRoot();
return worldStateArchive
.getMutable(parentHeader, false)
.getWorldState(withBlockHeaderAndNoUpdateNodeHead(parentHeader))
.orElseThrow(
() ->
new IllegalArgumentException(
@@ -337,7 +338,7 @@ public class TransactionSimulator {
private MutableWorldState getWorldState(final BlockHeader header) {
return worldStateArchive
.getMutable(header, false)
.getWorldState(withBlockHeaderAndNoUpdateNodeHead(header))
.orElseThrow(
() ->
new IllegalArgumentException(
@@ -586,7 +587,7 @@ public class TransactionSimulator {
public Optional<Boolean> doesAddressExistAtHead(final Address address) {
final BlockHeader header = blockchain.getChainHeadHeader();
try (final MutableWorldState worldState =
worldStateArchive.getMutable(header, false).orElseThrow()) {
worldStateArchive.getWorldState(withBlockHeaderAndNoUpdateNodeHead(header)).orElseThrow()) {
return doesAddressExist(worldState, address, header);
} catch (final Exception ex) {
return Optional.empty();

View File

@@ -14,6 +14,8 @@
*/
package org.hyperledger.besu.ethereum.trie.common;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig.createStatefulConfigWithTrie;
import org.hyperledger.besu.ethereum.core.MutableWorldState;
import org.hyperledger.besu.ethereum.storage.keyvalue.KeyValueStorageProvider;
import org.hyperledger.besu.ethereum.storage.keyvalue.WorldStatePreimageKeyValueStorage;
@@ -22,7 +24,6 @@ import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.cache.NoOpBonsaiCache
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.worldview.BonsaiWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.trielog.NoOpTrieLogManager;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.trie.forest.storage.ForestWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.forest.worldview.ForestMutableWorldState;
import org.hyperledger.besu.ethereum.worldstate.DataStorageConfiguration;
@@ -74,7 +75,7 @@ public class GenesisWorldStateProvider {
new NoOpBonsaiCachedWorldStorageManager(bonsaiWorldStateKeyValueStorage),
new NoOpTrieLogManager(),
EvmConfiguration.DEFAULT,
new DiffBasedWorldStateConfig());
createStatefulConfigWithTrie());
}
/**

View File

@@ -23,9 +23,8 @@ import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.cache.BonsaiCachedMer
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.cache.BonsaiCachedWorldStorageManager;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.worldview.BonsaiWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.DiffBasedWorldStateProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.DiffBasedWorldStateProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.common.trielog.TrieLogManager;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.trie.patricia.StoredMerklePatriciaTrie;
import org.hyperledger.besu.evm.internal.EvmConfiguration;
import org.hyperledger.besu.plugin.ServiceManager;
@@ -59,11 +58,9 @@ public class BonsaiWorldStateProvider extends DiffBasedWorldStateProvider {
this.bonsaiCachedMerkleTrieLoader = bonsaiCachedMerkleTrieLoader;
this.worldStateHealerSupplier = worldStateHealerSupplier;
provideCachedWorldStorageManager(
new BonsaiCachedWorldStorageManager(
this, worldStateKeyValueStorage, this::cloneBonsaiWorldStateConfig));
loadPersistedState(
new BonsaiWorldState(
this, worldStateKeyValueStorage, evmConfiguration, defaultWorldStateConfig));
new BonsaiCachedWorldStorageManager(this, worldStateKeyValueStorage, worldStateConfig));
loadHeadWorldState(
new BonsaiWorldState(this, worldStateKeyValueStorage, evmConfiguration, worldStateConfig));
}
@VisibleForTesting
@@ -79,9 +76,8 @@ public class BonsaiWorldStateProvider extends DiffBasedWorldStateProvider {
this.bonsaiCachedMerkleTrieLoader = bonsaiCachedMerkleTrieLoader;
this.worldStateHealerSupplier = worldStateHealerSupplier;
provideCachedWorldStorageManager(bonsaiCachedWorldStorageManager);
loadPersistedState(
new BonsaiWorldState(
this, worldStateKeyValueStorage, evmConfiguration, defaultWorldStateConfig));
loadHeadWorldState(
new BonsaiWorldState(this, worldStateKeyValueStorage, evmConfiguration, worldStateConfig));
}
public BonsaiCachedMerkleTrieLoader getCachedMerkleTrieLoader() {
@@ -113,7 +109,7 @@ public class BonsaiWorldStateProvider extends DiffBasedWorldStateProvider {
}
return node;
},
persistedState.getWorldStateRootHash(),
headWorldState.getWorldStateRootHash(),
Function.identity(),
Function.identity());
try {
@@ -154,10 +150,6 @@ public class BonsaiWorldStateProvider extends DiffBasedWorldStateProvider {
getBonsaiWorldStateKeyValueStorage().downgradeToPartialFlatDbMode();
}
private DiffBasedWorldStateConfig cloneBonsaiWorldStateConfig() {
return new DiffBasedWorldStateConfig(defaultWorldStateConfig);
}
@Override
public void heal(final Optional<Address> maybeAccountToRepair, final Bytes location) {
worldStateHealerSupplier.get().heal(maybeAccountToRepair, location);

View File

@@ -19,22 +19,20 @@ import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiSnapsho
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldStateLayerStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.worldview.BonsaiWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.DiffBasedWorldStateProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.common.cache.DiffBasedCachedWorldStorageManager;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.DiffBasedWorldStateProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.common.storage.DiffBasedWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig;
import org.hyperledger.besu.evm.internal.EvmConfiguration;
import java.util.function.Supplier;
public class BonsaiCachedWorldStorageManager extends DiffBasedCachedWorldStorageManager {
public BonsaiCachedWorldStorageManager(
final BonsaiWorldStateProvider archive,
final DiffBasedWorldStateKeyValueStorage worldStateKeyValueStorage,
final Supplier<DiffBasedWorldStateConfig> defaultBonsaiWorldStateConfigSupplier) {
super(archive, worldStateKeyValueStorage, defaultBonsaiWorldStateConfigSupplier);
final WorldStateConfig worldStateConfig) {
super(archive, worldStateKeyValueStorage, worldStateConfig);
}
@Override
@@ -46,7 +44,7 @@ public class BonsaiCachedWorldStorageManager extends DiffBasedCachedWorldStorage
(BonsaiWorldStateProvider) archive,
(BonsaiWorldStateKeyValueStorage) worldStateKeyValueStorage,
evmConfiguration,
defaultBonsaiWorldStateConfigSupplier.get());
WorldStateConfig.newBuilder(worldStateConfig).build());
}
@Override

View File

@@ -18,7 +18,7 @@ import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig;
import java.util.Optional;
import java.util.function.Function;
@@ -27,7 +27,7 @@ public class NoOpBonsaiCachedWorldStorageManager extends BonsaiCachedWorldStorag
public NoOpBonsaiCachedWorldStorageManager(
final BonsaiWorldStateKeyValueStorage bonsaiWorldStateKeyValueStorage) {
super(null, bonsaiWorldStateKeyValueStorage, DiffBasedWorldStateConfig::new);
super(null, bonsaiWorldStateKeyValueStorage, WorldStateConfig.createStatefulConfigWithTrie());
}
@Override

View File

@@ -35,7 +35,7 @@ import org.hyperledger.besu.ethereum.trie.diffbased.common.cache.DiffBasedCached
import org.hyperledger.besu.ethereum.trie.diffbased.common.storage.DiffBasedWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.common.trielog.TrieLogManager;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.accumulator.DiffBasedWorldStateUpdateAccumulator;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.accumulator.preload.StorageConsumingMap;
import org.hyperledger.besu.ethereum.trie.patricia.StoredMerklePatriciaTrie;
@@ -66,14 +66,14 @@ public class BonsaiWorldState extends DiffBasedWorldState {
final BonsaiWorldStateProvider archive,
final BonsaiWorldStateKeyValueStorage worldStateKeyValueStorage,
final EvmConfiguration evmConfiguration,
final DiffBasedWorldStateConfig diffBasedWorldStateConfig) {
final WorldStateConfig worldStateConfig) {
this(
worldStateKeyValueStorage,
archive.getCachedMerkleTrieLoader(),
archive.getCachedWorldStorageManager(),
archive.getTrieLogManager(),
evmConfiguration,
diffBasedWorldStateConfig);
worldStateConfig);
}
public BonsaiWorldState(
@@ -85,7 +85,7 @@ public class BonsaiWorldState extends DiffBasedWorldState {
worldState.cachedWorldStorageManager,
worldState.trieLogManager,
worldState.accumulator.getEvmConfiguration(),
new DiffBasedWorldStateConfig(worldState.worldStateConfig));
WorldStateConfig.newBuilder(worldState.worldStateConfig).build());
}
public BonsaiWorldState(
@@ -94,12 +94,8 @@ public class BonsaiWorldState extends DiffBasedWorldState {
final DiffBasedCachedWorldStorageManager cachedWorldStorageManager,
final TrieLogManager trieLogManager,
final EvmConfiguration evmConfiguration,
final DiffBasedWorldStateConfig diffBasedWorldStateConfig) {
super(
worldStateKeyValueStorage,
cachedWorldStorageManager,
trieLogManager,
diffBasedWorldStateConfig);
final WorldStateConfig worldStateConfig) {
super(worldStateKeyValueStorage, cachedWorldStorageManager, trieLogManager, worldStateConfig);
this.bonsaiCachedMerkleTrieLoader = bonsaiCachedMerkleTrieLoader;
this.worldStateKeyValueStorage = worldStateKeyValueStorage;
this.setAccumulator(
@@ -448,8 +444,8 @@ public class BonsaiWorldState extends DiffBasedWorldState {
}
@Override
public MutableWorldState freeze() {
this.worldStateConfig.setFrozen(true);
public MutableWorldState freezeStorage() {
this.isStorageFrozen = true;
this.worldStateKeyValueStorage = new BonsaiWorldStateLayerStorage(getWorldStateStorage());
return this;
}

View File

@@ -14,16 +14,18 @@
*/
package org.hyperledger.besu.ethereum.trie.diffbased.common.cache;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndNoUpdateNodeHead;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.chain.Blockchain;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.worldview.BonsaiWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.DiffBasedWorldStateProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.common.StorageSubscriber;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.DiffBasedWorldStateProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.common.storage.DiffBasedLayeredWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.common.storage.DiffBasedWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig;
import org.hyperledger.besu.evm.internal.EvmConfiguration;
import java.util.ArrayList;
@@ -34,7 +36,6 @@ import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import java.util.function.Supplier;
import com.github.benmanes.caffeine.cache.Cache;
import com.github.benmanes.caffeine.cache.Caffeine;
@@ -48,7 +49,7 @@ public abstract class DiffBasedCachedWorldStorageManager implements StorageSubsc
LoggerFactory.getLogger(DiffBasedCachedWorldStorageManager.class);
private final DiffBasedWorldStateProvider archive;
private final EvmConfiguration evmConfiguration;
protected final Supplier<DiffBasedWorldStateConfig> defaultBonsaiWorldStateConfigSupplier;
protected final WorldStateConfig worldStateConfig;
private final Cache<Hash, BlockHeader> stateRootToBlockHeaderCache =
Caffeine.newBuilder()
.maximumSize(RETAINED_LAYERS)
@@ -63,25 +64,25 @@ public abstract class DiffBasedCachedWorldStorageManager implements StorageSubsc
final DiffBasedWorldStateKeyValueStorage worldStateKeyValueStorage,
final Map<Bytes32, DiffBasedCachedWorldView> cachedWorldStatesByHash,
final EvmConfiguration evmConfiguration,
final Supplier<DiffBasedWorldStateConfig> defaultBonsaiWorldStateConfigSupplier) {
final WorldStateConfig worldStateConfig) {
worldStateKeyValueStorage.subscribe(this);
this.rootWorldStateStorage = worldStateKeyValueStorage;
this.cachedWorldStatesByHash = cachedWorldStatesByHash;
this.archive = archive;
this.evmConfiguration = evmConfiguration;
this.defaultBonsaiWorldStateConfigSupplier = defaultBonsaiWorldStateConfigSupplier;
this.worldStateConfig = worldStateConfig;
}
public DiffBasedCachedWorldStorageManager(
final DiffBasedWorldStateProvider archive,
final DiffBasedWorldStateKeyValueStorage worldStateKeyValueStorage,
final Supplier<DiffBasedWorldStateConfig> defaultBonsaiWorldStateConfigSupplier) {
final WorldStateConfig worldStateConfig) {
this(
archive,
worldStateKeyValueStorage,
new ConcurrentHashMap<>(),
EvmConfiguration.DEFAULT,
defaultBonsaiWorldStateConfigSupplier);
worldStateConfig);
}
public synchronized void addCachedLayer(
@@ -92,7 +93,7 @@ public abstract class DiffBasedCachedWorldStorageManager implements StorageSubsc
Optional.ofNullable(this.cachedWorldStatesByHash.get(blockHeader.getBlockHash()));
if (cachedDiffBasedWorldView.isPresent()) {
// only replace if it is a layered storage
if (forWorldState.isPersisted()
if (forWorldState.isModifyingHeadWorldState()
&& cachedDiffBasedWorldView.get().getWorldStateStorage()
instanceof DiffBasedLayeredWorldStateKeyValueStorage) {
LOG.atDebug()
@@ -111,7 +112,7 @@ public abstract class DiffBasedCachedWorldStorageManager implements StorageSubsc
.addArgument(blockHeader::toLogString)
.addArgument(worldStateRootHash::toShortHexString)
.log();
if (forWorldState.isPersisted()) {
if (forWorldState.isModifyingHeadWorldState()) {
cachedWorldStatesByHash.put(
blockHeader.getHash(),
new DiffBasedCachedWorldView(
@@ -252,7 +253,9 @@ public abstract class DiffBasedCachedWorldStorageManager implements StorageSubsc
() -> {
// if not cached already, maybe fetch and cache this worldstate
var maybeWorldState =
archive.getMutable(header, false).map(BonsaiWorldState.class::cast);
archive
.getWorldState(withBlockHeaderAndNoUpdateNodeHead(header))
.map(BonsaiWorldState.class::cast);
maybeWorldState.ifPresent(
ws -> addCachedLayer(header, header.getStateRoot(), ws));
return maybeWorldState.map(BonsaiWorldState::getWorldStateStorage);

View File

@@ -12,7 +12,9 @@
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.ethereum.trie.diffbased.common;
package org.hyperledger.besu.ethereum.trie.diffbased.common.provider;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndNoUpdateNodeHead;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
@@ -26,7 +28,7 @@ import org.hyperledger.besu.ethereum.trie.diffbased.common.cache.DiffBasedCached
import org.hyperledger.besu.ethereum.trie.diffbased.common.storage.DiffBasedWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.common.trielog.TrieLogManager;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.accumulator.DiffBasedWorldStateUpdateAccumulator;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
@@ -52,27 +54,24 @@ public abstract class DiffBasedWorldStateProvider implements WorldStateArchive {
protected final TrieLogManager trieLogManager;
protected DiffBasedCachedWorldStorageManager cachedWorldStorageManager;
protected DiffBasedWorldState persistedState;
protected DiffBasedWorldState headWorldState;
protected final DiffBasedWorldStateKeyValueStorage worldStateKeyValueStorage;
protected final DiffBasedWorldStateConfig defaultWorldStateConfig;
// Configuration that will be shared by all instances of world state at their creation
protected final WorldStateConfig worldStateConfig;
public DiffBasedWorldStateProvider(
final DiffBasedWorldStateKeyValueStorage worldStateKeyValueStorage,
final Blockchain blockchain,
final Optional<Long> maxLayersToLoad,
final ServiceManager pluginContext) {
this.worldStateKeyValueStorage = worldStateKeyValueStorage;
// TODO: de-dup constructors
this.trieLogManager =
this(
worldStateKeyValueStorage,
blockchain,
new TrieLogManager(
blockchain,
worldStateKeyValueStorage,
maxLayersToLoad.orElse(DiffBasedCachedWorldStorageManager.RETAINED_LAYERS),
pluginContext);
this.blockchain = blockchain;
this.defaultWorldStateConfig = new DiffBasedWorldStateConfig();
pluginContext));
}
public DiffBasedWorldStateProvider(
@@ -81,10 +80,10 @@ public abstract class DiffBasedWorldStateProvider implements WorldStateArchive {
final TrieLogManager trieLogManager) {
this.worldStateKeyValueStorage = worldStateKeyValueStorage;
// TODO: de-dup constructors
this.trieLogManager = trieLogManager;
this.blockchain = blockchain;
this.defaultWorldStateConfig = new DiffBasedWorldStateConfig();
this.worldStateConfig = WorldStateConfig.newBuilder().build();
;
}
protected void provideCachedWorldStorageManager(
@@ -92,14 +91,14 @@ public abstract class DiffBasedWorldStateProvider implements WorldStateArchive {
this.cachedWorldStorageManager = cachedWorldStorageManager;
}
protected void loadPersistedState(final DiffBasedWorldState persistedState) {
this.persistedState = persistedState;
protected void loadHeadWorldState(final DiffBasedWorldState headWorldState) {
this.headWorldState = headWorldState;
blockchain
.getBlockHeader(persistedState.getWorldStateBlockHash())
.getBlockHeader(headWorldState.getWorldStateBlockHash())
.ifPresent(
blockHeader ->
this.cachedWorldStorageManager.addCachedLayer(
blockHeader, persistedState.getWorldStateRootHash(), persistedState));
blockHeader, headWorldState.getWorldStateRootHash(), headWorldState));
}
@Override
@@ -108,8 +107,8 @@ public abstract class DiffBasedWorldStateProvider implements WorldStateArchive {
.getWorldState(blockHash)
.or(
() -> {
if (blockHash.equals(persistedState.blockHash())) {
return Optional.of(persistedState);
if (blockHash.equals(headWorldState.blockHash())) {
return Optional.of(headWorldState);
} else {
return Optional.empty();
}
@@ -120,40 +119,120 @@ public abstract class DiffBasedWorldStateProvider implements WorldStateArchive {
@Override
public boolean isWorldStateAvailable(final Hash rootHash, final Hash blockHash) {
return cachedWorldStorageManager.contains(blockHash)
|| persistedState.blockHash().equals(blockHash)
|| headWorldState.blockHash().equals(blockHash)
|| worldStateKeyValueStorage.isWorldStateAvailable(rootHash, blockHash);
}
/**
* Gets a mutable world state based on the provided query parameters.
*
* <p>This method checks if the world state is configured to be stateful. If it is, it retrieves
* the full world state using the provided query parameters. If the world state is not configured
* to be full, the stateless one will be returned.
*
* <p>The method follows these steps: 1. Check if the world state is configured to be stateful. 2.
* If true, call {@link #getFullWorldState(WorldStateQueryParams)} with the query parameters. 3.
* If false, throw a RuntimeException indicating that stateless mode is not yet available.
*
* @param queryParams the query parameters
* @return the mutable world state, if available
* @throws RuntimeException if the world state is not configured to be stateful
*/
@Override
public Optional<MutableWorldState> getMutable(
final BlockHeader blockHeader, final boolean shouldPersistState) {
if (shouldPersistState) {
return getMutable(blockHeader.getStateRoot(), blockHeader.getHash());
public Optional<MutableWorldState> getWorldState(final WorldStateQueryParams queryParams) {
if (worldStateConfig.isStateful()) {
return getFullWorldState(queryParams);
} else {
final BlockHeader chainHeadBlockHeader = blockchain.getChainHeadHeader();
if (chainHeadBlockHeader.getNumber() - blockHeader.getNumber()
>= trieLogManager.getMaxLayersToLoad()) {
LOG.warn(
"Exceeded the limit of historical blocks that can be loaded ({}). If you need to make older historical queries, configure your `--bonsai-historical-block-limit`.",
trieLogManager.getMaxLayersToLoad());
return Optional.empty();
}
return cachedWorldStorageManager
.getWorldState(blockHeader.getHash())
.or(() -> cachedWorldStorageManager.getNearestWorldState(blockHeader))
.or(() -> cachedWorldStorageManager.getHeadWorldState(blockchain::getBlockHeader))
.flatMap(worldState -> rollMutableStateToBlockHash(worldState, blockHeader.getHash()))
.map(MutableWorldState::freeze);
throw new RuntimeException("stateless mode is not yet available");
}
}
/**
* Gets the head world state.
*
* <p>This method returns the head world state, which is the most recent state of the world.
*
* @return the head world state
*/
@Override
public synchronized Optional<MutableWorldState> getMutable(
final Hash rootHash, final Hash blockHash) {
return rollMutableStateToBlockHash(persistedState, blockHash);
public MutableWorldState getWorldState() {
return headWorldState;
}
Optional<MutableWorldState> rollMutableStateToBlockHash(
/**
* Gets the full world state based on the provided query parameters.
*
* <p>This method determines whether to retrieve the full world state from the head or from the
* cache based on the query parameters. If the query parameters indicate that the world state
* should update the head, the method retrieves the full world state from the head. Otherwise, it
* retrieves the full world state from the cache.
*
* <p>The method follows these steps: 1. Check if the query parameters indicate that the world
* state should update the head. 2. If true, call {@link #getFullWorldStateFromHead(Hash)} with
* the block hash from the query parameters. 3. If false, call {@link
* #getFullWorldStateFromCache(BlockHeader)} with the block header from the query parameters.
*
* @param queryParams the query parameters
* @return the stateful world state, if available
*/
private Optional<MutableWorldState> getFullWorldState(final WorldStateQueryParams queryParams) {
return queryParams.shouldWorldStateUpdateHead()
? getFullWorldStateFromHead(queryParams.getBlockHash())
: getFullWorldStateFromCache(queryParams.getBlockHeader());
}
/**
* Gets the full world state from the head based on the provided block hash.
*
* <p>This method attempts to roll the head world state to the specified block hash. If the block
* hash matches the block hash of the head world state, the head world state is returned.
* Otherwise, the method attempts to roll the full world state to the specified block hash.
*
* <p>The method follows these steps: 1. Check if the block hash matches the block hash of the
* head world state. 2. If it matches, return the head world state. 3. If it does not match,
* attempt to roll the full world state to the specified block hash.
*
* @param blockHash the block hash
* @return the full world state, if available
*/
private Optional<MutableWorldState> getFullWorldStateFromHead(final Hash blockHash) {
return rollFullWorldStateToBlockHash(headWorldState, blockHash);
}
/**
* Gets the full world state from the cache based on the provided block header.
*
* <p>This method attempts to retrieve the world state from the cache using the block header. If
* the block header is too old (i.e., the number of blocks between the chain head and the provided
* block header exceeds the maximum layers to load), a warning is logged and an empty Optional is
* returned.
*
* <p>The method follows these steps: 1. Check if the world state for the given block header is
* available in the cache. 2. If not, attempt to get the nearest world state from the cache. 3. If
* still not found, attempt to get the head world state. 4. If a world state is found, roll it to
* the block hash of the provided block header. 5. Freeze the world state and return it.
*
* @param blockHeader the block header
* @return the full world state, if available
*/
private Optional<MutableWorldState> getFullWorldStateFromCache(final BlockHeader blockHeader) {
final BlockHeader chainHeadBlockHeader = blockchain.getChainHeadHeader();
if (chainHeadBlockHeader.getNumber() - blockHeader.getNumber()
>= trieLogManager.getMaxLayersToLoad()) {
LOG.warn(
"Exceeded the limit of historical blocks that can be loaded ({}). If you need to make older historical queries, configure your `--bonsai-historical-block-limit`.",
trieLogManager.getMaxLayersToLoad());
return Optional.empty();
}
return cachedWorldStorageManager
.getWorldState(blockHeader.getHash())
.or(() -> cachedWorldStorageManager.getNearestWorldState(blockHeader))
.or(() -> cachedWorldStorageManager.getHeadWorldState(blockchain::getBlockHeader))
.flatMap(worldState -> rollFullWorldStateToBlockHash(worldState, blockHeader.getHash()))
.map(MutableWorldState::freezeStorage);
}
private Optional<MutableWorldState> rollFullWorldStateToBlockHash(
final DiffBasedWorldState mutableState, final Hash blockHash) {
if (blockHash.equals(mutableState.blockHash())) {
return Optional.of(mutableState);
@@ -251,18 +330,8 @@ public abstract class DiffBasedWorldStateProvider implements WorldStateArchive {
}
}
@Override
public MutableWorldState getMutable() {
return persistedState;
}
public DiffBasedWorldStateConfig getDefaultWorldStateConfig() {
return defaultWorldStateConfig;
}
public void disableTrie() {
defaultWorldStateConfig.setTrieDisabled(true);
worldStateKeyValueStorage.clearTrie();
public WorldStateConfig getWorldStateSharedSpec() {
return worldStateConfig;
}
public DiffBasedWorldStateKeyValueStorage getWorldStateKeyValueStorage() {
@@ -279,10 +348,10 @@ public abstract class DiffBasedWorldStateProvider implements WorldStateArchive {
@Override
public void resetArchiveStateTo(final BlockHeader blockHeader) {
persistedState.resetWorldStateTo(blockHeader);
headWorldState.resetWorldStateTo(blockHeader);
this.cachedWorldStorageManager.reset();
this.cachedWorldStorageManager.addCachedLayer(
blockHeader, persistedState.getWorldStateRootHash(), persistedState);
blockHeader, headWorldState.getWorldStateRootHash(), headWorldState);
}
@Override
@@ -292,7 +361,8 @@ public abstract class DiffBasedWorldStateProvider implements WorldStateArchive {
final List<UInt256> accountStorageKeys,
final Function<Optional<WorldStateProof>, ? extends Optional<U>> mapper) {
try (DiffBasedWorldState ws =
(DiffBasedWorldState) getMutable(blockHeader, false).orElse(null)) {
(DiffBasedWorldState)
getWorldState(withBlockHeaderAndNoUpdateNodeHead(blockHeader)).orElse(null)) {
if (ws != null) {
final WorldStateProofProvider worldStateProofProvider =
new WorldStateProofProvider(

View File

@@ -0,0 +1,237 @@
/*
* 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.trie.diffbased.common.provider;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import java.util.Objects;
import java.util.Optional;
/** Parameters for querying the world state. */
public class WorldStateQueryParams {
private final BlockHeader blockHeader;
private final boolean shouldWorldStateUpdateHead;
private final Hash blockHash;
private final Optional<Hash> stateRoot;
/**
* Private constructor to enforce the use of the Builder.
*
* @param builder the builder to create an instance of WorldStateQueryParams
*/
private WorldStateQueryParams(final Builder builder) {
this.blockHeader = builder.blockHeader;
this.shouldWorldStateUpdateHead = builder.shouldWorldStateUpdateHead;
this.blockHash = builder.blockHash;
this.stateRoot = builder.stateRoot;
}
/**
* Gets the block header.
*
* @return the block header
*/
public BlockHeader getBlockHeader() {
return blockHeader;
}
/**
* Checks if the world state should update the node head.
*
* @return true if the world state should update the node head, false otherwise
*/
public boolean shouldWorldStateUpdateHead() {
return shouldWorldStateUpdateHead;
}
/**
* Gets the block hash.
*
* @return the block hash
*/
public Hash getBlockHash() {
return blockHash;
}
/**
* Gets the state root.
*
* @return the state root
*/
public Optional<Hash> getStateRoot() {
return stateRoot;
}
/**
* Creates a new builder for WorldStateQueryParams.
*
* @return a new builder
*/
public static Builder newBuilder() {
return new Builder();
}
/**
* Creates an instance with a block header and updates the node head.
*
* @param blockHeader the block header
* @return an instance of WorldStateQueryParams
*/
public static WorldStateQueryParams withBlockHeaderAndUpdateNodeHead(
final BlockHeader blockHeader) {
return newBuilder().withBlockHeader(blockHeader).withShouldWorldStateUpdateHead(true).build();
}
/**
* Creates an instance with a block header and does not update the node head.
*
* @param blockHeader the block header
* @return an instance of WorldStateQueryParams
*/
public static WorldStateQueryParams withBlockHeaderAndNoUpdateNodeHead(
final BlockHeader blockHeader) {
return newBuilder().withBlockHeader(blockHeader).withShouldWorldStateUpdateHead(false).build();
}
/**
* Should return a worldstate instance with a state root, block hash, and should update the node
* head.
*
* @param stateRoot the state root
* @param blockHash the block hash
* @return an instance of WorldStateQueryParams
*/
public static WorldStateQueryParams withStateRootAndBlockHashAndUpdateNodeHead(
final Hash stateRoot, final Hash blockHash) {
return newBuilder()
.withStateRoot(stateRoot)
.withBlockHash(blockHash)
.withShouldWorldStateUpdateHead(true)
.build();
}
/**
* Should return a worldstate instance with a state root and should update the node head.
*
* @param stateRoot the state root
* @return an instance of WorldStateQueryParams
*/
public static WorldStateQueryParams withStateRootAndUpdateNodeHead(final Hash stateRoot) {
return newBuilder().withStateRoot(stateRoot).withShouldWorldStateUpdateHead(true).build();
}
/**
* Creates an instance with a state root, block hash, and does not update the node head.
*
* @param stateRoot the state root
* @param blockHash the block hash
* @return an instance of WorldStateQueryParams
*/
public static WorldStateQueryParams withStateRootAndBlockHashAndNoUpdateNodeHead(
final Hash stateRoot, final Hash blockHash) {
return newBuilder()
.withStateRoot(stateRoot)
.withBlockHash(blockHash)
.withShouldWorldStateUpdateHead(false)
.build();
}
@Override
public boolean equals(final Object o) {
if (o == null || getClass() != o.getClass()) return false;
WorldStateQueryParams that = (WorldStateQueryParams) o;
return shouldWorldStateUpdateHead == that.shouldWorldStateUpdateHead
&& Objects.equals(blockHeader, that.blockHeader)
&& Objects.equals(blockHash, that.blockHash)
&& Objects.equals(stateRoot, that.stateRoot);
}
@Override
public int hashCode() {
return Objects.hash(blockHeader, shouldWorldStateUpdateHead, blockHash, stateRoot);
}
public static class Builder {
private BlockHeader blockHeader;
private boolean shouldWorldStateUpdateHead = false;
private Hash blockHash;
private Optional<Hash> stateRoot = Optional.empty();
private Builder() {}
/**
* Sets the block header.
*
* @param blockHeader the block header
* @return the builder
*/
public Builder withBlockHeader(final BlockHeader blockHeader) {
this.blockHeader = blockHeader;
this.blockHash = blockHeader.getBlockHash();
this.stateRoot = Optional.of(blockHeader.getStateRoot());
return this;
}
/**
* Sets whether the world state should update the node head.
*
* @param shouldWorldStateUpdateHead true if the world state should update the node head, false
* otherwise
* @return the builder
*/
public Builder withShouldWorldStateUpdateHead(final boolean shouldWorldStateUpdateHead) {
this.shouldWorldStateUpdateHead = shouldWorldStateUpdateHead;
return this;
}
/**
* Sets the block hash.
*
* @param blockHash the block hash
* @return the builder
*/
public Builder withBlockHash(final Hash blockHash) {
this.blockHash = blockHash;
return this;
}
/**
* Sets the state root.
*
* @param stateRoot the state root
* @return the builder
*/
public Builder withStateRoot(final Hash stateRoot) {
this.stateRoot = Optional.ofNullable(stateRoot);
return this;
}
/**
* Builds an instance of WorldStateQueryParams.
*
* @return an instance of WorldStateQueryParams
*/
public WorldStateQueryParams build() {
if (blockHash == null && stateRoot.isEmpty() && blockHeader == null) {
throw new IllegalArgumentException(
"Either blockHash, stateRoot, or blockHeader must be provided");
}
return new WorldStateQueryParams(this);
}
}
}

View File

@@ -60,13 +60,25 @@ public abstract class DiffBasedWorldState
protected Hash worldStateRootHash;
protected Hash worldStateBlockHash;
protected DiffBasedWorldStateConfig worldStateConfig;
// configuration parameters for the world state.
protected WorldStateConfig worldStateConfig;
/*
* Indicates whether the world state is in "frozen" mode.
*
* When `isStorageFrozen` is true:
* - Changes to accounts, code, or slots will not affect the underlying storage.
* - The state root can still be recalculated, and a trie log can be generated.
* - All modifications are temporary and will be lost once the world state is discarded.
*/
protected boolean isStorageFrozen;
protected DiffBasedWorldState(
final DiffBasedWorldStateKeyValueStorage worldStateKeyValueStorage,
final DiffBasedCachedWorldStorageManager cachedWorldStorageManager,
final TrieLogManager trieLogManager,
final DiffBasedWorldStateConfig diffBasedWorldStateConfig) {
final WorldStateConfig worldStateConfig) {
this.worldStateKeyValueStorage = worldStateKeyValueStorage;
this.worldStateRootHash =
Hash.wrap(
@@ -77,7 +89,8 @@ public abstract class DiffBasedWorldState
Bytes32.wrap(worldStateKeyValueStorage.getWorldStateBlockHash().orElse(Hash.ZERO)));
this.cachedWorldStorageManager = cachedWorldStorageManager;
this.trieLogManager = trieLogManager;
this.worldStateConfig = diffBasedWorldStateConfig;
this.worldStateConfig = worldStateConfig;
this.isStorageFrozen = false;
}
/**
@@ -108,12 +121,21 @@ public abstract class DiffBasedWorldState
return worldStateRootHash;
}
/**
* Determines whether the current world state is directly modifying the "head" state of the
* blockchain. A world state modifying the head directly updates the latest state of the node,
* while a world state derived from a snapshot or historical view (e.g., layered or snapshot world
* state) does not directly modify the head
*
* @return {@code true} if the current world state is modifying the head, {@code false} otherwise.
*/
@Override
public boolean isPersisted() {
return isPersisted(worldStateKeyValueStorage);
public boolean isModifyingHeadWorldState() {
return isModifyingHeadWorldState(worldStateKeyValueStorage);
}
private boolean isPersisted(final WorldStateKeyValueStorage worldStateKeyValueStorage) {
private boolean isModifyingHeadWorldState(
final WorldStateKeyValueStorage worldStateKeyValueStorage) {
return !(worldStateKeyValueStorage instanceof DiffBasedSnapshotWorldStateKeyValueStorage);
}
@@ -140,8 +162,7 @@ public abstract class DiffBasedWorldState
final BlockHeader blockHeader,
final DiffBasedWorldStateKeyValueStorage.Updater stateUpdater) {
// calling calculateRootHash in order to update the state
calculateRootHash(
worldStateConfig.isFrozen() ? Optional.empty() : Optional.of(stateUpdater), accumulator);
calculateRootHash(isStorageFrozen ? Optional.empty() : Optional.of(stateUpdater), accumulator);
return blockHeader.getStateRoot();
}
@@ -167,8 +188,7 @@ public abstract class DiffBasedWorldState
if (blockHeader == null || !worldStateConfig.isTrieDisabled()) {
calculatedRootHash =
calculateRootHash(
worldStateConfig.isFrozen() ? Optional.empty() : Optional.of(stateUpdater),
accumulator);
isStorageFrozen ? Optional.empty() : Optional.of(stateUpdater), accumulator);
} else {
// if the trie is disabled, we cannot calculate the state root, so we directly use the root
// of the block. It's important to understand that in all networks,
@@ -186,7 +206,7 @@ public abstract class DiffBasedWorldState
() -> {
trieLogManager.saveTrieLog(localCopy, calculatedRootHash, blockHeader, this);
// not save a frozen state in the cache
if (!worldStateConfig.isFrozen()) {
if (!isStorageFrozen) {
cachedWorldStorageManager.addCachedLayer(blockHeader, calculatedRootHash, this);
}
};
@@ -234,7 +254,7 @@ public abstract class DiffBasedWorldState
@Override
public Hash rootHash() {
if (worldStateConfig.isFrozen() && accumulator.isAccumulatorStateChanged()) {
if (isStorageFrozen && accumulator.isAccumulatorStateChanged()) {
worldStateRootHash = calculateRootHash(Optional.empty(), accumulator.copy());
accumulator.resetAccumulatorStateChanged();
}
@@ -307,9 +327,9 @@ public abstract class DiffBasedWorldState
@Override
public void close() {
try {
if (!isPersisted()) {
if (!isModifyingHeadWorldState()) {
this.worldStateKeyValueStorage.close();
if (worldStateConfig.isFrozen()) {
if (isStorageFrozen) {
closeFrozenStorage();
}
}
@@ -322,7 +342,7 @@ public abstract class DiffBasedWorldState
try {
final DiffBasedLayeredWorldStateKeyValueStorage worldStateLayerStorage =
(DiffBasedLayeredWorldStateKeyValueStorage) worldStateKeyValueStorage;
if (!isPersisted(worldStateLayerStorage.getParentWorldStateStorage())) {
if (!isModifyingHeadWorldState(worldStateLayerStorage.getParentWorldStateStorage())) {
worldStateLayerStorage.getParentWorldStateStorage().close();
}
} catch (Exception e) {
@@ -333,8 +353,21 @@ public abstract class DiffBasedWorldState
@Override
public abstract Hash frontierRootHash();
/**
* Configures the current world state to operate in "frozen" mode.
*
* <p>In this mode: - Changes (to accounts, code, or slots) are isolated and not applied to the
* underlying storage. - The state root can be recalculated, and a trie log can be generated, but
* updates will not affect the world state storage. - All modifications are temporary and will be
* lost once the world state is discarded.
*
* <p>Use Cases: - Calculating the state root after updates without altering the storage. -
* Generating a trie log.
*
* @return The current world state in "frozen" mode.
*/
@Override
public abstract MutableWorldState freeze();
public abstract MutableWorldState freezeStorage();
@Override
public abstract Account get(final Address address);

View File

@@ -1,79 +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.trie.diffbased.common.worldview;
public class DiffBasedWorldStateConfig {
private boolean isFrozen;
private boolean isTrieDisabled;
public DiffBasedWorldStateConfig() {
this(false, false);
}
public DiffBasedWorldStateConfig(final boolean isTrieDisabled) {
this(false, isTrieDisabled);
}
public DiffBasedWorldStateConfig(final DiffBasedWorldStateConfig config) {
this(config.isFrozen(), config.isTrieDisabled());
}
public DiffBasedWorldStateConfig(final boolean isFrozen, final boolean isTrieDisabled) {
this.isFrozen = isFrozen;
this.isTrieDisabled = isTrieDisabled;
}
/**
* Checks if the world state is frozen. When the world state is frozen, it cannot mutate.
*
* @return true if the world state is frozen, false otherwise.
*/
public boolean isFrozen() {
return isFrozen;
}
/**
* Sets the frozen status of the world state. When the world state is frozen, it cannot mutate.
*
* @param frozen the new frozen status to set.
*/
public void setFrozen(final boolean frozen) {
isFrozen = frozen;
}
/**
* Checks if the trie is disabled for the world state. When the trie is disabled, the world state
* will only work with the flat database and not the trie. In this mode, it's impossible to verify
* the state root.
*
* @return true if the trie is disabled, false otherwise.
*/
public boolean isTrieDisabled() {
return isTrieDisabled;
}
/**
* Sets the disabled status of the trie for the world state. When the trie is disabled, the world
* state will only work with the flat database and not the trie. In this mode, it's impossible to
* verify the state root.
*
* @param trieDisabled the new disabled status to set for the trie.
*/
public void setTrieDisabled(final boolean trieDisabled) {
isTrieDisabled = trieDisabled;
}
}

View File

@@ -55,7 +55,7 @@ public interface DiffBasedWorldView extends WorldView {
return out.encoded();
}
boolean isPersisted();
boolean isModifyingHeadWorldState();
DiffBasedWorldStateKeyValueStorage getWorldStateStorage();

View File

@@ -0,0 +1,98 @@
/*
* 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.trie.diffbased.common.worldview;
/** WorldStateConfig encapsulates the shared configuration parameters for the world state. */
public class WorldStateConfig {
/**
* Indicates whether the trie is disabled for the world state. When the trie is disabled, the
* world state will only work with the flat database and not the trie. In this mode, it's
* impossible to verify the state root.
*/
private boolean isTrieDisabled;
/** Indicates whether the mode is stateful. Default is true. */
private boolean isStateful;
private WorldStateConfig(final Builder builder) {
this.isTrieDisabled = builder.isTrieDisabled;
this.isStateful = builder.isStateful;
}
public boolean isTrieDisabled() {
return isTrieDisabled;
}
public boolean isStateful() {
return isStateful;
}
public void setTrieDisabled(final boolean trieDisabled) {
isTrieDisabled = trieDisabled;
}
public void setStateful(final boolean stateful) {
isStateful = stateful;
}
/**
* Merges this WorldStateConfig with another WorldStateConfig and returns a new instance.
*
* @param other the other WorldStateConfig to merge with
* @return a new WorldStateConfig instance with merged values
*/
public WorldStateConfig apply(final WorldStateConfig other) {
return new Builder(this).trieDisabled(other.isTrieDisabled).stateful(other.isStateful).build();
}
public static Builder newBuilder() {
return new Builder();
}
public static Builder newBuilder(final WorldStateConfig worldStateConfig) {
return new Builder(worldStateConfig);
}
public static WorldStateConfig createStatefulConfigWithTrie() {
return newBuilder().stateful(true).trieDisabled(false).build();
}
public static class Builder {
private boolean isStateful = true;
private boolean isTrieDisabled = false;
public Builder() {}
public Builder(final WorldStateConfig spec) {
this.isTrieDisabled = spec.isTrieDisabled();
this.isStateful = spec.isStateful();
}
public Builder trieDisabled(final boolean trieDisabled) {
this.isTrieDisabled = trieDisabled;
return this;
}
public Builder stateful(final boolean stateful) {
this.isStateful = stateful;
return this;
}
public WorldStateConfig build() {
return new WorldStateConfig(this);
}
}
}

View File

@@ -338,11 +338,6 @@ public abstract class DiffBasedWorldStateUpdateAccumulator<ACCOUNT extends DiffB
return getDeletedAccounts();
}
@Override
public void revert() {
super.reset();
}
@Override
public void commit() {
this.isAccumulatorStateChanged = true;
@@ -604,7 +599,7 @@ public abstract class DiffBasedWorldStateUpdateAccumulator<ACCOUNT extends DiffB
}
@Override
public boolean isPersisted() {
public boolean isModifyingHeadWorldState() {
return true;
}
@@ -867,6 +862,25 @@ public abstract class DiffBasedWorldStateUpdateAccumulator<ACCOUNT extends DiffB
isAccumulatorStateChanged = false;
}
/**
* Reverts all changes that have not yet been committed.
*
* <p>This method calls the `reset` method of the superclass, which cancels all changes that have
* not yet been committed. This effectively reverts the state to the last committed state.
*/
@Override
public void revert() {
super.reset();
}
/**
* Resets the accumulator by clearing all changes, including those that have been committed.
*
* <p>This method clears all internal maps and data structures that track changes. This includes
* clearing the storage to clear, storage to update, code to update, accounts to update, and other
* related data structures. This effectively removes all changes, even those that have been
* committed in the accumulator.
*/
@Override
public void reset() {
storageToClear.clear();

View File

@@ -21,6 +21,7 @@ import org.hyperledger.besu.ethereum.core.MutableWorldState;
import org.hyperledger.besu.ethereum.proof.WorldStateProof;
import org.hyperledger.besu.ethereum.proof.WorldStateProofProvider;
import org.hyperledger.besu.ethereum.trie.MerkleTrie;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams;
import org.hyperledger.besu.ethereum.trie.forest.storage.ForestWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.forest.worldview.ForestMutableWorldState;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
@@ -57,7 +58,7 @@ public class ForestWorldStateArchive implements WorldStateArchive {
@Override
public Optional<WorldState> get(final Hash rootHash, final Hash blockHash) {
return getMutable(rootHash, blockHash).map(state -> state);
return getWorldState(rootHash).map(state -> state);
}
@Override
@@ -66,13 +67,20 @@ public class ForestWorldStateArchive implements WorldStateArchive {
}
@Override
public Optional<MutableWorldState> getMutable(
final BlockHeader blockHeader, final boolean isPersistingState) {
return getMutable(blockHeader.getStateRoot(), blockHeader.getHash());
public Optional<MutableWorldState> getWorldState(final WorldStateQueryParams queryParams) {
if (queryParams.getStateRoot().isEmpty()) {
throw new IllegalArgumentException(
"State root cannot be empty. A valid state root is required to retrieve the world state.");
}
return getWorldState(queryParams.getStateRoot().get());
}
@Override
public Optional<MutableWorldState> getMutable(final Hash rootHash, final Hash blockHash) {
public MutableWorldState getWorldState() {
return getWorldState(EMPTY_ROOT_HASH).get();
}
private Optional<MutableWorldState> getWorldState(final Hash rootHash) {
if (!worldStateKeyValueStorage.isWorldStateAvailable(rootHash)) {
return Optional.empty();
}
@@ -81,11 +89,6 @@ public class ForestWorldStateArchive implements WorldStateArchive {
rootHash, worldStateKeyValueStorage, preimageStorage, evmConfiguration));
}
@Override
public MutableWorldState getMutable() {
return getMutable(EMPTY_ROOT_HASH, null).get();
}
@Override
public void resetArchiveStateTo(final BlockHeader blockHeader) {
// ignore for forest

View File

@@ -405,7 +405,6 @@ public class ForestMutableWorldState implements MutableWorldState {
for (final UpdateTrackingAccount<WorldStateAccount> updated : getUpdatedAccounts()) {
final WorldStateAccount origin = updated.getWrappedAccount();
// Save the code in key-value storage ...
Hash codeHash = origin == null ? Hash.EMPTY : origin.getCodeHash();
if (updated.codeWasUpdated()) {

View File

@@ -19,6 +19,7 @@ import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.MutableWorldState;
import org.hyperledger.besu.ethereum.proof.WorldStateProof;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams;
import org.hyperledger.besu.evm.worldstate.WorldState;
import java.io.Closeable;
@@ -34,11 +35,26 @@ public interface WorldStateArchive extends Closeable {
boolean isWorldStateAvailable(Hash rootHash, Hash blockHash);
Optional<MutableWorldState> getMutable(BlockHeader blockHeader, boolean isPersistingState);
/**
* Gets a mutable world state based on the provided query parameters.
*
* <p>This method retrieves the mutable world state using the provided query parameters. The query
* parameters can specify various conditions and filters to determine the specific world state to
* be retrieved.
*
* @param worldStateQueryParams the query parameters
* @return the mutable world state, if available
*/
Optional<MutableWorldState> getWorldState(WorldStateQueryParams worldStateQueryParams);
Optional<MutableWorldState> getMutable(Hash rootHash, Hash blockHash);
MutableWorldState getMutable();
/**
* Gets the head world state.
*
* <p>This method returns the head world state, which is the most recent state of the world.
*
* @return the head world state
*/
MutableWorldState getWorldState();
/**
* Resetting the archive cache and adding the new pivot as the only entry

View File

@@ -124,7 +124,7 @@ public class BlockDataGenerator {
final List<UInt256> storageKeys) {
final List<Block> seq = new ArrayList<>(count);
final MutableWorldState worldState = worldStateArchive.getMutable();
final MutableWorldState worldState = worldStateArchive.getWorldState();
long nextBlockNumber = nextBlock;
Hash parentHash = parent;

View File

@@ -181,7 +181,7 @@ public class BlockchainSetupUtil {
: createInMemoryWorldStateArchive();
final TransactionPool transactionPool = mock(TransactionPool.class);
genesisState.writeStateTo(worldArchive.getMutable());
genesisState.writeStateTo(worldArchive.getWorldState());
final ProtocolContext protocolContext = protocolContextProvider.get(blockchain, worldArchive);
final Path blocksPath = Path.of(chainResources.getBlocksURL().toURI());

View File

@@ -78,7 +78,7 @@ public class ExecutionContextTestFixture {
this.protocolContext =
new ProtocolContext(
blockchain, stateArchive, new ConsensusContextFixture(), new BadBlockManager());
genesisState.writeStateTo(stateArchive.getMutable());
genesisState.writeStateTo(stateArchive.getWorldState());
}
public static ExecutionContextTestFixture create() {

View File

@@ -191,7 +191,7 @@ public class MessageFrameTestFixture {
}
private WorldUpdater createDefaultWorldUpdater() {
return getOrCreateExecutionContextTestFixture().getStateArchive().getMutable().updater();
return getOrCreateExecutionContextTestFixture().getStateArchive().getWorldState().updater();
}
private Blockchain createDefaultBlockchain() {

View File

@@ -15,6 +15,7 @@
package org.hyperledger.besu.ethereum;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig.createStatefulConfigWithTrie;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
@@ -46,7 +47,6 @@ import org.hyperledger.besu.ethereum.storage.StorageProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.BonsaiWorldStateProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.worldview.BonsaiWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.worldstate.DataStorageConfiguration;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
@@ -105,7 +105,7 @@ class BlockImportExceptionHandlingTest {
(BonsaiWorldStateKeyValueStorage)
worldStateStorageCoordinator.worldStateKeyValueStorage(),
EvmConfiguration.DEFAULT,
new DiffBasedWorldStateConfig()));
createStatefulConfigWithTrie()));
private final BadBlockManager badBlockManager = new BadBlockManager();
@@ -128,8 +128,8 @@ class BlockImportExceptionHandlingTest {
void shouldNotBadBlockWhenInternalErrorDuringPersisting() {
Mockito.doThrow(new StorageException("database problem")).when(persisted).persist(any());
Mockito.doReturn(persisted).when(worldStateArchive).getMutable();
Mockito.doReturn(Optional.of(persisted)).when(worldStateArchive).getMutable(any(), any());
Mockito.doReturn(persisted).when(worldStateArchive).getWorldState();
Mockito.doReturn(Optional.of(persisted)).when(worldStateArchive).getWorldState(any());
Block goodBlock =
new BlockDataGenerator()
@@ -234,8 +234,8 @@ class BlockImportExceptionHandlingTest {
@Test
void shouldNotBadBlockWhenInternalErrorDuringValidateBody() {
Mockito.doNothing().when(persisted).persist(any());
Mockito.doReturn(persisted).when(worldStateArchive).getMutable();
Mockito.doReturn(Optional.of(persisted)).when(worldStateArchive).getMutable(any(), any());
Mockito.doReturn(persisted).when(worldStateArchive).getWorldState();
Mockito.doReturn(Optional.of(persisted)).when(worldStateArchive).getWorldState(any());
Block goodBlock =
new BlockDataGenerator()

View File

@@ -15,9 +15,9 @@
package org.hyperledger.besu.ethereum;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndUpdateNodeHead;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.doThrow;
@@ -38,6 +38,7 @@ import org.hyperledger.besu.ethereum.mainnet.BlockProcessor;
import org.hyperledger.besu.ethereum.mainnet.BodyValidationMode;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import org.hyperledger.besu.ethereum.trie.MerkleTrieException;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.plugin.services.exception.StorageException;
@@ -50,6 +51,7 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import org.mockito.ArgumentCaptor;
public class MainnetBlockValidatorTest {
@@ -91,11 +93,9 @@ public class MainnetBlockValidatorTest {
when(protocolContext.getBlockchain()).thenReturn(blockchain);
when(protocolContext.getWorldStateArchive()).thenReturn(worldStateArchive);
when(worldStateArchive.getMutable(any(BlockHeader.class), anyBoolean()))
.thenReturn(Optional.of(worldState));
when(worldStateArchive.getMutable(any(Hash.class), any(Hash.class)))
.thenReturn(Optional.of(worldState));
when(worldStateArchive.getMutable()).thenReturn(worldState);
when(worldStateArchive.getWorldState(any())).thenReturn(Optional.of(worldState));
when(worldStateArchive.getWorldState(any())).thenReturn(Optional.of(worldState));
when(worldStateArchive.getWorldState()).thenReturn(worldState);
when(blockHeaderValidator.validateHeader(any(), any(), any())).thenReturn(true);
when(blockHeaderValidator.validateHeader(any(), any(), any(), any())).thenReturn(true);
when(blockBodyValidator.validateBody(any(), any(), any(), any(), any(), any()))
@@ -182,8 +182,17 @@ public class MainnetBlockValidatorTest {
@Test
public void validateAndProcessBlock_whenParentWorldStateNotAvailable() {
when(worldStateArchive.getMutable(eq(blockParent.getHeader()), anyBoolean()))
.thenReturn(Optional.empty());
final ArgumentCaptor<WorldStateQueryParams> captor =
ArgumentCaptor.forClass(WorldStateQueryParams.class);
when(worldStateArchive.getWorldState(captor.capture()))
.thenAnswer(
invocation -> {
WorldStateQueryParams capturedParams = captor.getValue();
if (capturedParams.getBlockHeader().equals(blockParent.getHeader())) {
return Optional.empty();
}
return null;
});
BlockProcessingResult result =
mainnetBlockValidator.validateAndProcessBlock(
@@ -258,7 +267,9 @@ public class MainnetBlockValidatorTest {
public void validateAndProcessBlock_whenStorageExceptionThrownGettingWorldState(
final String caseName, final Exception storageException) {
final BlockHeader parentHeader = blockParent.getHeader();
doThrow(storageException).when(worldStateArchive).getMutable(eq(parentHeader), anyBoolean());
doThrow(storageException)
.when(worldStateArchive)
.getWorldState(eq(withBlockHeaderAndUpdateNodeHead(parentHeader)));
BlockProcessingResult result =
mainnetBlockValidator.validateAndProcessBlock(

View File

@@ -216,7 +216,7 @@ class AbstractBlockProcessorIntegrationTest {
createTransferTransaction(
0, 2_000_000_000_000_000_000L, 300000L, 5L, 7L, ACCOUNT_3, ACCOUNT_GENESIS_2_KEYPAIR);
MutableWorldState worldState = worldStateArchive.getMutable();
MutableWorldState worldState = worldStateArchive.getWorldState();
BonsaiAccount senderAccount1 = (BonsaiAccount) worldState.get(transactionTransfer1.getSender());
BonsaiAccount senderAccount2 = (BonsaiAccount) worldState.get(transactionTransfer2.getSender());
@@ -258,7 +258,7 @@ class AbstractBlockProcessorIntegrationTest {
createTransferTransaction(
2, 3_000_000_000_000_000_000L, 300000L, 5L, 7L, ACCOUNT_6, ACCOUNT_GENESIS_1_KEYPAIR);
MutableWorldState worldState = worldStateArchive.getMutable();
MutableWorldState worldState = worldStateArchive.getWorldState();
BonsaiAccount senderAccount = (BonsaiAccount) worldState.get(transferTransaction1.getSender());
Block blockWithTransactions =
@@ -311,7 +311,7 @@ class AbstractBlockProcessorIntegrationTest {
ACCOUNT_2,
ACCOUNT_GENESIS_2_KEYPAIR); // ACCOUNT_GENESIS_2 -> ACCOUNT_2
MutableWorldState worldState = worldStateArchive.getMutable();
MutableWorldState worldState = worldStateArchive.getWorldState();
BonsaiAccount transferTransaction1Sender =
(BonsaiAccount) worldState.get(transferTransaction1.getSender());
@@ -370,7 +370,7 @@ class AbstractBlockProcessorIntegrationTest {
coinbase.toHexString(),
ACCOUNT_GENESIS_2_KEYPAIR); // ACCOUNT_GENESIS_2 -> COINBASE
MutableWorldState worldState = worldStateArchive.getMutable();
MutableWorldState worldState = worldStateArchive.getWorldState();
BonsaiAccount transferTransaction1Sender =
(BonsaiAccount) worldState.get(transferTransaction1.getSender());
Block blockWithTransactions =
@@ -427,7 +427,7 @@ class AbstractBlockProcessorIntegrationTest {
setSlot3Transaction,
setSlot4Transaction);
MutableWorldState worldState = worldStateArchive.getMutable();
MutableWorldState worldState = worldStateArchive.getWorldState();
BlockProcessingResult blockProcessingResult =
blockProcessor.processBlock(blockchain, worldState, blockWithTransactions);
@@ -462,7 +462,7 @@ class AbstractBlockProcessorIntegrationTest {
setSlot1Transaction,
setSlo2Transaction,
setSlot3Transaction);
MutableWorldState worldState = worldStateArchive.getMutable();
MutableWorldState worldState = worldStateArchive.getWorldState();
BlockProcessingResult blockProcessingResult =
blockProcessor.processBlock(blockchain, worldState, blockWithTransactions);
@@ -505,7 +505,7 @@ class AbstractBlockProcessorIntegrationTest {
transactionTransfer,
getcontractBalanceTransaction,
sendEthFromContractTransaction);
MutableWorldState worldState = worldStateArchive.getMutable();
MutableWorldState worldState = worldStateArchive.getWorldState();
BlockProcessingResult blockProcessingResult =
blockProcessor.processBlock(blockchain, worldState, blockWithTransactions);
@@ -551,7 +551,7 @@ class AbstractBlockProcessorIntegrationTest {
transactionTransfer,
sendEthFromContractTransaction,
getcontractBalanceTransaction);
MutableWorldState worldState = worldStateArchive.getMutable();
MutableWorldState worldState = worldStateArchive.getWorldState();
BlockProcessingResult blockProcessingResult =
blockProcessor.processBlock(blockchain, worldState, blockWithTransactions);
@@ -596,7 +596,7 @@ class AbstractBlockProcessorIntegrationTest {
transactionTransfer,
getcontractBalanceTransaction,
sendEthFromContractTransaction);
MutableWorldState worldState = worldStateArchive.getMutable();
MutableWorldState worldState = worldStateArchive.getWorldState();
BlockProcessingResult blockProcessingResult =
blockProcessor.processBlock(blockchain, worldState, blockWithTransactions);
@@ -642,7 +642,7 @@ class AbstractBlockProcessorIntegrationTest {
transactionTransfer,
sendEthFromContractTransaction,
getcontractBalanceTransaction);
MutableWorldState worldState = worldStateArchive.getMutable();
MutableWorldState worldState = worldStateArchive.getWorldState();
BlockProcessingResult blockProcessingResult =
blockProcessor.processBlock(blockchain, worldState, blockWithTransactions);

View File

@@ -157,10 +157,9 @@ class PrivacyBlockProcessorTest {
when(blockchain.getBlockHeader(any())).thenReturn(Optional.of(firstBlock.getHeader()));
final ProtocolSpec protocolSpec = mockProtocolSpec();
when(protocolSchedule.getByBlockHeader(any())).thenReturn(protocolSpec);
when(publicWorldStateArchive.getMutable(any(), any()))
.thenReturn(Optional.of(mutableWorldState));
when(publicWorldStateArchive.getWorldState(any())).thenReturn(Optional.of(mutableWorldState));
final MutableWorldState mockPrivateStateArchive = mockPrivateStateArchive();
when(privateWorldStateArchive.getMutable(any(), any()))
when(privateWorldStateArchive.getWorldState(any()))
.thenReturn(Optional.of(mockPrivateStateArchive));
final PrivacyGroupHeadBlockMap expected =

View File

@@ -14,6 +14,7 @@
*/
package org.hyperledger.besu.ethereum.mainnet.parallelization;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig.createStatefulConfigWithTrie;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
@@ -39,7 +40,6 @@ import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.cache.NoopBonsaiCache
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.worldview.BonsaiWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.trielog.NoOpTrieLogManager;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.accumulator.DiffBasedWorldStateUpdateAccumulator;
import org.hyperledger.besu.ethereum.worldstate.DataStorageConfiguration;
import org.hyperledger.besu.evm.blockhash.BlockHashLookup;
@@ -88,7 +88,7 @@ class ParallelizedConcurrentTransactionProcessorTest {
new NoOpBonsaiCachedWorldStorageManager(bonsaiWorldStateKeyValueStorage),
new NoOpTrieLogManager(),
EvmConfiguration.DEFAULT,
new DiffBasedWorldStateConfig());
createStatefulConfigWithTrie());
when(transactionCollisionDetector.hasCollision(any(), any(), any(), any())).thenReturn(false);
}

View File

@@ -105,9 +105,10 @@ public class FlexiblePrivacyPrecompiledContractTest {
@BeforeEach
public void setUp() {
final MutableWorldState mutableWorldState = mock(MutableWorldState.class);
when(mutableWorldState.rootHash()).thenReturn(Hash.EMPTY);
when(mutableWorldState.updater()).thenReturn(mock(WorldUpdater.class));
when(worldStateArchive.getMutable()).thenReturn(mutableWorldState);
when(worldStateArchive.getMutable(any(), any())).thenReturn(Optional.of(mutableWorldState));
when(worldStateArchive.getWorldState()).thenReturn(mutableWorldState);
when(worldStateArchive.getWorldState(any())).thenReturn(Optional.of(mutableWorldState));
final PrivateStateStorage.Updater storageUpdater = mock(PrivateStateStorage.Updater.class);
when(privateStateStorage.getPrivacyGroupHeadBlockMap(any()))

View File

@@ -113,8 +113,8 @@ public class PrivacyPrecompiledContractTest {
public void setUp() {
final MutableWorldState mutableWorldState = mock(MutableWorldState.class);
when(mutableWorldState.updater()).thenReturn(mock(WorldUpdater.class));
when(worldStateArchive.getMutable()).thenReturn(mutableWorldState);
when(worldStateArchive.getMutable(any(), any())).thenReturn(Optional.of(mutableWorldState));
when(worldStateArchive.getWorldState()).thenReturn(mutableWorldState);
when(worldStateArchive.getWorldState(any())).thenReturn(Optional.of(mutableWorldState));
when(privateMetadataUpdater.getPrivacyGroupHeadBlockMap())
.thenReturn(PrivacyGroupHeadBlockMap.empty());

View File

@@ -21,6 +21,7 @@ import static org.hyperledger.besu.ethereum.core.PrivateTransactionDataFixture.p
import static org.hyperledger.besu.ethereum.privacy.PrivateStateRootResolver.EMPTY_ROOT_HASH;
import static org.hyperledger.besu.ethereum.privacy.storage.PrivateStateKeyValueStorage.SCHEMA_VERSION_1_0_0;
import static org.hyperledger.besu.ethereum.privacy.storage.PrivateStateKeyValueStorage.SCHEMA_VERSION_1_4_0;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndUpdateNodeHead;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.lenient;
@@ -313,7 +314,7 @@ public class PrivateStorageMigrationTest {
when(blockchain.getBlockHeader(blockHash)).thenReturn(Optional.of(blockHeader));
when(blockchain.getBlockBody(blockHash)).thenReturn(Optional.of(block.getBody()));
when(publicWorldStateArchive.getMutable(blockHeader.getStateRoot(), blockHash))
when(publicWorldStateArchive.getWorldState(withBlockHeaderAndUpdateNodeHead(blockHeader)))
.thenReturn(Optional.of(publicMutableWorldState));
}

View File

@@ -15,12 +15,12 @@
package org.hyperledger.besu.ethereum.transaction;
import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndNoUpdateNodeHead;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@@ -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.feemarket.FeeMarket;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams;
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
import org.hyperledger.besu.evm.account.MutableAccount;
import org.hyperledger.besu.evm.worldstate.WorldUpdater;
@@ -98,20 +99,20 @@ public class BlockSimulatorTest {
@Test
public void shouldProcessWithValidWorldState() {
when(worldStateArchive.getMutable(any(BlockHeader.class), eq(false)))
when(worldStateArchive.getWorldState(withBlockHeaderAndNoUpdateNodeHead(blockHeader)))
.thenReturn(Optional.of(mutableWorldState));
List<BlockSimulationResult> results =
blockSimulator.process(blockHeader, Collections.emptyList());
assertNotNull(results);
verify(worldStateArchive).getMutable(any(BlockHeader.class), eq(false));
verify(worldStateArchive).getWorldState(withBlockHeaderAndNoUpdateNodeHead(blockHeader));
}
@Test
public void shouldNotProcessWithInvalidWorldState() {
when(worldStateArchive.getMutable(any(BlockHeader.class), eq(false)))
.thenReturn(Optional.empty());
when(worldStateArchive.getWorldState(any(WorldStateQueryParams.class)))
.thenAnswer(invocation -> Optional.empty());
IllegalArgumentException exception =
assertThrows(

View File

@@ -15,6 +15,7 @@
package org.hyperledger.besu.ethereum.transaction;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndNoUpdateNodeHead;
import static org.hyperledger.besu.evm.tracing.OperationTracer.NO_TRACING;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
@@ -812,7 +813,7 @@ public class TransactionSimulatorTest {
final BlockHeader blockHeader, final Address address, final long nonce) {
final Account account = mock(Account.class);
when(account.getNonce()).thenReturn(nonce);
when(worldStateArchive.getMutable(eq(blockHeader), anyBoolean()))
when(worldStateArchive.getWorldState(withBlockHeaderAndNoUpdateNodeHead(blockHeader)))
.thenReturn(Optional.of(worldState));
final WorldUpdater updater = mock(WorldUpdater.class);
when(updater.get(address)).thenReturn(account);
@@ -820,7 +821,7 @@ public class TransactionSimulatorTest {
}
private void mockWorldStateForAbsentAccount(final BlockHeader blockHeader) {
when(worldStateArchive.getMutable(eq(blockHeader), anyBoolean()))
when(worldStateArchive.getWorldState(withBlockHeaderAndNoUpdateNodeHead(blockHeader)))
.thenReturn(Optional.of(worldState));
final WorldUpdater updater = mock(WorldUpdater.class);
when(updater.get(any())).thenReturn(null);

View File

@@ -171,7 +171,7 @@ public abstract class AbstractIsolationTests {
null,
EvmConfiguration.DEFAULT,
throwingWorldStateHealerSupplier());
var ws = archive.getMutable();
var ws = archive.getWorldState();
genesisState.writeStateTo(ws);
protocolContext =
new ProtocolContext(

View File

@@ -15,6 +15,8 @@
package org.hyperledger.besu.ethereum.trie.diffbased.bonsai;
import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndNoUpdateNodeHead;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withStateRootAndBlockHashAndUpdateNodeHead;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Wei;
@@ -32,11 +34,15 @@ public class BonsaiSnapshotIsolationTests extends AbstractIsolationTests {
@Test
public void ensureTruncateDoesNotCauseSegfault() {
var preTruncatedWorldState = archive.getMutable(genesisState.getBlock().getHeader(), false);
var preTruncatedWorldState =
archive.getWorldState(
withBlockHeaderAndNoUpdateNodeHead(genesisState.getBlock().getHeader()));
assertThat(preTruncatedWorldState)
.isPresent(); // really just assert that we have not segfaulted after truncating
worldStateKeyValueStorage.clear();
var postTruncatedWorldState = archive.getMutable(genesisState.getBlock().getHeader(), false);
var postTruncatedWorldState =
archive.getWorldState(
withBlockHeaderAndNoUpdateNodeHead(genesisState.getBlock().getHeader()));
assertThat(postTruncatedWorldState).isEmpty();
// assert that trying to access pre-worldstate does not segfault after truncating
preTruncatedWorldState.get().get(accounts.get(0).address());
@@ -47,14 +53,17 @@ public class BonsaiSnapshotIsolationTests extends AbstractIsolationTests {
public void testIsolatedFromHead_behindHead() {
Address testAddress = Address.fromHexString("0xdeadbeef");
// assert we can mutate head without mutating the isolated snapshot
var isolated = archive.getMutable(genesisState.getBlock().getHeader(), false);
var isolated =
archive.getWorldState(
withBlockHeaderAndNoUpdateNodeHead(genesisState.getBlock().getHeader()));
var firstBlock = forTransactions(List.of(burnTransaction(sender1, 0L, testAddress)));
var res = executeBlock(archive.getMutable(), firstBlock);
var res = executeBlock(archive.getWorldState(), firstBlock);
var isolated2 = archive.getMutable(firstBlock.getHeader(), false);
var isolated2 =
archive.getWorldState(withBlockHeaderAndNoUpdateNodeHead(firstBlock.getHeader()));
var secondBlock = forTransactions(List.of(burnTransaction(sender1, 1L, testAddress)));
var res2 = executeBlock(archive.getMutable(), secondBlock);
var res2 = executeBlock(archive.getWorldState(), secondBlock);
assertThat(res.isSuccessful()).isTrue();
assertThat(res2.isSuccessful()).isTrue();
@@ -62,8 +71,8 @@ public class BonsaiSnapshotIsolationTests extends AbstractIsolationTests {
assertThat(archive.getCachedWorldStorageManager().contains(firstBlock.getHash())).isTrue();
assertThat(archive.getCachedWorldStorageManager().contains(secondBlock.getHash())).isTrue();
assertThat(archive.getMutable().get(testAddress)).isNotNull();
assertThat(archive.getMutable().get(testAddress).getBalance())
assertThat(archive.getWorldState().get(testAddress)).isNotNull();
assertThat(archive.getWorldState().get(testAddress).getBalance())
.isEqualTo(Wei.of(2_000_000_000_000_000_000L));
assertThat(isolated.get().get(testAddress)).isNull();
@@ -89,13 +98,13 @@ public class BonsaiSnapshotIsolationTests extends AbstractIsolationTests {
Address testAddress = Address.fromHexString("0xdeadbeef");
var block1 = forTransactions(List.of(burnTransaction(sender1, 0L, testAddress)));
var res = executeBlock(archive.getMutable(), block1);
var res = executeBlock(archive.getWorldState(), block1);
var block2 = forTransactions(List.of(burnTransaction(sender1, 1L, testAddress)));
var res2 = executeBlock(archive.getMutable(), block2);
var res2 = executeBlock(archive.getWorldState(), block2);
var block3 = forTransactions(List.of(burnTransaction(sender1, 2L, testAddress)));
var res3 = executeBlock(archive.getMutable(), block3);
var res3 = executeBlock(archive.getWorldState(), block3);
assertThat(res.isSuccessful()).isTrue();
assertThat(res2.isSuccessful()).isTrue();
@@ -103,7 +112,8 @@ public class BonsaiSnapshotIsolationTests extends AbstractIsolationTests {
// roll chain and worldstate to block 2
blockchain.rewindToBlock(2L);
var block1State = archive.getMutable(null, block2.getHash());
var block1State =
archive.getWorldState(withStateRootAndBlockHashAndUpdateNodeHead(null, block2.getHash()));
// BonsaiWorldState should be at block 2
assertThat(block1State.get().get(testAddress)).isNotNull();
@@ -111,7 +121,8 @@ public class BonsaiSnapshotIsolationTests extends AbstractIsolationTests {
.isEqualTo(Wei.of(2_000_000_000_000_000_000L));
assertThat(block1State.get().rootHash()).isEqualTo(block2.getHeader().getStateRoot());
var isolatedRollForward = archive.getMutable(block3.getHeader(), false);
var isolatedRollForward =
archive.getWorldState(withBlockHeaderAndNoUpdateNodeHead(block3.getHeader()));
// we should be at block 3, one block ahead of BonsaiPersistatedWorldState
assertThat(isolatedRollForward.get().get(testAddress)).isNotNull();
@@ -120,7 +131,8 @@ public class BonsaiSnapshotIsolationTests extends AbstractIsolationTests {
assertThat(isolatedRollForward.get().rootHash()).isEqualTo(block3.getHeader().getStateRoot());
// we should be at block 1, one block behind BonsaiPersistatedWorldState
var isolatedRollBack = archive.getMutable(block1.getHeader(), false);
var isolatedRollBack =
archive.getWorldState(withBlockHeaderAndNoUpdateNodeHead(block1.getHeader()));
assertThat(isolatedRollBack.get().get(testAddress)).isNotNull();
assertThat(isolatedRollBack.get().get(testAddress).getBalance())
.isEqualTo(Wei.of(1_000_000_000_000_000_000L));
@@ -138,10 +150,12 @@ public class BonsaiSnapshotIsolationTests extends AbstractIsolationTests {
public void assertCloseDisposesOfStateWithoutCommitting() {
Address testAddress = Address.fromHexString("0xdeadbeef");
var head = archive.getMutable();
var head = archive.getWorldState();
try (var shouldCloseSnapshot =
archive.getMutable(genesisState.getBlock().getHeader(), false).get()) {
archive
.getWorldState(withBlockHeaderAndNoUpdateNodeHead(genesisState.getBlock().getHeader()))
.get()) {
var tx1 = burnTransaction(sender1, 0L, testAddress);
Block oneTx = forTransactions(List.of(tx1));

View File

@@ -18,6 +18,9 @@ import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.ethereum.core.WorldStateHealerHelper.throwingWorldStateHealerSupplier;
import static org.hyperledger.besu.ethereum.storage.keyvalue.KeyValueSegmentIdentifier.BLOCKCHAIN;
import static org.hyperledger.besu.ethereum.storage.keyvalue.KeyValueSegmentIdentifier.TRIE_BRANCH_STORAGE;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndNoUpdateNodeHead;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndUpdateNodeHead;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withStateRootAndBlockHashAndUpdateNodeHead;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.storage.DiffBasedWorldStateKeyValueStorage.WORLD_BLOCK_HASH_KEY;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.storage.DiffBasedWorldStateKeyValueStorage.WORLD_ROOT_HASH_KEY;
import static org.mockito.ArgumentMatchers.any;
@@ -115,7 +118,7 @@ class BonsaiWorldStateProviderTest {
EvmConfiguration.DEFAULT,
throwingWorldStateHealerSupplier());
assertThat(bonsaiWorldStateArchive.getMutable(chainHead, true))
assertThat(bonsaiWorldStateArchive.getWorldState(withBlockHeaderAndUpdateNodeHead(chainHead)))
.containsInstanceOf(BonsaiWorldState.class);
}
@@ -136,7 +139,9 @@ class BonsaiWorldStateProviderTest {
final BlockHeader blockHeader = blockBuilder.number(0).buildHeader();
final BlockHeader chainHead = blockBuilder.number(512).buildHeader();
when(blockchain.getChainHeadHeader()).thenReturn(chainHead);
assertThat(bonsaiWorldStateArchive.getMutable(blockHeader, false)).isEmpty();
assertThat(
bonsaiWorldStateArchive.getWorldState(withBlockHeaderAndNoUpdateNodeHead(blockHeader)))
.isEmpty();
verify(cachedWorldStorageManager, Mockito.never()).getWorldState(any(Hash.class));
}
@@ -159,13 +164,14 @@ class BonsaiWorldStateProviderTest {
final BlockHeader chainHead = blockBuilder.number(511).buildHeader();
final BonsaiWorldState mockWorldState = mock(BonsaiWorldState.class);
when(mockWorldState.blockHash()).thenReturn(blockHeader.getHash());
when(mockWorldState.freeze()).thenReturn(mockWorldState);
when(mockWorldState.freezeStorage()).thenReturn(mockWorldState);
when(trieLogManager.getMaxLayersToLoad()).thenReturn(Long.valueOf(512));
when(cachedWorldStorageManager.getWorldState(blockHeader.getHash()))
.thenReturn(Optional.of(mockWorldState));
when(blockchain.getChainHeadHeader()).thenReturn(chainHead);
assertThat(bonsaiWorldStateArchive.getMutable(blockHeader, false))
assertThat(
bonsaiWorldStateArchive.getWorldState(withBlockHeaderAndNoUpdateNodeHead(blockHeader)))
.containsInstanceOf(BonsaiWorldState.class);
}
@@ -195,7 +201,9 @@ class BonsaiWorldStateProviderTest {
when(blockchain.getBlockHeader(blockHeader.getHash())).thenReturn(Optional.of(blockHeader));
assertThat(bonsaiWorldStateArchive.getMutable(null, blockHeader.getHash()))
assertThat(
bonsaiWorldStateArchive.getWorldState(
withStateRootAndBlockHashAndUpdateNodeHead(null, blockHeader.getHash())))
.containsInstanceOf(BonsaiWorldState.class);
// verify is trying to get the trie log layer to rollback
@@ -227,7 +235,9 @@ class BonsaiWorldStateProviderTest {
when(blockchain.getBlockHeader(blockHeader.getHash())).thenReturn(Optional.of(blockHeader));
when(blockchain.getBlockHeader(Hash.ZERO)).thenReturn(Optional.of(blockHeader));
assertThat(bonsaiWorldStateArchive.getMutable(null, blockHeader.getHash()))
assertThat(
bonsaiWorldStateArchive.getWorldState(
withStateRootAndBlockHashAndUpdateNodeHead(null, blockHeader.getHash())))
.containsInstanceOf(BonsaiWorldState.class);
// verify is not trying to get the trie log layer to rollback when block is present
@@ -269,7 +279,9 @@ class BonsaiWorldStateProviderTest {
.thenReturn(Optional.of(blockHeaderChainB));
when(blockchain.getBlockHeader(genesis.getHash())).thenReturn(Optional.of(genesis));
assertThat(bonsaiWorldStateArchive.getMutable(null, blockHeaderChainB.getHash()))
assertThat(
bonsaiWorldStateArchive.getWorldState(
withStateRootAndBlockHashAndUpdateNodeHead(null, blockHeaderChainB.getHash())))
.containsInstanceOf(BonsaiWorldState.class);
// verify is trying to get the trie log layers to rollback and roll forward
@@ -279,7 +291,7 @@ class BonsaiWorldStateProviderTest {
@Test
// TODO: refactor to test original intent
@Disabled("needs refactor, getMutable(hash, hash) cannot trigger saveTrieLog")
@Disabled("needs refactor, getWorldState(hash, hash) cannot trigger saveTrieLog")
void testGetMutableWithRollbackNotOverrideTrieLogLayer() {
when(segmentedKeyValueStorage.startTransaction())
.thenReturn(segmentedKeyValueStorageTransaction);
@@ -321,7 +333,9 @@ class BonsaiWorldStateProviderTest {
.thenReturn(Optional.of(blockHeaderChainB));
when(blockchain.getBlockHeader(genesis.getHash())).thenReturn(Optional.of(genesis));
assertThat(bonsaiWorldStateArchive.getMutable(null, blockHeaderChainB.getHash()))
assertThat(
bonsaiWorldStateArchive.getWorldState(
withStateRootAndBlockHashAndUpdateNodeHead(null, blockHeaderChainB.getHash())))
.containsInstanceOf(BonsaiWorldState.class);
// verify is not persisting if already present

View File

@@ -15,6 +15,7 @@
package org.hyperledger.besu.ethereum.trie.diffbased.bonsai;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig.createStatefulConfigWithTrie;
import static org.mockito.Mockito.mock;
import org.hyperledger.besu.datatypes.Address;
@@ -32,7 +33,6 @@ import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.trielog.TrieLogFactor
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.worldview.BonsaiWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.worldview.BonsaiWorldStateUpdateAccumulator;
import org.hyperledger.besu.ethereum.trie.diffbased.common.trielog.TrieLogLayer;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.worldstate.DataStorageConfiguration;
import org.hyperledger.besu.evm.account.MutableAccount;
import org.hyperledger.besu.evm.internal.EvmConfiguration;
@@ -164,7 +164,7 @@ class LogRollingTests {
new BonsaiWorldStateKeyValueStorage(
provider, new NoOpMetricsSystem(), DataStorageConfiguration.DEFAULT_BONSAI_CONFIG),
EvmConfiguration.DEFAULT,
new DiffBasedWorldStateConfig());
createStatefulConfigWithTrie());
final WorldUpdater updater = worldState.updater();
final MutableAccount mutableAccount = updater.createAccount(addressOne, 1, Wei.of(1L));
@@ -181,7 +181,7 @@ class LogRollingTests {
new NoOpMetricsSystem(),
DataStorageConfiguration.DEFAULT_BONSAI_CONFIG),
EvmConfiguration.DEFAULT,
new DiffBasedWorldStateConfig());
createStatefulConfigWithTrie());
final BonsaiWorldStateUpdateAccumulator secondUpdater =
(BonsaiWorldStateUpdateAccumulator) secondWorldState.updater();
@@ -214,7 +214,7 @@ class LogRollingTests {
new BonsaiWorldStateKeyValueStorage(
provider, new NoOpMetricsSystem(), DataStorageConfiguration.DEFAULT_BONSAI_CONFIG),
EvmConfiguration.DEFAULT,
new DiffBasedWorldStateConfig());
createStatefulConfigWithTrie());
final WorldUpdater updater = worldState.updater();
final MutableAccount mutableAccount = updater.createAccount(addressOne, 1, Wei.of(1L));
@@ -239,7 +239,7 @@ class LogRollingTests {
new NoOpMetricsSystem(),
DataStorageConfiguration.DEFAULT_BONSAI_CONFIG),
EvmConfiguration.DEFAULT,
new DiffBasedWorldStateConfig());
createStatefulConfigWithTrie());
final BonsaiWorldStateUpdateAccumulator secondUpdater =
(BonsaiWorldStateUpdateAccumulator) secondWorldState.updater();
@@ -273,7 +273,7 @@ class LogRollingTests {
new BonsaiWorldStateKeyValueStorage(
provider, new NoOpMetricsSystem(), DataStorageConfiguration.DEFAULT_BONSAI_CONFIG),
EvmConfiguration.DEFAULT,
new DiffBasedWorldStateConfig());
createStatefulConfigWithTrie());
final WorldUpdater updater = worldState.updater();
final MutableAccount mutableAccount = updater.createAccount(addressOne, 1, Wei.of(1L));
@@ -305,7 +305,7 @@ class LogRollingTests {
new NoOpMetricsSystem(),
DataStorageConfiguration.DEFAULT_BONSAI_CONFIG),
EvmConfiguration.DEFAULT,
new DiffBasedWorldStateConfig());
createStatefulConfigWithTrie());
final WorldUpdater secondUpdater = secondWorldState.updater();
final MutableAccount secondMutableAccount =

View File

@@ -19,6 +19,7 @@ import static org.hyperledger.besu.ethereum.storage.keyvalue.KeyValueSegmentIden
import static org.hyperledger.besu.ethereum.storage.keyvalue.KeyValueSegmentIdentifier.ACCOUNT_STORAGE_STORAGE;
import static org.hyperledger.besu.ethereum.storage.keyvalue.KeyValueSegmentIdentifier.CODE_STORAGE;
import static org.hyperledger.besu.ethereum.storage.keyvalue.KeyValueSegmentIdentifier.TRIE_BRANCH_STORAGE;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig.createStatefulConfigWithTrie;
import org.hyperledger.besu.ethereum.core.InMemoryKeyValueStorageProvider;
import org.hyperledger.besu.ethereum.rlp.BytesValueRLPInput;
@@ -28,7 +29,6 @@ import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.trielog.TrieLogFactor
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.worldview.BonsaiWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.worldview.BonsaiWorldStateUpdateAccumulator;
import org.hyperledger.besu.ethereum.trie.diffbased.common.trielog.TrieLogLayer;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.worldstate.DataStorageConfiguration;
import org.hyperledger.besu.evm.internal.EvmConfiguration;
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
@@ -59,7 +59,7 @@ public class RollingImport {
new BonsaiWorldStateKeyValueStorage(
provider, new NoOpMetricsSystem(), DataStorageConfiguration.DEFAULT_BONSAI_CONFIG),
EvmConfiguration.DEFAULT,
new DiffBasedWorldStateConfig());
createStatefulConfigWithTrie());
final SegmentedInMemoryKeyValueStorage worldStateKeyValueStorage =
(SegmentedInMemoryKeyValueStorage)
provider.getStorageBySegmentIdentifiers(

View File

@@ -14,6 +14,7 @@
*/
package org.hyperledger.besu.ethereum.trie.diffbased.bonsai.worldview;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig.createStatefulConfigWithTrie;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoInteractions;
import static org.mockito.Mockito.when;
@@ -24,7 +25,6 @@ import org.hyperledger.besu.ethereum.chain.Blockchain;
import org.hyperledger.besu.ethereum.core.InMemoryKeyValueStorageProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldStateKeyValueStorage;
import org.hyperledger.besu.ethereum.trie.diffbased.common.DiffBasedValue;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.evm.internal.EvmConfiguration;
import java.util.HashMap;
@@ -63,7 +63,7 @@ class BonsaiWorldStateTest {
InMemoryKeyValueStorageProvider.createBonsaiInMemoryWorldStateArchive(blockchain),
bonsaiWorldStateKeyValueStorage,
EvmConfiguration.DEFAULT,
new DiffBasedWorldStateConfig());
createStatefulConfigWithTrie());
}
@ParameterizedTest

View File

@@ -131,7 +131,7 @@ public class WorldStateDownloaderBenchmark {
private Hash createExistingWorldState() {
// Setup existing state
final WorldStateArchive worldStateArchive = createInMemoryWorldStateArchive();
final MutableWorldState worldState = worldStateArchive.getMutable();
final MutableWorldState worldState = worldStateArchive.getWorldState();
dataGen.createRandomAccounts(worldState, 10000);

View File

@@ -18,6 +18,7 @@ import static org.hyperledger.besu.ethereum.transaction.TransactionInvalidReason
import static org.hyperledger.besu.ethereum.transaction.TransactionInvalidReason.CHAIN_HEAD_WORLD_STATE_NOT_AVAILABLE;
import static org.hyperledger.besu.ethereum.transaction.TransactionInvalidReason.INTERNAL_ERROR;
import static org.hyperledger.besu.ethereum.transaction.TransactionInvalidReason.TRANSACTION_ALREADY_KNOWN;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams.withBlockHeaderAndNoUpdateNodeHead;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.BlobsWithCommitments;
@@ -476,7 +477,7 @@ public class TransactionPool implements BlockAddedObserver {
try (final var worldState =
protocolContext
.getWorldStateArchive()
.getMutable(chainHeadBlockHeader, false)
.getWorldState(withBlockHeaderAndNoUpdateNodeHead(chainHeadBlockHeader))
.orElseThrow()) {
final Account senderAccount = worldState.get(transaction.getSender());
return new ValidationResultAndAccount(

View File

@@ -27,7 +27,7 @@ import org.hyperledger.besu.ethereum.eth.messages.snap.GetAccountRangeMessage;
import org.hyperledger.besu.ethereum.eth.sync.snapsync.ImmutableSnapSyncConfiguration;
import org.hyperledger.besu.ethereum.eth.sync.snapsync.SnapSyncConfiguration;
import org.hyperledger.besu.ethereum.mainnet.HeaderValidationMode;
import org.hyperledger.besu.ethereum.trie.diffbased.common.DiffBasedWorldStateProvider;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.DiffBasedWorldStateProvider;
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
@@ -379,7 +379,7 @@ public class SnapServerGetAccountRangeTest {
@SuppressWarnings("UnusedVariable")
private void initAccounts() {
rootHash = protocolContext.getWorldStateArchive().getMutable().rootHash();
rootHash = protocolContext.getWorldStateArchive().getWorldState().rootHash();
GetAccountRangeMessage requestMessage =
GetAccountRangeMessage.create(rootHash, Hash.ZERO, Hash.LAST, BigInteger.valueOf(4000));
AccountRangeMessage resultMessage =

View File

@@ -201,7 +201,7 @@ class FastWorldStateDownloaderTest {
void downloadAlreadyAvailableWorldState() {
// Setup existing state
final ForestWorldStateArchive worldStateArchive = createInMemoryWorldStateArchive();
final MutableWorldState worldState = worldStateArchive.getMutable();
final MutableWorldState worldState = worldStateArchive.getWorldState();
// Generate accounts and save corresponding state root
dataGen.createRandomAccounts(worldState, 20);
@@ -252,7 +252,7 @@ class FastWorldStateDownloaderTest {
// Setup "remote" state
final WorldStateArchive remoteWorldStateArchive = createInMemoryWorldStateArchive();
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getMutable();
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getWorldState();
// Generate accounts and save corresponding state root
final List<Account> accounts = dataGen.createRandomAccounts(remoteWorldState, 20);
@@ -309,7 +309,7 @@ class FastWorldStateDownloaderTest {
void doesNotRequestKnownCodeFromNetwork() {
// Setup "remote" state
final WorldStateArchive remoteWorldStateArchive = createInMemoryWorldStateArchive();
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getMutable();
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getWorldState();
// Generate accounts and save corresponding state root
final List<Account> accounts =
@@ -397,7 +397,7 @@ class FastWorldStateDownloaderTest {
// Setup "remote" state
final WorldStateArchive remoteWorldStateArchive = createInMemoryWorldStateArchive();
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getMutable();
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getWorldState();
// Generate accounts and save corresponding state root
dataGen.createRandomContractAccountsWithNonEmptyStorage(remoteWorldState, 20);
@@ -474,7 +474,7 @@ class FastWorldStateDownloaderTest {
new WorldStateStorageCoordinator(remoteStorage),
createPreimageStorage(),
EvmConfiguration.DEFAULT);
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getMutable();
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getWorldState();
// Generate accounts and save corresponding state root
final List<Account> accounts =
@@ -565,7 +565,7 @@ class FastWorldStateDownloaderTest {
new WorldStateStorageCoordinator(remoteStorage),
createPreimageStorage(),
EvmConfiguration.DEFAULT);
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getMutable();
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getWorldState();
// Generate accounts and save corresponding state root
final List<Account> accounts =
@@ -679,7 +679,7 @@ class FastWorldStateDownloaderTest {
new WorldStateStorageCoordinator(remoteStorage),
createPreimageStorage(),
EvmConfiguration.DEFAULT);
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getMutable();
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getWorldState();
// Generate accounts and save corresponding state root
dataGen.createRandomAccounts(remoteWorldState, 10);
@@ -743,7 +743,7 @@ class FastWorldStateDownloaderTest {
new WorldStateStorageCoordinator(remoteStorage),
createPreimageStorage(),
EvmConfiguration.DEFAULT);
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getMutable();
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getWorldState();
// Generate accounts and save corresponding state root
List<Account> accounts = dataGen.createRandomAccounts(remoteWorldState, 10);
@@ -886,7 +886,7 @@ class FastWorldStateDownloaderTest {
new WorldStateStorageCoordinator(remoteStorage),
createPreimageStorage(),
EvmConfiguration.DEFAULT);
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getMutable();
final MutableWorldState remoteWorldState = remoteWorldStateArchive.getWorldState();
// Generate accounts and save corresponding state root
final List<Account> accounts = dataGen.createRandomAccounts(remoteWorldState, accountCount);

View File

@@ -134,7 +134,7 @@ public class TestNode implements Closeable {
final MutableBlockchain blockchain =
createInMemoryBlockchain(genesisState.getBlock(), blockHeaderFunctions);
final WorldStateArchive worldStateArchive = createInMemoryWorldStateArchive();
genesisState.writeStateTo(worldStateArchive.getMutable());
genesisState.writeStateTo(worldStateArchive.getWorldState());
final ProtocolContext protocolContext =
new ProtocolContext(
blockchain, worldStateArchive, mock(ConsensusContext.class), new BadBlockManager());

View File

@@ -31,6 +31,7 @@ import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
import org.hyperledger.besu.ethereum.referencetests.BlockchainReferenceTestCaseSpec;
import org.hyperledger.besu.ethereum.referencetests.ReferenceTestProtocolSchedules;
import org.hyperledger.besu.ethereum.rlp.RLPException;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams;
import org.hyperledger.besu.evm.EVM;
import org.hyperledger.besu.evm.EvmSpecVersion;
import org.hyperledger.besu.evm.account.AccountState;
@@ -168,7 +169,9 @@ public class BlockchainTestSubCommand implements Runnable {
final BlockHeader genesisBlockHeader = spec.getGenesisBlockHeader();
final MutableWorldState worldState =
spec.getWorldStateArchive()
.getMutable(genesisBlockHeader.getStateRoot(), genesisBlockHeader.getHash())
.getWorldState(
WorldStateQueryParams.withStateRootAndBlockHashAndUpdateNodeHead(
genesisBlockHeader.getStateRoot(), genesisBlockHeader.getHash()))
.orElseThrow();
final ProtocolSchedule schedule =

View File

@@ -68,7 +68,7 @@ public class NodeSmartContractPermissioningControllerTest {
final MutableBlockchain blockchain = createInMemoryBlockchain(genesisState.getBlock());
final WorldStateArchive worldArchive = createInMemoryWorldStateArchive();
genesisState.writeStateTo(worldArchive.getMutable());
genesisState.writeStateTo(worldArchive.getWorldState());
final TransactionSimulator ts =
new TransactionSimulator(

View File

@@ -66,7 +66,7 @@ public class TransactionSmartContractPermissioningControllerTest {
final MutableBlockchain blockchain = createInMemoryBlockchain(genesisState.getBlock());
final WorldStateArchive worldArchive = createInMemoryWorldStateArchive();
genesisState.writeStateTo(worldArchive.getMutable());
genesisState.writeStateTo(worldArchive.getWorldState());
final TransactionSimulator ts =
new TransactionSimulator(

View File

@@ -73,7 +73,7 @@ public class BlockchainReferenceTestCaseSpec {
final Map<String, ReferenceTestWorldState.AccountMock> accounts) {
final WorldStateArchive worldStateArchive = createInMemoryWorldStateArchive();
final MutableWorldState worldState = worldStateArchive.getMutable();
final MutableWorldState worldState = worldStateArchive.getWorldState();
final WorldUpdater updater = worldState.updater();
for (final Map.Entry<String, ReferenceTestWorldState.AccountMock> entry : accounts.entrySet()) {

View File

@@ -14,6 +14,8 @@
*/
package org.hyperledger.besu.ethereum.referencetests;
import static org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.WorldStateConfig.createStatefulConfigWithTrie;
import org.hyperledger.besu.datatypes.Address;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.core.BlockHeader;
@@ -28,7 +30,6 @@ import org.hyperledger.besu.ethereum.trie.diffbased.common.cache.DiffBasedCached
import org.hyperledger.besu.ethereum.trie.diffbased.common.trielog.TrieLogAddedEvent;
import org.hyperledger.besu.ethereum.trie.diffbased.common.trielog.TrieLogManager;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldState;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.DiffBasedWorldStateConfig;
import org.hyperledger.besu.ethereum.trie.diffbased.common.worldview.accumulator.DiffBasedWorldStateUpdateAccumulator;
import org.hyperledger.besu.ethereum.worldstate.DataStorageConfiguration;
import org.hyperledger.besu.evm.internal.EvmConfiguration;
@@ -74,7 +75,7 @@ public class BonsaiReferenceTestWorldState extends BonsaiWorldState
cachedWorldStorageManager,
trieLogManager,
evmConfiguration,
new DiffBasedWorldStateConfig());
createStatefulConfigWithTrie());
this.refTestStorage = worldStateKeyValueStorage;
this.preImageProxy = preImageProxy;
this.evmConfiguration = evmConfiguration;
@@ -205,7 +206,7 @@ public class BonsaiReferenceTestWorldState extends BonsaiWorldState
private BonsaiWorldState createBonsaiWorldState(final boolean isFrozen) {
final BonsaiReferenceTestWorldState copy = (BonsaiReferenceTestWorldState) this.copy();
if (isFrozen) {
copy.freeze();
copy.freezeStorage();
}
return copy;
}

View File

@@ -30,6 +30,7 @@ import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec;
import org.hyperledger.besu.ethereum.referencetests.BlockchainReferenceTestCaseSpec;
import org.hyperledger.besu.ethereum.referencetests.ReferenceTestProtocolSchedules;
import org.hyperledger.besu.ethereum.rlp.RLPException;
import org.hyperledger.besu.ethereum.trie.diffbased.common.provider.WorldStateQueryParams;
import org.hyperledger.besu.evm.EVM;
import org.hyperledger.besu.evm.EvmSpecVersion;
import org.hyperledger.besu.evm.account.AccountState;
@@ -109,7 +110,7 @@ public class BlockchainReferenceTestTools {
final BlockHeader genesisBlockHeader = spec.getGenesisBlockHeader();
final MutableWorldState worldState =
spec.getWorldStateArchive()
.getMutable(genesisBlockHeader.getStateRoot(), genesisBlockHeader.getHash())
.getWorldState(WorldStateQueryParams.withStateRootAndBlockHashAndUpdateNodeHead(genesisBlockHeader.getStateRoot(), genesisBlockHeader.getHash()))
.orElseThrow();
final ProtocolSchedule schedule =

View File

@@ -71,7 +71,7 @@ Calculated : ${currentHash}
tasks.register('checkAPIChanges', FileStateChecker) {
description = "Checks that the API for the Plugin-API project does not change without deliberate thought"
files = sourceSets.main.allJava.files
knownHash = 'V3YwoXiJjbbrtpr7DmbebhJwAcj40J/3gb6VZcoFlF8='
knownHash = 'E2b/W+IKnNxo6L7cHuijBMBUewHHRrkQ8dEVlcql5KE='
}
check.dependsOn('checkAPIChanges')

View File

@@ -1,26 +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.plugin.services.sync;
/** interface for worldstate configuration * */
public interface WorldStateConfiguration {
/**
* Returns whether the trie is disabled.
*
* @return true if the trie is disabled, false otherwise.
*/
boolean isTrieDisabled();
}