Skip to content

Commit

Permalink
[Pruning Bugfix] Prevent race condition in key deletion. (#760)
Browse files Browse the repository at this point in the history
* add doomed key check (busy-waiting for now)

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* optional and logging

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* remove logging

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* sleeping and hardening

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* rename segments

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* move away from atomic references to regular vars

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* remove hardened segment parameter

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* increase sleep

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* spotless

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* remove unnecessary interface

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* rename

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* move commit waiting outside of timer

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* set default lock timeout to 1ms

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* add default lock timeout to tests

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* Revert "rename segments"

This reverts commit 184eefa.

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* fix jmh compilation error

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* add documentation

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* bump up sleep to 1ms

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* (POC) Add lock to ensure that we don't prune while comitting

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* remove unnecessary persist (#569)

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* flesh out @mbaxter's idea and remove my code

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* iterator changes

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* hybridize with doomed key

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* comment

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* move doomed key unset to after node added listener

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* update instead of getting and setting doomedKeyRef in commit

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* comment

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* invert condition

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* remove locks

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* remove `removeAllKeysUnless`

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* more remove removeAllKeysUnless

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* reuse streamKeys

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* remove test

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* set default lock timeout to 1ms

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* add default lock timeout to tests

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* fix jmh compilation error

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* revert back to locks instead of doomedkey

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* change delete to not guarantee deletion

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* plugin hash

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* javadoc

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* Revert "change delete to not guarantee deletion"

This reverts commit 2289bb3.

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* skip key deletion on timeout

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* clear in rollback

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* Revert "fix jmh compilation error"

This reverts commit b64ecf8.

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* Revert "add default lock timeout to tests"

This reverts commit aff6aa6.

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* Revert "set default lock timeout to 1ms"

This reverts commit 267fe0a.

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* use noSlowDown write option instead of global timeout

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* add back tests

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* close tryDeleteOptions

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* remove unnecessary lock

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* move increment inside try

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* use StorageException subclass instead of field

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* revert accidental deletion in javadoc

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* tryDelete javadoc

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* add trace for skipping key deletion

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* merge catch and finally try blocks

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* switch from exception to boolean return value

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* tweak

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* changelog changes

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* add api back

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

* add back throws javadoc

Signed-off-by: Ratan Rai Sur <ratan.r.sur@gmail.com>

Co-authored-by: Meredith Baxter <meredith.baxter@consensys.net>
Co-authored-by: MadelineMurray <43356962+MadelineMurray@users.noreply.github.com>
  • Loading branch information
3 people authored May 6, 2020
1 parent 69f6493 commit 43eccbb
Show file tree
Hide file tree
Showing 14 changed files with 360 additions and 79 deletions.
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ permissions on the directory allow other users and groups to r/w. Ideally this s

### Bug Fixes

- MerkleTrieException when pruning should no longer occur. Using `--pruning-enabled=true` will no longer accidentaly delete keys in certain edge cases. [\#760](https://github.com/hyperledger/besu/pull/760).
- Full help not displayed unless explicitly requested. [\#437](https://github.com/hyperledger/besu/pull/437)
- Compatibility with undocumented Geth `eth_subscribe` fields [\#654](https://github.com/hyperledger/besu/pull/654)
- Current block number included as part of `eth_getWork` response [\#849](https://github.com/hyperledger/besu/pull/849)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,14 @@
import org.hyperledger.besu.plugin.services.storage.KeyValueStorageTransaction;
import org.hyperledger.besu.util.Subscribers;

import java.util.ArrayList;
import java.util.List;
import java.util.HashSet;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Predicate;
import java.util.stream.Stream;

import org.apache.tuweni.bytes.Bytes;
import org.apache.tuweni.bytes.Bytes32;
Expand All @@ -33,6 +37,7 @@ public class WorldStateKeyValueStorage implements WorldStateStorage {

private final Subscribers<NodesAddedListener> nodeAddedListeners = Subscribers.create();
private final KeyValueStorage keyValueStorage;
private final ReentrantLock lock = new ReentrantLock();

public WorldStateKeyValueStorage(final KeyValueStorage keyValueStorage) {
this.keyValueStorage = keyValueStorage;
Expand Down Expand Up @@ -83,12 +88,27 @@ public boolean isWorldStateAvailable(final Bytes32 rootHash) {

@Override
public Updater updater() {
return new Updater(keyValueStorage.startTransaction(), nodeAddedListeners);
return new Updater(lock, keyValueStorage.startTransaction(), nodeAddedListeners);
}

@Override
public long prune(final Predicate<byte[]> inUseCheck) {
return keyValueStorage.removeAllKeysUnless(inUseCheck);
final AtomicInteger prunedKeys = new AtomicInteger(0);
try (final Stream<byte[]> keys = keyValueStorage.streamKeys()) {
keys.forEach(
key -> {
lock.lock();
try {
if (!inUseCheck.test(key) && keyValueStorage.tryDelete(key)) {
prunedKeys.incrementAndGet();
}
} finally {
lock.unlock();
}
});
}

return prunedKeys.get();
}

@Override
Expand All @@ -105,11 +125,14 @@ public static class Updater implements WorldStateStorage.Updater {

private final KeyValueStorageTransaction transaction;
private final Subscribers<NodesAddedListener> nodeAddedListeners;
private final List<Bytes32> addedNodes = new ArrayList<>();
private final Set<Bytes32> addedNodes = new HashSet<>();
private final Lock lock;

public Updater(
final Lock lock,
final KeyValueStorageTransaction transaction,
final Subscribers<NodesAddedListener> nodeAddedListeners) {
this.lock = lock;
this.transaction = transaction;
this.nodeAddedListeners = nodeAddedListeners;
}
Expand Down Expand Up @@ -156,12 +179,18 @@ public Updater putAccountStorageTrieNode(final Bytes32 nodeHash, final Bytes nod

@Override
public void commit() {
nodeAddedListeners.forEach(listener -> listener.onNodesAdded(addedNodes));
transaction.commit();
lock.lock();
try {
nodeAddedListeners.forEach(listener -> listener.onNodesAdded(addedNodes));
transaction.commit();
} finally {
lock.unlock();
}
}

@Override
public void rollback() {
addedNodes.clear();
transaction.rollback();
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,8 +56,9 @@ public class MarkSweepPrunerTest {
private final BlockDataGenerator gen = new BlockDataGenerator();
private final NoOpMetricsSystem metricsSystem = new NoOpMetricsSystem();
private final Map<Bytes, byte[]> hashValueStore = spy(new HashMap<>());
private final InMemoryKeyValueStorage stateStorage = spy(new TestInMemoryStorage(hashValueStore));
private final WorldStateStorage worldStateStorage = new WorldStateKeyValueStorage(stateStorage);
private final InMemoryKeyValueStorage stateStorage = new TestInMemoryStorage(hashValueStore);
private final WorldStateStorage worldStateStorage =
spy(new WorldStateKeyValueStorage(stateStorage));
private final WorldStateArchive worldStateArchive =
new WorldStateArchive(
worldStateStorage, new WorldStatePreimageKeyValueStorage(new InMemoryKeyValueStorage()));
Expand Down Expand Up @@ -137,11 +138,11 @@ public void sweepBefore_shouldSweepStateRootFirst() {
pruner.sweepBefore(markBlock.getNumber());

// Check stateRoots are marked first
InOrder inOrder = inOrder(hashValueStore, stateStorage);
InOrder inOrder = inOrder(hashValueStore, worldStateStorage);
for (Bytes32 stateRoot : stateRoots) {
inOrder.verify(hashValueStore).remove(stateRoot);
}
inOrder.verify(stateStorage).removeAllKeysUnless(any());
inOrder.verify(worldStateStorage).prune(any());
}

@Test
Expand Down Expand Up @@ -172,11 +173,11 @@ public void sweepBefore_shouldNotRemoveMarkedStateRoots() {
pruner.sweepBefore(markBlock.getNumber());

// Check stateRoots are marked first
InOrder inOrder = inOrder(hashValueStore, stateStorage);
InOrder inOrder = inOrder(hashValueStore, worldStateStorage);
for (Bytes32 stateRoot : stateRoots) {
inOrder.verify(hashValueStore).remove(stateRoot);
}
inOrder.verify(stateStorage).removeAllKeysUnless(any());
inOrder.verify(worldStateStorage).prune(any());

assertThat(stateStorage.containsKey(markedRoot.toArray())).isTrue();
}
Expand Down
2 changes: 1 addition & 1 deletion plugin-api/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ Calculated : ${currentHash}
tasks.register('checkAPIChanges', FileStateChecker) {
description = "Checks that the API for the Plugin-API project does not change without deliberate thought"
files = sourceSets.main.allJava.files
knownHash = 'wjORmE9I35CVqZIk78t8zgZUo9oIRy2fvu0x+CFnzaU='
knownHash = 'x6hLH75JBKLM8mNFz7n0ALRw4TWiDONJuum53da4jTY='
}
check.dependsOn('checkAPIChanges')

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.util.Optional;
import java.util.Set;
import java.util.function.Predicate;
import java.util.stream.Stream;

/**
* Responsible for storing values against keys.
Expand Down Expand Up @@ -60,17 +61,36 @@ public interface KeyValueStorage extends Closeable {
*/
Optional<byte[]> get(byte[] key) throws StorageException;

/**
* Returns a stream of all keys.
*
* @return A stream of all keys in storage.
* @throws StorageException problem encountered during the retrieval attempt.
*/
Stream<byte[]> streamKeys() throws StorageException;

/**
* Performs an evaluation against each key in the store, keeping the entries that pass, removing
* those that fail.
*
* @param retainCondition predicate to evaluate each key against, unless the result is {@code
* null}, both the key and associated value must be removed.
* @return the number of keys removed.
* @throws StorageException problem encountered when removing data.
* @throws StorageException problem encountered during the retrieval attempt.
*/
long removeAllKeysUnless(Predicate<byte[]> retainCondition) throws StorageException;

/**
* Delete the value corresponding to the given key if a write lock can be instantly acquired on
* the underlying storage. Do nothing otherwise.
*
* @param key The key to delete.
* @throws StorageException any problem encountered during the deletion attempt.
* @return false if the lock on the underlying storage could not be instantly acquired, true
* otherwise
*/
boolean tryDelete(byte[] key) throws StorageException;

/**
* Performs an evaluation against each key in the store, returning the set of entries that pass.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
/*
* Copyright 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.
*
* SPDX-License-Identifier: Apache-2.0
*/

package org.hyperledger.besu.plugin.services.storage.rocksdb;

import static com.google.common.base.Preconditions.checkState;

import java.util.Iterator;
import java.util.NoSuchElementException;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;

import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.rocksdb.RocksDBException;
import org.rocksdb.RocksIterator;

public class RocksDbKeyIterator implements Iterator<byte[]>, AutoCloseable {
private static final Logger LOG = LogManager.getLogger();

private final RocksIterator rocksIterator;
private final AtomicBoolean closed = new AtomicBoolean(false);

private RocksDbKeyIterator(final RocksIterator rocksIterator) {
this.rocksIterator = rocksIterator;
}

public static RocksDbKeyIterator create(final RocksIterator rocksIterator) {
return new RocksDbKeyIterator(rocksIterator);
}

@Override
public boolean hasNext() {
assertOpen();
return rocksIterator.isValid();
}

@Override
public byte[] next() {
assertOpen();
try {
rocksIterator.status();
} catch (final RocksDBException e) {
LOG.error(
String.format("%s encountered a problem while iterating.", getClass().getSimpleName()),
e);
}
if (!hasNext()) {
throw new NoSuchElementException();
}
final byte[] key = rocksIterator.key();
rocksIterator.next();
return key;
}

public Stream<byte[]> toStream() {
assertOpen();
final Spliterator<byte[]> spliterator =
Spliterators.spliteratorUnknownSize(
this,
Spliterator.IMMUTABLE
| Spliterator.DISTINCT
| Spliterator.NONNULL
| Spliterator.ORDERED
| Spliterator.SORTED);

return StreamSupport.stream(spliterator, false).onClose(this::close);
}

private void assertOpen() {
checkState(
!closed.get(),
String.format("Attempt to read from a closed %s", getClass().getSimpleName()));
}

@Override
public void close() {
if (closed.compareAndSet(false, true)) {
rocksIterator.close();
}
}
}
Loading

0 comments on commit 43eccbb

Please sign in to comment.