Merge pull request #291 from meshplus/refactor-mempool

refactor(mempool): refactor mempool for raft
This commit is contained in:
Aiden X 2020-12-21 15:46:09 +08:00 committed by GitHub
commit 1a4bf5f46a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
41 changed files with 1408 additions and 2852 deletions

View File

@ -15,7 +15,6 @@ disable_proposal_forwarding = true # This prevents blocks from being accidentall
batch_tick = "0.3s" # Block packaging time period.
tx_slice_timeout = "0.1s" # Node broadcasts transactions if there are cached transactions, although set_size isn't reached yet
fetch_timeout = "3s" # How long to wait before fetching missing transactions finished
[rbft] #RBFT configurations

4
go.sum
View File

@ -605,6 +605,10 @@ github.com/meshplus/bitxhub-kit v1.1.2-0.20201203072410-8a0383a6870d h1:J9tzTNf2
github.com/meshplus/bitxhub-kit v1.1.2-0.20201203072410-8a0383a6870d/go.mod h1:KR7ZlXhII9n0Bu8viaZTScvXCYn0MCQnYlsTvHPp0XA=
github.com/meshplus/bitxhub-model v1.0.0-rc3/go.mod h1:ZCctQIYTlE3vJ8Lhkrgs9bWwNA+Dw4JzojOSIzLVU6E=
github.com/meshplus/bitxhub-model v1.1.2-0.20201021152621-0b3c17c54b23/go.mod h1:4qWBZx5wv7WZzUqiuBsbkQqQ2Ju8aOFpsoNpBBNy8Us=
github.com/meshplus/bitxhub-model v1.1.2-0.20201118055706-510eb971b4c6 h1:M2j5n9XRgNc9baS6JEfpRW+ygKiKqorUVfvjpdlOld0=
github.com/meshplus/bitxhub-model v1.1.2-0.20201118055706-510eb971b4c6/go.mod h1:4qWBZx5wv7WZzUqiuBsbkQqQ2Ju8aOFpsoNpBBNy8Us=
github.com/meshplus/bitxhub-model v1.1.2-0.20201209072914-6846fa78ff35 h1:fuY1VlVKUa58108lODiQpsVypGiuYjPlEawx6xd6uyA=
github.com/meshplus/bitxhub-model v1.1.2-0.20201209072914-6846fa78ff35/go.mod h1:sk7glP/0M9G9On4SN6rMPaLGqet8Uu35wA65Mxc3Cms=
github.com/meshplus/bitxhub-model v1.1.2-0.20201218090311-9e471bb1654c h1:xVrWp1PlY9kxUQkb8Go/MKbm0PhYGi4yo+YBUDkGx0A=
github.com/meshplus/bitxhub-model v1.1.2-0.20201218090311-9e471bb1654c/go.mod h1:HpL39K8fuh3nDu+HuprYuDcS/C3eFOQBvFkd1bPViwA=
github.com/meshplus/go-lightp2p v0.0.0-20201203044909-e09b34cd93ab h1:JclTakVV0dcXxl/dScmN77htnYe3n19hh7m2eMk9Abs=

View File

