Files
seaweedFS/weed/server/raft_common.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

47 lines
1.4 KiB
Go

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