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
This commit is contained in:
@@ -77,6 +77,8 @@ type MasterServer struct {
|
||||
|
||||
grpcDialOption grpc.DialOption
|
||||
|
||||
topologyIdGenLock sync.Mutex
|
||||
|
||||
MasterClient *wdclient.MasterClient
|
||||
|
||||
adminLocks *AdminLocks
|
||||
@@ -209,6 +211,9 @@ func (ms *MasterServer) SetRaftServer(raftServer *RaftServer) {
|
||||
if ms.Topo.RaftServer.Leader() != "" {
|
||||
glog.V(0).Infof("[%s] %s becomes leader.", ms.Topo.RaftServer.Name(), ms.Topo.RaftServer.Leader())
|
||||
ms.Topo.LastLeaderChangeTime = time.Now()
|
||||
if ms.Topo.RaftServer.Leader() == ms.Topo.RaftServer.Name() {
|
||||
go ms.ensureTopologyId()
|
||||
}
|
||||
}
|
||||
})
|
||||
raftServerName = fmt.Sprintf("[%s]", ms.Topo.RaftServer.Name())
|
||||
@@ -236,6 +241,42 @@ func (ms *MasterServer) SetRaftServer(raftServer *RaftServer) {
|
||||
}
|
||||
}
|
||||
|
||||
func (ms *MasterServer) ensureTopologyId() {
|
||||
ms.topologyIdGenLock.Lock()
|
||||
defer ms.topologyIdGenLock.Unlock()
|
||||
|
||||
// Send a no-op command to ensure all previous logs are applied (barrier)
|
||||
// This handles the case where log replay is still in progress
|
||||
glog.V(1).Infof("ensureTopologyId: sending barrier command")
|
||||
for {
|
||||
if !ms.Topo.IsLeader() {
|
||||
glog.V(1).Infof("lost leadership while sending barrier command for topologyId")
|
||||
return
|
||||
}
|
||||
if _, err := ms.Topo.RaftServer.Do(topology.NewMaxVolumeIdCommand(ms.Topo.GetMaxVolumeId(), ms.Topo.GetTopologyId())); err != nil {
|
||||
glog.Errorf("failed to sync raft for topologyId: %v, retrying in 1s", err)
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
break
|
||||
}
|
||||
glog.V(1).Infof("ensureTopologyId: barrier command completed")
|
||||
|
||||
if !ms.Topo.IsLeader() {
|
||||
return
|
||||
}
|
||||
|
||||
currentId := ms.Topo.GetTopologyId()
|
||||
glog.V(1).Infof("ensureTopologyId: current TopologyId after barrier: %s", currentId)
|
||||
|
||||
EnsureTopologyId(ms.Topo, func() bool {
|
||||
return ms.Topo.IsLeader()
|
||||
}, func(topologyId string) error {
|
||||
_, err := ms.Topo.RaftServer.Do(topology.NewMaxVolumeIdCommand(ms.Topo.GetMaxVolumeId(), topologyId))
|
||||
return err
|
||||
})
|
||||
}
|
||||
|
||||
func (ms *MasterServer) proxyToLeader(f http.HandlerFunc) http.HandlerFunc {
|
||||
return func(w http.ResponseWriter, r *http.Request) {
|
||||
if ms.Topo.IsLeader() {
|
||||
|
||||
46
weed/server/raft_common.go
Normal file
46
weed/server/raft_common.go
Normal file
@@ -0,0 +1,46 @@
|
||||
package weed_server
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/topology"
|
||||
)
|
||||
|
||||
// EnsureTopologyId ensures that a TopologyId is generated and persisted if it's currently missing.
|
||||
// It uses the provided checkLeaderFn to verify leadership and persistFn to save the new ID.
|
||||
func EnsureTopologyId(topo *topology.Topology, checkLeaderFn func() bool, persistFn func(string) error) {
|
||||
if topo.GetTopologyId() != "" {
|
||||
return
|
||||
}
|
||||
|
||||
topologyId := uuid.New().String()
|
||||
for {
|
||||
if !checkLeaderFn() {
|
||||
glog.V(0).Infof("lost leadership while saving topologyId")
|
||||
return
|
||||
}
|
||||
|
||||
// Another concurrent operation may have set the ID between generation and now.
|
||||
if latestId := topo.GetTopologyId(); latestId != "" {
|
||||
glog.V(1).Infof("topologyId was set concurrently to %s, aborting generation", latestId)
|
||||
return
|
||||
}
|
||||
|
||||
if err := persistFn(topologyId); err != nil {
|
||||
glog.Errorf("failed to save topologyId, will retry: %v", err)
|
||||
time.Sleep(time.Second)
|
||||
continue
|
||||
}
|
||||
|
||||
// Verify that the topology ID was actually applied as expected.
|
||||
appliedId := topo.GetTopologyId()
|
||||
if appliedId != "" && appliedId != topologyId {
|
||||
glog.V(0).Infof("TopologyId generation race: expected %s, but current TopologyId is %s", topologyId, appliedId)
|
||||
} else {
|
||||
glog.V(0).Infof("TopologyId generated: %s", topologyId)
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
@@ -4,6 +4,7 @@ package weed_server
|
||||
// https://github.com/Jille/raft-grpc-example/blob/cd5bcab0218f008e044fbeee4facdd01b06018ad/application.go#L18
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"math/rand/v2"
|
||||
"os"
|
||||
@@ -17,10 +18,12 @@ import (
|
||||
"github.com/armon/go-metrics"
|
||||
"github.com/armon/go-metrics/prometheus"
|
||||
"github.com/hashicorp/raft"
|
||||
hashicorpRaft "github.com/hashicorp/raft"
|
||||
boltdb "github.com/hashicorp/raft-boltdb/v2"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/stats"
|
||||
"github.com/seaweedfs/seaweedfs/weed/topology"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
@@ -72,6 +75,17 @@ func (s *RaftServer) monitorLeaderLoop(updatePeers bool) {
|
||||
|
||||
s.topo.DoBarrier()
|
||||
|
||||
EnsureTopologyId(s.topo, func() bool {
|
||||
return s.RaftHashicorp.State() == hashicorpRaft.Leader
|
||||
}, func(topologyId string) error {
|
||||
command := topology.NewMaxVolumeIdCommand(s.topo.GetMaxVolumeId(), topologyId)
|
||||
b, err := json.Marshal(command)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return s.RaftHashicorp.Apply(b, 5*time.Second).Error()
|
||||
})
|
||||
|
||||
stats.MasterLeaderChangeCounter.WithLabelValues(fmt.Sprintf("%+v", leader)).Inc()
|
||||
} else {
|
||||
s.topo.BarrierReset()
|
||||
|
||||
@@ -54,6 +54,7 @@ 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)
|
||||
@@ -67,6 +68,10 @@ func (s StateMachine) Recovery(data []byte) error {
|
||||
}
|
||||
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
|
||||
}
|
||||
|
||||
@@ -78,6 +83,14 @@ func (s *StateMachine) Apply(l *hashicorpRaft.Log) interface{} {
|
||||
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
|
||||
@@ -86,6 +99,7 @@ func (s *StateMachine) Apply(l *hashicorpRaft.Log) interface{} {
|
||||
func (s *StateMachine) Snapshot() (hashicorpRaft.FSMSnapshot, error) {
|
||||
return &topology.MaxVolumeIdCommand{
|
||||
MaxVolumeId: s.topo.GetMaxVolumeId(),
|
||||
TopologyId: s.topo.GetTopologyId(),
|
||||
}, nil
|
||||
}
|
||||
|
||||
@@ -118,9 +132,9 @@ func NewRaftServer(option *RaftServerOption) (*RaftServer, error) {
|
||||
transporter := raft.NewGrpcTransporter(option.GrpcDialOption)
|
||||
glog.V(0).Infof("Starting RaftServer with %v", option.ServerAddr)
|
||||
|
||||
// always clear previous log to avoid server is promotable
|
||||
os.RemoveAll(path.Join(s.dataDir, "log"))
|
||||
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"))
|
||||
|
||||
Reference in New Issue
Block a user