@ -57,13 +57,12 @@ func NewBitXHub(rep *repo.Repo) (*BitXHub, error) {
order.WithPluginPath(rep.Config.Plugin),
order.WithNodes(m),
order.WithID(rep.NetworkConfig.ID),
order.WithIsNew(rep.NetworkConfig.IsNew),
order.WithIsNew(rep.NetworkConfig.New),
order.WithPeerManager(bxh.PeerMgr),
order.WithLogger(loggers.Logger(loggers.Order)),
order.WithApplied(chainMeta.Height),
order.WithDigest(chainMeta.BlockHash.String()),
order.WithGetChainMetaFunc(bxh.Ledger.GetChainMeta),
order.WithGetTransactionFunc(bxh.Ledger.GetTransaction),
order.WithGetBlockByHeightFunc(bxh.Ledger.GetBlock),
)
if err != nil {

View File

@ -1,8 +1,6 @@
package app
import (
"context"
"github.com/meshplus/bitxhub/internal/model/events"
"github.com/sirupsen/logrus"
)
@ -38,11 +36,11 @@ func (bxh *BitXHub) listenEvent() {
for {
select {
case ev := <-blockCh:
go bxh.Order.ReportState(ev.Block.BlockHeader.Number, ev.Block.BlockHash)
go bxh.Order.ReportState(ev.Block.BlockHeader.Number, ev.Block.BlockHash, ev.TxHashList)
go bxh.Router.PutBlockAndMeta(ev.Block, ev.InterchainMeta)
case ev := <-orderMsgCh:
go func() {
if err := bxh.Order.Step(context.Background(), ev.Data); err != nil {
if err := bxh.Order.Step(ev.Data); err != nil {
bxh.logger.Error(err)
}
}()

View File

@ -22,7 +22,7 @@ import (
type NetworkConfig struct {
ID uint64 `toml:"id" json:"id"`
N uint64 `toml:"n" json:"n"`
IsNew bool `toml:"new" json:"new"`
New bool `toml:"new" json:"new"`
LocalAddr string `toml:"local_addr, omitempty" json:"local_addr"`
Nodes []*NetworkNodes `toml:"nodes" json:"nodes"`
Genesis Genesis `toml:"genesis, omitempty" json:"genesis"`
@ -164,7 +164,7 @@ func RewriteNetworkConfig(repoRoot string, infos map[uint64]*pb.VpInfo, isNew bo
}
networkConfig.Nodes = nodes
networkConfig.N = uint64(len(nodes))
networkConfig.IsNew = isNew
networkConfig.New = isNew
data, err := toml.Marshal(*networkConfig)
if err != nil {
return err

View File

@ -4,7 +4,6 @@ import (
"fmt"
"github.com/meshplus/bitxhub-kit/crypto"
"github.com/meshplus/bitxhub-kit/types"
"github.com/meshplus/bitxhub-model/pb"
"github.com/meshplus/bitxhub/pkg/peermgr"
"github.com/sirupsen/logrus"
@ -22,7 +21,6 @@ type Config struct {
Nodes map[uint64]*pb.VpInfo
Applied uint64
Digest string
GetTransactionFunc func(hash *types.Hash) (*pb.Transaction, error)
GetChainMetaFunc func() *pb.ChainMeta
GetBlockByHeight func(height uint64) (*pb.Block, error)
}
@ -101,12 +99,6 @@ func WithGetChainMetaFunc(f func() *pb.ChainMeta) Option {
}
}
func WithGetTransactionFunc(f func(hash *types.Hash) (*pb.Transaction, error)) Option {
return func(config *Config) {
config.GetTransactionFunc = f
}
}
func WithGetBlockByHeightFunc(f func(height uint64) (*pb.Block, error)) Option {
return func(config *Config) {
config.GetBlockByHeight = f

View File

@ -19,7 +19,6 @@ type MempoolConfig struct {
TxSliceSize uint64 `mapstructure:"tx_slice_size"`
BatchTick time.Duration `mapstructure:"batch_tick"`
FetchTimeout time.Duration `mapstructure:"fetch_timeout"`
TxSliceTimeout time.Duration `mapstructure:"tx_slice_timeout"`
}
@ -84,7 +83,6 @@ func generateMempoolConfig(repoRoot string) (*MempoolConfig, error) {
mempoolConf.PoolSize = readConfig.RAFT.MempoolConfig.PoolSize
mempoolConf.TxSliceSize = readConfig.RAFT.MempoolConfig.TxSliceSize
mempoolConf.BatchTick = readConfig.RAFT.MempoolConfig.BatchTick
mempoolConf.FetchTimeout = readConfig.RAFT.MempoolConfig.FetchTimeout
mempoolConf.TxSliceTimeout = readConfig.RAFT.MempoolConfig.TxSliceTimeout
return mempoolConf, nil
}

View File

@ -2,7 +2,6 @@ package etcdraft
import (
"context"
"encoding/binary"
"errors"
"fmt"
"path/filepath"
@ -12,7 +11,6 @@ import (
"github.com/coreos/etcd/raft"
"github.com/coreos/etcd/raft/raftpb"
"github.com/gogo/protobuf/sortkeys"
"github.com/meshplus/bitxhub-kit/storage"
"github.com/meshplus/bitxhub-kit/types"
"github.com/meshplus/bitxhub-model/pb"
@ -26,42 +24,39 @@ import (
var defaultSnapshotCount uint64 = 10000
type Node struct {
id uint64 // raft id
leader uint64 // leader id
node raft.Node // raft node
peerMgr peermgr.PeerManager // network manager
peers []raft.Peer // raft peers
proposeC chan *raftproto.Ready // proposed ready, input channel
confChangeC <-chan raftpb.ConfChange // proposed cluster config changes
confState raftpb.ConfState // raft requires ConfState to be persisted within snapshot
commitC chan *pb.Block // the hash commit channel
errorC chan<- error // errors from raft session
tickTimeout time.Duration // tick timeout
raftStorage *RaftStorage // the raft backend storage system
storage storage.Storage // db
mempool mempool.MemPool // transaction pool
id uint64 // raft id
leader uint64 // leader id
repoRoot string // project path
logger logrus.FieldLogger // logger
blockAppliedIndex sync.Map // mapping of block height and apply index in raft log
appliedIndex uint64 // current apply index in raft log
snapCount uint64 // snapshot count
snapshotIndex uint64 // current snapshot apply index in raft log
lastIndex uint64 // last apply index in raft log
node raft.Node // raft node
peerMgr peermgr.PeerManager // network manager
peers []raft.Peer // raft peers
raftStorage *RaftStorage // the raft backend storage system
storage storage.Storage // db
mempool mempool.MemPool // transaction pool
txCache *mempool.TxCache // cache the transactions received from api
batchTimerMgr *batchTimer
readyPool *sync.Pool // ready pool, avoiding memory growth fast
readyCache sync.Map // ready cache
justElected bool
isRestart bool
// TODO (YH): refactor
ctx context.Context // context
haltC chan struct{} // exit signal
proposeC chan *raftproto.RequestBatch // proposed ready, input channel
confChangeC <-chan raftpb.ConfChange // proposed cluster config changes
commitC chan *pb.Block // the hash commit channel
errorC chan<- error // errors from raft session
tickTimeout time.Duration // tick timeout
msgC chan []byte // receive messages from remote peer
stateC chan *mempool.ChainState // receive the executed block state
confState raftpb.ConfState // raft requires ConfState to be persisted within snapshot
blockAppliedIndex sync.Map // mapping of block height and apply index in raft log
appliedIndex uint64 // current apply index in raft log
snapCount uint64 // snapshot count
snapshotIndex uint64 // current snapshot apply index in raft log
lastIndex uint64 // last apply index in raft log
lastExec uint64 // the index of the last-applied block
readyPool *sync.Pool // ready pool, avoiding memory growth fast
justElected bool // track new leader status
ctx context.Context // context
haltC chan struct{} // exit signal
}
// NewNode new raft node
@ -83,72 +78,85 @@ func NewNode(opts ...order.Option) (order.Order, error) {
}
// generate raft peers
peers, err := GenerateRaftPeers(config)
peers, err := generateRaftPeers(config)
if err != nil {
return nil, fmt.Errorf("generate raft peers: %w", err)
}
batchC := make(chan *raftproto.Ready)
memConfig, err := generateMempoolConfig(repoRoot)
if err != nil {
return nil, fmt.Errorf("generate raft txpool config: %w", err)
}
mempoolConf := &mempool.Config{
ID: config.ID,
PeerMgr: config.PeerMgr,
ChainHeight: config.Applied,
GetTransactionFunc: config.GetTransactionFunc,
Logger: config.Logger,
ID: config.ID,
ChainHeight: config.Applied,
Logger: config.Logger,
BatchSize: memConfig.BatchSize,
BatchTick: memConfig.BatchTick,
PoolSize: memConfig.PoolSize,
TxSliceSize: memConfig.TxSliceSize,
FetchTimeout: memConfig.FetchTimeout,
TxSliceTimeout: memConfig.TxSliceTimeout,
}
mempoolInst := mempool.NewMempool(mempoolConf, dbStorage, batchC)
mempoolInst := mempool.NewMempool(mempoolConf)
var batchTick time.Duration
if mempoolConf.BatchTick == 0 {
batchTick = DefaultBatchTick
} else {
batchTick = mempoolConf.BatchTick
}
batchTimerMgr := newTimer(batchTick, config.Logger)
txCache := mempool.NewTxCache(mempoolConf.TxSliceTimeout, mempoolConf.TxSliceSize, config.Logger)
readyPool := &sync.Pool{New: func() interface{} {
return new(raftproto.Ready)
return new(raftproto.RequestBatch)
}}
return &Node{
id: config.ID,
proposeC: batchC,
confChangeC: make(chan raftpb.ConfChange),
commitC: make(chan *pb.Block, 1024),
errorC: make(chan<- error),
repoRoot: repoRoot,
snapCount: defaultSnapshotCount,
peerMgr: config.PeerMgr,
peers: peers,
logger: config.Logger,
storage: dbStorage,
raftStorage: raftStorage,
readyPool: readyPool,
ctx: context.Background(),
mempool: mempoolInst,
}, nil
node := &Node{
id: config.ID,
lastExec: config.Applied,
confChangeC: make(chan raftpb.ConfChange),
commitC: make(chan *pb.Block, 1024),
errorC: make(chan<- error),
msgC: make(chan []byte),
stateC: make(chan *mempool.ChainState),
proposeC: make(chan *raftproto.RequestBatch),
repoRoot: repoRoot,
snapCount: defaultSnapshotCount,
peerMgr: config.PeerMgr,
txCache: txCache,
batchTimerMgr: batchTimerMgr,
peers: peers,
logger: config.Logger,
storage: dbStorage,
raftStorage: raftStorage,
readyPool: readyPool,
ctx: context.Background(),
mempool: mempoolInst,
}
node.logger.Infof("========= Raft localID = %d", node.id)
node.logger.Infof("========= Raft lastExec = %d ", node.lastExec)
node.logger.Infof("========= Raft defaultSnapshotCount = %d", node.snapCount)
return node, nil
}
// Start or restart raft node
func (n *Node) Start() error {
n.blockAppliedIndex.Store(n.mempool.GetChainHeight(), n.loadAppliedIndex())
n.blockAppliedIndex.Store(n.lastExec, n.loadAppliedIndex())
rc, tickTimeout, err := generateRaftConfig(n.id, n.repoRoot, n.logger, n.raftStorage.ram)
if err != nil {
return fmt.Errorf("generate raft config: %w", err)
}
if restart {
n.node = raft.RestartNode(rc)
n.isRestart = true
} else {
n.node = raft.StartNode(rc, n.peers)
}
n.tickTimeout = tickTimeout
go n.run()
n.mempool.Start()
go n.txCache.ListenEvent()
n.logger.Info("Consensus module started")
return nil
@ -156,7 +164,6 @@ func (n *Node) Start() error {
// Stop the raft node
func (n *Node) Stop() {
n.mempool.Stop()
n.node.Stop()
n.logger.Infof("Consensus stopped")
}
@ -166,90 +173,35 @@ func (n *Node) Prepare(tx *pb.Transaction) error {
if err := n.Ready(); err != nil {
return err
}
return n.mempool.RecvTransaction(tx)
if n.txCache.IsFull() && n.mempool.IsPoolFull() {
return errors.New("transaction cache are full, we will drop this transaction")
}
n.txCache.RecvTxC <- tx
return nil
}
func (n *Node) Commit() chan *pb.Block {
return n.commitC
}
func (n *Node) ReportState(height uint64, hash *types.Hash) {
if height%10 == 0 {
n.logger.WithFields(logrus.Fields{
"height": height,
}).Info("Report checkpoint")
func (n *Node) ReportState(height uint64, blockHash *types.Hash, txHashList []*types.Hash) {
state := &mempool.ChainState{
Height: height,
BlockHash: blockHash,
TxHashList: txHashList,
}
appliedIndex, ok := n.blockAppliedIndex.Load(height)
if !ok {
n.logger.Debugf("can not found appliedIndex:", height)
return
}
// block already persisted, record the apply index in db
n.writeAppliedIndex(appliedIndex.(uint64))
n.blockAppliedIndex.Delete(height - 1)
// TODO: delete readyCache
readyBytes, ok := n.readyCache.Load(height)
if !ok {
n.logger.Debugf("can not found ready:", height)
return
}
ready := readyBytes.(*raftproto.Ready)
// clean related mempool info
n.mempool.CommitTransactions(ready)
n.readyCache.Delete(height)
n.stateC <- state
}
func (n *Node) Quorum() uint64 {
return uint64(len(n.peers)/2 + 1)
}
func (n *Node) Step(ctx context.Context, msg []byte) error {
rm := &raftproto.RaftMessage{}
if err := rm.Unmarshal(msg); err != nil {
return err
}
switch rm.Type {
case raftproto.RaftMessage_CONSENSUS:
msg := &raftpb.Message{}
if err := msg.Unmarshal(rm.Data); err != nil {
return err
}
return n.node.Step(ctx, *msg)
case raftproto.RaftMessage_GET_TX:
fetchTxnRequest := &mempool.FetchTxnRequest{}
if err := fetchTxnRequest.Unmarshal(rm.Data); err != nil {
return err
}
n.mempool.RecvFetchTxnRequest(fetchTxnRequest)
case raftproto.RaftMessage_GET_TX_ACK:
fetchTxnResponse := &mempool.FetchTxnResponse{}
if err := fetchTxnResponse.Unmarshal(rm.Data); err != nil {
return err
}
n.mempool.RecvFetchTxnResponse(fetchTxnResponse)
case raftproto.RaftMessage_BROADCAST_TX:
txSlice := &mempool.TxSlice{}
if err := txSlice.Unmarshal(rm.Data); err != nil {
return err
}
n.mempool.RecvForwardTxs(txSlice)
default:
return fmt.Errorf("unexpected raft message received")
}
func (n *Node) Step(msg []byte) error {
n.msgC <- msg
return nil
}
func (n *Node) IsLeader() bool {
return n.leader == n.id
}
func (n *Node) Ready() error {
hasLeader := n.leader != 0
if !hasLeader {
@ -258,6 +210,15 @@ func (n *Node) Ready() error {
return nil
}
func (n *Node) GetPendingNonceByAccount(account string) uint64 {
return n.mempool.GetPendingNonceByAccount(account)
}
// DelNode sends a delete vp request by given id.
func (n *Node) DelNode(delID uint64) error {
return nil
}
// main work loop
func (n *Node) run() {
snap, err := n.raftStorage.ram.Snapshot()
@ -279,18 +240,14 @@ func (n *Node) run() {
for n.proposeC != nil && n.confChangeC != nil {
select {
case ready, ok := <-n.proposeC:
if !ok {
n.proposeC = nil
} else {
data, err := ready.Marshal()
if err != nil {
n.logger.Panic(err)
}
n.logger.Debugf("Proposed block %d to raft core consensus", ready.Height)
if err := n.node.Propose(n.ctx, data); err != nil {
n.logger.Errorf("Failed to propose block [%d] to raft: %s", ready.Height, err)
}
case batch := <-n.proposeC:
data, err := batch.Marshal()
if err != nil {
n.logger.Error("Marshal batch failed")
}
n.logger.Debugf("Proposed block %d to raft core consensus", batch.Height)
if err := n.node.Propose(n.ctx, data); err != nil {
n.logger.Errorf("Failed to propose block [%d] to raft: %s", batch.Height, err)
}
case cc, ok := <-n.confChangeC:
if !ok {
@ -314,8 +271,46 @@ func (n *Node) run() {
select {
case <-ticker.C:
n.node.Tick()
// when the node is first ready it gives us entries to commit and messages
// to immediately publish
case msg := <-n.msgC:
if err := n.processRaftCoreMsg(msg); err != nil {
n.logger.Errorf("Process consensus message failed, err: %s", err.Error())
}
case txSet := <-n.txCache.TxSetC:
// 1. send transactions to other peer
data, err := txSet.Marshal()
if err != nil {
n.logger.Errorf("Marshal failed, err: %s", err.Error())
return
}
pbMsg := msgToConsensusPbMsg(data, raftproto.RaftMessage_BROADCAST_TX, n.id)
_ = n.peerMgr.Broadcast(pbMsg)
// 2. process transactions
n.processTransactions(txSet.TxList)
case state := <-n.stateC:
n.reportState(state)
case <-n.batchTimerMgr.timeoutEventC:
n.batchTimerMgr.stopBatchTimer()
// call txPool module to generate a tx batch
if n.isLeader() {
n.logger.Debug("Leader batch timer expired, try to create a batch")
if n.mempool.HasPendingRequest() {
if batch := n.mempool.GenerateBlock(); batch != nil {
n.postProposal(batch)
}
} else {
n.logger.Debug("The length of priorityIndex is 0, skip the batch timer")
}
} else {
n.logger.Warningf("Replica %d try to generate batch, but the leader is %d", n.id, n.leader)
}
// when the node is first ready it gives us entries to commit and messages
// to immediately publish
case rd := <-n.node.Ready():
// 1: Write HardState, Entries, and Snapshot to persistent storage if they
// are not empty.
@ -326,22 +321,16 @@ func (n *Node) run() {
if rd.SoftState != nil {
newLeader := atomic.LoadUint64(&rd.SoftState.Lead)
if newLeader != n.leader {
n.logger.Infof("Raft leader changed: %d -> %d", n.leader, newLeader)
oldLeader := n.leader
n.leader = newLeader
// new leader should not serve requests directly.
if newLeader == n.id {
// If the cluster is started for the first time, the leader node starts listening requests directly.
if !n.isRestart && n.getBlockAppliedIndex() == uint64(0) {
n.mempool.UpdateLeader(n.leader)
} else {
// new leader should not serve requests
n.justElected = true
}
n.justElected = true
}
// old leader node stop batch block
if oldLeader == n.id {
n.mempool.UpdateLeader(n.leader)
// notify old leader to stop batching
if n.leader == n.id {
n.becomeFollower()
}
n.logger.Infof("Raft leader changed: %d -> %d", n.leader, newLeader)
n.leader = newLeader
}
}
// 2: Apply Snapshot (if any) and CommittedEntries to the state machine.
@ -355,10 +344,9 @@ func (n *Node) run() {
if n.justElected {
msgInflight := n.ramLastIndex() > n.appliedIndex+1
if msgInflight {
n.logger.Debugf("There are in flight blocks, new leader should not serve requests")
n.logger.Debug("There are in flight blocks, new leader should not generate new batches")
} else {
n.justElected = false
n.mempool.UpdateLeader(n.leader)
}
}
@ -375,10 +363,186 @@ func (n *Node) run() {
}
}
func (n *Node) ramLastIndex() uint64 {
i, _ := n.raftStorage.ram.LastIndex()
n.logger.Infof("New Leader's last index is %d, appliedIndex is %d", i, n.appliedIndex)
return i
func (n *Node) processTransactions(txList []*pb.Transaction) {
// leader node would check if this transaction triggered generating a batch or not
if n.isLeader() {
// start batch timer when this node receives the first transaction
if !n.batchTimerMgr.isBatchTimerActive() {
n.batchTimerMgr.startBatchTimer()
}
// If this transaction triggers generating a batch, stop batch timer
if batch := n.mempool.ProcessTransactions(txList, true); batch != nil {
n.batchTimerMgr.stopBatchTimer()
n.postProposal(batch)
}
} else {
n.mempool.ProcessTransactions(txList, false)
}
}
func (n *Node) publishEntries(ents []raftpb.Entry) bool {
for i := range ents {
switch ents[i].Type {
case raftpb.EntryNormal:
if len(ents[i].Data) == 0 {
// ignore empty messages
break
}
// This can happen:
//
// if (1) we crashed after applying this block to the chain, but
// before writing appliedIndex to LDB.
// or (2) we crashed in a scenario where we applied further than
// raft *durably persisted* its committed index (see
// https://github.com/coreos/etcd/pull/7899). In this
// scenario, when the node comes back up, we will re-apply
// a few entries.
blockAppliedIndex := n.getBlockAppliedIndex()
if blockAppliedIndex >= ents[i].Index {
n.appliedIndex = ents[i].Index
continue
}
requestBatch := n.readyPool.Get().(*raftproto.RequestBatch)
if err := requestBatch.Unmarshal(ents[i].Data); err != nil {
n.logger.Error(err)
continue
}
// strictly avoid writing the same block
if requestBatch.Height != n.lastExec+1 {
n.logger.Warningf("Replica %d expects to execute seq=%d, but get seq=%d, ignore it",
n.id, n.lastExec+1, requestBatch.Height)
continue
}
n.mint(requestBatch)
n.blockAppliedIndex.Store(requestBatch.Height, ents[i].Index)
n.setLastExec(requestBatch.Height)
// update followers' batch sequence number
if !n.isLeader() {
n.mempool.SetBatchSeqNo(requestBatch.Height)
}
case raftpb.EntryConfChange:
var cc raftpb.ConfChange
if err := cc.Unmarshal(ents[i].Data); err != nil {
continue
}
n.confState = *n.node.ApplyConfChange(cc)
switch cc.Type {
case raftpb.ConfChangeAddNode:
//if len(cc.Context) > 0 {
// _ := types.Bytes2Address(cc.Context)
//}
case raftpb.ConfChangeRemoveNode:
//if cc.NodeID == n.id {
// n.logger.Infoln("I've been removed from the cluster! Shutting down.")
// continue
//}
}
}
// after commit, update appliedIndex
n.appliedIndex = ents[i].Index
// special nil commit to signal replay has finished
if ents[i].Index == n.lastIndex {
select {
case n.commitC <- nil:
case <-n.haltC:
return false
}
}
}
return true
}
// mint the block
func (n *Node) mint(requestBatch *raftproto.RequestBatch) {
n.logger.WithFields(logrus.Fields{
"height": requestBatch.Height,
"count": len(requestBatch.TxList),
}).Debugln("block will be mint")
n.logger.Infof("======== Replica %d call execute, height=%d", n.id, requestBatch.Height)
block := &pb.Block{
BlockHeader: &pb.BlockHeader{
Version: []byte("1.0.0"),
Number: requestBatch.Height,
Timestamp: time.Now().UnixNano(),
},
Transactions: requestBatch.TxList,
}
n.commitC <- block
}
//Determines whether the current apply index triggers a snapshot
func (n *Node) maybeTriggerSnapshot() {
if n.appliedIndex-n.snapshotIndex <= n.snapCount {
return
}
data := n.raftStorage.Snapshot().Data
n.logger.Infof("Start snapshot [applied index: %d | last snapshot index: %d]", n.appliedIndex, n.snapshotIndex)
snap, err := n.raftStorage.ram.CreateSnapshot(n.appliedIndex, &n.confState, data)
if err != nil {
panic(err)
}
if err := n.raftStorage.saveSnap(snap); err != nil {
panic(err)
}
compactIndex := uint64(1)
if n.appliedIndex > n.raftStorage.SnapshotCatchUpEntries {
compactIndex = n.appliedIndex - n.raftStorage.SnapshotCatchUpEntries
}
if err := n.raftStorage.ram.Compact(compactIndex); err != nil {
panic(err)
}
n.logger.Infof("compacted log at index %d", compactIndex)
n.snapshotIndex = n.appliedIndex
}
func (n *Node) reportState(state *mempool.ChainState) {
height := state.Height
if height%10 == 0 {
n.logger.WithFields(logrus.Fields{
"height": height,
}).Info("Report checkpoint")
}
appliedIndex, ok := n.blockAppliedIndex.Load(height)
if !ok {
n.logger.Debugf("can not found appliedIndex:", height)
return
}
// block already persisted, record the apply index in db
n.writeAppliedIndex(appliedIndex.(uint64))
n.blockAppliedIndex.Delete(height - 1)
n.mempool.CommitTransactions(state)
}
func (n *Node) processRaftCoreMsg(msg []byte) error {
rm := &raftproto.RaftMessage{}
if err := rm.Unmarshal(msg); err != nil {
return err
}
switch rm.Type {
case raftproto.RaftMessage_CONSENSUS:
msg := &raftpb.Message{}
if err := msg.Unmarshal(rm.Data); err != nil {
return err
}
return n.node.Step(context.Background(), *msg)
case raftproto.RaftMessage_BROADCAST_TX:
txSlice := &raftproto.TxSlice{}
if err := txSlice.Unmarshal(rm.Data); err != nil {
return err
}
n.processTransactions(txSlice.TxList)
default:
return fmt.Errorf("unexpected raft message received")
}
return nil
}
// send raft consensus message
@ -429,242 +593,16 @@ func (n *Node) send(messages []raftpb.Message) {
}
}
func (n *Node) publishEntries(ents []raftpb.Entry) bool {
for i := range ents {
switch ents[i].Type {
case raftpb.EntryNormal:
if len(ents[i].Data) == 0 {
// ignore empty messages
break
}
// This can happen:
//
// if (1) we crashed after applying this block to the chain, but
// before writing appliedIndex to LDB.
// or (2) we crashed in a scenario where we applied further than
// raft *durably persisted* its committed index (see
// https://github.com/coreos/etcd/pull/7899). In this
// scenario, when the node comes back up, we will re-apply
// a few entries.
blockAppliedIndex := n.getBlockAppliedIndex()
if blockAppliedIndex >= ents[i].Index {
n.appliedIndex = ents[i].Index
continue
}
ready := n.readyPool.Get().(*raftproto.Ready)
if err := ready.Unmarshal(ents[i].Data); err != nil {
n.logger.Error(err)
continue
}
n.mint(ready)
n.blockAppliedIndex.Store(ready.Height, ents[i].Index)
case raftpb.EntryConfChange:
var cc raftpb.ConfChange
if err := cc.Unmarshal(ents[i].Data); err != nil {
continue
}
n.confState = *n.node.ApplyConfChange(cc)
switch cc.Type {
case raftpb.ConfChangeAddNode:
//if len(cc.Context) > 0 {
// _ := types.Bytes2Address(cc.Context)
//}
case raftpb.ConfChangeRemoveNode:
//if cc.NodeID == n.id {
// n.logger.Infoln("I've been removed from the cluster! Shutting down.")
// continue
//}
}
}
// after commit, update appliedIndex
n.appliedIndex = ents[i].Index
// special nil commit to signal replay has finished
if ents[i].Index == n.lastIndex {
select {
case n.commitC <- nil:
case <-n.haltC:
return false
}
}
}
return true
func (n *Node) postProposal(batch *raftproto.RequestBatch) {
n.proposeC <- batch
n.batchTimerMgr.startBatchTimer()
}
// mint the block
func (n *Node) mint(ready *raftproto.Ready) {
n.logger.WithFields(logrus.Fields{
"height": ready.Height,
"count": len(ready.TxHashes),
}).Debugln("block will be generated")
// follower node update the block height
expectHeight := n.mempool.GetChainHeight()
isLeader := n.IsLeader()
if !isLeader && expectHeight != ready.Height-1 {
n.logger.Warningf("Receive batch %d, but not match, expect height: %d", ready.Height, expectHeight+1)
return
}
missingTxsHash, txList := n.mempool.GetBlockByHashList(ready)
// handle missing txs
if len(missingTxsHash) != 0 {
waitLostTxnC := make(chan bool)
lostTxnEvent := &mempool.LocalMissingTxnEvent{
Height: ready.Height,
WaitC: waitLostTxnC,
MissingTxnHashList: missingTxsHash,
}
// NOTE!!! block until finishing fetching the missing txs
n.mempool.FetchTxn(lostTxnEvent)
select {
case isSuccess := <-waitLostTxnC:
if !isSuccess {
n.logger.Error("Fetch missing txn failed")
return
}
n.logger.Debug("Fetch missing transactions success")
case <-time.After(mempool.DefaultFetchTxnTimeout):
// TODO: add fetch request resend timer
n.logger.Debugf("Fetch missing transactions timeout, block height: %d", ready.Height)
return
}
if missingTxsHash, txList = n.mempool.GetBlockByHashList(ready); len(missingTxsHash) != 0 {
n.logger.Error("Still missing transaction")
return
}
}
if !isLeader {
n.mempool.IncreaseChainHeight()
}
block := &pb.Block{
BlockHeader: &pb.BlockHeader{
Version: []byte("1.0.0"),
Number: ready.Height,
Timestamp: time.Now().UnixNano(),
},
Transactions: txList,
}
n.readyCache.Store(ready.Height, ready)
n.commitC <- block
func (n *Node) becomeFollower() {
n.logger.Debugf("Replica %d became follower", n.id)
n.batchTimerMgr.stopBatchTimer()
}
//Determine whether the current apply index is normal
func (n *Node) entriesToApply(allEntries []raftpb.Entry) (entriesToApply []raftpb.Entry) {
if len(allEntries) == 0 {
return
}
firstIdx := allEntries[0].Index
if firstIdx > n.appliedIndex+1 {
n.logger.Fatalf("first index of committed entry[%d] should <= progress.appliedIndex[%d]+1", firstIdx, n.appliedIndex)
}
if n.appliedIndex-firstIdx+1 < uint64(len(allEntries)) {
entriesToApply = allEntries[n.appliedIndex-firstIdx+1:]
}
return entriesToApply
}
//Determines whether the current apply index triggers a snapshot
func (n *Node) maybeTriggerSnapshot() {
if n.appliedIndex-n.snapshotIndex <= n.snapCount {
return
}
data := n.raftStorage.Snapshot().Data
n.logger.Infof("Start snapshot [applied index: %d | last snapshot index: %d]", n.appliedIndex, n.snapshotIndex)
snap, err := n.raftStorage.ram.CreateSnapshot(n.appliedIndex, &n.confState, data)
if err != nil {
panic(err)
}
if err := n.raftStorage.saveSnap(snap); err != nil {
panic(err)
}
compactIndex := uint64(1)
if n.appliedIndex > n.raftStorage.SnapshotCatchUpEntries {
compactIndex = n.appliedIndex - n.raftStorage.SnapshotCatchUpEntries
}
if err := n.raftStorage.ram.Compact(compactIndex); err != nil {
panic(err)
}
n.logger.Infof("compacted log at index %d", compactIndex)
n.snapshotIndex = n.appliedIndex
}
func GenerateRaftPeers(config *order.Config) ([]raft.Peer, error) {
nodes := config.Nodes
peers := make([]raft.Peer, 0, len(nodes))
// sort by node id
idSlice := make([]uint64, len(nodes))
i := 0
for id := range nodes {
idSlice[i] = id
i++
}
sortkeys.Uint64s(idSlice)
for _, id := range idSlice {
addr := nodes[id]
addBytes,err := addr.Marshal()
if err != nil {
return nil, err
}
peers = append(peers, raft.Peer{ID: id, Context: addBytes})
}
return peers, nil
}
//Get the raft apply index of the highest block
func (n *Node) getBlockAppliedIndex() uint64 {
height := uint64(0)
n.blockAppliedIndex.Range(
func(key, value interface{}) bool {
k := key.(uint64)
if k > height {
height = k
}
return true
})
appliedIndex, ok := n.blockAppliedIndex.Load(height)
if !ok {
return 0
}
return appliedIndex.(uint64)
}
//Load the lastAppliedIndex of block height
func (n *Node) loadAppliedIndex() uint64 {
dat := n.storage.Get(appliedDbKey)
var lastAppliedIndex uint64
if dat == nil {
lastAppliedIndex = 0
} else {
lastAppliedIndex = binary.LittleEndian.Uint64(dat)
}
return lastAppliedIndex
}
//Write the lastAppliedIndex
func (n *Node) writeAppliedIndex(index uint64) {
buf := make([]byte, 8)
binary.LittleEndian.PutUint64(buf, index)
n.storage.Put(appliedDbKey, buf)
}
// GetPendingNonceByAccount gets pending nonce by given account.
func (n *Node) GetPendingNonceByAccount(account string) uint64 {
return n.mempool.GetPendingNonceByAccount(account)
}
// DelNode sends a delete vp request by given id.
func (n *Node) DelNode(delID uint64) error {
return nil
func (n *Node) setLastExec(height uint64) {
n.lastExec = height
}

View File

@ -1,7 +1,6 @@
package etcdraft
import (
"context"
"fmt"
"io/ioutil"
"os"
@ -44,7 +43,7 @@ func TestNode_Start(t *testing.T) {
Account: types.NewAddressByStr("000000000000000000000000000000000000000a").String(),
}
nodes[ID] = vpInfo
fileData, err := ioutil.ReadFile("../../../config/order.toml")
fileData, err := ioutil.ReadFile("./testdata/order.toml")
require.Nil(t, err)
err = ioutil.WriteFile(filepath.Join(repoRoot, "order.toml"), fileData, 0644)
require.Nil(t, err)
@ -54,6 +53,7 @@ func TestNode_Start(t *testing.T) {
mockPeermgr := mock_peermgr.NewMockPeerManager(mockCtl)
peers := make(map[uint64]*pb.VpInfo)
mockPeermgr.EXPECT().Peers().Return(peers).AnyTimes()
mockPeermgr.EXPECT().Broadcast(gomock.Any()).AnyTimes()
order, err := NewNode(
order.WithRepoRoot(repoRoot),
@ -150,7 +150,7 @@ func listen(t *testing.T, order order.Order, swarm *peermgr.Swarm) {
for {
select {
case ev := <-orderMsgCh:
err := order.Step(context.Background(), ev.Data)
err := order.Step(ev.Data)
require.Nil(t, err)
}
}

View File

@ -8,6 +8,7 @@ import (
_ "github.com/gogo/protobuf/gogoproto"
proto "github.com/gogo/protobuf/proto"
github_com_meshplus_bitxhub_kit_types "github.com/meshplus/bitxhub-kit/types"
pb "github.com/meshplus/bitxhub-model/pb"
io "io"
math "math"
)
@ -28,22 +29,16 @@ type RaftMessage_Type int32
const (
RaftMessage_CONSENSUS RaftMessage_Type = 0
RaftMessage_BROADCAST_TX RaftMessage_Type = 1
RaftMessage_GET_TX RaftMessage_Type = 2
RaftMessage_GET_TX_ACK RaftMessage_Type = 3
)
var RaftMessage_Type_name = map[int32]string{
0: "CONSENSUS",
1: "BROADCAST_TX",
2: "GET_TX",
3: "GET_TX_ACK",
}
var RaftMessage_Type_value = map[string]int32{
"CONSENSUS": 0,
"BROADCAST_TX": 1,
"GET_TX": 2,
"GET_TX_ACK": 3,
}
func (x RaftMessage_Type) String() string {
@ -114,6 +109,66 @@ func (m *RaftMessage) GetData() []byte {
return nil
}
type RequestBatch struct {
Digest string `protobuf:"bytes,1,opt,name=digest,proto3" json:"digest,omitempty"`
TxList []*pb.Transaction `protobuf:"bytes,2,rep,name=txList,proto3" json:"txList,omitempty"`
Height uint64 `protobuf:"varint,3,opt,name=height,proto3" json:"height,omitempty"`
}
func (m *RequestBatch) Reset() { *m = RequestBatch{} }
func (m *RequestBatch) String() string { return proto.CompactTextString(m) }
func (*RequestBatch) ProtoMessage() {}
func (*RequestBatch) Descriptor() ([]byte, []int) {
return fileDescriptor_33c57e4bae7b9afd, []int{1}
}
func (m *RequestBatch) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *RequestBatch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_RequestBatch.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalTo(b)
if err != nil {
return nil, err
}
return b[:n], nil
}
}
func (m *RequestBatch) XXX_Merge(src proto.Message) {
xxx_messageInfo_RequestBatch.Merge(m, src)
}
func (m *RequestBatch) XXX_Size() int {
return m.Size()
}
func (m *RequestBatch) XXX_DiscardUnknown() {
xxx_messageInfo_RequestBatch.DiscardUnknown(m)
}
var xxx_messageInfo_RequestBatch proto.InternalMessageInfo
func (m *RequestBatch) GetDigest() string {
if m != nil {
return m.Digest
}
return ""
}
func (m *RequestBatch) GetTxList() []*pb.Transaction {
if m != nil {
return m.TxList
}
return nil
}
func (m *RequestBatch) GetHeight() uint64 {
if m != nil {
return m.Height
}
return 0
}
type Ready struct {
TxHashes []github_com_meshplus_bitxhub_kit_types.Hash `protobuf:"bytes,1,rep,name=txHashes,proto3,customtype=github.com/meshplus/bitxhub-kit/types.Hash" json:"txHashes,omitempty"`
Height uint64 `protobuf:"varint,2,opt,name=height,proto3" json:"height,omitempty"`
@ -123,7 +178,7 @@ func (m *Ready) Reset() { *m = Ready{} }
func (m *Ready) String() string { return proto.CompactTextString(m) }
func (*Ready) ProtoMessage() {}
func (*Ready) Descriptor() ([]byte, []int) {
return fileDescriptor_33c57e4bae7b9afd, []int{1}
return fileDescriptor_33c57e4bae7b9afd, []int{2}
}
func (m *Ready) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@ -159,36 +214,87 @@ func (m *Ready) GetHeight() uint64 {
return 0
}
type TxSlice struct {
TxList []*pb.Transaction `protobuf:"bytes,1,rep,name=TxList,proto3" json:"TxList,omitempty"`
}
func (m *TxSlice) Reset() { *m = TxSlice{} }
func (m *TxSlice) String() string { return proto.CompactTextString(m) }
func (*TxSlice) ProtoMessage() {}
func (*TxSlice) Descriptor() ([]byte, []int) {
return fileDescriptor_33c57e4bae7b9afd, []int{3}
}
func (m *TxSlice) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *TxSlice) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_TxSlice.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalTo(b)
if err != nil {
return nil, err
}
return b[:n], nil
}
}
func (m *TxSlice) XXX_Merge(src proto.Message) {
xxx_messageInfo_TxSlice.Merge(m, src)
}
func (m *TxSlice) XXX_Size() int {
return m.Size()
}
func (m *TxSlice) XXX_DiscardUnknown() {
xxx_messageInfo_TxSlice.DiscardUnknown(m)
}
var xxx_messageInfo_TxSlice proto.InternalMessageInfo
func (m *TxSlice) GetTxList() []*pb.Transaction {
if m != nil {
return m.TxList
}
return nil
}
func init() {
proto.RegisterEnum("proto.RaftMessage_Type", RaftMessage_Type_name, RaftMessage_Type_value)
proto.RegisterType((*RaftMessage)(nil), "proto.RaftMessage")
proto.RegisterType((*RequestBatch)(nil), "proto.request_batch")
proto.RegisterType((*Ready)(nil), "proto.Ready")
proto.RegisterType((*TxSlice)(nil), "proto.tx_slice")
}
func init() { proto.RegisterFile("message.proto", fileDescriptor_33c57e4bae7b9afd) }
var fileDescriptor_33c57e4bae7b9afd = []byte{
// 318 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x4c, 0x90, 0xcf, 0x4a, 0xf3, 0x40,
0x14, 0xc5, 0x33, 0x6d, 0x5a, 0xbe, 0xef, 0xda, 0x96, 0x30, 0x0b, 0x0d, 0x2e, 0x62, 0xe8, 0x2a,
0x28, 0x4d, 0x40, 0x9f, 0xa0, 0x8d, 0xc5, 0x7f, 0xd8, 0xc2, 0xa4, 0x82, 0xbb, 0x32, 0xb1, 0xd3,
0x4c, 0xa8, 0x61, 0x42, 0x67, 0x02, 0xed, 0x5b, 0xf8, 0x1e, 0xbe, 0x88, 0xcb, 0x2e, 0xc5, 0x85,
0x48, 0xfb, 0x22, 0x92, 0x89, 0x4a, 0x57, 0x73, 0xce, 0x70, 0x7e, 0x97, 0x7b, 0x2e, 0xb4, 0x33,
0x26, 0x25, 0x4d, 0x98, 0x9f, 0x2f, 0x85, 0x12, 0xb8, 0xa1, 0x9f, 0xe3, 0x5e, 0x92, 0x2a, 0x5e,
0xc4, 0xfe, 0x93, 0xc8, 0x82, 0x44, 0x24, 0x22, 0xd0, 0xdf, 0x71, 0x31, 0xd7, 0x4e, 0x1b, 0xad,
0x2a, 0xaa, 0xfb, 0x8a, 0xe0, 0x80, 0xd0, 0xb9, 0xba, 0xaf, 0x66, 0xe1, 0x33, 0x30, 0xd5, 0x3a,
0x67, 0x36, 0x72, 0x91, 0xd7, 0x39, 0x3f, 0xaa, 0x52, 0xfe, 0x5e, 0xc2, 0x9f, 0xac, 0x73, 0x46,
0x74, 0x08, 0x1f, 0x42, 0x73, 0xbe, 0x14, 0xd9, 0xcd, 0xcc, 0xae, 0xb9, 0xc8, 0x33, 0xc9, 0x8f,
0xc3, 0x18, 0xcc, 0x19, 0x55, 0xd4, 0xae, 0xbb, 0xc8, 0x6b, 0x11, 0xad, 0xbb, 0x21, 0x98, 0x25,
0x89, 0xdb, 0xf0, 0x3f, 0x1c, 0x8f, 0xa2, 0xe1, 0x28, 0x7a, 0x88, 0x2c, 0x03, 0x5b, 0xd0, 0x1a,
0x90, 0x71, 0xff, 0x32, 0xec, 0x47, 0x93, 0xe9, 0xe4, 0xd1, 0x42, 0x18, 0xa0, 0x79, 0x35, 0xd4,
0xba, 0x86, 0x3b, 0x00, 0x95, 0x9e, 0xf6, 0xc3, 0x3b, 0xab, 0xde, 0x5d, 0x40, 0x83, 0x30, 0x3a,
0x5b, 0xe3, 0x5b, 0xf8, 0xa7, 0x56, 0xd7, 0x54, 0x72, 0x26, 0x6d, 0xe4, 0xd6, 0xbd, 0xd6, 0xc0,
0xff, 0xf8, 0x3c, 0x39, 0xdd, 0xeb, 0x9e, 0x31, 0xc9, 0xf3, 0xe7, 0x42, 0x06, 0x71, 0xaa, 0x56,
0xbc, 0x88, 0x7b, 0x8b, 0x54, 0x05, 0xe5, 0xd6, 0xd2, 0x2f, 0x39, 0xf2, 0xc7, 0x97, 0x2d, 0x38,
0x4b, 0x13, 0xae, 0x7e, 0x5b, 0x54, 0x6e, 0x60, 0xbf, 0x6d, 0x1d, 0xb4, 0xd9, 0x3a, 0xe8, 0x6b,
0xeb, 0xa0, 0x97, 0x9d, 0x63, 0x6c, 0x76, 0x8e, 0xf1, 0xbe, 0x73, 0x8c, 0xb8, 0xa9, 0xaf, 0x72,
0xf1, 0x1d, 0x00, 0x00, 0xff, 0xff, 0xec, 0x5d, 0x36, 0x19, 0x82, 0x01, 0x00, 0x00,
// 398 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x51, 0x41, 0x6f, 0xd3, 0x30,
0x14, 0xae, 0xdb, 0xac, 0x62, 0x5e, 0x0b, 0x95, 0x0f, 0x10, 0xed, 0x10, 0xa2, 0x5e, 0x16, 0x81,
0xe6, 0x48, 0x9b, 0xc4, 0x7d, 0x1d, 0x48, 0x80, 0x60, 0x93, 0xdc, 0x20, 0x71, 0xab, 0xec, 0xc4,
0x8d, 0xad, 0x35, 0x73, 0x88, 0x5f, 0xa4, 0xf6, 0x57, 0xc0, 0xcf, 0xe2, 0xd8, 0x23, 0xe2, 0x80,
0x50, 0xfb, 0x47, 0x50, 0x9c, 0x08, 0xca, 0x01, 0x4e, 0x7e, 0x9f, 0xdf, 0xf7, 0xf9, 0xfb, 0xfc,
0x1e, 0x1e, 0x17, 0xd2, 0x5a, 0x9e, 0x4b, 0x5a, 0x56, 0x06, 0x0c, 0x39, 0x72, 0xc7, 0xe9, 0x79,
0xae, 0x41, 0xd5, 0x82, 0xa6, 0xa6, 0x88, 0x73, 0x93, 0x9b, 0xd8, 0x5d, 0x8b, 0x7a, 0xe9, 0x90,
0x03, 0xae, 0x6a, 0x55, 0xa7, 0x2f, 0x0e, 0xe8, 0x85, 0xb4, 0xaa, 0x5c, 0xd5, 0x36, 0x16, 0x1a,
0xd6, 0xaa, 0x16, 0xe7, 0x85, 0xc9, 0xe4, 0x2a, 0x2e, 0x45, 0x0c, 0x15, 0xbf, 0xb7, 0x3c, 0x05,
0x6d, 0xee, 0x5b, 0xdd, 0xf4, 0x33, 0xc2, 0x27, 0x8c, 0x2f, 0xe1, 0x7d, 0x9b, 0x81, 0x3c, 0xc7,
0x1e, 0x6c, 0x4a, 0xe9, 0xa3, 0x10, 0x45, 0x0f, 0x2f, 0x9e, 0xb4, 0x2c, 0x7a, 0xc0, 0xa0, 0xc9,
0xa6, 0x94, 0xcc, 0x91, 0xc8, 0x63, 0x3c, 0x5c, 0x56, 0xa6, 0x78, 0x93, 0xf9, 0xfd, 0x10, 0x45,
0x1e, 0xeb, 0x10, 0x21, 0xd8, 0xcb, 0x38, 0x70, 0x7f, 0x10, 0xa2, 0x68, 0xc4, 0x5c, 0x3d, 0x3d,
0xc3, 0x5e, 0xa3, 0x24, 0x63, 0x7c, 0x7c, 0x7d, 0x7b, 0x33, 0x7f, 0x75, 0x33, 0xff, 0x30, 0x9f,
0xf4, 0xc8, 0x04, 0x8f, 0x66, 0xec, 0xf6, 0xea, 0xe5, 0xf5, 0xd5, 0x3c, 0x59, 0x24, 0x1f, 0x27,
0x68, 0xaa, 0xf0, 0xb8, 0x92, 0x9f, 0x6a, 0x69, 0x61, 0x21, 0x38, 0xa4, 0xaa, 0x71, 0xc9, 0x74,
0x2e, 0x2d, 0xb8, 0x50, 0xc7, 0xac, 0x43, 0xe4, 0x0c, 0x0f, 0x61, 0xfd, 0x4e, 0x5b, 0xf0, 0xfb,
0xe1, 0x20, 0x3a, 0xb9, 0x78, 0x44, 0x4b, 0x41, 0x93, 0x3f, 0x3f, 0x64, 0x5d, 0xbb, 0x79, 0x40,
0x49, 0x9d, 0x2b, 0x70, 0x81, 0x3c, 0xd6, 0xa1, 0xe9, 0x1d, 0x3e, 0x62, 0x92, 0x67, 0x1b, 0xf2,
0x16, 0x3f, 0x80, 0xf5, 0x6b, 0x6e, 0x95, 0xb4, 0x3e, 0x0a, 0x07, 0xd1, 0x68, 0x46, 0xbf, 0xff,
0x78, 0xfa, 0xec, 0x7f, 0x23, 0xbd, 0xd3, 0x10, 0x37, 0x33, 0xb0, 0xb4, 0xd1, 0xb1, 0xdf, 0xfa,
0x03, 0xb3, 0xfe, 0x5f, 0x66, 0x97, 0x8d, 0xc7, 0xc2, 0xae, 0x74, 0x2a, 0x9b, 0xe4, 0x49, 0x9b,
0x1c, 0xfd, 0x23, 0x79, 0xdb, 0x9e, 0xf9, 0x5f, 0x77, 0x01, 0xda, 0xee, 0x02, 0xf4, 0x73, 0x17,
0xa0, 0x2f, 0xfb, 0xa0, 0xb7, 0xdd, 0x07, 0xbd, 0x6f, 0xfb, 0xa0, 0x27, 0x86, 0x6e, 0x31, 0x97,
0xbf, 0x02, 0x00, 0x00, 0xff, 0xff, 0xe4, 0x5e, 0x21, 0x90, 0x3d, 0x02, 0x00, 0x00,
}
func (m *RaftMessage) Marshal() (dAtA []byte, err error) {
@ -225,6 +331,47 @@ func (m *RaftMessage) MarshalTo(dAtA []byte) (int, error) {
return i, nil
}
func (m *RequestBatch) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalTo(dAtA)
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *RequestBatch) MarshalTo(dAtA []byte) (int, error) {
var i int
_ = i
var l int
_ = l
if len(m.Digest) > 0 {
dAtA[i] = 0xa
i++
i = encodeVarintMessage(dAtA, i, uint64(len(m.Digest)))
i += copy(dAtA[i:], m.Digest)
}
if len(m.TxList) > 0 {
for _, msg := range m.TxList {
dAtA[i] = 0x12
i++
i = encodeVarintMessage(dAtA, i, uint64(msg.Size()))
n, err := msg.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
i += n
}
}
if m.Height != 0 {
dAtA[i] = 0x18
i++
i = encodeVarintMessage(dAtA, i, uint64(m.Height))
}
return i, nil
}
func (m *Ready) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -260,6 +407,36 @@ func (m *Ready) MarshalTo(dAtA []byte) (int, error) {
return i, nil
}
func (m *TxSlice) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalTo(dAtA)
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *TxSlice) MarshalTo(dAtA []byte) (int, error) {
var i int
_ = i
var l int
_ = l
if len(m.TxList) > 0 {
for _, msg := range m.TxList {
dAtA[i] = 0xa
i++
i = encodeVarintMessage(dAtA, i, uint64(msg.Size()))
n, err := msg.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
i += n
}
}
return i, nil
}
func encodeVarintMessage(dAtA []byte, offset int, v uint64) int {
for v >= 1<<7 {
dAtA[offset] = uint8(v&0x7f | 0x80)
@ -288,6 +465,28 @@ func (m *RaftMessage) Size() (n int) {
return n
}
func (m *RequestBatch) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
l = len(m.Digest)
if l > 0 {
n += 1 + l + sovMessage(uint64(l))
}
if len(m.TxList) > 0 {
for _, e := range m.TxList {
l = e.Size()
n += 1 + l + sovMessage(uint64(l))
}
}
if m.Height != 0 {
n += 1 + sovMessage(uint64(m.Height))
}
return n
}
func (m *Ready) Size() (n int) {
if m == nil {
return 0
@ -306,6 +505,21 @@ func (m *Ready) Size() (n int) {
return n
}
func (m *TxSlice) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
if len(m.TxList) > 0 {
for _, e := range m.TxList {
l = e.Size()
n += 1 + l + sovMessage(uint64(l))
}
}
return n
}
func sovMessage(x uint64) (n int) {
for {
n++
@ -444,6 +658,144 @@ func (m *RaftMessage) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *RequestBatch) 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 ErrIntOverflowMessage
}
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: request_batch: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: request_batch: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Digest", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowMessage
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
stringLen |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
intStringLen := int(stringLen)
if intStringLen < 0 {
return ErrInvalidLengthMessage
}
postIndex := iNdEx + intStringLen
if postIndex < 0 {
return ErrInvalidLengthMessage
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Digest = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field TxList", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowMessage
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthMessage
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthMessage
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.TxList = append(m.TxList, &pb.Transaction{})
if err := m.TxList[len(m.TxList)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
case 3:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field Height", wireType)
}
m.Height = 0
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowMessage
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
m.Height |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
default:
iNdEx = preIndex
skippy, err := skipMessage(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthMessage
}
if (iNdEx + skippy) < 0 {
return ErrInvalidLengthMessage
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *Ready) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
@ -551,6 +903,93 @@ func (m *Ready) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *TxSlice) 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 ErrIntOverflowMessage
}
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: tx_slice: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: tx_slice: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field TxList", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowMessage
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthMessage
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthMessage
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.TxList = append(m.TxList, &pb.Transaction{})
if err := m.TxList[len(m.TxList)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipMessage(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthMessage
}
if (iNdEx + skippy) < 0 {
return ErrInvalidLengthMessage
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func skipMessage(dAtA []byte) (n int, err error) {
l := len(dAtA)
iNdEx := 0

View File

@ -2,20 +2,29 @@ syntax = "proto3";
package proto;
import "github.com/gogo/protobuf/gogoproto/gogo.proto";
import "github.com/meshplus/bitxhub-model/pb/transaction.proto";
message RaftMessage {
enum Type {
CONSENSUS = 0;
BROADCAST_TX = 1;
GET_TX = 2;
GET_TX_ACK = 3;
}
Type type = 1;
uint64 fromId = 2;
bytes data = 3;
}
message request_batch {
string digest = 1;
repeated pb.Transaction txList = 2;
uint64 height = 3;
}
message Ready {
repeated bytes txHashes = 1 [(gogoproto.customtype) = "github.com/meshplus/bitxhub-kit/types.Hash"];
uint64 height = 2;
}
message tx_slice {
repeated pb.Transaction TxList = 1;
}

17
pkg/order/etcdraft/testdata/order.toml vendored Normal file
View File

@ -0,0 +1,17 @@
[raft]
tick_timeout = "0.1s" # TickTimeout is the internal logical clock for the Node by a single tick, Election timeouts and heartbeat timeouts are in units of ticks.
election_tick = 10 # ElectionTick is the number of Node.Tick invocations that must pass between elections.
heartbeat_tick = 1 # HeartbeatTick is the number of Node.Tick invocations that must pass between heartbeats.
max_size_per_msg = 1048576 # 1024*1024, MaxSizePerMsg limits the max size of each append message.
max_inflight_msgs = 500 # MaxInflightMsgs limits the max number of in-flight append messages during optimistic replication phase.
check_quorum = true # Leader steps down when quorum is not active for an electionTimeout.
pre_vote = true # PreVote prevents reconnected node from disturbing network.
disable_proposal_forwarding = true # This prevents blocks from being accidentally proposed by followers.
[raft.mempool]
batch_size = 1 # How many transactions should the primary pack.
pool_size = 50000 # How many transactions could the txPool stores in total.
tx_slice_size = 1 # How many transactions should the node broadcast at once
batch_tick = "0.3s" # Block packaging time period.
tx_slice_timeout = "0.1s" # Node broadcasts transactions if there are cached transactions, although set_size isn't reached yet

View File

@ -0,0 +1,75 @@
package etcdraft
import (
"github.com/meshplus/bitxhub-model/pb"
raftproto "github.com/meshplus/bitxhub/pkg/order/etcdraft/proto"
cmap "github.com/orcaman/concurrent-map"
"github.com/sirupsen/logrus"
"strconv"
"time"
)
type batchTimer struct {
logger logrus.FieldLogger
seqNo uint64
timeout time.Duration // default timeout of this timer
isActive cmap.ConcurrentMap // track all the timers with this timerName if it is active now
timeoutEventC chan bool
}
// newTimer news a timer with default timeout.
func newTimer(d time.Duration, logger logrus.FieldLogger) *batchTimer {
return &batchTimer{
timeout: d,
isActive: cmap.New(),
timeoutEventC: make(chan bool),
logger: logger,
}
}
func (timer *batchTimer) isBatchTimerActive() bool {
return !timer.isActive.IsEmpty()
}
// TODO (YH): add restartTimer???
// startBatchTimer starts the batch timer and reset the batchTimerActive to true.
func (timer *batchTimer) startBatchTimer() {
// stop old timer
timer.stopBatchTimer()
timer.logger.Debug("Leader start batch timer")
timestamp := time.Now().UnixNano()
key := strconv.FormatInt(timestamp, 10)
timer.isActive.Set(key, true)
time.AfterFunc(timer.timeout, func() {
if timer.isActive.Has(key) {
timer.timeoutEventC <- true
}
})
}
// stopBatchTimer stops the batch timer and reset the batchTimerActive to false.
func (timer *batchTimer) stopBatchTimer() {
if timer.isActive.IsEmpty() {
return
}
timer.logger.Debugf("Leader stop batch timer")
timer.isActive = cmap.New()
}
func msgToConsensusPbMsg(data []byte, tyr raftproto.RaftMessage_Type, replicaID uint64) *pb.Message {
rm := &raftproto.RaftMessage{
Type: tyr,
FromId: replicaID,
Data: data,
}
cmData, err := rm.Marshal()
if err != nil {
return nil
}
msg := &pb.Message{
Type: pb.Message_CONSENSUS,
Data: cmData,
}
return msg
}

View File

@ -0,0 +1,93 @@
package etcdraft
import (
"encoding/binary"
"sort"
"time"
"github.com/coreos/etcd/raft"
"github.com/coreos/etcd/raft/raftpb"
"github.com/meshplus/bitxhub/pkg/order"
)
const (
DefaultBatchTick = 500 * time.Millisecond
)
func generateRaftPeers(config *order.Config) ([]raft.Peer, error) {
peers := make([]raft.Peer, 0)
for id, vpInfo := range config.Nodes {
vpIngoBytes, err := vpInfo.Marshal()
if err != nil {
return nil, err
}
peers = append(peers, raft.Peer{ID: id, Context: vpIngoBytes})
}
sort.Slice(peers, func(i, j int) bool {
return peers[i].ID < peers[j].ID
})
return peers, nil
}
//Get the raft apply index of the highest block
func (n *Node) getBlockAppliedIndex() uint64 {
height := uint64(0)
n.blockAppliedIndex.Range(
func(key, value interface{}) bool {
k := key.(uint64)
if k > height {
height = k
}
return true
})
appliedIndex, ok := n.blockAppliedIndex.Load(height)
if !ok {
return 0
}
return appliedIndex.(uint64)
}
//Load the lastAppliedIndex of block height
func (n *Node) loadAppliedIndex() uint64 {
dat := n.storage.Get(appliedDbKey)
var lastAppliedIndex uint64
if dat == nil {
lastAppliedIndex = 0
} else {
lastAppliedIndex = binary.LittleEndian.Uint64(dat)
}
return lastAppliedIndex
}
//Write the lastAppliedIndex
func (n *Node) writeAppliedIndex(index uint64) {
buf := make([]byte, 8)
binary.LittleEndian.PutUint64(buf, index)
n.storage.Put(appliedDbKey, buf)
}
func (n *Node) isLeader() bool {
return n.leader == n.id
}
//Determine whether the current apply index is normal
func (n *Node) entriesToApply(allEntries []raftpb.Entry) (entriesToApply []raftpb.Entry) {
if len(allEntries) == 0 {
return
}
firstIdx := allEntries[0].Index
if firstIdx > n.appliedIndex+1 {
n.logger.Fatalf("first index of committed entry[%d] should <= progress.appliedIndex[%d]+1", firstIdx, n.appliedIndex)
}
if n.appliedIndex-firstIdx+1 < uint64(len(allEntries)) {
entriesToApply = allEntries[n.appliedIndex-firstIdx+1:]
}
return entriesToApply
}
func (n *Node) ramLastIndex() uint64 {
i, _ := n.raftStorage.ram.LastIndex()
n.logger.Infof("New Leader's last index is %d, appliedIndex is %d", i, n.appliedIndex)
return i
}

View File

@ -1,146 +1,81 @@
package mempool
import (
"errors"
"time"
"github.com/meshplus/bitxhub-kit/storage"
"github.com/meshplus/bitxhub-model/pb"
raftproto "github.com/meshplus/bitxhub/pkg/order/etcdraft/proto"
)
var _ MemPool = (*mempoolImpl)(nil)
//go:generate mockgen -destination mock_mempool/mock_mempool.go -package mock_mempool -source types.go
type MemPool interface {
// Start starts mempool service.
Start() error
// ProcessTransactions process transaction from api and other vp nodes.
ProcessTransactions(txs []*pb.Transaction, isLeader bool) *raftproto.RequestBatch
// Stop stops mempool service.
Stop()
// RecvTransaction receives transaction from API.
RecvTransaction(tx *pb.Transaction) error
// RecvForwardTxs receives transactions from other vp nodes.
RecvForwardTxs(txSlice *TxSlice)
// UpdateLeader updates the leader node info when the consensus module updates the leader status.
UpdateLeader(uint64)
// FetchTxn sends the fetching missing transactions request.
FetchTxn(lostTxnEvent *LocalMissingTxnEvent)
// RecvFetchTxnRequest receives the fetching missing transactions request,
// and load the related transactions from its mempool or storage.
RecvFetchTxnRequest(fetchTxnRequest *FetchTxnRequest)
// RecvFetchTxnResponse receives the fetching missing transactions response,
// and posts to consensus module.
RecvFetchTxnResponse(fetchTxnResponse *FetchTxnResponse)
// GetChainHeight gets the sequence number of block.
GetChainHeight() uint64
// IncreaseChainHeight increases the sequence number of block.
IncreaseChainHeight()
// GetBlock return the transactions list by given ready.
// If the corresponding transaction cannot be found, a list of missing transactions will be returned.
GetBlockByHashList(ready *raftproto.Ready) (map[uint64]string, []*pb.Transaction)
// GenerateBlock generate a block
GenerateBlock() *raftproto.RequestBatch
// Remove removes the committed transactions from mempool
CommitTransactions(ready *raftproto.Ready)
CommitTransactions(state *ChainState)
// HasPendingRequest checks if there is non-batched tx(s) in mempool pool or not
HasPendingRequest() bool
SetBatchSeqNo(batchSeq uint64)
External
}
// External is a concurrent and safe interface, which can be called by api module directly.
type External interface {
// GetPendingNonceByAccount will return the latest pending nonce of a given account
GetPendingNonceByAccount(account string) uint64
// IsPoolFull check if memPool has exceeded the limited txSize.
IsPoolFull() bool
}
// NewMempool return the mempool instance.
func NewMempool(config *Config, storage storage.Storage, batchC chan *raftproto.Ready) MemPool {
return newMempoolImpl(config, storage, batchC)
func NewMempool(config *Config) MemPool {
return newMempoolImpl(config)
}
// RecvTransaction receives transaction from api and other vp nodes.
func (mpi *mempoolImpl) RecvTransaction(tx *pb.Transaction) error {
if mpi.txCache.IsFull() && mpi.poolIsFull() {
return errors.New("transaction cache and pool are full, we will drop this transaction")
// GenerateRequestBatch generates a transaction batch and post it
// to outside if there are transactions in txPool.
func (mpi *mempoolImpl) GenerateBlock() *raftproto.RequestBatch {
if mpi.txStore.priorityNonBatchSize == 0 {
mpi.logger.Debug("Mempool is empty")
return nil
}
// TODOYH: how to inform the client that the nonce of is wrong, need to sync to correct nonce.
mpi.txCache.recvTxC <- tx
return nil
}
// RecvTransaction receives transaction from api and other vp nodes.
func (mpi *mempoolImpl) RecvForwardTxs(txSlice *TxSlice) {
mpi.subscribe.txForwardC <- txSlice
}
// UpdateLeader updates the
func (mpi *mempoolImpl) UpdateLeader(newLeader uint64) {
// TODO (YH): should block until mempool finishing updating the leader info.
mpi.subscribe.updateLeaderC <- newLeader
}
// FetchTxn sends the fetch request.
func (mpi *mempoolImpl) FetchTxn(lostTxnEvent *LocalMissingTxnEvent) {
mpi.subscribe.localMissingTxnEvent <- lostTxnEvent
}
func (mpi *mempoolImpl) RecvFetchTxnRequest(fetchTxnRequest *FetchTxnRequest) {
mpi.subscribe.fetchTxnRequestC <- fetchTxnRequest
}
func (mpi *mempoolImpl) RecvFetchTxnResponse(fetchTxnResponse *FetchTxnResponse) {
mpi.subscribe.fetchTxnResponseC <- fetchTxnResponse
}
// Start starts the mempool service.
func (mpi *mempoolImpl) Start() error {
mpi.logger.Debug("Start Listen mempool events")
go mpi.listenEvent()
go mpi.txCache.listenEvent()
return nil
}
func (mpi *mempoolImpl) Stop() {
if mpi.close != nil {
close(mpi.close)
}
if mpi.txCache.close != nil {
close(mpi.txCache.close)
batch, err := mpi.generateBlock()
if err != nil {
mpi.logger.Error("Generator batch failed")
return nil
}
return batch
}
func (mpi *mempoolImpl) GetChainHeight() uint64 {
return mpi.getBatchSeqNo()
func (mpi *mempoolImpl) HasPendingRequest() bool {
return mpi.txStore.priorityNonBatchSize > 0
}
func (mpi *mempoolImpl) IncreaseChainHeight() {
mpi.increaseBatchSeqNo()
}
func (mpi *mempoolImpl) GetBlockByHashList(ready *raftproto.Ready) (missingTxnHashList map[uint64]string, txList []*pb.Transaction) {
waitC := make(chan *mempoolBatch)
getBlock := &constructBatchEvent{
ready: ready,
result: waitC,
}
mpi.subscribe.getBlockC <- getBlock
// block until finishing constructing related batch
batch := <-waitC
return batch.missingTxnHashList, batch.txList
}
func (mpi *mempoolImpl) CommitTransactions(ready *raftproto.Ready) {
mpi.subscribe.commitTxnC <- ready
func (mpi *mempoolImpl) CommitTransactions(state *ChainState) {
gcStartTime := time.Now()
mpi.processCommitTransactions(state)
duration := time.Now().Sub(gcStartTime).Nanoseconds()
mpi.logger.Debugf("GC duration %v", duration)
}
func (mpi *mempoolImpl) GetPendingNonceByAccount(account string) uint64 {
waitC := make(chan uint64)
getNonceRequest := &getNonceRequest{
account: account,
waitC: waitC,
}
mpi.subscribe.pendingNonceC <- getNonceRequest
return <-waitC
return mpi.txStore.nonceCache.getPendingNonce(account)
}
func (mpi *mempoolImpl) IsPoolFull() bool {
return uint64(len(mpi.txStore.txHashMap)) >= mpi.poolSize
}
func (mpi *mempoolImpl) SetBatchSeqNo(batchSeq uint64) {
mpi.batchSeqNo = batchSeq
}

View File

@ -1,158 +1,46 @@
package mempool
import (
"errors"
"fmt"
"sync"
"sync/atomic"
"time"
"github.com/meshplus/bitxhub-kit/storage"
"github.com/meshplus/bitxhub-kit/types"
"github.com/google/btree"
"github.com/meshplus/bitxhub-model/pb"
raftproto "github.com/meshplus/bitxhub/pkg/order/etcdraft/proto"
"github.com/meshplus/bitxhub/pkg/peermgr"
"github.com/google/btree"
"github.com/sirupsen/logrus"
"sync"
)
type mempoolImpl struct {
localID uint64
leader uint64 // leader node id
batchSize uint64
batchSeqNo uint64 // track the sequence number of block
poolSize uint64
logger logrus.FieldLogger
batchC chan *raftproto.Ready
close chan bool
txStore *transactionStore // store all transactions info
txCache *TxCache // cache the transactions received from api
subscribe *subscribeEvent
storage storage.Storage
peerMgr peermgr.PeerManager //network manager
batchTimerMgr *timerManager
ledgerHelper func(hash *types.Hash) (*pb.Transaction, error)
txStore *transactionStore // store all transactions info
}
func newMempoolImpl(config *Config, storage storage.Storage, batchC chan *raftproto.Ready) *mempoolImpl {
func newMempoolImpl(config *Config) *mempoolImpl {
mpi := &mempoolImpl{
localID: config.ID,
peerMgr: config.PeerMgr,
batchSeqNo: config.ChainHeight,
ledgerHelper: config.GetTransactionFunc,
logger: config.Logger,
batchC: batchC,
storage: storage,
localID: config.ID,
batchSeqNo: config.ChainHeight,
logger: config.Logger,
}
mpi.txStore = newTransactionStore()
mpi.txCache = newTxCache(config.TxSliceTimeout, config.TxSliceSize, config.Logger)
mpi.subscribe = newSubscribe()
if config.BatchSize == 0 {
mpi.batchSize = DefaultBatchSize
} else {
mpi.batchSize = config.BatchSize
}
var batchTick time.Duration
if config.BatchTick == 0 {
batchTick = DefaultBatchTick
if config.PoolSize == 0 {
mpi.poolSize = DefaultPoolSize
} else {
batchTick = config.BatchTick
mpi.poolSize = config.PoolSize
}
mpi.batchTimerMgr = newTimer(batchTick)
mpi.logger.Infof("MemPool batch size = %d", mpi.batchSize)
mpi.logger.Infof("MemPool batch seqNo = %d", mpi.batchSeqNo)
mpi.logger.Infof("MemPool pool size = %d", mpi.poolSize)
return mpi
}
// TODO (YH): refactor listenEvent by mu
func (mpi *mempoolImpl) listenEvent() {
waitC := make(chan bool)
for {
select {
case <-mpi.close:
mpi.logger.Info("----- Exit listen loop -----")
return
case newLeader := <-mpi.subscribe.updateLeaderC:
if newLeader == mpi.localID {
mpi.logger.Info("----- Become the leader node -----")
}
mpi.leader = newLeader
case txSet := <-mpi.txCache.txSetC:
// 1. send transactions to other peer
data, err := txSet.Marshal()
if err != nil {
mpi.logger.Errorf("Marshal failed, err: %s", err.Error())
return
}
pbMsg := mpi.msgToConsensusPbMsg(data, raftproto.RaftMessage_BROADCAST_TX)
mpi.broadcast(pbMsg)
// 2. process transactions
if err := mpi.processTransactions(txSet.TxList); err != nil {
mpi.logger.Errorf("Process transactions failed, err: %s", err.Error())
}
case txSlice := <-mpi.subscribe.txForwardC:
if err := mpi.processTransactions(txSlice.TxList); err != nil {
mpi.logger.Errorf("Process transactions failed, err: %s", err.Error())
}
case res := <-mpi.subscribe.getBlockC:
result := mpi.getBlockByHashList(res.ready)
res.result <- result
case <-mpi.batchTimerMgr.timeoutEventC:
if mpi.isBatchTimerActive() {
mpi.stopBatchTimer(StopReason1)
mpi.logger.Debug("Batch timer expired, try to create a batch")
if mpi.txStore.priorityNonBatchSize > 0 {
ready, err := mpi.generateBlock(true)
if err != nil {
mpi.logger.Errorf("Generator batch failed")
continue
}
mpi.batchC <- ready
} else {
mpi.logger.Debug("The length of priorityIndex is 0, skip the batch timer")
}
}
case commitReady := <-mpi.subscribe.commitTxnC:
gcStartTime := time.Now()
mpi.processCommitTransactions(commitReady)
duration := time.Now().Sub(gcStartTime).Nanoseconds()
mpi.logger.Debugf("GC duration %v", duration)
case lostTxnEvent := <-mpi.subscribe.localMissingTxnEvent:
if err := mpi.sendFetchTxnRequest(lostTxnEvent.Height, lostTxnEvent.MissingTxnHashList); err != nil {
mpi.logger.Errorf("Process fetch txn failed, err: %s", err.Error())
lostTxnEvent.WaitC <- false
} else {
mpi.logger.Debug("Process fetch txn success")
waitC = lostTxnEvent.WaitC
}
case fetchRequest := <-mpi.subscribe.fetchTxnRequestC:
if err := mpi.processFetchTxnRequest(fetchRequest); err != nil {
mpi.logger.Error("Process fetchTxnRequest failed")
}
case fetchRes := <-mpi.subscribe.fetchTxnResponseC:
if err := mpi.processFetchTxnResponse(fetchRes); err != nil {
waitC <- false
continue
}
waitC <- true
case getNonceRequest := <-mpi.subscribe.pendingNonceC:
pendingNonce := mpi.txStore.nonceCache.getPendingNonce(getNonceRequest.account)
getNonceRequest.waitC <- pendingNonce
}
}
}
func (mpi *mempoolImpl) processTransactions(txs []*pb.Transaction) error {
func (mpi *mempoolImpl) ProcessTransactions(txs []*pb.Transaction, isLeader bool) *raftproto.RequestBatch {
validTxs := make(map[string][]*pb.Transaction)
for _, tx := range txs {
// check the sequence number of tx
@ -181,22 +69,14 @@ func (mpi *mempoolImpl) processTransactions(txs []*pb.Transaction) error {
// send tx to mempool store
mpi.processDirtyAccount(dirtyAccounts)
if mpi.isLeader() {
// start batch timer when this node receives the first transaction set of a batch
if !mpi.isBatchTimerActive() {
mpi.startBatchTimer(StartReason1)
}
// generator batch by block size
if mpi.txStore.priorityNonBatchSize >= mpi.batchSize {
ready, err := mpi.generateBlock(false)
if err != nil {
return errors.New("generator batch fai")
}
// stop batch timer
mpi.stopBatchTimer(StopReason2)
mpi.batchC <- ready
// generator batch by block size
if isLeader && mpi.txStore.priorityNonBatchSize >= mpi.batchSize {
batch, err := mpi.generateBlock()
if err != nil {
mpi.logger.Errorf("Generator batch failed")
return nil
}
return batch
}
return nil
}
@ -226,11 +106,18 @@ func (mpi *mempoolImpl) processDirtyAccount(dirtyAccounts map[string]bool) {
// getBlock fetches next block of transactions for consensus,
// batchedTx are all txs sent to consensus but were not committed yet, mempool should filter out such txs.
func (mpi *mempoolImpl) generateBlock(isTimeout bool) (*raftproto.Ready, error) {
result := make([]orderedIndexKey, 0, mpi.batchSize)
func (mpi *mempoolImpl) generateBlock() (*raftproto.RequestBatch, error) {
// txs has lower nonce will be observed first in priority index iterator.
mpi.logger.Infof("Length of priority index: %v", mpi.txStore.priorityIndex.data.Len())
mpi.logger.Debugf("Length of non-batched transactions: %d", mpi.txStore.priorityNonBatchSize)
var batchSize uint64
if poolLen := mpi.txStore.priorityNonBatchSize; poolLen > mpi.batchSize {
batchSize = mpi.batchSize
} else {
batchSize = mpi.txStore.priorityNonBatchSize
}
skippedTxs := make(map[orderedIndexKey]bool)
result := make([]orderedIndexKey, 0, mpi.batchSize)
mpi.txStore.priorityIndex.data.Ascend(func(a btree.Item) bool {
tx := a.(*orderedIndexKey)
// if tx has existed in bathedTxs,
@ -249,116 +136,61 @@ func (mpi *mempoolImpl) generateBlock(isTimeout bool) (*raftproto.Ready, error)
ptr := orderedIndexKey{account: tx.account, nonce: tx.nonce}
mpi.txStore.batchedTxs[ptr] = true
result = append(result, ptr)
// batched by batch size or timeout
condition1 := uint64(len(result)) == mpi.batchSize
condition2 := isTimeout && uint64(len(result)) == mpi.txStore.priorityNonBatchSize
if condition1 || condition2 {
if uint64(len(result)) == batchSize {
return false
}
// check if we can now include some txs that were skipped before for given account
skippedTxn := orderedIndexKey{account: tx.account, nonce: tx.nonce + 1}
for {
if _, ok := skippedTxs[skippedTxn]; !ok {
break
}
mpi.txStore.batchedTxs[ptr] = true
result = append(result, skippedTxn)
if uint64(len(result)) == batchSize {
return false
}
skippedTxn.nonce++
}
} else {
skippedTxs[orderedIndexKey{tx.account, tx.nonce}] = true
}
return true
})
if len(result) == 0 && mpi.txStore.priorityNonBatchSize > 0 {
mpi.logger.Error("===== NOTE!!! Leader generate a batch with 0 txs")
mpi.txStore.priorityNonBatchSize = 0
return nil, nil
}
// convert transaction pointers to real values
hashList := make([]types.Hash, len(result))
txList := make([]*pb.Transaction, len(result))
for i, v := range result {
rawTransaction := mpi.txStore.getTxByOrderKey(v.account, v.nonce)
hashList[i] = *rawTransaction.TransactionHash
txList[i] = rawTransaction
}
mpi.increaseBatchSeqNo()
batchSeqNo := mpi.getBatchSeqNo()
ready := &raftproto.Ready{
TxHashes: hashList,
Height: batchSeqNo,
mpi.batchSeqNo++
batchSeqNo := mpi.batchSeqNo
batch := &raftproto.RequestBatch{
TxList: txList,
Height: batchSeqNo,
}
// store the batch to cache
if _, ok := mpi.txStore.batchedCache[batchSeqNo]; ok {
mpi.logger.Errorf("Generate block with height %d, but there is already block at this height", batchSeqNo)
return nil, errors.New("wrong block height ")
if mpi.txStore.priorityNonBatchSize >= uint64(len(txList)) {
mpi.txStore.priorityNonBatchSize = mpi.txStore.priorityNonBatchSize - uint64(len(txList))
}
// store the batch to cache
mpi.txStore.batchedCache[batchSeqNo] = txList
// store the batch to db
mpi.batchStore(txList)
mpi.txStore.priorityNonBatchSize = mpi.txStore.priorityNonBatchSize - uint64(len(hashList))
mpi.logger.Infof("Generated block %d with %d txs", batchSeqNo, len(txList))
return ready, nil
}
func (mpi *mempoolImpl) getBlockByHashList(ready *raftproto.Ready) *mempoolBatch {
res := &mempoolBatch{}
// leader get the block directly from batchedCache
if mpi.isLeader() {
if txList, ok := mpi.txStore.batchedCache[ready.Height]; !ok {
mpi.logger.Warningf("Leader get block failed, can't find block %d from batchedCache", ready.Height)
missingTxnHashList := make(map[uint64]string)
for i, txHash := range ready.TxHashes {
missingTxnHashList[uint64(i)] = txHash.String()
}
res.missingTxnHashList = missingTxnHashList
} else {
// TODO (YH): check tx hash and length
res.txList = txList
}
return res
}
// follower construct the same batch by given ready.
return mpi.constructSameBatch(ready)
}
// constructSameBatch only be called by follower, constructs a batch by given ready info.
func (mpi *mempoolImpl) constructSameBatch(ready *raftproto.Ready) *mempoolBatch {
res := &mempoolBatch{}
if txList, ok := mpi.txStore.batchedCache[ready.Height]; ok {
mpi.logger.Warningf("Batch %d already exists in batchedCache", ready.Height)
// TODO (YH): check tx hash and length
res.txList = txList
return res
}
missingTxList := make(map[uint64]string)
txList := make([]*pb.Transaction, 0)
for index, txHash := range ready.TxHashes {
var (
txPointer *orderedIndexKey
txMap *txSortedMap
txItem *txItem
ok bool
)
strHash := txHash.String()
if txPointer, _ = mpi.txStore.txHashMap[strHash]; txPointer == nil {
missingTxList[uint64(index)] = strHash
continue
}
if txMap, ok = mpi.txStore.allTxs[txPointer.account]; !ok {
mpi.logger.Warningf("Transaction %s exist in txHashMap but not in allTxs", strHash)
missingTxList[uint64(index)] = strHash
continue
}
if txItem, ok = txMap.items[txPointer.nonce]; !ok {
mpi.logger.Warningf("Transaction %s exist in txHashMap but not in allTxs", strHash)
missingTxList[uint64(index)] = strHash
continue
}
txList = append(txList, txItem.tx)
mpi.txStore.batchedTxs[*txPointer] = true
}
res.missingTxnHashList = missingTxList
res.txList = txList
// persist the correct batch
if len(res.missingTxnHashList) == 0 {
// store the batch to cache
mpi.txStore.batchedCache[ready.Height] = txList
}
return res
mpi.logger.Debugf("Leader generate a batch with %d txs, which height is %d, and now there are %d pending txs.", len(txList), batchSeqNo, mpi.txStore.priorityNonBatchSize)
return batch, nil
}
// processCommitTransactions removes the transactions in ready.
func (mpi *mempoolImpl) processCommitTransactions(ready *raftproto.Ready) {
func (mpi *mempoolImpl) processCommitTransactions(state *ChainState) {
dirtyAccounts := make(map[string]bool)
// update current cached commit nonce for account
for _, txHash := range ready.TxHashes {
updateAccounts := make(map[string]uint64)
// update current cached commit nonce for account
for _, txHash := range state.TxHashList {
strHash := txHash.String()
txPointer := mpi.txStore.txHashMap[strHash]
txPointer, ok := mpi.txStore.txHashMap[strHash]
@ -370,6 +202,12 @@ func (mpi *mempoolImpl) processCommitTransactions(ready *raftproto.Ready) {
newCommitNonce := txPointer.nonce + 1
if preCommitNonce < newCommitNonce {
mpi.txStore.nonceCache.setCommitNonce(txPointer.account, newCommitNonce)
// Note!!! updating pendingNonce to commitNonce for the restart node
pendingNonce := mpi.txStore.nonceCache.getPendingNonce(txPointer.account)
if pendingNonce < newCommitNonce {
updateAccounts[txPointer.account] = newCommitNonce
mpi.txStore.nonceCache.setPendingNonce(txPointer.account, newCommitNonce)
}
}
delete(mpi.txStore.txHashMap, strHash)
delete(mpi.txStore.batchedTxs, *txPointer)
@ -397,158 +235,17 @@ func (mpi *mempoolImpl) processCommitTransactions(ready *raftproto.Ready) {
mpi.txStore.parkingLotIndex.removeByOrderedQueueKey(removedTxs)
}(removedTxs)
wg.Wait()
delta := int32(len(removedTxs))
atomic.AddInt32(&mpi.txStore.poolSize, -delta)
}
}
if mpi.isLeader() {
mpi.batchDelete(ready.TxHashes)
readyNum := uint64(mpi.txStore.priorityIndex.size())
// set priorityNonBatchSize to min(nonBatchedTxs, readyNum),
if mpi.txStore.priorityNonBatchSize > readyNum {
mpi.txStore.priorityNonBatchSize = readyNum
}
delete(mpi.txStore.batchedCache, ready.Height)
// restart batch timer for remain txs.
if mpi.isLeader() {
mpi.startBatchTimer(StartReason2)
for account, pendingNonce := range updateAccounts {
mpi.logger.Debugf("Account %s update its pendingNonce to %d by commitNonce", account, pendingNonce)
}
mpi.logger.Debugf("Replica removes batch %d in mempool, and now there are %d batches, "+
"priority len: %d, parkingLot len: %d", ready.Height, len(mpi.txStore.batchedCache),
mpi.txStore.priorityIndex.size(), mpi.txStore.parkingLotIndex.size())
}
// sendFetchTxnRequest sends fetching missing transactions request to leader node.
func (mpi *mempoolImpl) sendFetchTxnRequest(height uint64, lostTxnHashList map[uint64]string) error {
filterFetchTxHashList := &FetchTxnRequest{
ReplicaId: mpi.localID,
Height: height,
MissingTxHashes: lostTxnHashList,
}
missingHashListBytes, err := filterFetchTxHashList.Marshal()
if err != nil {
mpi.logger.Error("Marshal MissingHashList fail")
return err
}
pbMsg := mpi.msgToConsensusPbMsg(missingHashListBytes, raftproto.RaftMessage_GET_TX)
mpi.logger.Debugf("Send fetch transactions request to replica %d", mpi.leader)
mpi.unicast(mpi.leader, pbMsg)
mpi.txStore.missingBatch[height] = lostTxnHashList
return nil
}
// processFetchTxnRequest processes fetch request...
func (mpi *mempoolImpl) processFetchTxnRequest(fetchTxnRequest *FetchTxnRequest) error {
txList := make(map[uint64]*pb.Transaction, len(fetchTxnRequest.MissingTxHashes))
var err error
if txList, err = mpi.loadTxnFromCache(fetchTxnRequest); err != nil {
if txList, err = mpi.loadTxnFromStorage(fetchTxnRequest); err != nil {
if txList, err = mpi.loadTxnFromLedger(fetchTxnRequest); err != nil {
mpi.logger.Errorf("Process fetch txn request [peer: %s, block height: %d] failed",
fetchTxnRequest.ReplicaId, fetchTxnRequest.Height)
return err
}
}
}
fetchTxnResponse := &FetchTxnResponse{
ReplicaId: mpi.localID,
Height: fetchTxnRequest.Height,
MissingTxnList: txList,
}
resBytes, err := fetchTxnResponse.Marshal()
if err != nil {
return err
}
pbMsg := mpi.msgToConsensusPbMsg(resBytes, raftproto.RaftMessage_GET_TX_ACK)
mpi.logger.Debugf("Send fetch missing transactions response to replica %d", fetchTxnRequest.ReplicaId)
mpi.unicast(fetchTxnRequest.ReplicaId, pbMsg)
return nil
}
func (mpi *mempoolImpl) loadTxnFromCache(fetchTxnRequest *FetchTxnRequest) (map[uint64]*pb.Transaction, error) {
missingHashList := fetchTxnRequest.MissingTxHashes
targetHeight := fetchTxnRequest.Height
for _, txHash := range missingHashList {
if txPointer, _ := mpi.txStore.txHashMap[txHash]; txPointer == nil {
return nil, fmt.Errorf("transaction %s dones't exist in txHashMap", txHash)
}
}
var targetBatch []*pb.Transaction
var ok bool
if targetBatch, ok = mpi.txStore.batchedCache[targetHeight]; !ok {
return nil, fmt.Errorf("batch %d dones't exist in batchedCache", targetHeight)
}
targetBatchLen := uint64(len(targetBatch))
txList := make(map[uint64]*pb.Transaction, len(missingHashList))
for index, txHash := range missingHashList {
if index > targetBatchLen || targetBatch[index].TransactionHash.String() != txHash {
return nil, fmt.Errorf("find invaild transaction, index: %d, targetHash: %s", index, txHash)
}
txList[index] = targetBatch[index]
}
return txList, nil
}
// TODO (YH): restore txn from wal
func (mpi *mempoolImpl) loadTxnFromStorage(fetchTxnRequest *FetchTxnRequest) (map[uint64]*pb.Transaction, error) {
missingHashList := fetchTxnRequest.MissingTxHashes
txList := make(map[uint64]*pb.Transaction)
for index, txHash := range missingHashList {
var (
tx *pb.Transaction
rawHash []byte
err error
ok bool
)
if rawHash, err = types.HexDecodeString(txHash); err != nil {
return nil, err
}
if tx, ok = mpi.load(rawHash); !ok {
return nil, errors.New("can't load tx from storage")
}
txList[index] = tx
}
return txList, nil
}
// loadTxnFromLedger find missing transactions from ledger.
func (mpi *mempoolImpl) loadTxnFromLedger(fetchTxnRequest *FetchTxnRequest) (map[uint64]*pb.Transaction, error) {
missingHashList := fetchTxnRequest.MissingTxHashes
txList := make(map[uint64]*pb.Transaction)
for index, txHash := range missingHashList {
var (
tx *pb.Transaction
err error
)
hash := types.NewHashByStr(txHash)
if hash == nil {
return nil, errors.New("nil hash")
}
if tx, err = mpi.ledgerHelper(hash); err != nil {
return nil, err
}
txList[index] = tx
}
return txList, nil
}
func (mpi *mempoolImpl) processFetchTxnResponse(fetchTxnResponse *FetchTxnResponse) error {
mpi.logger.Debugf("Receive fetch transactions response from replica %d", fetchTxnResponse.ReplicaId)
if _, ok := mpi.txStore.missingBatch[fetchTxnResponse.Height]; !ok {
return errors.New("can't find batch %d from missingBatch")
}
expectLen := len(mpi.txStore.missingBatch[fetchTxnResponse.Height])
recvLen := len(fetchTxnResponse.MissingTxnList)
if recvLen != expectLen {
return fmt.Errorf("receive unmatched fetching txn response, expect length: %d, received length: %d", expectLen, recvLen)
}
validTxn := make([]*pb.Transaction, 0)
targetBatch := mpi.txStore.missingBatch[fetchTxnResponse.Height]
for index, tx := range fetchTxnResponse.MissingTxnList {
if tx.TransactionHash.String() != targetBatch[index] {
return errors.New("find a hash mismatch tx")
}
validTxn = append(validTxn, tx)
}
if err := mpi.processTransactions(validTxn); err != nil {
return err
}
delete(mpi.txStore.missingBatch, fetchTxnResponse.Height)
return nil
mpi.logger.Debugf("Replica %d removes batches in mempool, and now there are %d non-batched txs,"+
"priority len: %d, parkingLot len: %d, batchedTx len: %d, txHashMap len: %d", mpi.localID, mpi.txStore.priorityNonBatchSize,
mpi.txStore.priorityIndex.size(), mpi.txStore.parkingLotIndex.size(), len(mpi.txStore.batchedTxs), len(mpi.txStore.txHashMap))
}

View File

@ -1,121 +0,0 @@
package mempool
import (
"testing"
"github.com/meshplus/bitxhub-model/pb"
"github.com/stretchr/testify/assert"
)
func TestProcessTransactions(t *testing.T) {
ast := assert.New(t)
mpi, batchC := mockMempoolImpl()
defer cleanTestData()
txList := make([]*pb.Transaction, 0)
privKey1 := genPrivKey()
account1, _ := privKey1.PublicKey().Address()
privKey2 := genPrivKey()
account2, _ := privKey2.PublicKey().Address()
tx1 := constructTx(uint64(1), &privKey1)
tx2 := constructTx(uint64(2), &privKey1)
tx3 := constructTx(uint64(1), &privKey2)
tx4 := constructTx(uint64(2), &privKey2)
tx5 := constructTx(uint64(4), &privKey2)
txList = append(txList, tx1, tx2, tx3, tx4, tx5)
err := mpi.processTransactions(txList)
ast.Nil(err)
ast.Equal(4, mpi.txStore.priorityIndex.size())
ast.Equal(1, mpi.txStore.parkingLotIndex.size())
ast.Equal(5, len(mpi.txStore.txHashMap))
ast.Equal(0, len(mpi.txStore.batchedCache))
ast.Equal(2, mpi.txStore.allTxs[account1.String()].index.size())
ast.Equal(3, mpi.txStore.allTxs[account2.String()].index.size())
ast.Equal(uint64(1), mpi.txStore.nonceCache.getCommitNonce(account1.String()))
ast.Equal(uint64(3), mpi.txStore.nonceCache.getPendingNonce(account1.String()))
ast.Equal(uint64(1), mpi.txStore.nonceCache.getCommitNonce(account2.String()))
ast.Equal(uint64(3), mpi.txStore.nonceCache.getPendingNonce(account2.String()))
go func() {
mpi.batchSize = 4
mpi.leader = mpi.localID
tx6 := constructTx(uint64(3), &privKey1)
tx7 := constructTx(uint64(5), &privKey2)
txList = make([]*pb.Transaction, 0)
txList = append(txList, tx6, tx7)
err = mpi.processTransactions(txList)
ast.Nil(err)
}()
select {
case batch := <-batchC:
ast.Equal(4, len(batch.TxHashes))
ast.Equal(uint64(2), batch.Height)
ast.Equal(uint64(1), mpi.txStore.priorityNonBatchSize)
ast.Equal(5, mpi.txStore.priorityIndex.size())
ast.Equal(2, mpi.txStore.parkingLotIndex.size())
ast.Equal(7, len(mpi.txStore.txHashMap))
ast.Equal(1, len(mpi.txStore.batchedCache))
ast.Equal(4, len(mpi.txStore.batchedCache[uint64(2)]))
ast.Equal(3, mpi.txStore.allTxs[account1.String()].index.size())
ast.Equal(4, mpi.txStore.allTxs[account2.String()].index.size())
ast.Equal(uint64(4), mpi.txStore.nonceCache.getPendingNonce(account1.String()))
ast.Equal(uint64(3), mpi.txStore.nonceCache.getPendingNonce(account2.String()))
}
}
func TestProcessFetchTxnRequest(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
defer cleanTestData()
privKey1 := genPrivKey()
tx1 := constructTx(uint64(1), &privKey1)
var txList []*pb.Transaction
txList = append(txList, tx1)
missingList := make(map[uint64]string)
missingList[0] = tx1.TransactionHash.String()
fetchTxnRequest := &FetchTxnRequest{
Height: uint64(2),
MissingTxHashes: missingList,
}
err := mpi.processFetchTxnRequest(fetchTxnRequest)
ast.NotNil(err, " can't find the missing tx from local")
// load tx from cache
err = mpi.processTransactions(txList)
mpi.txStore.batchedCache[uint64(2)] = txList
err = mpi.processFetchTxnRequest(fetchTxnRequest)
ast.Nil(err)
}
func TestProcessFetchTxnResponse(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
defer cleanTestData()
fetchTxnResponse := &FetchTxnResponse{
Height: uint64(2),
}
err := mpi.processFetchTxnResponse(fetchTxnResponse)
ast.NotNil(err, "can't find batch 2 from missingBatch")
privKey1 := genPrivKey()
tx1 := constructTx(uint64(1), &privKey1)
missingList := make(map[uint64]string)
missingList[0] = tx1.TransactionHash.String()
mpi.txStore.missingBatch[uint64(2)] = missingList
fetchTxnResponse.MissingTxnList = make(map[uint64]*pb.Transaction)
fetchTxnResponse.MissingTxnList[0] = tx1
fetchTxnResponse.MissingTxnList[1] = tx1
err = mpi.processFetchTxnResponse(fetchTxnResponse)
ast.NotNil(err, "length mismatch")
delete(fetchTxnResponse.MissingTxnList, 1)
err = mpi.processFetchTxnResponse(fetchTxnResponse)
ast.Nil(err)
ast.Equal(0, len(mpi.txStore.missingBatch))
}

View File

@ -6,144 +6,38 @@ import (
"github.com/meshplus/bitxhub-kit/types"
"github.com/meshplus/bitxhub-model/pb"
raftproto "github.com/meshplus/bitxhub/pkg/order/etcdraft/proto"
"github.com/stretchr/testify/assert"
)
func TestRecvTransaction(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
defer cleanTestData()
privKey1 := genPrivKey()
tx1 := constructTx(uint64(1), &privKey1)
go mpi.txCache.listenEvent()
go func() {
_ = mpi.RecvTransaction(tx1)
}()
select {
case txSet := <-mpi.txCache.txSetC:
ast.Equal(1, len(txSet.TxList))
}
err := mpi.Start()
ast.Nil(err)
privKey2 := genPrivKey()
go func() {
_ = mpi.RecvTransaction(tx1)
}()
time.Sleep(1 * time.Millisecond)
ast.Equal(1, mpi.txStore.priorityIndex.size())
ast.Equal(0, mpi.txStore.parkingLotIndex.size())
tx2 := constructTx(uint64(2), &privKey1)
tx3 := constructTx(uint64(1), &privKey2)
tx4 := constructTx(uint64(2), &privKey2)
go func() {
_ = mpi.RecvTransaction(tx4)
}()
time.Sleep(1 * time.Millisecond)
ast.Equal(1, mpi.txStore.priorityIndex.size())
ast.Equal(1, mpi.txStore.parkingLotIndex.size())
go func() {
_ = mpi.RecvTransaction(tx2)
}()
time.Sleep(1 * time.Millisecond)
ast.Equal(2, mpi.txStore.priorityIndex.size())
ast.Equal(1, mpi.txStore.parkingLotIndex.size())
go func() {
_ = mpi.RecvTransaction(tx3)
}()
time.Sleep(1 * time.Millisecond)
ast.Equal(4, mpi.txStore.priorityIndex.size())
ast.Equal(1, mpi.txStore.parkingLotIndex.size(), "delete tx4 until finishing executor")
mpi.Stop()
}
func TestRecvForwardTxs(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
defer cleanTestData()
privKey1 := genPrivKey()
tx := constructTx(uint64(1), &privKey1)
txList := []*pb.Transaction{tx}
txSlice := &TxSlice{TxList: txList}
go mpi.RecvForwardTxs(txSlice)
select {
case txSet := <-mpi.subscribe.txForwardC:
ast.Equal(1, len(txSet.TxList))
}
}
func TestUpdateLeader(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
mpi.Start()
defer cleanTestData()
go mpi.UpdateLeader(uint64(2))
time.Sleep(1 * time.Millisecond)
ast.Equal(uint64(2), mpi.leader)
}
func TestGetBlock(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
err := mpi.Start()
ast.Nil(err)
defer cleanTestData()
privKey1 := genPrivKey()
privKey2 := genPrivKey()
tx1 := constructTx(uint64(1), &privKey1)
tx2 := constructTx(uint64(2), &privKey1)
tx3 := constructTx(uint64(2), &privKey2)
tx4 := constructTx(uint64(4), &privKey2)
tx5 := constructTx(uint64(1), &privKey2)
var txList []*pb.Transaction
var txHashList []types.Hash
txList = append(txList, tx1, tx2, tx3, tx4)
txHashList = append(txHashList, *tx1.TransactionHash, *tx2.TransactionHash, *tx3.TransactionHash, *tx5.TransactionHash)
err = mpi.processTransactions(txList)
ast.Nil(err)
ready := &raftproto.Ready{
Height: uint64(2),
TxHashes: txHashList,
}
missingTxnHashList, txList := mpi.GetBlockByHashList(ready)
ast.Equal(1, len(missingTxnHashList), "missing tx5")
ast.Equal(3, len(txList))
// mock leader to getBlock
mpi.leader = uint64(1)
missingTxnHashList, txList = mpi.GetBlockByHashList(ready)
ast.Equal(4, len(missingTxnHashList))
ast.Equal(0, len(txList))
// mock follower
mpi.leader = uint64(2)
txList = []*pb.Transaction{}
txList = append(txList, tx5)
err = mpi.processTransactions(txList)
missingTxnHashList, txList = mpi.GetBlockByHashList(ready)
ast.Equal(0, len(missingTxnHashList))
ast.Equal(4, len(txList))
batch := mpi.ProcessTransactions(txList, false)
ast.Nil(batch)
// mock leader
batch = mpi.ProcessTransactions(txList, true)
ast.Nil(batch)
// mock leader to getBlock
mpi.leader = uint64(1)
missingTxnHashList, txList = mpi.GetBlockByHashList(ready)
ast.Equal(0, len(missingTxnHashList))
ast.Equal(4, len(txList))
txList = make([]*pb.Transaction,0)
tx5 := constructTx(uint64(1), &privKey2)
txList = append(txList, tx5)
batch = mpi.ProcessTransactions(txList, true)
ast.Equal(4, len(batch.TxList))
}
func TestGetPendingNonceByAccount(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
err := mpi.Start()
ast.Nil(err)
defer cleanTestData()
privKey1 := genPrivKey()
account1, _ := privKey1.PublicKey().Address()
nonce := mpi.GetPendingNonceByAccount(account1.String())
@ -158,8 +52,8 @@ func TestGetPendingNonceByAccount(t *testing.T) {
tx5 := constructTx(uint64(4), &privKey2)
var txList []*pb.Transaction
txList = append(txList, tx1, tx2, tx3, tx4, tx5)
err = mpi.processTransactions(txList)
ast.Nil(err)
batch := mpi.ProcessTransactions(txList, false)
ast.Nil(batch)
nonce = mpi.GetPendingNonceByAccount(account1.String())
ast.Equal(uint64(3), nonce)
nonce = mpi.GetPendingNonceByAccount(account2.String())
@ -168,16 +62,11 @@ func TestGetPendingNonceByAccount(t *testing.T) {
func TestCommitTransactions(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
err := mpi.Start()
ast.Nil(err)
defer cleanTestData()
mpi, batchC := mockMempoolImpl()
privKey1 := genPrivKey()
account1, _ := privKey1.PublicKey().Address()
nonce := mpi.GetPendingNonceByAccount(account1.String())
ast.Equal(uint64(1), nonce)
privKey2 := genPrivKey()
tx1 := constructTx(uint64(1), &privKey1)
tx2 := constructTx(uint64(2), &privKey1)
@ -185,63 +74,111 @@ func TestCommitTransactions(t *testing.T) {
tx4 := constructTx(uint64(4), &privKey2)
var txList []*pb.Transaction
txList = append(txList, tx1, tx2, tx3, tx4)
mpi.leader = uint64(1)
err = mpi.processTransactions(txList)
batch := mpi.ProcessTransactions(txList, true)
ast.Nil(batch)
ast.Equal(3, mpi.txStore.priorityIndex.size())
ast.Equal(1, mpi.txStore.parkingLotIndex.size())
ast.Equal(0, len(mpi.txStore.batchedCache))
go func() {
<-mpi.batchC
<-batchC
}()
tx5 := constructTx(uint64(2), &privKey2)
txList = []*pb.Transaction{}
txList = append(txList, tx5)
err = mpi.processTransactions(txList)
batch = mpi.ProcessTransactions(txList, true)
ast.Equal(4, len(batch.TxList))
ast.Equal(4, mpi.txStore.priorityIndex.size())
ast.Equal(1, mpi.txStore.parkingLotIndex.size())
ast.Equal(1, len(mpi.txStore.batchedCache))
height := mpi.GetChainHeight()
ast.Equal(uint64(2), height)
ast.Equal(uint64(2), mpi.batchSeqNo)
var txHashList []types.Hash
txHashList = append(txHashList, *tx1.TransactionHash, *tx2.TransactionHash, *tx3.TransactionHash, *tx5.TransactionHash)
ready := &raftproto.Ready{
var txHashList []*types.Hash
txHashList = append(txHashList, tx1.TransactionHash, tx2.TransactionHash, tx3.TransactionHash, tx5.TransactionHash)
state := &ChainState{
TxHashList: txHashList,
Height: uint64(2),
TxHashes: txHashList,
}
mpi.CommitTransactions(ready)
mpi.CommitTransactions(state)
time.Sleep(100 * time.Millisecond)
ast.Equal(0, mpi.txStore.priorityIndex.size())
ast.Equal(1, mpi.txStore.parkingLotIndex.size())
ast.Equal(0, len(mpi.txStore.batchedCache))
}
func TestFetchTxn(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
err := mpi.Start()
ast.Nil(err)
defer cleanTestData()
missingList := make(map[uint64]string)
missingList[0] = "tx1"
lostTxnEvent := &LocalMissingTxnEvent{
Height: uint64(2),
MissingTxnHashList: missingList,
WaitC: make(chan bool),
}
mpi.FetchTxn(lostTxnEvent)
time.Sleep(10 * time.Millisecond)
ast.Equal(1, len(mpi.txStore.missingBatch))
}
func TestIncreaseChainHeight(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
defer cleanTestData()
ast.Equal(uint64(1), mpi.GetChainHeight())
mpi.increaseBatchSeqNo()
ast.Equal(uint64(2), mpi.GetChainHeight())
ast.Equal(uint64(1), mpi.batchSeqNo)
mpi.batchSeqNo++
mpi.SetBatchSeqNo(mpi.batchSeqNo)
ast.Equal(uint64(2), mpi.batchSeqNo)
}
func TestProcessTransactions(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
txList := make([]*pb.Transaction, 0)
privKey1 := genPrivKey()
account1, _ := privKey1.PublicKey().Address()
privKey2 := genPrivKey()
account2, _ := privKey2.PublicKey().Address()
tx1 := constructTx(uint64(1), &privKey1)
tx2 := constructTx(uint64(2), &privKey1)
tx3 := constructTx(uint64(1), &privKey2)
tx4 := constructTx(uint64(2), &privKey2)
tx5 := constructTx(uint64(4), &privKey2)
txList = append(txList, tx1, tx2, tx3, tx4, tx5)
batch := mpi.ProcessTransactions(txList, false)
ast.Nil(batch)
ast.Equal(4, mpi.txStore.priorityIndex.size())
ast.Equal(1, mpi.txStore.parkingLotIndex.size())
ast.Equal(5, len(mpi.txStore.txHashMap))
ast.Equal(2, mpi.txStore.allTxs[account1.String()].index.size())
ast.Equal(3, mpi.txStore.allTxs[account2.String()].index.size())
ast.Equal(uint64(1), mpi.txStore.nonceCache.getCommitNonce(account1.String()))
ast.Equal(uint64(3), mpi.txStore.nonceCache.getPendingNonce(account1.String()))
ast.Equal(uint64(1), mpi.txStore.nonceCache.getCommitNonce(account2.String()))
ast.Equal(uint64(3), mpi.txStore.nonceCache.getPendingNonce(account2.String()))
mpi.batchSize = 4
tx6 := constructTx(uint64(3), &privKey1)
tx7 := constructTx(uint64(5), &privKey2)
txList = make([]*pb.Transaction, 0)
txList = append(txList, tx6, tx7)
batch = mpi.ProcessTransactions(txList, true)
ast.Equal(4, len(batch.TxList))
ast.Equal(uint64(2), batch.Height)
ast.Equal(uint64(1), mpi.txStore.priorityNonBatchSize)
ast.Equal(5, mpi.txStore.priorityIndex.size())
ast.Equal(2, mpi.txStore.parkingLotIndex.size())
ast.Equal(7, len(mpi.txStore.txHashMap))
ast.Equal(3, mpi.txStore.allTxs[account1.String()].index.size())
ast.Equal(4, mpi.txStore.allTxs[account2.String()].index.size())
ast.Equal(uint64(4), mpi.txStore.nonceCache.getPendingNonce(account1.String()))
ast.Equal(uint64(3), mpi.txStore.nonceCache.getPendingNonce(account2.String()))
}
func TestForward(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
txList := make([]*pb.Transaction, 0)
privKey1 := genPrivKey()
account1, _ := privKey1.PublicKey().Address()
tx1 := constructTx(uint64(1), &privKey1)
tx2 := constructTx(uint64(2), &privKey1)
tx3 := constructTx(uint64(3), &privKey1)
tx4 := constructTx(uint64(4), &privKey1)
tx5 := constructTx(uint64(6), &privKey1)
txList = append(txList, tx1, tx2, tx3, tx4, tx5)
batch := mpi.ProcessTransactions(txList, false)
ast.Nil(batch)
list := mpi.txStore.allTxs[account1.String()]
ast.Equal(5, list.index.size())
ast.Equal(4, mpi.txStore.priorityIndex.size())
ast.Equal(1, mpi.txStore.parkingLotIndex.size())
removeList := list.forward(uint64(3))
ast.Equal(1, len(removeList))
ast.Equal(2, len(removeList[account1.String()]))
ast.Equal(uint64(1), removeList[account1.String()][0].Nonce)
ast.Equal(uint64(2), removeList[account1.String()][1].Nonce)
}

View File

@ -1,23 +1,14 @@
package mempool
import (
"errors"
"math/rand"
"os"
"time"
"github.com/ethereum/go-ethereum/event"
"github.com/libp2p/go-libp2p-core/peer"
"github.com/meshplus/bitxhub-kit/crypto"
"github.com/meshplus/bitxhub-kit/crypto/asym"
"github.com/meshplus/bitxhub-kit/log"
"github.com/meshplus/bitxhub-kit/storage/leveldb"
"github.com/meshplus/bitxhub-kit/types"
"github.com/meshplus/bitxhub-model/pb"
"github.com/meshplus/bitxhub/internal/model/events"
raftproto "github.com/meshplus/bitxhub/pkg/order/etcdraft/proto"
network "github.com/meshplus/go-lightp2p"
"github.com/stretchr/testify/mock"
)
var (
@ -28,33 +19,24 @@ const (
DefaultTestChainHeight = uint64(1)
DefaultTestBatchSize = uint64(4)
DefaultTestTxSetSize = uint64(1)
LevelDBDir = "test-db"
)
func mockMempoolImpl() (*mempoolImpl, chan *raftproto.Ready) {
config := &Config{
ID: 1,
ChainHeight: DefaultTestChainHeight,
BatchSize: DefaultTestBatchSize,
PoolSize: DefaultPoolSize,
TxSliceSize: DefaultTestTxSetSize,
BatchTick: DefaultBatchTick,
FetchTimeout: DefaultFetchTxnTimeout,
TxSliceTimeout: DefaultTxSetTick,
Logger: log.NewWithModule("consensus"),
GetTransactionFunc: getTransactionFunc,
ID: 1,
ChainHeight: DefaultTestChainHeight,
BatchSize: DefaultTestBatchSize,
PoolSize: DefaultPoolSize,
TxSliceSize: DefaultTestTxSetSize,
BatchTick: DefaultBatchTick,
TxSliceTimeout: DefaultTxSetTick,
Logger: log.NewWithModule("consensus"),
}
config.PeerMgr = newMockPeerMgr()
db, _ := leveldb.New(LevelDBDir)
proposalC := make(chan *raftproto.Ready)
mempool := newMempoolImpl(config, db, proposalC)
mempool := newMempoolImpl(config)
return mempool, proposalC
}
func getTransactionFunc(hash *types.Hash) (*pb.Transaction, error) {
return nil,errors.New("can't find transaction")
}
func genPrivKey() crypto.PrivateKey {
privKey, _ := asym.GenerateKeyPair(crypto.Secp256k1)
return privKey
@ -76,80 +58,3 @@ func constructTx(nonce uint64, privKey *crypto.PrivateKey) *pb.Transaction {
tx.TransactionHash = tx.Hash()
return tx
}
func cleanTestData() bool {
err := os.RemoveAll(LevelDBDir)
if err != nil {
return false
}
return true
}
func newHash() *types.Hash {
hashBytes := make([]byte,types.HashLength)
rand.Read(hashBytes)
return types.NewHash(hashBytes)
}
type mockPeerMgr struct {
mock.Mock
EventChan chan *pb.Message
}
func newMockPeerMgr() *mockPeerMgr {
return &mockPeerMgr{}
}
func (mpm *mockPeerMgr) Broadcast(msg *pb.Message) error {
return nil
}
func (mpm *mockPeerMgr) AsyncSend(id uint64, msg *pb.Message) error {
mpm.EventChan <- msg
return nil
}
func (mpm *mockPeerMgr) Start() error {
return nil
}
func (mpm *mockPeerMgr) Stop() error {
return nil
}
func (mpm *mockPeerMgr) SendWithStream(network.Stream, *pb.Message) error {
return nil
}
func (mpm *mockPeerMgr) Send(uint64, *pb.Message) (*pb.Message, error) {
return nil, nil
}
func (mpm *mockPeerMgr)AddNode(newNodeID uint64, vpInfo *pb.VpInfo) {}
func (mpm *mockPeerMgr)DelNode(delID uint64){}
func (mpm *mockPeerMgr)UpdateRouter(vpInfos map[uint64]*pb.VpInfo, isNew bool) bool {
return false
}
func (mpm *mockPeerMgr)Disconnect(vpInfos map[uint64]*pb.VpInfo) {}
func (mpm *mockPeerMgr) Peers() map[uint64]*pb.VpInfo {
peers := make(map[uint64]*pb.VpInfo, 3)
id1 := "peer1"
peers[0] = &pb.VpInfo{Pid: id1}
id1 = "peer2"
peers[1] = &pb.VpInfo{Pid: id1}
id1 = "peer3"
peers[2] = &pb.VpInfo{Pid: id1}
return peers
}
func (mpm *mockPeerMgr) OtherPeers() map[uint64]*peer.AddrInfo {
return nil
}
func (mpm *mockPeerMgr) SubscribeOrderMessage(ch chan<- events.OrderMessageEvent) event.Subscription {
return nil
}

File diff suppressed because it is too large Load Diff

View File

@ -1,23 +0,0 @@
syntax = "proto3";
package mempool;
import "github.com/meshplus/bitxhub-model/pb/transaction.proto";
message tx_slice {
repeated pb.Transaction TxList = 1;
}
message fetch_txn_request {
uint64 replicaId = 1;
uint64 height = 2;
map<uint64, string> missing_tx_hashes = 3;
}
message fetch_txn_response {
uint64 replicaId = 1;
uint64 height = 2;
map<uint64, pb.Transaction> missing_txn_list = 3;
}

View File

@ -1,27 +0,0 @@
package mempool
import (
"github.com/meshplus/bitxhub-model/pb"
)
// broadcast the new transaction to other nodes
func (mpi *mempoolImpl) broadcast(m *pb.Message) {
for id := range mpi.peerMgr.Peers() {
if id == mpi.localID {
continue
}
go func(id uint64) {
if err := mpi.peerMgr.AsyncSend(id, m); err != nil {
mpi.logger.Debugf("Send tx slice to peer %d failed, err: %s", id, err.Error())
}
}(id)
}
}
func (mpi *mempoolImpl) unicast(to uint64, m *pb.Message) {
go func() {
if err := mpi.peerMgr.AsyncSend(to, m); err != nil {
mpi.logger.Warningf("Send message to peer %d failed, err: %s", to, err.Error())
}
}()
}

View File

@ -1,48 +0,0 @@
package mempool
import (
"fmt"
"github.com/meshplus/bitxhub-kit/types"
"github.com/meshplus/bitxhub-model/pb"
)
// batchStore persists batch into DB, only be called by leader.
func (mpi *mempoolImpl) batchStore(txList []*pb.Transaction) {
batch := mpi.storage.NewBatch()
for _, tx := range txList {
txKey := compositeKey(tx.TransactionHash.Bytes())
txData, _ := tx.Marshal()
batch.Put(txKey, txData)
}
batch.Commit()
}
// batchDelete delete txs from DB by given hash list.
func (mpi *mempoolImpl) batchDelete(hashes []types.Hash) {
batch := mpi.storage.NewBatch()
for _, hash := range hashes {
txKey := compositeKey(hash.Bytes())
batch.Delete(txKey)
}
batch.Commit()
}
func (mpi *mempoolImpl) load(hash []byte) (*pb.Transaction, bool) {
txKey := compositeKey(hash)
txData := mpi.storage.Get(txKey)
if txData == nil {
return nil, false
}
var tx pb.Transaction
if err := tx.Unmarshal(txData); err != nil {
mpi.logger.Error(err)
return nil, false
}
return &tx, true
}
func compositeKey(value interface{}) []byte {
var prefix = []byte("tx-")
return append(prefix, []byte(fmt.Sprintf("%v", value))...)
}

View File

@ -1,33 +0,0 @@
package mempool
import (
"testing"
"github.com/meshplus/bitxhub-kit/types"
"github.com/meshplus/bitxhub-model/pb"
"github.com/stretchr/testify/assert"
)
func TestStorage(t *testing.T) {
ast := assert.New(t)
mempool, _ := mockMempoolImpl()
defer cleanTestData()
txList := make([]*pb.Transaction, 0)
txHashList := make([]types.Hash, 0)
txHash1 := newHash()
tx1 := &pb.Transaction{Nonce: uint64(1), TransactionHash: txHash1}
txHash2 := newHash()
tx2 := &pb.Transaction{Nonce: uint64(1), TransactionHash: txHash2}
txList = append(txList, tx1, tx2)
txHashList = append(txHashList, *txHash1, *txHash1)
mempool.batchStore(txList)
tx, ok := mempool.load(txHash1.Bytes())
ast.Equal(true, ok)
ast.Equal(uint64(1), tx.Nonce)
mempool.batchDelete(txHashList)
tx, ok = mempool.load(txHash1.Bytes())
ast.Equal(false, ok)
ast.Nil(tx)
}

View File

@ -1,6 +1,7 @@
package mempool
import (
raftproto "github.com/meshplus/bitxhub/pkg/order/etcdraft/proto"
"time"
"github.com/meshplus/bitxhub-model/pb"
@ -8,23 +9,23 @@ import (
)
type TxCache struct {
recvTxC chan *pb.Transaction
txSetC chan *TxSlice
txSet []*pb.Transaction
logger logrus.FieldLogger
TxSetC chan *raftproto.TxSlice
RecvTxC chan *pb.Transaction
txSet []*pb.Transaction
timerC chan bool
stopTimerC chan bool
close chan bool
txSetTick time.Duration
txSetSize uint64
logger logrus.FieldLogger
}
func newTxCache(txSliceTimeout time.Duration, txSetSize uint64, logger logrus.FieldLogger) *TxCache {
func NewTxCache(txSliceTimeout time.Duration, txSetSize uint64, logger logrus.FieldLogger) *TxCache {
txCache := &TxCache{}
txCache.recvTxC = make(chan *pb.Transaction, DefaultTxCacheSize)
txCache.RecvTxC = make(chan *pb.Transaction, DefaultTxCacheSize)
txCache.close = make(chan bool)
txCache.txSetC = make(chan *TxSlice)
txCache.TxSetC = make(chan *raftproto.TxSlice)
txCache.timerC = make(chan bool)
txCache.stopTimerC = make(chan bool)
txCache.txSet = make([]*pb.Transaction, 0)
@ -42,14 +43,14 @@ func newTxCache(txSliceTimeout time.Duration, txSetSize uint64, logger logrus.Fi
return txCache
}
func (tc *TxCache) listenEvent() {
func (tc *TxCache) ListenEvent() {
for {
select {
case <-tc.close:
tc.logger.Info("Exit transaction cache")
return
case tx := <-tc.recvTxC:
case tx := <-tc.RecvTxC:
tc.appendTx(tx)
case <-tc.timerC:
@ -77,15 +78,15 @@ func (tc *TxCache) appendTx(tx *pb.Transaction) {
func (tc *TxCache) postTxSet() {
dst := make([]*pb.Transaction, len(tc.txSet))
copy(dst, tc.txSet)
txSet := &TxSlice{
txSet := &raftproto.TxSlice{
TxList: dst,
}
tc.txSetC <- txSet
tc.TxSetC <- txSet
tc.txSet = make([]*pb.Transaction, 0)
}
func (tc *TxCache) IsFull() bool {
return len(tc.recvTxC) == DefaultTxCacheSize
return len(tc.RecvTxC) == DefaultTxCacheSize
}
func (tc *TxCache) startTxSetTimer() {

View File

@ -13,8 +13,8 @@ func TestAppendTx(t *testing.T) {
ast := assert.New(t)
logger := log.NewWithModule("consensus")
sliceTimeout := 1 * time.Millisecond
txCache := newTxCache(sliceTimeout, 2, logger)
go txCache.listenEvent()
txCache := NewTxCache(sliceTimeout, 2, logger)
go txCache.ListenEvent()
tx := &pb.Transaction{}
txCache.appendTx(nil)
@ -23,7 +23,7 @@ func TestAppendTx(t *testing.T) {
tx = &pb.Transaction{Nonce: 1}
txCache.appendTx(tx)
select {
case txSet := <-txCache.txSetC:
case txSet := <-txCache.TxSetC:
ast.Equal(1, len(txSet.TxList), "post tx set by timeout")
ast.Equal(0, len(txCache.txSet))
}
@ -35,7 +35,7 @@ func TestAppendTx(t *testing.T) {
go txCache.appendTx(tx1)
go txCache.appendTx(tx2)
select {
case txSet := <-txCache.txSetC:
case txSet := <-txCache.TxSetC:
ast.Equal(2, len(txSet.TxList), "post tx set by size")
ast.Equal(0, len(txCache.txSet))
}

View File

@ -3,7 +3,7 @@ package mempool
import (
"github.com/google/btree"
"github.com/meshplus/bitxhub-model/pb"
"sync/atomic"
"sync"
)
type transactionStore struct {
@ -20,13 +20,6 @@ type transactionStore struct {
priorityIndex *btreeIndex
// cache all the batched txs which haven't executed.
batchedTxs map[orderedIndexKey]bool
// cache all batches created by current primary in order, removed after they are been executed.
// TODO (YH): change the type of key from height to digest.
batchedCache map[uint64][]*pb.Transaction
// trace the missing transaction
missingBatch map[uint64]map[uint64]string
// track the current size of mempool
poolSize int32
// track the non-batch priority transaction.
priorityNonBatchSize uint64
}
@ -36,8 +29,6 @@ func newTransactionStore() *transactionStore {
txHashMap: make(map[string]*orderedIndexKey, 0),
allTxs: make(map[string]*txSortedMap),
batchedTxs: make(map[orderedIndexKey]bool),
missingBatch: make(map[uint64]map[uint64]string),
batchedCache: make(map[uint64][]*pb.Transaction),
parkingLotIndex: newBtreeIndex(),
priorityIndex: newBtreeIndex(),
nonceCache: newNonceCache(),
@ -66,7 +57,6 @@ func (txStore *transactionStore) insertTxs(txs map[string][]*pb.Transaction) map
}
txList.items[tx.Nonce] = txItem
txList.index.insertBySortedNonceKey(tx)
atomic.AddInt32(&txStore.poolSize, 1)
}
dirtyAccounts[account] = true
}
@ -137,15 +127,16 @@ func (m *txSortedMap) forward(commitNonce uint64) map[string][]*pb.Transaction {
return removedTxs
}
// TODO (YH): persist and restore commitNonce and pendingNonce from db.
type nonceCache struct {
// commitNonces records each account's latest committed nonce in ledger.
commitNonces map[string]uint64
// pendingNonces records each account's latest nonce which has been included in
// priority queue. Invariant: pendingNonces[account] >= commitNonces[account]
pendingNonces map[string]uint64
pendingMu sync.RWMutex
}
// TODO (YH): restore commitNonce and pendingNonce from db.
func newNonceCache() *nonceCache {
return &nonceCache{
commitNonces: make(map[string]uint64),
@ -166,6 +157,8 @@ func (nc *nonceCache) setCommitNonce(account string, nonce uint64) {
}
func (nc *nonceCache) getPendingNonce(account string) uint64 {
nc.pendingMu.RLock()
defer nc.pendingMu.RUnlock()
nonce, ok := nc.pendingNonces[account]
if !ok {
return 1
@ -174,5 +167,7 @@ func (nc *nonceCache) getPendingNonce(account string) uint64 {
}
func (nc *nonceCache) setPendingNonce(account string, nonce uint64) {
nc.pendingMu.Lock()
defer nc.pendingMu.Unlock()
nc.pendingNonces[account] = nonce
}

View File

@ -1,37 +0,0 @@
package mempool
import (
"testing"
"github.com/meshplus/bitxhub-model/pb"
"github.com/stretchr/testify/assert"
)
func TestForward(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
defer cleanTestData()
txList := make([]*pb.Transaction, 0)
privKey1 := genPrivKey()
account1, _ := privKey1.PublicKey().Address()
tx1 := constructTx(uint64(1), &privKey1)
tx2 := constructTx(uint64(2), &privKey1)
tx3 := constructTx(uint64(3), &privKey1)
tx4 := constructTx(uint64(4), &privKey1)
tx5 := constructTx(uint64(6), &privKey1)
txList = append(txList, tx1, tx2, tx3, tx4, tx5)
err := mpi.processTransactions(txList)
ast.Nil(err)
list := mpi.txStore.allTxs[account1.String()]
ast.Equal(5, list.index.size())
ast.Equal(4, mpi.txStore.priorityIndex.size())
ast.Equal(1, mpi.txStore.parkingLotIndex.size())
removeList := list.forward(uint64(3))
ast.Equal(1, len(removeList))
ast.Equal(2, len(removeList[account1.String()]))
ast.Equal(uint64(1), removeList[account1.String()][0].Nonce)
ast.Equal(uint64(2), removeList[account1.String()][1].Nonce)
}

View File

@ -5,10 +5,6 @@ import (
"github.com/meshplus/bitxhub-kit/types"
"github.com/meshplus/bitxhub-model/pb"
raftproto "github.com/meshplus/bitxhub/pkg/order/etcdraft/proto"
"github.com/meshplus/bitxhub/pkg/peermgr"
cmap "github.com/orcaman/concurrent-map"
"github.com/sirupsen/logrus"
)
@ -24,74 +20,27 @@ const (
DefaultBatchTick = 500 * time.Millisecond
DefaultTxSetTick = 100 * time.Millisecond
DefaultFetchTxnTimeout = 3 * time.Second
)
// batch timer reasons
const (
StartReason1 = "first transaction set"
StartReason2 = "finish executing a batch"
StopReason1 = "generated a batch by batch timer"
StopReason2 = "generated a batch by batch size"
StopReason3 = "restart batch timer"
)
type LocalMissingTxnEvent struct {
Height uint64
MissingTxnHashList map[uint64]string
WaitC chan bool
}
type subscribeEvent struct {
txForwardC chan *TxSlice
localMissingTxnEvent chan *LocalMissingTxnEvent
fetchTxnRequestC chan *FetchTxnRequest
fetchTxnResponseC chan *FetchTxnResponse
getBlockC chan *constructBatchEvent
commitTxnC chan *raftproto.Ready
updateLeaderC chan uint64
pendingNonceC chan *getNonceRequest
}
type mempoolBatch struct {
missingTxnHashList map[uint64]string
txList []*pb.Transaction
}
type constructBatchEvent struct {
ready *raftproto.Ready
result chan *mempoolBatch
}
type Config struct {
ID uint64
BatchSize uint64
PoolSize uint64
TxSliceSize uint64
BatchTick time.Duration
FetchTimeout time.Duration
TxSliceTimeout time.Duration
PeerMgr peermgr.PeerManager
GetTransactionFunc func(hash *types.Hash) (*pb.Transaction, error)
ChainHeight uint64
Logger logrus.FieldLogger
}
type timerManager struct {
timeout time.Duration // default timeout of this timer
isActive cmap.ConcurrentMap // track all the timers with this timerName if it is active now
timeoutEventC chan bool
}
type txItem struct {
account string
tx *pb.Transaction
}
type getNonceRequest struct {
account string
waitC chan uint64
}
type ChainState struct {
Height uint64
BlockHash *types.Hash
TxHashList []*types.Hash
}

View File

@ -1,19 +0,0 @@
package mempool
import (
"testing"
"github.com/stretchr/testify/assert"
)
func TestPoolIsFull(t *testing.T) {
ast := assert.New(t)
mpi, _ := mockMempoolImpl()
defer cleanTestData()
isFull := mpi.poolIsFull()
ast.Equal(false, isFull)
mpi.txStore.poolSize = DefaultPoolSize
isFull = mpi.poolIsFull()
ast.Equal(true, isFull)
}

View File

@ -1,96 +0,0 @@
package mempool
import (
"strconv"
"sync/atomic"
"time"
"github.com/meshplus/bitxhub-model/pb"
raftproto "github.com/meshplus/bitxhub/pkg/order/etcdraft/proto"
cmap "github.com/orcaman/concurrent-map"
)
func (mpi *mempoolImpl) getBatchSeqNo() uint64 {
return atomic.LoadUint64(&mpi.batchSeqNo)
}
func (mpi *mempoolImpl) increaseBatchSeqNo() {
atomic.AddUint64(&mpi.batchSeqNo, 1)
}
func (mpi *mempoolImpl) msgToConsensusPbMsg(data []byte, tyr raftproto.RaftMessage_Type) *pb.Message {
rm := &raftproto.RaftMessage{
Type: tyr,
FromId: mpi.localID,
Data: data,
}
cmData, err := rm.Marshal()
if err != nil {
return nil
}
msg := &pb.Message{
Type: pb.Message_CONSENSUS,
Data: cmData,
}
return msg
}
func newSubscribe() *subscribeEvent {
return &subscribeEvent{
txForwardC: make(chan *TxSlice),
localMissingTxnEvent: make(chan *LocalMissingTxnEvent),
fetchTxnRequestC: make(chan *FetchTxnRequest),
updateLeaderC: make(chan uint64),
fetchTxnResponseC: make(chan *FetchTxnResponse),
commitTxnC: make(chan *raftproto.Ready),
getBlockC: make(chan *constructBatchEvent),
pendingNonceC: make(chan *getNonceRequest),
}
}
func (mpi *mempoolImpl) poolIsFull() bool {
return atomic.LoadInt32(&mpi.txStore.poolSize) >= DefaultPoolSize
}
func (mpi *mempoolImpl) isLeader() bool {
return mpi.leader == mpi.localID
}
func (mpi *mempoolImpl) isBatchTimerActive() bool {
return !mpi.batchTimerMgr.isActive.IsEmpty()
}
// startBatchTimer starts the batch timer and reset the batchTimerActive to true.
func (mpi *mempoolImpl) startBatchTimer(reason string) {
// stop old timer
mpi.stopBatchTimer(StopReason3)
mpi.logger.Debugf("Start batch timer, reason: %s", reason)
timestamp := time.Now().UnixNano()
key := strconv.FormatInt(timestamp, 10)
mpi.batchTimerMgr.isActive.Set(key, true)
time.AfterFunc(mpi.batchTimerMgr.timeout, func() {
if mpi.batchTimerMgr.isActive.Has(key) {
mpi.batchTimerMgr.timeoutEventC <- true
}
})
}
// stopBatchTimer stops the batch timer and reset the batchTimerActive to false.
func (mpi *mempoolImpl) stopBatchTimer(reason string) {
if mpi.batchTimerMgr.isActive.IsEmpty() {
return
}
mpi.logger.Debugf("Stop batch timer, reason: %s", reason)
mpi.batchTimerMgr.isActive = cmap.New()
}
// newTimer news a timer with default timeout.
func newTimer(d time.Duration) *timerManager {
return &timerManager{
timeout: d,
isActive: cmap.New(),
timeoutEventC: make(chan bool),
}
}

View File

@ -1,7 +1,6 @@
package order
import (
"context"
"github.com/meshplus/bitxhub-kit/types"
"github.com/meshplus/bitxhub-model/pb"
)
@ -21,13 +20,13 @@ type Order interface {
Commit() chan *pb.Block
// Step send msg to the consensus engine
Step(ctx context.Context, msg []byte) error
Step(msg []byte) error
// Ready means whether order has finished electing leader
Ready() error
// ReportState means block was persisted and report it to the consensus engine
ReportState(height uint64, hash *types.Hash)
ReportState(height uint64, blockHash *types.Hash, txHashList []*types.Hash)
// Quorum means minimum number of nodes in the cluster that can work
Quorum() uint64

View File

@ -5,17 +5,13 @@ import (
"fmt"
"path/filepath"
"sync"
"testing"
"time"
"github.com/golang/mock/gomock"
"github.com/meshplus/bitxhub-kit/storage/leveldb"
"github.com/meshplus/bitxhub-kit/types"
"github.com/meshplus/bitxhub-model/pb"
"github.com/meshplus/bitxhub/pkg/order"
raftproto "github.com/meshplus/bitxhub/pkg/order/etcdraft/proto"
"github.com/meshplus/bitxhub/pkg/order/mempool"
"github.com/meshplus/bitxhub/pkg/peermgr/mock_peermgr"
"github.com/sirupsen/logrus"
"github.com/spf13/viper"
)
@ -23,13 +19,13 @@ import (
type Node struct {
ID uint64
sync.RWMutex
height uint64 // current block height
commitC chan *pb.Block // block channel
logger logrus.FieldLogger // logger
reqLookUp *order.ReqLookUp // bloom filter
getTransactionFunc func(hash *types.Hash) (*pb.Transaction, error)
mempool mempool.MemPool // transaction pool
proposeC chan *raftproto.Ready // proposed listenReadyBlock, input channel
mempool mempool.MemPool // transaction pool
proposeC chan *raftproto.RequestBatch // proposed listenReadyBlock, input channel
stateC chan *mempool.ChainState
txCache *mempool.TxCache // cache the transactions received from api
lastExec uint64 // the index of the last-applied block
packSize int // maximum number of transaction packages
blockTick time.Duration // block packed period
@ -39,16 +35,12 @@ type Node struct {
}
func (n *Node) Start() error {
go n.txCache.ListenEvent()
go n.listenReadyBlock()
if err := n.mempool.Start(); err != nil {
return err
}
n.mempool.UpdateLeader(n.ID)
return nil
}
func (n *Node) Stop() {
n.mempool.Stop()
n.cancel()
}
@ -64,14 +56,15 @@ func (n *Node) Prepare(tx *pb.Transaction) error {
if err := n.Ready(); err != nil {
return err
}
return n.mempool.RecvTransaction(tx)
n.txCache.RecvTxC <- tx
return nil
}
func (n *Node) Commit() chan *pb.Block {
return n.commitC
}
func (n *Node) Step(ctx context.Context, msg []byte) error {
func (n *Node) Step(msg []byte) error {
return nil
}
@ -79,17 +72,13 @@ func (n *Node) Ready() error {
return nil
}
func (n *Node) ReportState(height uint64, hash *types.Hash) {
if err := n.reqLookUp.Build(); err != nil {
n.logger.Errorf("bloom filter persistence error", err)
}
if height%10 == 0 {
n.logger.WithFields(logrus.Fields{
"height": height,
"hash": hash.String(),
}).Info("Report checkpoint")
func (n *Node) ReportState(height uint64, blockHash *types.Hash, txHashList []*types.Hash) {
state := &mempool.ChainState{
Height: height,
BlockHash: blockHash,
TxHashList: txHashList,
}
n.stateC <- state
}
func (n *Node) Quorum() uint64 {
@ -101,77 +90,91 @@ func NewNode(opts ...order.Option) (order.Order, error) {
if err != nil {
return nil, fmt.Errorf("generate config: %w", err)
}
storage, err := leveldb.New(config.StoragePath)
if err != nil {
return nil, fmt.Errorf("new leveldb: %w", err)
}
reqLookUp, err := order.NewReqLookUp(storage, config.Logger)
if err != nil {
return nil, fmt.Errorf("new bloom filter: %w", err)
}
ctx, cancel := context.WithCancel(context.Background())
mockCtl := gomock.NewController(&testing.T{})
peerMgr := mock_peermgr.NewMockPeerManager(mockCtl)
peerMgr.EXPECT().Peers().Return(map[uint64]*pb.VpInfo{}).AnyTimes()
memConfig, err := generateMempoolConfig(config.RepoRoot)
mempoolConf := &mempool.Config{
ID: config.ID,
ChainHeight: config.Applied,
GetTransactionFunc: config.GetTransactionFunc,
PeerMgr: peerMgr,
Logger: config.Logger,
ID: config.ID,
ChainHeight: config.Applied,
Logger: config.Logger,
BatchSize: memConfig.BatchSize,
BatchTick: memConfig.BatchTick,
PoolSize: memConfig.PoolSize,
TxSliceSize: memConfig.TxSliceSize,
FetchTimeout: memConfig.FetchTimeout,
TxSliceTimeout: memConfig.TxSliceTimeout,
}
batchC := make(chan *raftproto.Ready, 10)
mempoolInst := mempool.NewMempool(mempoolConf, storage, batchC)
return &Node{
batchC := make(chan *raftproto.RequestBatch)
mempoolInst := mempool.NewMempool(mempoolConf)
txCache := mempool.NewTxCache(mempoolConf.TxSliceTimeout, mempoolConf.TxSliceSize, config.Logger)
soloNode := &Node{
ID: config.ID,
height: config.Applied,
commitC: make(chan *pb.Block, 1024),
reqLookUp: reqLookUp,
getTransactionFunc: config.GetTransactionFunc,
stateC: make(chan *mempool.ChainState),
lastExec: config.Applied,
mempool: mempoolInst,
txCache: txCache,
proposeC: batchC,
logger: config.Logger,
ctx: ctx,
cancel: cancel,
}, nil
}
soloNode.logger.Infof("========= SOLO lastExec = %d ", soloNode.lastExec)
return soloNode, nil
}
// Schedule to collect txs to the listenReadyBlock channel
func (n *Node) listenReadyBlock() {
go func() {
for {
select {
case proposal := <-n.proposeC:
n.logger.WithFields(logrus.Fields{
"proposal_height": proposal.Height,
"tx_count": len(proposal.TxList),
}).Debugf("Receive proposal from mempool")
if proposal.Height != n.lastExec+1 {
n.logger.Warningf("Expects to execute seq=%d, but get seq=%d, ignore it", n.lastExec+1, proposal.Height)
return
}
n.logger.Infof("======== Call execute, height=%d", proposal.Height)
block := &pb.Block{
BlockHeader: &pb.BlockHeader{
Version: []byte("1.0.0"),
Number: proposal.Height,
Timestamp: time.Now().UnixNano(),
},
Transactions: proposal.TxList,
}
n.commitC <- block
n.lastExec++
}
}
}()
for {
select {
case <-n.ctx.Done():
n.logger.Info("----- Exit listen ready block loop -----")
return
case proposal := <-n.proposeC:
n.logger.WithFields(logrus.Fields{
"proposal_height": proposal.Height,
"tx_count": len(proposal.TxHashes),
}).Debugf("Receive proposal from mempool")
// collect txs from proposalC
_, txs := n.mempool.GetBlockByHashList(proposal)
n.height++
block := &pb.Block{
BlockHeader: &pb.BlockHeader{
Version: []byte("1.0.0"),
Number: n.height,
Timestamp: time.Now().UnixNano(),
},
Transactions: txs,
case txSet := <-n.txCache.TxSetC:
if batch := n.mempool.ProcessTransactions(txSet.TxList, true); batch != nil {
n.proposeC <- batch
}
n.mempool.CommitTransactions(proposal)
n.mempool.IncreaseChainHeight()
n.commitC <- block
case state := <-n.stateC:
if state.Height%10 == 0 {
n.logger.WithFields(logrus.Fields{
"height": state.Height,
"hash": state.BlockHash.String(),
}).Info("Report checkpoint")
}
n.mempool.CommitTransactions(state)
}
}
}

View File

@ -28,7 +28,7 @@ func TestNode_Start(t *testing.T) {
assert.Nil(t, err)
// write config file for order module
fileData, err := ioutil.ReadFile("../../../config/order.toml")
fileData, err := ioutil.ReadFile("./testdata/order.toml")
require.Nil(t, err)
err = ioutil.WriteFile(filepath.Join(repoRoot, "order.toml"), fileData, 0644)
require.Nil(t, err)
@ -39,8 +39,8 @@ func TestNode_Start(t *testing.T) {
mockPeermgr.EXPECT().Peers().Return(peers).AnyTimes()
nodes := make(map[uint64]*pb.VpInfo)
vpInfo := & pb.VpInfo{
Id: uint64(1),
vpInfo := &pb.VpInfo{
Id: uint64(1),
Account: types.NewAddressByStr("000000000000000000000000000000000000000a").String(),
}
nodes[1] = vpInfo
@ -57,9 +57,8 @@ func TestNode_Start(t *testing.T) {
)
require.Nil(t, err)
err = order.Start()
_ = order.Start()
require.Nil(t, err)
privKey, err := asym.GenerateKeyPair(crypto.Secp256k1)
require.Nil(t, err)
@ -91,6 +90,8 @@ func TestNode_Start(t *testing.T) {
require.Equal(t, uint64(2), block.BlockHeader.Number)
require.Equal(t, 1, len(block.Transactions))
order.ReportState(block.Height(), block.BlockHash)
txHashList := make([]*types.Hash, 0)
txHashList = append(txHashList, tx.TransactionHash)
order.ReportState(block.Height(), block.BlockHash, txHashList)
order.Stop()
}

17
pkg/order/solo/testdata/order.toml vendored Normal file
View File

@ -0,0 +1,17 @@
[raft]
tick_timeout = "0.1s" # TickTimeout is the internal logical clock for the Node by a single tick, Election timeouts and heartbeat timeouts are in units of ticks.
election_tick = 10 # ElectionTick is the number of Node.Tick invocations that must pass between elections.
heartbeat_tick = 1 # HeartbeatTick is the number of Node.Tick invocations that must pass between heartbeats.
max_size_per_msg = 1048576 # 1024*1024, MaxSizePerMsg limits the max size of each append message.
max_inflight_msgs = 500 # MaxInflightMsgs limits the max number of in-flight append messages during optimistic replication phase.
check_quorum = true # Leader steps down when quorum is not active for an electionTimeout.
pre_vote = true # PreVote prevents reconnected node from disturbing network.
disable_proposal_forwarding = true # This prevents blocks from being accidentally proposed by followers.
[raft.mempool]
batch_size = 1 # How many transactions should the primary pack.
pool_size = 50000 # How many transactions could the txPool stores in total.
tx_slice_size = 1 # How many transactions should the node broadcast at once
batch_tick = "0.3s" # Block packaging time period.
tx_slice_timeout = "0.1s" # Node broadcasts transactions if there are cached transactions, although set_size isn't reached yet

View File

@ -3,7 +3,6 @@ package peermgr
import (
"context"
"fmt"
ma "github.com/multiformats/go-multiaddr"
"sync"
"time"
@ -20,6 +19,7 @@ import (
"github.com/meshplus/bitxhub/pkg/cert"
network "github.com/meshplus/go-lightp2p"
"github.com/sirupsen/logrus"
ma "github.com/multiformats/go-multiaddr"
)
const (

View File

@ -4,24 +4,24 @@ new = false
[[nodes]]
account = "0xc7F999b83Af6DF9e67d0a37Ee7e900bF38b3D013"
hosts = ["/ip4/127.0.0.1/tcp/4001/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5001/p2p/"]
id = 1
pid = "QmXi58fp9ZczF3Z5iz1yXAez3Hy5NYo1R8STHWKEM9XnTL"
[[nodes]]
account = "0x79a1215469FaB6f9c63c1816b45183AD3624bE34"
hosts = ["/ip4/127.0.0.1/tcp/4002/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5002/p2p/"]
id = 2
pid = "QmbmD1kzdsxRiawxu7bRrteDgW1ituXupR8GH6E2EUAHY4"
[[nodes]]
account = "0x97c8B516D19edBf575D72a172Af7F418BE498C37"
hosts = ["/ip4/127.0.0.1/tcp/4003/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5003/p2p/"]
id = 3
pid = "QmQUcDYCtqbpn5Nhaw4FAGxQaSSNvdWfAFcpQT9SPiezbS"
[[nodes]]
account = "0xc0Ff2e0b3189132D815b8eb325bE17285AC898f8"
hosts = ["/ip4/127.0.0.1/tcp/4004/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5004/p2p/"]
id = 4
pid = "QmQW3bFn8XX1t4W14Pmn37bPJUpUVBrBjnPuBZwPog3Qdy"

View File

@ -4,24 +4,24 @@ new = false
[[nodes]]
account = "0xc7F999b83Af6DF9e67d0a37Ee7e900bF38b3D013"
hosts = ["/ip4/127.0.0.1/tcp/4001/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5001/p2p/"]
id = 1
pid = "QmXi58fp9ZczF3Z5iz1yXAez3Hy5NYo1R8STHWKEM9XnTL"
[[nodes]]
account = "0x79a1215469FaB6f9c63c1816b45183AD3624bE34"
hosts = ["/ip4/127.0.0.1/tcp/4002/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5002/p2p/"]
id = 2
pid = "QmbmD1kzdsxRiawxu7bRrteDgW1ituXupR8GH6E2EUAHY4"
[[nodes]]
account = "0x97c8B516D19edBf575D72a172Af7F418BE498C37"
hosts = ["/ip4/127.0.0.1/tcp/4003/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5003/p2p/"]
id = 3
pid = "QmQUcDYCtqbpn5Nhaw4FAGxQaSSNvdWfAFcpQT9SPiezbS"
[[nodes]]
account = "0xc0Ff2e0b3189132D815b8eb325bE17285AC898f8"
hosts = ["/ip4/127.0.0.1/tcp/4004/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5004/p2p/"]
id = 4
pid = "QmQW3bFn8XX1t4W14Pmn37bPJUpUVBrBjnPuBZwPog3Qdy"

View File

@ -1,27 +1,26 @@
id = 3
n = 4
new = false
id = 3 # self id
n = 4 # the number of cluster nodes
[[nodes]]
account = "0xc7F999b83Af6DF9e67d0a37Ee7e900bF38b3D013"
hosts = ["/ip4/127.0.0.1/tcp/4001/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5001/p2p/"]
id = 1
pid = "QmXi58fp9ZczF3Z5iz1yXAez3Hy5NYo1R8STHWKEM9XnTL"
[[nodes]]
account = "0x79a1215469FaB6f9c63c1816b45183AD3624bE34"
hosts = ["/ip4/127.0.0.1/tcp/4002/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5002/p2p/"]
id = 2
pid = "QmbmD1kzdsxRiawxu7bRrteDgW1ituXupR8GH6E2EUAHY4"
[[nodes]]
account = "0x97c8B516D19edBf575D72a172Af7F418BE498C37"
hosts = ["/ip4/127.0.0.1/tcp/4003/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5003/p2p/"]
id = 3
pid = "QmQUcDYCtqbpn5Nhaw4FAGxQaSSNvdWfAFcpQT9SPiezbS"
[[nodes]]
account = "0xc0Ff2e0b3189132D815b8eb325bE17285AC898f8"
hosts = ["/ip4/127.0.0.1/tcp/4004/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5004/p2p/"]
id = 4
pid = "QmQW3bFn8XX1t4W14Pmn37bPJUpUVBrBjnPuBZwPog3Qdy"

View File

@ -4,24 +4,24 @@ new = false
[[nodes]]
account = "0xc7F999b83Af6DF9e67d0a37Ee7e900bF38b3D013"
hosts = ["/ip4/127.0.0.1/tcp/4001/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5001/p2p/"]
id = 1
pid = "QmXi58fp9ZczF3Z5iz1yXAez3Hy5NYo1R8STHWKEM9XnTL"
[[nodes]]
account = "0x79a1215469FaB6f9c63c1816b45183AD3624bE34"
hosts = ["/ip4/127.0.0.1/tcp/4002/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5002/p2p/"]
id = 2
pid = "QmbmD1kzdsxRiawxu7bRrteDgW1ituXupR8GH6E2EUAHY4"
[[nodes]]
account = "0x97c8B516D19edBf575D72a172Af7F418BE498C37"
hosts = ["/ip4/127.0.0.1/tcp/4003/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5003/p2p/"]
id = 3
pid = "QmQUcDYCtqbpn5Nhaw4FAGxQaSSNvdWfAFcpQT9SPiezbS"
[[nodes]]
account = "0xc0Ff2e0b3189132D815b8eb325bE17285AC898f8"
hosts = ["/ip4/127.0.0.1/tcp/4004/p2p/"]
hosts = ["/ip4/127.0.0.1/tcp/5004/p2p/"]
id = 4
pid = "QmQW3bFn8XX1t4W14Pmn37bPJUpUVBrBjnPuBZwPog3Qdy"

View File

@ -3,6 +3,8 @@ package tester
import (
"context"
"fmt"
"os"
"path"
"testing"
"time"
@ -44,6 +46,7 @@ func TestTester(t *testing.T) {
}
func setupNode(t *testing.T, path string) api.CoreAPI {
cleanStorage(path)
repoRoot, err := repo.PathRootWithDefault(path)
require.Nil(t, err)
@ -84,13 +87,12 @@ func newTesterBitXHub(rep *repo.Repo) (*app.BitXHub, error) {
order.WithPluginPath(rep.Config.Plugin),
order.WithNodes(m),
order.WithID(rep.NetworkConfig.ID),
order.WithIsNew(rep.NetworkConfig.IsNew),
order.WithIsNew(rep.NetworkConfig.New),
order.WithPeerManager(bxh.PeerMgr),
order.WithLogger(loggers.Logger(loggers.Order)),
order.WithApplied(chainMeta.Height),
order.WithDigest(chainMeta.BlockHash.String()),
order.WithGetChainMetaFunc(bxh.Ledger.GetChainMeta),
order.WithGetTransactionFunc(bxh.Ledger.GetTransaction),
order.WithGetBlockByHeightFunc(bxh.Ledger.GetBlock),
)
@ -111,3 +113,12 @@ func newTesterBitXHub(rep *repo.Repo) (*app.BitXHub, error) {
return bxh, nil
}
func cleanStorage(basePath string) {
filePath := path.Join(basePath,"storage")
err := os.RemoveAll(filePath)
if err != nil {
fmt.Printf("Clean storage failed, error: %s", err.Error())
return
}
}