// ChangeGroupMembership submits a proposed membership change to the cluster. // Payload is an opaque blob that will be returned in EventMembershipChangeCommitted. func (m *MultiRaft) ChangeGroupMembership(groupID roachpb.RangeID, commandID string, changeType raftpb.ConfChangeType, replica roachpb.ReplicaDescriptor, payload []byte) <-chan error { if log.V(6) { log.Infof("node %v proposing membership change to group %v", m.nodeID, groupID) } ch := make(chan error, 1) if err := replica.Validate(); err != nil { ch <- err return ch } m.proposalChan <- &proposal{ groupID: groupID, commandID: commandID, fn: func() { ctx := ConfChangeContext{ CommandID: commandID, Payload: payload, Replica: replica, } encodedCtx, err := ctx.Marshal() if err != nil { log.Errorf("node %v: error encoding context protobuf", m.nodeID) return } if err := m.multiNode.ProposeConfChange(context.Background(), uint64(groupID), raftpb.ConfChange{ Type: changeType, NodeID: uint64(replica.ReplicaID), Context: encodedCtx, }, ); err != nil { log.Errorf("node %v: error proposing membership change to node %v: %s", m.nodeID, groupID, err) return } }, ch: ch, } return ch }
// sendMessage sends a raft message on the given group. Coalesced heartbeats // address nodes, not groups; they will use the noGroup constant as groupID. func (s *state) sendMessage(g *group, msg raftpb.Message) { if log.V(6) { log.Infof("node %v sending message %.200s to %v", s.nodeID, raft.DescribeMessage(msg, s.EntryFormatter), msg.To) } groupID := noGroup var toReplica roachpb.ReplicaDescriptor var fromReplica roachpb.ReplicaDescriptor if g == nil { // No group (a coalesced heartbeat): To/From fields are NodeIDs. // TODO(bdarnell): test transports route by store ID, not node ID. // In tests they're always the same, so we can hack it here but // it would be better to fix the transports. // I think we need to fix this before we can support a range // with two replicas on different stores of the same node. toReplica.NodeID = roachpb.NodeID(msg.To) toReplica.StoreID = roachpb.StoreID(msg.To) fromReplica.NodeID = roachpb.NodeID(msg.From) fromReplica.StoreID = roachpb.StoreID(msg.From) } else { // Regular message: To/From fields are replica IDs. groupID = g.id var err error toReplica, err = s.ReplicaDescriptor(groupID, roachpb.ReplicaID(msg.To)) if err != nil { log.Warningf("failed to lookup recipient replica %d in group %d: %s", msg.To, groupID, err) return } fromReplica, err = s.ReplicaDescriptor(groupID, roachpb.ReplicaID(msg.From)) if err != nil { log.Warningf("failed to lookup sender replica %d in group %d: %s", msg.From, groupID, err) return } } if _, ok := s.nodes[toReplica.NodeID]; !ok { if log.V(4) { log.Infof("node %v: connecting to new node %v", s.nodeID, toReplica.NodeID) } if err := s.addNode(toReplica.NodeID, g); err != nil { log.Errorf("node %v: error adding group %v to node %v: %v", s.nodeID, groupID, toReplica.NodeID, err) } } err := s.Transport.Send(&RaftMessageRequest{ GroupID: groupID, ToReplica: toReplica, FromReplica: fromReplica, Message: msg, }) snapStatus := raft.SnapshotFinish if err != nil { log.Warningf("node %v failed to send message to %v: %s", s.nodeID, toReplica.NodeID, err) if groupID != noGroup { s.multiNode.ReportUnreachable(msg.To, uint64(groupID)) } snapStatus = raft.SnapshotFailure } if msg.Type == raftpb.MsgSnap { // TODO(bdarnell): add an ack for snapshots and don't report status until // ack, error, or timeout. if groupID != noGroup { s.multiNode.ReportSnapshot(msg.To, uint64(groupID), snapStatus) } } }