mirror of
https://github.com/talent-plan/tinykv.git
synced 2025-01-13 13:50:43 +08:00
rename batchsystem to raftstore
Signed-off-by: Connor1996 <zbk602423539@gmail.com>
This commit is contained in:
parent
5691c2c3bc
commit
55c6151c9b
@ -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{
|
||||
|
@ -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
|
||||
|
||||
|
@ -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
|
||||
}
|
@ -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
|
||||
|
@ -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 {
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user