Files
seaweedFS/weed/server/raft_server.go
Chris Lu 753e1db096 Prevent split-brain: Persistent ClusterID and Join Validation (#8022)
* Prevent split-brain: Persistent ClusterID and Join Validation

- Persist ClusterId in Raft store to survive restarts.
- Validate ClusterId on Raft command application (piggybacked on MaxVolumeId).
- Prevent masters with conflicting ClusterIds from joining/operating together.
- Update Telemetry to report the persistent ClusterId.

* Refine ClusterID validation based on feedback

- Improved error message in cluster_commands.go.
- Added ClusterId mismatch check in RaftServer.Recovery.

* Handle Raft errors and support Hashicorp Raft for ClusterId

- Check for errors when persisting ClusterId in legacy Raft.
- Implement ClusterId generation and persistence for Hashicorp Raft leader changes.
- Ensure consistent error logging.

* Refactor ClusterId validation

- Centralize ClusterId mismatch check in Topology.SetClusterId.
- Simplify MaxVolumeIdCommand.Apply and RaftServer.Recovery to rely on SetClusterId.

* Fix goroutine leak and add timeout

- Handle channel closure in Hashicorp Raft leader listener.
- Add timeout to Raft Apply call to prevent blocking.

* Fix deadlock in legacy Raft listener

- Wrap ClusterId generation/persistence in a goroutine to avoid blocking the Raft event loop (deadlock).

* Rename ClusterId to SystemId

- Renamed ClusterId to SystemId across the codebase (protobuf, topology, server, telemetry).
- Regenerated telemetry.pb.go with new field.

* Rename SystemId to TopologyId

- Rename to SystemId was intermediate step.
- Final name is TopologyId for the persistent cluster identifier.
- Updated protobuf, topology, raft server, master server, and telemetry.

* Optimize Hashicorp Raft listener

- Integrated TopologyId generation into existing monitorLeaderLoop.
- Removed extra goroutine in master_server.go.

* Fix optimistic TopologyId update

- Removed premature local state update of TopologyId in master_server.go and raft_hashicorp.go.
- State is now solely updated via the Raft state machine Apply/Restore methods after consensus.

* Add explicit log for recovered TopologyId

- Added glog.V(0) info log in RaftServer.Recovery to print the recovered TopologyId on startup.

* Add Raft barrier to prevent TopologyId race condition

- Implement ensureTopologyId helper method
- Send no-op MaxVolumeIdCommand to sync Raft log before checking TopologyId
- Ensures persisted TopologyId is recovered before generating new one
- Prevents race where generation happens during log replay

* Serialize TopologyId generation with mutex

- Add topologyIdGenLock mutex to MasterServer struct
- Wrap ensureTopologyId method with lock to prevent concurrent generation
- Fixes race where event listener and manual leadership check both generate IDs
- Second caller waits for first to complete and sees the generated ID

* Add TopologyId recovery logging to Apply method

- Change log level from V(1) to V(0) for visibility
- Log 'Recovered TopologyId' when applying from Raft log
- Ensures recovery is visible whether from snapshot or log replay
- Matches Recovery() method logging for consistency

* Fix Raft barrier timing issue

- Add 100ms delay after barrier command to ensure log application completes
- Add debug logging to track barrier execution and TopologyId state
- Return early if barrier command fails
- Prevents TopologyId generation before old logs are fully applied

* ensure leader

* address comments

* address comments

* redundant

* clean up

* double check

* refactoring

* comment
2026-01-18 14:02:34 -08:00

214 lines
5.4 KiB
Go

package weed_server
import (
"encoding/json"
"io"
"math/rand/v2"
"os"
"path"
"time"
transport "github.com/Jille/raft-grpc-transport"
"google.golang.org/grpc"
"github.com/seaweedfs/seaweedfs/weed/pb"
hashicorpRaft "github.com/hashicorp/raft"
"github.com/seaweedfs/raft"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/topology"
)
type RaftServerOption struct {
GrpcDialOption grpc.DialOption
Peers map[string]pb.ServerAddress
ServerAddr pb.ServerAddress
DataDir string
Topo *topology.Topology
RaftResumeState bool
HeartbeatInterval time.Duration
ElectionTimeout time.Duration
RaftBootstrap bool
}
type RaftServer struct {
peers map[string]pb.ServerAddress // initial peers to join with
raftServer raft.Server
RaftHashicorp *hashicorpRaft.Raft
TransportManager *transport.Manager
dataDir string
serverAddr pb.ServerAddress
topo *topology.Topology
*raft.GrpcServer
}
type StateMachine struct {
raft.StateMachine
topo *topology.Topology
}
var _ hashicorpRaft.FSM = &StateMachine{}
func (s StateMachine) Save() ([]byte, error) {
state := topology.MaxVolumeIdCommand{
MaxVolumeId: s.topo.GetMaxVolumeId(),
TopologyId: s.topo.GetTopologyId(),
}
glog.V(1).Infof("Save raft state %+v", state)
return json.Marshal(state)
}
func (s StateMachine) Recovery(data []byte) error {
state := topology.MaxVolumeIdCommand{}
err := json.Unmarshal(data, &state)
if err != nil {
return err
}
glog.V(1).Infof("Recovery raft state %+v", state)
s.topo.UpAdjustMaxVolumeId(state.MaxVolumeId)
if state.TopologyId != "" {
s.topo.SetTopologyId(state.TopologyId)
glog.V(0).Infof("Recovered TopologyId: %s", state.TopologyId)
}
return nil
}
func (s *StateMachine) Apply(l *hashicorpRaft.Log) interface{} {
before := s.topo.GetMaxVolumeId()
state := topology.MaxVolumeIdCommand{}
err := json.Unmarshal(l.Data, &state)
if err != nil {
return err
}
s.topo.UpAdjustMaxVolumeId(state.MaxVolumeId)
if state.TopologyId != "" {
prevTopologyId := s.topo.GetTopologyId()
s.topo.SetTopologyId(state.TopologyId)
// Log when recovering TopologyId from Raft log replay, or setting it for the first time.
if prevTopologyId == "" {
glog.V(0).Infof("Set TopologyId from raft log: %s", state.TopologyId)
}
}
glog.V(1).Infoln("max volume id", before, "==>", s.topo.GetMaxVolumeId())
return nil
}
func (s *StateMachine) Snapshot() (hashicorpRaft.FSMSnapshot, error) {
return &topology.MaxVolumeIdCommand{
MaxVolumeId: s.topo.GetMaxVolumeId(),
TopologyId: s.topo.GetTopologyId(),
}, nil
}
func (s *StateMachine) Restore(r io.ReadCloser) error {
b, err := io.ReadAll(r)
if err != nil {
return err
}
if err := s.Recovery(b); err != nil {
return err
}
return nil
}
func NewRaftServer(option *RaftServerOption) (*RaftServer, error) {
s := &RaftServer{
peers: option.Peers,
serverAddr: option.ServerAddr,
dataDir: option.DataDir,
topo: option.Topo,
}
if glog.V(4) {
raft.SetLogLevel(2)
}
raft.RegisterCommand(&topology.MaxVolumeIdCommand{})
var err error
transporter := raft.NewGrpcTransporter(option.GrpcDialOption)
glog.V(0).Infof("Starting RaftServer with %v", option.ServerAddr)
if !option.RaftResumeState {
// clear previous log to ensure fresh start
os.RemoveAll(path.Join(s.dataDir, "log"))
// always clear previous metadata
os.RemoveAll(path.Join(s.dataDir, "conf"))
os.RemoveAll(path.Join(s.dataDir, "snapshot"))
}
if err := os.MkdirAll(path.Join(s.dataDir, "snapshot"), os.ModePerm); err != nil {
return nil, err
}
stateMachine := StateMachine{topo: option.Topo}
s.raftServer, err = raft.NewServer(string(s.serverAddr), s.dataDir, transporter, stateMachine, option.Topo, s.serverAddr.ToGrpcAddress())
if err != nil {
glog.V(0).Infoln(err)
return nil, err
}
heartbeatInterval := time.Duration(float64(option.HeartbeatInterval) * (rand.Float64()*0.25 + 1))
s.raftServer.SetHeartbeatInterval(heartbeatInterval)
s.raftServer.SetElectionTimeout(option.ElectionTimeout)
if err := s.raftServer.LoadSnapshot(); err != nil {
return nil, err
}
if err := s.raftServer.Start(); err != nil {
return nil, err
}
for name, peer := range s.peers {
if err := s.raftServer.AddPeer(name, peer.ToGrpcAddress()); err != nil {
return nil, err
}
}
// Remove deleted peers
for existsPeerName := range s.raftServer.Peers() {
if existingPeer, found := s.peers[existsPeerName]; !found {
if err := s.raftServer.RemovePeer(existsPeerName); err != nil {
glog.V(0).Infoln(err)
return nil, err
} else {
glog.V(0).Infof("removing old peer: %s", existingPeer)
}
}
}
s.GrpcServer = raft.NewGrpcServer(s.raftServer)
glog.V(0).Infof("current cluster leader: %v", s.raftServer.Leader())
return s, nil
}
func (s *RaftServer) Peers() (members []string) {
if s.raftServer != nil {
peers := s.raftServer.Peers()
for _, p := range peers {
members = append(members, p.Name)
}
} else if s.RaftHashicorp != nil {
cfg := s.RaftHashicorp.GetConfiguration()
for _, p := range cfg.Configuration().Servers {
members = append(members, string(p.ID))
}
}
return
}
func (s *RaftServer) DoJoinCommand() {
glog.V(0).Infoln("Initializing new cluster")
if _, err := s.raftServer.Do(&raft.DefaultJoinCommand{
Name: s.raftServer.Name(),
ConnectionString: s.serverAddr.ToGrpcAddress(),
}); err != nil {
glog.Errorf("fail to send join command: %v", err)
}
}