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

242 lines
7.1 KiB
Go

package weed_server
// https://yusufs.medium.com/creating-distributed-kv-database-by-implementing-raft-consensus-using-golang-d0884eef2e28
// https://github.com/Jille/raft-grpc-example/blob/cd5bcab0218f008e044fbeee4facdd01b06018ad/application.go#L18
import (
"encoding/json"
"fmt"
"math/rand/v2"
"os"
"path"
"path/filepath"
"sort"
"strings"
"time"
transport "github.com/Jille/raft-grpc-transport"
"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"
)
const (
ldbFile = "logs.dat"
sdbFile = "stable.dat"
updatePeersTimeout = 15 * time.Minute
)
func getPeerIdx(self pb.ServerAddress, mapPeers map[string]pb.ServerAddress) int {
peers := make([]pb.ServerAddress, 0, len(mapPeers))
for _, peer := range mapPeers {
peers = append(peers, peer)
}
sort.Slice(peers, func(i, j int) bool {
return strings.Compare(string(peers[i]), string(peers[j])) < 0
})
for i, peer := range peers {
if string(peer) == string(self) {
return i
}
}
return -1
}
func (s *RaftServer) AddPeersConfiguration() (cfg raft.Configuration) {
for _, peer := range s.peers {
cfg.Servers = append(cfg.Servers, raft.Server{
Suffrage: raft.Voter,
ID: raft.ServerID(peer),
Address: raft.ServerAddress(peer.ToGrpcAddress()),
})
}
return cfg
}
func (s *RaftServer) monitorLeaderLoop(updatePeers bool) {
for {
prevLeader, _ := s.RaftHashicorp.LeaderWithID()
select {
case isLeader := <-s.RaftHashicorp.LeaderCh():
leader, _ := s.RaftHashicorp.LeaderWithID()
if isLeader {
if updatePeers {
s.updatePeers()
updatePeers = false
}
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()
}
glog.V(0).Infof("is leader %+v change event: %+v => %+v", isLeader, prevLeader, leader)
prevLeader = leader
s.topo.LastLeaderChangeTime = time.Now()
}
}
}
func (s *RaftServer) updatePeers() {
peerLeader := string(s.serverAddr)
existsPeerName := make(map[string]bool)
for _, server := range s.RaftHashicorp.GetConfiguration().Configuration().Servers {
if string(server.ID) == peerLeader {
continue
}
existsPeerName[string(server.ID)] = true
}
for _, peer := range s.peers {
peerName := string(peer)
if peerName == peerLeader || existsPeerName[peerName] {
continue
}
glog.V(0).Infof("adding new peer: %s", peerName)
s.RaftHashicorp.AddVoter(
raft.ServerID(peerName), raft.ServerAddress(peer.ToGrpcAddress()), 0, 0)
}
for peer := range existsPeerName {
if _, found := s.peers[peer]; !found {
glog.V(0).Infof("removing old peer: %s", peer)
s.RaftHashicorp.RemoveServer(raft.ServerID(peer), 0, 0)
}
}
if _, found := s.peers[peerLeader]; !found {
glog.V(0).Infof("removing old leader peer: %s", peerLeader)
s.RaftHashicorp.RemoveServer(raft.ServerID(peerLeader), 0, 0)
}
}
func NewHashicorpRaftServer(option *RaftServerOption) (*RaftServer, error) {
s := &RaftServer{
peers: option.Peers,
serverAddr: option.ServerAddr,
dataDir: option.DataDir,
topo: option.Topo,
}
c := raft.DefaultConfig()
c.LocalID = raft.ServerID(s.serverAddr) // TODO maybee the IP:port address will change
c.HeartbeatTimeout = time.Duration(float64(option.HeartbeatInterval) * (rand.Float64()*0.25 + 1))
c.ElectionTimeout = option.ElectionTimeout
if c.LeaderLeaseTimeout > c.HeartbeatTimeout {
c.LeaderLeaseTimeout = c.HeartbeatTimeout
}
if glog.V(4) {
c.LogLevel = "Debug"
} else if glog.V(2) {
c.LogLevel = "Info"
} else if glog.V(1) {
c.LogLevel = "Warn"
} else if glog.V(0) {
c.LogLevel = "Error"
}
if err := raft.ValidateConfig(c); err != nil {
return nil, fmt.Errorf("raft.ValidateConfig: %w", err)
}
if option.RaftBootstrap {
os.RemoveAll(path.Join(s.dataDir, ldbFile))
os.RemoveAll(path.Join(s.dataDir, sdbFile))
os.RemoveAll(path.Join(s.dataDir, "snapshots"))
}
if err := os.MkdirAll(path.Join(s.dataDir, "snapshots"), os.ModePerm); err != nil {
return nil, err
}
baseDir := s.dataDir
ldb, err := boltdb.NewBoltStore(filepath.Join(baseDir, ldbFile))
if err != nil {
return nil, fmt.Errorf("boltdb.NewBoltStore(%q): %v", filepath.Join(baseDir, "logs.dat"), err)
}
sdb, err := boltdb.NewBoltStore(filepath.Join(baseDir, sdbFile))
if err != nil {
return nil, fmt.Errorf("boltdb.NewBoltStore(%q): %v", filepath.Join(baseDir, "stable.dat"), err)
}
fss, err := raft.NewFileSnapshotStore(baseDir, 3, os.Stderr)
if err != nil {
return nil, fmt.Errorf("raft.NewFileSnapshotStore(%q, ...): %v", baseDir, err)
}
s.TransportManager = transport.New(raft.ServerAddress(s.serverAddr), []grpc.DialOption{option.GrpcDialOption})
stateMachine := StateMachine{topo: option.Topo}
s.RaftHashicorp, err = raft.NewRaft(c, &stateMachine, ldb, sdb, fss, s.TransportManager.Transport())
if err != nil {
return nil, fmt.Errorf("raft.NewRaft: %w", err)
}
updatePeers := false
if option.RaftBootstrap || len(s.RaftHashicorp.GetConfiguration().Configuration().Servers) == 0 {
cfg := s.AddPeersConfiguration()
// Need to get lock, in case all servers do this at the same time.
peerIdx := getPeerIdx(s.serverAddr, s.peers)
timeSleep := time.Duration(float64(c.LeaderLeaseTimeout) * (rand.Float64()*0.25 + 1) * float64(peerIdx))
glog.V(0).Infof("Bootstrapping idx: %d sleep: %v new cluster: %+v", peerIdx, timeSleep, cfg)
time.Sleep(timeSleep)
f := s.RaftHashicorp.BootstrapCluster(cfg)
if err := f.Error(); err != nil {
return nil, fmt.Errorf("raft.Raft.BootstrapCluster: %w", err)
}
} else {
updatePeers = true
}
go s.monitorLeaderLoop(updatePeers)
ticker := time.NewTicker(c.HeartbeatTimeout * 10)
if glog.V(4) {
go func() {
for {
select {
case <-ticker.C:
cfuture := s.RaftHashicorp.GetConfiguration()
if err = cfuture.Error(); err != nil {
glog.Fatalf("error getting config: %s", err)
}
configuration := cfuture.Configuration()
glog.V(4).Infof("Showing peers known by %s:\n%+v", s.RaftHashicorp.String(), configuration.Servers)
}
}
}()
}
// Configure a prometheus sink as the raft metrics sink
if sink, err := prometheus.NewPrometheusSinkFrom(prometheus.PrometheusOpts{
Registerer: stats.Gather,
}); err != nil {
return nil, fmt.Errorf("NewPrometheusSink: %w", err)
} else {
metricsConf := metrics.DefaultConfig(stats.Namespace)
metricsConf.EnableRuntimeMetrics = false
if _, err = metrics.NewGlobal(metricsConf, sink); err != nil {
return nil, fmt.Errorf("metrics.NewGlobal: %w", err)
}
}
return s, nil
}