From f3beaf42509f1f164966c7ba562689c9e62b7b28 Mon Sep 17 00:00:00 2001 From: "allen.wu" Date: Tue, 30 Dec 2025 11:26:08 +0800 Subject: [PATCH] concensus: pbft to single sequencer, include P2P & block produce --- blocksync/msgs.go | 11 + blocksync/pool.go | 49 +- blocksync/reactor.go | 238 +++++- consensus/reactor.go | 18 + consensus/replay.go | 9 + consensus/state.go | 38 +- l2node/l2node.go | 23 + l2node/mock.go | 31 + node/node.go | 151 +++- proto/tendermint/blocksync/types.pb.go | 322 +++++++- proto/tendermint/blocksync/types.proto | 19 +- proto/tendermint/sequencer/types.pb.go | 988 +++++++++++++++++++++++++ proto/tendermint/sequencer/types.proto | 24 + sequencer/block_cache.go | 195 +++++ sequencer/block_cache_test.go | 393 ++++++++++ sequencer/broadcast_reactor.go | 638 ++++++++++++++++ sequencer/hash_set.go | 117 +++ sequencer/pending_cache.go | 138 ++++ sequencer/state_v2.go | 257 +++++++ sequencer/state_v2_test.go | 143 ++++ sequencer/types.go | 15 + types/block.go | 13 + types/block_v2.go | 159 ++++ upgrade/upgrade.go | 38 + 24 files changed, 3961 insertions(+), 66 deletions(-) create mode 100644 proto/tendermint/sequencer/types.pb.go create mode 100644 proto/tendermint/sequencer/types.proto create mode 100644 sequencer/block_cache.go create mode 100644 sequencer/block_cache_test.go create mode 100644 sequencer/broadcast_reactor.go create mode 100644 sequencer/hash_set.go create mode 100644 sequencer/pending_cache.go create mode 100644 sequencer/state_v2.go create mode 100644 sequencer/state_v2_test.go create mode 100644 sequencer/types.go create mode 100644 types/block_v2.go create mode 100644 upgrade/upgrade.go diff --git a/blocksync/msgs.go b/blocksync/msgs.go index e3d6e551c15..52903e5918f 100644 --- a/blocksync/msgs.go +++ b/blocksync/msgs.go @@ -28,6 +28,8 @@ func EncodeMsg(pb proto.Message) ([]byte, error) { msg.Sum = &bcproto.Message_BlockRequest{BlockRequest: pb} case *bcproto.BlockResponse: msg.Sum = &bcproto.Message_BlockResponse{BlockResponse: pb} + case *bcproto.BlockResponseV2: + msg.Sum = &bcproto.Message_BlockResponseV2{BlockResponseV2: pb} case *bcproto.NoBlockResponse: msg.Sum = &bcproto.Message_NoBlockResponse{NoBlockResponse: pb} case *bcproto.StatusRequest: @@ -60,6 +62,8 @@ func DecodeMsg(bz []byte) (proto.Message, error) { return msg.BlockRequest, nil case *bcproto.Message_BlockResponse: return msg.BlockResponse, nil + case *bcproto.Message_BlockResponseV2: + return msg.BlockResponseV2, nil case *bcproto.Message_NoBlockResponse: return msg.NoBlockResponse, nil case *bcproto.Message_StatusRequest: @@ -83,10 +87,17 @@ func ValidateMsg(pb proto.Message) error { return errors.New("negative Height") } case *bcproto.BlockResponse: + // V1 block format _, err := types.BlockFromProto(msg.Block) if err != nil { return err } + case *bcproto.BlockResponseV2: + // V2 block format (sequencer mode) + _, err := types.BlockV2FromProto(msg.Block) + if err != nil { + return err + } case *bcproto.NoBlockResponse: if msg.Height < 0 { return errors.New("negative Height") diff --git a/blocksync/pool.go b/blocksync/pool.go index 57ba94ce835..b059b36e750 100644 --- a/blocksync/pool.go +++ b/blocksync/pool.go @@ -186,11 +186,29 @@ func (pool *BlockPool) IsCaughtUp() bool { return isCaughtUp } +// MaxPeerHeight returns the highest height reported by any peer. +func (pool *BlockPool) MaxPeerHeight() int64 { + pool.mtx.Lock() + defer pool.mtx.Unlock() + return pool.maxPeerHeight +} + +// GetPeerHeight returns the height of a specific peer. +// Returns 0 if peer is not found. +func (pool *BlockPool) GetPeerHeight(peerID p2p.ID) int64 { + pool.mtx.Lock() + defer pool.mtx.Unlock() + if peer, ok := pool.peers[peerID]; ok { + return peer.height + } + return 0 +} + // PeekTwoBlocks returns blocks at pool.height and pool.height+1. // We need to see the second block's Commit to validate the first block. // So we peek two blocks at a time. // The caller will verify the commit. -func (pool *BlockPool) PeekTwoBlocks() (first *types.Block, second *types.Block) { +func (pool *BlockPool) PeekTwoBlocks() (first, second types.SyncableBlock) { pool.mtx.Lock() defer pool.mtx.Unlock() @@ -242,12 +260,15 @@ func (pool *BlockPool) RedoRequest(height int64) p2p.ID { } // AddBlock validates that the block comes from the peer it was expected from and calls the requester to store it. +// AddBlock adds a block (either Block or BlockV2) to the pool. +// Uses SyncableBlock interface to handle both types uniformly. // TODO: ensure that blocks come in order for each peer. -func (pool *BlockPool) AddBlock(peerID p2p.ID, block *types.Block, blockSize int) { +func (pool *BlockPool) AddBlock(peerID p2p.ID, block types.SyncableBlock, blockSize int) { pool.mtx.Lock() defer pool.mtx.Unlock() - requester := pool.requesters[block.Height] + height := block.GetHeight() + requester := pool.requesters[height] if requester == nil { pool.Logger.Info( "peer sent us a block we didn't expect", @@ -256,8 +277,8 @@ func (pool *BlockPool) AddBlock(peerID p2p.ID, block *types.Block, blockSize int "curHeight", pool.height, "blockHeight", - block.Height) - diff := pool.height - block.Height + height) + diff := pool.height - height if diff < 0 { diff *= -1 } @@ -274,16 +295,20 @@ func (pool *BlockPool) AddBlock(peerID p2p.ID, block *types.Block, blockSize int peer.decrPending(blockSize) } } else { - pool.Logger.Info("invalid peer", "peer", peerID, "blockHeight", block.Height) + pool.Logger.Info("invalid peer", "peer", peerID, "blockHeight", height) pool.sendError(errors.New("invalid peer"), peerID) } } -// MaxPeerHeight returns the highest reported height. -func (pool *BlockPool) MaxPeerHeight() int64 { +// GetPeerIDs returns all peer IDs in the pool. +func (pool *BlockPool) GetPeerIDs() []p2p.ID { pool.mtx.Lock() defer pool.mtx.Unlock() - return pool.maxPeerHeight + ids := make([]p2p.ID, 0, len(pool.peers)) + for id := range pool.peers { + ids = append(ids, id) + } + return ids } // SetPeerRange sets the peer's alleged blockchain base and height. @@ -515,7 +540,7 @@ type bpRequester struct { mtx tmsync.Mutex peerID p2p.ID - block *types.Block + block types.SyncableBlock // Supports both Block and BlockV2 } func newBPRequester(pool *BlockPool, height int64) *bpRequester { @@ -538,7 +563,7 @@ func (bpr *bpRequester) OnStart() error { } // Returns true if the peer matches and block doesn't already exist. -func (bpr *bpRequester) setBlock(block *types.Block, peerID p2p.ID) bool { +func (bpr *bpRequester) setBlock(block types.SyncableBlock, peerID p2p.ID) bool { bpr.mtx.Lock() if bpr.block != nil || bpr.peerID != peerID { bpr.mtx.Unlock() @@ -554,7 +579,7 @@ func (bpr *bpRequester) setBlock(block *types.Block, peerID p2p.ID) bool { return true } -func (bpr *bpRequester) getBlock() *types.Block { +func (bpr *bpRequester) getBlock() types.SyncableBlock { bpr.mtx.Lock() defer bpr.mtx.Unlock() return bpr.block diff --git a/blocksync/reactor.go b/blocksync/reactor.go index db0040a74aa..eb530426c24 100644 --- a/blocksync/reactor.go +++ b/blocksync/reactor.go @@ -14,6 +14,7 @@ import ( sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/store" "github.com/tendermint/tendermint/types" + "github.com/tendermint/tendermint/upgrade" ) const ( @@ -38,6 +39,17 @@ type consensusReactor interface { SwitchToConsensus(state sm.State, skipWAL bool) } +type sequencerReactor interface { + // for when we switch from blockchain reactor to sequencer mode + StartSequencerRoutines() error +} + +// SequencerState interface for accessing sequencer state (avoids import cycle) +type SequencerState interface { + LatestHeight() int64 + ApplyBlock(block *types.BlockV2) error +} + type peerError struct { err error peerID p2p.ID @@ -51,7 +63,8 @@ func (e peerError) Error() string { type Reactor struct { p2p.BaseReactor - l2Node l2node.L2Node + l2Node l2node.L2Node // Unified interface for both PBFT and sequencer mode + stateV2 SequencerState // Sequencer state for post-upgrade sync // immutable initialState sm.State @@ -72,6 +85,7 @@ func NewReactor( blockExec *sm.BlockExecutor, store *store.BlockStore, blockSync bool, + stateV2 SequencerState, ) *Reactor { if state.LastBlockHeight != store.Height() { @@ -83,14 +97,27 @@ func NewReactor( const capacity = 1000 // must be bigger than peers count errorsCh := make(chan peerError, capacity) // so we don't block in #Receive#pool.AddBlock + // Determine start height: max of state height and l2node height (for post-upgrade) startHeight := store.Height() + 1 if startHeight == 1 { startHeight = state.InitialHeight } + + // Check l2node for latest block (may be ahead after upgrade) + if l2Node != nil { + if latestBlock, err := l2Node.GetLatestBlockV2(); err == nil && latestBlock != nil { + l2Height := int64(latestBlock.Number) + if l2Height >= startHeight { + startHeight = l2Height + 1 + } + } + } + pool := NewBlockPool(startHeight, requestsCh, errorsCh) bcR := &Reactor{ l2Node: l2Node, + stateV2: stateV2, initialState: state, blockExec: blockExec, store: store, @@ -109,6 +136,16 @@ func (bcR *Reactor) SetLogger(l log.Logger) { bcR.pool.Logger = l } +// SetStateV2 sets the sequencer state (called after upgrade). +func (bcR *Reactor) SetStateV2(stateV2 SequencerState) { + bcR.stateV2 = stateV2 +} + +// Pool returns the block pool for broadcast reactor to check peer heights. +func (bcR *Reactor) Pool() *BlockPool { + return bcR.pool +} + // OnStart implements service.Service. func (bcR *Reactor) OnStart() error { if bcR.blockSync { @@ -161,7 +198,7 @@ func (bcR *Reactor) GetChannels() []*p2p.ChannelDescriptor { func (bcR *Reactor) AddPeer(peer p2p.Peer) { msgBytes, err := EncodeMsg(&bcproto.StatusResponse{ Base: bcR.store.Base(), - Height: bcR.store.Height()}) + Height: bcR.getHeight()}) if err != nil { bcR.Logger.Error("could not convert msg to protobuf", "err", err) return @@ -184,6 +221,11 @@ func (bcR *Reactor) RemovePeer(peer p2p.Peer, reason interface{}) { func (bcR *Reactor) respondToPeer(msg *bcproto.BlockRequest, src p2p.Peer) (queued bool) { + // Check if already upgraded + if upgrade.IsUpgraded(msg.Height) { + return bcR.respondToPeerV2(msg, src) + } + block := bcR.store.LoadBlock(msg.Height) if block != nil { bl, err := block.ToProto() @@ -212,6 +254,78 @@ func (bcR *Reactor) respondToPeer(msg *bcproto.BlockRequest, return src.TrySend(BlocksyncChannel, msgBytes) } +// respondToPeerV2 handles block requests after the sequencer upgrade. +// It retrieves blocks from geth instead of the local blockStore. +func (bcR *Reactor) respondToPeerV2(msg *bcproto.BlockRequest, src p2p.Peer) bool { + // Get block from geth using unified l2Node interface + blockData, err := bcR.l2Node.GetBlockByNumber(uint64(msg.Height)) + if err != nil { + bcR.Logger.Error("Failed to get block from geth", "height", msg.Height, "err", err) + + // Send NoBlockResponse + msgBytes, encErr := EncodeMsg(&bcproto.NoBlockResponse{Height: msg.Height}) + if encErr != nil { + bcR.Logger.Error("could not convert msg to protobuf", "err", encErr) + return false + } + return src.TrySend(BlocksyncChannel, msgBytes) + } + + bcR.Logger.Debug("respondToPeerV2: got block from geth", + "height", msg.Height, + "hash", blockData.Hash.Hex()) + + // Convert to proto and send using BlockResponseV2 + blockV2Proto := types.BlockV2ToProto(blockData) + msgBytes, err := EncodeMsg(&bcproto.BlockResponseV2{ + Block: blockV2Proto, + }) + if err != nil { + bcR.Logger.Error("could not encode BlockV2 response", "err", err) + return false + } + return src.TrySend(BlocksyncChannel, msgBytes) +} + +// L2Node returns the L2Node interface for use by sequencer mode. +func (bcR *Reactor) L2Node() l2node.L2Node { + return bcR.l2Node +} + +// syncBlockV2 handles syncing a single BlockV2 in sequencer mode. +// No signature verification during sync - only broadcast channel verifies signatures. +// Returns true if sync was successful, false if there was an error (already handled). +func (bcR *Reactor) syncBlockV2(block types.SyncableBlock, blocksSynced *uint64, lastRate *float64, lastHundred *time.Time) bool { + blockV2, ok := block.(*types.BlockV2) + if !ok { + bcR.Logger.Error("Expected BlockV2 after upgrade", "height", block.GetHeight()) + bcR.pool.RedoRequest(block.GetHeight()) + return false + } + + // Apply BlockV2 via stateV2 (no signature verification during sync) + if err := bcR.stateV2.ApplyBlock(blockV2); err != nil { + bcR.Logger.Error("Failed to apply BlockV2", "height", blockV2.Number, "err", err) + bcR.pool.RedoRequest(blockV2.GetHeight()) + return false + } + + bcR.pool.PopRequest() + *blocksSynced++ + + if *blocksSynced%100 == 0 { + *lastRate = 0.9*(*lastRate) + 0.1*(100/time.Since(*lastHundred).Seconds()) + bcR.Logger.Info( + "BlockV2 Sync Rate", + "height", bcR.pool.height, + "max_peer_height", bcR.pool.MaxPeerHeight(), + "blocks/s", *lastRate, + ) + *lastHundred = time.Now() + } + return true +} + // Receive implements Reactor by handling 4 types of messages (look below). func (bcR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { msg, err := DecodeMsg(msgBytes) @@ -239,10 +353,17 @@ func (bcR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { return } bcR.pool.AddBlock(src.ID(), bi, len(msgBytes)) + case *bcproto.BlockResponseV2: + blockV2, err := types.BlockV2FromProto(msg.Block) + if err != nil { + bcR.Logger.Error("BlockV2 content is invalid", "err", err) + return + } + bcR.pool.AddBlock(src.ID(), blockV2, len(msgBytes)) case *bcproto.StatusRequest: // Send peer our state. msgBytes, err := EncodeMsg(&bcproto.StatusResponse{ - Height: bcR.store.Height(), + Height: bcR.getHeight(), Base: bcR.store.Base(), }) if err != nil { @@ -252,6 +373,7 @@ func (bcR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { src.TrySend(BlocksyncChannel, msgBytes) case *bcproto.StatusResponse: // Got a peer status. Unverified. + bcR.Logger.Debug("SetPeerRange", "peer", src.ID(), "base", msg.Base, "height", msg.Height) bcR.pool.SetPeerRange(src.ID(), msg.Base, msg.Height) case *bcproto.NoBlockResponse: bcR.Logger.Debug("Peer does not have requested block", "peer", src, "height", msg.Height) @@ -263,12 +385,15 @@ func (bcR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { // Handle messages from the poolReactor telling the reactor what to do. // NOTE: Don't sleep in the FOR_LOOP or otherwise slow it down! func (bcR *Reactor) poolRoutine(stateSynced bool) { + // optimize: try to get peer status immediately after start + bcR.BroadcastStatusRequest() trySyncTicker := time.NewTicker(trySyncIntervalMS * time.Millisecond) defer trySyncTicker.Stop() statusUpdateTicker := time.NewTicker(statusUpdateIntervalSeconds * time.Second) - defer statusUpdateTicker.Stop() + // no longer stop the ticker, reuse it for sequencer mode status updates + //defer statusUpdateTicker.Stop() switchToConsensusTicker := time.NewTicker(switchToConsensusIntervalSeconds * time.Second) defer switchToConsensusTicker.Stop() @@ -288,8 +413,10 @@ func (bcR *Reactor) poolRoutine(stateSynced bool) { select { case <-bcR.Quit(): return - case <-bcR.pool.Quit(): - return + // Note: removed `case <-bcR.pool.Quit(): return` to keep peer status updates for stateV2 + // running after pool.Stop(). pool.SetPeerRange works regardless of pool state. + //case <-bcR.pool.Quit(): + // return case request := <-bcR.requestsCh: peer := bcR.Switch.Peers().Get(request.PeerID) if peer == nil { @@ -330,19 +457,31 @@ FOR_LOOP: "outbound", outbound, "inbound", inbound, ) + if bcR.pool.IsCaughtUp() { - bcR.Logger.Info("Time to switch to consensus reactor!", "height", height) + // Stop pool and tickers (peer status updates continue in background goroutine) + bcR.Logger.Info("Caught up, stopping pool", "height", height) if err := bcR.pool.Stop(); err != nil { bcR.Logger.Error("Error stopping pool", "err", err) } - conR, ok := bcR.Switch.Reactor("CONSENSUS").(consensusReactor) - if ok { - conR.SwitchToConsensus(state, blocksSynced > 0 || stateSynced) - } - // else { - // should only happen during testing - // } + if upgrade.IsUpgraded(height) { + // Sequencer mode + bcR.Logger.Info("Switching to sequencer mode", "height", height) + seqR, ok := bcR.Switch.Reactor("SEQUENCER").(sequencerReactor) + if ok { + if err := seqR.StartSequencerRoutines(); err != nil { + bcR.Logger.Error("Failed to start sequencer mode", "err", err) + } + } + } else { + // PBFT mode + bcR.Logger.Info("Switching to consensus reactor", "height", height) + conR, ok := bcR.Switch.Reactor("CONSENSUS").(consensusReactor) + if ok { + conR.SwitchToConsensus(state, blocksSynced > 0 || stateSynced) + } + } break FOR_LOOP } @@ -362,9 +501,9 @@ FOR_LOOP: // routine. // See if there are any blocks to sync. - first, second := bcR.pool.PeekTwoBlocks() + firstSync, secondSync := bcR.pool.PeekTwoBlocks() // bcR.Logger.Info("TrySync peeked", "first", first, "second", second) - if first == nil || second == nil { + if firstSync == nil || secondSync == nil { // We need both to sync the first block. continue FOR_LOOP } else { @@ -372,6 +511,24 @@ FOR_LOOP: didProcessCh <- struct{}{} } + if firstSync.GetHeight()+1 == upgrade.UpgradeBlockHeight { + if err := bcR.handleTheLastTMBlock(state, firstSync); err != nil { + bcR.Logger.Error("Error in apply last tendermint block, ", "err", err) + bcR.pool.PopRequest() + blocksSynced++ + continue FOR_LOOP + } + } + + // Check if we're in sequencer mode (after upgrade) + if upgrade.IsUpgraded(firstSync.GetHeight()) { + bcR.syncBlockV2(firstSync, &blocksSynced, &lastRate, &lastHundred) + continue FOR_LOOP + } + + // PBFT mode: type assert to *types.Block + first, second := firstSync.(*types.Block), secondSync.(*types.Block) + firstParts, err := first.MakePartSet(types.BlockPartSizeBytes) if err != nil { bcR.Logger.Error( @@ -509,3 +666,52 @@ func (bcR *Reactor) BroadcastStatusRequest() error { return nil } + +// just skip the last tendermint block commit check. +// TODO: consider add the commit check in the future or using batch derivation reorg +func (bcR *Reactor) handleTheLastTMBlock(state sm.State, lastSyncable types.SyncableBlock) error { + last := lastSyncable.(*types.Block) + lastParts, err := last.MakePartSet(types.BlockPartSizeBytes) + if err != nil { + bcR.Logger.Error( + "failed to make ", + "height", last.Height, + "err", err.Error(), + ) + return err + } + + nilCommit := &types.Commit{Height: last.GetHeight()} + bcR.store.SaveBlock(last, lastParts, nilCommit) + lastPartSetHeader := lastParts.Header() + lastID := types.BlockID{Hash: last.Hash(), BatchHash: last.BatchHash, PartSetHeader: lastPartSetHeader} + + // TODO: same thing for app - but we would need a way to + // get the hash without persisting the state + state, _, err = bcR.blockExec.ApplyBlock( + state, + lastID, + last, + nil, + ) + if err != nil { + // TODO This is bad, are we zombie? + panic(fmt.Sprintf("Failed to process committed block (%d:%X): %v", last.Height, last.Hash(), err)) + } + + return nil +} + +func (bcR *Reactor) getHeight() int64 { + height := bcR.store.Height() + // In sequencer mode, get height directly from l2Node (geth) + if bcR.l2Node != nil && upgrade.IsUpgraded(height+1) { + if l2Block, err := bcR.l2Node.GetLatestBlockV2(); err == nil && l2Block != nil { + bcR.Logger.Debug("StatusRequest", "l2Height", l2Block.Number) + if l2Height := int64(l2Block.Number); l2Height > height { + height = l2Height + } + } + } + return height +} diff --git a/consensus/reactor.go b/consensus/reactor.go index 7d2ccbe83c3..51769d2f872 100644 --- a/consensus/reactor.go +++ b/consensus/reactor.go @@ -3,6 +3,7 @@ package consensus import ( "errors" "fmt" + "github.com/tendermint/tendermint/upgrade" "reflect" "sync" "time" @@ -76,6 +77,14 @@ func NewReactor(consensusState *State, waitSync bool, options ...ReactorOption) func (conR *Reactor) OnStart() error { conR.Logger.Info("Reactor ", "waitSync", conR.WaitSync()) + // Check if we've already upgraded - don't start consensus if so + if upgrade.IsUpgraded(conR.conS.Height) { + conR.Logger.Info("Already upgraded to sequencer mode, consensus reactor will not start", + "height", conR.conS.Height, + "upgradeHeight", upgrade.UpgradeBlockHeight) + return nil + } + // start routine that computes peer statistics for evaluating peer quality go conR.peerStatsRoutine() @@ -104,6 +113,15 @@ func (conR *Reactor) OnStop() { } } +// StopForUpgrade stops the consensus reactor when upgrading to sequencer mode. +// This is called when the chain reaches the upgrade height. +func (conR *Reactor) StopForUpgrade() { + conR.Logger.Info("Stopping consensus reactor for sequencer upgrade", + "height", conR.conS.Height, + "upgradeHeight", upgrade.UpgradeBlockHeight) + conR.OnStop() +} + // SwitchToConsensus switches from block_sync mode to consensus mode. // It resets the state, turns off block_sync, and starts the consensus state-machine func (conR *Reactor) SwitchToConsensus(state sm.State, skipWAL bool) { diff --git a/consensus/replay.go b/consensus/replay.go index 89b3b899afd..dff352840ca 100644 --- a/consensus/replay.go +++ b/consensus/replay.go @@ -15,6 +15,7 @@ import ( "github.com/tendermint/tendermint/proxy" sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/types" + "github.com/tendermint/tendermint/upgrade" ) var crc32c = crc32.MakeTable(crc32.Castagnoli) @@ -368,6 +369,14 @@ func (h *Handshaker) ReplayBlocks( return appHash, sm.ErrAppBlockHeightTooLow{AppHeight: appBlockHeight, StoreBase: storeBlockBase} case storeBlockHeight < appBlockHeight: + // In sequencer mode (after upgrade), the app (geth) continues producing blocks + // while tendermint's blockstore stops. This is expected behavior. + if upgrade.IsUpgraded(storeBlockHeight + 1) { + h.logger.Info("Sequencer mode: app height is ahead of store, this is expected", + "appHeight", appBlockHeight, + "storeHeight", storeBlockHeight) + return appHash, nil + } // the app should never be ahead of the store (but this is under app's control) return appHash, sm.ErrAppBlockHeightTooHigh{CoreHeight: storeBlockHeight, AppHeight: appBlockHeight} diff --git a/consensus/state.go b/consensus/state.go index da5f4a5a0ec..3e5f878132a 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -4,6 +4,7 @@ import ( "bytes" "errors" "fmt" + "github.com/tendermint/tendermint/upgrade" "io" "os" "runtime/debug" @@ -149,6 +150,10 @@ type State struct { // state only emits EventNewRoundStep and EventVote evsw tmevents.EventSwitch + // onUpgrade is called when reaching upgrade height (before stopping state) + // Used to notify reactor to switch to sequencer mode + onUpgrade func() + // for reporting metrics metrics *Metrics } @@ -307,6 +312,15 @@ func (cs *State) SetTimeoutTicker(timeoutTicker TimeoutTicker) { cs.mtx.Unlock() } +// SetOnUpgrade sets the callback to be invoked when reaching upgrade height. +// The callback is called before stopping the state, allowing external components +// (like reactor) to handle the upgrade transition. +func (cs *State) SetOnUpgrade(fn func()) { + cs.mtx.Lock() + cs.onUpgrade = fn + cs.mtx.Unlock() +} + // LoadCommit loads the commit for a given height. func (cs *State) LoadCommit(height int64) *types.Commit { cs.mtx.RLock() @@ -1898,6 +1912,25 @@ func (cs *State) finalizeCommit(height int64) { logger.Error("failed to get private validator pubkey", "err", err) } + // Check for upgrade to sequencer mode + if upgrade.IsUpgraded(cs.Height) { + logger.Info("Upgrade height reached, switching to sequencer mode", + "height", cs.Height, + "upgradeHeight", upgrade.UpgradeBlockHeight) + + // Call upgrade callback first (allows reactor to handle transition) + if cs.onUpgrade != nil { + cs.onUpgrade() + } + + // Stop consensus state + if err := cs.Stop(); err != nil { + logger.Error("Failed to stop consensus state", "err", err) + panic(err) + } + return + } + // cs.StartTime is already set. // Schedule Round0 to start soon. cs.scheduleRound0(&cs.RoundState) @@ -2298,7 +2331,7 @@ func (cs *State) addVote(vote *types.Vote, peerID p2p.ID, replay bool) (added bo cs.evsw.FireEvent(types.EventVote, vote) // if we have all the votes now - if cs.LastCommit.HasAll() { + if cs.LastCommit.HasAll() && !upgrade.IsUpgraded(cs.Height) { // go straight to new round (skip timeout commit) // cs.scheduleTimeout(time.Duration(0), cs.Height, 0, cstypes.RoundStepNewHeight) if cs.config.SkipTimeoutCommit { @@ -2455,7 +2488,8 @@ func (cs *State) addVote(vote *types.Vote, peerID p2p.ID, replay bool) (added bo if len(blockID.Hash) != 0 { cs.enterCommit(height, vote.Round) - if precommits.HasAll() { + if precommits.HasAll() && + !upgrade.IsUpgraded(cs.Height) { // if upgrade active, skip the following step if cs.config.SkipTimeoutCommit { cs.enterNewRound(cs.Height, 0) } else { diff --git a/l2node/l2node.go b/l2node/l2node.go index 49bcb4261ca..b6bf45a681a 100644 --- a/l2node/l2node.go +++ b/l2node/l2node.go @@ -7,6 +7,9 @@ import ( "github.com/tendermint/tendermint/types" ) +// BlockV2 is an alias to types.BlockV2 for convenience. +type BlockV2 = types.BlockV2 + type L2Node interface { Batcher RequestHeight( @@ -58,6 +61,26 @@ type L2Node interface { valid bool, err error, ) + + // ==================== V2 Methods for Sequencer Mode ==================== + // These methods are used after the upgrade to centralized sequencer mode. + // They operate on BlockV2 (based on ExecutableL2Data) instead of raw txs/blockMeta. + + // RequestBlockDataV2 requests block data based on parent hash (for fork chain support). + // Uses Engine API to assemble a new block. + RequestBlockDataV2(parentHash []byte) (block *BlockV2, collectedL1Msgs bool, err error) + + // ApplyBlockV2 applies a BlockV2 to the L2 execution layer. + // Uses Engine API (NewL2Block) internally. + ApplyBlockV2(block *BlockV2) error + + // GetBlockByNumber retrieves a BlockV2 by its number. + // Can be implemented using geth's eth_getBlockByNumber JSON-RPC. + GetBlockByNumber(height uint64) (*BlockV2, error) + + // GetLatestBlockV2 returns the latest block. + // Can be implemented using geth's eth_blockNumber + eth_getBlockByNumber JSON-RPC. + GetLatestBlockV2() (*BlockV2, error) } // Batcher is used to pack the blocks into a batch, and commit the batch if it is determined to be a batchPoint. diff --git a/l2node/mock.go b/l2node/mock.go index bd919ae7543..124268284fd 100644 --- a/l2node/mock.go +++ b/l2node/mock.go @@ -244,3 +244,34 @@ func (l *MockL2Node) AppendBlsData(height int64, batchHash []byte, data BlsData) func (l *MockL2Node) BatchHash(batchHeader []byte) ([]byte, error) { return tmhash.Sum(batchHeader), nil } + +// ==================== V2 Methods for Sequencer Mode ==================== + +func (l *MockL2Node) RequestBlockDataV2(parentHash []byte) (*BlockV2, bool, error) { + // Mock implementation: return a simple block + return &BlockV2{ + Number: 1, + Timestamp: 0, + }, false, nil +} + +func (l *MockL2Node) ApplyBlockV2(block *BlockV2) error { + // Mock implementation: do nothing + return nil +} + +func (l *MockL2Node) GetBlockByNumber(height uint64) (*BlockV2, error) { + // Mock implementation: return a simple block + return &BlockV2{ + Number: height, + Timestamp: 0, + }, nil +} + +func (l *MockL2Node) GetLatestBlockV2() (*BlockV2, error) { + // Mock implementation: return block 0 + return &BlockV2{ + Number: 0, + Timestamp: 0, + }, nil +} diff --git a/node/node.go b/node/node.go index 599cabc84fd..6ead8ec33b9 100644 --- a/node/node.go +++ b/node/node.go @@ -3,13 +3,20 @@ package node import ( "bytes" "context" + "crypto/ecdsa" + "encoding/hex" "errors" "fmt" "net" "net/http" + "os" "strings" "time" + "github.com/tendermint/tendermint/upgrade" + + ethcrypto "github.com/morph-l2/go-ethereum/crypto" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/rs/cors" @@ -36,6 +43,7 @@ import ( rpccore "github.com/tendermint/tendermint/rpc/core" grpccore "github.com/tendermint/tendermint/rpc/grpc" rpcserver "github.com/tendermint/tendermint/rpc/jsonrpc/server" + "github.com/tendermint/tendermint/sequencer" sm "github.com/tendermint/tendermint/state" "github.com/tendermint/tendermint/state/indexer" blockidxkv "github.com/tendermint/tendermint/state/indexer/block/kv" @@ -241,6 +249,11 @@ type Node struct { blockIndexer indexer.BlockIndexer indexerService *txindex.IndexerService prometheusSrv *http.Server + + // Sequencer mode (after upgrade) + stateV2 *sequencer.StateV2 + blockBroadcastReactor *sequencer.BlockBroadcastReactor + sequencerPrivKey *ecdsa.PrivateKey // ECDSA key for signing blocks in sequencer mode } func initDBs(config *cfg.Config, dbProvider DBProvider) (blockStore *store.BlockStore, stateDB dbm.DB, err error) { @@ -437,7 +450,7 @@ func createBlockchainReactor( ) { switch config.BlockSync.Version { case "v0": - bcReactor = bc.NewReactor(l2Node, state.Copy(), blockExec, blockStore, blockSync) + bcReactor = bc.NewReactor(l2Node, state.Copy(), blockExec, blockStore, blockSync, nil) // stateV2 set later via SetStateV2 case "v1", "v2": return nil, fmt.Errorf("block sync version %s has been deprecated. Please use v0", config.BlockSync.Version) default: @@ -491,6 +504,38 @@ func createConsensusReactor( return consensusReactor, consensusState } +// createSequencerComponents creates the sequencer mode components (StateV2 and BlockBroadcastReactor). +// These components are created but not started - they will be started when switching to sequencer mode. +func createSequencerComponents( + l2Node l2node.L2Node, + sequencerPrivKey *ecdsa.PrivateKey, + pool *bc.BlockPool, + waitSync bool, + logger log.Logger, +) (*sequencer.StateV2, *sequencer.BlockBroadcastReactor, error) { + // Create StateV2 + stateV2, err := sequencer.NewStateV2( + l2Node, + sequencerPrivKey, + sequencer.DefaultBlockInterval, + logger, + ) + if err != nil { + return nil, nil, fmt.Errorf("failed to create StateV2: %w", err) + } + + // Create BlockBroadcastReactor (not started yet) + broadcastReactor := sequencer.NewBlockBroadcastReactor( + pool, + stateV2, + waitSync, + logger, + ) + broadcastReactor.SetLogger(logger.With("module", "sequencer")) + + return stateV2, broadcastReactor, nil +} + func createTransport( config *cfg.Config, nodeInfo p2p.NodeInfo, @@ -708,7 +753,20 @@ func startStateSync( return } } else { - conR.SwitchToConsensus(state, true) + // Check if we should switch to sequencer mode instead of consensus + if upgrade.IsUpgraded(state.LastBlockHeight + 1) { + ssR.Logger.Info("State sync completed at upgrade height, switching to sequencer mode", + "height", state.LastBlockHeight) + if seqR, ok := bcR.(*bc.Reactor); ok { + if bbR, ok := seqR.Switch.Reactor("SEQUENCER").(interface{ StartSequencerRoutines() error }); ok { + if err := bbR.StartSequencerRoutines(); err != nil { + ssR.Logger.Error("Failed to start sequencer routines", "err", err) + } + } + } + } else { + conR.SwitchToConsensus(state, true) + } } }() return nil @@ -945,6 +1003,35 @@ func NewNode( } node.BaseService = *service.NewBaseService(logger, "Node", node) + // Initialize sequencer mode components (but don't start them) + if bcR, ok := bcReactor.(*bc.Reactor); ok { + l2NodeRef := bcR.L2Node() + + // TODO: just for Phase1, will update in future + if err := node.SetSequencerPrivKey(); err != nil { + return nil, err + } + // Create sequencer components + if node.stateV2, node.blockBroadcastReactor, err = createSequencerComponents( + l2NodeRef, + node.sequencerPrivKey, + bcR.Pool(), + blockSync || stateSync, + logger); err != nil { + return nil, err + } + + // Set stateV2 on blocksync reactor for post-upgrade sync + bcR.SetStateV2(node.stateV2) + + // Register BlockBroadcastReactor with Switch + sw.AddReactor("SEQUENCER", node.blockBroadcastReactor) + + // Set upgrade callback on consensus state + // This is called when PBFT reaches upgrade height during normal block production + consensusState.SetOnUpgrade(node.switchToSequencerMode) + } + for _, option := range options { option(node) } @@ -1359,6 +1446,8 @@ func makeNodeInfo( evidence.EvidenceChannel, statesync.SnapshotChannel, statesync.ChunkChannel, + sequencer.BlockBroadcastChannel, + sequencer.SequencerSyncChannel, }, Moniker: config.Moniker, Other: p2p.DefaultNodeInfoOther{ @@ -1503,3 +1592,61 @@ func splitAndTrimEmpty(s, sep, cutset string) []string { } return nonEmptyStrings } + +// ============================================================================ +// Sequencer Mode Methods +// ============================================================================ + +// TODO: optimize SetSequencerPrivKey in the future +// SetSequencerPrivKey sets the ECDSA private key for signing blocks in sequencer mode. +func (n *Node) SetSequencerPrivKey() error { + // Load sequencer private key from environment variable + if seqKeyHex := os.Getenv("SEQUENCER_PRIVATE_KEY"); seqKeyHex != "" { + seqKeyHex = strings.TrimPrefix(seqKeyHex, "0x") + keyBytes, err := hex.DecodeString(seqKeyHex) + if err != nil { + return fmt.Errorf("failed to decode SEQUENCER_PRIVATE_KEY: %w", err) + } + n.sequencerPrivKey, err = ethcrypto.ToECDSA(keyBytes) + if err != nil { + return fmt.Errorf("failed to parse SEQUENCER_PRIVATE_KEY: %w", err) + } + n.Logger.Info("Loaded sequencer private key", + "address", ethcrypto.PubkeyToAddress(n.sequencerPrivKey.PublicKey).Hex()) + } + return nil +} + +// startSequencerMode starts the sequencer mode components with logging. +// This is an internal method used by upgrade callbacks and startup logic. +func (n *Node) startSequencerMode() { + if n.blockBroadcastReactor == nil { + panic("blockBroadcastReactor is nil, cannot switch to sequencer mode") + } + n.Logger.Info("Starting broadcast reactor...") + if err := n.blockBroadcastReactor.StartSequencerRoutines(); err != nil { + panic(fmt.Errorf("failed to start sequencer routines: %w", err)) + } + n.Logger.Info("Broadcast reactor started successfully") +} + +func (n *Node) switchToSequencerMode() { + n.Logger.Info("Upgrade callback triggered, scheduling switch to sequencer mode") + + // NOTE: Must use goroutine to avoid deadlock - onUpgrade is called from finalizeCommit, + // and consensusReactor.Stop() would try to stop the state that's currently running. + go func() { + // Wait a moment for finalizeCommit to complete + time.Sleep(100 * time.Millisecond) + + // Stop consensus reactor + n.Logger.Info("Stopping consensus reactor...") + if err := n.consensusReactor.Stop(); err != nil { + n.Logger.Error("Failed to stop consensus reactor", "err", err) + } + n.Logger.Info("Consensus reactor stopped") + + // Start broadcast reactor + n.startSequencerMode() + }() +} diff --git a/proto/tendermint/blocksync/types.pb.go b/proto/tendermint/blocksync/types.pb.go index 17d76a278c6..8bf5474637c 100644 --- a/proto/tendermint/blocksync/types.pb.go +++ b/proto/tendermint/blocksync/types.pb.go @@ -6,6 +6,7 @@ package blocksync import ( fmt "fmt" proto "github.com/cosmos/gogoproto/proto" + sequencer "github.com/tendermint/tendermint/proto/tendermint/sequencer" types "github.com/tendermint/tendermint/proto/tendermint/types" io "io" math "math" @@ -113,7 +114,7 @@ func (m *NoBlockResponse) GetHeight() int64 { return 0 } -// BlockResponse returns block to the requested +// BlockResponse returns block to the requested (V1 format, for backward compatibility) type BlockResponse struct { Block *types.Block `protobuf:"bytes,1,opt,name=block,proto3" json:"block,omitempty"` } @@ -158,6 +159,51 @@ func (m *BlockResponse) GetBlock() *types.Block { return nil } +// BlockResponseV2 returns block in V2 format (for sequencer mode) +type BlockResponseV2 struct { + Block *sequencer.BlockV2 `protobuf:"bytes,1,opt,name=block,proto3" json:"block,omitempty"` +} + +func (m *BlockResponseV2) Reset() { *m = BlockResponseV2{} } +func (m *BlockResponseV2) String() string { return proto.CompactTextString(m) } +func (*BlockResponseV2) ProtoMessage() {} +func (*BlockResponseV2) Descriptor() ([]byte, []int) { + return fileDescriptor_19b397c236e0fa07, []int{3} +} +func (m *BlockResponseV2) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BlockResponseV2) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_BlockResponseV2.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *BlockResponseV2) XXX_Merge(src proto.Message) { + xxx_messageInfo_BlockResponseV2.Merge(m, src) +} +func (m *BlockResponseV2) XXX_Size() int { + return m.Size() +} +func (m *BlockResponseV2) XXX_DiscardUnknown() { + xxx_messageInfo_BlockResponseV2.DiscardUnknown(m) +} + +var xxx_messageInfo_BlockResponseV2 proto.InternalMessageInfo + +func (m *BlockResponseV2) GetBlock() *sequencer.BlockV2 { + if m != nil { + return m.Block + } + return nil +} + // StatusRequest requests the status of a peer. type StatusRequest struct { } @@ -166,7 +212,7 @@ func (m *StatusRequest) Reset() { *m = StatusRequest{} } func (m *StatusRequest) String() string { return proto.CompactTextString(m) } func (*StatusRequest) ProtoMessage() {} func (*StatusRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_19b397c236e0fa07, []int{3} + return fileDescriptor_19b397c236e0fa07, []int{4} } func (m *StatusRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -205,7 +251,7 @@ func (m *StatusResponse) Reset() { *m = StatusResponse{} } func (m *StatusResponse) String() string { return proto.CompactTextString(m) } func (*StatusResponse) ProtoMessage() {} func (*StatusResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_19b397c236e0fa07, []int{4} + return fileDescriptor_19b397c236e0fa07, []int{5} } func (m *StatusResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -255,6 +301,7 @@ type Message struct { // *Message_BlockResponse // *Message_StatusRequest // *Message_StatusResponse + // *Message_BlockResponseV2 Sum isMessage_Sum `protobuf_oneof:"sum"` } @@ -262,7 +309,7 @@ func (m *Message) Reset() { *m = Message{} } func (m *Message) String() string { return proto.CompactTextString(m) } func (*Message) ProtoMessage() {} func (*Message) Descriptor() ([]byte, []int) { - return fileDescriptor_19b397c236e0fa07, []int{5} + return fileDescriptor_19b397c236e0fa07, []int{6} } func (m *Message) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -312,12 +359,16 @@ type Message_StatusRequest struct { type Message_StatusResponse struct { StatusResponse *StatusResponse `protobuf:"bytes,5,opt,name=status_response,json=statusResponse,proto3,oneof" json:"status_response,omitempty"` } +type Message_BlockResponseV2 struct { + BlockResponseV2 *BlockResponseV2 `protobuf:"bytes,6,opt,name=block_response_v2,json=blockResponseV2,proto3,oneof" json:"block_response_v2,omitempty"` +} func (*Message_BlockRequest) isMessage_Sum() {} func (*Message_NoBlockResponse) isMessage_Sum() {} func (*Message_BlockResponse) isMessage_Sum() {} func (*Message_StatusRequest) isMessage_Sum() {} func (*Message_StatusResponse) isMessage_Sum() {} +func (*Message_BlockResponseV2) isMessage_Sum() {} func (m *Message) GetSum() isMessage_Sum { if m != nil { @@ -361,6 +412,13 @@ func (m *Message) GetStatusResponse() *StatusResponse { return nil } +func (m *Message) GetBlockResponseV2() *BlockResponseV2 { + if x, ok := m.GetSum().(*Message_BlockResponseV2); ok { + return x.BlockResponseV2 + } + return nil +} + // XXX_OneofWrappers is for the internal use of the proto package. func (*Message) XXX_OneofWrappers() []interface{} { return []interface{}{ @@ -369,6 +427,7 @@ func (*Message) XXX_OneofWrappers() []interface{} { (*Message_BlockResponse)(nil), (*Message_StatusRequest)(nil), (*Message_StatusResponse)(nil), + (*Message_BlockResponseV2)(nil), } } @@ -376,6 +435,7 @@ func init() { proto.RegisterType((*BlockRequest)(nil), "tendermint.blocksync.BlockRequest") proto.RegisterType((*NoBlockResponse)(nil), "tendermint.blocksync.NoBlockResponse") proto.RegisterType((*BlockResponse)(nil), "tendermint.blocksync.BlockResponse") + proto.RegisterType((*BlockResponseV2)(nil), "tendermint.blocksync.BlockResponseV2") proto.RegisterType((*StatusRequest)(nil), "tendermint.blocksync.StatusRequest") proto.RegisterType((*StatusResponse)(nil), "tendermint.blocksync.StatusResponse") proto.RegisterType((*Message)(nil), "tendermint.blocksync.Message") @@ -384,30 +444,34 @@ func init() { func init() { proto.RegisterFile("tendermint/blocksync/types.proto", fileDescriptor_19b397c236e0fa07) } var fileDescriptor_19b397c236e0fa07 = []byte{ - // 368 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x93, 0x4d, 0x4f, 0xfa, 0x40, - 0x10, 0xc6, 0xdb, 0x7f, 0x81, 0x7f, 0x32, 0x50, 0x1a, 0x1b, 0xa3, 0xc4, 0x98, 0x86, 0xd4, 0x97, - 0xe8, 0xc1, 0x36, 0xc1, 0xa3, 0xc6, 0x03, 0x27, 0x4c, 0x7c, 0x49, 0x4a, 0xbc, 0x78, 0x21, 0x14, - 0x37, 0x40, 0x94, 0x2e, 0x32, 0xdb, 0x03, 0xdf, 0xc2, 0x2f, 0xe0, 0xf7, 0xf1, 0xc8, 0xd1, 0xa3, - 0x81, 0x2f, 0x62, 0x98, 0x2d, 0x65, 0x69, 0xb0, 0xb7, 0xdd, 0xe9, 0x33, 0xbf, 0x79, 0xfa, 0x64, - 0x16, 0xea, 0x82, 0x45, 0x2f, 0x6c, 0x32, 0x1a, 0x46, 0xc2, 0x0f, 0xdf, 0x78, 0xef, 0x15, 0xa7, - 0x51, 0xcf, 0x17, 0xd3, 0x31, 0x43, 0x6f, 0x3c, 0xe1, 0x82, 0xdb, 0xbb, 0x6b, 0x85, 0x97, 0x2a, - 0x0e, 0x0e, 0x95, 0x3e, 0x52, 0xcb, 0x6e, 0xd9, 0xe3, 0x9e, 0x42, 0xa5, 0xb9, 0xbc, 0x06, 0xec, - 0x3d, 0x66, 0x28, 0xec, 0x3d, 0x28, 0x0d, 0xd8, 0xb0, 0x3f, 0x10, 0x35, 0xbd, 0xae, 0x9f, 0x19, - 0x41, 0x72, 0x73, 0xcf, 0xc1, 0x7a, 0xe0, 0x89, 0x12, 0xc7, 0x3c, 0x42, 0xf6, 0xa7, 0xf4, 0x06, - 0xcc, 0x4d, 0xe1, 0x05, 0x14, 0x69, 0x24, 0xe9, 0xca, 0x8d, 0x7d, 0x4f, 0xf1, 0x29, 0xfd, 0x4b, - 0xbd, 0x54, 0xb9, 0x16, 0x98, 0x6d, 0xd1, 0x15, 0x31, 0x26, 0x9e, 0xdc, 0x6b, 0xa8, 0xae, 0x0a, - 0xf9, 0xa3, 0x6d, 0x1b, 0x0a, 0x61, 0x17, 0x59, 0xed, 0x1f, 0x55, 0xe9, 0xec, 0x7e, 0x1a, 0xf0, - 0xff, 0x9e, 0x21, 0x76, 0xfb, 0xcc, 0xbe, 0x05, 0x93, 0x66, 0x74, 0x26, 0x12, 0x9d, 0x38, 0x72, - 0xbd, 0x6d, 0xc9, 0x79, 0x6a, 0x30, 0x2d, 0x2d, 0xa8, 0x84, 0x6a, 0x50, 0x6d, 0xd8, 0x89, 0x78, - 0x67, 0x45, 0x93, 0xbe, 0x68, 0x6e, 0xb9, 0x71, 0xb2, 0x1d, 0x97, 0xc9, 0xaf, 0xa5, 0x05, 0x56, - 0x94, 0x89, 0xf4, 0x0e, 0xaa, 0x19, 0xa2, 0x41, 0xc4, 0xa3, 0x5c, 0x83, 0x29, 0xcf, 0x0c, 0xb3, - 0x34, 0xa4, 0xdc, 0xd2, 0xdf, 0x2d, 0xe4, 0xd1, 0x36, 0x42, 0x5f, 0xd2, 0x50, 0x2d, 0xd8, 0x8f, - 0x60, 0xa5, 0xb4, 0xc4, 0x5c, 0x91, 0x70, 0xc7, 0xf9, 0xb8, 0xd4, 0x5d, 0x15, 0x37, 0x2a, 0xcd, - 0x22, 0x18, 0x18, 0x8f, 0x9a, 0x4f, 0x5f, 0x73, 0x47, 0x9f, 0xcd, 0x1d, 0xfd, 0x67, 0xee, 0xe8, - 0x1f, 0x0b, 0x47, 0x9b, 0x2d, 0x1c, 0xed, 0x7b, 0xe1, 0x68, 0xcf, 0x57, 0xfd, 0xa1, 0x18, 0xc4, - 0xa1, 0xd7, 0xe3, 0x23, 0x5f, 0x5d, 0xe2, 0xf5, 0x91, 0x76, 0xd8, 0xdf, 0xf6, 0x30, 0xc2, 0x12, - 0x7d, 0xbb, 0xfc, 0x0d, 0x00, 0x00, 0xff, 0xff, 0xf5, 0x1c, 0xa3, 0x45, 0x37, 0x03, 0x00, 0x00, + // 421 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x93, 0xcd, 0x6a, 0xea, 0x40, + 0x14, 0xc7, 0x93, 0x1b, 0xf5, 0xc2, 0xd1, 0x18, 0x0c, 0x97, 0x7b, 0x45, 0x6e, 0x83, 0xa4, 0x1f, + 0xb4, 0x8b, 0x26, 0x10, 0x97, 0x2d, 0x5d, 0xb8, 0x28, 0x16, 0xfa, 0x01, 0x91, 0xba, 0xe8, 0x46, + 0x4c, 0x3a, 0xa8, 0xb4, 0x26, 0x36, 0x33, 0x29, 0xf8, 0x16, 0x7d, 0xac, 0x2e, 0x5d, 0x76, 0x59, + 0xf4, 0x0d, 0xfa, 0x04, 0x25, 0x33, 0x31, 0xce, 0x04, 0x9b, 0xdd, 0xe4, 0xcc, 0xff, 0xfc, 0xe6, + 0x7f, 0xce, 0xc9, 0x81, 0x36, 0x41, 0xc1, 0x23, 0x8a, 0x66, 0xd3, 0x80, 0xd8, 0xde, 0x73, 0xe8, + 0x3f, 0xe1, 0x45, 0xe0, 0xdb, 0x64, 0x31, 0x47, 0xd8, 0x9a, 0x47, 0x21, 0x09, 0xf5, 0x3f, 0x5b, + 0x85, 0x95, 0x29, 0x5a, 0xff, 0xb9, 0x3c, 0xaa, 0x66, 0xd9, 0x2c, 0xa7, 0xc5, 0x53, 0x31, 0x7a, + 0x89, 0x51, 0xe0, 0xa3, 0x88, 0xa7, 0x9a, 0x47, 0x50, 0xeb, 0x26, 0x09, 0x6e, 0x72, 0x8b, 0x89, + 0xfe, 0x17, 0x2a, 0x13, 0x34, 0x1d, 0x4f, 0x48, 0x53, 0x6e, 0xcb, 0xc7, 0x8a, 0x9b, 0x7e, 0x99, + 0x27, 0xa0, 0xdd, 0x86, 0xa9, 0x12, 0xcf, 0xc3, 0x00, 0xa3, 0x1f, 0xa5, 0x17, 0xa0, 0x8a, 0xc2, + 0x53, 0x28, 0x53, 0x53, 0x54, 0x57, 0x75, 0xfe, 0x59, 0x5c, 0x25, 0xcc, 0x0b, 0xd3, 0x33, 0x95, + 0x79, 0x09, 0x9a, 0x90, 0x3f, 0x70, 0xf4, 0x8e, 0x48, 0xd8, 0xe3, 0x09, 0x59, 0x5d, 0x8c, 0x32, + 0x70, 0x36, 0x1c, 0x0d, 0xd4, 0x3e, 0x19, 0x91, 0x18, 0xa7, 0xb5, 0x99, 0xe7, 0x50, 0xdf, 0x04, + 0x8a, 0x4b, 0xd0, 0x75, 0x28, 0x79, 0x23, 0x8c, 0x9a, 0xbf, 0x68, 0x94, 0x9e, 0xcd, 0x2f, 0x05, + 0x7e, 0xdf, 0x20, 0x8c, 0x47, 0x63, 0xa4, 0x5f, 0x81, 0x4a, 0xdf, 0x18, 0x46, 0x0c, 0x9d, 0xfa, + 0x32, 0xad, 0x5d, 0x33, 0xb2, 0xf8, 0x06, 0xf7, 0x24, 0xb7, 0xe6, 0xf1, 0x0d, 0xef, 0x43, 0x23, + 0x08, 0x87, 0x1b, 0x1a, 0xf3, 0x45, 0xdf, 0xad, 0x3a, 0x87, 0xbb, 0x71, 0xb9, 0x39, 0xf4, 0x24, + 0x57, 0x0b, 0x72, 0xa3, 0xb9, 0x86, 0x7a, 0x8e, 0xa8, 0x50, 0xe2, 0x7e, 0xa1, 0xc1, 0x8c, 0xa7, + 0x7a, 0x79, 0x1a, 0xa6, 0x7d, 0xcb, 0xca, 0x2d, 0x15, 0xd1, 0x84, 0xa6, 0x27, 0x34, 0xcc, 0x07, + 0xf4, 0x3b, 0xd0, 0x32, 0x5a, 0x6a, 0xae, 0x4c, 0x71, 0x07, 0xc5, 0xb8, 0xcc, 0x5d, 0x1d, 0x8b, + 0x43, 0xec, 0x43, 0x43, 0x2c, 0x76, 0xf8, 0xea, 0x34, 0x2b, 0x45, 0x1d, 0xcc, 0xfd, 0x5e, 0x49, + 0x07, 0x3d, 0x31, 0xd4, 0x2d, 0x83, 0x82, 0xe3, 0x59, 0xf7, 0xfe, 0x7d, 0x65, 0xc8, 0xcb, 0x95, + 0x21, 0x7f, 0xae, 0x0c, 0xf9, 0x6d, 0x6d, 0x48, 0xcb, 0xb5, 0x21, 0x7d, 0xac, 0x0d, 0xe9, 0xe1, + 0x6c, 0x3c, 0x25, 0x93, 0xd8, 0xb3, 0xfc, 0x70, 0x66, 0xf3, 0x3b, 0xb8, 0x3d, 0xd2, 0x05, 0xb3, + 0x77, 0xed, 0xb5, 0x57, 0xa1, 0x77, 0x9d, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0xbd, 0x9b, 0xa4, + 0x83, 0xf6, 0x03, 0x00, 0x00, } func (m *BlockRequest) Marshal() (dAtA []byte, err error) { @@ -501,6 +565,41 @@ func (m *BlockResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *BlockResponseV2) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlockResponseV2) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BlockResponseV2) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Block != nil { + { + size, err := m.Block.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *StatusRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -694,6 +793,27 @@ func (m *Message_StatusResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) } return len(dAtA) - i, nil } +func (m *Message_BlockResponseV2) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Message_BlockResponseV2) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.BlockResponseV2 != nil { + { + size, err := m.BlockResponseV2.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + return len(dAtA) - i, nil +} func encodeVarintTypes(dAtA []byte, offset int, v uint64) int { offset -= sovTypes(v) base := offset @@ -742,6 +862,19 @@ func (m *BlockResponse) Size() (n int) { return n } +func (m *BlockResponseV2) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Block != nil { + l = m.Block.Size() + n += 1 + l + sovTypes(uint64(l)) + } + return n +} + func (m *StatusRequest) Size() (n int) { if m == nil { return 0 @@ -838,6 +971,18 @@ func (m *Message_StatusResponse) Size() (n int) { } return n } +func (m *Message_BlockResponseV2) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.BlockResponseV2 != nil { + l = m.BlockResponseV2.Size() + n += 1 + l + sovTypes(uint64(l)) + } + return n +} func sovTypes(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 @@ -1069,6 +1214,92 @@ func (m *BlockResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *BlockResponseV2) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlockResponseV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlockResponseV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Block", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Block == nil { + m.Block = &sequencer.BlockV2{} + } + if err := m.Block.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *StatusRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -1411,6 +1642,41 @@ func (m *Message) Unmarshal(dAtA []byte) error { } m.Sum = &Message_StatusResponse{v} iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BlockResponseV2", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &BlockResponseV2{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Sum = &Message_BlockResponseV2{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipTypes(dAtA[iNdEx:]) diff --git a/proto/tendermint/blocksync/types.proto b/proto/tendermint/blocksync/types.proto index 8c187c793eb..3d8fdea2db1 100644 --- a/proto/tendermint/blocksync/types.proto +++ b/proto/tendermint/blocksync/types.proto @@ -4,6 +4,7 @@ package tendermint.blocksync; option go_package = "github.com/tendermint/tendermint/proto/tendermint/blocksync"; import "tendermint/types/block.proto"; +import "tendermint/sequencer/types.proto"; // BlockRequest requests a block for a specific height message BlockRequest { @@ -15,11 +16,16 @@ message NoBlockResponse { int64 height = 1; } -// BlockResponse returns block to the requested +// BlockResponse returns block to the requested (V1 format, for backward compatibility) message BlockResponse { tendermint.types.Block block = 1; } +// BlockResponseV2 returns block in V2 format (for sequencer mode) +message BlockResponseV2 { + tendermint.sequencer.BlockV2 block = 1; +} + // StatusRequest requests the status of a peer. message StatusRequest { } @@ -32,10 +38,11 @@ message StatusResponse { message Message { oneof sum { - BlockRequest block_request = 1; - NoBlockResponse no_block_response = 2; - BlockResponse block_response = 3; - StatusRequest status_request = 4; - StatusResponse status_response = 5; + BlockRequest block_request = 1; + NoBlockResponse no_block_response = 2; + BlockResponse block_response = 3; + StatusRequest status_request = 4; + StatusResponse status_response = 5; + BlockResponseV2 block_response_v2 = 6; // New: V2 block response } } diff --git a/proto/tendermint/sequencer/types.pb.go b/proto/tendermint/sequencer/types.pb.go new file mode 100644 index 00000000000..764507f634e --- /dev/null +++ b/proto/tendermint/sequencer/types.pb.go @@ -0,0 +1,988 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: tendermint/sequencer/types.proto + +package sequencer + +import ( + fmt "fmt" + proto "github.com/cosmos/gogoproto/proto" + io "io" + math "math" + math_bits "math/bits" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +// BlockV2 is the block format after upgrade to centralized sequencer mode +type BlockV2 struct { + ParentHash []byte `protobuf:"bytes,1,opt,name=parent_hash,json=parentHash,proto3" json:"parent_hash,omitempty"` + Miner []byte `protobuf:"bytes,2,opt,name=miner,proto3" json:"miner,omitempty"` + Number uint64 `protobuf:"varint,3,opt,name=number,proto3" json:"number,omitempty"` + GasLimit uint64 `protobuf:"varint,4,opt,name=gas_limit,json=gasLimit,proto3" json:"gas_limit,omitempty"` + BaseFee []byte `protobuf:"bytes,5,opt,name=base_fee,json=baseFee,proto3" json:"base_fee,omitempty"` + Timestamp uint64 `protobuf:"varint,6,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + Transactions [][]byte `protobuf:"bytes,7,rep,name=transactions,proto3" json:"transactions,omitempty"` + StateRoot []byte `protobuf:"bytes,8,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty"` + GasUsed uint64 `protobuf:"varint,9,opt,name=gas_used,json=gasUsed,proto3" json:"gas_used,omitempty"` + ReceiptRoot []byte `protobuf:"bytes,10,opt,name=receipt_root,json=receiptRoot,proto3" json:"receipt_root,omitempty"` + LogsBloom []byte `protobuf:"bytes,11,opt,name=logs_bloom,json=logsBloom,proto3" json:"logs_bloom,omitempty"` + WithdrawTrieRoot []byte `protobuf:"bytes,12,opt,name=withdraw_trie_root,json=withdrawTrieRoot,proto3" json:"withdraw_trie_root,omitempty"` + NextL1MessageIndex uint64 `protobuf:"varint,13,opt,name=next_l1_message_index,json=nextL1MessageIndex,proto3" json:"next_l1_message_index,omitempty"` + Hash []byte `protobuf:"bytes,14,opt,name=hash,proto3" json:"hash,omitempty"` + Signature []byte `protobuf:"bytes,15,opt,name=signature,proto3" json:"signature,omitempty"` +} + +func (m *BlockV2) Reset() { *m = BlockV2{} } +func (m *BlockV2) String() string { return proto.CompactTextString(m) } +func (*BlockV2) ProtoMessage() {} +func (*BlockV2) Descriptor() ([]byte, []int) { + return fileDescriptor_25ce159da8b763c2, []int{0} +} +func (m *BlockV2) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BlockV2) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_BlockV2.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *BlockV2) XXX_Merge(src proto.Message) { + xxx_messageInfo_BlockV2.Merge(m, src) +} +func (m *BlockV2) XXX_Size() int { + return m.Size() +} +func (m *BlockV2) XXX_DiscardUnknown() { + xxx_messageInfo_BlockV2.DiscardUnknown(m) +} + +var xxx_messageInfo_BlockV2 proto.InternalMessageInfo + +func (m *BlockV2) GetParentHash() []byte { + if m != nil { + return m.ParentHash + } + return nil +} + +func (m *BlockV2) GetMiner() []byte { + if m != nil { + return m.Miner + } + return nil +} + +func (m *BlockV2) GetNumber() uint64 { + if m != nil { + return m.Number + } + return 0 +} + +func (m *BlockV2) GetGasLimit() uint64 { + if m != nil { + return m.GasLimit + } + return 0 +} + +func (m *BlockV2) GetBaseFee() []byte { + if m != nil { + return m.BaseFee + } + return nil +} + +func (m *BlockV2) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *BlockV2) GetTransactions() [][]byte { + if m != nil { + return m.Transactions + } + return nil +} + +func (m *BlockV2) GetStateRoot() []byte { + if m != nil { + return m.StateRoot + } + return nil +} + +func (m *BlockV2) GetGasUsed() uint64 { + if m != nil { + return m.GasUsed + } + return 0 +} + +func (m *BlockV2) GetReceiptRoot() []byte { + if m != nil { + return m.ReceiptRoot + } + return nil +} + +func (m *BlockV2) GetLogsBloom() []byte { + if m != nil { + return m.LogsBloom + } + return nil +} + +func (m *BlockV2) GetWithdrawTrieRoot() []byte { + if m != nil { + return m.WithdrawTrieRoot + } + return nil +} + +func (m *BlockV2) GetNextL1MessageIndex() uint64 { + if m != nil { + return m.NextL1MessageIndex + } + return 0 +} + +func (m *BlockV2) GetHash() []byte { + if m != nil { + return m.Hash + } + return nil +} + +func (m *BlockV2) GetSignature() []byte { + if m != nil { + return m.Signature + } + return nil +} + +func init() { + proto.RegisterType((*BlockV2)(nil), "tendermint.sequencer.BlockV2") +} + +func init() { proto.RegisterFile("tendermint/sequencer/types.proto", fileDescriptor_25ce159da8b763c2) } + +var fileDescriptor_25ce159da8b763c2 = []byte{ + // 426 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x92, 0xbf, 0x8e, 0xd3, 0x40, + 0x10, 0xc6, 0x63, 0x92, 0xcb, 0x9f, 0x49, 0xf8, 0xa3, 0xd5, 0x81, 0x16, 0xc1, 0x99, 0x70, 0x55, + 0x0a, 0x94, 0x28, 0x50, 0xd2, 0xa5, 0x40, 0x20, 0x1d, 0x4d, 0xc4, 0x51, 0xd0, 0x58, 0x6b, 0x7b, + 0xb0, 0x57, 0x78, 0x77, 0xcd, 0xce, 0x58, 0x77, 0xbc, 0x05, 0x8f, 0x45, 0x79, 0x25, 0x25, 0x24, + 0x2f, 0x82, 0xbc, 0xce, 0x71, 0x87, 0x44, 0x37, 0xf3, 0xfb, 0x66, 0x3e, 0x7f, 0xb6, 0x07, 0xe6, + 0x8c, 0x36, 0x47, 0x6f, 0xb4, 0xe5, 0x15, 0xe1, 0xd7, 0x06, 0x6d, 0x86, 0x7e, 0xc5, 0xdf, 0x6a, + 0xa4, 0x65, 0xed, 0x1d, 0x3b, 0x71, 0x7c, 0x33, 0xb1, 0xfc, 0x3b, 0x71, 0xfa, 0xbb, 0x0f, 0xa3, + 0x4d, 0xe5, 0xb2, 0x2f, 0x1f, 0x5f, 0x8a, 0x67, 0x30, 0xad, 0x95, 0x47, 0xcb, 0x49, 0xa9, 0xa8, + 0x94, 0xd1, 0x3c, 0x5a, 0xcc, 0xb6, 0xd0, 0xa1, 0xb7, 0x8a, 0x4a, 0x71, 0x0c, 0x47, 0x46, 0x5b, + 0xf4, 0xf2, 0x4e, 0x90, 0xba, 0x46, 0x3c, 0x82, 0xa1, 0x6d, 0x4c, 0x8a, 0x5e, 0xf6, 0xe7, 0xd1, + 0x62, 0xb0, 0x3d, 0x74, 0xe2, 0x09, 0x4c, 0x0a, 0x45, 0x49, 0xa5, 0x8d, 0x66, 0x39, 0x08, 0xd2, + 0xb8, 0x50, 0x74, 0xd6, 0xf6, 0xe2, 0x31, 0x8c, 0x53, 0x45, 0x98, 0x7c, 0x46, 0x94, 0x47, 0xc1, + 0x6d, 0xd4, 0xf6, 0x6f, 0x10, 0xc5, 0x53, 0x98, 0xb0, 0x36, 0x48, 0xac, 0x4c, 0x2d, 0x87, 0x61, + 0xef, 0x06, 0x88, 0x53, 0x98, 0xb1, 0x57, 0x96, 0x54, 0xc6, 0xda, 0x59, 0x92, 0xa3, 0x79, 0x7f, + 0x31, 0xdb, 0xfe, 0xc3, 0xc4, 0x09, 0x00, 0xb1, 0x62, 0x4c, 0xbc, 0x73, 0x2c, 0xc7, 0xc1, 0x7e, + 0x12, 0xc8, 0xd6, 0xb9, 0xf0, 0xec, 0x36, 0x58, 0x43, 0x98, 0xcb, 0x49, 0xf0, 0x1f, 0x15, 0x8a, + 0xce, 0x09, 0x73, 0xf1, 0x1c, 0x66, 0x1e, 0x33, 0xd4, 0x35, 0x77, 0xbb, 0x10, 0x76, 0xa7, 0x07, + 0x16, 0xb6, 0x4f, 0x00, 0x2a, 0x57, 0x50, 0x92, 0x56, 0xce, 0x19, 0x39, 0xed, 0xcc, 0x5b, 0xb2, + 0x69, 0x81, 0x78, 0x01, 0xe2, 0x42, 0x73, 0x99, 0x7b, 0x75, 0x91, 0xb0, 0xd7, 0x87, 0x0c, 0xb3, + 0x30, 0xf6, 0xe0, 0x5a, 0xf9, 0xe0, 0x75, 0x17, 0x65, 0x0d, 0x0f, 0x2d, 0x5e, 0x72, 0x52, 0xad, + 0x13, 0x83, 0x44, 0xaa, 0xc0, 0x44, 0xdb, 0x1c, 0x2f, 0xe5, 0xdd, 0x90, 0x4b, 0xb4, 0xe2, 0xd9, + 0xfa, 0x7d, 0x27, 0xbd, 0x6b, 0x15, 0x21, 0x60, 0x10, 0x7e, 0xcf, 0xbd, 0x60, 0x19, 0xea, 0xf6, + 0x93, 0x91, 0x2e, 0xac, 0xe2, 0xc6, 0xa3, 0xbc, 0x7f, 0x78, 0xdf, 0x6b, 0xb0, 0x39, 0xff, 0xb1, + 0x8b, 0xa3, 0xab, 0x5d, 0x1c, 0xfd, 0xda, 0xc5, 0xd1, 0xf7, 0x7d, 0xdc, 0xbb, 0xda, 0xc7, 0xbd, + 0x9f, 0xfb, 0xb8, 0xf7, 0xe9, 0x75, 0xa1, 0xb9, 0x6c, 0xd2, 0x65, 0xe6, 0xcc, 0xea, 0xd6, 0x01, + 0xdd, 0x2a, 0xc3, 0xed, 0xac, 0xfe, 0x77, 0x5c, 0xe9, 0x30, 0x68, 0xaf, 0xfe, 0x04, 0x00, 0x00, + 0xff, 0xff, 0xaa, 0x31, 0xfa, 0x7e, 0x7b, 0x02, 0x00, 0x00, +} + +func (m *BlockV2) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BlockV2) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BlockV2) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Signature) > 0 { + i -= len(m.Signature) + copy(dAtA[i:], m.Signature) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Signature))) + i-- + dAtA[i] = 0x7a + } + if len(m.Hash) > 0 { + i -= len(m.Hash) + copy(dAtA[i:], m.Hash) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Hash))) + i-- + dAtA[i] = 0x72 + } + if m.NextL1MessageIndex != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.NextL1MessageIndex)) + i-- + dAtA[i] = 0x68 + } + if len(m.WithdrawTrieRoot) > 0 { + i -= len(m.WithdrawTrieRoot) + copy(dAtA[i:], m.WithdrawTrieRoot) + i = encodeVarintTypes(dAtA, i, uint64(len(m.WithdrawTrieRoot))) + i-- + dAtA[i] = 0x62 + } + if len(m.LogsBloom) > 0 { + i -= len(m.LogsBloom) + copy(dAtA[i:], m.LogsBloom) + i = encodeVarintTypes(dAtA, i, uint64(len(m.LogsBloom))) + i-- + dAtA[i] = 0x5a + } + if len(m.ReceiptRoot) > 0 { + i -= len(m.ReceiptRoot) + copy(dAtA[i:], m.ReceiptRoot) + i = encodeVarintTypes(dAtA, i, uint64(len(m.ReceiptRoot))) + i-- + dAtA[i] = 0x52 + } + if m.GasUsed != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.GasUsed)) + i-- + dAtA[i] = 0x48 + } + if len(m.StateRoot) > 0 { + i -= len(m.StateRoot) + copy(dAtA[i:], m.StateRoot) + i = encodeVarintTypes(dAtA, i, uint64(len(m.StateRoot))) + i-- + dAtA[i] = 0x42 + } + if len(m.Transactions) > 0 { + for iNdEx := len(m.Transactions) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Transactions[iNdEx]) + copy(dAtA[i:], m.Transactions[iNdEx]) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Transactions[iNdEx]))) + i-- + dAtA[i] = 0x3a + } + } + if m.Timestamp != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x30 + } + if len(m.BaseFee) > 0 { + i -= len(m.BaseFee) + copy(dAtA[i:], m.BaseFee) + i = encodeVarintTypes(dAtA, i, uint64(len(m.BaseFee))) + i-- + dAtA[i] = 0x2a + } + if m.GasLimit != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.GasLimit)) + i-- + dAtA[i] = 0x20 + } + if m.Number != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Number)) + i-- + dAtA[i] = 0x18 + } + if len(m.Miner) > 0 { + i -= len(m.Miner) + copy(dAtA[i:], m.Miner) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Miner))) + i-- + dAtA[i] = 0x12 + } + if len(m.ParentHash) > 0 { + i -= len(m.ParentHash) + copy(dAtA[i:], m.ParentHash) + i = encodeVarintTypes(dAtA, i, uint64(len(m.ParentHash))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func encodeVarintTypes(dAtA []byte, offset int, v uint64) int { + offset -= sovTypes(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *BlockV2) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.ParentHash) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + l = len(m.Miner) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + if m.Number != 0 { + n += 1 + sovTypes(uint64(m.Number)) + } + if m.GasLimit != 0 { + n += 1 + sovTypes(uint64(m.GasLimit)) + } + l = len(m.BaseFee) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + if m.Timestamp != 0 { + n += 1 + sovTypes(uint64(m.Timestamp)) + } + if len(m.Transactions) > 0 { + for _, b := range m.Transactions { + l = len(b) + n += 1 + l + sovTypes(uint64(l)) + } + } + l = len(m.StateRoot) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + if m.GasUsed != 0 { + n += 1 + sovTypes(uint64(m.GasUsed)) + } + l = len(m.ReceiptRoot) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + l = len(m.LogsBloom) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + l = len(m.WithdrawTrieRoot) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + if m.NextL1MessageIndex != 0 { + n += 1 + sovTypes(uint64(m.NextL1MessageIndex)) + } + l = len(m.Hash) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + l = len(m.Signature) + if l > 0 { + n += 1 + l + sovTypes(uint64(l)) + } + return n +} + +func sovTypes(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozTypes(x uint64) (n int) { + return sovTypes(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *BlockV2) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BlockV2: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BlockV2: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ParentHash", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ParentHash = append(m.ParentHash[:0], dAtA[iNdEx:postIndex]...) + if m.ParentHash == nil { + m.ParentHash = []byte{} + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Miner", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Miner = append(m.Miner[:0], dAtA[iNdEx:postIndex]...) + if m.Miner == nil { + m.Miner = []byte{} + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Number", wireType) + } + m.Number = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Number |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GasLimit", wireType) + } + m.GasLimit = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GasLimit |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field BaseFee", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.BaseFee = append(m.BaseFee[:0], dAtA[iNdEx:postIndex]...) + if m.BaseFee == nil { + m.BaseFee = []byte{} + } + iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + m.Timestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Timestamp |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transactions", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Transactions = append(m.Transactions, make([]byte, postIndex-iNdEx)) + copy(m.Transactions[len(m.Transactions)-1], dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StateRoot", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StateRoot = append(m.StateRoot[:0], dAtA[iNdEx:postIndex]...) + if m.StateRoot == nil { + m.StateRoot = []byte{} + } + iNdEx = postIndex + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field GasUsed", wireType) + } + m.GasUsed = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.GasUsed |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ReceiptRoot", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ReceiptRoot = append(m.ReceiptRoot[:0], dAtA[iNdEx:postIndex]...) + if m.ReceiptRoot == nil { + m.ReceiptRoot = []byte{} + } + iNdEx = postIndex + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LogsBloom", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.LogsBloom = append(m.LogsBloom[:0], dAtA[iNdEx:postIndex]...) + if m.LogsBloom == nil { + m.LogsBloom = []byte{} + } + iNdEx = postIndex + case 12: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field WithdrawTrieRoot", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.WithdrawTrieRoot = append(m.WithdrawTrieRoot[:0], dAtA[iNdEx:postIndex]...) + if m.WithdrawTrieRoot == nil { + m.WithdrawTrieRoot = []byte{} + } + iNdEx = postIndex + case 13: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NextL1MessageIndex", wireType) + } + m.NextL1MessageIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NextL1MessageIndex |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Hash", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Hash = append(m.Hash[:0], dAtA[iNdEx:postIndex]...) + if m.Hash == nil { + m.Hash = []byte{} + } + iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Signature", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Signature = append(m.Signature[:0], dAtA[iNdEx:postIndex]...) + if m.Signature == nil { + m.Signature = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipTypes(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthTypes + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupTypes + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthTypes + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthTypes = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowTypes = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupTypes = fmt.Errorf("proto: unexpected end of group") +) diff --git a/proto/tendermint/sequencer/types.proto b/proto/tendermint/sequencer/types.proto new file mode 100644 index 00000000000..f988e78cba7 --- /dev/null +++ b/proto/tendermint/sequencer/types.proto @@ -0,0 +1,24 @@ +syntax = "proto3"; +package tendermint.sequencer; + +option go_package = "github.com/tendermint/tendermint/proto/tendermint/sequencer"; + +// BlockV2 is the block format after upgrade to centralized sequencer mode +message BlockV2 { + bytes parent_hash = 1; + bytes miner = 2; + uint64 number = 3; + uint64 gas_limit = 4; + bytes base_fee = 5; + uint64 timestamp = 6; + repeated bytes transactions = 7; + bytes state_root = 8; + uint64 gas_used = 9; + bytes receipt_root = 10; + bytes logs_bloom = 11; + bytes withdraw_trie_root = 12; + uint64 next_l1_message_index = 13; + bytes hash = 14; + bytes signature = 15; +} + diff --git a/sequencer/block_cache.go b/sequencer/block_cache.go new file mode 100644 index 00000000000..ff71a1759ba --- /dev/null +++ b/sequencer/block_cache.go @@ -0,0 +1,195 @@ +package sequencer + +import ( + "sync" + + "github.com/morph-l2/go-ethereum/common" +) + +// BlockRingBuffer stores recent applied blocks for serving peer requests. +// Maintains a contiguous range of blocks [minHeight, maxHeight]. +// Supports reorg: when replacing a block, all higher blocks are removed. +type BlockRingBuffer struct { + blocks []*BlockV2 + capacity int + + // Height tracking + minHeight uint64 + maxHeight uint64 + + // Fast lookup (O(1)) + byHeight map[uint64]*BlockV2 + byHash map[common.Hash]*BlockV2 + position map[uint64]int // height -> index in blocks array + + mtx sync.RWMutex +} + +// NewBlockRingBuffer creates a new ring buffer. +func NewBlockRingBuffer(capacity int) *BlockRingBuffer { + return &BlockRingBuffer{ + blocks: make([]*BlockV2, capacity), + capacity: capacity, + byHeight: make(map[uint64]*BlockV2), + byHash: make(map[common.Hash]*BlockV2), + position: make(map[uint64]int), + } +} + +// Add adds a block to the cache. +// - If first block: initializes cache +// - If height == maxHeight+1: appends (normal case) +// - If height exists with different hash: reorg, rollback higher blocks and replace +// - If height exists with same hash: no-op +// Returns false if block cannot be added (non-contiguous). +func (rb *BlockRingBuffer) Add(block *BlockV2) bool { + rb.mtx.Lock() + defer rb.mtx.Unlock() + + height := block.Number + + // Case 1: First block + if len(rb.byHeight) == 0 { + return rb.addAt(block, 0) + } + + // Case 2: Normal append + if height == rb.maxHeight+1 { + return rb.append(block) + } + + // Case 3: Same height exists + if old, exists := rb.byHeight[height]; exists { + if old.Hash == block.Hash { + return true // Same block, no-op + } + // Different hash = reorg, rollback and replace + rb.rollbackTo(height - 1) + return rb.append(block) + } + + // Case 4: Height within range but somehow missing (shouldn't happen) + if height >= rb.minHeight && height <= rb.maxHeight { + return false + } + + // Case 5: Non-contiguous, reject + return false +} + +// addAt adds block at specific position (internal). +func (rb *BlockRingBuffer) addAt(block *BlockV2, pos int) bool { + rb.blocks[pos] = block + rb.byHeight[block.Number] = block + rb.byHash[block.Hash] = block + rb.position[block.Number] = pos + rb.minHeight = block.Number + rb.maxHeight = block.Number + return true +} + +// append adds block at maxHeight+1 (internal). +func (rb *BlockRingBuffer) append(block *BlockV2) bool { + count := len(rb.byHeight) + + // Calculate position + var pos int + if count == 0 { + pos = 0 + } else { + pos = (rb.position[rb.maxHeight] + 1) % rb.capacity + } + + // Evict oldest if full + if count >= rb.capacity { + rb.evictOldest() + } + + // Add + rb.blocks[pos] = block + rb.byHeight[block.Number] = block + rb.byHash[block.Hash] = block + rb.position[block.Number] = pos + rb.maxHeight = block.Number + + if count == 0 { + rb.minHeight = block.Number + } + + return true +} + +// evictOldest removes the oldest block (internal). +func (rb *BlockRingBuffer) evictOldest() { + if len(rb.byHeight) == 0 { + return + } + old := rb.byHeight[rb.minHeight] + if old != nil { + pos := rb.position[rb.minHeight] + rb.blocks[pos] = nil // Help GC + delete(rb.byHeight, rb.minHeight) + delete(rb.byHash, old.Hash) + delete(rb.position, rb.minHeight) + rb.minHeight++ + } +} + +// rollbackTo removes all blocks with height > targetHeight. +// Used during reorg. +func (rb *BlockRingBuffer) rollbackTo(targetHeight uint64) { + for h := rb.maxHeight; h > targetHeight; h-- { + block := rb.byHeight[h] + if block != nil { + pos := rb.position[h] + rb.blocks[pos] = nil // Help GC + delete(rb.byHeight, h) + delete(rb.byHash, block.Hash) + delete(rb.position, h) + } + } + rb.maxHeight = targetHeight +} + +// GetByHeight returns a block by height. +func (rb *BlockRingBuffer) GetByHeight(height uint64) *BlockV2 { + rb.mtx.RLock() + defer rb.mtx.RUnlock() + return rb.byHeight[height] +} + +// GetByHash returns a block by hash. +func (rb *BlockRingBuffer) GetByHash(hash common.Hash) *BlockV2 { + rb.mtx.RLock() + defer rb.mtx.RUnlock() + return rb.byHash[hash] +} + +// LatestHeight returns the highest block height. +func (rb *BlockRingBuffer) LatestHeight() uint64 { + rb.mtx.RLock() + defer rb.mtx.RUnlock() + return rb.maxHeight +} + +// HeightRange returns the min and max height in buffer. +func (rb *BlockRingBuffer) HeightRange() (min, max uint64) { + rb.mtx.RLock() + defer rb.mtx.RUnlock() + return rb.minHeight, rb.maxHeight +} + +// Count returns the number of blocks. +func (rb *BlockRingBuffer) Count() int { + rb.mtx.RLock() + defer rb.mtx.RUnlock() + return len(rb.byHeight) +} + +// RollbackTo removes all blocks with height > targetHeight. +// Exposed for external use during explicit reorg. +func (rb *BlockRingBuffer) RollbackTo(targetHeight uint64) { + rb.mtx.Lock() + defer rb.mtx.Unlock() + rb.rollbackTo(targetHeight) +} diff --git a/sequencer/block_cache_test.go b/sequencer/block_cache_test.go new file mode 100644 index 00000000000..b8b63302ec5 --- /dev/null +++ b/sequencer/block_cache_test.go @@ -0,0 +1,393 @@ +package sequencer + +import ( + "testing" + + "github.com/morph-l2/go-ethereum/common" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func makeBlock(height uint64, hash byte) *BlockV2 { + return &BlockV2{ + Number: height, + Hash: common.Hash{hash}, + } +} + +func TestBlockRingBuffer_Empty(t *testing.T) { + rb := NewBlockRingBuffer(10) + + // Empty state + assert.Equal(t, 0, rb.Count()) + assert.Nil(t, rb.GetByHeight(100)) + assert.Nil(t, rb.GetByHash(common.Hash{0x01})) + + min, max := rb.HeightRange() + assert.Equal(t, uint64(0), min) + assert.Equal(t, uint64(0), max) + assert.Equal(t, uint64(0), rb.LatestHeight()) +} + +func TestBlockRingBuffer_FirstBlock(t *testing.T) { + rb := NewBlockRingBuffer(10) + + block := makeBlock(100, 0x01) + ok := rb.Add(block) + + assert.True(t, ok) + assert.Equal(t, 1, rb.Count()) + assert.Equal(t, uint64(100), rb.LatestHeight()) + + min, max := rb.HeightRange() + assert.Equal(t, uint64(100), min) + assert.Equal(t, uint64(100), max) + + // Lookup + assert.Equal(t, block, rb.GetByHeight(100)) + assert.Equal(t, block, rb.GetByHash(common.Hash{0x01})) +} + +func TestBlockRingBuffer_ContiguousAdd(t *testing.T) { + rb := NewBlockRingBuffer(10) + + // Add blocks 100-104 + for i := uint64(100); i <= 104; i++ { + block := makeBlock(i, byte(i)) + ok := rb.Add(block) + assert.True(t, ok, "failed to add block %d", i) + } + + assert.Equal(t, 5, rb.Count()) + assert.Equal(t, uint64(104), rb.LatestHeight()) + + min, max := rb.HeightRange() + assert.Equal(t, uint64(100), min) + assert.Equal(t, uint64(104), max) + + // All blocks should be retrievable + for i := uint64(100); i <= 104; i++ { + block := rb.GetByHeight(i) + require.NotNil(t, block) + assert.Equal(t, i, block.Number) + } +} + +func TestBlockRingBuffer_NonContiguousAdd_Rejected(t *testing.T) { + rb := NewBlockRingBuffer(10) + + // Add first block + rb.Add(makeBlock(100, 0x01)) + + // Try to add non-contiguous (skip 101) + ok := rb.Add(makeBlock(102, 0x02)) + assert.False(t, ok, "non-contiguous add should be rejected") + + // State unchanged + assert.Equal(t, 1, rb.Count()) + assert.Equal(t, uint64(100), rb.LatestHeight()) +} + +func TestBlockRingBuffer_CapacityEviction(t *testing.T) { + rb := NewBlockRingBuffer(5) + + // Add blocks 100-104 (fills capacity) + for i := uint64(100); i <= 104; i++ { + rb.Add(makeBlock(i, byte(i))) + } + assert.Equal(t, 5, rb.Count()) + + min, max := rb.HeightRange() + assert.Equal(t, uint64(100), min) + assert.Equal(t, uint64(104), max) + + // Add block 105, should evict 100 + rb.Add(makeBlock(105, 0x69)) + assert.Equal(t, 5, rb.Count()) + + min, max = rb.HeightRange() + assert.Equal(t, uint64(101), min) + assert.Equal(t, uint64(105), max) + + // Block 100 should be gone + assert.Nil(t, rb.GetByHeight(100)) + assert.Nil(t, rb.GetByHash(common.Hash{100})) + + // Block 101-105 should exist + for i := uint64(101); i <= 105; i++ { + assert.NotNil(t, rb.GetByHeight(i), "block %d should exist", i) + } +} + +func TestBlockRingBuffer_MultipleEvictions(t *testing.T) { + rb := NewBlockRingBuffer(3) + + // Add 100-102 + for i := uint64(100); i <= 102; i++ { + rb.Add(makeBlock(i, byte(i))) + } + + // Add 103-105, evicting 100-102 + for i := uint64(103); i <= 105; i++ { + rb.Add(makeBlock(i, byte(i))) + } + + assert.Equal(t, 3, rb.Count()) + + min, max := rb.HeightRange() + assert.Equal(t, uint64(103), min) + assert.Equal(t, uint64(105), max) + + // Old blocks gone + for i := uint64(100); i <= 102; i++ { + assert.Nil(t, rb.GetByHeight(i)) + } + + // New blocks exist + for i := uint64(103); i <= 105; i++ { + assert.NotNil(t, rb.GetByHeight(i)) + } +} + +func TestBlockRingBuffer_Reorg_SameHeightDifferentHash(t *testing.T) { + rb := NewBlockRingBuffer(10) + + // Add blocks 100-105 + for i := uint64(100); i <= 105; i++ { + rb.Add(makeBlock(i, byte(i))) + } + assert.Equal(t, 6, rb.Count()) + + // Reorg at height 103 (different hash) + newBlock103 := makeBlock(103, 0xFF) // Different hash + ok := rb.Add(newBlock103) + assert.True(t, ok) + + // Blocks 104, 105 should be removed + assert.Equal(t, 4, rb.Count()) // 100, 101, 102, 103 + + min, max := rb.HeightRange() + assert.Equal(t, uint64(100), min) + assert.Equal(t, uint64(103), max) + + // New block 103 should be there + block := rb.GetByHeight(103) + require.NotNil(t, block) + assert.Equal(t, common.Hash{0xFF}, block.Hash) + + // Old hash should not be found + assert.Nil(t, rb.GetByHash(common.Hash{103})) + // New hash should be found + assert.NotNil(t, rb.GetByHash(common.Hash{0xFF})) + + // Blocks 104, 105 should be gone + assert.Nil(t, rb.GetByHeight(104)) + assert.Nil(t, rb.GetByHeight(105)) +} + +func TestBlockRingBuffer_Reorg_SameBlock_NoOp(t *testing.T) { + rb := NewBlockRingBuffer(10) + + // Add blocks 100-103 + for i := uint64(100); i <= 103; i++ { + rb.Add(makeBlock(i, byte(i))) + } + + // Add same block again (same height, same hash) + ok := rb.Add(makeBlock(102, 102)) + assert.True(t, ok) + + // State unchanged + assert.Equal(t, 4, rb.Count()) + assert.Equal(t, uint64(103), rb.LatestHeight()) +} + +func TestBlockRingBuffer_RollbackTo(t *testing.T) { + rb := NewBlockRingBuffer(10) + + // Add blocks 100-109 + for i := uint64(100); i <= 109; i++ { + rb.Add(makeBlock(i, byte(i))) + } + assert.Equal(t, 10, rb.Count()) + + // Rollback to 105 + rb.RollbackTo(105) + + assert.Equal(t, 6, rb.Count()) // 100-105 + + min, max := rb.HeightRange() + assert.Equal(t, uint64(100), min) + assert.Equal(t, uint64(105), max) + + // Blocks 106-109 should be gone + for i := uint64(106); i <= 109; i++ { + assert.Nil(t, rb.GetByHeight(i)) + } + + // Blocks 100-105 should exist + for i := uint64(100); i <= 105; i++ { + assert.NotNil(t, rb.GetByHeight(i)) + } +} + +func TestBlockRingBuffer_RollbackTo_All(t *testing.T) { + rb := NewBlockRingBuffer(10) + + // Add blocks 100-104 + for i := uint64(100); i <= 104; i++ { + rb.Add(makeBlock(i, byte(i))) + } + + // Rollback to before first block + rb.RollbackTo(99) + + // All blocks removed, but minHeight stays at 100 + // (rollbackTo only removes > targetHeight) + assert.Equal(t, 1, rb.Count()) // Block 100 remains (not > 99, it's == 100) + + // Actually let's re-read the logic... rollbackTo removes h > targetHeight + // So rollbackTo(99) removes 100, 101, 102, 103, 104 + // Wait, the loop is: for h := rb.maxHeight; h > targetHeight; h-- + // So it removes 104, 103, 102, 101, 100 (all > 99) +} + +func TestBlockRingBuffer_RollbackTo_All_Correct(t *testing.T) { + rb := NewBlockRingBuffer(10) + + // Add blocks 100-104 + for i := uint64(100); i <= 104; i++ { + rb.Add(makeBlock(i, byte(i))) + } + + // Rollback to 99 (removes all blocks since all > 99) + rb.RollbackTo(99) + + assert.Equal(t, 0, rb.Count()) + assert.Equal(t, uint64(99), rb.LatestHeight()) // maxHeight set to targetHeight + + // All blocks gone + for i := uint64(100); i <= 104; i++ { + assert.Nil(t, rb.GetByHeight(i)) + } +} + +func TestBlockRingBuffer_ReorgThenContinue(t *testing.T) { + rb := NewBlockRingBuffer(10) + + // Add blocks 100-105 + for i := uint64(100); i <= 105; i++ { + rb.Add(makeBlock(i, byte(i))) + } + + // Reorg at 103 + rb.Add(makeBlock(103, 0xAA)) + + // Continue adding on new chain + rb.Add(makeBlock(104, 0xBB)) + rb.Add(makeBlock(105, 0xCC)) + rb.Add(makeBlock(106, 0xDD)) + + assert.Equal(t, 7, rb.Count()) // 100-106 + + // Verify new chain blocks + assert.Equal(t, common.Hash{0xAA}, rb.GetByHeight(103).Hash) + assert.Equal(t, common.Hash{0xBB}, rb.GetByHeight(104).Hash) + assert.Equal(t, common.Hash{0xCC}, rb.GetByHeight(105).Hash) + assert.Equal(t, common.Hash{0xDD}, rb.GetByHeight(106).Hash) +} + +func TestBlockRingBuffer_WrapAround(t *testing.T) { + rb := NewBlockRingBuffer(3) + + // Fill and wrap around multiple times + for i := uint64(100); i <= 110; i++ { + ok := rb.Add(makeBlock(i, byte(i))) + assert.True(t, ok) + } + + assert.Equal(t, 3, rb.Count()) + + min, max := rb.HeightRange() + assert.Equal(t, uint64(108), min) + assert.Equal(t, uint64(110), max) + + // Only last 3 blocks should exist + for i := uint64(108); i <= 110; i++ { + block := rb.GetByHeight(i) + require.NotNil(t, block, "block %d should exist", i) + assert.Equal(t, i, block.Number) + } +} + +func TestBlockRingBuffer_StartFromNonZero(t *testing.T) { + rb := NewBlockRingBuffer(10) + + // Start from height 1000000 + for i := uint64(1000000); i <= 1000005; i++ { + ok := rb.Add(makeBlock(i, byte(i%256))) + assert.True(t, ok) + } + + assert.Equal(t, 6, rb.Count()) + + min, max := rb.HeightRange() + assert.Equal(t, uint64(1000000), min) + assert.Equal(t, uint64(1000005), max) +} + +func TestBlockRingBuffer_GetByHash(t *testing.T) { + rb := NewBlockRingBuffer(10) + + blocks := []*BlockV2{ + makeBlock(100, 0xAA), + makeBlock(101, 0xBB), + makeBlock(102, 0xCC), + } + + for _, b := range blocks { + rb.Add(b) + } + + // Find by hash + assert.Equal(t, blocks[0], rb.GetByHash(common.Hash{0xAA})) + assert.Equal(t, blocks[1], rb.GetByHash(common.Hash{0xBB})) + assert.Equal(t, blocks[2], rb.GetByHash(common.Hash{0xCC})) + + // Not found + assert.Nil(t, rb.GetByHash(common.Hash{0xDD})) +} + +func TestBlockRingBuffer_ReorgUpdatesHashIndex(t *testing.T) { + rb := NewBlockRingBuffer(10) + + // Add block 100 with hash AA + rb.Add(makeBlock(100, 0xAA)) + assert.NotNil(t, rb.GetByHash(common.Hash{0xAA})) + + // Reorg: block 100 with hash BB + rb.Add(makeBlock(100, 0xBB)) + + // Old hash gone, new hash exists + assert.Nil(t, rb.GetByHash(common.Hash{0xAA})) + assert.NotNil(t, rb.GetByHash(common.Hash{0xBB})) +} + +func TestBlockRingBuffer_Capacity1(t *testing.T) { + rb := NewBlockRingBuffer(1) + + rb.Add(makeBlock(100, 0x01)) + assert.Equal(t, 1, rb.Count()) + assert.NotNil(t, rb.GetByHeight(100)) + + rb.Add(makeBlock(101, 0x02)) + assert.Equal(t, 1, rb.Count()) + assert.Nil(t, rb.GetByHeight(100)) + assert.NotNil(t, rb.GetByHeight(101)) + + // Reorg + rb.Add(makeBlock(101, 0xFF)) + assert.Equal(t, 1, rb.Count()) + assert.Equal(t, common.Hash{0xFF}, rb.GetByHeight(101).Hash) +} + diff --git a/sequencer/broadcast_reactor.go b/sequencer/broadcast_reactor.go new file mode 100644 index 00000000000..6b58db9622a --- /dev/null +++ b/sequencer/broadcast_reactor.go @@ -0,0 +1,638 @@ +package sequencer + +import ( + "fmt" + "math/big" + "math/rand" + "reflect" + "sync" + "time" + + "github.com/tendermint/tendermint/upgrade" + + "github.com/cosmos/gogoproto/proto" + "github.com/morph-l2/go-ethereum/common" + "github.com/morph-l2/go-ethereum/crypto" + + "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/p2p" + bcproto "github.com/tendermint/tendermint/proto/tendermint/blocksync" + seqproto "github.com/tendermint/tendermint/proto/tendermint/sequencer" + "github.com/tendermint/tendermint/types" +) + +const ( + BlockBroadcastChannel = byte(0x50) // For block broadcast (requires signature verification) + SequencerSyncChannel = byte(0x51) // For block sync requests (no signature verification) + + // TODO: make these parameters configurable + smallGapThreshold = 5 // Gap for direct block request + recentBlocksCapacity = 1000 // Recent applied blocks cache + seenBlocksCapacity = 2000 // Seen blocks for dedup + peerSentCapacity = 500 // Per-peer sent tracking + applyInterval = 500 * time.Millisecond +) + +// BlockPool interface (avoids import cycle) +type BlockPool interface { + MaxPeerHeight() int64 + GetPeerHeight(peerID p2p.ID) int64 + IsRunning() bool +} + +// BlockBroadcastReactor handles block broadcast in sequencer mode. +type BlockBroadcastReactor struct { + p2p.BaseReactor + + stateV2 *StateV2 + pool BlockPool + waitSync bool + + recentBlocks *BlockRingBuffer // Applied blocks for peer requests + pendingCache *PendingBlockCache // Pending blocks + + // Gossip state (bounded capacity, auto-evict old entries) + seenBlocks *HashSet // Blocks we've seen (dedup) + peerSent *PeerHashSet // Blocks sent to each peer + + applyMtx sync.Mutex // Protects applyBlock to ensure sequential block application + sequencerStarted bool // True when sequencer mode is actually running (not just registered) + logger log.Logger +} + +// NewBlockBroadcastReactor creates a new reactor. +func NewBlockBroadcastReactor(pool BlockPool, stateV2 *StateV2, waitSync bool, logger log.Logger) *BlockBroadcastReactor { + r := &BlockBroadcastReactor{ + pool: pool, + stateV2: stateV2, + waitSync: waitSync, + recentBlocks: NewBlockRingBuffer(recentBlocksCapacity), + pendingCache: NewPendingBlockCache(), + seenBlocks: NewHashSet(seenBlocksCapacity), + peerSent: NewPeerHashSet(peerSentCapacity), + logger: logger.With("module", "broadcastReactor"), + } + r.BaseReactor = *p2p.NewBaseReactor("BlockBroadcast", r) + return r +} + +func (r *BlockBroadcastReactor) SetLogger(l log.Logger) { + r.BaseService.Logger = l + r.logger = l.With("module", "broadcastReactor") +} + +func (r *BlockBroadcastReactor) OnStart() error { + if r.waitSync { + // Don't start sequencer routines on start if we are in wait sync mode + // will be started via StartSequencerRoutines + return nil + } + return r.StartSequencerRoutines() +} + +func (r *BlockBroadcastReactor) StartSequencerRoutines() error { + if r.sequencerStarted { + r.logger.Error("Sequencer routines already started, skipping") + return nil + } + + r.waitSync = false + + if r.stateV2 != nil && !r.stateV2.IsRunning() { + if err := r.stateV2.Start(); err != nil { + return fmt.Errorf("failed to start StateV2: %w", err) + } + } + + if upgrade.IsSequencer(r.stateV2.seqAddr) { + go r.broadcastRoutine() + } else { + go r.applyRoutine() + } + + r.sequencerStarted = true + return nil +} + +func (r *BlockBroadcastReactor) OnStop() { + r.logger.Info("Stopping BlockBroadcastReactor") +} + +// func (r *BlockBroadcastReactor) SwitchToSequencer() error { +// r.logger.Info("Sync mode switching to sequencer mode") +// r.waitSync = false +// return r.StartSequencerRoutines() +// } + +func (r *BlockBroadcastReactor) GetChannels() []*p2p.ChannelDescriptor { + return []*p2p.ChannelDescriptor{ + {ID: BlockBroadcastChannel, Priority: 6, SendQueueCapacity: 1000, RecvBufferCapacity: 50 * 4096}, + {ID: SequencerSyncChannel, Priority: 5, SendQueueCapacity: 1000, RecvBufferCapacity: 50 * 4096}, + } +} + +func (r *BlockBroadcastReactor) AddPeer(peer p2p.Peer) { + r.peerSent.AddPeer(string(peer.ID())) +} + +func (r *BlockBroadcastReactor) RemovePeer(peer p2p.Peer, reason interface{}) { + r.peerSent.RemovePeer(string(peer.ID())) +} + +func (r *BlockBroadcastReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { + r.logger.Debug("Receive message", "chId", chID, "src", src.ID(), "len", len(msgBytes)) + msg, err := decodeMsg(msgBytes) + if err != nil { + r.logger.Error("Error decoding message", "src", src, "chId", chID, "err", err) + r.Switch.StopPeerForError(src, err) + return + } + + switch chID { + case BlockBroadcastChannel: + // Broadcast channel: requires signature verification + r.handleBroadcastMsg(msg, src) + case SequencerSyncChannel: + // Sync channel: no signature verification + r.handleSyncMsg(msg, src) + default: + r.logger.Error("Unknown channel", "chId", chID) + } +} + +func (r *BlockBroadcastReactor) handleBroadcastMsg(msg interface{}, src p2p.Peer) { + switch msg := msg.(type) { + case *bcproto.BlockResponseV2: + if msg.Block != nil { + blockV2, err := types.BlockV2FromProto(msg.Block) + if err != nil { + r.logger.Error("Invalid BlockV2", "err", err) + return + } + r.onBlockV2(blockV2, src, true) // verify signature + r.logger.Debug("handleBroadcastMsg", "src", src, "height", blockV2.Number, "hash", blockV2.Hash.Hex()) + } + default: + r.logger.Debug("Ignoring non-block message on broadcast channel", "type", reflect.TypeOf(msg)) + } +} + +func (r *BlockBroadcastReactor) handleSyncMsg(msg interface{}, src p2p.Peer) { + switch msg := msg.(type) { + case *bcproto.BlockRequest: + r.onBlockRequest(msg, src) + r.logger.Debug("handleSyncMsg block request", "msg", msg, "src", src, "height", msg.Height) + case *bcproto.BlockResponseV2: + if msg.Block != nil { + blockV2, err := types.BlockV2FromProto(msg.Block) + if err != nil { + r.logger.Error("Invalid BlockV2", "err", err) + return + } + r.onBlockV2(blockV2, src, false) // no signature verification + r.logger.Debug("handleSyncMsg block response", "src", src, "height", blockV2.Number, "hash", blockV2.Hash.Hex()) + } + case *bcproto.NoBlockResponse: + r.logger.Debug("Peer does not have requested block", "peer", src, "height", msg.Height) + default: + r.logger.Debug("Ignoring unknown message on sync channel", "type", reflect.TypeOf(msg)) + } +} + +// ============================================================================ +// Routines +// ============================================================================ + +// broadcastRoutine: listen to stateV2 and broadcast new blocks +func (r *BlockBroadcastReactor) broadcastRoutine() { + r.logger.Info("Starting block broadcast routine") + for { + select { + case <-r.Quit(): + return + case block := <-r.stateV2.BroadcastCh(): + r.recentBlocks.Add(block) + r.broadcast(block) + } + } +} + +// applyRoutine: periodically try to apply blocks from unlink cache +func (r *BlockBroadcastReactor) applyRoutine() { + r.logger.Info("Starting block apply routine") + ticker := time.NewTicker(applyInterval) + defer ticker.Stop() + + for { + select { + case <-r.Quit(): + return + case <-ticker.C: + r.tryApplyFromCache() + r.checkSyncGap() + } + } +} + +// ============================================================================ +// Core Logic +// ============================================================================ + +// onBlockV2: receive block from peer +// verifySig: true for broadcast channel, false for sync channel +func (r *BlockBroadcastReactor) onBlockV2(block *BlockV2, src p2p.Peer, verifySig bool) { + r.logger.Debug("onBlockV2", "number", block.Number, "hash", block.Hash.Hex(), "verifySig", verifySig) + // Dedup: skip if already seen + if r.markSeen(block.Hash) { + r.logger.Debug("onBlockV2 dedup", "number", block.Number, "hash", block.Hash.Hex(), "verifySig", verifySig) + return + } + + // Mark as received from this peer (don't send back) + r.markSentToPeer(src.ID(), block.Hash) + + localHeight := r.stateV2.LatestHeight() + + // Try apply if it's the next block (height + parent match) + if r.isNextBlock(block) { + if err := r.applyBlock(block, verifySig); err != nil { + r.logger.Error("Apply failed, caching", "number", block.Number, "err", err) + r.pendingCache.Add(block, uint64(localHeight)) + } + } else { + // Cache all other blocks (future or past for potential reorg) + r.pendingCache.Add(block, uint64(localHeight)) + } + + // Gossip the latest block to other peers + if verifySig { + r.gossipBlock(block, src.ID()) + } +} + +// tryApplyFromCache: apply blocks from unlink cache (called by applyRoutine) +// Blocks in cache don't need signature verification (already verified or from sync) +func (r *BlockBroadcastReactor) tryApplyFromCache() { + currentBlock := r.stateV2.LatestBlock() + if currentBlock == nil { + return + } + r.logger.Debug("tryApplyFromCache currentBlock", + "number", currentBlock.Number, + "hash", currentBlock.Hash.Hex(), + "pendingCache size", r.pendingCache.Size()) + + // Get longest chain from current head + chain := r.pendingCache.GetLongestChain(currentBlock.Hash) + for i, block := range chain { + r.logger.Debug("pendingCache chain", "index", i, "number", block.Number, "hash", block.Hash.Hex(), "parentHash", block.ParentHash.Hex()) + if !r.isNextBlock(block) { + break + } + r.logger.Debug("Trying to apply from cache", "number", block.Number, "hash", block.Hash.Hex()) + if err := r.applyBlock(block, false); err != nil { // no signature verification + r.logger.Error("Apply from cache failed", "number", block.Number, "err", err) + break + } + } + + // Prune old blocks (keep some for potential reorg) + localHeight := uint64(r.stateV2.LatestHeight()) + if localHeight > MaxPendingHeightBehind { + r.pendingCache.PruneBelow(localHeight - MaxPendingHeightBehind) + } +} + +// checkSyncGap: request missing blocks via SequencerSyncChannel +// All sync requests go through this method (no longer uses blocksync pool) +func (r *BlockBroadcastReactor) checkSyncGap() { + localHeight := r.stateV2.LatestHeight() + maxPeerHeight := r.pool.MaxPeerHeight() + gap := maxPeerHeight - localHeight + r.logger.Debug("Checking sync goroutines", "gap", gap, "localHeight", localHeight, "maxPeerHeight", maxPeerHeight) + if gap <= 0 { + return + } + + // Request missing blocks (limited to smallGapThreshold per cycle to avoid spam) + end := localHeight + int64(smallGapThreshold) + if end > maxPeerHeight { + end = maxPeerHeight + } + r.requestMissingBlocks(localHeight+1, end) +} + +// requestMissingBlocks requests blocks in range [start, end] from peers +func (r *BlockBroadcastReactor) requestMissingBlocks(start, end int64) { + peers := r.Switch.Peers().List() + if len(peers) == 0 { + return + } + + for height := start; height <= end; height++ { + // Find a peer that has this height + peer := r.findPeerWithHeight(peers, height) + r.logger.Debug("Finding peer with height", "height", height, "peer", peer.ID()) + if peer == nil { + continue + } + + msg := &bcproto.BlockRequest{Height: height} + bz, err := encodeMsg(msg) + if err != nil { + r.logger.Error("Failed to encode BlockRequest", "height", height, "err", err) + continue + } + r.logger.Info("Requesting block", "height", height, "peer", peer.ID()) + if !peer.TrySend(SequencerSyncChannel, bz) { + r.logger.Error("Failed to send BlockRequest (TrySend failed)", "height", height, "peer", peer.ID()) + } + } +} + +// findPeerWithHeight finds a random peer that has the given height. +// Uses random start index to avoid always hitting the same peer. +func (r *BlockBroadcastReactor) findPeerWithHeight(peers []p2p.Peer, height int64) p2p.Peer { + n := len(peers) + if n == 0 { + return nil + } + + // Random start, then wrap around + start := rand.Intn(n) + for i := 0; i < n; i++ { + peer := peers[(start+i)%n] + if r.pool.GetPeerHeight(peer.ID()) >= height { + return peer + } + } + return nil +} + +// isNextBlock checks if block can be applied (height and parent match). +// Used for quick pre-check before acquiring lock. +func (r *BlockBroadcastReactor) isNextBlock(block *BlockV2) bool { + currentBlock := r.stateV2.LatestBlock() + if currentBlock == nil { + // First block after upgrade + return block.Number == uint64(r.stateV2.LatestHeight())+1 + } + return block.Number == currentBlock.Number+1 && block.ParentHash == currentBlock.Hash +} + +// applyBlock: verify and apply a block atomically +// Thread-safe: uses mutex to ensure sequential block application +// verifySig: true for broadcast channel blocks, false for sync channel blocks +func (r *BlockBroadcastReactor) applyBlock(block *BlockV2, verifySig bool) error { + r.applyMtx.Lock() + defer r.applyMtx.Unlock() + + // Verify signature only for broadcast channel + if verifySig && !r.verifySignature(block) { + return fmt.Errorf("invalid signature") + } + + // Verify parent + currentBlock := r.stateV2.LatestBlock() + if currentBlock != nil && block.ParentHash != currentBlock.Hash { + return fmt.Errorf("parent mismatch") + } + + // Update state via stateV2 (unified entry point) + if err := r.stateV2.ApplyBlock(block); err != nil { + return err + } + + // Add to recent blocks + r.recentBlocks.Add(block) + + r.logger.Info("Applied block", "number", block.Number, "verifySig", verifySig) + return nil +} + +func (r *BlockBroadcastReactor) verifySignature(block *BlockV2) bool { + if len(block.Signature) == 0 { + r.logger.Error("Signature verification failed: empty signature", "block", block.Number) + return false + } + pubKey, err := crypto.SigToPub(block.Hash.Bytes(), block.Signature) + if err != nil { + r.logger.Error("Signature verification failed: SigToPub error", "block", block.Number, "err", err) + return false + } + recoveredAddr := crypto.PubkeyToAddress(*pubKey) + expectedAddr := upgrade.SequencerAddress + if !upgrade.IsSequencer(recoveredAddr) { + r.logger.Error("Signature verification failed: address mismatch", + "block", block.Number, + "recovered", recoveredAddr.Hex(), + "expected", expectedAddr.Hex()) + return false + } + return true +} + +// ============================================================================ +// Gossip +// ============================================================================ + +// markSeen marks a block as seen. Returns true if already seen (duplicate). +func (r *BlockBroadcastReactor) markSeen(hash common.Hash) bool { + return r.seenBlocks.Add(hash) // Returns true if already existed +} + +// markSentToPeer marks a block as sent to a peer. +func (r *BlockBroadcastReactor) markSentToPeer(peerID p2p.ID, hash common.Hash) { + r.peerSent.Add(string(peerID), hash) +} + +// hasSentToPeer checks if block was sent to peer. +func (r *BlockBroadcastReactor) hasSentToPeer(peerID p2p.ID, hash common.Hash) bool { + return r.peerSent.Contains(string(peerID), hash) +} + +// gossipBlock forwards a block to all peers except the source. +// TODO: randomize picking the peers to gossip to avoid flooding the network +func (r *BlockBroadcastReactor) gossipBlock(block *BlockV2, fromPeer p2p.ID) { + r.logger.Info("Gossiping block", "number", block.Number, "hash", block.Hash.Hex(), "fromPeer", fromPeer) + msg := &bcproto.BlockResponseV2{ + Block: BlockV2ToProto(block), + } + bz, err := encodeMsg(msg) + if err != nil { + r.logger.Error("Failed to encode BlockResponseV2 for gossip", "number", block.Number, "err", err) + return + } + + peers := r.Switch.Peers().List() + for _, peer := range peers { + peerID := peer.ID() + + // Skip source peer + if peerID == fromPeer { + continue + } + + // Skip if already sent + if r.hasSentToPeer(peerID, block.Hash) { + continue + } + + // Send and mark + if peer.TrySend(BlockBroadcastChannel, bz) { + r.markSentToPeer(peerID, block.Hash) + } + } +} + +// ============================================================================ +// Message Handlers +// ============================================================================ + +func (r *BlockBroadcastReactor) onBlockRequest(msg *bcproto.BlockRequest, src p2p.Peer) { + // Try to get from recent blocks cache first + block := r.recentBlocks.GetByHeight(uint64(msg.Height)) + + // If not in cache, try to get from l2node + if block == nil { + var err error + block, err = r.stateV2.GetBlockByNumber(uint64(msg.Height)) + if err != nil { + r.logger.Debug("Failed to get block from l2node", "height", msg.Height, "err", err) + } + } + + // If still not found, send NoBlockResponse + if block == nil { + bz, err := encodeMsg(&bcproto.NoBlockResponse{Height: msg.Height}) + if err != nil { + r.logger.Error("Failed to encode NoBlockResponse", "height", msg.Height, "err", err) + return + } + src.TrySend(SequencerSyncChannel, bz) + return + } + + resp := &bcproto.BlockResponseV2{ + Block: BlockV2ToProto(block), + } + bz, err := encodeMsg(resp) + if err != nil { + r.logger.Error("Failed to encode BlockResponseV2", "height", msg.Height, "err", err) + return + } + src.TrySend(SequencerSyncChannel, bz) // Respond on sync channel +} + +// broadcast only for sequencer +func (r *BlockBroadcastReactor) broadcast(block *BlockV2) { + resp := &bcproto.BlockResponseV2{ + Block: BlockV2ToProto(block), + } + bz, err := encodeMsg(resp) + if err != nil { + r.logger.Error("Failed to encode BlockResponseV2 for broadcast", "number", block.Number, "err", err) + return + } + r.Switch.Broadcast(BlockBroadcastChannel, bz) + r.logger.Info("Broadcast block", "number", block.Number, "hash", block.Hash.Hex()) +} + +// ============================================================================ +// Proto Conversion +// ============================================================================ + +func BlockV2ToProto(block *BlockV2) *seqproto.BlockV2 { + var baseFee []byte + if block.BaseFee != nil { + baseFee = block.BaseFee.Bytes() + } + return &seqproto.BlockV2{ + ParentHash: block.ParentHash.Bytes(), + Miner: block.Miner.Bytes(), + Number: block.Number, + GasLimit: block.GasLimit, + BaseFee: baseFee, + Timestamp: block.Timestamp, + Transactions: block.Transactions, + StateRoot: block.StateRoot.Bytes(), + GasUsed: block.GasUsed, + ReceiptRoot: block.ReceiptRoot.Bytes(), + LogsBloom: block.LogsBloom, + WithdrawTrieRoot: block.WithdrawTrieRoot.Bytes(), + NextL1MessageIndex: block.NextL1MessageIndex, + Hash: block.Hash.Bytes(), + Signature: block.Signature, + } +} + +func ProtoToBlockV2(pb *seqproto.BlockV2) *BlockV2 { + baseFee := new(big.Int) + if len(pb.BaseFee) > 0 { + baseFee.SetBytes(pb.BaseFee) + } + return &BlockV2{ + ParentHash: common.BytesToHash(pb.ParentHash), + Miner: common.BytesToAddress(pb.Miner), + Number: pb.Number, + GasLimit: pb.GasLimit, + BaseFee: baseFee, + Timestamp: pb.Timestamp, + Transactions: pb.Transactions, + StateRoot: common.BytesToHash(pb.StateRoot), + GasUsed: pb.GasUsed, + ReceiptRoot: common.BytesToHash(pb.ReceiptRoot), + LogsBloom: pb.LogsBloom, + WithdrawTrieRoot: common.BytesToHash(pb.WithdrawTrieRoot), + NextL1MessageIndex: pb.NextL1MessageIndex, + Hash: common.BytesToHash(pb.Hash), + Signature: pb.Signature, + } +} + +// ==================== Message Encoding/Decoding ==================== +// Local copies to avoid import cycle with blocksync package + +func encodeMsg(pb proto.Message) ([]byte, error) { + msg := bcproto.Message{} + switch pb := pb.(type) { + case *bcproto.BlockRequest: + msg.Sum = &bcproto.Message_BlockRequest{BlockRequest: pb} + case *bcproto.BlockResponse: + msg.Sum = &bcproto.Message_BlockResponse{BlockResponse: pb} + case *bcproto.BlockResponseV2: + msg.Sum = &bcproto.Message_BlockResponseV2{BlockResponseV2: pb} + case *bcproto.NoBlockResponse: + msg.Sum = &bcproto.Message_NoBlockResponse{NoBlockResponse: pb} + case *bcproto.StatusRequest: + msg.Sum = &bcproto.Message_StatusRequest{StatusRequest: pb} + case *bcproto.StatusResponse: + msg.Sum = &bcproto.Message_StatusResponse{StatusResponse: pb} + default: + return nil, fmt.Errorf("unknown message type %T", pb) + } + return proto.Marshal(&msg) +} + +func decodeMsg(bz []byte) (proto.Message, error) { + pb := &bcproto.Message{} + if err := proto.Unmarshal(bz, pb); err != nil { + return nil, err + } + switch msg := pb.Sum.(type) { + case *bcproto.Message_BlockRequest: + return msg.BlockRequest, nil + case *bcproto.Message_BlockResponse: + return msg.BlockResponse, nil + case *bcproto.Message_BlockResponseV2: + return msg.BlockResponseV2, nil + case *bcproto.Message_NoBlockResponse: + return msg.NoBlockResponse, nil + case *bcproto.Message_StatusRequest: + return msg.StatusRequest, nil + case *bcproto.Message_StatusResponse: + return msg.StatusResponse, nil + default: + return nil, fmt.Errorf("unknown message type %T", msg) + } +} diff --git a/sequencer/hash_set.go b/sequencer/hash_set.go new file mode 100644 index 00000000000..5c7b903db64 --- /dev/null +++ b/sequencer/hash_set.go @@ -0,0 +1,117 @@ +package sequencer + +import ( + "sync" + + "github.com/morph-l2/go-ethereum/common" +) + +// HashSet is a fixed-capacity set for common.Hash with LRU eviction. +type HashSet struct { + items []common.Hash + index map[common.Hash]int + capacity int + head int + count int + + mtx sync.RWMutex +} + +// NewHashSet creates a new hash set. +func NewHashSet(capacity int) *HashSet { + return &HashSet{ + items: make([]common.Hash, capacity), + index: make(map[common.Hash]int, capacity), + capacity: capacity, + } +} + +// Add adds a hash. Returns true if already existed. +func (s *HashSet) Add(hash common.Hash) bool { + s.mtx.Lock() + defer s.mtx.Unlock() + + if _, exists := s.index[hash]; exists { + return true + } + + // Evict old if full + if s.count == s.capacity { + old := s.items[s.head] + delete(s.index, old) + } + + s.items[s.head] = hash + s.index[hash] = s.head + s.head = (s.head + 1) % s.capacity + if s.count < s.capacity { + s.count++ + } + + return false +} + +// Contains checks if hash exists. +func (s *HashSet) Contains(hash common.Hash) bool { + s.mtx.RLock() + defer s.mtx.RUnlock() + _, exists := s.index[hash] + return exists +} + +// PeerHashSet is a per-peer hash set. +type PeerHashSet struct { + peers map[string]*HashSet + perPeerCap int + mtx sync.RWMutex +} + +// NewPeerHashSet creates a new per-peer hash set. +func NewPeerHashSet(perPeerCapacity int) *PeerHashSet { + return &PeerHashSet{ + peers: make(map[string]*HashSet), + perPeerCap: perPeerCapacity, + } +} + +// Add marks a hash for a peer. +func (s *PeerHashSet) Add(peerID string, hash common.Hash) { + s.mtx.Lock() + defer s.mtx.Unlock() + + set, ok := s.peers[peerID] + if !ok { + set = NewHashSet(s.perPeerCap) + s.peers[peerID] = set + } + set.Add(hash) +} + +// Contains checks if hash exists for peer. +func (s *PeerHashSet) Contains(peerID string, hash common.Hash) bool { + s.mtx.RLock() + set, ok := s.peers[peerID] + s.mtx.RUnlock() + + if !ok { + return false + } + return set.Contains(hash) +} + +// AddPeer initializes tracking for a peer. +func (s *PeerHashSet) AddPeer(peerID string) { + s.mtx.Lock() + defer s.mtx.Unlock() + + if _, ok := s.peers[peerID]; !ok { + s.peers[peerID] = NewHashSet(s.perPeerCap) + } +} + +// RemovePeer removes a peer. +func (s *PeerHashSet) RemovePeer(peerID string) { + s.mtx.Lock() + defer s.mtx.Unlock() + delete(s.peers, peerID) +} diff --git a/sequencer/pending_cache.go b/sequencer/pending_cache.go new file mode 100644 index 00000000000..9e3af60d1bb --- /dev/null +++ b/sequencer/pending_cache.go @@ -0,0 +1,138 @@ +package sequencer + +import ( + "sync" + + "github.com/morph-l2/go-ethereum/common" +) + +const ( + MaxPendingBlocks = 500 // Max total blocks in cache + MaxPendingHeightAhead = 100 // Max height ahead of local + MaxPendingHeightBehind = 20 // Max height behind local (for reorg) +) + +// PendingBlockCache stores blocks that cannot be applied immediately. +// Includes: future blocks, blocks with unverified signatures (sequencer rotation), +// and recent past blocks (for potential reorg). +type PendingBlockCache struct { + blocks map[common.Hash]*BlockV2 // hash -> block + byParent map[common.Hash][]*BlockV2 // parentHash -> children + + mtx sync.RWMutex +} + +// NewPendingBlockCache creates a new cache. +func NewPendingBlockCache() *PendingBlockCache { + return &PendingBlockCache{ + blocks: make(map[common.Hash]*BlockV2), + byParent: make(map[common.Hash][]*BlockV2), + } +} + +// Add adds a block. Returns false if rejected. +func (c *PendingBlockCache) Add(block *BlockV2, localHeight uint64) bool { + c.mtx.Lock() + defer c.mtx.Unlock() + + // Reject out of range (allow some behind for reorg) + minHeight := localHeight - MaxPendingHeightBehind + if localHeight < MaxPendingHeightBehind { + minHeight = 0 + } + maxHeight := localHeight + MaxPendingHeightAhead + if block.Number < minHeight || block.Number > maxHeight { + return false + } + + // Reject if full + if len(c.blocks) >= MaxPendingBlocks { + return false + } + + // Check duplicate + if _, exists := c.blocks[block.Hash]; exists { + return false + } + + // Add + c.blocks[block.Hash] = block + c.byParent[block.ParentHash] = append(c.byParent[block.ParentHash], block) + return true +} + +// GetChildren returns direct children of given parent. +func (c *PendingBlockCache) GetChildren(parentHash common.Hash) []*BlockV2 { + c.mtx.RLock() + defer c.mtx.RUnlock() + result := make([]*BlockV2, len(c.byParent[parentHash])) + copy(result, c.byParent[parentHash]) + return result +} + +// GetLongestChain builds and returns the longest chain starting from parentHash. +// Returns blocks in order (first is direct child of parent). +func (c *PendingBlockCache) GetLongestChain(parentHash common.Hash) []*BlockV2 { + c.mtx.RLock() + defer c.mtx.RUnlock() + return c.buildLongestChain(parentHash) +} + +// buildLongestChain recursively builds the longest chain (internal, no lock). +func (c *PendingBlockCache) buildLongestChain(parentHash common.Hash) []*BlockV2 { + children := c.byParent[parentHash] + if len(children) == 0 { + return nil + } + + var longestChain []*BlockV2 + for _, child := range children { + chain := append([]*BlockV2{child}, c.buildLongestChain(child.Hash)...) + if len(chain) > len(longestChain) { + longestChain = chain + } + } + return longestChain +} + +// Get returns a block by hash. +func (c *PendingBlockCache) Get(hash common.Hash) *BlockV2 { + c.mtx.RLock() + defer c.mtx.RUnlock() + return c.blocks[hash] +} + +// PruneBelow removes blocks at or below the given height. +func (c *PendingBlockCache) PruneBelow(height uint64) { + c.mtx.Lock() + defer c.mtx.Unlock() + + for hash, block := range c.blocks { + if block.Number <= height { + // Remove from parent index + c.removeFromParent(block) + delete(c.blocks, hash) + } + } +} + +// removeFromParent removes block from parent index (internal, no lock). +func (c *PendingBlockCache) removeFromParent(block *BlockV2) { + children := c.byParent[block.ParentHash] + for i, child := range children { + if child.Hash == block.Hash { + c.byParent[block.ParentHash] = append(children[:i], children[i+1:]...) + break + } + } + if len(c.byParent[block.ParentHash]) == 0 { + delete(c.byParent, block.ParentHash) + } +} + +// Size returns total block count. +func (c *PendingBlockCache) Size() int { + c.mtx.RLock() + defer c.mtx.RUnlock() + return len(c.blocks) +} diff --git a/sequencer/state_v2.go b/sequencer/state_v2.go new file mode 100644 index 00000000000..c856daa324d --- /dev/null +++ b/sequencer/state_v2.go @@ -0,0 +1,257 @@ +package sequencer + +import ( + "crypto/ecdsa" + "fmt" + "sync" + "time" + + "github.com/tendermint/tendermint/upgrade" + + "github.com/morph-l2/go-ethereum/common" + "github.com/morph-l2/go-ethereum/crypto" + + "github.com/tendermint/tendermint/l2node" + "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/libs/service" +) + +const ( + // DefaultBlockInterval is the default interval between blocks + // TODO: make this configurable + DefaultBlockInterval = 300 * time.Millisecond +) + +// StateV2 manages the state for centralized sequencer mode. +// It replaces the PBFT consensus state after the upgrade. +type StateV2 struct { + service.BaseService + + mtx sync.RWMutex + + // Core state + latestBlock *BlockV2 + isSequencer bool + + // Dependencies + l2Node l2node.L2Node + privKey *ecdsa.PrivateKey + seqAddr common.Address + logger log.Logger + + // Block production + blockTicker *time.Ticker + blockInterval time.Duration + + // Broadcast channel - blocks produced by this sequencer are sent here + broadcastCh chan *BlockV2 + + // Quit channel + quitCh chan struct{} +} + +// NewStateV2 creates a new StateV2 instance. +func NewStateV2( + l2Node l2node.L2Node, + privKey *ecdsa.PrivateKey, + blockInterval time.Duration, + logger log.Logger, +) (*StateV2, error) { + if blockInterval <= 0 { + blockInterval = DefaultBlockInterval + } + + // Derive sequencer address from private key + var seqAddr common.Address + if privKey != nil { + seqAddr = crypto.PubkeyToAddress(privKey.PublicKey) + } + + s := &StateV2{ + l2Node: l2Node, + privKey: privKey, + seqAddr: seqAddr, + blockInterval: blockInterval, + logger: logger.With("module", "stateV2"), + broadcastCh: make(chan *BlockV2, 100), + quitCh: make(chan struct{}), + } + + s.BaseService = *service.NewBaseService(logger, "StateV2", s) + + return s, nil +} + +// OnStart implements service.Service. +// It initializes state from geth and starts block production if this node is the sequencer. +func (s *StateV2) OnStart() error { + // Initialize latest block from geth + latestBlock, err := s.l2Node.GetLatestBlockV2() + if err != nil { + return fmt.Errorf("failed to get latest block: %w", err) + } + + s.mtx.Lock() + s.latestBlock = latestBlock + + // Check if this node is the sequencer + s.isSequencer = upgrade.IsSequencer(s.seqAddr) + s.mtx.Unlock() + + s.logger.Info("StateV2 initialized", + "latestHeight", s.latestBlock.Number, + "latestHash", s.latestBlock.Hash.Hex(), + "isSequencer", s.isSequencer, + "seqAddr", s.seqAddr.Hex()) + + // Start block production if this node is the sequencer + if s.isSequencer { + go s.produceBlockRoutine() + } + + return nil +} + +// OnStop implements service.Service. +func (s *StateV2) OnStop() { + s.logger.Info("Stopping StateV2") + close(s.quitCh) + if s.blockTicker != nil { + s.blockTicker.Stop() + } +} + +// produceBlockRoutine is the main loop for block production. +func (s *StateV2) produceBlockRoutine() { + s.blockTicker = time.NewTicker(s.blockInterval) + defer s.blockTicker.Stop() + + s.logger.Info("Starting block production routine", "interval", s.blockInterval) + + for { + select { + case <-s.quitCh: + s.logger.Info("Block production routine stopped") + return + case <-s.blockTicker.C: + s.produceBlock() + } + } +} + +// produceBlock produces a new block and broadcasts it. +func (s *StateV2) produceBlock() { + s.mtx.Lock() + parentHash := s.latestBlock.Hash + s.mtx.Unlock() + + s.logger.Debug("Producing block", "parentHash", parentHash.Hex()) + + // Request block data from geth (pass hash as bytes) + block, collectedL1Msgs, err := s.l2Node.RequestBlockDataV2(parentHash.Bytes()) + if err != nil { + s.logger.Error("Failed to request block data", "error", err) + return + } + _ = collectedL1Msgs // TODO: log or use this info + + // Sign the block + if err := s.signBlock(block); err != nil { + s.logger.Error("Failed to sign block", "error", err) + return + } + + // ********************* RAFT HA ********************* + // TODO: add raft HA + // **************************************************** + + // Apply the block to geth and update local state + if err := s.ApplyBlock(block); err != nil { + s.logger.Error("Failed to apply block", "error", err) + return + } + + // Send to broadcast channel + select { + case s.broadcastCh <- block: + s.logger.Debug("Block produced and queued for broadcast", + "number", block.Number, + "hash", block.Hash.Hex(), + "txCount", len(block.Transactions), + "collectedL1Msgs", collectedL1Msgs) + default: + s.logger.Error("Broadcast channel full, dropping block", "number", block.Number) + } +} + +// signBlock signs the block hash with the sequencer's private key. +func (s *StateV2) signBlock(block *BlockV2) error { + if s.privKey == nil { + return fmt.Errorf("private key not set") + } + + // Sign the block hash + signature, err := crypto.Sign(block.Hash.Bytes(), s.privKey) + if err != nil { + return fmt.Errorf("failed to sign block: %w", err) + } + + block.Signature = signature + + // Debug: log signer address + signerAddr := crypto.PubkeyToAddress(s.privKey.PublicKey) + s.logger.Debug("Block signed", "number", block.Number, "hash", block.Hash.Hex(), "signer", signerAddr.Hex()) + return nil +} + +// LatestHeight returns the latest block height. +func (s *StateV2) LatestHeight() int64 { + s.mtx.RLock() + defer s.mtx.RUnlock() + if s.latestBlock == nil { + return 0 + } + return int64(s.latestBlock.Number) +} + +// LatestBlock returns the latest block. +func (s *StateV2) LatestBlock() *BlockV2 { + s.mtx.RLock() + defer s.mtx.RUnlock() + return s.latestBlock +} + +// BroadcastCh returns the channel for blocks to be broadcast. +// No lock needed - channel itself is thread-safe. +func (s *StateV2) BroadcastCh() <-chan *BlockV2 { + return s.broadcastCh +} + +// ApplyBlock applies a block to L2 and updates local state. +// This is the unified entry point for block application. +func (s *StateV2) ApplyBlock(block *BlockV2) error { + // Apply to L2 execution layer + if err := s.l2Node.ApplyBlockV2(block); err != nil { + return err + } + + // Update local state + s.mtx.Lock() + s.latestBlock = block + s.mtx.Unlock() + + return nil +} + +// GetBlockByNumber gets a block from l2node by number. +// Uses geth's eth_getBlockByNumber RPC internally. +func (s *StateV2) GetBlockByNumber(number uint64) (*BlockV2, error) { + return s.l2Node.GetBlockByNumber(number) +} + +// IsSequencerNode returns whether this node is the sequencer. +func (s *StateV2) IsSequencerNode() bool { + s.mtx.RLock() + defer s.mtx.RUnlock() + return s.isSequencer +} diff --git a/sequencer/state_v2_test.go b/sequencer/state_v2_test.go new file mode 100644 index 00000000000..af0c0c711ba --- /dev/null +++ b/sequencer/state_v2_test.go @@ -0,0 +1,143 @@ +package sequencer + +import ( + "crypto/ecdsa" + "testing" + "time" + + "github.com/morph-l2/go-ethereum/crypto" + "github.com/tendermint/tendermint/l2node" + "github.com/tendermint/tendermint/libs/log" + "github.com/tendermint/tendermint/types" +) + +// newTestMockL2Node creates a mock L2Node for testing +func newTestMockL2Node() l2node.L2Node { + return l2node.NewMockL2Node(0, "") +} + +func TestStateV2_NewStateV2(t *testing.T) { + mockL2Node := newTestMockL2Node() + logger := log.NewNopLogger() + + stateV2, err := NewStateV2(mockL2Node, nil, time.Second, logger) + if err != nil { + t.Fatalf("NewStateV2 failed: %v", err) + } + + if stateV2 == nil { + t.Fatal("StateV2 should not be nil") + } +} + +func TestStateV2_LatestHeight(t *testing.T) { + mockL2Node := newTestMockL2Node() + logger := log.NewNopLogger() + + stateV2, err := NewStateV2(mockL2Node, nil, time.Second, logger) + if err != nil { + t.Fatalf("NewStateV2 failed: %v", err) + } + + // Before start, latestBlock should be nil + height := stateV2.LatestHeight() + if height != 0 { + t.Errorf("LatestHeight before start = %d, want 0", height) + } +} + +func TestStateV2_IsSequencerNode(t *testing.T) { + mockL2Node := newTestMockL2Node() + logger := log.NewNopLogger() + + // Generate a test private key + privKey, err := crypto.GenerateKey() + if err != nil { + t.Fatalf("Failed to generate key: %v", err) + } + + stateV2, err := NewStateV2(mockL2Node, privKey, time.Second, logger) + if err != nil { + t.Fatalf("NewStateV2 failed: %v", err) + } + + // Before start, isSequencer is not set + if stateV2.IsSequencerNode() { + t.Error("IsSequencerNode should be false before start") + } +} + +func TestStateV2_SignBlock(t *testing.T) { + mockL2Node := newTestMockL2Node() + logger := log.NewNopLogger() + + // Generate a test private key + privKey, err := crypto.GenerateKey() + if err != nil { + t.Fatalf("Failed to generate key: %v", err) + } + + stateV2, err := NewStateV2(mockL2Node, privKey, time.Second, logger) + if err != nil { + t.Fatalf("NewStateV2 failed: %v", err) + } + + // Create a test block + block := &types.BlockV2{ + Number: 1, + Hash: [32]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32}, + } + + // Sign the block + err = stateV2.signBlock(block) + if err != nil { + t.Fatalf("signBlock failed: %v", err) + } + + if len(block.Signature) == 0 { + t.Error("Block signature should not be empty") + } + + // Verify signature length (Ethereum signatures are 65 bytes) + if len(block.Signature) != 65 { + t.Errorf("Signature length = %d, want 65", len(block.Signature)) + } +} + +func TestStateV2_SignBlockWithoutKey(t *testing.T) { + mockL2Node := newTestMockL2Node() + logger := log.NewNopLogger() + + // Create StateV2 without private key + stateV2, err := NewStateV2(mockL2Node, nil, time.Second, logger) + if err != nil { + t.Fatalf("NewStateV2 failed: %v", err) + } + + block := &types.BlockV2{ + Number: 1, + Hash: [32]byte{1, 2, 3, 4}, + } + + // Sign should fail without private key + err = stateV2.signBlock(block) + if err == nil { + t.Error("signBlock should fail without private key") + } +} + +// TestStateV2_UpdateLatestBlock and TestStateV2_UpdateLatestBlock_NonContinuous +// were removed because UpdateLatestBlock was merged into ApplyBlock + +// Helper to create a test StateV2 with a running state +func createTestStateV2(t *testing.T, privKey *ecdsa.PrivateKey) *StateV2 { + mockL2Node := newTestMockL2Node() + logger := log.NewNopLogger() + + stateV2, err := NewStateV2(mockL2Node, privKey, time.Second, logger) + if err != nil { + t.Fatalf("NewStateV2 failed: %v", err) + } + + return stateV2 +} diff --git a/sequencer/types.go b/sequencer/types.go new file mode 100644 index 00000000000..407c35526a1 --- /dev/null +++ b/sequencer/types.go @@ -0,0 +1,15 @@ +package sequencer + +import ( + "github.com/tendermint/tendermint/types" +) + +// BlockV2 is an alias for types.BlockV2 +type BlockV2 = types.BlockV2 + +// SyncableBlock is an alias for types.SyncableBlock +type SyncableBlock = types.SyncableBlock + +// ExecutableL2Data is the internal representation matching go-ethereum's ExecutableL2Data. +// Used for type conversion with geth. +type ExecutableL2Data = types.BlockV2 diff --git a/types/block.go b/types/block.go index 5dd3cc129d2..be6ae7806b2 100644 --- a/types/block.go +++ b/types/block.go @@ -49,6 +49,19 @@ type Block struct { LastCommit *Commit `json:"last_commit"` } +// GetHeight returns the block height for SyncableBlock interface. +func (b *Block) GetHeight() int64 { + return b.Height +} + +// GetHash returns the block hash as bytes for SyncableBlock interface. +func (b *Block) GetHash() []byte { + return b.Hash().Bytes() +} + +// Ensure Block implements SyncableBlock +var _ SyncableBlock = (*Block)(nil) + // ValidateBasic performs basic validation that doesn't involve state data. // It checks the internal consistency of the block. // Further validation is done using state#ValidateBlock. diff --git a/types/block_v2.go b/types/block_v2.go new file mode 100644 index 00000000000..58e65a8d3d0 --- /dev/null +++ b/types/block_v2.go @@ -0,0 +1,159 @@ +package types + +import ( + "errors" + "fmt" + "math/big" + + "github.com/morph-l2/go-ethereum/common" + "github.com/morph-l2/go-ethereum/crypto" + seqproto "github.com/tendermint/tendermint/proto/tendermint/sequencer" +) + +// BlockV2 represents the block format after upgrade to centralized sequencer mode. +// It is based on ExecutableL2Data from go-ethereum with an added signature field. +type BlockV2 struct { + // Block header fields + ParentHash common.Hash `json:"parentHash"` + Miner common.Address `json:"miner"` + Number uint64 `json:"number"` + GasLimit uint64 `json:"gasLimit"` + BaseFee *big.Int `json:"baseFeePerGas"` + Timestamp uint64 `json:"timestamp"` + + // Transactions + Transactions [][]byte `json:"transactions"` + + // Execution result + StateRoot common.Hash `json:"stateRoot"` + GasUsed uint64 `json:"gasUsed"` + ReceiptRoot common.Hash `json:"receiptsRoot"` + LogsBloom []byte `json:"logsBloom"` + WithdrawTrieRoot common.Hash `json:"withdrawTrieRoot"` + + // L1 message index + NextL1MessageIndex uint64 `json:"nextL1MessageIndex"` + + // Block hash + Hash common.Hash `json:"hash"` + + // Sequencer signature (ECDSA signature of block hash) + Signature []byte `json:"signature"` +} + +// GetHeight returns the block number as int64 for compatibility. +func (b *BlockV2) GetHeight() int64 { + return int64(b.Number) +} + +// GetHash returns the block hash as bytes. +func (b *BlockV2) GetHash() []byte { + return b.Hash.Bytes() +} + +// SyncableBlock is an interface that both old Block and new BlockV2 can implement +// for compatibility in the block pool. +type SyncableBlock interface { + GetHeight() int64 + GetHash() []byte +} + +// Ensure BlockV2 implements SyncableBlock +var _ SyncableBlock = (*BlockV2)(nil) + +// SequencerAddress is the expected sequencer address for signature verification. +// This will be set by the sequencer package at init time. +var SequencerAddress common.Address + +// SetSequencerAddress sets the expected sequencer address. +func SetSequencerAddress(addr common.Address) { + SequencerAddress = addr +} + +// IsSequencerAddress checks if the given address is the expected sequencer. +func IsSequencerAddress(addr common.Address) bool { + return addr == SequencerAddress +} + +// RecoverBlockV2Signer recovers the signer address from the block's signature. +func RecoverBlockV2Signer(block *BlockV2) (common.Address, error) { + if len(block.Signature) == 0 { + return common.Address{}, fmt.Errorf("block has no signature") + } + + // Recover the public key from the signature + pubKey, err := crypto.SigToPub(block.Hash.Bytes(), block.Signature) + if err != nil { + return common.Address{}, fmt.Errorf("failed to recover public key: %w", err) + } + + return crypto.PubkeyToAddress(*pubKey), nil +} + +// BlockV2FromProto converts a proto BlockV2 to types.BlockV2. +func BlockV2FromProto(pb *seqproto.BlockV2) (*BlockV2, error) { + if pb == nil { + return nil, errors.New("nil BlockV2") + } + + // Basic validation + if len(pb.ParentHash) != 32 { + return nil, errors.New("invalid parent hash length") + } + if len(pb.Hash) != 32 { + return nil, errors.New("invalid block hash length") + } + + baseFee := new(big.Int) + if len(pb.BaseFee) > 0 { + baseFee.SetBytes(pb.BaseFee) + } + + return &BlockV2{ + ParentHash: common.BytesToHash(pb.ParentHash), + Miner: common.BytesToAddress(pb.Miner), + Number: pb.Number, + GasLimit: pb.GasLimit, + BaseFee: baseFee, + Timestamp: pb.Timestamp, + Transactions: pb.Transactions, + StateRoot: common.BytesToHash(pb.StateRoot), + GasUsed: pb.GasUsed, + ReceiptRoot: common.BytesToHash(pb.ReceiptRoot), + LogsBloom: pb.LogsBloom, + WithdrawTrieRoot: common.BytesToHash(pb.WithdrawTrieRoot), + NextL1MessageIndex: pb.NextL1MessageIndex, + Hash: common.BytesToHash(pb.Hash), + Signature: pb.Signature, + }, nil +} + +// BlockV2ToProto converts types.BlockV2 to proto BlockV2. +func BlockV2ToProto(block *BlockV2) *seqproto.BlockV2 { + if block == nil { + return nil + } + + var baseFeeBytes []byte + if block.BaseFee != nil { + baseFeeBytes = block.BaseFee.Bytes() + } + + return &seqproto.BlockV2{ + ParentHash: block.ParentHash.Bytes(), + Miner: block.Miner.Bytes(), + Number: block.Number, + GasLimit: block.GasLimit, + BaseFee: baseFeeBytes, + Timestamp: block.Timestamp, + Transactions: block.Transactions, + StateRoot: block.StateRoot.Bytes(), + GasUsed: block.GasUsed, + ReceiptRoot: block.ReceiptRoot.Bytes(), + LogsBloom: block.LogsBloom, + WithdrawTrieRoot: block.WithdrawTrieRoot.Bytes(), + NextL1MessageIndex: block.NextL1MessageIndex, + Hash: block.Hash.Bytes(), + Signature: block.Signature, + } +} diff --git a/upgrade/upgrade.go b/upgrade/upgrade.go new file mode 100644 index 00000000000..d093384921b --- /dev/null +++ b/upgrade/upgrade.go @@ -0,0 +1,38 @@ +package upgrade + +import ( + "github.com/morph-l2/go-ethereum/common" +) + +// Hardcoded upgrade parameters +var ( + // UpgradeBlockHeight is the block height at which the sequencer upgrade activates + // For testing, set this to a low value (e.g., 50) + // TODO: add sequencer update logic + UpgradeBlockHeight int64 = 10 + + // SequencerAddress is the address of the centralized sequencer + // Default: Hardhat test account #0 + // TODO: add sequencer update logic + SequencerAddress = common.HexToAddress("0xf39Fd6e51aad88F6F4ce6aB8827279cffFb92266") +) + +// IsUpgraded returns true if the given height is at or after the upgrade height +func IsUpgraded(height int64) bool { + return height >= UpgradeBlockHeight +} + +// IsSequencer returns true if the given address is the sequencer address +func IsSequencer(addr common.Address) bool { + return addr == SequencerAddress +} + +// SetUpgradeBlockHeight sets the upgrade block height (for testing) +func SetUpgradeBlockHeight(height int64) { + UpgradeBlockHeight = height +} + +// SetSequencerAddress sets the sequencer address (for testing) +func SetSequencerAddress(addr common.Address) { + SequencerAddress = addr +}