Skip to content

Commit

Permalink
[PAN-2422] Add in-memory cache over world state download queue (#1087)
Browse files Browse the repository at this point in the history
Signed-off-by: Adrian Sutton <adrian.sutton@consensys.net>
  • Loading branch information
mbaxter authored Mar 13, 2019
1 parent 6104403 commit fafd7e6
Show file tree
Hide file tree
Showing 20 changed files with 565 additions and 168 deletions.
2 changes: 1 addition & 1 deletion ethereum/eth/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ dependencies {
implementation project(':ethereum:permissioning')
implementation project(':metrics')
implementation project(':services:kvstore')
implementation project(':services:queue')
implementation project(':services:tasks')

implementation 'io.vertx:vertx-core'
implementation 'com.google.guava:guava'
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,8 @@
import tech.pegasys.pantheon.metrics.MetricsSystem;
import tech.pegasys.pantheon.metrics.noop.NoOpMetricsSystem;
import tech.pegasys.pantheon.services.kvstore.InMemoryKeyValueStorage;
import tech.pegasys.pantheon.services.queue.RocksDbTaskQueue;
import tech.pegasys.pantheon.services.queue.TaskQueue;
import tech.pegasys.pantheon.services.tasks.CachingTaskCollection;
import tech.pegasys.pantheon.services.tasks.RocksDbTaskQueue;
import tech.pegasys.pantheon.util.bytes.BytesValue;

import java.nio.file.Path;
Expand Down Expand Up @@ -61,7 +61,7 @@ public class WorldStateDownloaderBenchmark {
private WorldStateStorage worldStateStorage;
private RespondingEthPeer peer;
private Responder responder;
private TaskQueue<NodeDataRequest> pendingRequests;
private CachingTaskCollection<NodeDataRequest> pendingRequests;
private StorageProvider storageProvider;
private EthProtocolManager ethProtocolManager;
private InMemoryKeyValueStorage remoteKeyValueStorage;
Expand Down Expand Up @@ -89,11 +89,13 @@ public void setUpUnchangedState() throws Exception {
worldStateStorage = storageProvider.createWorldStateStorage();

pendingRequests =
RocksDbTaskQueue.create(
tempDir.resolve("fastsync"),
NodeDataRequest::serialize,
NodeDataRequest::deserialize,
metricsSystem);
new CachingTaskCollection<>(
RocksDbTaskQueue.create(
tempDir.resolve("fastsync"),
NodeDataRequest::serialize,
NodeDataRequest::deserialize,
metricsSystem),
0);
worldStateDownloader =
new WorldStateDownloader(
ethContext,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,10 @@
import tech.pegasys.pantheon.ethereum.mainnet.ProtocolSchedule;
import tech.pegasys.pantheon.ethereum.mainnet.ScheduleBasedBlockHashFunction;
import tech.pegasys.pantheon.ethereum.worldstate.WorldStateStorage;
import tech.pegasys.pantheon.metrics.MetricCategory;
import tech.pegasys.pantheon.metrics.MetricsSystem;
import tech.pegasys.pantheon.services.queue.RocksDbTaskQueue;
import tech.pegasys.pantheon.services.queue.TaskQueue;
import tech.pegasys.pantheon.services.tasks.CachingTaskCollection;
import tech.pegasys.pantheon.services.tasks.RocksDbTaskQueue;

import java.io.File;
import java.io.IOException;
Expand All @@ -45,19 +46,19 @@ class FastSynchronizer<C> {

private final FastSyncDownloader<C> fastSyncDownloader;
private final Path fastSyncDataDirectory;
private final TaskQueue<NodeDataRequest> stateQueue;
private final CachingTaskCollection<NodeDataRequest> taskCollection;
private final WorldStateDownloader worldStateDownloader;
private final FastSyncState initialSyncState;

private FastSynchronizer(
final FastSyncDownloader<C> fastSyncDownloader,
final Path fastSyncDataDirectory,
final TaskQueue<NodeDataRequest> stateQueue,
final CachingTaskCollection<NodeDataRequest> taskCollection,
final WorldStateDownloader worldStateDownloader,
final FastSyncState initialSyncState) {
this.fastSyncDownloader = fastSyncDownloader;
this.fastSyncDataDirectory = fastSyncDataDirectory;
this.stateQueue = stateQueue;
this.taskCollection = taskCollection;
this.worldStateDownloader = worldStateDownloader;
this.initialSyncState = initialSyncState;
}
Expand Down Expand Up @@ -88,13 +89,14 @@ public static <C> Optional<FastSynchronizer<C>> create(
return Optional.empty();
}

final TaskQueue<NodeDataRequest> stateQueue =
createWorldStateDownloaderQueue(getStateQueueDirectory(dataDirectory), metricsSystem);
final CachingTaskCollection<NodeDataRequest> taskCollection =
createWorldStateDownloaderTaskCollection(
getStateQueueDirectory(dataDirectory), metricsSystem);
final WorldStateDownloader worldStateDownloader =
new WorldStateDownloader(
ethContext,
worldStateStorage,
stateQueue,
taskCollection,
syncConfig.getWorldStateHashCountPerRequest(),
syncConfig.getWorldStateRequestParallelism(),
syncConfig.getWorldStateMaxRequestsWithoutProgress(),
Expand All @@ -114,7 +116,7 @@ public static <C> Optional<FastSynchronizer<C>> create(
new FastSynchronizer<>(
fastSyncDownloader,
fastSyncDataDirectory,
stateQueue,
taskCollection,
worldStateDownloader,
fastSyncState));
}
Expand All @@ -128,7 +130,7 @@ public void deleteFastSyncState() {
// Make sure downloader is stopped before we start cleaning up its dependencies
worldStateDownloader.cancel();
try {
stateQueue.close();
taskCollection.close();
if (fastSyncDataDirectory.toFile().exists()) {
// Clean up this data for now (until fast sync resume functionality is in place)
MoreFiles.deleteRecursively(fastSyncDataDirectory, RecursiveDeleteOption.ALLOW_INSECURE);
Expand Down Expand Up @@ -156,9 +158,33 @@ private static void ensureDirectoryExists(final File dir) {
}
}

private static TaskQueue<NodeDataRequest> createWorldStateDownloaderQueue(
private static CachingTaskCollection<NodeDataRequest> createWorldStateDownloaderTaskCollection(
final Path dataDirectory, final MetricsSystem metricsSystem) {
return RocksDbTaskQueue.create(
dataDirectory, NodeDataRequest::serialize, NodeDataRequest::deserialize, metricsSystem);
final CachingTaskCollection<NodeDataRequest> taskCollection =
new CachingTaskCollection<>(
RocksDbTaskQueue.create(
dataDirectory,
NodeDataRequest::serialize,
NodeDataRequest::deserialize,
metricsSystem));

metricsSystem.createLongGauge(
MetricCategory.SYNCHRONIZER,
"world_state_pending_requests_current",
"Number of pending requests for fast sync world state download",
taskCollection::size);

metricsSystem.createIntegerGauge(
MetricCategory.SYNCHRONIZER,
"world_state_pending_requests_cache_size",
"Pending request cache size for fast sync world state download",
taskCollection::cacheSize);

// We're using the CachingTaskCollection which isn't designed to reliably persist all
// added tasks. We therefore can't resume from previously added tasks.
// So for now, clear tasks when we start up.
taskCollection.clear();

return taskCollection;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,8 @@
import tech.pegasys.pantheon.ethereum.eth.manager.task.EthTask;
import tech.pegasys.pantheon.ethereum.worldstate.WorldStateStorage;
import tech.pegasys.pantheon.ethereum.worldstate.WorldStateStorage.Updater;
import tech.pegasys.pantheon.services.queue.TaskQueue;
import tech.pegasys.pantheon.services.queue.TaskQueue.Task;
import tech.pegasys.pantheon.services.tasks.CachingTaskCollection;
import tech.pegasys.pantheon.services.tasks.Task;
import tech.pegasys.pantheon.util.ExceptionUtils;
import tech.pegasys.pantheon.util.bytes.BytesValue;

Expand All @@ -38,7 +38,7 @@ class WorldDownloadState {
private static final Logger LOG = LogManager.getLogger();

private final boolean downloadWasResumed;
private final TaskQueue<NodeDataRequest> pendingRequests;
private final CachingTaskCollection<NodeDataRequest> pendingRequests;
private final ArrayBlockingQueue<Task<NodeDataRequest>> requestsToPersist;
private final int maxOutstandingRequests;
private final int maxRequestsWithoutProgress;
Expand All @@ -54,7 +54,7 @@ class WorldDownloadState {
private EthTask<?> persistenceTask;

public WorldDownloadState(
final TaskQueue<NodeDataRequest> pendingRequests,
final CachingTaskCollection<NodeDataRequest> pendingRequests,
final ArrayBlockingQueue<Task<NodeDataRequest>> requestsToPersist,
final int maxOutstandingRequests,
final int maxRequestsWithoutProgress) {
Expand Down Expand Up @@ -151,21 +151,21 @@ public synchronized void setPersistenceTask(final EthTask<?> persistenceTask) {

public synchronized void enqueueRequest(final NodeDataRequest request) {
if (!internalFuture.isDone()) {
pendingRequests.enqueue(request);
pendingRequests.add(request);
}
}

public synchronized void enqueueRequests(final Collection<NodeDataRequest> requests) {
if (!internalFuture.isDone()) {
requests.forEach(pendingRequests::enqueue);
requests.forEach(pendingRequests::add);
}
}

public synchronized Task<NodeDataRequest> dequeueRequest() {
if (internalFuture.isDone()) {
return null;
}
return pendingRequests.dequeue();
return pendingRequests.remove();
}

public synchronized void setRootNodeData(final BytesValue rootNodeData) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@
import tech.pegasys.pantheon.metrics.Counter;
import tech.pegasys.pantheon.metrics.MetricCategory;
import tech.pegasys.pantheon.metrics.MetricsSystem;
import tech.pegasys.pantheon.services.queue.TaskQueue;
import tech.pegasys.pantheon.services.queue.TaskQueue.Task;
import tech.pegasys.pantheon.services.tasks.CachingTaskCollection;
import tech.pegasys.pantheon.services.tasks.Task;
import tech.pegasys.pantheon.util.ExceptionUtils;
import tech.pegasys.pantheon.util.bytes.BytesValue;

Expand Down Expand Up @@ -61,7 +61,7 @@ public class WorldStateDownloader {
private final MetricsSystem metricsSystem;

private final EthContext ethContext;
private final TaskQueue<NodeDataRequest> taskQueue;
private final CachingTaskCollection<NodeDataRequest> taskCollection;
private final int hashCountPerRequest;
private final int maxOutstandingRequests;
private final int maxNodeRequestsWithoutProgress;
Expand All @@ -72,25 +72,19 @@ public class WorldStateDownloader {
public WorldStateDownloader(
final EthContext ethContext,
final WorldStateStorage worldStateStorage,
final TaskQueue<NodeDataRequest> taskQueue,
final CachingTaskCollection<NodeDataRequest> taskCollection,
final int hashCountPerRequest,
final int maxOutstandingRequests,
final int maxNodeRequestsWithoutProgress,
final MetricsSystem metricsSystem) {
this.ethContext = ethContext;
this.worldStateStorage = worldStateStorage;
this.taskQueue = taskQueue;
this.taskCollection = taskCollection;
this.hashCountPerRequest = hashCountPerRequest;
this.maxOutstandingRequests = maxOutstandingRequests;
this.maxNodeRequestsWithoutProgress = maxNodeRequestsWithoutProgress;
this.metricsSystem = metricsSystem;

metricsSystem.createLongGauge(
MetricCategory.SYNCHRONIZER,
"world_state_pending_requests_current",
"Number of pending requests for fast sync world state download",
taskQueue::size);

completedRequestsCounter =
metricsSystem.createCounter(
MetricCategory.SYNCHRONIZER,
Expand Down Expand Up @@ -159,7 +153,7 @@ public CompletableFuture<Void> run(final BlockHeader header) {
final int persistenceQueueCapacity = hashCountPerRequest * maxOutstandingRequests * 2;
final WorldDownloadState newDownloadState =
new WorldDownloadState(
taskQueue,
taskCollection,
new ArrayBlockingQueue<>(persistenceQueueCapacity),
maxOutstandingRequests,
maxNodeRequestsWithoutProgress);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,9 @@
import tech.pegasys.pantheon.ethereum.storage.keyvalue.KeyValueStorageWorldStateStorage;
import tech.pegasys.pantheon.ethereum.worldstate.WorldStateStorage;
import tech.pegasys.pantheon.services.kvstore.InMemoryKeyValueStorage;
import tech.pegasys.pantheon.services.queue.InMemoryTaskQueue;
import tech.pegasys.pantheon.services.queue.TaskQueue.Task;
import tech.pegasys.pantheon.services.tasks.CachingTaskCollection;
import tech.pegasys.pantheon.services.tasks.InMemoryTaskQueue;
import tech.pegasys.pantheon.services.tasks.Task;
import tech.pegasys.pantheon.util.bytes.BytesValue;

import java.util.Arrays;
Expand All @@ -51,7 +52,8 @@ public class WorldDownloadStateTest {

private final BlockHeader header =
new BlockHeaderTestFixture().stateRoot(ROOT_NODE_HASH).buildHeader();
private final InMemoryTaskQueue<NodeDataRequest> pendingRequests = new InMemoryTaskQueue<>();
private final CachingTaskCollection<NodeDataRequest> pendingRequests =
new CachingTaskCollection<>(new InMemoryTaskQueue<>());
private final ArrayBlockingQueue<Task<NodeDataRequest>> requestsToPersist =
new ArrayBlockingQueue<>(100);

Expand Down Expand Up @@ -94,7 +96,7 @@ public void shouldStoreRootNodeBeforeReturnedFutureCompletes() {

@Test
public void shouldNotCompleteWhenThereArePendingTasks() {
pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));

downloadState.checkCompletion(worldStateStorage, header);

Expand All @@ -115,8 +117,8 @@ public void shouldCancelOutstandingTasksWhenFutureIsCancelled() {
downloadState.addOutstandingTask(outstandingTask1);
downloadState.addOutstandingTask(outstandingTask2);

pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY));
requestsToPersist.add(toPersist1);
requestsToPersist.add(toPersist2);

Expand All @@ -133,15 +135,15 @@ public void shouldCancelOutstandingTasksWhenFutureIsCancelled() {

@Test
public void shouldNotSendAdditionalRequestsWhenWaitingForANewPeer() {
pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));

downloadState.setWaitingForNewPeer(true);
downloadState.whileAdditionalRequestsCanBeSent(mustNotBeCalled());
}

@Test
public void shouldResumeSendingAdditionalRequestsWhenNoLongerWaitingForPeer() {
pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
final Runnable sendRequest =
mockWithAction(() -> downloadState.addOutstandingTask(mock(EthTask.class)));

Expand All @@ -155,18 +157,18 @@ public void shouldResumeSendingAdditionalRequestsWhenNoLongerWaitingForPeer() {

@Test
public void shouldStopSendingAdditionalRequestsWhenPendingRequestsIsEmpty() {
pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));

final Runnable sendRequest = mockWithAction(pendingRequests::dequeue);
final Runnable sendRequest = mockWithAction(pendingRequests::remove);
downloadState.whileAdditionalRequestsCanBeSent(sendRequest);

verify(sendRequest, times(2)).run();
}

@Test
public void shouldStopSendingAdditionalRequestsWhenMaximumOutstandingRequestCountReached() {
pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
final Runnable sendRequest =
mockWithAction(() -> downloadState.addOutstandingTask(mock(EthTask.class)));

Expand All @@ -176,7 +178,7 @@ public void shouldStopSendingAdditionalRequestsWhenMaximumOutstandingRequestCoun

@Test
public void shouldStopSendingAdditionalRequestsWhenFutureIsCancelled() {
pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
final Runnable sendRequest = mockWithAction(() -> future.cancel(true));

downloadState.whileAdditionalRequestsCanBeSent(sendRequest);
Expand All @@ -185,7 +187,7 @@ public void shouldStopSendingAdditionalRequestsWhenFutureIsCancelled() {

@Test
public void shouldStopSendingAdditionalRequestsWhenDownloadIsMarkedAsStalled() {
pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
final Runnable sendRequest = mockWithAction(() -> downloadState.requestComplete(false));

downloadState.whileAdditionalRequestsCanBeSent(sendRequest);
Expand All @@ -210,7 +212,7 @@ public void shouldResetRequestsSinceProgressCountWhenProgressIsMade() {

@Test
public void shouldNotAllowMultipleCallsToSendAdditionalRequestsAtOnce() {
pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
final Runnable sendRequest =
mockWithAction(
() -> {
Expand All @@ -223,7 +225,7 @@ public void shouldNotAllowMultipleCallsToSendAdditionalRequestsAtOnce() {
}

@Test
public void shouldNotEnqueueRequestsAfterDownloadIsStalled() {
public void shouldNotAddRequestsAfterDownloadIsStalled() {
downloadState.checkCompletion(worldStateStorage, header);

downloadState.enqueueRequests(Arrays.asList(createAccountDataRequest(Hash.EMPTY_TRIE_HASH)));
Expand All @@ -235,7 +237,7 @@ public void shouldNotEnqueueRequestsAfterDownloadIsStalled() {
@Test // Sanity check for the test structure
public void shouldFailWhenMustNotBeCalledIsCalled() {

pendingRequests.enqueue(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
pendingRequests.add(createAccountDataRequest(Hash.EMPTY_TRIE_HASH));
assertThatThrownBy(() -> downloadState.whileAdditionalRequestsCanBeSent(mustNotBeCalled()))
.hasMessage("Unexpected invocation");
}
Expand Down
Loading

0 comments on commit fafd7e6

Please sign in to comment.