server: Move wal bootstrap from cluster to storage

This commit is contained in:
Marek Siarkowicz 2021-07-21 15:57:53 +02:00
parent d3abf774ea
commit 4884e7d8cf
2 changed files with 69 additions and 61 deletions

View File

@ -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

View File

@ -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,