From 20cc1fac45fbf8d56117802cab6bbc379a4a88e1 Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Mon, 11 Dec 2023 13:32:05 +0100 Subject: [PATCH 01/12] Merge --- pkg/protocol/engine/eviction/state.go | 55 +++++++++++++-------------- pkg/storage/permanent/permanent.go | 8 ---- pkg/storage/permanent/settings.go | 12 +++--- 3 files changed, 32 insertions(+), 43 deletions(-) diff --git a/pkg/protocol/engine/eviction/state.go b/pkg/protocol/engine/eviction/state.go index e09eafdfa..a08b28d00 100644 --- a/pkg/protocol/engine/eviction/state.go +++ b/pkg/protocol/engine/eviction/state.go @@ -6,10 +6,10 @@ import ( "github.com/iotaledger/hive.go/ierrors" "github.com/iotaledger/hive.go/kvstore" "github.com/iotaledger/hive.go/lo" - "github.com/iotaledger/hive.go/runtime/options" "github.com/iotaledger/hive.go/runtime/syncutils" "github.com/iotaledger/hive.go/serializer/v2" "github.com/iotaledger/hive.go/serializer/v2/stream" + "github.com/iotaledger/iota-core/pkg/protocol/engine" "github.com/iotaledger/iota-core/pkg/storage/prunable/slotstore" iotago "github.com/iotaledger/iota.go/v4" ) @@ -20,25 +20,21 @@ const latestNonEmptySlotKey = 1 type State struct { Events *Events + apiProvider iotago.APIProvider rootBlockStorageFunc func(iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error) lastEvictedSlot iotago.SlotIndex latestNonEmptyStore kvstore.KVStore evictionMutex syncutils.RWMutex - - genesisRootBlockFunc func() iotago.BlockID - - optsRootBlocksEvictionDelay iotago.SlotIndex } // NewState creates a new eviction State. -func NewState(latestNonEmptyStore kvstore.KVStore, rootBlockStorageFunc func(iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error), genesisRootBlockFunc func() iotago.BlockID, opts ...options.Option[State]) (state *State) { - return options.Apply(&State{ - Events: NewEvents(), - rootBlockStorageFunc: rootBlockStorageFunc, - latestNonEmptyStore: latestNonEmptyStore, - genesisRootBlockFunc: genesisRootBlockFunc, - optsRootBlocksEvictionDelay: 3, - }, opts) +func NewState(engine *engine.Engine, latestNonEmptyStore kvstore.KVStore, rootBlockStorageFunc func(iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error)) (state *State) { + return &State{ + apiProvider: engine, + Events: NewEvents(), + rootBlockStorageFunc: rootBlockStorageFunc, + latestNonEmptyStore: latestNonEmptyStore, + } } func (s *State) Initialize(lastCommittedSlot iotago.SlotIndex) { @@ -48,19 +44,25 @@ func (s *State) Initialize(lastCommittedSlot iotago.SlotIndex) { func (s *State) AdvanceActiveWindowToIndex(slot iotago.SlotIndex) { s.evictionMutex.Lock() - s.lastEvictedSlot = slot - if delayedIndex, shouldEvictRootBlocks := s.delayedBlockEvictionThreshold(slot); shouldEvictRootBlocks { - // Remember the last slot outside our cache window that has root blocks. - if storage, err := s.rootBlockStorageFunc(delayedIndex); err != nil { - _ = storage.StreamKeys(func(_ iotago.BlockID) error { - s.setLatestNonEmptySlot(delayedIndex) + protocolParams := s.apiProvider.APIForSlot(slot).ProtocolParameters() + genesisSlot := protocolParams.GenesisSlot() + maxCommittableAge := protocolParams.MaxCommittableAge() - return ierrors.New("stop iteration") - }) - } + if slot < maxCommittableAge+genesisSlot { + return + } + + // We allow a maxCommittableAge window of available root blocks. + evictionSlot := slot - maxCommittableAge + + // We do not evict slots that are empty + if evictionSlot >= s.latestNonEmptySlot() { + return } + s.lastEvictedSlot = slot + s.evictionMutex.Unlock() s.Events.SlotEvicted.Trigger(slot) @@ -118,6 +120,8 @@ func (s *State) AddRootBlock(id iotago.BlockID, commitmentID iotago.CommitmentID if err := lo.PanicOnErr(s.rootBlockStorageFunc(id.Slot())).Store(id, commitmentID); err != nil { panic(ierrors.Wrapf(err, "failed to store root block %s", id)) } + + s.setLatestNonEmptySlot(id.Slot()) } // RemoveRootBlock removes a solid entry points from the map. @@ -374,10 +378,3 @@ func (s *State) delayedBlockEvictionThreshold(slot iotago.SlotIndex) (thresholdS return genesisSlot, false } - -// WithRootBlocksEvictionDelay sets the time since confirmation threshold. -func WithRootBlocksEvictionDelay(evictionDelay iotago.SlotIndex) options.Option[State] { - return func(s *State) { - s.optsRootBlocksEvictionDelay = evictionDelay - } -} diff --git a/pkg/storage/permanent/permanent.go b/pkg/storage/permanent/permanent.go index c26bfc32c..94bbb85d5 100644 --- a/pkg/storage/permanent/permanent.go +++ b/pkg/storage/permanent/permanent.go @@ -82,14 +82,6 @@ func (p *Permanent) Accounts(optRealm ...byte) kvstore.KVStore { return lo.PanicOnErr(p.accounts.WithExtendedRealm(optRealm)) } -func (p *Permanent) LatestNonEmptySlot(optRealm ...byte) kvstore.KVStore { - if len(optRealm) == 0 { - return p.latestNonEmptySlot - } - - return lo.PanicOnErr(p.latestNonEmptySlot.WithExtendedRealm(optRealm)) -} - func (p *Permanent) UTXOLedger() *utxoledger.Manager { return p.utxoLedger } diff --git a/pkg/storage/permanent/settings.go b/pkg/storage/permanent/settings.go index 96c89497b..b18939d10 100644 --- a/pkg/storage/permanent/settings.go +++ b/pkg/storage/permanent/settings.go @@ -33,8 +33,9 @@ type Settings struct { store kvstore.KVStore storeSnapshotImported *kvstore.TypedValue[bool] storeLatestCommitment *kvstore.TypedValue[*model.Commitment] + storeLatestNonEmptySlot *kvstore.TypedValue[iotago.SlotIndex] storeLatestFinalizedSlot *kvstore.TypedValue[iotago.SlotIndex] - storeLatestProcessedSlot *kvstore.TypedValue[iotago.SlotIndex] + storeLatestStoredSlot *kvstore.TypedValue[iotago.SlotIndex] storeLatestIssuedValidationBlock *kvstore.TypedValue[*model.Block] mutex syncutils.RWMutex @@ -77,7 +78,7 @@ func NewSettings(store kvstore.KVStore, opts ...options.Option[iotago.EpochBased iotago.SlotIndex.Bytes, iotago.SlotIndexFromBytes, ), - storeLatestProcessedSlot: kvstore.NewTypedValue( + storeLatestStoredSlot: kvstore.NewTypedValue( store, []byte{latestStoredSlotKey}, iotago.SlotIndex.Bytes, @@ -258,7 +259,6 @@ func (s *Settings) SetLatestCommitment(latestCommitment *model.Commitment) (err func (s *Settings) latestCommitment() *model.Commitment { commitment, err := s.storeLatestCommitment.Get() - if err != nil { if ierrors.Is(err, kvstore.ErrKeyNotFound) { return model.NewEmptyCommitment(s.apiProvider.CommittedAPI()) @@ -289,11 +289,11 @@ func (s *Settings) LatestFinalizedSlot() iotago.SlotIndex { } func (s *Settings) LatestStoredSlot() iotago.SlotIndex { - return read(s.storeLatestProcessedSlot) + return read(s.storeLatestStoredSlot) } func (s *Settings) SetLatestStoredSlot(slot iotago.SlotIndex) (err error) { - return s.storeLatestProcessedSlot.Set(slot) + return s.storeLatestStoredSlot.Set(slot) } func (s *Settings) AdvanceLatestStoredSlot(slot iotago.SlotIndex) (err error) { @@ -303,7 +303,7 @@ func (s *Settings) AdvanceLatestStoredSlot(slot iotago.SlotIndex) (err error) { return nil } - if _, err = s.storeLatestProcessedSlot.Compute(func(latestStoredSlot iotago.SlotIndex, _ bool) (newValue iotago.SlotIndex, err error) { + if _, err = s.storeLatestStoredSlot.Compute(func(latestStoredSlot iotago.SlotIndex, _ bool) (newValue iotago.SlotIndex, err error) { if latestStoredSlot >= slot { return latestStoredSlot, kvstore.ErrTypedValueNotChanged } From e17e6aa6f3d8c111167b416a2e526f33511c038c Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Mon, 11 Dec 2023 15:22:29 +0100 Subject: [PATCH 02/12] Refactor root blocks completely --- components/inx/server_blocks.go | 7 +- .../blockgadget/testframework_test.go | 8 +- pkg/protocol/engine/engine.go | 13 +- pkg/protocol/engine/eviction/state.go | 178 +++++++----------- pkg/protocol/engine/eviction/state_test.go | 17 +- .../engine/eviction/testframework_test.go | 2 +- .../engine/tipselection/v1/provider.go | 2 +- .../tipselection/v1/test_framework_test.go | 4 +- .../engine/tipselection/v1/tip_selection.go | 25 +-- pkg/protocol/engines.go | 2 +- pkg/storage/permanent/permanent.go | 7 +- pkg/storage/permanent/settings.go | 35 ++++ pkg/storage/storage_permanent.go | 4 - pkg/testsuite/eviction.go | 2 +- .../mock/blockissuer_acceptance_loss.go | 14 +- 15 files changed, 144 insertions(+), 176 deletions(-) diff --git a/components/inx/server_blocks.go b/components/inx/server_blocks.go index 1838d7ee6..22ddd9910 100644 --- a/components/inx/server_blocks.go +++ b/components/inx/server_blocks.go @@ -19,9 +19,12 @@ import ( ) func (s *Server) ReadActiveRootBlocks(_ context.Context, _ *inx.NoParams) (*inx.RootBlocksResponse, error) { - activeRootBlocks := deps.Protocol.Engines.Main.Get().EvictionState.ActiveRootBlocks() + latestRootBlockID, latestRootBlockSlotCommitmentID := deps.Protocol.Engines.Main.Get().EvictionState.LatestActiveRootBlock() - return inx.WrapRootBlocks(activeRootBlocks), nil + // TODO: we should maybe change the INX API to return only the latest rootblock instead of the whole active set. + return inx.WrapRootBlocks(map[iotago.BlockID]iotago.CommitmentID{ + latestRootBlockID: latestRootBlockSlotCommitmentID, + }), nil } func (s *Server) ReadBlock(_ context.Context, blockID *inx.BlockId) (*inx.RawBlock, error) { diff --git a/pkg/protocol/engine/consensus/blockgadget/testframework_test.go b/pkg/protocol/engine/consensus/blockgadget/testframework_test.go index 486b4a8c9..db41ad1af 100644 --- a/pkg/protocol/engine/consensus/blockgadget/testframework_test.go +++ b/pkg/protocol/engine/consensus/blockgadget/testframework_test.go @@ -19,6 +19,7 @@ import ( "github.com/iotaledger/iota-core/pkg/protocol/engine/consensus/blockgadget/thresholdblockgadget" "github.com/iotaledger/iota-core/pkg/protocol/engine/eviction" "github.com/iotaledger/iota-core/pkg/protocol/sybilprotection/seatmanager/mock" + "github.com/iotaledger/iota-core/pkg/storage/permanent" "github.com/iotaledger/iota-core/pkg/storage/prunable/epochstore" "github.com/iotaledger/iota-core/pkg/storage/prunable/slotstore" iotago "github.com/iotaledger/iota.go/v4" @@ -45,15 +46,16 @@ func NewTestFramework(test *testing.T) *TestFramework { SeatManager: mock.NewManualPOA(iotago.SingleVersionProvider(tpkg.ZeroCostTestAPI), epochstore.NewStore(kvstore.Realm{}, mapdb.NewMapDB(), 0, (*account.Accounts).Bytes, account.AccountsFromBytes)), } - evictionState := eviction.NewState(mapdb.NewMapDB(), func(slot iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error) { + newSettings := permanent.NewSettings(mapdb.NewMapDB()) + newSettings.StoreProtocolParametersForStartEpoch(tpkg.ZeroCostTestAPI.ProtocolParameters(), 0) + + evictionState := eviction.NewState(newSettings, func(slot iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error) { return slotstore.NewStore(slot, mapdb.NewMapDB(), iotago.BlockID.Bytes, iotago.BlockIDFromBytes, iotago.CommitmentID.Bytes, iotago.CommitmentIDFromBytes, ), nil - }, func() iotago.BlockID { - return tpkg.ZeroCostTestAPI.ProtocolParameters().GenesisBlockID() }) t.blockCache = blocks.New(evictionState, iotago.SingleVersionProvider(tpkg.ZeroCostTestAPI)) diff --git a/pkg/protocol/engine/engine.go b/pkg/protocol/engine/engine.go index f6f241ecc..e399e2588 100644 --- a/pkg/protocol/engine/engine.go +++ b/pkg/protocol/engine/engine.go @@ -122,7 +122,7 @@ func New( &Engine{ Events: NewEvents(), Storage: storageInstance, - EvictionState: eviction.NewState(storageInstance.LatestNonEmptySlot(), storageInstance.RootBlocks, storageInstance.GenesisRootBlockID), + EvictionState: eviction.NewState(storageInstance.Settings(), storageInstance.RootBlocks), RootCommitment: reactive.NewVariable[*model.Commitment](), LatestCommitment: reactive.NewVariable[*model.Commitment](), Workers: workers, @@ -466,16 +466,7 @@ func (e *Engine) setupEvictionState() { wp := e.Workers.CreatePool("EvictionState", workerpool.WithWorkerCount(1)) // Using just 1 worker to avoid contention e.Events.BlockGadget.BlockAccepted.Hook(func(block *blocks.Block) { - block.ForEachParent(func(parent iotago.Parent) { - if parent.ID.Slot() < block.ID().Slot() && !e.EvictionState.IsRootBlock(parent.ID) { - parentBlock, exists := e.Block(parent.ID) - if !exists { - e.errorHandler(ierrors.Errorf("cannot store root block (%s) because it is missing", parent.ID)) - return - } - e.EvictionState.AddRootBlock(parentBlock.ID(), parentBlock.ProtocolBlock().Header.SlotCommitmentID) - } - }) + e.EvictionState.AddRootBlock(block.ID(), block.SlotCommitmentID()) }, event.WithWorkerPool(wp)) e.Events.Notarization.LatestCommitmentUpdated.Hook(func(commitment *model.Commitment) { diff --git a/pkg/protocol/engine/eviction/state.go b/pkg/protocol/engine/eviction/state.go index a08b28d00..c1bbfe395 100644 --- a/pkg/protocol/engine/eviction/state.go +++ b/pkg/protocol/engine/eviction/state.go @@ -1,15 +1,19 @@ package eviction import ( + "errors" "io" "github.com/iotaledger/hive.go/ierrors" - "github.com/iotaledger/hive.go/kvstore" "github.com/iotaledger/hive.go/lo" "github.com/iotaledger/hive.go/runtime/syncutils" "github.com/iotaledger/hive.go/serializer/v2" "github.com/iotaledger/hive.go/serializer/v2/stream" - "github.com/iotaledger/iota-core/pkg/protocol/engine" + + // "github.com/iotaledger/hive.go/serializer/v2" + // "github.com/iotaledger/hive.go/serializer/v2/stream" + "github.com/iotaledger/iota-core/pkg/model" + "github.com/iotaledger/iota-core/pkg/storage/permanent" "github.com/iotaledger/iota-core/pkg/storage/prunable/slotstore" iotago "github.com/iotaledger/iota.go/v4" ) @@ -20,20 +24,18 @@ const latestNonEmptySlotKey = 1 type State struct { Events *Events - apiProvider iotago.APIProvider + settings *permanent.Settings rootBlockStorageFunc func(iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error) lastEvictedSlot iotago.SlotIndex - latestNonEmptyStore kvstore.KVStore evictionMutex syncutils.RWMutex } // NewState creates a new eviction State. -func NewState(engine *engine.Engine, latestNonEmptyStore kvstore.KVStore, rootBlockStorageFunc func(iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error)) (state *State) { +func NewState(settings *permanent.Settings, rootBlockStorageFunc func(iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error)) (state *State) { return &State{ - apiProvider: engine, + settings: settings, Events: NewEvents(), rootBlockStorageFunc: rootBlockStorageFunc, - latestNonEmptyStore: latestNonEmptyStore, } } @@ -45,7 +47,7 @@ func (s *State) Initialize(lastCommittedSlot iotago.SlotIndex) { func (s *State) AdvanceActiveWindowToIndex(slot iotago.SlotIndex) { s.evictionMutex.Lock() - protocolParams := s.apiProvider.APIForSlot(slot).ProtocolParameters() + protocolParams := s.settings.APIProvider().APIForSlot(slot).ProtocolParameters() genesisSlot := protocolParams.GenesisSlot() maxCommittableAge := protocolParams.MaxCommittableAge() @@ -57,7 +59,7 @@ func (s *State) AdvanceActiveWindowToIndex(slot iotago.SlotIndex) { evictionSlot := slot - maxCommittableAge // We do not evict slots that are empty - if evictionSlot >= s.latestNonEmptySlot() { + if evictionSlot >= s.settings.LatestNonEmptySlot() { return } @@ -83,12 +85,12 @@ func (s *State) InRootBlockSlot(id iotago.BlockID) bool { return s.withinActiveIndexRange(id.Slot()) } -func (s *State) ActiveRootBlocks() map[iotago.BlockID]iotago.CommitmentID { +func (s *State) AllActiveRootBlocks() map[iotago.BlockID]iotago.CommitmentID { s.evictionMutex.RLock() defer s.evictionMutex.RUnlock() activeRootBlocks := make(map[iotago.BlockID]iotago.CommitmentID) - startSlot, endSlot := s.activeIndexRange() + startSlot, endSlot := s.activeIndexRange(s.lastEvictedSlot) for slot := startSlot; slot <= endSlot; slot++ { // We assume the cache is always populated for the latest slots. storage, err := s.rootBlockStorageFunc(slot) @@ -107,13 +109,47 @@ func (s *State) ActiveRootBlocks() map[iotago.BlockID]iotago.CommitmentID { return activeRootBlocks } +func (s *State) LatestActiveRootBlock() (iotago.BlockID, iotago.CommitmentID) { + s.evictionMutex.RLock() + defer s.evictionMutex.RUnlock() + + startSlot, endSlot := s.activeIndexRange(s.lastEvictedSlot) + for slot := endSlot; slot >= startSlot; slot-- { + // We assume the cache is always populated for the latest slots. + storage, err := s.rootBlockStorageFunc(slot) + // Slot too old, it was pruned. + if err != nil { + continue + } + + var latestRootBlock iotago.BlockID + var latestSlotCommitmentID iotago.CommitmentID + + _ = storage.Stream(func(id iotago.BlockID, commitmentID iotago.CommitmentID) error { + latestRootBlock = id + latestSlotCommitmentID = commitmentID + + // We want the newest rootblock. + return errors.New("stop iteration") + }) + + // We found the most recent root block in this slot. + if latestRootBlock != iotago.EmptyBlockID { + return latestRootBlock, latestSlotCommitmentID + } + } + + // Fallback to genesis block and genesis commitment if we have no active root blocks. + return s.settings.APIProvider().CommittedAPI().ProtocolParameters().GenesisBlockID(), model.NewEmptyCommitment(s.settings.APIProvider().CommittedAPI()).ID() +} + // AddRootBlock inserts a solid entry point to the seps map. func (s *State) AddRootBlock(id iotago.BlockID, commitmentID iotago.CommitmentID) { s.evictionMutex.RLock() defer s.evictionMutex.RUnlock() // The rootblock is too old, ignore it. - if id.Slot() < lo.Return1(s.activeIndexRange()) { + if id.Slot() < lo.Return1(s.activeIndexRange(s.lastEvictedSlot)) { return } @@ -121,7 +157,7 @@ func (s *State) AddRootBlock(id iotago.BlockID, commitmentID iotago.CommitmentID panic(ierrors.Wrapf(err, "failed to store root block %s", id)) } - s.setLatestNonEmptySlot(id.Slot()) + s.settings.AdvanceLatestNonEmptySlot(id.Slot()) } // RemoveRootBlock removes a solid entry points from the map. @@ -182,10 +218,8 @@ func (s *State) Export(writer io.WriteSeeker, lowerTarget iotago.SlotIndex, targ s.evictionMutex.RLock() defer s.evictionMutex.RUnlock() - start, _ := s.delayedBlockEvictionThreshold(lowerTarget) - - genesisSlot := s.genesisRootBlockFunc().Slot() - latestNonEmptySlot := genesisSlot + start, _ := s.activeIndexRange(lowerTarget) + latestNonEmptySlot := s.settings.APIProvider().APIForSlot(targetSlot).ProtocolParameters().GenesisSlot() if err := stream.WriteCollection(writer, serializer.SeriLengthPrefixTypeAsUint32, func() (elementsCount int, err error) { for currentSlot := start; currentSlot <= targetSlot; currentSlot++ { @@ -217,12 +251,6 @@ func (s *State) Export(writer io.WriteSeeker, lowerTarget iotago.SlotIndex, targ return ierrors.Wrap(err, "failed to write root blocks") } - if latestNonEmptySlot > s.optsRootBlocksEvictionDelay { - latestNonEmptySlot -= s.optsRootBlocksEvictionDelay - } else { - latestNonEmptySlot = genesisSlot - } - if err := stream.Write(writer, latestNonEmptySlot); err != nil { return ierrors.Wrap(err, "failed to write latest non empty slot") } @@ -257,20 +285,19 @@ func (s *State) Import(reader io.ReadSeeker) error { return ierrors.Wrap(err, "failed to read latest non empty slot") } - s.setLatestNonEmptySlot(latestNonEmptySlot) + s.settings.SetLatestNonEmptySlot(latestNonEmptySlot) return nil } -func (s *State) Rollback(lowerTarget iotago.SlotIndex, targetIndex iotago.SlotIndex) error { +func (s *State) Rollback(lowerTarget iotago.SlotIndex, targetSlot iotago.SlotIndex) error { s.evictionMutex.RLock() defer s.evictionMutex.RUnlock() - start, _ := s.delayedBlockEvictionThreshold(lowerTarget) - genesisSlot := s.genesisRootBlockFunc().Slot() - latestNonEmptySlot := genesisSlot + start, _ := s.activeIndexRange(lowerTarget) + latestNonEmptySlot := s.settings.APIProvider().APIForSlot(targetSlot).ProtocolParameters().GenesisSlot() - for currentSlot := start; currentSlot <= targetIndex; currentSlot++ { + for currentSlot := start; currentSlot <= targetSlot; currentSlot++ { _, err := s.rootBlockStorageFunc(currentSlot) if err != nil { continue @@ -279,102 +306,33 @@ func (s *State) Rollback(lowerTarget iotago.SlotIndex, targetIndex iotago.SlotIn latestNonEmptySlot = currentSlot } - if latestNonEmptySlot > s.optsRootBlocksEvictionDelay { - latestNonEmptySlot -= s.optsRootBlocksEvictionDelay - } else { - latestNonEmptySlot = genesisSlot - } - - if err := s.latestNonEmptyStore.Set([]byte{latestNonEmptySlotKey}, latestNonEmptySlot.MustBytes()); err != nil { - return ierrors.Wrap(err, "failed to store latest non empty slot") - } + s.settings.SetLatestNonEmptySlot(latestNonEmptySlot) return nil } func (s *State) Reset() { /* nothing to reset but comply with interface */ } -// latestNonEmptySlot returns the latest slot that contains a rootblock. -func (s *State) latestNonEmptySlot() iotago.SlotIndex { - latestNonEmptySlotBytes, err := s.latestNonEmptyStore.Get([]byte{latestNonEmptySlotKey}) - if err != nil { - if ierrors.Is(err, kvstore.ErrKeyNotFound) { - return 0 - } - panic(ierrors.Wrap(err, "failed to get latest non empty slot")) - } - - latestNonEmptySlot, _, err := iotago.SlotIndexFromBytes(latestNonEmptySlotBytes) - if err != nil { - panic(ierrors.Wrap(err, "failed to parse latest non empty slot")) - } - - return latestNonEmptySlot -} +func (s *State) activeIndexRange(targetSlot iotago.SlotIndex) (startSlot iotago.SlotIndex, endSlot iotago.SlotIndex) { + protocolParams := s.settings.APIProvider().APIForSlot(targetSlot).ProtocolParameters() + genesisSlot := protocolParams.GenesisSlot() + maxCommittableAge := protocolParams.MaxCommittableAge() -// setLatestNonEmptySlot sets the latest slot that contains a rootblock. -func (s *State) setLatestNonEmptySlot(slot iotago.SlotIndex) { - if err := s.latestNonEmptyStore.Set([]byte{latestNonEmptySlotKey}, slot.MustBytes()); err != nil { - panic(ierrors.Wrap(err, "failed to store latest non empty slot")) + if targetSlot < maxCommittableAge+genesisSlot { + return genesisSlot, targetSlot } -} -func (s *State) activeIndexRange() (startSlot iotago.SlotIndex, endSlot iotago.SlotIndex) { - lastCommittedSlot := s.lastEvictedSlot - delayedSlot, valid := s.delayedBlockEvictionThreshold(lastCommittedSlot) + rootBlocksWindowStart := targetSlot - maxCommittableAge - if !valid { - return 0, lastCommittedSlot + if latestNonEmptySlot := s.settings.LatestNonEmptySlot(); rootBlocksWindowStart > latestNonEmptySlot { + rootBlocksWindowStart = latestNonEmptySlot } - if delayedSlot+1 > lastCommittedSlot { - return delayedSlot, lastCommittedSlot - } - - return delayedSlot + 1, lastCommittedSlot + return rootBlocksWindowStart, targetSlot } func (s *State) withinActiveIndexRange(slot iotago.SlotIndex) bool { - startSlot, endSlot := s.activeIndexRange() + startSlot, endSlot := s.activeIndexRange(slot) return slot >= startSlot && slot <= endSlot } - -// delayedBlockEvictionThreshold returns the slot index that is the threshold for delayed rootblocks eviction. -func (s *State) delayedBlockEvictionThreshold(slot iotago.SlotIndex) (thresholdSlot iotago.SlotIndex, shouldEvict bool) { - genesisSlot := s.genesisRootBlockFunc().Slot() - if slot < genesisSlot+s.optsRootBlocksEvictionDelay { - return genesisSlot, false - } - - var rootBlockInWindow bool - // Check if we have root blocks up to the eviction point. - for ; slot >= s.lastEvictedSlot; slot-- { - storage := lo.PanicOnErr(s.rootBlockStorageFunc(slot)) - - _ = storage.StreamKeys(func(_ iotago.BlockID) error { - if slot >= s.optsRootBlocksEvictionDelay { - thresholdSlot = slot - s.optsRootBlocksEvictionDelay - shouldEvict = true - } else { - thresholdSlot = genesisSlot - shouldEvict = false - } - - rootBlockInWindow = true - - return ierrors.New("stop iteration") - }) - } - - if rootBlockInWindow { - return thresholdSlot, shouldEvict - } - - // If we didn't find any root blocks, we have to fallback to the latestNonEmptySlot before the eviction point. - if latestNonEmptySlot := s.latestNonEmptySlot(); latestNonEmptySlot >= s.optsRootBlocksEvictionDelay { - return latestNonEmptySlot - s.optsRootBlocksEvictionDelay, true - } - - return genesisSlot, false -} diff --git a/pkg/protocol/engine/eviction/state_test.go b/pkg/protocol/engine/eviction/state_test.go index 515dae6da..5ee233d7a 100644 --- a/pkg/protocol/engine/eviction/state_test.go +++ b/pkg/protocol/engine/eviction/state_test.go @@ -7,7 +7,9 @@ import ( "github.com/iotaledger/hive.go/kvstore/mapdb" "github.com/iotaledger/iota-core/pkg/protocol/engine/eviction" "github.com/iotaledger/iota-core/pkg/storage/database" + "github.com/iotaledger/iota-core/pkg/storage/permanent" "github.com/iotaledger/iota-core/pkg/storage/prunable" + "github.com/iotaledger/iota-core/pkg/storage/prunable/slotstore" iotago "github.com/iotaledger/iota.go/v4" "github.com/iotaledger/iota.go/v4/tpkg" ) @@ -21,9 +23,18 @@ func TestState_RootBlocks(t *testing.T) { Directory: t.TempDir(), }, iotago.SingleVersionProvider(tpkg.ZeroCostTestAPI), errorHandler) - ts := NewTestFramework(t, prunableStorage, eviction.NewState(mapdb.NewMapDB(), prunableStorage.RootBlocks, func() iotago.BlockID { - return tpkg.ZeroCostTestAPI.ProtocolParameters().GenesisBlockID() - }, eviction.WithRootBlocksEvictionDelay(3))) + newSettings := permanent.NewSettings(mapdb.NewMapDB()) + newSettings.StoreProtocolParametersForStartEpoch(tpkg.ZeroCostTestAPI.ProtocolParameters(), 0) + + ts := NewTestFramework(t, prunableStorage, eviction.NewState(newSettings, func(slot iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error) { + return slotstore.NewStore(slot, mapdb.NewMapDB(), + iotago.BlockID.Bytes, + iotago.BlockIDFromBytes, + iotago.CommitmentID.Bytes, + iotago.CommitmentIDFromBytes, + ), nil + })) + ts.CreateAndAddRootBlock("Genesis", 0, iotago.NewEmptyCommitment(tpkg.ZeroCostTestAPI).MustID()) ts.RequireActiveRootBlocks("Genesis") ts.RequireLastEvictedSlot(0) diff --git a/pkg/protocol/engine/eviction/testframework_test.go b/pkg/protocol/engine/eviction/testframework_test.go index 33d097036..a0ab7224c 100644 --- a/pkg/protocol/engine/eviction/testframework_test.go +++ b/pkg/protocol/engine/eviction/testframework_test.go @@ -81,7 +81,7 @@ func (t *TestFramework) RootBlocks(aliases ...string) map[iotago.BlockID]iotago. func (t *TestFramework) RequireActiveRootBlocks(expected ...string) { expectedRootBlocks := t.RootBlocks(expected...) - gotActiveRootBlocks := t.Instance.ActiveRootBlocks() + gotActiveRootBlocks := t.Instance.AllActiveRootBlocks() require.Equalf(t.Testing, expectedRootBlocks, gotActiveRootBlocks, "active root blocks do not match, expected: %v, got: %v", expectedRootBlocks, gotActiveRootBlocks) } diff --git a/pkg/protocol/engine/tipselection/v1/provider.go b/pkg/protocol/engine/tipselection/v1/provider.go index be28ebc50..c2eacce8a 100644 --- a/pkg/protocol/engine/tipselection/v1/provider.go +++ b/pkg/protocol/engine/tipselection/v1/provider.go @@ -21,7 +21,7 @@ func NewProvider(opts ...options.Option[TipSelection]) module.Provider[*engine.E e.Constructed.OnTrigger(func() { // wait for submodules to be constructed (so all of their properties are available) module.OnAllConstructed(func() { - t.Construct(e.TipManager, e.Ledger.SpendDAG(), e.Ledger.MemPool().TransactionMetadata, func() iotago.BlockIDs { return lo.Keys(e.EvictionState.ActiveRootBlocks()) }, DynamicLivenessThreshold(e.SybilProtection.SeatManager().OnlineCommittee().Size)) + t.Construct(e.TipManager, e.Ledger.SpendDAG(), e.Ledger.MemPool().TransactionMetadata, func() iotago.BlockID { return lo.Return1(e.EvictionState.LatestActiveRootBlock()) }, DynamicLivenessThreshold(e.SybilProtection.SeatManager().OnlineCommittee().Size)) }, e.TipManager, e.Ledger, e.SybilProtection) }) diff --git a/pkg/protocol/engine/tipselection/v1/test_framework_test.go b/pkg/protocol/engine/tipselection/v1/test_framework_test.go index d7288ae0e..0ce8b8739 100644 --- a/pkg/protocol/engine/tipselection/v1/test_framework_test.go +++ b/pkg/protocol/engine/tipselection/v1/test_framework_test.go @@ -41,8 +41,8 @@ func NewTestFramework(test *testing.T, opts ...options.Option[TestFramework]) *T return nil, false } - rootBlocksRetriever := func() iotago.BlockIDs { - return iotago.BlockIDs{iotago.EmptyBlockID} + rootBlocksRetriever := func() iotago.BlockID { + return iotago.EmptyBlockID } t.TipManager = tipmanagertests.NewTestFramework(test) diff --git a/pkg/protocol/engine/tipselection/v1/tip_selection.go b/pkg/protocol/engine/tipselection/v1/tip_selection.go index e3e4a323e..452f4a953 100644 --- a/pkg/protocol/engine/tipselection/v1/tip_selection.go +++ b/pkg/protocol/engine/tipselection/v1/tip_selection.go @@ -3,8 +3,6 @@ package tipselectionv1 import ( "time" - "golang.org/x/exp/slices" - "github.com/iotaledger/hive.go/ds" "github.com/iotaledger/hive.go/ds/reactive" "github.com/iotaledger/hive.go/ierrors" @@ -30,8 +28,8 @@ type TipSelection struct { // spendDAG is the SpendDAG that is used to track spenders. spendDAG spenddag.SpendDAG[iotago.TransactionID, mempool.StateID, ledger.BlockVoteRank] - // rootBlocks is a function that returns the current root blocks. - rootBlocks func() iotago.BlockIDs + // rootBlock is a function that returns the current root blocks. + rootBlock func() iotago.BlockID // livenessThreshold is a function that is used to determine the liveness threshold for a tip. livenessThreshold func(tip tipmanager.TipMetadata) time.Duration @@ -85,11 +83,11 @@ func New(opts ...options.Option[TipSelection]) *TipSelection { // // This method is separated from the constructor so the TipSelection can be initialized lazily after all dependencies // are available. -func (t *TipSelection) Construct(tipManager tipmanager.TipManager, spendDAG spenddag.SpendDAG[iotago.TransactionID, mempool.StateID, ledger.BlockVoteRank], transactionMetadataRetriever func(iotago.TransactionID) (mempool.TransactionMetadata, bool), rootBlocksRetriever func() iotago.BlockIDs, livenessThresholdFunc func(tipmanager.TipMetadata) time.Duration) *TipSelection { +func (t *TipSelection) Construct(tipManager tipmanager.TipManager, spendDAG spenddag.SpendDAG[iotago.TransactionID, mempool.StateID, ledger.BlockVoteRank], transactionMetadataRetriever func(iotago.TransactionID) (mempool.TransactionMetadata, bool), rootBlocksRetriever func() iotago.BlockID, livenessThresholdFunc func(tipmanager.TipMetadata) time.Duration) *TipSelection { t.tipManager = tipManager t.spendDAG = spendDAG t.transactionMetadata = transactionMetadataRetriever - t.rootBlocks = rootBlocksRetriever + t.rootBlock = rootBlocksRetriever t.livenessThreshold = livenessThresholdFunc t.TriggerConstructed() @@ -127,20 +125,7 @@ func (t *TipSelection) SelectTips(amount int) (references model.ParentReferences previousLikedInsteadConflicts = updatedLikedInsteadConflicts } }, amount); len(references[iotago.StrongParentType]) == 0 { - rootBlocks := t.rootBlocks() - - // Sort the rootBlocks in descending order according to their slot. - slices.SortFunc(rootBlocks, func(i iotago.BlockID, j iotago.BlockID) int { - if i.Slot() == j.Slot() { - return 0 - } else if i.Slot() < j.Slot() { - return 1 - } - - return -1 - }) - - references[iotago.StrongParentType] = rootBlocks[:lo.Min(len(rootBlocks), t.optMaxStrongParents)] + references[iotago.StrongParentType] = iotago.BlockIDs{t.rootBlock()} } t.collectReferences(references, iotago.WeakParentType, t.tipManager.WeakTips, func(tip tipmanager.TipMetadata) { diff --git a/pkg/protocol/engines.go b/pkg/protocol/engines.go index 847713cd2..d803f4af0 100644 --- a/pkg/protocol/engines.go +++ b/pkg/protocol/engines.go @@ -90,7 +90,7 @@ func (e *Engines) ForkAtSlot(slot iotago.SlotIndex) (*engine.Engine, error) { return nil, ierrors.Wrap(err, "failed to rollback commitments") } // create temporary components and rollback their permanent state, which will be reflected on disk. - evictionState := eviction.NewState(newStorage.LatestNonEmptySlot(), newStorage.RootBlocks, newStorage.GenesisRootBlockID) + evictionState := eviction.NewState(newStorage.Settings(), newStorage.RootBlocks) evictionState.Initialize(latestCommitment.Slot()) blockCache := blocks.New(evictionState, newStorage.Settings().APIProvider()) diff --git a/pkg/storage/permanent/permanent.go b/pkg/storage/permanent/permanent.go index 94bbb85d5..8fde86a13 100644 --- a/pkg/storage/permanent/permanent.go +++ b/pkg/storage/permanent/permanent.go @@ -18,7 +18,6 @@ const ( commitmentsPrefix ledgerPrefix accountsPrefix - latestNonEmptySlotPrefix ) type Permanent struct { @@ -29,9 +28,8 @@ type Permanent struct { settings *Settings commitments *Commitments - utxoLedger *utxoledger.Manager - accounts kvstore.KVStore - latestNonEmptySlot kvstore.KVStore + utxoLedger *utxoledger.Manager + accounts kvstore.KVStore optsEpochBasedProvider []options.Option[iotago.EpochBasedProvider] } @@ -48,7 +46,6 @@ func New(dbConfig database.Config, errorHandler func(error), opts ...options.Opt p.commitments = NewCommitments(lo.PanicOnErr(p.store.KVStore().WithExtendedRealm(kvstore.Realm{commitmentsPrefix})), p.settings.APIProvider()) p.utxoLedger = utxoledger.New(lo.PanicOnErr(p.store.KVStore().WithExtendedRealm(kvstore.Realm{ledgerPrefix})), p.settings.APIProvider()) p.accounts = lo.PanicOnErr(p.store.KVStore().WithExtendedRealm(kvstore.Realm{accountsPrefix})) - p.latestNonEmptySlot = lo.PanicOnErr(p.store.KVStore().WithExtendedRealm(kvstore.Realm{latestNonEmptySlotPrefix})) }) } diff --git a/pkg/storage/permanent/settings.go b/pkg/storage/permanent/settings.go index b18939d10..876bfad1d 100644 --- a/pkg/storage/permanent/settings.go +++ b/pkg/storage/permanent/settings.go @@ -23,6 +23,7 @@ const ( latestCommitmentKey latestFinalizedSlotKey latestStoredSlotKey + latestNonEmptySlotKey protocolVersionEpochMappingKey futureProtocolParametersKey protocolParametersKey @@ -84,6 +85,12 @@ func NewSettings(store kvstore.KVStore, opts ...options.Option[iotago.EpochBased iotago.SlotIndex.Bytes, iotago.SlotIndexFromBytes, ), + storeLatestNonEmptySlot: kvstore.NewTypedValue( + store, + []byte{latestNonEmptySlotKey}, + iotago.SlotIndex.Bytes, + iotago.SlotIndexFromBytes, + ), storeLatestIssuedValidationBlock: kvstore.NewTypedValue( store, []byte{latestIssuedValidationBlock}, @@ -316,6 +323,34 @@ func (s *Settings) AdvanceLatestStoredSlot(slot iotago.SlotIndex) (err error) { return nil } +func (s *Settings) LatestNonEmptySlot() iotago.SlotIndex { + return read(s.storeLatestNonEmptySlot) +} + +func (s *Settings) SetLatestNonEmptySlot(slot iotago.SlotIndex) (err error) { + return s.storeLatestNonEmptySlot.Set(slot) +} + +func (s *Settings) AdvanceLatestNonEmptySlot(slot iotago.SlotIndex) (err error) { + // We don't need to advance the latest stored slot if it's already ahead of the given slot. + // We check this before Compute to avoid contention inside the TypedValue. + if s.LatestNonEmptySlot() >= slot { + return nil + } + + if _, err = s.storeLatestNonEmptySlot.Compute(func(latestNonEmptySlot iotago.SlotIndex, _ bool) (newValue iotago.SlotIndex, err error) { + if latestNonEmptySlot >= slot { + return latestNonEmptySlot, kvstore.ErrTypedValueNotChanged + } + + return slot, nil + }); err != nil { + return ierrors.Wrap(err, "failed to advance latest non-empty slot") + } + + return nil +} + func (s *Settings) LatestIssuedValidationBlock() *model.Block { return read(s.storeLatestIssuedValidationBlock) } diff --git a/pkg/storage/storage_permanent.go b/pkg/storage/storage_permanent.go index cc5036f25..7738a96bb 100644 --- a/pkg/storage/storage_permanent.go +++ b/pkg/storage/storage_permanent.go @@ -19,10 +19,6 @@ func (s *Storage) Accounts(optRealm ...byte) kvstore.KVStore { return s.permanent.Accounts(optRealm...) } -func (s *Storage) LatestNonEmptySlot(optRealm ...byte) kvstore.KVStore { - return s.permanent.LatestNonEmptySlot(optRealm...) -} - // Ledger returns the ledger storage (or a specialized sub-storage if a realm is provided). func (s *Storage) Ledger() *utxoledger.Manager { return s.permanent.UTXOLedger() diff --git a/pkg/testsuite/eviction.go b/pkg/testsuite/eviction.go index 8b9500a1b..f5a57964d 100644 --- a/pkg/testsuite/eviction.go +++ b/pkg/testsuite/eviction.go @@ -20,7 +20,7 @@ func (t *TestSuite) AssertActiveRootBlocks(expectedBlocks []*blocks.Block, nodes for _, node := range nodes { t.Eventually(func() error { - activeRootBlocks := node.Protocol.Engines.Main.Get().EvictionState.ActiveRootBlocks() + activeRootBlocks := node.Protocol.Engines.Main.Get().EvictionState.AllActiveRootBlocks() if !assert.Equal(t.fakeTesting, expectedRootBlocks, activeRootBlocks) { return ierrors.Errorf("AssertActiveRootBlocks: %s: expected %v, got %v", node.Name, expectedRootBlocks, activeRootBlocks) diff --git a/pkg/testsuite/mock/blockissuer_acceptance_loss.go b/pkg/testsuite/mock/blockissuer_acceptance_loss.go index e295819b5..c06f8c143 100644 --- a/pkg/testsuite/mock/blockissuer_acceptance_loss.go +++ b/pkg/testsuite/mock/blockissuer_acceptance_loss.go @@ -4,6 +4,7 @@ import ( "time" "github.com/iotaledger/hive.go/ierrors" + "github.com/iotaledger/hive.go/lo" iotago "github.com/iotaledger/iota.go/v4" ) @@ -12,18 +13,7 @@ func (i *BlockIssuer) reviveChain(issuingTime time.Time, node *Node) (*iotago.Co apiForSlot := node.Protocol.APIForSlot(lastCommittedSlot) // Get a rootblock as recent as possible for the parent. - parentBlockID := iotago.EmptyBlockID - for rootBlock := range node.Protocol.Engines.Main.Get().EvictionState.ActiveRootBlocks() { - if rootBlock.Slot() > parentBlockID.Slot() { - parentBlockID = rootBlock - } - - // Exit the loop if we found a rootblock in the last committed slot (which is the highest we can get). - if parentBlockID.Slot() == lastCommittedSlot { - break - } - } - + parentBlockID := lo.Return1(node.Protocol.Engines.Main.Get().EvictionState.LatestActiveRootBlock()) issuingSlot := apiForSlot.TimeProvider().SlotFromTime(issuingTime) // Force commitments until minCommittableAge relative to the block's issuing time. We basically "pretend" that From fec8afbc0910e16524c82987e9eff3ef26b8ea9b Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Mon, 11 Dec 2023 16:44:46 +0100 Subject: [PATCH 03/12] Fix deadlock and rootblock evaluation when attaching --- pkg/protocol/engine/eviction/state.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/protocol/engine/eviction/state.go b/pkg/protocol/engine/eviction/state.go index c1bbfe395..811e74628 100644 --- a/pkg/protocol/engine/eviction/state.go +++ b/pkg/protocol/engine/eviction/state.go @@ -52,6 +52,7 @@ func (s *State) AdvanceActiveWindowToIndex(slot iotago.SlotIndex) { maxCommittableAge := protocolParams.MaxCommittableAge() if slot < maxCommittableAge+genesisSlot { + s.evictionMutex.Unlock() return } @@ -60,6 +61,7 @@ func (s *State) AdvanceActiveWindowToIndex(slot iotago.SlotIndex) { // We do not evict slots that are empty if evictionSlot >= s.settings.LatestNonEmptySlot() { + s.evictionMutex.Unlock() return } @@ -332,7 +334,7 @@ func (s *State) activeIndexRange(targetSlot iotago.SlotIndex) (startSlot iotago. } func (s *State) withinActiveIndexRange(slot iotago.SlotIndex) bool { - startSlot, endSlot := s.activeIndexRange(slot) + startSlot, endSlot := s.activeIndexRange(s.lastEvictedSlot) return slot >= startSlot && slot <= endSlot } From d32a747cec08f9f770656c81c1b15e6cc22f1fc7 Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Mon, 11 Dec 2023 17:34:15 +0100 Subject: [PATCH 04/12] Fix eviction test --- go.mod | 14 +++++------ go.sum | 14 +++++++++++ pkg/model/commitment.go | 1 - .../blockgadget/testframework_test.go | 18 ++++++------- pkg/protocol/engine/eviction/state.go | 25 +++++++++++-------- pkg/protocol/engine/eviction/state_test.go | 25 +++++++++---------- .../engine/eviction/testframework_test.go | 2 +- pkg/testsuite/blocks.go | 14 +++++++++++ 8 files changed, 72 insertions(+), 41 deletions(-) diff --git a/go.mod b/go.mod index 2295c1ea1..a0a8dcad0 100644 --- a/go.mod +++ b/go.mod @@ -12,21 +12,21 @@ require ( github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 github.com/iotaledger/hive.go/ads v0.0.0-20231205131244-472357435a39 github.com/iotaledger/hive.go/app v0.0.0-20231206114953-6a65a82e30ad - github.com/iotaledger/hive.go/constraints v0.0.0-20231206114953-6a65a82e30ad + github.com/iotaledger/hive.go/constraints v0.0.0-20231207181026-f482ac139305 github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231206114953-6a65a82e30ad - github.com/iotaledger/hive.go/crypto v0.0.0-20231206114953-6a65a82e30ad + github.com/iotaledger/hive.go/crypto v0.0.0-20231207181026-f482ac139305 github.com/iotaledger/hive.go/ds v0.0.0-20231206114953-6a65a82e30ad - github.com/iotaledger/hive.go/ierrors v0.0.0-20231206114953-6a65a82e30ad + github.com/iotaledger/hive.go/ierrors v0.0.0-20231207181026-f482ac139305 github.com/iotaledger/hive.go/kvstore v0.0.0-20231205131244-472357435a39 - github.com/iotaledger/hive.go/lo v0.0.0-20231206114953-6a65a82e30ad + github.com/iotaledger/hive.go/lo v0.0.0-20231207181026-f482ac139305 github.com/iotaledger/hive.go/log v0.0.0-20231206113509-4b4ff95ac61c github.com/iotaledger/hive.go/logger v0.0.0-20231206114953-6a65a82e30ad - github.com/iotaledger/hive.go/runtime v0.0.0-20231206114953-6a65a82e30ad + github.com/iotaledger/hive.go/runtime v0.0.0-20231207181026-f482ac139305 github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231206114953-6a65a82e30ad - github.com/iotaledger/hive.go/stringify v0.0.0-20231206114953-6a65a82e30ad + github.com/iotaledger/hive.go/stringify v0.0.0-20231207181026-f482ac139305 github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231206124511-b78dc962031f github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231206124145-f773dfe3927e - github.com/iotaledger/iota.go/v4 v4.0.0-20231206123921-2af411eef0b5 + github.com/iotaledger/iota.go/v4 v4.0.0-20231211160706-492c65d5e3f5 github.com/labstack/echo/v4 v4.11.3 github.com/labstack/gommon v0.4.1 github.com/libp2p/go-libp2p v0.32.0 diff --git a/go.sum b/go.sum index ea71d4d30..3e7ce36c9 100644 --- a/go.sum +++ b/go.sum @@ -281,34 +281,48 @@ github.com/iotaledger/hive.go/app v0.0.0-20231206114953-6a65a82e30ad h1:v7dkbVLS github.com/iotaledger/hive.go/app v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:hTHKGFbZnuiW8yEgDuuL7ZjQTCnl8bXyHLmj3LPa648= github.com/iotaledger/hive.go/constraints v0.0.0-20231206114953-6a65a82e30ad h1:4XL7IIvdsWHxSKQfU+sgq3H9egN54053LF9TwMfDcTg= github.com/iotaledger/hive.go/constraints v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s= +github.com/iotaledger/hive.go/constraints v0.0.0-20231207181026-f482ac139305 h1:ar+IWfqO7B1M5+kuKGUJnfg0i/YuuM1oN5i8byp/F7A= +github.com/iotaledger/hive.go/constraints v0.0.0-20231207181026-f482ac139305/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s= github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231206114953-6a65a82e30ad h1:iNzb/Oy/nucIOXOzRcwSqqFsaeKwr2JZpZYSLp8xjlE= github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231206114953-6a65a82e30ad/go.mod h1:CO28KMA6Pp5LJPiigPQQ276zQofES+jMod08U5pyRFA= github.com/iotaledger/hive.go/crypto v0.0.0-20231206114953-6a65a82e30ad h1:pUL2UZbF4S8FIV7uKo9p+IGfZ658K1VNorQ6rzDMRvs= github.com/iotaledger/hive.go/crypto v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:7vHoF//1Pt3nu0l8nDIw7bEgv2GfbL3kSgjp7Rdqhd4= +github.com/iotaledger/hive.go/crypto v0.0.0-20231207181026-f482ac139305 h1:OR2TClxTtst906F4tok9xzhBTKO81qrUFdxIAoaZVvE= +github.com/iotaledger/hive.go/crypto v0.0.0-20231207181026-f482ac139305/go.mod h1:7vHoF//1Pt3nu0l8nDIw7bEgv2GfbL3kSgjp7Rdqhd4= github.com/iotaledger/hive.go/ds v0.0.0-20231206114953-6a65a82e30ad h1:adLrD6dOEkM5Xdg6AOPt9/HYqy/pQ5FrprDpW4/VqUU= github.com/iotaledger/hive.go/ds v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:NmZRIoxtL6iQdVK6n5W+JOx58K/0Yn8k7WuSvpKPQ+M= github.com/iotaledger/hive.go/ierrors v0.0.0-20231206114953-6a65a82e30ad h1:WDl58zJKHfwbzHs+ZB8Jq3YNgVQE5Neu2NeaX3FZuyU= github.com/iotaledger/hive.go/ierrors v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8= +github.com/iotaledger/hive.go/ierrors v0.0.0-20231207181026-f482ac139305 h1:v7/zMhNcr6hibXFZXZ4xV/S27ESUytQFgUQ1oo10iic= +github.com/iotaledger/hive.go/ierrors v0.0.0-20231207181026-f482ac139305/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8= github.com/iotaledger/hive.go/kvstore v0.0.0-20231205131244-472357435a39 h1:Gp2h+Els9cTVYYnYsHX3zLuixb0XggIj2okK570aKww= github.com/iotaledger/hive.go/kvstore v0.0.0-20231205131244-472357435a39/go.mod h1:ytfKoHr/nF8u0y0G4mamfG0yjFtJiJVk0kgjnPOtsSY= github.com/iotaledger/hive.go/lo v0.0.0-20231206114953-6a65a82e30ad h1:qpCsjw+InLL824QPu3lY/osck4DhucBKhCs5/E8OH+A= github.com/iotaledger/hive.go/lo v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:ETXGXymFyNcUq2t4I9e7ZK18f9bxUWYat4pjZ9W0rWc= +github.com/iotaledger/hive.go/lo v0.0.0-20231207181026-f482ac139305 h1:zxVbTEWutMvZhS0VLu/OmBk2WpMjrXQ7l67VBwsExtc= +github.com/iotaledger/hive.go/lo v0.0.0-20231207181026-f482ac139305/go.mod h1:ETXGXymFyNcUq2t4I9e7ZK18f9bxUWYat4pjZ9W0rWc= github.com/iotaledger/hive.go/log v0.0.0-20231206113509-4b4ff95ac61c h1:Ksts6VjPj9y0o2Nis+2tHtDGWITNJ4yju87ZlHLPuOo= github.com/iotaledger/hive.go/log v0.0.0-20231206113509-4b4ff95ac61c/go.mod h1:Td3R6QmYw0svZI1GuZ/tN9s0VNaHassXSKLCc70aX9w= github.com/iotaledger/hive.go/logger v0.0.0-20231206114953-6a65a82e30ad h1:fazCxogqOLDEPNDPWYDLTDpYmwgTJgIaC2Z6VN52S4M= github.com/iotaledger/hive.go/logger v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:hVaVODS+Uik0obf3SVEHFQNruUko/uqIgD/GKwhn49M= github.com/iotaledger/hive.go/runtime v0.0.0-20231206114953-6a65a82e30ad h1:HpupWK8iqFt+Sdogkh2/N8ojalmevYy+FzhjOuy7Y7E= github.com/iotaledger/hive.go/runtime v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:Z9NFsByMh1Kf98f3v3ifeZRycbS2db1hjswTQG1MxnE= +github.com/iotaledger/hive.go/runtime v0.0.0-20231207181026-f482ac139305 h1:7CW1/EbG+RvkjbyOf6JA1u1feax/cpex/6a8CLbaA4I= +github.com/iotaledger/hive.go/runtime v0.0.0-20231207181026-f482ac139305/go.mod h1:Z9NFsByMh1Kf98f3v3ifeZRycbS2db1hjswTQG1MxnE= github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231206114953-6a65a82e30ad h1:c8uwbBZDqpiCNN9/9Jji7Z4lL0GdVnORp8WMouiuknk= github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231206114953-6a65a82e30ad/go.mod h1:FoH3T6yKlZJp8xm8K+zsQiibSynp32v21CpWx8xkek8= github.com/iotaledger/hive.go/stringify v0.0.0-20231206114953-6a65a82e30ad h1:VC3OgdSbyngY7/gxVj66fKd/nGmN6P0/myr348nx7vA= github.com/iotaledger/hive.go/stringify v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs= +github.com/iotaledger/hive.go/stringify v0.0.0-20231207181026-f482ac139305 h1:KjbaklWvZb4zIcXBETHzl6XFTAf8wtAlFDfaF0Z1Daw= +github.com/iotaledger/hive.go/stringify v0.0.0-20231207181026-f482ac139305/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs= github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231206124511-b78dc962031f h1:V68Ijq1A64gB9r0Rhc4ybLGH66rXqZ2Ly0L4uuaLrMg= github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231206124511-b78dc962031f/go.mod h1:Dy3Gv4Dn1zufB177x6IXETP3zTeiWQ1+HMVQR0Bt/ew= github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231206124145-f773dfe3927e h1:jbtiUlmTpTdGiRBW1pniPSqRcDMJaIW8fGS+uORryas= github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231206124145-f773dfe3927e/go.mod h1:zEb9onVHnDUStl5SsFBj7H0HBKfIN0c/pUND8Llfqp8= github.com/iotaledger/iota.go/v4 v4.0.0-20231206123921-2af411eef0b5 h1:0KgQFpVRnKd6CdCwXo3Kg/SL27xkeKh2SMoU5G1TkZk= github.com/iotaledger/iota.go/v4 v4.0.0-20231206123921-2af411eef0b5/go.mod h1:tiswk1O1wSAi9GE6tD1j43+bLmWU9Je07aZLaJ0+Ha0= +github.com/iotaledger/iota.go/v4 v4.0.0-20231211160706-492c65d5e3f5 h1:2iQUEuYvuyeYtZBr6bRoM4xFLxRiQ66aBPgKuJTirh0= +github.com/iotaledger/iota.go/v4 v4.0.0-20231211160706-492c65d5e3f5/go.mod h1:jU3Jlnhwv4KWlpft84lu9/M0QkCykESl29bL4oCenKs= github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI= github.com/ipfs/boxo v0.13.1/go.mod h1:btrtHy0lmO1ODMECbbEY1pxNtrLilvKSYLoGQt1yYCk= github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s= diff --git a/pkg/model/commitment.go b/pkg/model/commitment.go index f80f41824..673116ab1 100644 --- a/pkg/model/commitment.go +++ b/pkg/model/commitment.go @@ -22,7 +22,6 @@ type Commitment struct { func NewEmptyCommitment(api iotago.API) *Commitment { emptyCommitment := iotago.NewEmptyCommitment(api) - emptyCommitment.ReferenceManaCost = api.ProtocolParameters().CongestionControlParameters().MinReferenceManaCost return lo.PanicOnErr(CommitmentFromCommitment(emptyCommitment, api)) } diff --git a/pkg/protocol/engine/consensus/blockgadget/testframework_test.go b/pkg/protocol/engine/consensus/blockgadget/testframework_test.go index db41ad1af..00b2ae52a 100644 --- a/pkg/protocol/engine/consensus/blockgadget/testframework_test.go +++ b/pkg/protocol/engine/consensus/blockgadget/testframework_test.go @@ -10,6 +10,7 @@ import ( "github.com/iotaledger/hive.go/ds/shrinkingmap" "github.com/iotaledger/hive.go/kvstore" + hivedb "github.com/iotaledger/hive.go/kvstore/database" "github.com/iotaledger/hive.go/kvstore/mapdb" "github.com/iotaledger/hive.go/lo" "github.com/iotaledger/iota-core/pkg/core/account" @@ -19,9 +20,10 @@ import ( "github.com/iotaledger/iota-core/pkg/protocol/engine/consensus/blockgadget/thresholdblockgadget" "github.com/iotaledger/iota-core/pkg/protocol/engine/eviction" "github.com/iotaledger/iota-core/pkg/protocol/sybilprotection/seatmanager/mock" + "github.com/iotaledger/iota-core/pkg/storage/database" "github.com/iotaledger/iota-core/pkg/storage/permanent" + "github.com/iotaledger/iota-core/pkg/storage/prunable" "github.com/iotaledger/iota-core/pkg/storage/prunable/epochstore" - "github.com/iotaledger/iota-core/pkg/storage/prunable/slotstore" iotago "github.com/iotaledger/iota.go/v4" "github.com/iotaledger/iota.go/v4/builder" "github.com/iotaledger/iota.go/v4/tpkg" @@ -46,17 +48,15 @@ func NewTestFramework(test *testing.T) *TestFramework { SeatManager: mock.NewManualPOA(iotago.SingleVersionProvider(tpkg.ZeroCostTestAPI), epochstore.NewStore(kvstore.Realm{}, mapdb.NewMapDB(), 0, (*account.Accounts).Bytes, account.AccountsFromBytes)), } + prunableStorage := prunable.New(database.Config{ + Engine: hivedb.EngineMapDB, + Directory: t.TempDir(), + }, iotago.SingleVersionProvider(tpkg.ZeroCostTestAPI), func(err error) { panic(err) }) + newSettings := permanent.NewSettings(mapdb.NewMapDB()) newSettings.StoreProtocolParametersForStartEpoch(tpkg.ZeroCostTestAPI.ProtocolParameters(), 0) - evictionState := eviction.NewState(newSettings, func(slot iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error) { - return slotstore.NewStore(slot, mapdb.NewMapDB(), - iotago.BlockID.Bytes, - iotago.BlockIDFromBytes, - iotago.CommitmentID.Bytes, - iotago.CommitmentIDFromBytes, - ), nil - }) + evictionState := eviction.NewState(newSettings, prunableStorage.RootBlocks) t.blockCache = blocks.New(evictionState, iotago.SingleVersionProvider(tpkg.ZeroCostTestAPI)) instance := thresholdblockgadget.New(t.blockCache, t.SeatManager, func(err error) { diff --git a/pkg/protocol/engine/eviction/state.go b/pkg/protocol/engine/eviction/state.go index 811e74628..fbf3ef4f9 100644 --- a/pkg/protocol/engine/eviction/state.go +++ b/pkg/protocol/engine/eviction/state.go @@ -26,7 +26,7 @@ type State struct { settings *permanent.Settings rootBlockStorageFunc func(iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error) - lastEvictedSlot iotago.SlotIndex + lastCommittedSlot iotago.SlotIndex evictionMutex syncutils.RWMutex } @@ -41,7 +41,7 @@ func NewState(settings *permanent.Settings, rootBlockStorageFunc func(iotago.Slo func (s *State) Initialize(lastCommittedSlot iotago.SlotIndex) { // This marks the slot from which we only have root blocks, so starting with 0 is valid here, since we only have a root block for genesis. - s.lastEvictedSlot = lastCommittedSlot + s.lastCommittedSlot = lastCommittedSlot } func (s *State) AdvanceActiveWindowToIndex(slot iotago.SlotIndex) { @@ -51,6 +51,8 @@ func (s *State) AdvanceActiveWindowToIndex(slot iotago.SlotIndex) { genesisSlot := protocolParams.GenesisSlot() maxCommittableAge := protocolParams.MaxCommittableAge() + s.lastCommittedSlot = slot + if slot < maxCommittableAge+genesisSlot { s.evictionMutex.Unlock() return @@ -65,18 +67,16 @@ func (s *State) AdvanceActiveWindowToIndex(slot iotago.SlotIndex) { return } - s.lastEvictedSlot = slot - s.evictionMutex.Unlock() - s.Events.SlotEvicted.Trigger(slot) + s.Events.SlotEvicted.Trigger(evictionSlot) } func (s *State) LastEvictedSlot() iotago.SlotIndex { s.evictionMutex.RLock() defer s.evictionMutex.RUnlock() - return s.lastEvictedSlot + return s.lastCommittedSlot } // InRootBlockSlot checks if the Block associated with the given id is too old. @@ -92,7 +92,7 @@ func (s *State) AllActiveRootBlocks() map[iotago.BlockID]iotago.CommitmentID { defer s.evictionMutex.RUnlock() activeRootBlocks := make(map[iotago.BlockID]iotago.CommitmentID) - startSlot, endSlot := s.activeIndexRange(s.lastEvictedSlot) + startSlot, endSlot := s.activeIndexRange(s.lastCommittedSlot) for slot := startSlot; slot <= endSlot; slot++ { // We assume the cache is always populated for the latest slots. storage, err := s.rootBlockStorageFunc(slot) @@ -108,6 +108,11 @@ func (s *State) AllActiveRootBlocks() map[iotago.BlockID]iotago.CommitmentID { }) } + // We include genesis as a root block if the start of our active window is the genesis slot. + if startSlot == s.settings.APIProvider().APIForSlot(s.lastCommittedSlot).ProtocolParameters().GenesisSlot() { + activeRootBlocks[s.settings.APIProvider().CommittedAPI().ProtocolParameters().GenesisBlockID()] = model.NewEmptyCommitment(s.settings.APIProvider().CommittedAPI()).ID() + } + return activeRootBlocks } @@ -115,7 +120,7 @@ func (s *State) LatestActiveRootBlock() (iotago.BlockID, iotago.CommitmentID) { s.evictionMutex.RLock() defer s.evictionMutex.RUnlock() - startSlot, endSlot := s.activeIndexRange(s.lastEvictedSlot) + startSlot, endSlot := s.activeIndexRange(s.lastCommittedSlot) for slot := endSlot; slot >= startSlot; slot-- { // We assume the cache is always populated for the latest slots. storage, err := s.rootBlockStorageFunc(slot) @@ -151,7 +156,7 @@ func (s *State) AddRootBlock(id iotago.BlockID, commitmentID iotago.CommitmentID defer s.evictionMutex.RUnlock() // The rootblock is too old, ignore it. - if id.Slot() < lo.Return1(s.activeIndexRange(s.lastEvictedSlot)) { + if id.Slot() < lo.Return1(s.activeIndexRange(s.lastCommittedSlot)) { return } @@ -334,7 +339,7 @@ func (s *State) activeIndexRange(targetSlot iotago.SlotIndex) (startSlot iotago. } func (s *State) withinActiveIndexRange(slot iotago.SlotIndex) bool { - startSlot, endSlot := s.activeIndexRange(s.lastEvictedSlot) + startSlot, endSlot := s.activeIndexRange(s.lastCommittedSlot) return slot >= startSlot && slot <= endSlot } diff --git a/pkg/protocol/engine/eviction/state_test.go b/pkg/protocol/engine/eviction/state_test.go index 5ee233d7a..7affc269f 100644 --- a/pkg/protocol/engine/eviction/state_test.go +++ b/pkg/protocol/engine/eviction/state_test.go @@ -9,7 +9,6 @@ import ( "github.com/iotaledger/iota-core/pkg/storage/database" "github.com/iotaledger/iota-core/pkg/storage/permanent" "github.com/iotaledger/iota-core/pkg/storage/prunable" - "github.com/iotaledger/iota-core/pkg/storage/prunable/slotstore" iotago "github.com/iotaledger/iota.go/v4" "github.com/iotaledger/iota.go/v4/tpkg" ) @@ -18,24 +17,24 @@ func TestState_RootBlocks(t *testing.T) { errorHandler := func(err error) { t.Error(err) } + + TestAPISmallMCA := iotago.V3API(iotago.NewV3SnapshotProtocolParameters( + iotago.WithStorageOptions(0, 0, 0, 0, 0, 0), // zero storage score + iotago.WithWorkScoreOptions(0, 1, 0, 0, 0, 0, 0, 0, 0, 0), // all blocks workscore = 1 + iotago.WithLivenessOptions(5, 9, 1, 2, 4), + )) + prunableStorage := prunable.New(database.Config{ Engine: hivedb.EngineMapDB, Directory: t.TempDir(), }, iotago.SingleVersionProvider(tpkg.ZeroCostTestAPI), errorHandler) newSettings := permanent.NewSettings(mapdb.NewMapDB()) - newSettings.StoreProtocolParametersForStartEpoch(tpkg.ZeroCostTestAPI.ProtocolParameters(), 0) - - ts := NewTestFramework(t, prunableStorage, eviction.NewState(newSettings, func(slot iotago.SlotIndex) (*slotstore.Store[iotago.BlockID, iotago.CommitmentID], error) { - return slotstore.NewStore(slot, mapdb.NewMapDB(), - iotago.BlockID.Bytes, - iotago.BlockIDFromBytes, - iotago.CommitmentID.Bytes, - iotago.CommitmentIDFromBytes, - ), nil - })) - - ts.CreateAndAddRootBlock("Genesis", 0, iotago.NewEmptyCommitment(tpkg.ZeroCostTestAPI).MustID()) + newSettings.StoreProtocolParametersForStartEpoch(TestAPISmallMCA.ProtocolParameters(), 0) + + ts := NewTestFramework(t, prunableStorage, eviction.NewState(newSettings, prunableStorage.RootBlocks)) + + ts.CreateAndAddRootBlock("Genesis", 0, iotago.NewEmptyCommitment(TestAPISmallMCA).MustID()) ts.RequireActiveRootBlocks("Genesis") ts.RequireLastEvictedSlot(0) diff --git a/pkg/protocol/engine/eviction/testframework_test.go b/pkg/protocol/engine/eviction/testframework_test.go index a0ab7224c..45317cb6c 100644 --- a/pkg/protocol/engine/eviction/testframework_test.go +++ b/pkg/protocol/engine/eviction/testframework_test.go @@ -99,7 +99,7 @@ func (t *TestFramework) RequireStorageRootBlocks(expected ...string) { loadedCommitmentID, exists, err := rootBlockStorage.Load(blockID) require.NoError(t.Testing, err) - require.True(t.Testing, exists) + require.True(t.Testing, exists, "expected blockID %s to exist in rootBlockStorage", blockID) require.Equal(t.Testing, commitmentID, loadedCommitmentID) } } diff --git a/pkg/testsuite/blocks.go b/pkg/testsuite/blocks.go index 40bdd265b..bfce8b917 100644 --- a/pkg/testsuite/blocks.go +++ b/pkg/testsuite/blocks.go @@ -1,6 +1,9 @@ package testsuite import ( + "fmt" + "strings" + "github.com/stretchr/testify/assert" "github.com/iotaledger/hive.go/ds" @@ -15,19 +18,30 @@ import ( func (t *TestSuite) AssertBlock(block *blocks.Block, node *mock.Node) *model.Block { var loadedBlock *model.Block t.Eventually(func() error { + fmt.Println("Exists? ", block.ID()) + + if strings.HasPrefix(block.ID().String(), "BlockID(3.1-node") { + fmt.Println("break me") + } + var exists bool loadedBlock, exists = node.Protocol.Engines.Main.Get().Block(block.ID()) if !exists { + fmt.Println("loadedBlock ", block.ID()) return ierrors.Errorf("AssertBlock: %s: block %s does not exist", node.Name, block.ID()) } if block.ID() != loadedBlock.ID() { + fmt.Println("return2 ", block.ID()) return ierrors.Errorf("AssertBlock: %s: expected %s, got %s", node.Name, block.ID(), loadedBlock.ID()) } if !assert.Equal(t.fakeTesting, block.ModelBlock().Data(), loadedBlock.Data()) { + fmt.Println("return3 ", block.ID()) return ierrors.Errorf("AssertBlock: %s: expected %s, got %s", node.Name, block.ModelBlock().Data(), loadedBlock.Data()) } + fmt.Println("return4 nil nil nil ", block.ID()) + return nil }) From e6fe5f0b94ad2aa4de9e07dfca4686907c61b253 Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Mon, 11 Dec 2023 17:47:59 +0100 Subject: [PATCH 05/12] Fix more tests --- go.mod | 4 ++-- go.sum | 18 ++---------------- pkg/protocol/engine/eviction/state.go | 2 +- pkg/protocol/engine/eviction/state_test.go | 2 +- pkg/tests/protocol_startup_test.go | 22 +++++++++++----------- 5 files changed, 17 insertions(+), 31 deletions(-) diff --git a/go.mod b/go.mod index a0a8dcad0..7711ab1f8 100644 --- a/go.mod +++ b/go.mod @@ -42,7 +42,6 @@ require ( go.uber.org/atomic v1.11.0 go.uber.org/dig v1.17.1 golang.org/x/crypto v0.16.0 - golang.org/x/exp v0.0.0-20231006140011-7918f672742d google.golang.org/grpc v1.59.0 google.golang.org/protobuf v1.31.0 ) @@ -89,6 +88,7 @@ require ( github.com/huin/goupnp v1.3.0 // indirect github.com/iancoleman/orderedmap v0.3.0 // indirect github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 // indirect + github.com/iotaledger/iota-crypto-demo v0.0.0-20231208171603-786bb32fdb00 // indirect github.com/ipfs/boxo v0.13.1 // indirect github.com/ipfs/go-cid v0.4.1 // indirect github.com/ipfs/go-datastore v0.6.0 // indirect @@ -160,7 +160,6 @@ require ( github.com/valyala/bytebufferpool v1.0.0 // indirect github.com/valyala/fasttemplate v1.2.2 // indirect github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 // indirect - github.com/wollac/iota-crypto-demo v0.0.0-20221117162917-b10619eccb98 // indirect go.opencensus.io v0.24.0 // indirect go.opentelemetry.io/otel v1.19.0 // indirect go.opentelemetry.io/otel/metric v1.19.0 // indirect @@ -169,6 +168,7 @@ require ( go.uber.org/mock v0.3.0 // indirect go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.26.0 // indirect + golang.org/x/exp v0.0.0-20231006140011-7918f672742d // indirect golang.org/x/image v0.13.0 // indirect golang.org/x/mod v0.13.0 // indirect golang.org/x/net v0.19.0 // indirect diff --git a/go.sum b/go.sum index 3e7ce36c9..af0e4ef3e 100644 --- a/go.sum +++ b/go.sum @@ -279,48 +279,36 @@ github.com/iotaledger/hive.go/ads v0.0.0-20231205131244-472357435a39 h1:jxoBAPgC github.com/iotaledger/hive.go/ads v0.0.0-20231205131244-472357435a39/go.mod h1:gbUvr01B5ha530GnNm8K2OsHXOd2BtzBYOMxyTX3iDg= github.com/iotaledger/hive.go/app v0.0.0-20231206114953-6a65a82e30ad h1:v7dkbVLSsmzgOWT2vjvv1MdKQXvqFbvIkx8mvh6VK7g= github.com/iotaledger/hive.go/app v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:hTHKGFbZnuiW8yEgDuuL7ZjQTCnl8bXyHLmj3LPa648= -github.com/iotaledger/hive.go/constraints v0.0.0-20231206114953-6a65a82e30ad h1:4XL7IIvdsWHxSKQfU+sgq3H9egN54053LF9TwMfDcTg= -github.com/iotaledger/hive.go/constraints v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s= github.com/iotaledger/hive.go/constraints v0.0.0-20231207181026-f482ac139305 h1:ar+IWfqO7B1M5+kuKGUJnfg0i/YuuM1oN5i8byp/F7A= github.com/iotaledger/hive.go/constraints v0.0.0-20231207181026-f482ac139305/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s= github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231206114953-6a65a82e30ad h1:iNzb/Oy/nucIOXOzRcwSqqFsaeKwr2JZpZYSLp8xjlE= github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231206114953-6a65a82e30ad/go.mod h1:CO28KMA6Pp5LJPiigPQQ276zQofES+jMod08U5pyRFA= -github.com/iotaledger/hive.go/crypto v0.0.0-20231206114953-6a65a82e30ad h1:pUL2UZbF4S8FIV7uKo9p+IGfZ658K1VNorQ6rzDMRvs= -github.com/iotaledger/hive.go/crypto v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:7vHoF//1Pt3nu0l8nDIw7bEgv2GfbL3kSgjp7Rdqhd4= github.com/iotaledger/hive.go/crypto v0.0.0-20231207181026-f482ac139305 h1:OR2TClxTtst906F4tok9xzhBTKO81qrUFdxIAoaZVvE= github.com/iotaledger/hive.go/crypto v0.0.0-20231207181026-f482ac139305/go.mod h1:7vHoF//1Pt3nu0l8nDIw7bEgv2GfbL3kSgjp7Rdqhd4= github.com/iotaledger/hive.go/ds v0.0.0-20231206114953-6a65a82e30ad h1:adLrD6dOEkM5Xdg6AOPt9/HYqy/pQ5FrprDpW4/VqUU= github.com/iotaledger/hive.go/ds v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:NmZRIoxtL6iQdVK6n5W+JOx58K/0Yn8k7WuSvpKPQ+M= -github.com/iotaledger/hive.go/ierrors v0.0.0-20231206114953-6a65a82e30ad h1:WDl58zJKHfwbzHs+ZB8Jq3YNgVQE5Neu2NeaX3FZuyU= -github.com/iotaledger/hive.go/ierrors v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8= github.com/iotaledger/hive.go/ierrors v0.0.0-20231207181026-f482ac139305 h1:v7/zMhNcr6hibXFZXZ4xV/S27ESUytQFgUQ1oo10iic= github.com/iotaledger/hive.go/ierrors v0.0.0-20231207181026-f482ac139305/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8= github.com/iotaledger/hive.go/kvstore v0.0.0-20231205131244-472357435a39 h1:Gp2h+Els9cTVYYnYsHX3zLuixb0XggIj2okK570aKww= github.com/iotaledger/hive.go/kvstore v0.0.0-20231205131244-472357435a39/go.mod h1:ytfKoHr/nF8u0y0G4mamfG0yjFtJiJVk0kgjnPOtsSY= -github.com/iotaledger/hive.go/lo v0.0.0-20231206114953-6a65a82e30ad h1:qpCsjw+InLL824QPu3lY/osck4DhucBKhCs5/E8OH+A= -github.com/iotaledger/hive.go/lo v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:ETXGXymFyNcUq2t4I9e7ZK18f9bxUWYat4pjZ9W0rWc= github.com/iotaledger/hive.go/lo v0.0.0-20231207181026-f482ac139305 h1:zxVbTEWutMvZhS0VLu/OmBk2WpMjrXQ7l67VBwsExtc= github.com/iotaledger/hive.go/lo v0.0.0-20231207181026-f482ac139305/go.mod h1:ETXGXymFyNcUq2t4I9e7ZK18f9bxUWYat4pjZ9W0rWc= github.com/iotaledger/hive.go/log v0.0.0-20231206113509-4b4ff95ac61c h1:Ksts6VjPj9y0o2Nis+2tHtDGWITNJ4yju87ZlHLPuOo= github.com/iotaledger/hive.go/log v0.0.0-20231206113509-4b4ff95ac61c/go.mod h1:Td3R6QmYw0svZI1GuZ/tN9s0VNaHassXSKLCc70aX9w= github.com/iotaledger/hive.go/logger v0.0.0-20231206114953-6a65a82e30ad h1:fazCxogqOLDEPNDPWYDLTDpYmwgTJgIaC2Z6VN52S4M= github.com/iotaledger/hive.go/logger v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:hVaVODS+Uik0obf3SVEHFQNruUko/uqIgD/GKwhn49M= -github.com/iotaledger/hive.go/runtime v0.0.0-20231206114953-6a65a82e30ad h1:HpupWK8iqFt+Sdogkh2/N8ojalmevYy+FzhjOuy7Y7E= -github.com/iotaledger/hive.go/runtime v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:Z9NFsByMh1Kf98f3v3ifeZRycbS2db1hjswTQG1MxnE= github.com/iotaledger/hive.go/runtime v0.0.0-20231207181026-f482ac139305 h1:7CW1/EbG+RvkjbyOf6JA1u1feax/cpex/6a8CLbaA4I= github.com/iotaledger/hive.go/runtime v0.0.0-20231207181026-f482ac139305/go.mod h1:Z9NFsByMh1Kf98f3v3ifeZRycbS2db1hjswTQG1MxnE= github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231206114953-6a65a82e30ad h1:c8uwbBZDqpiCNN9/9Jji7Z4lL0GdVnORp8WMouiuknk= github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231206114953-6a65a82e30ad/go.mod h1:FoH3T6yKlZJp8xm8K+zsQiibSynp32v21CpWx8xkek8= -github.com/iotaledger/hive.go/stringify v0.0.0-20231206114953-6a65a82e30ad h1:VC3OgdSbyngY7/gxVj66fKd/nGmN6P0/myr348nx7vA= -github.com/iotaledger/hive.go/stringify v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs= github.com/iotaledger/hive.go/stringify v0.0.0-20231207181026-f482ac139305 h1:KjbaklWvZb4zIcXBETHzl6XFTAf8wtAlFDfaF0Z1Daw= github.com/iotaledger/hive.go/stringify v0.0.0-20231207181026-f482ac139305/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs= github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231206124511-b78dc962031f h1:V68Ijq1A64gB9r0Rhc4ybLGH66rXqZ2Ly0L4uuaLrMg= github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231206124511-b78dc962031f/go.mod h1:Dy3Gv4Dn1zufB177x6IXETP3zTeiWQ1+HMVQR0Bt/ew= github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231206124145-f773dfe3927e h1:jbtiUlmTpTdGiRBW1pniPSqRcDMJaIW8fGS+uORryas= github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231206124145-f773dfe3927e/go.mod h1:zEb9onVHnDUStl5SsFBj7H0HBKfIN0c/pUND8Llfqp8= -github.com/iotaledger/iota.go/v4 v4.0.0-20231206123921-2af411eef0b5 h1:0KgQFpVRnKd6CdCwXo3Kg/SL27xkeKh2SMoU5G1TkZk= -github.com/iotaledger/iota.go/v4 v4.0.0-20231206123921-2af411eef0b5/go.mod h1:tiswk1O1wSAi9GE6tD1j43+bLmWU9Je07aZLaJ0+Ha0= +github.com/iotaledger/iota-crypto-demo v0.0.0-20231208171603-786bb32fdb00 h1:j5udgLtSN6wQgFI9vnhkdJsqsVdJmwtoc0yOmT/Ila4= +github.com/iotaledger/iota-crypto-demo v0.0.0-20231208171603-786bb32fdb00/go.mod h1:gt+URx7DZu414nZME7jtGgxR4DVTSnNa1jF2trTUTZ0= github.com/iotaledger/iota.go/v4 v4.0.0-20231211160706-492c65d5e3f5 h1:2iQUEuYvuyeYtZBr6bRoM4xFLxRiQ66aBPgKuJTirh0= github.com/iotaledger/iota.go/v4 v4.0.0-20231211160706-492c65d5e3f5/go.mod h1:jU3Jlnhwv4KWlpft84lu9/M0QkCykESl29bL4oCenKs= github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI= @@ -672,8 +660,6 @@ github.com/warpfork/go-wish v0.0.0-20220906213052-39a1cc7a02d0 h1:GDDkbFiaK8jsSD github.com/warpfork/go-wish v0.0.0-20220906213052-39a1cc7a02d0/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 h1:EKhdznlJHPMoKr0XTrX+IlJs1LH3lyx2nfr1dOlZ79k= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1/go.mod h1:8UvriyWtv5Q5EOgjHaSseUEdkQfvwFv1I/In/O2M9gc= -github.com/wollac/iota-crypto-demo v0.0.0-20221117162917-b10619eccb98 h1:i7k63xHOX2ntuHrhHewfKro67c834jug2DIk599fqAA= -github.com/wollac/iota-crypto-demo v0.0.0-20221117162917-b10619eccb98/go.mod h1:Knu2XMRWe8SkwTlHc/+ghP+O9DEaZRQQEyTjvLJ5Cck= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= diff --git a/pkg/protocol/engine/eviction/state.go b/pkg/protocol/engine/eviction/state.go index fbf3ef4f9..db36b0c79 100644 --- a/pkg/protocol/engine/eviction/state.go +++ b/pkg/protocol/engine/eviction/state.go @@ -329,7 +329,7 @@ func (s *State) activeIndexRange(targetSlot iotago.SlotIndex) (startSlot iotago. return genesisSlot, targetSlot } - rootBlocksWindowStart := targetSlot - maxCommittableAge + rootBlocksWindowStart := targetSlot - maxCommittableAge + 1 if latestNonEmptySlot := s.settings.LatestNonEmptySlot(); rootBlocksWindowStart > latestNonEmptySlot { rootBlocksWindowStart = latestNonEmptySlot diff --git a/pkg/protocol/engine/eviction/state_test.go b/pkg/protocol/engine/eviction/state_test.go index 7affc269f..30f36a844 100644 --- a/pkg/protocol/engine/eviction/state_test.go +++ b/pkg/protocol/engine/eviction/state_test.go @@ -21,7 +21,7 @@ func TestState_RootBlocks(t *testing.T) { TestAPISmallMCA := iotago.V3API(iotago.NewV3SnapshotProtocolParameters( iotago.WithStorageOptions(0, 0, 0, 0, 0, 0), // zero storage score iotago.WithWorkScoreOptions(0, 1, 0, 0, 0, 0, 0, 0, 0, 0), // all blocks workscore = 1 - iotago.WithLivenessOptions(5, 9, 1, 2, 4), + iotago.WithLivenessOptions(5, 9, 1, 3, 4), )) prunableStorage := prunable.New(database.Config{ diff --git a/pkg/tests/protocol_startup_test.go b/pkg/tests/protocol_startup_test.go index c40c8421a..c9e86a652 100644 --- a/pkg/tests/protocol_startup_test.go +++ b/pkg/tests/protocol_startup_test.go @@ -91,12 +91,12 @@ func Test_BookInCommittedSlot(t *testing.T) { ts.AssertBlocksInCachePreAccepted(ts.BlocksWithPrefixes("7.3"), true, ts.Nodes()...) var expectedActiveRootBlocks []*blocks.Block - for _, slot := range []iotago.SlotIndex{3, 4, 5} { - expectedActiveRootBlocks = append(expectedActiveRootBlocks, ts.BlocksWithPrefix(fmt.Sprintf("%d.3-", slot))...) + for _, slot := range []iotago.SlotIndex{2, 3, 4, 5} { + expectedActiveRootBlocks = append(expectedActiveRootBlocks, ts.BlocksWithPrefix(fmt.Sprintf("%d.", slot))...) } for _, slot := range []iotago.SlotIndex{1, 2, 3, 4, 5, 6} { - expectedStorageRootBlocksFrom0 = append(expectedStorageRootBlocksFrom0, ts.BlocksWithPrefix(fmt.Sprintf("%d.3-", slot))...) + expectedStorageRootBlocksFrom0 = append(expectedStorageRootBlocksFrom0, ts.BlocksWithPrefix(fmt.Sprintf("%d.", slot))...) } ts.AssertNodeState(ts.Nodes(), @@ -211,12 +211,12 @@ func Test_StartNodeFromSnapshotAndDisk(t *testing.T) { ts.AssertBlocksInCacheConfirmed(ts.BlocksWithPrefixes("6", "7.0", "7.1"), true, ts.Nodes()...) var expectedActiveRootBlocks []*blocks.Block - for _, slot := range []iotago.SlotIndex{3, 4, 5} { - expectedActiveRootBlocks = append(expectedActiveRootBlocks, ts.BlocksWithPrefix(fmt.Sprintf("%d.3-", slot))...) + for _, slot := range []iotago.SlotIndex{2, 3, 4, 5} { + expectedActiveRootBlocks = append(expectedActiveRootBlocks, ts.BlocksWithPrefix(fmt.Sprintf("%d.", slot))...) } for _, slot := range []iotago.SlotIndex{1, 2, 3, 4, 5, 6} { - expectedStorageRootBlocksFrom0 = append(expectedStorageRootBlocksFrom0, ts.BlocksWithPrefix(fmt.Sprintf("%d.3-", slot))...) + expectedStorageRootBlocksFrom0 = append(expectedStorageRootBlocksFrom0, ts.BlocksWithPrefix(fmt.Sprintf("%d.", slot))...) } ts.AssertNodeState(ts.Nodes(), @@ -257,14 +257,14 @@ func Test_StartNodeFromSnapshotAndDisk(t *testing.T) { ts.AssertBlocksInCacheConfirmed(ts.BlocksWithPrefixes("12", "13.0", "13.1", "13.2", "13.3"), true, ts.Nodes()...) var expectedActiveRootBlocks []*blocks.Block - for _, slot := range []iotago.SlotIndex{9, 11} { - b := ts.BlocksWithPrefix(fmt.Sprintf("%d.5-", slot)) + for _, slot := range []iotago.SlotIndex{8, 9, 11} { + b := ts.BlocksWithPrefix(fmt.Sprintf("%d.", slot)) expectedActiveRootBlocks = append(expectedActiveRootBlocks, b...) expectedStorageRootBlocksFrom9 = append(expectedStorageRootBlocksFrom9, b...) } for _, slot := range []iotago.SlotIndex{8, 9, 11} { - expectedStorageRootBlocksFrom0 = append(expectedStorageRootBlocksFrom0, ts.BlocksWithPrefix(fmt.Sprintf("%d.5-", slot))...) + expectedStorageRootBlocksFrom0 = append(expectedStorageRootBlocksFrom0, ts.BlocksWithPrefix(fmt.Sprintf("%d.", slot))...) } ts.AssertNodeState(ts.Nodes(), @@ -404,8 +404,8 @@ func Test_StartNodeFromSnapshotAndDisk(t *testing.T) { ts.IssueBlocksAtEpoch("", 4, 4, "31.3", ts.Nodes(), true, false) var expectedActiveRootBlocks []*blocks.Block - for _, slot := range []iotago.SlotIndex{35, 36, 37} { - expectedActiveRootBlocks = append(expectedActiveRootBlocks, ts.BlocksWithPrefix(fmt.Sprintf("%d.3-", slot))...) + for _, slot := range []iotago.SlotIndex{34, 35, 36, 37} { + expectedActiveRootBlocks = append(expectedActiveRootBlocks, ts.BlocksWithPrefix(fmt.Sprintf("%d.", slot))...) } ts.AssertNodeState(ts.Nodes(), From bf9d3e5e894ddccdd99b2eac6d813de6e76c96d5 Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Mon, 11 Dec 2023 18:14:22 +0100 Subject: [PATCH 06/12] Fix moar tests --- pkg/protocol/engine/eviction/state.go | 22 +++++----------------- pkg/tests/booker_test.go | 6 +++--- pkg/testsuite/blocks.go | 11 ----------- 3 files changed, 8 insertions(+), 31 deletions(-) diff --git a/pkg/protocol/engine/eviction/state.go b/pkg/protocol/engine/eviction/state.go index db36b0c79..0568652c7 100644 --- a/pkg/protocol/engine/eviction/state.go +++ b/pkg/protocol/engine/eviction/state.go @@ -47,29 +47,17 @@ func (s *State) Initialize(lastCommittedSlot iotago.SlotIndex) { func (s *State) AdvanceActiveWindowToIndex(slot iotago.SlotIndex) { s.evictionMutex.Lock() - protocolParams := s.settings.APIProvider().APIForSlot(slot).ProtocolParameters() - genesisSlot := protocolParams.GenesisSlot() - maxCommittableAge := protocolParams.MaxCommittableAge() - - s.lastCommittedSlot = slot - - if slot < maxCommittableAge+genesisSlot { + if slot <= s.lastCommittedSlot { s.evictionMutex.Unlock() return } - // We allow a maxCommittableAge window of available root blocks. - evictionSlot := slot - maxCommittableAge - - // We do not evict slots that are empty - if evictionSlot >= s.settings.LatestNonEmptySlot() { - s.evictionMutex.Unlock() - return - } + s.lastCommittedSlot = slot s.evictionMutex.Unlock() - s.Events.SlotEvicted.Trigger(evictionSlot) + // We only delay eviction in the Eviction State, but components evict on committment, which in this context is slot. + s.Events.SlotEvicted.Trigger(slot) } func (s *State) LastEvictedSlot() iotago.SlotIndex { @@ -121,7 +109,7 @@ func (s *State) LatestActiveRootBlock() (iotago.BlockID, iotago.CommitmentID) { defer s.evictionMutex.RUnlock() startSlot, endSlot := s.activeIndexRange(s.lastCommittedSlot) - for slot := endSlot; slot >= startSlot; slot-- { + for slot := endSlot; slot >= startSlot && slot > 0; slot-- { // We assume the cache is always populated for the latest slots. storage, err := s.rootBlockStorageFunc(slot) // Slot too old, it was pruned. diff --git a/pkg/tests/booker_test.go b/pkg/tests/booker_test.go index 36b31c2e0..b5407ba42 100644 --- a/pkg/tests/booker_test.go +++ b/pkg/tests/booker_test.go @@ -723,7 +723,7 @@ func Test_SpendPendingCommittedRace(t *testing.T) { ts.AssertBlocksInCacheConflicts(map[*blocks.Block][]string{ ts.Block("block2.1"): {"tx1"}, ts.Block("n2-pending-genesis"): {"tx1"}, - ts.Block("n2-pending-commit1"): {}, // no conflits inherited as the block merges orphaned conflicts. + // ts.Block("n2-pending-commit1"): {}, // no conflits inherited as the block merges orphaned conflicts. }, node2) } @@ -753,7 +753,7 @@ func Test_SpendPendingCommittedRace(t *testing.T) { ts.AssertBlocksInCacheConflicts(map[*blocks.Block][]string{ ts.Block("block2.1"): {"tx1"}, ts.Block("n2-pending-genesis"): {"tx1"}, - ts.Block("n2-pending-commit1"): {}, // no conflits inherited as the block merges orphaned conflicts. + // ts.Block("n2-pending-commit1"): {}, // no conflits inherited as the block merges orphaned conflicts. }, node1, node2) ts.AssertTransactionsInCachePending(wallet.Transactions("tx1", "tx2"), true, node1, node2) @@ -820,7 +820,7 @@ func Test_RootBlockShallowLike(t *testing.T) { ts.IssueBlocksAtSlots("", []iotago.SlotIndex{2, 3, 4}, 2, "block", ts.Nodes(), true, false) - ts.AssertActiveRootBlocks(ts.Blocks("Genesis", "block1", "block2", "2.1-node1"), ts.Nodes()...) + ts.AssertActiveRootBlocks(append(ts.Blocks("Genesis", "block1", "block2"), ts.BlocksWithPrefix("2.")...), ts.Nodes()...) ts.IssueBasicBlockWithOptions("block-shallow-like-valid", wallet, &iotago.TaggedData{}, mock.WithStrongParents(ts.BlockID("4.1-node1")), mock.WithShallowLikeParents(ts.BlockID("block1")), mock.WithIssuingTime(ts.API.TimeProvider().SlotStartTime(5))) ts.AssertBlocksInCacheBooked(ts.Blocks("block-shallow-like-valid"), true, node1) diff --git a/pkg/testsuite/blocks.go b/pkg/testsuite/blocks.go index bfce8b917..f8a35ea47 100644 --- a/pkg/testsuite/blocks.go +++ b/pkg/testsuite/blocks.go @@ -2,7 +2,6 @@ package testsuite import ( "fmt" - "strings" "github.com/stretchr/testify/assert" @@ -18,12 +17,6 @@ import ( func (t *TestSuite) AssertBlock(block *blocks.Block, node *mock.Node) *model.Block { var loadedBlock *model.Block t.Eventually(func() error { - fmt.Println("Exists? ", block.ID()) - - if strings.HasPrefix(block.ID().String(), "BlockID(3.1-node") { - fmt.Println("break me") - } - var exists bool loadedBlock, exists = node.Protocol.Engines.Main.Get().Block(block.ID()) if !exists { @@ -32,16 +25,12 @@ func (t *TestSuite) AssertBlock(block *blocks.Block, node *mock.Node) *model.Blo } if block.ID() != loadedBlock.ID() { - fmt.Println("return2 ", block.ID()) return ierrors.Errorf("AssertBlock: %s: expected %s, got %s", node.Name, block.ID(), loadedBlock.ID()) } if !assert.Equal(t.fakeTesting, block.ModelBlock().Data(), loadedBlock.Data()) { - fmt.Println("return3 ", block.ID()) return ierrors.Errorf("AssertBlock: %s: expected %s, got %s", node.Name, block.ModelBlock().Data(), loadedBlock.Data()) } - fmt.Println("return4 nil nil nil ", block.ID()) - return nil }) From dfec0bc053c6c7e186e0435e22eadac35277ceab Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Tue, 12 Dec 2023 12:24:16 +0100 Subject: [PATCH 07/12] Fix root blocks identification: active vs non-active --- .../blockdag/inmemoryblockdag/blockdag.go | 6 ++--- pkg/protocol/engine/blocks/blocks.go | 13 ++++++----- pkg/protocol/engine/eviction/state.go | 22 ++++++++----------- 3 files changed, 19 insertions(+), 22 deletions(-) diff --git a/pkg/protocol/engine/blockdag/inmemoryblockdag/blockdag.go b/pkg/protocol/engine/blockdag/inmemoryblockdag/blockdag.go index 6a9c3cdf4..0e690bd11 100644 --- a/pkg/protocol/engine/blockdag/inmemoryblockdag/blockdag.go +++ b/pkg/protocol/engine/blockdag/inmemoryblockdag/blockdag.go @@ -191,7 +191,7 @@ func (b *BlockDAG) attach(data *model.Block) (block *blocks.Block, wasAttached b // canAttach determines if the Block can be attached (does not exist and addresses a recent slot). func (b *BlockDAG) shouldAttach(data *model.Block) (shouldAttach bool, err error) { - if b.evictionState.InRootBlockSlot(data.ID()) && !b.evictionState.IsRootBlock(data.ID()) { + if b.evictionState.InActiveRootBlockRange(data.ID()) && !b.evictionState.IsActiveRootBlock(data.ID()) { b.retainBlockFailure(data.ID(), api.BlockFailureIsTooOld) return false, ierrors.Errorf("block data with %s is too old (issued at: %s)", data.ID(), data.ProtocolBlock().Header.IssuingTime) } @@ -218,7 +218,7 @@ func (b *BlockDAG) shouldAttach(data *model.Block) (shouldAttach bool, err error // this condition but exists as a missing entry, we mark it as invalid. func (b *BlockDAG) canAttachToParents(modelBlock *model.Block) (parentsValid bool, err error) { for _, parentID := range modelBlock.ProtocolBlock().Parents() { - if b.evictionState.InRootBlockSlot(parentID) && !b.evictionState.IsRootBlock(parentID) { + if b.evictionState.InActiveRootBlockRange(parentID) && !b.evictionState.IsActiveRootBlock(parentID) { b.retainBlockFailure(modelBlock.ID(), api.BlockFailureParentIsTooOld) return false, ierrors.Errorf("parent %s of block %s is too old", parentID, modelBlock.ID()) } @@ -230,7 +230,7 @@ func (b *BlockDAG) canAttachToParents(modelBlock *model.Block) (parentsValid boo // registerChild registers the given Block as a child of the parent. It triggers a BlockMissing event if the referenced // Block does not exist, yet. func (b *BlockDAG) registerChild(child *blocks.Block, parent iotago.Parent) { - if b.evictionState.IsRootBlock(parent.ID) { + if b.evictionState.IsActiveRootBlock(parent.ID) { return } diff --git a/pkg/protocol/engine/blocks/blocks.go b/pkg/protocol/engine/blocks/blocks.go index 3ae4e69a7..facb969c0 100644 --- a/pkg/protocol/engine/blocks/blocks.go +++ b/pkg/protocol/engine/blocks/blocks.go @@ -39,16 +39,17 @@ func (b *Blocks) Block(id iotago.BlockID) (block *Block, exists bool) { b.evictionMutex.RLock() defer b.evictionMutex.RUnlock() - if commitmentID, isRootBlock := b.evictionState.RootBlockCommitmentID(id); isRootBlock { - return NewRootBlock(id, commitmentID, b.apiProvider.APIForSlot(id.Slot()).TimeProvider().SlotEndTime(id.Slot())), true + if storage := b.blocks.Get(id.Slot()); storage != nil { + if block, exists = storage.Get(id); exists { + return block, true + } } - storage := b.blocks.Get(id.Slot(), false) - if storage == nil { - return nil, false + if commitmentID, isRootBlock := b.evictionState.RootBlockCommitmentID(id); isRootBlock { + return NewRootBlock(id, commitmentID, b.apiProvider.APIForSlot(id.Slot()).TimeProvider().SlotEndTime(id.Slot())), true } - return storage.Get(id) + return nil, false } func (b *Blocks) StoreOrUpdate(data *model.Block) (storedBlock *Block, evicted bool, updated bool) { diff --git a/pkg/protocol/engine/eviction/state.go b/pkg/protocol/engine/eviction/state.go index 0568652c7..f98d60e94 100644 --- a/pkg/protocol/engine/eviction/state.go +++ b/pkg/protocol/engine/eviction/state.go @@ -10,8 +10,6 @@ import ( "github.com/iotaledger/hive.go/serializer/v2" "github.com/iotaledger/hive.go/serializer/v2/stream" - // "github.com/iotaledger/hive.go/serializer/v2" - // "github.com/iotaledger/hive.go/serializer/v2/stream" "github.com/iotaledger/iota-core/pkg/model" "github.com/iotaledger/iota-core/pkg/storage/permanent" "github.com/iotaledger/iota-core/pkg/storage/prunable/slotstore" @@ -67,8 +65,9 @@ func (s *State) LastEvictedSlot() iotago.SlotIndex { return s.lastCommittedSlot } -// InRootBlockSlot checks if the Block associated with the given id is too old. -func (s *State) InRootBlockSlot(id iotago.BlockID) bool { +// InActiveRootBlockRange checks if the Block associated with the given id is within the active root block range with +// respect to the latest committed slot. +func (s *State) InActiveRootBlockRange(id iotago.BlockID) bool { s.evictionMutex.RLock() defer s.evictionMutex.RUnlock() @@ -165,8 +164,8 @@ func (s *State) RemoveRootBlock(id iotago.BlockID) { } } -// IsRootBlock returns true if the given block is a root block. -func (s *State) IsRootBlock(id iotago.BlockID) (has bool) { +// IsActiveRootBlock returns true if the given block is an _active_ root block. +func (s *State) IsActiveRootBlock(id iotago.BlockID) (has bool) { s.evictionMutex.RLock() defer s.evictionMutex.RUnlock() @@ -182,15 +181,11 @@ func (s *State) IsRootBlock(id iotago.BlockID) (has bool) { return lo.PanicOnErr(storage.Has(id)) } -// RootBlockCommitmentID returns the commitmentID if it is a known root block. +// RootBlockCommitmentID returns the commitmentID if it is a known root block, _no matter if active or not_. func (s *State) RootBlockCommitmentID(id iotago.BlockID) (commitmentID iotago.CommitmentID, exists bool) { s.evictionMutex.RLock() defer s.evictionMutex.RUnlock() - if !s.withinActiveIndexRange(id.Slot()) { - return iotago.CommitmentID{}, false - } - storage, err := s.rootBlockStorageFunc(id.Slot()) if err != nil { return iotago.EmptyCommitmentID, false @@ -214,6 +209,7 @@ func (s *State) Export(writer io.WriteSeeker, lowerTarget iotago.SlotIndex, targ defer s.evictionMutex.RUnlock() start, _ := s.activeIndexRange(lowerTarget) + latestNonEmptySlot := s.settings.APIProvider().APIForSlot(targetSlot).ProtocolParameters().GenesisSlot() if err := stream.WriteCollection(writer, serializer.SeriLengthPrefixTypeAsUint32, func() (elementsCount int, err error) { @@ -233,12 +229,12 @@ func (s *State) Export(writer io.WriteSeeker, lowerTarget iotago.SlotIndex, targ elementsCount++ + latestNonEmptySlot = currentSlot + return }); err != nil { return 0, ierrors.Wrap(err, "failed to stream root blocks") } - - latestNonEmptySlot = currentSlot } return elementsCount, nil From 7cf41f79ee916f627a110007e9b9b47de9bd2a4d Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Tue, 12 Dec 2023 12:35:53 +0100 Subject: [PATCH 08/12] Fix upgrade signaling test --- pkg/tests/upgrade_signaling_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/tests/upgrade_signaling_test.go b/pkg/tests/upgrade_signaling_test.go index a9d81f508..ce7ae6a3b 100644 --- a/pkg/tests/upgrade_signaling_test.go +++ b/pkg/tests/upgrade_signaling_test.go @@ -239,8 +239,8 @@ func Test_Upgrade_Signaling(t *testing.T) { { var expectedRootBlocks []*blocks.Block - for _, slot := range []iotago.SlotIndex{39, 40, 41} { - expectedRootBlocks = append(expectedRootBlocks, ts.BlocksWithPrefix(fmt.Sprintf("%d.3-", slot))...) + for _, slot := range []iotago.SlotIndex{38, 39, 40, 41} { + expectedRootBlocks = append(expectedRootBlocks, ts.BlocksWithPrefix(fmt.Sprintf("%d.", slot))...) } ts.AssertNodeState(ts.Nodes(), @@ -297,8 +297,8 @@ func Test_Upgrade_Signaling(t *testing.T) { // Restart node (and add protocol parameters) and add another node from snapshot (also with protocol parameters already set). { var expectedRootBlocks []*blocks.Block - for _, slot := range []iotago.SlotIndex{59, 60, 61} { - expectedRootBlocks = append(expectedRootBlocks, ts.BlocksWithPrefix(fmt.Sprintf("%d.3-", slot))...) + for _, slot := range []iotago.SlotIndex{58, 59, 60, 61} { + expectedRootBlocks = append(expectedRootBlocks, ts.BlocksWithPrefix(fmt.Sprintf("%d.", slot))...) } ts.AssertNodeState(ts.Nodes(), From 8b0c85ea77a486b2634b14d1e307395b9d4a174b Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Tue, 12 Dec 2023 13:03:14 +0100 Subject: [PATCH 09/12] Tidy go mod --- tools/gendoc/go.mod | 16 ++++++++-------- tools/gendoc/go.sum | 32 ++++++++++++++++---------------- tools/genesis-snapshot/go.mod | 16 ++++++++-------- tools/genesis-snapshot/go.sum | 32 ++++++++++++++++---------------- 4 files changed, 48 insertions(+), 48 deletions(-) diff --git a/tools/gendoc/go.mod b/tools/gendoc/go.mod index 13874cbf0..c4fae86ad 100644 --- a/tools/gendoc/go.mod +++ b/tools/gendoc/go.mod @@ -57,21 +57,22 @@ require ( github.com/iancoleman/orderedmap v0.3.0 // indirect github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 // indirect github.com/iotaledger/hive.go/ads v0.0.0-20231205131244-472357435a39 // indirect - github.com/iotaledger/hive.go/constraints v0.0.0-20231206114953-6a65a82e30ad // indirect + github.com/iotaledger/hive.go/constraints v0.0.0-20231207181026-f482ac139305 // indirect github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231206114953-6a65a82e30ad // indirect - github.com/iotaledger/hive.go/crypto v0.0.0-20231206114953-6a65a82e30ad // indirect + github.com/iotaledger/hive.go/crypto v0.0.0-20231207181026-f482ac139305 // indirect github.com/iotaledger/hive.go/ds v0.0.0-20231206114953-6a65a82e30ad // indirect - github.com/iotaledger/hive.go/ierrors v0.0.0-20231206114953-6a65a82e30ad // indirect + github.com/iotaledger/hive.go/ierrors v0.0.0-20231207181026-f482ac139305 // indirect github.com/iotaledger/hive.go/kvstore v0.0.0-20231205131244-472357435a39 // indirect - github.com/iotaledger/hive.go/lo v0.0.0-20231206114953-6a65a82e30ad // indirect + github.com/iotaledger/hive.go/lo v0.0.0-20231207181026-f482ac139305 // indirect github.com/iotaledger/hive.go/log v0.0.0-20231206113509-4b4ff95ac61c // indirect github.com/iotaledger/hive.go/logger v0.0.0-20231206114953-6a65a82e30ad // indirect - github.com/iotaledger/hive.go/runtime v0.0.0-20231206114953-6a65a82e30ad // indirect + github.com/iotaledger/hive.go/runtime v0.0.0-20231207181026-f482ac139305 // indirect github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231206114953-6a65a82e30ad // indirect - github.com/iotaledger/hive.go/stringify v0.0.0-20231206114953-6a65a82e30ad // indirect + github.com/iotaledger/hive.go/stringify v0.0.0-20231207181026-f482ac139305 // indirect github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231206124511-b78dc962031f // indirect github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231206124145-f773dfe3927e // indirect - github.com/iotaledger/iota.go/v4 v4.0.0-20231206123921-2af411eef0b5 // indirect + github.com/iotaledger/iota-crypto-demo v0.0.0-20231208171603-786bb32fdb00 // indirect + github.com/iotaledger/iota.go/v4 v4.0.0-20231211160706-492c65d5e3f5 // indirect github.com/ipfs/boxo v0.13.1 // indirect github.com/ipfs/go-cid v0.4.1 // indirect github.com/ipfs/go-datastore v0.6.0 // indirect @@ -152,7 +153,6 @@ require ( github.com/valyala/bytebufferpool v1.0.0 // indirect github.com/valyala/fasttemplate v1.2.2 // indirect github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 // indirect - github.com/wollac/iota-crypto-demo v0.0.0-20221117162917-b10619eccb98 // indirect github.com/zyedidia/generic v1.2.1 // indirect go.opencensus.io v0.24.0 // indirect go.opentelemetry.io/otel v1.19.0 // indirect diff --git a/tools/gendoc/go.sum b/tools/gendoc/go.sum index 3d2ccafdd..c8adb9d42 100644 --- a/tools/gendoc/go.sum +++ b/tools/gendoc/go.sum @@ -283,36 +283,38 @@ github.com/iotaledger/hive.go/app v0.0.0-20231206114953-6a65a82e30ad h1:v7dkbVLS github.com/iotaledger/hive.go/app v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:hTHKGFbZnuiW8yEgDuuL7ZjQTCnl8bXyHLmj3LPa648= github.com/iotaledger/hive.go/apputils v0.0.0-20230829152614-7afc7a4d89b3 h1:4aVJTc0KS77uEw0Tny4r0n1ORwcbAQDECaCclgf/6lE= github.com/iotaledger/hive.go/apputils v0.0.0-20230829152614-7afc7a4d89b3/go.mod h1:TZeAqieDu+xDOZp2e9+S+8pZp1PrfgcwLUnxmd8IgLU= -github.com/iotaledger/hive.go/constraints v0.0.0-20231206114953-6a65a82e30ad h1:4XL7IIvdsWHxSKQfU+sgq3H9egN54053LF9TwMfDcTg= -github.com/iotaledger/hive.go/constraints v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s= +github.com/iotaledger/hive.go/constraints v0.0.0-20231207181026-f482ac139305 h1:ar+IWfqO7B1M5+kuKGUJnfg0i/YuuM1oN5i8byp/F7A= +github.com/iotaledger/hive.go/constraints v0.0.0-20231207181026-f482ac139305/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s= github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231206114953-6a65a82e30ad h1:iNzb/Oy/nucIOXOzRcwSqqFsaeKwr2JZpZYSLp8xjlE= github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231206114953-6a65a82e30ad/go.mod h1:CO28KMA6Pp5LJPiigPQQ276zQofES+jMod08U5pyRFA= -github.com/iotaledger/hive.go/crypto v0.0.0-20231206114953-6a65a82e30ad h1:pUL2UZbF4S8FIV7uKo9p+IGfZ658K1VNorQ6rzDMRvs= -github.com/iotaledger/hive.go/crypto v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:7vHoF//1Pt3nu0l8nDIw7bEgv2GfbL3kSgjp7Rdqhd4= +github.com/iotaledger/hive.go/crypto v0.0.0-20231207181026-f482ac139305 h1:OR2TClxTtst906F4tok9xzhBTKO81qrUFdxIAoaZVvE= +github.com/iotaledger/hive.go/crypto v0.0.0-20231207181026-f482ac139305/go.mod h1:7vHoF//1Pt3nu0l8nDIw7bEgv2GfbL3kSgjp7Rdqhd4= github.com/iotaledger/hive.go/ds v0.0.0-20231206114953-6a65a82e30ad h1:adLrD6dOEkM5Xdg6AOPt9/HYqy/pQ5FrprDpW4/VqUU= github.com/iotaledger/hive.go/ds v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:NmZRIoxtL6iQdVK6n5W+JOx58K/0Yn8k7WuSvpKPQ+M= -github.com/iotaledger/hive.go/ierrors v0.0.0-20231206114953-6a65a82e30ad h1:WDl58zJKHfwbzHs+ZB8Jq3YNgVQE5Neu2NeaX3FZuyU= -github.com/iotaledger/hive.go/ierrors v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8= +github.com/iotaledger/hive.go/ierrors v0.0.0-20231207181026-f482ac139305 h1:v7/zMhNcr6hibXFZXZ4xV/S27ESUytQFgUQ1oo10iic= +github.com/iotaledger/hive.go/ierrors v0.0.0-20231207181026-f482ac139305/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8= github.com/iotaledger/hive.go/kvstore v0.0.0-20231205131244-472357435a39 h1:Gp2h+Els9cTVYYnYsHX3zLuixb0XggIj2okK570aKww= github.com/iotaledger/hive.go/kvstore v0.0.0-20231205131244-472357435a39/go.mod h1:ytfKoHr/nF8u0y0G4mamfG0yjFtJiJVk0kgjnPOtsSY= -github.com/iotaledger/hive.go/lo v0.0.0-20231206114953-6a65a82e30ad h1:qpCsjw+InLL824QPu3lY/osck4DhucBKhCs5/E8OH+A= -github.com/iotaledger/hive.go/lo v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:ETXGXymFyNcUq2t4I9e7ZK18f9bxUWYat4pjZ9W0rWc= +github.com/iotaledger/hive.go/lo v0.0.0-20231207181026-f482ac139305 h1:zxVbTEWutMvZhS0VLu/OmBk2WpMjrXQ7l67VBwsExtc= +github.com/iotaledger/hive.go/lo v0.0.0-20231207181026-f482ac139305/go.mod h1:ETXGXymFyNcUq2t4I9e7ZK18f9bxUWYat4pjZ9W0rWc= github.com/iotaledger/hive.go/log v0.0.0-20231206113509-4b4ff95ac61c h1:Ksts6VjPj9y0o2Nis+2tHtDGWITNJ4yju87ZlHLPuOo= github.com/iotaledger/hive.go/log v0.0.0-20231206113509-4b4ff95ac61c/go.mod h1:Td3R6QmYw0svZI1GuZ/tN9s0VNaHassXSKLCc70aX9w= github.com/iotaledger/hive.go/logger v0.0.0-20231206114953-6a65a82e30ad h1:fazCxogqOLDEPNDPWYDLTDpYmwgTJgIaC2Z6VN52S4M= github.com/iotaledger/hive.go/logger v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:hVaVODS+Uik0obf3SVEHFQNruUko/uqIgD/GKwhn49M= -github.com/iotaledger/hive.go/runtime v0.0.0-20231206114953-6a65a82e30ad h1:HpupWK8iqFt+Sdogkh2/N8ojalmevYy+FzhjOuy7Y7E= -github.com/iotaledger/hive.go/runtime v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:Z9NFsByMh1Kf98f3v3ifeZRycbS2db1hjswTQG1MxnE= +github.com/iotaledger/hive.go/runtime v0.0.0-20231207181026-f482ac139305 h1:7CW1/EbG+RvkjbyOf6JA1u1feax/cpex/6a8CLbaA4I= +github.com/iotaledger/hive.go/runtime v0.0.0-20231207181026-f482ac139305/go.mod h1:Z9NFsByMh1Kf98f3v3ifeZRycbS2db1hjswTQG1MxnE= github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231206114953-6a65a82e30ad h1:c8uwbBZDqpiCNN9/9Jji7Z4lL0GdVnORp8WMouiuknk= github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231206114953-6a65a82e30ad/go.mod h1:FoH3T6yKlZJp8xm8K+zsQiibSynp32v21CpWx8xkek8= -github.com/iotaledger/hive.go/stringify v0.0.0-20231206114953-6a65a82e30ad h1:VC3OgdSbyngY7/gxVj66fKd/nGmN6P0/myr348nx7vA= -github.com/iotaledger/hive.go/stringify v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs= +github.com/iotaledger/hive.go/stringify v0.0.0-20231207181026-f482ac139305 h1:KjbaklWvZb4zIcXBETHzl6XFTAf8wtAlFDfaF0Z1Daw= +github.com/iotaledger/hive.go/stringify v0.0.0-20231207181026-f482ac139305/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs= github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231206124511-b78dc962031f h1:V68Ijq1A64gB9r0Rhc4ybLGH66rXqZ2Ly0L4uuaLrMg= github.com/iotaledger/inx-app v1.0.0-rc.3.0.20231206124511-b78dc962031f/go.mod h1:Dy3Gv4Dn1zufB177x6IXETP3zTeiWQ1+HMVQR0Bt/ew= github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231206124145-f773dfe3927e h1:jbtiUlmTpTdGiRBW1pniPSqRcDMJaIW8fGS+uORryas= github.com/iotaledger/inx/go v1.0.0-rc.2.0.20231206124145-f773dfe3927e/go.mod h1:zEb9onVHnDUStl5SsFBj7H0HBKfIN0c/pUND8Llfqp8= -github.com/iotaledger/iota.go/v4 v4.0.0-20231206123921-2af411eef0b5 h1:0KgQFpVRnKd6CdCwXo3Kg/SL27xkeKh2SMoU5G1TkZk= -github.com/iotaledger/iota.go/v4 v4.0.0-20231206123921-2af411eef0b5/go.mod h1:tiswk1O1wSAi9GE6tD1j43+bLmWU9Je07aZLaJ0+Ha0= +github.com/iotaledger/iota-crypto-demo v0.0.0-20231208171603-786bb32fdb00 h1:j5udgLtSN6wQgFI9vnhkdJsqsVdJmwtoc0yOmT/Ila4= +github.com/iotaledger/iota-crypto-demo v0.0.0-20231208171603-786bb32fdb00/go.mod h1:gt+URx7DZu414nZME7jtGgxR4DVTSnNa1jF2trTUTZ0= +github.com/iotaledger/iota.go/v4 v4.0.0-20231211160706-492c65d5e3f5 h1:2iQUEuYvuyeYtZBr6bRoM4xFLxRiQ66aBPgKuJTirh0= +github.com/iotaledger/iota.go/v4 v4.0.0-20231211160706-492c65d5e3f5/go.mod h1:jU3Jlnhwv4KWlpft84lu9/M0QkCykESl29bL4oCenKs= github.com/ipfs/boxo v0.13.1 h1:nQ5oQzcMZR3oL41REJDcTbrvDvuZh3J9ckc9+ILeRQI= github.com/ipfs/boxo v0.13.1/go.mod h1:btrtHy0lmO1ODMECbbEY1pxNtrLilvKSYLoGQt1yYCk= github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s= @@ -662,8 +664,6 @@ github.com/warpfork/go-wish v0.0.0-20220906213052-39a1cc7a02d0 h1:GDDkbFiaK8jsSD github.com/warpfork/go-wish v0.0.0-20220906213052-39a1cc7a02d0/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1 h1:EKhdznlJHPMoKr0XTrX+IlJs1LH3lyx2nfr1dOlZ79k= github.com/whyrusleeping/go-keyspace v0.0.0-20160322163242-5b898ac5add1/go.mod h1:8UvriyWtv5Q5EOgjHaSseUEdkQfvwFv1I/In/O2M9gc= -github.com/wollac/iota-crypto-demo v0.0.0-20221117162917-b10619eccb98 h1:i7k63xHOX2ntuHrhHewfKro67c834jug2DIk599fqAA= -github.com/wollac/iota-crypto-demo v0.0.0-20221117162917-b10619eccb98/go.mod h1:Knu2XMRWe8SkwTlHc/+ghP+O9DEaZRQQEyTjvLJ5Cck= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= diff --git a/tools/genesis-snapshot/go.mod b/tools/genesis-snapshot/go.mod index 756d9ce51..85197502e 100644 --- a/tools/genesis-snapshot/go.mod +++ b/tools/genesis-snapshot/go.mod @@ -5,12 +5,12 @@ go 1.21 replace github.com/iotaledger/iota-core => ../../ require ( - github.com/iotaledger/hive.go/crypto v0.0.0-20231206114953-6a65a82e30ad - github.com/iotaledger/hive.go/ierrors v0.0.0-20231206114953-6a65a82e30ad - github.com/iotaledger/hive.go/lo v0.0.0-20231206114953-6a65a82e30ad - github.com/iotaledger/hive.go/runtime v0.0.0-20231206114953-6a65a82e30ad + github.com/iotaledger/hive.go/crypto v0.0.0-20231207181026-f482ac139305 + github.com/iotaledger/hive.go/ierrors v0.0.0-20231207181026-f482ac139305 + github.com/iotaledger/hive.go/lo v0.0.0-20231207181026-f482ac139305 + github.com/iotaledger/hive.go/runtime v0.0.0-20231207181026-f482ac139305 github.com/iotaledger/iota-core v0.0.0-00010101000000-000000000000 - github.com/iotaledger/iota.go/v4 v4.0.0-20231206123921-2af411eef0b5 + github.com/iotaledger/iota.go/v4 v4.0.0-20231211160706-492c65d5e3f5 github.com/mr-tron/base58 v1.2.0 github.com/spf13/pflag v1.0.5 golang.org/x/crypto v0.16.0 @@ -27,13 +27,14 @@ require ( github.com/iancoleman/orderedmap v0.3.0 // indirect github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 // indirect github.com/iotaledger/hive.go/ads v0.0.0-20231205131244-472357435a39 // indirect - github.com/iotaledger/hive.go/constraints v0.0.0-20231206114953-6a65a82e30ad // indirect + github.com/iotaledger/hive.go/constraints v0.0.0-20231207181026-f482ac139305 // indirect github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231206114953-6a65a82e30ad // indirect github.com/iotaledger/hive.go/ds v0.0.0-20231206114953-6a65a82e30ad // indirect github.com/iotaledger/hive.go/kvstore v0.0.0-20231205131244-472357435a39 // indirect github.com/iotaledger/hive.go/log v0.0.0-20231206113509-4b4ff95ac61c // indirect github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231206114953-6a65a82e30ad // indirect - github.com/iotaledger/hive.go/stringify v0.0.0-20231206114953-6a65a82e30ad // indirect + github.com/iotaledger/hive.go/stringify v0.0.0-20231207181026-f482ac139305 // indirect + github.com/iotaledger/iota-crypto-demo v0.0.0-20231208171603-786bb32fdb00 // indirect github.com/ipfs/go-cid v0.4.1 // indirect github.com/klauspost/cpuid/v2 v2.2.5 // indirect github.com/kr/text v0.2.0 // indirect @@ -56,7 +57,6 @@ require ( github.com/sasha-s/go-deadlock v0.3.1 // indirect github.com/spaolacci/murmur3 v1.1.0 // indirect github.com/stretchr/testify v1.8.4 // indirect - github.com/wollac/iota-crypto-demo v0.0.0-20221117162917-b10619eccb98 // indirect github.com/zyedidia/generic v1.2.1 // indirect go.uber.org/atomic v1.11.0 // indirect golang.org/x/exp v0.0.0-20231006140011-7918f672742d // indirect diff --git a/tools/genesis-snapshot/go.sum b/tools/genesis-snapshot/go.sum index 578c00428..7249508ff 100644 --- a/tools/genesis-snapshot/go.sum +++ b/tools/genesis-snapshot/go.sum @@ -30,30 +30,32 @@ github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7 h1:dTrD7X2PT github.com/iotaledger/grocksdb v1.7.5-0.20230220105546-5162e18885c7/go.mod h1:ZRdPu684P0fQ1z8sXz4dj9H5LWHhz4a9oCtvjunkSrw= github.com/iotaledger/hive.go/ads v0.0.0-20231205131244-472357435a39 h1:jxoBAPgC4I73pAwvEWI2IUCxiI1xN68IaFZ5WC1D3ek= github.com/iotaledger/hive.go/ads v0.0.0-20231205131244-472357435a39/go.mod h1:gbUvr01B5ha530GnNm8K2OsHXOd2BtzBYOMxyTX3iDg= -github.com/iotaledger/hive.go/constraints v0.0.0-20231206114953-6a65a82e30ad h1:4XL7IIvdsWHxSKQfU+sgq3H9egN54053LF9TwMfDcTg= -github.com/iotaledger/hive.go/constraints v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s= +github.com/iotaledger/hive.go/constraints v0.0.0-20231207181026-f482ac139305 h1:ar+IWfqO7B1M5+kuKGUJnfg0i/YuuM1oN5i8byp/F7A= +github.com/iotaledger/hive.go/constraints v0.0.0-20231207181026-f482ac139305/go.mod h1:dOBOM2s4se3HcWefPe8sQLUalGXJ8yVXw58oK8jke3s= github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231206114953-6a65a82e30ad h1:iNzb/Oy/nucIOXOzRcwSqqFsaeKwr2JZpZYSLp8xjlE= github.com/iotaledger/hive.go/core v1.0.0-rc.3.0.20231206114953-6a65a82e30ad/go.mod h1:CO28KMA6Pp5LJPiigPQQ276zQofES+jMod08U5pyRFA= -github.com/iotaledger/hive.go/crypto v0.0.0-20231206114953-6a65a82e30ad h1:pUL2UZbF4S8FIV7uKo9p+IGfZ658K1VNorQ6rzDMRvs= -github.com/iotaledger/hive.go/crypto v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:7vHoF//1Pt3nu0l8nDIw7bEgv2GfbL3kSgjp7Rdqhd4= +github.com/iotaledger/hive.go/crypto v0.0.0-20231207181026-f482ac139305 h1:OR2TClxTtst906F4tok9xzhBTKO81qrUFdxIAoaZVvE= +github.com/iotaledger/hive.go/crypto v0.0.0-20231207181026-f482ac139305/go.mod h1:7vHoF//1Pt3nu0l8nDIw7bEgv2GfbL3kSgjp7Rdqhd4= github.com/iotaledger/hive.go/ds v0.0.0-20231206114953-6a65a82e30ad h1:adLrD6dOEkM5Xdg6AOPt9/HYqy/pQ5FrprDpW4/VqUU= github.com/iotaledger/hive.go/ds v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:NmZRIoxtL6iQdVK6n5W+JOx58K/0Yn8k7WuSvpKPQ+M= -github.com/iotaledger/hive.go/ierrors v0.0.0-20231206114953-6a65a82e30ad h1:WDl58zJKHfwbzHs+ZB8Jq3YNgVQE5Neu2NeaX3FZuyU= -github.com/iotaledger/hive.go/ierrors v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8= +github.com/iotaledger/hive.go/ierrors v0.0.0-20231207181026-f482ac139305 h1:v7/zMhNcr6hibXFZXZ4xV/S27ESUytQFgUQ1oo10iic= +github.com/iotaledger/hive.go/ierrors v0.0.0-20231207181026-f482ac139305/go.mod h1:HcE8B5lP96enc/OALTb2/rIIi+yOLouRoHOKRclKmC8= github.com/iotaledger/hive.go/kvstore v0.0.0-20231205131244-472357435a39 h1:Gp2h+Els9cTVYYnYsHX3zLuixb0XggIj2okK570aKww= github.com/iotaledger/hive.go/kvstore v0.0.0-20231205131244-472357435a39/go.mod h1:ytfKoHr/nF8u0y0G4mamfG0yjFtJiJVk0kgjnPOtsSY= -github.com/iotaledger/hive.go/lo v0.0.0-20231206114953-6a65a82e30ad h1:qpCsjw+InLL824QPu3lY/osck4DhucBKhCs5/E8OH+A= -github.com/iotaledger/hive.go/lo v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:ETXGXymFyNcUq2t4I9e7ZK18f9bxUWYat4pjZ9W0rWc= +github.com/iotaledger/hive.go/lo v0.0.0-20231207181026-f482ac139305 h1:zxVbTEWutMvZhS0VLu/OmBk2WpMjrXQ7l67VBwsExtc= +github.com/iotaledger/hive.go/lo v0.0.0-20231207181026-f482ac139305/go.mod h1:ETXGXymFyNcUq2t4I9e7ZK18f9bxUWYat4pjZ9W0rWc= github.com/iotaledger/hive.go/log v0.0.0-20231206113509-4b4ff95ac61c h1:Ksts6VjPj9y0o2Nis+2tHtDGWITNJ4yju87ZlHLPuOo= github.com/iotaledger/hive.go/log v0.0.0-20231206113509-4b4ff95ac61c/go.mod h1:Td3R6QmYw0svZI1GuZ/tN9s0VNaHassXSKLCc70aX9w= -github.com/iotaledger/hive.go/runtime v0.0.0-20231206114953-6a65a82e30ad h1:HpupWK8iqFt+Sdogkh2/N8ojalmevYy+FzhjOuy7Y7E= -github.com/iotaledger/hive.go/runtime v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:Z9NFsByMh1Kf98f3v3ifeZRycbS2db1hjswTQG1MxnE= +github.com/iotaledger/hive.go/runtime v0.0.0-20231207181026-f482ac139305 h1:7CW1/EbG+RvkjbyOf6JA1u1feax/cpex/6a8CLbaA4I= +github.com/iotaledger/hive.go/runtime v0.0.0-20231207181026-f482ac139305/go.mod h1:Z9NFsByMh1Kf98f3v3ifeZRycbS2db1hjswTQG1MxnE= github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231206114953-6a65a82e30ad h1:c8uwbBZDqpiCNN9/9Jji7Z4lL0GdVnORp8WMouiuknk= github.com/iotaledger/hive.go/serializer/v2 v2.0.0-rc.1.0.20231206114953-6a65a82e30ad/go.mod h1:FoH3T6yKlZJp8xm8K+zsQiibSynp32v21CpWx8xkek8= -github.com/iotaledger/hive.go/stringify v0.0.0-20231206114953-6a65a82e30ad h1:VC3OgdSbyngY7/gxVj66fKd/nGmN6P0/myr348nx7vA= -github.com/iotaledger/hive.go/stringify v0.0.0-20231206114953-6a65a82e30ad/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs= -github.com/iotaledger/iota.go/v4 v4.0.0-20231206123921-2af411eef0b5 h1:0KgQFpVRnKd6CdCwXo3Kg/SL27xkeKh2SMoU5G1TkZk= -github.com/iotaledger/iota.go/v4 v4.0.0-20231206123921-2af411eef0b5/go.mod h1:tiswk1O1wSAi9GE6tD1j43+bLmWU9Je07aZLaJ0+Ha0= +github.com/iotaledger/hive.go/stringify v0.0.0-20231207181026-f482ac139305 h1:KjbaklWvZb4zIcXBETHzl6XFTAf8wtAlFDfaF0Z1Daw= +github.com/iotaledger/hive.go/stringify v0.0.0-20231207181026-f482ac139305/go.mod h1:FTo/UWzNYgnQ082GI9QVM9HFDERqf9rw9RivNpqrnTs= +github.com/iotaledger/iota-crypto-demo v0.0.0-20231208171603-786bb32fdb00 h1:j5udgLtSN6wQgFI9vnhkdJsqsVdJmwtoc0yOmT/Ila4= +github.com/iotaledger/iota-crypto-demo v0.0.0-20231208171603-786bb32fdb00/go.mod h1:gt+URx7DZu414nZME7jtGgxR4DVTSnNa1jF2trTUTZ0= +github.com/iotaledger/iota.go/v4 v4.0.0-20231211160706-492c65d5e3f5 h1:2iQUEuYvuyeYtZBr6bRoM4xFLxRiQ66aBPgKuJTirh0= +github.com/iotaledger/iota.go/v4 v4.0.0-20231211160706-492c65d5e3f5/go.mod h1:jU3Jlnhwv4KWlpft84lu9/M0QkCykESl29bL4oCenKs= github.com/ipfs/go-cid v0.4.1 h1:A/T3qGvxi4kpKWWcPC/PgbvDA2bjVLO7n4UeVwnbs/s= github.com/ipfs/go-cid v0.4.1/go.mod h1:uQHwDeX4c6CtyrFwdqyhpNcxVewur1M7l7fNU7LKwZk= github.com/klauspost/cpuid/v2 v2.2.5 h1:0E5MSMDEoAulmXNFquVs//DdoomxaoTY1kUhbc/qbZg= @@ -116,8 +118,6 @@ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= -github.com/wollac/iota-crypto-demo v0.0.0-20221117162917-b10619eccb98 h1:i7k63xHOX2ntuHrhHewfKro67c834jug2DIk599fqAA= -github.com/wollac/iota-crypto-demo v0.0.0-20221117162917-b10619eccb98/go.mod h1:Knu2XMRWe8SkwTlHc/+ghP+O9DEaZRQQEyTjvLJ5Cck= github.com/zyedidia/generic v1.2.1 h1:Zv5KS/N2m0XZZiuLS82qheRG4X1o5gsWreGb0hR7XDc= github.com/zyedidia/generic v1.2.1/go.mod h1:ly2RBz4mnz1yeuVbQA/VFwGjK3mnHGRj1JuoG336Bis= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= From b56e5b4fe31946cba77413a0ab166c0bbf79d7dd Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Tue, 12 Dec 2023 13:09:37 +0100 Subject: [PATCH 10/12] Make doggo happy --- pkg/protocol/engine/eviction/state.go | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/pkg/protocol/engine/eviction/state.go b/pkg/protocol/engine/eviction/state.go index f98d60e94..fbe025cb8 100644 --- a/pkg/protocol/engine/eviction/state.go +++ b/pkg/protocol/engine/eviction/state.go @@ -1,7 +1,6 @@ package eviction import ( - "errors" "io" "github.com/iotaledger/hive.go/ierrors" @@ -16,8 +15,6 @@ import ( iotago "github.com/iotaledger/iota.go/v4" ) -const latestNonEmptySlotKey = 1 - // State represents the state of the eviction and keeps track of the root blocks. type State struct { Events *Events @@ -54,7 +51,7 @@ func (s *State) AdvanceActiveWindowToIndex(slot iotago.SlotIndex) { s.evictionMutex.Unlock() - // We only delay eviction in the Eviction State, but components evict on committment, which in this context is slot. + // We only delay eviction in the Eviction State, but components evict on commitment, which in this context is slot. s.Events.SlotEvicted.Trigger(slot) } @@ -124,7 +121,7 @@ func (s *State) LatestActiveRootBlock() (iotago.BlockID, iotago.CommitmentID) { latestSlotCommitmentID = commitmentID // We want the newest rootblock. - return errors.New("stop iteration") + return ierrors.New("stop iteration") }) // We found the most recent root block in this slot. @@ -151,7 +148,9 @@ func (s *State) AddRootBlock(id iotago.BlockID, commitmentID iotago.CommitmentID panic(ierrors.Wrapf(err, "failed to store root block %s", id)) } - s.settings.AdvanceLatestNonEmptySlot(id.Slot()) + if err := s.settings.AdvanceLatestNonEmptySlot(id.Slot()); err != nil { + panic(ierrors.Wrapf(err, "failed to advance latest non empty slot to %d", id.Slot())) + } } // RemoveRootBlock removes a solid entry points from the map. @@ -276,7 +275,9 @@ func (s *State) Import(reader io.ReadSeeker) error { return ierrors.Wrap(err, "failed to read latest non empty slot") } - s.settings.SetLatestNonEmptySlot(latestNonEmptySlot) + if err := s.settings.SetLatestNonEmptySlot(latestNonEmptySlot); err != nil { + return ierrors.Wrapf(err, "failed to set latest non empty slot to %d", latestNonEmptySlot) + } return nil } @@ -297,7 +298,9 @@ func (s *State) Rollback(lowerTarget iotago.SlotIndex, targetSlot iotago.SlotInd latestNonEmptySlot = currentSlot } - s.settings.SetLatestNonEmptySlot(latestNonEmptySlot) + if err := s.settings.SetLatestNonEmptySlot(latestNonEmptySlot); err != nil { + return ierrors.Wrapf(err, "failed to set latest non empty slot to %d", latestNonEmptySlot) + } return nil } From 7bac67923837603c3b87f0b67ed2c7964fe9a8d2 Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Tue, 12 Dec 2023 15:59:18 +0100 Subject: [PATCH 11/12] Address review comments --- pkg/protocol/engine/eviction/state.go | 10 +++------- .../engine/tipselection/v1/test_framework_test.go | 4 ++-- pkg/protocol/engine/tipselection/v1/tip_selection.go | 6 +++--- pkg/storage/permanent/settings.go | 6 ------ pkg/tests/booker_test.go | 4 ++-- pkg/testsuite/blocks.go | 3 --- 6 files changed, 10 insertions(+), 23 deletions(-) diff --git a/pkg/protocol/engine/eviction/state.go b/pkg/protocol/engine/eviction/state.go index fbe025cb8..51054709b 100644 --- a/pkg/protocol/engine/eviction/state.go +++ b/pkg/protocol/engine/eviction/state.go @@ -4,6 +4,7 @@ import ( "io" "github.com/iotaledger/hive.go/ierrors" + "github.com/iotaledger/hive.go/kvstore" "github.com/iotaledger/hive.go/lo" "github.com/iotaledger/hive.go/runtime/syncutils" "github.com/iotaledger/hive.go/serializer/v2" @@ -92,11 +93,6 @@ func (s *State) AllActiveRootBlocks() map[iotago.BlockID]iotago.CommitmentID { }) } - // We include genesis as a root block if the start of our active window is the genesis slot. - if startSlot == s.settings.APIProvider().APIForSlot(s.lastCommittedSlot).ProtocolParameters().GenesisSlot() { - activeRootBlocks[s.settings.APIProvider().CommittedAPI().ProtocolParameters().GenesisBlockID()] = model.NewEmptyCommitment(s.settings.APIProvider().CommittedAPI()).ID() - } - return activeRootBlocks } @@ -148,7 +144,7 @@ func (s *State) AddRootBlock(id iotago.BlockID, commitmentID iotago.CommitmentID panic(ierrors.Wrapf(err, "failed to store root block %s", id)) } - if err := s.settings.AdvanceLatestNonEmptySlot(id.Slot()); err != nil { + if err := s.settings.AdvanceLatestNonEmptySlot(id.Slot()); err != nil && !ierrors.Is(err, kvstore.ErrTypedValueNotChanged) { panic(ierrors.Wrapf(err, "failed to advance latest non empty slot to %d", id.Slot())) } } @@ -316,7 +312,7 @@ func (s *State) activeIndexRange(targetSlot iotago.SlotIndex) (startSlot iotago. return genesisSlot, targetSlot } - rootBlocksWindowStart := targetSlot - maxCommittableAge + 1 + rootBlocksWindowStart := (targetSlot - maxCommittableAge) + 1 if latestNonEmptySlot := s.settings.LatestNonEmptySlot(); rootBlocksWindowStart > latestNonEmptySlot { rootBlocksWindowStart = latestNonEmptySlot diff --git a/pkg/protocol/engine/tipselection/v1/test_framework_test.go b/pkg/protocol/engine/tipselection/v1/test_framework_test.go index 0ce8b8739..4ccfc6170 100644 --- a/pkg/protocol/engine/tipselection/v1/test_framework_test.go +++ b/pkg/protocol/engine/tipselection/v1/test_framework_test.go @@ -41,7 +41,7 @@ func NewTestFramework(test *testing.T, opts ...options.Option[TestFramework]) *T return nil, false } - rootBlocksRetriever := func() iotago.BlockID { + rootBlockRetriever := func() iotago.BlockID { return iotago.EmptyBlockID } @@ -51,7 +51,7 @@ func NewTestFramework(test *testing.T, opts ...options.Option[TestFramework]) *T t.TipManager.Instance, spenddagv1.New[iotago.TransactionID, mempool.StateID, ledger.BlockVoteRank](t.CommitteeSize), transactionMetadataRetriever, - rootBlocksRetriever, + rootBlockRetriever, t.expectedLivenessDuration, ) }) diff --git a/pkg/protocol/engine/tipselection/v1/tip_selection.go b/pkg/protocol/engine/tipselection/v1/tip_selection.go index 452f4a953..e9312d8e4 100644 --- a/pkg/protocol/engine/tipselection/v1/tip_selection.go +++ b/pkg/protocol/engine/tipselection/v1/tip_selection.go @@ -28,7 +28,7 @@ type TipSelection struct { // spendDAG is the SpendDAG that is used to track spenders. spendDAG spenddag.SpendDAG[iotago.TransactionID, mempool.StateID, ledger.BlockVoteRank] - // rootBlock is a function that returns the current root blocks. + // rootBlock is a function that returns the latest root block. rootBlock func() iotago.BlockID // livenessThreshold is a function that is used to determine the liveness threshold for a tip. @@ -83,11 +83,11 @@ func New(opts ...options.Option[TipSelection]) *TipSelection { // // This method is separated from the constructor so the TipSelection can be initialized lazily after all dependencies // are available. -func (t *TipSelection) Construct(tipManager tipmanager.TipManager, spendDAG spenddag.SpendDAG[iotago.TransactionID, mempool.StateID, ledger.BlockVoteRank], transactionMetadataRetriever func(iotago.TransactionID) (mempool.TransactionMetadata, bool), rootBlocksRetriever func() iotago.BlockID, livenessThresholdFunc func(tipmanager.TipMetadata) time.Duration) *TipSelection { +func (t *TipSelection) Construct(tipManager tipmanager.TipManager, spendDAG spenddag.SpendDAG[iotago.TransactionID, mempool.StateID, ledger.BlockVoteRank], transactionMetadataRetriever func(iotago.TransactionID) (mempool.TransactionMetadata, bool), rootBlockRetriever func() iotago.BlockID, livenessThresholdFunc func(tipmanager.TipMetadata) time.Duration) *TipSelection { t.tipManager = tipManager t.spendDAG = spendDAG t.transactionMetadata = transactionMetadataRetriever - t.rootBlock = rootBlocksRetriever + t.rootBlock = rootBlockRetriever t.livenessThreshold = livenessThresholdFunc t.TriggerConstructed() diff --git a/pkg/storage/permanent/settings.go b/pkg/storage/permanent/settings.go index 876bfad1d..6720617d0 100644 --- a/pkg/storage/permanent/settings.go +++ b/pkg/storage/permanent/settings.go @@ -332,12 +332,6 @@ func (s *Settings) SetLatestNonEmptySlot(slot iotago.SlotIndex) (err error) { } func (s *Settings) AdvanceLatestNonEmptySlot(slot iotago.SlotIndex) (err error) { - // We don't need to advance the latest stored slot if it's already ahead of the given slot. - // We check this before Compute to avoid contention inside the TypedValue. - if s.LatestNonEmptySlot() >= slot { - return nil - } - if _, err = s.storeLatestNonEmptySlot.Compute(func(latestNonEmptySlot iotago.SlotIndex, _ bool) (newValue iotago.SlotIndex, err error) { if latestNonEmptySlot >= slot { return latestNonEmptySlot, kvstore.ErrTypedValueNotChanged diff --git a/pkg/tests/booker_test.go b/pkg/tests/booker_test.go index b5407ba42..9929243be 100644 --- a/pkg/tests/booker_test.go +++ b/pkg/tests/booker_test.go @@ -723,7 +723,7 @@ func Test_SpendPendingCommittedRace(t *testing.T) { ts.AssertBlocksInCacheConflicts(map[*blocks.Block][]string{ ts.Block("block2.1"): {"tx1"}, ts.Block("n2-pending-genesis"): {"tx1"}, - // ts.Block("n2-pending-commit1"): {}, // no conflits inherited as the block merges orphaned conflicts. + ts.Block("n2-pending-commit1"): {}, // no conflits inherited as the block merges orphaned conflicts. }, node2) } @@ -753,7 +753,7 @@ func Test_SpendPendingCommittedRace(t *testing.T) { ts.AssertBlocksInCacheConflicts(map[*blocks.Block][]string{ ts.Block("block2.1"): {"tx1"}, ts.Block("n2-pending-genesis"): {"tx1"}, - // ts.Block("n2-pending-commit1"): {}, // no conflits inherited as the block merges orphaned conflicts. + ts.Block("n2-pending-commit1"): {}, // no conflits inherited as the block merges orphaned conflicts. }, node1, node2) ts.AssertTransactionsInCachePending(wallet.Transactions("tx1", "tx2"), true, node1, node2) diff --git a/pkg/testsuite/blocks.go b/pkg/testsuite/blocks.go index f8a35ea47..40bdd265b 100644 --- a/pkg/testsuite/blocks.go +++ b/pkg/testsuite/blocks.go @@ -1,8 +1,6 @@ package testsuite import ( - "fmt" - "github.com/stretchr/testify/assert" "github.com/iotaledger/hive.go/ds" @@ -20,7 +18,6 @@ func (t *TestSuite) AssertBlock(block *blocks.Block, node *mock.Node) *model.Blo var exists bool loadedBlock, exists = node.Protocol.Engines.Main.Get().Block(block.ID()) if !exists { - fmt.Println("loadedBlock ", block.ID()) return ierrors.Errorf("AssertBlock: %s: block %s does not exist", node.Name, block.ID()) } From a69e15f31cf908157e1bfe68811e2c07e685d09d Mon Sep 17 00:00:00 2001 From: Andrea V <1577639+karimodm@users.noreply.github.com> Date: Tue, 12 Dec 2023 16:18:50 +0100 Subject: [PATCH 12/12] Avoid contention with early check --- pkg/storage/permanent/settings.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/storage/permanent/settings.go b/pkg/storage/permanent/settings.go index 6720617d0..9ef1cebe2 100644 --- a/pkg/storage/permanent/settings.go +++ b/pkg/storage/permanent/settings.go @@ -332,6 +332,11 @@ func (s *Settings) SetLatestNonEmptySlot(slot iotago.SlotIndex) (err error) { } func (s *Settings) AdvanceLatestNonEmptySlot(slot iotago.SlotIndex) (err error) { + // Avoid write-locking within the Compute with an early check. + if s.LatestNonEmptySlot() >= slot { + return nil + } + if _, err = s.storeLatestNonEmptySlot.Compute(func(latestNonEmptySlot iotago.SlotIndex, _ bool) (newValue iotago.SlotIndex, err error) { if latestNonEmptySlot >= slot { return latestNonEmptySlot, kvstore.ErrTypedValueNotChanged