mirror of
https://github.com/fluencelabs/tendermint
synced 2025-05-29 06:01:21 +00:00
Reactors can be stopped or started at any time.
This commit is contained in:
parent
612f8bab9d
commit
08a83aa9fb
@ -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++
|
||||
|
226
blockchain/reactor.go
Normal file
226
blockchain/reactor.go
Normal file
@ -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)
|
||||
}
|
@ -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))
|
||||
}
|
||||
|
@ -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{}),
|
||||
|
@ -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,
|
||||
|
@ -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)
|
||||
|
@ -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()
|
||||
}
|
||||
|
||||
|
@ -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")
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
103
p2p/switch.go
103
p2p/switch.go
@ -29,71 +29,82 @@ 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
|
||||
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")
|
||||
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)
|
||||
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()
|
||||
@ -103,15 +114,15 @@ func (sw *Switch) Stop() {
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
||||
|
@ -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"
|
||||
|
||||
|
15
types/block_meta.go
Normal file
15
types/block_meta.go
Normal file
@ -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(),
|
||||
}
|
||||
}
|
Loading…
x
Reference in New Issue
Block a user