diff --git a/block/block.go b/block/block.go index de09ae536..544fa0286 100644 --- a/block/block.go +++ b/block/block.go @@ -157,8 +157,8 @@ func (m *Manager) applyBlock(block *types.Block, commit *types.Commit, blockMeta return fmt.Errorf("save proposer: %w", err) } - // 2. Update the proposer in the state in case of rotation. - switchRole := m.Executor.UpdateProposerFromBlock(m.State, m.Sequencers, block) + // 2. Update the proposer in the state in case of rotation happened on the rollapp level (not necessarily on the hub yet). + isProposerUpdated := m.Executor.UpdateProposerFromBlock(m.State, m.Sequencers, block) // 3. Save the state to the store (independently of the height). Here the proposer might differ from (1). batch, err = m.Store.SaveState(m.State, batch) @@ -175,18 +175,20 @@ func (m *Manager) applyBlock(block *types.Block, commit *types.Commit, blockMeta m.blockCache.Delete(block.Header.Height) - if switchRole { - // TODO: graceful role change (https://github.com/dymensionxyz/dymint/issues/1008) - m.logger.Info("Node changing to proposer role") - panic("sequencer is no longer the proposer") - } - // validate whether configuration params and rollapp consensus params keep in line, after rollapp params are updated from the responses received in the block execution err = m.ValidateConfigWithRollappParams() if err != nil { return err } + // Check if there was an Update for the proposer and if I am the new proposer. + // If so, restart so I can start as the proposer. + // For current proposer, we don't want to restart because we still need to send the last batch. + // This will be done as part of the `rotate` function. + if isProposerUpdated && m.AmIProposerOnRollapp() { + panic("I'm the new Proposer now. restarting as a proposer") + } + return nil } diff --git a/block/initchain.go b/block/initchain.go index db65485df..926724633 100644 --- a/block/initchain.go +++ b/block/initchain.go @@ -8,7 +8,7 @@ import ( ) func (m *Manager) RunInitChain(ctx context.Context) error { - // get the proposer's consensus pubkey + // FIXME: We want to get the initial proposer and not current one proposer := m.SLClient.GetProposer() if proposer == nil { return errors.New("failed to get proposer") diff --git a/block/manager.go b/block/manager.go index 309d44c7f..8c071da55 100644 --- a/block/manager.go +++ b/block/manager.go @@ -14,7 +14,6 @@ import ( "github.com/dymensionxyz/dymint/indexers/txindex" "github.com/dymensionxyz/dymint/store" uerrors "github.com/dymensionxyz/dymint/utils/errors" - uevent "github.com/dymensionxyz/dymint/utils/event" "github.com/dymensionxyz/dymint/version" "github.com/libp2p/go-libp2p/core/crypto" @@ -191,17 +190,29 @@ func (m *Manager) Start(ctx context.Context) error { } } - // Check if the chain is halted - err := m.isChainHalted() - if err != nil { - return err + // Check if a proposer on the rollapp is set. In case no proposer is set on the Rollapp, fallback to the hub proposer (If such exists). + // No proposer on the rollapp means that at some point there was no available proposer. + // In case there is also no proposer on the hub to our current height, it means that the chain is halted. + // FIXME: In case we are syncing we would like to get the proposer from the hub relevant to the current height. + if m.State.GetProposer() == nil { + m.logger.Info("No proposer on the rollapp, fallback to the hub proposer, if available") + SLProposer := m.SLClient.GetProposer() + if SLProposer == nil { + return fmt.Errorf("no proposer available. chain is halted") + } + m.State.SetProposer(SLProposer) } - isProposer := m.IsProposer() - m.logger.Info("starting block manager", "proposer", isProposer) + // checks if the the current node is the proposer either on rollapp or on the hub. + // In case of sequencer rotation, there's a phase where proposer rotated on Rollapp but hasn't yet rotated on hub. + // for this case, 2 nodes will get `true` for `AmIProposer` so the l2 proposer can produce blocks and the hub proposer can submit his last batch. + // The hub proposer, after sending the last state update, will panic and restart as full node. + amIProposer := m.AmIProposerOnSL() || m.AmIProposerOnRollapp() + + m.logger.Info("starting block manager", "mode", map[bool]string{true: "proposer", false: "full node"}[amIProposer]) // update local state from latest state in settlement - err = m.updateFromLastSettlementState() + err := m.updateFromLastSettlementState() if err != nil { return fmt.Errorf("sync block manager from settlement: %w", err) } @@ -223,100 +234,17 @@ func (m *Manager) Start(ctx context.Context) error { return m.SettlementSyncLoop(ctx) }) + // Monitor sequencer set updates uerrors.ErrGroupGoLog(eg, m.logger, func() error { return m.MonitorSequencerSetUpdates(ctx) }) - /* ----------------------------- full node mode ----------------------------- */ - if !isProposer { - - // update latest finalized height - err = m.updateLastFinalizedHeightFromSettlement() - if err != nil { - return fmt.Errorf("sync block manager from settlement: %w", err) - } - - // Start the settlement validation loop in the background - uerrors.ErrGroupGoLog(eg, m.logger, func() error { - return m.SettlementValidateLoop(ctx) - }) - - // Subscribe to new (or finalized) state updates events. - go uevent.MustSubscribe(ctx, m.Pubsub, "syncLoop", settlement.EventQueryNewSettlementBatchAccepted, m.onNewStateUpdate, m.logger) - go uevent.MustSubscribe(ctx, m.Pubsub, "validateLoop", settlement.EventQueryNewSettlementBatchFinalized, m.onNewStateUpdateFinalized, m.logger) - - // Subscribe to P2P received blocks events (used for P2P syncing). - go uevent.MustSubscribe(ctx, m.Pubsub, "applyGossipedBlocksLoop", p2p.EventQueryNewGossipedBlock, m.OnReceivedBlock, m.logger) - go uevent.MustSubscribe(ctx, m.Pubsub, "applyBlockSyncBlocksLoop", p2p.EventQueryNewBlockSyncBlock, m.OnReceivedBlock, m.logger) - - return nil - } - - /* ----------------------------- sequencer mode ----------------------------- */ - // Subscribe to batch events, to update last submitted height in case batch confirmation was lost. This could happen if the sequencer crash/restarted just after submitting a batch to the settlement and by the time we query the last batch, this batch wasn't accepted yet. - go uevent.MustSubscribe(ctx, m.Pubsub, "updateSubmittedHeightLoop", settlement.EventQueryNewSettlementBatchAccepted, m.UpdateLastSubmittedHeight, m.logger) - - // Sequencer must wait till the DA light client is synced. Otherwise it will fail when submitting blocks. - // Full-nodes does not need to wait, but if it tries to fetch blocks from DA heights previous to the DA light client height it will fail, and it will retry till it reaches the height. - m.DAClient.WaitForSyncing() - - // Sequencer must wait till node is synced till last submittedHeight, in case it is not - m.waitForSettlementSyncing() - - // check if sequencer in the middle of rotation - nextSeqAddr, missing, err := m.MissingLastBatch() - if err != nil { - return fmt.Errorf("checking if missing last batch: %w", err) - } - // if sequencer is in the middle of rotation, complete rotation instead of running the main loop - if missing { - m.handleRotationReq(ctx, nextSeqAddr) - return nil + // run based on the node role + if !amIProposer { + return m.runAsFullNode(ctx, eg) } - // populate the bytes produced channel - bytesProducedC := make(chan int) - - // channel to signal sequencer rotation started - rotateSequencerC := make(chan string, 1) - - uerrors.ErrGroupGoLog(eg, m.logger, func() error { - return m.SubmitLoop(ctx, bytesProducedC) - }) - uerrors.ErrGroupGoLog(eg, m.logger, func() error { - bytesProducedC <- m.GetUnsubmittedBytes() // load unsubmitted bytes from previous run - return m.ProduceBlockLoop(ctx, bytesProducedC) - }) - uerrors.ErrGroupGoLog(eg, m.logger, func() error { - return m.MonitorSequencerRotation(ctx, rotateSequencerC) - }) - - go func() { - _ = eg.Wait() - // Check if exited due to sequencer rotation signal - select { - case nextSeqAddr := <-rotateSequencerC: - m.handleRotationReq(ctx, nextSeqAddr) - default: - m.logger.Info("Block manager err group finished.") - } - }() - - return nil -} - -func (m *Manager) isChainHalted() error { - if m.State.GetProposerPubKey() == nil { - // if no proposer set in state, try to update it from the hub - err := m.UpdateProposerFromSL() - if err != nil { - return fmt.Errorf("update proposer: %w", err) - } - if m.State.GetProposerPubKey() == nil { - return fmt.Errorf("no proposer pubkey found. chain is halted") - } - } - return nil + return m.runAsProposer(ctx, eg) } func (m *Manager) NextHeightToSubmit() uint64 { diff --git a/block/modes.go b/block/modes.go new file mode 100644 index 000000000..1c8c3fd60 --- /dev/null +++ b/block/modes.go @@ -0,0 +1,73 @@ +package block + +import ( + "context" + "fmt" + + "github.com/dymensionxyz/dymint/p2p" + "github.com/dymensionxyz/dymint/settlement" + uerrors "github.com/dymensionxyz/dymint/utils/errors" + uevent "github.com/dymensionxyz/dymint/utils/event" + "golang.org/x/sync/errgroup" +) + +// setFraudHandler sets the fraud handler for the block manager. +func (m *Manager) runAsFullNode(ctx context.Context, eg *errgroup.Group) error { + // update latest finalized height + err := m.updateLastFinalizedHeightFromSettlement() + if err != nil { + return fmt.Errorf("sync block manager from settlement: %w", err) + } + + // Start the settlement validation loop in the background + uerrors.ErrGroupGoLog(eg, m.logger, func() error { + return m.SettlementValidateLoop(ctx) + }) + + // Subscribe to new (or finalized) state updates events. + go uevent.MustSubscribe(ctx, m.Pubsub, "syncLoop", settlement.EventQueryNewSettlementBatchAccepted, m.onNewStateUpdate, m.logger) + go uevent.MustSubscribe(ctx, m.Pubsub, "validateLoop", settlement.EventQueryNewSettlementBatchFinalized, m.onNewStateUpdateFinalized, m.logger) + + // Subscribe to P2P received blocks events (used for P2P syncing). + go uevent.MustSubscribe(ctx, m.Pubsub, "applyGossipedBlocksLoop", p2p.EventQueryNewGossipedBlock, m.OnReceivedBlock, m.logger) + go uevent.MustSubscribe(ctx, m.Pubsub, "applyBlockSyncBlocksLoop", p2p.EventQueryNewBlockSyncBlock, m.OnReceivedBlock, m.logger) + + return nil +} + +func (m *Manager) runAsProposer(ctx context.Context, eg *errgroup.Group) error { + // Subscribe to batch events, to update last submitted height in case batch confirmation was lost. This could happen if the sequencer crash/restarted just after submitting a batch to the settlement and by the time we query the last batch, this batch wasn't accepted yet. + go uevent.MustSubscribe(ctx, m.Pubsub, "updateSubmittedHeightLoop", settlement.EventQueryNewSettlementBatchAccepted, m.UpdateLastSubmittedHeight, m.logger) + + // Sequencer must wait till the DA light client is synced. Otherwise it will fail when submitting blocks. + // Full-nodes does not need to wait, but if it tries to fetch blocks from DA heights previous to the DA light client height it will fail, and it will retry till it reaches the height. + m.DAClient.WaitForSyncing() + + // Sequencer must wait till node is synced till last submittedHeight, in case it is not + m.waitForSettlementSyncing() + + // check if we should rotate + shouldRotate, err := m.ShouldRotate() + if err != nil { + return fmt.Errorf("checking should rotate: %w", err) + } + if shouldRotate { + m.rotate(ctx) + } + + // populate the bytes produced channel + bytesProducedC := make(chan int) + + uerrors.ErrGroupGoLog(eg, m.logger, func() error { + return m.SubmitLoop(ctx, bytesProducedC) + }) + uerrors.ErrGroupGoLog(eg, m.logger, func() error { + bytesProducedC <- m.GetUnsubmittedBytes() // load unsubmitted bytes from previous run + return m.ProduceBlockLoop(ctx, bytesProducedC) + }) + + // Monitor and handling of the rotation + go m.MonitorProposerRotation(ctx) + + return nil +} diff --git a/block/produce.go b/block/produce.go index f46876c41..90062ba8b 100644 --- a/block/produce.go +++ b/block/produce.go @@ -40,6 +40,10 @@ func (m *Manager) ProduceBlockLoop(ctx context.Context, bytesProducedC chan int) case <-ctx.Done(): return nil case <-ticker.C: + // Only produce if I'm the current rollapp proposer. + if !m.AmIProposerOnRollapp() { + continue + } // if empty blocks are configured to be enabled, and one is scheduled... produceEmptyBlock := firstBlock || m.Conf.MaxIdleTime == 0 || nextEmptyBlock.Before(time.Now()) @@ -107,6 +111,7 @@ func (m *Manager) ProduceApplyGossipBlock(ctx context.Context, allowEmpty bool) } func (m *Manager) produceApplyGossip(ctx context.Context, allowEmpty bool, nextProposerHash *[32]byte) (block *types.Block, commit *types.Commit, err error) { + // If I'm not the current rollapp proposer, I should not produce a blocks. block, commit, err = m.produceBlock(allowEmpty, nextProposerHash) if err != nil { return nil, nil, fmt.Errorf("produce block: %w", err) diff --git a/block/pruning.go b/block/pruning.go index 4d2e3f1f5..06c2ce4c5 100644 --- a/block/pruning.go +++ b/block/pruning.go @@ -44,7 +44,7 @@ func (m *Manager) PruningLoop(ctx context.Context) error { case retainHeight := <-m.pruningC: var pruningHeight uint64 - if m.IsProposer() { // do not delete anything that we might submit in future + if m.AmIProposerOnSL() || m.AmIProposerOnRollapp() { // do not delete anything that we might submit in future pruningHeight = min(m.NextHeightToSubmit(), uint64(retainHeight)) } else { // do not delete anything that is not validated yet pruningHeight = min(m.SettlementValidator.NextValidationHeight(), uint64(retainHeight)) diff --git a/block/sequencers.go b/block/sequencers.go index 4b7ff0fe0..a32db3865 100644 --- a/block/sequencers.go +++ b/block/sequencers.go @@ -6,48 +6,28 @@ import ( "fmt" "time" - "github.com/dymensionxyz/dymint/settlement" "github.com/dymensionxyz/dymint/types" ) -func (m *Manager) MonitorSequencerRotation(ctx context.Context, rotateC chan string) error { - sequencerRotationEventClient := "sequencer_rotation" - subscription, err := m.Pubsub.Subscribe(ctx, sequencerRotationEventClient, settlement.EventQueryRotationStarted) - if err != nil { - panic("Error subscribing to events") - } - defer m.Pubsub.UnsubscribeAll(ctx, sequencerRotationEventClient) //nolint:errcheck - +func (m *Manager) MonitorProposerRotation(ctx context.Context) { ticker := time.NewTicker(3 * time.Minute) // TODO: make this configurable defer ticker.Stop() - var nextSeqAddr string for { select { case <-ctx.Done(): - return nil + return case <-ticker.C: - next, err := m.SLClient.CheckRotationInProgress() + next, err := m.SLClient.GetNextProposer() if err != nil { m.logger.Error("Check rotation in progress", "err", err) continue } - if next == nil { - continue + if next != nil { + m.rotate(ctx) } - nextSeqAddr = next.SettlementAddress - case event := <-subscription.Out(): - eventData, _ := event.Data().(*settlement.EventDataRotationStarted) - nextSeqAddr = eventData.NextSeqAddr } - break // break out of the loop after getting the next sequencer address } - // we get here once a sequencer rotation signal is received - m.logger.Info("Sequencer rotation started.", "next_seq", nextSeqAddr) - go func() { - rotateC <- nextSeqAddr - }() - return fmt.Errorf("sequencer rotation started. signal to stop production") } func (m *Manager) MonitorSequencerSetUpdates(ctx context.Context) error { @@ -68,90 +48,71 @@ func (m *Manager) MonitorSequencerSetUpdates(ctx context.Context) error { } } -// IsProposer checks if the local node is the proposer +// AmIPropoesr checks if the the current node is the proposer either on L2 or on the hub. // In case of sequencer rotation, there's a phase where proposer rotated on L2 but hasn't yet rotated on hub. -// for this case, the old proposer counts as "sequencer" as well, so he'll be able to submit the last state update. -func (m *Manager) IsProposer() bool { - localProposerKey, _ := m.LocalKey.GetPublic().Raw() - l2Proposer := m.State.GetProposerPubKey().Bytes() +// for this case, 2 nodes will get `true` for `AmIProposer` so the l2 proposer can produce blocks and the hub proposer can submit his last batch. +func (m *Manager) AmIProposer() bool { + return m.AmIProposerOnSL() || m.AmIProposerOnRollapp() +} - var expectedHubProposer []byte +// AmIProposerOnSL checks if the current node is the proposer on the hub +// Proposer on the Hub is not necessarily the proposer on the L2 during rotation phase. +func (m *Manager) AmIProposerOnSL() bool { + localProposerKeyBytes, _ := m.LocalKey.GetPublic().Raw() + + // get hub proposer key + var hubProposerKeyBytes []byte hubProposer := m.SLClient.GetProposer() if hubProposer != nil { - expectedHubProposer = hubProposer.PubKey().Bytes() + hubProposerKeyBytes = hubProposer.PubKey().Bytes() } + return bytes.Equal(hubProposerKeyBytes, localProposerKeyBytes) +} - // check if recovering from halt - if l2Proposer == nil && hubProposer != nil { - m.State.SetProposer(hubProposer) - } +// AmIProposerOnRollapp checks if the current node is the proposer on the rollapp. +// Proposer on the rollapp is not necessarily the proposer on the hub during rotation phase. +func (m *Manager) AmIProposerOnRollapp() bool { + localProposerKeyBytes, _ := m.LocalKey.GetPublic().Raw() + rollappProposer := m.State.GetProposerPubKey().Bytes() - // we run sequencer flow if we're proposer on L2 or hub (can be different during rotation phase, before hub receives the last state update) - return bytes.Equal(l2Proposer, localProposerKey) || bytes.Equal(expectedHubProposer, localProposerKey) + return bytes.Equal(rollappProposer, localProposerKeyBytes) } -// MissingLastBatch checks if the sequencer is in the middle of rotation (I'm the proposer, but needs to complete rotation) -// returns the next sequencer address and a flag if the sequencer is the old proposer and the hub waits for the last batch -func (m *Manager) MissingLastBatch() (string, bool, error) { - localProposerKey, _ := m.LocalKey.GetPublic().Raw() - next, err := m.SLClient.CheckRotationInProgress() +// ShouldRotate checks if the we are in the middle of rotation and we are the rotating proposer (i.e current proposer on the hub). +// We check it by checking if there is a "next" proposer on the hub which is not us. +func (m *Manager) ShouldRotate() (bool, error) { + nextProposer, err := m.SLClient.GetNextProposer() if err != nil { - return "", false, err + return false, err } - if next == nil { - return "", false, nil + if nextProposer == nil { + return false, nil } - // rotation in progress, - // check if we're the old proposer and needs to complete rotation - curr := m.SLClient.GetProposer() - isProposer := bytes.Equal(curr.PubKey().Bytes(), localProposerKey) - return next.SettlementAddress, isProposer, nil + // At this point we know that there is a next proposer, + // so we should rotate only if we are the current proposer on the hub + return m.AmIProposerOnSL(), nil } -// handleRotationReq completes the rotation flow once a signal is received from the SL -// this called after manager shuts down the block producer and submitter -func (m *Manager) handleRotationReq(ctx context.Context, nextSeqAddr string) { - m.logger.Info("Sequencer rotation started. Production stopped on this sequencer", "nextSeqAddr", nextSeqAddr) - - // Update sequencers list from SL - err := m.UpdateSequencerSetFromSL() +// rotate rotates current proposer by doing the following: +// 1. Creating last block with the new proposer, which will stop him from producing blocks. +// 2. Submitting the last batch +// 3. Panicing so the node restarts as full node +// Note: In case he already created his last block, he will only try to submit the last batch. +func (m *Manager) rotate(ctx context.Context) { + // Get Next Proposer from SL. We assume such exists (even if empty proposer) otherwise function wouldn't be called. + nextProposer, err := m.SLClient.GetNextProposer() if err != nil { - // this error is not critical, try to complete the rotation anyway - m.logger.Error("Cannot fetch sequencer set from the Hub", "error", err) + panic(fmt.Sprintf("rotate: fetch next proposer set from Hub: %v", err)) } - err = m.CompleteRotation(ctx, nextSeqAddr) + err = m.CreateAndPostLastBatch(ctx, [32]byte(nextProposer.MustHash())) if err != nil { - panic(err) + panic(fmt.Sprintf("rotate: create and post last batch: %v", err)) } - // TODO: graceful fallback to full node (https://github.com/dymensionxyz/dymint/issues/1008) - m.logger.Info("Sequencer is no longer the proposer") - panic("sequencer is no longer the proposer") -} - -// CompleteRotation completes the sequencer rotation flow -// the sequencer will create his last block, with the next sequencer's hash, to handover the proposer role -// then it will submit all the data accumulated thus far and mark the last state update -// if nextSeqAddr is empty, the nodes will halt after applying the block produced -func (m *Manager) CompleteRotation(ctx context.Context, nextSeqAddr string) error { - // validate nextSeq is in the bonded set - var nextSeqHash [32]byte - if nextSeqAddr != "" { - seq, found := m.Sequencers.GetByAddress(nextSeqAddr) - if !found { - return types.ErrMissingProposerPubKey - } - copy(nextSeqHash[:], seq.MustHash()) - } + m.logger.Info("Sequencer rotation completed. sequencer is no longer the proposer", "nextSeqAddr", nextProposer.SettlementAddress) - err := m.CreateAndPostLastBatch(ctx, nextSeqHash) - if err != nil { - return fmt.Errorf("create and post last batch: %w", err) - } - - m.logger.Info("Sequencer rotation completed. sequencer is no longer the proposer", "nextSeqAddr", nextSeqAddr) - return nil + panic("rotate: sequencer is no longer the proposer. restarting as a full node") } // CreateAndPostLastBatch creates and posts the last batch to the hub @@ -163,7 +124,8 @@ func (m *Manager) CreateAndPostLastBatch(ctx context.Context, nextSeqHash [32]by return fmt.Errorf("load block: height: %d: %w", h, err) } - // check if the last block already produced with nextProposerHash set + // check if the last block already produced with nextProposerHash set. + // After creating the last block, the sequencer will be restarted so it will not be able to produce blocks anymore. if bytes.Equal(block.Header.NextSequencersHash[:], nextSeqHash[:]) { m.logger.Debug("Last block already produced and applied.") } else { @@ -220,9 +182,3 @@ func (m *Manager) HandleSequencerSetUpdate(newSet []types.Sequencer) error { m.Sequencers.Set(newSet) return nil } - -// UpdateProposerFromSL updates the proposer from the hub -func (m *Manager) UpdateProposerFromSL() error { - m.State.SetProposer(m.SLClient.GetProposer()) - return nil -} diff --git a/block/state.go b/block/state.go index b423c30d3..6c68e400a 100644 --- a/block/state.go +++ b/block/state.go @@ -137,8 +137,6 @@ func (e *Executor) UpdateStateAfterCommit(s *types.State, resp *tmstate.ABCIResp // UpdateProposerFromBlock updates the proposer from the block // The next proposer is defined in the block header (NextSequencersHash) -// In case of a node that a becomes the proposer, we return true to mark the role change -// currently the node will rebooted to apply the new role // TODO: (https://github.com/dymensionxyz/dymint/issues/1008) func (e *Executor) UpdateProposerFromBlock(s *types.State, seqSet *types.SequencerSet, block *types.Block) bool { // no sequencer change @@ -151,22 +149,17 @@ func (e *Executor) UpdateProposerFromBlock(s *types.State, seqSet *types.Sequenc // TODO: recover from halt (https://github.com/dymensionxyz/dymint/issues/1021) e.logger.Info("rollapp left with no proposer. chain is halted") s.SetProposer(nil) - return false + return true } // if hash changed, update the proposer + // We assume here that we're updated with the latest sequencer set + // FIXME: Think how to handle not being updated with the latest sequencer set seq, found := seqSet.GetByHash(block.Header.NextSequencersHash[:]) if !found { e.logger.Error("cannot find proposer by hash") panic("cannot find proposer by hash") } s.SetProposer(&seq) - - // check if this node becomes a proposer - localSeq, found := seqSet.GetByConsAddress(e.localAddress) - if found && bytes.Equal(localSeq.MustHash(), block.Header.NextSequencersHash[:]) { - return true - } - - return false + return true } diff --git a/mocks/github.com/dymensionxyz/dymint/settlement/mock_ClientI.go b/mocks/github.com/dymensionxyz/dymint/settlement/mock_ClientI.go index c21146436..3e3bcc9b6 100644 --- a/mocks/github.com/dymensionxyz/dymint/settlement/mock_ClientI.go +++ b/mocks/github.com/dymensionxyz/dymint/settlement/mock_ClientI.go @@ -26,12 +26,12 @@ func (_m *MockClientI) EXPECT() *MockClientI_Expecter { return &MockClientI_Expecter{mock: &_m.Mock} } -// CheckRotationInProgress provides a mock function with given fields: -func (_m *MockClientI) CheckRotationInProgress() (*types.Sequencer, error) { +// GetNextProposer provides a mock function with given fields: +func (_m *MockClientI) GetNextProposer() (*types.Sequencer, error) { ret := _m.Called() if len(ret) == 0 { - panic("no return value specified for CheckRotationInProgress") + panic("no return value specified for GetNextProposer") } var r0 *types.Sequencer @@ -56,29 +56,29 @@ func (_m *MockClientI) CheckRotationInProgress() (*types.Sequencer, error) { return r0, r1 } -// MockClientI_CheckRotationInProgress_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CheckRotationInProgress' -type MockClientI_CheckRotationInProgress_Call struct { +// MockClientI_GetNextProposer_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetNextProposer' +type MockClientI_GetNextProposer_Call struct { *mock.Call } -// CheckRotationInProgress is a helper method to define mock.On call -func (_e *MockClientI_Expecter) CheckRotationInProgress() *MockClientI_CheckRotationInProgress_Call { - return &MockClientI_CheckRotationInProgress_Call{Call: _e.mock.On("CheckRotationInProgress")} +// GetNextProposer is a helper method to define mock.On call +func (_e *MockClientI_Expecter) GetNextProposer() *MockClientI_GetNextProposer_Call { + return &MockClientI_GetNextProposer_Call{Call: _e.mock.On("GetNextProposer")} } -func (_c *MockClientI_CheckRotationInProgress_Call) Run(run func()) *MockClientI_CheckRotationInProgress_Call { +func (_c *MockClientI_GetNextProposer_Call) Run(run func()) *MockClientI_GetNextProposer_Call { _c.Call.Run(func(args mock.Arguments) { run() }) return _c } -func (_c *MockClientI_CheckRotationInProgress_Call) Return(_a0 *types.Sequencer, _a1 error) *MockClientI_CheckRotationInProgress_Call { +func (_c *MockClientI_GetNextProposer_Call) Return(_a0 *types.Sequencer, _a1 error) *MockClientI_GetNextProposer_Call { _c.Call.Return(_a0, _a1) return _c } -func (_c *MockClientI_CheckRotationInProgress_Call) RunAndReturn(run func() (*types.Sequencer, error)) *MockClientI_CheckRotationInProgress_Call { +func (_c *MockClientI_GetNextProposer_Call) RunAndReturn(run func() (*types.Sequencer, error)) *MockClientI_GetNextProposer_Call { _c.Call.Return(run) return _c } diff --git a/settlement/dymension/dymension.go b/settlement/dymension/dymension.go index 1c549996b..6f9820d62 100644 --- a/settlement/dymension/dymension.go +++ b/settlement/dymension/dymension.go @@ -31,7 +31,8 @@ import ( ) const ( - addressPrefix = "dym" + addressPrefix = "dym" + UNAVAILABLE_PROPOSER = "" ) const ( @@ -505,8 +506,11 @@ func (c *Client) GetBondedSequencers() ([]types.Sequencer, error) { return sequencerList, nil } -// CheckRotationInProgress implements settlement.ClientI. -func (c *Client) CheckRotationInProgress() (*types.Sequencer, error) { +// GetNextProposer returns the next proposer on the hub. +// In case the current proposer is the next proposer, it returns nil. +// in case there is no next proposer, it returns an empty sequencer struct. +// in case there is a next proposer, it returns the next proposer. +func (c *Client) GetNextProposer() (*types.Sequencer, error) { var ( nextAddr string found bool @@ -532,7 +536,7 @@ func (c *Client) CheckRotationInProgress() (*types.Sequencer, error) { if !found { return nil, nil } - if nextAddr == "" { + if nextAddr == UNAVAILABLE_PROPOSER { return &types.Sequencer{}, nil } diff --git a/settlement/grpc/grpc.go b/settlement/grpc/grpc.go index 1eaf1c609..367cff312 100644 --- a/settlement/grpc/grpc.go +++ b/settlement/grpc/grpc.go @@ -282,8 +282,8 @@ func (c *Client) GetBondedSequencers() ([]types.Sequencer, error) { return []types.Sequencer{*c.GetProposer()}, nil } -// CheckRotationInProgress implements settlement.ClientI. -func (c *Client) CheckRotationInProgress() (*types.Sequencer, error) { +// GetNextProposer implements settlement.ClientI. +func (c *Client) GetNextProposer() (*types.Sequencer, error) { return nil, nil } diff --git a/settlement/local/local.go b/settlement/local/local.go index 7714b0e20..3de07099d 100644 --- a/settlement/local/local.go +++ b/settlement/local/local.go @@ -243,8 +243,8 @@ func (c *Client) GetBondedSequencers() ([]types.Sequencer, error) { return []types.Sequencer{*c.GetProposer()}, nil } -// CheckRotationInProgress implements settlement.ClientI. -func (c *Client) CheckRotationInProgress() (*types.Sequencer, error) { +// GetNextProposer implements settlement.ClientI. +func (c *Client) GetNextProposer() (*types.Sequencer, error) { return nil, nil } diff --git a/settlement/settlement.go b/settlement/settlement.go index 3a851a416..b4f57d7e7 100644 --- a/settlement/settlement.go +++ b/settlement/settlement.go @@ -89,7 +89,7 @@ type ClientI interface { GetBondedSequencers() ([]types.Sequencer, error) // GetProposer returns the current proposer for this chain. GetProposer() *types.Sequencer - // CheckRotationInProgress returns the next proposer for this chain in case of a rotation. + // GetNextProposer returns the next proposer for this chain in case of a rotation. // If no rotation is in progress, it should return nil. - CheckRotationInProgress() (*types.Sequencer, error) + GetNextProposer() (*types.Sequencer, error) }