From 5d193b0a91b836620b8b170b33655b5bcab4eb06 Mon Sep 17 00:00:00 2001 From: garyschulte Date: Wed, 13 Sep 2023 21:30:18 -0700 Subject: [PATCH] Bonsai based reference test worldstate (#5686) * create a bonsai based reference test worldstate -> getOrCreate in BonsaiWorldStateUpdateAccumulator - do not throw if we discover an empty account in a non-null BonsaiValue -> add curentStateRoot to t8n -> storageEntriesFrom and streamAccounts implemented in BonsaiWorldStateKeyValueStorage -> add endKey version of streamFromKey * bonsai fix for self-destruct and create2 at the same address and same block Signed-off-by: garyschulte Signed-off-by: Karim TAAM Co-authored-by: Karim TAAM --- .../besu/ethereum/bonsai/BonsaiAccount.java | 2 +- .../bonsai/BonsaiWorldStateProvider.java | 2 +- .../bonsai/storage/BonsaiPreImageProxy.java | 68 +++++++ .../BonsaiWorldStateKeyValueStorage.java | 8 +- .../storage/flat/FlatDbReaderStrategy.java | 13 +- .../bonsai/worldview/BonsaiWorldState.java | 80 ++++---- .../BonsaiWorldStateUpdateAccumulator.java | 60 ++++-- .../mainnet/MainnetBlockProcessorTest.java | 3 + .../backwardsync/BackwardSyncContextTest.java | 7 +- .../backwardsync/ForwardSyncStepTest.java | 5 +- .../besu/evmtool/StateTestSubCommand.java | 7 +- .../hyperledger/besu/evmtool/T8nExecutor.java | 6 +- .../besu/evmtool/T8nSubCommand.java | 3 +- .../besu/evmtool/state-test/blockhash.json | 4 +- .../t8n/berlin-calculate-difficulty.json | 3 +- .../besu/evmtool/t8n/berlin-example-yul.json | 3 +- .../t8n/berlin-negative-cli-reward.json | 3 +- .../evmtool/t8n/berlin-no-cli-reward.json | 3 +- .../besu/evmtool/t8n/berlin-no-tx.json | 3 +- .../besu/evmtool/t8n/berlin-simple.json | 3 +- .../t8n/cancun-6780-selfdestruct-sweep.json | 3 +- .../t8n/cancun-6780-selfdestruct-to-self.json | 3 +- .../cancun-6780-selfdestruct-transient.json | 3 +- .../besu/evmtool/t8n/cancun-blobs-per-tx.json | 3 +- .../evmtool/t8n/istanbul-cumulative-gas.json | 1 + .../evmtool/t8n/london-env-no-basefee.json | 3 +- .../besu/evmtool/t8n/london-hex.json | 3 +- .../besu/evmtool/t8n/shanghai-blockhash.json | 3 +- .../besu/evmtool/t8n/shanghai-init-code.json | 3 +- .../besu/evmtool/t8n/shanghai-invalidTx.json | 3 +- .../t8n/shanghai-withdrawals-no-nonce.json | 3 +- .../t8n/shanghai-withdrawals-overflow.json | 3 +- .../evmtool/t8n/shanghai-withdrawals.json | 3 +- ethereum/referencetests/build.gradle | 3 + .../BonsaiReferenceTestUpdateAccumulator.java | 44 +++++ .../BonsaiReferenceTestWorldState.java | 182 ++++++++++++++++++ .../BonsaiReferenceTestWorldStateStorage.java | 85 ++++++++ .../DefaultReferenceTestWorldState.java | 60 ++++++ .../GeneralStateTestCaseEipSpec.java | 7 +- .../GeneralStateTestCaseSpec.java | 11 +- .../referencetests/ReferenceTestEnv.java | 3 +- .../ReferenceTestWorldState.java | 30 +-- .../vm/GeneralStateReferenceTestTools.java | 8 +- plugin-api/build.gradle | 2 +- .../services/storage/KeyValueStorage.java | 11 ++ .../storage/SegmentedKeyValueStorage.java | 13 ++ .../RocksDBColumnarKeyValueSnapshot.java | 9 +- .../RocksDBColumnarKeyValueStorage.java | 11 ++ .../segmented/RocksDBSnapshotTransaction.java | 43 +++++ .../kvstore/LayeredKeyValueStorage.java | 12 +- .../LimitedInMemoryKeyValueStorage.java | 7 + .../SegmentedInMemoryKeyValueStorage.java | 13 +- .../SegmentedKeyValueStorageAdapter.java | 10 +- 53 files changed, 746 insertions(+), 141 deletions(-) create mode 100644 ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/BonsaiPreImageProxy.java create mode 100644 ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestUpdateAccumulator.java create mode 100644 ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestWorldState.java create mode 100644 ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestWorldStateStorage.java create mode 100644 ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/DefaultReferenceTestWorldState.java diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/BonsaiAccount.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/BonsaiAccount.java index a5488cb9a2e..7362d345ce4 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/BonsaiAccount.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/BonsaiAccount.java @@ -219,7 +219,7 @@ public UInt256 getOriginalStorageValue(final UInt256 key) { @Override public NavigableMap storageEntriesFrom( final Bytes32 startKeyHash, final int limit) { - throw new RuntimeException("Bonsai Tries does not currently support enumerating storage"); + return context.getWorldStateStorage().storageEntriesFrom(this.addressHash, startKeyHash, limit); } public Bytes serializeAccount() { diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/BonsaiWorldStateProvider.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/BonsaiWorldStateProvider.java index 986e736745a..6388d62a584 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/BonsaiWorldStateProvider.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/BonsaiWorldStateProvider.java @@ -102,8 +102,8 @@ public BonsaiWorldStateProvider( pluginContext); this.blockchain = blockchain; this.worldStateStorage = worldStateStorage; - this.persistedState = new BonsaiWorldState(this, worldStateStorage); this.cachedMerkleTrieLoader = cachedMerkleTrieLoader; + this.persistedState = new BonsaiWorldState(this, worldStateStorage); blockchain .getBlockHeader(persistedState.getWorldStateBlockHash()) .ifPresent( diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/BonsaiPreImageProxy.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/BonsaiPreImageProxy.java new file mode 100644 index 00000000000..248e5d895fd --- /dev/null +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/BonsaiPreImageProxy.java @@ -0,0 +1,68 @@ +/* + * Copyright Hyperledger Besu Contributors. + * + * 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.bonsai.storage; + +import org.hyperledger.besu.datatypes.Address; +import org.hyperledger.besu.datatypes.Hash; +import org.hyperledger.besu.ethereum.worldstate.WorldStatePreimageStorage; + +import java.util.Optional; + +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; +import org.apache.tuweni.bytes.Bytes; +import org.apache.tuweni.bytes.Bytes32; +import org.apache.tuweni.units.bigints.UInt256; + +/** Acts as both a Hasher and PreImageStorage for Bonsai storage format. */ +public interface BonsaiPreImageProxy extends WorldStatePreimageStorage { + /** + * If this value is not already present, save in preImage store and return the hash value. + * + * @param value value to hash + * @return Hash of value + */ + Hash hashAndSavePreImage(Bytes value); + + /** + * A caching PreImageProxy suitable for ReferenceTestWorldState which saves hashes in an unbounded + * BiMap. + */ + class BonsaiReferenceTestPreImageProxy implements BonsaiPreImageProxy { + BiMap preImageCache = HashBiMap.create(); + + @Override + public synchronized Hash hashAndSavePreImage(final Bytes value) { + return preImageCache.inverse().computeIfAbsent(value, Hash::hash); + } + + @Override + public Optional getStorageTrieKeyPreimage(final Bytes32 trieKey) { + return Optional.ofNullable(preImageCache.get(trieKey)).map(UInt256::fromBytes); + } + + @Override + public Optional
getAccountTrieKeyPreimage(final Bytes32 trieKey) { + return Optional.ofNullable(preImageCache.get(trieKey)).map(Address::wrap); + } + + @Override + public Updater updater() { + throw new UnsupportedOperationException( + "BonsaiReferenceTestPreImageProxy does not implement an updater"); + } + } +} diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/BonsaiWorldStateKeyValueStorage.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/BonsaiWorldStateKeyValueStorage.java index 4ceb63ef219..1201fffb3b8 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/BonsaiWorldStateKeyValueStorage.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/BonsaiWorldStateKeyValueStorage.java @@ -31,6 +31,7 @@ import org.hyperledger.besu.ethereum.worldstate.FlatDbMode; import org.hyperledger.besu.ethereum.worldstate.StateTrieAccountValue; import org.hyperledger.besu.ethereum.worldstate.WorldStateStorage; +import org.hyperledger.besu.evm.account.AccountStorageEntry; import org.hyperledger.besu.metrics.ObservableMetricsSystem; import org.hyperledger.besu.plugin.services.storage.KeyValueStorage; import org.hyperledger.besu.plugin.services.storage.KeyValueStorageTransaction; @@ -41,6 +42,7 @@ import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; +import java.util.NavigableMap; import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Predicate; @@ -53,7 +55,6 @@ @SuppressWarnings("unused") public class BonsaiWorldStateKeyValueStorage implements WorldStateStorage, AutoCloseable { - private static final Logger LOG = LoggerFactory.getLogger(BonsaiWorldStateKeyValueStorage.class); // 0x776f726c64526f6f74 @@ -250,6 +251,11 @@ public Map streamFlatStorages( composedWorldStateStorage, accountHash, startKeyHash, endKeyHash, max); } + public NavigableMap storageEntriesFrom( + final Hash addressHash, final Bytes32 startKeyHash, final int limit) { + throw new RuntimeException("Bonsai Tries does not currently support enumerating storage"); + } + @Override public Optional getNodeData(final Bytes location, final Bytes32 hash) { return Optional.empty(); diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/flat/FlatDbReaderStrategy.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/flat/FlatDbReaderStrategy.java index 1a35dbb93fe..74c7b101c25 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/flat/FlatDbReaderStrategy.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/storage/flat/FlatDbReaderStrategy.java @@ -135,10 +135,10 @@ public Map streamAccountFlatDatabase( final long max) { final Stream> pairStream = storage - .streamFromKey(ACCOUNT_INFO_STATE, startKeyHash.toArrayUnsafe()) + .streamFromKey( + ACCOUNT_INFO_STATE, startKeyHash.toArrayUnsafe(), endKeyHash.toArrayUnsafe()) .limit(max) - .map(pair -> new Pair<>(Bytes32.wrap(pair.getKey()), Bytes.wrap(pair.getValue()))) - .takeWhile(pair -> pair.getFirst().compareTo(endKeyHash) <= 0); + .map(pair -> new Pair<>(Bytes32.wrap(pair.getKey()), Bytes.wrap(pair.getValue()))); final TreeMap collected = pairStream.collect( @@ -157,15 +157,14 @@ public Map streamStorageFlatDatabase( storage .streamFromKey( ACCOUNT_STORAGE_STORAGE, - Bytes.concatenate(accountHash, startKeyHash).toArrayUnsafe()) - .takeWhile(pair -> Bytes.wrap(pair.getKey()).slice(0, Hash.SIZE).equals(accountHash)) + Bytes.concatenate(accountHash, startKeyHash).toArrayUnsafe(), + Bytes.concatenate(accountHash, endKeyHash).toArrayUnsafe()) .limit(max) .map( pair -> new Pair<>( Bytes32.wrap(Bytes.wrap(pair.getKey()).slice(Hash.SIZE)), - RLP.encodeValue(Bytes.wrap(pair.getValue()).trimLeadingZeros()))) - .takeWhile(pair -> pair.getFirst().compareTo(endKeyHash) <= 0); + RLP.encodeValue(Bytes.wrap(pair.getValue()).trimLeadingZeros()))); final TreeMap collected = pairStream.collect( diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/worldview/BonsaiWorldState.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/worldview/BonsaiWorldState.java index 865471eca99..f28e24b0970 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/worldview/BonsaiWorldState.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/worldview/BonsaiWorldState.java @@ -27,6 +27,7 @@ import org.hyperledger.besu.ethereum.bonsai.BonsaiAccount; import org.hyperledger.besu.ethereum.bonsai.BonsaiValue; import org.hyperledger.besu.ethereum.bonsai.BonsaiWorldStateProvider; +import org.hyperledger.besu.ethereum.bonsai.cache.CachedMerkleTrieLoader; import org.hyperledger.besu.ethereum.bonsai.storage.BonsaiSnapshotWorldStateKeyValueStorage; import org.hyperledger.besu.ethereum.bonsai.storage.BonsaiWorldStateKeyValueStorage; import org.hyperledger.besu.ethereum.bonsai.storage.BonsaiWorldStateKeyValueStorage.BonsaiStorageSubscriber; @@ -65,51 +66,52 @@ public class BonsaiWorldState private static final Logger LOG = LoggerFactory.getLogger(BonsaiWorldState.class); - private BonsaiWorldStateKeyValueStorage worldStateStorage; + protected BonsaiWorldStateKeyValueStorage worldStateStorage; - private final BonsaiWorldStateProvider archive; - private final BonsaiWorldStateUpdateAccumulator accumulator; + protected final CachedMerkleTrieLoader cachedMerkleTrieLoader; + protected final TrieLogManager trieLogManager; + private BonsaiWorldStateUpdateAccumulator accumulator; - private Hash worldStateRootHash; + protected Hash worldStateRootHash; Hash worldStateBlockHash; - private boolean isFrozen; public BonsaiWorldState( final BonsaiWorldStateProvider archive, final BonsaiWorldStateKeyValueStorage worldStateStorage) { - this.archive = archive; + this(worldStateStorage, archive.getCachedMerkleTrieLoader(), archive.getTrieLogManager()); + } + + protected BonsaiWorldState( + final BonsaiWorldStateKeyValueStorage worldStateStorage, + final CachedMerkleTrieLoader cachedMerkleTrieLoader, + final TrieLogManager trieLogManager) { this.worldStateStorage = worldStateStorage; - worldStateRootHash = + this.worldStateRootHash = Hash.wrap( Bytes32.wrap(worldStateStorage.getWorldStateRootHash().orElse(Hash.EMPTY_TRIE_HASH))); - worldStateBlockHash = + this.worldStateBlockHash = Hash.wrap(Bytes32.wrap(worldStateStorage.getWorldStateBlockHash().orElse(Hash.ZERO))); - accumulator = + this.accumulator = new BonsaiWorldStateUpdateAccumulator( this, (addr, value) -> - archive - .getCachedMerkleTrieLoader() - .preLoadAccount(getWorldStateStorage(), worldStateRootHash, addr), + cachedMerkleTrieLoader.preLoadAccount( + getWorldStateStorage(), worldStateRootHash, addr), (addr, value) -> - archive - .getCachedMerkleTrieLoader() - .preLoadStorageSlot(getWorldStateStorage(), addr, value)); + cachedMerkleTrieLoader.preLoadStorageSlot(getWorldStateStorage(), addr, value)); + this.cachedMerkleTrieLoader = cachedMerkleTrieLoader; + this.trieLogManager = trieLogManager; } - public BonsaiWorldState( - final BonsaiWorldStateProvider archive, - final BonsaiWorldStateKeyValueStorage worldStateStorage, - final BonsaiWorldStateUpdateAccumulator updater) { - this.archive = archive; - this.worldStateStorage = worldStateStorage; - this.worldStateRootHash = - Hash.wrap( - Bytes32.wrap(worldStateStorage.getWorldStateRootHash().orElse(Hash.EMPTY_TRIE_HASH))); - this.worldStateBlockHash = - Hash.wrap(Bytes32.wrap(worldStateStorage.getWorldStateBlockHash().orElse(Hash.ZERO))); - this.accumulator = updater; + /** + * Having a protected method to override the accumulator solves the chicken-egg problem of needing + * a worldstate reference (this) when construction the Accumulator. + * + * @param accumulator accumulator to use. + */ + protected void setAccumulator(final BonsaiWorldStateUpdateAccumulator accumulator) { + this.accumulator = accumulator; } /** @@ -130,10 +132,6 @@ public Hash getWorldStateRootHash() { return worldStateRootHash; } - public BonsaiWorldStateProvider getArchive() { - return archive; - } - @Override public boolean isPersisted() { return isPersisted(worldStateStorage); @@ -189,9 +187,7 @@ private Hash calculateRootHash( final StoredMerklePatriciaTrie accountTrie = createTrie( (location, hash) -> - archive - .getCachedMerkleTrieLoader() - .getAccountStateTrieNode(worldStateStorage, location, hash), + cachedMerkleTrieLoader.getAccountStateTrieNode(worldStateStorage, location, hash), worldStateRootHash); // for manicured tries and composting, collect branches here (not implemented) @@ -223,7 +219,7 @@ private void updateTheAccounts( final BonsaiAccount updatedAccount = bonsaiValue.getUpdated(); try { if (updatedAccount == null) { - final Hash addressHash = Hash.hash(accountKey); + final Hash addressHash = hashAndSavePreImage(accountKey); accountTrie.remove(addressHash); maybeStateUpdater.ifPresent( bonsaiUpdater -> bonsaiUpdater.removeAccountInfoState(addressHash)); @@ -232,7 +228,7 @@ private void updateTheAccounts( final Bytes accountValue = updatedAccount.serializeAccount(); maybeStateUpdater.ifPresent( bonsaiUpdater -> - bonsaiUpdater.putAccountInfoState(Hash.hash(accountKey), accountValue)); + bonsaiUpdater.putAccountInfoState(hashAndSavePreImage(accountKey), accountValue)); accountTrie.put(addressHash, accountValue); } } catch (MerkleTrieException e) { @@ -277,10 +273,8 @@ private void updateAccountStorageState( final StoredMerklePatriciaTrie storageTrie = createTrie( (location, key) -> - archive - .getCachedMerkleTrieLoader() - .getAccountStorageTrieNode( - worldStateStorage, updatedAddressHash, location, key), + cachedMerkleTrieLoader.getAccountStorageTrieNode( + worldStateStorage, updatedAddressHash, location, key), storageRoot); // for manicured tries and composting, collect branches here (not implemented) @@ -405,7 +399,6 @@ public void persist(final BlockHeader blockHeader) { } saveTrieLog = () -> { - final TrieLogManager trieLogManager = archive.getTrieLogManager(); trieLogManager.saveTrieLog(localCopy, newWorldStateRootHash, blockHeader, this); // not save a frozen state in the cache if (!isFrozen) { @@ -626,4 +619,9 @@ private void closeFrozenStorage() { // no op } } + + protected Hash hashAndSavePreImage(final Bytes value) { + // by default do not save has preImages + return Hash.hash(value); + } } diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/worldview/BonsaiWorldStateUpdateAccumulator.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/worldview/BonsaiWorldStateUpdateAccumulator.java index b1cd64378b9..23fd7625fa3 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/worldview/BonsaiWorldStateUpdateAccumulator.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/bonsai/worldview/BonsaiWorldStateUpdateAccumulator.java @@ -127,14 +127,18 @@ public MutableAccount createAccount(final Address address, final long nonce, fin bonsaiValue = new BonsaiValue<>(null, null); accountsToUpdate.put(address, bonsaiValue); } else if (bonsaiValue.getUpdated() != null) { - throw new IllegalStateException("Cannot create an account when one already exists"); + if (bonsaiValue.getUpdated().isEmpty()) { + return track(new UpdateTrackingAccount<>(bonsaiValue.getUpdated())); + } else { + throw new IllegalStateException("Cannot create an account when one already exists"); + } } final BonsaiAccount newAccount = new BonsaiAccount( this, address, - address.addressHash(), + hashAndSavePreImage(address), nonce, balance, Hash.EMPTY_TRIE_HASH, @@ -288,6 +292,19 @@ public void commit() { final BonsaiAccount updatedAccount; final BonsaiValue updatedAccountValue = accountsToUpdate.get(updatedAddress); + + final Map> pendingStorageUpdates = + storageToUpdate.computeIfAbsent( + updatedAddress, + k -> + new StorageConsumingMap<>( + updatedAddress, new ConcurrentHashMap<>(), storagePreloader)); + + if (tracked.getStorageWasCleared()) { + storageToClear.add(updatedAddress); + pendingStorageUpdates.clear(); + } + if (tracked.getWrappedAccount() == null) { updatedAccount = new BonsaiAccount(this, tracked); tracked.setWrappedAccount(updatedAccount); @@ -307,6 +324,17 @@ public void commit() { } if (tracked.getStorageWasCleared()) { updatedAccount.clearStorage(); + wrappedWorldView() + .getAllAccountStorage(updatedAddress, updatedAccount.getStorageRoot()) + .forEach( + (keyHash, entryValue) -> { + final StorageSlotKey storageSlotKey = + new StorageSlotKey(Hash.wrap(keyHash), Optional.empty()); + final UInt256 value = UInt256.fromBytes(RLP.decodeOne(entryValue)); + pendingStorageUpdates.put( + storageSlotKey, new BonsaiValue<>(value, null, true)); + }); + updatedAccount.setStorageRoot(Hash.EMPTY_TRIE_HASH); } tracked.getUpdatedStorage().forEach(updatedAccount::setStorageValue); } @@ -329,24 +357,12 @@ public void commit() { pendingCode.setUpdated(updatedAccount.getCode()); } - // This is especially to avoid unnecessary computation for withdrawals + // This is especially to avoid unnecessary computation for withdrawals and + // self-destruct beneficiaries if (updatedAccount.getUpdatedStorage().isEmpty()) { return; } - final StorageConsumingMap> - pendingStorageUpdates = - storageToUpdate.computeIfAbsent( - updatedAddress, - __ -> - new StorageConsumingMap<>( - updatedAddress, new ConcurrentHashMap<>(), storagePreloader)); - - if (tracked.getStorageWasCleared()) { - storageToClear.add(updatedAddress); - pendingStorageUpdates.clear(); - } - final TreeSet> entries = new TreeSet<>(Map.Entry.comparingByKey()); entries.addAll(updatedAccount.getUpdatedStorage().entrySet()); @@ -355,7 +371,7 @@ public void commit() { entries.forEach( storageUpdate -> { final UInt256 keyUInt = storageUpdate.getKey(); - final Hash slotHash = Hash.hash(keyUInt); + final Hash slotHash = hashAndSavePreImage(keyUInt); final StorageSlotKey slotKey = new StorageSlotKey(slotHash, Optional.of(keyUInt)); final UInt256 value = storageUpdate.getValue(); @@ -399,7 +415,8 @@ public Optional getCode(final Address address, final Hash codeHash) { @Override public UInt256 getStorageValue(final Address address, final UInt256 slotKey) { - StorageSlotKey storageSlotKey = new StorageSlotKey(Hash.hash(slotKey), Optional.of(slotKey)); + StorageSlotKey storageSlotKey = + new StorageSlotKey(hashAndSavePreImage(slotKey), Optional.of(slotKey)); return getStorageValueByStorageSlotKey(address, storageSlotKey).orElse(UInt256.ZERO); } @@ -443,7 +460,7 @@ public Optional getStorageValueByStorageSlotKey( public UInt256 getPriorStorageValue(final Address address, final UInt256 storageKey) { // TODO maybe log the read into the trie layer? StorageSlotKey storageSlotKey = - new StorageSlotKey(Hash.hash(storageKey), Optional.of(storageKey)); + new StorageSlotKey(hashAndSavePreImage(storageKey), Optional.of(storageKey)); final Map> localAccountStorage = storageToUpdate.get(address); if (localAccountStorage != null) { @@ -816,4 +833,9 @@ protected Map delegate() { public interface Consumer { void process(final Address address, T value); } + + protected Hash hashAndSavePreImage(final Bytes bytes) { + // by default do not save hash preImages + return Hash.hash(bytes); + } } diff --git a/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/MainnetBlockProcessorTest.java b/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/MainnetBlockProcessorTest.java index 6bd6978acb1..1b106686bfc 100644 --- a/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/MainnetBlockProcessorTest.java +++ b/ethereum/core/src/test/java/org/hyperledger/besu/ethereum/mainnet/MainnetBlockProcessorTest.java @@ -94,6 +94,9 @@ public void accountCreatedWhenBlockRewardIsZeroAndNotSkipped() { final BlockHeader emptyBlockHeader = new BlockHeaderTestFixture() .transactionsRoot(Hash.EMPTY_LIST_HASH) + .stateRoot( + Hash.fromHexString( + "0xa6b5d50f7b3c39b969c2fe8fed091939c674fef49b4826309cb6994361e39b71")) .ommersHash(Hash.EMPTY_LIST_HASH) .buildHeader(); blockProcessor.processBlock(blockchain, worldState, emptyBlockHeader, emptyList(), emptyList()); diff --git a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java index 3ed28e66f4d..b5909c4d429 100644 --- a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java +++ b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/BackwardSyncContextTest.java @@ -47,7 +47,7 @@ import org.hyperledger.besu.ethereum.mainnet.MainnetProtocolSchedule; import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec; -import org.hyperledger.besu.ethereum.referencetests.ReferenceTestWorldState; +import org.hyperledger.besu.ethereum.referencetests.DefaultReferenceTestWorldState; import org.hyperledger.besu.plugin.services.MetricsSystem; import org.hyperledger.besu.services.kvstore.InMemoryKeyValueStorage; @@ -149,7 +149,10 @@ public void setup() { return new BlockProcessingResult( Optional.of( new BlockProcessingOutputs( - new ReferenceTestWorldState(), blockDataGenerator.receipts(block)))); + // use forest-based worldstate since it does not require + // blockheader stateroot to match actual worldstate root + DefaultReferenceTestWorldState.create(Collections.emptyMap()), + blockDataGenerator.receipts(block)))); }); backwardChain = inMemoryBackwardChain(); diff --git a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStepTest.java b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStepTest.java index 77308ebcc3b..e38a9647b79 100644 --- a/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStepTest.java +++ b/ethereum/eth/src/test/java/org/hyperledger/besu/ethereum/eth/sync/backwardsync/ForwardSyncStepTest.java @@ -38,7 +38,7 @@ import org.hyperledger.besu.ethereum.mainnet.MainnetBlockHeaderFunctions; import org.hyperledger.besu.ethereum.mainnet.MainnetProtocolSchedule; import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; -import org.hyperledger.besu.ethereum.referencetests.ReferenceTestWorldState; +import org.hyperledger.besu.ethereum.referencetests.DefaultReferenceTestWorldState; import org.hyperledger.besu.services.kvstore.InMemoryKeyValueStorage; import java.nio.charset.StandardCharsets; @@ -138,7 +138,8 @@ public void setup() { return new BlockProcessingResult( Optional.of( new BlockProcessingOutputs( - new ReferenceTestWorldState(), blockDataGenerator.receipts(block)))); + DefaultReferenceTestWorldState.create(Collections.emptyMap()), + blockDataGenerator.receipts(block)))); }); } diff --git a/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/StateTestSubCommand.java b/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/StateTestSubCommand.java index 95af770c589..2bc5f815309 100644 --- a/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/StateTestSubCommand.java +++ b/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/StateTestSubCommand.java @@ -37,12 +37,10 @@ import org.hyperledger.besu.ethereum.referencetests.ReferenceTestBlockchain; import org.hyperledger.besu.ethereum.referencetests.ReferenceTestProtocolSchedules; import org.hyperledger.besu.ethereum.rlp.RLP; -import org.hyperledger.besu.ethereum.worldstate.DefaultMutableWorldState; import org.hyperledger.besu.evm.account.Account; import org.hyperledger.besu.evm.log.Log; import org.hyperledger.besu.evm.tracing.OperationTracer; import org.hyperledger.besu.evm.tracing.StandardJsonTracer; -import org.hyperledger.besu.evm.worldstate.WorldState; import org.hyperledger.besu.evm.worldstate.WorldUpdater; import org.hyperledger.besu.evmtool.exception.UnsupportedForkException; import org.hyperledger.besu.util.LogConfigurator; @@ -200,9 +198,7 @@ private void traceTestSpecs(final String test, final List transactions, final List rejections, final TracerManager tracerManager) { @@ -238,7 +238,7 @@ static T8nResult runTest( ReferenceTestProtocolSchedules.create( new StubGenesisConfigOptions().chainId(BigInteger.valueOf(chainId))); - final MutableWorldState worldState = new DefaultMutableWorldState(initialWorldState); + final MutableWorldState worldState = initialWorldState.copy(); final ProtocolSchedule protocolSchedule = referenceTestProtocolSchedules.getByName(fork); if (protocolSchedule == null) { diff --git a/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/T8nSubCommand.java b/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/T8nSubCommand.java index 1571dc3551e..ea2984c5c90 100644 --- a/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/T8nSubCommand.java +++ b/ethereum/evmtool/src/main/java/org/hyperledger/besu/evmtool/T8nSubCommand.java @@ -21,7 +21,6 @@ import org.hyperledger.besu.crypto.SignatureAlgorithmFactory; import org.hyperledger.besu.datatypes.Hash; -import org.hyperledger.besu.ethereum.core.MutableWorldState; import org.hyperledger.besu.ethereum.core.Transaction; import org.hyperledger.besu.ethereum.referencetests.ReferenceTestEnv; import org.hyperledger.besu.ethereum.referencetests.ReferenceTestWorldState; @@ -175,7 +174,7 @@ public void run() { final ObjectMapper objectMapper = JsonUtils.createObjectMapper(); final ObjectReader t8nReader = objectMapper.reader(); - MutableWorldState initialWorldState; + ReferenceTestWorldState initialWorldState; ReferenceTestEnv referenceTestEnv; List transactions = new ArrayList<>(); List rejections = new ArrayList<>(); diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/state-test/blockhash.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/state-test/blockhash.json index b5bac5462ef..de44541641d 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/state-test/blockhash.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/state-test/blockhash.json @@ -61,7 +61,7 @@ "post": { "Shanghai": [ { - "hash": "0x0000000000000000000000000000000000000000000000000000000000000000", + "hash": "0xd14c10ed22a1cfb642e374be985ac581c39f3969bd59249e0405aca3beb47a47", "logs": "0x0000000000000000000000000000000000000000000000000000000000000000", "indexes": { "data": 0, @@ -101,4 +101,4 @@ {"pc":83,"op":62,"gas":"0x79bc18","gasCost":"0x0","memSize":0,"stack":["0x0","0x1","0x1","0x2","0x2","0xffff","0x1f4","0x78859e5b97166c486532b1595a673e9f9073643f1b519c6f18511b9913","0x2","0x389","0x0","0x0","0x1","0x0","0x3e3d6d5ff042148d326c1898713a76759ca273","0xb94f5374fce5edbc8e2a8697c15331677e6ebf1b"],"depth":1,"refund":0,"opName":"RETURNDATACOPY","error":"Out of bounds"}, {"output":"","gasUsed":"0x7a1200","test":"00000936-mixed-1","fork":"Shanghai","d":0,"g":0,"v":0,"postHash":"0xd14c10ed22a1cfb642e374be985ac581c39f3969bd59249e0405aca3beb47a47","postLogsHash":"0x1dcc4de8dec75d7aab85b567b6ccd41ad312451b948a7413f0a142fd40d49347","pass":false} ] -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-calculate-difficulty.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-calculate-difficulty.json index 1a3c7a178c6..005cb1498ad 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-calculate-difficulty.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-calculate-difficulty.json @@ -48,6 +48,7 @@ "currentGasLimit": "100000000000000000", "currentNumber": "1", "currentTimestamp": "1000", + "currentStateRoot": "0xd921e74bfe864514fa508003336b8f66eb98d748c5163749827029a1ed7db265", "parentDifficulty": "131072", "parentGasUsed": "50000000000000000", "parentGasLimit": "100000000000000000", @@ -101,4 +102,4 @@ "gasUsed": "0xa8ad" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-example-yul.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-example-yul.json index 1a3c7a178c6..005cb1498ad 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-example-yul.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-example-yul.json @@ -48,6 +48,7 @@ "currentGasLimit": "100000000000000000", "currentNumber": "1", "currentTimestamp": "1000", + "currentStateRoot": "0xd921e74bfe864514fa508003336b8f66eb98d748c5163749827029a1ed7db265", "parentDifficulty": "131072", "parentGasUsed": "50000000000000000", "parentGasLimit": "100000000000000000", @@ -101,4 +102,4 @@ "gasUsed": "0xa8ad" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-negative-cli-reward.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-negative-cli-reward.json index 4ef273955c4..f41eb4989c4 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-negative-cli-reward.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-negative-cli-reward.json @@ -46,6 +46,7 @@ "env": { "currentCoinbase": "0x0000000000000000000000000000000000000000", "currentDifficulty": "0x0", + "currentStateRoot": "0xde762e485411037c02119ff1115b422945b37efe91f4135dd442c3346629d0ef", "currentGasLimit": "0x0", "currentNumber": "0", "currentTimestamp": "0" @@ -93,4 +94,4 @@ "gasUsed": "0xa8ad" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-no-cli-reward.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-no-cli-reward.json index d06c75ccfd2..4aba3a31a1d 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-no-cli-reward.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-no-cli-reward.json @@ -45,6 +45,7 @@ "env": { "currentCoinbase": "0x0000000000000000000000000000000000000000", "currentDifficulty": "0x0", + "currentStateRoot": "0x9580f6eec8ad54773eff8e370d12dfea7b3216e929657c48a509af64c4190be6", "currentGasLimit": "0x0", "currentNumber": "0", "currentTimestamp": "0" @@ -92,4 +93,4 @@ "gasUsed": "0xa8ad" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-no-tx.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-no-tx.json index 5abcf7bb347..8c655b71ec9 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-no-tx.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-no-tx.json @@ -30,6 +30,7 @@ "env": { "currentCoinbase": "0x0000000000000000000000000000000000000000", "currentDifficulty": "0x0", + "currentStateRoot": "0x369ba8b2e5b32d675d07933d6fb851d97d3ca66c60a829f7356163d92ae0439a", "currentGasLimit": "0x0", "currentNumber": "0", "currentTimestamp": "0" @@ -57,4 +58,4 @@ "gasUsed": "0x0" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-simple.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-simple.json index e4555f220b3..3fef5b5a296 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-simple.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/berlin-simple.json @@ -46,6 +46,7 @@ "env": { "currentCoinbase": "0x0000000000000000000000000000000000000000", "currentDifficulty": "0x0", + "currentStateRoot": "0xde762e485411037c02119ff1115b422945b37efe91f4135dd442c3346629d0ef", "currentGasLimit": "0x0", "currentNumber": "0", "currentTimestamp": "0" @@ -93,4 +94,4 @@ "gasUsed": "0xa8ad" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-sweep.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-sweep.json index bc7d113ec79..7e1ffaf68d1 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-sweep.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-sweep.json @@ -49,6 +49,7 @@ "currentTimestamp": "1000", "currentRandom": "0", "currentDifficulty": "0", + "currentStateRoot": "0x3a0e532de836d767cae901aba671040fedc07557d277f7203066f640ed95f78d", "parentDifficulty": "0", "parentBaseFee": "7", "parentGasUsed": "0", @@ -113,4 +114,4 @@ "currentExcessBlobGas": "0x0" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-to-self.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-to-self.json index 7d8907d9bfb..d922c54baa7 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-to-self.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-to-self.json @@ -51,6 +51,7 @@ "currentTimestamp": "1000", "currentRandom": "0", "currentDifficulty": "0", + "currentStateRoot" : "0xddd3a541e86e2dd0293959736de63e1fad74ae95149f34740b1173378e82527a", "parentDifficulty": "0", "parentBaseFee": "7", "parentGasUsed": "0", @@ -112,4 +113,4 @@ "currentExcessBlobGas": "0x0" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-transient.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-transient.json index 1860990b0f7..997b29f76bd 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-transient.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-6780-selfdestruct-transient.json @@ -43,6 +43,7 @@ "currentTimestamp": "1000", "currentRandom": "0", "currentDifficulty": "0", + "currentStateRoot": "0xe8a9461dcfdbaa48bbddca4f4baa439e45f1489827e5deeb2797363323e34769", "parentDifficulty": "0", "parentBaseFee": "7", "parentGasUsed": "0", @@ -97,4 +98,4 @@ "currentExcessBlobGas": "0x0" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-blobs-per-tx.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-blobs-per-tx.json index b2ed324b3d6..0912b52b81a 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-blobs-per-tx.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/cancun-blobs-per-tx.json @@ -53,6 +53,7 @@ "currentTimestamp": "24", "currentRandom": "0", "currentDifficulty": "0", + "currentStateRoot": "0xd29f5a8dd1a63e0a299009f546bdf447fb61f1604d95e737bd8eb3c089d78060", "parentDifficulty": "0", "parentBaseFee": "7", "parentGasUsed": "21000", @@ -114,4 +115,4 @@ "currentExcessBlobGas": "0x0" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/istanbul-cumulative-gas.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/istanbul-cumulative-gas.json index d3dd7021df7..087eae7745b 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/istanbul-cumulative-gas.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/istanbul-cumulative-gas.json @@ -363,6 +363,7 @@ "currentGasLimit": "100000000000000000", "currentNumber": "1", "currentTimestamp": "1000", + "currentStateRoot": "0xf4f4aed0d1813d2880d8bb1cb5697303c20002ac6bfc206635645f43805ccbcb", "parentDifficulty": "131072", "parentGasUsed": "50000000000000000", "parentGasLimit": "100000000000000000", diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/london-env-no-basefee.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/london-env-no-basefee.json index 052ebb5260b..bb2d2da5dbc 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/london-env-no-basefee.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/london-env-no-basefee.json @@ -48,6 +48,7 @@ "currentGasLimit": "100000000000000000", "currentNumber": "1", "currentTimestamp": "1000", + "currentStateRoot": "0x5c0f211cd1bea44ac01bd3fd7e0ab742d316adabd139bcf60a7aac6c458d596d", "parentDifficulty": "131072", "parentBaseFee": "7", "parentGasUsed": "50000000000000000", @@ -103,4 +104,4 @@ "currentBaseFee": "0x7" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/london-hex.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/london-hex.json index 502ac628de6..1ccce1739d6 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/london-hex.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/london-hex.json @@ -48,6 +48,7 @@ "currentNumber": "0x1", "currentTimestamp": "0x0", "currentGasLimit": "0xff112233445566", + "currentStateRoot": "0x5775e461ac316f8c02466d23e4492d24bbe546336f6fc632dd1311d6a5b67b22", "previousHash": "0xb271e9e5796d0ff5a2fd519ba666393e42d4f38680854761121d84a7a96ff017", "parentTimestamp": "0x00", "parentDifficulty": "0x00", @@ -102,4 +103,4 @@ "currentBaseFee": "0xa" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-blockhash.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-blockhash.json index e1c331206d2..9b2908976fc 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-blockhash.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-blockhash.json @@ -48,6 +48,7 @@ "currentTimestamp": "24", "currentRandom": "0", "currentDifficulty": "0", + "currentStateRoot": "0x89412c0f1bb31b983d0317e6e2801a4e604e1ef9987a132ab63c52f2d5a3994b", "parentDifficulty": "0", "parentGasUsed": "0", "parentGasLimit": "100000000000000000", @@ -106,4 +107,4 @@ "withdrawalsRoot": "0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-init-code.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-init-code.json index d536a6a7260..6c502e80cc8 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-init-code.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-init-code.json @@ -42,6 +42,7 @@ "currentTimestamp": "12", "currentRandom": "0", "currentDifficulty": "0", + "currentStateRoot": "0x70c42824108fafccadbfce71e6e22660c4fad89be18be324cd15ef351969a8c8", "parentDifficulty": "0", "parentBaseFee": "7", "parentGasUsed": "0", @@ -81,4 +82,4 @@ "withdrawalsRoot": "0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-invalidTx.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-invalidTx.json index 7ec8a1d7763..79087bf545a 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-invalidTx.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-invalidTx.json @@ -42,6 +42,7 @@ "currentNumber": "1", "currentTimestamp": "1000", "currentDifficulty": "0x20000", + "currentStateRoot": "0xec92f4c572101075d17eb5aaf15c33df92b6d5519cbed635fc53353b99e8e6da", "currentBaseFee": "0xa", "blockHashes": { "0": "0x5e20a0453cecd065ea59c37ac63e079ee08998b6045136a8ce6635c7912ec0b6" @@ -84,4 +85,4 @@ "withdrawalsRoot": "0x56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals-no-nonce.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals-no-nonce.json index 86cefab468f..4e9f1febebe 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals-no-nonce.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals-no-nonce.json @@ -40,6 +40,7 @@ "currentTimestamp": "24", "currentRandom": "0", "currentDifficulty": "0", + "currentStateRoot": "0x41855ebd4b425591ae60f57de6ec700c8a1128c7761b63371b09671b5f1abb3f", "parentDifficulty": "0", "parentGasUsed": "0", "parentGasLimit": "100000000000000000", @@ -98,4 +99,4 @@ "withdrawalsRoot": "0xa485a3bd07e29cb8234b5f093d5216eb8b965fb2693c66fea254f6cacef97a6f" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals-overflow.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals-overflow.json index 8f8b29b7525..e334d94cbff 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals-overflow.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals-overflow.json @@ -34,6 +34,7 @@ "currentTimestamp": "12", "currentRandom": "0", "currentDifficulty": "0", + "currentStateRoot": "0x2a23ea102cd739aa3ad83a9be1555d621e50b3a3f780a7369ec2a6c03f22e18d", "parentDifficulty": "0", "parentBaseFee": "7", "parentGasUsed": "0", @@ -78,4 +79,4 @@ "withdrawalsRoot": "0x7bae13217931ff0d7a6c5823c996492f28cd92da5a7788a9a68856bb2206741a" } } -} \ No newline at end of file +} diff --git a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals.json b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals.json index 5a4efc2eea0..946a97ceac6 100644 --- a/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals.json +++ b/ethereum/evmtool/src/test/resources/org/hyperledger/besu/evmtool/t8n/shanghai-withdrawals.json @@ -35,6 +35,7 @@ "currentTimestamp": "12", "currentRandom": "0", "currentDifficulty": "0", + "currentStateRoot": "0x7075d070d6ec70a91216fc2dba6ceae6b8d30815166f6b7b7249776767fb9855", "parentDifficulty": "0", "parentBaseFee": "7", "parentGasUsed": "0", @@ -81,4 +82,4 @@ "withdrawalsRoot": "0xa485a3bd07e29cb8234b5f093d5216eb8b965fb2693c66fea254f6cacef97a6f" } } -} \ No newline at end of file +} diff --git a/ethereum/referencetests/build.gradle b/ethereum/referencetests/build.gradle index a0b7db03d48..cf1a6e1f8a8 100644 --- a/ethereum/referencetests/build.gradle +++ b/ethereum/referencetests/build.gradle @@ -147,11 +147,14 @@ dependencies { implementation project(':crypto:algorithms') implementation project(':datatypes') implementation project(':ethereum:core') + implementation project(':metrics:core') + implementation project(':util') implementation project(path: ':ethereum:core', configuration: 'testSupportArtifacts') implementation project(':ethereum:rlp') implementation project(':evm') implementation project(':services:kvstore') + implementation 'io.tmio:tuweni-rlp' implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'com.google.guava:guava' diff --git a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestUpdateAccumulator.java b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestUpdateAccumulator.java new file mode 100644 index 00000000000..c8349b7f790 --- /dev/null +++ b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestUpdateAccumulator.java @@ -0,0 +1,44 @@ +/* + * Copyright Hyperledger Besu Contributors. + * + * 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.referencetests; + +import org.hyperledger.besu.datatypes.Hash; +import org.hyperledger.besu.datatypes.StorageSlotKey; +import org.hyperledger.besu.ethereum.bonsai.BonsaiAccount; +import org.hyperledger.besu.ethereum.bonsai.BonsaiValue; +import org.hyperledger.besu.ethereum.bonsai.storage.BonsaiPreImageProxy; +import org.hyperledger.besu.ethereum.bonsai.worldview.BonsaiWorldStateUpdateAccumulator; +import org.hyperledger.besu.ethereum.bonsai.worldview.BonsaiWorldView; + +import org.apache.tuweni.bytes.Bytes; + +public class BonsaiReferenceTestUpdateAccumulator extends BonsaiWorldStateUpdateAccumulator { + private final BonsaiPreImageProxy preImageProxy; + + public BonsaiReferenceTestUpdateAccumulator( + final BonsaiWorldView world, + final Consumer> accountPreloader, + final Consumer storagePreloader, + final BonsaiPreImageProxy preImageProxy) { + super(world, accountPreloader, storagePreloader); + this.preImageProxy = preImageProxy; + } + + @Override + protected Hash hashAndSavePreImage(final Bytes bytes) { + // by default do not save hash preImages + return preImageProxy.hashAndSavePreImage(bytes); + } +} diff --git a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestWorldState.java b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestWorldState.java new file mode 100644 index 00000000000..e40b3774a97 --- /dev/null +++ b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestWorldState.java @@ -0,0 +1,182 @@ +/* + * Copyright Hyperledger Besu Contributors. + * + * 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.referencetests; + +import org.hyperledger.besu.datatypes.Address; +import org.hyperledger.besu.datatypes.Hash; +import org.hyperledger.besu.ethereum.bonsai.cache.CachedMerkleTrieLoader; +import org.hyperledger.besu.ethereum.bonsai.storage.BonsaiPreImageProxy; +import org.hyperledger.besu.ethereum.bonsai.storage.BonsaiWorldStateKeyValueStorage; +import org.hyperledger.besu.ethereum.bonsai.trielog.TrieLogAddedEvent; +import org.hyperledger.besu.ethereum.bonsai.trielog.TrieLogFactoryImpl; +import org.hyperledger.besu.ethereum.bonsai.trielog.TrieLogManager; +import org.hyperledger.besu.ethereum.bonsai.worldview.BonsaiWorldState; +import org.hyperledger.besu.ethereum.bonsai.worldview.BonsaiWorldStateUpdateAccumulator; +import org.hyperledger.besu.ethereum.core.BlockHeader; +import org.hyperledger.besu.ethereum.core.InMemoryKeyValueStorageProvider; +import org.hyperledger.besu.evm.worldstate.WorldUpdater; +import org.hyperledger.besu.metrics.ObservableMetricsSystem; +import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem; +import org.hyperledger.besu.plugin.services.trielogs.TrieLog; +import org.hyperledger.besu.plugin.services.trielogs.TrieLogEvent; +import org.hyperledger.besu.plugin.services.trielogs.TrieLogFactory; +import org.hyperledger.besu.util.Subscribers; + +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Stream; + +import com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.tuweni.bytes.Bytes; +import org.apache.tuweni.bytes.Bytes32; + +public class BonsaiReferenceTestWorldState extends BonsaiWorldState + implements ReferenceTestWorldState { + + private final BonsaiReferenceTestWorldStateStorage refTestStorage; + private final BonsaiPreImageProxy preImageProxy; + + protected BonsaiReferenceTestWorldState( + final BonsaiReferenceTestWorldStateStorage worldStateStorage, + final CachedMerkleTrieLoader cachedMerkleTrieLoader, + final TrieLogManager trieLogManager, + final BonsaiPreImageProxy preImageProxy) { + super(worldStateStorage, cachedMerkleTrieLoader, trieLogManager); + this.refTestStorage = worldStateStorage; + this.preImageProxy = preImageProxy; + setAccumulator( + new BonsaiReferenceTestUpdateAccumulator( + this, + (addr, value) -> + cachedMerkleTrieLoader.preLoadAccount( + getWorldStateStorage(), worldStateRootHash, addr), + (addr, value) -> + cachedMerkleTrieLoader.preLoadStorageSlot(getWorldStateStorage(), addr, value), + preImageProxy)); + } + + @Override + public ReferenceTestWorldState copy() { + var layerCopy = new BonsaiReferenceTestWorldStateStorage(worldStateStorage, preImageProxy); + return new BonsaiReferenceTestWorldState( + layerCopy, cachedMerkleTrieLoader, trieLogManager, preImageProxy); + } + + @JsonCreator + public static BonsaiReferenceTestWorldState create( + final Map accounts) { + final ObservableMetricsSystem metricsSystem = new NoOpMetricsSystem(); + final CachedMerkleTrieLoader cachedMerkleTrieLoader = new CachedMerkleTrieLoader(metricsSystem); + final TrieLogManager trieLogManager = new NoOpTrieLogManager(); + final BonsaiPreImageProxy preImageProxy = + new BonsaiPreImageProxy.BonsaiReferenceTestPreImageProxy(); + + final BonsaiReferenceTestWorldStateStorage worldStateStorage = + new BonsaiReferenceTestWorldStateStorage( + new BonsaiWorldStateKeyValueStorage( + new InMemoryKeyValueStorageProvider(), metricsSystem), + preImageProxy); + + final BonsaiReferenceTestWorldState worldState = + new BonsaiReferenceTestWorldState( + worldStateStorage, cachedMerkleTrieLoader, trieLogManager, preImageProxy); + + final WorldUpdater updater = worldState.updater(); + for (final Map.Entry entry : accounts.entrySet()) { + ReferenceTestWorldState.insertAccount( + updater, Address.fromHexString(entry.getKey()), entry.getValue()); + } + updater.commit(); + return worldState; + } + + @Override + public Stream streamAccounts(final Bytes32 startKeyHash, final int limit) { + return this.refTestStorage.streamAccounts(this, startKeyHash, limit); + } + + static class NoOpTrieLogManager implements TrieLogManager { + private final Subscribers trieLogObservers = Subscribers.create(); + private final TrieLogFactory trieLogFactory = new TrieLogFactoryImpl(); + + @Override + public void saveTrieLog( + final BonsaiWorldStateUpdateAccumulator localUpdater, + final Hash forWorldStateRootHash, + final BlockHeader forBlockHeader, + final BonsaiWorldState forWorldState) { + // notify trie log added observers, synchronously + TrieLog trieLog = trieLogFactory.create(localUpdater, forBlockHeader); + trieLogObservers.forEach(o -> o.onTrieLogAdded(new TrieLogAddedEvent(trieLog))); + } + + @Override + public void addCachedLayer( + final BlockHeader blockHeader, + final Hash worldStateRootHash, + final BonsaiWorldState forWorldState) {} + + @Override + public boolean containWorldStateStorage(final Hash blockHash) { + return false; + } + + @Override + public Optional getWorldState(final Hash blockHash) { + return Optional.empty(); + } + + @Override + public Optional getNearestWorldState(final BlockHeader blockHeader) { + return Optional.empty(); + } + + @Override + public Optional getHeadWorldState( + final Function> hashBlockHeaderFunction) { + return Optional.empty(); + } + + @Override + public long getMaxLayersToLoad() { + return 0; + } + + @Override + public void reset() {} + + @Override + public Optional getTrieLogLayer(final Hash blockHash) { + return Optional.empty(); + } + + @Override + public synchronized long subscribe(final TrieLogEvent.TrieLogObserver sub) { + return trieLogObservers.subscribe(sub); + } + + @Override + public synchronized void unsubscribe(final long id) { + trieLogObservers.unsubscribe(id); + } + } + + @Override + protected Hash hashAndSavePreImage(final Bytes value) { + // by default do not save has preImages + return preImageProxy.hashAndSavePreImage(value); + } +} diff --git a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestWorldStateStorage.java b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestWorldStateStorage.java new file mode 100644 index 00000000000..b998bf7d25b --- /dev/null +++ b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/BonsaiReferenceTestWorldStateStorage.java @@ -0,0 +1,85 @@ +/* + * Copyright Hyperledger Besu Contributors. + * + * 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.referencetests; + +import org.hyperledger.besu.datatypes.Address; +import org.hyperledger.besu.datatypes.Hash; +import org.hyperledger.besu.ethereum.bonsai.BonsaiAccount; +import org.hyperledger.besu.ethereum.bonsai.storage.BonsaiPreImageProxy; +import org.hyperledger.besu.ethereum.bonsai.storage.BonsaiWorldStateKeyValueStorage; +import org.hyperledger.besu.ethereum.bonsai.storage.BonsaiWorldStateLayerStorage; +import org.hyperledger.besu.ethereum.bonsai.worldview.BonsaiWorldView; +import org.hyperledger.besu.evm.account.AccountStorageEntry; +import org.hyperledger.besu.evm.worldstate.WorldState; + +import java.util.Comparator; +import java.util.NavigableMap; +import java.util.Optional; +import java.util.TreeMap; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.tuweni.bytes.Bytes32; +import org.apache.tuweni.rlp.RLP; +import org.apache.tuweni.units.bigints.UInt256; + +public class BonsaiReferenceTestWorldStateStorage extends BonsaiWorldStateLayerStorage { + private final BonsaiPreImageProxy preImageProxy; + + public BonsaiReferenceTestWorldStateStorage( + final BonsaiWorldStateKeyValueStorage parent, final BonsaiPreImageProxy preImageProxy) { + super(parent); + this.preImageProxy = preImageProxy; + } + + @Override + public NavigableMap storageEntriesFrom( + final Hash addressHash, final Bytes32 startKeyHash, final int limit) { + return streamFlatStorages(addressHash, startKeyHash, UInt256.MAX_VALUE, limit) + .entrySet() + // map back to slot keys using preImage provider: + .stream() + .collect( + Collectors.toMap( + e -> e.getKey(), + e -> + AccountStorageEntry.create( + UInt256.fromBytes(RLP.decodeValue(e.getValue())), + Hash.wrap(e.getKey()), + preImageProxy.getStorageTrieKeyPreimage(e.getKey())), + (a, b) -> a, + TreeMap::new)); + } + + public Stream streamAccounts( + final BonsaiWorldView context, final Bytes32 startKeyHash, final int limit) { + return streamFlatAccounts(startKeyHash, UInt256.MAX_VALUE, limit) + .entrySet() + // map back to addresses using preImage provider: + .stream() + .map( + entry -> + preImageProxy + .getAccountTrieKeyPreimage(entry.getKey()) + .map( + address -> + new WorldState.StreamableAccount( + Optional.of(address), + BonsaiAccount.fromRLP(context, address, entry.getValue(), false)))) + .filter(Optional::isPresent) + .map(Optional::get) + .sorted(Comparator.comparing(account -> account.getAddress().orElse(Address.ZERO))); + } +} diff --git a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/DefaultReferenceTestWorldState.java b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/DefaultReferenceTestWorldState.java new file mode 100644 index 00000000000..ec0b058ba99 --- /dev/null +++ b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/DefaultReferenceTestWorldState.java @@ -0,0 +1,60 @@ +/* + * Copyright Hyperledger Besu Contributors. + * + * 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.referencetests; + +import org.hyperledger.besu.datatypes.Address; +import org.hyperledger.besu.ethereum.storage.keyvalue.WorldStateKeyValueStorage; +import org.hyperledger.besu.ethereum.storage.keyvalue.WorldStatePreimageKeyValueStorage; +import org.hyperledger.besu.ethereum.worldstate.DefaultMutableWorldState; +import org.hyperledger.besu.evm.worldstate.WorldState; +import org.hyperledger.besu.evm.worldstate.WorldUpdater; +import org.hyperledger.besu.services.kvstore.InMemoryKeyValueStorage; + +import java.util.Map; + +import com.fasterxml.jackson.annotation.JsonCreator; + +public class DefaultReferenceTestWorldState extends DefaultMutableWorldState + implements ReferenceTestWorldState { + + DefaultReferenceTestWorldState() { + super( + new WorldStateKeyValueStorage(new InMemoryKeyValueStorage()), + new WorldStatePreimageKeyValueStorage(new InMemoryKeyValueStorage())); + } + + public DefaultReferenceTestWorldState(final WorldState worldState) { + super(worldState); + } + + @Override + public ReferenceTestWorldState copy() { + return new DefaultReferenceTestWorldState(this); + } + + @JsonCreator + public static ReferenceTestWorldState create(final Map accounts) { + final ReferenceTestWorldState worldState = new DefaultReferenceTestWorldState(); + final WorldUpdater updater = worldState.updater(); + + for (final Map.Entry entry : accounts.entrySet()) { + ReferenceTestWorldState.insertAccount( + updater, Address.fromHexString(entry.getKey()), entry.getValue()); + } + + updater.commit(); + return worldState; + } +} diff --git a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/GeneralStateTestCaseEipSpec.java b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/GeneralStateTestCaseEipSpec.java index b396eff98f8..def8c14f17e 100644 --- a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/GeneralStateTestCaseEipSpec.java +++ b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/GeneralStateTestCaseEipSpec.java @@ -18,7 +18,6 @@ import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.ethereum.core.Transaction; -import org.hyperledger.besu.evm.worldstate.WorldState; import java.util.function.Supplier; @@ -37,7 +36,7 @@ public class GeneralStateTestCaseEipSpec { // is why this is a supplier: calling get() actually does the signing. private final Supplier transactionSupplier; - private final WorldState initialWorldState; + private final ReferenceTestWorldState initialWorldState; private final Hash expectedRootHash; @@ -54,7 +53,7 @@ public class GeneralStateTestCaseEipSpec { GeneralStateTestCaseEipSpec( final String fork, final Supplier transactionSupplier, - final WorldState initialWorldState, + final ReferenceTestWorldState initialWorldState, final Hash expectedRootHash, final Hash expectedLogsHash, final BlockHeader blockHeader, @@ -78,7 +77,7 @@ public String getFork() { return fork; } - public WorldState getInitialWorldState() { + public ReferenceTestWorldState getInitialWorldState() { return initialWorldState; } diff --git a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/GeneralStateTestCaseSpec.java b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/GeneralStateTestCaseSpec.java index 328922e1876..59f0a86fe09 100644 --- a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/GeneralStateTestCaseSpec.java +++ b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/GeneralStateTestCaseSpec.java @@ -17,7 +17,10 @@ import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.ethereum.core.BlockHeader; +import org.hyperledger.besu.ethereum.core.BlockHeaderBuilder; +import org.hyperledger.besu.ethereum.core.BlockHeaderFunctions; import org.hyperledger.besu.ethereum.core.Transaction; +import org.hyperledger.besu.ethereum.mainnet.MainnetBlockHeaderFunctions; import java.util.ArrayList; import java.util.LinkedHashMap; @@ -37,6 +40,7 @@ public class GeneralStateTestCaseSpec { private final Map> finalStateSpecs; + private static final BlockHeaderFunctions MAINNET_FUNCTIONS = new MainnetBlockHeaderFunctions(); @JsonCreator public GeneralStateTestCaseSpec( @@ -49,7 +53,7 @@ public GeneralStateTestCaseSpec( } private Map> generate( - final BlockHeader blockHeader, + final BlockHeader rawBlockHeader, final ReferenceTestWorldState initialWorldState, final Map> postSections, final StateTestVersionedTransaction versionedTransaction) { @@ -62,6 +66,11 @@ private Map> generate( final List post = entry.getValue(); final List specs = new ArrayList<>(post.size()); for (final PostSection p : post) { + final BlockHeader blockHeader = + BlockHeaderBuilder.fromHeader(rawBlockHeader) + .stateRoot(p.rootHash) + .blockHeaderFunctions(MAINNET_FUNCTIONS) + .buildBlockHeader(); final Supplier txSupplier = () -> versionedTransaction.get(p.indexes); specs.add( new GeneralStateTestCaseEipSpec( diff --git a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/ReferenceTestEnv.java b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/ReferenceTestEnv.java index 79356331fb6..5f7af60719a 100644 --- a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/ReferenceTestEnv.java +++ b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/ReferenceTestEnv.java @@ -104,6 +104,7 @@ public ReferenceTestEnv( @JsonProperty("currentBaseFee") final String baseFee, @JsonProperty("currentTimestamp") final String timestamp, @JsonProperty("currentRandom") final String random, + @JsonProperty("currentStateRoot") final String stateRoot, @JsonProperty("previousHash") final String previousHash, @JsonProperty("parentDifficulty") final String parentDifficulty, @JsonProperty("parentBaseFee") final String parentBaseFee, @@ -127,7 +128,7 @@ public ReferenceTestEnv( generateTestBlockHash(previousHash, number), Hash.EMPTY_LIST_HASH, // ommersHash Address.fromHexString(coinbase), - Hash.EMPTY, // stateRoot + Optional.ofNullable(stateRoot).map(Hash::fromHexString).orElse(Hash.EMPTY), // stateRoot Hash.EMPTY, // transactionsRoot Hash.EMPTY, // receiptsRoot new LogsBloomFilter(), diff --git a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/ReferenceTestWorldState.java b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/ReferenceTestWorldState.java index 659847c1211..825d7a2cd4b 100644 --- a/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/ReferenceTestWorldState.java +++ b/ethereum/referencetests/src/main/java/org/hyperledger/besu/ethereum/referencetests/ReferenceTestWorldState.java @@ -17,12 +17,9 @@ import org.hyperledger.besu.datatypes.Address; import org.hyperledger.besu.datatypes.Wei; -import org.hyperledger.besu.ethereum.storage.keyvalue.WorldStateKeyValueStorage; -import org.hyperledger.besu.ethereum.storage.keyvalue.WorldStatePreimageKeyValueStorage; -import org.hyperledger.besu.ethereum.worldstate.DefaultMutableWorldState; +import org.hyperledger.besu.ethereum.core.MutableWorldState; import org.hyperledger.besu.evm.account.MutableAccount; import org.hyperledger.besu.evm.worldstate.WorldUpdater; -import org.hyperledger.besu.services.kvstore.InMemoryKeyValueStorage; import java.util.HashMap; import java.util.Map; @@ -35,9 +32,9 @@ /** Represent a worldState for testing. */ @JsonIgnoreProperties(ignoreUnknown = true) -public class ReferenceTestWorldState extends DefaultMutableWorldState { +public interface ReferenceTestWorldState extends MutableWorldState { - public static class AccountMock { + class AccountMock { private final long nonce; private final Wei balance; private final Bytes code; @@ -90,22 +87,11 @@ static void insertAccount( } } - @JsonCreator - public static ReferenceTestWorldState create(final Map accounts) { - final ReferenceTestWorldState worldState = new ReferenceTestWorldState(); - final WorldUpdater updater = worldState.updater(); - - for (final Map.Entry entry : accounts.entrySet()) { - insertAccount(updater, Address.fromHexString(entry.getKey()), entry.getValue()); - } + ReferenceTestWorldState copy(); - updater.commit(); - return worldState; - } - - public ReferenceTestWorldState() { - super( - new WorldStateKeyValueStorage(new InMemoryKeyValueStorage()), - new WorldStatePreimageKeyValueStorage(new InMemoryKeyValueStorage())); + @JsonCreator + static ReferenceTestWorldState create(final Map accounts) { + // delegate to a Bonsai reference test world state: + return BonsaiReferenceTestWorldState.create(accounts); } } diff --git a/ethereum/referencetests/src/reference-test/java/org/hyperledger/besu/ethereum/vm/GeneralStateReferenceTestTools.java b/ethereum/referencetests/src/reference-test/java/org/hyperledger/besu/ethereum/vm/GeneralStateReferenceTestTools.java index e01e08c0414..83388fe2526 100644 --- a/ethereum/referencetests/src/reference-test/java/org/hyperledger/besu/ethereum/vm/GeneralStateReferenceTestTools.java +++ b/ethereum/referencetests/src/reference-test/java/org/hyperledger/besu/ethereum/vm/GeneralStateReferenceTestTools.java @@ -21,8 +21,10 @@ import org.hyperledger.besu.datatypes.Wei; import org.hyperledger.besu.ethereum.core.BlockHeader; import org.hyperledger.besu.ethereum.core.BlockHeaderBuilder; +import org.hyperledger.besu.ethereum.core.BlockHeaderFunctions; import org.hyperledger.besu.ethereum.core.MutableWorldState; import org.hyperledger.besu.ethereum.core.Transaction; +import org.hyperledger.besu.ethereum.mainnet.MainnetBlockHeaderFunctions; import org.hyperledger.besu.ethereum.mainnet.MainnetTransactionProcessor; import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec; import org.hyperledger.besu.ethereum.mainnet.TransactionValidationParams; @@ -31,6 +33,7 @@ import org.hyperledger.besu.ethereum.referencetests.GeneralStateTestCaseSpec; import org.hyperledger.besu.ethereum.referencetests.ReferenceTestBlockchain; import org.hyperledger.besu.ethereum.referencetests.ReferenceTestProtocolSchedules; +import org.hyperledger.besu.ethereum.referencetests.ReferenceTestWorldState; import org.hyperledger.besu.ethereum.rlp.RLP; import org.hyperledger.besu.ethereum.worldstate.DefaultMutableWorldState; import org.hyperledger.besu.evm.account.Account; @@ -123,7 +126,7 @@ public static Collection generateTestParametersForConfig(final String[ public static void executeTest(final GeneralStateTestCaseEipSpec spec) { final BlockHeader blockHeader = spec.getBlockHeader(); - final WorldState initialWorldState = spec.getInitialWorldState(); + final ReferenceTestWorldState initialWorldState = spec.getInitialWorldState(); final Transaction transaction = spec.getTransaction(); // Sometimes the tests ask us assemble an invalid transaction. If we have @@ -136,7 +139,7 @@ public static void executeTest(final GeneralStateTestCaseEipSpec spec) { return; } - final MutableWorldState worldState = new DefaultMutableWorldState(initialWorldState); + final MutableWorldState worldState = initialWorldState.copy(); // Several of the GeneralStateTests check if the transaction could potentially // consume more gas than is left for the block it's attempted to be included in. // This check is performed within the `BlockImporter` rather than inside the @@ -178,6 +181,7 @@ public static void executeTest(final GeneralStateTestCaseEipSpec spec) { worldStateUpdater.deleteAccount(coinbase.getAddress()); } worldStateUpdater.commit(); + worldState.persist(blockHeader); // Check the world state root hash. final Hash expectedRootHash = spec.getExpectedRootHash(); diff --git a/plugin-api/build.gradle b/plugin-api/build.gradle index aedc706bab1..e544419b8ab 100644 --- a/plugin-api/build.gradle +++ b/plugin-api/build.gradle @@ -69,7 +69,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 = 'fPOd/MnNB1PwfTV7HDTXc3oYRcoeUzMJrlzDUdg/HNk=' + knownHash = 'tpSnjt4HgqSiOTJhBbYdB0r1nFX4QZbicjfloI71Wf0=' } check.dependsOn('checkAPIChanges') diff --git a/plugin-api/src/main/java/org/hyperledger/besu/plugin/services/storage/KeyValueStorage.java b/plugin-api/src/main/java/org/hyperledger/besu/plugin/services/storage/KeyValueStorage.java index 6ccdb3b0a24..3d0e9e5efea 100644 --- a/plugin-api/src/main/java/org/hyperledger/besu/plugin/services/storage/KeyValueStorage.java +++ b/plugin-api/src/main/java/org/hyperledger/besu/plugin/services/storage/KeyValueStorage.java @@ -82,6 +82,17 @@ public interface KeyValueStorage extends Closeable { */ Stream> streamFromKey(final byte[] startKey); + /** + * Returns a stream of key-value pairs starting from the specified key, ending at the specified + * key. This method is used to retrieve a stream of data from the storage, starting from the given + * key. If no data is available from the specified key onwards, an empty stream is returned. + * + * @param startKey The key from which the stream should start. + * @param endKey The key at which the stream should stop. + * @return A stream of key-value pairs starting from the specified key. + */ + Stream> streamFromKey(final byte[] startKey, final byte[] endKey); + /** * Returns a stream of all keys. * diff --git a/plugin-api/src/main/java/org/hyperledger/besu/plugin/services/storage/SegmentedKeyValueStorage.java b/plugin-api/src/main/java/org/hyperledger/besu/plugin/services/storage/SegmentedKeyValueStorage.java index 21e6ecca6f6..55d04d470c4 100644 --- a/plugin-api/src/main/java/org/hyperledger/besu/plugin/services/storage/SegmentedKeyValueStorage.java +++ b/plugin-api/src/main/java/org/hyperledger/besu/plugin/services/storage/SegmentedKeyValueStorage.java @@ -78,6 +78,19 @@ default boolean containsKey(final SegmentIdentifier segment, final byte[] key) Stream> streamFromKey( final SegmentIdentifier segmentIdentifier, final byte[] startKey); + /** + * Returns a stream of key-value pairs starting from the specified key, ending at the specified + * key. This method is used to retrieve a stream of data from the storage, starting from the given + * key. If no data is available from the specified key onwards, an empty stream is returned. + * + * @param segmentIdentifier The segment identifier whose keys we want to stream. + * @param startKey The key from which the stream should start. + * @param endKey The key at which the stream should stop. + * @return A stream of key-value pairs starting from the specified key. + */ + Stream> streamFromKey( + final SegmentIdentifier segmentIdentifier, final byte[] startKey, final byte[] endKey); + /** * Stream keys. * diff --git a/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBColumnarKeyValueSnapshot.java b/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBColumnarKeyValueSnapshot.java index d51bdb310b8..6a123069001 100644 --- a/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBColumnarKeyValueSnapshot.java +++ b/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBColumnarKeyValueSnapshot.java @@ -33,7 +33,6 @@ import java.util.stream.Stream; import org.apache.commons.lang3.tuple.Pair; -import org.apache.tuweni.bytes.Bytes; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.OptimisticTransactionDB; import org.slf4j.Logger; @@ -83,7 +82,13 @@ public Stream> stream(final SegmentIdentifier segment) { @Override public Stream> streamFromKey( final SegmentIdentifier segment, final byte[] startKey) { - return stream(segment).filter(e -> Bytes.wrap(startKey).compareTo(Bytes.wrap(e.getKey())) <= 0); + return snapTx.streamFromKey(segment, startKey); + } + + @Override + public Stream> streamFromKey( + final SegmentIdentifier segment, final byte[] startKey, final byte[] endKey) { + return snapTx.streamFromKey(segment, startKey, endKey); } @Override diff --git a/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBColumnarKeyValueStorage.java b/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBColumnarKeyValueStorage.java index 3d98be37523..9462f9bcb14 100644 --- a/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBColumnarKeyValueStorage.java +++ b/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBColumnarKeyValueStorage.java @@ -333,6 +333,17 @@ public Stream> streamFromKey( return RocksDbIterator.create(rocksIterator).toStream(); } + @Override + public Stream> streamFromKey( + final SegmentIdentifier segmentIdentifier, final byte[] startKey, final byte[] endKey) { + final Bytes endKeyBytes = Bytes.wrap(endKey); + final RocksIterator rocksIterator = getDB().newIterator(safeColumnHandle(segmentIdentifier)); + rocksIterator.seek(startKey); + return RocksDbIterator.create(rocksIterator) + .toStream() + .takeWhile(e -> endKeyBytes.compareTo(Bytes.wrap(e.getKey())) >= 0); + } + @Override public Stream streamKeys(final SegmentIdentifier segmentIdentifier) { final RocksIterator rocksIterator = getDB().newIterator(safeColumnHandle(segmentIdentifier)); diff --git a/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBSnapshotTransaction.java b/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBSnapshotTransaction.java index c1c404628d9..0897493eb19 100644 --- a/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBSnapshotTransaction.java +++ b/plugins/rocksdb/src/main/java/org/hyperledger/besu/plugin/services/storage/rocksdb/segmented/RocksDBSnapshotTransaction.java @@ -28,6 +28,7 @@ import java.util.stream.Stream; import org.apache.commons.lang3.tuple.Pair; +import org.apache.tuweni.bytes.Bytes; import org.rocksdb.ColumnFamilyHandle; import org.rocksdb.OptimisticTransactionDB; import org.rocksdb.ReadOptions; @@ -165,6 +166,48 @@ public Stream streamKeys(final SegmentIdentifier segmentId) { rocksIterator.seekToFirst(); return RocksDbIterator.create(rocksIterator).toStreamKeys(); } + /** + * Returns a stream of key-value pairs starting from the specified key. This method is used to + * retrieve a stream of data reading through the transaction, starting from the given key. If no + * data is available from the specified key onwards, an empty stream is returned. + * + * @param segment The segment identifier whose keys we want to stream. + * @param startKey The key from which the stream should start. + * @return A stream of key-value pairs starting from the specified key. + */ + public Stream> streamFromKey( + final SegmentIdentifier segment, final byte[] startKey) { + throwIfClosed(); + + final RocksIterator rocksIterator = + db.newIterator(columnFamilyMapper.apply(segment), readOptions); + rocksIterator.seek(startKey); + return RocksDbIterator.create(rocksIterator).toStream(); + } + + /** + * Returns a stream of key-value pairs starting from the specified key, ending at the specified + * key. This method is used to retrieve a stream of data reading through the transaction, starting + * from the given key. If no data is available from the specified key onwards, an empty stream is + * returned. + * + * @param segment The segment identifier whose keys we want to stream. + * @param startKey The key from which the stream should start. + * @param endKey The key at which the stream should stop. + * @return A stream of key-value pairs starting from the specified key. + */ + public Stream> streamFromKey( + final SegmentIdentifier segment, final byte[] startKey, final byte[] endKey) { + throwIfClosed(); + final Bytes endKeyBytes = Bytes.wrap(endKey); + + final RocksIterator rocksIterator = + db.newIterator(columnFamilyMapper.apply(segment), readOptions); + rocksIterator.seek(startKey); + return RocksDbIterator.create(rocksIterator) + .toStream() + .takeWhile(e -> endKeyBytes.compareTo(Bytes.wrap(e.getKey())) >= 0); + } @Override public void commit() throws StorageException { diff --git a/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/LayeredKeyValueStorage.java b/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/LayeredKeyValueStorage.java index 9a028becdf8..ffc57530682 100644 --- a/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/LayeredKeyValueStorage.java +++ b/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/LayeredKeyValueStorage.java @@ -122,8 +122,18 @@ public Stream> stream(final SegmentIdentifier segmentId) { @Override public Stream> streamFromKey( final SegmentIdentifier segmentId, final byte[] startKey) { + final Bytes startKeyBytes = Bytes.wrap(startKey); + return stream(segmentId).filter(e -> startKeyBytes.compareTo(Bytes.wrap(e.getKey())) <= 0); + } + + @Override + public Stream> streamFromKey( + final SegmentIdentifier segmentId, final byte[] startKey, final byte[] endKey) { + final Bytes startKeyBytes = Bytes.wrap(startKey); + final Bytes endKeyBytes = Bytes.wrap(endKey); return stream(segmentId) - .filter(e -> Bytes.wrap(startKey).compareTo(Bytes.wrap(e.getKey())) <= 0); + .filter(e -> startKeyBytes.compareTo(Bytes.wrap(e.getKey())) <= 0) + .filter(e -> endKeyBytes.compareTo(Bytes.wrap(e.getKey())) >= 0); } @Override diff --git a/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/LimitedInMemoryKeyValueStorage.java b/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/LimitedInMemoryKeyValueStorage.java index 7ab04ec26e8..aa52e49284f 100644 --- a/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/LimitedInMemoryKeyValueStorage.java +++ b/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/LimitedInMemoryKeyValueStorage.java @@ -118,6 +118,13 @@ public Stream> streamFromKey(final byte[] startKey) { return stream().filter(e -> Bytes.wrap(startKey).compareTo(Bytes.wrap(e.getKey())) <= 0); } + @Override + public Stream> streamFromKey(final byte[] startKey, final byte[] endKey) { + return stream() + .filter(e -> Bytes.wrap(startKey).compareTo(Bytes.wrap(e.getKey())) <= 0) + .takeWhile(e -> Bytes.wrap(endKey).compareTo(Bytes.wrap(e.getKey())) >= 0); + } + @Override public Stream streamKeys() { final Lock lock = rwLock.readLock(); diff --git a/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/SegmentedInMemoryKeyValueStorage.java b/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/SegmentedInMemoryKeyValueStorage.java index adc9a4c71dc..66d70bac4dc 100644 --- a/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/SegmentedInMemoryKeyValueStorage.java +++ b/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/SegmentedInMemoryKeyValueStorage.java @@ -149,8 +149,19 @@ public Stream> stream(final SegmentIdentifier segmentIdenti @Override public Stream> streamFromKey( final SegmentIdentifier segmentIdentifier, final byte[] startKey) { + final Bytes startKeyBytes = Bytes.wrap(startKey); return stream(segmentIdentifier) - .filter(e -> Bytes.wrap(startKey).compareTo(Bytes.wrap(e.getKey())) <= 0); + .filter(e -> startKeyBytes.compareTo(Bytes.wrap(e.getKey())) <= 0); + } + + @Override + public Stream> streamFromKey( + final SegmentIdentifier segmentIdentifier, final byte[] startKey, final byte[] endKey) { + final Bytes startKeyHash = Bytes.wrap(startKey); + final Bytes endKeyHash = Bytes.wrap(endKey); + return stream(segmentIdentifier) + .filter(e -> startKeyHash.compareTo(Bytes.wrap(e.getKey())) <= 0) + .filter(e -> endKeyHash.compareTo(Bytes.wrap(e.getKey())) >= 0); } @Override diff --git a/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/SegmentedKeyValueStorageAdapter.java b/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/SegmentedKeyValueStorageAdapter.java index 81a492624f2..a02e9284fa6 100644 --- a/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/SegmentedKeyValueStorageAdapter.java +++ b/services/kvstore/src/main/java/org/hyperledger/besu/services/kvstore/SegmentedKeyValueStorageAdapter.java @@ -88,8 +88,14 @@ public Stream> stream() { } @Override - public Stream> streamFromKey(final byte[] startKey) throws StorageException { - return storage.streamFromKey(segmentIdentifier, startKey); + public Stream> streamFromKey(final byte[] startKeyHash) + throws StorageException { + return storage.streamFromKey(segmentIdentifier, startKeyHash); + } + + @Override + public Stream> streamFromKey(final byte[] startKey, final byte[] endKey) { + return storage.streamFromKey(segmentIdentifier, startKey, endKey); } @Override