rename batchsystem to raftstore

Signed-off-by: Connor1996 <zbk602423539@gmail.com>
This commit is contained in:
Connor1996 2020-04-23 13:49:04 +08:00 committed by Connor
parent 5691c2c3bc
commit 55c6151c9b
7 changed files with 23 additions and 20 deletions

View File

@ -22,11 +22,11 @@ type Node struct {
clusterID uint64
store *metapb.Store
cfg *config.Config
system *RaftBatchSystem
system *Raftstore
schedulerClient scheduler_client.Client
}
func NewNode(system *RaftBatchSystem, cfg *config.Config, schedulerClient scheduler_client.Client) *Node {
func NewNode(system *Raftstore, cfg *config.Config, schedulerClient scheduler_client.Client) *Node {
return &Node{
clusterID: schedulerClient.GetClusterID((context.TODO())),
store: &metapb.Store{

View File

@ -10,6 +10,9 @@ import (
type raftWorker struct {
pr *router
// receiver of messages should sent to raft, including:
// * raft command from `raftStorage`
// * raft inner messages from other peers sent by network
raftCh chan message.Msg
ctx *GlobalContext

View File

@ -105,7 +105,7 @@ type Transport interface {
/// loadPeers loads peers in this store. It scans the db engine, loads all regions and their peers from it
/// WARN: This store should not be used before initialized.
func (bs *RaftBatchSystem) loadPeers() ([]*peer, error) {
func (bs *Raftstore) loadPeers() ([]*peer, error) {
// Scan region meta to get saved regions.
startKey := meta.RegionMetaMinKey
endKey := meta.RegionMetaMaxKey
@ -175,7 +175,7 @@ func (bs *RaftBatchSystem) loadPeers() ([]*peer, error) {
return regionPeers, nil
}
func (bs *RaftBatchSystem) clearStaleMeta(kvWB, raftWB *engine_util.WriteBatch, originState *rspb.RegionLocalState) {
func (bs *Raftstore) clearStaleMeta(kvWB, raftWB *engine_util.WriteBatch, originState *rspb.RegionLocalState) {
region := originState.Region
raftState, err := meta.GetRaftLocalState(bs.ctx.engine.Raft, region.Id)
if err != nil {
@ -199,7 +199,7 @@ type workers struct {
wg *sync.WaitGroup
}
type RaftBatchSystem struct {
type Raftstore struct {
ctx *GlobalContext
storeState *storeState
router *router
@ -209,7 +209,7 @@ type RaftBatchSystem struct {
wg *sync.WaitGroup
}
func (bs *RaftBatchSystem) start(
func (bs *Raftstore) start(
meta *metapb.Store,
cfg *config.Config,
engines *engine_util.Engines,
@ -260,7 +260,7 @@ func (bs *RaftBatchSystem) start(
return nil
}
func (bs *RaftBatchSystem) startWorkers(peers []*peer) {
func (bs *Raftstore) startWorkers(peers []*peer) {
ctx := bs.ctx
workers := bs.workers
router := bs.router
@ -283,7 +283,7 @@ func (bs *RaftBatchSystem) startWorkers(peers []*peer) {
go bs.tickDriver.run()
}
func (bs *RaftBatchSystem) shutDown() {
func (bs *Raftstore) shutDown() {
close(bs.closeCh)
bs.wg.Wait()
bs.tickDriver.stop()
@ -299,15 +299,15 @@ func (bs *RaftBatchSystem) shutDown() {
workers.wg.Wait()
}
func CreateRaftBatchSystem(cfg *config.Config) (*RaftstoreRouter, *RaftBatchSystem) {
func CreateRaftstore(cfg *config.Config) (*RaftstoreRouter, *Raftstore) {
storeSender, storeState := newStoreState(cfg)
router := newRouter(storeSender)
raftBatchSystem := &RaftBatchSystem{
raftstore := &Raftstore{
router: router,
storeState: storeState,
tickDriver: newTickDriver(cfg.RaftBaseTickInterval, router, storeState.ticker),
closeCh: make(chan struct{}),
wg: new(sync.WaitGroup),
}
return NewRaftstoreRouter(router), raftBatchSystem
return NewRaftstoreRouter(router), raftstore
}

View File

@ -31,7 +31,7 @@ type RaftStorage struct {
node *raftstore.Node
snapManager *snap.SnapManager
raftRouter *raftstore.RaftstoreRouter
batchSystem *raftstore.RaftBatchSystem
raftSystem *raftstore.Raftstore
resolveWorker *worker.Worker
snapWorker *worker.Worker
@ -182,7 +182,7 @@ func (rs *RaftStorage) Start() error {
if err != nil {
return err
}
rs.raftRouter, rs.batchSystem = raftstore.CreateRaftBatchSystem(cfg)
rs.raftRouter, rs.raftSystem = raftstore.CreateRaftstore(cfg)
rs.resolveWorker = worker.NewWorker("resolver", &rs.wg)
resolveSender := rs.resolveWorker.Sender()
@ -198,7 +198,7 @@ func (rs *RaftStorage) Start() error {
raftClient := newRaftClient(cfg)
trans := NewServerTransport(raftClient, snapSender, rs.raftRouter, resolveSender)
rs.node = raftstore.NewNode(rs.batchSystem, rs.config, schedulerClient)
rs.node = raftstore.NewNode(rs.raftSystem, rs.config, schedulerClient)
err = rs.node.Start(context.TODO(), rs.engines, trans, rs.snapManager)
if err != nil {
return err

View File

@ -148,9 +148,9 @@ func (c *NodeSimulator) RunStore(cfg *config.Config, engine *engine_util.Engines
c.Lock()
defer c.Unlock()
raftRouter, batchSystem := raftstore.CreateRaftBatchSystem(cfg)
raftRouter, raftSystem := raftstore.CreateRaftstore(cfg)
snapManager := snap.NewSnapManager(cfg.DBPath + "/snap")
node := raftstore.NewNode(batchSystem, cfg, c.schedulerClient)
node := raftstore.NewNode(raftSystem, cfg, c.schedulerClient)
err := node.Start(ctx, engine, c.trans, snapManager)
if err != nil {

View File

@ -634,7 +634,7 @@ func (m *Response) GetSnap() *SnapResponse {
}
type ChangePeerRequest struct {
// This can be only called in internal RaftStore now.
// This can be only called in internal Raftstore now.
ChangeType eraftpb.ConfChangeType `protobuf:"varint,1,opt,name=change_type,json=changeType,proto3,enum=eraftpb.ConfChangeType" json:"change_type,omitempty"`
Peer *metapb.Peer `protobuf:"bytes,2,opt,name=peer" json:"peer,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -737,7 +737,7 @@ func (m *ChangePeerResponse) GetRegion() *metapb.Region {
}
type SplitRequest struct {
// This can be only called in internal RaftStore now.
// This can be only called in internal Raftstore now.
// The split_key has to exist in the splitting region.
SplitKey []byte `protobuf:"bytes,1,opt,name=split_key,json=splitKey,proto3" json:"split_key,omitempty"`
// We split the region into two. The first uses the origin

View File

@ -60,7 +60,7 @@ message Response {
}
message ChangePeerRequest {
// This can be only called in internal RaftStore now.
// This can be only called in internal Raftstore now.
eraftpb.ConfChangeType change_type = 1;
metapb.Peer peer = 2;
}
@ -70,7 +70,7 @@ message ChangePeerResponse {
}
message SplitRequest {
// This can be only called in internal RaftStore now.
// This can be only called in internal Raftstore now.
// The split_key has to exist in the splitting region.
bytes split_key = 1;
// We split the region into two. The first uses the origin