mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
server: Move wal bootstrap from cluster to storage
This commit is contained in:
parent
d3abf774ea
commit
4884e7d8cf
@ -72,21 +72,42 @@ func bootstrap(cfg config.ServerConfig) (b *bootstrappedServer, err error) {
|
||||
if terr := fileutil.TouchDirAll(cfg.MemberDir()); terr != nil {
|
||||
return nil, fmt.Errorf("cannot access member directory: %v", terr)
|
||||
}
|
||||
|
||||
haveWAL := wal.Exist(cfg.WALDir())
|
||||
s, err := bootstrapStorage(cfg, haveWAL, ss, prt)
|
||||
st := v2store.New(StoreClusterPrefix, StoreKeysPrefix)
|
||||
backend, err := bootstrapBackend(cfg, haveWAL, st, ss)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var (
|
||||
bwal *bootstrappedWAL
|
||||
)
|
||||
|
||||
if haveWAL {
|
||||
if err = fileutil.IsDirWriteable(cfg.WALDir()); err != nil {
|
||||
return nil, fmt.Errorf("cannot write to WAL directory: %v", err)
|
||||
}
|
||||
bwal = bootstrapWALFromSnapshot(cfg, backend.snapshot)
|
||||
}
|
||||
|
||||
cluster, err := bootstrapCluster(cfg, bwal, prt)
|
||||
if err != nil {
|
||||
backend.be.Close()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
cluster, err := bootstrapCluster(cfg, haveWAL, s, prt, ss)
|
||||
s, err := bootstrapStorage(cfg, st, backend, bwal, cluster)
|
||||
if err != nil {
|
||||
s.backend.be.Close()
|
||||
backend.be.Close()
|
||||
return nil, err
|
||||
}
|
||||
raft := bootstrapRaft(cfg, haveWAL, cluster.cl, cluster.wal)
|
||||
if !haveWAL {
|
||||
cluster.cl.SetID(cluster.nodeID, cluster.cl.ID())
|
||||
|
||||
err = cluster.Finalize(cfg, s)
|
||||
if err != nil {
|
||||
backend.be.Close()
|
||||
return nil, err
|
||||
}
|
||||
raft := bootstrapRaft(cfg, cluster, s.wal)
|
||||
return &bootstrappedServer{
|
||||
prt: prt,
|
||||
ss: ss,
|
||||
@ -106,6 +127,7 @@ type bootstrappedServer struct {
|
||||
|
||||
type bootstrappedStorage struct {
|
||||
backend *bootstrappedBackend
|
||||
wal *bootstrappedWAL
|
||||
st v2store.Store
|
||||
}
|
||||
|
||||
@ -119,7 +141,6 @@ type bootstrappedBackend struct {
|
||||
|
||||
type bootstrapedCluster struct {
|
||||
remotes []*membership.Member
|
||||
wal *bootstrappedWAL
|
||||
cl *membership.RaftCluster
|
||||
nodeID types.ID
|
||||
}
|
||||
@ -133,17 +154,15 @@ type bootstrappedRaft struct {
|
||||
storage *raft.MemoryStorage
|
||||
}
|
||||
|
||||
func bootstrapStorage(cfg config.ServerConfig, haveWAL bool, ss *snap.Snapshotter, prt http.RoundTripper) (b *bootstrappedStorage, err error) {
|
||||
st := v2store.New(StoreClusterPrefix, StoreKeysPrefix)
|
||||
|
||||
backend, err := bootstrapBackend(cfg, haveWAL, st, ss)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
func bootstrapStorage(cfg config.ServerConfig, st v2store.Store, be *bootstrappedBackend, wal *bootstrappedWAL, cl *bootstrapedCluster) (b *bootstrappedStorage, err error) {
|
||||
if wal == nil {
|
||||
wal = bootstrapNewWAL(cfg, cl)
|
||||
}
|
||||
|
||||
return &bootstrappedStorage{
|
||||
backend: backend,
|
||||
backend: be,
|
||||
st: st,
|
||||
wal: wal,
|
||||
}, nil
|
||||
}
|
||||
|
||||
@ -233,49 +252,19 @@ func maybeDefragBackend(cfg config.ServerConfig, be backend.Backend) error {
|
||||
return be.Defrag()
|
||||
}
|
||||
|
||||
func bootstrapCluster(cfg config.ServerConfig, haveWAL bool, storage *bootstrappedStorage, prt http.RoundTripper, ss *snap.Snapshotter) (c *bootstrapedCluster, err error) {
|
||||
c = &bootstrapedCluster{}
|
||||
var (
|
||||
bwal *bootstrappedWAL
|
||||
)
|
||||
if haveWAL {
|
||||
if err = fileutil.IsDirWriteable(cfg.WALDir()); err != nil {
|
||||
return nil, fmt.Errorf("cannot write to WAL directory: %v", err)
|
||||
}
|
||||
bwal = bootstrapWALFromSnapshot(cfg, storage.backend.snapshot)
|
||||
}
|
||||
|
||||
func bootstrapCluster(cfg config.ServerConfig, bwal *bootstrappedWAL, prt http.RoundTripper) (c *bootstrapedCluster, err error) {
|
||||
switch {
|
||||
case !haveWAL && !cfg.NewCluster:
|
||||
case bwal == nil && !cfg.NewCluster:
|
||||
c, err = bootstrapExistingClusterNoWAL(cfg, prt)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
c.wal = bootstrapNewWAL(cfg, c.cl, c.nodeID)
|
||||
case !haveWAL && cfg.NewCluster:
|
||||
case bwal == nil && cfg.NewCluster:
|
||||
c, err = bootstrapNewClusterNoWAL(cfg, prt)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
c.wal = bootstrapNewWAL(cfg, c.cl, c.nodeID)
|
||||
case haveWAL:
|
||||
case bwal != nil && bwal.haveWAL:
|
||||
c, err = bootstrapClusterWithWAL(cfg, bwal.meta)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
c.wal = bwal
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported bootstrap config")
|
||||
}
|
||||
c.cl.SetStore(storage.st)
|
||||
c.cl.SetBackend(schema.NewMembershipBackend(cfg.Logger, storage.backend.be))
|
||||
if haveWAL {
|
||||
c.cl.Recover(api.UpdateCapability)
|
||||
if c.cl.Version() != nil && !c.cl.Version().LessThan(semver.Version{Major: 3}) && !storage.backend.beExist {
|
||||
bepath := cfg.BackendPath()
|
||||
os.RemoveAll(bepath)
|
||||
return nil, fmt.Errorf("database file (%v) of the backend is missing", bepath)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return c, nil
|
||||
}
|
||||
@ -425,13 +414,30 @@ func recoverSnapshot(cfg config.ServerConfig, st v2store.Store, be backend.Backe
|
||||
return snapshot, be, nil
|
||||
}
|
||||
|
||||
func bootstrapRaft(cfg config.ServerConfig, haveWAL bool, cl *membership.RaftCluster, bwal *bootstrappedWAL) *bootstrappedRaft {
|
||||
switch {
|
||||
case !haveWAL && !cfg.NewCluster:
|
||||
return bootstrapRaftFromCluster(cfg, cl, nil, bwal)
|
||||
case !haveWAL && cfg.NewCluster:
|
||||
return bootstrapRaftFromCluster(cfg, cl, cl.MemberIDs(), bwal)
|
||||
case haveWAL:
|
||||
func (c *bootstrapedCluster) Finalize(cfg config.ServerConfig, s *bootstrappedStorage) error {
|
||||
if !s.wal.haveWAL {
|
||||
c.cl.SetID(c.nodeID, c.cl.ID())
|
||||
}
|
||||
c.cl.SetStore(s.st)
|
||||
c.cl.SetBackend(schema.NewMembershipBackend(cfg.Logger, s.backend.be))
|
||||
if s.wal.haveWAL {
|
||||
c.cl.Recover(api.UpdateCapability)
|
||||
if c.cl.Version() != nil && !c.cl.Version().LessThan(semver.Version{Major: 3}) && !s.backend.beExist {
|
||||
bepath := cfg.BackendPath()
|
||||
os.RemoveAll(bepath)
|
||||
return fmt.Errorf("database file (%v) of the backend is missing", bepath)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func bootstrapRaft(cfg config.ServerConfig, cluster *bootstrapedCluster, bwal *bootstrappedWAL) *bootstrappedRaft {
|
||||
switch {
|
||||
case !bwal.haveWAL && !cfg.NewCluster:
|
||||
return bootstrapRaftFromCluster(cfg, cluster.cl, nil, bwal)
|
||||
case !bwal.haveWAL && cfg.NewCluster:
|
||||
return bootstrapRaftFromCluster(cfg, cluster.cl, cluster.cl.MemberIDs(), bwal)
|
||||
case bwal.haveWAL:
|
||||
return bootstrapRaftFromWAL(cfg, bwal)
|
||||
default:
|
||||
cfg.Logger.Panic("unsupported bootstrap config")
|
||||
@ -520,6 +526,7 @@ func bootstrapWALFromSnapshot(cfg config.ServerConfig, snapshot *raftpb.Snapshot
|
||||
ents: ents,
|
||||
snapshot: snap,
|
||||
meta: meta,
|
||||
haveWAL: true,
|
||||
}
|
||||
|
||||
if cfg.ForceNewCluster {
|
||||
@ -590,11 +597,11 @@ type snapshotMetadata struct {
|
||||
nodeID, clusterID types.ID
|
||||
}
|
||||
|
||||
func bootstrapNewWAL(cfg config.ServerConfig, cl *membership.RaftCluster, nodeID types.ID) *bootstrappedWAL {
|
||||
func bootstrapNewWAL(cfg config.ServerConfig, cl *bootstrapedCluster) *bootstrappedWAL {
|
||||
metadata := pbutil.MustMarshal(
|
||||
&etcdserverpb.Metadata{
|
||||
NodeID: uint64(nodeID),
|
||||
ClusterID: uint64(cl.ID()),
|
||||
NodeID: uint64(cl.nodeID),
|
||||
ClusterID: uint64(cl.cl.ID()),
|
||||
},
|
||||
)
|
||||
w, err := wal.Create(cfg.Logger, cfg.WALDir(), metadata)
|
||||
@ -613,6 +620,7 @@ func bootstrapNewWAL(cfg config.ServerConfig, cl *membership.RaftCluster, nodeID
|
||||
type bootstrappedWAL struct {
|
||||
lg *zap.Logger
|
||||
|
||||
haveWAL bool
|
||||
w *wal.WAL
|
||||
st *raftpb.HardState
|
||||
ents []raftpb.Entry
|
||||
|
@ -320,7 +320,7 @@ func NewServer(cfg config.ServerConfig) (srv *EtcdServer, err error) {
|
||||
errorc: make(chan error, 1),
|
||||
v2store: b.storage.st,
|
||||
snapshotter: b.ss,
|
||||
r: *b.raft.newRaftNode(b.ss, b.cluster.wal.w, b.cluster.cl),
|
||||
r: *b.raft.newRaftNode(b.ss, b.storage.wal.w, b.cluster.cl),
|
||||
id: b.cluster.nodeID,
|
||||
attributes: membership.Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
|
||||
cluster: b.cluster.cl,
|
||||
|
Loading…
x
Reference in New Issue
Block a user