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:
Chris Lu
2026-01-18 14:02:34 -08:00
committed by GitHub
parent ce23c4fca7
commit 753e1db096
12 changed files with 267 additions and 159 deletions

View File

@@ -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() {