refactor(mempool): refactor code according to comments

This commit is contained in:
Alexader 2021-01-18 20:17:04 +08:00
parent f328d1d6a1
commit 2aed7f6ecd
8 changed files with 134 additions and 85 deletions

View File

@ -1,6 +1,6 @@
[raft]
batch_timeout = "0.3s" # Block packaging time period.
rebroadcast_timeout = "3m" # Node timeout tick to check if there exist txs to be rebroadcast
check_interval = "3m" # Node check interval to check if there exist txs to be rebroadcast
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.

View File

@ -27,7 +27,7 @@ type SyncerConfig struct {
type RAFT struct {
BatchTimeout time.Duration `mapstructure:"batch_timeout"`
RebroadcastTimeout time.Duration `mapstructure:"rebroadcast_timeout"`
CheckInterval time.Duration `mapstructure:"check_interval"`
TickTimeout time.Duration `mapstructure:"tick_timeout"`
ElectionTick int `mapstructure:"election_tick"`
HeartbeatTick int `mapstructure:"heartbeat_tick"`

View File

@ -39,15 +39,15 @@ type Node struct {
txCache *mempool.TxCache // cache the transactions received from api
batchTimerMgr *BatchTimer
proposeC chan *raftproto.RequestBatch // proposed ready, input channel
confChangeC <-chan raftpb.ConfChange // proposed cluster config changes
commitC chan *pb.CommitEvent // the hash commit channel
errorC chan<- error // errors from raft session
tickTimeout time.Duration // tick timeout
rebroadcastTimeout time.Duration // rebroadcast timeout
msgC chan []byte // receive messages from remote peer
stateC chan *mempool.ChainState // receive the executed block state
rebroadcastTicker chan *raftproto.TxSlice // receive the executed block state
proposeC chan *raftproto.RequestBatch // proposed ready, input channel
confChangeC <-chan raftpb.ConfChange // proposed cluster config changes
commitC chan *pb.CommitEvent // the hash commit channel
errorC chan<- error // errors from raft session
tickTimeout time.Duration // tick timeout
checkInterval time.Duration // interval for rebroadcast
msgC chan []byte // receive messages from remote peer
stateC chan *mempool.ChainState // receive the executed block state
rebroadcastTicker chan *raftproto.TxSlice // 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
@ -122,36 +122,36 @@ func NewNode(opts ...order.Option) (order.Order, error) {
snapCount = DefaultSnapshotCount
}
var rebroadcastTimeout time.Duration
if raftConfig.RAFT.RebroadcastTimeout == 0 {
rebroadcastTimeout = DefaultRebroadcastTimeout
var checkInterval time.Duration
if raftConfig.RAFT.CheckInterval == 0 {
checkInterval = DefaultCheckInterval
} else {
rebroadcastTimeout = raftConfig.RAFT.RebroadcastTimeout
checkInterval = raftConfig.RAFT.CheckInterval
}
node := &Node{
id: config.ID,
lastExec: config.Applied,
confChangeC: make(chan raftpb.ConfChange),
commitC: make(chan *pb.CommitEvent, 1024),
errorC: make(chan<- error),
msgC: make(chan []byte),
stateC: make(chan *mempool.ChainState),
proposeC: make(chan *raftproto.RequestBatch),
snapCount: snapCount,
repoRoot: repoRoot,
peerMgr: config.PeerMgr,
txCache: txCache,
batchTimerMgr: batchTimerMgr,
peers: peers,
logger: config.Logger,
getChainMetaFunc: config.GetChainMetaFunc,
storage: dbStorage,
raftStorage: raftStorage,
readyPool: readyPool,
ctx: context.Background(),
mempool: mempoolInst,
rebroadcastTimeout: rebroadcastTimeout,
id: config.ID,
lastExec: config.Applied,
confChangeC: make(chan raftpb.ConfChange),
commitC: make(chan *pb.CommitEvent, 1024),
errorC: make(chan<- error),
msgC: make(chan []byte),
stateC: make(chan *mempool.ChainState),
proposeC: make(chan *raftproto.RequestBatch),
snapCount: snapCount,
repoRoot: repoRoot,
peerMgr: config.PeerMgr,
txCache: txCache,
batchTimerMgr: batchTimerMgr,
peers: peers,
logger: config.Logger,
getChainMetaFunc: config.GetChainMetaFunc,
storage: dbStorage,
raftStorage: raftStorage,
readyPool: readyPool,
ctx: context.Background(),
mempool: mempoolInst,
checkInterval: checkInterval,
}
node.raftStorage.SnapshotCatchUpEntries = node.snapCount
@ -260,7 +260,7 @@ func (n *Node) run() {
n.snapshotIndex = snap.Metadata.Index
n.appliedIndex = snap.Metadata.Index
ticker := time.NewTicker(n.tickTimeout)
rebroadcastTicker := time.NewTicker(n.rebroadcastTimeout)
rebroadcastTicker := time.NewTicker(n.checkInterval)
defer ticker.Stop()
defer rebroadcastTicker.Stop()
@ -328,7 +328,7 @@ func (n *Node) run() {
case <-rebroadcastTicker.C:
// check periodically if there are long-pending txs in mempool
rebroadcastTxs := n.mempool.GetTimeoutTransactions(n.rebroadcastTimeout)
rebroadcastTxs := n.mempool.GetTimeoutTransactions(n.checkInterval)
for _, txSlice := range rebroadcastTxs {
txSet := &raftproto.TxSlice{TxList: txSlice}
data, err := txSet.Marshal()

View File

@ -14,9 +14,9 @@ import (
)
const (
DefaultBatchTick = 500 * time.Millisecond
DefaultSnapshotCount = 1000
DefaultRebroadcastTimeout = 3 * time.Minute
DefaultBatchTick = 500 * time.Millisecond
DefaultSnapshotCount = 1000
DefaultCheckInterval = 3 * time.Minute
)
func generateRaftPeers(config *order.Config) ([]raft.Peer, error) {

View File

@ -24,17 +24,6 @@ func (oik *orderedIndexKey) Less(than btree.Item) bool {
return oik.nonce < other.nonce
}
type sortedTtlKey struct {
account string
nonce uint64
liveTime int64 // the latest live time of tx after it is received or rebroadcast
}
func (stk *sortedTtlKey) Less(than btree.Item) bool {
other := than.(*sortedTtlKey)
return stk.liveTime < other.liveTime
}
type sortedNonceKey struct {
nonce uint64
}
@ -48,7 +37,7 @@ func (snk *sortedNonceKey) Less(item btree.Item) bool {
type orderedTimeoutKey struct {
account string
nonce uint64
timestamp int64 // the timestamp of tx when it is received
timestamp int64 // the timestamp of index key created
}
func (otk *orderedTimeoutKey) Less(than btree.Item) bool {

View File

@ -101,14 +101,10 @@ func (mpi *mempoolImpl) processDirtyAccount(dirtyAccounts map[string]bool) {
readyTxs, nonReadyTxs, nextDemandNonce := list.filterReady(pendingNonce)
mpi.txStore.nonceCache.setPendingNonce(account, nextDemandNonce)
// insert ready txs into priorityIndex and set ttlIndex for these txs.
// insert ready txs into priorityIndex.
for _, tx := range readyTxs {
if !mpi.txStore.priorityIndex.data.Has(makeTimeoutKey(account, tx)) {
mpi.txStore.priorityIndex.insertByTimeoutKey(account, tx)
if list.items[tx.Nonce].local {
// no need to rebroadcast tx from other nodes to reduce network overhead
mpi.txStore.ttlIndex.insertByTtlKey(account, tx.Nonce, tx.Timestamp)
}
}
}
mpi.txStore.updateEarliestTimestamp()
@ -281,67 +277,60 @@ func (mpi *mempoolImpl) processCommitTransactions(state *ChainState) {
func (mpi *mempoolImpl) GetTimeoutTransactions(rebroadcastDuration time.Duration) [][]*pb.Transaction {
// all the tx whose live time is less than lowBoundTime should be rebroadcast
mpi.logger.Debugf("------- Start gathering timeout txs, ttl index len is %d -----------", mpi.txStore.ttlIndex.index.Len())
mpi.logger.Debugf("Start gathering timeout txs, ttl index len is %d", mpi.txStore.ttlIndex.index.Len())
currentTime := time.Now().UnixNano()
if currentTime < mpi.txStore.earliestTimestamp+rebroadcastDuration.Nanoseconds() {
// if the latest incoming tx has not exceeded the timeout limit, then none will be timeout
return [][]*pb.Transaction{}
}
txList := make([]*pb.Transaction, 0)
timeoutItems := make([]*sortedTtlKey, 0)
timeoutItems := make([]*orderedTimeoutKey, 0)
mpi.txStore.ttlIndex.index.Ascend(func(i btree.Item) bool {
item := i.(*sortedTtlKey)
if item.liveTime > math.MaxInt64 {
item := i.(*orderedTimeoutKey)
if item.timestamp > math.MaxInt64 {
// TODO(tyx): if this tx has rebroadcast many times and exceeded a final limit,
// it is expired and will be removed from mempool
return true
}
// if this tx has not exceeded the rebroadcast duration, break iteration
timeoutTime := item.liveTime + rebroadcastDuration.Nanoseconds()
txMap, ok := mpi.txStore.allTxs[item.account]
timeoutTime := item.timestamp + rebroadcastDuration.Nanoseconds()
_, ok := mpi.txStore.allTxs[item.account]
if !ok || currentTime < timeoutTime {
return false
}
tx := txMap.items[item.nonce].tx
txList = append(txList, tx)
timeoutItems = append(timeoutItems, item)
return true
})
if len(txList) == 0 {
return [][]*pb.Transaction{}
}
for _, item := range timeoutItems {
// update the liveTime of timeout txs
item.liveTime = currentTime
item.timestamp = currentTime
mpi.txStore.ttlIndex.items[makeAccountNonceKey(item.account, item.nonce)] = currentTime
}
// shard txList into fixed size in case txList is too large to broadcast one time
return shardTxList(txList, mpi.txSliceSize)
return mpi.shardTxList(timeoutItems, mpi.txSliceSize)
}
func shardTxList(txs []*pb.Transaction, batchLen uint64) [][]*pb.Transaction {
func (mpi *mempoolImpl) shardTxList(timeoutItems []*orderedTimeoutKey, batchLen uint64) [][]*pb.Transaction {
begin := uint64(0)
end := uint64(len(txs)) - 1
totalLen := uint64(len(txs))
end := uint64(len(timeoutItems)) - 1
totalLen := uint64(len(timeoutItems))
// shape txs to fixed size in case totalLen is too large
// shape timeout txs to batch size in case totalLen is too large
batchNums := totalLen / batchLen
if totalLen%batchLen != 0 {
batchNums++
}
shardedLists := make([][]*pb.Transaction, 0, batchNums)
for i := uint64(0); i <= batchNums; i++ {
for i := uint64(0); i < batchNums; i++ {
actualLen := batchLen
if end-begin+1 < batchLen {
actualLen = end - begin + 1
}
if actualLen == 0 {
continue
}
shardedList := make([]*pb.Transaction, actualLen)
for j := uint64(0); j < batchLen && begin <= end; j++ {
shardedList[j] = txs[begin]
txMap, _ := mpi.txStore.allTxs[timeoutItems[begin].account]
shardedList[j] = txMap.items[timeoutItems[begin].nonce].tx
begin++
}
shardedLists = append(shardedLists, shardedList)

View File

@ -65,6 +65,10 @@ func (txStore *transactionStore) insertTxs(txs map[string][]*pb.Transaction, isL
}
txList.items[tx.Nonce] = txItem
txList.index.insertBySortedNonceKey(tx)
if isLocal {
// no need to rebroadcast tx from other nodes to reduce network overhead
txStore.ttlIndex.insertByTtlKey(account, tx.Nonce, tx.Timestamp)
}
}
dirtyAccounts[account] = true
}
@ -88,7 +92,7 @@ func (txStore *transactionStore) updateEarliestTimestamp() {
earliestTime := int64(math.MaxInt64)
latestItem := txStore.ttlIndex.index.Min()
if latestItem != nil {
earliestTime = latestItem.(*sortedTtlKey).liveTime
earliestTime = latestItem.(*orderedTimeoutKey).timestamp
}
txStore.earliestTimestamp = earliestTime
}
@ -205,7 +209,7 @@ func newTxLiveTimeMap() *txLiveTimeMap {
}
func (tlm *txLiveTimeMap) insertByTtlKey(account string, nonce uint64, liveTime int64) {
tlm.index.ReplaceOrInsert(&sortedTtlKey{account, nonce, liveTime})
tlm.index.ReplaceOrInsert(&orderedTimeoutKey{account, nonce, liveTime})
tlm.items[makeAccountNonceKey(account, nonce)] = liveTime
}
@ -216,7 +220,7 @@ func (tlm *txLiveTimeMap) removeByTtlKey(txs map[string][]*pb.Transaction) {
if !ok {
return
}
tlm.index.Delete(&sortedTtlKey{account, tx.Nonce, liveTime})
tlm.index.Delete(&orderedTimeoutKey{account, tx.Nonce, liveTime})
delete(tlm.items, makeAccountNonceKey(account, tx.Nonce))
}
}

View File

@ -10,6 +10,7 @@ import (
peer "github.com/libp2p/go-libp2p-core/peer"
pb "github.com/meshplus/bitxhub-model/pb"
events "github.com/meshplus/bitxhub/internal/model/events"
peermgr "github.com/meshplus/bitxhub/pkg/peermgr"
network "github.com/meshplus/go-lightp2p"
reflect "reflect"
)
@ -227,3 +228,69 @@ func (mr *MockPeerManagerMockRecorder) Disconnect(vpInfos interface{}) *gomock.C
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Disconnect", reflect.TypeOf((*MockPeerManager)(nil).Disconnect), vpInfos)
}
// PierManager mocks base method
func (m *MockPeerManager) PierManager() peermgr.PierManager {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "PierManager")
ret0, _ := ret[0].(peermgr.PierManager)
return ret0
}
// PierManager indicates an expected call of PierManager
func (mr *MockPeerManagerMockRecorder) PierManager() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "PierManager", reflect.TypeOf((*MockPeerManager)(nil).PierManager))
}
// MockPierManager is a mock of PierManager interface
type MockPierManager struct {
ctrl *gomock.Controller
recorder *MockPierManagerMockRecorder
}
// MockPierManagerMockRecorder is the mock recorder for MockPierManager
type MockPierManagerMockRecorder struct {
mock *MockPierManager
}
// NewMockPierManager creates a new mock instance
func NewMockPierManager(ctrl *gomock.Controller) *MockPierManager {
mock := &MockPierManager{ctrl: ctrl}
mock.recorder = &MockPierManagerMockRecorder{mock}
return mock
}
// EXPECT returns an object that allows the caller to indicate expected use
func (m *MockPierManager) EXPECT() *MockPierManagerMockRecorder {
return m.recorder
}
// Piers mocks base method
func (m *MockPierManager) Piers() *peermgr.Piers {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "Piers")
ret0, _ := ret[0].(*peermgr.Piers)
return ret0
}
// Piers indicates an expected call of Piers
func (mr *MockPierManagerMockRecorder) Piers() *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Piers", reflect.TypeOf((*MockPierManager)(nil).Piers))
}
// AskPierMaster mocks base method
func (m *MockPierManager) AskPierMaster(arg0 string) (bool, error) {
m.ctrl.T.Helper()
ret := m.ctrl.Call(m, "AskPierMaster", arg0)
ret0, _ := ret[0].(bool)
ret1, _ := ret[1].(error)
return ret0, ret1
}
// AskPierMaster indicates an expected call of AskPierMaster
func (mr *MockPierManagerMockRecorder) AskPierMaster(arg0 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AskPierMaster", reflect.TypeOf((*MockPierManager)(nil).AskPierMaster), arg0)
}