mirror of
https://github.com/vacp2p/linea-besu.git
synced 2026-01-08 04:33:56 -05:00
[PAN-2819] [PAN-2820] Mark Sweep Pruner
* Mark Sweep Pruner * add `unload` method on Node interface, which is a noop everywhere but on `StoredNode` Signed-off-by: Adrian Sutton <adrian.sutton@consensys.net>
This commit is contained in:
@@ -18,6 +18,7 @@ import tech.pegasys.pantheon.ethereum.privacy.PrivateStateStorage;
|
||||
import tech.pegasys.pantheon.ethereum.privacy.PrivateTransactionStorage;
|
||||
import tech.pegasys.pantheon.ethereum.worldstate.WorldStatePreimageStorage;
|
||||
import tech.pegasys.pantheon.ethereum.worldstate.WorldStateStorage;
|
||||
import tech.pegasys.pantheon.services.kvstore.KeyValueStorage;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
@@ -32,4 +33,6 @@ public interface StorageProvider extends Closeable {
|
||||
PrivateTransactionStorage createPrivateTransactionStorage();
|
||||
|
||||
PrivateStateStorage createPrivateStateStorage();
|
||||
|
||||
KeyValueStorage createPruningStorage();
|
||||
}
|
||||
|
||||
@@ -76,6 +76,7 @@ public class KeyValueStorageProvider implements StorageProvider {
|
||||
return new PrivateStateKeyValueStorage(privateStateStorage);
|
||||
}
|
||||
|
||||
@Override
|
||||
public KeyValueStorage createPruningStorage() {
|
||||
return pruningStorage;
|
||||
}
|
||||
|
||||
@@ -0,0 +1,181 @@
|
||||
/*
|
||||
* 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 volatile long nodeAddedListenerId;
|
||||
private final ReentrantLock markLock = new ReentrantLock(true);
|
||||
private final Set<BytesValue> 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);
|
||||
}
|
||||
|
||||
public void cleanup() {
|
||||
worldStateStorage.removeNodeAddedListener(nodeAddedListenerId);
|
||||
}
|
||||
|
||||
public void mark(final Hash rootHash) {
|
||||
markOperationCounter.inc();
|
||||
markStorage.clear();
|
||||
createStateTrie(rootHash)
|
||||
.visitAll(
|
||||
node -> {
|
||||
if (Thread.interrupted()) {
|
||||
// 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());
|
||||
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(markStorage::containsKey);
|
||||
sweptNodesCounter.inc(prunedNodeCount);
|
||||
worldStateStorage.removeNodeAddedListener(nodeAddedListenerId);
|
||||
markStorage.clear();
|
||||
LOG.info("Completed sweeping unused nodes");
|
||||
}
|
||||
|
||||
private MerklePatriciaTrie<Bytes32, BytesValue> createStateTrie(final Bytes32 rootHash) {
|
||||
return new StoredMerklePatriciaTrie<>(
|
||||
worldStateStorage::getAccountStateTrieNode,
|
||||
rootHash,
|
||||
Function.identity(),
|
||||
Function.identity());
|
||||
}
|
||||
|
||||
private MerklePatriciaTrie<Bytes32, BytesValue> createStorageTrie(final Bytes32 rootHash) {
|
||||
return new StoredMerklePatriciaTrie<>(
|
||||
worldStateStorage::getAccountStorageTrieNode,
|
||||
rootHash,
|
||||
Function.identity(),
|
||||
Function.identity());
|
||||
}
|
||||
|
||||
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<Bytes32> nodeHashes) {
|
||||
markedNodesCounter.inc(nodeHashes.size());
|
||||
markLock.lock();
|
||||
try {
|
||||
pendingMarks.addAll(nodeHashes);
|
||||
maybeFlushPendingMarks();
|
||||
} finally {
|
||||
markLock.unlock();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,127 @@
|
||||
/*
|
||||
* 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.BlockAddedEvent;
|
||||
import tech.pegasys.pantheon.ethereum.chain.Blockchain;
|
||||
import tech.pegasys.pantheon.ethereum.core.BlockHeader;
|
||||
import tech.pegasys.pantheon.ethereum.core.Hash;
|
||||
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
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 Blockchain blockchain;
|
||||
private final ExecutorService executorService;
|
||||
private final long retentionPeriodInBlocks;
|
||||
private final AtomicReference<State> state = new AtomicReference<>(State.IDLE);
|
||||
private volatile long markBlockNumber = 0;
|
||||
private volatile BlockHeader markedBlockHeader;
|
||||
private long transientForkThreshold;
|
||||
|
||||
public Pruner(
|
||||
final MarkSweepPruner pruningStrategy,
|
||||
final Blockchain blockchain,
|
||||
final ExecutorService executorService,
|
||||
final long transientForkThreshold,
|
||||
final long retentionPeriodInBlocks) {
|
||||
this.pruningStrategy = pruningStrategy;
|
||||
this.executorService = executorService;
|
||||
this.blockchain = blockchain;
|
||||
if (transientForkThreshold < 0 || retentionPeriodInBlocks < 0) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(
|
||||
"TransientForkThreshold and RetentionPeriodInBlocks must be non-negative. transientForkThreshold=%d, retentionPeriodInBlocks=%d",
|
||||
transientForkThreshold, retentionPeriodInBlocks));
|
||||
}
|
||||
this.retentionPeriodInBlocks = retentionPeriodInBlocks;
|
||||
this.transientForkThreshold = transientForkThreshold;
|
||||
}
|
||||
|
||||
public void start() {
|
||||
blockchain.observeBlockAdded((event, blockchain) -> handleNewBlock(event));
|
||||
}
|
||||
|
||||
public void stop() throws InterruptedException {
|
||||
pruningStrategy.cleanup();
|
||||
executorService.awaitTermination(10, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
private void handleNewBlock(final BlockAddedEvent event) {
|
||||
if (!event.isNewCanonicalHead()) {
|
||||
return;
|
||||
}
|
||||
|
||||
final long blockNumber = event.getBlock().getHeader().getNumber();
|
||||
if (state.compareAndSet(State.IDLE, State.TRANSIENT_FORK_OUTLIVING)) {
|
||||
pruningStrategy.prepare();
|
||||
markBlockNumber = blockNumber;
|
||||
} 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();
|
||||
}
|
||||
}
|
||||
|
||||
private void mark(final BlockHeader header) {
|
||||
markBlockNumber = header.getNumber();
|
||||
final Hash stateRoot = header.getStateRoot();
|
||||
LOG.info(
|
||||
"Begin marking used nodes for pruning. Block number: {} State root: {}",
|
||||
markBlockNumber,
|
||||
stateRoot);
|
||||
execute(
|
||||
() -> {
|
||||
pruningStrategy.mark(stateRoot);
|
||||
state.compareAndSet(State.MARKING, State.MARKING_COMPLETE);
|
||||
});
|
||||
}
|
||||
|
||||
private void sweep() {
|
||||
LOG.info(
|
||||
"Begin sweeping unused nodes for pruning. Retention period: {}", retentionPeriodInBlocks);
|
||||
execute(
|
||||
() -> {
|
||||
pruningStrategy.sweep();
|
||||
state.compareAndSet(State.SWEEPING, State.IDLE);
|
||||
});
|
||||
}
|
||||
|
||||
private void execute(final Runnable action) {
|
||||
try {
|
||||
executorService.execute(action);
|
||||
} catch (final Throwable t) {
|
||||
LOG.error("Pruning failed", t);
|
||||
state.set(State.IDLE);
|
||||
}
|
||||
}
|
||||
|
||||
private enum State {
|
||||
IDLE,
|
||||
TRANSIENT_FORK_OUTLIVING,
|
||||
MARKING,
|
||||
MARKING_COMPLETE,
|
||||
SWEEPING;
|
||||
}
|
||||
}
|
||||
@@ -32,6 +32,7 @@ import tech.pegasys.pantheon.ethereum.worldstate.WorldStatePreimageStorage;
|
||||
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 {
|
||||
|
||||
@@ -86,6 +87,11 @@ public class InMemoryStorageProvider implements StorageProvider {
|
||||
return new PrivateStateKeyValueStorage(new InMemoryKeyValueStorage());
|
||||
}
|
||||
|
||||
@Override
|
||||
public KeyValueStorage createPruningStorage() {
|
||||
return new InMemoryKeyValueStorage();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,70 @@
|
||||
/*
|
||||
* 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.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;
|
||||
|
||||
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,
|
||||
new WorldStatePreimageKeyValueStorage(new InMemoryKeyValueStorage()));
|
||||
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<BytesValue> 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);
|
||||
assertThat(markStorage.keySet()).isEmpty();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,173 @@
|
||||
/*
|
||||
* 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.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;
|
||||
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;
|
||||
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 java.util.concurrent.ExecutorService;
|
||||
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.MockitoJUnitRunner;
|
||||
|
||||
@RunWith(MockitoJUnitRunner.class)
|
||||
public class PrunerTest {
|
||||
|
||||
private final NoOpMetricsSystem metricsSystem = new NoOpMetricsSystem();
|
||||
|
||||
private final BlockDataGenerator gen = new BlockDataGenerator();
|
||||
|
||||
@Mock private MarkSweepPruner markSweepPruner;
|
||||
private final ExecutorService mockExecutorService = new MockExecutorService();
|
||||
|
||||
private final Block genesisBlock = gen.genesisBlock();
|
||||
|
||||
@Test
|
||||
public void shouldMarkCorrectBlockAndSweep() throws InterruptedException {
|
||||
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();
|
||||
|
||||
final Block block1 = appendBlockWithParent(blockchain, genesisBlock);
|
||||
appendBlockWithParent(blockchain, block1);
|
||||
appendBlockWithParent(blockchain, blockchain.getChainHeadBlock());
|
||||
|
||||
verify(markSweepPruner).mark(block1.getHeader().getStateRoot());
|
||||
verify(markSweepPruner).sweep();
|
||||
pruner.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldOnlySweepAfterTransientForkPeriodAndRetentionPeriodEnds()
|
||||
throws InterruptedException {
|
||||
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();
|
||||
|
||||
final Hash markBlockStateRootHash =
|
||||
appendBlockWithParent(blockchain, genesisBlock).getHeader().getStateRoot();
|
||||
verify(markSweepPruner, never()).mark(markBlockStateRootHash);
|
||||
verify(markSweepPruner, never()).sweep();
|
||||
|
||||
appendBlockWithParent(blockchain, blockchain.getChainHeadBlock());
|
||||
verify(markSweepPruner).mark(markBlockStateRootHash);
|
||||
verify(markSweepPruner, never()).sweep();
|
||||
|
||||
appendBlockWithParent(blockchain, blockchain.getChainHeadBlock());
|
||||
verify(markSweepPruner).sweep();
|
||||
pruner.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void abortsPruningWhenFullyMarkedBlockNoLongerOnCanonicalChain()
|
||||
throws InterruptedException {
|
||||
final BlockchainStorage blockchainStorage =
|
||||
new KeyValueStoragePrefixedKeyBlockchainStorage(
|
||||
new InMemoryKeyValueStorage(), new MainnetBlockHeaderFunctions());
|
||||
final DefaultMutableBlockchain blockchain =
|
||||
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);
|
||||
pruner.start();
|
||||
|
||||
/*
|
||||
Set up pre-marking state:
|
||||
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 initiallyCanonicalBlock = appendBlockWithParent(blockchain, genesisBlock);
|
||||
appendBlockWithParent(blockchain, initiallyCanonicalBlock);
|
||||
final Block forkBlock = appendBlockWithParent(blockchain, genesisBlock);
|
||||
/*
|
||||
Cause reorg:
|
||||
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(initiallyCanonicalBlock.getHeader().getStateRoot());
|
||||
verify(markSweepPruner, never()).sweep();
|
||||
pruner.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldRejectInvalidArguments() {
|
||||
final Blockchain mockchain = mock(Blockchain.class);
|
||||
assertThatThrownBy(() -> new Pruner(markSweepPruner, mockchain, mockExecutorService, -1, -2))
|
||||
.isInstanceOf(IllegalArgumentException.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldCleanUpPruningStrategyOnShutdown() throws InterruptedException {
|
||||
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();
|
||||
}
|
||||
|
||||
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<TransactionReceipt> receipts = gen.receipts(newBlock);
|
||||
blockchain.appendBlock(newBlock, receipts);
|
||||
return newBlock;
|
||||
}
|
||||
}
|
||||
@@ -26,6 +26,7 @@ import tech.pegasys.pantheon.ethereum.eth.sync.fullsync.FullSyncDownloader;
|
||||
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,8 @@ public class DefaultSynchronizer<C> 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<SyncStatusListener> syncStatusListeners = Subscribers.create();
|
||||
@@ -57,11 +60,13 @@ public class DefaultSynchronizer<C> implements Synchronizer {
|
||||
final ProtocolContext<C> 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 +169,9 @@ public class DefaultSynchronizer<C> implements Synchronizer {
|
||||
|
||||
private void startFullSync() {
|
||||
fullSyncDownloader.start();
|
||||
if (PRUNING_ENABLED) {
|
||||
pruner.start();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -19,6 +19,8 @@ import static org.mockito.ArgumentMatchers.eq;
|
||||
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;
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -40,7 +40,6 @@ import tech.pegasys.pantheon.ethereum.eth.manager.EthContext;
|
||||
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;
|
||||
@@ -64,6 +63,7 @@ import tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem;
|
||||
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;
|
||||
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* 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<V> implements NodeVisitor<V> {
|
||||
|
||||
private final Consumer<Node<V>> handler;
|
||||
|
||||
AllNodesVisitor(final Consumer<Node<V>> handler) {
|
||||
this.handler = handler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(final ExtensionNode<V> extensionNode) {
|
||||
handler.accept(extensionNode);
|
||||
acceptAndUnload(extensionNode.getChild());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(final BranchNode<V> branchNode) {
|
||||
handler.accept(branchNode);
|
||||
for (byte i = 0; i < BranchNode.RADIX; i++) {
|
||||
acceptAndUnload(branchNode.child(i));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(final LeafNode<V> leafNode) {
|
||||
handler.accept(leafNode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visit(final NullNode<V> nullNode) {}
|
||||
|
||||
private void acceptAndUnload(final Node<V> storedNode) {
|
||||
storedNode.accept(this);
|
||||
storedNode.unload();
|
||||
}
|
||||
}
|
||||
@@ -20,6 +20,7 @@ import tech.pegasys.pantheon.util.bytes.BytesValue;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
/** An Merkle Patricial Trie. */
|
||||
public interface MerklePatriciaTrie<K, V> {
|
||||
@@ -74,4 +75,6 @@ public interface MerklePatriciaTrie<K, V> {
|
||||
* @return the requested storage entries as a map of key hash to value.
|
||||
*/
|
||||
Map<Bytes32, V> entriesFrom(Bytes32 startKeyHash, int limit);
|
||||
|
||||
void visitAll(Consumer<Node<V>> visitor);
|
||||
}
|
||||
|
||||
@@ -56,4 +56,7 @@ public interface Node<V> {
|
||||
boolean isDirty();
|
||||
|
||||
String print();
|
||||
|
||||
/** Unloads the node if it is, for example, a StoredNode. */
|
||||
default void unload() {}
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ import tech.pegasys.pantheon.util.bytes.BytesValue;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
@@ -82,4 +83,9 @@ public class SimpleMerklePatriciaTrie<K extends BytesValue, V> implements Merkle
|
||||
public Map<Bytes32, V> entriesFrom(final Bytes32 startKeyHash, final int limit) {
|
||||
return StorageEntriesCollector.collectEntries(root, startKeyHash, limit);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visitAll(final Consumer<Node<V>> visitor) {
|
||||
root.accept(new AllNodesVisitor<>(visitor));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ import tech.pegasys.pantheon.util.bytes.BytesValue;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
@@ -109,6 +110,11 @@ public class StoredMerklePatriciaTrie<K extends BytesValue, V> implements Merkle
|
||||
return StorageEntriesCollector.collectEntries(root, startKeyHash, limit);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void visitAll(final Consumer<Node<V>> visitor) {
|
||||
root.accept(new AllNodesVisitor<>(visitor));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bytes32 getRootHash() {
|
||||
return root.getHash();
|
||||
|
||||
@@ -108,6 +108,11 @@ class StoredNode<V> implements Node<V> {
|
||||
return loaded;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unload() {
|
||||
loaded = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String print() {
|
||||
if (loaded == null) {
|
||||
|
||||
@@ -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");
|
||||
|
||||
@@ -44,6 +44,8 @@ import tech.pegasys.pantheon.ethereum.mainnet.ProtocolSchedule;
|
||||
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;
|
||||
@@ -56,7 +58,9 @@ import java.util.ArrayList;
|
||||
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;
|
||||
|
||||
@@ -199,6 +203,30 @@ public abstract class PantheonControllerBuilder<C> {
|
||||
|
||||
final MutableBlockchain blockchain = protocolContext.getBlockchain();
|
||||
|
||||
final Pruner pruner =
|
||||
new Pruner(
|
||||
new MarkSweepPruner(
|
||||
protocolContext.getWorldStateArchive().getWorldStateStorage(),
|
||||
storageProvider.createPruningStorage(),
|
||||
metricsSystem),
|
||||
protocolContext.getBlockchain(),
|
||||
Executors.newSingleThreadExecutor(
|
||||
new ThreadFactoryBuilder()
|
||||
.setDaemon(true)
|
||||
.setPriority(Thread.MIN_PRIORITY)
|
||||
.setNameFormat("StatePruning-%d")
|
||||
.build()),
|
||||
10,
|
||||
1000);
|
||||
addShutdownAction(
|
||||
() -> {
|
||||
try {
|
||||
pruner.stop();
|
||||
} catch (InterruptedException ie) {
|
||||
throw new RuntimeException(ie);
|
||||
}
|
||||
});
|
||||
|
||||
final boolean fastSyncEnabled = syncConfig.getSyncMode().equals(SyncMode.FAST);
|
||||
ethProtocolManager = createEthProtocolManager(protocolContext, fastSyncEnabled);
|
||||
final SyncState syncState =
|
||||
@@ -210,6 +238,7 @@ public abstract class PantheonControllerBuilder<C> {
|
||||
protocolContext,
|
||||
protocolContext.getWorldStateArchive().getWorldStateStorage(),
|
||||
ethProtocolManager.getBlockBroadcaster(),
|
||||
pruner,
|
||||
ethProtocolManager.ethContext(),
|
||||
syncState,
|
||||
dataDirectory,
|
||||
|
||||
@@ -14,6 +14,7 @@ package tech.pegasys.pantheon.services.kvstore;
|
||||
|
||||
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 abstract class AbstractKeyValueStorageTest {
|
||||
Optional.of(BytesValue.fromHexString("0DEF")), store.get(BytesValue.fromHexString("0F")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void containsKey() 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();
|
||||
|
||||
@@ -30,4 +30,5 @@ dependencies {
|
||||
implementation 'com.google.guava:guava'
|
||||
implementation 'com.squareup.okhttp3:okhttp'
|
||||
implementation 'net.consensys:orion'
|
||||
implementation 'org.mockito:mockito-core'
|
||||
}
|
||||
|
||||
@@ -10,7 +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;
|
||||
package tech.pegasys.pantheon.testutil;
|
||||
|
||||
import static org.mockito.Mockito.spy;
|
||||
|
||||
Reference in New Issue
Block a user