@@ -500,6 +500,28 @@ func (n *node) applyConfChange(e raftpb.Entry) {
500500 n .DeletePeer (cc .NodeID )
501501 n .server .removeZero (cc .NodeID )
502502
503+ } else if cc .Type == raftpb .ConfChangeUpdateNode && len (cc .Context ) > 0 {
504+ // ConfChangeUpdateNode is a Raft-provided mechanism for updating node
505+ // metadata (e.g. addresses) without changing cluster membership. It is
506+ // a no-op inside the Raft library; the application is responsible for
507+ // interpreting the Context bytes. We use it to fix stale Zero addresses
508+ // that were baked into the WAL at initial bootstrap.
509+ var rc pb.RaftContext
510+ x .Check (proto .Unmarshal (cc .Context , & rc ))
511+
512+ // Drop the stale pool before connecting to the new address so that
513+ // repeated dial errors to the old address are eliminated immediately.
514+ if oldMember , ok := n .server .membershipState ().GetZeros ()[rc .Id ]; ok {
515+ if oldMember .GetAddr () != rc .Addr {
516+ conn .GetPools ().Remove (oldMember .GetAddr ())
517+ }
518+ }
519+ go n .Connect (rc .Id , rc .Addr )
520+
521+ m := & pb.Member {Id : rc .Id , Addr : rc .Addr , GroupId : 0 , Learner : rc .IsLearner }
522+ n .server .storeZero (m )
523+ glog .Infof ("Applied ConfChangeUpdateNode for Zero %#x: addr=%q" , rc .Id , rc .Addr )
524+
503525 } else if len (cc .Context ) > 0 {
504526 var rc pb.RaftContext
505527 x .Check (proto .Unmarshal (cc .Context , & rc ))
@@ -696,12 +718,89 @@ func (n *node) updateZeroMembershipPeriodically(closer *z.Closer) {
696718 select {
697719 case <- ticker :
698720 n .server .updateZeroLeader ()
721+ n .reconcileZeroAddresses ()
699722 case <- closer .HasBeenClosed ():
700723 return
701724 }
702725 }
703726}
704727
728+ // reconcileZeroAddresses detects mismatches between the current --my address
729+ // (or transport-layer peer addresses) and what is stored in MembershipState,
730+ // then proposes a ConfChangeUpdateNode through Raft to correct them. This
731+ // ensures that stale addresses baked into the WAL at initial bootstrap are
732+ // replaced with the current addresses after a restart. Only the leader can
733+ // propose, so this function is a no-op on followers.
734+ func (n * node ) reconcileZeroAddresses () {
735+ if ! n .AmLeader () {
736+ return
737+ }
738+
739+ state := n .server .membershipState ()
740+ if state == nil {
741+ return
742+ }
743+
744+ for id , zero := range state .GetZeros () {
745+ var correctAddr string
746+
747+ if id == n .Id {
748+ correctAddr = n .RaftContext .Addr
749+ } else if peerAddr , ok := n .Peer (id ); ok {
750+ correctAddr = peerAddr
751+ } else {
752+ continue
753+ }
754+
755+ if correctAddr == zero .GetAddr () {
756+ continue
757+ }
758+
759+ // Validate: ensure no other Zero already claims this address.
760+ duplicate := false
761+ for otherId , otherZero := range state .GetZeros () {
762+ if otherId != id && otherZero .GetAddr () == correctAddr {
763+ glog .Warningf ("Skipping address reconciliation for Zero %#x: " +
764+ "address %q already used by Zero %#x" , id , correctAddr , otherId )
765+ duplicate = true
766+ break
767+ }
768+ }
769+ if duplicate {
770+ continue
771+ }
772+
773+ glog .Infof ("Zero %#x address mismatch: MembershipState has %q, expected %q. " +
774+ "Proposing ConfChangeUpdateNode." , id , zero .GetAddr (), correctAddr )
775+
776+ rc := & pb.RaftContext {
777+ Id : id ,
778+ Addr : correctAddr ,
779+ Group : 0 ,
780+ }
781+ data , err := proto .Marshal (rc )
782+ if err != nil {
783+ glog .Errorf ("Error marshalling RaftContext for address update: %v" , err )
784+ continue
785+ }
786+
787+ cc := raftpb.ConfChange {
788+ Type : raftpb .ConfChangeUpdateNode ,
789+ NodeID : id ,
790+ Context : data ,
791+ }
792+ if err := n .ProposeConfChange (n .ctx , cc ); err != nil {
793+ glog .Errorf ("Failed to propose ConfChangeUpdateNode for Zero %#x: %v" , id , err )
794+ }
795+ // Propose one update at a time to respect Raft's single pending
796+ // ConfChange constraint. The next mismatch will be handled in the
797+ // following periodic cycle.
798+ return
799+ }
800+ // V(2): emitted on every 10s tick, too noisy for production at Info level.
801+ glog .V (2 ).Infof ("Zero address reconciliation complete: all addresses up to date" )
802+ }
803+
705804func (n * node ) checkQuorum (closer * z.Closer ) {
706805 defer closer .Done ()
707806 ticker := time .Tick (time .Second )
@@ -923,6 +1022,10 @@ func (n *node) Run() {
9231022 if rd .RaftState == raft .StateLeader && ! leader {
9241023 glog .Infoln ("I've become the leader, updating leases." )
9251024 n .server .updateLeases ()
1025+ // Eagerly reconcile on leader election so stale WAL
1026+ // addresses are corrected without waiting for the
1027+ // periodic 10s tick.
1028+ go n .reconcileZeroAddresses ()
9261029 }
9271030 leader = rd .RaftState == raft .StateLeader
9281031 // group id hardcoded as 0
0 commit comments