diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/SwirldsPlatform.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/SwirldsPlatform.java index c75dfd52adc9..080fb5cedd70 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/SwirldsPlatform.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/SwirldsPlatform.java @@ -287,6 +287,8 @@ public class SwirldsPlatform implements Platform { /** thread-queue responsible for hashing states */ private final QueueThread stateHashSignQueue; + private final AncientMode ancientMode; + /** * the browser gives the Platform what app to run. There can be multiple Platforms on one computer. * @@ -314,6 +316,11 @@ public class SwirldsPlatform implements Platform { this.platformContext = Objects.requireNonNull(platformContext, "platformContext"); + ancientMode = platformContext + .getConfiguration() + .getConfigData(EventConfig.class) + .getAncientMode(); + this.emergencyRecoveryManager = Objects.requireNonNull(emergencyRecoveryManager, "emergencyRecoveryManager"); final Time time = Time.getCurrent(); @@ -403,12 +410,6 @@ public class SwirldsPlatform implements Platform { // When we perform the migration to using birth round bounding, we will need to read // the old type and start writing the new type. - final AncientMode currentFileType = platformContext - .getConfiguration() - .getConfigData(EventConfig.class) - .useBirthRoundAncientThreshold() - ? AncientMode.BIRTH_ROUND_THRESHOLD - : AncientMode.GENERATION_THRESHOLD; initialPcesFiles = PcesFileReader.readFilesFromDisk( platformContext, @@ -416,7 +417,7 @@ public class SwirldsPlatform implements Platform { databaseDirectory, initialState.getRound(), preconsensusEventStreamConfig.permitGaps(), - currentFileType); + ancientMode); preconsensusEventFileManager = new PcesFileManager(platformContext, initialPcesFiles, selfId, initialState.getRound()); @@ -683,7 +684,6 @@ public class SwirldsPlatform implements Platform { epochHash, shadowGraph, emergencyRecoveryManager, - consensusRef, platformWiring.getGossipEventInput()::put, platformWiring.getIntakeQueueSizeSupplier(), swirldStateManager, @@ -699,10 +699,7 @@ public class SwirldsPlatform implements Platform { platformContext.getConfiguration().getConfigData(ConsensusConfig.class), consensusMetrics, getAddressBook(), - platformContext - .getConfiguration() - .getConfigData(EventConfig.class) - .getAncientMode())); + ancientMode)); if (startedFromGenesis) { initialMinimumGenerationNonAncient = 0; @@ -845,7 +842,15 @@ private void loadStateIntoConsensus(@NonNull final SignedState signedState) { Objects.requireNonNull(signedState); consensusRef.get().loadFromSignedState(signedState); - shadowGraph.startWithExpiredThreshold(consensusRef.get().getMinGenerationNonAncient()); + + // FUTURE WORK: this needs to be updated for birth round compatibility. + final NonAncientEventWindow eventWindow = new NonAncientEventWindow( + signedState.getRound(), + signedState.getState().getPlatformState().getMinimumGenerationNonAncient(), + signedState.getState().getPlatformState().getMinimumGenerationNonAncient(), + ancientMode); + + shadowGraph.startWithEventWindow(eventWindow); gossip.loadFromSignedState(signedState); } @@ -912,7 +917,7 @@ private void loadReconnectState(final SignedState signedState) { signedState.getRound(), signedState.getMinRoundGeneration(), signedState.getMinRoundGeneration(), - AncientMode.getAncientMode(platformContext))); + ancientMode)); platformWiring.updateRunningHash(new RunningEventHashUpdate(signedState.getHashEventsCons(), true)); platformWiring.getPcesWriterRegisterDiscontinuityInput().inject(signedState.getRound()); diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/consensus/NonAncientEventWindow.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/consensus/NonAncientEventWindow.java index 36459976a92a..296903c5b961 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/consensus/NonAncientEventWindow.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/consensus/NonAncientEventWindow.java @@ -116,12 +116,21 @@ public AncientMode getAncientMode() { return ancientMode; } + /** + * The round that has come to consensus most recently. + * + * @return the latest round that has come to consensus + */ + public long getLatestConsensusRound() { + return latestConsensusRound; + } + /** * The round that will come to consensus next. * * @return the pending round coming to consensus, i.e. 1 + the latestConsensusRound */ - public long pendingConsensusRound() { + public long getPendingConsensusRound() { return latestConsensusRound + 1; } diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/FutureEventBuffer.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/FutureEventBuffer.java index b6b8cd769ad6..52fb513ad127 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/FutureEventBuffer.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/FutureEventBuffer.java @@ -89,7 +89,7 @@ public List addEvent(@NonNull final GossipEvent event) { if (eventWindow.isAncient(event)) { // we can safely ignore ancient events return null; - } else if (event.getHashedData().getBirthRound() <= eventWindow.pendingConsensusRound()) { + } else if (event.getHashedData().getBirthRound() <= eventWindow.getPendingConsensusRound()) { // this is not a future event, no need to buffer it return List.of(event); } @@ -113,7 +113,7 @@ public List updateEventWindow(@NonNull final NonAncientEventWindow this.eventWindow = Objects.requireNonNull(eventWindow); final List events = new ArrayList<>(); - futureEvents.shiftWindow(eventWindow.pendingConsensusRound(), (round, roundEvents) -> { + futureEvents.shiftWindow(eventWindow.getPendingConsensusRound(), (round, roundEvents) -> { for (final GossipEvent event : roundEvents) { if (!eventWindow.isAncient(event)) { events.add(event); diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/creation/tipset/TipsetEventCreator.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/creation/tipset/TipsetEventCreator.java index 206e1f396ebb..ccc76cb828e9 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/creation/tipset/TipsetEventCreator.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/creation/tipset/TipsetEventCreator.java @@ -438,7 +438,7 @@ private GossipEvent assembleEventObject(@Nullable final EventDescriptor otherPar lastSelfEvent, otherParent == null ? Collections.emptyList() : Collections.singletonList(otherParent), nonAncientEventWindow.getAncientMode() == AncientMode.BIRTH_ROUND_THRESHOLD - ? nonAncientEventWindow.pendingConsensusRound() + ? nonAncientEventWindow.getPendingConsensusRound() : ConsensusConstants.ROUND_FIRST, timeCreated, transactionSupplier.getTransactions()); diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/validation/InternalEventValidator.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/validation/InternalEventValidator.java index 0da957d37458..eb28e453c64f 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/validation/InternalEventValidator.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/event/validation/InternalEventValidator.java @@ -20,7 +20,7 @@ import static com.swirlds.metrics.api.Metrics.PLATFORM_CATEGORY; import static com.swirlds.platform.consensus.ConsensusConstants.ROUND_FIRST; import static com.swirlds.platform.consensus.ConsensusConstants.ROUND_NEGATIVE_INFINITY; -import static com.swirlds.platform.consensus.GraphGenerations.FIRST_GENERATION; +import static com.swirlds.platform.system.events.EventConstants.FIRST_GENERATION; import com.swirlds.base.time.Time; import com.swirlds.common.context.PlatformContext; diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/GossipFactory.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/GossipFactory.java index 3869c8c4dcb1..07f070083a44 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/GossipFactory.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/GossipFactory.java @@ -24,7 +24,6 @@ import com.swirlds.common.notification.NotificationEngine; import com.swirlds.common.platform.NodeId; import com.swirlds.common.threading.manager.ThreadManager; -import com.swirlds.platform.Consensus; import com.swirlds.platform.crypto.KeysAndCerts; import com.swirlds.platform.event.GossipEvent; import com.swirlds.platform.gossip.shadowgraph.Shadowgraph; @@ -42,7 +41,6 @@ import edu.umd.cs.findbugs.annotations.NonNull; import edu.umd.cs.findbugs.annotations.Nullable; import java.util.Objects; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.LongSupplier; import java.util.function.Supplier; @@ -72,7 +70,6 @@ private GossipFactory() {} * @param epochHash the epoch hash of the initial state * @param shadowGraph contains non-ancient events * @param emergencyRecoveryManager handles emergency recovery - * @param consensusRef a pointer to consensus * @param receivedEventHandler handles events received from other nodes * @param intakeQueueSizeSupplier a supplier for the size of the event intake queue * @param swirldStateManager manages the mutable state @@ -97,7 +94,6 @@ public static Gossip buildGossip( @Nullable final Hash epochHash, @NonNull final Shadowgraph shadowGraph, @NonNull final EmergencyRecoveryManager emergencyRecoveryManager, - @NonNull final AtomicReference consensusRef, @NonNull final Consumer receivedEventHandler, @NonNull final LongSupplier intakeQueueSizeSupplier, @NonNull final SwirldStateManager swirldStateManager, @@ -119,7 +115,6 @@ public static Gossip buildGossip( Objects.requireNonNull(appVersion); Objects.requireNonNull(shadowGraph); Objects.requireNonNull(emergencyRecoveryManager); - Objects.requireNonNull(consensusRef); Objects.requireNonNull(receivedEventHandler); Objects.requireNonNull(intakeQueueSizeSupplier); Objects.requireNonNull(swirldStateManager); @@ -160,7 +155,6 @@ public static Gossip buildGossip( epochHash, shadowGraph, emergencyRecoveryManager, - consensusRef, receivedEventHandler, intakeQueueSizeSupplier, swirldStateManager, diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/ReservedEventWindow.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/ReservedEventWindow.java new file mode 100644 index 000000000000..350fe8d8b669 --- /dev/null +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/ReservedEventWindow.java @@ -0,0 +1,71 @@ +/* + * Copyright (C) 2024 Hedera Hashgraph, LLC + * + * 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 com.swirlds.platform.gossip.shadowgraph; + +import com.swirlds.platform.consensus.NonAncientEventWindow; +import edu.umd.cs.findbugs.annotations.NonNull; +import java.util.Objects; + +/** + * An event window that has been reserved in the shadowgraph. While this reservation is held, the shadowgraph will not + * unlink any events that are non-expired with respect to the reserved window. + */ +public class ReservedEventWindow implements AutoCloseable { + private final NonAncientEventWindow eventWindow; + private final ShadowgraphReservation shadowgraphReservation; + + /** + * Constructor. + * + * @param eventWindow the event window + * @param shadowgraphReservation the shadowgraph reservation + */ + public ReservedEventWindow( + @NonNull final NonAncientEventWindow eventWindow, + @NonNull final ShadowgraphReservation shadowgraphReservation) { + this.eventWindow = Objects.requireNonNull(eventWindow); + this.shadowgraphReservation = Objects.requireNonNull(shadowgraphReservation); + } + + /** + * {@inheritDoc} + */ + @Override + public void close() { + shadowgraphReservation.close(); + } + + /** + * Get the reserved event window. No events will be unlinked that are non-expired with respect to this window, as + * long as this reservation is held. + * + * @return the reserved event window + */ + @NonNull + public NonAncientEventWindow getEventWindow() { + return eventWindow; + } + + /** + * Get the number of reservations in the underlying shadowgraph reservation. Exposed for unit tests. + * + * @return the number of reservations + */ + public long getReservationCount() { + return shadowgraphReservation.getReservationCount(); + } +} diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/Shadowgraph.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/Shadowgraph.java index 38c0a275dd2d..c878364a519c 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/Shadowgraph.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/Shadowgraph.java @@ -77,12 +77,6 @@ public class Shadowgraph implements Clearable { */ private final HashSet tips; - /** - * The indicator describing the expiration boundary. All events with an older (smaller) ancient indicator should be - * expired, when possible. - */ - private long expireBelow; - /** * The oldest ancient indicator that has not yet been expired */ @@ -108,6 +102,11 @@ public class Shadowgraph implements Clearable { */ private final AncientMode ancientMode; + /** + * The most recent event window we know about. + */ + private NonAncientEventWindow eventWindow; + /** * Constructor. * @@ -123,7 +122,7 @@ public Shadowgraph(@NonNull final PlatformContext platformContext, @NonNull fina this.metrics = new ShadowgraphMetrics(platformContext); this.numberOfNodes = addressBook.getSize(); - expireBelow = ancientMode.getGenesisIndicator(); + eventWindow = NonAncientEventWindow.getGenesisNonAncientEventWindow(ancientMode); oldestUnexpiredIndicator = ancientMode.getGenesisIndicator(); tips = new HashSet<>(); hashToShadowEvent = new HashMap<>(); @@ -132,21 +131,24 @@ public Shadowgraph(@NonNull final PlatformContext platformContext, @NonNull fina } /** - * Define the starting non-expired threshold for the shadowgraph, it will not keep any events older than this + * Define the starting event window for the shadowgraph * - * @param expiredEventThreshold the starting non-expired threshold - */ - public synchronized void startWithExpiredThreshold(final long expiredEventThreshold) { - expireBelow = expiredEventThreshold; - oldestUnexpiredIndicator = expiredEventThreshold; - logger.info(STARTUP.getMarker(), "Shadowgraph starting from expiration threshold {}", expiredEventThreshold); + * @param eventWindow the starting event window + */ + public synchronized void startWithEventWindow(@NonNull final NonAncientEventWindow eventWindow) { + this.eventWindow = eventWindow; + oldestUnexpiredIndicator = eventWindow.getExpiredThreshold(); + logger.info( + STARTUP.getMarker(), + "Shadowgraph starting from expiration threshold {}", + eventWindow.getExpiredThreshold()); } /** * Reset the shadowgraph manager to its constructed state. */ public synchronized void clear() { - expireBelow = ancientMode.getGenesisIndicator(); + eventWindow = NonAncientEventWindow.getGenesisNonAncientEventWindow(ancientMode); oldestUnexpiredIndicator = ancientMode.getGenesisIndicator(); disconnectShadowEvents(); tips.clear(); @@ -171,19 +173,45 @@ private void disconnectShadowEvents() { * * @return the reservation instance, must be closed when the reservation is no longer needed */ - public synchronized ShadowgraphReservation reserve() { + @NonNull + public synchronized ReservedEventWindow reserve() { if (reservationList.isEmpty()) { - return newReservation(); + // If we are not currently holding any reservations, we need to create a new one. + return new ReservedEventWindow(eventWindow, newReservation()); } + + // Check to see if an existing reservation is good enough. + final ShadowgraphReservation lastReservation = reservationList.getLast(); - if (lastReservation.getReservedIndicator() == expireBelow) { + + final long previouslyReservedThreshold = lastReservation.getReservedThreshold(); + final long thresholdWeWantToReserve = eventWindow.getExpiredThreshold(); + + if (previouslyReservedThreshold == thresholdWeWantToReserve) { + + // The latest reservation is against the same expired threshold that we currently want to reserve. + // We can reuse that reservation instead of creating a new one. We still need to package that + // reservation with the most recent eventWindow we know about. + lastReservation.incrementReservations(); - return lastReservation; + return new ReservedEventWindow(eventWindow, lastReservation); } else { - return newReservation(); + + // We want a reservation on an expired threshold that isn't currently reserved. + // Create a new reservation. + + return new ReservedEventWindow(eventWindow, newReservation()); } } + /** + * Get the latest event window known to the shadowgraph. + */ + @NonNull + public synchronized NonAncientEventWindow getEventWindow() { + return eventWindow; + } + /** * Determines if the provided {@code hash} is in the shadowgraph. * @@ -313,32 +341,29 @@ public synchronized Collection findByAncientIndicator( * * @param eventWindow describes the current window of non-expired events */ - public synchronized void updateNonExpiredEventWindow(@NonNull final NonAncientEventWindow eventWindow) { - + public synchronized void updateEventWindow(@NonNull final NonAncientEventWindow eventWindow) { final long expiredThreshold = eventWindow.getExpiredThreshold(); - if (expiredThreshold < expireBelow) { + if (expiredThreshold < eventWindow.getExpiredThreshold()) { logger.error( EXCEPTION.getMarker(), "A request to expire below {} is less than request of {}. Ignoring expiration request", expiredThreshold, - expireBelow); + eventWindow.getExpiredThreshold()); // The value of expireBelow must never decrease, so if we receive an invalid request like this, ignore it return; } - - // Update the smallest threshold that should not be expired - expireBelow = expiredThreshold; + this.eventWindow = eventWindow; // Remove reservations for events that can and should be expired, and // keep track of the oldest threshold that can be expired long oldestReservedIndicator = pruneReservationList(); if (oldestReservedIndicator == NO_RESERVATION) { - oldestReservedIndicator = expireBelow; + oldestReservedIndicator = eventWindow.getExpiredThreshold(); } - metrics.updateIndicatorsWaitingForExpiry(expireBelow - oldestReservedIndicator); + metrics.updateIndicatorsWaitingForExpiry(eventWindow.getExpiredThreshold() - oldestReservedIndicator); // Expire events that can and should be expired, starting with the oldest non-expired ancient indicator // and working up until we reach an indicator that should not or cannot be expired. @@ -346,7 +371,7 @@ public synchronized void updateNonExpiredEventWindow(@NonNull final NonAncientEv // This process must be separate from iterating through the reservations because even if there are no // reservations, expiry should still function correctly. - final long minimumIndicatorToKeep = Math.min(expireBelow, oldestReservedIndicator); + final long minimumIndicatorToKeep = Math.min(eventWindow.getExpiredThreshold(), oldestReservedIndicator); while (oldestUnexpiredIndicator < minimumIndicatorToKeep) { final Set shadowsToExpire = indicatorToShadowEvent.remove(oldestUnexpiredIndicator); @@ -367,7 +392,6 @@ public synchronized void updateNonExpiredEventWindow(@NonNull final NonAncientEv * Removes reservations that can and should be expired, starting with the oldest ancient indicator reservation. * * @return the oldest ancient indicator with at least one reservation, or {@code -1} if there are no reservations - * @see Shadowgraph#expireBelow */ private long pruneReservationList() { long oldestReservedIndicator = NO_RESERVATION; @@ -377,13 +401,13 @@ private long pruneReservationList() { final Iterator iterator = reservationList.iterator(); while (iterator.hasNext()) { final ShadowgraphReservation reservation = iterator.next(); - final long reservedIndicator = reservation.getReservedIndicator(); + final long reservedIndicator = reservation.getReservedThreshold(); if (reservation.getReservationCount() > 0) { // As soon as we find a reserved indicator, stop iterating - oldestReservedIndicator = reservation.getReservedIndicator(); + oldestReservedIndicator = reservation.getReservedThreshold(); break; - } else if (reservedIndicator < expireBelow) { + } else if (reservedIndicator < eventWindow.getExpiredThreshold()) { // If the number of reservations is 0 and the // indicator should be expired, remove the reservation iterator.remove(); @@ -488,13 +512,13 @@ public synchronized boolean addEvent(final EventImpl e) throws ShadowgraphInsert logger.info( SYNC_INFO.getMarker(), "tips size is {} after adding {}. Esp null:{} Ssp null:{}\n" - + "expireBelow: {} oldestUnexpiredIndicator: {}\n" + + "eventWindow.getExpiredThreshold: {} oldestUnexpiredIndicator: {}\n" + "current tips:{}", tips::size, () -> EventStrings.toMediumString(e), () -> e.getSelfParent() == null, () -> s.getSelfParent() == null, - () -> expireBelow, + () -> eventWindow.getExpiredThreshold(), () -> oldestUnexpiredIndicator, () -> tips.stream() .map(sh -> EventStrings.toShortString(sh.getEvent())) @@ -524,7 +548,7 @@ public synchronized boolean addEvent(final EventImpl e) throws ShadowgraphInsert } private ShadowgraphReservation newReservation() { - final ShadowgraphReservation reservation = new ShadowgraphReservation(expireBelow); + final ShadowgraphReservation reservation = new ShadowgraphReservation(eventWindow.getExpiredThreshold()); reservationList.addLast(reservation); return reservation; } diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/ShadowgraphReservation.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/ShadowgraphReservation.java index eaa4d3c9c489..ce8debc5d6ba 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/ShadowgraphReservation.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/ShadowgraphReservation.java @@ -27,17 +27,24 @@ public final class ShadowgraphReservation implements AutoCloseable { /** - * The event ancient indicator that is reserved + * The threshold that is being reserved. No event with an ancient indicator greater than or equal to this value may + * be expired. */ - private final long reservedIndicator; + private final long reservedThreshold; /** * The number of reservations on this ancient indicator. */ private final AtomicInteger reservationCount; - public ShadowgraphReservation(final long reservedIndicator) { - this.reservedIndicator = reservedIndicator; + /** + * Constructor. + * + * @param reservedThreshold the ancient indicator that is being reserved, no event with an ancient indicator greater + * than or equal to this value may be expired + */ + public ShadowgraphReservation(final long reservedThreshold) { + this.reservedThreshold = reservedThreshold; reservationCount = new AtomicInteger(1); } @@ -71,7 +78,7 @@ public void close() { * * @return the ancient indicator that is reserved */ - public long getReservedIndicator() { - return reservedIndicator; + public long getReservedThreshold() { + return reservedThreshold; } } diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/ShadowgraphSynchronizer.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/ShadowgraphSynchronizer.java index fcea83824826..52b35766c691 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/ShadowgraphSynchronizer.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/ShadowgraphSynchronizer.java @@ -22,20 +22,20 @@ import static com.swirlds.platform.gossip.shadowgraph.SyncUtils.getTheirTipsIHave; import static com.swirlds.platform.gossip.shadowgraph.SyncUtils.readEventsINeed; import static com.swirlds.platform.gossip.shadowgraph.SyncUtils.readMyTipsTheyHave; -import static com.swirlds.platform.gossip.shadowgraph.SyncUtils.readTheirTipsAndGenerations; +import static com.swirlds.platform.gossip.shadowgraph.SyncUtils.readTheirTipsAndEventWindow; import static com.swirlds.platform.gossip.shadowgraph.SyncUtils.sendEventsTheyNeed; -import static com.swirlds.platform.gossip.shadowgraph.SyncUtils.writeFirstByte; -import static com.swirlds.platform.gossip.shadowgraph.SyncUtils.writeMyTipsAndGenerations; +import static com.swirlds.platform.gossip.shadowgraph.SyncUtils.writeMyTipsAndEventWindow; import static com.swirlds.platform.gossip.shadowgraph.SyncUtils.writeTheirTipsIHave; import com.swirlds.base.time.Time; import com.swirlds.common.context.PlatformContext; import com.swirlds.common.platform.NodeId; -import com.swirlds.common.threading.interrupt.InterruptableRunnable; import com.swirlds.common.threading.pool.ParallelExecutionException; import com.swirlds.common.threading.pool.ParallelExecutor; -import com.swirlds.platform.consensus.GraphGenerations; +import com.swirlds.platform.consensus.NonAncientEventWindow; +import com.swirlds.platform.event.AncientMode; import com.swirlds.platform.event.GossipEvent; +import com.swirlds.platform.eventhandling.EventConfig; import com.swirlds.platform.gossip.FallenBehindManager; import com.swirlds.platform.gossip.IntakeEventCounter; import com.swirlds.platform.gossip.SyncException; @@ -57,7 +57,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import java.util.function.Predicate; -import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -89,12 +88,6 @@ public class ShadowgraphSynchronizer { */ private final SyncMetrics syncMetrics; - /** - * provides the current consensus instance, a supplier is used because this instance will change after a reconnect, - * so we have to make sure we always get the latest one - */ - private final Supplier generationsSupplier; - /** * consumes events received by the peer */ @@ -116,16 +109,6 @@ public class ShadowgraphSynchronizer { */ private final ParallelExecutor executor; - /** - * if set to true, send and receive initial negotiation bytes at the start of the sync - */ - private final boolean sendRecInitBytes; - - /** - * executed before fetching the tips from the shadowgraph for the second time in phase 3 - */ - private final InterruptableRunnable executePreFetchTips; - private final Time time; /** @@ -141,18 +124,32 @@ public class ShadowgraphSynchronizer { */ private final Duration nonAncestorFilterThreshold; + /** + * The current ancient mode. + */ + private final AncientMode ancientMode; + + /** + * Constructs a new ShadowgraphSynchronizer. + * + * @param platformContext the platform context + * @param shadowGraph stores events to sync + * @param numberOfNodes number of nodes in the network + * @param syncMetrics metrics for sync + * @param receivedEventHandler events that are received are passed here + * @param fallenBehindManager tracks if we have fallen behind + * @param intakeEventCounter used for tracking events in the intake pipeline per peer + * @param executor for executing read/write tasks in parallel + */ public ShadowgraphSynchronizer( @NonNull final PlatformContext platformContext, @NonNull final Shadowgraph shadowGraph, final int numberOfNodes, @NonNull final SyncMetrics syncMetrics, - @NonNull final Supplier generationsSupplier, @NonNull final Consumer receivedEventHandler, @NonNull final FallenBehindManager fallenBehindManager, @NonNull final IntakeEventCounter intakeEventCounter, - @NonNull final ParallelExecutor executor, - final boolean sendRecInitBytes, - @NonNull final InterruptableRunnable executePreFetchTips) { + @NonNull final ParallelExecutor executor) { Objects.requireNonNull(platformContext); @@ -160,18 +157,20 @@ public ShadowgraphSynchronizer( this.shadowGraph = Objects.requireNonNull(shadowGraph); this.numberOfNodes = numberOfNodes; this.syncMetrics = Objects.requireNonNull(syncMetrics); - this.generationsSupplier = Objects.requireNonNull(generationsSupplier); this.fallenBehindManager = Objects.requireNonNull(fallenBehindManager); this.intakeEventCounter = Objects.requireNonNull(intakeEventCounter); this.executor = Objects.requireNonNull(executor); - this.sendRecInitBytes = sendRecInitBytes; - this.executePreFetchTips = Objects.requireNonNull(executePreFetchTips); this.eventHandler = Objects.requireNonNull(receivedEventHandler); final SyncConfig syncConfig = platformContext.getConfiguration().getConfigData(SyncConfig.class); this.nonAncestorFilterThreshold = syncConfig.nonAncestorFilterThreshold(); this.filterLikelyDuplicates = syncConfig.filterLikelyDuplicates(); + + this.ancientMode = platformContext + .getConfiguration() + .getConfigData(EventConfig.class) + .getAncientMode(); } /** @@ -206,36 +205,26 @@ private boolean reserveSynchronize( // reporting and performance analysis final SyncTiming timing = new SyncTiming(); final List sendList; - try (final ShadowgraphReservation reservation = shadowGraph.reserve()) { + try (final ReservedEventWindow reservation = shadowGraph.reserve()) { connection.initForSync(); timing.start(); - if (sendRecInitBytes) { - writeFirstByte(connection); - } + // Step 1: each peer tells the other about its tips and event windows - // Step 1: each peer tells the other about its tips and generations + final NonAncientEventWindow myWindow = reservation.getEventWindow(); - // the generation we reserved is our minimum round generation - // the ShadowGraph guarantees it won't be expired until we release it - final Generations myGenerations = getGenerations(reservation.getReservedIndicator()); final List myTips = getTips(); - // READ and WRITE generation numbers & tip hashes - final TheirTipsAndGenerations theirTipsAndGenerations = readWriteParallel( - readTheirTipsAndGenerations(connection, numberOfNodes, sendRecInitBytes), - writeMyTipsAndGenerations(connection, myGenerations, myTips), + // READ and WRITE event windows numbers & tip hashes + final TheirTipsAndEventWindow theirTipsAndEventWindow = readWriteParallel( + readTheirTipsAndEventWindow(connection, numberOfNodes, ancientMode), + writeMyTipsAndEventWindow(connection, myWindow, myTips), connection); timing.setTimePoint(1); - if (theirTipsAndGenerations.isSyncRejected()) { - logger.info(SYNC_INFO.getMarker(), "{} sync rejected by other", connection.getDescription()); - return false; - } + syncMetrics.eventWindow(myWindow, theirTipsAndEventWindow.eventWindow()); - syncMetrics.generations(myGenerations, theirTipsAndGenerations.getGenerations()); - - if (fallenBehind(myGenerations, theirTipsAndGenerations.getGenerations(), connection)) { + if (fallenBehind(myWindow, theirTipsAndEventWindow.eventWindow(), connection)) { // aborting the sync since someone has fallen behind return false; } @@ -244,7 +233,7 @@ private boolean reserveSynchronize( final Set eventsTheyHave = new HashSet<>(); // process the hashes received - final List theirTips = shadowGraph.shadows(theirTipsAndGenerations.getTips()); + final List theirTips = shadowGraph.shadows(theirTipsAndEventWindow.tips()); // For each tip they send us, determine if we have that event. // For each tip, send true if we have the event and false if we don't. @@ -268,7 +257,7 @@ private boolean reserveSynchronize( // create a send list based on the known set sendList = createSendList( - connection.getSelfId(), eventsTheyHave, myGenerations, theirTipsAndGenerations.getGenerations()); + connection.getSelfId(), eventsTheyHave, myWindow, theirTipsAndEventWindow.eventWindow()); } final SyncConfig syncConfig = platformContext.getConfiguration().getConfigData(SyncConfig.class); @@ -277,14 +266,6 @@ private boolean reserveSynchronize( connection, timing, sendList, syncConfig.syncKeepalivePeriod(), syncConfig.maxSyncTime()); } - @NonNull - private Generations getGenerations(final long minRoundGen) { - return new Generations( - minRoundGen, - generationsSupplier.get().getMinGenerationNonAncient(), - generationsSupplier.get().getMaxRoundGeneration()); - } - @NonNull private List getTips() { final List myTips = shadowGraph.getTips(); @@ -296,13 +277,15 @@ private List getTips() { /** * Decide if we have fallen behind with respect to this peer. * - * @param self our generations - * @param other their generations + * @param self our event window + * @param other their event window * @param connection the connection to use * @return true if we have fallen behind, false otherwise */ private boolean fallenBehind( - @NonNull final Generations self, @NonNull final Generations other, @NonNull final Connection connection) { + @NonNull final NonAncientEventWindow self, + @NonNull final NonAncientEventWindow other, + @NonNull final Connection connection) { Objects.requireNonNull(self); Objects.requireNonNull(other); Objects.requireNonNull(connection); @@ -325,26 +308,25 @@ private boolean fallenBehind( * @param selfId the id of this node * @param knownSet the set of events that the peer already has (this is incomplete at this stage and is * added to during this method) - * @param myGenerations the generations of this node - * @param theirGenerations the generations of the peer + * @param myEventWindow the event window of this node + * @param theirEventWindow the event window of the peer * @return a list of events to send to the peer */ @NonNull private List createSendList( @NonNull final NodeId selfId, @NonNull final Set knownSet, - @NonNull final Generations myGenerations, - @NonNull final Generations theirGenerations) - throws InterruptedException { + @NonNull final NonAncientEventWindow myEventWindow, + @NonNull final NonAncientEventWindow theirEventWindow) { Objects.requireNonNull(selfId); Objects.requireNonNull(knownSet); - Objects.requireNonNull(myGenerations); - Objects.requireNonNull(theirGenerations); + Objects.requireNonNull(myEventWindow); + Objects.requireNonNull(theirEventWindow); // add to knownSet all the ancestors of each known event final Set knownAncestors = shadowGraph.findAncestors( - knownSet, SyncUtils.unknownNonAncient(knownSet, myGenerations, theirGenerations)); + knownSet, SyncUtils.unknownNonAncient(knownSet, myEventWindow, theirEventWindow, ancientMode)); // since knownAncestors is a lot bigger than knownSet, it is a lot cheaper to add knownSet to knownAncestors // then vice versa @@ -354,9 +336,8 @@ private List createSendList( // predicate used to search for events to send final Predicate knownAncestorsPredicate = - SyncUtils.unknownNonAncient(knownAncestors, myGenerations, theirGenerations); + SyncUtils.unknownNonAncient(knownAncestors, myEventWindow, theirEventWindow, ancientMode); - executePreFetchTips.run(); // in order to get the peer the latest events, we get a new set of tips to search from final List myNewTips = shadowGraph.getTips(); @@ -472,20 +453,4 @@ private T readWriteParallel( return executor.doParallel(readTask, writeTask, connection::disconnect); } - - /** - * Reject a sync - * - * @param connection the connection over which the sync was initiated - * @throws IOException if there are any connection issues - */ - public void rejectSync(@NonNull final Connection connection) throws IOException { - Objects.requireNonNull(connection); - try { - connection.initForSync(); - SyncUtils.rejectSync(connection, numberOfNodes); - } finally { - logger.info(SYNC_INFO.getMarker(), "{} sync rejected by self", connection.getDescription()); - } - } } diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/SyncFallenBehindStatus.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/SyncFallenBehindStatus.java index 3ae92eed22bc..c70080c86197 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/SyncFallenBehindStatus.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/SyncFallenBehindStatus.java @@ -16,18 +16,28 @@ package com.swirlds.platform.gossip.shadowgraph; -import com.swirlds.platform.consensus.GraphGenerations; +import com.swirlds.platform.consensus.NonAncientEventWindow; +import edu.umd.cs.findbugs.annotations.NonNull; public enum SyncFallenBehindStatus { NONE_FALLEN_BEHIND, SELF_FALLEN_BEHIND, OTHER_FALLEN_BEHIND; - public static SyncFallenBehindStatus getStatus(final GraphGenerations self, final GraphGenerations other) { - if (other.getMinGenerationNonAncient() < self.getMinRoundGeneration()) { + /** + * Compute the fallen behind status between ourselves and a peer. + * + * @param self our event window + * @param other the peer's event window + * @return the status + */ + @NonNull + public static SyncFallenBehindStatus getStatus( + @NonNull final NonAncientEventWindow self, @NonNull final NonAncientEventWindow other) { + if (other.getAncientThreshold() < self.getExpiredThreshold()) { return OTHER_FALLEN_BEHIND; } - if (self.getMinGenerationNonAncient() < other.getMinRoundGeneration()) { + if (self.getAncientThreshold() < other.getExpiredThreshold()) { return SELF_FALLEN_BEHIND; } return NONE_FALLEN_BEHIND; diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/SyncUtils.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/SyncUtils.java index 48769df91e25..b233f81c9839 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/SyncUtils.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/SyncUtils.java @@ -20,8 +20,11 @@ import static com.swirlds.logging.legacy.LogMarker.SYNC_INFO; import com.swirlds.common.crypto.Hash; +import com.swirlds.common.io.streams.SerializableDataInputStream; +import com.swirlds.common.io.streams.SerializableDataOutputStream; import com.swirlds.common.platform.NodeId; -import com.swirlds.platform.consensus.GraphGenerations; +import com.swirlds.platform.consensus.NonAncientEventWindow; +import com.swirlds.platform.event.AncientMode; import com.swirlds.platform.event.GossipEvent; import com.swirlds.platform.gossip.IntakeEventCounter; import com.swirlds.platform.gossip.SyncException; @@ -31,11 +34,9 @@ import com.swirlds.platform.network.Connection; import com.swirlds.platform.system.events.EventDescriptor; import edu.umd.cs.findbugs.annotations.NonNull; -import edu.umd.cs.findbugs.annotations.Nullable; import java.io.IOException; import java.time.Duration; import java.time.Instant; -import java.time.temporal.ChronoUnit; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -67,48 +68,32 @@ public final class SyncUtils { */ private SyncUtils() {} - public static void writeFirstByte(final Connection conn) throws IOException { - if (conn.isOutbound()) { // caller WRITE sync request - // try to initiate a sync - conn.getDos().requestSync(); - } else { // listener WRITE sync request response - conn.getDos().acceptSync(); - } - // no need to flush, since we will write more data right after - } - - public static void rejectSync(final Connection conn, final int numberOfNodes) throws IOException { - // respond with a nack - conn.getDos().rejectSync(); - conn.getDos().flush(); - - // read data and ignore since we rejected the sync - conn.getDis().readGenerations(); - conn.getDis().readTipHashes(numberOfNodes); - } - /** - * Write the tips and generations to the peer. This is the first data exchanged during a sync (after protocol - * negotiation). The complementary function to {@link #readTheirTipsAndGenerations(Connection, int, boolean)}. + * Send the tips and event window to the peer. This is the first data exchanged during a sync (after protocol + * negotiation). The complementary function to {@link #readTheirTipsAndEventWindow(Connection, int, AncientMode)}. * * @param connection the connection to write to - * @param generations the generations to write + * @param eventWindow the event window to write * @param tips the tips to write - * @return a {@link Callable} that writes the tips and generations + * @return a {@link Callable} that writes the tips and event window */ - public static Callable writeMyTipsAndGenerations( - final Connection connection, final Generations generations, final List tips) { + public static Callable writeMyTipsAndEventWindow( + @NonNull final Connection connection, + @NonNull final NonAncientEventWindow eventWindow, + @NonNull final List tips) { return () -> { final List tipHashes = tips.stream().map(ShadowEvent::getEventBaseHash).collect(Collectors.toList()); - connection.getDos().writeGenerations(generations); + + serializeEventWindow(connection.getDos(), eventWindow); + connection.getDos().writeTipHashes(tipHashes); connection.getDos().flush(); logger.info( SYNC_INFO.getMarker(), - "{} sent generations: {}", + "{} sent event window: {}", connection::getDescription, - generations::toString); + eventWindow::toString); logger.info( SYNC_INFO.getMarker(), "{} sent tips: {}", @@ -119,49 +104,34 @@ public static Callable writeMyTipsAndGenerations( } /** - * Read the tips and generations from the peer. This is the first data exchanged during a sync (after protocol - * negotiation). The complementary function to {@link #writeMyTipsAndGenerations(Connection, Generations, List)}. + * Read the tips and event window from the peer. This is the first data exchanged during a sync (after protocol + * negotiation). The complementary function to + * {@link #writeMyTipsAndEventWindow(Connection, NonAncientEventWindow, List)}. * * @param connection the connection to read from * @param numberOfNodes the number of nodes in the network - * @param readInitByte if true, read the first byte of the sync request response - * @return a {@link Callable} that reads the tips and generations + * @param ancientMode the current ancient mode + * @return a {@link Callable} that reads the tips and event window */ - public static Callable readTheirTipsAndGenerations( - final Connection connection, final int numberOfNodes, final boolean readInitByte) { + public static Callable readTheirTipsAndEventWindow( + final Connection connection, final int numberOfNodes, @NonNull final AncientMode ancientMode) { return () -> { - // Caller thread requested a sync, so now caller thread reads if its request was accepted. - if (connection.isOutbound() && readInitByte) { - try { - if (!connection.getDis().readSyncRequestResponse()) { - // sync rejected - return TheirTipsAndGenerations.syncRejected(); - } - } catch (final SyncException | IOException e) { - final Instant sentTime = connection.getDos().getRequestSentTime(); - final long inMs = sentTime == null ? -1 : sentTime.until(Instant.now(), ChronoUnit.MILLIS); - throw new SyncException( - connection, - "Problem while reading sync request response. Request was sent " + inMs + "ms ago", - e); - } - } + final NonAncientEventWindow eventWindow = deserializeEventWindow(connection.getDis(), ancientMode); - final Generations generations = connection.getDis().readGenerations(); final List tips = connection.getDis().readTipHashes(numberOfNodes); logger.info( SYNC_INFO.getMarker(), - "{} received generations: {}", + "{} received event window: {}", connection::getDescription, - generations::toString); + eventWindow::toString); logger.info( SYNC_INFO.getMarker(), "{} received tips: {}", connection::getDescription, () -> SyncLogging.toShortHashes(tips)); - return TheirTipsAndGenerations.create(generations, tips); + return new TheirTipsAndEventWindow(eventWindow, tips); }; } @@ -379,26 +349,6 @@ private static void checkEventExchangeTime(@NonNull final Duration maxSyncTime, } } - /** - * Get the latest self event in the shadowgraph. - * - * @param shadowGraph the shadow graph - * @param selfId the id of the node - * @return the latest self event in the shadowgraph, or null if there is none - */ - @Nullable - private static ShadowEvent getLatestSelfEventInShadowgraph( - @NonNull final Shadowgraph shadowGraph, @NonNull final NodeId selfId) { - - final List tips = shadowGraph.getTips(); - for (final ShadowEvent tip : tips) { - if (tip.getEvent().getCreatorId().equals(selfId)) { - return tip; - } - } - return null; - } - /** * Given a list of events we think the other node may not have, reduce that list to events that we think they do not * have and that are unlikely to end up being duplicate events. @@ -478,22 +428,34 @@ private static boolean haveWeKnownAboutEventForALongTime( } /** - * Returns a predicate that determines if a {@link ShadowEvent}'s generation is non-ancient for the peer and greater - * than this node's minimum non-expired generation, and is not already known. + * Returns a predicate that determines if a {@link ShadowEvent}'s ancient indicator is non-ancient for the peer and + * greater than this node's minimum non-expired threshold, and is not already known. * * @param knownShadows the {@link ShadowEvent}s that are already known and should therefore be rejected by the * predicate - * @param myGenerations the generations of this node - * @param theirGenerations the generations of the peer node + * @param myEventWindow the event window of this node + * @param theirEventWindow the event window of the peer node + * @param ancientMode the current ancient mode * @return the predicate */ + @NonNull public static Predicate unknownNonAncient( - final Collection knownShadows, - final GraphGenerations myGenerations, - final GraphGenerations theirGenerations) { - long minSearchGen = - Math.max(myGenerations.getMinRoundGeneration(), theirGenerations.getMinGenerationNonAncient()); - return s -> s.getEvent().getGeneration() >= minSearchGen && !knownShadows.contains(s); + @NonNull final Collection knownShadows, + @NonNull final NonAncientEventWindow myEventWindow, + @NonNull final NonAncientEventWindow theirEventWindow, + @NonNull final AncientMode ancientMode) { + + // When searching for events, we don't want to send any events that are known to be ancient to the peer. + // We should never be syncing with a peer if their ancient threshold is less than our expired threshold + // (if this is the case, then the peer is "behind"), so in practice the minimumSearchThreshold will always + // be the same as the peer's ancient threshold. However, in an abundance of caution, we use the maximum of + // the two thresholds to ensure that we don't ever attempt to traverse over events that are expired to us, + // since those events may be unlinked and could cause race conditions if accessed. + + final long minimumSearchThreshold = + Math.max(myEventWindow.getExpiredThreshold(), theirEventWindow.getAncientThreshold()); + return s -> s.getEvent().getBaseEvent().getAncientIndicator(ancientMode) >= minimumSearchThreshold + && !knownShadows.contains(s); } /** @@ -530,9 +492,13 @@ public static int computeMultiTipCount(Iterable tips) { } /** + * Performs a topological sort on the given list of events (i.e. where parents always come before their children). + * * @param sendList The list of events to sort. */ - static void sort(final List sendList) { + static void sort(@NonNull final List sendList) { + // Note: regardless of ancient mode, sorting uses generations and not birth rounds. + // Sorting by generations yields a list in topological order, sorting by birth rounds does not. sendList.sort((EventImpl e1, EventImpl e2) -> (int) (e1.getGeneration() - e2.getGeneration())); } @@ -589,4 +555,39 @@ static List getMyTipsTheyKnow( return knownTips; } + + /** + * Serialize an event window to the given output stream. + * + * @param out the output stream + * @param eventWindow the event window + */ + public static void serializeEventWindow( + @NonNull final SerializableDataOutputStream out, @NonNull final NonAncientEventWindow eventWindow) + throws IOException { + + out.writeLong(eventWindow.getLatestConsensusRound()); + out.writeLong(eventWindow.getAncientThreshold()); + out.writeLong(eventWindow.getExpiredThreshold()); + + // Intentionally don't bother writing ancient mode, the peer will always be using the same ancient mode as us + } + + /** + * Deserialize an event window from the given input stream. + * + * @param in the input stream + * @param ancientMode the currently configured ancient mode + * @return the deserialized event window + */ + @NonNull + public static NonAncientEventWindow deserializeEventWindow( + @NonNull final SerializableDataInputStream in, @NonNull final AncientMode ancientMode) throws IOException { + + final long latestConsensusRound = in.readLong(); + final long ancientThreshold = in.readLong(); + final long expiredThreshold = in.readLong(); + + return new NonAncientEventWindow(latestConsensusRound, ancientThreshold, expiredThreshold, ancientMode); + } } diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/TheirTipsAndEventWindow.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/TheirTipsAndEventWindow.java new file mode 100644 index 000000000000..5a40eaad68de --- /dev/null +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/TheirTipsAndEventWindow.java @@ -0,0 +1,28 @@ +/* + * Copyright (C) 2021-2024 Hedera Hashgraph, LLC + * + * 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 com.swirlds.platform.gossip.shadowgraph; + +import com.swirlds.common.crypto.Hash; +import com.swirlds.platform.consensus.NonAncientEventWindow; +import edu.umd.cs.findbugs.annotations.NonNull; +import java.util.List; + +/** + * The tips and event window of the sync peer. This is the first thing sent/received during a sync (after protocol + * negotiation). + */ +public record TheirTipsAndEventWindow(@NonNull NonAncientEventWindow eventWindow, @NonNull List tips) {} diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/TheirTipsAndGenerations.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/TheirTipsAndGenerations.java deleted file mode 100644 index cb2a000ad942..000000000000 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/shadowgraph/TheirTipsAndGenerations.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Copyright (C) 2021-2024 Hedera Hashgraph, LLC - * - * 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 com.swirlds.platform.gossip.shadowgraph; - -import com.swirlds.common.crypto.Hash; -import java.util.List; -import java.util.Objects; - -/** - * The tips and generations of the sync peer. This is the first thing sent/received during a sync (after protocol - * negotiation). - */ -public final class TheirTipsAndGenerations { - private static final TheirTipsAndGenerations SYNC_REJECTED_RESPONSE = new TheirTipsAndGenerations(null, null); - - private final Generations generations; - private final List tips; - - private TheirTipsAndGenerations(final Generations generations, final List tips) { - this.generations = generations; - this.tips = tips; - } - - public static TheirTipsAndGenerations create(final Generations generations, final List tips) { - Objects.requireNonNull(generations, "generations cannot be null"); - Objects.requireNonNull(tips, "tips cannot be null"); - return new TheirTipsAndGenerations(generations, tips); - } - - public static TheirTipsAndGenerations syncRejected() { - return SYNC_REJECTED_RESPONSE; - } - - public Generations getGenerations() { - return generations; - } - - public List getTips() { - return tips; - } - - public boolean isSyncRejected() { - return this == SYNC_REJECTED_RESPONSE; - } -} diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncGossip.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncGossip.java index ed42cb8bf954..c1610a03127a 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncGossip.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncGossip.java @@ -30,7 +30,6 @@ import com.swirlds.common.threading.manager.ThreadManager; import com.swirlds.common.threading.pool.CachedPoolParallelExecutor; import com.swirlds.common.threading.pool.ParallelExecutor; -import com.swirlds.platform.Consensus; import com.swirlds.platform.config.BasicConfig; import com.swirlds.platform.crypto.KeysAndCerts; import com.swirlds.platform.event.GossipEvent; @@ -69,7 +68,6 @@ import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.LongSupplier; import java.util.function.Supplier; @@ -109,7 +107,6 @@ public class SyncGossip extends AbstractGossip { * @param epochHash the epoch hash of the initial state * @param shadowGraph contains non-ancient events * @param emergencyRecoveryManager handles emergency recovery - * @param consensusRef a pointer to consensus * @param receivedEventHandler handles events received from other nodes * @param intakeQueueSizeSupplier a supplier for the size of the event intake queue * @param swirldStateManager manages the mutable state @@ -133,7 +130,6 @@ public SyncGossip( @Nullable final Hash epochHash, @NonNull final Shadowgraph shadowGraph, @NonNull final EmergencyRecoveryManager emergencyRecoveryManager, - @NonNull final AtomicReference consensusRef, @NonNull final Consumer receivedEventHandler, @NonNull final LongSupplier intakeQueueSizeSupplier, @NonNull final SwirldStateManager swirldStateManager, @@ -172,14 +168,10 @@ public SyncGossip( shadowGraph, addressBook.getSize(), syncMetrics, - consensusRef::get, receivedEventHandler, syncManager, intakeEventCounter, - shadowgraphExecutor, - // don't send or receive init bytes if running sync as a protocol. the negotiator handles this - false, - () -> {}); + shadowgraphExecutor); final ReconnectConfig reconnectConfig = platformContext.getConfiguration().getConfigData(ReconnectConfig.class); diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncInputStream.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncInputStream.java index e819b5d7c4f3..15d9cb497408 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncInputStream.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncInputStream.java @@ -23,9 +23,6 @@ import com.swirlds.common.io.extendable.extensions.CountingStreamExtension; import com.swirlds.common.io.streams.SerializableDataInputStream; import com.swirlds.platform.event.GossipEvent; -import com.swirlds.platform.gossip.SyncException; -import com.swirlds.platform.gossip.shadowgraph.Generations; -import com.swirlds.platform.network.ByteConstants; import com.swirlds.platform.network.SocketConfig; import edu.umd.cs.findbugs.annotations.NonNull; import java.io.BufferedInputStream; @@ -73,35 +70,6 @@ public CountingStreamExtension getSyncByteCounter() { return syncByteCounter; } - /** - * Reads a sync request response from the stream - * - * @return true if the sync has been accepted, false if it was rejected - * @throws IOException if a stream exception occurs - * @throws SyncException if something unexpected has been read from the stream - */ - public boolean readSyncRequestResponse() throws IOException, SyncException { - final byte b = readByte(); - if (b == ByteConstants.COMM_SYNC_NACK) { - // sync rejected - return false; - } - if (b != ByteConstants.COMM_SYNC_ACK) { - throw new SyncException(String.format( - "COMM_SYNC_REQUEST was sent but reply was %02x instead of COMM_SYNC_ACK or COMM_SYNC_NACK", b)); - } - return true; - } - - /** - * Read the other node's generation numbers from an input stream - * - * @throws IOException if a stream exception occurs - */ - public Generations readGenerations() throws IOException { - return readSerializable(false, Generations::new); - } - /** * Read the other node's tip hashes * diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncOutputStream.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncOutputStream.java index bf10619e3e69..e9775e4a5cb5 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncOutputStream.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/gossip/sync/SyncOutputStream.java @@ -22,9 +22,7 @@ import com.swirlds.common.crypto.Hash; import com.swirlds.common.io.extendable.extensions.CountingStreamExtension; import com.swirlds.common.io.streams.SerializableDataOutputStream; -import com.swirlds.platform.gossip.shadowgraph.Generations; import com.swirlds.platform.internal.EventImpl; -import com.swirlds.platform.network.ByteConstants; import com.swirlds.platform.network.SocketConfig; import edu.umd.cs.findbugs.annotations.NonNull; import java.io.BufferedOutputStream; @@ -81,50 +79,6 @@ public CountingStreamExtension getConnectionByteCounter() { return connectionByteCounter; } - /** - * @return the time the last sync request was sent - */ - public Instant getRequestSentTime() { - return requestSent.get(); - } - - /** - * Send a sync request - * - * @throws IOException if a stream exception occurs - */ - public void requestSync() throws IOException { - writeByte(ByteConstants.COMM_SYNC_REQUEST); - requestSent.set(Instant.now()); - } - - /** - * Accepts a previously requested sync - * - * @throws IOException if a stream exception occurs - */ - public void acceptSync() throws IOException { - writeByte(ByteConstants.COMM_SYNC_ACK); - } - - /** - * Rejects a previously requested sync - * - * @throws IOException if a stream exception occurs - */ - public void rejectSync() throws IOException { - writeByte(ByteConstants.COMM_SYNC_NACK); - } - - /** - * Write this node's generation numbers to an output stream - * - * @throws IOException if a stream exception occurs - */ - public void writeGenerations(final Generations generations) throws IOException { - writeSerializable(generations, false); - } - /** * Write to the {@link SyncOutputStream} the hashes of the tip events from this node's shadow graph * diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/metrics/SyncMetrics.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/metrics/SyncMetrics.java index bc440348d47b..9996afe2d354 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/metrics/SyncMetrics.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/metrics/SyncMetrics.java @@ -26,7 +26,7 @@ import com.swirlds.common.metrics.RunningAverageMetric; import com.swirlds.common.metrics.extensions.CountPerSecond; import com.swirlds.metrics.api.Metrics; -import com.swirlds.platform.consensus.GraphGenerations; +import com.swirlds.platform.consensus.NonAncientEventWindow; import com.swirlds.platform.gossip.shadowgraph.ShadowgraphSynchronizer; import com.swirlds.platform.gossip.shadowgraph.SyncResult; import com.swirlds.platform.gossip.shadowgraph.SyncTiming; @@ -37,6 +37,7 @@ import com.swirlds.platform.stats.AverageTimeStat; import com.swirlds.platform.stats.MaxStat; import com.swirlds.platform.system.PlatformStatNames; +import edu.umd.cs.findbugs.annotations.NonNull; import java.time.temporal.ChronoUnit; /** @@ -142,7 +143,7 @@ public class SyncMetrics { private final RunningAverageMetric tipsPerSync; - private final AverageStat syncGenerationDiff; + private final AverageStat syncIndicatorDiff; private final AverageStat eventRecRate; private final AverageTimeStat avgSyncDuration1; private final AverageTimeStat avgSyncDuration2; @@ -199,11 +200,11 @@ public SyncMetrics(final Metrics metrics) { "number of events received per successful sync", FORMAT_8_1); - syncGenerationDiff = new AverageStat( + syncIndicatorDiff = new AverageStat( metrics, INTERNAL_CATEGORY, - "syncGenDiff", - "number of generation ahead (positive) or behind (negative) when syncing", + "syncIndicatorDiff", + "number of ancient indicators ahead (positive) or behind (negative) when syncing", FORMAT_8_1, AverageStat.WEIGHT_VOLATILE); eventRecRate = new AverageStat( @@ -264,13 +265,13 @@ public SyncMetrics(final Metrics metrics) { } /** - * Supplies the generation numbers of a sync for statistics + * Supplies the event window numbers of a sync for statistics * - * @param self generations of our graph at the start of the sync - * @param other generations of their graph at the start of the sync + * @param self event window of our graph at the start of the sync + * @param other event window of their graph at the start of the sync */ - public void generations(final GraphGenerations self, final GraphGenerations other) { - syncGenerationDiff.update(self.getMaxRoundGeneration() - other.getMaxRoundGeneration()); + public void eventWindow(@NonNull final NonAncientEventWindow self, @NonNull final NonAncientEventWindow other) { + syncIndicatorDiff.update(self.getAncientThreshold() - other.getAncientThreshold()); } /** diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/wiring/PlatformWiring.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/wiring/PlatformWiring.java index cf9370285c76..af2dd6ebb6a2 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/wiring/PlatformWiring.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/wiring/PlatformWiring.java @@ -234,7 +234,7 @@ private void solderNonAncientEventWindow() { nonAncientEventWindowOutputWire.solderTo(inOrderLinkerWiring.nonAncientEventWindowInput(), INJECT); nonAncientEventWindowOutputWire.solderTo(pcesWriterWiring.nonAncientEventWindowInput(), INJECT); nonAncientEventWindowOutputWire.solderTo(eventCreationManagerWiring.nonAncientEventWindowInput(), INJECT); - nonAncientEventWindowOutputWire.solderTo(shadowgraphWiring.nonExpiredEventWindowInput(), INJECT); + nonAncientEventWindowOutputWire.solderTo(shadowgraphWiring.eventWindowInput(), INJECT); nonAncientEventWindowOutputWire.solderTo(futureEventBufferWiring.eventWindowInput(), INJECT); } @@ -545,6 +545,10 @@ public void updateRunningHash(@NonNull final RunningEventHashUpdate runningHashU */ public void updateNonAncientEventWindow(@NonNull final NonAncientEventWindow nonAncientEventWindow) { eventWindowManagerWiring.manualWindowInput().inject(nonAncientEventWindow); + + // Since there is asynchronous access to the shadowgraph, it's important to ensure that + // it has fully ingested the new event window before continuing. + shadowgraphWiring.flushRunnable().run(); } /** diff --git a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/wiring/components/ShadowgraphWiring.java b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/wiring/components/ShadowgraphWiring.java index 86ad1328ce91..9d782559efde 100644 --- a/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/wiring/components/ShadowgraphWiring.java +++ b/platform-sdk/swirlds-platform-core/src/main/java/com/swirlds/platform/wiring/components/ShadowgraphWiring.java @@ -27,13 +27,13 @@ /** * Wiring for the {@link Shadowgraph}. * - * @param eventInput the input wire for events to be added to the shadow graph - * @param nonExpiredEventWindowInput the input wire for the non-expired event window - * @param flushRunnable the runnable to flush the task scheduler + * @param eventInput the input wire for events to be added to the shadow graph + * @param eventWindowInput the input wire for the non-expired event window + * @param flushRunnable the runnable to flush the task scheduler */ public record ShadowgraphWiring( @NonNull InputWire eventInput, - @NonNull InputWire nonExpiredEventWindowInput, + @NonNull InputWire eventWindowInput, @NonNull Runnable flushRunnable) { /** @@ -57,7 +57,6 @@ public static ShadowgraphWiring create(@NonNull final TaskScheduler taskSc */ public void bind(@NonNull final Shadowgraph shadowgraph) { ((BindableInputWire) eventInput).bind(shadowgraph::addEvent); - ((BindableInputWire) nonExpiredEventWindowInput) - .bind(shadowgraph::updateNonExpiredEventWindow); + ((BindableInputWire) eventWindowInput).bind(shadowgraph::updateEventWindow); } } diff --git a/platform-sdk/swirlds-platform-core/src/test/java/com/swirlds/platform/event/FutureEventBufferTests.java b/platform-sdk/swirlds-platform-core/src/test/java/com/swirlds/platform/event/FutureEventBufferTests.java index 852d7b403edb..8210b09766bf 100644 --- a/platform-sdk/swirlds-platform-core/src/test/java/com/swirlds/platform/event/FutureEventBufferTests.java +++ b/platform-sdk/swirlds-platform-core/src/test/java/com/swirlds/platform/event/FutureEventBufferTests.java @@ -115,7 +115,7 @@ void futureEventsBufferedTest() { if (eventWindow.isAncient(event)) { // Ancient events should be discarded. assertNull(returnedEvent); - } else if (event.getHashedData().getBirthRound() <= eventWindow.pendingConsensusRound()) { + } else if (event.getHashedData().getBirthRound() <= eventWindow.getPendingConsensusRound()) { // Non-future events should be returned immediately. assertSame(event, returnedEvent); } else { @@ -195,7 +195,7 @@ void eventsGoAncientWhileBufferedTest() { if (eventWindow.isAncient(event)) { // Ancient events should be discarded. assertNull(returnedEvent); - } else if (event.getHashedData().getBirthRound() <= eventWindow.pendingConsensusRound()) { + } else if (event.getHashedData().getBirthRound() <= eventWindow.getPendingConsensusRound()) { // Non-future events should be returned immediately. assertSame(event, returnedEvent); } else { diff --git a/platform-sdk/swirlds-platform-core/src/testFixtures/java/com/swirlds/platform/test/fixtures/event/source/AbstractEventSource.java b/platform-sdk/swirlds-platform-core/src/testFixtures/java/com/swirlds/platform/test/fixtures/event/source/AbstractEventSource.java index a83b53ce215f..4d281bef4fa2 100644 --- a/platform-sdk/swirlds-platform-core/src/testFixtures/java/com/swirlds/platform/test/fixtures/event/source/AbstractEventSource.java +++ b/platform-sdk/swirlds-platform-core/src/testFixtures/java/com/swirlds/platform/test/fixtures/event/source/AbstractEventSource.java @@ -16,6 +16,7 @@ package com.swirlds.platform.test.fixtures.event.source; +import static com.swirlds.platform.system.events.EventConstants.FIRST_GENERATION; import static com.swirlds.platform.test.fixtures.event.EventUtils.integerPowerDistribution; import static com.swirlds.platform.test.fixtures.event.EventUtils.staticDynamicValue; @@ -228,10 +229,12 @@ public IndexedEvent generateEvent( // work for consensus, and we will have to upgrade our simulation framework prior to converting // consensus to use the birth round. final IndexedEvent latestSelfEvent = getLatestEvent(random); - final long birthRound = Math.max( - otherParentEvent == null ? 0 : otherParentEvent.getGeneration(), - latestSelfEvent == null ? 0 : latestSelfEvent.getGeneration()) + final long generation = Math.max( + otherParentEvent == null ? (FIRST_GENERATION - 1) : otherParentEvent.getGeneration(), + latestSelfEvent == null ? (FIRST_GENERATION - 1) : latestSelfEvent.getGeneration()) + 1; + // First generation is 0, but first birth round is 1. + final long birthRound = generation + 1; event = RandomEventUtils.randomEventWithTimestamp( random, diff --git a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/main/java/com/swirlds/platform/test/consensus/TestIntake.java b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/main/java/com/swirlds/platform/test/consensus/TestIntake.java index 8b2ebd32b275..ebc31ea35335 100644 --- a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/main/java/com/swirlds/platform/test/consensus/TestIntake.java +++ b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/main/java/com/swirlds/platform/test/consensus/TestIntake.java @@ -17,6 +17,8 @@ package com.swirlds.platform.test.consensus; import static com.swirlds.common.wiring.wires.SolderType.INJECT; +import static com.swirlds.platform.consensus.ConsensusConstants.ROUND_FIRST; +import static com.swirlds.platform.event.AncientMode.GENERATION_THRESHOLD; import static org.mockito.Mockito.mock; import com.swirlds.base.time.Time; @@ -32,7 +34,6 @@ import com.swirlds.platform.consensus.ConsensusConfig; import com.swirlds.platform.consensus.ConsensusSnapshot; import com.swirlds.platform.consensus.NonAncientEventWindow; -import com.swirlds.platform.event.AncientMode; import com.swirlds.platform.event.GossipEvent; import com.swirlds.platform.event.hashing.EventHasher; import com.swirlds.platform.event.linking.InOrderLinker; @@ -88,7 +89,7 @@ public TestIntake(@NonNull final AddressBook addressBook, @NonNull final Consens // FUTURE WORK: Broaden this test sweet to include testing ancient threshold via birth round. consensus = new ConsensusImpl( - consensusConfig, ConsensusUtils.NOOP_CONSENSUS_METRICS, addressBook, AncientMode.GENERATION_THRESHOLD); + consensusConfig, ConsensusUtils.NOOP_CONSENSUS_METRICS, addressBook, GENERATION_THRESHOLD); final PlatformContext platformContext = TestPlatformContextBuilder.create() .withConfiguration(new TestConfigBuilder().getOrCreateConfig()) @@ -218,17 +219,24 @@ public void loadSnapshot(@NonNull final ConsensusSnapshot snapshot) { consensus.getLastRoundDecided(), consensus.getMinGenerationNonAncient(), consensus.getMinRoundGeneration(), - AncientMode.GENERATION_THRESHOLD)); + GENERATION_THRESHOLD)); linkerWiring .nonAncientEventWindowInput() .put(new NonAncientEventWindow( consensus.getLastRoundDecided(), consensus.getMinGenerationNonAncient(), consensus.getMinRoundGeneration(), - AncientMode.GENERATION_THRESHOLD)); + GENERATION_THRESHOLD)); shadowGraph.clear(); - shadowGraph.startWithExpiredThreshold(consensus.getMinGenerationNonAncient()); + + final NonAncientEventWindow eventWindow = new NonAncientEventWindow( + ROUND_FIRST, + consensus.getMinGenerationNonAncient(), + consensus.getMinGenerationNonAncient(), + GENERATION_THRESHOLD); + + shadowGraph.startWithEventWindow(eventWindow); } public void flush() { diff --git a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/ShadowGraphTest.java b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/ShadowGraphTest.java index 7ebc85c530fd..735cc0d39c75 100644 --- a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/ShadowGraphTest.java +++ b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/ShadowGraphTest.java @@ -34,10 +34,10 @@ import com.swirlds.common.test.fixtures.platform.TestPlatformContextBuilder; import com.swirlds.common.utility.CommonUtils; import com.swirlds.platform.consensus.NonAncientEventWindow; +import com.swirlds.platform.gossip.shadowgraph.ReservedEventWindow; import com.swirlds.platform.gossip.shadowgraph.ShadowEvent; import com.swirlds.platform.gossip.shadowgraph.Shadowgraph; import com.swirlds.platform.gossip.shadowgraph.ShadowgraphInsertionException; -import com.swirlds.platform.gossip.shadowgraph.ShadowgraphReservation; import com.swirlds.platform.internal.EventImpl; import com.swirlds.platform.system.address.AddressBook; import com.swirlds.platform.test.event.emitter.EventEmitterFactory; @@ -52,7 +52,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Random; import java.util.Set; import java.util.function.Predicate; @@ -106,9 +105,9 @@ private void initShadowgraph(final Random random, final int numEvents, final int shadowgraph = new Shadowgraph(platformContext, mock(AddressBook.class)); for (int i = 0; i < numEvents; i++) { - IndexedEvent event = emitter.emitEvent(); + final IndexedEvent event = emitter.emitEvent(); - Hash hash = event.getBaseHash(); + final Hash hash = event.getBaseHash(); ancestorsMap.put(hash, ancestorsOf(event.getSelfParentHash(), event.getOtherParentHash())); assertDoesNotThrow(() -> shadowgraph.addEvent(event), "Unable to insert event into shadow graph."); assertTrue( @@ -134,40 +133,40 @@ private void initShadowgraph(final Random random, final int numEvents, final int @ParameterizedTest @MethodSource("graphSizes") void testFindAncestorsForMultipleEvents(final int numEvents, final int numNodes) { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, numEvents, numNodes); - Set generatedShadows = + final Set generatedShadows = generatedEvents.stream().map(shadowgraph::shadow).collect(Collectors.toSet()); - Set generatedShadowsSubset = generatedShadows.stream() + final Set generatedShadowsSubset = generatedShadows.stream() .filter((hash) -> random.nextDouble() < 0.5) .collect(Collectors.toSet()); - Set actualAncestors = shadowgraph.findAncestors(generatedShadowsSubset, (e) -> true).stream() + final Set actualAncestors = shadowgraph.findAncestors(generatedShadowsSubset, (e) -> true).stream() .map(ShadowEvent::getEventBaseHash) .collect(Collectors.toSet()); - for (ShadowEvent shadowEvent : generatedShadowsSubset) { + for (final ShadowEvent shadowEvent : generatedShadowsSubset) { assertSetsContainSameHashes(ancestorsMap.get(shadowEvent.getEventBaseHash()), actualAncestors); } } @RepeatedTest(10) void testFindAncestorsExcludesExpiredEvents() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, 100, 4); - long expireBelowGen = random.nextInt(10) + 1; + final long expireBelowGen = random.nextInt(10) + 1; final NonAncientEventWindow eventWindow = new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 0 /* ignored by shadowgraph */, expireBelowGen, GENERATION_THRESHOLD); - shadowgraph.updateNonExpiredEventWindow(eventWindow); + shadowgraph.updateEventWindow(eventWindow); - Set allEvents = shadowgraph.findAncestors(shadowgraph.getTips(), (e) -> true); - for (ShadowEvent event : allEvents) { + final Set allEvents = shadowgraph.findAncestors(shadowgraph.getTips(), (e) -> true); + for (final ShadowEvent event : allEvents) { assertTrue( event.getEvent().getGeneration() >= expireBelowGen, "Ancestors should not include expired events."); } @@ -182,7 +181,7 @@ private void assertSetsContainSameHashes(final Set expected, final Set ancestorsOf(final Hash selfParent, final Hash otherParent) { - Set ancestorSet = new HashSet<>(); + final Set ancestorSet = new HashSet<>(); if (selfParent != null) { ancestorSet.add(selfParent); if (ancestorsMap.containsKey(selfParent)) { @@ -198,34 +197,23 @@ private Set ancestorsOf(final Hash selfParent, final Hash otherParent) { return ancestorSet; } - private void printAncestors() { - for (Map.Entry> entry : ancestorsMap.entrySet()) { - String hash = CommonUtils.hex(entry.getKey().getValue(), 4); - List parents = entry.getValue().stream() - .filter(Objects::nonNull) - .map(h -> CommonUtils.hex(h.getValue(), 4)) - .collect(Collectors.toList()); - String parent1 = parents.size() > 0 ? parents.get(0) : "none"; - String parent2 = parents.size() > 1 ? parents.get(1) : "none"; - System.out.printf("\n%s = (%s, %s)", hash, parent1, parent2); - } - } - /** * This test verifies a single reservation can be made and closed without any event expiry. * * @param numEvents the number of events to put in the shadow graph * @param numNodes the number of nodes in the shadow graph - * @throws Exception if there was an error closing the reservation */ @ParameterizedTest @MethodSource("graphSizes") - void testSingleReservation(final int numEvents, final int numNodes) throws Exception { - Random random = RandomUtils.getRandomPrintSeed(); + void testSingleReservation(final int numEvents, final int numNodes) { + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, numEvents, numNodes); - ShadowgraphReservation r1 = shadowgraph.reserve(); - assertEquals(FIRST_GENERATION, r1.getReservedIndicator(), "First reservation should reserve generation 1"); + final ReservedEventWindow r1 = shadowgraph.reserve(); + assertEquals( + FIRST_GENERATION, + r1.getEventWindow().getExpiredThreshold(), + "First reservation should reserve generation 1"); assertEquals( 1, r1.getReservationCount(), @@ -234,11 +222,11 @@ void testSingleReservation(final int numEvents, final int numNodes) throws Excep r1.close(); assertEquals( FIRST_GENERATION, - r1.getReservedIndicator(), + r1.getEventWindow().getExpiredThreshold(), "The generation should not be affected by a reservation being closed."); assertEquals( 0, - r1.getReservationCount(), + r1.getEventWindow().getExpiredThreshold(), "Closing the second reservation should decrement the number of reservations."); } @@ -247,29 +235,27 @@ void testSingleReservation(final int numEvents, final int numNodes) throws Excep * * @param numEvents the number of events to put in the shadow graph * @param numNodes the number of nodes in the shadow graph - * @throws Exception if there was an error closing the reservation */ @ParameterizedTest @MethodSource("graphSizes") - void testMultipleReservationsNoExpiry(final int numEvents, final int numNodes) throws Exception { - Random random = RandomUtils.getRandomPrintSeed(); + void testMultipleReservationsNoExpiry(final int numEvents, final int numNodes) { + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, numEvents, numNodes); - ShadowgraphReservation r1 = shadowgraph.reserve(); - ShadowgraphReservation r2 = shadowgraph.reserve(); + final ReservedEventWindow r1 = shadowgraph.reserve(); + final ReservedEventWindow r2 = shadowgraph.reserve(); + assertEquals(r1.getEventWindow(), r2.getEventWindow()); assertEquals( - r1, - r2, - "The second call to reserve() prior to the first being closed should return the same object as the " - + "first reservation."); - assertEquals(FIRST_GENERATION, r2.getReservedIndicator(), "Second reservation should reserve generation 1"); + FIRST_GENERATION, + r2.getEventWindow().getExpiredThreshold(), + "Second reservation should reserve generation 1"); assertEquals(2, r2.getReservationCount(), "The second call to reserve() should result in 2 reservations."); r2.close(); assertEquals( FIRST_GENERATION, - r1.getReservedIndicator(), + r1.getEventWindow().getExpiredThreshold(), "The generation should not be affected by a reservation being closed."); assertEquals( 1, @@ -280,7 +266,7 @@ void testMultipleReservationsNoExpiry(final int numEvents, final int numNodes) t assertEquals( FIRST_GENERATION, - r1.getReservedIndicator(), + r1.getEventWindow().getExpiredThreshold(), "The generation should not be affected by a reservation being closed."); assertEquals( 0, @@ -293,22 +279,21 @@ void testMultipleReservationsNoExpiry(final int numEvents, final int numNodes) t * * @param numEvents the number of events to put in the shadow graph * @param numNodes the number of nodes in the shadow graph - * @throws Exception if there was an error closing the reservation */ @ParameterizedTest @MethodSource("graphSizes") - void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) throws Exception { - Random random = RandomUtils.getRandomPrintSeed(); + void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) { + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, numEvents, numNodes); - long expireBelowGen = FIRST_GENERATION + 1; + final long expireBelowGen = FIRST_GENERATION + 1; - ShadowgraphReservation r1 = shadowgraph.reserve(); + final ReservedEventWindow r1 = shadowgraph.reserve(); final NonAncientEventWindow eventWindow = new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 0 /* ignored by shadowgraph */, expireBelowGen, GENERATION_THRESHOLD); - shadowgraph.updateNonExpiredEventWindow(eventWindow); + shadowgraph.updateEventWindow(eventWindow); - ShadowgraphReservation r2 = shadowgraph.reserve(); + final ReservedEventWindow r2 = shadowgraph.reserve(); assertNotEquals( r1, r2, @@ -316,7 +301,7 @@ void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) + "instance."); assertEquals( expireBelowGen, - r2.getReservedIndicator(), + r2.getEventWindow().getExpiredThreshold(), "Reservation after call to expire() should reserve the expired generation + 1"); assertEquals( 1, r2.getReservationCount(), "The first reservation after expire() should result in 1 reservation."); @@ -325,7 +310,7 @@ void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) assertEquals( expireBelowGen, - r2.getReservedIndicator(), + r2.getEventWindow().getExpiredThreshold(), "The generation should not be affected by a reservation being closed."); assertEquals( 0, @@ -334,7 +319,7 @@ void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) assertEquals( FIRST_GENERATION, - r1.getReservedIndicator(), + r1.getEventWindow().getExpiredThreshold(), "The generation should not be affected by a reservation being closed."); assertEquals( 1, @@ -345,7 +330,7 @@ void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) assertEquals( FIRST_GENERATION, - r1.getReservedIndicator(), + r1.getEventWindow().getExpiredThreshold(), "The generation should not be affected by a reservation being closed."); assertEquals( 0, @@ -358,18 +343,17 @@ void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) * * @param numEvents the number of events to put in the shadow graph * @param numNodes the number of nodes in the shadow graph - * @throws Exception if there was an error closing the reservation */ @ParameterizedTest @MethodSource("graphSizes") void testExpireNoReservations(final int numEvents, final int numNodes) { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, numEvents, numNodes); - long expireBelowGen = random.nextInt((int) maxGen) + 2; + final long expireBelowGen = random.nextInt((int) maxGen) + 2; final NonAncientEventWindow eventWindow = new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 0 /* ignored by shadowgraph */, expireBelowGen, GENERATION_THRESHOLD); - shadowgraph.updateNonExpiredEventWindow(eventWindow); + shadowgraph.updateEventWindow(eventWindow); assertEventsBelowGenAreExpired(expireBelowGen); } @@ -399,28 +383,27 @@ private void assertEventsBelowGenAreExpired(final long expireBelowGen) { * * @param numEvents the number of events to put in the shadow graph * @param numNodes the number of nodes in the shadow graph - * @throws Exception if there was an error closing the reservation */ @ParameterizedTest @MethodSource("graphSizes") - void testExpireWithReservation(final int numEvents, final int numNodes) throws Exception { - Random random = RandomUtils.getRandomPrintSeed(); + void testExpireWithReservation(final int numEvents, final int numNodes) { + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, numEvents, numNodes); SyncTestUtils.printEvents("generated events", generatedEvents); - ShadowgraphReservation r0 = shadowgraph.reserve(); - shadowgraph.updateNonExpiredEventWindow(new NonAncientEventWindow( + final ReservedEventWindow r0 = shadowgraph.reserve(); + shadowgraph.updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 0 /* ignored by shadowgraph */, FIRST_GENERATION + 1, GENERATION_THRESHOLD)); - ShadowgraphReservation r1 = shadowgraph.reserve(); - shadowgraph.updateNonExpiredEventWindow(new NonAncientEventWindow( + final ReservedEventWindow r1 = shadowgraph.reserve(); + shadowgraph.updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 0 /* ignored by shadowgraph */, FIRST_GENERATION + 2, GENERATION_THRESHOLD)); - ShadowgraphReservation r2 = shadowgraph.reserve(); + final ReservedEventWindow r2 = shadowgraph.reserve(); // release the middle reservation to ensure that generations // greater than the lowest reserved generation are not expired. @@ -431,7 +414,7 @@ void testExpireWithReservation(final int numEvents, final int numNodes) throws E r2.close(); // Attempt to expire everything up to - shadowgraph.updateNonExpiredEventWindow(new NonAncientEventWindow( + shadowgraph.updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 0 /* ignored by shadowgraph */, FIRST_GENERATION + 2, @@ -441,7 +424,7 @@ void testExpireWithReservation(final int numEvents, final int numNodes) throws E assertEventsBelowGenAreExpired(0); r0.close(); - shadowgraph.updateNonExpiredEventWindow(new NonAncientEventWindow( + shadowgraph.updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 0 /* ignored by shadowgraph */, FIRST_GENERATION + 2, @@ -455,7 +438,7 @@ void testExpireWithReservation(final int numEvents, final int numNodes) throws E void testShadow() { initShadowgraph(RandomUtils.getRandomPrintSeed(), 0, 4); assertNull(shadowgraph.shadow(null), "Passing null should return null."); - IndexedEvent event = emitter.emitEvent(); + final IndexedEvent event = emitter.emitEvent(); assertDoesNotThrow(() -> shadowgraph.addEvent(event), "Adding an tip event should succeed."); assertEquals( event.getBaseHash(), @@ -475,17 +458,17 @@ void testShadowsNullListThrowsNPE() { @Test void testShadows() { initShadowgraph(RandomUtils.getRandomPrintSeed(), 0, 4); - List events = emitter.emitEvents(10); + final List events = emitter.emitEvents(10); events.forEach(e -> assertDoesNotThrow(() -> shadowgraph.addEvent(e), "Adding new tip events should succeed.")); - List hashes = events.stream().map(EventImpl::getBaseHash).collect(Collectors.toList()); - List shadows = shadowgraph.shadows(hashes); + final List hashes = events.stream().map(EventImpl::getBaseHash).collect(Collectors.toList()); + final List shadows = shadowgraph.shadows(hashes); assertEquals( events.size(), shadows.size(), "The number of shadow events should match the number of events provided."); - for (ShadowEvent shadow : shadows) { + for (final ShadowEvent shadow : shadows) { assertTrue( hashes.contains(shadow.getEventBaseHash()), "Each event provided should have a shadow event with the same hash."); @@ -495,26 +478,27 @@ void testShadows() { @Test void testShadowsWithUnknownEvents() { initShadowgraph(RandomUtils.getRandomPrintSeed(), 0, 4); - List events = emitter.emitEvents(10); + final List events = emitter.emitEvents(10); events.forEach(e -> assertDoesNotThrow(() -> shadowgraph.addEvent(e), "Adding new tip events should succeed.")); - List knownHashes = events.stream().map(EventImpl::getBaseHash).collect(Collectors.toList()); - List unknownHashes = + final List knownHashes = + events.stream().map(EventImpl::getBaseHash).collect(Collectors.toList()); + final List unknownHashes = emitter.emitEvents(10).stream().map(EventImpl::getBaseHash).collect(Collectors.toList()); - List allHashes = new ArrayList<>(knownHashes.size() + unknownHashes.size()); + final List allHashes = new ArrayList<>(knownHashes.size() + unknownHashes.size()); allHashes.addAll(knownHashes); allHashes.addAll(unknownHashes); Collections.shuffle(allHashes); - List shadows = shadowgraph.shadows(allHashes); + final List shadows = shadowgraph.shadows(allHashes); assertEquals( allHashes.size(), shadows.size(), "The number of shadow events should match the number of hashes provided."); for (int i = 0; i < allHashes.size(); i++) { - Hash hash = allHashes.get(i); + final Hash hash = allHashes.get(i); if (knownHashes.contains(hash)) { assertEquals( hash, @@ -537,9 +521,9 @@ void testAddNullEvent() { @RepeatedTest(10) void testAddDuplicateEvent() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, 10, 4); - IndexedEvent randomDuplicateEvent = generatedEvents.get(random.nextInt(generatedEvents.size())); + final IndexedEvent randomDuplicateEvent = generatedEvents.get(random.nextInt(generatedEvents.size())); assertThrows( ShadowgraphInsertionException.class, () -> shadowgraph.addEvent(randomDuplicateEvent), @@ -553,7 +537,7 @@ void testAddDuplicateEvent() { void testAddEventWithExpiredGeneration() { initShadowgraph(RandomUtils.getRandomPrintSeed(), 100, 4); - shadowgraph.updateNonExpiredEventWindow(new NonAncientEventWindow( + shadowgraph.updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 0 /* ignored by shadowgraph */, FIRST_GENERATION + 1, @@ -570,7 +554,7 @@ void testAddEventWithExpiredGeneration() { void testAddEventWithUnknownOtherParent() { initShadowgraph(RandomUtils.getRandomPrintSeed(), 100, 4); - IndexedEvent newEvent = emitter.emitEvent(); + final IndexedEvent newEvent = emitter.emitEvent(); newEvent.setOtherParent(emitter.emitEvent()); assertDoesNotThrow( @@ -581,7 +565,7 @@ void testAddEventWithUnknownOtherParent() { void testAddEventWithUnknownSelfParent() { initShadowgraph(RandomUtils.getRandomPrintSeed(), 100, 4); - IndexedEvent newEvent = emitter.emitEvent(); + final IndexedEvent newEvent = emitter.emitEvent(); newEvent.setSelfParent(emitter.emitEvent()); assertDoesNotThrow(() -> shadowgraph.addEvent(newEvent), "Events with an unknown self parent should be added."); @@ -591,13 +575,13 @@ void testAddEventWithUnknownSelfParent() { void testAddEventWithExpiredParents() { initShadowgraph(RandomUtils.getRandomPrintSeed(), 100, 4); - IndexedEvent newEvent = emitter.emitEvent(); + final IndexedEvent newEvent = emitter.emitEvent(); final NonAncientEventWindow eventWindow = new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 0 /* ignored by shadowgraph */, newEvent.getGeneration(), GENERATION_THRESHOLD); - shadowgraph.updateNonExpiredEventWindow(eventWindow); + shadowgraph.updateEventWindow(eventWindow); assertDoesNotThrow(() -> shadowgraph.addEvent(newEvent), "Events with expired parents should be added."); } @@ -606,15 +590,15 @@ void testAddEventWithExpiredParents() { void testAddEventUpdatesTips() { initShadowgraph(RandomUtils.getRandomPrintSeed(), 100, 4); - int tipsSize = shadowgraph.getTips().size(); - int additionalEvents = 100; + final int tipsSize = shadowgraph.getTips().size(); + final int additionalEvents = 100; for (int i = 0; i < additionalEvents; i++) { - IndexedEvent newTip = emitter.emitEvent(); + final IndexedEvent newTip = emitter.emitEvent(); assertNull(shadowgraph.shadow(newTip), "The shadow graph should not contain the new event."); assertDoesNotThrow(() -> shadowgraph.addEvent(newTip), "The new tip should be added to the shadow graph."); - ShadowEvent tipShadow = shadowgraph.shadow(newTip); + final ShadowEvent tipShadow = shadowgraph.shadow(newTip); assertEquals( tipsSize, @@ -636,10 +620,10 @@ void testHashgraphEventWithNullHash() { @RepeatedTest(10) void testHashgraphEventWithExistingHash() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, 100, 4); - IndexedEvent randomExistingEvent = generatedEvents.get(random.nextInt(generatedEvents.size())); + final IndexedEvent randomExistingEvent = generatedEvents.get(random.nextInt(generatedEvents.size())); assertEquals( randomExistingEvent, shadowgraph.hashgraphEvent(randomExistingEvent.getBaseHash()), @@ -648,23 +632,26 @@ void testHashgraphEventWithExistingHash() { @Test void testClear() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, 100, 4); - ShadowgraphReservation r0 = shadowgraph.reserve(); - ShadowgraphReservation r1 = shadowgraph.reserve(); + ReservedEventWindow r0 = shadowgraph.reserve(); + final ReservedEventWindow r1 = shadowgraph.reserve(); r0.close(); r1.close(); shadowgraph.clear(); assertEquals(0, shadowgraph.getTips().size(), "Shadow graph should not have any tips after being cleared."); - for (IndexedEvent generatedEvent : generatedEvents) { + for (final IndexedEvent generatedEvent : generatedEvents) { assertNull( shadowgraph.shadow(generatedEvent), "Shadow graph should not have any events after being cleared."); } r0 = shadowgraph.reserve(); - assertEquals(0, r0.getReservedIndicator(), "The first reservation after clearing should reserve generation 0."); + assertEquals( + 0, + r0.getEventWindow().getExpiredThreshold(), + "The first reservation after clearing should reserve generation 0."); assertEquals( 1, r0.getReservationCount(), "The first reservation after clearing should have a single reservation."); } @@ -672,12 +659,12 @@ void testClear() { @Test @DisplayName("Test that clear() disconnect all shadow events in the shadow graph") void testClearDisconnects() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, 100, 4); - List tips = shadowgraph.getTips(); - Set shadows = new HashSet<>(); - for (ShadowEvent tip : tips) { + final List tips = shadowgraph.getTips(); + final Set shadows = new HashSet<>(); + for (final ShadowEvent tip : tips) { ShadowEvent sp = tip.getSelfParent(); while (sp != null) { shadows.add(sp); @@ -688,7 +675,7 @@ void testClearDisconnects() { shadowgraph.clear(); - for (ShadowEvent s : shadows) { + for (final ShadowEvent s : shadows) { assertNull(s.getSelfParent(), "after a clear, all parents should be disconnected"); assertNull(s.getOtherParent(), "after a clear, all parents should be disconnected"); } @@ -696,27 +683,27 @@ void testClearDisconnects() { @RepeatedTest(10) void testTipsExpired() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowgraph(random, 100, 4); long oldestTipGen = Long.MAX_VALUE; - List tipsToExpire = new ArrayList<>(); - for (ShadowEvent tip : shadowgraph.getTips()) { + final List tipsToExpire = new ArrayList<>(); + for (final ShadowEvent tip : shadowgraph.getTips()) { oldestTipGen = Math.min(oldestTipGen, tip.getEvent().getGeneration()); } - for (ShadowEvent tip : shadowgraph.getTips()) { + for (final ShadowEvent tip : shadowgraph.getTips()) { if (tip.getEvent().getGeneration() == oldestTipGen) { tipsToExpire.add(tip); } } - int numTipsBeforeExpiry = shadowgraph.getTips().size(); + final int numTipsBeforeExpiry = shadowgraph.getTips().size(); assertTrue(numTipsBeforeExpiry > 0, "Shadow graph should have tips after events are added."); final NonAncientEventWindow eventWindow = new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 0 /* ignored by shadowgraph */, oldestTipGen + 1, GENERATION_THRESHOLD); - shadowgraph.updateNonExpiredEventWindow(eventWindow); + shadowgraph.updateEventWindow(eventWindow); assertEquals( numTipsBeforeExpiry - tipsToExpire.size(), diff --git a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/ShadowgraphByBirthRoundTests.java b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/ShadowgraphByBirthRoundTests.java index d2cbfac4824c..6d98a3663e44 100644 --- a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/ShadowgraphByBirthRoundTests.java +++ b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/ShadowgraphByBirthRoundTests.java @@ -37,10 +37,10 @@ import com.swirlds.config.extensions.test.fixtures.TestConfigBuilder; import com.swirlds.platform.consensus.NonAncientEventWindow; import com.swirlds.platform.eventhandling.EventConfig_; +import com.swirlds.platform.gossip.shadowgraph.ReservedEventWindow; import com.swirlds.platform.gossip.shadowgraph.ShadowEvent; import com.swirlds.platform.gossip.shadowgraph.Shadowgraph; import com.swirlds.platform.gossip.shadowgraph.ShadowgraphInsertionException; -import com.swirlds.platform.gossip.shadowgraph.ShadowgraphReservation; import com.swirlds.platform.internal.EventImpl; import com.swirlds.platform.system.address.AddressBook; import com.swirlds.platform.test.event.emitter.EventEmitterFactory; @@ -53,7 +53,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Random; import java.util.Set; import java.util.function.Predicate; @@ -112,9 +111,9 @@ private void initShadowGraph(final Random random, final int numEvents, final int shadowGraph = new Shadowgraph(platformContext, mock(AddressBook.class)); for (int i = 0; i < numEvents; i++) { - IndexedEvent event = emitter.emitEvent(); + final IndexedEvent event = emitter.emitEvent(); - Hash hash = event.getBaseHash(); + final Hash hash = event.getBaseHash(); ancestorsMap.put(hash, ancestorsOf(event.getSelfParentHash(), event.getOtherParentHash())); assertDoesNotThrow(() -> shadowGraph.addEvent(event), "Unable to insert event into shadow graph."); assertTrue( @@ -143,32 +142,32 @@ private void initShadowGraph(final Random random, final int numEvents, final int @ParameterizedTest @MethodSource("graphSizes") void testFindAncestorsForMultipleEvents(final int numEvents, final int numNodes) { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowGraph(random, numEvents, numNodes); - Set generatedShadows = + final Set generatedShadows = generatedEvents.stream().map(shadowGraph::shadow).collect(Collectors.toSet()); - Set generatedShadowsSubset = generatedShadows.stream() + final Set generatedShadowsSubset = generatedShadows.stream() .filter((hash) -> random.nextDouble() < 0.5) .collect(Collectors.toSet()); - Set actualAncestors = shadowGraph.findAncestors(generatedShadowsSubset, (e) -> true).stream() + final Set actualAncestors = shadowGraph.findAncestors(generatedShadowsSubset, (e) -> true).stream() .map(ShadowEvent::getEventBaseHash) .collect(Collectors.toSet()); - for (ShadowEvent shadowEvent : generatedShadowsSubset) { + for (final ShadowEvent shadowEvent : generatedShadowsSubset) { assertSetsContainSameHashes(ancestorsMap.get(shadowEvent.getEventBaseHash()), actualAncestors); } } @RepeatedTest(10) void testFindAncestorsExcludesExpiredEvents() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowGraph(random, 100, 4); - long expireBelowBirthRound = random.nextInt(10) + 1; + final long expireBelowBirthRound = random.nextInt(10) + 1; final NonAncientEventWindow eventWindow = new NonAncientEventWindow( 0 /* ignored by shadowgraph */, @@ -176,10 +175,10 @@ void testFindAncestorsExcludesExpiredEvents() { expireBelowBirthRound, BIRTH_ROUND_THRESHOLD); - shadowGraph.updateNonExpiredEventWindow(eventWindow); + shadowGraph.updateEventWindow(eventWindow); - Set allEvents = shadowGraph.findAncestors(shadowGraph.getTips(), (e) -> true); - for (ShadowEvent event : allEvents) { + final Set allEvents = shadowGraph.findAncestors(shadowGraph.getTips(), (e) -> true); + for (final ShadowEvent event : allEvents) { assertTrue( event.getEvent().getBaseEvent().getAncientIndicator(BIRTH_ROUND_THRESHOLD) >= expireBelowBirthRound, "Ancestors should not include expired events."); @@ -195,7 +194,7 @@ private void assertSetsContainSameHashes(final Set expected, final Set ancestorsOf(final Hash selfParent, final Hash otherParent) { - Set ancestorSet = new HashSet<>(); + final Set ancestorSet = new HashSet<>(); if (selfParent != null) { ancestorSet.add(selfParent); if (ancestorsMap.containsKey(selfParent)) { @@ -211,34 +210,23 @@ private Set ancestorsOf(final Hash selfParent, final Hash otherParent) { return ancestorSet; } - private void printAncestors() { - for (Map.Entry> entry : ancestorsMap.entrySet()) { - String hash = CommonUtils.hex(entry.getKey().getValue(), 4); - List parents = entry.getValue().stream() - .filter(Objects::nonNull) - .map(h -> CommonUtils.hex(h.getValue(), 4)) - .collect(Collectors.toList()); - String parent1 = parents.size() > 0 ? parents.get(0) : "none"; - String parent2 = parents.size() > 1 ? parents.get(1) : "none"; - System.out.printf("\n%s = (%s, %s)", hash, parent1, parent2); - } - } - /** * This test verifies a single reservation can be made and closed without any event expiry. * * @param numEvents the number of events to put in the shadow graph * @param numNodes the number of nodes in the shadow graph - * @throws Exception if there was an error closing the reservation */ @ParameterizedTest @MethodSource("graphSizes") - void testSingleReservation(final int numEvents, final int numNodes) throws Exception { - Random random = RandomUtils.getRandomPrintSeed(); + void testSingleReservation(final int numEvents, final int numNodes) { + final Random random = RandomUtils.getRandomPrintSeed(); initShadowGraph(random, numEvents, numNodes); - ShadowgraphReservation r1 = shadowGraph.reserve(); - assertEquals(ROUND_FIRST, r1.getReservedIndicator(), "First reservation should reserve birth round 1"); + final ReservedEventWindow r1 = shadowGraph.reserve(); + assertEquals( + ROUND_FIRST, + r1.getEventWindow().getExpiredThreshold(), + "First reservation should reserve birth round 1"); assertEquals( 1, r1.getReservationCount(), @@ -247,7 +235,7 @@ void testSingleReservation(final int numEvents, final int numNodes) throws Excep r1.close(); assertEquals( ROUND_FIRST, - r1.getReservedIndicator(), + r1.getEventWindow().getExpiredThreshold(), "The birth round should not be affected by a reservation being closed."); assertEquals( 0, @@ -260,29 +248,27 @@ void testSingleReservation(final int numEvents, final int numNodes) throws Excep * * @param numEvents the number of events to put in the shadow graph * @param numNodes the number of nodes in the shadow graph - * @throws Exception if there was an error closing the reservation */ @ParameterizedTest @MethodSource("graphSizes") - void testMultipleReservationsNoExpiry(final int numEvents, final int numNodes) throws Exception { - Random random = RandomUtils.getRandomPrintSeed(); + void testMultipleReservationsNoExpiry(final int numEvents, final int numNodes) { + final Random random = RandomUtils.getRandomPrintSeed(); initShadowGraph(random, numEvents, numNodes); - ShadowgraphReservation r1 = shadowGraph.reserve(); - ShadowgraphReservation r2 = shadowGraph.reserve(); + final ReservedEventWindow r1 = shadowGraph.reserve(); + final ReservedEventWindow r2 = shadowGraph.reserve(); + assertEquals(r1.getEventWindow(), r2.getEventWindow()); assertEquals( - r1, - r2, - "The second call to reserve() prior to the first being closed should return the same object as the " - + "first reservation."); - assertEquals(ROUND_FIRST, r2.getReservedIndicator(), "Second reservation should reserve birth round 1"); + ROUND_FIRST, + r2.getEventWindow().getExpiredThreshold(), + "Second reservation should reserve birth round 1"); assertEquals(2, r2.getReservationCount(), "The second call to reserve() should result in 2 reservations."); r2.close(); assertEquals( ROUND_FIRST, - r1.getReservedIndicator(), + r1.getEventWindow().getExpiredThreshold(), "The birth round should not be affected by a reservation being closed."); assertEquals( 1, @@ -293,7 +279,7 @@ void testMultipleReservationsNoExpiry(final int numEvents, final int numNodes) t assertEquals( ROUND_FIRST, - r1.getReservedIndicator(), + r1.getEventWindow().getExpiredThreshold(), "The birth round should not be affected by a reservation being closed."); assertEquals( 0, @@ -306,25 +292,24 @@ void testMultipleReservationsNoExpiry(final int numEvents, final int numNodes) t * * @param numEvents the number of events to put in the shadow graph * @param numNodes the number of nodes in the shadow graph - * @throws Exception if there was an error closing the reservation */ @ParameterizedTest @MethodSource("graphSizes") - void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) throws Exception { - Random random = RandomUtils.getRandomPrintSeed(); + void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) { + final Random random = RandomUtils.getRandomPrintSeed(); initShadowGraph(random, numEvents, numNodes); - long expireBelowBirthRound = ROUND_FIRST + 1; + final long expireBelowBirthRound = ROUND_FIRST + 1; - ShadowgraphReservation r1 = shadowGraph.reserve(); + final ReservedEventWindow r1 = shadowGraph.reserve(); final NonAncientEventWindow eventWindow = new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 1 /* ignored by shadowgraph */, expireBelowBirthRound, BIRTH_ROUND_THRESHOLD); - shadowGraph.updateNonExpiredEventWindow(eventWindow); + shadowGraph.updateEventWindow(eventWindow); - ShadowgraphReservation r2 = shadowGraph.reserve(); + final ReservedEventWindow r2 = shadowGraph.reserve(); assertNotEquals( r1, r2, @@ -332,7 +317,7 @@ void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) + "instance."); assertEquals( expireBelowBirthRound, - r2.getReservedIndicator(), + r2.getEventWindow().getExpiredThreshold(), "Reservation after call to expire() should reserve the expired birth round + 1"); assertEquals( 1, r2.getReservationCount(), "The first reservation after expire() should result in 1 reservation."); @@ -341,7 +326,7 @@ void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) assertEquals( expireBelowBirthRound, - r2.getReservedIndicator(), + r2.getEventWindow().getExpiredThreshold(), "The birth round should not be affected by a reservation being closed."); assertEquals( 0, @@ -350,7 +335,7 @@ void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) assertEquals( ROUND_FIRST, - r1.getReservedIndicator(), + r1.getEventWindow().getExpiredThreshold(), "The birth round should not be affected by a reservation being closed."); assertEquals( 1, @@ -361,7 +346,7 @@ void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) assertEquals( ROUND_FIRST, - r1.getReservedIndicator(), + r1.getEventWindow().getExpiredThreshold(), "The birth round should not be affected by a reservation being closed."); assertEquals( 0, @@ -374,7 +359,6 @@ void testMultipleReservationsWithExpiry(final int numEvents, final int numNodes) * * @param numEvents the number of events to put in the shadow graph * @param numNodes the number of nodes in the shadow graph - * @throws Exception if there was an error closing the reservation */ @ParameterizedTest @MethodSource("graphSizes") @@ -388,7 +372,7 @@ void testExpireNoReservations(final int numEvents, final int numNodes) { 1 /* ignored by shadowgraph */, expireBelowBirthRound, BIRTH_ROUND_THRESHOLD); - shadowGraph.updateNonExpiredEventWindow(eventWindow); + shadowGraph.updateEventWindow(eventWindow); assertEventsBelowBirthRoundAreExpired(expireBelowBirthRound); } @@ -418,28 +402,27 @@ private void assertEventsBelowBirthRoundAreExpired(final long expireBelowBirthRo * * @param numEvents the number of events to put in the shadow graph * @param numNodes the number of nodes in the shadow graph - * @throws Exception if there was an error closing the reservation */ @ParameterizedTest @MethodSource("graphSizes") - void testExpireWithReservation(final int numEvents, final int numNodes) throws Exception { - Random random = RandomUtils.getRandomPrintSeed(); + void testExpireWithReservation(final int numEvents, final int numNodes) { + final Random random = RandomUtils.getRandomPrintSeed(); initShadowGraph(random, numEvents, numNodes); SyncTestUtils.printEvents("generated events", generatedEvents); - ShadowgraphReservation r0 = shadowGraph.reserve(); - shadowGraph.updateNonExpiredEventWindow(new NonAncientEventWindow( + final ReservedEventWindow r0 = shadowGraph.reserve(); + shadowGraph.updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 1 /* ignored by shadowgraph */, ROUND_FIRST + 1, BIRTH_ROUND_THRESHOLD)); - ShadowgraphReservation r1 = shadowGraph.reserve(); - shadowGraph.updateNonExpiredEventWindow(new NonAncientEventWindow( + final ReservedEventWindow r1 = shadowGraph.reserve(); + shadowGraph.updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 1 /* ignored by shadowgraph */, ROUND_FIRST + 2, BIRTH_ROUND_THRESHOLD)); - ShadowgraphReservation r2 = shadowGraph.reserve(); + final ReservedEventWindow r2 = shadowGraph.reserve(); // release the middle reservation to ensure that birth rounds // greater than the lowest reserved birth round are not expired. @@ -450,7 +433,7 @@ void testExpireWithReservation(final int numEvents, final int numNodes) throws E r2.close(); // Attempt to expire everything up to - shadowGraph.updateNonExpiredEventWindow(new NonAncientEventWindow( + shadowGraph.updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 1 /* ignored by shadowgraph */, ROUND_FIRST + 2, @@ -460,7 +443,7 @@ void testExpireWithReservation(final int numEvents, final int numNodes) throws E assertEventsBelowBirthRoundAreExpired(0); r0.close(); - shadowGraph.updateNonExpiredEventWindow(new NonAncientEventWindow( + shadowGraph.updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 1 /* ignored by shadowgraph */, ROUND_FIRST + 2, @@ -474,7 +457,7 @@ void testExpireWithReservation(final int numEvents, final int numNodes) throws E void testShadow() { initShadowGraph(RandomUtils.getRandomPrintSeed(), 0, 4); assertNull(shadowGraph.shadow(null), "Passing null should return null."); - IndexedEvent event = emitter.emitEvent(); + final IndexedEvent event = emitter.emitEvent(); assertDoesNotThrow(() -> shadowGraph.addEvent(event), "Adding an tip event should succeed."); assertEquals( event.getBaseHash(), @@ -494,17 +477,17 @@ void testShadowsNullListThrowsNPE() { @Test void testShadows() { initShadowGraph(RandomUtils.getRandomPrintSeed(), 0, 4); - List events = emitter.emitEvents(10); + final List events = emitter.emitEvents(10); events.forEach(e -> assertDoesNotThrow(() -> shadowGraph.addEvent(e), "Adding new tip events should succeed.")); - List hashes = events.stream().map(EventImpl::getBaseHash).collect(Collectors.toList()); - List shadows = shadowGraph.shadows(hashes); + final List hashes = events.stream().map(EventImpl::getBaseHash).collect(Collectors.toList()); + final List shadows = shadowGraph.shadows(hashes); assertEquals( events.size(), shadows.size(), "The number of shadow events should match the number of events provided."); - for (ShadowEvent shadow : shadows) { + for (final ShadowEvent shadow : shadows) { assertTrue( hashes.contains(shadow.getEventBaseHash()), "Each event provided should have a shadow event with the same hash."); @@ -514,26 +497,27 @@ void testShadows() { @Test void testShadowsWithUnknownEvents() { initShadowGraph(RandomUtils.getRandomPrintSeed(), 0, 4); - List events = emitter.emitEvents(10); + final List events = emitter.emitEvents(10); events.forEach(e -> assertDoesNotThrow(() -> shadowGraph.addEvent(e), "Adding new tip events should succeed.")); - List knownHashes = events.stream().map(EventImpl::getBaseHash).collect(Collectors.toList()); - List unknownHashes = + final List knownHashes = + events.stream().map(EventImpl::getBaseHash).collect(Collectors.toList()); + final List unknownHashes = emitter.emitEvents(10).stream().map(EventImpl::getBaseHash).collect(Collectors.toList()); - List allHashes = new ArrayList<>(knownHashes.size() + unknownHashes.size()); + final List allHashes = new ArrayList<>(knownHashes.size() + unknownHashes.size()); allHashes.addAll(knownHashes); allHashes.addAll(unknownHashes); Collections.shuffle(allHashes); - List shadows = shadowGraph.shadows(allHashes); + final List shadows = shadowGraph.shadows(allHashes); assertEquals( allHashes.size(), shadows.size(), "The number of shadow events should match the number of hashes provided."); for (int i = 0; i < allHashes.size(); i++) { - Hash hash = allHashes.get(i); + final Hash hash = allHashes.get(i); if (knownHashes.contains(hash)) { assertEquals( hash, @@ -556,9 +540,9 @@ void testAddNullEvent() { @RepeatedTest(10) void testAddDuplicateEvent() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowGraph(random, 10, 4); - IndexedEvent randomDuplicateEvent = generatedEvents.get(random.nextInt(generatedEvents.size())); + final IndexedEvent randomDuplicateEvent = generatedEvents.get(random.nextInt(generatedEvents.size())); assertThrows( ShadowgraphInsertionException.class, () -> shadowGraph.addEvent(randomDuplicateEvent), @@ -572,7 +556,7 @@ void testAddDuplicateEvent() { void testAddEventWithExpiredBirthRound() { initShadowGraph(RandomUtils.getRandomPrintSeed(), 100, 4); - shadowGraph.updateNonExpiredEventWindow(new NonAncientEventWindow( + shadowGraph.updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 1 /* ignored by shadowgraph */, ROUND_FIRST + 1, @@ -589,7 +573,7 @@ void testAddEventWithExpiredBirthRound() { void testAddEventWithUnknownOtherParent() { initShadowGraph(RandomUtils.getRandomPrintSeed(), 100, 4); - IndexedEvent newEvent = emitter.emitEvent(); + final IndexedEvent newEvent = emitter.emitEvent(); newEvent.setOtherParent(emitter.emitEvent()); assertDoesNotThrow( @@ -600,7 +584,7 @@ void testAddEventWithUnknownOtherParent() { void testAddEventWithUnknownSelfParent() { initShadowGraph(RandomUtils.getRandomPrintSeed(), 100, 4); - IndexedEvent newEvent = emitter.emitEvent(); + final IndexedEvent newEvent = emitter.emitEvent(); newEvent.setSelfParent(emitter.emitEvent()); assertDoesNotThrow(() -> shadowGraph.addEvent(newEvent), "Events with an unknown self parent should be added."); @@ -610,13 +594,13 @@ void testAddEventWithUnknownSelfParent() { void testAddEventWithExpiredParents() { initShadowGraph(RandomUtils.getRandomPrintSeed(), 100, 4); - IndexedEvent newEvent = emitter.emitEvent(); + final IndexedEvent newEvent = emitter.emitEvent(); final NonAncientEventWindow eventWindow = new NonAncientEventWindow( 0 /* ignored by shadowgraph */, 1 /* ignored by shadowgraph */, newEvent.getBaseEvent().getAncientIndicator(BIRTH_ROUND_THRESHOLD), BIRTH_ROUND_THRESHOLD); - shadowGraph.updateNonExpiredEventWindow(eventWindow); + shadowGraph.updateEventWindow(eventWindow); assertDoesNotThrow(() -> shadowGraph.addEvent(newEvent), "Events with expired parents should be added."); } @@ -625,15 +609,15 @@ void testAddEventWithExpiredParents() { void testAddEventUpdatesTips() { initShadowGraph(RandomUtils.getRandomPrintSeed(), 100, 4); - int tipsSize = shadowGraph.getTips().size(); - int additionalEvents = 100; + final int tipsSize = shadowGraph.getTips().size(); + final int additionalEvents = 100; for (int i = 0; i < additionalEvents; i++) { - IndexedEvent newTip = emitter.emitEvent(); + final IndexedEvent newTip = emitter.emitEvent(); assertNull(shadowGraph.shadow(newTip), "The shadow graph should not contain the new event."); assertDoesNotThrow(() -> shadowGraph.addEvent(newTip), "The new tip should be added to the shadow graph."); - ShadowEvent tipShadow = shadowGraph.shadow(newTip); + final ShadowEvent tipShadow = shadowGraph.shadow(newTip); assertEquals( tipsSize, @@ -655,10 +639,10 @@ void testHashgraphEventWithNullHash() { @RepeatedTest(10) void testHashgraphEventWithExistingHash() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowGraph(random, 100, 4); - IndexedEvent randomExistingEvent = generatedEvents.get(random.nextInt(generatedEvents.size())); + final IndexedEvent randomExistingEvent = generatedEvents.get(random.nextInt(generatedEvents.size())); assertEquals( randomExistingEvent, shadowGraph.hashgraphEvent(randomExistingEvent.getBaseHash()), @@ -667,24 +651,26 @@ void testHashgraphEventWithExistingHash() { @Test void testClear() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowGraph(random, 100, 4); - ShadowgraphReservation r0 = shadowGraph.reserve(); - ShadowgraphReservation r1 = shadowGraph.reserve(); + ReservedEventWindow r0 = shadowGraph.reserve(); + final ReservedEventWindow r1 = shadowGraph.reserve(); r0.close(); r1.close(); shadowGraph.clear(); assertEquals(0, shadowGraph.getTips().size(), "Shadow graph should not have any tips after being cleared."); - for (IndexedEvent generatedEvent : generatedEvents) { + for (final IndexedEvent generatedEvent : generatedEvents) { assertNull( shadowGraph.shadow(generatedEvent), "Shadow graph should not have any events after being cleared."); } r0 = shadowGraph.reserve(); assertEquals( - 1, r0.getReservedIndicator(), "The first reservation after clearing should reserve birth round 1."); + 1, + r0.getEventWindow().getExpiredThreshold(), + "The first reservation after clearing should reserve birth round 1."); assertEquals( 1, r0.getReservationCount(), "The first reservation after clearing should have a single reservation."); } @@ -692,12 +678,12 @@ void testClear() { @Test @DisplayName("Test that clear() disconnect all shadow events in the shadow graph") void testClearDisconnects() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowGraph(random, 100, 4); - List tips = shadowGraph.getTips(); - Set shadows = new HashSet<>(); - for (ShadowEvent tip : tips) { + final List tips = shadowGraph.getTips(); + final Set shadows = new HashSet<>(); + for (final ShadowEvent tip : tips) { ShadowEvent sp = tip.getSelfParent(); while (sp != null) { shadows.add(sp); @@ -708,7 +694,7 @@ void testClearDisconnects() { shadowGraph.clear(); - for (ShadowEvent s : shadows) { + for (final ShadowEvent s : shadows) { assertNull(s.getSelfParent(), "after a clear, all parents should be disconnected"); assertNull(s.getOtherParent(), "after a clear, all parents should be disconnected"); } @@ -716,23 +702,23 @@ void testClearDisconnects() { @RepeatedTest(10) void testTipsExpired() { - Random random = RandomUtils.getRandomPrintSeed(); + final Random random = RandomUtils.getRandomPrintSeed(); initShadowGraph(random, 100, 4); long oldestTipBirthRound = Long.MAX_VALUE; - List tipsToExpire = new ArrayList<>(); - for (ShadowEvent tip : shadowGraph.getTips()) { + final List tipsToExpire = new ArrayList<>(); + for (final ShadowEvent tip : shadowGraph.getTips()) { oldestTipBirthRound = Math.min( oldestTipBirthRound, tip.getEvent().getBaseEvent().getAncientIndicator(BIRTH_ROUND_THRESHOLD)); } - for (ShadowEvent tip : shadowGraph.getTips()) { + for (final ShadowEvent tip : shadowGraph.getTips()) { if (tip.getEvent().getBaseEvent().getAncientIndicator(BIRTH_ROUND_THRESHOLD) == oldestTipBirthRound) { tipsToExpire.add(tip); } } - int numTipsBeforeExpiry = shadowGraph.getTips().size(); + final int numTipsBeforeExpiry = shadowGraph.getTips().size(); assertTrue(numTipsBeforeExpiry > 0, "Shadow graph should have tips after events are added."); final NonAncientEventWindow eventWindow = new NonAncientEventWindow( @@ -740,7 +726,7 @@ void testTipsExpired() { 1 /* ignored by shadowgraph */, oldestTipBirthRound + 1, BIRTH_ROUND_THRESHOLD); - shadowGraph.updateNonExpiredEventWindow(eventWindow); + shadowGraph.updateEventWindow(eventWindow); assertEquals( numTipsBeforeExpiry - tipsToExpire.size(), diff --git a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncNode.java b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncNode.java index e13bdf00aaf9..6b3bf8e9adc4 100644 --- a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncNode.java +++ b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncNode.java @@ -17,7 +17,6 @@ package com.swirlds.platform.test.sync; import static com.swirlds.common.threading.manager.AdHocThreadManager.getStaticThreadManager; -import static com.swirlds.platform.event.AncientMode.GENERATION_THRESHOLD; import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; @@ -29,21 +28,25 @@ import com.swirlds.common.threading.pool.ParallelExecutor; import com.swirlds.config.api.Configuration; import com.swirlds.config.extensions.test.fixtures.TestConfigBuilder; -import com.swirlds.platform.Consensus; import com.swirlds.platform.consensus.NonAncientEventWindow; +import com.swirlds.platform.event.AncientMode; import com.swirlds.platform.event.GossipEvent; +import com.swirlds.platform.eventhandling.EventConfig_; import com.swirlds.platform.gossip.IntakeEventCounter; import com.swirlds.platform.gossip.shadowgraph.Shadowgraph; import com.swirlds.platform.gossip.shadowgraph.ShadowgraphInsertionException; import com.swirlds.platform.gossip.shadowgraph.ShadowgraphSynchronizer; +import com.swirlds.platform.gossip.sync.config.SyncConfig_; import com.swirlds.platform.metrics.SyncMetrics; import com.swirlds.platform.network.Connection; import com.swirlds.platform.system.address.AddressBook; import com.swirlds.platform.test.event.emitter.EventEmitter; import com.swirlds.platform.test.fixtures.event.IndexedEvent; +import edu.umd.cs.findbugs.annotations.NonNull; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; +import java.util.Objects; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicInteger; @@ -69,15 +72,14 @@ public class SyncNode { private int eventsEmitted = 0; private final TestingSyncManager syncManager; private final Shadowgraph shadowGraph; - private final Consensus consensus; private ParallelExecutor executor; private Connection connection; private boolean saveGeneratedEvents; private boolean shouldAcceptSync = true; private boolean reconnected = false; - private boolean sendRecInitBytes = true; + private final AncientMode ancientMode; - private long oldestGeneration; + private long expirationThreshold; private Exception syncException; private final AtomicInteger sleepAfterEventReadMillis = new AtomicInteger(0); @@ -89,20 +91,32 @@ public class SyncNode { private final PlatformContext platformContext; - public SyncNode(final int numNodes, final long nodeId, final EventEmitter eventEmitter) { - this(numNodes, nodeId, eventEmitter, new CachedPoolParallelExecutor(getStaticThreadManager(), "sync-node")); + public SyncNode( + final int numNodes, + final long nodeId, + final EventEmitter eventEmitter, + @NonNull final AncientMode ancientMode) { + + this( + numNodes, + nodeId, + eventEmitter, + new CachedPoolParallelExecutor(getStaticThreadManager(), "sync-node"), + ancientMode); } public SyncNode( final int numNodes, final long nodeId, final EventEmitter eventEmitter, - final ParallelExecutor executor) { + final ParallelExecutor executor, + @NonNull final AncientMode ancientMode) { if (executor.isMutable()) { executor.start(); } + this.ancientMode = Objects.requireNonNull(ancientMode); this.numNodes = numNodes; this.nodeId = new NodeId(nodeId); this.eventEmitter = eventEmitter; @@ -117,7 +131,10 @@ public SyncNode( // The original sync tests are incompatible with event filtering. final Configuration configuration = new TestConfigBuilder() - .withValue("sync.filterLikelyDuplicates", false) + .withValue(SyncConfig_.FILTER_LIKELY_DUPLICATES, false) + .withValue( + EventConfig_.USE_BIRTH_ROUND_ANCIENT_THRESHOLD, + ancientMode == AncientMode.BIRTH_ROUND_THRESHOLD) .getOrCreateConfig(); platformContext = TestPlatformContextBuilder.create() @@ -125,7 +142,6 @@ public SyncNode( .build(); shadowGraph = new Shadowgraph(platformContext, mock(AddressBook.class)); - consensus = mock(Consensus.class); this.executor = executor; } @@ -224,7 +240,10 @@ public ShadowgraphSynchronizer getSynchronizer() { // The original sync tests are incompatible with event filtering. final Configuration configuration = new TestConfigBuilder() - .withValue("sync.filterLikelyDuplicates", false) + .withValue(SyncConfig_.FILTER_LIKELY_DUPLICATES, false) + .withValue( + EventConfig_.USE_BIRTH_ROUND_ANCIENT_THRESHOLD, + ancientMode == AncientMode.BIRTH_ROUND_THRESHOLD) .getOrCreateConfig(); final PlatformContext platformContext = TestPlatformContextBuilder.create() @@ -237,33 +256,33 @@ public ShadowgraphSynchronizer getSynchronizer() { shadowGraph, numNodes, mock(SyncMetrics.class), - this::getConsensus, eventHandler, syncManager, mock(IntakeEventCounter.class), - executor, - sendRecInitBytes, - () -> {}); + executor); } /** *

Calls the - * {@link Shadowgraph#updateNonExpiredEventWindow(com.swirlds.platform.consensus.NonAncientEventWindow)} method and - * saves the {@code expireBelow} value for use in validation. For the purposes of these tests, the - * {@code expireBelow} value becomes the oldest non-expired generation in the shadow graph returned by - * {@link SyncNode#getOldestGeneration()} . In order words, these tests assume there are no generation reservations - * prior to the sync that occurs in the test.

+ * {@link Shadowgraph#updateEventWindow(com.swirlds.platform.consensus.NonAncientEventWindow)} method and saves the + * {@code expireBelow} value for use in validation. For the purposes of these tests, the {@code expireBelow} value + * becomes the oldest non-expired ancient indicator in the shadow graph returned by + * {@link SyncNode#getExpirationThreshold()} . In order words, these tests assume there are no reservations prior + * to the sync that occurs in the test.

* - *

The {@link SyncNode#getOldestGeneration()} value is used to determine which events should not be send to the - * peer because they are expired.

+ *

The {@link SyncNode#getExpirationThreshold()} value is used to determine which events should not be send + * to + * the peer because they are expired.

*/ - public void expireBelow(final long expireBelow) { - this.oldestGeneration = expireBelow; + public void expireBelow(final long expirationThreshold) { + this.expirationThreshold = expirationThreshold; + + final long ancientThreshold = shadowGraph.getEventWindow().getAncientThreshold(); final NonAncientEventWindow eventWindow = new NonAncientEventWindow( - 0 /* ignored by shadowgraph */, 0 /* ignored by shadowgraph */, expireBelow, GENERATION_THRESHOLD); + 0 /* ignored by shadowgraph */, ancientThreshold, expirationThreshold, ancientMode); - shadowGraph.updateNonExpiredEventWindow(eventWindow); + updateEventWindow(eventWindow); } public NodeId getNodeId() { @@ -282,6 +301,13 @@ public Shadowgraph getShadowGraph() { return shadowGraph; } + /** + * Sets the current {@link NonAncientEventWindow} for the {@link Shadowgraph}. + */ + public void updateEventWindow(@NonNull final NonAncientEventWindow eventWindow) { + shadowGraph.updateEventWindow(eventWindow); + } + public TestingSyncManager getSyncManager() { return syncManager; } @@ -326,12 +352,12 @@ public void setParallelExecutor(final ParallelExecutor executor) { this.executor = executor; } - public Consensus getConsensus() { - return consensus; + public long getCurrentAncientThreshold() { + return shadowGraph.getEventWindow().getAncientThreshold(); } - public long getOldestGeneration() { - return oldestGeneration; + public long getExpirationThreshold() { + return expirationThreshold; } public int getSleepAfterEventReadMillis() { @@ -342,14 +368,6 @@ public void setSleepAfterEventReadMillis(final int sleepAfterEventReadMillis) { this.sleepAfterEventReadMillis.set(sleepAfterEventReadMillis); } - public boolean isSendRecInitBytes() { - return sendRecInitBytes; - } - - public void setSendRecInitBytes(final boolean sendRecInitBytes) { - this.sendRecInitBytes = sendRecInitBytes; - } - public void setSynchronizerReturn(final Boolean value) { synchronizerReturn.set(value); } diff --git a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestExecutor.java b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestExecutor.java index 1aa860a4bb97..5a2c0ffb9ebd 100644 --- a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestExecutor.java +++ b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestExecutor.java @@ -17,12 +17,13 @@ package com.swirlds.platform.test.sync; import static com.swirlds.common.threading.manager.AdHocThreadManager.getStaticThreadManager; -import static org.mockito.Mockito.when; import com.swirlds.base.utility.Pair; import com.swirlds.common.test.fixtures.RandomUtils; import com.swirlds.common.threading.pool.CachedPoolParallelExecutor; import com.swirlds.common.threading.pool.ParallelExecutor; +import com.swirlds.platform.consensus.NonAncientEventWindow; +import com.swirlds.platform.event.AncientMode; import com.swirlds.platform.gossip.shadowgraph.ShadowEvent; import com.swirlds.platform.network.Connection; import com.swirlds.platform.system.address.AddressBook; @@ -63,14 +64,19 @@ public class SyncTestExecutor { private BiConsumer customInitialization; private BiConsumer graphCustomization; private BiConsumer customPreSyncConfiguration; - private BiConsumer generationDefinitions; + private BiConsumer eventWindowDefinitions; private Predicate callerAddToGraphTest; private Predicate listenerAddToGraphTest; - /** A randomly generated address book from the number of nodes in the parameters of the test. */ + private final AncientMode ancientMode; + + /** + * A randomly generated address book from the number of nodes in the parameters of the test. + */ private AddressBook addressBook; public SyncTestExecutor(final SyncTestParams params) { this.params = params; + this.ancientMode = params.getAncientMode(); this.addressBook = new RandomAddressBookGenerator() .setSize(params.getNumNetworkNodes()) .build(); @@ -89,12 +95,17 @@ public SyncTestExecutor(final SyncTestParams params) { return executor; }; callerSupplier = (factory) -> new SyncNode( - params.getNumNetworkNodes(), 0, factory.newShuffledFromSourceFactory(), callerExecutorSupplier.get()); + params.getNumNetworkNodes(), + 0, + factory.newShuffledFromSourceFactory(), + callerExecutorSupplier.get(), + ancientMode); listenerSupplier = (factory) -> new SyncNode( params.getNumNetworkNodes(), params.getNumNetworkNodes() - 1, factory.newShuffledFromSourceFactory(), - listenerExecutorSupplier.get()); + listenerExecutorSupplier.get(), + ancientMode); initialGraphCreation = (caller, listener) -> { for (final SyncNode node : List.of(caller, listener)) { @@ -105,7 +116,7 @@ public SyncTestExecutor(final SyncTestParams params) { }; customInitialization = (caller, listener) -> {}; graphCustomization = (caller, listener) -> {}; - generationDefinitions = getDefaultGenerationDefinitions(); + eventWindowDefinitions = updateDefaultEventWindow(); customPreSyncConfiguration = (caller, listener) -> {}; callerAddToGraphTest = (indexedEvent -> true); listenerAddToGraphTest = (indexedEvent -> true); @@ -177,7 +188,7 @@ private void initialize() throws IOException { *
  • Caller Graph Additions
  • *
  • Listener Graph Additions
  • * - * + *

    * Does not include graph validation. */ private void createGraphs() { @@ -187,43 +198,48 @@ private void createGraphs() { caller.generateAndAdd(params.getNumCallerEvents(), callerAddToGraphTest); listener.getEmitter().setCheckpoint(params.getNumListenerEvents()); listener.generateAndAdd(params.getNumListenerEvents(), listenerAddToGraphTest); - generationDefinitions.accept(caller, listener); + eventWindowDefinitions.accept(caller, listener); } - private BiConsumer getDefaultGenerationDefinitions() { + private BiConsumer updateDefaultEventWindow() { return (caller, listener) -> { - List callerTips = caller.getShadowGraph().getTips(); - List listenerTips = listener.getShadowGraph().getTips(); - - long listenerMinGen = - SyncTestUtils.getMinGen(listener.getShadowGraph().findAncestors(listenerTips, (e) -> true)); - long listenerMaxGen = SyncTestUtils.getMaxGen(listenerTips); - long callerMinGen = SyncTestUtils.getMinGen(caller.getShadowGraph().findAncestors(callerTips, (e) -> true)); - long callerMaxGen = SyncTestUtils.getMaxGen(callerTips); - - long listenerMinNonAncientGen = listenerMinGen; - double listenerDif = listenerMaxGen - listenerMinGen; + final List callerTips = caller.getShadowGraph().getTips(); + final List listenerTips = listener.getShadowGraph().getTips(); + + final long listenerExpiredThreshold = SyncTestUtils.getMinIndicator( + listener.getShadowGraph().findAncestors(listenerTips, (e) -> true), ancientMode); + final long listenerMaxIndicator = SyncTestUtils.getMaxIndicator(listenerTips, ancientMode); + final long callerExpiredThreshold = SyncTestUtils.getMinIndicator( + caller.getShadowGraph().findAncestors(callerTips, (e) -> true), ancientMode); + final long callerMaxIndicator = SyncTestUtils.getMaxIndicator(callerTips, ancientMode); + + long listenerAncientThreshold = listenerExpiredThreshold; + final double listenerDif = listenerMaxIndicator - listenerExpiredThreshold; if (listenerDif >= 3) { - listenerMinNonAncientGen += Math.floor(listenerDif / 3); + listenerAncientThreshold += Math.floor(listenerDif / 3); } else if (listenerDif == 2) { - listenerMinNonAncientGen++; + listenerAncientThreshold++; } - long callerMinNonAncientGen = callerMinGen; - double callerDif = callerMaxGen - callerMinGen; + long callerAncientThreshold = callerExpiredThreshold; + final double callerDif = callerMaxIndicator - callerExpiredThreshold; if (callerDif >= 3) { - callerMinNonAncientGen += Math.floor(callerDif / 3); + callerAncientThreshold += Math.floor(callerDif / 3); } else if (callerDif == 2) { - callerMinNonAncientGen++; + callerAncientThreshold++; } - when(caller.getConsensus().getMaxRoundGeneration()).thenReturn(callerMaxGen); - when(caller.getConsensus().getMinRoundGeneration()).thenReturn(callerMinGen); - when(caller.getConsensus().getMinGenerationNonAncient()).thenReturn(callerMinNonAncientGen); - - when(listener.getConsensus().getMaxRoundGeneration()).thenReturn(listenerMaxGen); - when(listener.getConsensus().getMinRoundGeneration()).thenReturn(listenerMinGen); - when(listener.getConsensus().getMinGenerationNonAncient()).thenReturn(listenerMinNonAncientGen); + caller.updateEventWindow(new NonAncientEventWindow( + ancientMode.getGenesisIndicator(), + Math.max(ancientMode.getGenesisIndicator(), callerAncientThreshold), + Math.max(ancientMode.getGenesisIndicator(), callerExpiredThreshold), + ancientMode)); + + listener.updateEventWindow(new NonAncientEventWindow( + ancientMode.getGenesisIndicator(), + Math.max(ancientMode.getGenesisIndicator(), listenerAncientThreshold), + Math.max(ancientMode.getGenesisIndicator(), listenerExpiredThreshold), + ancientMode)); }; } @@ -256,8 +272,7 @@ private void sync() throws Exception { } /** - * @param executorSupplier - * supplies the ParallelExecutor for both of the {@link SyncNode}s + * @param executorSupplier supplies the ParallelExecutor for both of the {@link SyncNode}s */ public void setExecutorSupplier(final Supplier executorSupplier) { this.callerExecutorSupplier = executorSupplier; @@ -279,21 +294,20 @@ public void setCallerExecutorSupplier(final Supplier callerExe } /** - * @param factoryConfig - * a method that configures the event factory for the particular test + * @param factoryConfig a method that configures the event factory for the particular test */ public void setFactoryConfig(final Consumer factoryConfig) { this.factoryConfig = factoryConfig; } /** - * Sets a custom generation definition function that defines what generations the caller and listener will use in + * Sets a custom event window definition function that defines what event window the caller and listener will use in * the sync. * - * @param generationDefinitions + * @param eventWindowDefinitions a function that defines the event window for the caller and listener */ - public void setGenerationDefinitions(final BiConsumer generationDefinitions) { - this.generationDefinitions = generationDefinitions; + public void setEventWindowDefinitions(final BiConsumer eventWindowDefinitions) { + this.eventWindowDefinitions = eventWindowDefinitions; } /** @@ -319,9 +333,8 @@ public void setCallerSupplier(final Function call * {@link ParallelExecutor} or an {@link EventEmitter} other than a {@link ShuffledEventEmitter}.

    * *

    Please note that is an {@link EventEmitter} other than {@link ShuffledEventEmitter} is used to create this - * node, the {@link SyncTestExecutor#setInitialGraphCreation(BiConsumer)} must be called with a compatible {@link - * BiConsumer} - * .

    + * node, the {@link SyncTestExecutor#setInitialGraphCreation(BiConsumer)} must be called with a compatible + * {@link BiConsumer} .

    */ public void setListenerSupplier(final Function listenerSupplier) { this.listenerSupplier = listenerSupplier; @@ -340,10 +353,11 @@ public void setInitialGraphCreation(final BiConsumer initial } /** - * Defines the custom initialization function to be performed after common initialization of the caller and - * listener nodes but before graph creation. - * - * For example, this could be used to turn on saving events for one or both of the node prior to any event creation. + * Defines the custom initialization function to be performed after common initialization of the caller and listener + * nodes but before graph creation. + *

    + * For example, this could be used to turn on saving events for one or both of the node prior to any event + * creation. * * @param customInitialization */ @@ -352,10 +366,10 @@ public void setCustomInitialization(final BiConsumer customI } /** - * Defines a graph customization function to be performed after initial graph creation, but before {@link - * SyncTestParams#getNumCallerEvents()} and {@link SyncTestParams#getNumListenerEvents()} - * events are added to the caller and listener shadow graphs. - * + * Defines a graph customization function to be performed after initial graph creation, but before + * {@link SyncTestParams#getNumCallerEvents()} and {@link SyncTestParams#getNumListenerEvents()} events are added to + * the caller and listener shadow graphs. + *

    * For example, this method could be used to setup the sync nodes to partition for the remaining events. * * @param graphCustomization @@ -384,9 +398,8 @@ public void setCallerAddToGraphTest(final Predicate callerAddToGra } /** - * Defines a custom predicate that determines if listener events generated after the common events should be - * added to - * the caller's shadow graph. + * Defines a custom predicate that determines if listener events generated after the common events should be added + * to the caller's shadow graph. * * @param listenerAddToGraphTest */ diff --git a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestParams.java b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestParams.java index 3673002136b1..ddfb4a984f8b 100644 --- a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestParams.java +++ b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestParams.java @@ -17,6 +17,9 @@ package com.swirlds.platform.test.sync; import com.swirlds.base.utility.ToStringBuilder; +import com.swirlds.platform.event.AncientMode; +import edu.umd.cs.findbugs.annotations.NonNull; +import java.util.Objects; /** * Data holder for parameters needed for every sync unit test. @@ -28,22 +31,30 @@ public class SyncTestParams { private final int numCallerEvents; private final int numListenerEvents; private final Long customSeed; + private final AncientMode ancientMode; public SyncTestParams( - int numNetworkNodes, int numCommonEvents, int numCallerEvents, int numListenerEvents, Long customSeed) { + int numNetworkNodes, + int numCommonEvents, + int numCallerEvents, + int numListenerEvents, + Long customSeed, + @NonNull final AncientMode ancientMode) { this.numNetworkNodes = numNetworkNodes; this.numCommonEvents = numCommonEvents; this.numCallerEvents = numCallerEvents; this.numListenerEvents = numListenerEvents; this.customSeed = customSeed; + this.ancientMode = Objects.requireNonNull(ancientMode); } public SyncTestParams( final int numNetworkNodes, final int numCommonEvents, final int numCallerEvents, - final int numListenerEvents) { - this(numNetworkNodes, numCommonEvents, numCallerEvents, numListenerEvents, null); + final int numListenerEvents, + @NonNull final AncientMode ancientMode) { + this(numNetworkNodes, numCommonEvents, numCallerEvents, numListenerEvents, null, ancientMode); } /** @@ -61,16 +72,16 @@ public int getNumCommonEvents() { } /** - * The number of events to insert into the caller's shadow graph in addition to {@link - * SyncTestParams#numCommonEvents}. + * The number of events to insert into the caller's shadow graph in addition to + * {@link SyncTestParams#numCommonEvents}. */ public int getNumCallerEvents() { return numCallerEvents; } /** - * The number of events to insert into the listener's shadow graph in addition to {@link - * SyncTestParams#numCommonEvents}. + * The number of events to insert into the listener's shadow graph in addition to + * {@link SyncTestParams#numCommonEvents}. */ public int getNumListenerEvents() { return numListenerEvents; @@ -83,6 +94,13 @@ public Long getCustomSeed() { return customSeed; } + /** + * @return the ancient mode set for this test + */ + public AncientMode getAncientMode() { + return ancientMode; + } + @Override public String toString() { return new ToStringBuilder(this) @@ -91,6 +109,7 @@ public String toString() { .append("numCallerEvents", numCallerEvents) .append("numListenerEvents", numListenerEvents) .append("customSeed", customSeed) + .append("ancientMode", ancientMode) .toString(); } } diff --git a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestUtils.java b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestUtils.java index 3c6ec00d9a1c..fca5b6665055 100644 --- a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestUtils.java +++ b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestUtils.java @@ -17,9 +17,11 @@ package com.swirlds.platform.test.sync; import com.swirlds.platform.EventStrings; +import com.swirlds.platform.event.AncientMode; import com.swirlds.platform.event.GossipEvent; import com.swirlds.platform.gossip.shadowgraph.ShadowEvent; import com.swirlds.platform.internal.EventImpl; +import edu.umd.cs.findbugs.annotations.NonNull; import java.util.Collection; import java.util.List; import java.util.Set; @@ -41,19 +43,19 @@ public static void printTipSet(final String nodeName, final SyncNode node) { node.getShadowGraph().getTips().forEach(tip -> System.out.println(EventStrings.toMediumString(tip.getEvent()))); } - public static long getMaxGen(final List tips) { - long maxGen = 0; - for (ShadowEvent tip : tips) { - maxGen = Math.max(tip.getEvent().getGeneration(), maxGen); + public static long getMaxIndicator(final List tips, @NonNull final AncientMode ancientMode) { + long maxIndicator = ancientMode.getGenesisIndicator(); + for (final ShadowEvent tip : tips) { + maxIndicator = Math.max(tip.getEvent().getBaseEvent().getAncientIndicator(ancientMode), maxIndicator); } - return maxGen; + return maxIndicator; } - public static long getMinGen(final Set events) { - long minGen = Long.MAX_VALUE; - for (ShadowEvent event : events) { - minGen = Math.min(event.getEvent().getGeneration(), minGen); + public static long getMinIndicator(@NonNull final Set events, @NonNull final AncientMode ancientMode) { + long minIndicator = Long.MAX_VALUE; + for (final ShadowEvent event : events) { + minIndicator = Math.min(event.getEvent().getBaseEvent().getAncientIndicator(ancientMode), minIndicator); } - return minGen == Long.MAX_VALUE ? 0 : minGen; + return minIndicator == Long.MAX_VALUE ? ancientMode.getGenesisIndicator() : minIndicator; } } diff --git a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestUtilsTest.java b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestUtilsTest.java index 92784df33303..81279b370971 100644 --- a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestUtilsTest.java +++ b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTestUtilsTest.java @@ -16,11 +16,11 @@ package com.swirlds.platform.test.sync; +import static com.swirlds.platform.event.AncientMode.GENERATION_THRESHOLD; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; -import com.swirlds.platform.consensus.GraphGenerations; -import com.swirlds.platform.gossip.shadowgraph.Generations; +import com.swirlds.platform.consensus.NonAncientEventWindow; import com.swirlds.platform.gossip.shadowgraph.ShadowEvent; import com.swirlds.platform.gossip.shadowgraph.SyncUtils; import java.util.HashSet; @@ -55,10 +55,10 @@ void testUnknownNonAncient() { knownSet.add(e3); knownSet.add(e1); - final GraphGenerations generations = new Generations(0, 1, 3); + final NonAncientEventWindow eventWindow = new NonAncientEventWindow(0, 1, 0, GENERATION_THRESHOLD); final Predicate unknownNonAncient = - SyncUtils.unknownNonAncient(knownSet, generations, generations); + SyncUtils.unknownNonAncient(knownSet, eventWindow, eventWindow, GENERATION_THRESHOLD); assertFalse(unknownNonAncient.test(e1), "e1 is both ancient and known, should be false"); assertFalse(unknownNonAncient.test(e2), "e2 is ancient, should be false"); diff --git a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTests.java b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTests.java index 06de480d10c0..7e23542e0b2f 100644 --- a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTests.java +++ b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncTests.java @@ -19,6 +19,8 @@ import static com.swirlds.common.test.fixtures.io.ResourceLoader.loadLog4jContext; import static com.swirlds.common.threading.manager.AdHocThreadManager.getStaticThreadManager; import static com.swirlds.common.utility.CompareTo.max; +import static com.swirlds.platform.consensus.ConsensusConstants.ROUND_FIRST; +import static com.swirlds.platform.event.AncientMode.BIRTH_ROUND_THRESHOLD; import static com.swirlds.platform.event.AncientMode.GENERATION_THRESHOLD; import static com.swirlds.platform.test.fixtures.event.EventUtils.integerPowerDistribution; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -26,7 +28,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; -import static org.mockito.Mockito.when; import com.swirlds.common.constructable.ConstructableRegistry; import com.swirlds.common.constructable.ConstructableRegistryException; @@ -37,10 +38,9 @@ import com.swirlds.common.threading.pool.ParallelExecutionException; import com.swirlds.common.threading.pool.ParallelExecutor; import com.swirlds.config.extensions.test.fixtures.TestConfigBuilder; -import com.swirlds.platform.consensus.GraphGenerations; import com.swirlds.platform.consensus.NonAncientEventWindow; +import com.swirlds.platform.event.AncientMode; import com.swirlds.platform.gossip.shadowgraph.ShadowEvent; -import com.swirlds.platform.internal.EventImpl; import com.swirlds.platform.system.events.EventConstants; import com.swirlds.platform.test.event.emitter.EventEmitterFactory; import com.swirlds.platform.test.event.emitter.StandardEventEmitter; @@ -50,6 +50,7 @@ import com.swirlds.platform.test.graph.OtherParentMatrixFactory; import com.swirlds.platform.test.graph.PartitionedGraphCreator; import com.swirlds.platform.test.graph.SplitForkGraphCreator; +import edu.umd.cs.findbugs.annotations.NonNull; import java.io.FileNotFoundException; import java.net.SocketException; import java.util.List; @@ -60,9 +61,7 @@ import java.util.stream.IntStream; import java.util.stream.Stream; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -72,25 +71,38 @@ public class SyncTests { private static final boolean platformLoggingEnabled = true; + private static Stream bothAncientModes() { + return Stream.of(Arguments.of(GENERATION_THRESHOLD), Arguments.of(BIRTH_ROUND_THRESHOLD)); + } + private static Stream fourNodeGraphParams() { return Stream.of( - Arguments.of(new SyncTestParams(4, 100, 20, 0)), - Arguments.of(new SyncTestParams(4, 100, 0, 20)), - Arguments.of(new SyncTestParams(4, 100, 20, 20))); + Arguments.of(new SyncTestParams(4, 100, 20, 0, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 0, 20, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 20, 20, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 20, 0, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 0, 20, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 20, 20, BIRTH_ROUND_THRESHOLD))); } private static Stream tenNodeGraphParams() { return Stream.of( - Arguments.of(new SyncTestParams(10, 100, 50, 0)), - Arguments.of(new SyncTestParams(10, 100, 0, 50)), - Arguments.of(new SyncTestParams(10, 100, 50, 50))); + Arguments.of(new SyncTestParams(10, 100, 50, 0, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 0, 50, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 50, 50, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 50, 0, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 0, 50, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 50, 50, BIRTH_ROUND_THRESHOLD))); } private static Stream tenNodeBigGraphParams() { return Stream.of( - Arguments.of(new SyncTestParams(10, 1000, 2000, 0)), - Arguments.of(new SyncTestParams(10, 1000, 0, 2000)), - Arguments.of(new SyncTestParams(10, 1000, 2000, 2000))); + Arguments.of(new SyncTestParams(10, 1000, 2000, 0, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 1000, 0, 2000, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 1000, 2000, 2000, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 1000, 2000, 0, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 1000, 0, 2000, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 1000, 2000, 2000, BIRTH_ROUND_THRESHOLD))); } /** @@ -98,11 +110,16 @@ private static Stream tenNodeBigGraphParams() { */ private static Stream simpleFourNodeGraphParams() { return Stream.of( - Arguments.of(new SyncTestParams(4, 10, 0, 0)), - Arguments.of(new SyncTestParams(4, 10, 1, 0)), - Arguments.of(new SyncTestParams(4, 10, 0, 1)), - Arguments.of(new SyncTestParams(4, 10, 1, 1)), - Arguments.of(new SyncTestParams(4, 10, 8, 1))); + Arguments.of(new SyncTestParams(4, 10, 0, 0, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 1, 0, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 0, 1, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 1, 1, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 8, 1, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 0, 0, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 1, 0, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 0, 1, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 1, 1, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 8, 1, BIRTH_ROUND_THRESHOLD))); } /** @@ -110,10 +127,14 @@ private static Stream simpleFourNodeGraphParams() { */ private static Stream edgeCaseGraphParams() { return Stream.of( - Arguments.of(new SyncTestParams(10, 0, 20, 0)), - Arguments.of(new SyncTestParams(10, 0, 0, 20)), - Arguments.of(new SyncTestParams(10, 0, 0, 0)), - Arguments.of(new SyncTestParams(10, 20, 0, 0))); + Arguments.of(new SyncTestParams(10, 0, 20, 0, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 0, 20, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 0, 0, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 20, 0, 0, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 20, 0, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 0, 20, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 0, 0, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 20, 0, 0, BIRTH_ROUND_THRESHOLD))); } /** @@ -122,15 +143,25 @@ private static Stream edgeCaseGraphParams() { private static Stream partitionedGraphParams() { return Stream.of( // Partitioned graphs with common events - Arguments.of(new SyncTestParams(10, 30, 5, 5)), - Arguments.of(new SyncTestParams(10, 30, 5, 5)), - Arguments.of(new SyncTestParams(20, 200, 100, 0)), - Arguments.of(new SyncTestParams(20, 200, 0, 100)), - Arguments.of(new SyncTestParams(20, 200, 100, 100)), + Arguments.of(new SyncTestParams(10, 30, 5, 5, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 30, 5, 5, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(20, 200, 100, 0, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(20, 200, 0, 100, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(20, 200, 100, 100, GENERATION_THRESHOLD)), + // Partitioned graphs with no common events + Arguments.of(new SyncTestParams(20, 0, 100, 0, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(20, 0, 0, 100, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(20, 0, 100, 100, GENERATION_THRESHOLD)), + // Partitioned graphs with common events + Arguments.of(new SyncTestParams(10, 30, 5, 5, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 30, 5, 5, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(20, 200, 100, 0, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(20, 200, 0, 100, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(20, 200, 100, 100, BIRTH_ROUND_THRESHOLD)), // Partitioned graphs with no common events - Arguments.of(new SyncTestParams(20, 0, 100, 0)), - Arguments.of(new SyncTestParams(20, 0, 0, 100)), - Arguments.of(new SyncTestParams(20, 0, 100, 100))); + Arguments.of(new SyncTestParams(20, 0, 100, 0, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(20, 0, 0, 100, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(20, 0, 100, 100, BIRTH_ROUND_THRESHOLD))); } /** @@ -139,25 +170,40 @@ private static Stream partitionedGraphParams() { */ private static Stream exceptionParams() { return Stream.of( - Arguments.of(new SyncTestParams(10, 100, 50, 50), 1, 1), - Arguments.of(new SyncTestParams(10, 100, 50, 50), 1, 2), - Arguments.of(new SyncTestParams(10, 100, 50, 50), 2, 1), - Arguments.of(new SyncTestParams(10, 100, 50, 50), 2, 2), - Arguments.of(new SyncTestParams(10, 100, 50, 50), 3, 1), - Arguments.of(new SyncTestParams(10, 100, 50, 50), 3, 2)); + Arguments.of(new SyncTestParams(10, 100, 50, 50, GENERATION_THRESHOLD), 1, 1), + Arguments.of(new SyncTestParams(10, 100, 50, 50, GENERATION_THRESHOLD), 1, 2), + Arguments.of(new SyncTestParams(10, 100, 50, 50, GENERATION_THRESHOLD), 2, 1), + Arguments.of(new SyncTestParams(10, 100, 50, 50, GENERATION_THRESHOLD), 2, 2), + Arguments.of(new SyncTestParams(10, 100, 50, 50, GENERATION_THRESHOLD), 3, 1), + Arguments.of(new SyncTestParams(10, 100, 50, 50, GENERATION_THRESHOLD), 3, 2), + Arguments.of(new SyncTestParams(10, 100, 50, 50, BIRTH_ROUND_THRESHOLD), 1, 1), + Arguments.of(new SyncTestParams(10, 100, 50, 50, BIRTH_ROUND_THRESHOLD), 1, 2), + Arguments.of(new SyncTestParams(10, 100, 50, 50, BIRTH_ROUND_THRESHOLD), 2, 1), + Arguments.of(new SyncTestParams(10, 100, 50, 50, BIRTH_ROUND_THRESHOLD), 2, 2), + Arguments.of(new SyncTestParams(10, 100, 50, 50, BIRTH_ROUND_THRESHOLD), 3, 1), + Arguments.of(new SyncTestParams(10, 100, 50, 50, BIRTH_ROUND_THRESHOLD), 3, 2)); } private static Stream splitForkParams() { return Stream.of( // This seed makes the caller send the whole graph, should not be the case once we change the tip // definition - Arguments.of(new SyncTestParams(4, 100, 20, 1, 4956163591276672768L)), - Arguments.of(new SyncTestParams(4, 100, 20, 1)), - Arguments.of(new SyncTestParams(4, 100, 1, 20)), - Arguments.of(new SyncTestParams(4, 100, 20, 20)), - Arguments.of(new SyncTestParams(10, 100, 50, 50)), - Arguments.of(new SyncTestParams(10, 100, 100, 50)), - Arguments.of(new SyncTestParams(10, 100, 50, 100))); + Arguments.of(new SyncTestParams(4, 100, 20, 1, 4956163591276672768L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 20, 1, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 1, 20, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 20, 20, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 50, 50, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 100, 50, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 50, 100, GENERATION_THRESHOLD)), + // This seed makes the caller send the whole graph, should not be the case once we change the tip + // definition + Arguments.of(new SyncTestParams(4, 100, 20, 1, 4956163591276672768L, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 20, 1, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 1, 20, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 20, 20, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 50, 50, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 100, 50, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 50, 100, BIRTH_ROUND_THRESHOLD))); } private static Stream splitForkParamsBreakingSeed() { @@ -165,56 +211,82 @@ private static Stream splitForkParamsBreakingSeed() { // This seed used to make the caller send the whole graph back when the definition of a tip was an // event with no children (self or other). Now that the definition of a tip is an event with no // self-child, this seed passes. - Arguments.of(new SyncTestParams(4, 100, 20, 1, 4956163591276672768L))); + Arguments.of(new SyncTestParams(4, 100, 20, 1, 4956163591276672768L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 20, 1, 4956163591276672768L, BIRTH_ROUND_THRESHOLD))); } private static Stream largeGraphParams() { return Stream.of( - Arguments.of(new SyncTestParams(10, 1000, 500, 200)), - Arguments.of(new SyncTestParams(10, 1000, 200, 500)), - Arguments.of(new SyncTestParams(10, 1000, 500, 500))); + Arguments.of(new SyncTestParams(10, 1000, 500, 200, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 1000, 200, 500, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 1000, 500, 500, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 1000, 500, 200, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 1000, 200, 500, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 1000, 500, 500, BIRTH_ROUND_THRESHOLD))); } private static Stream noCommonEventsParams() { return Stream.of( - Arguments.of(new SyncTestParams(4, 0, 50, 20)), - Arguments.of(new SyncTestParams(4, 0, 20, 50)), - Arguments.of(new SyncTestParams(4, 0, 50, 50)), - Arguments.of(new SyncTestParams(10, 0, 500, 200)), - Arguments.of(new SyncTestParams(10, 0, 200, 500)), - Arguments.of(new SyncTestParams(10, 0, 500, 500))); + Arguments.of(new SyncTestParams(4, 0, 50, 20, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 0, 20, 50, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 0, 50, 50, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 500, 200, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 200, 500, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 500, 500, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 0, 50, 20, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 0, 20, 50, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 0, 50, 50, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 500, 200, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 200, 500, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 500, 500, BIRTH_ROUND_THRESHOLD))); } private static Stream tipsChangeBreakingSeed() { return Stream.of( - Arguments.of(new SyncTestParams(10, 0, 20, 0, 6238590233436833292L)), - Arguments.of(new SyncTestParams(4, 10, 8, 1, 8824331216639179768L)), - Arguments.of(new SyncTestParams(10, 0, 0, 20, -909134053413382981L)), - Arguments.of(new SyncTestParams(10, 0, 0, 20, 5236225801504915258L)), - Arguments.of(new SyncTestParams(4, 10, 1, 1, -3204404663467002969L)), - Arguments.of(new SyncTestParams(10, 0, 0, 20, -4776092416980912346L))); + Arguments.of(new SyncTestParams(10, 0, 20, 0, 6238590233436833292L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 8, 1, 8824331216639179768L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 0, 20, -909134053413382981L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 0, 20, 5236225801504915258L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 1, 1, -3204404663467002969L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 0, 20, -4776092416980912346L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 20, 0, 6238590233436833292L, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 8, 1, 8824331216639179768L, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 0, 20, -909134053413382981L, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 0, 20, 5236225801504915258L, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 10, 1, 1, -3204404663467002969L, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 0, 0, 20, -4776092416980912346L, BIRTH_ROUND_THRESHOLD))); } private static Stream simpleGraphBreakingSeed() { return Stream.of( - Arguments.of(new SyncTestParams(4, 100, 20, 20, -5979073137457357235L)), - Arguments.of(new SyncTestParams(10, 100, 50, 50, 1861589538493329478L))); + Arguments.of(new SyncTestParams(4, 100, 20, 20, -5979073137457357235L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 50, 50, 1861589538493329478L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(4, 100, 20, 20, -5979073137457357235L, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 50, 50, 1861589538493329478L, BIRTH_ROUND_THRESHOLD))); } private static Stream tipExpiresBreakingSeed() { return Stream.of( - Arguments.of(new SyncTestParams(10, 100, 0, 50, 1152284535185134815L)), - Arguments.of(new SyncTestParams(10, 100, 0, 50, -8664085824668001150L))); + Arguments.of(new SyncTestParams(10, 100, 0, 50, 1152284535185134815L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 0, 50, -8664085824668001150L, GENERATION_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 0, 50, 1152284535185134815L, BIRTH_ROUND_THRESHOLD)), + Arguments.of(new SyncTestParams(10, 100, 0, 50, -8664085824668001150L, BIRTH_ROUND_THRESHOLD))); } private static Stream requiredEventsExpire() { return Stream.of( - Arguments.of(1, new SyncTestParams(10, 100, 0, 1000)), - Arguments.of(1, new SyncTestParams(10, 200, 100, 1000)), - Arguments.of(1, new SyncTestParams(10, 200, 200, 1000)), - Arguments.of(2, new SyncTestParams(10, 100, 0, 1000)), - Arguments.of(2, new SyncTestParams(10, 200, 100, 1000)), - Arguments.of(2, new SyncTestParams(10, 200, 200, 1000))); + Arguments.of(1, new SyncTestParams(10, 100, 0, 1000, GENERATION_THRESHOLD)), + Arguments.of(1, new SyncTestParams(10, 200, 100, 1000, GENERATION_THRESHOLD)), + Arguments.of(1, new SyncTestParams(10, 200, 200, 1000, GENERATION_THRESHOLD)), + Arguments.of(2, new SyncTestParams(10, 100, 0, 1000, GENERATION_THRESHOLD)), + Arguments.of(2, new SyncTestParams(10, 200, 100, 1000, GENERATION_THRESHOLD)), + Arguments.of(2, new SyncTestParams(10, 200, 200, 1000, GENERATION_THRESHOLD)), + Arguments.of(1, new SyncTestParams(10, 100, 0, 1000, BIRTH_ROUND_THRESHOLD)), + Arguments.of(1, new SyncTestParams(10, 200, 100, 1000, BIRTH_ROUND_THRESHOLD)), + Arguments.of(1, new SyncTestParams(10, 200, 200, 1000, BIRTH_ROUND_THRESHOLD)), + Arguments.of(2, new SyncTestParams(10, 100, 0, 1000, BIRTH_ROUND_THRESHOLD)), + Arguments.of(2, new SyncTestParams(10, 200, 100, 1000, BIRTH_ROUND_THRESHOLD)), + Arguments.of(2, new SyncTestParams(10, 200, 200, 1000, BIRTH_ROUND_THRESHOLD))); } @BeforeAll @@ -249,7 +321,8 @@ void simpleGraph(final SyncTestParams params) throws Exception { executor.execute(); - SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertOnlyRequiredEventsTransferred( + executor.getCaller(), executor.getListener(), params.getAncientMode()); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } @@ -264,13 +337,12 @@ void skipInit(final SyncTestParams params) throws Exception { executor.setGraphCustomization((caller, listener) -> { caller.setSaveGeneratedEvents(true); listener.setSaveGeneratedEvents(true); - caller.setSendRecInitBytes(false); - listener.setSendRecInitBytes(false); }); executor.execute(); - SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertOnlyRequiredEventsTransferred( + executor.getCaller(), executor.getListener(), params.getAncientMode()); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } @@ -282,16 +354,20 @@ void skipInit(final SyncTestParams params) throws Exception { void forkingGraph(final SyncTestParams params) throws Exception { final SyncTestExecutor executor = new SyncTestExecutor(params); - executor.setCallerSupplier( - (factory) -> new SyncNode(params.getNumNetworkNodes(), 0, factory.newForkingShuffledGenerator())); + executor.setCallerSupplier((factory) -> new SyncNode( + params.getNumNetworkNodes(), 0, factory.newForkingShuffledGenerator(), params.getAncientMode())); executor.setListenerSupplier((factory) -> new SyncNode( - params.getNumNetworkNodes(), params.getNumNetworkNodes() - 1, factory.newForkingShuffledGenerator())); + params.getNumNetworkNodes(), + params.getNumNetworkNodes() - 1, + factory.newForkingShuffledGenerator(), + params.getAncientMode())); executor.execute(); // Some extra events could be transferred in the case of a split fork graph. This is explicitly tested in // splitForkGraph() - SyncValidator.assertRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertRequiredEventsTransferred( + executor.getCaller(), executor.getListener(), params.getAncientMode()); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } @@ -307,10 +383,13 @@ void splitForkGraph(final SyncTestParams params) throws Exception { final int callerOtherParent = 1; final int listenerOtherParent = 2; - executor.setCallerSupplier( - (factory) -> new SyncNode(params.getNumNetworkNodes(), 0, factory.newStandardEmitter())); + executor.setCallerSupplier((factory) -> + new SyncNode(params.getNumNetworkNodes(), 0, factory.newStandardEmitter(), params.getAncientMode())); executor.setListenerSupplier((factory) -> new SyncNode( - params.getNumNetworkNodes(), params.getNumNetworkNodes() - 1, factory.newStandardEmitter())); + params.getNumNetworkNodes(), + params.getNumNetworkNodes() - 1, + factory.newStandardEmitter(), + params.getAncientMode())); executor.setInitialGraphCreation((caller, listener) -> { caller.generateAndAdd(params.getNumCommonEvents()); @@ -332,7 +411,8 @@ void splitForkGraph(final SyncTestParams params) throws Exception { // In split fork graphs, some extra events will be sent because each node has a different tip for the same // creator, causing each to think the other does not have any ancestors of that creator's event when they in // fact do. - SyncValidator.assertRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertRequiredEventsTransferred( + executor.getCaller(), executor.getListener(), params.getAncientMode()); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } @@ -352,10 +432,13 @@ void partitionedGraph(final SyncTestParams params) throws Exception { .boxed() .collect(Collectors.toList()); - executor.setCallerSupplier( - (factory) -> new SyncNode(params.getNumNetworkNodes(), 0, factory.newStandardEmitter())); + executor.setCallerSupplier((factory) -> + new SyncNode(params.getNumNetworkNodes(), 0, factory.newStandardEmitter(), params.getAncientMode())); executor.setListenerSupplier((factory) -> new SyncNode( - params.getNumNetworkNodes(), params.getNumNetworkNodes() - 1, factory.newStandardEmitter())); + params.getNumNetworkNodes(), + params.getNumNetworkNodes() - 1, + factory.newStandardEmitter(), + params.getAncientMode())); executor.setInitialGraphCreation((caller, listener) -> { caller.generateAndAdd(params.getNumCommonEvents()); @@ -372,23 +455,8 @@ void partitionedGraph(final SyncTestParams params) throws Exception { executor.execute(); - SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener()); - SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); - } - - /** - * Test a sync where the listener rejects the sync. - */ - @ParameterizedTest - @MethodSource({"simpleFourNodeGraphParams", "fourNodeGraphParams", "tenNodeGraphParams", "edgeCaseGraphParams"}) - void testSyncRejected(final SyncTestParams params) throws Exception { - final SyncTestExecutor executor = new SyncTestExecutor(params); - - executor.setCustomInitialization((caller, listener) -> listener.setCanAcceptSync(false)); - - executor.execute(); - - SyncValidator.assertNoEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertOnlyRequiredEventsTransferred( + executor.getCaller(), executor.getListener(), params.getAncientMode()); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } @@ -406,13 +474,14 @@ void testNoBooleansReceived(final SyncTestParams params) { listenerExecutor.start(); // Setup parallel executors - executor.setCallerSupplier((factory) -> - new SyncNode(params.getNumNetworkNodes(), 0, factory.newShuffledEmitter(), callerExecutor)); + executor.setCallerSupplier((factory) -> new SyncNode( + params.getNumNetworkNodes(), 0, factory.newShuffledEmitter(), callerExecutor, params.getAncientMode())); executor.setListenerSupplier((factory) -> new SyncNode( params.getNumNetworkNodes(), params.getNumNetworkNodes() - 1, factory.newShuffledEmitter(), - listenerExecutor)); + listenerExecutor, + params.getAncientMode())); assertThrows(ParallelExecutionException.class, executor::execute, "Unexpected exception during sync."); @@ -437,8 +506,8 @@ void testCallerExceptionDuringSyncPhase(final SyncTestParams params, final int p throw new SocketException(); }); - executor.setCallerSupplier((factory) -> - new SyncNode(params.getNumNetworkNodes(), 0, factory.newShuffledEmitter(), callerExecutor)); + executor.setCallerSupplier((factory) -> new SyncNode( + params.getNumNetworkNodes(), 0, factory.newShuffledEmitter(), callerExecutor, params.getAncientMode())); runExceptionDuringSyncPhase(phaseToThrowIn, taskNum, executor); } @@ -460,8 +529,12 @@ void testListenerExceptionDuringSyncPhase(final SyncTestParams params, final int throw new SocketException(); }); - executor.setListenerSupplier((factory) -> - new SyncNode(params.getNumNetworkNodes(), 0, factory.newShuffledEmitter(), listenerExecutor)); + executor.setListenerSupplier((factory) -> new SyncNode( + params.getNumNetworkNodes(), + 0, + factory.newShuffledEmitter(), + listenerExecutor, + params.getAncientMode())); runExceptionDuringSyncPhase(phaseToThrowIn, taskNum, executor); } @@ -527,62 +600,80 @@ void testUnknownCreator(final SyncTestParams params) throws Exception { executor.execute(); - SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertOnlyRequiredEventsTransferred( + executor.getCaller(), executor.getListener(), params.getAncientMode()); } /** * Tests fallen behind detection works */ - @Test - void fallenBehind() throws Exception { - final SyncTestParams params = new SyncTestParams(4, 100, 20, 20); + @ParameterizedTest + @MethodSource("bothAncientModes") + void fallenBehind(@NonNull final AncientMode ancientMode) throws Exception { + final SyncTestParams params = new SyncTestParams(4, 100, 20, 20, ancientMode); - final long callerMinGen = 100; - final long callerMaxGen = 200; - final long listenerMinGen = 300; - final long listenerMaxGen = 400; + final long callerExpiredThreshold = 100; + final long callerMaximumIndicator = 200; + final long listenerExpiredThreshold = 300; + final long listenerMaximumIndicator = 400; - runFallenBehindTest(params, callerMinGen, callerMaxGen, listenerMinGen, listenerMaxGen); + runFallenBehindTest( + params, + callerExpiredThreshold, + callerMaximumIndicator, + listenerExpiredThreshold, + listenerMaximumIndicator); } /** * Tests fallen behind detection works with one node at genesis */ - @Test - void fallenBehindAtGenesis() throws Exception { - final SyncTestParams params = new SyncTestParams(4, 0, 1, 100); + @ParameterizedTest + @MethodSource("bothAncientModes") + void fallenBehindAtGenesis(@NonNull final AncientMode ancientMode) throws Exception { + final SyncTestParams params = new SyncTestParams(4, 0, 1, 100, ancientMode); - final long callerMinGen = GraphGenerations.FIRST_GENERATION; - final long callerMaxGen = GraphGenerations.FIRST_GENERATION; - final long listenerMinGen = 200; - final long listenerMaxGen = 300; + final long callerExpiredThreshold = ancientMode.getGenesisIndicator(); + final long callerMaximumIndicator = ancientMode.getGenesisIndicator(); + final long listenerAncientThreshold = 200; + final long listenerMaximumIndicator = 300; - runFallenBehindTest(params, callerMinGen, callerMaxGen, listenerMinGen, listenerMaxGen); + runFallenBehindTest( + params, + callerExpiredThreshold, + callerMaximumIndicator, + listenerAncientThreshold, + listenerMaximumIndicator); } private void runFallenBehindTest( final SyncTestParams params, - final long callerMinGen, - final long callerMaxGen, - final long listenerMinGen, - final long listenerMaxGen) + final long callerExpiredThreshold, + final long callerMaximumIndicator, + final long listenerExpiredThreshold, + final long listenerMaximumIndicator) throws Exception { - assertTrue(callerMinGen <= callerMaxGen, "Caller generations provided do not represent a fallen behind node."); assertTrue( - listenerMinGen <= listenerMaxGen, - "Listener generations provided do not represent a fallen behind node."); + callerExpiredThreshold <= callerMaximumIndicator, + "Caller event window provided does not represent a fallen behind node."); + assertTrue( + listenerExpiredThreshold <= listenerMaximumIndicator, + "Listener event window provided does not represent a fallen behind node."); assertTrue( - callerMaxGen < listenerMinGen || listenerMaxGen < callerMinGen, - "Generations provided do not represent a fallen behind node."); + callerMaximumIndicator < listenerExpiredThreshold || listenerMaximumIndicator < callerExpiredThreshold, + "Event window provided does not represent a fallen behind node."); - final boolean callerFallenBehind = callerMaxGen < listenerMinGen; + final boolean callerFallenBehind = callerMaximumIndicator < listenerExpiredThreshold; final SyncTestExecutor executor = new SyncTestExecutor(params); - executor.setCallerSupplier( - (factory) -> new SyncNode(params.getNumNetworkNodes(), 0, factory.newStandardEmitter())); + executor.setCallerSupplier((factory) -> + new SyncNode(params.getNumNetworkNodes(), 0, factory.newStandardEmitter(), params.getAncientMode())); executor.setListenerSupplier((factory) -> new SyncNode( - params.getNumNetworkNodes(), params.getNumNetworkNodes() - 1, factory.newStandardEmitter())); + params.getNumNetworkNodes(), + params.getNumNetworkNodes() - 1, + factory.newStandardEmitter(), + params.getAncientMode())); executor.setInitialGraphCreation((caller, listener) -> { caller.setSaveGeneratedEvents(true); @@ -590,21 +681,17 @@ private void runFallenBehindTest( }); executor.setCustomPreSyncConfiguration((c, l) -> { - when(c.getConsensus().getMinGenerationNonAncient()).thenReturn(callerMinGen); - when(c.getConsensus().getMaxRoundGeneration()).thenReturn(callerMaxGen); c.getShadowGraph() - .updateNonExpiredEventWindow(new NonAncientEventWindow( + .updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, - 0 /* ignored by shadowgraph */, - callerMinGen, + callerMaximumIndicator, + callerExpiredThreshold, GENERATION_THRESHOLD)); - when(l.getConsensus().getMinGenerationNonAncient()).thenReturn(listenerMinGen); - when(l.getConsensus().getMaxRoundGeneration()).thenReturn(listenerMaxGen); l.getShadowGraph() - .updateNonExpiredEventWindow(new NonAncientEventWindow( - 0 /* ignored by shadowgraph */, + .updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, - listenerMinGen, + listenerMaximumIndicator, + listenerExpiredThreshold, GENERATION_THRESHOLD)); }); @@ -618,53 +705,70 @@ private void runFallenBehindTest( SyncValidator.assertNoEventsReceived("caller", executor.getCaller()); } - @Test - void testBarelyNotFallenBehind() throws Exception { - final SyncTestParams params = new SyncTestParams(4, 200, 200, 0); + @ParameterizedTest + @MethodSource("bothAncientModes") + void testBarelyNotFallenBehind(@NonNull final AncientMode ancientMode) throws Exception { + final SyncTestParams params = new SyncTestParams(4, 200, 200, 0, ancientMode); final SyncTestExecutor executor = new SyncTestExecutor(params); - executor.setGenerationDefinitions((caller, listener) -> { - long listenerMaxGen = - SyncTestUtils.getMaxGen(listener.getShadowGraph().getTips()); - // make the min non-ancient gen slightly below the max gen - long listenerMinNonAncient = listenerMaxGen - (listenerMaxGen / 10); - long listenerMinGen = SyncTestUtils.getMinGen(listener.getShadowGraph() - .findAncestors(listener.getShadowGraph().getTips(), (e) -> true)); - - // Expire everything below the listener's min non-ancient gen on the caller - // so that the listener's maxGen == caller's min non-ancient gen - caller.expireBelow(listenerMinNonAncient); - - long callerMaxGen = SyncTestUtils.getMaxGen(caller.getShadowGraph().getTips()); - // make the min non-ancient gen slightly below the max gen - long callerMinNonAncient = callerMaxGen - (callerMaxGen / 10); - long callerMinGen = SyncTestUtils.getMinGen(caller.getShadowGraph() - .findAncestors(caller.getShadowGraph().getTips(), (e) -> true)); - - assertEquals(listenerMinNonAncient, callerMinGen, "listener max gen and caller min gen should be equal."); - - when(listener.getConsensus().getMaxRoundGeneration()).thenReturn(listenerMaxGen); - when(listener.getConsensus().getMinGenerationNonAncient()).thenReturn(listenerMinNonAncient); - when(listener.getConsensus().getMinRoundGeneration()).thenReturn(listenerMinGen); - when(caller.getConsensus().getMaxRoundGeneration()).thenReturn(callerMaxGen); - when(caller.getConsensus().getMinGenerationNonAncient()).thenReturn(callerMinNonAncient); - when(caller.getConsensus().getMinRoundGeneration()).thenReturn(callerMinGen); + executor.setEventWindowDefinitions((caller, listener) -> { + long listenerMaxIndicator = + SyncTestUtils.getMaxIndicator(listener.getShadowGraph().getTips(), ancientMode); + // make the min non-ancient indicator slightly below the max indicator + long listenerNonAncientThreshold = listenerMaxIndicator - (listenerMaxIndicator / 10); + long listenerMinIndicator = SyncTestUtils.getMinIndicator( + listener.getShadowGraph() + .findAncestors(listener.getShadowGraph().getTips(), (e) -> true), + ancientMode); + + // Expire everything below the listener's min non-ancient indicator on the caller + // so that the listener's maxIndicator == caller's min non-ancient indicator + caller.expireBelow(listenerNonAncientThreshold); + + long callerMaxIndicator = + SyncTestUtils.getMaxIndicator(caller.getShadowGraph().getTips(), ancientMode); + // make the min non-ancient indicator slightly below the max indicator + long callerNonAncientThreshold = callerMaxIndicator - (callerMaxIndicator / 10); + long callerMinIndicator = SyncTestUtils.getMinIndicator( + caller.getShadowGraph() + .findAncestors(caller.getShadowGraph().getTips(), (e) -> true), + ancientMode); + + assertEquals( + listenerNonAncientThreshold, + callerMinIndicator, + "listener max indicator and caller min indicator should be equal."); + + listener.getShadowGraph() + .updateEventWindow(new NonAncientEventWindow( + ROUND_FIRST /* ignored */, + Math.max(ancientMode.getGenesisIndicator(), listenerNonAncientThreshold), + Math.max(ancientMode.getGenesisIndicator(), listenerMinIndicator), + ancientMode)); + + caller.getShadowGraph() + .updateEventWindow(new NonAncientEventWindow( + ROUND_FIRST /* ignored */, + Math.max(ancientMode.getGenesisIndicator(), callerNonAncientThreshold), + Math.max(ancientMode.getGenesisIndicator(), callerMinIndicator), + ancientMode)); }); executor.execute(); - SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener(), ancientMode); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } /** * Verifies that even if events are expired right before sending, they are still sent. */ - @Test - void testSendExpiredEvents() throws Exception { - final SyncTestParams params = new SyncTestParams(4, 20, 10, 0); + @ParameterizedTest + @MethodSource("bothAncientModes") + void testSendExpiredEvents(@NonNull final AncientMode ancientMode) throws Exception { + final SyncTestParams params = new SyncTestParams(4, 20, 10, 0, ancientMode); final SyncTestExecutor executor = new SyncTestExecutor(params); - final AtomicLong genToExpire = new AtomicLong(EventConstants.GENERATION_UNDEFINED); + final AtomicLong indicatorToExpire = new AtomicLong(ancientMode.getGenesisIndicator() - 1); // before phase 3, expire all events so that expired events are sent executor.setCallerExecutorSupplier(() -> new SyncPhaseParallelExecutor( @@ -681,20 +785,20 @@ void testSendExpiredEvents() throws Exception { // Expire the events from the shadow graph final NonAncientEventWindow eventWindow = new NonAncientEventWindow( 0 /* ignored by shadowgraph */, - 0 /* ignored by shadowgraph */, - genToExpire.get() + 1, - GENERATION_THRESHOLD); - executor.getCaller().getShadowGraph().updateNonExpiredEventWindow(eventWindow); + ancientMode.getGenesisIndicator(), + indicatorToExpire.get() + 1, + ancientMode); + executor.getCaller().getShadowGraph().updateEventWindow(eventWindow); }, false)); - // we save the max generation of node 0, so we know what we need to expire to remove a tip - executor.setCustomPreSyncConfiguration((caller, listener) -> - genToExpire.set(SyncTestUtils.getMaxGen(caller.getShadowGraph().getTips()))); + // we save the max indicator of node 0, so we know what we need to expire to remove a tip + executor.setCustomPreSyncConfiguration((caller, listener) -> indicatorToExpire.set( + SyncTestUtils.getMaxIndicator(caller.getShadowGraph().getTips(), ancientMode))); executor.execute(); - SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener(), ancientMode); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } @@ -705,7 +809,7 @@ void testSendExpiredEvents() throws Exception { @MethodSource({"tenNodeGraphParams", "tenNodeBigGraphParams", "tipExpiresBreakingSeed"}) void tipExpiresAfterPhase1(final SyncTestParams params) throws Exception { final SyncTestExecutor executor = new SyncTestExecutor(params); - final AtomicLong maxGen = new AtomicLong(EventConstants.GENERATION_UNDEFINED); + final AtomicLong maximumIndicator = new AtomicLong(EventConstants.GENERATION_UNDEFINED); final int creatorIndexToExpire = 0; final NodeId creatorIdToExpire = executor.getAddressBook().getNodeId(creatorIndexToExpire); @@ -724,52 +828,49 @@ void tipExpiresAfterPhase1(final SyncTestParams params) throws Exception { return source0; })); - // we save the max generation of node 0, so we know what we need to expire to remove a tip + // we save the max indicator of node 0, so we know what we need to expire to remove a tip executor.setGraphCustomization((caller, listener) -> { caller.setSaveGeneratedEvents(true); listener.setSaveGeneratedEvents(true); - maxGen.set(caller.getEmitter().getGraphGenerator().getMaxGeneration(creatorIdToExpire)); + // As a hack, birth round is equal to generation + 1 + maximumIndicator.set(caller.getEmitter().getGraphGenerator().getMaxGeneration(creatorIdToExpire) + + (params.getAncientMode() == BIRTH_ROUND_THRESHOLD ? 1 : 0)); }); // before the sync, expire the tip on the listener executor.setCustomPreSyncConfiguration((c, l) -> { l.getShadowGraph() - .updateNonExpiredEventWindow(new NonAncientEventWindow( - 0 /* ignored by shadowgraph */, - 0 /* ignored by shadowgraph */, - maxGen.get() + 1, - GENERATION_THRESHOLD)); - when(l.getConsensus().getMinGenerationNonAncient()).thenReturn(maxGen.get() + 2); - when(l.getConsensus().getMaxRoundGeneration()).thenReturn(maxGen.get() + 3); - - c.getShadowGraph() - .updateNonExpiredEventWindow(new NonAncientEventWindow( - 0 /* ignored by shadowgraph */, + .updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, - max(EventConstants.FIRST_GENERATION, maxGen.get() - 1), - GENERATION_THRESHOLD)); - when(c.getConsensus().getMinGenerationNonAncient()).thenReturn(maxGen.get() + 2); - when(c.getConsensus().getMaxRoundGeneration()).thenReturn(maxGen.get() + 3); + maximumIndicator.get() + 2, + maximumIndicator.get() + 1, + params.getAncientMode())); + + c.updateEventWindow(new NonAncientEventWindow( + 0 /* ignored by shadowgraph */, + maximumIndicator.get() + 2, + max(params.getAncientMode().getGenesisIndicator(), maximumIndicator.get() - 1), + params.getAncientMode())); }); // after phase 1, expire the tip on the caller final SyncPhaseParallelExecutor parallelExecutor = new SyncPhaseParallelExecutor( getStaticThreadManager(), () -> executor.getCaller() - .getShadowGraph() - .updateNonExpiredEventWindow(new NonAncientEventWindow( - 0 /* ignored by shadowgraph */, + .updateEventWindow(new NonAncientEventWindow( 0 /* ignored by shadowgraph */, - maxGen.get() + 1, - GENERATION_THRESHOLD)), + params.getAncientMode().getGenesisIndicator(), + maximumIndicator.get() + 1, + params.getAncientMode())), null, true); executor.setExecutorSupplier(() -> parallelExecutor); executor.execute(); - SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertOnlyRequiredEventsTransferred( + executor.getCaller(), executor.getListener(), params.getAncientMode()); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } @@ -806,7 +907,8 @@ void tipsChangeAfterPhase1(final SyncTestParams params) throws Exception { // since we get a new set of tips before phase 3, it is possible to transfer some duplicate events that were // added after the initial tips were exchanged - SyncValidator.assertRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertRequiredEventsTransferred( + executor.getCaller(), executor.getListener(), params.getAncientMode()); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } @@ -843,13 +945,15 @@ void tipsChangeAfterPhase2(final SyncTestParams params) throws Exception { // since we get a new set of tips before phase 3, it is possible to transfer some duplicate events that were // added after the initial tips were exchanged - SyncValidator.assertRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertRequiredEventsTransferred( + executor.getCaller(), executor.getListener(), params.getAncientMode()); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } /** * Tests scenarios in which events that need to be sent to the peer are requested to be expired before they are - * sent. Because generations are reserved in a sync, the events should not be expired while a sync is in progress. + * sent. Because ancient indicators are reserved in a sync, the events should not be expired while a sync is in + * progress. * * @param expireAfterPhase the phase after which events that need to be sent should be requested to be expired * @param params Sync parameters @@ -858,24 +962,25 @@ void tipsChangeAfterPhase2(final SyncTestParams params) throws Exception { @MethodSource("requiredEventsExpire") void requiredEventsExpire(final int expireAfterPhase, final SyncTestParams params) throws Exception { final SyncTestExecutor executor = new SyncTestExecutor(params); - final AtomicLong genToExpire = new AtomicLong(0); + final AtomicLong indicatorToExpire = new AtomicLong(0); final NodeId creatorId = executor.getAddressBook().getNodeId(0); - // Set the generation to expire such that half the listener's graph, and therefore some events that need - // to be sent to the caller, will be expired - executor.setCustomPreSyncConfiguration( - (c, l) -> genToExpire.set(l.getEmitter().getGraphGenerator().getMaxGeneration(creatorId) / 2)); + // Set the indicator to expire such that half the listener's graph, and therefore some events that need + // to be sent to the caller, will be expired. Since we are hacking birth rounds to be the same as generations, + // we can use the same indicator for both. + executor.setCustomPreSyncConfiguration((c, l) -> + indicatorToExpire.set(l.getEmitter().getGraphGenerator().getMaxGeneration(creatorId) / 2)); final NonAncientEventWindow eventWindow = new NonAncientEventWindow( 0 /* ignored by shadowgraph */, - 0 /* ignored by shadowgraph */, - genToExpire.get(), - GENERATION_THRESHOLD); + params.getAncientMode().getGenesisIndicator(), + Math.max(params.getAncientMode().getGenesisIndicator(), indicatorToExpire.get()), + params.getAncientMode()); // Expire events from the listener's graph after the supplied phase final Runnable expireEvents = - () -> executor.getListener().getShadowGraph().updateNonExpiredEventWindow(eventWindow); + () -> executor.getListener().getShadowGraph().updateEventWindow(eventWindow); final SyncPhaseParallelExecutor parallelExecutor = new SyncPhaseParallelExecutor( getStaticThreadManager(), expireAfterPhase == 1 ? expireEvents : null, @@ -885,7 +990,8 @@ void requiredEventsExpire(final int expireAfterPhase, final SyncTestParams param executor.execute(); - SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertOnlyRequiredEventsTransferred( + executor.getCaller(), executor.getListener(), params.getAncientMode()); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } @@ -931,10 +1037,13 @@ private void runOldParentGraphTest(final SyncTestParams params, final Consumer new SyncNode(params.getNumNetworkNodes(), 0, factory.newStandardFromSourceFactory())); + executor.setCallerSupplier((factory) -> new SyncNode( + params.getNumNetworkNodes(), 0, factory.newStandardFromSourceFactory(), params.getAncientMode())); executor.setListenerSupplier((factory) -> new SyncNode( - params.getNumNetworkNodes(), params.getNumNetworkNodes() - 1, factory.newStandardFromSourceFactory())); + params.getNumNetworkNodes(), + params.getNumNetworkNodes() - 1, + factory.newStandardFromSourceFactory(), + params.getAncientMode())); executor.setInitialGraphCreation((caller, listener) -> { for (final SyncNode node : List.of(caller, listener)) { @@ -945,7 +1054,8 @@ private void runOldParentGraphTest(final SyncTestParams params, final Consumer executor.getCaller().getConnection().disconnect(), false); - executor.setCallerSupplier((factory) -> - new SyncNode(params.getNumNetworkNodes(), 0, factory.newShuffledFromSourceFactory(), parallelExecutor)); + executor.setCallerSupplier((factory) -> new SyncNode( + params.getNumNetworkNodes(), + 0, + factory.newShuffledFromSourceFactory(), + parallelExecutor, + params.getAncientMode())); try { executor.execute(); @@ -980,119 +1094,42 @@ void testConnectionClosedBehavior(final SyncTestParams params) { } /** - * Tests if a sync is interrupted because it takes too long. The caller has 300 events to send, but the listener - * sleeps after reading every event. Eventually, the sync will time out and will be aborted. - */ - @Test - @Disabled("Test takes a minute to finish, so disabled by default") - void testSyncTimeExceeded() { - final SyncTestParams params = new SyncTestParams(10, 1000, 500, 200); - final SyncTestExecutor executor = new SyncTestExecutor(params); - final int sleep = 1000; - - executor.setGraphCustomization((caller, listener) -> { - caller.setSaveGeneratedEvents(true); - listener.setSaveGeneratedEvents(true); - }); - - executor.setCustomInitialization((c, l) -> l.setSleepAfterEventReadMillis(sleep)); - - assertThrows(Exception.class, executor::execute, "the sync should time out and an exception should be thrown"); - } - - /** - * Tests if a sync is aborted when there is any issue on the readers side (in this case a socket timeout), while the - * listener is blocked while writing to the socket, because the buffer is full. The only was to unlock the writer is - * to close the connection, which is what the synchronizer does. - */ - @Test - @Disabled("Not reliable enough for CCI") - void testTimoutWriterStuck() { - final SyncTestParams params = new SyncTestParams(10, 1000, 5000, 200); - final SyncTestExecutor executor = new SyncTestExecutor(params); - final int sleep = 5000; - - executor.setConnectionFactory(ConnectionFactory::createSocketConnections); - - executor.setGraphCustomization((caller, listener) -> { - caller.setSaveGeneratedEvents(true); - listener.setSaveGeneratedEvents(true); - }); - - executor.setCustomInitialization((c, l) -> l.setSleepAfterEventReadMillis(sleep)); - - assertThrows( - Exception.class, - executor::execute, - "the socket read should time out and an exception should be thrown"); - } - - /** - * Tests that events from a signed state are not gossiped and that such a sync is properly aborted + * Tests that a sync works if one node has no events at all in the graph and also has a non-ancient indicator that + * is not 0 */ - @Test - void signedStateEvents() throws Exception { - final SyncTestParams params = new SyncTestParams(10, 50, 2, 1); + @ParameterizedTest + @MethodSource("bothAncientModes") + void noEventsStartIndicator(@NonNull final AncientMode ancientMode) throws Exception { + final SyncTestParams params = new SyncTestParams(4, 0, 100, 0, ancientMode); final SyncTestExecutor executor = new SyncTestExecutor(params); - executor.setGraphCustomization((caller, listener) -> { caller.setSaveGeneratedEvents(true); listener.setSaveGeneratedEvents(true); }); - // the caller will have only signed state events - executor.setCustomPreSyncConfiguration((caller, listener) -> { - caller.getGeneratedEvents().forEach(EventImpl::markAsSignedStateEvent); - - // a signed sent event needs to be identified as needed by the peer - // if it is ancient, it will not be marked as needed, so need to make sure no events are ancient - when(caller.getConsensus().getMinRoundGeneration()).thenReturn(GraphGenerations.FIRST_GENERATION); - when(caller.getConsensus().getMinGenerationNonAncient()).thenReturn(GraphGenerations.FIRST_GENERATION); - when(listener.getConsensus().getMinRoundGeneration()).thenReturn(GraphGenerations.FIRST_GENERATION); - when(listener.getConsensus().getMinGenerationNonAncient()).thenReturn(GraphGenerations.FIRST_GENERATION); - }); + executor.setEventWindowDefinitions((caller, listener) -> { + final long callerMaximumIndicator = + SyncTestUtils.getMaxIndicator(caller.getShadowGraph().getTips(), ancientMode); + final long callerAncientIndicator = SyncTestUtils.getMinIndicator( + caller.getShadowGraph() + .findAncestors(caller.getShadowGraph().getTips(), (e) -> true), + ancientMode); - executor.execute(); - - // the caller should not have sent any events to the listener - SyncValidator.assertNoEventsReceived(executor.getListener()); - SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); - - // both should be aware that the sync was aborted - assertFalse(executor.getCaller().getSynchronizerReturn()); - assertFalse(executor.getListener().getSynchronizerReturn()); - } + listener.getShadowGraph() + .updateEventWindow(new NonAncientEventWindow( + ROUND_FIRST /* ignored */, + callerMaximumIndicator / 2, + callerAncientIndicator, + ancientMode)); - /** - * Tests that a sync works if one node has no events at all in the graph and also has a non-ancient generation that - * is not 0 - */ - @Test - void noEventsStartGeneration() throws Exception { - final SyncTestParams params = new SyncTestParams(4, 0, 100, 0); - final SyncTestExecutor executor = new SyncTestExecutor(params); - executor.setGraphCustomization((caller, listener) -> { - caller.setSaveGeneratedEvents(true); - listener.setSaveGeneratedEvents(true); - }); - executor.setGenerationDefinitions((caller, listener) -> { - long callerMaxGen = SyncTestUtils.getMaxGen(caller.getShadowGraph().getTips()); - long callerMinGen = SyncTestUtils.getMinGen(caller.getShadowGraph() - .findAncestors(caller.getShadowGraph().getTips(), (e) -> true)); - - when(listener.getConsensus().getMaxRoundGeneration()).thenReturn(callerMaxGen); - when(listener.getConsensus().getMinGenerationNonAncient()).thenReturn(callerMaxGen / 2); - when(listener.getConsensus().getMinRoundGeneration()).thenReturn(callerMinGen); - - when(caller.getConsensus().getMaxRoundGeneration()).thenReturn(callerMaxGen); - when(caller.getConsensus().getMinGenerationNonAncient()).thenReturn(callerMaxGen / 2); - when(caller.getConsensus().getMinRoundGeneration()).thenReturn(callerMinGen); - }); - executor.setCustomPreSyncConfiguration((caller, listener) -> { listener.getShadowGraph() - .startWithExpiredThreshold(caller.getConsensus().getMinGenerationNonAncient()); + .updateEventWindow(new NonAncientEventWindow( + ROUND_FIRST /* ignored */, + callerMaximumIndicator / 2, + callerAncientIndicator, + ancientMode)); }); executor.execute(); - SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener()); + SyncValidator.assertOnlyRequiredEventsTransferred(executor.getCaller(), executor.getListener(), ancientMode); SyncValidator.assertStreamsEmpty(executor.getCaller(), executor.getListener()); } } diff --git a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncValidator.java b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncValidator.java index 4b0ff1ccb0d5..8bfb1c42cc95 100644 --- a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncValidator.java +++ b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/SyncValidator.java @@ -26,12 +26,14 @@ import static org.junit.jupiter.api.Assertions.fail; import com.swirlds.common.utility.CommonUtils; +import com.swirlds.platform.event.AncientMode; import com.swirlds.platform.event.GossipEvent; import com.swirlds.platform.internal.EventImpl; import com.swirlds.platform.network.Connection; import com.swirlds.platform.system.events.BaseEventHashedData; import com.swirlds.platform.system.events.BaseEventUnhashedData; import com.swirlds.platform.test.fixtures.event.IndexedEvent; +import edu.umd.cs.findbugs.annotations.NonNull; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -67,20 +69,22 @@ public static void assertNoEventsTransferred(final SyncNode caller, final SyncNo assertNoEventsReceived("listener", listener); } - public static void assertRequiredEventsTransferred(final SyncNode caller, final SyncNode listener) { + public static void assertRequiredEventsTransferred( + final SyncNode caller, final SyncNode listener, @NonNull final AncientMode ancientMode) { if (enableLogging) { printTipSet("Caller's Tip Set", caller); printTipSet("Listener's Tip Set", listener); System.out.println("*** Asserting that required events were transferred ***"); } - compareEventLists(caller, listener, false); + compareEventLists(caller, listener, false, ancientMode); } - public static void assertOnlyRequiredEventsTransferred(final SyncNode caller, final SyncNode listener) { + public static void assertOnlyRequiredEventsTransferred( + final SyncNode caller, final SyncNode listener, @NonNull final AncientMode ancientMode) { if (enableLogging) { System.out.println("*** Asserting that only required events were transferred ***"); } - compareEventLists(caller, listener, true); + compareEventLists(caller, listener, true, ancientMode); } public static void assertFallenBehindDetection(final boolean fellBehind, final SyncNode... nodes) { @@ -121,7 +125,11 @@ public static void assertExceptionThrown(final SyncNode caller, final SyncNode l assertNotNull(listener.getSyncException(), "Expected the listener to have thrown an exception."); } - private static void compareEventLists(final SyncNode caller, final SyncNode listener, final boolean strictCompare) { + private static void compareEventLists( + final SyncNode caller, + final SyncNode listener, + final boolean strictCompare, + @NonNull final AncientMode ancientMode) { // Determine the unique events for the caller and listener, since they could have added some of the // same events from step 2. final Collection expectedCallerSendList = new ArrayList<>(caller.getGeneratedEvents()); @@ -131,14 +139,16 @@ private static void compareEventLists(final SyncNode caller, final SyncNode list expectedListenerSendList.removeAll(caller.getGeneratedEvents()); // Remove expired events - expectedCallerSendList.removeIf(e -> e.getGeneration() < caller.getOldestGeneration()); - expectedListenerSendList.removeIf(e -> e.getGeneration() < listener.getOldestGeneration()); + expectedCallerSendList.removeIf( + e -> e.getBaseEvent().getAncientIndicator(ancientMode) < caller.getExpirationThreshold()); + expectedListenerSendList.removeIf( + e -> e.getBaseEvent().getAncientIndicator(ancientMode) < listener.getExpirationThreshold()); // Remove events that are ancient for the peer expectedCallerSendList.removeIf( - e -> e.getGeneration() < listener.getConsensus().getMinGenerationNonAncient()); + e -> e.getBaseEvent().getAncientIndicator(ancientMode) < listener.getCurrentAncientThreshold()); expectedListenerSendList.removeIf( - e -> e.getGeneration() < caller.getConsensus().getMinGenerationNonAncient()); + e -> e.getBaseEvent().getAncientIndicator(ancientMode) < caller.getCurrentAncientThreshold()); // Get the events each received from the other in the sync final List callerReceivedEvents = caller.getReceivedEvents(); @@ -157,15 +167,16 @@ private static void compareEventLists(final SyncNode caller, final SyncNode list } // Assert that the event each received are the unique events in the other's shadow graph - compareEventLists("listener", expectedCallerSendList, listener, strictCompare); - compareEventLists("caller", expectedListenerSendList, caller, strictCompare); + compareEventLists("listener", expectedCallerSendList, listener, strictCompare, ancientMode); + compareEventLists("caller", expectedListenerSendList, caller, strictCompare, ancientMode); } private static void compareEventLists( final String node, final Collection expectedList, final SyncNode receiver, - final boolean strictCompare) { + final boolean strictCompare, + @NonNull final AncientMode ancientMode) { Collection actualList = receiver.getReceivedEvents(); diff --git a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/Synchronizer.java b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/Synchronizer.java index 406c7b540be1..ef0ad74f6d14 100644 --- a/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/Synchronizer.java +++ b/platform-sdk/swirlds-unit-tests/core/swirlds-platform-test/src/test/java/com/swirlds/platform/test/sync/Synchronizer.java @@ -67,18 +67,11 @@ public void synchronize(final SyncNode caller, final SyncNode listener) throws E return null; }, () -> { - if (listener.isSendRecInitBytes()) { - // Read the COMM_SYNC_REQUEST byte on the listener prior to calling synchronize to match the - // production code and align the streams - listener.getConnection().getDis().readByte(); - } try { if (listener.isCanAcceptSync()) { final boolean synchronize = listener.getSynchronizer().synchronize(platformContext, listener.getConnection()); listener.setSynchronizerReturn(synchronize); - } else { - listener.getSynchronizer().rejectSync(listener.getConnection()); } } catch (final Exception e) { listener.setSynchronizerReturn(null);