diff --git a/CHANGELOG.md b/CHANGELOG.md index 79bdb0f879d..cf5b8029b84 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -17,5 +17,6 @@ the [releases page](https://github.com/Consensys/teku/releases). - Deposit tree snapshots will be loaded from database as a default unless custom snapshot has been provided. - Added hidden option `--Xdeposit-contract-logs-syncing-enabled` to allow disabling the syncing of the deposit contract logs from the EL. This is useful when running a non-validating node. It is advisable to be used alongside with `--Xeth1-missing-deposits-event-logging-enabled=false` to avoid unnecessary logging of missing deposits. - Updated the bootnodes for Chiado and Gnosis networks +- Added hidden option `--Xp2p-dumps-to-file-enabled` to enable saving p2p dumps to file. ### Bug Fixes diff --git a/eth-reference-tests/src/referenceTest/java/tech/pegasys/teku/reference/phase0/forkchoice/ForkChoiceTestExecutor.java b/eth-reference-tests/src/referenceTest/java/tech/pegasys/teku/reference/phase0/forkchoice/ForkChoiceTestExecutor.java index 5e66788b95f..ca69233ee55 100644 --- a/eth-reference-tests/src/referenceTest/java/tech/pegasys/teku/reference/phase0/forkchoice/ForkChoiceTestExecutor.java +++ b/eth-reference-tests/src/referenceTest/java/tech/pegasys/teku/reference/phase0/forkchoice/ForkChoiceTestExecutor.java @@ -15,6 +15,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.mockito.Mockito.mock; import static tech.pegasys.teku.infrastructure.async.SafeFutureAssert.safeJoin; import static tech.pegasys.teku.infrastructure.time.TimeUtilities.secondsToMillis; @@ -69,6 +70,7 @@ import tech.pegasys.teku.statetransition.forkchoice.MergeTransitionBlockValidator; import tech.pegasys.teku.statetransition.forkchoice.NoopForkChoiceNotifier; import tech.pegasys.teku.statetransition.forkchoice.TickProcessor; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.BlockBroadcastValidator; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; import tech.pegasys.teku.storage.client.RecentChainData; @@ -139,6 +141,7 @@ spec, new SignedBlockAndState(anchorBlock, anchorState)), new TickProcessor(spec, recentChainData), transitionBlockValidator, true, + mock(DebugDataDumper.class), storageSystem.getMetricsSystem()); final ExecutionLayerChannelStub executionLayer = new ExecutionLayerChannelStub(spec, false, Optional.empty()); diff --git a/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoice.java b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoice.java index a4bf4ad30c7..74b3d53de06 100644 --- a/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoice.java +++ b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoice.java @@ -74,6 +74,8 @@ import tech.pegasys.teku.statetransition.attestation.DeferredAttestations; import tech.pegasys.teku.statetransition.blobs.BlobSidecarManager; import tech.pegasys.teku.statetransition.block.BlockImportPerformance; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; +import tech.pegasys.teku.statetransition.util.noop.NoOpDebugDataDumper; import tech.pegasys.teku.statetransition.validation.AttestationStateSelector; import tech.pegasys.teku.statetransition.validation.BlockBroadcastValidator; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; @@ -108,6 +110,8 @@ public class ForkChoice implements ForkChoiceUpdatedResultSubscriber { private final LabelledMetric getProposerHeadSelectedCounter; + private final DebugDataDumper debugDataDumper; + public ForkChoice( final Spec spec, final EventThread forkChoiceExecutor, @@ -118,6 +122,7 @@ public ForkChoice( final TickProcessor tickProcessor, final MergeTransitionBlockValidator transitionBlockValidator, final boolean forkChoiceLateBlockReorgEnabled, + final DebugDataDumper debugDataDumper, final MetricsSystem metricsSystem) { this.spec = spec; this.forkChoiceExecutor = forkChoiceExecutor; @@ -132,6 +137,7 @@ public ForkChoice( this.forkChoiceLateBlockReorgEnabled = forkChoiceLateBlockReorgEnabled; this.lastProcessHeadSlot.set(UInt64.ZERO); LOG.debug("forkChoiceLateBlockReorgEnabled is set to {}", forkChoiceLateBlockReorgEnabled); + this.debugDataDumper = debugDataDumper; getProposerHeadSelectedCounter = metricsSystem.createLabelledCounter( TekuMetricCategory.BEACON, @@ -161,6 +167,7 @@ public ForkChoice( new TickProcessor(spec, recentChainData), transitionBlockValidator, false, + new NoOpDebugDataDumper(), metricsSystem); } @@ -753,6 +760,8 @@ private void reportInvalidBlock(final SignedBeaconBlock block, final BlockImport if (result.getFailureReason() == FailureReason.BLOCK_IS_FROM_FUTURE) { return; } + debugDataDumper.saveInvalidBlockToFile( + block, result.getFailureReason().name(), result.getFailureCause()); P2P_LOG.onInvalidBlock( block.getSlot(), block.getRoot(), diff --git a/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/util/DebugDataDumper.java b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/util/DebugDataDumper.java new file mode 100644 index 00000000000..3aa4fc80002 --- /dev/null +++ b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/util/DebugDataDumper.java @@ -0,0 +1,207 @@ +/* + * Copyright Consensys Software Inc., 2024 + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package tech.pegasys.teku.statetransition.util; + +import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.sql.Date; +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.util.Optional; +import java.util.function.Supplier; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.tuweni.bytes.Bytes; +import org.apache.tuweni.bytes.Bytes32; +import tech.pegasys.teku.infrastructure.time.SystemTimeProvider; +import tech.pegasys.teku.infrastructure.time.TimeProvider; +import tech.pegasys.teku.infrastructure.unsigned.UInt64; +import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; + +public class DebugDataDumper { + private static final Logger LOG = LogManager.getLogger(); + + private static final String GOSSIP_MESSAGES_DIR = "gossip_messages"; + private static final String DECODING_ERROR_SUB_DIR = "decoding_error"; + private static final String REJECTED_SUB_DIR = "rejected"; + private static final String INVALID_BLOCK_DIR = "invalid_blocks"; + + private boolean enabled; + private final Path directory; + + public DebugDataDumper(final Path directory) { + this.enabled = true; + this.directory = directory; + + final Path gossipMessagesPath = this.directory.resolve(GOSSIP_MESSAGES_DIR); + createDirectory(gossipMessagesPath, GOSSIP_MESSAGES_DIR, "gossip messages"); + createDirectory( + gossipMessagesPath.resolve(DECODING_ERROR_SUB_DIR), + DECODING_ERROR_SUB_DIR, + "gossip messages with decoding errors"); + createDirectory( + gossipMessagesPath.resolve(REJECTED_SUB_DIR), REJECTED_SUB_DIR, "rejected gossip messages"); + createDirectory(this.directory.resolve(INVALID_BLOCK_DIR), INVALID_BLOCK_DIR, "invalid blocks"); + } + + public void saveGossipMessageDecodingError( + final String topic, + final Optional arrivalTimestamp, + final Supplier originalMessage, + final Throwable error) { + if (!enabled) { + return; + } + final String formattedTimestamp = formatOptionalTimestamp(arrivalTimestamp); + final String fileName = String.format("%s.ssz", formattedTimestamp); + final Path topicPath = + Path.of(GOSSIP_MESSAGES_DIR) + .resolve(DECODING_ERROR_SUB_DIR) + .resolve(topic.replaceAll("/", "_")); + final boolean success = + saveBytesToFile( + "gossip message with decoding error", + topicPath.resolve(fileName), + originalMessage.get()); + if (success) { + LOG.warn("Failed to decode gossip message on topic {}", topic, error); + } + } + + public void saveGossipRejectedMessageToFile( + final String topic, + final Optional arrivalTimestamp, + final Supplier decodedMessage, + final Optional reason) { + if (!enabled) { + return; + } + final String formattedTimestamp = formatOptionalTimestamp(arrivalTimestamp); + final String fileName = String.format("%s.ssz", formattedTimestamp); + final Path topicPath = + Path.of(GOSSIP_MESSAGES_DIR).resolve(REJECTED_SUB_DIR).resolve(topic.replaceAll("/", "_")); + final boolean success = + saveBytesToFile( + "rejected gossip message", topicPath.resolve(fileName), decodedMessage.get()); + if (success) { + LOG.warn( + "Rejecting gossip message on topic {}, reason: {}", + topic, + reason.orElse("failed validation")); + } + } + + public void saveInvalidBlockToFile( + final SignedBeaconBlock block, + final String failureReason, + final Optional failureCause) { + if (!enabled) { + return; + } + final UInt64 slot = block.getSlot(); + final Bytes32 blockRoot = block.getRoot(); + final String fileName = String.format("%s_%s.ssz", slot, blockRoot.toUnprefixedHexString()); + final boolean success = + saveBytesToFile( + "invalid block", Path.of(INVALID_BLOCK_DIR).resolve(fileName), block.sszSerialize()); + if (success) { + LOG.warn( + "Rejecting invalid block at slot {} with root {} because {}", + slot, + blockRoot, + failureReason, + failureCause.orElse(null)); + } + } + + @VisibleForTesting + protected boolean saveBytesToFile( + final String description, final Path relativeFilePath, final Bytes bytes) { + final Path path = directory.resolve(relativeFilePath); + try { + Files.write(path, bytes.toArray()); + } catch (NoSuchFileException e) { + return saveAfterCreatingTopicDirectory(description, path, relativeFilePath, bytes); + } catch (IOException e) { + LOG.error("Failed to save {} bytes to file.", description, e); + return false; + } + return true; + } + + private boolean saveAfterCreatingTopicDirectory( + final String description, final Path path, final Path relativeFilePath, final Bytes bytes) { + if (!path.getParent().toFile().mkdirs()) { + LOG.error( + "Failed to save {} bytes to file. No such directory {} to save file.", + description, + relativeFilePath.getParent()); + return false; + } + try { + Files.write(path, bytes.toArray()); + } catch (IOException e) { + LOG.error("Failed to save {} bytes to file.", description, e); + if (!path.getParent().toFile().exists()) { + this.enabled = false; + LOG.error( + "{} directory does not exist. Disabling saving debug data to file.", + relativeFilePath.getParent()); + } + return false; + } + return true; + } + + private void createDirectory( + final Path path, final String directoryName, final String description) { + if (!enabled) { + return; + } + if (path.toFile().mkdirs()) { + LOG.debug("{} directory has been created to save {}.", directoryName, description); + } else { + if (!path.toFile().exists()) { + this.enabled = false; + LOG.error( + "Unable to create {} directory to save {}. Disabling saving debug data to file.", + directoryName, + description); + } + } + } + + private String formatOptionalTimestamp(final Optional maybeTimestamp) { + return formatOptionalTimestamp(maybeTimestamp, new SystemTimeProvider()); + } + + @VisibleForTesting + String formatOptionalTimestamp( + final Optional maybeTimestamp, final TimeProvider timeProvider) { + final DateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH_mm_ss.SS"); + final Date date = + maybeTimestamp + .map(timestamp -> new Date(timestamp.longValue())) + .orElse(new Date(timeProvider.getTimeInMillis().longValue())); + return df.format(date); + } + + @VisibleForTesting + boolean isEnabled() { + return enabled; + } +} diff --git a/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/util/noop/NoOpDebugDataDumper.java b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/util/noop/NoOpDebugDataDumper.java new file mode 100644 index 00000000000..a52c3a410d0 --- /dev/null +++ b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/util/noop/NoOpDebugDataDumper.java @@ -0,0 +1,49 @@ +/* + * Copyright Consensys Software Inc., 2024 + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package tech.pegasys.teku.statetransition.util.noop; + +import java.nio.file.Path; +import java.util.Optional; +import java.util.function.Supplier; +import org.apache.tuweni.bytes.Bytes; +import tech.pegasys.teku.infrastructure.unsigned.UInt64; +import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; + +public class NoOpDebugDataDumper extends DebugDataDumper { + + public NoOpDebugDataDumper() { + super(Path.of(".")); + } + + @Override + public void saveGossipMessageDecodingError( + final String topic, + final Optional arrivalTimestamp, + final Supplier originalMessage, + final Throwable error) {} + + @Override + public void saveGossipRejectedMessageToFile( + final String topic, + final Optional arrivalTimestamp, + final Supplier decodedMessage, + final Optional reason) {} + + @Override + public void saveInvalidBlockToFile( + final SignedBeaconBlock block, + final String failureReason, + final Optional failureCause) {} +} diff --git a/ethereum/statetransition/src/test/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceTest.java b/ethereum/statetransition/src/test/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceTest.java index a8d96ea1206..54e09be8627 100644 --- a/ethereum/statetransition/src/test/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceTest.java +++ b/ethereum/statetransition/src/test/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceTest.java @@ -95,6 +95,7 @@ import tech.pegasys.teku.statetransition.blobs.BlobSidecarManager; import tech.pegasys.teku.statetransition.forkchoice.ForkChoice.OptimisticHeadSubscriber; import tech.pegasys.teku.statetransition.forkchoice.ForkChoiceUpdatedResultSubscriber.ForkChoiceUpdatedResultNotification; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.BlockBroadcastValidator; import tech.pegasys.teku.statetransition.validation.BlockBroadcastValidator.BroadcastValidationResult; import tech.pegasys.teku.storage.api.TrackingChainHeadChannel.ReorgEvent; @@ -128,6 +129,7 @@ class ForkChoiceTest { mock(BlockBroadcastValidator.class); private final MergeTransitionBlockValidator transitionBlockValidator = mock(MergeTransitionBlockValidator.class); + private DebugDataDumper debugDataDumper = mock(DebugDataDumper.class); private final InlineEventThread eventThread = new InlineEventThread(); @@ -164,6 +166,7 @@ private void setupWithSpec(final Spec unmockedSpec) { new TickProcessor(spec, recentChainData), transitionBlockValidator, DEFAULT_FORK_CHOICE_LATE_BLOCK_REORG_ENABLED, + debugDataDumper, metricsSystem); // Starting and mocks @@ -304,11 +307,17 @@ void onBlock_consensusValidationShouldNotResolveWhenStateTransitionFails() final BlockProcessor blockProcessor = mock(BlockProcessor.class); when(spec.getBlockProcessor(blockAndState.getSlot())).thenReturn(blockProcessor); + final Exception blockException = new StateTransitionException("error!"); when(blockProcessor.processAndValidateBlock(any(), any(), any(), any())) - .thenThrow(new StateTransitionException("error!")); + .thenThrow(blockException); importBlockAndAssertFailure(blockAndState, FailureReason.FAILED_STATE_TRANSITION); + verify(debugDataDumper) + .saveInvalidBlockToFile( + eq(blockAndState.getBlock()), + eq(FailureReason.FAILED_STATE_TRANSITION.toString()), + eq(Optional.of(blockException))); verify(blockBroadcastValidator, never()).onConsensusValidationSucceeded(); } @@ -338,8 +347,9 @@ void onBlock_consensusValidationShouldReturnRegardlessExecutionPayloadValidation // resolve with a failure payloadStatusSafeFuture.complete(PayloadStatus.invalid(Optional.empty(), Optional.empty())); - assertBlockImportFailure(importResult, FailureReason.FAILED_STATE_TRANSITION); + verify(debugDataDumper) + .saveInvalidBlockToFile(any(), eq(FailureReason.FAILED_STATE_TRANSITION.toString()), any()); } @Test @@ -419,6 +429,7 @@ void onBlock_shouldReorgWhenProposerWeightingMakesForkBestChain( new TickProcessor(spec, recentChainData), transitionBlockValidator, DEFAULT_FORK_CHOICE_LATE_BLOCK_REORG_ENABLED, + mock(DebugDataDumper.class), metricsSystem); final UInt64 currentSlot = recentChainData.getCurrentSlot().orElseThrow(); @@ -738,6 +749,11 @@ void onBlock_shouldNotOptimisticallyImportInvalidExecutionPayload() { storageSystem.chainUpdater().setCurrentSlot(slotToImport.increment()); importBlockAndAssertFailure( chainBuilder.generateNextBlock(), FailureReason.FAILED_STATE_TRANSITION); + verify(debugDataDumper) + .saveInvalidBlockToFile( + eq(chainBuilder.getLatestBlockAndState().getBlock()), + eq(FailureReason.FAILED_STATE_TRANSITION.toString()), + any()); } @Test @@ -774,7 +790,11 @@ void onBlock_shouldChangeForkChoiceForLatestValidHashOnInvalidExecutionPayload() SignedBlockAndState invalidBlock = chainBuilder.generateNextBlock(); importBlockAndAssertFailure(invalidBlock, FailureReason.FAILED_STATE_TRANSITION); assertThat(forkChoice.processHead(invalidBlock.getSlot())).isCompleted(); - + verify(debugDataDumper) + .saveInvalidBlockToFile( + eq(invalidBlock.getBlock()), + eq(FailureReason.FAILED_STATE_TRANSITION.toString()), + any()); assertHeadIsOptimistic(maybeValidBlock); assertThat(forkChoiceStrategy.getChainHeads().get(0).getRoot()) .isEqualTo(maybeValidBlock.getRoot()); diff --git a/ethereum/statetransition/src/test/java/tech/pegasys/teku/statetransition/util/DebugDataDumperTest.java b/ethereum/statetransition/src/test/java/tech/pegasys/teku/statetransition/util/DebugDataDumperTest.java new file mode 100644 index 00000000000..a36921a0f93 --- /dev/null +++ b/ethereum/statetransition/src/test/java/tech/pegasys/teku/statetransition/util/DebugDataDumperTest.java @@ -0,0 +1,162 @@ +/* + * Copyright Consensys Software Inc., 2024 + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package tech.pegasys.teku.statetransition.util; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.fail; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.sql.Date; +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.util.Optional; +import org.apache.tuweni.bytes.Bytes; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.DisabledOnOs; +import org.junit.jupiter.api.condition.OS; +import org.junit.jupiter.api.io.TempDir; +import tech.pegasys.teku.infrastructure.time.StubTimeProvider; +import tech.pegasys.teku.infrastructure.unsigned.UInt64; +import tech.pegasys.teku.spec.TestSpecFactory; +import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; +import tech.pegasys.teku.spec.util.DataStructureUtil; + +class DebugDataDumperTest { + final DataStructureUtil dataStructureUtil = + new DataStructureUtil(TestSpecFactory.createDefault()); + private final StubTimeProvider timeProvider = StubTimeProvider.withTimeInSeconds(10_000); + + @Test + void saveGossipMessageDecodingError_shouldSaveToFile(@TempDir Path tempDir) { + final DebugDataDumper manager = new DebugDataDumper(tempDir); + final Bytes messageBytes = dataStructureUtil.stateBuilderPhase0().build().sszSerialize(); + final Optional arrivalTimestamp = Optional.of(timeProvider.getTimeInMillis()); + manager.saveGossipMessageDecodingError( + "/eth/test/topic", arrivalTimestamp, () -> messageBytes, new Throwable()); + + final String fileName = + String.format("%s.ssz", formatTimestamp(timeProvider.getTimeInMillis().longValue())); + final Path expectedFile = + tempDir + .resolve("gossip_messages") + .resolve("decoding_error") + .resolve("_eth_test_topic") + .resolve(fileName); + checkBytesSavedToFile(expectedFile, messageBytes); + } + + @Test + void saveGossipRejectedMessageToFile_shouldSaveToFile(@TempDir Path tempDir) { + final DebugDataDumper manager = new DebugDataDumper(tempDir); + final Bytes messageBytes = dataStructureUtil.stateBuilderPhase0().build().sszSerialize(); + final Optional arrivalTimestamp = Optional.of(timeProvider.getTimeInMillis()); + manager.saveGossipRejectedMessageToFile( + "/eth/test/topic", arrivalTimestamp, () -> messageBytes, Optional.of("reason")); + + final String fileName = + String.format("%s.ssz", formatTimestamp(timeProvider.getTimeInMillis().longValue())); + final Path expectedFile = + tempDir + .resolve("gossip_messages") + .resolve("rejected") + .resolve("_eth_test_topic") + .resolve(fileName); + checkBytesSavedToFile(expectedFile, messageBytes); + } + + @Test + void saveInvalidBlockToFile_shouldSaveToFile(@TempDir Path tempDir) { + final DebugDataDumper manager = new DebugDataDumper(tempDir); + final SignedBeaconBlock block = dataStructureUtil.randomSignedBeaconBlock(); + manager.saveInvalidBlockToFile(block, "reason", Optional.of(new Throwable())); + + final String fileName = + String.format("%s_%s.ssz", block.getSlot(), block.getRoot().toUnprefixedHexString()); + final Path expectedFile = tempDir.resolve("invalid_blocks").resolve(fileName); + checkBytesSavedToFile(expectedFile, block.sszSerialize()); + } + + @Test + void saveBytesToFile_shouldNotThrowExceptionWhenNoDirectory(@TempDir Path tempDir) { + final DebugDataDumper manager = new DebugDataDumper(tempDir); + assertDoesNotThrow( + () -> { + final boolean success = + manager.saveBytesToFile( + "object", Path.of("invalid").resolve("file.ssz"), Bytes.EMPTY); + assertThat(success).isTrue(); // creates directory + }); + } + + @Test + @DisabledOnOs(OS.WINDOWS) // Can't set permissions on Windows + void saveBytesToFile_shouldNotEscalateWhenIOException(@TempDir Path tempDir) { + final DebugDataDumper manager = new DebugDataDumper(tempDir); + final File invalidPath = tempDir.resolve("invalid").toFile(); + assertThat(invalidPath.mkdirs()).isTrue(); + assertThat(invalidPath.setWritable(false)).isTrue(); + assertDoesNotThrow( + () -> { + final boolean success = + manager.saveBytesToFile( + "object", Path.of("invalid").resolve("file.ssz"), Bytes.EMPTY); + assertThat(success).isFalse(); + }); + } + + @Test + @DisabledOnOs(OS.WINDOWS) // Can't set permissions on Windows + void constructionOfDirectories_shouldDisableWhenFailedToCreate(@TempDir Path tempDir) { + assertThat(tempDir.toFile().setWritable(false)).isTrue(); + final DebugDataDumper manager = new DebugDataDumper(tempDir); + assertThat(manager.isEnabled()).isFalse(); + } + + @Test + void formatOptionalTimestamp_shouldFormatTimestamp(@TempDir Path tempDir) { + final DebugDataDumper manager = new DebugDataDumper(tempDir); + final String formattedTimestamp = + manager.formatOptionalTimestamp(Optional.of(timeProvider.getTimeInMillis()), timeProvider); + assertThat(formattedTimestamp) + .isEqualTo(formatTimestamp(timeProvider.getTimeInMillis().longValue())); + } + + @Test + void formatOptionalTimestamp_shouldGenerateTimestamp(@TempDir Path tempDir) { + final DebugDataDumper manager = new DebugDataDumper(tempDir); + final String formattedTimestamp = + manager.formatOptionalTimestamp(Optional.empty(), timeProvider); + assertThat(formattedTimestamp) + .isEqualTo(formatTimestamp(timeProvider.getTimeInMillis().longValue())); + } + + private void checkBytesSavedToFile(final Path path, final Bytes expectedBytes) { + try { + final Bytes bytes = Bytes.wrap(Files.readAllBytes(path)); + assertThat(bytes).isEqualTo(expectedBytes); + } catch (IOException e) { + fail(); + } + } + + private String formatTimestamp(final long timeInMillis) { + final DateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH_mm_ss.SS"); + final Date date = new Date(timeInMillis); + return df.format(date); + } +} diff --git a/infrastructure/serviceutils/src/main/java/tech/pegasys/teku/service/serviceutils/layout/DataDirLayout.java b/infrastructure/serviceutils/src/main/java/tech/pegasys/teku/service/serviceutils/layout/DataDirLayout.java index 2ea3e24299f..2e9a2f3018e 100644 --- a/infrastructure/serviceutils/src/main/java/tech/pegasys/teku/service/serviceutils/layout/DataDirLayout.java +++ b/infrastructure/serviceutils/src/main/java/tech/pegasys/teku/service/serviceutils/layout/DataDirLayout.java @@ -29,4 +29,6 @@ static DataDirLayout createFrom(final DataConfig dataConfig) { Path getBeaconDataDirectory(); Path getValidatorDataDirectory(); + + Path getDebugDataDirectory(); } diff --git a/infrastructure/serviceutils/src/main/java/tech/pegasys/teku/service/serviceutils/layout/SeparateServiceDataDirLayout.java b/infrastructure/serviceutils/src/main/java/tech/pegasys/teku/service/serviceutils/layout/SeparateServiceDataDirLayout.java index 602b6b8d3ef..2fa8f14958b 100644 --- a/infrastructure/serviceutils/src/main/java/tech/pegasys/teku/service/serviceutils/layout/SeparateServiceDataDirLayout.java +++ b/infrastructure/serviceutils/src/main/java/tech/pegasys/teku/service/serviceutils/layout/SeparateServiceDataDirLayout.java @@ -19,8 +19,10 @@ public class SeparateServiceDataDirLayout implements DataDirLayout { static final String BEACON_DATA_DIR_NAME = "beacon"; static final String VALIDATOR_DATA_DIR_NAME = "validator"; + static final String DEBUG_DIR_NAME = "debug"; private final Path beaconNodeDataDir; private final Path validatorDataDir; + private final Path debugDataDir; public SeparateServiceDataDirLayout( final Path baseDir, @@ -29,6 +31,7 @@ public SeparateServiceDataDirLayout( beaconNodeDataDir = beaconDataDirectory.orElseGet(() -> baseDir.resolve(BEACON_DATA_DIR_NAME)); validatorDataDir = validatorDataDirectory.orElseGet(() -> baseDir.resolve(VALIDATOR_DATA_DIR_NAME)); + debugDataDir = baseDir.resolve(DEBUG_DIR_NAME); } @Override @@ -40,4 +43,9 @@ public Path getBeaconDataDirectory() { public Path getValidatorDataDirectory() { return validatorDataDir; } + + @Override + public Path getDebugDataDirectory() { + return debugDataDir; + } } diff --git a/infrastructure/serviceutils/src/testFixtures/java/tech/pegasys/techu/service/serviceutils/layout/SimpleDataDirLayout.java b/infrastructure/serviceutils/src/testFixtures/java/tech/pegasys/techu/service/serviceutils/layout/SimpleDataDirLayout.java index 74a087be1e6..41df2354fdf 100644 --- a/infrastructure/serviceutils/src/testFixtures/java/tech/pegasys/techu/service/serviceutils/layout/SimpleDataDirLayout.java +++ b/infrastructure/serviceutils/src/testFixtures/java/tech/pegasys/techu/service/serviceutils/layout/SimpleDataDirLayout.java @@ -32,4 +32,9 @@ public Path getBeaconDataDirectory() { public Path getValidatorDataDirectory() { return path; } + + @Override + public Path getDebugDataDirectory() { + return path; + } } diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/Eth2P2PNetworkBuilder.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/Eth2P2PNetworkBuilder.java index 4ba768c9504..370c5ab788c 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/Eth2P2PNetworkBuilder.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/Eth2P2PNetworkBuilder.java @@ -80,6 +80,7 @@ import tech.pegasys.teku.spec.datastructures.state.Checkpoint; import tech.pegasys.teku.spec.datastructures.util.ForkAndSpecMilestone; import tech.pegasys.teku.spec.schemas.SchemaDefinitionsSupplier; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.CombinedChainDataClient; import tech.pegasys.teku.storage.store.KeyValueStore; @@ -124,6 +125,7 @@ public class Eth2P2PNetworkBuilder { protected StatusMessageFactory statusMessageFactory; protected KZG kzg; protected boolean recordMessageArrival; + protected DebugDataDumper debugDataDumper; protected Eth2P2PNetworkBuilder() {} @@ -219,7 +221,8 @@ private GossipForkSubscriptions createSubscriptions( gossipedAggregateProcessor, gossipedAttesterSlashingConsumer, gossipedProposerSlashingConsumer, - gossipedVoluntaryExitConsumer); + gossipedVoluntaryExitConsumer, + debugDataDumper); case ALTAIR -> new GossipForkSubscriptionsAltair( forkAndSpecMilestone.getFork(), spec, @@ -235,7 +238,8 @@ private GossipForkSubscriptions createSubscriptions( gossipedProposerSlashingConsumer, gossipedVoluntaryExitConsumer, gossipedSignedContributionAndProofProcessor, - gossipedSyncCommitteeMessageProcessor); + gossipedSyncCommitteeMessageProcessor, + debugDataDumper); case BELLATRIX -> new GossipForkSubscriptionsBellatrix( forkAndSpecMilestone.getFork(), spec, @@ -251,7 +255,8 @@ private GossipForkSubscriptions createSubscriptions( gossipedProposerSlashingConsumer, gossipedVoluntaryExitConsumer, gossipedSignedContributionAndProofProcessor, - gossipedSyncCommitteeMessageProcessor); + gossipedSyncCommitteeMessageProcessor, + debugDataDumper); case CAPELLA -> new GossipForkSubscriptionsCapella( forkAndSpecMilestone.getFork(), spec, @@ -268,7 +273,8 @@ private GossipForkSubscriptions createSubscriptions( gossipedVoluntaryExitConsumer, gossipedSignedContributionAndProofProcessor, gossipedSyncCommitteeMessageProcessor, - gossipedSignedBlsToExecutionChangeProcessor); + gossipedSignedBlsToExecutionChangeProcessor, + debugDataDumper); case DENEB -> new GossipForkSubscriptionsDeneb( forkAndSpecMilestone.getFork(), spec, @@ -286,7 +292,8 @@ private GossipForkSubscriptions createSubscriptions( gossipedVoluntaryExitConsumer, gossipedSignedContributionAndProofProcessor, gossipedSyncCommitteeMessageProcessor, - gossipedSignedBlsToExecutionChangeProcessor); + gossipedSignedBlsToExecutionChangeProcessor, + debugDataDumper); case ELECTRA -> new GossipForkSubscriptionsElectra( forkAndSpecMilestone.getFork(), spec, @@ -304,7 +311,8 @@ private GossipForkSubscriptions createSubscriptions( gossipedVoluntaryExitConsumer, gossipedSignedContributionAndProofProcessor, gossipedSyncCommitteeMessageProcessor, - gossipedSignedBlsToExecutionChangeProcessor); + gossipedSignedBlsToExecutionChangeProcessor, + debugDataDumper); }; } @@ -598,4 +606,9 @@ public Eth2P2PNetworkBuilder recordMessageArrival(final boolean recordMessageArr this.recordMessageArrival = recordMessageArrival; return this; } + + public Eth2P2PNetworkBuilder debugDataDumper(final DebugDataDumper debugDataDumper) { + this.debugDataDumper = debugDataDumper; + return this; + } } diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/P2PConfig.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/P2PConfig.java index 1edeac1a496..2008e6974d7 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/P2PConfig.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/P2PConfig.java @@ -41,6 +41,7 @@ public class P2PConfig { Math.max(2, Runtime.getRuntime().availableProcessors() / 2); public static final int DEFAULT_BATCH_VERIFY_QUEUE_CAPACITY = 15_000; public static final int DEFAULT_BATCH_VERIFY_MAX_BATCH_SIZE = 250; + public static final boolean DEFAULT_P2P_DUMPS_TO_FILE_ENABLED = false; public static final boolean DEFAULT_BATCH_VERIFY_STRICT_THREAD_LIMIT_ENABLED = false; private final Spec spec; @@ -58,6 +59,7 @@ public class P2PConfig { private final int batchVerifyQueueCapacity; private final int batchVerifyMaxBatchSize; private final boolean batchVerifyStrictThreadLimitEnabled; + private final boolean p2pDumpsToFileEnabled; private final boolean allTopicsFilterEnabled; @@ -75,6 +77,7 @@ private P2PConfig( final int batchVerifyQueueCapacity, final int batchVerifyMaxBatchSize, final boolean batchVerifyStrictThreadLimitEnabled, + final boolean p2pDumpsToFileEnabled, boolean allTopicsFilterEnabled) { this.spec = spec; this.networkConfig = networkConfig; @@ -90,6 +93,7 @@ private P2PConfig( this.batchVerifyMaxBatchSize = batchVerifyMaxBatchSize; this.batchVerifyStrictThreadLimitEnabled = batchVerifyStrictThreadLimitEnabled; this.networkingSpecConfig = spec.getNetworkingConfig(); + this.p2pDumpsToFileEnabled = p2pDumpsToFileEnabled; this.allTopicsFilterEnabled = allTopicsFilterEnabled; } @@ -149,6 +153,10 @@ public boolean isBatchVerifyStrictThreadLimitEnabled() { return batchVerifyStrictThreadLimitEnabled; } + public boolean isP2pDumpsToFileEnabled() { + return p2pDumpsToFileEnabled; + } + public NetworkingSpecConfig getNetworkingSpecConfig() { return networkingSpecConfig; } @@ -174,6 +182,7 @@ public static class Builder { private boolean batchVerifyStrictThreadLimitEnabled = DEFAULT_BATCH_VERIFY_STRICT_THREAD_LIMIT_ENABLED; private boolean allTopicsFilterEnabled = DEFAULT_PEER_ALL_TOPIC_FILTER_ENABLED; + private boolean p2pDumpsToFileEnabled = DEFAULT_P2P_DUMPS_TO_FILE_ENABLED; private Builder() {} @@ -216,6 +225,7 @@ public P2PConfig build() { batchVerifyQueueCapacity, batchVerifyMaxBatchSize, batchVerifyStrictThreadLimitEnabled, + p2pDumpsToFileEnabled, allTopicsFilterEnabled); } @@ -318,5 +328,10 @@ public Builder allTopicsFilterEnabled(final boolean allTopicsFilterEnabled) { this.allTopicsFilterEnabled = allTopicsFilterEnabled; return this; } + + public Builder p2pDumpsToFileEnabled(final boolean p2pDumpsToFileEnabled) { + this.p2pDumpsToFileEnabled = p2pDumpsToFileEnabled; + return this; + } } } diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AbstractGossipManager.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AbstractGossipManager.java index b6850a68017..fe8355163e3 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AbstractGossipManager.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AbstractGossipManager.java @@ -29,6 +29,7 @@ import tech.pegasys.teku.networking.p2p.gossip.TopicChannel; import tech.pegasys.teku.spec.config.NetworkingSpecConfig; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public abstract class AbstractGossipManager implements GossipManager { @@ -50,7 +51,8 @@ protected AbstractGossipManager( final OperationProcessor processor, final SszSchema gossipType, final Function getEpochForMessage, - final NetworkingSpecConfig networkingConfig) { + final NetworkingSpecConfig networkingConfig, + final DebugDataDumper debugDataDumper) { this.gossipNetwork = gossipNetwork; this.topicHandler = new Eth2TopicHandler<>( @@ -63,7 +65,8 @@ protected AbstractGossipManager( new OperationMilestoneValidator<>( recentChainData.getSpec(), forkInfo.getFork(), getEpochForMessage), gossipType, - networkingConfig); + networkingConfig, + debugDataDumper); this.gossipEncoding = gossipEncoding; } diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AggregateGossipManager.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AggregateGossipManager.java index cb0737a6db0..f7294fa323c 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AggregateGossipManager.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AggregateGossipManager.java @@ -22,6 +22,7 @@ import tech.pegasys.teku.spec.datastructures.attestation.ValidatableAttestation; import tech.pegasys.teku.spec.datastructures.operations.SignedAggregateAndProof; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class AggregateGossipManager extends AbstractGossipManager { @@ -33,7 +34,8 @@ public AggregateGossipManager( final GossipNetwork gossipNetwork, final GossipEncoding gossipEncoding, final ForkInfo forkInfo, - final OperationProcessor processor) { + final OperationProcessor processor, + final DebugDataDumper debugDataDumper) { super( recentChainData, GossipTopicName.BEACON_AGGREGATE_AND_PROOF, @@ -50,7 +52,8 @@ public AggregateGossipManager( .getSchemaDefinitions() .getSignedAggregateAndProofSchema(), message -> spec.computeEpochAtSlot(message.getMessage().getAggregate().getData().getSlot()), - spec.getNetworkingConfig()); + spec.getNetworkingConfig(), + debugDataDumper); } public void onNewAggregate(final ValidatableAttestation validatableAttestation) { diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AttesterSlashingGossipManager.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AttesterSlashingGossipManager.java index 49a865d1843..3eb4167c3b8 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AttesterSlashingGossipManager.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/AttesterSlashingGossipManager.java @@ -21,6 +21,7 @@ import tech.pegasys.teku.spec.Spec; import tech.pegasys.teku.spec.datastructures.operations.AttesterSlashing; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class AttesterSlashingGossipManager extends AbstractGossipManager { @@ -32,7 +33,8 @@ public AttesterSlashingGossipManager( final GossipNetwork gossipNetwork, final GossipEncoding gossipEncoding, final ForkInfo forkInfo, - final OperationProcessor processor) { + final OperationProcessor processor, + final DebugDataDumper debugDataDumper) { super( recentChainData, GossipTopicName.ATTESTER_SLASHING, @@ -45,7 +47,8 @@ public AttesterSlashingGossipManager( .getSchemaDefinitions() .getAttesterSlashingSchema(), message -> spec.computeEpochAtSlot(message.getAttestation1().getData().getSlot()), - spec.getNetworkingConfig()); + spec.getNetworkingConfig(), + debugDataDumper); } public void publishAttesterSlashing(final AttesterSlashing message) { diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/BlobSidecarGossipManager.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/BlobSidecarGossipManager.java index dec9c6a8807..85fb9631084 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/BlobSidecarGossipManager.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/BlobSidecarGossipManager.java @@ -35,6 +35,7 @@ import tech.pegasys.teku.spec.datastructures.blobs.versions.deneb.BlobSidecarSchema; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.schemas.SchemaDefinitionsDeneb; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; import tech.pegasys.teku.storage.client.RecentChainData; @@ -54,7 +55,8 @@ public static BlobSidecarGossipManager create( final GossipNetwork gossipNetwork, final GossipEncoding gossipEncoding, final ForkInfo forkInfo, - final OperationProcessor processor) { + final OperationProcessor processor, + final DebugDataDumper debugDataDumper) { final SpecVersion forkSpecVersion = spec.atEpoch(forkInfo.getFork().getEpoch()); final BlobSidecarSchema gossipType = SchemaDefinitionsDeneb.required(forkSpecVersion.getSchemaDefinitions()) @@ -74,7 +76,8 @@ public static BlobSidecarGossipManager create( processor, gossipEncoding, forkInfo, - gossipType); + gossipType, + debugDataDumper); subnetIdToTopicHandler.put(subnetId, topicHandler); }); return new BlobSidecarGossipManager( @@ -135,7 +138,8 @@ private static Eth2TopicHandler createBlobSidecarTopicHandler( final OperationProcessor processor, final GossipEncoding gossipEncoding, final ForkInfo forkInfo, - final BlobSidecarSchema gossipType) { + final BlobSidecarSchema gossipType, + final DebugDataDumper debugDataDumper) { return new Eth2TopicHandler<>( recentChainData, asyncRunner, @@ -148,7 +152,8 @@ private static Eth2TopicHandler createBlobSidecarTopicHandler( forkInfo.getFork(), blobSidecar -> spec.computeEpochAtSlot(blobSidecar.getSlot())), gossipType, - spec.getNetworkingConfig()); + spec.getNetworkingConfig(), + debugDataDumper); } private record TopicSubnetIdAwareOperationProcessor( diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/BlockGossipManager.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/BlockGossipManager.java index 0502c0b5df5..c078e76fdd2 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/BlockGossipManager.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/BlockGossipManager.java @@ -21,6 +21,7 @@ import tech.pegasys.teku.spec.Spec; import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class BlockGossipManager extends AbstractGossipManager { @@ -32,7 +33,8 @@ public BlockGossipManager( final GossipNetwork gossipNetwork, final GossipEncoding gossipEncoding, final ForkInfo forkInfo, - final OperationProcessor processor) { + final OperationProcessor processor, + final DebugDataDumper debugDataDumper) { super( recentChainData, GossipTopicName.BEACON_BLOCK, @@ -45,7 +47,8 @@ public BlockGossipManager( .getSchemaDefinitions() .getSignedBeaconBlockSchema(), block -> spec.computeEpochAtSlot(block.getSlot()), - spec.getNetworkingConfig()); + spec.getNetworkingConfig(), + debugDataDumper); } public void publishBlock(final SignedBeaconBlock message) { diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/ProposerSlashingGossipManager.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/ProposerSlashingGossipManager.java index 48965957713..8f3e707913e 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/ProposerSlashingGossipManager.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/ProposerSlashingGossipManager.java @@ -21,6 +21,7 @@ import tech.pegasys.teku.spec.config.NetworkingSpecConfig; import tech.pegasys.teku.spec.datastructures.operations.ProposerSlashing; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class ProposerSlashingGossipManager extends AbstractGossipManager { @@ -32,7 +33,8 @@ public ProposerSlashingGossipManager( final GossipEncoding gossipEncoding, final ForkInfo forkInfo, final OperationProcessor processor, - final NetworkingSpecConfig networkingConfig) { + final NetworkingSpecConfig networkingConfig, + final DebugDataDumper debugDataDumper) { super( recentChainData, GossipTopicName.PROPOSER_SLASHING, @@ -46,7 +48,8 @@ public ProposerSlashingGossipManager( recentChainData .getSpec() .computeEpochAtSlot(message.getHeader1().getMessage().getSlot()), - networkingConfig); + networkingConfig, + debugDataDumper); } public void publishProposerSlashing(final ProposerSlashing message) { diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/SignedBlsToExecutionChangeGossipManager.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/SignedBlsToExecutionChangeGossipManager.java index a57b45e7bbc..ee169cefeb7 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/SignedBlsToExecutionChangeGossipManager.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/SignedBlsToExecutionChangeGossipManager.java @@ -22,6 +22,7 @@ import tech.pegasys.teku.spec.datastructures.operations.SignedBlsToExecutionChange; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.schemas.SchemaDefinitionsCapella; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class SignedBlsToExecutionChangeGossipManager @@ -35,7 +36,8 @@ public SignedBlsToExecutionChangeGossipManager( final GossipEncoding gossipEncoding, final ForkInfo forkInfo, final OperationProcessor processor, - final NetworkingSpecConfig networkingConfig) { + final NetworkingSpecConfig networkingConfig, + final DebugDataDumper debugDataDumper) { super( recentChainData, GossipTopicName.BLS_TO_EXECUTION_CHANGE, @@ -48,7 +50,8 @@ public SignedBlsToExecutionChangeGossipManager( // BLS changes don't have a fork they apply to so are always considered to match the fork // of the topic they arrived on (ie disable fork checking at this level) message -> forkInfo.getFork().getEpoch(), - networkingConfig); + networkingConfig, + debugDataDumper); } public void publish(final SignedBlsToExecutionChange message) { diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/SignedContributionAndProofGossipManager.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/SignedContributionAndProofGossipManager.java index aaedb081eed..971bb3be02d 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/SignedContributionAndProofGossipManager.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/SignedContributionAndProofGossipManager.java @@ -22,6 +22,7 @@ import tech.pegasys.teku.spec.datastructures.operations.versions.altair.SignedContributionAndProof; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.schemas.SchemaDefinitionsAltair; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class SignedContributionAndProofGossipManager @@ -35,7 +36,8 @@ public SignedContributionAndProofGossipManager( final GossipEncoding gossipEncoding, final ForkInfo forkInfo, final OperationProcessor processor, - final NetworkingSpecConfig networkingConfig) { + final NetworkingSpecConfig networkingConfig, + final DebugDataDumper debugDataDumper) { super( recentChainData, GossipTopicName.SYNC_COMMITTEE_CONTRIBUTION_AND_PROOF, @@ -49,7 +51,8 @@ public SignedContributionAndProofGossipManager( recentChainData .getSpec() .computeEpochAtSlot(message.getMessage().getContribution().getSlot()), - networkingConfig); + networkingConfig, + debugDataDumper); } public void publishContribution(final SignedContributionAndProof message) { diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/VoluntaryExitGossipManager.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/VoluntaryExitGossipManager.java index 4bcb2da4863..b3a36456018 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/VoluntaryExitGossipManager.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/VoluntaryExitGossipManager.java @@ -21,6 +21,7 @@ import tech.pegasys.teku.spec.config.NetworkingSpecConfig; import tech.pegasys.teku.spec.datastructures.operations.SignedVoluntaryExit; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class VoluntaryExitGossipManager extends AbstractGossipManager { @@ -32,7 +33,8 @@ public VoluntaryExitGossipManager( final GossipEncoding gossipEncoding, final ForkInfo forkInfo, final OperationProcessor processor, - final NetworkingSpecConfig networkingConfig) { + final NetworkingSpecConfig networkingConfig, + final DebugDataDumper debugDataDumper) { super( recentChainData, GossipTopicName.VOLUNTARY_EXIT, @@ -43,7 +45,8 @@ public VoluntaryExitGossipManager( processor, SignedVoluntaryExit.SSZ_SCHEMA, exit -> exit.getMessage().getEpoch(), - networkingConfig); + networkingConfig, + debugDataDumper); } public void publishVoluntaryExit(final SignedVoluntaryExit message) { diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsAltair.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsAltair.java index aa7a0a4a082..018f97ef9ed 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsAltair.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsAltair.java @@ -34,6 +34,7 @@ import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.schemas.SchemaDefinitionsAltair; import tech.pegasys.teku.statetransition.synccommittee.SyncCommitteeStateUtils; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class GossipForkSubscriptionsAltair extends GossipForkSubscriptionsPhase0 { @@ -62,7 +63,8 @@ public GossipForkSubscriptionsAltair( final OperationProcessor signedContributionAndProofOperationProcessor, final OperationProcessor - syncCommitteeMessageOperationProcessor) { + syncCommitteeMessageOperationProcessor, + final DebugDataDumper debugDataDumper) { super( fork, spec, @@ -76,7 +78,8 @@ public GossipForkSubscriptionsAltair( aggregateProcessor, attesterSlashingProcessor, proposerSlashingProcessor, - voluntaryExitProcessor); + voluntaryExitProcessor, + debugDataDumper); this.signedContributionAndProofOperationProcessor = signedContributionAndProofOperationProcessor; this.syncCommitteeMessageOperationProcessor = syncCommitteeMessageOperationProcessor; @@ -95,7 +98,8 @@ void addSignedContributionAndProofGossipManager(final ForkInfo forkInfo) { gossipEncoding, forkInfo, signedContributionAndProofOperationProcessor, - specConfig.getNetworkingConfig()); + specConfig.getNetworkingConfig(), + debugDataDumper); addGossipManager(syncCommitteeContributionGossipManager); } @@ -111,7 +115,8 @@ void addSyncCommitteeMessageGossipManager(final ForkInfo forkInfo) { schemaDefinitions, asyncRunner, syncCommitteeMessageOperationProcessor, - forkInfo); + forkInfo, + debugDataDumper); syncCommitteeMessageGossipManager = new SyncCommitteeMessageGossipManager( metricsSystem, diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsBellatrix.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsBellatrix.java index 8ec2f8d46b4..f7ebdbba8c7 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsBellatrix.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsBellatrix.java @@ -27,6 +27,7 @@ import tech.pegasys.teku.spec.datastructures.operations.versions.altair.SignedContributionAndProof; import tech.pegasys.teku.spec.datastructures.operations.versions.altair.ValidatableSyncCommitteeMessage; import tech.pegasys.teku.spec.datastructures.state.Fork; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class GossipForkSubscriptionsBellatrix extends GossipForkSubscriptionsAltair { @@ -48,7 +49,8 @@ public GossipForkSubscriptionsBellatrix( final OperationProcessor signedContributionAndProofOperationProcessor, final OperationProcessor - syncCommitteeMessageOperationProcessor) { + syncCommitteeMessageOperationProcessor, + final DebugDataDumper debugDataDumper) { super( fork, spec, @@ -64,6 +66,7 @@ public GossipForkSubscriptionsBellatrix( proposerSlashingProcessor, voluntaryExitProcessor, signedContributionAndProofOperationProcessor, - syncCommitteeMessageOperationProcessor); + syncCommitteeMessageOperationProcessor, + debugDataDumper); } } diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsCapella.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsCapella.java index ea6eea41786..38ffb2ea1ee 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsCapella.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsCapella.java @@ -33,6 +33,7 @@ import tech.pegasys.teku.spec.datastructures.state.Fork; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.schemas.SchemaDefinitionsCapella; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class GossipForkSubscriptionsCapella extends GossipForkSubscriptionsBellatrix { @@ -61,7 +62,8 @@ public GossipForkSubscriptionsCapella( final OperationProcessor syncCommitteeMessageOperationProcessor, final OperationProcessor - signedBlsToExecutionChangeOperationProcessor) { + signedBlsToExecutionChangeOperationProcessor, + final DebugDataDumper debugDataDumper) { super( fork, spec, @@ -77,7 +79,8 @@ public GossipForkSubscriptionsCapella( proposerSlashingProcessor, voluntaryExitProcessor, signedContributionAndProofOperationProcessor, - syncCommitteeMessageOperationProcessor); + syncCommitteeMessageOperationProcessor, + debugDataDumper); this.signedBlsToExecutionChangeOperationProcessor = signedBlsToExecutionChangeOperationProcessor; @@ -97,7 +100,8 @@ void addSignedBlsToExecutionChangeGossipManager(final ForkInfo forkInfo) { gossipEncoding, forkInfo, signedBlsToExecutionChangeOperationProcessor, - spec.getNetworkingConfig()); + spec.getNetworkingConfig(), + debugDataDumper); addGossipManager(gossipManager); this.signedBlsToExecutionChangeGossipManager = Optional.of(gossipManager); diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsDeneb.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsDeneb.java index 5b6b3ac71e4..87d5d6e68ca 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsDeneb.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsDeneb.java @@ -31,6 +31,7 @@ import tech.pegasys.teku.spec.datastructures.operations.versions.altair.ValidatableSyncCommitteeMessage; import tech.pegasys.teku.spec.datastructures.state.Fork; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class GossipForkSubscriptionsDeneb extends GossipForkSubscriptionsCapella { @@ -59,7 +60,8 @@ public GossipForkSubscriptionsDeneb( final OperationProcessor syncCommitteeMessageOperationProcessor, final OperationProcessor - signedBlsToExecutionChangeOperationProcessor) { + signedBlsToExecutionChangeOperationProcessor, + final DebugDataDumper debugDataDumper) { super( fork, spec, @@ -76,7 +78,8 @@ public GossipForkSubscriptionsDeneb( voluntaryExitProcessor, signedContributionAndProofOperationProcessor, syncCommitteeMessageOperationProcessor, - signedBlsToExecutionChangeOperationProcessor); + signedBlsToExecutionChangeOperationProcessor, + debugDataDumper); this.blobSidecarProcessor = blobSidecarProcessor; } @@ -95,7 +98,8 @@ void addBlobSidecarGossipManager(final ForkInfo forkInfo) { discoveryNetwork, gossipEncoding, forkInfo, - blobSidecarProcessor); + blobSidecarProcessor, + debugDataDumper); addGossipManager(blobSidecarGossipManager); } diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsElectra.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsElectra.java index 5ef8d21e34b..1c8e81d2b94 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsElectra.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsElectra.java @@ -29,6 +29,7 @@ import tech.pegasys.teku.spec.datastructures.operations.versions.altair.SignedContributionAndProof; import tech.pegasys.teku.spec.datastructures.operations.versions.altair.ValidatableSyncCommitteeMessage; import tech.pegasys.teku.spec.datastructures.state.Fork; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class GossipForkSubscriptionsElectra extends GossipForkSubscriptionsDeneb { @@ -53,7 +54,8 @@ public GossipForkSubscriptionsElectra( final OperationProcessor syncCommitteeMessageOperationProcessor, final OperationProcessor - signedBlsToExecutionChangeOperationProcessor) { + signedBlsToExecutionChangeOperationProcessor, + final DebugDataDumper debugDataDumper) { super( fork, spec, @@ -71,6 +73,7 @@ public GossipForkSubscriptionsElectra( voluntaryExitProcessor, signedContributionAndProofOperationProcessor, syncCommitteeMessageOperationProcessor, - signedBlsToExecutionChangeOperationProcessor); + signedBlsToExecutionChangeOperationProcessor, + debugDataDumper); } } diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsPhase0.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsPhase0.java index 8758eed7d3a..766775e413d 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsPhase0.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsPhase0.java @@ -39,6 +39,7 @@ import tech.pegasys.teku.spec.datastructures.operations.SignedVoluntaryExit; import tech.pegasys.teku.spec.datastructures.state.Fork; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class GossipForkSubscriptionsPhase0 implements GossipForkSubscriptions { @@ -65,6 +66,7 @@ public class GossipForkSubscriptionsPhase0 implements GossipForkSubscriptions { private VoluntaryExitGossipManager voluntaryExitGossipManager; private ProposerSlashingGossipManager proposerSlashingGossipManager; private AttesterSlashingGossipManager attesterSlashingGossipManager; + protected DebugDataDumper debugDataDumper; public GossipForkSubscriptionsPhase0( final Fork fork, @@ -79,7 +81,8 @@ public GossipForkSubscriptionsPhase0( final OperationProcessor aggregateProcessor, final OperationProcessor attesterSlashingProcessor, final OperationProcessor proposerSlashingProcessor, - final OperationProcessor voluntaryExitProcessor) { + final OperationProcessor voluntaryExitProcessor, + final DebugDataDumper debugDataDumper) { this.fork = fork; this.spec = spec; this.asyncRunner = asyncRunner; @@ -93,6 +96,7 @@ public GossipForkSubscriptionsPhase0( this.attesterSlashingProcessor = attesterSlashingProcessor; this.proposerSlashingProcessor = proposerSlashingProcessor; this.voluntaryExitProcessor = voluntaryExitProcessor; + this.debugDataDumper = debugDataDumper; } @Override @@ -121,7 +125,8 @@ void addAttestationGossipManager(final ForkInfo forkInfo) { gossipEncoding, recentChainData, attestationProcessor, - forkInfo); + forkInfo, + debugDataDumper); attestationGossipManager = new AttestationGossipManager(metricsSystem, attestationSubnetSubscriptions); @@ -137,7 +142,8 @@ void addBlockGossipManager(final ForkInfo forkInfo) { discoveryNetwork, gossipEncoding, forkInfo, - blockProcessor); + blockProcessor, + debugDataDumper); addGossipManager(blockGossipManager); } @@ -150,7 +156,8 @@ void addAggregateGossipManager(final ForkInfo forkInfo) { discoveryNetwork, gossipEncoding, forkInfo, - aggregateProcessor); + aggregateProcessor, + debugDataDumper); addGossipManager(aggregateGossipManager); } @@ -163,7 +170,8 @@ void addVoluntaryExitGossipManager(final ForkInfo forkInfo) { gossipEncoding, forkInfo, voluntaryExitProcessor, - spec.getNetworkingConfig()); + spec.getNetworkingConfig(), + debugDataDumper); addGossipManager(voluntaryExitGossipManager); } @@ -176,7 +184,8 @@ void addProposerSlashingGossipManager(final ForkInfo forkInfo) { gossipEncoding, forkInfo, proposerSlashingProcessor, - spec.getNetworkingConfig()); + spec.getNetworkingConfig(), + debugDataDumper); addGossipManager(proposerSlashingGossipManager); } @@ -189,7 +198,8 @@ void addAttesterSlashingGossipManager(final ForkInfo forkInfo) { discoveryNetwork, gossipEncoding, forkInfo, - attesterSlashingProcessor); + attesterSlashingProcessor, + debugDataDumper); addGossipManager(attesterSlashingGossipManager); } diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/subnets/AttestationSubnetSubscriptions.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/subnets/AttestationSubnetSubscriptions.java index 79d47422724..178e6bc982c 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/subnets/AttestationSubnetSubscriptions.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/subnets/AttestationSubnetSubscriptions.java @@ -30,6 +30,7 @@ import tech.pegasys.teku.spec.datastructures.operations.Attestation; import tech.pegasys.teku.spec.datastructures.operations.Attestation.AttestationSchema; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class AttestationSubnetSubscriptions extends CommitteeSubnetSubscriptions { @@ -40,6 +41,7 @@ public class AttestationSubnetSubscriptions extends CommitteeSubnetSubscriptions private final OperationProcessor processor; private final ForkInfo forkInfo; private final AttestationSchema attestationSchema; + private final DebugDataDumper debugDataDumper; public AttestationSubnetSubscriptions( final Spec spec, @@ -48,7 +50,8 @@ public AttestationSubnetSubscriptions( final GossipEncoding gossipEncoding, final RecentChainData recentChainData, final OperationProcessor processor, - final ForkInfo forkInfo) { + final ForkInfo forkInfo, + final DebugDataDumper debugDataDumper) { super(gossipNetwork, gossipEncoding); this.spec = spec; this.asyncRunner = asyncRunner; @@ -57,6 +60,7 @@ public AttestationSubnetSubscriptions( this.forkInfo = forkInfo; attestationSchema = spec.atEpoch(forkInfo.getFork().getEpoch()).getSchemaDefinitions().getAttestationSchema(); + this.debugDataDumper = debugDataDumper; } public SafeFuture gossip(final Attestation attestation) { @@ -85,6 +89,7 @@ SafeFuture> getChannel(final Attestation attestation) { @Override protected Eth2TopicHandler createTopicHandler(final int subnetId) { final String topicName = GossipTopicName.getAttestationSubnetTopicName(subnetId); + return SingleAttestationTopicHandler.createHandler( recentChainData, asyncRunner, @@ -93,7 +98,8 @@ protected Eth2TopicHandler createTopicHandler(final int subnetId) { forkInfo, topicName, attestationSchema, - subnetId); + subnetId, + debugDataDumper); } private SafeFuture> computeSubnetForAttestation(final Attestation attestation) { diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/subnets/SyncCommitteeSubnetSubscriptions.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/subnets/SyncCommitteeSubnetSubscriptions.java index e044761b363..5bd63e3ac62 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/subnets/SyncCommitteeSubnetSubscriptions.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/subnets/SyncCommitteeSubnetSubscriptions.java @@ -27,6 +27,7 @@ import tech.pegasys.teku.spec.datastructures.operations.versions.altair.ValidatableSyncCommitteeMessage; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.schemas.SchemaDefinitionsAltair; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class SyncCommitteeSubnetSubscriptions extends CommitteeSubnetSubscriptions { @@ -37,6 +38,7 @@ public class SyncCommitteeSubnetSubscriptions extends CommitteeSubnetSubscriptio private final AsyncRunner asyncRunner; private final OperationProcessor processor; private final ForkInfo forkInfo; + private final DebugDataDumper debugDataDumper; public SyncCommitteeSubnetSubscriptions( final Spec spec, @@ -46,7 +48,8 @@ public SyncCommitteeSubnetSubscriptions( final SchemaDefinitionsAltair schemaDefinitions, final AsyncRunner asyncRunner, final OperationProcessor processor, - final ForkInfo forkInfo) { + final ForkInfo forkInfo, + final DebugDataDumper debugDataDumper) { super(gossipNetwork, gossipEncoding); this.spec = spec; this.recentChainData = recentChainData; @@ -54,6 +57,7 @@ public SyncCommitteeSubnetSubscriptions( this.asyncRunner = asyncRunner; this.processor = processor; this.forkInfo = forkInfo; + this.debugDataDumper = debugDataDumper; } public SafeFuture gossip(final SyncCommitteeMessage message, final int subnetId) { @@ -81,6 +85,7 @@ protected Eth2TopicHandler createTopicHandler(final int subnetId) { forkInfo.getFork(), message -> spec.computeEpochAtSlot(message.getSlot())), schemaDefinitions.getSyncCommitteeMessageSchema(), - spec.getNetworkingConfig()); + spec.getNetworkingConfig(), + debugDataDumper); } } diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/topics/topichandlers/Eth2TopicHandler.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/topics/topichandlers/Eth2TopicHandler.java index 3aa03596cc8..67f99ae1641 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/topics/topichandlers/Eth2TopicHandler.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/topics/topichandlers/Eth2TopicHandler.java @@ -40,6 +40,7 @@ import tech.pegasys.teku.networking.p2p.gossip.TopicHandler; import tech.pegasys.teku.service.serviceutils.ServiceCapacityExceededException; import tech.pegasys.teku.spec.config.NetworkingSpecConfig; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; import tech.pegasys.teku.storage.client.RecentChainData; @@ -54,6 +55,7 @@ public class Eth2TopicHandler implements TopicHandler private final Eth2PreparedGossipMessageFactory preparedGossipMessageFactory; private final OperationMilestoneValidator forkValidator; private final NetworkingSpecConfig networkingConfig; + private final DebugDataDumper debugDataDumper; public Eth2TopicHandler( final RecentChainData recentChainData, @@ -64,7 +66,8 @@ public Eth2TopicHandler( final String topicName, final OperationMilestoneValidator forkValidator, final SszSchema messageType, - final NetworkingSpecConfig networkingConfig) { + final NetworkingSpecConfig networkingConfig, + final DebugDataDumper debugDataDumper) { this.asyncRunner = asyncRunner; this.processor = processor; this.gossipEncoding = gossipEncoding; @@ -76,6 +79,7 @@ public Eth2TopicHandler( this.preparedGossipMessageFactory = gossipEncoding.createPreparedGossipMessageFactory( recentChainData::getMilestoneByForkDigest); + this.debugDataDumper = debugDataDumper; } public Eth2TopicHandler( @@ -87,7 +91,8 @@ public Eth2TopicHandler( final GossipTopicName topicName, final OperationMilestoneValidator forkValidator, final SszSchema messageType, - final NetworkingSpecConfig networkingConfig) { + final NetworkingSpecConfig networkingConfig, + final DebugDataDumper debugDataDumper) { this( recentChainData, asyncRunner, @@ -97,7 +102,8 @@ public Eth2TopicHandler( topicName.toString(), forkValidator, messageType, - networkingConfig); + networkingConfig, + debugDataDumper); } @Override @@ -129,6 +135,11 @@ private void processMessage( final PreparedGossipMessage message) { switch (internalValidationResult.code()) { case REJECT: + debugDataDumper.saveGossipRejectedMessageToFile( + getTopic(), + message.getArrivalTimestamp(), + () -> message.getDecodedMessage().getDecodedMessage().orElse(Bytes.EMPTY), + internalValidationResult.getDescription()); P2P_LOG.onGossipRejected( getTopic(), message.getDecodedMessage().getDecodedMessage().orElse(Bytes.EMPTY), @@ -160,6 +171,9 @@ protected ValidationResult handleMessageProcessingError( final PreparedGossipMessage message, final Throwable err) { final ValidationResult response; if (ExceptionUtil.hasCause(err, DecodingException.class)) { + + debugDataDumper.saveGossipMessageDecodingError( + getTopic(), message.getArrivalTimestamp(), message::getOriginalMessage, err); P2P_LOG.onGossipMessageDecodingError(getTopic(), message.getOriginalMessage(), err); response = ValidationResult.Invalid; } else if (ExceptionUtil.hasCause(err, RejectedExecutionException.class)) { diff --git a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/topics/topichandlers/SingleAttestationTopicHandler.java b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/topics/topichandlers/SingleAttestationTopicHandler.java index f88a7d6ee9c..1dd00defd83 100644 --- a/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/topics/topichandlers/SingleAttestationTopicHandler.java +++ b/networking/eth2/src/main/java/tech/pegasys/teku/networking/eth2/gossip/topics/topichandlers/SingleAttestationTopicHandler.java @@ -22,6 +22,7 @@ import tech.pegasys.teku.spec.datastructures.operations.Attestation; import tech.pegasys.teku.spec.datastructures.operations.Attestation.AttestationSchema; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; public class SingleAttestationTopicHandler { @@ -34,7 +35,8 @@ public static Eth2TopicHandler createHandler( final ForkInfo forkInfo, final String topicName, final AttestationSchema attestationSchema, - final int subnetId) { + final int subnetId, + final DebugDataDumper debugDataDumper) { final Spec spec = recentChainData.getSpec(); OperationProcessor convertingProcessor = @@ -54,6 +56,7 @@ public static Eth2TopicHandler createHandler( forkInfo.getFork(), message -> spec.computeEpochAtSlot(message.getData().getSlot())), attestationSchema, - spec.getNetworkingConfig()); + spec.getNetworkingConfig(), + debugDataDumper); } } diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AbstractGossipManagerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AbstractGossipManagerTest.java index 45c3128500a..57390f5eb95 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AbstractGossipManagerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AbstractGossipManagerTest.java @@ -39,6 +39,7 @@ import tech.pegasys.teku.spec.config.NetworkingSpecConfig; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.util.DataStructureUtil; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.RecentChainData; import tech.pegasys.teku.storage.storageSystem.InMemoryStorageSystemBuilder; import tech.pegasys.teku.storage.storageSystem.StorageSystem; @@ -176,7 +177,8 @@ protected TestGossipManager( processor, gossipType, message -> UInt64.ZERO, - networkingConfig); + networkingConfig, + mock(DebugDataDumper.class)); } } } diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AggregateGossipManagerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AggregateGossipManagerTest.java index ffd2053721b..5af68eba3f6 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AggregateGossipManagerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AggregateGossipManagerTest.java @@ -35,6 +35,7 @@ import tech.pegasys.teku.spec.datastructures.operations.SignedAggregateAndProof; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.util.DataStructureUtil; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.storageSystem.InMemoryStorageSystemBuilder; import tech.pegasys.teku.storage.storageSystem.StorageSystem; @@ -70,7 +71,8 @@ public void setup() { gossipNetwork, gossipEncoding, forkInfo, - processor); + processor, + mock(DebugDataDumper.class)); gossipManager.subscribe(); } diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AttestationGossipManagerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AttestationGossipManagerTest.java index 933a63c6183..55f846a1503 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AttestationGossipManagerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/AttestationGossipManagerTest.java @@ -41,6 +41,7 @@ import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.util.DataStructureUtil; import tech.pegasys.teku.statetransition.BeaconChainUtil; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.MemoryOnlyRecentChainData; import tech.pegasys.teku.storage.client.RecentChainData; @@ -69,7 +70,8 @@ public class AttestationGossipManagerTest { gossipEncoding, recentChainData, gossipedAttestationProcessor, - forkInfo); + forkInfo, + mock(DebugDataDumper.class)); @BeforeEach public void setup() { diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/BlobSidecarGossipManagerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/BlobSidecarGossipManagerTest.java index 4bd85f91b0a..adb92b46f20 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/BlobSidecarGossipManagerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/BlobSidecarGossipManagerTest.java @@ -47,6 +47,7 @@ import tech.pegasys.teku.spec.datastructures.blobs.versions.deneb.BlobSidecar; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.util.DataStructureUtil; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; import tech.pegasys.teku.storage.storageSystem.InMemoryStorageSystemBuilder; import tech.pegasys.teku.storage.storageSystem.StorageSystem; @@ -103,7 +104,8 @@ public void setup() { gossipNetwork, gossipEncoding, forkInfo, - processor); + processor, + mock(DebugDataDumper.class)); blobSidecarGossipManager.subscribe(); } diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/BlockGossipManagerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/BlockGossipManagerTest.java index 01efd8afb8b..be076f28556 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/BlockGossipManagerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/BlockGossipManagerTest.java @@ -34,6 +34,7 @@ import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.util.DataStructureUtil; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.storageSystem.InMemoryStorageSystemBuilder; import tech.pegasys.teku.storage.storageSystem.StorageSystem; @@ -68,7 +69,8 @@ public void setup() { gossipNetwork, gossipEncoding, forkInfo, - processor); + processor, + mock(DebugDataDumper.class)); blockGossipManager.subscribe(); } diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsCapellaTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsCapellaTest.java index 76b9fb572e4..5be0c6c36ed 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsCapellaTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsCapellaTest.java @@ -31,6 +31,7 @@ import tech.pegasys.teku.spec.datastructures.operations.SignedBlsToExecutionChange; import tech.pegasys.teku.spec.datastructures.state.Fork; import tech.pegasys.teku.spec.util.DataStructureUtil; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.MemoryOnlyRecentChainData; import tech.pegasys.teku.storage.client.RecentChainData; @@ -81,6 +82,7 @@ private GossipForkSubscriptionsCapella createGossipForkSubscriptionCapella() { noopOperationProcessor, noopOperationProcessor, noopOperationProcessor, - noopOperationProcessor); + noopOperationProcessor, + mock(DebugDataDumper.class)); } } diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsDenebTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsDenebTest.java index 78d04e8e7e3..3eafbd0b885 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsDenebTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/forks/versions/GossipForkSubscriptionsDenebTest.java @@ -32,6 +32,7 @@ import tech.pegasys.teku.spec.datastructures.state.Fork; import tech.pegasys.teku.spec.datastructures.state.ForkInfo; import tech.pegasys.teku.spec.util.DataStructureUtil; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.MemoryOnlyRecentChainData; import tech.pegasys.teku.storage.client.RecentChainData; @@ -81,6 +82,7 @@ private GossipForkSubscriptionsDeneb createGossipForkSubscriptionDeneb() { noopOperationProcessor, noopOperationProcessor, noopOperationProcessor, - noopOperationProcessor); + noopOperationProcessor, + mock(DebugDataDumper.class)); } } diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/subnets/AttestationSubnetSubscriptionsTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/subnets/AttestationSubnetSubscriptionsTest.java index b9e40ea6172..25cda79aafe 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/subnets/AttestationSubnetSubscriptionsTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/subnets/AttestationSubnetSubscriptionsTest.java @@ -38,6 +38,7 @@ import tech.pegasys.teku.spec.datastructures.operations.Attestation; import tech.pegasys.teku.spec.util.DataStructureUtil; import tech.pegasys.teku.statetransition.BeaconChainUtil; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.client.MemoryOnlyRecentChainData; import tech.pegasys.teku.storage.client.RecentChainData; @@ -67,7 +68,8 @@ void setUp() { gossipEncoding, recentChainData, processor, - recentChainData.getCurrentForkInfo().orElseThrow()); + recentChainData.getCurrentForkInfo().orElseThrow(), + mock(DebugDataDumper.class)); subnetSubscriptions.subscribe(); when(gossipNetwork.subscribe(any(), any())).thenReturn(mock(TopicChannel.class)); diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/AggregateTopicHandlerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/AggregateTopicHandlerTest.java index ca276059c89..258515cd549 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/AggregateTopicHandlerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/AggregateTopicHandlerTest.java @@ -14,6 +14,7 @@ package tech.pegasys.teku.networking.eth2.gossip.topics; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; @@ -24,6 +25,7 @@ import tech.pegasys.teku.networking.eth2.gossip.AggregateGossipManager; import tech.pegasys.teku.networking.eth2.gossip.topics.topichandlers.Eth2TopicHandler; import tech.pegasys.teku.spec.datastructures.attestation.ValidatableAttestation; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; public class AggregateTopicHandlerTest extends AbstractTopicHandlerTest { @@ -31,7 +33,14 @@ public class AggregateTopicHandlerTest extends AbstractTopicHandlerTest createHandler() { return new AggregateGossipManager( - spec, recentChainData, asyncRunner, gossipNetwork, gossipEncoding, forkInfo, processor) + spec, + recentChainData, + asyncRunner, + gossipNetwork, + gossipEncoding, + forkInfo, + processor, + mock(DebugDataDumper.class)) .getTopicHandler(); } diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/AttesterSlashingTopicHandlerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/AttesterSlashingTopicHandlerTest.java index b2954d77c7e..91b98a5b8fc 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/AttesterSlashingTopicHandlerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/AttesterSlashingTopicHandlerTest.java @@ -14,6 +14,7 @@ package tech.pegasys.teku.networking.eth2.gossip.topics; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; @@ -25,6 +26,7 @@ import tech.pegasys.teku.networking.eth2.gossip.AttesterSlashingGossipManager; import tech.pegasys.teku.networking.eth2.gossip.topics.topichandlers.Eth2TopicHandler; import tech.pegasys.teku.spec.datastructures.operations.AttesterSlashing; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; public class AttesterSlashingTopicHandlerTest extends AbstractTopicHandlerTest { @@ -33,7 +35,14 @@ public class AttesterSlashingTopicHandlerTest extends AbstractTopicHandlerTest createHandler() { final AttesterSlashingGossipManager gossipManager = new AttesterSlashingGossipManager( - spec, recentChainData, asyncRunner, null, gossipEncoding, forkInfo, processor); + spec, + recentChainData, + asyncRunner, + null, + gossipEncoding, + forkInfo, + processor, + mock(DebugDataDumper.class)); return gossipManager.getTopicHandler(); } diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/BlockTopicHandlerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/BlockTopicHandlerTest.java index 9bde5f5cec3..fea1f9f1b85 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/BlockTopicHandlerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/BlockTopicHandlerTest.java @@ -16,6 +16,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; @@ -29,6 +30,7 @@ import tech.pegasys.teku.networking.eth2.gossip.topics.topichandlers.Eth2TopicHandler; import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; import tech.pegasys.teku.spec.generator.ChainBuilder.BlockOptions; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; public class BlockTopicHandlerTest extends AbstractTopicHandlerTest { @@ -36,7 +38,14 @@ public class BlockTopicHandlerTest extends AbstractTopicHandlerTest createHandler() { return new BlockGossipManager( - recentChainData, spec, asyncRunner, gossipNetwork, gossipEncoding, forkInfo, processor) + recentChainData, + spec, + asyncRunner, + gossipNetwork, + gossipEncoding, + forkInfo, + processor, + mock(DebugDataDumper.class)) .getTopicHandler(); } diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/Eth2TopicHandlerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/Eth2TopicHandlerTest.java index e7cd99dcabd..02bd0d9aa86 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/Eth2TopicHandlerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/Eth2TopicHandlerTest.java @@ -13,6 +13,10 @@ package tech.pegasys.teku.networking.eth2.gossip.topics; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static tech.pegasys.teku.infrastructure.async.SafeFutureAssert.assertThatSafeFuture; import io.libp2p.core.pubsub.ValidationResult; @@ -37,6 +41,7 @@ import tech.pegasys.teku.spec.TestSpecFactory; import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; import tech.pegasys.teku.spec.util.DataStructureUtil; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; import tech.pegasys.teku.storage.client.RecentChainData; import tech.pegasys.teku.storage.storageSystem.InMemoryStorageSystemBuilder; @@ -50,6 +55,7 @@ public class Eth2TopicHandlerTest { private final SignedBeaconBlock block = dataStructureUtil.randomSignedBeaconBlock(1); private final Bytes blockBytes = GossipEncoding.SSZ_SNAPPY.encode(block); private final StubAsyncRunner asyncRunner = new StubAsyncRunner(); + private final DebugDataDumper debugDataDumper = mock(DebugDataDumper.class); @BeforeEach public void setup() { @@ -63,7 +69,8 @@ public void handleMessage_valid() { recentChainData, spec, asyncRunner, - (b, __) -> SafeFuture.completedFuture(InternalValidationResult.ACCEPT)); + (b, __) -> SafeFuture.completedFuture(InternalValidationResult.ACCEPT), + debugDataDumper); final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); @@ -78,11 +85,14 @@ public void handleMessage_invalid() { recentChainData, spec, asyncRunner, - (b, __) -> SafeFuture.completedFuture(InternalValidationResult.reject("Nope"))); + (b, __) -> SafeFuture.completedFuture(InternalValidationResult.reject("Nope")), + debugDataDumper); final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); asyncRunner.executeQueuedActions(); + verify(debugDataDumper) + .saveGossipRejectedMessageToFile(eq(topicHandler.getTopic()), any(), any(), any()); assertThatSafeFuture(result).isCompletedWithValue(ValidationResult.Invalid); } @@ -93,7 +103,8 @@ public void handleMessage_ignore() { recentChainData, spec, asyncRunner, - (b, __) -> SafeFuture.completedFuture(InternalValidationResult.IGNORE)); + (b, __) -> SafeFuture.completedFuture(InternalValidationResult.IGNORE), + debugDataDumper); final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); @@ -108,10 +119,13 @@ public void handleMessage_invalidBytes() { recentChainData, spec, asyncRunner, - (b, __) -> SafeFuture.completedFuture(InternalValidationResult.ACCEPT)); + (b, __) -> SafeFuture.completedFuture(InternalValidationResult.ACCEPT), + debugDataDumper); final Bytes invalidBytes = Bytes.fromHexString("0x0102"); final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(invalidBytes, Optional.empty())); + verify(debugDataDumper) + .saveGossipMessageDecodingError(eq(topicHandler.getTopic()), any(), any(), any()); asyncRunner.executeQueuedActions(); assertThatSafeFuture(result).isCompletedWithValue(ValidationResult.Invalid); @@ -124,7 +138,8 @@ public void handleMessage_errorWhileProcessing_decodingException() { recentChainData, spec, asyncRunner, - (b, __) -> SafeFuture.completedFuture(InternalValidationResult.ACCEPT)); + (b, __) -> SafeFuture.completedFuture(InternalValidationResult.ACCEPT), + debugDataDumper); topicHandler.setDeserializer( (b) -> { throw new DecodingException("oops"); @@ -132,6 +147,8 @@ public void handleMessage_errorWhileProcessing_decodingException() { final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); + verify(debugDataDumper) + .saveGossipMessageDecodingError(eq(topicHandler.getTopic()), any(), any(), any()); asyncRunner.executeQueuedActions(); assertThatSafeFuture(result).isCompletedWithValue(ValidationResult.Invalid); @@ -144,7 +161,8 @@ public void handleMessage_errorWhileProcessing_wrappedDecodingException() { recentChainData, spec, asyncRunner, - (b, __) -> SafeFuture.completedFuture(InternalValidationResult.ACCEPT)); + (b, __) -> SafeFuture.completedFuture(InternalValidationResult.ACCEPT), + debugDataDumper); topicHandler.setDeserializer( (b) -> { throw new CompletionException(new DecodingException("oops")); @@ -152,6 +170,8 @@ public void handleMessage_errorWhileProcessing_wrappedDecodingException() { final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); + verify(debugDataDumper) + .saveGossipMessageDecodingError(eq(topicHandler.getTopic()), any(), any(), any()); asyncRunner.executeQueuedActions(); assertThatSafeFuture(result).isCompletedWithValue(ValidationResult.Invalid); @@ -164,7 +184,8 @@ public void handleMessage_errorWhileProcessing_decodingExceptionWithCause() { recentChainData, spec, asyncRunner, - (b, __) -> SafeFuture.completedFuture(InternalValidationResult.ACCEPT)); + (b, __) -> SafeFuture.completedFuture(InternalValidationResult.ACCEPT), + debugDataDumper); topicHandler.setDeserializer( (b) -> { throw new DecodingException("oops", new RuntimeException("oops")); @@ -172,6 +193,8 @@ public void handleMessage_errorWhileProcessing_decodingExceptionWithCause() { final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); + verify(debugDataDumper) + .saveGossipMessageDecodingError(eq(topicHandler.getTopic()), any(), any(), any()); asyncRunner.executeQueuedActions(); assertThatSafeFuture(result).isCompletedWithValue(ValidationResult.Invalid); @@ -186,7 +209,8 @@ public void handleMessage_errorWhileProcessing_rejectedExecution() { asyncRunner, (b, __) -> { throw new RejectedExecutionException("No more capacity"); - }); + }, + debugDataDumper); final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); @@ -204,7 +228,8 @@ public void handleMessage_errorWhileProcessing_wrappedRejectedExecution() { asyncRunner, (b, __) -> { throw new CompletionException(new RejectedExecutionException("No more capacity")); - }); + }, + debugDataDumper); final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); @@ -222,7 +247,8 @@ public void handleMessage_errorWhileProcessing_rejectedExecutionWithRootCause() asyncRunner, (b, __) -> { throw new RejectedExecutionException("No more capacity", new NullPointerException()); - }); + }, + debugDataDumper); final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); @@ -240,7 +266,8 @@ public void handleMessage_errorWhileProcessing_serviceCapacityExceededExecution( asyncRunner, (b, __) -> { throw new ServiceCapacityExceededException("No more capacity"); - }); + }, + debugDataDumper); final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); @@ -259,7 +286,8 @@ public void handleMessage_errorWhileProcessing_wrappedServiceCapacityExceededExe (b, __) -> { throw new CompletionException( new ServiceCapacityExceededException("No more capacity")); - }); + }, + debugDataDumper); final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); @@ -277,7 +305,8 @@ public void handleMessage_errorWhileProcessing_unknownError() { asyncRunner, (b, __) -> { throw new NullPointerException(); - }); + }, + debugDataDumper); final SafeFuture result = topicHandler.handleMessage(topicHandler.prepareMessage(blockBytes, Optional.empty())); @@ -295,7 +324,8 @@ protected MockEth2TopicHandler( final RecentChainData recentChainData, final Spec spec, final AsyncRunner asyncRunner, - final OperationProcessor processor) { + final OperationProcessor processor, + final DebugDataDumper debugDataDumper) { super( recentChainData, asyncRunner, @@ -306,7 +336,8 @@ protected MockEth2TopicHandler( new OperationMilestoneValidator<>( spec, spec.getForkSchedule().getFork(UInt64.ZERO), message -> UInt64.ZERO), spec.getGenesisSchemaDefinitions().getSignedBeaconBlockSchema(), - spec.getNetworkingConfig()); + spec.getNetworkingConfig(), + debugDataDumper); this.forkDigest = recentChainData.getForkDigestByMilestone(SpecMilestone.PHASE0).orElseThrow(); deserializer = diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/ProposerSlashingTopicHandlerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/ProposerSlashingTopicHandlerTest.java index 32b80a67a17..a7dac19ba8a 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/ProposerSlashingTopicHandlerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/ProposerSlashingTopicHandlerTest.java @@ -14,6 +14,7 @@ package tech.pegasys.teku.networking.eth2.gossip.topics; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; @@ -26,6 +27,7 @@ import tech.pegasys.teku.networking.eth2.gossip.ProposerSlashingGossipManager; import tech.pegasys.teku.networking.eth2.gossip.topics.topichandlers.Eth2TopicHandler; import tech.pegasys.teku.spec.datastructures.operations.ProposerSlashing; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; public class ProposerSlashingTopicHandlerTest extends AbstractTopicHandlerTest { @@ -39,7 +41,8 @@ protected Eth2TopicHandler createHandler() { gossipEncoding, forkInfo, processor, - spec.getNetworkingConfig()) + spec.getNetworkingConfig(), + mock(DebugDataDumper.class)) .getTopicHandler(); } diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/SingleAttestationTopicHandlerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/SingleAttestationTopicHandlerTest.java index 3f9607898d9..608850e2193 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/SingleAttestationTopicHandlerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/SingleAttestationTopicHandlerTest.java @@ -14,6 +14,7 @@ package tech.pegasys.teku.networking.eth2.gossip.topics; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; @@ -30,6 +31,7 @@ import tech.pegasys.teku.spec.datastructures.attestation.ValidatableAttestation; import tech.pegasys.teku.spec.datastructures.blocks.StateAndBlockSummary; import tech.pegasys.teku.spec.generator.AttestationGenerator; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; public class SingleAttestationTopicHandlerTest @@ -48,7 +50,8 @@ protected Eth2TopicHandler createHandler() { forkInfo, GossipTopicName.getAttestationSubnetTopicName(SUBNET_ID), spec.getGenesisSchemaDefinitions().getAttestationSchema(), - SUBNET_ID); + SUBNET_ID, + mock(DebugDataDumper.class)); } @Test diff --git a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/VoluntaryExitTopicHandlerTest.java b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/VoluntaryExitTopicHandlerTest.java index b45c6561293..aa332537957 100644 --- a/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/VoluntaryExitTopicHandlerTest.java +++ b/networking/eth2/src/test/java/tech/pegasys/teku/networking/eth2/gossip/topics/VoluntaryExitTopicHandlerTest.java @@ -14,6 +14,7 @@ package tech.pegasys.teku.networking.eth2.gossip.topics; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; import static tech.pegasys.teku.infrastructure.async.SafeFutureAssert.safeJoin; @@ -28,6 +29,7 @@ import tech.pegasys.teku.spec.datastructures.operations.SignedVoluntaryExit; import tech.pegasys.teku.spec.datastructures.state.beaconstate.BeaconState; import tech.pegasys.teku.spec.generator.VoluntaryExitGenerator; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.validation.InternalValidationResult; import tech.pegasys.teku.storage.storageSystem.InMemoryStorageSystemBuilder; import tech.pegasys.teku.storage.storageSystem.StorageSystem; @@ -47,7 +49,8 @@ protected Eth2TopicHandler createHandler() { gossipEncoding, forkInfo, processor, - spec.getNetworkingConfig()) + spec.getNetworkingConfig(), + mock(DebugDataDumper.class)) .getTopicHandler(); } diff --git a/networking/eth2/src/testFixtures/java/tech/pegasys/teku/networking/eth2/Eth2P2PNetworkFactory.java b/networking/eth2/src/testFixtures/java/tech/pegasys/teku/networking/eth2/Eth2P2PNetworkFactory.java index f6da52cb668..cce64085e3a 100644 --- a/networking/eth2/src/testFixtures/java/tech/pegasys/teku/networking/eth2/Eth2P2PNetworkFactory.java +++ b/networking/eth2/src/testFixtures/java/tech/pegasys/teku/networking/eth2/Eth2P2PNetworkFactory.java @@ -98,6 +98,7 @@ import tech.pegasys.teku.spec.schemas.SchemaDefinitionsSupplier; import tech.pegasys.teku.statetransition.BeaconChainUtil; import tech.pegasys.teku.statetransition.block.VerifiedBlockOperationsListener; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.storage.api.StorageQueryChannel; import tech.pegasys.teku.storage.api.StubStorageQueryChannel; import tech.pegasys.teku.storage.client.CombinedChainDataClient; @@ -156,6 +157,7 @@ public class Eth2P2PNetworkBuilder { protected Duration eth2StatusUpdateInterval; protected Spec spec = TestSpecFactory.createMinimalPhase0(); private int earliestAvailableBlockSlotFrequency = 0; + protected DebugDataDumper debugDataDumper; public Eth2P2PNetwork startNetwork() throws Exception { setDefaults(); @@ -355,7 +357,8 @@ private GossipForkSubscriptions createSubscriptions( gossipedAggregateProcessor, attesterSlashingProcessor, proposerSlashingProcessor, - voluntaryExitProcessor); + voluntaryExitProcessor, + debugDataDumper); case ALTAIR -> new GossipForkSubscriptionsAltair( forkAndSpecMilestone.getFork(), spec, @@ -371,7 +374,8 @@ private GossipForkSubscriptions createSubscriptions( proposerSlashingProcessor, voluntaryExitProcessor, signedContributionAndProofProcessor, - syncCommitteeMessageProcessor); + syncCommitteeMessageProcessor, + debugDataDumper); case BELLATRIX -> new GossipForkSubscriptionsBellatrix( forkAndSpecMilestone.getFork(), spec, @@ -387,7 +391,8 @@ private GossipForkSubscriptions createSubscriptions( proposerSlashingProcessor, voluntaryExitProcessor, signedContributionAndProofProcessor, - syncCommitteeMessageProcessor); + syncCommitteeMessageProcessor, + debugDataDumper); case CAPELLA -> new GossipForkSubscriptionsCapella( forkAndSpecMilestone.getFork(), spec, @@ -404,7 +409,8 @@ private GossipForkSubscriptions createSubscriptions( voluntaryExitProcessor, signedContributionAndProofProcessor, syncCommitteeMessageProcessor, - signedBlsToExecutionChangeProcessor); + signedBlsToExecutionChangeProcessor, + debugDataDumper); case DENEB -> new GossipForkSubscriptionsDeneb( forkAndSpecMilestone.getFork(), spec, @@ -422,7 +428,8 @@ private GossipForkSubscriptions createSubscriptions( voluntaryExitProcessor, signedContributionAndProofProcessor, syncCommitteeMessageProcessor, - signedBlsToExecutionChangeProcessor); + signedBlsToExecutionChangeProcessor, + debugDataDumper); case ELECTRA -> new GossipForkSubscriptionsElectra( forkAndSpecMilestone.getFork(), spec, @@ -440,7 +447,8 @@ private GossipForkSubscriptions createSubscriptions( voluntaryExitProcessor, signedContributionAndProofProcessor, syncCommitteeMessageProcessor, - signedBlsToExecutionChangeProcessor); + signedBlsToExecutionChangeProcessor, + debugDataDumper); }; } @@ -715,5 +723,11 @@ public Eth2P2PNetworkBuilder eth2StatusUpdateInterval(Duration eth2StatusUpdateI this.eth2StatusUpdateInterval = eth2StatusUpdateInterval; return this; } + + public Eth2P2PNetworkBuilder debugDataDumper(final DebugDataDumper debugDataDumper) { + checkNotNull(debugDataDumper); + this.debugDataDumper = debugDataDumper; + return this; + } } } diff --git a/services/beaconchain/src/main/java/tech/pegasys/teku/services/beaconchain/BeaconChainController.java b/services/beaconchain/src/main/java/tech/pegasys/teku/services/beaconchain/BeaconChainController.java index 9a11564014b..d950b816fb7 100644 --- a/services/beaconchain/src/main/java/tech/pegasys/teku/services/beaconchain/BeaconChainController.java +++ b/services/beaconchain/src/main/java/tech/pegasys/teku/services/beaconchain/BeaconChainController.java @@ -149,9 +149,11 @@ import tech.pegasys.teku.statetransition.synccommittee.SyncCommitteeMessageValidator; import tech.pegasys.teku.statetransition.synccommittee.SyncCommitteeStateUtils; import tech.pegasys.teku.statetransition.util.BlockBlobSidecarsTrackersPoolImpl; +import tech.pegasys.teku.statetransition.util.DebugDataDumper; import tech.pegasys.teku.statetransition.util.FutureItems; import tech.pegasys.teku.statetransition.util.PendingPool; import tech.pegasys.teku.statetransition.util.PoolFactory; +import tech.pegasys.teku.statetransition.util.noop.NoOpDebugDataDumper; import tech.pegasys.teku.statetransition.validation.AggregateAttestationValidator; import tech.pegasys.teku.statetransition.validation.AttestationValidator; import tech.pegasys.teku.statetransition.validation.AttesterSlashingValidator; @@ -287,6 +289,7 @@ public class BeaconChainController extends Service implements BeaconChainControl protected PoolFactory poolFactory; protected SettableLabelledGauge futureItemsMetric; protected IntSupplier rejectedExecutionCountSupplier; + protected DebugDataDumper debugDataDumper; public BeaconChainController( final ServiceConfig serviceConfig, final BeaconChainConfiguration beaconConfig) { @@ -318,6 +321,10 @@ public BeaconChainController( this.receivedBlockEventsChannelPublisher = eventChannels.getPublisher(ReceivedBlockEventsChannel.class); this.forkChoiceExecutor = new AsyncRunnerEventThread("forkchoice", asyncRunnerFactory); + this.debugDataDumper = + beaconConfig.p2pConfig().isP2pDumpsToFileEnabled() + ? new DebugDataDumper(serviceConfig.getDataDirLayout().getDebugDataDirectory()) + : new NoOpDebugDataDumper(); this.futureItemsMetric = SettableLabelledGauge.create( metricsSystem, @@ -791,6 +798,7 @@ protected void initForkChoice() { new TickProcessor(spec, recentChainData), new MergeTransitionBlockValidator(spec, recentChainData, executionLayer), beaconConfig.eth2NetworkConfig().isForkChoiceLateBlockReorgEnabled(), + debugDataDumper, metricsSystem); forkChoiceTrigger = new ForkChoiceTrigger(forkChoice); } @@ -1106,6 +1114,7 @@ protected void initP2PNetwork() { .specProvider(spec) .kzg(kzg) .recordMessageArrival(true) + .debugDataDumper(debugDataDumper) .build(); syncCommitteeMessagePool.subscribeOperationAdded( diff --git a/teku/src/main/java/tech/pegasys/teku/cli/options/P2POptions.java b/teku/src/main/java/tech/pegasys/teku/cli/options/P2POptions.java index 3af1824c318..f7d5329a260 100644 --- a/teku/src/main/java/tech/pegasys/teku/cli/options/P2POptions.java +++ b/teku/src/main/java/tech/pegasys/teku/cli/options/P2POptions.java @@ -284,6 +284,17 @@ public class P2POptions { hidden = true) private int batchVerifyMaxBatchSize = P2PConfig.DEFAULT_BATCH_VERIFY_MAX_BATCH_SIZE; + @Option( + names = {"--Xp2p-dumps-to-file-enabled"}, + paramLabel = "", + showDefaultValue = Visibility.ALWAYS, + description = + "Save objects to file that cause problems when processing, for example rejected blocks or invalid gossip.", + hidden = true, + arity = "0..1", + fallbackValue = "true") + private boolean p2pDumpsToFileEnabled = P2PConfig.DEFAULT_P2P_DUMPS_TO_FILE_ENABLED; + @Option( names = {"--Xp2p-batch-verify-signatures-strict-thread-limit-enabled"}, paramLabel = "", @@ -350,7 +361,8 @@ public void configure(final TekuConfiguration.Builder builder) { .isGossipScoringEnabled(gossipScoringEnabled) .peerRateLimit(peerRateLimit) .allTopicsFilterEnabled(allTopicsFilterEnabled) - .peerRequestLimit(peerRequestLimit)) + .peerRequestLimit(peerRequestLimit) + .p2pDumpsToFileEnabled(p2pDumpsToFileEnabled)) .discovery( d -> { if (p2pDiscoveryBootnodes != null) { diff --git a/teku/src/test/java/tech/pegasys/teku/cli/options/P2POptionsTest.java b/teku/src/test/java/tech/pegasys/teku/cli/options/P2POptionsTest.java index 9f7c3aca6c4..3f1e71b2ea3 100644 --- a/teku/src/test/java/tech/pegasys/teku/cli/options/P2POptionsTest.java +++ b/teku/src/test/java/tech/pegasys/teku/cli/options/P2POptionsTest.java @@ -37,6 +37,7 @@ public void shouldReadFromConfigurationFile() { assertThat(p2pConfig.getTargetSubnetSubscriberCount()).isEqualTo(5); assertThat(p2pConfig.getPeerRateLimit()).isEqualTo(100); assertThat(p2pConfig.getPeerRequestLimit()).isEqualTo(101); + assertThat(p2pConfig.isP2pDumpsToFileEnabled()).isFalse(); final DiscoveryConfig discoConfig = tekuConfig.discovery(); assertThat(discoConfig.isDiscoveryEnabled()).isTrue(); @@ -139,6 +140,26 @@ void advertisedUdpPort_shouldDefaultToTcpListenPortWhenNeitherSet() { .isEqualTo(tekuConfig.network().getAdvertisedPort()); } + @Test + void p2pDumpsToFileEnabled_shouldDefaultFalse() { + final TekuConfiguration tekuConfig = getTekuConfigurationFromArguments(); + assertThat(tekuConfig.p2p().isP2pDumpsToFileEnabled()).isEqualTo(false); + } + + @Test + void p2pDumpsToFileEnabled_shouldTrue() { + final TekuConfiguration tekuConfig = + getTekuConfigurationFromArguments("--Xp2p-dumps-to-file-enabled=true"); + assertThat(tekuConfig.p2p().isP2pDumpsToFileEnabled()).isEqualTo(true); + } + + @Test + void p2pDumpsToFileEnabled_shouldNotRequireAValue() { + final TekuConfiguration tekuConfig = + getTekuConfigurationFromArguments("--Xp2p-dumps-to-file-enabled"); + assertThat(tekuConfig.p2p().isP2pDumpsToFileEnabled()).isEqualTo(true); + } + @Test void advertisedUdpPort_shouldDefaultToTcpListenPortWhenListenPortSet() { TekuConfiguration tekuConfiguration = getTekuConfigurationFromArguments("--p2p-port=8000");