From 064409b92c0e7ac2e7eb13a8196b0362c0ec0396 Mon Sep 17 00:00:00 2001 From: Adrian Sutton Date: Wed, 8 May 2019 13:56:15 +1000 Subject: [PATCH 01/53] Mark Sweep Pruner - first pass --- .../ethereum/blockcreation/BlockMiner.java | 5 + .../ethereum/storage/StorageProvider.java | 3 + .../keyvalue/KeyValueStorageProvider.java | 1 + .../worldstate/DefaultMutableWorldState.java | 4 +- .../ethereum/worldstate/MarkSweepPruner.java | 185 ++++++++++++++++++ .../pantheon/ethereum/worldstate/Pruner.java | 122 ++++++++++++ .../core/InMemoryStorageProvider.java | 6 + .../worldstate/MarkSweepPrunerTest.java | 65 ++++++ .../eth/sync/DefaultSynchronizer.java | 5 + .../worldstate/WorldStateDownloaderTest.java | 3 +- .../ethereum/trie/AllNodesVisitor.java | 46 +++++ .../ethereum/trie/MerklePatriciaTrie.java | 3 + .../trie/SimpleMerklePatriciaTrie.java | 6 + .../trie/StoredMerklePatriciaTrie.java | 30 ++- .../pantheon/ethereum/trie/StoredNode.java | 7 +- .../ethereum/trie/StoredNodeFactory.java | 9 +- .../ethereum/trie/TrieNodeDecoder.java | 5 +- .../trie/StoredMerklePatriciaTrieTest.java | 24 +-- .../ethereum/trie/TrieNodeDecoderTest.java | 12 +- .../metrics/PantheonMetricCategory.java | 1 + .../controller/PantheonControllerBuilder.java | 13 ++ .../kvstore/RocksDbConfiguration.java | 2 +- .../kvstore/AbstractKeyValueStorageTest.java | 15 ++ 23 files changed, 541 insertions(+), 31 deletions(-) create mode 100644 ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java create mode 100644 ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java create mode 100644 ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java create mode 100644 ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/AllNodesVisitor.java diff --git a/ethereum/blockcreation/src/main/java/tech/pegasys/pantheon/ethereum/blockcreation/BlockMiner.java b/ethereum/blockcreation/src/main/java/tech/pegasys/pantheon/ethereum/blockcreation/BlockMiner.java index c16861f9fc..6fee690064 100644 --- a/ethereum/blockcreation/src/main/java/tech/pegasys/pantheon/ethereum/blockcreation/BlockMiner.java +++ b/ethereum/blockcreation/src/main/java/tech/pegasys/pantheon/ethereum/blockcreation/BlockMiner.java @@ -80,6 +80,11 @@ public void run() { Thread.currentThread().interrupt(); } catch (final Exception ex) { LOG.error("Block mining threw an unhandled exception.", ex); + try { + Thread.sleep(TimeUnit.SECONDS.toMillis(5)); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } } } } diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/storage/StorageProvider.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/storage/StorageProvider.java index 34232afa13..a119a42e45 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/storage/StorageProvider.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/storage/StorageProvider.java @@ -17,6 +17,7 @@ import tech.pegasys.pantheon.ethereum.privacy.PrivateStateStorage; import tech.pegasys.pantheon.ethereum.privacy.PrivateTransactionStorage; import tech.pegasys.pantheon.ethereum.worldstate.WorldStateStorage; +import tech.pegasys.pantheon.services.kvstore.KeyValueStorage; import java.io.Closeable; @@ -29,4 +30,6 @@ public interface StorageProvider extends Closeable { PrivateTransactionStorage createPrivateTransactionStorage(); PrivateStateStorage createPrivateStateStorage(); + + KeyValueStorage createPruningStorage(); } diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/storage/keyvalue/KeyValueStorageProvider.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/storage/keyvalue/KeyValueStorageProvider.java index aafa31387a..7dc466c7b8 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/storage/keyvalue/KeyValueStorageProvider.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/storage/keyvalue/KeyValueStorageProvider.java @@ -71,6 +71,7 @@ public PrivateStateStorage createPrivateStateStorage() { return new PrivateStateKeyValueStorage(privateStateStorage); } + @Override public KeyValueStorage createPruningStorage() { return pruningStorage; } diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/DefaultMutableWorldState.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/DefaultMutableWorldState.java index df77dfb3e4..e009d9aea7 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/DefaultMutableWorldState.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/DefaultMutableWorldState.java @@ -73,12 +73,12 @@ public DefaultMutableWorldState(final WorldState worldState) { private MerklePatriciaTrie newAccountStateTrie(final Bytes32 rootHash) { return new StoredMerklePatriciaTrie<>( - worldStateStorage::getAccountStateTrieNode, rootHash, b -> b, b -> b); + worldStateStorage::getAccountStateTrieNode, rootHash, b -> b, b -> b, false); } private MerklePatriciaTrie newAccountStorageTrie(final Bytes32 rootHash) { return new StoredMerklePatriciaTrie<>( - worldStateStorage::getAccountStorageTrieNode, rootHash, b -> b, b -> b); + worldStateStorage::getAccountStorageTrieNode, rootHash, b -> b, b -> b, false); } @Override diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java new file mode 100644 index 0000000000..2a9ed547b0 --- /dev/null +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java @@ -0,0 +1,185 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * 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. + */ +package tech.pegasys.pantheon.ethereum.worldstate; + +import tech.pegasys.pantheon.ethereum.core.Hash; +import tech.pegasys.pantheon.ethereum.rlp.RLP; +import tech.pegasys.pantheon.ethereum.trie.MerklePatriciaTrie; +import tech.pegasys.pantheon.ethereum.trie.StoredMerklePatriciaTrie; +import tech.pegasys.pantheon.metrics.Counter; +import tech.pegasys.pantheon.metrics.MetricsSystem; +import tech.pegasys.pantheon.metrics.PantheonMetricCategory; +import tech.pegasys.pantheon.services.kvstore.KeyValueStorage; +import tech.pegasys.pantheon.services.kvstore.KeyValueStorage.Transaction; +import tech.pegasys.pantheon.util.bytes.Bytes32; +import tech.pegasys.pantheon.util.bytes.BytesValue; + +import java.util.Collection; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Function; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +public class MarkSweepPruner { + private static final Logger LOG = LogManager.getLogger(); + private static final BytesValue IN_USE = BytesValue.of(1); + private static final int MARKS_PER_TRANSACTION = 1000; + private final WorldStateStorage worldStateStorage; + private final KeyValueStorage markStorage; + private final Counter markedNodesCounter; + private final Counter markOperationCounter; + private final Counter sweepOperationCounter; + private final Counter sweptNodesCounter; + private long nodeAddedListenerId; + private final ReentrantLock markLock = new ReentrantLock(true); + private final Set pendingMarks = Collections.newSetFromMap(new ConcurrentHashMap<>()); + + public MarkSweepPruner( + final WorldStateStorage worldStateStorage, + final KeyValueStorage markStorage, + final MetricsSystem metricsSystem) { + this.worldStateStorage = worldStateStorage; + this.markStorage = markStorage; + + markedNodesCounter = + metricsSystem.createCounter( + PantheonMetricCategory.PRUNER, + "marked_nodes_total", + "Total number of nodes marked as in use"); + markOperationCounter = + metricsSystem.createCounter( + PantheonMetricCategory.PRUNER, + "mark_operations_total", + "Total number of mark operations performed"); + + sweptNodesCounter = + metricsSystem.createCounter( + PantheonMetricCategory.PRUNER, + "swept_nodes_total", + "Total number of unused nodes removed"); + sweepOperationCounter = + metricsSystem.createCounter( + PantheonMetricCategory.PRUNER, + "sweep_operations_total", + "Total number of sweep operations performed"); + } + + public void prepare() { + worldStateStorage.removeNodeAddedListener(nodeAddedListenerId); // Just in case. + nodeAddedListenerId = worldStateStorage.addNodeAddedListener(this::markNewNodes); + } + + // Note chainHeadStateRoot must be the state root of the current chain head. + // We can delay the actual sweep until a certain number of blocks in the future if we want to + // have a certain number of block history available. + // TODO: Need to ensure we only start marking when new world states aren't being persisted + // Once we add our node added listener persisting world states can continue but we can't start + // half way through persisting a world state. + public void mark(final Hash rootHash) { + markOperationCounter.inc(); + markStorage.clear(); + createStateTrie(rootHash) + .visitAll( + node -> { + if (Thread.interrupted()) { + // TODO: Probably need a better abort process than this.... + throw new RuntimeException("Interrupted while marking"); + } + markNode(node.getHash()); + node.getValue().ifPresent(this::processAccountState); + }); + LOG.info("Completed marking used nodes for pruning"); + } + + public void sweep() { + flushPendingMarks(); + sweepOperationCounter.inc(); + LOG.info("Sweeping unused nodes"); + final long prunedNodeCount = + worldStateStorage.prune(key -> pendingMarks.contains(key) || markStorage.containsKey(key)); + sweptNodesCounter.inc(prunedNodeCount); + worldStateStorage.removeNodeAddedListener(nodeAddedListenerId); + markStorage.clear(); + LOG.info("Completed sweeping unused nodes"); + } + + private MerklePatriciaTrie createStateTrie(final Bytes32 rootHash) { + return new StoredMerklePatriciaTrie<>( + worldStateStorage::getAccountStateTrieNode, + rootHash, + Function.identity(), + Function.identity(), + true); + } + + private MerklePatriciaTrie createStorageTrie(final Bytes32 rootHash) { + return new StoredMerklePatriciaTrie<>( + worldStateStorage::getAccountStorageTrieNode, + rootHash, + Function.identity(), + Function.identity(), + true); + } + + private void processAccountState(final BytesValue value) { + final StateTrieAccountValue accountValue = StateTrieAccountValue.readFrom(RLP.input(value)); + markNode(accountValue.getCodeHash()); + + createStorageTrie(accountValue.getStorageRoot()) + .visitAll(storageNode -> markNode(storageNode.getHash())); + } + + private void markNode(final Bytes32 hash) { + markedNodesCounter.inc(); + markLock.lock(); + try { + pendingMarks.add(hash); + maybeFlushPendingMarks(); + } finally { + markLock.unlock(); + } + } + + private void maybeFlushPendingMarks() { + if (pendingMarks.size() > MARKS_PER_TRANSACTION) { + flushPendingMarks(); + } + } + + void flushPendingMarks() { + markLock.lock(); + try { + final Transaction transaction = markStorage.startTransaction(); + pendingMarks.forEach(node -> transaction.put(node, IN_USE)); + transaction.commit(); + pendingMarks.clear(); + } finally { + markLock.unlock(); + } + } + + private void markNewNodes(final Collection nodeHashes) { + markedNodesCounter.inc(nodeHashes.size()); + markLock.lock(); + try { + pendingMarks.addAll(nodeHashes); + maybeFlushPendingMarks(); + } finally { + markLock.unlock(); + } + } +} diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java new file mode 100644 index 0000000000..e3ea7be675 --- /dev/null +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -0,0 +1,122 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * 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. + */ +package tech.pegasys.pantheon.ethereum.worldstate; + +import tech.pegasys.pantheon.ethereum.ProtocolContext; +import tech.pegasys.pantheon.ethereum.chain.BlockAddedEvent; +import tech.pegasys.pantheon.ethereum.core.BlockHeader; +import tech.pegasys.pantheon.ethereum.core.Hash; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicReference; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +public class Pruner { + private static final Logger LOG = LogManager.getLogger(); + private final MarkSweepPruner pruningStrategy; + private final ProtocolContext protocolContext; + private final ExecutorService executor; + private final long retentionPeriodInBlocks; + private final AtomicReference state = new AtomicReference<>(State.IDLE); + private volatile long markedBlockNumber = 0; + + public Pruner( + final MarkSweepPruner pruningStrategy, + final ProtocolContext protocolContext, + final long retentionPeriodInBlocks) { + this.pruningStrategy = pruningStrategy; + this.protocolContext = protocolContext; + this.executor = + Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setPriority(Thread.MIN_PRIORITY) + .setNameFormat("StatePruning-%d") + .build()); + this.retentionPeriodInBlocks = retentionPeriodInBlocks; + } + + public void start() { + // TODO: Don't attempt pruning while a fast sync is in progress. + protocolContext.getBlockchain().observeBlockAdded((event, blockchain) -> handleNewBlock(event)); + } + + public void stop() { + executor.shutdownNow(); + } + + private void handleNewBlock(final BlockAddedEvent event) { + if (!event.isNewCanonicalHead()) { + return; + } + final BlockHeader header = event.getBlock().getHeader(); + // Only kick off pruning every million blocks. + if (state.compareAndSet(State.IDLE, State.MARKING)) { + /* TODO: We don't currently handle re-orgs. + Will need to: + 1. start listening for new nodes at this point + 2. then after a few blocks mark the world state of the block when we started listening (now a few blocks back) + 3. check the block we marked is still on the canonical chain when we begin sweeping and abort that round of pruning if not + */ + + mark(header); + } else if (header.getNumber() > markedBlockNumber + retentionPeriodInBlocks + && state.compareAndSet(State.MARKING_COMPLETE, State.SWEEPING)) { + sweep(); + } + } + + private void mark(final BlockHeader header) { + markedBlockNumber = header.getNumber(); + final Hash stateRoot = header.getStateRoot(); + LOG.info( + "Begin marking used nodes for pruning. Block number: {} State root: {}", + markedBlockNumber, + stateRoot); + pruningStrategy.prepare(); + execute( + () -> { + pruningStrategy.mark(stateRoot); + state.compareAndSet(State.MARKING, State.MARKING_COMPLETE); + }); + } + + private void sweep() { + execute( + () -> { + pruningStrategy.sweep(); + state.compareAndSet(State.SWEEPING, State.IDLE); + }); + } + + private void execute(final Runnable action) { + try { + executor.execute(action); + } catch (final Throwable t) { + LOG.error("Pruning failed", t); + state.set(State.IDLE); + System.exit(1); + } + } + + private enum State { + IDLE, + MARKING, + MARKING_COMPLETE, + SWEEPING + } +} diff --git a/ethereum/core/src/test-support/java/tech/pegasys/pantheon/ethereum/core/InMemoryStorageProvider.java b/ethereum/core/src/test-support/java/tech/pegasys/pantheon/ethereum/core/InMemoryStorageProvider.java index 51a8670c1e..eabde8e624 100644 --- a/ethereum/core/src/test-support/java/tech/pegasys/pantheon/ethereum/core/InMemoryStorageProvider.java +++ b/ethereum/core/src/test-support/java/tech/pegasys/pantheon/ethereum/core/InMemoryStorageProvider.java @@ -29,6 +29,7 @@ import tech.pegasys.pantheon.ethereum.worldstate.WorldStateStorage; import tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem; import tech.pegasys.pantheon.services.kvstore.InMemoryKeyValueStorage; +import tech.pegasys.pantheon.services.kvstore.KeyValueStorage; public class InMemoryStorageProvider implements StorageProvider { @@ -70,6 +71,11 @@ public PrivateStateStorage createPrivateStateStorage() { return new PrivateStateKeyValueStorage(new InMemoryKeyValueStorage()); } + @Override + public KeyValueStorage createPruningStorage() { + return new InMemoryKeyValueStorage(); + } + @Override public void close() {} } diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java new file mode 100644 index 0000000000..80def9016d --- /dev/null +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java @@ -0,0 +1,65 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * 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. + */ +package tech.pegasys.pantheon.ethereum.worldstate; + +import static org.assertj.core.api.Assertions.assertThat; + +import tech.pegasys.pantheon.ethereum.core.BlockDataGenerator; +import tech.pegasys.pantheon.ethereum.core.Hash; +import tech.pegasys.pantheon.ethereum.core.MutableWorldState; +import tech.pegasys.pantheon.ethereum.storage.keyvalue.WorldStateKeyValueStorage; +import tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem; +import tech.pegasys.pantheon.services.kvstore.InMemoryKeyValueStorage; +import tech.pegasys.pantheon.util.bytes.BytesValue; + +import java.util.HashSet; +import java.util.Set; + +import org.junit.Test; + +public class MarkSweepPrunerTest { + + private final BlockDataGenerator gen = new BlockDataGenerator(); + private final NoOpMetricsSystem metricsSystem = new NoOpMetricsSystem(); + + @Test + public void shouldMarkAllNodesInCurrentWorldState() { + + // Setup "remote" state + final InMemoryKeyValueStorage markStorage = new InMemoryKeyValueStorage(); + final InMemoryKeyValueStorage stateStorage = new InMemoryKeyValueStorage(); + final WorldStateStorage worldStateStorage = new WorldStateKeyValueStorage(stateStorage); + final WorldStateArchive worldStateArchive = new WorldStateArchive(worldStateStorage); + final MutableWorldState worldState = worldStateArchive.getMutable(); + + // Generate accounts and save corresponding state root + gen.createRandomContractAccountsWithNonEmptyStorage(worldState, 20); + final Hash stateRoot = worldState.rootHash(); + + final MarkSweepPruner pruner = + new MarkSweepPruner(worldStateStorage, markStorage, metricsSystem); + pruner.mark(stateRoot); + pruner.flushPendingMarks(); + + final Set keysToKeep = new HashSet<>(stateStorage.keySet()); + assertThat(markStorage.keySet()).containsExactlyInAnyOrderElementsOf(keysToKeep); + + // Generate some more nodes from a world state we didn't mark + gen.createRandomContractAccountsWithNonEmptyStorage(worldStateArchive.getMutable(), 10); + assertThat(stateStorage.keySet()).hasSizeGreaterThan(keysToKeep.size()); + + // All those new nodes should be removed when we sweep + pruner.sweep(); + assertThat(stateStorage.keySet()).containsExactlyInAnyOrderElementsOf(keysToKeep); + } +} diff --git a/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java b/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java index a6f8d06888..6191410f28 100644 --- a/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java +++ b/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java @@ -26,6 +26,7 @@ import tech.pegasys.pantheon.ethereum.eth.sync.state.PendingBlocks; import tech.pegasys.pantheon.ethereum.eth.sync.state.SyncState; import tech.pegasys.pantheon.ethereum.mainnet.ProtocolSchedule; +import tech.pegasys.pantheon.ethereum.worldstate.Pruner; import tech.pegasys.pantheon.ethereum.worldstate.WorldStateStorage; import tech.pegasys.pantheon.metrics.MetricsSystem; import tech.pegasys.pantheon.metrics.PantheonMetricCategory; @@ -44,6 +45,7 @@ public class DefaultSynchronizer implements Synchronizer { private static final Logger LOG = LogManager.getLogger(); + private final Pruner pruner; private final SyncState syncState; private final AtomicBoolean running = new AtomicBoolean(false); private final Subscribers syncStatusListeners = Subscribers.create(); @@ -57,11 +59,13 @@ public DefaultSynchronizer( final ProtocolContext protocolContext, final WorldStateStorage worldStateStorage, final BlockBroadcaster blockBroadcaster, + final Pruner pruner, final EthContext ethContext, final SyncState syncState, final Path dataDirectory, final Clock clock, final MetricsSystem metricsSystem) { + this.pruner = pruner; this.syncState = syncState; ChainHeadTracker.trackChainHeadForPeers( @@ -164,6 +168,7 @@ private void handleFastSyncResult(final FastSyncState result, final Throwable er private void startFullSync() { fullSyncDownloader.start(); + pruner.start(); } @Override diff --git a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java index b80dfdb78f..10d367bc3c 100644 --- a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java +++ b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java @@ -534,7 +534,8 @@ public void doesNotRequestKnownStorageTrieNodesFromNetwork() { remoteStorage::getNodeData, remoteWorldState.rootHash(), Function.identity(), - Function.identity()) + Function.identity(), + true) .entriesFrom(Bytes32.ZERO, 5).values().stream() .map(RLP::input) .map(StateTrieAccountValue::readFrom) diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/AllNodesVisitor.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/AllNodesVisitor.java new file mode 100644 index 0000000000..dba9b9048b --- /dev/null +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/AllNodesVisitor.java @@ -0,0 +1,46 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * 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. + */ +package tech.pegasys.pantheon.ethereum.trie; + +import java.util.function.Consumer; + +public class AllNodesVisitor implements NodeVisitor { + + private final Consumer> handler; + + public AllNodesVisitor(final Consumer> handler) { + this.handler = handler; + } + + @Override + public void visit(final ExtensionNode extensionNode) { + handler.accept(extensionNode); + extensionNode.getChild().accept(this); + } + + @Override + public void visit(final BranchNode branchNode) { + handler.accept(branchNode); + for (byte i = 0; i < BranchNode.RADIX; i++) { + branchNode.child(i).accept(this); + } + } + + @Override + public void visit(final LeafNode leafNode) { + handler.accept(leafNode); + } + + @Override + public void visit(final NullNode nullNode) {} +} diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/MerklePatriciaTrie.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/MerklePatriciaTrie.java index 1b869ada5e..1cac37b452 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/MerklePatriciaTrie.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/MerklePatriciaTrie.java @@ -20,6 +20,7 @@ import java.util.Map; import java.util.Optional; +import java.util.function.Consumer; /** An Merkle Patricial Trie. */ public interface MerklePatriciaTrie { @@ -74,4 +75,6 @@ public interface MerklePatriciaTrie { * @return the requested storage entries as a map of key hash to value. */ Map entriesFrom(Bytes32 startKeyHash, int limit); + + void visitAll(Consumer> visitor); } diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/SimpleMerklePatriciaTrie.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/SimpleMerklePatriciaTrie.java index dde962259b..25fda9395c 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/SimpleMerklePatriciaTrie.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/SimpleMerklePatriciaTrie.java @@ -20,6 +20,7 @@ import java.util.Map; import java.util.Optional; +import java.util.function.Consumer; import java.util.function.Function; /** @@ -82,4 +83,9 @@ public void commit(final NodeUpdater nodeUpdater) { public Map entriesFrom(final Bytes32 startKeyHash, final int limit) { return StorageEntriesCollector.collectEntries(root, startKeyHash, limit); } + + @Override + public void visitAll(final Consumer> visitor) { + root.accept(new AllNodesVisitor<>(visitor)); + } } diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java index 63f6f3648c..a45cf83f9c 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java @@ -20,6 +20,7 @@ import java.util.Map; import java.util.Optional; +import java.util.function.Consumer; import java.util.function.Function; /** @@ -31,6 +32,7 @@ public class StoredMerklePatriciaTrie implements Merkle private final GetVisitor getVisitor = new GetVisitor<>(); private final RemoveVisitor removeVisitor = new RemoveVisitor<>(); private final StoredNodeFactory nodeFactory; + private final boolean singleUseNodes; private Node root; @@ -40,12 +42,19 @@ public class StoredMerklePatriciaTrie implements Merkle * @param nodeLoader The {@link NodeLoader} to retrieve node data from. * @param valueSerializer A function for serializing values to bytes. * @param valueDeserializer A function for deserializing values from bytes. + * @param singleUseNodes whether nodes should unload data after being visited */ public StoredMerklePatriciaTrie( final NodeLoader nodeLoader, final Function valueSerializer, - final Function valueDeserializer) { - this(nodeLoader, MerklePatriciaTrie.EMPTY_TRIE_NODE_HASH, valueSerializer, valueDeserializer); + final Function valueDeserializer, + final boolean singleUseNodes) { + this( + nodeLoader, + MerklePatriciaTrie.EMPTY_TRIE_NODE_HASH, + valueSerializer, + valueDeserializer, + singleUseNodes); } /** @@ -56,17 +65,21 @@ public StoredMerklePatriciaTrie( * storage}. * @param valueSerializer A function for serializing values to bytes. * @param valueDeserializer A function for deserializing values from bytes. + * @param singleUseNodes whether nodes should unload data after being visited */ public StoredMerklePatriciaTrie( final NodeLoader nodeLoader, final Bytes32 rootHash, final Function valueSerializer, - final Function valueDeserializer) { - this.nodeFactory = new StoredNodeFactory<>(nodeLoader, valueSerializer, valueDeserializer); + final Function valueDeserializer, + final boolean singleUseNodes) { + this.singleUseNodes = singleUseNodes; + this.nodeFactory = + new StoredNodeFactory<>(nodeLoader, valueSerializer, valueDeserializer, singleUseNodes); this.root = rootHash.equals(MerklePatriciaTrie.EMPTY_TRIE_NODE_HASH) ? NullNode.instance() - : new StoredNode<>(nodeFactory, rootHash); + : new StoredNode<>(nodeFactory, rootHash, singleUseNodes); } @Override @@ -101,7 +114,7 @@ public void commit(final NodeUpdater nodeUpdater) { this.root = rootHash.equals(MerklePatriciaTrie.EMPTY_TRIE_NODE_HASH) ? NullNode.instance() - : new StoredNode<>(nodeFactory, rootHash); + : new StoredNode<>(nodeFactory, rootHash, singleUseNodes); } @Override @@ -109,6 +122,11 @@ public Map entriesFrom(final Bytes32 startKeyHash, final int limit) return StorageEntriesCollector.collectEntries(root, startKeyHash, limit); } + @Override + public void visitAll(final Consumer> visitor) { + root.accept(new AllNodesVisitor<>(visitor)); + } + @Override public Bytes32 getRootHash() { return root.getHash(); diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java index 6b409aba23..2ce1120fa4 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java @@ -22,11 +22,13 @@ class StoredNode implements Node { private final StoredNodeFactory nodeFactory; private final Bytes32 hash; + private final boolean singleUse; private Node loaded; - StoredNode(final StoredNodeFactory nodeFactory, final Bytes32 hash) { + StoredNode(final StoredNodeFactory nodeFactory, final Bytes32 hash, final boolean singleUse) { this.nodeFactory = nodeFactory; this.hash = hash; + this.singleUse = singleUse; } /** @return True if the node needs to be persisted. */ @@ -52,6 +54,9 @@ public Node accept(final PathNodeVisitor visitor, final BytesValue path) { public void accept(final NodeVisitor visitor) { final Node node = load(); node.accept(visitor); + if (singleUse) { + loaded = null; + } } @Override diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNodeFactory.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNodeFactory.java index 41dca7855e..87b07f9cb1 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNodeFactory.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNodeFactory.java @@ -33,14 +33,17 @@ class StoredNodeFactory implements NodeFactory { private final NodeLoader nodeLoader; private final Function valueSerializer; private final Function valueDeserializer; + private final boolean singleUseNodes; StoredNodeFactory( final NodeLoader nodeLoader, final Function valueSerializer, - final Function valueDeserializer) { + final Function valueDeserializer, + final boolean singleUseNodes) { this.nodeLoader = nodeLoader; this.valueSerializer = valueSerializer; this.valueDeserializer = valueDeserializer; + this.singleUseNodes = singleUseNodes; } @Override @@ -156,7 +159,7 @@ private Node decodeExtension( return new ExtensionNode<>(path, childNode, this); } else { final Bytes32 childHash = childRlp.readBytes32(); - final StoredNode childNode = new StoredNode<>(this, childHash); + final StoredNode childNode = new StoredNode<>(this, childHash, singleUseNodes); return new ExtensionNode<>(path, childNode, this); } } @@ -173,7 +176,7 @@ private BranchNode decodeBranch(final RLPInput nodeRLPs, final Supplier(this, childHash)); + children.add(new StoredNode<>(this, childHash, singleUseNodes)); } } diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoder.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoder.java index 5212e90df0..12c742f705 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoder.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoder.java @@ -30,7 +30,8 @@ public class TrieNodeDecoder { private static final StoredNodeFactory emptyNodeFactory = - new StoredNodeFactory<>((h) -> Optional.empty(), Function.identity(), Function.identity()); + new StoredNodeFactory<>( + (h) -> Optional.empty(), Function.identity(), Function.identity(), false); // Hide constructor for static utility class private TrieNodeDecoder() {} @@ -116,7 +117,7 @@ private static class BreadthFirstIterator implements Iterator> BreadthFirstIterator(final NodeLoader nodeLoader, final Bytes32 rootHash, final int maxDepth) { this.maxDepth = maxDepth; this.nodeFactory = - new StoredNodeFactory<>(nodeLoader, Function.identity(), Function.identity()); + new StoredNodeFactory<>(nodeLoader, Function.identity(), Function.identity(), false); nodeLoader.getNode(rootHash).map(TrieNodeDecoder::decode).ifPresent(currentNodes::add); } diff --git a/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java b/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java index cb8932d8d8..b10ab4b64a 100644 --- a/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java +++ b/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java @@ -41,7 +41,9 @@ public void setup() { valueSerializer = value -> (value != null) ? BytesValue.wrap(value.getBytes(Charset.forName("UTF-8"))) : null; valueDeserializer = bytes -> new String(bytes.getArrayUnsafe(), Charset.forName("UTF-8")); - trie = new StoredMerklePatriciaTrie<>(merkleStorage::get, valueSerializer, valueDeserializer); + trie = + new StoredMerklePatriciaTrie<>( + merkleStorage::get, valueSerializer, valueDeserializer, false); } @Test @@ -329,21 +331,21 @@ public void canReloadTrieFromHash() { // Create new tries from root hashes and check that we find expected values trie = new StoredMerklePatriciaTrie<>( - merkleStorage::get, hash1, valueSerializer, valueDeserializer); + merkleStorage::get, hash1, valueSerializer, valueDeserializer, false); assertThat(trie.get(key1)).isEqualTo(Optional.of("value1")); assertThat(trie.get(key2)).isEqualTo(Optional.empty()); assertThat(trie.get(key3)).isEqualTo(Optional.empty()); trie = new StoredMerklePatriciaTrie<>( - merkleStorage::get, hash2, valueSerializer, valueDeserializer); + merkleStorage::get, hash2, valueSerializer, valueDeserializer, false); assertThat(trie.get(key1)).isEqualTo(Optional.of("value1")); assertThat(trie.get(key2)).isEqualTo(Optional.of("value2")); assertThat(trie.get(key3)).isEqualTo(Optional.of("value3")); trie = new StoredMerklePatriciaTrie<>( - merkleStorage::get, hash3, valueSerializer, valueDeserializer); + merkleStorage::get, hash3, valueSerializer, valueDeserializer, false); assertThat(trie.get(key1)).isEqualTo(Optional.of("value4")); assertThat(trie.get(key2)).isEqualTo(Optional.of("value2")); assertThat(trie.get(key3)).isEqualTo(Optional.of("value3")); @@ -353,21 +355,21 @@ public void canReloadTrieFromHash() { final MerkleStorage newMerkleStorage = new KeyValueMerkleStorage(keyValueStore); trie = new StoredMerklePatriciaTrie<>( - newMerkleStorage::get, hash1, valueSerializer, valueDeserializer); + newMerkleStorage::get, hash1, valueSerializer, valueDeserializer, false); assertThat(trie.get(key1)).isEqualTo(Optional.of("value1")); assertThat(trie.get(key2)).isEqualTo(Optional.empty()); assertThat(trie.get(key3)).isEqualTo(Optional.empty()); trie = new StoredMerklePatriciaTrie<>( - newMerkleStorage::get, hash2, valueSerializer, valueDeserializer); + newMerkleStorage::get, hash2, valueSerializer, valueDeserializer, false); assertThat(trie.get(key1)).isEqualTo(Optional.of("value1")); assertThat(trie.get(key2)).isEqualTo(Optional.of("value2")); assertThat(trie.get(key3)).isEqualTo(Optional.of("value3")); trie = new StoredMerklePatriciaTrie<>( - newMerkleStorage::get, hash3, valueSerializer, valueDeserializer); + newMerkleStorage::get, hash3, valueSerializer, valueDeserializer, false); assertThat(trie.get(key1)).isEqualTo(Optional.of("value4")); assertThat(trie.get(key2)).isEqualTo(Optional.of("value2")); assertThat(trie.get(key3)).isEqualTo(Optional.of("value3")); @@ -378,7 +380,7 @@ public void shouldRetrieveStoredExtensionWithInlinedChild() { final KeyValueStorage keyValueStorage = new InMemoryKeyValueStorage(); final MerkleStorage merkleStorage = new KeyValueMerkleStorage(keyValueStorage); final StoredMerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>(merkleStorage::get, b -> b, b -> b); + new StoredMerklePatriciaTrie<>(merkleStorage::get, b -> b, b -> b, false); // Both of these can be inlined in its parent branch and the branch // itself can be inlined into its parent extension. @@ -389,7 +391,7 @@ public void shouldRetrieveStoredExtensionWithInlinedChild() { // Ensure the extension branch can be loaded correct with its inlined child. final Bytes32 rootHash = trie.getRootHash(); final StoredMerklePatriciaTrie newTrie = - new StoredMerklePatriciaTrie<>(merkleStorage::get, rootHash, b -> b, b -> b); + new StoredMerklePatriciaTrie<>(merkleStorage::get, rootHash, b -> b, b -> b, false); newTrie.get(BytesValue.fromHexString("0x0401")); } @@ -400,7 +402,7 @@ public void shouldInlineNodesInParentAcrossModifications() { final KeyValueStorage keyValueStorage = new InMemoryKeyValueStorage(); final MerkleStorage merkleStorage = new KeyValueMerkleStorage(keyValueStorage); final StoredMerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>(merkleStorage::get, b -> b, b -> b); + new StoredMerklePatriciaTrie<>(merkleStorage::get, b -> b, b -> b, false); // Both of these can be inlined in its parent branch. trie.put(BytesValue.fromHexString("0x0400"), BytesValue.of(1)); @@ -409,7 +411,7 @@ public void shouldInlineNodesInParentAcrossModifications() { final Bytes32 rootHash = trie.getRootHash(); final StoredMerklePatriciaTrie newTrie = - new StoredMerklePatriciaTrie<>(merkleStorage::get, rootHash, b -> b, b -> b); + new StoredMerklePatriciaTrie<>(merkleStorage::get, rootHash, b -> b, b -> b, false); newTrie.put(BytesValue.fromHexString("0x0800"), BytesValue.of(3)); newTrie.get(BytesValue.fromHexString("0x0401")); diff --git a/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoderTest.java b/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoderTest.java index 1361722cdb..86786e1166 100644 --- a/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoderTest.java +++ b/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoderTest.java @@ -36,7 +36,8 @@ public void decodeNodes() { // Build a small trie MerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>(storage::get, Function.identity(), Function.identity()); + new StoredMerklePatriciaTrie<>( + storage::get, Function.identity(), Function.identity(), false); trie.put(BytesValue.fromHexString("0x100000"), BytesValue.of(1)); trie.put(BytesValue.fromHexString("0x200000"), BytesValue.of(2)); trie.put(BytesValue.fromHexString("0x300000"), BytesValue.of(3)); @@ -80,7 +81,8 @@ public void breadthFirstDecode_smallTrie() { // Build a small trie MerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>(storage::get, Function.identity(), Function.identity()); + new StoredMerklePatriciaTrie<>( + storage::get, Function.identity(), Function.identity(), false); trie.put(BytesValue.fromHexString("0x100000"), BytesValue.of(1)); trie.put(BytesValue.fromHexString("0x200000"), BytesValue.of(2)); trie.put(BytesValue.fromHexString("0x300000"), BytesValue.of(3)); @@ -153,7 +155,8 @@ public void breadthFirstDecode_partialTrie() { // Build a small trie MerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>(fullStorage::get, Function.identity(), Function.identity()); + new StoredMerklePatriciaTrie<>( + fullStorage::get, Function.identity(), Function.identity(), false); final Random random = new Random(1); for (int i = 0; i < 30; i++) { byte[] key = new byte[4]; @@ -195,7 +198,8 @@ public void breadthFirstDecode_singleNodeTrie() { final InMemoryKeyValueStorage storage = new InMemoryKeyValueStorage(); MerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>(storage::get, Function.identity(), Function.identity()); + new StoredMerklePatriciaTrie<>( + storage::get, Function.identity(), Function.identity(), false); trie.put(BytesValue.fromHexString("0x100000"), BytesValue.of(1)); // Save nodes to storage diff --git a/metrics/core/src/main/java/tech/pegasys/pantheon/metrics/PantheonMetricCategory.java b/metrics/core/src/main/java/tech/pegasys/pantheon/metrics/PantheonMetricCategory.java index 6b54cd1b7b..18d439ef18 100644 --- a/metrics/core/src/main/java/tech/pegasys/pantheon/metrics/PantheonMetricCategory.java +++ b/metrics/core/src/main/java/tech/pegasys/pantheon/metrics/PantheonMetricCategory.java @@ -27,6 +27,7 @@ public enum PantheonMetricCategory implements MetricCategory { PERMISSIONING("permissioning"), KVSTORE_ROCKSDB("rocksdb"), KVSTORE_ROCKSDB_STATS("rocksdb", false), + PRUNER("pruner"), RPC("rpc"), SYNCHRONIZER("synchronizer"), TRANSACTION_POOL("transaction_pool"); diff --git a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java index 27352dcaa1..51f4a7a2a4 100644 --- a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java +++ b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java @@ -44,6 +44,8 @@ import tech.pegasys.pantheon.ethereum.p2p.config.SubProtocolConfiguration; import tech.pegasys.pantheon.ethereum.storage.StorageProvider; import tech.pegasys.pantheon.ethereum.storage.keyvalue.RocksDbStorageProvider; +import tech.pegasys.pantheon.ethereum.worldstate.MarkSweepPruner; +import tech.pegasys.pantheon.ethereum.worldstate.Pruner; import tech.pegasys.pantheon.ethereum.worldstate.WorldStateArchive; import tech.pegasys.pantheon.metrics.MetricsSystem; import tech.pegasys.pantheon.services.kvstore.RocksDbConfiguration; @@ -199,6 +201,16 @@ public PantheonController build() throws IOException { final MutableBlockchain blockchain = protocolContext.getBlockchain(); + final Pruner pruner = + new Pruner( + new MarkSweepPruner( + protocolContext.getWorldStateArchive().getStorage(), + storageProvider.createPruningStorage(), + metricsSystem), + protocolContext, + 1000); + addShutdownAction(pruner::stop); + final boolean fastSyncEnabled = syncConfig.getSyncMode().equals(SyncMode.FAST); ethProtocolManager = createEthProtocolManager(protocolContext, fastSyncEnabled); final SyncState syncState = @@ -210,6 +222,7 @@ public PantheonController build() throws IOException { protocolContext, protocolContext.getWorldStateArchive().getStorage(), ethProtocolManager.getBlockBroadcaster(), + pruner, ethProtocolManager.ethContext(), syncState, dataDirectory, diff --git a/services/kvstore/src/main/java/tech/pegasys/pantheon/services/kvstore/RocksDbConfiguration.java b/services/kvstore/src/main/java/tech/pegasys/pantheon/services/kvstore/RocksDbConfiguration.java index ef6538689e..9025f66280 100644 --- a/services/kvstore/src/main/java/tech/pegasys/pantheon/services/kvstore/RocksDbConfiguration.java +++ b/services/kvstore/src/main/java/tech/pegasys/pantheon/services/kvstore/RocksDbConfiguration.java @@ -89,7 +89,7 @@ public static class Builder { long cacheCapacity = DEFAULT_CACHE_CAPACITY; int maxBackgroundCompactions = DEFAULT_MAX_BACKGROUND_COMPACTIONS; int backgroundThreadCount = DEFAULT_BACKGROUND_THREAD_COUNT; - boolean useColumns = false; + boolean useColumns = true; private Builder() {} diff --git a/services/kvstore/src/test/java/tech/pegasys/pantheon/services/kvstore/AbstractKeyValueStorageTest.java b/services/kvstore/src/test/java/tech/pegasys/pantheon/services/kvstore/AbstractKeyValueStorageTest.java index 4ad98f657b..839df858e9 100644 --- a/services/kvstore/src/test/java/tech/pegasys/pantheon/services/kvstore/AbstractKeyValueStorageTest.java +++ b/services/kvstore/src/test/java/tech/pegasys/pantheon/services/kvstore/AbstractKeyValueStorageTest.java @@ -14,6 +14,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import tech.pegasys.pantheon.services.kvstore.KeyValueStorage.Transaction; @@ -62,6 +63,20 @@ public void put() throws Exception { Optional.of(BytesValue.fromHexString("0DEF")), store.get(BytesValue.fromHexString("0F"))); } + @Test + public void mayContainKey() throws Exception { + final KeyValueStorage store = createStore(); + final BytesValue key = BytesValue.fromHexString("ABCD"); + + assertFalse(store.containsKey(key)); + + final Transaction transaction = store.startTransaction(); + transaction.put(key, BytesValue.fromHexString("DEFF")); + transaction.commit(); + + assertTrue(store.containsKey(key)); + } + @Test public void removeExisting() throws Exception { final KeyValueStorage store = createStore(); From ee116fb558d5c7a1969bd17cf3035ea90b95ed0a Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Wed, 26 Jun 2019 13:27:57 -0400 Subject: [PATCH 02/53] narrow Pruner dependency: ProtocolContext -> Blockchain --- .../pegasys/pantheon/ethereum/worldstate/Pruner.java | 11 ++++++----- .../controller/PantheonControllerBuilder.java | 2 +- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index e3ea7be675..7182d2a85e 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -12,8 +12,8 @@ */ package tech.pegasys.pantheon.ethereum.worldstate; -import tech.pegasys.pantheon.ethereum.ProtocolContext; import tech.pegasys.pantheon.ethereum.chain.BlockAddedEvent; +import tech.pegasys.pantheon.ethereum.chain.Blockchain; import tech.pegasys.pantheon.ethereum.core.BlockHeader; import tech.pegasys.pantheon.ethereum.core.Hash; @@ -27,8 +27,9 @@ public class Pruner { private static final Logger LOG = LogManager.getLogger(); + private final MarkSweepPruner pruningStrategy; - private final ProtocolContext protocolContext; + private final Blockchain blockchain; private final ExecutorService executor; private final long retentionPeriodInBlocks; private final AtomicReference state = new AtomicReference<>(State.IDLE); @@ -36,10 +37,10 @@ public class Pruner { public Pruner( final MarkSweepPruner pruningStrategy, - final ProtocolContext protocolContext, + final Blockchain blockchain, final long retentionPeriodInBlocks) { this.pruningStrategy = pruningStrategy; - this.protocolContext = protocolContext; + this.blockchain = blockchain; this.executor = Executors.newSingleThreadExecutor( new ThreadFactoryBuilder() @@ -52,7 +53,7 @@ public Pruner( public void start() { // TODO: Don't attempt pruning while a fast sync is in progress. - protocolContext.getBlockchain().observeBlockAdded((event, blockchain) -> handleNewBlock(event)); + blockchain.observeBlockAdded((event, blockchain) -> handleNewBlock(event)); } public void stop() { diff --git a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java index 51f4a7a2a4..461ed9b67c 100644 --- a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java +++ b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java @@ -207,7 +207,7 @@ public PantheonController build() throws IOException { protocolContext.getWorldStateArchive().getStorage(), storageProvider.createPruningStorage(), metricsSystem), - protocolContext, + protocolContext.getBlockchain(), 1000); addShutdownAction(pruner::stop); From 0c6af0d099879a1dbfcdb3240ceb05a55dde6ef7 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Wed, 26 Jun 2019 13:28:09 -0400 Subject: [PATCH 03/53] test scaffolding --- .../ethereum/worldstate/PrunerTest.java | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) create mode 100644 ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java new file mode 100644 index 0000000000..16b16c99ad --- /dev/null +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -0,0 +1,39 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * 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. + */ +package tech.pegasys.pantheon.ethereum.worldstate; + +import tech.pegasys.pantheon.ethereum.chain.Blockchain; +import tech.pegasys.pantheon.ethereum.storage.keyvalue.WorldStateKeyValueStorage; +import tech.pegasys.pantheon.ethereum.vm.TestBlockchain; +import tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem; +import tech.pegasys.pantheon.services.kvstore.InMemoryKeyValueStorage; + +import org.junit.Test; + +public class PrunerTest { + + private final NoOpMetricsSystem metricsSystem = new NoOpMetricsSystem(); + + @Test + public void doesNotLoseStateDuringReorg() { + final InMemoryKeyValueStorage markStorage = new InMemoryKeyValueStorage(); + final InMemoryKeyValueStorage stateStorage = new InMemoryKeyValueStorage(); + final WorldStateStorage worldStateStorage = new WorldStateKeyValueStorage(stateStorage); + final WorldStateArchive worldStateArchive = new WorldStateArchive(worldStateStorage); + final Blockchain blockchain = new TestBlockchain(); + final Pruner pruner = + new Pruner( + new MarkSweepPruner(worldStateStorage, markStorage, metricsSystem), blockchain, 2); + pruner.start(); + } +} From 7073faef3865512161d1b9998ee7d8fbec97e36f Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Thu, 27 Jun 2019 09:52:38 -0400 Subject: [PATCH 04/53] explainer comments and beginnings of block creation --- .../ethereum/worldstate/PrunerTest.java | 57 ++++++++++++++++++- 1 file changed, 55 insertions(+), 2 deletions(-) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 16b16c99ad..12ba69d611 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -12,13 +12,26 @@ */ package tech.pegasys.pantheon.ethereum.worldstate; +import tech.pegasys.pantheon.consensus.clique.CliqueBlockHeaderFunctions; import tech.pegasys.pantheon.ethereum.chain.Blockchain; +import tech.pegasys.pantheon.ethereum.chain.BlockchainStorage; +import tech.pegasys.pantheon.ethereum.chain.DefaultMutableBlockchain; +import tech.pegasys.pantheon.ethereum.chain.DefaultMutableBlockchainTest; +import tech.pegasys.pantheon.ethereum.core.Block; +import tech.pegasys.pantheon.ethereum.core.BlockDataGenerator; +import tech.pegasys.pantheon.ethereum.core.BlockDataGenerator.BlockOptions; +import tech.pegasys.pantheon.ethereum.core.TransactionReceipt; +import tech.pegasys.pantheon.ethereum.mainnet.MainnetBlockHeaderFunctions; +import tech.pegasys.pantheon.ethereum.storage.keyvalue.KeyValueStoragePrefixedKeyBlockchainStorage; import tech.pegasys.pantheon.ethereum.storage.keyvalue.WorldStateKeyValueStorage; import tech.pegasys.pantheon.ethereum.vm.TestBlockchain; import tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem; import tech.pegasys.pantheon.services.kvstore.InMemoryKeyValueStorage; import org.junit.Test; +import tech.pegasys.pantheon.services.kvstore.KeyValueStorage; + +import java.util.List; public class PrunerTest { @@ -29,11 +42,51 @@ public void doesNotLoseStateDuringReorg() { final InMemoryKeyValueStorage markStorage = new InMemoryKeyValueStorage(); final InMemoryKeyValueStorage stateStorage = new InMemoryKeyValueStorage(); final WorldStateStorage worldStateStorage = new WorldStateKeyValueStorage(stateStorage); - final WorldStateArchive worldStateArchive = new WorldStateArchive(worldStateStorage); - final Blockchain blockchain = new TestBlockchain(); +// final WorldStateArchive worldStateArchive = new WorldStateArchive(worldStateStorage); + + final BlockDataGenerator gen = new BlockDataGenerator(); + final KeyValueStorage kvStore = new InMemoryKeyValueStorage(); + final Block genesisBlock = gen.genesisBlock(); + final DefaultMutableBlockchain blockchain = createBlockchain(kvStore, genesisBlock); + + /* + Set up pre-marking state: + O <--- this is a fork as of now (non-canonical) + | O <--- marking of the full state trie will begin at this block + \/ + O <--- the common ancestor when the reorg happens + */ + BlockOptions options = + new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); + Block newBlock = gen.block(options); + List receipts = gen.receipts(newBlock); + blockchain.appendBlock(newBlock, receipts); + + options = new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); + newBlock = gen.block(options); + receipts = gen.receipts(newBlock); + blockchain.appendBlock(newBlock, receipts); + /* + Cause reorg: + O <-- this block causes a reorg, this branch becomes canonical + | + O <-- which means that state here is referring to nodes from the common ancestor block, + | which is not fully marked + | O <--- because this was the last fully marked block + \/ + O + */ final Pruner pruner = new Pruner( new MarkSweepPruner(worldStateStorage, markStorage, metricsSystem), blockchain, 2); pruner.start(); } + + private DefaultMutableBlockchain createBlockchain( + final KeyValueStorage kvStore, final Block genesisBlock) { + return new DefaultMutableBlockchain( + genesisBlock, + new KeyValueStoragePrefixedKeyBlockchainStorage(kvStore, new MainnetBlockHeaderFunctions()), + new NoOpMetricsSystem()); + } } From a95ab9924bb2ed65fa2310c74558c96fe2890985 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Thu, 27 Jun 2019 13:06:56 -0400 Subject: [PATCH 05/53] make Pruner.State public and add getter --- .../tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 7182d2a85e..fdbff547e8 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -51,6 +51,10 @@ public Pruner( this.retentionPeriodInBlocks = retentionPeriodInBlocks; } + public AtomicReference getState() { + return state; + } + public void start() { // TODO: Don't attempt pruning while a fast sync is in progress. blockchain.observeBlockAdded((event, blockchain) -> handleNewBlock(event)); @@ -114,7 +118,7 @@ private void execute(final Runnable action) { } } - private enum State { + public enum State { IDLE, MARKING, MARKING_COMPLETE, From 023dee91bcdabaf341b2e8979f803e98cae936e5 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Thu, 27 Jun 2019 13:07:04 -0400 Subject: [PATCH 06/53] sweeping log --- .../java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index fdbff547e8..c885d0f577 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -101,6 +101,8 @@ private void mark(final BlockHeader header) { } private void sweep() { + LOG.info( + "Begin sweeping unused nodes for pruning. Retention period: {}", retentionPeriodInBlocks); execute( () -> { pruningStrategy.sweep(); From aef1c1eec6505a37932d7b91e0c40f597693d68f Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Thu, 27 Jun 2019 13:08:23 -0400 Subject: [PATCH 07/53] get test properly failing --- .../ethereum/worldstate/PrunerTest.java | 87 +++++++++---------- 1 file changed, 43 insertions(+), 44 deletions(-) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 12ba69d611..31e1165133 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -12,81 +12,80 @@ */ package tech.pegasys.pantheon.ethereum.worldstate; -import tech.pegasys.pantheon.consensus.clique.CliqueBlockHeaderFunctions; -import tech.pegasys.pantheon.ethereum.chain.Blockchain; -import tech.pegasys.pantheon.ethereum.chain.BlockchainStorage; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + import tech.pegasys.pantheon.ethereum.chain.DefaultMutableBlockchain; -import tech.pegasys.pantheon.ethereum.chain.DefaultMutableBlockchainTest; import tech.pegasys.pantheon.ethereum.core.Block; import tech.pegasys.pantheon.ethereum.core.BlockDataGenerator; import tech.pegasys.pantheon.ethereum.core.BlockDataGenerator.BlockOptions; import tech.pegasys.pantheon.ethereum.core.TransactionReceipt; import tech.pegasys.pantheon.ethereum.mainnet.MainnetBlockHeaderFunctions; import tech.pegasys.pantheon.ethereum.storage.keyvalue.KeyValueStoragePrefixedKeyBlockchainStorage; -import tech.pegasys.pantheon.ethereum.storage.keyvalue.WorldStateKeyValueStorage; -import tech.pegasys.pantheon.ethereum.vm.TestBlockchain; import tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem; import tech.pegasys.pantheon.services.kvstore.InMemoryKeyValueStorage; -import org.junit.Test; -import tech.pegasys.pantheon.services.kvstore.KeyValueStorage; - import java.util.List; +import org.junit.Test; +import org.mockito.Mock; + public class PrunerTest { private final NoOpMetricsSystem metricsSystem = new NoOpMetricsSystem(); - @Test - public void doesNotLoseStateDuringReorg() { - final InMemoryKeyValueStorage markStorage = new InMemoryKeyValueStorage(); - final InMemoryKeyValueStorage stateStorage = new InMemoryKeyValueStorage(); - final WorldStateStorage worldStateStorage = new WorldStateKeyValueStorage(stateStorage); -// final WorldStateArchive worldStateArchive = new WorldStateArchive(worldStateStorage); + @Mock private final BlockDataGenerator gen = new BlockDataGenerator(); - final BlockDataGenerator gen = new BlockDataGenerator(); - final KeyValueStorage kvStore = new InMemoryKeyValueStorage(); + @Mock private MarkSweepPruner markSweepPruner; + + @Test + public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { + markSweepPruner = mock(MarkSweepPruner.class); final Block genesisBlock = gen.genesisBlock(); - final DefaultMutableBlockchain blockchain = createBlockchain(kvStore, genesisBlock); + final DefaultMutableBlockchain blockchain = + new DefaultMutableBlockchain( + genesisBlock, + new KeyValueStoragePrefixedKeyBlockchainStorage( + new InMemoryKeyValueStorage(), new MainnetBlockHeaderFunctions()), + metricsSystem); + + // start pruner so it can start handling block added events + final Pruner pruner = new Pruner(markSweepPruner, blockchain, 0); + pruner.start(); /* Set up pre-marking state: - O <--- this is a fork as of now (non-canonical) - | O <--- marking of the full state trie will begin at this block + O <---- this is a fork as of now (non-canonical) + | O <- marking of the full state trie will begin at this block \/ - O <--- the common ancestor when the reorg happens + O <--- the common ancestor when the reorg happens */ BlockOptions options = - new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); - Block newBlock = gen.block(options); - List receipts = gen.receipts(newBlock); - blockchain.appendBlock(newBlock, receipts); + new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); + final Block canonicalHead = gen.block(options); + List receipts = gen.receipts(canonicalHead); + blockchain.appendBlock(canonicalHead, receipts); options = new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); - newBlock = gen.block(options); - receipts = gen.receipts(newBlock); - blockchain.appendBlock(newBlock, receipts); + final Block forkBlock = gen.block(options); + receipts = gen.receipts(forkBlock); + blockchain.appendBlock(forkBlock, receipts); /* Cause reorg: - O <-- this block causes a reorg, this branch becomes canonical + O <---- this block causes a reorg; this branch becomes canonical | - O <-- which means that state here is referring to nodes from the common ancestor block, - | which is not fully marked - | O <--- because this was the last fully marked block + O <---- which means that state here is referring to nodes from the common ancestor block, + | which is not fully marked + | O <- because this was block at which marking began \/ O */ - final Pruner pruner = - new Pruner( - new MarkSweepPruner(worldStateStorage, markStorage, metricsSystem), blockchain, 2); - pruner.start(); - } - - private DefaultMutableBlockchain createBlockchain( - final KeyValueStorage kvStore, final Block genesisBlock) { - return new DefaultMutableBlockchain( - genesisBlock, - new KeyValueStoragePrefixedKeyBlockchainStorage(kvStore, new MainnetBlockHeaderFunctions()), - new NoOpMetricsSystem()); + options = new BlockOptions().setBlockNumber(2L).setParentHash(forkBlock.getHash()); + final Block reorgCausingBlock = gen.block(options); + receipts = gen.receipts(reorgCausingBlock); + blockchain.appendBlock(reorgCausingBlock, receipts); + // after the reorg, the sweep should have been aborted, so the genesis block state should remain + verify(markSweepPruner, never()).sweep(); } } From 1af66ea8f856e62a6d92725f896996528e6cac28 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Mon, 1 Jul 2019 23:24:24 -0400 Subject: [PATCH 08/53] gate sweeping on marked block being on canonical chain --- .../java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index c885d0f577..657b3425aa 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -34,6 +34,7 @@ public class Pruner { private final long retentionPeriodInBlocks; private final AtomicReference state = new AtomicReference<>(State.IDLE); private volatile long markedBlockNumber = 0; + private volatile Hash markedBlockHash; public Pruner( final MarkSweepPruner pruningStrategy, @@ -80,6 +81,7 @@ private void handleNewBlock(final BlockAddedEvent event) { mark(header); } else if (header.getNumber() > markedBlockNumber + retentionPeriodInBlocks + && blockchain.blockIsOnCanonicalChain(markedBlockHash) && state.compareAndSet(State.MARKING_COMPLETE, State.SWEEPING)) { sweep(); } @@ -87,6 +89,7 @@ private void handleNewBlock(final BlockAddedEvent event) { private void mark(final BlockHeader header) { markedBlockNumber = header.getNumber(); + markedBlockHash = header.getHash(); final Hash stateRoot = header.getStateRoot(); LOG.info( "Begin marking used nodes for pruning. Block number: {} State root: {}", From 0bccea4961ed35dac4eb8b1015edc2a7dc89bbe6 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 2 Jul 2019 12:30:40 -0400 Subject: [PATCH 09/53] privitize state and remove getter --- .../tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 657b3425aa..ce08b405d0 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -52,10 +52,6 @@ public Pruner( this.retentionPeriodInBlocks = retentionPeriodInBlocks; } - public AtomicReference getState() { - return state; - } - public void start() { // TODO: Don't attempt pruning while a fast sync is in progress. blockchain.observeBlockAdded((event, blockchain) -> handleNewBlock(event)); @@ -123,7 +119,7 @@ private void execute(final Runnable action) { } } - public enum State { + private enum State { IDLE, MARKING, MARKING_COMPLETE, From adbc86acc748b5e6688bce8e35948a8084137d90 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 2 Jul 2019 13:08:20 -0400 Subject: [PATCH 10/53] add transient fork outliving --- .../pantheon/ethereum/worldstate/Pruner.java | 41 ++++++++++--------- .../ethereum/worldstate/PrunerTest.java | 2 +- .../controller/PantheonControllerBuilder.java | 3 +- 3 files changed, 24 insertions(+), 22 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index ce08b405d0..f71ea20dfa 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -33,13 +33,15 @@ public class Pruner { private final ExecutorService executor; private final long retentionPeriodInBlocks; private final AtomicReference state = new AtomicReference<>(State.IDLE); - private volatile long markedBlockNumber = 0; - private volatile Hash markedBlockHash; + private volatile long markBlockNumber = 0; + private volatile BlockHeader markedBlockHeader; + private long transientForkThreshold; public Pruner( final MarkSweepPruner pruningStrategy, final Blockchain blockchain, - final long retentionPeriodInBlocks) { + final long retentionPeriodInBlocks, + final long transientForkThreshold) { this.pruningStrategy = pruningStrategy; this.blockchain = blockchain; this.executor = @@ -50,6 +52,7 @@ public Pruner( .setNameFormat("StatePruning-%d") .build()); this.retentionPeriodInBlocks = retentionPeriodInBlocks; + this.transientForkThreshold = transientForkThreshold; } public void start() { @@ -66,32 +69,29 @@ private void handleNewBlock(final BlockAddedEvent event) { return; } final BlockHeader header = event.getBlock().getHeader(); - // Only kick off pruning every million blocks. - if (state.compareAndSet(State.IDLE, State.MARKING)) { - /* TODO: We don't currently handle re-orgs. - Will need to: - 1. start listening for new nodes at this point - 2. then after a few blocks mark the world state of the block when we started listening (now a few blocks back) - 3. check the block we marked is still on the canonical chain when we begin sweeping and abort that round of pruning if not - */ - - mark(header); - } else if (header.getNumber() > markedBlockNumber + retentionPeriodInBlocks - && blockchain.blockIsOnCanonicalChain(markedBlockHash) + if (state.compareAndSet(State.IDLE, State.TRANSIENT_FORK_OUTLIVING)) { + pruningStrategy + .prepare(); // implications for calling prepare here when blocks can come from multiple + // threads? + markBlockNumber = header.getNumber(); + } else if (header.getNumber() > markBlockNumber + transientForkThreshold + && state.compareAndSet(State.TRANSIENT_FORK_OUTLIVING, State.MARKING)) { + markedBlockHeader = blockchain.getBlockHeader(markBlockNumber).get(); + mark(markedBlockHeader); + } else if (header.getNumber() > markBlockNumber + retentionPeriodInBlocks + && blockchain.blockIsOnCanonicalChain(markedBlockHeader.getHash()) && state.compareAndSet(State.MARKING_COMPLETE, State.SWEEPING)) { sweep(); } } private void mark(final BlockHeader header) { - markedBlockNumber = header.getNumber(); - markedBlockHash = header.getHash(); + markBlockNumber = header.getNumber(); final Hash stateRoot = header.getStateRoot(); LOG.info( "Begin marking used nodes for pruning. Block number: {} State root: {}", - markedBlockNumber, + markBlockNumber, stateRoot); - pruningStrategy.prepare(); execute( () -> { pruningStrategy.mark(stateRoot); @@ -121,8 +121,9 @@ private void execute(final Runnable action) { private enum State { IDLE, + TRANSIENT_FORK_OUTLIVING, MARKING, MARKING_COMPLETE, - SWEEPING + SWEEPING; } } diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 31e1165133..c8846f13d2 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -51,7 +51,7 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { metricsSystem); // start pruner so it can start handling block added events - final Pruner pruner = new Pruner(markSweepPruner, blockchain, 0); + final Pruner pruner = new Pruner(markSweepPruner, blockchain, 0, 0); pruner.start(); /* diff --git a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java index 461ed9b67c..0d692461ff 100644 --- a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java +++ b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java @@ -208,7 +208,8 @@ public PantheonController build() throws IOException { storageProvider.createPruningStorage(), metricsSystem), protocolContext.getBlockchain(), - 1000); + 1000, + 10); addShutdownAction(pruner::stop); final boolean fastSyncEnabled = syncConfig.getSyncMode().equals(SyncMode.FAST); From 216907a4a8e01816abb5f45923a2fe420893b1e4 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Wed, 3 Jul 2019 11:31:28 -0400 Subject: [PATCH 11/53] add test check that markStorage is empty after sweeping --- .../pantheon/ethereum/worldstate/MarkSweepPrunerTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java index 80def9016d..3f39909b36 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java @@ -61,5 +61,6 @@ public void shouldMarkAllNodesInCurrentWorldState() { // All those new nodes should be removed when we sweep pruner.sweep(); assertThat(stateStorage.keySet()).containsExactlyInAnyOrderElementsOf(keysToKeep); + assertThat(markStorage.keySet()).isEmpty(); } } From a69bf0f20e5d8bb6a385f8a4a2f432c95f9cbdf7 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Wed, 3 Jul 2019 12:09:44 -0400 Subject: [PATCH 12/53] disable pruner for now --- .../pantheon/ethereum/eth/sync/DefaultSynchronizer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java b/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java index 6191410f28..b83f33097e 100644 --- a/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java +++ b/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java @@ -168,7 +168,8 @@ private void handleFastSyncResult(final FastSyncState result, final Throwable er private void startFullSync() { fullSyncDownloader.start(); - pruner.start(); + // TODO: enable when pruning cli implemented + // pruner.start(); } @Override From 4615be33f78a6ca6cdca5c6224ad39ba800d3624 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Wed, 3 Jul 2019 12:23:44 -0400 Subject: [PATCH 13/53] set useColumns to false --- .../pegasys/pantheon/services/kvstore/RocksDbConfiguration.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/services/kvstore/src/main/java/tech/pegasys/pantheon/services/kvstore/RocksDbConfiguration.java b/services/kvstore/src/main/java/tech/pegasys/pantheon/services/kvstore/RocksDbConfiguration.java index 9025f66280..ef6538689e 100644 --- a/services/kvstore/src/main/java/tech/pegasys/pantheon/services/kvstore/RocksDbConfiguration.java +++ b/services/kvstore/src/main/java/tech/pegasys/pantheon/services/kvstore/RocksDbConfiguration.java @@ -89,7 +89,7 @@ public static class Builder { long cacheCapacity = DEFAULT_CACHE_CAPACITY; int maxBackgroundCompactions = DEFAULT_MAX_BACKGROUND_COMPACTIONS; int backgroundThreadCount = DEFAULT_BACKGROUND_THREAD_COUNT; - boolean useColumns = true; + boolean useColumns = false; private Builder() {} From f268c57b70598f78c6b0e7f8e18ca446560da661 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Wed, 3 Jul 2019 12:28:24 -0400 Subject: [PATCH 14/53] remove unused variable --- .../pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java b/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java index b83f33097e..d9fcbd31e7 100644 --- a/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java +++ b/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java @@ -45,7 +45,6 @@ public class DefaultSynchronizer implements Synchronizer { private static final Logger LOG = LogManager.getLogger(); - private final Pruner pruner; private final SyncState syncState; private final AtomicBoolean running = new AtomicBoolean(false); private final Subscribers syncStatusListeners = Subscribers.create(); @@ -65,7 +64,6 @@ public DefaultSynchronizer( final Path dataDirectory, final Clock clock, final MetricsSystem metricsSystem) { - this.pruner = pruner; this.syncState = syncState; ChainHeadTracker.trackChainHeadForPeers( From ef2499cddfd1a95cbdf3a1722056652a070aa460 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Fri, 5 Jul 2019 09:40:48 -0400 Subject: [PATCH 15/53] remove resolved TODOs --- .../pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java | 3 --- .../java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 1 - 2 files changed, 4 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java index 2a9ed547b0..f7d335ff2d 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java @@ -83,9 +83,6 @@ public void prepare() { nodeAddedListenerId = worldStateStorage.addNodeAddedListener(this::markNewNodes); } - // Note chainHeadStateRoot must be the state root of the current chain head. - // We can delay the actual sweep until a certain number of blocks in the future if we want to - // have a certain number of block history available. // TODO: Need to ensure we only start marking when new world states aren't being persisted // Once we add our node added listener persisting world states can continue but we can't start // half way through persisting a world state. diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index f71ea20dfa..8a8d2a35f0 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -56,7 +56,6 @@ public Pruner( } public void start() { - // TODO: Don't attempt pruning while a fast sync is in progress. blockchain.observeBlockAdded((event, blockchain) -> handleNewBlock(event)); } From 26a597299b6fe8efd2a1c5590de943ad2c59f9aa Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Fri, 5 Jul 2019 10:21:31 -0400 Subject: [PATCH 16/53] remove TODO that is now tracked by JIRA issue --- .../pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java index f7d335ff2d..914642a4f3 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java @@ -83,9 +83,6 @@ public void prepare() { nodeAddedListenerId = worldStateStorage.addNodeAddedListener(this::markNewNodes); } - // TODO: Need to ensure we only start marking when new world states aren't being persisted - // Once we add our node added listener persisting world states can continue but we can't start - // half way through persisting a world state. public void mark(final Hash rootHash) { markOperationCounter.inc(); markStorage.clear(); From d26b30c20e53eef0c506caf21df96321d178bf05 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Fri, 5 Jul 2019 10:22:17 -0400 Subject: [PATCH 17/53] unmock BlockDataGenerator --- .../tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index c8846f13d2..b6a0226e29 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -35,7 +35,7 @@ public class PrunerTest { private final NoOpMetricsSystem metricsSystem = new NoOpMetricsSystem(); - @Mock private final BlockDataGenerator gen = new BlockDataGenerator(); + private final BlockDataGenerator gen = new BlockDataGenerator(); @Mock private MarkSweepPruner markSweepPruner; From c7224cd5ebdfaf3ab55b948786fe3a6bb276314a Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Thu, 11 Jul 2019 14:33:17 -0400 Subject: [PATCH 18/53] remove outdated comment --- .../tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index b6a0226e29..251a2fa724 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -85,7 +85,6 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { final Block reorgCausingBlock = gen.block(options); receipts = gen.receipts(reorgCausingBlock); blockchain.appendBlock(reorgCausingBlock, receipts); - // after the reorg, the sweep should have been aborted, so the genesis block state should remain verify(markSweepPruner, never()).sweep(); } } From 6eaded261d1fbe05ebcd175bd2e7fca7722ef91e Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Mon, 15 Jul 2019 10:27:47 -0400 Subject: [PATCH 19/53] remove mark/sweep irrelevant changes --- .../pegasys/pantheon/ethereum/blockcreation/BlockMiner.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/ethereum/blockcreation/src/main/java/tech/pegasys/pantheon/ethereum/blockcreation/BlockMiner.java b/ethereum/blockcreation/src/main/java/tech/pegasys/pantheon/ethereum/blockcreation/BlockMiner.java index 6fee690064..c16861f9fc 100644 --- a/ethereum/blockcreation/src/main/java/tech/pegasys/pantheon/ethereum/blockcreation/BlockMiner.java +++ b/ethereum/blockcreation/src/main/java/tech/pegasys/pantheon/ethereum/blockcreation/BlockMiner.java @@ -80,11 +80,6 @@ public void run() { Thread.currentThread().interrupt(); } catch (final Exception ex) { LOG.error("Block mining threw an unhandled exception.", ex); - try { - Thread.sleep(TimeUnit.SECONDS.toMillis(5)); - } catch (final InterruptedException e) { - Thread.currentThread().interrupt(); - } } } } From 33a5b9decb509bda67c6075f897f1a28cb7bca63 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Mon, 15 Jul 2019 10:28:37 -0400 Subject: [PATCH 20/53] make nodeAddedListenerID volatile --- .../pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java | 2 +- .../tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java index 914642a4f3..484563a7d5 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java @@ -44,7 +44,7 @@ public class MarkSweepPruner { private final Counter markOperationCounter; private final Counter sweepOperationCounter; private final Counter sweptNodesCounter; - private long nodeAddedListenerId; + private volatile long nodeAddedListenerId; private final ReentrantLock markLock = new ReentrantLock(true); private final Set pendingMarks = Collections.newSetFromMap(new ConcurrentHashMap<>()); diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 8a8d2a35f0..62c9772a42 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -69,9 +69,7 @@ private void handleNewBlock(final BlockAddedEvent event) { } final BlockHeader header = event.getBlock().getHeader(); if (state.compareAndSet(State.IDLE, State.TRANSIENT_FORK_OUTLIVING)) { - pruningStrategy - .prepare(); // implications for calling prepare here when blocks can come from multiple - // threads? + pruningStrategy.prepare(); markBlockNumber = header.getNumber(); } else if (header.getNumber() > markBlockNumber + transientForkThreshold && state.compareAndSet(State.TRANSIENT_FORK_OUTLIVING, State.MARKING)) { From 36411593e8fd19d436efa58409198f7176d5f1bd Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Mon, 15 Jul 2019 10:31:34 -0400 Subject: [PATCH 21/53] don't exit on pruning failure --- .../java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 1 - 1 file changed, 1 deletion(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 62c9772a42..c402485bc2 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -112,7 +112,6 @@ private void execute(final Runnable action) { } catch (final Throwable t) { LOG.error("Pruning failed", t); state.set(State.IDLE); - System.exit(1); } } From a395a3d6ccbfa3d529d0eddd103d5b888ac86b32 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Mon, 15 Jul 2019 11:01:47 -0400 Subject: [PATCH 22/53] add mockito runner to pruner tests --- .../tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 251a2fa724..78914d6c30 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -16,6 +16,8 @@ import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; import tech.pegasys.pantheon.ethereum.chain.DefaultMutableBlockchain; import tech.pegasys.pantheon.ethereum.core.Block; import tech.pegasys.pantheon.ethereum.core.BlockDataGenerator; @@ -31,6 +33,7 @@ import org.junit.Test; import org.mockito.Mock; +@RunWith(MockitoJUnitRunner.class) public class PrunerTest { private final NoOpMetricsSystem metricsSystem = new NoOpMetricsSystem(); @@ -41,7 +44,6 @@ public class PrunerTest { @Test public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { - markSweepPruner = mock(MarkSweepPruner.class); final Block genesisBlock = gen.genesisBlock(); final DefaultMutableBlockchain blockchain = new DefaultMutableBlockchain( From 7e004b1ba7235c6628a6d335280a7c437cea4266 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Mon, 15 Jul 2019 16:29:02 -0400 Subject: [PATCH 23/53] awaitTermination of Pruner instead of shutdownNow --- .../tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index c402485bc2..ebe956bbe5 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -19,6 +19,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -59,8 +60,8 @@ public void start() { blockchain.observeBlockAdded((event, blockchain) -> handleNewBlock(event)); } - public void stop() { - executor.shutdownNow(); + public void stop() throws InterruptedException { + executor.awaitTermination(30, TimeUnit.SECONDS); } private void handleNewBlock(final BlockAddedEvent event) { From e57885b3685f5ec7a5e15372bd87f5dc98f7614d Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Mon, 15 Jul 2019 16:33:45 -0400 Subject: [PATCH 24/53] gate pruning with boolean instead of comment --- .../pantheon/ethereum/eth/sync/DefaultSynchronizer.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java b/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java index d9fcbd31e7..e53105d931 100644 --- a/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java +++ b/ethereum/eth/src/main/java/tech/pegasys/pantheon/ethereum/eth/sync/DefaultSynchronizer.java @@ -45,6 +45,8 @@ public class DefaultSynchronizer implements Synchronizer { private static final Logger LOG = LogManager.getLogger(); + private static final boolean PRUNING_ENABLED = false; + private final Pruner pruner; private final SyncState syncState; private final AtomicBoolean running = new AtomicBoolean(false); private final Subscribers syncStatusListeners = Subscribers.create(); @@ -64,6 +66,7 @@ public DefaultSynchronizer( final Path dataDirectory, final Clock clock, final MetricsSystem metricsSystem) { + this.pruner = pruner; this.syncState = syncState; ChainHeadTracker.trackChainHeadForPeers( @@ -166,8 +169,9 @@ private void handleFastSyncResult(final FastSyncState result, final Throwable er private void startFullSync() { fullSyncDownloader.start(); - // TODO: enable when pruning cli implemented - // pruner.start(); + if (PRUNING_ENABLED) { + pruner.start(); + } } @Override From 89a44adbdf3cc8eedbd15c583666cc6c5d482156 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Mon, 15 Jul 2019 17:27:16 -0400 Subject: [PATCH 25/53] inject ExecutorService dependency into Pruner and move MockExecutorService to testutil --- .../pantheon/ethereum/worldstate/Pruner.java | 16 ++++++---------- .../ethereum/worldstate/PrunerTest.java | 2 +- .../eth/manager/DeterministicEthScheduler.java | 2 ++ .../ethereum/eth/manager/EthSchedulerTest.java | 2 ++ .../eth/manager/MockScheduledExecutor.java | 2 ++ .../worldstate/WorldStateDownloaderTest.java | 2 +- .../controller/PantheonControllerBuilder.java | 17 ++++++++++++++++- testutil/build.gradle | 1 + .../pantheon/testutil}/MockExecutorService.java | 8 ++++---- 9 files changed, 35 insertions(+), 17 deletions(-) rename {ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager => testutil/src/main/java/tech/pegasys/pantheon/testutil}/MockExecutorService.java (97%) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index ebe956bbe5..7d2cfa6587 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -17,6 +17,7 @@ import tech.pegasys.pantheon.ethereum.core.BlockHeader; import tech.pegasys.pantheon.ethereum.core.Hash; +import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -39,19 +40,14 @@ public class Pruner { private long transientForkThreshold; public Pruner( - final MarkSweepPruner pruningStrategy, - final Blockchain blockchain, + final MarkSweepPruner pruningStrategy, + final Blockchain blockchain, + final ExecutorService executor, final long retentionPeriodInBlocks, - final long transientForkThreshold) { + final long transientForkThreshold) { this.pruningStrategy = pruningStrategy; this.blockchain = blockchain; - this.executor = - Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder() - .setDaemon(true) - .setPriority(Thread.MIN_PRIORITY) - .setNameFormat("StatePruning-%d") - .build()); + this.executor = executor; this.retentionPeriodInBlocks = retentionPeriodInBlocks; this.transientForkThreshold = transientForkThreshold; } diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 78914d6c30..668157052b 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -53,7 +53,7 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { metricsSystem); // start pruner so it can start handling block added events - final Pruner pruner = new Pruner(markSweepPruner, blockchain, 0, 0); + final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 0); pruner.start(); /* diff --git a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/DeterministicEthScheduler.java b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/DeterministicEthScheduler.java index 6073bb7ee1..eac8b55102 100644 --- a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/DeterministicEthScheduler.java +++ b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/DeterministicEthScheduler.java @@ -12,6 +12,8 @@ */ package tech.pegasys.pantheon.ethereum.eth.manager; +import tech.pegasys.pantheon.testutil.MockExecutorService; + import java.time.Duration; import java.util.Arrays; import java.util.List; diff --git a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/EthSchedulerTest.java b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/EthSchedulerTest.java index dfcbc08131..6eb273a8aa 100644 --- a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/EthSchedulerTest.java +++ b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/EthSchedulerTest.java @@ -19,6 +19,8 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import tech.pegasys.pantheon.testutil.MockExecutorService; + import java.time.Duration; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; diff --git a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/MockScheduledExecutor.java b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/MockScheduledExecutor.java index 48172114c8..650476e050 100644 --- a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/MockScheduledExecutor.java +++ b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/MockScheduledExecutor.java @@ -12,6 +12,8 @@ */ package tech.pegasys.pantheon.ethereum.eth.manager; +import tech.pegasys.pantheon.testutil.MockExecutorService; + import java.util.concurrent.Callable; import java.util.concurrent.Delayed; import java.util.concurrent.ExecutionException; diff --git a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java index 10d367bc3c..78a6b86f16 100644 --- a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java +++ b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java @@ -38,7 +38,6 @@ import tech.pegasys.pantheon.ethereum.eth.manager.EthProtocolManager; import tech.pegasys.pantheon.ethereum.eth.manager.EthProtocolManagerTestUtil; import tech.pegasys.pantheon.ethereum.eth.manager.EthScheduler; -import tech.pegasys.pantheon.ethereum.eth.manager.MockExecutorService; import tech.pegasys.pantheon.ethereum.eth.manager.RespondingEthPeer; import tech.pegasys.pantheon.ethereum.eth.manager.RespondingEthPeer.Responder; import tech.pegasys.pantheon.ethereum.eth.messages.EthPV63; @@ -60,6 +59,7 @@ import tech.pegasys.pantheon.services.kvstore.InMemoryKeyValueStorage; import tech.pegasys.pantheon.services.tasks.CachingTaskCollection; import tech.pegasys.pantheon.services.tasks.InMemoryTaskQueue; +import tech.pegasys.pantheon.testutil.MockExecutorService; import tech.pegasys.pantheon.testutil.TestClock; import tech.pegasys.pantheon.util.bytes.Bytes32; import tech.pegasys.pantheon.util.bytes.BytesValue; diff --git a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java index 0d692461ff..178b699ad3 100644 --- a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java +++ b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java @@ -58,7 +58,9 @@ import java.util.Collections; import java.util.List; import java.util.OptionalLong; +import java.util.concurrent.Executors; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -208,9 +210,22 @@ public PantheonController build() throws IOException { storageProvider.createPruningStorage(), metricsSystem), protocolContext.getBlockchain(), + Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setPriority(Thread.MIN_PRIORITY) + .setNameFormat("StatePruning-%d") + .build()), 1000, 10); - addShutdownAction(pruner::stop); + addShutdownAction( + () -> { + try { + pruner.stop(); + } catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + }); final boolean fastSyncEnabled = syncConfig.getSyncMode().equals(SyncMode.FAST); ethProtocolManager = createEthProtocolManager(protocolContext, fastSyncEnabled); diff --git a/testutil/build.gradle b/testutil/build.gradle index 7fcad75c79..b72215da56 100644 --- a/testutil/build.gradle +++ b/testutil/build.gradle @@ -30,4 +30,5 @@ dependencies { implementation 'com.google.guava:guava' implementation 'com.squareup.okhttp3:okhttp' implementation 'net.consensys:orion' + implementation 'org.mockito:mockito-core' } diff --git a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/MockExecutorService.java b/testutil/src/main/java/tech/pegasys/pantheon/testutil/MockExecutorService.java similarity index 97% rename from ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/MockExecutorService.java rename to testutil/src/main/java/tech/pegasys/pantheon/testutil/MockExecutorService.java index f8f62b481c..edc7b61daa 100644 --- a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/manager/MockExecutorService.java +++ b/testutil/src/main/java/tech/pegasys/pantheon/testutil/MockExecutorService.java @@ -10,9 +10,7 @@ * 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. */ -package tech.pegasys.pantheon.ethereum.eth.manager; - -import static org.mockito.Mockito.spy; +package tech.pegasys.pantheon.testutil; import java.util.ArrayList; import java.util.Collection; @@ -27,6 +25,8 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; +import org.mockito.Mockito; + public class MockExecutorService implements ExecutorService { private boolean autoRun = true; @@ -146,7 +146,7 @@ private static class ExecutorTask { private boolean isPending = true; private ExecutorTask(final Callable taskRunner) { - this.future = spy(new CompletableFuture<>()); + this.future = Mockito.spy(new CompletableFuture<>()); this.taskRunner = taskRunner; } From e4632a46f3fcc6d1260a8f9c981148691fceb7bb Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 16 Jul 2019 10:45:35 -0400 Subject: [PATCH 26/53] add happy path testing --- .../pantheon/ethereum/worldstate/Pruner.java | 24 ++++++------ .../ethereum/worldstate/PrunerTest.java | 38 +++++++++++++++---- 2 files changed, 43 insertions(+), 19 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 7d2cfa6587..d98ab0d983 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -17,13 +17,10 @@ import tech.pegasys.pantheon.ethereum.core.BlockHeader; import tech.pegasys.pantheon.ethereum.core.Hash; -import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -40,11 +37,11 @@ public class Pruner { private long transientForkThreshold; public Pruner( - final MarkSweepPruner pruningStrategy, - final Blockchain blockchain, - final ExecutorService executor, + final MarkSweepPruner pruningStrategy, + final Blockchain blockchain, + final ExecutorService executor, final long retentionPeriodInBlocks, - final long transientForkThreshold) { + final long transientForkThreshold) { this.pruningStrategy = pruningStrategy; this.blockchain = blockchain; this.executor = executor; @@ -64,15 +61,20 @@ private void handleNewBlock(final BlockAddedEvent event) { if (!event.isNewCanonicalHead()) { return; } - final BlockHeader header = event.getBlock().getHeader(); + + final long blockNumber = event.getBlock().getHeader().getNumber(); if (state.compareAndSet(State.IDLE, State.TRANSIENT_FORK_OUTLIVING)) { pruningStrategy.prepare(); - markBlockNumber = header.getNumber(); - } else if (header.getNumber() > markBlockNumber + transientForkThreshold + markBlockNumber = blockNumber; + } + + if (blockNumber >= markBlockNumber + transientForkThreshold && state.compareAndSet(State.TRANSIENT_FORK_OUTLIVING, State.MARKING)) { markedBlockHeader = blockchain.getBlockHeader(markBlockNumber).get(); mark(markedBlockHeader); - } else if (header.getNumber() > markBlockNumber + retentionPeriodInBlocks + } + + if (blockNumber >= markBlockNumber + retentionPeriodInBlocks && blockchain.blockIsOnCanonicalChain(markedBlockHeader.getHash()) && state.compareAndSet(State.MARKING_COMPLETE, State.SWEEPING)) { sweep(); diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 668157052b..2e1bef6c19 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -12,12 +12,10 @@ */ package tech.pegasys.pantheon.ethereum.worldstate; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; -import org.junit.runner.RunWith; -import org.mockito.junit.MockitoJUnitRunner; +import tech.pegasys.pantheon.ethereum.chain.BlockchainStorage; import tech.pegasys.pantheon.ethereum.chain.DefaultMutableBlockchain; import tech.pegasys.pantheon.ethereum.core.Block; import tech.pegasys.pantheon.ethereum.core.BlockDataGenerator; @@ -27,11 +25,14 @@ import tech.pegasys.pantheon.ethereum.storage.keyvalue.KeyValueStoragePrefixedKeyBlockchainStorage; import tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem; import tech.pegasys.pantheon.services.kvstore.InMemoryKeyValueStorage; +import tech.pegasys.pantheon.testutil.MockExecutorService; import java.util.List; import org.junit.Test; +import org.junit.runner.RunWith; import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; @RunWith(MockitoJUnitRunner.class) public class PrunerTest { @@ -41,16 +42,37 @@ public class PrunerTest { private final BlockDataGenerator gen = new BlockDataGenerator(); @Mock private MarkSweepPruner markSweepPruner; + private final MockExecutorService mockExecutorService = new MockExecutorService(); + + @Test + public void shouldMarkCorrectBlockAndSweep() { + final Block genesisBlock = gen.genesisBlock(); + final BlockchainStorage blockchainStorage = + new KeyValueStoragePrefixedKeyBlockchainStorage( + new InMemoryKeyValueStorage(), new MainnetBlockHeaderFunctions()); + final DefaultMutableBlockchain blockchain = + new DefaultMutableBlockchain(genesisBlock, blockchainStorage, metricsSystem); + + final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 0); + pruner.start(); + + BlockOptions options = + new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); + final Block canonicalHead = gen.block(options); + List receipts = gen.receipts(canonicalHead); + blockchain.appendBlock(canonicalHead, receipts); + verify(markSweepPruner).mark(canonicalHead.getHeader().getStateRoot()); + verify(markSweepPruner).sweep(); + } @Test public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { final Block genesisBlock = gen.genesisBlock(); + final BlockchainStorage blockchainStorage = + new KeyValueStoragePrefixedKeyBlockchainStorage( + new InMemoryKeyValueStorage(), new MainnetBlockHeaderFunctions()); final DefaultMutableBlockchain blockchain = - new DefaultMutableBlockchain( - genesisBlock, - new KeyValueStoragePrefixedKeyBlockchainStorage( - new InMemoryKeyValueStorage(), new MainnetBlockHeaderFunctions()), - metricsSystem); + new DefaultMutableBlockchain(genesisBlock, blockchainStorage, metricsSystem); // start pruner so it can start handling block added events final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 0); From 5a129cfd7d71527a39eac4bbd7a543b5ba5fc2c1 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 16 Jul 2019 11:08:48 -0400 Subject: [PATCH 27/53] swap pruner parameters --- .../tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 4 ++-- .../pantheon/controller/PantheonControllerBuilder.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index d98ab0d983..091c13ea27 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -40,8 +40,8 @@ public Pruner( final MarkSweepPruner pruningStrategy, final Blockchain blockchain, final ExecutorService executor, - final long retentionPeriodInBlocks, - final long transientForkThreshold) { + final long transientForkThreshold, + final long retentionPeriodInBlocks) { this.pruningStrategy = pruningStrategy; this.blockchain = blockchain; this.executor = executor; diff --git a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java index 178b699ad3..69c8a70a07 100644 --- a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java +++ b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java @@ -216,8 +216,8 @@ public PantheonController build() throws IOException { .setPriority(Thread.MIN_PRIORITY) .setNameFormat("StatePruning-%d") .build()), - 1000, - 10); + 10, + 1000); addShutdownAction( () -> { try { From 47fdc47ff45d91fb75ea22550bdee29a44536171 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 16 Jul 2019 11:13:35 -0400 Subject: [PATCH 28/53] add transient fork and retention period test --- .../ethereum/worldstate/PrunerTest.java | 41 +++++++++++++++++-- 1 file changed, 37 insertions(+), 4 deletions(-) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 2e1bef6c19..0d21218276 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -20,6 +20,7 @@ import tech.pegasys.pantheon.ethereum.core.Block; import tech.pegasys.pantheon.ethereum.core.BlockDataGenerator; import tech.pegasys.pantheon.ethereum.core.BlockDataGenerator.BlockOptions; +import tech.pegasys.pantheon.ethereum.core.Hash; import tech.pegasys.pantheon.ethereum.core.TransactionReceipt; import tech.pegasys.pantheon.ethereum.mainnet.MainnetBlockHeaderFunctions; import tech.pegasys.pantheon.ethereum.storage.keyvalue.KeyValueStoragePrefixedKeyBlockchainStorage; @@ -58,10 +59,42 @@ public void shouldMarkCorrectBlockAndSweep() { BlockOptions options = new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); - final Block canonicalHead = gen.block(options); - List receipts = gen.receipts(canonicalHead); - blockchain.appendBlock(canonicalHead, receipts); - verify(markSweepPruner).mark(canonicalHead.getHeader().getStateRoot()); + final Block newBlock = gen.block(options); + List receipts = gen.receipts(newBlock); + blockchain.appendBlock(newBlock, receipts); + verify(markSweepPruner).mark(newBlock.getHeader().getStateRoot()); + verify(markSweepPruner).sweep(); + } + + @Test + public void shouldOnlySweepAfterTransientForkPeriodAndRetentionPeriodEnds() { + final Block genesisBlock = gen.genesisBlock(); + final BlockchainStorage blockchainStorage = + new KeyValueStoragePrefixedKeyBlockchainStorage( + new InMemoryKeyValueStorage(), new MainnetBlockHeaderFunctions()); + final DefaultMutableBlockchain blockchain = + new DefaultMutableBlockchain(genesisBlock, blockchainStorage, metricsSystem); + + final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 1, 2); + pruner.start(); + + BlockOptions options = + new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); + Block newBlock = gen.block(options); + List receipts = gen.receipts(newBlock); + blockchain.appendBlock(newBlock, receipts); + final Hash markBlockStateRootHash = newBlock.getHeader().getStateRoot(); + + options = options.setBlockNumber(2L).setParentHash(newBlock.getHash()); + newBlock = gen.block(options); + receipts = gen.receipts(newBlock); + blockchain.appendBlock(newBlock, receipts); + verify(markSweepPruner).mark(markBlockStateRootHash); + + options = options.setBlockNumber(3L).setParentHash(newBlock.getHash()); + newBlock = gen.block(options); + receipts = gen.receipts(newBlock); + blockchain.appendBlock(newBlock, receipts); verify(markSweepPruner).sweep(); } From 94062e806ef48a52de64d03fcf3907c2b45d35b2 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 16 Jul 2019 11:19:02 -0400 Subject: [PATCH 29/53] add negative checks to period testing --- .../tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 0d21218276..5d5affda7c 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -84,12 +84,15 @@ public void shouldOnlySweepAfterTransientForkPeriodAndRetentionPeriodEnds() { List receipts = gen.receipts(newBlock); blockchain.appendBlock(newBlock, receipts); final Hash markBlockStateRootHash = newBlock.getHeader().getStateRoot(); + verify(markSweepPruner, never()).mark(markBlockStateRootHash); + verify(markSweepPruner, never()).sweep(); options = options.setBlockNumber(2L).setParentHash(newBlock.getHash()); newBlock = gen.block(options); receipts = gen.receipts(newBlock); blockchain.appendBlock(newBlock, receipts); verify(markSweepPruner).mark(markBlockStateRootHash); + verify(markSweepPruner, never()).sweep(); options = options.setBlockNumber(3L).setParentHash(newBlock.getHash()); newBlock = gen.block(options); From d27d7cd84068bb4900f7994f1deaa9db45376251 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 16 Jul 2019 11:21:25 -0400 Subject: [PATCH 30/53] rename mayContainKey test to containsKey --- .../pantheon/services/kvstore/AbstractKeyValueStorageTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/services/kvstore/src/test/java/tech/pegasys/pantheon/services/kvstore/AbstractKeyValueStorageTest.java b/services/kvstore/src/test/java/tech/pegasys/pantheon/services/kvstore/AbstractKeyValueStorageTest.java index 839df858e9..e5a94ff93e 100644 --- a/services/kvstore/src/test/java/tech/pegasys/pantheon/services/kvstore/AbstractKeyValueStorageTest.java +++ b/services/kvstore/src/test/java/tech/pegasys/pantheon/services/kvstore/AbstractKeyValueStorageTest.java @@ -64,7 +64,7 @@ public void put() throws Exception { } @Test - public void mayContainKey() throws Exception { + public void containsKey() throws Exception { final KeyValueStorage store = createStore(); final BytesValue key = BytesValue.fromHexString("ABCD"); From ea2bc3805b78bf7a0c57cfd961a3f2b6135a8532 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 16 Jul 2019 11:57:36 -0400 Subject: [PATCH 31/53] Pruner IllegalArgumentException testing --- .../pantheon/ethereum/worldstate/Pruner.java | 7 +++++++ .../pantheon/ethereum/worldstate/PrunerTest.java | 14 ++++++++++++++ 2 files changed, 21 insertions(+) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 091c13ea27..acee1c1ec0 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -45,6 +45,13 @@ public Pruner( this.pruningStrategy = pruningStrategy; this.blockchain = blockchain; this.executor = executor; + if (!(transientForkThreshold >= 0 && retentionPeriodInBlocks >= transientForkThreshold)) { + throw new IllegalArgumentException( + String.format( + "Expected transientForkThreshold >= 0 and retentionPeriodInBlocks >= transientForkThreshold. transientForkThreshold={}, retentionPeriodInBlocks={}", + transientForkThreshold, + retentionPeriodInBlocks)); + } this.retentionPeriodInBlocks = retentionPeriodInBlocks; this.transientForkThreshold = transientForkThreshold; } diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 5d5affda7c..30d435ae25 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -12,9 +12,12 @@ */ package tech.pegasys.pantheon.ethereum.worldstate; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; +import tech.pegasys.pantheon.ethereum.chain.Blockchain; import tech.pegasys.pantheon.ethereum.chain.BlockchainStorage; import tech.pegasys.pantheon.ethereum.chain.DefaultMutableBlockchain; import tech.pegasys.pantheon.ethereum.core.Block; @@ -147,4 +150,15 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { blockchain.appendBlock(reorgCausingBlock, receipts); verify(markSweepPruner, never()).sweep(); } + + @Test + public void shouldThrowIllegalArgumentExceptionCorrectly() { + final Blockchain mockchain = mock(Blockchain.class); + assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, -1, -2)) + .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, 0, -1)) + .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, 1, 0)) + .isInstanceOf(IllegalArgumentException.class); + } } From f4de1b40d34a99dfb6d0a92ad0a704ffcb9ae41f Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 16 Jul 2019 12:03:33 -0400 Subject: [PATCH 32/53] fix exception string formatting --- .../java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index acee1c1ec0..371c487ec2 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -48,7 +48,7 @@ public Pruner( if (!(transientForkThreshold >= 0 && retentionPeriodInBlocks >= transientForkThreshold)) { throw new IllegalArgumentException( String.format( - "Expected transientForkThreshold >= 0 and retentionPeriodInBlocks >= transientForkThreshold. transientForkThreshold={}, retentionPeriodInBlocks={}", + "Expected transientForkThreshold >= 0 and retentionPeriodInBlocks >= transientForkThreshold. transientForkThreshold=%d, retentionPeriodInBlocks=%d", transientForkThreshold, retentionPeriodInBlocks)); } From fcb96b9727acc7759c53e6413d320a31b6b17771 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 16 Jul 2019 12:28:01 -0400 Subject: [PATCH 33/53] spotless --- .../java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 371c487ec2..513f8bd3c7 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -49,8 +49,7 @@ public Pruner( throw new IllegalArgumentException( String.format( "Expected transientForkThreshold >= 0 and retentionPeriodInBlocks >= transientForkThreshold. transientForkThreshold=%d, retentionPeriodInBlocks=%d", - transientForkThreshold, - retentionPeriodInBlocks)); + transientForkThreshold, retentionPeriodInBlocks)); } this.retentionPeriodInBlocks = retentionPeriodInBlocks; this.transientForkThreshold = transientForkThreshold; From cd2fbecd7e32960cd81bde14131a56fef8b3896d Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 16 Jul 2019 15:07:33 -0400 Subject: [PATCH 34/53] fix abort test after pruner state changing change --- .../tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 30d435ae25..b72f0eab39 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -114,7 +114,7 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { new DefaultMutableBlockchain(genesisBlock, blockchainStorage, metricsSystem); // start pruner so it can start handling block added events - final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 0); + final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 1); pruner.start(); /* @@ -148,6 +148,7 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { final Block reorgCausingBlock = gen.block(options); receipts = gen.receipts(reorgCausingBlock); blockchain.appendBlock(reorgCausingBlock, receipts); + verify(markSweepPruner).mark(canonicalHead.getHeader().getStateRoot()); verify(markSweepPruner, never()).sweep(); } From e57cd0042b69ccedb755f9a06705afd75da0693a Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 23 Jul 2019 11:34:32 -0400 Subject: [PATCH 35/53] build ExecutorService on demand --- .../worldstate/ExecutorServiceBuilder.java | 19 +++++++++++++++++++ .../pantheon/ethereum/worldstate/Pruner.java | 12 ++++++++---- .../ethereum/worldstate/PrunerTest.java | 17 ++++++++++------- .../controller/PantheonControllerBuilder.java | 13 +++++++------ 4 files changed, 44 insertions(+), 17 deletions(-) create mode 100644 ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/ExecutorServiceBuilder.java diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/ExecutorServiceBuilder.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/ExecutorServiceBuilder.java new file mode 100644 index 0000000000..63619fceca --- /dev/null +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/ExecutorServiceBuilder.java @@ -0,0 +1,19 @@ +/* + * Copyright 2019 ConsenSys AG. + * + * 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. + */ +package tech.pegasys.pantheon.ethereum.worldstate; + +import java.util.concurrent.ExecutorService; + +public interface ExecutorServiceBuilder { + ExecutorService build(); +} diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 513f8bd3c7..819abbcb8b 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -29,7 +29,8 @@ public class Pruner { private final MarkSweepPruner pruningStrategy; private final Blockchain blockchain; - private final ExecutorService executor; + private final ExecutorServiceBuilder executorServiceBuilder; + private ExecutorService executor; private final long retentionPeriodInBlocks; private final AtomicReference state = new AtomicReference<>(State.IDLE); private volatile long markBlockNumber = 0; @@ -39,12 +40,12 @@ public class Pruner { public Pruner( final MarkSweepPruner pruningStrategy, final Blockchain blockchain, - final ExecutorService executor, + final ExecutorServiceBuilder executorServiceBuilder, final long transientForkThreshold, final long retentionPeriodInBlocks) { this.pruningStrategy = pruningStrategy; + this.executorServiceBuilder = executorServiceBuilder; this.blockchain = blockchain; - this.executor = executor; if (!(transientForkThreshold >= 0 && retentionPeriodInBlocks >= transientForkThreshold)) { throw new IllegalArgumentException( String.format( @@ -56,11 +57,14 @@ public Pruner( } public void start() { + executor = executorServiceBuilder.build(); blockchain.observeBlockAdded((event, blockchain) -> handleNewBlock(event)); } public void stop() throws InterruptedException { - executor.awaitTermination(30, TimeUnit.SECONDS); + if (executor != null) { + executor.awaitTermination(30, TimeUnit.SECONDS); + } } private void handleNewBlock(final BlockAddedEvent event) { diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index b72f0eab39..f727be1af4 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -46,7 +46,7 @@ public class PrunerTest { private final BlockDataGenerator gen = new BlockDataGenerator(); @Mock private MarkSweepPruner markSweepPruner; - private final MockExecutorService mockExecutorService = new MockExecutorService(); + private final ExecutorServiceBuilder mockExecutorServiceBuilder = MockExecutorService::new; @Test public void shouldMarkCorrectBlockAndSweep() { @@ -57,7 +57,7 @@ public void shouldMarkCorrectBlockAndSweep() { final DefaultMutableBlockchain blockchain = new DefaultMutableBlockchain(genesisBlock, blockchainStorage, metricsSystem); - final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 0); + final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorServiceBuilder, 0, 0); pruner.start(); BlockOptions options = @@ -78,7 +78,7 @@ public void shouldOnlySweepAfterTransientForkPeriodAndRetentionPeriodEnds() { final DefaultMutableBlockchain blockchain = new DefaultMutableBlockchain(genesisBlock, blockchainStorage, metricsSystem); - final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 1, 2); + final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorServiceBuilder, 1, 2); pruner.start(); BlockOptions options = @@ -114,7 +114,7 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { new DefaultMutableBlockchain(genesisBlock, blockchainStorage, metricsSystem); // start pruner so it can start handling block added events - final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 1); + final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorServiceBuilder, 0, 1); pruner.start(); /* @@ -155,11 +155,14 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { @Test public void shouldThrowIllegalArgumentExceptionCorrectly() { final Blockchain mockchain = mock(Blockchain.class); - assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, -1, -2)) + assertThatThrownBy( + () -> new Pruner(markSweepPruner, mockchain, mockExecutorServiceBuilder, -1, -2)) .isInstanceOf(IllegalArgumentException.class); - assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, 0, -1)) + assertThatThrownBy( + () -> new Pruner(markSweepPruner, mockchain, mockExecutorServiceBuilder, 0, -1)) .isInstanceOf(IllegalArgumentException.class); - assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, 1, 0)) + assertThatThrownBy( + () -> new Pruner(markSweepPruner, mockchain, mockExecutorServiceBuilder, 1, 0)) .isInstanceOf(IllegalArgumentException.class); } } diff --git a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java index 69c8a70a07..c038fb9bda 100644 --- a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java +++ b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java @@ -210,12 +210,13 @@ public PantheonController build() throws IOException { storageProvider.createPruningStorage(), metricsSystem), protocolContext.getBlockchain(), - Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder() - .setDaemon(true) - .setPriority(Thread.MIN_PRIORITY) - .setNameFormat("StatePruning-%d") - .build()), + () -> + Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setPriority(Thread.MIN_PRIORITY) + .setNameFormat("StatePruning-%d") + .build()), 10, 1000); addShutdownAction( From 18de8a9febecaeb28cae3225ed588fa3e6f450ed Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 23 Jul 2019 11:41:38 -0400 Subject: [PATCH 36/53] rename test --- .../tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index f727be1af4..4ac124e9b1 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -153,7 +153,7 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { } @Test - public void shouldThrowIllegalArgumentExceptionCorrectly() { + public void shouldRejectInvalidArguments() { final Blockchain mockchain = mock(Blockchain.class); assertThatThrownBy( () -> new Pruner(markSweepPruner, mockchain, mockExecutorServiceBuilder, -1, -2)) From 71f2c61b1edf3acf65d78e43accb5859cd3e1f69 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 23 Jul 2019 12:53:27 -0400 Subject: [PATCH 37/53] add pruner stopping to tests --- .../pantheon/ethereum/worldstate/PrunerTest.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 4ac124e9b1..cc62e26b4a 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -49,7 +49,7 @@ public class PrunerTest { private final ExecutorServiceBuilder mockExecutorServiceBuilder = MockExecutorService::new; @Test - public void shouldMarkCorrectBlockAndSweep() { + public void shouldMarkCorrectBlockAndSweep() throws InterruptedException { final Block genesisBlock = gen.genesisBlock(); final BlockchainStorage blockchainStorage = new KeyValueStoragePrefixedKeyBlockchainStorage( @@ -67,10 +67,12 @@ public void shouldMarkCorrectBlockAndSweep() { blockchain.appendBlock(newBlock, receipts); verify(markSweepPruner).mark(newBlock.getHeader().getStateRoot()); verify(markSweepPruner).sweep(); + pruner.stop(); } @Test - public void shouldOnlySweepAfterTransientForkPeriodAndRetentionPeriodEnds() { + public void shouldOnlySweepAfterTransientForkPeriodAndRetentionPeriodEnds() + throws InterruptedException { final Block genesisBlock = gen.genesisBlock(); final BlockchainStorage blockchainStorage = new KeyValueStoragePrefixedKeyBlockchainStorage( @@ -102,10 +104,12 @@ public void shouldOnlySweepAfterTransientForkPeriodAndRetentionPeriodEnds() { receipts = gen.receipts(newBlock); blockchain.appendBlock(newBlock, receipts); verify(markSweepPruner).sweep(); + pruner.stop(); } @Test - public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { + public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() + throws InterruptedException { final Block genesisBlock = gen.genesisBlock(); final BlockchainStorage blockchainStorage = new KeyValueStoragePrefixedKeyBlockchainStorage( @@ -150,6 +154,7 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() { blockchain.appendBlock(reorgCausingBlock, receipts); verify(markSweepPruner).mark(canonicalHead.getHeader().getStateRoot()); verify(markSweepPruner, never()).sweep(); + pruner.stop(); } @Test From 4367f7f4b2d8a9986196017343b5e19d200aeb32 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 23 Jul 2019 14:05:34 -0400 Subject: [PATCH 38/53] give unloading responsibility to AllNodesVisitor --- .../worldstate/DefaultMutableWorldState.java | 4 ++-- .../ethereum/worldstate/MarkSweepPruner.java | 6 ++---- .../worldstate/WorldStateDownloaderTest.java | 3 +-- .../ethereum/trie/AllNodesVisitor.java | 11 +++++++--- .../pegasys/pantheon/ethereum/trie/Node.java | 3 +++ .../trie/StoredMerklePatriciaTrie.java | 20 +++++-------------- .../pantheon/ethereum/trie/StoredNode.java | 11 +++++----- .../ethereum/trie/StoredNodeFactory.java | 9 +++------ .../ethereum/trie/TrieNodeDecoder.java | 5 ++--- .../trie/StoredMerklePatriciaTrieTest.java | 16 +++++++-------- 10 files changed, 39 insertions(+), 49 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/DefaultMutableWorldState.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/DefaultMutableWorldState.java index e009d9aea7..df77dfb3e4 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/DefaultMutableWorldState.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/DefaultMutableWorldState.java @@ -73,12 +73,12 @@ public DefaultMutableWorldState(final WorldState worldState) { private MerklePatriciaTrie newAccountStateTrie(final Bytes32 rootHash) { return new StoredMerklePatriciaTrie<>( - worldStateStorage::getAccountStateTrieNode, rootHash, b -> b, b -> b, false); + worldStateStorage::getAccountStateTrieNode, rootHash, b -> b, b -> b); } private MerklePatriciaTrie newAccountStorageTrie(final Bytes32 rootHash) { return new StoredMerklePatriciaTrie<>( - worldStateStorage::getAccountStorageTrieNode, rootHash, b -> b, b -> b, false); + worldStateStorage::getAccountStorageTrieNode, rootHash, b -> b, b -> b); } @Override diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java index 484563a7d5..5ed82871c9 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java @@ -116,8 +116,7 @@ private MerklePatriciaTrie createStateTrie(final Bytes32 ro worldStateStorage::getAccountStateTrieNode, rootHash, Function.identity(), - Function.identity(), - true); + Function.identity()); } private MerklePatriciaTrie createStorageTrie(final Bytes32 rootHash) { @@ -125,8 +124,7 @@ private MerklePatriciaTrie createStorageTrie(final Bytes32 worldStateStorage::getAccountStorageTrieNode, rootHash, Function.identity(), - Function.identity(), - true); + Function.identity()); } private void processAccountState(final BytesValue value) { diff --git a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java index 78a6b86f16..2163001369 100644 --- a/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java +++ b/ethereum/eth/src/test/java/tech/pegasys/pantheon/ethereum/eth/sync/worldstate/WorldStateDownloaderTest.java @@ -534,8 +534,7 @@ public void doesNotRequestKnownStorageTrieNodesFromNetwork() { remoteStorage::getNodeData, remoteWorldState.rootHash(), Function.identity(), - Function.identity(), - true) + Function.identity()) .entriesFrom(Bytes32.ZERO, 5).values().stream() .map(RLP::input) .map(StateTrieAccountValue::readFrom) diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/AllNodesVisitor.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/AllNodesVisitor.java index dba9b9048b..3775e5f705 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/AllNodesVisitor.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/AllNodesVisitor.java @@ -18,21 +18,21 @@ public class AllNodesVisitor implements NodeVisitor { private final Consumer> handler; - public AllNodesVisitor(final Consumer> handler) { + AllNodesVisitor(final Consumer> handler) { this.handler = handler; } @Override public void visit(final ExtensionNode extensionNode) { handler.accept(extensionNode); - extensionNode.getChild().accept(this); + acceptAndUnload(extensionNode.getChild()); } @Override public void visit(final BranchNode branchNode) { handler.accept(branchNode); for (byte i = 0; i < BranchNode.RADIX; i++) { - branchNode.child(i).accept(this); + acceptAndUnload(branchNode.child(i)); } } @@ -43,4 +43,9 @@ public void visit(final LeafNode leafNode) { @Override public void visit(final NullNode nullNode) {} + + private void acceptAndUnload(final Node storedNode) { + storedNode.accept(this); + storedNode.unload(); + } } diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/Node.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/Node.java index e24eaaa4b5..47741f298a 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/Node.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/Node.java @@ -56,4 +56,7 @@ default boolean isReferencedByHash() { boolean isDirty(); String print(); + + /** Unloads the node if it is, for example, a StoredNode. */ + default void unload() {} } diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java index a45cf83f9c..70f24563c7 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java @@ -32,7 +32,6 @@ public class StoredMerklePatriciaTrie implements Merkle private final GetVisitor getVisitor = new GetVisitor<>(); private final RemoveVisitor removeVisitor = new RemoveVisitor<>(); private final StoredNodeFactory nodeFactory; - private final boolean singleUseNodes; private Node root; @@ -49,12 +48,7 @@ public StoredMerklePatriciaTrie( final Function valueSerializer, final Function valueDeserializer, final boolean singleUseNodes) { - this( - nodeLoader, - MerklePatriciaTrie.EMPTY_TRIE_NODE_HASH, - valueSerializer, - valueDeserializer, - singleUseNodes); + this(nodeLoader, MerklePatriciaTrie.EMPTY_TRIE_NODE_HASH, valueSerializer, valueDeserializer); } /** @@ -65,21 +59,17 @@ public StoredMerklePatriciaTrie( * storage}. * @param valueSerializer A function for serializing values to bytes. * @param valueDeserializer A function for deserializing values from bytes. - * @param singleUseNodes whether nodes should unload data after being visited */ public StoredMerklePatriciaTrie( final NodeLoader nodeLoader, final Bytes32 rootHash, final Function valueSerializer, - final Function valueDeserializer, - final boolean singleUseNodes) { - this.singleUseNodes = singleUseNodes; - this.nodeFactory = - new StoredNodeFactory<>(nodeLoader, valueSerializer, valueDeserializer, singleUseNodes); + final Function valueDeserializer) { + this.nodeFactory = new StoredNodeFactory<>(nodeLoader, valueSerializer, valueDeserializer); this.root = rootHash.equals(MerklePatriciaTrie.EMPTY_TRIE_NODE_HASH) ? NullNode.instance() - : new StoredNode<>(nodeFactory, rootHash, singleUseNodes); + : new StoredNode<>(nodeFactory, rootHash); } @Override @@ -114,7 +104,7 @@ public void commit(final NodeUpdater nodeUpdater) { this.root = rootHash.equals(MerklePatriciaTrie.EMPTY_TRIE_NODE_HASH) ? NullNode.instance() - : new StoredNode<>(nodeFactory, rootHash, singleUseNodes); + : new StoredNode<>(nodeFactory, rootHash); } @Override diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java index 2ce1120fa4..db1a93daf3 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java @@ -22,13 +22,11 @@ class StoredNode implements Node { private final StoredNodeFactory nodeFactory; private final Bytes32 hash; - private final boolean singleUse; private Node loaded; - StoredNode(final StoredNodeFactory nodeFactory, final Bytes32 hash, final boolean singleUse) { + StoredNode(final StoredNodeFactory nodeFactory, final Bytes32 hash) { this.nodeFactory = nodeFactory; this.hash = hash; - this.singleUse = singleUse; } /** @return True if the node needs to be persisted. */ @@ -54,9 +52,6 @@ public Node accept(final PathNodeVisitor visitor, final BytesValue path) { public void accept(final NodeVisitor visitor) { final Node node = load(); node.accept(visitor); - if (singleUse) { - loaded = null; - } } @Override @@ -113,6 +108,10 @@ private Node load() { return loaded; } + public void unload() { + loaded = null; + } + @Override public String print() { if (loaded == null) { diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNodeFactory.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNodeFactory.java index 87b07f9cb1..41dca7855e 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNodeFactory.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNodeFactory.java @@ -33,17 +33,14 @@ class StoredNodeFactory implements NodeFactory { private final NodeLoader nodeLoader; private final Function valueSerializer; private final Function valueDeserializer; - private final boolean singleUseNodes; StoredNodeFactory( final NodeLoader nodeLoader, final Function valueSerializer, - final Function valueDeserializer, - final boolean singleUseNodes) { + final Function valueDeserializer) { this.nodeLoader = nodeLoader; this.valueSerializer = valueSerializer; this.valueDeserializer = valueDeserializer; - this.singleUseNodes = singleUseNodes; } @Override @@ -159,7 +156,7 @@ private Node decodeExtension( return new ExtensionNode<>(path, childNode, this); } else { final Bytes32 childHash = childRlp.readBytes32(); - final StoredNode childNode = new StoredNode<>(this, childHash, singleUseNodes); + final StoredNode childNode = new StoredNode<>(this, childHash); return new ExtensionNode<>(path, childNode, this); } } @@ -176,7 +173,7 @@ private BranchNode decodeBranch(final RLPInput nodeRLPs, final Supplier(this, childHash, singleUseNodes)); + children.add(new StoredNode<>(this, childHash)); } } diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoder.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoder.java index 12c742f705..5212e90df0 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoder.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoder.java @@ -30,8 +30,7 @@ public class TrieNodeDecoder { private static final StoredNodeFactory emptyNodeFactory = - new StoredNodeFactory<>( - (h) -> Optional.empty(), Function.identity(), Function.identity(), false); + new StoredNodeFactory<>((h) -> Optional.empty(), Function.identity(), Function.identity()); // Hide constructor for static utility class private TrieNodeDecoder() {} @@ -117,7 +116,7 @@ private static class BreadthFirstIterator implements Iterator> BreadthFirstIterator(final NodeLoader nodeLoader, final Bytes32 rootHash, final int maxDepth) { this.maxDepth = maxDepth; this.nodeFactory = - new StoredNodeFactory<>(nodeLoader, Function.identity(), Function.identity(), false); + new StoredNodeFactory<>(nodeLoader, Function.identity(), Function.identity()); nodeLoader.getNode(rootHash).map(TrieNodeDecoder::decode).ifPresent(currentNodes::add); } diff --git a/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java b/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java index b10ab4b64a..ca6711ef16 100644 --- a/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java +++ b/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java @@ -331,21 +331,21 @@ public void canReloadTrieFromHash() { // Create new tries from root hashes and check that we find expected values trie = new StoredMerklePatriciaTrie<>( - merkleStorage::get, hash1, valueSerializer, valueDeserializer, false); + merkleStorage::get, hash1, valueSerializer, valueDeserializer); assertThat(trie.get(key1)).isEqualTo(Optional.of("value1")); assertThat(trie.get(key2)).isEqualTo(Optional.empty()); assertThat(trie.get(key3)).isEqualTo(Optional.empty()); trie = new StoredMerklePatriciaTrie<>( - merkleStorage::get, hash2, valueSerializer, valueDeserializer, false); + merkleStorage::get, hash2, valueSerializer, valueDeserializer); assertThat(trie.get(key1)).isEqualTo(Optional.of("value1")); assertThat(trie.get(key2)).isEqualTo(Optional.of("value2")); assertThat(trie.get(key3)).isEqualTo(Optional.of("value3")); trie = new StoredMerklePatriciaTrie<>( - merkleStorage::get, hash3, valueSerializer, valueDeserializer, false); + merkleStorage::get, hash3, valueSerializer, valueDeserializer); assertThat(trie.get(key1)).isEqualTo(Optional.of("value4")); assertThat(trie.get(key2)).isEqualTo(Optional.of("value2")); assertThat(trie.get(key3)).isEqualTo(Optional.of("value3")); @@ -355,21 +355,21 @@ public void canReloadTrieFromHash() { final MerkleStorage newMerkleStorage = new KeyValueMerkleStorage(keyValueStore); trie = new StoredMerklePatriciaTrie<>( - newMerkleStorage::get, hash1, valueSerializer, valueDeserializer, false); + newMerkleStorage::get, hash1, valueSerializer, valueDeserializer); assertThat(trie.get(key1)).isEqualTo(Optional.of("value1")); assertThat(trie.get(key2)).isEqualTo(Optional.empty()); assertThat(trie.get(key3)).isEqualTo(Optional.empty()); trie = new StoredMerklePatriciaTrie<>( - newMerkleStorage::get, hash2, valueSerializer, valueDeserializer, false); + newMerkleStorage::get, hash2, valueSerializer, valueDeserializer); assertThat(trie.get(key1)).isEqualTo(Optional.of("value1")); assertThat(trie.get(key2)).isEqualTo(Optional.of("value2")); assertThat(trie.get(key3)).isEqualTo(Optional.of("value3")); trie = new StoredMerklePatriciaTrie<>( - newMerkleStorage::get, hash3, valueSerializer, valueDeserializer, false); + newMerkleStorage::get, hash3, valueSerializer, valueDeserializer); assertThat(trie.get(key1)).isEqualTo(Optional.of("value4")); assertThat(trie.get(key2)).isEqualTo(Optional.of("value2")); assertThat(trie.get(key3)).isEqualTo(Optional.of("value3")); @@ -391,7 +391,7 @@ public void shouldRetrieveStoredExtensionWithInlinedChild() { // Ensure the extension branch can be loaded correct with its inlined child. final Bytes32 rootHash = trie.getRootHash(); final StoredMerklePatriciaTrie newTrie = - new StoredMerklePatriciaTrie<>(merkleStorage::get, rootHash, b -> b, b -> b, false); + new StoredMerklePatriciaTrie<>(merkleStorage::get, rootHash, b -> b, b -> b); newTrie.get(BytesValue.fromHexString("0x0401")); } @@ -411,7 +411,7 @@ public void shouldInlineNodesInParentAcrossModifications() { final Bytes32 rootHash = trie.getRootHash(); final StoredMerklePatriciaTrie newTrie = - new StoredMerklePatriciaTrie<>(merkleStorage::get, rootHash, b -> b, b -> b, false); + new StoredMerklePatriciaTrie<>(merkleStorage::get, rootHash, b -> b, b -> b); newTrie.put(BytesValue.fromHexString("0x0800"), BytesValue.of(3)); newTrie.get(BytesValue.fromHexString("0x0401")); From 4280ecc45ee62cbd3da205c037554c0170302b98 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 23 Jul 2019 14:27:59 -0400 Subject: [PATCH 39/53] add override --- .../java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java | 1 + 1 file changed, 1 insertion(+) diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java index db1a93daf3..09c512418c 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredNode.java @@ -108,6 +108,7 @@ private Node load() { return loaded; } + @Override public void unload() { loaded = null; } From af061e9b5573ab763f0b23549fb724a93cf06959 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 23 Jul 2019 15:22:07 -0400 Subject: [PATCH 40/53] remove more references to singleUseNodes --- .../ethereum/trie/StoredMerklePatriciaTrie.java | 4 +--- .../ethereum/trie/StoredMerklePatriciaTrieTest.java | 8 +++----- .../pantheon/ethereum/trie/TrieNodeDecoderTest.java | 12 ++++-------- 3 files changed, 8 insertions(+), 16 deletions(-) diff --git a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java index 70f24563c7..78438cff60 100644 --- a/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java +++ b/ethereum/trie/src/main/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrie.java @@ -41,13 +41,11 @@ public class StoredMerklePatriciaTrie implements Merkle * @param nodeLoader The {@link NodeLoader} to retrieve node data from. * @param valueSerializer A function for serializing values to bytes. * @param valueDeserializer A function for deserializing values from bytes. - * @param singleUseNodes whether nodes should unload data after being visited */ public StoredMerklePatriciaTrie( final NodeLoader nodeLoader, final Function valueSerializer, - final Function valueDeserializer, - final boolean singleUseNodes) { + final Function valueDeserializer) { this(nodeLoader, MerklePatriciaTrie.EMPTY_TRIE_NODE_HASH, valueSerializer, valueDeserializer); } diff --git a/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java b/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java index ca6711ef16..cb8932d8d8 100644 --- a/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java +++ b/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/StoredMerklePatriciaTrieTest.java @@ -41,9 +41,7 @@ public void setup() { valueSerializer = value -> (value != null) ? BytesValue.wrap(value.getBytes(Charset.forName("UTF-8"))) : null; valueDeserializer = bytes -> new String(bytes.getArrayUnsafe(), Charset.forName("UTF-8")); - trie = - new StoredMerklePatriciaTrie<>( - merkleStorage::get, valueSerializer, valueDeserializer, false); + trie = new StoredMerklePatriciaTrie<>(merkleStorage::get, valueSerializer, valueDeserializer); } @Test @@ -380,7 +378,7 @@ public void shouldRetrieveStoredExtensionWithInlinedChild() { final KeyValueStorage keyValueStorage = new InMemoryKeyValueStorage(); final MerkleStorage merkleStorage = new KeyValueMerkleStorage(keyValueStorage); final StoredMerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>(merkleStorage::get, b -> b, b -> b, false); + new StoredMerklePatriciaTrie<>(merkleStorage::get, b -> b, b -> b); // Both of these can be inlined in its parent branch and the branch // itself can be inlined into its parent extension. @@ -402,7 +400,7 @@ public void shouldInlineNodesInParentAcrossModifications() { final KeyValueStorage keyValueStorage = new InMemoryKeyValueStorage(); final MerkleStorage merkleStorage = new KeyValueMerkleStorage(keyValueStorage); final StoredMerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>(merkleStorage::get, b -> b, b -> b, false); + new StoredMerklePatriciaTrie<>(merkleStorage::get, b -> b, b -> b); // Both of these can be inlined in its parent branch. trie.put(BytesValue.fromHexString("0x0400"), BytesValue.of(1)); diff --git a/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoderTest.java b/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoderTest.java index 86786e1166..1361722cdb 100644 --- a/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoderTest.java +++ b/ethereum/trie/src/test/java/tech/pegasys/pantheon/ethereum/trie/TrieNodeDecoderTest.java @@ -36,8 +36,7 @@ public void decodeNodes() { // Build a small trie MerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>( - storage::get, Function.identity(), Function.identity(), false); + new StoredMerklePatriciaTrie<>(storage::get, Function.identity(), Function.identity()); trie.put(BytesValue.fromHexString("0x100000"), BytesValue.of(1)); trie.put(BytesValue.fromHexString("0x200000"), BytesValue.of(2)); trie.put(BytesValue.fromHexString("0x300000"), BytesValue.of(3)); @@ -81,8 +80,7 @@ public void breadthFirstDecode_smallTrie() { // Build a small trie MerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>( - storage::get, Function.identity(), Function.identity(), false); + new StoredMerklePatriciaTrie<>(storage::get, Function.identity(), Function.identity()); trie.put(BytesValue.fromHexString("0x100000"), BytesValue.of(1)); trie.put(BytesValue.fromHexString("0x200000"), BytesValue.of(2)); trie.put(BytesValue.fromHexString("0x300000"), BytesValue.of(3)); @@ -155,8 +153,7 @@ public void breadthFirstDecode_partialTrie() { // Build a small trie MerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>( - fullStorage::get, Function.identity(), Function.identity(), false); + new StoredMerklePatriciaTrie<>(fullStorage::get, Function.identity(), Function.identity()); final Random random = new Random(1); for (int i = 0; i < 30; i++) { byte[] key = new byte[4]; @@ -198,8 +195,7 @@ public void breadthFirstDecode_singleNodeTrie() { final InMemoryKeyValueStorage storage = new InMemoryKeyValueStorage(); MerklePatriciaTrie trie = - new StoredMerklePatriciaTrie<>( - storage::get, Function.identity(), Function.identity(), false); + new StoredMerklePatriciaTrie<>(storage::get, Function.identity(), Function.identity()); trie.put(BytesValue.fromHexString("0x100000"), BytesValue.of(1)); // Save nodes to storage From 61645effb733c7d5d256f5506664cd735234bdc3 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 23 Jul 2019 15:26:44 -0400 Subject: [PATCH 41/53] remove unnecessary intellij refactoring --- .../tech/pegasys/pantheon/testutil/MockExecutorService.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/testutil/src/main/java/tech/pegasys/pantheon/testutil/MockExecutorService.java b/testutil/src/main/java/tech/pegasys/pantheon/testutil/MockExecutorService.java index edc7b61daa..f6d0758557 100644 --- a/testutil/src/main/java/tech/pegasys/pantheon/testutil/MockExecutorService.java +++ b/testutil/src/main/java/tech/pegasys/pantheon/testutil/MockExecutorService.java @@ -12,6 +12,8 @@ */ package tech.pegasys.pantheon.testutil; +import static org.mockito.Mockito.spy; + import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -25,8 +27,6 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; -import org.mockito.Mockito; - public class MockExecutorService implements ExecutorService { private boolean autoRun = true; @@ -146,7 +146,7 @@ private static class ExecutorTask { private boolean isPending = true; private ExecutorTask(final Callable taskRunner) { - this.future = Mockito.spy(new CompletableFuture<>()); + this.future = spy(new CompletableFuture<>()); this.taskRunner = taskRunner; } From 105ff70a2565f423a0134b63e377fb9fd7495775 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Thu, 1 Aug 2019 10:29:59 -0400 Subject: [PATCH 42/53] update to reflect WorldStatePreimage changes --- .../pantheon/ethereum/worldstate/MarkSweepPrunerTest.java | 6 +++++- .../pantheon/controller/PantheonControllerBuilder.java | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java index 3f39909b36..e509dbf631 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPrunerTest.java @@ -18,6 +18,7 @@ import tech.pegasys.pantheon.ethereum.core.Hash; import tech.pegasys.pantheon.ethereum.core.MutableWorldState; import tech.pegasys.pantheon.ethereum.storage.keyvalue.WorldStateKeyValueStorage; +import tech.pegasys.pantheon.ethereum.storage.keyvalue.WorldStatePreimageKeyValueStorage; import tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem; import tech.pegasys.pantheon.services.kvstore.InMemoryKeyValueStorage; import tech.pegasys.pantheon.util.bytes.BytesValue; @@ -39,7 +40,10 @@ public void shouldMarkAllNodesInCurrentWorldState() { final InMemoryKeyValueStorage markStorage = new InMemoryKeyValueStorage(); final InMemoryKeyValueStorage stateStorage = new InMemoryKeyValueStorage(); final WorldStateStorage worldStateStorage = new WorldStateKeyValueStorage(stateStorage); - final WorldStateArchive worldStateArchive = new WorldStateArchive(worldStateStorage); + final WorldStateArchive worldStateArchive = + new WorldStateArchive( + worldStateStorage, + new WorldStatePreimageKeyValueStorage(new InMemoryKeyValueStorage())); final MutableWorldState worldState = worldStateArchive.getMutable(); // Generate accounts and save corresponding state root diff --git a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java index b712570643..22a905f87b 100644 --- a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java +++ b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java @@ -206,7 +206,7 @@ public PantheonController build() throws IOException { final Pruner pruner = new Pruner( new MarkSweepPruner( - protocolContext.getWorldStateArchive().getStorage(), + protocolContext.getWorldStateArchive().getWorldStateStorage(), storageProvider.createPruningStorage(), metricsSystem), protocolContext.getBlockchain(), From 2cffb78f7d66536ba2c7eb2fbd40b6d96035499d Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Fri, 2 Aug 2019 10:33:32 -0400 Subject: [PATCH 43/53] revert ExecutorServiceBuilder change and reduce shutdown timeout --- .../worldstate/ExecutorServiceBuilder.java | 19 ------------------- .../pantheon/ethereum/worldstate/Pruner.java | 14 +++++--------- .../ethereum/worldstate/PrunerTest.java | 18 ++++++++---------- .../controller/PantheonControllerBuilder.java | 13 ++++++------- 4 files changed, 19 insertions(+), 45 deletions(-) delete mode 100644 ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/ExecutorServiceBuilder.java diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/ExecutorServiceBuilder.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/ExecutorServiceBuilder.java deleted file mode 100644 index 63619fceca..0000000000 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/ExecutorServiceBuilder.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Copyright 2019 ConsenSys AG. - * - * 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. - */ -package tech.pegasys.pantheon.ethereum.worldstate; - -import java.util.concurrent.ExecutorService; - -public interface ExecutorServiceBuilder { - ExecutorService build(); -} diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 819abbcb8b..288be7aa78 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -29,8 +29,7 @@ public class Pruner { private final MarkSweepPruner pruningStrategy; private final Blockchain blockchain; - private final ExecutorServiceBuilder executorServiceBuilder; - private ExecutorService executor; + private ExecutorService executorService; private final long retentionPeriodInBlocks; private final AtomicReference state = new AtomicReference<>(State.IDLE); private volatile long markBlockNumber = 0; @@ -40,11 +39,11 @@ public class Pruner { public Pruner( final MarkSweepPruner pruningStrategy, final Blockchain blockchain, - final ExecutorServiceBuilder executorServiceBuilder, + final ExecutorService executorService, final long transientForkThreshold, final long retentionPeriodInBlocks) { this.pruningStrategy = pruningStrategy; - this.executorServiceBuilder = executorServiceBuilder; + this.executorService = executorService; this.blockchain = blockchain; if (!(transientForkThreshold >= 0 && retentionPeriodInBlocks >= transientForkThreshold)) { throw new IllegalArgumentException( @@ -57,14 +56,11 @@ public Pruner( } public void start() { - executor = executorServiceBuilder.build(); blockchain.observeBlockAdded((event, blockchain) -> handleNewBlock(event)); } public void stop() throws InterruptedException { - if (executor != null) { - executor.awaitTermination(30, TimeUnit.SECONDS); - } + executorService.awaitTermination(10, TimeUnit.SECONDS); } private void handleNewBlock(final BlockAddedEvent event) { @@ -117,7 +113,7 @@ private void sweep() { private void execute(final Runnable action) { try { - executor.execute(action); + executorService.execute(action); } catch (final Throwable t) { LOG.error("Pruning failed", t); state.set(State.IDLE); diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index cc62e26b4a..d31bb5e6d0 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -32,6 +32,7 @@ import tech.pegasys.pantheon.testutil.MockExecutorService; import java.util.List; +import java.util.concurrent.ExecutorService; import org.junit.Test; import org.junit.runner.RunWith; @@ -46,7 +47,7 @@ public class PrunerTest { private final BlockDataGenerator gen = new BlockDataGenerator(); @Mock private MarkSweepPruner markSweepPruner; - private final ExecutorServiceBuilder mockExecutorServiceBuilder = MockExecutorService::new; + private final ExecutorService mockExecutorService = new MockExecutorService(); @Test public void shouldMarkCorrectBlockAndSweep() throws InterruptedException { @@ -57,7 +58,7 @@ public void shouldMarkCorrectBlockAndSweep() throws InterruptedException { final DefaultMutableBlockchain blockchain = new DefaultMutableBlockchain(genesisBlock, blockchainStorage, metricsSystem); - final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorServiceBuilder, 0, 0); + final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 0); pruner.start(); BlockOptions options = @@ -80,7 +81,7 @@ public void shouldOnlySweepAfterTransientForkPeriodAndRetentionPeriodEnds() final DefaultMutableBlockchain blockchain = new DefaultMutableBlockchain(genesisBlock, blockchainStorage, metricsSystem); - final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorServiceBuilder, 1, 2); + final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 1, 2); pruner.start(); BlockOptions options = @@ -118,7 +119,7 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() new DefaultMutableBlockchain(genesisBlock, blockchainStorage, metricsSystem); // start pruner so it can start handling block added events - final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorServiceBuilder, 0, 1); + final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 1); pruner.start(); /* @@ -160,14 +161,11 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() @Test public void shouldRejectInvalidArguments() { final Blockchain mockchain = mock(Blockchain.class); - assertThatThrownBy( - () -> new Pruner(markSweepPruner, mockchain, mockExecutorServiceBuilder, -1, -2)) + assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, -1, -2)) .isInstanceOf(IllegalArgumentException.class); - assertThatThrownBy( - () -> new Pruner(markSweepPruner, mockchain, mockExecutorServiceBuilder, 0, -1)) + assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, 0, -1)) .isInstanceOf(IllegalArgumentException.class); - assertThatThrownBy( - () -> new Pruner(markSweepPruner, mockchain, mockExecutorServiceBuilder, 1, 0)) + assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, 1, 0)) .isInstanceOf(IllegalArgumentException.class); } } diff --git a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java index 22a905f87b..7bbb1ea843 100644 --- a/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java +++ b/pantheon/src/main/java/tech/pegasys/pantheon/controller/PantheonControllerBuilder.java @@ -210,13 +210,12 @@ public PantheonController build() throws IOException { storageProvider.createPruningStorage(), metricsSystem), protocolContext.getBlockchain(), - () -> - Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder() - .setDaemon(true) - .setPriority(Thread.MIN_PRIORITY) - .setNameFormat("StatePruning-%d") - .build()), + Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setPriority(Thread.MIN_PRIORITY) + .setNameFormat("StatePruning-%d") + .build()), 10, 1000); addShutdownAction( From 8e9536979ad95e1b693d0cf0e1ca3561fc01b46c Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Fri, 2 Aug 2019 10:53:32 -0400 Subject: [PATCH 44/53] make pruner cleanup pruning strategy on shutdown --- .../ethereum/worldstate/MarkSweepPruner.java | 4 ++++ .../pantheon/ethereum/worldstate/Pruner.java | 1 + .../pantheon/ethereum/worldstate/PrunerTest.java | 15 +++++++++++++++ 3 files changed, 20 insertions(+) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java index 5ed82871c9..0b22516bef 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java @@ -83,6 +83,10 @@ public void prepare() { nodeAddedListenerId = worldStateStorage.addNodeAddedListener(this::markNewNodes); } + public void cleanup() { + worldStateStorage.removeNodeAddedListener(nodeAddedListenerId); + } + public void mark(final Hash rootHash) { markOperationCounter.inc(); markStorage.clear(); diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 288be7aa78..ee9b7af227 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -60,6 +60,7 @@ public void start() { } public void stop() throws InterruptedException { + pruningStrategy.cleanup(); executorService.awaitTermination(10, TimeUnit.SECONDS); } diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index d31bb5e6d0..f9b91d0781 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -168,4 +168,19 @@ public void shouldRejectInvalidArguments() { assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, 1, 0)) .isInstanceOf(IllegalArgumentException.class); } + + @Test + public void shouldCleanUpPruningStrategyOnShutdown() throws InterruptedException { + final Block genesisBlock = gen.genesisBlock(); + final BlockchainStorage blockchainStorage = + new KeyValueStoragePrefixedKeyBlockchainStorage( + new InMemoryKeyValueStorage(), new MainnetBlockHeaderFunctions()); + final DefaultMutableBlockchain blockchain = + new DefaultMutableBlockchain(genesisBlock, blockchainStorage, metricsSystem); + + final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 0); + pruner.start(); + pruner.stop(); + verify(markSweepPruner).cleanup(); + } } From 896b5e39dcd4785210f672f166407a42a8c0e3d2 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Fri, 2 Aug 2019 10:56:25 -0400 Subject: [PATCH 45/53] nest ifs to draw attention to side-effecting checks --- .../pantheon/ethereum/worldstate/Pruner.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index ee9b7af227..4f05db6ad5 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -75,16 +75,18 @@ private void handleNewBlock(final BlockAddedEvent event) { markBlockNumber = blockNumber; } - if (blockNumber >= markBlockNumber + transientForkThreshold - && state.compareAndSet(State.TRANSIENT_FORK_OUTLIVING, State.MARKING)) { - markedBlockHeader = blockchain.getBlockHeader(markBlockNumber).get(); - mark(markedBlockHeader); + if (blockNumber >= markBlockNumber + transientForkThreshold) { + if (state.compareAndSet(State.TRANSIENT_FORK_OUTLIVING, State.MARKING)) { + markedBlockHeader = blockchain.getBlockHeader(markBlockNumber).get(); + mark(markedBlockHeader); + } } if (blockNumber >= markBlockNumber + retentionPeriodInBlocks - && blockchain.blockIsOnCanonicalChain(markedBlockHeader.getHash()) - && state.compareAndSet(State.MARKING_COMPLETE, State.SWEEPING)) { - sweep(); + && blockchain.blockIsOnCanonicalChain(markedBlockHeader.getHash())) { + if (state.compareAndSet(State.MARKING_COMPLETE, State.SWEEPING)) { + sweep(); + } } } From 707ebcac7f8dfc4e87f4cb3d9af27b28f2912ab4 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Sat, 3 Aug 2019 08:52:42 -0400 Subject: [PATCH 46/53] add comment explaining interrupt handling --- .../pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java index 0b22516bef..f2cc13c231 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java @@ -94,7 +94,8 @@ public void mark(final Hash rootHash) { .visitAll( node -> { if (Thread.interrupted()) { - // TODO: Probably need a better abort process than this.... + // Since we don't expect to abort marking ourselves, + // our abort process consists only of handling interrupts throw new RuntimeException("Interrupted while marking"); } markNode(node.getHash()); From 3b7a020764409a17fa0f27927a0ffc1bd8f9280d Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Sat, 3 Aug 2019 09:07:41 -0400 Subject: [PATCH 47/53] remove unnecessary check to pending marks when sweeping because it will be flushed --- .../pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java index f2cc13c231..57cd35f34b 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/MarkSweepPruner.java @@ -108,8 +108,7 @@ public void sweep() { flushPendingMarks(); sweepOperationCounter.inc(); LOG.info("Sweeping unused nodes"); - final long prunedNodeCount = - worldStateStorage.prune(key -> pendingMarks.contains(key) || markStorage.containsKey(key)); + final long prunedNodeCount = worldStateStorage.prune(markStorage::containsKey); sweptNodesCounter.inc(prunedNodeCount); worldStateStorage.removeNodeAddedListener(nodeAddedListenerId); markStorage.clear(); From bc032b1d81f7b18aeec364091fca9af930adc2bb Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 6 Aug 2019 14:20:07 -0400 Subject: [PATCH 48/53] relax argument requirements --- .../tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 4 ++-- .../tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java | 4 ---- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 4f05db6ad5..7ef0c77ef8 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -45,10 +45,10 @@ public Pruner( this.pruningStrategy = pruningStrategy; this.executorService = executorService; this.blockchain = blockchain; - if (!(transientForkThreshold >= 0 && retentionPeriodInBlocks >= transientForkThreshold)) { + if (transientForkThreshold < 0 || retentionPeriodInBlocks < 0) { throw new IllegalArgumentException( String.format( - "Expected transientForkThreshold >= 0 and retentionPeriodInBlocks >= transientForkThreshold. transientForkThreshold=%d, retentionPeriodInBlocks=%d", + "TransientForkThreshold and RetentionPeriodInBlocks should be positive. transientForkThreshold=%d, retentionPeriodInBlocks=%d", transientForkThreshold, retentionPeriodInBlocks)); } this.retentionPeriodInBlocks = retentionPeriodInBlocks; diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index f9b91d0781..b18ba578f4 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -163,10 +163,6 @@ public void shouldRejectInvalidArguments() { final Blockchain mockchain = mock(Blockchain.class); assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, -1, -2)) .isInstanceOf(IllegalArgumentException.class); - assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, 0, -1)) - .isInstanceOf(IllegalArgumentException.class); - assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, 1, 0)) - .isInstanceOf(IllegalArgumentException.class); } @Test From 54ba517f92b3ccd1203979e1ff45762a42d0e461 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 6 Aug 2019 14:58:33 -0400 Subject: [PATCH 49/53] extract block appending test util method --- .../ethereum/worldstate/PrunerTest.java | 60 +++++++------------ 1 file changed, 23 insertions(+), 37 deletions(-) diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index b18ba578f4..99c3e99062 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -49,9 +49,10 @@ public class PrunerTest { @Mock private MarkSweepPruner markSweepPruner; private final ExecutorService mockExecutorService = new MockExecutorService(); + private final Block genesisBlock = gen.genesisBlock(); + @Test public void shouldMarkCorrectBlockAndSweep() throws InterruptedException { - final Block genesisBlock = gen.genesisBlock(); final BlockchainStorage blockchainStorage = new KeyValueStoragePrefixedKeyBlockchainStorage( new InMemoryKeyValueStorage(), new MainnetBlockHeaderFunctions()); @@ -61,11 +62,8 @@ public void shouldMarkCorrectBlockAndSweep() throws InterruptedException { final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 0); pruner.start(); - BlockOptions options = - new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); - final Block newBlock = gen.block(options); - List receipts = gen.receipts(newBlock); - blockchain.appendBlock(newBlock, receipts); + final Block newBlock = appendBlockWithParent(blockchain, genesisBlock); + verify(markSweepPruner).mark(newBlock.getHeader().getStateRoot()); verify(markSweepPruner).sweep(); pruner.stop(); @@ -74,7 +72,6 @@ public void shouldMarkCorrectBlockAndSweep() throws InterruptedException { @Test public void shouldOnlySweepAfterTransientForkPeriodAndRetentionPeriodEnds() throws InterruptedException { - final Block genesisBlock = gen.genesisBlock(); final BlockchainStorage blockchainStorage = new KeyValueStoragePrefixedKeyBlockchainStorage( new InMemoryKeyValueStorage(), new MainnetBlockHeaderFunctions()); @@ -84,26 +81,16 @@ public void shouldOnlySweepAfterTransientForkPeriodAndRetentionPeriodEnds() final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 1, 2); pruner.start(); - BlockOptions options = - new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); - Block newBlock = gen.block(options); - List receipts = gen.receipts(newBlock); - blockchain.appendBlock(newBlock, receipts); - final Hash markBlockStateRootHash = newBlock.getHeader().getStateRoot(); + final Hash markBlockStateRootHash = + appendBlockWithParent(blockchain, genesisBlock).getHeader().getStateRoot(); verify(markSweepPruner, never()).mark(markBlockStateRootHash); verify(markSweepPruner, never()).sweep(); - options = options.setBlockNumber(2L).setParentHash(newBlock.getHash()); - newBlock = gen.block(options); - receipts = gen.receipts(newBlock); - blockchain.appendBlock(newBlock, receipts); + appendBlockWithParent(blockchain, blockchain.getChainHeadBlock()); verify(markSweepPruner).mark(markBlockStateRootHash); verify(markSweepPruner, never()).sweep(); - options = options.setBlockNumber(3L).setParentHash(newBlock.getHash()); - newBlock = gen.block(options); - receipts = gen.receipts(newBlock); - blockchain.appendBlock(newBlock, receipts); + appendBlockWithParent(blockchain, blockchain.getChainHeadBlock()); verify(markSweepPruner).sweep(); pruner.stop(); } @@ -111,7 +98,6 @@ public void shouldOnlySweepAfterTransientForkPeriodAndRetentionPeriodEnds() @Test public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() throws InterruptedException { - final Block genesisBlock = gen.genesisBlock(); final BlockchainStorage blockchainStorage = new KeyValueStoragePrefixedKeyBlockchainStorage( new InMemoryKeyValueStorage(), new MainnetBlockHeaderFunctions()); @@ -129,16 +115,8 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() \/ O <--- the common ancestor when the reorg happens */ - BlockOptions options = - new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); - final Block canonicalHead = gen.block(options); - List receipts = gen.receipts(canonicalHead); - blockchain.appendBlock(canonicalHead, receipts); - - options = new BlockOptions().setBlockNumber(1L).setParentHash(genesisBlock.getHash()); - final Block forkBlock = gen.block(options); - receipts = gen.receipts(forkBlock); - blockchain.appendBlock(forkBlock, receipts); + final Block canonicalHead = appendBlockWithParent(blockchain, genesisBlock); + final Block forkBlock = appendBlockWithParent(blockchain, genesisBlock); /* Cause reorg: O <---- this block causes a reorg; this branch becomes canonical @@ -149,10 +127,7 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() \/ O */ - options = new BlockOptions().setBlockNumber(2L).setParentHash(forkBlock.getHash()); - final Block reorgCausingBlock = gen.block(options); - receipts = gen.receipts(reorgCausingBlock); - blockchain.appendBlock(reorgCausingBlock, receipts); + appendBlockWithParent(blockchain, forkBlock); verify(markSweepPruner).mark(canonicalHead.getHeader().getStateRoot()); verify(markSweepPruner, never()).sweep(); pruner.stop(); @@ -167,7 +142,6 @@ public void shouldRejectInvalidArguments() { @Test public void shouldCleanUpPruningStrategyOnShutdown() throws InterruptedException { - final Block genesisBlock = gen.genesisBlock(); final BlockchainStorage blockchainStorage = new KeyValueStoragePrefixedKeyBlockchainStorage( new InMemoryKeyValueStorage(), new MainnetBlockHeaderFunctions()); @@ -179,4 +153,16 @@ public void shouldCleanUpPruningStrategyOnShutdown() throws InterruptedException pruner.stop(); verify(markSweepPruner).cleanup(); } + + private Block appendBlockWithParent( + final DefaultMutableBlockchain blockchain, final Block parent) { + BlockOptions options = + new BlockOptions() + .setBlockNumber(parent.getHeader().getNumber() + 1) + .setParentHash(parent.getHash()); + final Block newBlock = gen.block(options); + final List receipts = gen.receipts(newBlock); + blockchain.appendBlock(newBlock, receipts); + return newBlock; + } } From aaa01916ee9090ab0180031e9bd69f086a006f2c Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 6 Aug 2019 15:30:52 -0400 Subject: [PATCH 50/53] fix IllegalArgumentException error message --- .../java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 7ef0c77ef8..0b1d1b4351 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -48,7 +48,7 @@ public Pruner( if (transientForkThreshold < 0 || retentionPeriodInBlocks < 0) { throw new IllegalArgumentException( String.format( - "TransientForkThreshold and RetentionPeriodInBlocks should be positive. transientForkThreshold=%d, retentionPeriodInBlocks=%d", + "TransientForkThreshold and RetentionPeriodInBlocks should be non-negative. transientForkThreshold=%d, retentionPeriodInBlocks=%d", transientForkThreshold, retentionPeriodInBlocks)); } this.retentionPeriodInBlocks = retentionPeriodInBlocks; From 5f3e57ae0282ac98060a22b5d7ab90d164c89a56 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 6 Aug 2019 16:16:50 -0400 Subject: [PATCH 51/53] revert cascading between states and fix tests to accomodate --- .../pantheon/ethereum/worldstate/Pruner.java | 22 ++++++--------- .../ethereum/worldstate/PrunerTest.java | 27 +++++++++++-------- 2 files changed, 24 insertions(+), 25 deletions(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 0b1d1b4351..ae12695caf 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -73,20 +73,14 @@ private void handleNewBlock(final BlockAddedEvent event) { if (state.compareAndSet(State.IDLE, State.TRANSIENT_FORK_OUTLIVING)) { pruningStrategy.prepare(); markBlockNumber = blockNumber; - } - - if (blockNumber >= markBlockNumber + transientForkThreshold) { - if (state.compareAndSet(State.TRANSIENT_FORK_OUTLIVING, State.MARKING)) { - markedBlockHeader = blockchain.getBlockHeader(markBlockNumber).get(); - mark(markedBlockHeader); - } - } - - if (blockNumber >= markBlockNumber + retentionPeriodInBlocks - && blockchain.blockIsOnCanonicalChain(markedBlockHeader.getHash())) { - if (state.compareAndSet(State.MARKING_COMPLETE, State.SWEEPING)) { - sweep(); - } + } else if (blockNumber >= markBlockNumber + transientForkThreshold + && state.compareAndSet(State.TRANSIENT_FORK_OUTLIVING, State.MARKING)) { + markedBlockHeader = blockchain.getBlockHeader(markBlockNumber).get(); + mark(markedBlockHeader); + } else if (blockNumber >= markBlockNumber + retentionPeriodInBlocks + && blockchain.blockIsOnCanonicalChain(markedBlockHeader.getHash()) + && state.compareAndSet(State.MARKING_COMPLETE, State.SWEEPING)) { + sweep(); } } diff --git a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java index 99c3e99062..bdd9d87dc6 100644 --- a/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java +++ b/ethereum/core/src/test/java/tech/pegasys/pantheon/ethereum/worldstate/PrunerTest.java @@ -62,9 +62,11 @@ public void shouldMarkCorrectBlockAndSweep() throws InterruptedException { final Pruner pruner = new Pruner(markSweepPruner, blockchain, mockExecutorService, 0, 0); pruner.start(); - final Block newBlock = appendBlockWithParent(blockchain, genesisBlock); + final Block block1 = appendBlockWithParent(blockchain, genesisBlock); + appendBlockWithParent(blockchain, block1); + appendBlockWithParent(blockchain, blockchain.getChainHeadBlock()); - verify(markSweepPruner).mark(newBlock.getHeader().getStateRoot()); + verify(markSweepPruner).mark(block1.getHeader().getStateRoot()); verify(markSweepPruner).sweep(); pruner.stop(); } @@ -110,25 +112,28 @@ public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain() /* Set up pre-marking state: - O <---- this is a fork as of now (non-canonical) - | O <- marking of the full state trie will begin at this block + O <---- marking of the this block's parent will begin when this block is added + | + | O <- this is a fork as of now (non-canonical) + O | <- this is the initially canonical block that will be marked \/ O <--- the common ancestor when the reorg happens */ - final Block canonicalHead = appendBlockWithParent(blockchain, genesisBlock); + final Block initiallyCanonicalBlock = appendBlockWithParent(blockchain, genesisBlock); + appendBlockWithParent(blockchain, initiallyCanonicalBlock); final Block forkBlock = appendBlockWithParent(blockchain, genesisBlock); /* Cause reorg: - O <---- this block causes a reorg; this branch becomes canonical - | - O <---- which means that state here is referring to nodes from the common ancestor block, - | which is not fully marked - | O <- because this was block at which marking began + Set up pre-marking state: + O + | O <---- this block causes a reorg; this branch becomes canonical + | O <---- which means that state here is referring to nodes from the common ancestor block, + O | <- because this was block at which marking began \/ O */ appendBlockWithParent(blockchain, forkBlock); - verify(markSweepPruner).mark(canonicalHead.getHeader().getStateRoot()); + verify(markSweepPruner).mark(initiallyCanonicalBlock.getHeader().getStateRoot()); verify(markSweepPruner, never()).sweep(); pruner.stop(); } From 1e01636669484433895096336228cd3e67d4d1c1 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Tue, 6 Aug 2019 22:08:53 -0400 Subject: [PATCH 52/53] use "must" instead of "should" in exception message --- .../java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index ae12695caf..3287121f97 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -48,7 +48,7 @@ public Pruner( if (transientForkThreshold < 0 || retentionPeriodInBlocks < 0) { throw new IllegalArgumentException( String.format( - "TransientForkThreshold and RetentionPeriodInBlocks should be non-negative. transientForkThreshold=%d, retentionPeriodInBlocks=%d", + "TransientForkThreshold and RetentionPeriodInBlocks must be non-negative. transientForkThreshold=%d, retentionPeriodInBlocks=%d", transientForkThreshold, retentionPeriodInBlocks)); } this.retentionPeriodInBlocks = retentionPeriodInBlocks; From 245433f29296d1176c330e73c7c5010243c501a8 Mon Sep 17 00:00:00 2001 From: Ratan Rai Sur Date: Wed, 7 Aug 2019 11:37:55 -0400 Subject: [PATCH 53/53] make executor Service final --- .../java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java index 3287121f97..285527f08f 100644 --- a/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java +++ b/ethereum/core/src/main/java/tech/pegasys/pantheon/ethereum/worldstate/Pruner.java @@ -29,7 +29,7 @@ public class Pruner { private final MarkSweepPruner pruningStrategy; private final Blockchain blockchain; - private ExecutorService executorService; + private final ExecutorService executorService; private final long retentionPeriodInBlocks; private final AtomicReference state = new AtomicReference<>(State.IDLE); private volatile long markBlockNumber = 0;