mirror of
https://github.com/vacp2p/status-linea-besu.git
synced 2026-01-09 22:07:59 -05:00
Metrics for sync phases (#7390)
* add metrics to measure the time for chain and world state sync Signed-off-by: stefan.pingel@consensys.net <stefan.pingel@consensys.net> Signed-off-by: Stefan Pingel <16143240+pinges@users.noreply.github.com> Co-authored-by: Simon Dudley <simon.dudley@consensys.net> Co-authored-by: Sally MacFarlane <macfarla.github@gmail.com>
This commit is contained in:
@@ -43,6 +43,7 @@ import org.hyperledger.besu.ethereum.worldstate.DataStorageConfiguration;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateArchive;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.ObservableMetricsSystem;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.storage.rocksdb.RocksDBKeyValueStorageFactory;
|
||||
import org.hyperledger.besu.plugin.services.storage.rocksdb.RocksDBMetricsFactory;
|
||||
@@ -120,7 +121,8 @@ public class WorldStateDownloaderBenchmark {
|
||||
syncConfig.getWorldStateMaxRequestsWithoutProgress(),
|
||||
syncConfig.getWorldStateMinMillisBeforeStalling(),
|
||||
Clock.fixed(Instant.ofEpochSecond(1000), ZoneOffset.UTC),
|
||||
metricsSystem);
|
||||
metricsSystem,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
}
|
||||
|
||||
private Hash createExistingWorldState() {
|
||||
|
||||
@@ -38,6 +38,7 @@ import org.hyperledger.besu.ethereum.storage.StorageProvider;
|
||||
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.BonsaiWorldStateProvider;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.BesuMetricCategory;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.data.SyncStatus;
|
||||
import org.hyperledger.besu.plugin.services.BesuEvents;
|
||||
import org.hyperledger.besu.plugin.services.BesuEvents.SyncStatusListener;
|
||||
@@ -67,6 +68,7 @@ public class DefaultSynchronizer implements Synchronizer, UnverifiedForkchoiceLi
|
||||
private final AtomicBoolean running = new AtomicBoolean(false);
|
||||
private final Optional<BlockPropagationManager> blockPropagationManager;
|
||||
private final Supplier<Optional<FastSyncDownloader<?>>> fastSyncFactory;
|
||||
private final SyncDurationMetrics syncDurationMetrics;
|
||||
private Optional<FastSyncDownloader<?>> fastSyncDownloader;
|
||||
private final Optional<FullSyncDownloader> fullSyncDownloader;
|
||||
private final ProtocolContext protocolContext;
|
||||
@@ -118,6 +120,8 @@ public class DefaultSynchronizer implements Synchronizer, UnverifiedForkchoiceLi
|
||||
metricsSystem,
|
||||
blockBroadcaster));
|
||||
|
||||
syncDurationMetrics = new SyncDurationMetrics(metricsSystem);
|
||||
|
||||
this.fullSyncDownloader =
|
||||
terminationCondition.shouldStopDownload()
|
||||
? Optional.empty()
|
||||
@@ -129,7 +133,8 @@ public class DefaultSynchronizer implements Synchronizer, UnverifiedForkchoiceLi
|
||||
ethContext,
|
||||
syncState,
|
||||
metricsSystem,
|
||||
terminationCondition));
|
||||
terminationCondition,
|
||||
syncDurationMetrics));
|
||||
|
||||
if (SyncMode.FAST.equals(syncConfig.getSyncMode())) {
|
||||
this.fastSyncFactory =
|
||||
@@ -144,7 +149,8 @@ public class DefaultSynchronizer implements Synchronizer, UnverifiedForkchoiceLi
|
||||
ethContext,
|
||||
worldStateStorageCoordinator,
|
||||
syncState,
|
||||
clock);
|
||||
clock,
|
||||
syncDurationMetrics);
|
||||
} else if (syncConfig.getSyncMode() == SyncMode.CHECKPOINT) {
|
||||
this.fastSyncFactory =
|
||||
() ->
|
||||
@@ -159,7 +165,8 @@ public class DefaultSynchronizer implements Synchronizer, UnverifiedForkchoiceLi
|
||||
ethContext,
|
||||
worldStateStorageCoordinator,
|
||||
syncState,
|
||||
clock);
|
||||
clock,
|
||||
syncDurationMetrics);
|
||||
} else {
|
||||
this.fastSyncFactory =
|
||||
() ->
|
||||
@@ -174,7 +181,8 @@ public class DefaultSynchronizer implements Synchronizer, UnverifiedForkchoiceLi
|
||||
ethContext,
|
||||
worldStateStorageCoordinator,
|
||||
syncState,
|
||||
clock);
|
||||
clock,
|
||||
syncDurationMetrics);
|
||||
}
|
||||
|
||||
// create a non-resync fast sync downloader:
|
||||
@@ -205,6 +213,9 @@ public class DefaultSynchronizer implements Synchronizer, UnverifiedForkchoiceLi
|
||||
public CompletableFuture<Void> start() {
|
||||
if (running.compareAndSet(false, true)) {
|
||||
LOG.info("Starting synchronizer.");
|
||||
|
||||
syncDurationMetrics.startTimer(SyncDurationMetrics.Labels.TOTAL_SYNC_DURATION);
|
||||
|
||||
blockPropagationManager.ifPresent(
|
||||
manager -> {
|
||||
if (!manager.isRunning()) {
|
||||
@@ -390,6 +401,10 @@ public class DefaultSynchronizer implements Synchronizer, UnverifiedForkchoiceLi
|
||||
blockPropagationManager.ifPresent(BlockPropagationManager::stop);
|
||||
LOG.info("Stopping the pruner.");
|
||||
running.set(false);
|
||||
|
||||
syncDurationMetrics.stopTimer(SyncDurationMetrics.Labels.FLAT_DB_HEAL);
|
||||
syncDurationMetrics.stopTimer(SyncDurationMetrics.Labels.TOTAL_SYNC_DURATION);
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.hyperledger.besu.ethereum.eth.sync.state.SyncTarget;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.tasks.exceptions.InvalidBlockException;
|
||||
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.messages.DisconnectMessage.DisconnectReason;
|
||||
import org.hyperledger.besu.metrics.BesuMetricCategory;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.metrics.Counter;
|
||||
import org.hyperledger.besu.plugin.services.metrics.LabelledMetric;
|
||||
@@ -51,6 +52,7 @@ public class PipelineChainDownloader implements ChainDownloader {
|
||||
private final AtomicBoolean cancelled = new AtomicBoolean(false);
|
||||
private final Counter pipelineCompleteCounter;
|
||||
private final Counter pipelineErrorCounter;
|
||||
private final SyncDurationMetrics syncDurationMetrics;
|
||||
private Pipeline<?> currentDownloadPipeline;
|
||||
|
||||
public PipelineChainDownloader(
|
||||
@@ -58,11 +60,13 @@ public class PipelineChainDownloader implements ChainDownloader {
|
||||
final AbstractSyncTargetManager syncTargetManager,
|
||||
final DownloadPipelineFactory downloadPipelineFactory,
|
||||
final EthScheduler scheduler,
|
||||
final MetricsSystem metricsSystem) {
|
||||
final MetricsSystem metricsSystem,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
this.syncState = syncState;
|
||||
this.syncTargetManager = syncTargetManager;
|
||||
this.downloadPipelineFactory = downloadPipelineFactory;
|
||||
this.scheduler = scheduler;
|
||||
this.syncDurationMetrics = syncDurationMetrics;
|
||||
|
||||
final LabelledMetric<Counter> labelledCounter =
|
||||
metricsSystem.createLabelledCounter(
|
||||
@@ -79,6 +83,9 @@ public class PipelineChainDownloader implements ChainDownloader {
|
||||
if (!started.compareAndSet(false, true)) {
|
||||
throw new IllegalStateException("Cannot start a chain download twice");
|
||||
}
|
||||
|
||||
syncDurationMetrics.startTimer(SyncDurationMetrics.Labels.CHAIN_DOWNLOAD_DURATION);
|
||||
|
||||
return performDownload();
|
||||
}
|
||||
|
||||
|
||||
@@ -36,6 +36,7 @@ import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ScheduleBasedBlockHeaderFunctions;
|
||||
import org.hyperledger.besu.ethereum.trie.CompactEncoding;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.services.tasks.InMemoryTasksPriorityQueues;
|
||||
|
||||
@@ -62,7 +63,8 @@ public class CheckpointDownloaderFactory extends SnapDownloaderFactory {
|
||||
final EthContext ethContext,
|
||||
final WorldStateStorageCoordinator worldStateStorageCoordinator,
|
||||
final SyncState syncState,
|
||||
final Clock clock) {
|
||||
final Clock clock,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
|
||||
final Path fastSyncDataDirectory = dataDirectory.resolve(FAST_SYNC_FOLDER);
|
||||
final FastSyncStateStorage fastSyncStateStorage =
|
||||
@@ -149,7 +151,8 @@ public class CheckpointDownloaderFactory extends SnapDownloaderFactory {
|
||||
syncConfig.getWorldStateMaxRequestsWithoutProgress(),
|
||||
syncConfig.getWorldStateMinMillisBeforeStalling(),
|
||||
clock,
|
||||
metricsSystem);
|
||||
metricsSystem,
|
||||
syncDurationMetrics);
|
||||
final FastSyncDownloader<SnapDataRequest> fastSyncDownloader =
|
||||
new SnapSyncDownloader(
|
||||
fastSyncActions,
|
||||
@@ -158,7 +161,8 @@ public class CheckpointDownloaderFactory extends SnapDownloaderFactory {
|
||||
fastSyncStateStorage,
|
||||
snapTaskCollection,
|
||||
fastSyncDataDirectory,
|
||||
snapSyncState);
|
||||
snapSyncState,
|
||||
syncDurationMetrics);
|
||||
syncState.setWorldStateDownloadStatus(snapWorldStateDownloader);
|
||||
return Optional.of(fastSyncDownloader);
|
||||
}
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.hyperledger.besu.ethereum.eth.sync.fastsync.FastSyncState;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
|
||||
public class CheckpointSyncActions extends FastSyncActions {
|
||||
@@ -48,7 +49,8 @@ public class CheckpointSyncActions extends FastSyncActions {
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChainDownloader createChainDownloader(final FastSyncState currentState) {
|
||||
public ChainDownloader createChainDownloader(
|
||||
final FastSyncState currentState, final SyncDurationMetrics syncDurationMetrics) {
|
||||
return CheckpointSyncChainDownloader.create(
|
||||
syncConfig,
|
||||
worldStateStorageCoordinator,
|
||||
@@ -57,6 +59,7 @@ public class CheckpointSyncActions extends FastSyncActions {
|
||||
ethContext,
|
||||
syncState,
|
||||
metricsSystem,
|
||||
currentState);
|
||||
currentState,
|
||||
syncDurationMetrics);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -25,6 +25,7 @@ import org.hyperledger.besu.ethereum.eth.sync.fastsync.SyncTargetManager;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
|
||||
public class CheckpointSyncChainDownloader extends FastSyncChainDownloader {
|
||||
@@ -37,7 +38,8 @@ public class CheckpointSyncChainDownloader extends FastSyncChainDownloader {
|
||||
final EthContext ethContext,
|
||||
final SyncState syncState,
|
||||
final MetricsSystem metricsSystem,
|
||||
final FastSyncState fastSyncState) {
|
||||
final FastSyncState fastSyncState,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
|
||||
final SyncTargetManager syncTargetManager =
|
||||
new SyncTargetManager(
|
||||
@@ -55,6 +57,7 @@ public class CheckpointSyncChainDownloader extends FastSyncChainDownloader {
|
||||
new CheckpointSyncDownloadPipelineFactory(
|
||||
config, protocolSchedule, protocolContext, ethContext, fastSyncState, metricsSystem),
|
||||
ethContext.getScheduler(),
|
||||
metricsSystem);
|
||||
metricsSystem,
|
||||
syncDurationMetrics);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -28,6 +28,7 @@ import org.hyperledger.besu.ethereum.eth.sync.tasks.RetryingGetHeaderFromPeerByH
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.BesuMetricCategory;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.metrics.Counter;
|
||||
|
||||
@@ -155,7 +156,8 @@ public class FastSyncActions {
|
||||
return fastSyncState;
|
||||
}
|
||||
|
||||
public ChainDownloader createChainDownloader(final FastSyncState currentState) {
|
||||
public ChainDownloader createChainDownloader(
|
||||
final FastSyncState currentState, final SyncDurationMetrics syncDurationMetrics) {
|
||||
return FastSyncChainDownloader.create(
|
||||
syncConfig,
|
||||
worldStateStorageCoordinator,
|
||||
@@ -164,7 +166,8 @@ public class FastSyncActions {
|
||||
ethContext,
|
||||
syncState,
|
||||
metricsSystem,
|
||||
currentState);
|
||||
currentState,
|
||||
syncDurationMetrics);
|
||||
}
|
||||
|
||||
private CompletableFuture<FastSyncState> downloadPivotBlockHeader(final Hash hash) {
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.hyperledger.besu.ethereum.eth.sync.SynchronizerConfiguration;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
|
||||
public class FastSyncChainDownloader {
|
||||
@@ -36,7 +37,8 @@ public class FastSyncChainDownloader {
|
||||
final EthContext ethContext,
|
||||
final SyncState syncState,
|
||||
final MetricsSystem metricsSystem,
|
||||
final FastSyncState fastSyncState) {
|
||||
final FastSyncState fastSyncState,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
|
||||
final SyncTargetManager syncTargetManager =
|
||||
new SyncTargetManager(
|
||||
@@ -53,6 +55,7 @@ public class FastSyncChainDownloader {
|
||||
new FastSyncDownloadPipelineFactory(
|
||||
config, protocolSchedule, protocolContext, ethContext, fastSyncState, metricsSystem),
|
||||
ethContext.getScheduler(),
|
||||
metricsSystem);
|
||||
metricsSystem,
|
||||
syncDurationMetrics);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -23,6 +23,7 @@ import org.hyperledger.besu.ethereum.eth.sync.worldstate.StalledDownloadExceptio
|
||||
import org.hyperledger.besu.ethereum.eth.sync.worldstate.WorldStateDownloader;
|
||||
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
import org.hyperledger.besu.services.tasks.TaskCollection;
|
||||
import org.hyperledger.besu.util.ExceptionUtils;
|
||||
@@ -52,6 +53,7 @@ public class FastSyncDownloader<REQUEST> {
|
||||
private final WorldStateDownloader worldStateDownloader;
|
||||
private final TaskCollection<REQUEST> taskCollection;
|
||||
private final Path fastSyncDataDirectory;
|
||||
private final SyncDurationMetrics syncDurationMetrics;
|
||||
private volatile Optional<TrailingPeerRequirements> trailingPeerRequirements = Optional.empty();
|
||||
private final AtomicBoolean running = new AtomicBoolean(false);
|
||||
|
||||
@@ -66,7 +68,8 @@ public class FastSyncDownloader<REQUEST> {
|
||||
final FastSyncStateStorage fastSyncStateStorage,
|
||||
final TaskCollection<REQUEST> taskCollection,
|
||||
final Path fastSyncDataDirectory,
|
||||
final FastSyncState initialFastSyncState) {
|
||||
final FastSyncState initialFastSyncState,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
this.fastSyncActions = fastSyncActions;
|
||||
this.worldStateStorageCoordinator = worldStateStorageCoordinator;
|
||||
this.worldStateDownloader = worldStateDownloader;
|
||||
@@ -74,6 +77,7 @@ public class FastSyncDownloader<REQUEST> {
|
||||
this.taskCollection = taskCollection;
|
||||
this.fastSyncDataDirectory = fastSyncDataDirectory;
|
||||
this.initialFastSyncState = initialFastSyncState;
|
||||
this.syncDurationMetrics = syncDurationMetrics;
|
||||
}
|
||||
|
||||
public CompletableFuture<FastSyncState> start() {
|
||||
@@ -81,6 +85,7 @@ public class FastSyncDownloader<REQUEST> {
|
||||
throw new IllegalStateException("SyncDownloader already running");
|
||||
}
|
||||
LOG.info("Starting pivot-based sync");
|
||||
|
||||
return start(initialFastSyncState);
|
||||
}
|
||||
|
||||
@@ -189,7 +194,8 @@ public class FastSyncDownloader<REQUEST> {
|
||||
}
|
||||
final CompletableFuture<Void> worldStateFuture =
|
||||
worldStateDownloader.run(fastSyncActions, currentState);
|
||||
final ChainDownloader chainDownloader = fastSyncActions.createChainDownloader(currentState);
|
||||
final ChainDownloader chainDownloader =
|
||||
fastSyncActions.createChainDownloader(currentState, syncDurationMetrics);
|
||||
final CompletableFuture<Void> chainFuture = chainDownloader.start();
|
||||
|
||||
// If either download fails, cancel the other one.
|
||||
|
||||
@@ -30,6 +30,7 @@ import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ScheduleBasedBlockHeaderFunctions;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.BesuMetricCategory;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.services.tasks.InMemoryTasksPriorityQueues;
|
||||
|
||||
@@ -60,7 +61,8 @@ public class FastDownloaderFactory {
|
||||
final EthContext ethContext,
|
||||
final WorldStateStorageCoordinator worldStateStorageCoordinator,
|
||||
final SyncState syncState,
|
||||
final Clock clock) {
|
||||
final Clock clock,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
|
||||
final Path fastSyncDataDirectory = dataDirectory.resolve(FAST_SYNC_FOLDER);
|
||||
final FastSyncStateStorage fastSyncStateStorage =
|
||||
@@ -114,7 +116,8 @@ public class FastDownloaderFactory {
|
||||
syncConfig.getWorldStateMaxRequestsWithoutProgress(),
|
||||
syncConfig.getWorldStateMinMillisBeforeStalling(),
|
||||
clock,
|
||||
metricsSystem);
|
||||
metricsSystem,
|
||||
syncDurationMetrics);
|
||||
final FastSyncDownloader<NodeDataRequest> fastSyncDownloader =
|
||||
new FastSyncDownloader<>(
|
||||
new FastSyncActions(
|
||||
@@ -131,7 +134,8 @@ public class FastDownloaderFactory {
|
||||
fastSyncStateStorage,
|
||||
taskCollection,
|
||||
fastSyncDataDirectory,
|
||||
fastSyncState);
|
||||
fastSyncState,
|
||||
syncDurationMetrics);
|
||||
syncState.setWorldStateDownloadStatus(worldStateDownloader);
|
||||
return Optional.of(fastSyncDownloader);
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ import org.hyperledger.besu.ethereum.core.BlockHeader;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.worldstate.WorldDownloadState;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.services.tasks.InMemoryTasksPriorityQueues;
|
||||
|
||||
import java.time.Clock;
|
||||
@@ -37,13 +38,15 @@ public class FastWorldDownloadState extends WorldDownloadState<NodeDataRequest>
|
||||
final InMemoryTasksPriorityQueues<NodeDataRequest> pendingRequests,
|
||||
final int maxRequestsWithoutProgress,
|
||||
final long minMillisBeforeStalling,
|
||||
final Clock clock) {
|
||||
final Clock clock,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
super(
|
||||
worldStateStorageCoordinator,
|
||||
pendingRequests,
|
||||
maxRequestsWithoutProgress,
|
||||
minMillisBeforeStalling,
|
||||
clock);
|
||||
clock,
|
||||
syncDurationMetrics);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -70,7 +73,9 @@ public class FastWorldDownloadState extends WorldDownloadState<NodeDataRequest>
|
||||
// THere are no more inputs to process so make sure we wake up any threads waiting to dequeue
|
||||
// so they can give up waiting.
|
||||
notifyAll();
|
||||
|
||||
LOG.info("Finished downloading world state from peers");
|
||||
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.hyperledger.besu.ethereum.eth.sync.fastsync.FastSyncState;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.worldstate.WorldStateDownloader;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.BesuMetricCategory;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.services.tasks.InMemoryTasksPriorityQueues;
|
||||
|
||||
@@ -51,6 +52,7 @@ public class FastWorldStateDownloader implements WorldStateDownloader {
|
||||
private final WorldStateStorageCoordinator worldStateStorageCoordinator;
|
||||
|
||||
private final AtomicReference<FastWorldDownloadState> downloadState = new AtomicReference<>();
|
||||
private final SyncDurationMetrics syncDurationMetrics;
|
||||
|
||||
private Optional<CompleteTaskStep> maybeCompleteTask = Optional.empty();
|
||||
|
||||
@@ -63,7 +65,8 @@ public class FastWorldStateDownloader implements WorldStateDownloader {
|
||||
final int maxNodeRequestsWithoutProgress,
|
||||
final long minMillisBeforeStalling,
|
||||
final Clock clock,
|
||||
final MetricsSystem metricsSystem) {
|
||||
final MetricsSystem metricsSystem,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
this.ethContext = ethContext;
|
||||
this.worldStateStorageCoordinator = worldStateStorageCoordinator;
|
||||
this.taskCollection = taskCollection;
|
||||
@@ -73,6 +76,7 @@ public class FastWorldStateDownloader implements WorldStateDownloader {
|
||||
this.minMillisBeforeStalling = minMillisBeforeStalling;
|
||||
this.clock = clock;
|
||||
this.metricsSystem = metricsSystem;
|
||||
this.syncDurationMetrics = syncDurationMetrics;
|
||||
|
||||
metricsSystem.createIntegerGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
@@ -137,7 +141,8 @@ public class FastWorldStateDownloader implements WorldStateDownloader {
|
||||
taskCollection,
|
||||
maxNodeRequestsWithoutProgress,
|
||||
minMillisBeforeStalling,
|
||||
clock);
|
||||
clock,
|
||||
syncDurationMetrics);
|
||||
this.downloadState.set(newDownloadState);
|
||||
|
||||
if (!newDownloadState.downloadWasResumed()) {
|
||||
|
||||
@@ -21,6 +21,7 @@ import org.hyperledger.besu.ethereum.eth.sync.PipelineChainDownloader;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.SynchronizerConfiguration;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
|
||||
public class FullSyncChainDownloader {
|
||||
@@ -33,7 +34,8 @@ public class FullSyncChainDownloader {
|
||||
final EthContext ethContext,
|
||||
final SyncState syncState,
|
||||
final MetricsSystem metricsSystem,
|
||||
final SyncTerminationCondition terminationCondition) {
|
||||
final SyncTerminationCondition terminationCondition,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
|
||||
final FullSyncTargetManager syncTargetManager =
|
||||
new FullSyncTargetManager(
|
||||
@@ -55,6 +57,7 @@ public class FullSyncChainDownloader {
|
||||
metricsSystem,
|
||||
terminationCondition),
|
||||
ethContext.getScheduler(),
|
||||
metricsSystem);
|
||||
metricsSystem,
|
||||
syncDurationMetrics);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,6 +21,7 @@ import org.hyperledger.besu.ethereum.eth.sync.SynchronizerConfiguration;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.TrailingPeerRequirements;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
@@ -43,7 +44,8 @@ public class FullSyncDownloader {
|
||||
final EthContext ethContext,
|
||||
final SyncState syncState,
|
||||
final MetricsSystem metricsSystem,
|
||||
final SyncTerminationCondition terminationCondition) {
|
||||
final SyncTerminationCondition terminationCondition,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
this.syncConfig = syncConfig;
|
||||
this.protocolContext = protocolContext;
|
||||
this.syncState = syncState;
|
||||
@@ -56,7 +58,8 @@ public class FullSyncDownloader {
|
||||
ethContext,
|
||||
syncState,
|
||||
metricsSystem,
|
||||
terminationCondition);
|
||||
terminationCondition,
|
||||
syncDurationMetrics);
|
||||
}
|
||||
|
||||
public CompletableFuture<Void> start() {
|
||||
|
||||
@@ -33,6 +33,7 @@ import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ScheduleBasedBlockHeaderFunctions;
|
||||
import org.hyperledger.besu.ethereum.trie.CompactEncoding;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.services.tasks.InMemoryTasksPriorityQueues;
|
||||
|
||||
@@ -58,7 +59,8 @@ public class SnapDownloaderFactory extends FastDownloaderFactory {
|
||||
final EthContext ethContext,
|
||||
final WorldStateStorageCoordinator worldStateStorageCoordinator,
|
||||
final SyncState syncState,
|
||||
final Clock clock) {
|
||||
final Clock clock,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
|
||||
final Path fastSyncDataDirectory = dataDirectory.resolve(FAST_SYNC_FOLDER);
|
||||
final FastSyncStateStorage fastSyncStateStorage =
|
||||
@@ -93,10 +95,7 @@ public class SnapDownloaderFactory extends FastDownloaderFactory {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
final SnapSyncProcessState snapSyncState =
|
||||
new SnapSyncProcessState(
|
||||
fastSyncStateStorage.loadState(
|
||||
ScheduleBasedBlockHeaderFunctions.create(protocolSchedule)));
|
||||
final SnapSyncProcessState snapSyncState = new SnapSyncProcessState(fastSyncState);
|
||||
|
||||
final InMemoryTasksPriorityQueues<SnapDataRequest> snapTaskCollection =
|
||||
createSnapWorldStateDownloaderTaskCollection();
|
||||
@@ -112,7 +111,8 @@ public class SnapDownloaderFactory extends FastDownloaderFactory {
|
||||
syncConfig.getWorldStateMaxRequestsWithoutProgress(),
|
||||
syncConfig.getWorldStateMinMillisBeforeStalling(),
|
||||
clock,
|
||||
metricsSystem);
|
||||
metricsSystem,
|
||||
syncDurationMetrics);
|
||||
final FastSyncDownloader<SnapDataRequest> fastSyncDownloader =
|
||||
new SnapSyncDownloader(
|
||||
new FastSyncActions(
|
||||
@@ -129,7 +129,8 @@ public class SnapDownloaderFactory extends FastDownloaderFactory {
|
||||
fastSyncStateStorage,
|
||||
snapTaskCollection,
|
||||
fastSyncDataDirectory,
|
||||
snapSyncState);
|
||||
snapSyncState,
|
||||
syncDurationMetrics);
|
||||
syncState.setWorldStateDownloadStatus(snapWorldStateDownloader);
|
||||
return Optional.of(fastSyncDownloader);
|
||||
}
|
||||
|
||||
@@ -21,6 +21,7 @@ import org.hyperledger.besu.ethereum.eth.sync.fastsync.FastSyncStateStorage;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.snapsync.request.SnapDataRequest;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.worldstate.WorldStateDownloader;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.services.tasks.TaskCollection;
|
||||
|
||||
import java.nio.file.Path;
|
||||
@@ -35,7 +36,8 @@ public class SnapSyncDownloader extends FastSyncDownloader<SnapDataRequest> {
|
||||
final FastSyncStateStorage fastSyncStateStorage,
|
||||
final TaskCollection<SnapDataRequest> taskCollection,
|
||||
final Path fastSyncDataDirectory,
|
||||
final FastSyncState initialFastSyncState) {
|
||||
final FastSyncState initialFastSyncState,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
super(
|
||||
fastSyncActions,
|
||||
worldStateStorageCoordinator,
|
||||
@@ -43,7 +45,8 @@ public class SnapSyncDownloader extends FastSyncDownloader<SnapDataRequest> {
|
||||
fastSyncStateStorage,
|
||||
taskCollection,
|
||||
fastSyncDataDirectory,
|
||||
initialFastSyncState);
|
||||
initialFastSyncState,
|
||||
syncDurationMetrics);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -36,6 +36,8 @@ import org.hyperledger.besu.ethereum.worldstate.FlatDbMode;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.BesuMetricCategory;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
import org.hyperledger.besu.services.tasks.InMemoryTaskQueue;
|
||||
import org.hyperledger.besu.services.tasks.InMemoryTasksPriorityQueues;
|
||||
@@ -92,6 +94,8 @@ public class SnapWorldDownloadState extends WorldDownloadState<SnapDataRequest>
|
||||
// metrics around the snapsync
|
||||
private final SnapSyncMetricsManager metricsManager;
|
||||
|
||||
private final AtomicBoolean trieHealStartedBefore = new AtomicBoolean(false);
|
||||
|
||||
public SnapWorldDownloadState(
|
||||
final WorldStateStorageCoordinator worldStateStorageCoordinator,
|
||||
final SnapSyncStatePersistenceManager snapContext,
|
||||
@@ -102,13 +106,15 @@ public class SnapWorldDownloadState extends WorldDownloadState<SnapDataRequest>
|
||||
final long minMillisBeforeStalling,
|
||||
final SnapSyncMetricsManager metricsManager,
|
||||
final Clock clock,
|
||||
final EthContext ethContext) {
|
||||
final EthContext ethContext,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
super(
|
||||
worldStateStorageCoordinator,
|
||||
pendingRequests,
|
||||
maxRequestsWithoutProgress,
|
||||
minMillisBeforeStalling,
|
||||
clock);
|
||||
clock,
|
||||
syncDurationMetrics);
|
||||
this.snapContext = snapContext;
|
||||
this.blockchain = blockchain;
|
||||
this.snapSyncState = snapSyncState;
|
||||
@@ -116,46 +122,34 @@ public class SnapWorldDownloadState extends WorldDownloadState<SnapDataRequest>
|
||||
this.blockObserverId = blockchain.observeBlockAdded(createBlockchainObserver());
|
||||
this.ethContext = ethContext;
|
||||
|
||||
metricsManager
|
||||
.getMetricsSystem()
|
||||
.createLongGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
"snap_world_state_pending_account_requests_current",
|
||||
"Number of account pending requests for snap sync world state download",
|
||||
pendingAccountRequests::size);
|
||||
metricsManager
|
||||
.getMetricsSystem()
|
||||
.createLongGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
"snap_world_state_pending_storage_requests_current",
|
||||
"Number of storage pending requests for snap sync world state download",
|
||||
pendingStorageRequests::size);
|
||||
metricsManager
|
||||
.getMetricsSystem()
|
||||
.createLongGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
"snap_world_state_pending_big_storage_requests_current",
|
||||
"Number of storage pending requests for snap sync world state download",
|
||||
pendingLargeStorageRequests::size);
|
||||
metricsManager
|
||||
.getMetricsSystem()
|
||||
.createLongGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
"snap_world_state_pending_code_requests_current",
|
||||
"Number of code pending requests for snap sync world state download",
|
||||
pendingCodeRequests::size);
|
||||
metricsManager
|
||||
.getMetricsSystem()
|
||||
.createLongGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
"snap_world_state_pending_trie_node_requests_current",
|
||||
"Number of trie node pending requests for snap sync world state download",
|
||||
pendingTrieNodeRequests::size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void notifyTaskAvailable() {
|
||||
notifyAll();
|
||||
final MetricsSystem metricsSystem = metricsManager.getMetricsSystem();
|
||||
metricsSystem.createLongGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
"snap_world_state_pending_account_requests_current",
|
||||
"Number of account pending requests for snap sync world state download",
|
||||
pendingAccountRequests::size);
|
||||
metricsSystem.createLongGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
"snap_world_state_pending_storage_requests_current",
|
||||
"Number of storage pending requests for snap sync world state download",
|
||||
pendingStorageRequests::size);
|
||||
metricsSystem.createLongGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
"snap_world_state_pending_big_storage_requests_current",
|
||||
"Number of storage pending requests for snap sync world state download",
|
||||
pendingLargeStorageRequests::size);
|
||||
metricsSystem.createLongGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
"snap_world_state_pending_code_requests_current",
|
||||
"Number of code pending requests for snap sync world state download",
|
||||
pendingCodeRequests::size);
|
||||
metricsSystem.createLongGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
"snap_world_state_pending_trie_node_requests_current",
|
||||
"Number of trie node pending requests for snap sync world state download",
|
||||
pendingTrieNodeRequests::size);
|
||||
syncDurationMetrics.startTimer(
|
||||
SyncDurationMetrics.Labels.SNAP_INITIAL_WORLD_STATE_DOWNLOAD_DURATION);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -191,6 +185,9 @@ public class SnapWorldDownloadState extends WorldDownloadState<SnapDataRequest>
|
||||
// if all snapsync tasks are completed and the healing was running and the blockchain is not
|
||||
// behind the pivot block
|
||||
else {
|
||||
syncDurationMetrics.stopTimer(SyncDurationMetrics.Labels.SNAP_WORLD_STATE_HEALING_DURATION);
|
||||
syncDurationMetrics.stopTimer(SyncDurationMetrics.Labels.CHAIN_DOWNLOAD_DURATION);
|
||||
|
||||
// If the flat database healing process is not in progress and the flat database mode is
|
||||
// FULL
|
||||
if (!snapSyncState.isHealFlatDatabaseInProgress()
|
||||
@@ -217,6 +214,7 @@ public class SnapWorldDownloadState extends WorldDownloadState<SnapDataRequest>
|
||||
// Clear the snap context
|
||||
snapContext.clear();
|
||||
internalFuture.complete(null);
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -236,6 +234,12 @@ public class SnapWorldDownloadState extends WorldDownloadState<SnapDataRequest>
|
||||
|
||||
/** Method to start the healing process of the trie */
|
||||
public synchronized void startTrieHeal() {
|
||||
if (trieHealStartedBefore.compareAndSet(false, true)) {
|
||||
syncDurationMetrics.stopTimer(
|
||||
SyncDurationMetrics.Labels.SNAP_INITIAL_WORLD_STATE_DOWNLOAD_DURATION);
|
||||
|
||||
syncDurationMetrics.startTimer(SyncDurationMetrics.Labels.SNAP_WORLD_STATE_HEALING_DURATION);
|
||||
}
|
||||
snapContext.clearAccountRangeTasks();
|
||||
snapSyncState.setHealTrieStatus(true);
|
||||
// Try to find a new pivot block before starting the healing process
|
||||
@@ -272,6 +276,7 @@ public class SnapWorldDownloadState extends WorldDownloadState<SnapDataRequest>
|
||||
|
||||
public synchronized void startFlatDatabaseHeal(final BlockHeader header) {
|
||||
LOG.info("Initiating the healing process for the flat database");
|
||||
syncDurationMetrics.startTimer(SyncDurationMetrics.Labels.FLAT_DB_HEAL);
|
||||
snapSyncState.setHealFlatDatabaseInProgress(true);
|
||||
final Map<Bytes32, Bytes32> ranges = RangeManager.generateAllRanges(16);
|
||||
ranges.forEach(
|
||||
|
||||
@@ -31,6 +31,7 @@ import org.hyperledger.besu.ethereum.trie.RangeManager;
|
||||
import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.BesuMetricCategory;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
import org.hyperledger.besu.services.tasks.InMemoryTasksPriorityQueues;
|
||||
@@ -67,6 +68,7 @@ public class SnapWorldStateDownloader implements WorldStateDownloader {
|
||||
private final WorldStateStorageCoordinator worldStateStorageCoordinator;
|
||||
|
||||
private final AtomicReference<SnapWorldDownloadState> downloadState = new AtomicReference<>();
|
||||
private final SyncDurationMetrics syncDurationMetrics;
|
||||
|
||||
public SnapWorldStateDownloader(
|
||||
final EthContext ethContext,
|
||||
@@ -79,7 +81,8 @@ public class SnapWorldStateDownloader implements WorldStateDownloader {
|
||||
final int maxNodeRequestsWithoutProgress,
|
||||
final long minMillisBeforeStalling,
|
||||
final Clock clock,
|
||||
final MetricsSystem metricsSystem) {
|
||||
final MetricsSystem metricsSystem,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
this.ethContext = ethContext;
|
||||
this.protocolContext = protocolContext;
|
||||
this.worldStateStorageCoordinator = worldStateStorageCoordinator;
|
||||
@@ -91,6 +94,7 @@ public class SnapWorldStateDownloader implements WorldStateDownloader {
|
||||
this.minMillisBeforeStalling = minMillisBeforeStalling;
|
||||
this.clock = clock;
|
||||
this.metricsSystem = metricsSystem;
|
||||
this.syncDurationMetrics = syncDurationMetrics;
|
||||
|
||||
metricsSystem.createIntegerGauge(
|
||||
BesuMetricCategory.SYNCHRONIZER,
|
||||
@@ -148,7 +152,8 @@ public class SnapWorldStateDownloader implements WorldStateDownloader {
|
||||
minMillisBeforeStalling,
|
||||
snapsyncMetricsManager,
|
||||
clock,
|
||||
ethContext);
|
||||
ethContext,
|
||||
syncDurationMetrics);
|
||||
|
||||
final Map<Bytes32, Bytes32> ranges = RangeManager.generateAllRanges(16);
|
||||
snapsyncMetricsManager.initRange(ranges);
|
||||
|
||||
@@ -18,6 +18,7 @@ import org.hyperledger.besu.ethereum.core.BlockHeader;
|
||||
import org.hyperledger.besu.ethereum.eth.manager.EthScheduler;
|
||||
import org.hyperledger.besu.ethereum.eth.manager.task.EthTask;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.services.tasks.InMemoryTasksPriorityQueues;
|
||||
import org.hyperledger.besu.services.tasks.Task;
|
||||
import org.hyperledger.besu.services.tasks.TasksPriorityProvider;
|
||||
@@ -37,6 +38,7 @@ import org.slf4j.LoggerFactory;
|
||||
|
||||
public abstract class WorldDownloadState<REQUEST extends TasksPriorityProvider> {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(WorldDownloadState.class);
|
||||
protected final SyncDurationMetrics syncDurationMetrics;
|
||||
|
||||
private boolean downloadWasResumed;
|
||||
protected final InMemoryTasksPriorityQueues<REQUEST> pendingRequests;
|
||||
@@ -61,7 +63,8 @@ public abstract class WorldDownloadState<REQUEST extends TasksPriorityProvider>
|
||||
final InMemoryTasksPriorityQueues<REQUEST> pendingRequests,
|
||||
final int maxRequestsWithoutProgress,
|
||||
final long minMillisBeforeStalling,
|
||||
final Clock clock) {
|
||||
final Clock clock,
|
||||
final SyncDurationMetrics syncDurationMetrics) {
|
||||
this.worldStateStorageCoordinator = worldStateStorageCoordinator;
|
||||
this.minMillisBeforeStalling = minMillisBeforeStalling;
|
||||
this.timestampOfLastProgress = clock.millis();
|
||||
@@ -69,6 +72,7 @@ public abstract class WorldDownloadState<REQUEST extends TasksPriorityProvider>
|
||||
this.pendingRequests = pendingRequests;
|
||||
this.maxRequestsWithoutProgress = maxRequestsWithoutProgress;
|
||||
this.clock = clock;
|
||||
this.syncDurationMetrics = syncDurationMetrics;
|
||||
this.internalFuture = new CompletableFuture<>();
|
||||
this.downloadFuture = new CompletableFuture<>();
|
||||
this.internalFuture.whenComplete(this::cleanup);
|
||||
|
||||
@@ -34,6 +34,7 @@ import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.state.SyncTarget;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.tasks.exceptions.InvalidBlockException;
|
||||
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.messages.DisconnectMessage.DisconnectReason;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.services.pipeline.Pipeline;
|
||||
|
||||
@@ -69,13 +70,15 @@ public class PipelineChainDownloaderTest {
|
||||
public void setUp() {
|
||||
syncTarget = new SyncTarget(peer1, commonAncestor);
|
||||
syncTarget2 = new SyncTarget(peer2, commonAncestor);
|
||||
final NoOpMetricsSystem noOpMetricsSystem = new NoOpMetricsSystem();
|
||||
chainDownloader =
|
||||
new PipelineChainDownloader(
|
||||
syncState,
|
||||
syncTargetManager,
|
||||
downloadPipelineFactory,
|
||||
scheduler,
|
||||
new NoOpMetricsSystem());
|
||||
noOpMetricsSystem,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
@@ -40,6 +40,7 @@ import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldSt
|
||||
import org.hyperledger.besu.ethereum.trie.forest.storage.ForestWorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
|
||||
@@ -141,7 +142,8 @@ public class CheckPointSyncChainDownloaderTest {
|
||||
ethContext,
|
||||
syncState,
|
||||
new NoOpMetricsSystem(),
|
||||
new FastSyncState(otherBlockchain.getBlockHeader(pivotBlockNumber).get()));
|
||||
new FastSyncState(otherBlockchain.getBlockHeader(pivotBlockNumber).get()),
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
|
||||
@@ -35,6 +35,7 @@ import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldSt
|
||||
import org.hyperledger.besu.ethereum.trie.forest.storage.ForestWorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
|
||||
@@ -115,7 +116,8 @@ public class FastDownloaderFactoryTest {
|
||||
ethContext,
|
||||
worldStateStorageCoordinator,
|
||||
syncState,
|
||||
clock))
|
||||
clock,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.isInstanceOf(IllegalStateException.class);
|
||||
}
|
||||
|
||||
@@ -139,7 +141,8 @@ public class FastDownloaderFactoryTest {
|
||||
ethContext,
|
||||
worldStateStorageCoordinator,
|
||||
syncState,
|
||||
clock);
|
||||
clock,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
assertThat(result).isEmpty();
|
||||
}
|
||||
|
||||
@@ -166,7 +169,8 @@ public class FastDownloaderFactoryTest {
|
||||
ethContext,
|
||||
worldStateStorageCoordinator,
|
||||
syncState,
|
||||
clock);
|
||||
clock,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
|
||||
verify(mutableBlockchain).getChainHeadBlockNumber();
|
||||
}
|
||||
@@ -200,7 +204,8 @@ public class FastDownloaderFactoryTest {
|
||||
ethContext,
|
||||
worldStateStorageCoordinator,
|
||||
syncState,
|
||||
clock);
|
||||
clock,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
|
||||
verify(worldStateKeyValueStorage).clear();
|
||||
assertThat(Files.exists(stateQueueDir)).isFalse();
|
||||
@@ -236,7 +241,8 @@ public class FastDownloaderFactoryTest {
|
||||
ethContext,
|
||||
worldStateStorageCoordinator,
|
||||
syncState,
|
||||
clock))
|
||||
clock,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.isInstanceOf(IllegalStateException.class);
|
||||
}
|
||||
|
||||
|
||||
@@ -36,6 +36,7 @@ import org.hyperledger.besu.ethereum.eth.sync.SynchronizerConfiguration;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
|
||||
@@ -111,7 +112,8 @@ public class FastSyncChainDownloaderTest {
|
||||
ethContext,
|
||||
syncState,
|
||||
new NoOpMetricsSystem(),
|
||||
new FastSyncState(otherBlockchain.getBlockHeader(pivotBlockNumber).get()));
|
||||
new FastSyncState(otherBlockchain.getBlockHeader(pivotBlockNumber).get()),
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
|
||||
@@ -38,6 +38,7 @@ import org.hyperledger.besu.ethereum.trie.diffbased.bonsai.storage.BonsaiWorldSt
|
||||
import org.hyperledger.besu.ethereum.trie.forest.storage.ForestWorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
import org.hyperledger.besu.services.tasks.TaskCollection;
|
||||
|
||||
@@ -105,7 +106,8 @@ public class FastSyncDownloaderTest {
|
||||
storage,
|
||||
taskCollection,
|
||||
fastSyncDataDirectory,
|
||||
FastSyncState.EMPTY_SYNC_STATE);
|
||||
FastSyncState.EMPTY_SYNC_STATE,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@@ -119,7 +121,8 @@ public class FastSyncDownloaderTest {
|
||||
.thenReturn(completedFuture(selectPivotBlockState));
|
||||
when(fastSyncActions.downloadPivotBlockHeader(selectPivotBlockState))
|
||||
.thenReturn(completedFuture(downloadPivotBlockHeaderState));
|
||||
when(fastSyncActions.createChainDownloader(downloadPivotBlockHeaderState))
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(chainDownloader);
|
||||
when(chainDownloader.start()).thenReturn(completedFuture(null));
|
||||
when(worldStateDownloader.run(
|
||||
@@ -131,7 +134,9 @@ public class FastSyncDownloaderTest {
|
||||
verify(fastSyncActions).selectPivotBlock(FastSyncState.EMPTY_SYNC_STATE);
|
||||
verify(fastSyncActions).downloadPivotBlockHeader(selectPivotBlockState);
|
||||
verify(storage).storeState(downloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions).createChainDownloader(downloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions)
|
||||
.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
verify(chainDownloader).start();
|
||||
verify(worldStateDownloader)
|
||||
.run(any(FastSyncActions.class), eq(new FastSyncState(pivotBlockHeader)));
|
||||
@@ -148,7 +153,9 @@ public class FastSyncDownloaderTest {
|
||||
final CompletableFuture<FastSyncState> complete = completedFuture(fastSyncState);
|
||||
when(fastSyncActions.selectPivotBlock(fastSyncState)).thenReturn(complete);
|
||||
when(fastSyncActions.downloadPivotBlockHeader(fastSyncState)).thenReturn(complete);
|
||||
when(fastSyncActions.createChainDownloader(fastSyncState)).thenReturn(chainDownloader);
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
fastSyncState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(chainDownloader);
|
||||
when(chainDownloader.start()).thenReturn(completedFuture(null));
|
||||
when(worldStateDownloader.run(
|
||||
any(FastSyncActions.class), eq(new FastSyncState(pivotBlockHeader))))
|
||||
@@ -162,14 +169,16 @@ public class FastSyncDownloaderTest {
|
||||
storage,
|
||||
taskCollection,
|
||||
fastSyncDataDirectory,
|
||||
fastSyncState);
|
||||
fastSyncState,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
|
||||
final CompletableFuture<FastSyncState> result = resumedDownloader.start();
|
||||
|
||||
verify(fastSyncActions).selectPivotBlock(fastSyncState);
|
||||
verify(fastSyncActions).downloadPivotBlockHeader(fastSyncState);
|
||||
verify(storage).storeState(fastSyncState);
|
||||
verify(fastSyncActions).createChainDownloader(fastSyncState);
|
||||
verify(fastSyncActions)
|
||||
.createChainDownloader(fastSyncState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
verify(chainDownloader).start();
|
||||
verify(worldStateDownloader)
|
||||
.run(any(FastSyncActions.class), eq(new FastSyncState(pivotBlockHeader)));
|
||||
@@ -206,7 +215,8 @@ public class FastSyncDownloaderTest {
|
||||
.thenReturn(completedFuture(selectPivotBlockState));
|
||||
when(fastSyncActions.downloadPivotBlockHeader(selectPivotBlockState))
|
||||
.thenReturn(completedFuture(downloadPivotBlockHeaderState));
|
||||
when(fastSyncActions.createChainDownloader(downloadPivotBlockHeaderState))
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(chainDownloader);
|
||||
when(chainDownloader.start()).thenReturn(chainFuture);
|
||||
when(worldStateDownloader.run(
|
||||
@@ -218,7 +228,9 @@ public class FastSyncDownloaderTest {
|
||||
verify(fastSyncActions).selectPivotBlock(FastSyncState.EMPTY_SYNC_STATE);
|
||||
verify(fastSyncActions).downloadPivotBlockHeader(selectPivotBlockState);
|
||||
verify(storage).storeState(downloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions).createChainDownloader(downloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions)
|
||||
.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
verify(worldStateDownloader)
|
||||
.run(any(FastSyncActions.class), eq(new FastSyncState(pivotBlockHeader)));
|
||||
verifyNoMoreInteractions(fastSyncActions, worldStateDownloader, storage);
|
||||
@@ -246,7 +258,8 @@ public class FastSyncDownloaderTest {
|
||||
.thenReturn(completedFuture(selectPivotBlockState));
|
||||
when(fastSyncActions.downloadPivotBlockHeader(selectPivotBlockState))
|
||||
.thenReturn(completedFuture(downloadPivotBlockHeaderState));
|
||||
when(fastSyncActions.createChainDownloader(downloadPivotBlockHeaderState))
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(chainDownloader);
|
||||
when(chainDownloader.start()).thenReturn(chainFuture);
|
||||
when(worldStateDownloader.run(
|
||||
@@ -257,7 +270,9 @@ public class FastSyncDownloaderTest {
|
||||
|
||||
verify(fastSyncActions).selectPivotBlock(FastSyncState.EMPTY_SYNC_STATE);
|
||||
verify(fastSyncActions).downloadPivotBlockHeader(selectPivotBlockState);
|
||||
verify(fastSyncActions).createChainDownloader(downloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions)
|
||||
.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
verify(worldStateDownloader)
|
||||
.run(any(FastSyncActions.class), eq(new FastSyncState(pivotBlockHeader)));
|
||||
verifyNoMoreInteractions(fastSyncActions);
|
||||
@@ -321,7 +336,8 @@ public class FastSyncDownloaderTest {
|
||||
.thenReturn(completedFuture(selectPivotBlockState));
|
||||
when(fastSyncActions.downloadPivotBlockHeader(selectPivotBlockState))
|
||||
.thenReturn(completedFuture(downloadPivotBlockHeaderState));
|
||||
when(fastSyncActions.createChainDownloader(downloadPivotBlockHeaderState))
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(chainDownloader);
|
||||
when(chainDownloader.start()).thenReturn(chainFuture);
|
||||
when(worldStateDownloader.run(
|
||||
@@ -332,7 +348,9 @@ public class FastSyncDownloaderTest {
|
||||
|
||||
verify(fastSyncActions).selectPivotBlock(FastSyncState.EMPTY_SYNC_STATE);
|
||||
verify(fastSyncActions).downloadPivotBlockHeader(selectPivotBlockState);
|
||||
verify(fastSyncActions).createChainDownloader(downloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions)
|
||||
.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
verify(worldStateDownloader)
|
||||
.run(any(FastSyncActions.class), eq(new FastSyncState(pivotBlockHeader)));
|
||||
verifyNoMoreInteractions(fastSyncActions);
|
||||
@@ -359,7 +377,8 @@ public class FastSyncDownloaderTest {
|
||||
.thenReturn(completedFuture(selectPivotBlockState));
|
||||
when(fastSyncActions.downloadPivotBlockHeader(selectPivotBlockState))
|
||||
.thenReturn(completedFuture(downloadPivotBlockHeaderState));
|
||||
when(fastSyncActions.createChainDownloader(downloadPivotBlockHeaderState))
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(chainDownloader);
|
||||
when(chainDownloader.start()).thenReturn(chainFuture);
|
||||
when(worldStateDownloader.run(
|
||||
@@ -370,7 +389,9 @@ public class FastSyncDownloaderTest {
|
||||
|
||||
verify(fastSyncActions).selectPivotBlock(FastSyncState.EMPTY_SYNC_STATE);
|
||||
verify(fastSyncActions).downloadPivotBlockHeader(selectPivotBlockState);
|
||||
verify(fastSyncActions).createChainDownloader(downloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions)
|
||||
.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
verify(worldStateDownloader)
|
||||
.run(any(FastSyncActions.class), eq(new FastSyncState(pivotBlockHeader)));
|
||||
verifyNoMoreInteractions(fastSyncActions);
|
||||
@@ -407,7 +428,8 @@ public class FastSyncDownloaderTest {
|
||||
completedFuture(selectPivotBlockState), completedFuture(secondSelectPivotBlockState));
|
||||
when(fastSyncActions.downloadPivotBlockHeader(selectPivotBlockState))
|
||||
.thenReturn(completedFuture(downloadPivotBlockHeaderState));
|
||||
when(fastSyncActions.createChainDownloader(downloadPivotBlockHeaderState))
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(chainDownloader);
|
||||
when(chainDownloader.start()).thenReturn(chainFuture);
|
||||
when(worldStateDownloader.run(
|
||||
@@ -418,7 +440,8 @@ public class FastSyncDownloaderTest {
|
||||
when(fastSyncActions.downloadPivotBlockHeader(secondSelectPivotBlockState))
|
||||
.thenReturn(completedFuture(secondDownloadPivotBlockHeaderState));
|
||||
|
||||
when(fastSyncActions.createChainDownloader(secondDownloadPivotBlockHeaderState))
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
secondDownloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(secondChainDownloader);
|
||||
when(secondChainDownloader.start()).thenReturn(completedFuture(null));
|
||||
when(worldStateDownloader.run(
|
||||
@@ -430,7 +453,9 @@ public class FastSyncDownloaderTest {
|
||||
verify(fastSyncActions).selectPivotBlock(FastSyncState.EMPTY_SYNC_STATE);
|
||||
verify(fastSyncActions).downloadPivotBlockHeader(selectPivotBlockState);
|
||||
verify(storage).storeState(downloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions).createChainDownloader(downloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions)
|
||||
.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
verify(worldStateDownloader)
|
||||
.run(any(FastSyncActions.class), eq(new FastSyncState(pivotBlockHeader)));
|
||||
verifyNoMoreInteractions(fastSyncActions, worldStateDownloader, storage);
|
||||
@@ -446,7 +471,9 @@ public class FastSyncDownloaderTest {
|
||||
verify(fastSyncActions, times(2)).selectPivotBlock(FastSyncState.EMPTY_SYNC_STATE);
|
||||
verify(fastSyncActions).downloadPivotBlockHeader(secondSelectPivotBlockState);
|
||||
verify(storage).storeState(secondDownloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions).createChainDownloader(secondDownloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions)
|
||||
.createChainDownloader(
|
||||
secondDownloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
verify(worldStateDownloader)
|
||||
.run(any(FastSyncActions.class), eq(new FastSyncState(secondPivotBlockHeader)));
|
||||
verifyNoMoreInteractions(fastSyncActions, worldStateDownloader, storage);
|
||||
@@ -481,7 +508,8 @@ public class FastSyncDownloaderTest {
|
||||
completedFuture(selectPivotBlockState), completedFuture(secondSelectPivotBlockState));
|
||||
when(fastSyncActions.downloadPivotBlockHeader(selectPivotBlockState))
|
||||
.thenReturn(completedFuture(downloadPivotBlockHeaderState));
|
||||
when(fastSyncActions.createChainDownloader(downloadPivotBlockHeaderState))
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(chainDownloader);
|
||||
when(chainDownloader.start()).thenReturn(chainFuture);
|
||||
when(worldStateDownloader.run(
|
||||
@@ -494,7 +522,8 @@ public class FastSyncDownloaderTest {
|
||||
when(fastSyncActions.downloadPivotBlockHeader(secondSelectPivotBlockState))
|
||||
.thenReturn(completedFuture(secondDownloadPivotBlockHeaderState));
|
||||
|
||||
when(fastSyncActions.createChainDownloader(secondDownloadPivotBlockHeaderState))
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
secondDownloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(secondChainDownloader);
|
||||
when(secondChainDownloader.start()).thenReturn(completedFuture(null));
|
||||
when(worldStateDownloader.run(
|
||||
@@ -506,7 +535,9 @@ public class FastSyncDownloaderTest {
|
||||
verify(fastSyncActions).selectPivotBlock(FastSyncState.EMPTY_SYNC_STATE);
|
||||
verify(fastSyncActions).downloadPivotBlockHeader(selectPivotBlockState);
|
||||
verify(storage).storeState(downloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions).createChainDownloader(downloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions)
|
||||
.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
verify(worldStateDownloader)
|
||||
.run(any(FastSyncActions.class), eq(new FastSyncState(pivotBlockHeader)));
|
||||
verifyNoMoreInteractions(fastSyncActions, worldStateDownloader, storage);
|
||||
@@ -524,7 +555,9 @@ public class FastSyncDownloaderTest {
|
||||
verify(fastSyncActions, times(2)).selectPivotBlock(FastSyncState.EMPTY_SYNC_STATE);
|
||||
verify(fastSyncActions).downloadPivotBlockHeader(secondSelectPivotBlockState);
|
||||
verify(storage).storeState(secondDownloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions).createChainDownloader(secondDownloadPivotBlockHeaderState);
|
||||
verify(fastSyncActions)
|
||||
.createChainDownloader(
|
||||
secondDownloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
verify(worldStateDownloader)
|
||||
.run(any(FastSyncActions.class), eq(new FastSyncState(secondPivotBlockHeader)));
|
||||
verifyNoMoreInteractions(fastSyncActions, worldStateDownloader, storage);
|
||||
@@ -556,7 +589,8 @@ public class FastSyncDownloaderTest {
|
||||
.thenReturn(completedFuture(selectPivotBlockState));
|
||||
when(fastSyncActions.downloadPivotBlockHeader(selectPivotBlockState))
|
||||
.thenReturn(completedFuture(downloadPivotBlockHeaderState));
|
||||
when(fastSyncActions.createChainDownloader(downloadPivotBlockHeaderState))
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(chainDownloader);
|
||||
when(chainDownloader.start()).thenReturn(new CompletableFuture<>());
|
||||
when(worldStateDownloader.run(
|
||||
@@ -581,7 +615,8 @@ public class FastSyncDownloaderTest {
|
||||
.thenReturn(completedFuture(selectPivotBlockState));
|
||||
when(fastSyncActions.downloadPivotBlockHeader(selectPivotBlockState))
|
||||
.thenReturn(completedFuture(downloadPivotBlockHeaderState));
|
||||
when(fastSyncActions.createChainDownloader(downloadPivotBlockHeaderState))
|
||||
when(fastSyncActions.createChainDownloader(
|
||||
downloadPivotBlockHeaderState, SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS))
|
||||
.thenReturn(chainDownloader);
|
||||
when(chainDownloader.start()).thenReturn(completedFuture(null));
|
||||
when(worldStateDownloader.run(
|
||||
|
||||
@@ -31,6 +31,7 @@ import org.hyperledger.besu.ethereum.trie.forest.storage.ForestWorldStateKeyValu
|
||||
import org.hyperledger.besu.ethereum.worldstate.DataStorageConfiguration;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
import org.hyperledger.besu.services.kvstore.InMemoryKeyValueStorage;
|
||||
@@ -100,7 +101,8 @@ public class FastWorldDownloadStateTest {
|
||||
pendingRequests,
|
||||
MAX_REQUESTS_WITHOUT_PROGRESS,
|
||||
MIN_MILLIS_BEFORE_STALLING,
|
||||
clock);
|
||||
clock,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
assertThat(downloadState.isDownloading()).isTrue();
|
||||
downloadState.setRootNodeData(ROOT_NODE_DATA);
|
||||
future = downloadState.getDownloadFuture();
|
||||
|
||||
@@ -64,6 +64,7 @@ import org.hyperledger.besu.evm.account.Account;
|
||||
import org.hyperledger.besu.evm.account.AccountStorageEntry;
|
||||
import org.hyperledger.besu.evm.internal.EvmConfiguration;
|
||||
import org.hyperledger.besu.evm.worldstate.WorldState;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.services.kvstore.InMemoryKeyValueStorage;
|
||||
import org.hyperledger.besu.services.tasks.InMemoryTasksPriorityQueues;
|
||||
@@ -1051,7 +1052,8 @@ class FastWorldStateDownloaderTest {
|
||||
config.getWorldStateMaxRequestsWithoutProgress(),
|
||||
config.getWorldStateMinMillisBeforeStalling(),
|
||||
TestClock.fixed(),
|
||||
new NoOpMetricsSystem());
|
||||
new NoOpMetricsSystem(),
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
}
|
||||
|
||||
private WorldStatePreimageStorage createPreimageStorage() {
|
||||
|
||||
@@ -32,6 +32,7 @@ import org.hyperledger.besu.ethereum.eth.sync.SynchronizerConfiguration;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.messages.DisconnectMessage.DisconnectReason;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
|
||||
@@ -89,7 +90,8 @@ public class FullSyncChainDownloaderForkTest {
|
||||
ethContext,
|
||||
syncState,
|
||||
metricsSystem,
|
||||
SyncTerminationCondition.never());
|
||||
SyncTerminationCondition.never(),
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
}
|
||||
|
||||
private ChainDownloader downloader() {
|
||||
|
||||
@@ -42,6 +42,7 @@ import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.MessageData;
|
||||
import org.hyperledger.besu.ethereum.p2p.rlpx.wire.messages.DisconnectMessage.DisconnectReason;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
@@ -121,7 +122,8 @@ public class FullSyncChainDownloaderTest {
|
||||
ethContext,
|
||||
syncState,
|
||||
metricsSystem,
|
||||
SyncTerminationCondition.never());
|
||||
SyncTerminationCondition.never(),
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
}
|
||||
|
||||
private ChainDownloader downloader() {
|
||||
|
||||
@@ -31,6 +31,7 @@ import org.hyperledger.besu.ethereum.eth.sync.ChainDownloader;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.SynchronizerConfiguration;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
@@ -107,7 +108,8 @@ public class FullSyncChainDownloaderTotalTerminalDifficultyTest {
|
||||
ethContext,
|
||||
syncState,
|
||||
metricsSystem,
|
||||
terminalCondition);
|
||||
terminalCondition,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
}
|
||||
|
||||
private SynchronizerConfiguration.Builder syncConfigBuilder() {
|
||||
|
||||
@@ -29,6 +29,7 @@ import org.hyperledger.besu.ethereum.eth.sync.SynchronizerConfiguration;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.TrailingPeerRequirements;
|
||||
import org.hyperledger.besu.ethereum.eth.sync.state.SyncState;
|
||||
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
@@ -96,7 +97,8 @@ public class FullSyncDownloaderTest {
|
||||
ethContext,
|
||||
syncState,
|
||||
metricsSystem,
|
||||
SyncTerminationCondition.never());
|
||||
SyncTerminationCondition.never(),
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
|
||||
@@ -46,6 +46,7 @@ import org.hyperledger.besu.ethereum.trie.forest.storage.ForestWorldStateKeyValu
|
||||
import org.hyperledger.besu.ethereum.worldstate.DataStorageConfiguration;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateKeyValueStorage;
|
||||
import org.hyperledger.besu.ethereum.worldstate.WorldStateStorageCoordinator;
|
||||
import org.hyperledger.besu.metrics.SyncDurationMetrics;
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.storage.DataStorageFormat;
|
||||
import org.hyperledger.besu.services.kvstore.InMemoryKeyValueStorage;
|
||||
@@ -137,7 +138,8 @@ public class SnapWorldDownloadStateTest {
|
||||
MIN_MILLIS_BEFORE_STALLING,
|
||||
metricsManager,
|
||||
clock,
|
||||
ethContext);
|
||||
ethContext,
|
||||
SyncDurationMetrics.NO_OP_SYNC_DURATION_METRICS);
|
||||
final DynamicPivotBlockSelector dynamicPivotBlockManager =
|
||||
mock(DynamicPivotBlockSelector.class);
|
||||
doAnswer(
|
||||
|
||||
@@ -0,0 +1,91 @@
|
||||
/*
|
||||
* Copyright contributors to Hyperledger Besu.
|
||||
*
|
||||
* 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.metrics;
|
||||
|
||||
import org.hyperledger.besu.metrics.noop.NoOpMetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.MetricsSystem;
|
||||
import org.hyperledger.besu.plugin.services.metrics.LabelledMetric;
|
||||
import org.hyperledger.besu.plugin.services.metrics.OperationTimer;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
/**
|
||||
* This class manages the synchronization duration metrics for the Hyperledger Besu project. It
|
||||
* provides methods to start and stop timers for various synchronization phases.
|
||||
*/
|
||||
public class SyncDurationMetrics {
|
||||
|
||||
/** A {@link SyncDurationMetrics} instance that does not record any metrics. */
|
||||
public static final SyncDurationMetrics NO_OP_SYNC_DURATION_METRICS =
|
||||
new SyncDurationMetrics(new NoOpMetricsSystem());
|
||||
|
||||
private final LabelledMetric<OperationTimer> timer;
|
||||
|
||||
private final HashMap<String, OperationTimer.TimingContext> timers = new HashMap<>();
|
||||
|
||||
/**
|
||||
* Creates a new {@link SyncDurationMetrics} instance.
|
||||
*
|
||||
* @param metricsSystem The {@link MetricsSystem} to use to record metrics.
|
||||
*/
|
||||
public SyncDurationMetrics(final MetricsSystem metricsSystem) {
|
||||
timer =
|
||||
metricsSystem.createSimpleLabelledTimer(
|
||||
BesuMetricCategory.SYNCHRONIZER, "sync_duration", "Time taken to sync", "name");
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts a timer for the given synchronization phase.
|
||||
*
|
||||
* @param label The synchronization phase to start the timer for.
|
||||
*/
|
||||
public void startTimer(final Labels label) {
|
||||
timers.computeIfAbsent(label.name(), k -> timer.labels(label.name()).startTimer());
|
||||
}
|
||||
|
||||
/**
|
||||
* Stops the timer for the given synchronization phase.
|
||||
*
|
||||
* @param label The synchronization phase to stop the timer for.
|
||||
*/
|
||||
public void stopTimer(final Labels label) {
|
||||
OperationTimer.TimingContext context = timers.remove(label.name());
|
||||
if (context != null) {
|
||||
context.stopTimer();
|
||||
}
|
||||
}
|
||||
|
||||
/** Enum representing the different synchronization phases. */
|
||||
public enum Labels {
|
||||
/**
|
||||
* Total time taken to get into sync. It is useful for SNAP and CHECKPOINT sync-modes only.
|
||||
*
|
||||
* <p>Total sync duration includes the separate stages mentioned below, some of which occur in
|
||||
* parallel.
|
||||
*
|
||||
* <p>Total sync duration excludes the backwards sync stage due to implementation challenges.
|
||||
* The backwards sync should be a very short duration following the other sync stages.
|
||||
*/
|
||||
TOTAL_SYNC_DURATION,
|
||||
/** Time taken to download the chain data (headers, blocks, receipts). */
|
||||
CHAIN_DOWNLOAD_DURATION,
|
||||
/** Time taken to download the initial world state, before the healing step. */
|
||||
SNAP_INITIAL_WORLD_STATE_DOWNLOAD_DURATION,
|
||||
/** Time taken to heal the world state, after the initial download. */
|
||||
SNAP_WORLD_STATE_HEALING_DURATION,
|
||||
/** Time taken to do the flat database heal. */
|
||||
FLAT_DB_HEAL;
|
||||
}
|
||||
}
|
||||
@@ -104,6 +104,15 @@ public class NoOpMetricsSystem implements ObservableMetricsSystem {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public LabelledMetric<OperationTimer> createSimpleLabelledTimer(
|
||||
final MetricCategory category,
|
||||
final String name,
|
||||
final String help,
|
||||
final String... labelNames) {
|
||||
return getOperationTimerLabelledMetric(labelNames.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public LabelledMetric<OperationTimer> createLabelledTimer(
|
||||
final MetricCategory category,
|
||||
|
||||
@@ -231,6 +231,15 @@ public class OpenTelemetrySystem implements ObservableMetricsSystem {
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public LabelledMetric<OperationTimer> createSimpleLabelledTimer(
|
||||
final MetricCategory category,
|
||||
final String name,
|
||||
final String help,
|
||||
final String... labelNames) {
|
||||
return createLabelledTimer(category, name, help, labelNames);
|
||||
}
|
||||
|
||||
@Override
|
||||
public LabelledMetric<OperationTimer> createLabelledTimer(
|
||||
final MetricCategory category,
|
||||
|
||||
@@ -40,6 +40,7 @@ import io.prometheus.client.Collector.MetricFamilySamples;
|
||||
import io.prometheus.client.Collector.MetricFamilySamples.Sample;
|
||||
import io.prometheus.client.CollectorRegistry;
|
||||
import io.prometheus.client.Counter;
|
||||
import io.prometheus.client.Histogram;
|
||||
import io.prometheus.client.Summary;
|
||||
import io.prometheus.client.hotspot.BufferPoolsExports;
|
||||
import io.prometheus.client.hotspot.ClassLoadingExports;
|
||||
@@ -139,6 +140,27 @@ public class PrometheusMetricsSystem implements ObservableMetricsSystem {
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public LabelledMetric<OperationTimer> createSimpleLabelledTimer(
|
||||
final MetricCategory category,
|
||||
final String name,
|
||||
final String help,
|
||||
final String... labelNames) {
|
||||
final String metricName = convertToPrometheusName(category, name);
|
||||
return cachedTimers.computeIfAbsent(
|
||||
metricName,
|
||||
(k) -> {
|
||||
if (timersEnabled && isCategoryEnabled(category)) {
|
||||
final Histogram histogram =
|
||||
Histogram.build(metricName, help).labelNames(labelNames).buckets(1D).create();
|
||||
addCollectorUnchecked(category, histogram);
|
||||
return new PrometheusSimpleTimer(histogram);
|
||||
} else {
|
||||
return NoOpMetricsSystem.getOperationTimerLabelledMetric(labelNames.length);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createGauge(
|
||||
final MetricCategory category,
|
||||
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* 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.metrics.prometheus;
|
||||
|
||||
import org.hyperledger.besu.plugin.services.metrics.LabelledMetric;
|
||||
import org.hyperledger.besu.plugin.services.metrics.OperationTimer;
|
||||
|
||||
import io.prometheus.client.Histogram;
|
||||
|
||||
class PrometheusSimpleTimer implements LabelledMetric<OperationTimer> {
|
||||
|
||||
private final Histogram histogram;
|
||||
|
||||
public PrometheusSimpleTimer(final Histogram histogram) {
|
||||
this.histogram = histogram;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OperationTimer labels(final String... labels) {
|
||||
final Histogram.Child metric = histogram.labels(labels);
|
||||
return () -> metric.startTimer()::observeDuration;
|
||||
}
|
||||
}
|
||||
@@ -75,6 +75,15 @@ public class StubMetricsSystem implements ObservableMetricsSystem {
|
||||
return labelValues -> NoOpMetricsSystem.NO_OP_OPERATION_TIMER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LabelledMetric<OperationTimer> createSimpleLabelledTimer(
|
||||
final MetricCategory category,
|
||||
final String name,
|
||||
final String help,
|
||||
final String... labelNames) {
|
||||
return labelValues -> NoOpMetricsSystem.NO_OP_OPERATION_TIMER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createGauge(
|
||||
final MetricCategory category,
|
||||
|
||||
@@ -70,7 +70,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 = '2tFIKwEd8T5I37ywbFnVcMwTR8HiiCC6gO1Chd3hZp8='
|
||||
knownHash = 'V/bdVbzJLjdwch266dHHuxIGwiCRhS4w3jDwHt4TWqg='
|
||||
}
|
||||
check.dependsOn('checkAPIChanges')
|
||||
|
||||
|
||||
@@ -89,6 +89,34 @@ public interface MetricsSystem extends BesuService {
|
||||
LabelledMetric<OperationTimer> createLabelledTimer(
|
||||
MetricCategory category, String name, String help, String... labelNames);
|
||||
|
||||
/**
|
||||
* Creates a simple Timer.
|
||||
*
|
||||
* @param category The {@link MetricCategory} this timer is assigned to.
|
||||
* @param name A name for this metric.
|
||||
* @param help A human readable description of the metric.
|
||||
* @return The created Timer instance.
|
||||
*/
|
||||
default OperationTimer createSimpleTimer(
|
||||
final MetricCategory category, final String name, final String help) {
|
||||
return createSimpleLabelledTimer(category, name, help).labels();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a simple Timer with assigned labels.
|
||||
*
|
||||
* @param category The {@link MetricCategory} this timer is assigned to.
|
||||
* @param name A name for this metric.
|
||||
* @param help A human readable description of the metric.
|
||||
* @param labelNames An array of labels to assign to the Timer.
|
||||
* @return The created Timer instance.
|
||||
*/
|
||||
LabelledMetric<OperationTimer> createSimpleLabelledTimer(
|
||||
final MetricCategory category,
|
||||
final String name,
|
||||
final String help,
|
||||
final String... labelNames);
|
||||
|
||||
/**
|
||||
* Creates a gauge for displaying double vales. A gauge is a metric to report the current value.
|
||||
* The metric value may go up or down.
|
||||
|
||||
Reference in New Issue
Block a user