From 08a83aa9fbb08d85543710a5c88c7cc7d58bf230 Mon Sep 17 00:00:00 2001 From: Jae Kwon Date: Wed, 25 Mar 2015 00:15:18 -0700 Subject: [PATCH] Reactors can be stopped or started at any time. --- blockchain/pool.go | 2 +- blockchain/reactor.go | 226 ++++++++++++++++++++++++++++++++++++++++++ blockchain/store.go | 24 +---- consensus/reactor.go | 5 +- consensus/state.go | 5 +- consensus/test.go | 4 +- daemon/daemon.go | 23 +++-- p2p/connection.go | 1 + p2p/peer_set.go | 1 + p2p/switch.go | 123 ++++++++++++----------- rpc/rpc.go | 6 +- state/state_test.go | 1 - types/block_meta.go | 15 +++ 13 files changed, 340 insertions(+), 96 deletions(-) create mode 100644 blockchain/reactor.go create mode 100644 types/block_meta.go diff --git a/blockchain/pool.go b/blockchain/pool.go index 3e3313bf..099594c1 100644 --- a/blockchain/pool.go +++ b/blockchain/pool.go @@ -149,7 +149,7 @@ func (bp *BlockPool) RedoRequest(height uint) { if request.block == nil { panic("Expected block to be non-nil") } - bp.removePeer(request.peerId) + bp.RemovePeer(request.peerId) // Lock on peersMtx. request.block = nil request.peerId = "" bp.numPending++ diff --git a/blockchain/reactor.go b/blockchain/reactor.go new file mode 100644 index 00000000..1aba782e --- /dev/null +++ b/blockchain/reactor.go @@ -0,0 +1,226 @@ +package blockchain + +import ( + "bytes" + "errors" + "fmt" + "sync/atomic" + "time" + + "github.com/tendermint/tendermint/binary" + "github.com/tendermint/tendermint/p2p" + "github.com/tendermint/tendermint/types" +) + +const ( + BlockchainChannel = byte(0x40) + defaultChannelCapacity = 100 + defaultSleepIntervalMS = 500 +) + +// BlockchainReactor handles long-term catchup syncing. +type BlockchainReactor struct { + sw *p2p.Switch + store *BlockStore + pool *BlockPool + requestsCh chan BlockRequest + timeoutsCh chan string + lastBlock *types.Block + quit chan struct{} + started uint32 + stopped uint32 +} + +func NewBlockchainReactor(store *BlockStore) *BlockchainReactor { + requestsCh := make(chan BlockRequest, defaultChannelCapacity) + timeoutsCh := make(chan string, defaultChannelCapacity) + pool := NewBlockPool( + store.Height()+1, + requestsCh, + timeoutsCh, + ) + bcR := &BlockchainReactor{ + store: store, + pool: pool, + requestsCh: requestsCh, + timeoutsCh: timeoutsCh, + quit: make(chan struct{}), + started: 0, + stopped: 0, + } + return bcR +} + +// Implements Reactor +func (bcR *BlockchainReactor) Start(sw *p2p.Switch) { + if atomic.CompareAndSwapUint32(&bcR.started, 0, 1) { + log.Info("Starting BlockchainReactor") + bcR.sw = sw + bcR.pool.Start() + go bcR.poolRoutine() + } +} + +// Implements Reactor +func (bcR *BlockchainReactor) Stop() { + if atomic.CompareAndSwapUint32(&bcR.stopped, 0, 1) { + log.Info("Stopping BlockchainReactor") + close(bcR.quit) + bcR.pool.Stop() + } +} + +// Implements Reactor +func (bcR *BlockchainReactor) GetChannels() []*p2p.ChannelDescriptor { + return []*p2p.ChannelDescriptor{ + &p2p.ChannelDescriptor{ + Id: BlockchainChannel, + Priority: 5, + SendQueueCapacity: 20, // Queue 20 blocks to send to a peer. + }, + } +} + +// Implements Reactor +func (bcR *BlockchainReactor) AddPeer(peer *p2p.Peer) { + // Send peer our state. + peer.Send(BlockchainChannel, PeerStatusMessage{bcR.store.Height()}) +} + +// Implements Reactor +func (bcR *BlockchainReactor) RemovePeer(peer *p2p.Peer, reason interface{}) { + // Remove peer from the pool. + bcR.pool.RemovePeer(peer.Key) +} + +// Implements Reactor +func (bcR *BlockchainReactor) Receive(chId byte, src *p2p.Peer, msgBytes []byte) { + _, msg_, err := DecodeMessage(msgBytes) + if err != nil { + log.Warn("Error decoding message", "error", err) + return + } + log.Info("BlockchainReactor received message", "msg", msg_) + + switch msg := msg_.(type) { + case BlockRequestMessage: + log.Debug("Got BlockRequest", "msg", msg) + // Got a request for a block. Respond with block if we have it. + block := bcR.store.LoadBlock(msg.Height) + if block != nil { + msg := BlockResponseMessage{Block: block} + queued := src.TrySend(BlockchainChannel, msg) + if !queued { + // queue is full, just ignore. + } + } else { + // TODO peer is asking for things we don't have. + } + case BlockResponseMessage: + log.Debug("Got BlockResponse", "msg", msg) + // Got a block. + bcR.pool.AddBlock(msg.Block, src.Key) + case PeerStatusMessage: + log.Debug("Got PeerStatus", "msg", msg) + // Got a peer status. + bcR.pool.SetPeerHeight(src.Key, msg.Height) + default: + // Ignore unknown message + } +} + +func (bcR *BlockchainReactor) poolRoutine() { +FOR_LOOP: + for { + select { + case request := <-bcR.requestsCh: // chan BlockRequest + peer := bcR.sw.Peers().Get(request.PeerId) + if peer == nil { + // We can't fulfill the request. + continue FOR_LOOP + } + msg := BlockRequestMessage{request.Height} + queued := peer.TrySend(BlockchainChannel, msg) + if !queued { + // We couldn't queue the request. + time.Sleep(defaultSleepIntervalMS * time.Millisecond) + continue FOR_LOOP + } + case peerId := <-bcR.timeoutsCh: // chan string + // Peer timed out. + peer := bcR.sw.Peers().Get(peerId) + bcR.sw.StopPeerForError(peer, errors.New("BlockchainReactor Timeout")) + case <-bcR.quit: + break FOR_LOOP + } + } +} + +func (bcR *BlockchainReactor) BroadcastStatus() error { + bcR.sw.Broadcast(BlockchainChannel, PeerStatusMessage{bcR.store.Height()}) + return nil +} + +//----------------------------------------------------------------------------- +// Messages + +const ( + msgTypeUnknown = byte(0x00) + msgTypeBlockRequest = byte(0x10) + msgTypeBlockResponse = byte(0x11) + msgTypePeerStatus = byte(0x20) +) + +// TODO: check for unnecessary extra bytes at the end. +func DecodeMessage(bz []byte) (msgType byte, msg interface{}, err error) { + n := new(int64) + msgType = bz[0] + r := bytes.NewReader(bz) + switch msgType { + case msgTypeBlockRequest: + msg = binary.ReadBinary(BlockRequestMessage{}, r, n, &err) + case msgTypeBlockResponse: + msg = binary.ReadBinary(BlockResponseMessage{}, r, n, &err) + case msgTypePeerStatus: + msg = binary.ReadBinary(PeerStatusMessage{}, r, n, &err) + default: + msg = nil + } + return +} + +//------------------------------------- + +type BlockRequestMessage struct { + Height uint +} + +func (m BlockRequestMessage) TypeByte() byte { return msgTypeBlockRequest } + +func (m BlockRequestMessage) String() string { + return fmt.Sprintf("[BlockRequestMessage %v]", m.Height) +} + +//------------------------------------- + +type BlockResponseMessage struct { + Block *types.Block +} + +func (m BlockResponseMessage) TypeByte() byte { return msgTypeBlockResponse } + +func (m BlockResponseMessage) String() string { + return fmt.Sprintf("[BlockResponseMessage %v]", m.Block.Height) +} + +//------------------------------------- + +type PeerStatusMessage struct { + Height uint +} + +func (m PeerStatusMessage) TypeByte() byte { return msgTypePeerStatus } + +func (m PeerStatusMessage) String() string { + return fmt.Sprintf("[PeerStatusMessage %v]", m.Height) +} diff --git a/blockchain/store.go b/blockchain/store.go index 8938273c..f9d54cd2 100644 --- a/blockchain/store.go +++ b/blockchain/store.go @@ -57,7 +57,7 @@ func (bs *BlockStore) LoadBlock(height uint) *types.Block { if r == nil { panic(Fmt("Block does not exist at height %v", height)) } - meta := binary.ReadBinary(&BlockMeta{}, r, &n, &err).(*BlockMeta) + meta := binary.ReadBinary(&types.BlockMeta{}, r, &n, &err).(*types.BlockMeta) if err != nil { panic(Fmt("Error reading block meta: %v", err)) } @@ -87,14 +87,14 @@ func (bs *BlockStore) LoadBlockPart(height uint, index uint) *types.Part { return part } -func (bs *BlockStore) LoadBlockMeta(height uint) *BlockMeta { +func (bs *BlockStore) LoadBlockMeta(height uint) *types.BlockMeta { var n int64 var err error r := bs.GetReader(calcBlockMetaKey(height)) if r == nil { panic(Fmt("BlockMeta does not exist for height %v", height)) } - meta := binary.ReadBinary(&BlockMeta{}, r, &n, &err).(*BlockMeta) + meta := binary.ReadBinary(&types.BlockMeta{}, r, &n, &err).(*types.BlockMeta) if err != nil { panic(Fmt("Error reading block meta: %v", err)) } @@ -150,7 +150,7 @@ func (bs *BlockStore) SaveBlock(block *types.Block, blockParts *types.PartSet, s } // Save block meta - meta := makeBlockMeta(block, blockParts) + meta := types.NewBlockMeta(block, blockParts) metaBytes := binary.BinaryBytes(meta) bs.db.Set(calcBlockMetaKey(height), metaBytes) @@ -184,22 +184,6 @@ func (bs *BlockStore) saveBlockPart(height uint, index uint, part *types.Part) { //----------------------------------------------------------------------------- -type BlockMeta struct { - Hash []byte // The block hash - Header *types.Header // The block's Header - Parts types.PartSetHeader // The PartSetHeader, for transfer -} - -func makeBlockMeta(block *types.Block, blockParts *types.PartSet) *BlockMeta { - return &BlockMeta{ - Hash: block.Hash(), - Header: block.Header, - Parts: blockParts.Header(), - } -} - -//----------------------------------------------------------------------------- - func calcBlockMetaKey(height uint) []byte { return []byte(fmt.Sprintf("H:%v", height)) } diff --git a/consensus/reactor.go b/consensus/reactor.go index 6e6f3366..7abd75ec 100644 --- a/consensus/reactor.go +++ b/consensus/reactor.go @@ -9,6 +9,7 @@ import ( "time" "github.com/tendermint/tendermint/binary" + bc "github.com/tendermint/tendermint/blockchain" . "github.com/tendermint/tendermint/common" . "github.com/tendermint/tendermint/consensus/types" "github.com/tendermint/tendermint/p2p" @@ -34,11 +35,11 @@ type ConsensusReactor struct { stopped uint32 quit chan struct{} - blockStore *types.BlockStore + blockStore *bc.BlockStore conS *ConsensusState } -func NewConsensusReactor(consensusState *ConsensusState, blockStore *types.BlockStore) *ConsensusReactor { +func NewConsensusReactor(consensusState *ConsensusState, blockStore *bc.BlockStore) *ConsensusReactor { conR := &ConsensusReactor{ blockStore: blockStore, quit: make(chan struct{}), diff --git a/consensus/state.go b/consensus/state.go index 5fb4268d..ad8b79cf 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -62,6 +62,7 @@ import ( "github.com/tendermint/tendermint/account" "github.com/tendermint/tendermint/binary" + bc "github.com/tendermint/tendermint/blockchain" . "github.com/tendermint/tendermint/common" "github.com/tendermint/tendermint/config" . "github.com/tendermint/tendermint/consensus/types" @@ -234,7 +235,7 @@ type ConsensusState struct { stopped uint32 quit chan struct{} - blockStore *types.BlockStore + blockStore *bc.BlockStore mempoolReactor *mempl.MempoolReactor runActionCh chan RoundAction newStepCh chan *RoundState @@ -247,7 +248,7 @@ type ConsensusState struct { lastCommitVoteHeight uint // Last called commitVoteBlock() or saveCommitVoteBlock() on. } -func NewConsensusState(state *sm.State, blockStore *types.BlockStore, mempoolReactor *mempl.MempoolReactor) *ConsensusState { +func NewConsensusState(state *sm.State, blockStore *bc.BlockStore, mempoolReactor *mempl.MempoolReactor) *ConsensusState { cs := &ConsensusState{ quit: make(chan struct{}), blockStore: blockStore, diff --git a/consensus/test.go b/consensus/test.go index e86c6a07..397befa0 100644 --- a/consensus/test.go +++ b/consensus/test.go @@ -3,15 +3,15 @@ package consensus import ( "sort" + bc "github.com/tendermint/tendermint/blockchain" dbm "github.com/tendermint/tendermint/db" mempl "github.com/tendermint/tendermint/mempool" sm "github.com/tendermint/tendermint/state" - "github.com/tendermint/tendermint/types" ) func randConsensusState() (*ConsensusState, []*sm.PrivValidator) { state, _, privValidators := sm.RandGenesisState(20, false, 1000, 10, false, 1000) - blockStore := types.NewBlockStore(dbm.NewMemDB()) + blockStore := bc.NewBlockStore(dbm.NewMemDB()) mempool := mempl.NewMempool(state) mempoolReactor := mempl.NewMempoolReactor(mempool) cs := NewConsensusState(state, blockStore, mempoolReactor) diff --git a/daemon/daemon.go b/daemon/daemon.go index ccb2932d..89e5de69 100644 --- a/daemon/daemon.go +++ b/daemon/daemon.go @@ -4,6 +4,7 @@ import ( "os" "os/signal" + bc "github.com/tendermint/tendermint/blockchain" . "github.com/tendermint/tendermint/common" "github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/consensus" @@ -12,15 +13,15 @@ import ( "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/rpc" sm "github.com/tendermint/tendermint/state" - "github.com/tendermint/tendermint/types" ) type Node struct { lz []p2p.Listener sw *p2p.Switch book *p2p.AddrBook + blockStore *bc.BlockStore pexReactor *p2p.PEXReactor - blockStore *types.BlockStore + bcReactor *bc.BlockchainReactor mempoolReactor *mempl.MempoolReactor consensusState *consensus.ConsensusState consensusReactor *consensus.ConsensusReactor @@ -30,7 +31,7 @@ type Node struct { func NewNode() *Node { // Get BlockStore blockStoreDB := dbm.GetDB("blockstore") - blockStore := types.NewBlockStore(blockStoreDB) + blockStore := bc.NewBlockStore(blockStoreDB) // Get State stateDB := dbm.GetDB("state") @@ -53,6 +54,9 @@ func NewNode() *Node { book := p2p.NewAddrBook(config.App().GetString("AddrBookFile")) pexReactor := p2p.NewPEXReactor(book) + // Get BlockchainReactor + bcReactor := bc.NewBlockchainReactor(blockStore) + // Get MempoolReactor mempool := mempl.NewMempool(state.Copy()) mempoolReactor := mempl.NewMempoolReactor(mempool) @@ -64,14 +68,19 @@ func NewNode() *Node { consensusReactor.SetPrivValidator(privValidator) } - sw := p2p.NewSwitch([]p2p.Reactor{pexReactor, mempoolReactor, consensusReactor}) + sw := p2p.NewSwitch() sw.SetChainId(state.Hash(), config.App().GetString("Network")) + sw.AddReactor("PEX", pexReactor) + //sw.AddReactor("BLOCKCHAIN", bcReactor) + sw.AddReactor("MEMPOOL", mempoolReactor) + sw.AddReactor("CONSENSUS", consensusReactor) return &Node{ sw: sw, book: book, - pexReactor: pexReactor, blockStore: blockStore, + pexReactor: pexReactor, + bcReactor: bcReactor, mempoolReactor: mempoolReactor, consensusState: consensusState, consensusReactor: consensusReactor, @@ -85,13 +94,13 @@ func (n *Node) Start() { go n.inboundConnectionRoutine(l) } n.book.Start() - n.sw.Start() + n.sw.StartAll() } func (n *Node) Stop() { log.Info("Stopping Node") // TODO: gracefully disconnect from peers. - n.sw.Stop() + n.sw.StopAll() n.book.Stop() } diff --git a/p2p/connection.go b/p2p/connection.go index 578dea1b..0e26480f 100644 --- a/p2p/connection.go +++ b/p2p/connection.go @@ -468,6 +468,7 @@ type Channel struct { } func newChannel(conn *MConnection, desc *ChannelDescriptor) *Channel { + desc.FillDefaults() if desc.Priority <= 0 { panic("Channel default priority must be a postive integer") } diff --git a/p2p/peer_set.go b/p2p/peer_set.go index f365cd8e..23f49c51 100644 --- a/p2p/peer_set.go +++ b/p2p/peer_set.go @@ -7,6 +7,7 @@ import ( // IPeerSet has a (immutable) subset of the methods of PeerSet. type IPeerSet interface { Has(key string) bool + Get(key string) *Peer List() []*Peer Size() int } diff --git a/p2p/switch.go b/p2p/switch.go index 12267fa1..27c61cb4 100644 --- a/p2p/switch.go +++ b/p2p/switch.go @@ -29,89 +29,100 @@ or more `Channels`. So while sending outgoing messages is typically performed o incoming messages are received on the reactor. */ type Switch struct { - reactors []Reactor + chainId string + reactors map[string]Reactor chDescs []*ChannelDescriptor reactorsByCh map[byte]Reactor peers *PeerSet dialing *CMap - listeners *CMap // listenerName -> chan interface{} - quit chan struct{} - started uint32 - stopped uint32 - chainId string + listeners *CMap // listenerName -> chan interface{} + running uint32 // atomic } var ( - ErrSwitchStopped = errors.New("Switch already stopped") ErrSwitchDuplicatePeer = errors.New("Duplicate peer") + ErrSwitchStopped = errors.New("Switch stopped") ) const ( peerDialTimeoutSeconds = 3 ) -func NewSwitch(reactors []Reactor) *Switch { - - // Validate the reactors. no two reactors can share the same channel. - chDescs := []*ChannelDescriptor{} - reactorsByCh := make(map[byte]Reactor) - for _, reactor := range reactors { - reactorChannels := reactor.GetChannels() - for _, chDesc := range reactorChannels { - chId := chDesc.Id - if reactorsByCh[chId] != nil { - panic(fmt.Sprintf("Channel %X has multiple reactors %v & %v", chId, reactorsByCh[chId], reactor)) - } - chDescs = append(chDescs, chDesc) - reactorsByCh[chId] = reactor - } - } +func NewSwitch() *Switch { sw := &Switch{ - reactors: reactors, - chDescs: chDescs, - reactorsByCh: reactorsByCh, + chainId: "", + reactors: make(map[string]Reactor), + chDescs: make([]*ChannelDescriptor, 0), + reactorsByCh: make(map[byte]Reactor), peers: NewPeerSet(), dialing: NewCMap(), listeners: NewCMap(), - quit: make(chan struct{}), - stopped: 0, + running: 0, } return sw } -func (sw *Switch) Start() { - if atomic.CompareAndSwapUint32(&sw.started, 0, 1) { - log.Info("Starting Switch") - for _, reactor := range sw.reactors { - reactor.Start(sw) +func (sw *Switch) SetChainId(hash []byte, network string) { + sw.chainId = hex.EncodeToString(hash) + "-" + network +} + +func (sw *Switch) AddReactor(name string, reactor Reactor) { + // Validate the reactor. + // No two reactors can share the same channel. + reactorChannels := reactor.GetChannels() + for _, chDesc := range reactorChannels { + chId := chDesc.Id + if sw.reactorsByCh[chId] != nil { + panic(fmt.Sprintf("Channel %X has multiple reactors %v & %v", chId, sw.reactorsByCh[chId], reactor)) } + sw.chDescs = append(sw.chDescs, chDesc) + sw.reactorsByCh[chId] = reactor + } + sw.reactors[name] = reactor + time.Sleep(1 * time.Second) +} + +func (sw *Switch) StartReactor(name string) { + atomic.StoreUint32(&sw.running, 1) + sw.reactors[name].Start(sw) +} + +// Convenience function +func (sw *Switch) StartAll() { + atomic.StoreUint32(&sw.running, 1) + for _, reactor := range sw.reactors { + reactor.Start(sw) } } -func (sw *Switch) Stop() { - if atomic.CompareAndSwapUint32(&sw.stopped, 0, 1) { - log.Info("Stopping Switch") - close(sw.quit) - // Stop each peer. - for _, peer := range sw.peers.List() { - peer.stop() - } - sw.peers = NewPeerSet() - // Stop all reactors. - for _, reactor := range sw.reactors { - reactor.Stop() - } +func (sw *Switch) StopReactor(name string) { + sw.reactors[name].Stop() +} + +// Convenience function +// Not goroutine safe +func (sw *Switch) StopAll() { + atomic.StoreUint32(&sw.running, 0) + // Stop each peer. + for _, peer := range sw.peers.List() { + peer.stop() + } + sw.peers = NewPeerSet() + // Stop all reactors. + for _, reactor := range sw.reactors { + reactor.Stop() } } -func (sw *Switch) Reactors() []Reactor { +// Not goroutine safe +func (sw *Switch) Reactors() map[string]Reactor { return sw.reactors } func (sw *Switch) AddPeerWithConnection(conn net.Conn, outbound bool) (*Peer, error) { - if atomic.LoadUint32(&sw.stopped) == 1 { + if atomic.LoadUint32(&sw.running) == 0 { return nil, ErrSwitchStopped } @@ -125,12 +136,12 @@ func (sw *Switch) AddPeerWithConnection(conn net.Conn, outbound bool) (*Peer, er return nil, ErrSwitchDuplicatePeer } - // Start the peer - go peer.start() - // Notify listeners. sw.doAddPeer(peer) + // Start the peer + go peer.start() + // Send handshake msg := &pexHandshakeMessage{ChainId: sw.chainId} peer.Send(PexChannel, msg) @@ -139,7 +150,7 @@ func (sw *Switch) AddPeerWithConnection(conn net.Conn, outbound bool) (*Peer, er } func (sw *Switch) DialPeerWithAddress(addr *NetAddress) (*Peer, error) { - if atomic.LoadUint32(&sw.stopped) == 1 { + if atomic.LoadUint32(&sw.running) == 0 { return nil, ErrSwitchStopped } @@ -168,7 +179,7 @@ func (sw *Switch) IsDialing(addr *NetAddress) bool { // trying to send for defaultSendTimeoutSeconds. Returns a channel // which receives success values for each attempted send (false if times out) func (sw *Switch) Broadcast(chId byte, msg interface{}) chan bool { - if atomic.LoadUint32(&sw.stopped) == 1 { + if atomic.LoadUint32(&sw.running) == 0 { return nil } successChan := make(chan bool, len(sw.peers.List())) @@ -223,16 +234,12 @@ func (sw *Switch) StopPeerGracefully(peer *Peer) { sw.doRemovePeer(peer, nil) } -func (sw *Switch) SetChainId(hash []byte, network string) { - sw.chainId = hex.EncodeToString(hash) + "-" + network -} - func (sw *Switch) IsListening() bool { return sw.listeners.Size() > 0 } func (sw *Switch) doAddPeer(peer *Peer) { - for _, reactor := range sw.reactors { + for name, reactor := range sw.reactors { reactor.AddPeer(peer) } } diff --git a/rpc/rpc.go b/rpc/rpc.go index 94631249..8cf905d3 100644 --- a/rpc/rpc.go +++ b/rpc/rpc.go @@ -1,18 +1,18 @@ package rpc import ( + bc "github.com/tendermint/tendermint/blockchain" "github.com/tendermint/tendermint/consensus" mempl "github.com/tendermint/tendermint/mempool" "github.com/tendermint/tendermint/p2p" - "github.com/tendermint/tendermint/types" ) -var blockStore *types.BlockStore +var blockStore *bc.BlockStore var consensusState *consensus.ConsensusState var mempoolReactor *mempl.MempoolReactor var p2pSwitch *p2p.Switch -func SetRPCBlockStore(bs *types.BlockStore) { +func SetRPCBlockStore(bs *bc.BlockStore) { blockStore = bs } diff --git a/state/state_test.go b/state/state_test.go index 71efd80f..0d2c963b 100644 --- a/state/state_test.go +++ b/state/state_test.go @@ -2,7 +2,6 @@ package state import ( "github.com/tendermint/tendermint/account" - "github.com/tendermint/tendermint/binary" "github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/types" diff --git a/types/block_meta.go b/types/block_meta.go new file mode 100644 index 00000000..3e9ba8f9 --- /dev/null +++ b/types/block_meta.go @@ -0,0 +1,15 @@ +package types + +type BlockMeta struct { + Hash []byte // The block hash + Header *Header // The block's Header + Parts PartSetHeader // The PartSetHeader, for transfer +} + +func NewBlockMeta(block *Block, blockParts *PartSet) *BlockMeta { + return &BlockMeta{ + Hash: block.Hash(), + Header: block.Header, + Parts: blockParts.Header(), + } +}