diff --git a/kv/dist_sender.go b/kv/dist_sender.go index a1a84cf3ee73..d6bee30d2ae3 100644 --- a/kv/dist_sender.go +++ b/kv/dist_sender.go @@ -316,11 +316,11 @@ func (ds *DistSender) sendRPC(trace *tracer.Trace, rangeID proto.RangeID, replic // Build a slice of replica addresses (if gossiped). var addrs []net.Addr - replicaMap := map[string]*proto.Replica{} + replicaMap := map[string]*proto.ReplicaDescriptor{} for i := range replicas { addr := replicas[i].NodeDesc.Address addrs = append(addrs, addr) - replicaMap[addr.String()] = &replicas[i].Replica + replicaMap[addr.String()] = &replicas[i].ReplicaDescriptor } if len(addrs) == 0 { return nil, noNodeAddrsAvailError{} @@ -627,7 +627,7 @@ func (ds *DistSender) sendChunk(ctx context.Context, ba proto.BatchRequest) (*pr evictDesc() } } else { - newLeader = &proto.Replica{} + newLeader = &proto.ReplicaDescriptor{} } ds.updateLeaderCache(proto.RangeID(desc.RangeID), *newLeader) if log.V(1) { @@ -752,7 +752,7 @@ func (ds *DistSender) sendChunk(ctx context.Context, ba proto.BatchRequest) (*pr // updateLeaderCache updates the cached leader for the given range, // evicting any previous value in the process. -func (ds *DistSender) updateLeaderCache(rid proto.RangeID, leader proto.Replica) { +func (ds *DistSender) updateLeaderCache(rid proto.RangeID, leader proto.ReplicaDescriptor) { oldLeader := ds.leaderCache.Lookup(rid) if leader.StoreID != oldLeader.StoreID { if log.V(1) { diff --git a/kv/dist_sender_test.go b/kv/dist_sender_test.go index 7ce7be4af6c9..5ff003b41494 100644 --- a/kv/dist_sender_test.go +++ b/kv/dist_sender_test.go @@ -43,7 +43,7 @@ var testRangeDescriptor = proto.RangeDescriptor{ RangeID: 1, StartKey: proto.Key("a"), EndKey: proto.Key("z"), - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { NodeID: 1, StoreID: 1, @@ -86,16 +86,16 @@ func TestMoveLocalReplicaToFront(t *testing.T) { // No attribute prefix slice: replicaSlice{ replicaInfo{ - Replica: proto.Replica{NodeID: 2, StoreID: 2}, - NodeDesc: &proto.NodeDescriptor{NodeID: 2}, + ReplicaDescriptor: proto.ReplicaDescriptor{NodeID: 2, StoreID: 2}, + NodeDesc: &proto.NodeDescriptor{NodeID: 2}, }, replicaInfo{ - Replica: proto.Replica{NodeID: 3, StoreID: 3}, - NodeDesc: &proto.NodeDescriptor{NodeID: 3}, + ReplicaDescriptor: proto.ReplicaDescriptor{NodeID: 3, StoreID: 3}, + NodeDesc: &proto.NodeDescriptor{NodeID: 3}, }, replicaInfo{ - Replica: proto.Replica{NodeID: 1, StoreID: 1}, - NodeDesc: &proto.NodeDescriptor{NodeID: 1}, + ReplicaDescriptor: proto.ReplicaDescriptor{NodeID: 1, StoreID: 1}, + NodeDesc: &proto.NodeDescriptor{NodeID: 1}, }, }, localNodeDesc: proto.NodeDescriptor{NodeID: 1}, @@ -104,16 +104,16 @@ func TestMoveLocalReplicaToFront(t *testing.T) { // Sort replicas by attribute slice: replicaSlice{ replicaInfo{ - Replica: proto.Replica{NodeID: 2, StoreID: 2}, - NodeDesc: &proto.NodeDescriptor{NodeID: 2, Attrs: proto.Attributes{Attrs: []string{"ad"}}}, + ReplicaDescriptor: proto.ReplicaDescriptor{NodeID: 2, StoreID: 2}, + NodeDesc: &proto.NodeDescriptor{NodeID: 2, Attrs: proto.Attributes{Attrs: []string{"ad"}}}, }, replicaInfo{ - Replica: proto.Replica{NodeID: 3, StoreID: 3}, - NodeDesc: &proto.NodeDescriptor{NodeID: 3, Attrs: proto.Attributes{Attrs: []string{"ab", "c"}}}, + ReplicaDescriptor: proto.ReplicaDescriptor{NodeID: 3, StoreID: 3}, + NodeDesc: &proto.NodeDescriptor{NodeID: 3, Attrs: proto.Attributes{Attrs: []string{"ab", "c"}}}, }, replicaInfo{ - Replica: proto.Replica{NodeID: 1, StoreID: 1}, - NodeDesc: &proto.NodeDescriptor{NodeID: 1, Attrs: proto.Attributes{Attrs: []string{"ab"}}}, + ReplicaDescriptor: proto.ReplicaDescriptor{NodeID: 1, StoreID: 1}, + NodeDesc: &proto.NodeDescriptor{NodeID: 1, Attrs: proto.Attributes{Attrs: []string{"ab"}}}, }, }, localNodeDesc: proto.NodeDescriptor{NodeID: 1, Attrs: proto.Attributes{Attrs: []string{"ab"}}}, @@ -299,7 +299,7 @@ func TestSendRPCOrder(t *testing.T) { if err := g.AddInfoProto(gossip.MakeNodeIDKey(proto.NodeID(i)), nd, time.Hour); err != nil { t.Fatal(err) } - descriptor.Replicas = append(descriptor.Replicas, proto.Replica{ + descriptor.Replicas = append(descriptor.Replicas, proto.ReplicaDescriptor{ NodeID: proto.NodeID(i), StoreID: proto.StoreID(i), }) @@ -318,7 +318,7 @@ func TestSendRPCOrder(t *testing.T) { } } - ds.leaderCache.Update(proto.RangeID(rangeID), proto.Replica{}) + ds.leaderCache.Update(proto.RangeID(rangeID), proto.ReplicaDescriptor{}) if tc.leader > 0 { ds.leaderCache.Update(proto.RangeID(rangeID), descriptor.Replicas[tc.leader-1]) } @@ -363,7 +363,7 @@ func TestRetryOnNotLeaderError(t *testing.T) { defer leaktest.AfterTest(t) g, s := makeTestGossip(t) defer s() - leader := proto.Replica{ + leader := proto.ReplicaDescriptor{ NodeID: 99, StoreID: 999, } @@ -373,7 +373,7 @@ func TestRetryOnNotLeaderError(t *testing.T) { if first { reply := getReply() reply.(proto.Response).Header().SetGoError( - &proto.NotLeaderError{Leader: &leader, Replica: &proto.Replica{}}) + &proto.NotLeaderError{Leader: &leader, Replica: &proto.ReplicaDescriptor{}}) first = false return []gogoproto.Message{reply}, nil } @@ -459,7 +459,7 @@ func TestEvictCacheOnError(t *testing.T) { for i, tc := range testCases { g, s := makeTestGossip(t) defer s() - leader := proto.Replica{ + leader := proto.ReplicaDescriptor{ NodeID: 99, StoreID: 999, } @@ -498,7 +498,7 @@ func TestEvictCacheOnError(t *testing.T) { if _, err := batchutil.SendWrapped(ds, put); err != nil && !testutils.IsError(err, "boom") { t.Errorf("put encountered unexpected error: %s", err) } - if cur := ds.leaderCache.Lookup(1); reflect.DeepEqual(cur, &proto.Replica{}) && !tc.shouldClearLeader { + if cur := ds.leaderCache.Lookup(1); reflect.DeepEqual(cur, &proto.ReplicaDescriptor{}) && !tc.shouldClearLeader { t.Errorf("%d: leader cache eviction: shouldClearLeader=%t, but value is %v", i, tc.shouldClearLeader, cur) } _, cachedDesc := ds.rangeCache.getCachedRangeDescriptor(put.Key, false /* !inclusive */) @@ -626,7 +626,7 @@ func TestSendRPCRetry(t *testing.T) { t.Fatal(err) } - descriptor.Replicas = append(descriptor.Replicas, proto.Replica{ + descriptor.Replicas = append(descriptor.Replicas, proto.ReplicaDescriptor{ NodeID: proto.NodeID(i), StoreID: proto.StoreID(i), }) @@ -697,7 +697,7 @@ func TestMultiRangeMergeStaleDescriptor(t *testing.T) { RangeID: 1, StartKey: proto.Key("a"), EndKey: proto.Key("b"), - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { NodeID: 1, StoreID: 1, @@ -710,7 +710,7 @@ func TestMultiRangeMergeStaleDescriptor(t *testing.T) { RangeID: 1, StartKey: proto.Key("a"), EndKey: proto.KeyMax, - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { NodeID: 1, StoreID: 1, diff --git a/kv/leader_cache.go b/kv/leader_cache.go index ab53cb5974bc..ec4e76790ec3 100644 --- a/kv/leader_cache.go +++ b/kv/leader_cache.go @@ -47,20 +47,20 @@ func newLeaderCache(size int) *leaderCache { // Lookup consults the cache for the replica cached as the leader of // the given Raft consensus group. -func (lc *leaderCache) Lookup(group proto.RangeID) proto.Replica { +func (lc *leaderCache) Lookup(group proto.RangeID) proto.ReplicaDescriptor { lc.mu.Lock() defer lc.mu.Unlock() v, ok := lc.cache.Get(group) if !ok || v == nil { - return proto.Replica{} + return proto.ReplicaDescriptor{} } - return *(v.(*proto.Replica)) + return *(v.(*proto.ReplicaDescriptor)) } // Update invalidates the cached leader for the given Raft group. // If a replica is passed in, it is inserted into the cache. // A StoreID of 0 (empty replica) means evict. -func (lc *leaderCache) Update(group proto.RangeID, r proto.Replica) { +func (lc *leaderCache) Update(group proto.RangeID, r proto.ReplicaDescriptor) { lc.mu.Lock() defer lc.mu.Unlock() lc.cache.Del(group) diff --git a/kv/leader_cache_test.go b/kv/leader_cache_test.go index 2ef2bf92f4e7..b40797074d21 100644 --- a/kv/leader_cache_test.go +++ b/kv/leader_cache_test.go @@ -30,18 +30,18 @@ func TestLeaderCache(t *testing.T) { if r := lc.Lookup(12); r.StoreID != 0 { t.Fatalf("lookup of missing key returned replica: %v", r) } - replica := proto.Replica{StoreID: 1} + replica := proto.ReplicaDescriptor{StoreID: 1} lc.Update(5, replica) if r := lc.Lookup(5); r.StoreID != 1 { t.Errorf("expected %v, got %v", replica, r) } - newReplica := proto.Replica{StoreID: 7} + newReplica := proto.ReplicaDescriptor{StoreID: 7} lc.Update(5, newReplica) r := lc.Lookup(5) if r.StoreID != 7 { t.Errorf("expected %v, got %v", newReplica, r) } - lc.Update(5, proto.Replica{}) + lc.Update(5, proto.ReplicaDescriptor{}) r = lc.Lookup(5) if r.StoreID != 0 { t.Fatalf("evicted leader returned: %v", r) diff --git a/kv/local_sender.go b/kv/local_sender.go index 391733f6795f..f6441e26c427 100644 --- a/kv/local_sender.go +++ b/kv/local_sender.go @@ -118,7 +118,7 @@ func (ls *LocalSender) Send(ctx context.Context, ba proto.BatchRequest) (*proto. // If we aren't given a Replica, then a little bending over // backwards here. This case applies exclusively to unittests. if ba.RangeID == 0 || ba.Replica.StoreID == 0 { - var repl *proto.Replica + var repl *proto.ReplicaDescriptor var rangeID proto.RangeID rangeID, repl, err = ls.lookupReplica(ba.Key, ba.EndKey) if err == nil { @@ -167,7 +167,7 @@ func (ls *LocalSender) Send(ctx context.Context, ba proto.BatchRequest) (*proto. // Returns RangeID and replica on success; RangeKeyMismatch error // if not found. // This is only for testing usage; performance doesn't matter. -func (ls *LocalSender) lookupReplica(start, end proto.Key) (rangeID proto.RangeID, replica *proto.Replica, err error) { +func (ls *LocalSender) lookupReplica(start, end proto.Key) (rangeID proto.RangeID, replica *proto.ReplicaDescriptor, err error) { ls.mu.RLock() defer ls.mu.RUnlock() var rng *storage.Replica diff --git a/kv/local_sender_test.go b/kv/local_sender_test.go index 8d7c2dc70eb5..6eee4da55a52 100644 --- a/kv/local_sender_test.go +++ b/kv/local_sender_test.go @@ -105,7 +105,7 @@ func TestLocalSenderGetStore(t *testing.T) { defer leaktest.AfterTest(t) ls := NewLocalSender() store := storage.Store{} - replica := proto.Replica{StoreID: store.Ident.StoreID} + replica := proto.ReplicaDescriptor{StoreID: store.Ident.StoreID} s, err := ls.GetStore(replica.StoreID) if s != nil || err == nil { t.Errorf("expected no stores in new local sender") @@ -154,7 +154,7 @@ func TestLocalSenderLookupReplica(t *testing.T) { RangeID: proto.RangeID(i), StartKey: rng.start, EndKey: rng.end, - Replicas: []proto.Replica{{StoreID: rng.storeID}}, + Replicas: []proto.ReplicaDescriptor{{StoreID: rng.storeID}}, } newRng, err := storage.NewReplica(d[i], s[i]) if err != nil { diff --git a/kv/replica_slice.go b/kv/replica_slice.go index 76fabbdb3a70..a9694eb3899b 100644 --- a/kv/replica_slice.go +++ b/kv/replica_slice.go @@ -27,7 +27,7 @@ import ( // replicaInfo extends the Replica structure with the associated node // descriptor. type replicaInfo struct { - proto.Replica + proto.ReplicaDescriptor NodeDesc *proto.NodeDescriptor } @@ -55,8 +55,8 @@ func newReplicaSlice(gossip *gossip.Gossip, desc *proto.RangeDescriptor) replica continue } replicas = append(replicas, replicaInfo{ - Replica: r, - NodeDesc: nd, + ReplicaDescriptor: r, + NodeDesc: nd, }) } return replicas diff --git a/kv/replica_slice_test.go b/kv/replica_slice_test.go index e9db682dbe60..603e8c9991dd 100644 --- a/kv/replica_slice_test.go +++ b/kv/replica_slice_test.go @@ -92,7 +92,7 @@ func TestReplicaSetMoveToFront(t *testing.T) { defer leaktest.AfterTest(t) rs := replicaSlice(nil) for i := 0; i < 5; i++ { - rs = append(rs, replicaInfo{Replica: proto.Replica{StoreID: proto.StoreID(i + 1)}}) + rs = append(rs, replicaInfo{ReplicaDescriptor: proto.ReplicaDescriptor{StoreID: proto.StoreID(i + 1)}}) } rs.MoveToFront(0) exp := []proto.StoreID{1, 2, 3, 4, 5} diff --git a/kv/txn_coord_sender_test.go b/kv/txn_coord_sender_test.go index 012f97799009..fe95b4987b4e 100644 --- a/kv/txn_coord_sender_test.go +++ b/kv/txn_coord_sender_test.go @@ -515,7 +515,7 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { Txn: &proto.Transaction{ Name: "test txn", }, - Replica: proto.Replica{ + Replica: proto.ReplicaDescriptor{ NodeID: 12345, }, }, diff --git a/multiraft/events.go b/multiraft/events.go index df5c7f94baf9..e89e9ee58c01 100644 --- a/multiraft/events.go +++ b/multiraft/events.go @@ -26,9 +26,9 @@ import ( // An EventLeaderElection is broadcast when a group starts or completes // an election. NodeID is zero when an election is in progress. type EventLeaderElection struct { - GroupID proto.RangeID - NodeID proto.RaftNodeID - Term uint64 + GroupID proto.RangeID + ReplicaID proto.ReplicaID + Term uint64 } // An EventCommandCommitted is broadcast whenever a command has been committed. @@ -52,7 +52,7 @@ type EventMembershipChangeCommitted struct { GroupID proto.RangeID CommandID string Index uint64 - NodeID proto.RaftNodeID + Replica proto.ReplicaDescriptor ChangeType raftpb.ConfChangeType Payload []byte diff --git a/multiraft/multiraft.go b/multiraft/multiraft.go index b2bd237d95f2..fa3c33ae99bd 100644 --- a/multiraft/multiraft.go +++ b/multiraft/multiraft.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/proto" "github.com/cockroachdb/cockroach/util" + "github.com/cockroachdb/cockroach/util/cache" "github.com/cockroachdb/cockroach/util/log" "github.com/cockroachdb/cockroach/util/stop" "github.com/coreos/etcd/raft" @@ -35,6 +36,14 @@ const ( noGroup = proto.RangeID(0) reqBufferSize = 100 + + // TODO(bdarnell): Determine the right size for this cache. Should + // the cache be partitioned so that replica descriptors from the + // range descriptors (which are the bulk of the data and can be + // reloaded from disk as needed) don't crowd out the + // message/snapshot descriptors (whose necessity is short-lived but + // cannot be recovered through other means if evicted)? + maxReplicaDescCacheSize = 1000 ) // An ErrGroupDeleted is returned for commands which are pending while their @@ -95,7 +104,8 @@ type MultiRaft struct { stopper *stop.Stopper multiNode raft.MultiNode Events chan []interface{} - nodeID proto.RaftNodeID + nodeID proto.NodeID + storeID proto.StoreID reqChan chan *RaftMessageRequest createGroupChan chan *createGroupOp removeGroupChan chan *removeGroupOp @@ -109,9 +119,12 @@ type MultiRaft struct { type multiraftServer MultiRaft // NewMultiRaft creates a MultiRaft object. -func NewMultiRaft(nodeID proto.RaftNodeID, config *Config, stopper *stop.Stopper) (*MultiRaft, error) { - if nodeID == 0 { - return nil, util.Errorf("Invalid RaftNodeID") +func NewMultiRaft(nodeID proto.NodeID, storeID proto.StoreID, config *Config, stopper *stop.Stopper) (*MultiRaft, error) { + if nodeID <= 0 { + return nil, util.Errorf("invalid NodeID: %s", nodeID) + } + if storeID <= 0 { + return nil, util.Errorf("invalid StoreID: %s", storeID) } if err := config.validate(); err != nil { return nil, err @@ -138,8 +151,9 @@ func NewMultiRaft(nodeID proto.RaftNodeID, config *Config, stopper *stop.Stopper m := &MultiRaft{ Config: *config, stopper: stopper, - multiNode: raft.StartMultiNode(uint64(nodeID)), + multiNode: raft.StartMultiNode(0), nodeID: nodeID, + storeID: storeID, // Output channel. Events: make(chan []interface{}), @@ -152,7 +166,7 @@ func NewMultiRaft(nodeID proto.RaftNodeID, config *Config, stopper *stop.Stopper callbackChan: make(chan func()), } - if err := m.Transport.Listen(nodeID, (*multiraftServer)(m)); err != nil { + if err := m.Transport.Listen(storeID, (*multiraftServer)(m)); err != nil { return nil, err } @@ -185,7 +199,7 @@ func (s *state) sendEvent(event interface{}) { func (s *state) fanoutHeartbeat(req *RaftMessageRequest) { // A heartbeat message is expanded into a heartbeat for each group // that the remote node is a part of. - fromID := proto.RaftNodeID(req.Message.From) + fromID := proto.NodeID(req.Message.From) groupCount := 0 followerCount := 0 if originNode, ok := s.nodes[fromID]; ok { @@ -193,15 +207,40 @@ func (s *state) fanoutHeartbeat(req *RaftMessageRequest) { groupCount++ // If we don't think that the sending node is leading that group, don't // propagate. - if s.groups[groupID].leader != fromID || fromID == s.nodeID { + if s.groups[groupID].leader.NodeID != fromID || fromID == s.nodeID { if log.V(8) { - log.Infof("node %v: not fanning out heartbeat to %v, msg is from %d and leader is %d", - s.nodeID, req.Message.To, fromID, s.groups[groupID].leader) + log.Infof("node %s: not fanning out heartbeat to %s, msg is from %s and leader is %s", + s.nodeID, groupID, fromID, s.groups[groupID].leader) + } + continue + } + + fromRepID, err := s.Storage.ReplicaIDForStore(groupID, req.FromReplica.StoreID) + if err != nil { + if log.V(3) { + log.Infof("node %s: not fanning out heartbeat to %s, could not find replica id for sending store %s", + s.nodeID, groupID, req.FromReplica.StoreID) + } + continue + } + + toRepID, err := s.Storage.ReplicaIDForStore(groupID, req.ToReplica.StoreID) + if err != nil { + if log.V(3) { + log.Infof("node %s: not fanning out heartbeat to %s, could not find replica id for receiving store %s", + s.nodeID, groupID, req.ToReplica.StoreID) } continue } followerCount++ - if err := s.multiNode.Step(context.Background(), uint64(groupID), req.Message); err != nil { + + groupMsg := raftpb.Message{ + Type: raftpb.MsgHeartbeat, + To: uint64(toRepID), + From: uint64(fromRepID), + } + + if err := s.multiNode.Step(context.Background(), uint64(groupID), groupMsg); err != nil { if log.V(4) { log.Infof("node %v: coalesced heartbeat step to group %v failed for message %s", s.nodeID, groupID, raft.DescribeMessage(req.Message, s.EntryFormatter)) @@ -230,33 +269,51 @@ func (s *state) fanoutHeartbeat(req *RaftMessageRequest) { // fanoutHeartbeatResponse sends the given heartbeat response to all groups // which overlap with the sender's groups and consider themselves leader. -func (s *state) fanoutHeartbeatResponse(fromID proto.RaftNodeID) { +func (s *state) fanoutHeartbeatResponse(req *RaftMessageRequest) { + fromID := proto.NodeID(req.Message.From) originNode, ok := s.nodes[fromID] if !ok { log.Warningf("node %v: not fanning out heartbeat response from unknown node %v", s.nodeID, fromID) return } - // Term in HeartbeatResponse is no meaning in fanouting. Otherwise it - // will cause Leader change to Follower if another group's term is - // greater than this. - req := raftpb.Message{ - From: uint64(fromID), - To: uint64(s.nodeID), - Type: raftpb.MsgHeartbeatResp, - } cnt := 0 for groupID := range originNode.groupIDs { // If we don't think that the local node is leader, don't propagate. - if s.groups[groupID].leader != s.nodeID || fromID == s.nodeID { + if s.groups[groupID].leader.NodeID != s.nodeID || fromID == s.nodeID { if log.V(8) { log.Infof("node %v: not fanning out heartbeat response to %v, msg is from %v and leader is %v", s.nodeID, groupID, fromID, s.groups[groupID].leader) } continue } - if err := s.multiNode.Step(context.Background(), uint64(groupID), req); err != nil { + + fromRepID, err := s.Storage.ReplicaIDForStore(groupID, req.FromReplica.StoreID) + if err != nil { + if log.V(3) { + log.Infof("node %s: not fanning out heartbeat to %s, could not find replica id for sending store %s", + s.nodeID, groupID, req.FromReplica.StoreID) + } + continue + } + + toRepID, err := s.Storage.ReplicaIDForStore(groupID, req.ToReplica.StoreID) + if err != nil { + if log.V(3) { + log.Infof("node %s: not fanning out heartbeat to %s, could not find replica id for receiving store %s", + s.nodeID, groupID, req.ToReplica.StoreID) + } + continue + } + + msg := raftpb.Message{ + Type: raftpb.MsgHeartbeatResp, + From: uint64(fromRepID), + To: uint64(toRepID), + } + + if err := s.multiNode.Step(context.Background(), uint64(groupID), msg); err != nil { if log.V(4) { log.Infof("node %v: coalesced heartbeat response step to group %v failed", s.nodeID, groupID) } @@ -319,7 +376,7 @@ func (m *MultiRaft) SubmitCommand(groupID proto.RangeID, commandID string, comma // 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 proto.RangeID, commandID string, - changeType raftpb.ConfChangeType, nodeID proto.RaftNodeID, payload []byte) <-chan error { + changeType raftpb.ConfChangeType, replica proto.ReplicaDescriptor, payload []byte) <-chan error { if log.V(6) { log.Infof("node %v proposing membership change to group %v", m.nodeID, groupID) } @@ -328,17 +385,27 @@ func (m *MultiRaft) ChangeGroupMembership(groupID proto.RangeID, commandID strin 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(nodeID), - Context: encodeCommand(commandID, payload), + 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, } @@ -364,9 +431,9 @@ type group struct { // committedTerm is the term of the most recently committed entry. committedTerm uint64 - // leader is the node ID of the last known leader for this group, or - // 0 if an election is in progress. - leader proto.RaftNodeID + // leader is the last known leader for this group, or all zeros + // if an election is in progress. + leader proto.ReplicaDescriptor // pending contains all commands that have been proposed but not yet // committed in the current term. When a proposal is committed, nil @@ -379,7 +446,7 @@ type group struct { // an earlier incarnation to be fed into a later one. writing bool // nodeIDs track the remote nodes associated with this group. - nodeIDs []proto.RaftNodeID + nodeIDs []proto.NodeID // waitForCallback is true while a configuration change callback // is waiting to be called. It's a bool other than a counter // as only one configuration change should be pending in range leader. @@ -398,7 +465,7 @@ type removeGroupOp struct { // node represents a connection to a remote node. type node struct { - nodeID proto.RaftNodeID + nodeID proto.NodeID groupIDs map[proto.RangeID]struct{} } @@ -418,9 +485,10 @@ func (n *node) unregisterGroup(groupID proto.RangeID) { // synchronization. type state struct { *MultiRaft - groups map[proto.RangeID]*group - nodes map[proto.RaftNodeID]*node - writeTask *writeTask + groups map[proto.RangeID]*group + nodes map[proto.NodeID]*node + writeTask *writeTask + replicaDescCache *cache.UnorderedCache // Buffer the events and send them in batch to avoid the deadlock // between s.Events channel and callbackChan. pendingEvents []interface{} @@ -430,8 +498,14 @@ func newState(m *MultiRaft) *state { return &state{ MultiRaft: m, groups: make(map[proto.RangeID]*group), - nodes: make(map[proto.RaftNodeID]*node), + nodes: make(map[proto.NodeID]*node), writeTask: newWriteTask(m.Storage), + replicaDescCache: cache.NewUnorderedCache(cache.Config{ + Policy: cache.CacheLRU, + ShouldEvict: func(size int, key, value interface{}) bool { + return size > maxReplicaDescCacheSize + }, + }), } } @@ -498,8 +572,10 @@ func (s *state) start() { case raftpb.MsgHeartbeat: s.fanoutHeartbeat(req) case raftpb.MsgHeartbeatResp: - s.fanoutHeartbeatResponse(proto.RaftNodeID(req.Message.From)) + s.fanoutHeartbeatResponse(req) default: + s.CacheReplicaDescriptor(req.GroupID, req.FromReplica) + s.CacheReplicaDescriptor(req.GroupID, req.ToReplica) // We only want to lazily create the group if it's not heartbeat-related; // our heartbeats are coalesced and contain a dummy GroupID. // TODO(tschottdorf) still shouldn't hurt to move this part outside, @@ -508,7 +584,7 @@ func (s *state) start() { if log.V(1) { log.Infof("node %v: got message for unknown group %d; creating it", s.nodeID, req.GroupID) } - if err := s.createGroup(req.GroupID); err != nil { + if err := s.createGroup(req.GroupID, req.ToReplica.ReplicaID); err != nil { log.Warningf("Error creating group %d: %s", req.GroupID, err) break } @@ -525,7 +601,7 @@ func (s *state) start() { if log.V(6) { log.Infof("node %v: got op %#v", s.nodeID, op) } - op.ch <- s.createGroup(op.groupID) + op.ch <- s.createGroup(op.groupID, 0) case op := <-s.removeGroupChan: if log.V(6) { @@ -627,15 +703,15 @@ func (s *state) coalescedHeartbeat() { func (s *state) stop() { if log.V(6) { - log.Infof("node %v stopping", s.nodeID) + log.Infof("store %s stopping", s.storeID) } s.MultiRaft.multiNode.Stop() - s.MultiRaft.Transport.Stop(s.nodeID) + s.MultiRaft.Transport.Stop(s.storeID) } // addNode creates a node and registers the given group (if not nil) // for that node. -func (s *state) addNode(nodeID proto.RaftNodeID, g *group) error { +func (s *state) addNode(nodeID proto.NodeID, g *group) error { newNode, ok := s.nodes[nodeID] if !ok { s.nodes[nodeID] = &node{ @@ -653,7 +729,7 @@ func (s *state) addNode(nodeID proto.RaftNodeID, g *group) error { } // removeNode removes a node from a group. -func (s *state) removeNode(nodeID proto.RaftNodeID, g *group) error { +func (s *state) removeNode(nodeID proto.NodeID, g *group) error { node, ok := s.nodes[nodeID] if !ok { return util.Errorf("cannot remove unknown node %s", nodeID) @@ -679,7 +755,12 @@ func (s *state) removeNode(nodeID proto.RaftNodeID, g *group) error { return nil } -func (s *state) createGroup(groupID proto.RangeID) error { +// createGroup is called in two situations: by the application at +// startup (in which case the replicaID argument is zero and the +// replicaID will be loaded from storage), and in response to incoming +// messages (in which case the replicaID comes from the incoming +// message, since nothing is on disk yet). +func (s *state) createGroup(groupID proto.RangeID, replicaID proto.ReplicaID) error { if _, ok := s.groups[groupID]; ok { return nil } @@ -693,6 +774,33 @@ func (s *state) createGroup(groupID proto.RangeID) error { return err } + // Find our store ID in the replicas list. + for _, r := range cs.Nodes { + repDesc, err := s.ReplicaDescriptor(groupID, proto.ReplicaID(r)) + if err != nil { + return err + } + if repDesc.StoreID == s.storeID { + if replicaID == 0 { + replicaID = repDesc.ReplicaID + } else if replicaID != repDesc.ReplicaID { + return util.Errorf("inconsistent replica ID: passed %d, but found %s by scanning ConfState for store %s", + replicaID, repDesc.ReplicaID, s.storeID) + } + replicaID = repDesc.ReplicaID + break + } + } + if replicaID == 0 { + return util.Errorf("couldn't find replica ID for this store (%s) in range %d", + s.storeID, groupID) + } + s.CacheReplicaDescriptor(groupID, proto.ReplicaDescriptor{ + ReplicaID: replicaID, + NodeID: s.nodeID, + StoreID: s.storeID, + }) + var appliedIndex uint64 if s.StateMachine != nil { appliedIndex, err = s.StateMachine.AppliedIndex(groupID) @@ -702,6 +810,7 @@ func (s *state) createGroup(groupID proto.RangeID) error { } raftCfg := &raft.Config{ + ID: uint64(replicaID), Applied: appliedIndex, ElectionTick: s.ElectionTimeoutTicks, HeartbeatTick: s.HeartbeatIntervalTicks, @@ -720,8 +829,14 @@ func (s *state) createGroup(groupID proto.RangeID) error { } s.groups[groupID] = g - for _, nodeID := range cs.Nodes { - if err := s.addNode(proto.RaftNodeID(nodeID), g); err != nil { + for _, id := range cs.Nodes { + replicaID := proto.ReplicaID(id) + replica, err := s.ReplicaDescriptor(groupID, replicaID) + if err != nil { + return err + } + + if err := s.addNode(replica.NodeID, g); err != nil { return err } } @@ -742,8 +857,17 @@ func (s *state) createGroup(groupID proto.RangeID) error { // could happen is both nodes ending up in candidate state, timing // out and then voting again. This is expected to be an extremely // rare event. - if len(cs.Nodes) == 1 && s.MultiRaft.nodeID == proto.RaftNodeID(cs.Nodes[0]) { - return s.multiNode.Campaign(context.Background(), uint64(groupID)) + if len(cs.Nodes) == 1 { + replica, err := s.ReplicaDescriptor(groupID, proto.ReplicaID(cs.Nodes[0])) + if err != nil { + return err + } + if replica.StoreID == s.storeID { + log.Infof("node %s campaigning because initial confstate is %v", s.nodeID, cs.Nodes) + if err := s.multiNode.Campaign(context.Background(), uint64(groupID)); err != nil { + return err + } + } } return nil } @@ -896,14 +1020,27 @@ func (s *state) processCommittedEntry(groupID proto.RangeID, g *group, entry raf } var payload []byte if len(cc.Context) > 0 { - commandID, payload = decodeCommand(cc.Context) + var ctx ConfChangeContext + if err := ctx.Unmarshal(cc.Context); err != nil { + log.Fatalf("invalid ConfChangeContext: %s", err) + } + commandID = ctx.CommandID + payload = ctx.Payload + s.CacheReplicaDescriptor(groupID, ctx.Replica) + } + replica, err := s.ReplicaDescriptor(groupID, proto.ReplicaID(cc.NodeID)) + if err != nil { + // TODO(bdarnell): stash Replica information somewhere so we can have it here + // with no chance of failure. + log.Fatalf("could not look up replica info (node %s, group %d, replica %d): %s", + s.nodeID, groupID, cc.NodeID, err) } g.waitForCallback = true s.sendEvent(&EventMembershipChangeCommitted{ GroupID: groupID, CommandID: commandID, Index: entry.Index, - NodeID: proto.RaftNodeID(cc.NodeID), + Replica: replica, ChangeType: cc.Type, Payload: payload, Callback: func(err error) { @@ -916,9 +1053,9 @@ func (s *state) processCommittedEntry(groupID proto.RangeID, g *group, entry raf // TODO(bdarnell): dedupe by keeping a record of recently-applied commandIDs switch cc.Type { case raftpb.ConfChangeAddNode: - err = s.addNode(proto.RaftNodeID(cc.NodeID), g) + err = s.addNode(replica.NodeID, g) case raftpb.ConfChangeRemoveNode: - err = s.removeNode(proto.RaftNodeID(cc.NodeID), g) + err = s.removeNode(replica.NodeID, g) case raftpb.ConfChangeUpdateNode: // Updates don't concern multiraft, they are simply passed through. } @@ -954,24 +1091,53 @@ func (s *state) sendMessage(g *group, msg raftpb.Message) { log.Infof("node %v sending message %.200s to %v", s.nodeID, raft.DescribeMessage(msg, s.EntryFormatter), msg.To) } - nodeID := proto.RaftNodeID(msg.To) groupID := noGroup - if g != nil { + var toReplica proto.ReplicaDescriptor + var fromReplica proto.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 = proto.NodeID(msg.To) + toReplica.StoreID = proto.StoreID(msg.To) + fromReplica.NodeID = proto.NodeID(msg.From) + fromReplica.StoreID = proto.StoreID(msg.From) + } else { + // Regular message: To/From fields are replica IDs. groupID = g.id + var err error + toReplica, err = s.ReplicaDescriptor(groupID, proto.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, proto.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[nodeID]; !ok { + if _, ok := s.nodes[toReplica.NodeID]; !ok { if log.V(4) { - log.Infof("node %v: connecting to new node %v", s.nodeID, nodeID) + log.Infof("node %v: connecting to new node %v", s.nodeID, toReplica.NodeID) } - if err := s.addNode(nodeID, g); err != nil { + if err := s.addNode(toReplica.NodeID, g); err != nil { log.Errorf("node %v: error adding group %v to node %v: %v", - s.nodeID, groupID, nodeID, err) + s.nodeID, groupID, toReplica.NodeID, err) } } - err := s.Transport.Send(&RaftMessageRequest{groupID, msg}) + 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, nodeID, err) + 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)) } @@ -992,20 +1158,32 @@ func (s *state) sendMessage(g *group, msg raftpb.Message) { func (s *state) maybeSendLeaderEvent(groupID proto.RangeID, g *group, ready *raft.Ready) { term := g.committedTerm if ready.SoftState != nil { - // Always save the leader whenever we get a SoftState. - g.leader = proto.RaftNodeID(ready.SoftState.Lead) + // Always save the leader whenever it changes. + if proto.ReplicaID(ready.SoftState.Lead) != g.leader.ReplicaID { + if ready.SoftState.Lead == 0 { + g.leader = proto.ReplicaDescriptor{} + } else { + if repl, err := s.ReplicaDescriptor(g.id, proto.ReplicaID(ready.SoftState.Lead)); err != nil { + log.Warningf("node %s: failed to look up address of replica %d in group %d: %s", + s.nodeID, ready.SoftState.Lead, g.id, err) + g.leader = proto.ReplicaDescriptor{} + } else { + g.leader = repl + } + } + } } if len(ready.CommittedEntries) > 0 { term = ready.CommittedEntries[len(ready.CommittedEntries)-1].Term } - if term != g.committedTerm && g.leader != 0 { + if term != g.committedTerm && g.leader.ReplicaID != 0 { // Whenever the committed term has advanced and we know our leader, // emit an event. g.committedTerm = term s.sendEvent(&EventLeaderElection{ - GroupID: groupID, - NodeID: proto.RaftNodeID(g.leader), - Term: g.committedTerm, + GroupID: groupID, + ReplicaID: g.leader.ReplicaID, + Term: g.committedTerm, }) // Re-submit all pending proposals @@ -1053,11 +1231,15 @@ func (s *state) handleWriteResponse(response *writeResponse, readyGroups map[uin if !raft.IsEmptySnap(ready.Snapshot) { // Sync the group/node mapping with the information contained in the snapshot. - for _, nodeID := range ready.Snapshot.Metadata.ConfState.Nodes { + replicas, err := s.Storage.ReplicasFromSnapshot(ready.Snapshot) + if err != nil { + log.Fatalf("failed to parse snapshot: %s", err) + } + for _, rep := range replicas { // TODO(bdarnell): if we had any information that predated this snapshot // we must remove those nodes. - if err := s.addNode(proto.RaftNodeID(nodeID), g); err != nil { - log.Errorf("node %v: error adding node %v", s.nodeID, nodeID) + if err := s.addNode(rep.NodeID, g); err != nil { + log.Errorf("node %v: error adding node %v", s.nodeID, rep.NodeID) } } } @@ -1084,3 +1266,23 @@ func (s *state) handleWriteResponse(response *writeResponse, readyGroups map[uin } } } + +type replicaDescCacheKey struct { + groupID proto.RangeID + replicaID proto.ReplicaID +} + +func (s *state) ReplicaDescriptor(groupID proto.RangeID, replicaID proto.ReplicaID) (proto.ReplicaDescriptor, error) { + if rep, ok := s.replicaDescCache.Get(replicaDescCacheKey{groupID, replicaID}); ok { + return rep.(proto.ReplicaDescriptor), nil + } + rep, err := s.Storage.ReplicaDescriptor(groupID, replicaID) + if err == nil { + s.replicaDescCache.Add(replicaDescCacheKey{groupID, replicaID}, rep) + } + return rep, err +} + +func (s *state) CacheReplicaDescriptor(groupID proto.RangeID, replica proto.ReplicaDescriptor) { + s.replicaDescCache.Add(replicaDescCacheKey{groupID, replica.ReplicaID}, replica) +} diff --git a/multiraft/multiraft_test.go b/multiraft/multiraft_test.go index 9e0194668193..b681eafa25f3 100644 --- a/multiraft/multiraft_test.go +++ b/multiraft/multiraft_test.go @@ -76,7 +76,7 @@ func newTestCluster(transport Transport, size int, stopper *stop.Stopper, t *tes HeartbeatIntervalTicks: 1, TickInterval: time.Hour, // not in use } - mr, err := NewMultiRaft(proto.RaftNodeID(i+1), config, stopper) + mr, err := NewMultiRaft(proto.NodeID(i+1), proto.StoreID(i+1), config, stopper) if err != nil { t.Fatal(err) } @@ -155,7 +155,7 @@ func (c *testCluster) waitForElection(i int) *EventLeaderElection { panic("got nil LeaderElection event, channel likely closed") } // Ignore events with NodeID 0; these mark elections that are in progress. - if e.NodeID != 0 { + if e.ReplicaID != 0 { return e } } @@ -168,7 +168,8 @@ func (c *testCluster) elect(leaderIndex int, groupID proto.RangeID) { c.triggerElection(leaderIndex, groupID) for _, i := range c.groups[groupID] { el := c.waitForElection(i) - if el.NodeID != c.nodes[leaderIndex].nodeID { + // With the in-memory storage used in these tests, replica and node IDs are interchangeable. + if el.ReplicaID != proto.ReplicaID(c.nodes[leaderIndex].nodeID) { c.t.Fatalf("wrong leader elected; wanted node %d but got event %v", leaderIndex, el) } if el.GroupID != groupID { @@ -255,9 +256,9 @@ func TestLeaderElectionEvent(t *testing.T) { select { case e := <-cluster.events[1].LeaderElection: if !reflect.DeepEqual(e, &EventLeaderElection{ - GroupID: groupID, - NodeID: 3, - Term: 42, + GroupID: groupID, + ReplicaID: 3, + Term: 42, }) { t.Errorf("election event did not match expectations: %+v", e) } @@ -374,7 +375,11 @@ func TestMembershipChange(t *testing.T) { for i := 1; i < 4; i++ { ch := cluster.nodes[0].ChangeGroupMembership(groupID, makeCommandID(), raftpb.ConfChangeAddNode, - cluster.nodes[i].nodeID, nil) + proto.ReplicaDescriptor{ + NodeID: cluster.nodes[i].nodeID, + StoreID: proto.StoreID(cluster.nodes[i].nodeID), + ReplicaID: proto.ReplicaID(cluster.nodes[i].nodeID), + }, nil) <-ch } @@ -458,7 +463,11 @@ func TestRemoveLeader(t *testing.T) { log.Infof("adding node %d", i+groupSize) ch := cluster.nodes[i].ChangeGroupMembership(groupID, makeCommandID(), raftpb.ConfChangeAddNode, - cluster.nodes[i+groupSize].nodeID, nil) + proto.ReplicaDescriptor{ + NodeID: cluster.nodes[i+groupSize].nodeID, + StoreID: proto.StoreID(cluster.nodes[i+groupSize].nodeID), + ReplicaID: proto.ReplicaID(cluster.nodes[i+groupSize].nodeID), + }, nil) if err := <-ch; err != nil { t.Fatal(err) } @@ -466,7 +475,11 @@ func TestRemoveLeader(t *testing.T) { log.Infof("removing node %d", i) ch = cluster.nodes[i].ChangeGroupMembership(groupID, makeCommandID(), raftpb.ConfChangeRemoveNode, - cluster.nodes[i].nodeID, nil) + proto.ReplicaDescriptor{ + NodeID: cluster.nodes[i].nodeID, + StoreID: proto.StoreID(cluster.nodes[i].nodeID), + ReplicaID: proto.ReplicaID(cluster.nodes[i].nodeID), + }, nil) if err := <-ch; err != nil { t.Fatal(err) } diff --git a/multiraft/raft.go b/multiraft/raft.go index fcea40c26355..0723d14f6037 100644 --- a/multiraft/raft.go +++ b/multiraft/raft.go @@ -49,7 +49,7 @@ func (r *raftLogger) prependContext(format string, v []interface{}) string { var s string if r.group != 0 { v2 := append([]interface{}{r.group}, v...) - s = fmt.Sprintf("group %d "+format, v2...) + s = fmt.Sprintf("[group %d] "+format, v2...) } else { s = fmt.Sprintf(format, v...) } diff --git a/multiraft/rpc.pb.go b/multiraft/rpc.pb.go index 4b81b8be4dd8..5eab04d77f4b 100644 --- a/multiraft/rpc.pb.go +++ b/multiraft/rpc.pb.go @@ -11,12 +11,14 @@ It has these top-level messages: RaftMessageRequest RaftMessageResponse + ConfChangeContext */ package multiraft import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" +import cockroach_proto "github.com/cockroachdb/cockroach/proto" import raftpb "github.com/coreos/etcd/raft/raftpb" // discarding unused import gogoproto "github.com/cockroachdb/gogoproto" @@ -33,8 +35,10 @@ var _ = math.Inf // RaftMessageRequest is the request used to send raft messages using our // protobuf-based RPC codec. type RaftMessageRequest struct { - GroupID github_com_cockroachdb_cockroach_proto.RangeID `protobuf:"varint,1,opt,name=group_id,casttype=github.com/cockroachdb/cockroach/proto.RangeID" json:"group_id"` - Message raftpb.Message `protobuf:"bytes,2,opt,name=message" json:"message"` + GroupID github_com_cockroachdb_cockroach_proto.RangeID `protobuf:"varint,1,opt,name=group_id,casttype=github.com/cockroachdb/cockroach/proto.RangeID" json:"group_id"` + FromReplica cockroach_proto.ReplicaDescriptor `protobuf:"bytes,2,opt,name=from_replica" json:"from_replica"` + ToReplica cockroach_proto.ReplicaDescriptor `protobuf:"bytes,3,opt,name=to_replica" json:"to_replica"` + Message raftpb.Message `protobuf:"bytes,4,opt,name=message" json:"message"` } func (m *RaftMessageRequest) Reset() { *m = RaftMessageRequest{} } @@ -48,6 +52,20 @@ func (m *RaftMessageRequest) GetGroupID() github_com_cockroachdb_cockroach_proto return 0 } +func (m *RaftMessageRequest) GetFromReplica() cockroach_proto.ReplicaDescriptor { + if m != nil { + return m.FromReplica + } + return cockroach_proto.ReplicaDescriptor{} +} + +func (m *RaftMessageRequest) GetToReplica() cockroach_proto.ReplicaDescriptor { + if m != nil { + return m.ToReplica + } + return cockroach_proto.ReplicaDescriptor{} +} + func (m *RaftMessageRequest) GetMessage() raftpb.Message { if m != nil { return m.Message @@ -64,6 +82,41 @@ func (m *RaftMessageResponse) Reset() { *m = RaftMessageResponse{} } func (m *RaftMessageResponse) String() string { return proto.CompactTextString(m) } func (*RaftMessageResponse) ProtoMessage() {} +// ConfChangeContext is encoded in the raftpb.ConfChange.Context field. +type ConfChangeContext struct { + CommandID string `protobuf:"bytes,1,opt,name=command_id" json:"command_id"` + // Payload is the application-level command (i.e. an encoded + // proto.EndTransactionRequest). + Payload []byte `protobuf:"bytes,2,opt,name=payload" json:"payload,omitempty"` + // Replica contains full details about the replica being added or removed. + Replica cockroach_proto.ReplicaDescriptor `protobuf:"bytes,3,opt,name=replica" json:"replica"` +} + +func (m *ConfChangeContext) Reset() { *m = ConfChangeContext{} } +func (m *ConfChangeContext) String() string { return proto.CompactTextString(m) } +func (*ConfChangeContext) ProtoMessage() {} + +func (m *ConfChangeContext) GetCommandID() string { + if m != nil { + return m.CommandID + } + return "" +} + +func (m *ConfChangeContext) GetPayload() []byte { + if m != nil { + return m.Payload + } + return nil +} + +func (m *ConfChangeContext) GetReplica() cockroach_proto.ReplicaDescriptor { + if m != nil { + return m.Replica + } + return cockroach_proto.ReplicaDescriptor{} +} + func (m *RaftMessageRequest) Marshal() (data []byte, err error) { size := m.Size() data = make([]byte, size) @@ -84,12 +137,28 @@ func (m *RaftMessageRequest) MarshalTo(data []byte) (int, error) { i = encodeVarintRpc(data, i, uint64(m.GroupID)) data[i] = 0x12 i++ - i = encodeVarintRpc(data, i, uint64(m.Message.Size())) - n1, err := m.Message.MarshalTo(data[i:]) + i = encodeVarintRpc(data, i, uint64(m.FromReplica.Size())) + n1, err := m.FromReplica.MarshalTo(data[i:]) if err != nil { return 0, err } i += n1 + data[i] = 0x1a + i++ + i = encodeVarintRpc(data, i, uint64(m.ToReplica.Size())) + n2, err := m.ToReplica.MarshalTo(data[i:]) + if err != nil { + return 0, err + } + i += n2 + data[i] = 0x22 + i++ + i = encodeVarintRpc(data, i, uint64(m.Message.Size())) + n3, err := m.Message.MarshalTo(data[i:]) + if err != nil { + return 0, err + } + i += n3 return i, nil } @@ -111,6 +180,42 @@ func (m *RaftMessageResponse) MarshalTo(data []byte) (int, error) { return i, nil } +func (m *ConfChangeContext) Marshal() (data []byte, err error) { + size := m.Size() + data = make([]byte, size) + n, err := m.MarshalTo(data) + if err != nil { + return nil, err + } + return data[:n], nil +} + +func (m *ConfChangeContext) MarshalTo(data []byte) (int, error) { + var i int + _ = i + var l int + _ = l + data[i] = 0xa + i++ + i = encodeVarintRpc(data, i, uint64(len(m.CommandID))) + i += copy(data[i:], m.CommandID) + if m.Payload != nil { + data[i] = 0x12 + i++ + i = encodeVarintRpc(data, i, uint64(len(m.Payload))) + i += copy(data[i:], m.Payload) + } + data[i] = 0x1a + i++ + i = encodeVarintRpc(data, i, uint64(m.Replica.Size())) + n4, err := m.Replica.MarshalTo(data[i:]) + if err != nil { + return 0, err + } + i += n4 + return i, nil +} + func encodeFixed64Rpc(data []byte, offset int, v uint64) int { data[offset] = uint8(v) data[offset+1] = uint8(v >> 8) @@ -142,6 +247,10 @@ func (m *RaftMessageRequest) Size() (n int) { var l int _ = l n += 1 + sovRpc(uint64(m.GroupID)) + l = m.FromReplica.Size() + n += 1 + l + sovRpc(uint64(l)) + l = m.ToReplica.Size() + n += 1 + l + sovRpc(uint64(l)) l = m.Message.Size() n += 1 + l + sovRpc(uint64(l)) return n @@ -153,6 +262,20 @@ func (m *RaftMessageResponse) Size() (n int) { return n } +func (m *ConfChangeContext) Size() (n int) { + var l int + _ = l + l = len(m.CommandID) + n += 1 + l + sovRpc(uint64(l)) + if m.Payload != nil { + l = len(m.Payload) + n += 1 + l + sovRpc(uint64(l)) + } + l = m.Replica.Size() + n += 1 + l + sovRpc(uint64(l)) + return n +} + func sovRpc(x uint64) (n int) { for { n++ @@ -215,6 +338,66 @@ func (m *RaftMessageRequest) Unmarshal(data []byte) error { } } case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field FromReplica", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.FromReplica.Unmarshal(data[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ToReplica", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ToReplica.Unmarshal(data[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) } @@ -315,6 +498,143 @@ func (m *RaftMessageResponse) Unmarshal(data []byte) error { } return nil } +func (m *ConfChangeContext) Unmarshal(data []byte) error { + l := len(data) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ConfChangeContext: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ConfChangeContext: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommandID", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CommandID = string(data[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Payload", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + byteLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + byteLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Payload = append([]byte{}, data[iNdEx:postIndex]...) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Replica", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRpc + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := data[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRpc + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Replica.Unmarshal(data[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipRpc(data[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthRpc + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipRpc(data []byte) (n int, err error) { l := len(data) iNdEx := 0 diff --git a/multiraft/rpc.proto b/multiraft/rpc.proto index 4011bde49006..7f11ac08c83a 100644 --- a/multiraft/rpc.proto +++ b/multiraft/rpc.proto @@ -19,6 +19,7 @@ syntax = "proto2"; package cockroach.multiraft; option go_package = "multiraft"; +import "cockroach/proto/metadata.proto"; import "etcd/raft/raftpb/raft.proto"; import "gogoproto/gogo.proto"; @@ -34,10 +35,26 @@ message RaftMessageRequest { (gogoproto.customname) = "GroupID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/proto.RangeID"]; - optional raftpb.Message message = 2 [(gogoproto.nullable) = false]; + optional proto.ReplicaDescriptor from_replica = 2 [(gogoproto.nullable) = false]; + optional proto.ReplicaDescriptor to_replica = 3 [(gogoproto.nullable) = false]; + + optional raftpb.Message message = 4 [(gogoproto.nullable) = false]; } // RaftMessageResponse is an empty message returned by raft RPCs. If a // response is needed it will be sent as a separate message. message RaftMessageResponse { } + +// ConfChangeContext is encoded in the raftpb.ConfChange.Context field. +message ConfChangeContext { + optional string command_id = 1 [(gogoproto.nullable) = false, + (gogoproto.customname) = "CommandID"]; + + // Payload is the application-level command (i.e. an encoded + // proto.EndTransactionRequest). + optional bytes payload = 2; + + // Replica contains full details about the replica being added or removed. + optional proto.ReplicaDescriptor replica = 3 [(gogoproto.nullable) = false]; +} diff --git a/multiraft/storage.go b/multiraft/storage.go index 438d6a81f576..6a07ecb685c5 100644 --- a/multiraft/storage.go +++ b/multiraft/storage.go @@ -42,6 +42,9 @@ var _ WriteableGroupStorage = (*raft.MemoryStorage)(nil) // of raft data. type Storage interface { GroupStorage(groupID proto.RangeID) WriteableGroupStorage + ReplicaDescriptor(groupID proto.RangeID, replicaID proto.ReplicaID) (proto.ReplicaDescriptor, error) + ReplicaIDForStore(groupID proto.RangeID, storeID proto.StoreID) (proto.ReplicaID, error) + ReplicasFromSnapshot(snap raftpb.Snapshot) ([]proto.ReplicaDescriptor, error) } // The StateMachine interface is supplied by the application to manage a persistent @@ -81,6 +84,26 @@ func (m *MemoryStorage) GroupStorage(groupID proto.RangeID) WriteableGroupStorag return g } +// ReplicaDescriptor implements the Storage interface by returning a +// dummy descriptor. +func (m *MemoryStorage) ReplicaDescriptor(groupID proto.RangeID, replicaID proto.ReplicaID) (proto.ReplicaDescriptor, error) { + return proto.ReplicaDescriptor{ + ReplicaID: replicaID, + NodeID: proto.NodeID(replicaID), + StoreID: proto.StoreID(replicaID), + }, nil +} + +// ReplicaIDForStore implements the Storage interface. +func (m *MemoryStorage) ReplicaIDForStore(groupID proto.RangeID, storeID proto.StoreID) (proto.ReplicaID, error) { + return proto.ReplicaID(storeID), nil +} + +// ReplicasFromSnapshot implements the Storage interface. +func (m *MemoryStorage) ReplicasFromSnapshot(_ raftpb.Snapshot) ([]proto.ReplicaDescriptor, error) { + return nil, nil +} + // groupWriteRequest represents a set of changes to make to a group. type groupWriteRequest struct { state raftpb.HardState diff --git a/multiraft/storage_test.go b/multiraft/storage_test.go index e8f54f2d64d6..1e8aae506581 100644 --- a/multiraft/storage_test.go +++ b/multiraft/storage_test.go @@ -40,6 +40,18 @@ func (b *BlockableStorage) GroupStorage(g proto.RangeID) WriteableGroupStorage { return &blockableGroupStorage{b, b.storage.GroupStorage(g)} } +func (b *BlockableStorage) ReplicaDescriptor(groupID proto.RangeID, replicaID proto.ReplicaID) (proto.ReplicaDescriptor, error) { + return b.storage.ReplicaDescriptor(groupID, replicaID) +} + +func (b *BlockableStorage) ReplicaIDForStore(groupID proto.RangeID, storeID proto.StoreID) (proto.ReplicaID, error) { + return b.storage.ReplicaIDForStore(groupID, storeID) +} + +func (b *BlockableStorage) ReplicasFromSnapshot(snap raftpb.Snapshot) ([]proto.ReplicaDescriptor, error) { + return b.storage.ReplicasFromSnapshot(snap) +} + type blockableGroupStorage struct { b *BlockableStorage s WriteableGroupStorage diff --git a/multiraft/transport.go b/multiraft/transport.go index ad4c3a59d6ee..ceab137ca251 100644 --- a/multiraft/transport.go +++ b/multiraft/transport.go @@ -33,18 +33,15 @@ import ( ) // The Transport interface is supplied by the application to manage communication with -// other nodes. It is responsible for mapping from IDs to some communication channel -// (in the simplest case, a host:port pair could be used as an ID, although this would -// make it impossible to move an instance from one host to another except by syncing -// up a new node from scratch). +// other nodes. It is responsible for mapping from IDs to some communication channel. type Transport interface { - // Listen informs the Transport of the local node's ID and callback interface. + // Listen informs the Transport of a local store's ID and callback interface. // The Transport should associate the given id with the server object so other Transport's // Connect methods can find it. - Listen(id proto.RaftNodeID, server ServerInterface) error + Listen(id proto.StoreID, server ServerInterface) error // Stop undoes a previous Listen. - Stop(id proto.RaftNodeID) + Stop(id proto.StoreID) // Send a message to the node specified in the request's To field. Send(req *RaftMessageRequest) error @@ -64,8 +61,8 @@ var ( type localRPCTransport struct { mu sync.Mutex - servers map[proto.RaftNodeID]*crpc.Server - clients map[proto.RaftNodeID]*netrpc.Client + servers map[proto.StoreID]*crpc.Server + clients map[proto.StoreID]*netrpc.Client conns map[net.Conn]struct{} closed chan struct{} stopper *stop.Stopper @@ -78,15 +75,15 @@ type localRPCTransport struct { // Because this is just for local testing, it doesn't use TLS. func NewLocalRPCTransport(stopper *stop.Stopper) Transport { return &localRPCTransport{ - servers: make(map[proto.RaftNodeID]*crpc.Server), - clients: make(map[proto.RaftNodeID]*netrpc.Client), + servers: make(map[proto.StoreID]*crpc.Server), + clients: make(map[proto.StoreID]*netrpc.Client), conns: make(map[net.Conn]struct{}), closed: make(chan struct{}), stopper: stopper, } } -func (lt *localRPCTransport) Listen(id proto.RaftNodeID, server ServerInterface) error { +func (lt *localRPCTransport) Listen(id proto.StoreID, server ServerInterface) error { addr := util.CreateTestAddr("tcp") rpcServer := crpc.NewServer(addr, &crpc.Context{ Context: base.Context{ @@ -115,7 +112,7 @@ func (lt *localRPCTransport) Listen(id proto.RaftNodeID, server ServerInterface) return rpcServer.Start() } -func (lt *localRPCTransport) Stop(id proto.RaftNodeID) { +func (lt *localRPCTransport) Stop(id proto.StoreID) { lt.mu.Lock() defer lt.mu.Unlock() lt.servers[id].Close() @@ -126,7 +123,7 @@ func (lt *localRPCTransport) Stop(id proto.RaftNodeID) { } } -func (lt *localRPCTransport) getClient(id proto.RaftNodeID) (*netrpc.Client, error) { +func (lt *localRPCTransport) getClient(id proto.StoreID) (*netrpc.Client, error) { lt.mu.Lock() defer lt.mu.Unlock() @@ -158,7 +155,7 @@ func (lt *localRPCTransport) getClient(id proto.RaftNodeID) (*netrpc.Client, err } func (lt *localRPCTransport) Send(req *RaftMessageRequest) error { - client, err := lt.getClient(proto.RaftNodeID(req.Message.To)) + client, err := lt.getClient(req.ToReplica.StoreID) if err != nil { return err } diff --git a/multiraft/transport_test.go b/multiraft/transport_test.go index 539e2d84e336..08431fec404c 100644 --- a/multiraft/transport_test.go +++ b/multiraft/transport_test.go @@ -27,7 +27,7 @@ import ( type localInterceptableTransport struct { mu sync.Mutex - listeners map[proto.RaftNodeID]ServerInterface + listeners map[proto.StoreID]ServerInterface messages chan *RaftMessageRequest Events chan *interceptMessage stopper *stop.Stopper @@ -39,7 +39,7 @@ type localInterceptableTransport struct { // which they are queued, intercepted and blocked until acknowledged. func NewLocalInterceptableTransport(stopper *stop.Stopper) Transport { lt := &localInterceptableTransport{ - listeners: make(map[proto.RaftNodeID]ServerInterface), + listeners: make(map[proto.StoreID]ServerInterface), messages: make(chan *RaftMessageRequest, 100), Events: make(chan *interceptMessage), stopper: stopper, @@ -77,7 +77,7 @@ func (lt *localInterceptableTransport) handleMessage(msg *RaftMessageRequest) { lt.Events <- iMsg <-ack lt.mu.Lock() - srv, ok := lt.listeners[proto.RaftNodeID(msg.Message.To)] + srv, ok := lt.listeners[msg.ToReplica.StoreID] lt.mu.Unlock() if !ok { return @@ -90,14 +90,14 @@ func (lt *localInterceptableTransport) handleMessage(msg *RaftMessageRequest) { } } -func (lt *localInterceptableTransport) Listen(id proto.RaftNodeID, server ServerInterface) error { +func (lt *localInterceptableTransport) Listen(id proto.StoreID, server ServerInterface) error { lt.mu.Lock() defer lt.mu.Unlock() lt.listeners[id] = server return nil } -func (lt *localInterceptableTransport) Stop(id proto.RaftNodeID) { +func (lt *localInterceptableTransport) Stop(id proto.StoreID) { lt.mu.Lock() delete(lt.listeners, id) lt.mu.Unlock() diff --git a/proto/api.go b/proto/api.go index fd70e11f0ae3..fe55597e72dd 100644 --- a/proto/api.go +++ b/proto/api.go @@ -20,6 +20,7 @@ package proto import ( "fmt" "math/rand" + "strconv" gogoproto "github.com/gogo/protobuf/proto" ) @@ -27,6 +28,11 @@ import ( // A RangeID is a unique ID associated to a Raft consensus group. type RangeID int64 +// String implements the fmt.Stringer interface. +func (r RangeID) String() string { + return strconv.FormatInt(int64(r), 10) +} + // RangeIDSlice implements sort.Interface. type RangeIDSlice []RangeID diff --git a/proto/api.pb.go b/proto/api.pb.go index adfa3bd7464e..1565c724b3a5 100644 --- a/proto/api.pb.go +++ b/proto/api.pb.go @@ -101,7 +101,7 @@ RaftTruncatedState RaftSnapshotData Attributes - Replica + ReplicaDescriptor RangeDescriptor RangeTree RangeTreeNode @@ -270,7 +270,7 @@ type RequestHeader struct { EndKey Key `protobuf:"bytes,4,opt,name=end_key,casttype=Key" json:"end_key,omitempty"` // Replica specifies the destination for the request. This is a specific // instance of the available replicas belonging to RangeID. - Replica Replica `protobuf:"bytes,5,opt,name=replica" json:"replica"` + Replica ReplicaDescriptor `protobuf:"bytes,5,opt,name=replica" json:"replica"` // RangeID specifies the ID of the Raft consensus group which the key // range belongs to. This is used by the receiving node to route the // request to the correct range. @@ -331,11 +331,11 @@ func (m *RequestHeader) GetEndKey() Key { return nil } -func (m *RequestHeader) GetReplica() Replica { +func (m *RequestHeader) GetReplica() ReplicaDescriptor { if m != nil { return m.Replica } - return Replica{} + return ReplicaDescriptor{} } func (m *RequestHeader) GetRangeID() RangeID { diff --git a/proto/api.proto b/proto/api.proto index abcdd68d6ef9..3613202af55c 100644 --- a/proto/api.proto +++ b/proto/api.proto @@ -87,7 +87,7 @@ message RequestHeader { optional bytes end_key = 4 [(gogoproto.casttype) = "Key"]; // Replica specifies the destination for the request. This is a specific // instance of the available replicas belonging to RangeID. - optional Replica replica = 5 [(gogoproto.nullable) = false]; + optional ReplicaDescriptor replica = 5 [(gogoproto.nullable) = false]; // RangeID specifies the ID of the Raft consensus group which the key // range belongs to. This is used by the receiving node to route the // request to the correct range. diff --git a/proto/data.go b/proto/data.go index 85ca9770770b..1f4bcc9c0786 100644 --- a/proto/data.go +++ b/proto/data.go @@ -640,10 +640,9 @@ func (s Int32Slice) Less(i, j int) bool { return s[i] < s[j] } var _ fmt.Stringer = &Lease{} func (l Lease) String() string { - nodeID, storeID := DecodeRaftNodeID(RaftNodeID(l.RaftNodeID)) t := time.Unix(l.Start.WallTime/1E9, 0) tStr := t.Format("15:04:05.000") - return fmt.Sprintf("replica %d:%d %s +%.3fs", nodeID, storeID, tStr, float64(l.Expiration.WallTime-l.Start.WallTime)/1E9) + return fmt.Sprintf("replica %s %s +%.3fs", l.Replica, tStr, float64(l.Expiration.WallTime-l.Start.WallTime)/1E9) } // Covers returns true if the given timestamp is strictly less than the @@ -653,7 +652,7 @@ func (l Lease) Covers(timestamp Timestamp) bool { return timestamp.Less(l.Expiration) } -// OwnedBy returns whether the lease owner is equal to the given RaftNodeID. -func (l Lease) OwnedBy(id RaftNodeID) bool { - return l.RaftNodeID == id +// OwnedBy returns whether the given store is the lease owner. +func (l Lease) OwnedBy(storeID StoreID) bool { + return l.Replica.StoreID == storeID } diff --git a/proto/data.pb.go b/proto/data.pb.go index 9ce07a44362a..e29c21c3a38c 100644 --- a/proto/data.pb.go +++ b/proto/data.pb.go @@ -411,10 +411,10 @@ type ChangeReplicasTrigger struct { StoreID StoreID `protobuf:"varint,2,opt,name=store_id,casttype=StoreID" json:"store_id"` ChangeType ReplicaChangeType `protobuf:"varint,3,opt,name=change_type,enum=cockroach.proto.ReplicaChangeType" json:"change_type"` // The replica being modified. - Replica Replica `protobuf:"bytes,4,opt,name=replica" json:"replica"` + Replica ReplicaDescriptor `protobuf:"bytes,4,opt,name=replica" json:"replica"` // The new replica list with this change applied. - UpdatedReplicas []Replica `protobuf:"bytes,5,rep,name=updated_replicas" json:"updated_replicas"` - NextReplicaID ReplicaID `protobuf:"varint,6,opt,name=next_replica_id,casttype=ReplicaID" json:"next_replica_id"` + UpdatedReplicas []ReplicaDescriptor `protobuf:"bytes,5,rep,name=updated_replicas" json:"updated_replicas"` + NextReplicaID ReplicaID `protobuf:"varint,6,opt,name=next_replica_id,casttype=ReplicaID" json:"next_replica_id"` } func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } @@ -442,14 +442,14 @@ func (m *ChangeReplicasTrigger) GetChangeType() ReplicaChangeType { return ADD_REPLICA } -func (m *ChangeReplicasTrigger) GetReplica() Replica { +func (m *ChangeReplicasTrigger) GetReplica() ReplicaDescriptor { if m != nil { return m.Replica } - return Replica{} + return ReplicaDescriptor{} } -func (m *ChangeReplicasTrigger) GetUpdatedReplicas() []Replica { +func (m *ChangeReplicasTrigger) GetUpdatedReplicas() []ReplicaDescriptor { if m != nil { return m.UpdatedReplicas } @@ -704,8 +704,8 @@ type Lease struct { Start Timestamp `protobuf:"bytes,1,opt,name=start" json:"start"` // The expiration is a timestamp at which the lease will expire. Expiration Timestamp `protobuf:"bytes,2,opt,name=expiration" json:"expiration"` - // The Raft NodeID on which the would-be lease holder lives. - RaftNodeID RaftNodeID `protobuf:"varint,3,opt,name=raft_node_id,casttype=RaftNodeID" json:"raft_node_id"` + // The address of the would-be lease holder. + Replica ReplicaDescriptor `protobuf:"bytes,3,opt,name=replica" json:"replica"` } func (m *Lease) Reset() { *m = Lease{} } @@ -725,11 +725,11 @@ func (m *Lease) GetExpiration() Timestamp { return Timestamp{} } -func (m *Lease) GetRaftNodeID() RaftNodeID { +func (m *Lease) GetReplica() ReplicaDescriptor { if m != nil { - return m.RaftNodeID + return m.Replica } - return 0 + return ReplicaDescriptor{} } // Intent is used to communicate the location of an intent. @@ -1314,9 +1314,14 @@ func (m *Lease) MarshalTo(data []byte) (int, error) { return 0, err } i += n19 - data[i] = 0x18 + data[i] = 0x1a i++ - i = encodeVarintData(data, i, uint64(m.RaftNodeID)) + i = encodeVarintData(data, i, uint64(m.Replica.Size())) + n20, err := m.Replica.MarshalTo(data[i:]) + if err != nil { + return 0, err + } + i += n20 return i, nil } @@ -1350,11 +1355,11 @@ func (m *Intent) MarshalTo(data []byte) (int, error) { data[i] = 0x1a i++ i = encodeVarintData(data, i, uint64(m.Txn.Size())) - n20, err := m.Txn.MarshalTo(data[i:]) + n21, err := m.Txn.MarshalTo(data[i:]) if err != nil { return 0, err } - i += n20 + i += n21 return i, nil } @@ -1592,7 +1597,8 @@ func (m *Lease) Size() (n int) { n += 1 + l + sovData(uint64(l)) l = m.Expiration.Size() n += 1 + l + sovData(uint64(l)) - n += 1 + sovData(uint64(m.RaftNodeID)) + l = m.Replica.Size() + n += 1 + l + sovData(uint64(l)) return n } @@ -2549,7 +2555,7 @@ func (m *ChangeReplicasTrigger) Unmarshal(data []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.UpdatedReplicas = append(m.UpdatedReplicas, Replica{}) + m.UpdatedReplicas = append(m.UpdatedReplicas, ReplicaDescriptor{}) if err := m.UpdatedReplicas[len(m.UpdatedReplicas)-1].Unmarshal(data[iNdEx:postIndex]); err != nil { return err } @@ -3432,10 +3438,10 @@ func (m *Lease) Unmarshal(data []byte) error { } iNdEx = postIndex case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field RaftNodeID", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Replica", wireType) } - m.RaftNodeID = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowData @@ -3445,11 +3451,22 @@ func (m *Lease) Unmarshal(data []byte) error { } b := data[iNdEx] iNdEx++ - m.RaftNodeID |= (RaftNodeID(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthData + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Replica.Unmarshal(data[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipData(data[iNdEx:]) diff --git a/proto/data.proto b/proto/data.proto index 326f573fea64..1adf4776d30d 100644 --- a/proto/data.proto +++ b/proto/data.proto @@ -54,7 +54,7 @@ enum ValueType { FLOAT = 2; BYTES = 3; TIME = 4; - + // TIMESERIES is applied to values which contain InternalTimeSeriesData. TIMESERIES = 100; } @@ -135,9 +135,9 @@ message ChangeReplicasTrigger { (gogoproto.customname) = "StoreID", (gogoproto.casttype) = "StoreID"]; optional ReplicaChangeType change_type = 3 [(gogoproto.nullable) = false]; // The replica being modified. - optional Replica replica = 4 [(gogoproto.nullable) = false]; + optional ReplicaDescriptor replica = 4 [(gogoproto.nullable) = false]; // The new replica list with this change applied. - repeated Replica updated_replicas = 5 [(gogoproto.nullable) = false]; + repeated ReplicaDescriptor updated_replicas = 5 [(gogoproto.nullable) = false]; optional int32 next_replica_id = 6 [(gogoproto.nullable) = false, (gogoproto.customname) = "NextReplicaID", (gogoproto.casttype) = "ReplicaID"]; } @@ -268,9 +268,8 @@ message Lease { optional Timestamp start = 1 [(gogoproto.nullable) = false]; // The expiration is a timestamp at which the lease will expire. optional Timestamp expiration = 2 [(gogoproto.nullable) = false]; - // The Raft NodeID on which the would-be lease holder lives. - optional uint64 raft_node_id = 3 [(gogoproto.nullable) = false, - (gogoproto.customname) = "RaftNodeID", (gogoproto.casttype) = "RaftNodeID"]; + // The address of the would-be lease holder. + optional ReplicaDescriptor replica = 3 [(gogoproto.nullable) = false]; } // Intent is used to communicate the location of an intent. diff --git a/proto/errors.pb.go b/proto/errors.pb.go index f80d0ce41c45..127dda7a75a7 100644 --- a/proto/errors.pb.go +++ b/proto/errors.pb.go @@ -64,22 +64,22 @@ func (x *TransactionRestart) UnmarshalJSON(data []byte) error { // A NotLeaderError indicates that the current range is not the // leader. If the leader is known, its Replica is set in the error. type NotLeaderError struct { - Replica *Replica `protobuf:"bytes,1,opt,name=replica" json:"replica,omitempty"` - Leader *Replica `protobuf:"bytes,2,opt,name=leader" json:"leader,omitempty"` - RangeID RangeID `protobuf:"varint,3,opt,name=range_id,casttype=RangeID" json:"range_id"` + Replica *ReplicaDescriptor `protobuf:"bytes,1,opt,name=replica" json:"replica,omitempty"` + Leader *ReplicaDescriptor `protobuf:"bytes,2,opt,name=leader" json:"leader,omitempty"` + RangeID RangeID `protobuf:"varint,3,opt,name=range_id,casttype=RangeID" json:"range_id"` } func (m *NotLeaderError) Reset() { *m = NotLeaderError{} } func (*NotLeaderError) ProtoMessage() {} -func (m *NotLeaderError) GetReplica() *Replica { +func (m *NotLeaderError) GetReplica() *ReplicaDescriptor { if m != nil { return m.Replica } return nil } -func (m *NotLeaderError) GetLeader() *Replica { +func (m *NotLeaderError) GetLeader() *ReplicaDescriptor { if m != nil { return m.Leader } @@ -1772,7 +1772,7 @@ func (m *NotLeaderError) Unmarshal(data []byte) error { return io.ErrUnexpectedEOF } if m.Replica == nil { - m.Replica = &Replica{} + m.Replica = &ReplicaDescriptor{} } if err := m.Replica.Unmarshal(data[iNdEx:postIndex]); err != nil { return err @@ -1805,7 +1805,7 @@ func (m *NotLeaderError) Unmarshal(data []byte) error { return io.ErrUnexpectedEOF } if m.Leader == nil { - m.Leader = &Replica{} + m.Leader = &ReplicaDescriptor{} } if err := m.Leader.Unmarshal(data[iNdEx:postIndex]); err != nil { return err diff --git a/proto/errors.proto b/proto/errors.proto index 3bd0c6ce53f0..b19d6336db4e 100644 --- a/proto/errors.proto +++ b/proto/errors.proto @@ -33,8 +33,8 @@ option (gogoproto.goproto_unrecognized_all) = false; // A NotLeaderError indicates that the current range is not the // leader. If the leader is known, its Replica is set in the error. message NotLeaderError { - optional Replica replica = 1; - optional Replica leader = 2; + optional ReplicaDescriptor replica = 1; + optional ReplicaDescriptor leader = 2; optional int64 range_id = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "RangeID", (gogoproto.casttype) = "RangeID"]; } diff --git a/proto/internal.pb.go b/proto/internal.pb.go index e69888c482eb..e96445e82bb9 100644 --- a/proto/internal.pb.go +++ b/proto/internal.pb.go @@ -20,9 +20,9 @@ var _ = math.Inf // A RaftCommand is a command which can be serialized and sent via // raft. type RaftCommand struct { - RangeID RangeID `protobuf:"varint,1,opt,name=range_id,casttype=RangeID" json:"range_id"` - OriginNodeID RaftNodeID `protobuf:"varint,2,opt,name=origin_node_id,casttype=RaftNodeID" json:"origin_node_id"` - Cmd BatchRequest `protobuf:"bytes,3,opt,name=cmd" json:"cmd"` + RangeID RangeID `protobuf:"varint,1,opt,name=range_id,casttype=RangeID" json:"range_id"` + OriginReplica ReplicaDescriptor `protobuf:"bytes,2,opt,name=origin_replica" json:"origin_replica"` + Cmd BatchRequest `protobuf:"bytes,3,opt,name=cmd" json:"cmd"` } func (m *RaftCommand) Reset() { *m = RaftCommand{} } @@ -36,11 +36,11 @@ func (m *RaftCommand) GetRangeID() RangeID { return 0 } -func (m *RaftCommand) GetOriginNodeID() RaftNodeID { +func (m *RaftCommand) GetOriginReplica() ReplicaDescriptor { if m != nil { - return m.OriginNodeID + return m.OriginReplica } - return 0 + return ReplicaDescriptor{} } func (m *RaftCommand) GetCmd() BatchRequest { @@ -273,17 +273,22 @@ func (m *RaftCommand) MarshalTo(data []byte) (int, error) { data[i] = 0x8 i++ i = encodeVarintInternal(data, i, uint64(m.RangeID)) - data[i] = 0x10 + data[i] = 0x12 i++ - i = encodeVarintInternal(data, i, uint64(m.OriginNodeID)) + i = encodeVarintInternal(data, i, uint64(m.OriginReplica.Size())) + n1, err := m.OriginReplica.MarshalTo(data[i:]) + if err != nil { + return 0, err + } + i += n1 data[i] = 0x1a i++ i = encodeVarintInternal(data, i, uint64(m.Cmd.Size())) - n1, err := m.Cmd.MarshalTo(data[i:]) + n2, err := m.Cmd.MarshalTo(data[i:]) if err != nil { return 0, err } - i += n1 + i += n2 return i, nil } @@ -402,11 +407,11 @@ func (m *RaftSnapshotData) MarshalTo(data []byte) (int, error) { data[i] = 0xa i++ i = encodeVarintInternal(data, i, uint64(m.RangeDescriptor.Size())) - n2, err := m.RangeDescriptor.MarshalTo(data[i:]) + n3, err := m.RangeDescriptor.MarshalTo(data[i:]) if err != nil { return 0, err } - i += n2 + i += n3 if len(m.KV) > 0 { for _, msg := range m.KV { data[i] = 0x12 @@ -483,7 +488,8 @@ func (m *RaftCommand) Size() (n int) { var l int _ = l n += 1 + sovInternal(uint64(m.RangeID)) - n += 1 + sovInternal(uint64(m.OriginNodeID)) + l = m.OriginReplica.Size() + n += 1 + l + sovInternal(uint64(l)) l = m.Cmd.Size() n += 1 + l + sovInternal(uint64(l)) return n @@ -616,10 +622,10 @@ func (m *RaftCommand) Unmarshal(data []byte) error { } } case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field OriginNodeID", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OriginReplica", wireType) } - m.OriginNodeID = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowInternal @@ -629,11 +635,22 @@ func (m *RaftCommand) Unmarshal(data []byte) error { } b := data[iNdEx] iNdEx++ - m.OriginNodeID |= (RaftNodeID(b) & 0x7F) << shift + msglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + if msglen < 0 { + return ErrInvalidLengthInternal + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.OriginReplica.Unmarshal(data[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 3: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Cmd", wireType) diff --git a/proto/internal.proto b/proto/internal.proto index d9c04df0ba67..fd5543887e8d 100644 --- a/proto/internal.proto +++ b/proto/internal.proto @@ -33,8 +33,7 @@ option (gogoproto.goproto_unrecognized_all) = false; message RaftCommand { optional int64 range_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "RangeID", (gogoproto.casttype) = "RangeID"]; - optional uint64 origin_node_id = 2 [(gogoproto.nullable) = false, - (gogoproto.customname) = "OriginNodeID", (gogoproto.casttype) = "RaftNodeID"]; + optional ReplicaDescriptor origin_replica = 2 [(gogoproto.nullable) = false]; optional BatchRequest cmd = 3 [(gogoproto.nullable) = false]; } diff --git a/proto/metadata.go b/proto/metadata.go index 13ca18b415ae..b915365425a7 100644 --- a/proto/metadata.go +++ b/proto/metadata.go @@ -20,7 +20,6 @@ package proto import ( "bytes" - "fmt" "sort" "strconv" "strings" @@ -63,31 +62,9 @@ func (n StoreID) String() string { // ReplicaID is a custom type for a range replica ID. type ReplicaID int32 -// RaftNodeID is a custom type for a Raft node ID. A raft node ID -// is composed of a concatenation of NodeID + StoreID. -type RaftNodeID uint64 - -// Format implements the fmt.Formatter interface. -func (n RaftNodeID) Format(f fmt.State, verb rune) { - // Note: this implementation doesn't handle the width and precision - // specifiers such as "%20.10s". - fmt.Fprint(f, strconv.FormatInt(int64(n), 16)) -} - -// MakeRaftNodeID packs a NodeID and StoreID into a single uint64 for use in raft. -func MakeRaftNodeID(n NodeID, s StoreID) RaftNodeID { - if n < 0 || s <= 0 { - // Zeroes are likely the result of incomplete initialization. - // TODO(bdarnell): should we disallow NodeID==0? It should never occur in - // production but many tests use it. - panic("NodeID must be >= 0 and StoreID must be > 0") - } - return RaftNodeID(n)<<32 | RaftNodeID(s) -} - -// DecodeRaftNodeID converts a RaftNodeID into its component NodeID and StoreID. -func DecodeRaftNodeID(n RaftNodeID) (NodeID, StoreID) { - return NodeID(n >> 32), StoreID(n & 0xffffffff) +// String implements the fmt.Stringer interface. +func (r ReplicaID) String() string { + return strconv.FormatInt(int64(r), 10) } // IsSubset returns whether attributes list a is a subset of @@ -151,7 +128,7 @@ func (r *RangeDescriptor) ContainsKeyRange(start, end []byte) bool { // FindReplica returns the replica which matches the specified store // ID. If no replica matches, (-1, nil) is returned. -func (r *RangeDescriptor) FindReplica(storeID StoreID) (int, *Replica) { +func (r *RangeDescriptor) FindReplica(storeID StoreID) (int, *ReplicaDescriptor) { for i := range r.Replicas { if r.Replicas[i].StoreID == storeID { return i, &r.Replicas[i] diff --git a/proto/metadata.pb.go b/proto/metadata.pb.go index 0c27acd68e6f..fe6ca1495df5 100644 --- a/proto/metadata.pb.go +++ b/proto/metadata.pb.go @@ -34,10 +34,10 @@ func (m *Attributes) GetAttrs() []string { return nil } -// Replica describes a replica location by node ID (corresponds to a -// host:port via lookup on gossip network) and store ID (identifies the -// device). -type Replica struct { +// ReplicaDescriptor describes a replica location by node ID +// (corresponds to a host:port via lookup on gossip network) and store +// ID (identifies the device). +type ReplicaDescriptor struct { NodeID NodeID `protobuf:"varint,1,opt,name=node_id,casttype=NodeID" json:"node_id"` StoreID StoreID `protobuf:"varint,2,opt,name=store_id,casttype=StoreID" json:"store_id"` // ReplicaID uniquely identifies a replica instance. If a range is removed from @@ -46,25 +46,25 @@ type Replica struct { ReplicaID ReplicaID `protobuf:"varint,3,opt,name=replica_id,casttype=ReplicaID" json:"replica_id"` } -func (m *Replica) Reset() { *m = Replica{} } -func (m *Replica) String() string { return proto1.CompactTextString(m) } -func (*Replica) ProtoMessage() {} +func (m *ReplicaDescriptor) Reset() { *m = ReplicaDescriptor{} } +func (m *ReplicaDescriptor) String() string { return proto1.CompactTextString(m) } +func (*ReplicaDescriptor) ProtoMessage() {} -func (m *Replica) GetNodeID() NodeID { +func (m *ReplicaDescriptor) GetNodeID() NodeID { if m != nil { return m.NodeID } return 0 } -func (m *Replica) GetStoreID() StoreID { +func (m *ReplicaDescriptor) GetStoreID() StoreID { if m != nil { return m.StoreID } return 0 } -func (m *Replica) GetReplicaID() ReplicaID { +func (m *ReplicaDescriptor) GetReplicaID() ReplicaID { if m != nil { return m.ReplicaID } @@ -85,7 +85,7 @@ type RangeDescriptor struct { // Replicas is the set of nodes/stores on which replicas of this // range are stored, the ordering being arbitrary and subject to // permutation. - Replicas []Replica `protobuf:"bytes,4,rep,name=replicas" json:"replicas"` + Replicas []ReplicaDescriptor `protobuf:"bytes,4,rep,name=replicas" json:"replicas"` // NextReplicaID is a counter used to generate replica IDs. NextReplicaID ReplicaID `protobuf:"varint,5,opt,name=next_replica_id,casttype=ReplicaID" json:"next_replica_id"` } @@ -115,7 +115,7 @@ func (m *RangeDescriptor) GetEndKey() Key { return nil } -func (m *RangeDescriptor) GetReplicas() []Replica { +func (m *RangeDescriptor) GetReplicas() []ReplicaDescriptor { if m != nil { return m.Replicas } @@ -333,7 +333,7 @@ func (m *Attributes) MarshalTo(data []byte) (int, error) { return i, nil } -func (m *Replica) Marshal() (data []byte, err error) { +func (m *ReplicaDescriptor) Marshal() (data []byte, err error) { size := m.Size() data = make([]byte, size) n, err := m.MarshalTo(data) @@ -343,7 +343,7 @@ func (m *Replica) Marshal() (data []byte, err error) { return data[:n], nil } -func (m *Replica) MarshalTo(data []byte) (int, error) { +func (m *ReplicaDescriptor) MarshalTo(data []byte) (int, error) { var i int _ = i var l int @@ -630,7 +630,7 @@ func (m *Attributes) Size() (n int) { return n } -func (m *Replica) Size() (n int) { +func (m *ReplicaDescriptor) Size() (n int) { var l int _ = l n += 1 + sovMetadata(uint64(m.NodeID)) @@ -819,7 +819,7 @@ func (m *Attributes) Unmarshal(data []byte) error { } return nil } -func (m *Replica) Unmarshal(data []byte) error { +func (m *ReplicaDescriptor) Unmarshal(data []byte) error { l := len(data) iNdEx := 0 for iNdEx < l { @@ -842,10 +842,10 @@ func (m *Replica) Unmarshal(data []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Replica: wiretype end group for non-group") + return fmt.Errorf("proto: ReplicaDescriptor: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Replica: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReplicaDescriptor: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -1056,7 +1056,7 @@ func (m *RangeDescriptor) Unmarshal(data []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Replicas = append(m.Replicas, Replica{}) + m.Replicas = append(m.Replicas, ReplicaDescriptor{}) if err := m.Replicas[len(m.Replicas)-1].Unmarshal(data[iNdEx:postIndex]); err != nil { return err } diff --git a/proto/metadata.proto b/proto/metadata.proto index bd82637a0797..005b9d79f661 100644 --- a/proto/metadata.proto +++ b/proto/metadata.proto @@ -36,10 +36,10 @@ message Attributes { repeated string attrs = 1 [(gogoproto.moretags) = "yaml:\"attrs,flow\""]; } -// Replica describes a replica location by node ID (corresponds to a -// host:port via lookup on gossip network) and store ID (identifies the -// device). -message Replica { +// ReplicaDescriptor describes a replica location by node ID +// (corresponds to a host:port via lookup on gossip network) and store +// ID (identifies the device). +message ReplicaDescriptor { optional int32 node_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "NodeID", (gogoproto.casttype) = "NodeID"]; optional int32 store_id = 2 [(gogoproto.nullable) = false, @@ -67,7 +67,7 @@ message RangeDescriptor { // Replicas is the set of nodes/stores on which replicas of this // range are stored, the ordering being arbitrary and subject to // permutation. - repeated Replica replicas = 4 [(gogoproto.nullable) = false]; + repeated ReplicaDescriptor replicas = 4 [(gogoproto.nullable) = false]; // NextReplicaID is a counter used to generate replica IDs. optional int32 next_replica_id = 5 [(gogoproto.nullable) = false, diff --git a/proto/metadata_test.go b/proto/metadata_test.go index 2cd810726420..b30acab9c6d3 100644 --- a/proto/metadata_test.go +++ b/proto/metadata_test.go @@ -64,7 +64,7 @@ func TestAttributesSortedString(t *testing.T) { func TestRangeDescriptorFindReplica(t *testing.T) { desc := RangeDescriptor{ - Replicas: []Replica{ + Replicas: []ReplicaDescriptor{ {NodeID: 1, StoreID: 1}, {NodeID: 2, StoreID: 2}, {NodeID: 3, StoreID: 3}, diff --git a/server/raft_transport.go b/server/raft_transport.go index 282b277b2ffd..9dd400b37d2a 100644 --- a/server/raft_transport.go +++ b/server/raft_transport.go @@ -50,8 +50,8 @@ type rpcTransport struct { rpcServer *rpc.Server rpcContext *rpc.Context mu sync.Mutex - servers map[proto.RaftNodeID]multiraft.ServerInterface - queues map[proto.RaftNodeID]chan *multiraft.RaftMessageRequest + servers map[proto.StoreID]multiraft.ServerInterface + queues map[proto.StoreID]chan *multiraft.RaftMessageRequest } // newRPCTransport creates a new rpcTransport with specified gossip and rpc server. @@ -61,8 +61,8 @@ func newRPCTransport(gossip *gossip.Gossip, rpcServer *rpc.Server, rpcContext *r gossip: gossip, rpcServer: rpcServer, rpcContext: rpcContext, - servers: make(map[proto.RaftNodeID]multiraft.ServerInterface), - queues: make(map[proto.RaftNodeID]chan *multiraft.RaftMessageRequest), + servers: make(map[proto.StoreID]multiraft.ServerInterface), + queues: make(map[proto.StoreID]chan *multiraft.RaftMessageRequest), } if t.rpcServer != nil { @@ -80,7 +80,7 @@ func (t *rpcTransport) RaftMessage(args gogoproto.Message, callback func(gogopro req := args.(*multiraft.RaftMessageRequest) t.mu.Lock() - server, ok := t.servers[proto.RaftNodeID(req.Message.To)] + server, ok := t.servers[proto.StoreID(req.Message.To)] t.mu.Unlock() if !ok { @@ -101,7 +101,7 @@ func (t *rpcTransport) RaftMessage(args gogoproto.Message, callback func(gogopro // Listen implements the multiraft.Transport interface by registering a ServerInterface // to receive proxied messages. -func (t *rpcTransport) Listen(id proto.RaftNodeID, server multiraft.ServerInterface) error { +func (t *rpcTransport) Listen(id proto.StoreID, server multiraft.ServerInterface) error { t.mu.Lock() t.servers[id] = server t.mu.Unlock() @@ -109,7 +109,7 @@ func (t *rpcTransport) Listen(id proto.RaftNodeID, server multiraft.ServerInterf } // Stop implements the multiraft.Transport interface by unregistering the server id. -func (t *rpcTransport) Stop(id proto.RaftNodeID) { +func (t *rpcTransport) Stop(id proto.StoreID) { t.mu.Lock() defer t.mu.Unlock() delete(t.servers, id) @@ -122,9 +122,9 @@ func (t *rpcTransport) Stop(id proto.RaftNodeID) { // TODO(tschottdorf) should let MultiRaft know if the node is down; // need a feedback mechanism for that. Potentially easiest is to arrange for // the next call to Send() to fail appropriately. -func (t *rpcTransport) processQueue(raftNodeID proto.RaftNodeID) { +func (t *rpcTransport) processQueue(nodeID proto.NodeID, storeID proto.StoreID) { t.mu.Lock() - ch, ok := t.queues[raftNodeID] + ch, ok := t.queues[storeID] t.mu.Unlock() if !ok { return @@ -132,11 +132,10 @@ func (t *rpcTransport) processQueue(raftNodeID proto.RaftNodeID) { // Clean-up when the loop below shuts down. defer func() { t.mu.Lock() - delete(t.queues, raftNodeID) + delete(t.queues, storeID) t.mu.Unlock() }() - nodeID, _ := proto.DecodeRaftNodeID(raftNodeID) addr, err := t.gossip.GetNodeIDAddress(nodeID) if err != nil { log.Errorf("could not get address for node %d: %s", nodeID, err) @@ -182,22 +181,18 @@ func (t *rpcTransport) processQueue(raftNodeID proto.RaftNodeID) { return } - client.Go(raftMessageName, &multiraft.RaftMessageRequest{ - GroupID: req.GroupID, - Message: req.Message, - }, protoResp, done) + client.Go(raftMessageName, req, protoResp, done) } } // Send a message to the recipient specified in the request. func (t *rpcTransport) Send(req *multiraft.RaftMessageRequest) error { - raftNodeID := proto.RaftNodeID(req.Message.To) t.mu.Lock() - ch, ok := t.queues[raftNodeID] + ch, ok := t.queues[req.ToReplica.StoreID] if !ok { ch = make(chan *multiraft.RaftMessageRequest, raftSendBufferSize) - t.queues[raftNodeID] = ch - go t.processQueue(raftNodeID) + t.queues[req.ToReplica.StoreID] = ch + go t.processQueue(req.ToReplica.NodeID, req.ToReplica.StoreID) } t.mu.Unlock() diff --git a/server/raft_transport_test.go b/server/raft_transport_test.go index 59528f1c997d..be9b228a6e56 100644 --- a/server/raft_transport_test.go +++ b/server/raft_transport_test.go @@ -72,7 +72,7 @@ func TestSendAndReceive(t *testing.T) { servers := []*rpc.Server{} // All the rest have length numStores (note that several stores share a transport). nextNodeID := proto.NodeID(1) - nodeIDs := []proto.RaftNodeID{} + nodeIDs := []proto.NodeID{} transports := []multiraft.Transport{} channels := []channelServer{} for serverIndex := 0; serverIndex < numServers; serverIndex++ { @@ -89,17 +89,16 @@ func TestSendAndReceive(t *testing.T) { defer transport.Close() for store := 0; store < storesPerServer; store++ { - protoNodeID := nextNodeID - nodeID := proto.MakeRaftNodeID(protoNodeID, 1) + nodeID := nextNodeID nextNodeID++ channel := newChannelServer(10, 0) - if err := transport.Listen(nodeID, channel); err != nil { + if err := transport.Listen(proto.StoreID(nodeID), channel); err != nil { t.Fatal(err) } addr := server.Addr() - if err := g.AddInfoProto(gossip.MakeNodeIDKey(protoNodeID), + if err := g.AddInfoProto(gossip.MakeNodeIDKey(nodeID), &proto.NodeDescriptor{ Address: util.MakeUnresolvedAddr(addr.Network(), addr.String()), }, @@ -125,6 +124,16 @@ func TestSendAndReceive(t *testing.T) { To: uint64(nodeIDs[to]), Type: raftpb.MsgHeartbeat, }, + FromReplica: proto.ReplicaDescriptor{ + NodeID: nodeIDs[from], + StoreID: proto.StoreID(nodeIDs[from]), + ReplicaID: proto.ReplicaID(nodeIDs[from]), + }, + ToReplica: proto.ReplicaDescriptor{ + NodeID: nodeIDs[to], + StoreID: proto.StoreID(nodeIDs[to]), + ReplicaID: proto.ReplicaID(nodeIDs[to]), + }, } if err := transports[from].Send(req); err != nil { @@ -174,19 +183,18 @@ func TestInOrderDelivery(t *testing.T) { defer server.Close() const numMessages = 100 - protoNodeID := proto.NodeID(1) - raftNodeID := proto.MakeRaftNodeID(protoNodeID, 1) + nodeID := proto.NodeID(1) serverTransport, err := newRPCTransport(g, server, nodeRPCContext) if err != nil { t.Fatal(err) } defer serverTransport.Close() serverChannel := newChannelServer(numMessages, 10*time.Millisecond) - if err := serverTransport.Listen(raftNodeID, serverChannel); err != nil { + if err := serverTransport.Listen(proto.StoreID(nodeID), serverChannel); err != nil { t.Fatal(err) } addr := server.Addr() - if err := g.AddInfoProto(gossip.MakeNodeIDKey(protoNodeID), + if err := g.AddInfoProto(gossip.MakeNodeIDKey(nodeID), &proto.NodeDescriptor{ Address: util.MakeUnresolvedAddr(addr.Network(), addr.String()), }, @@ -194,7 +202,7 @@ func TestInOrderDelivery(t *testing.T) { t.Fatal(err) } - clientNodeID := proto.MakeRaftNodeID(2, 2) + clientNodeID := proto.NodeID(2) clientTransport, err := newRPCTransport(g, nil, nodeRPCContext) if err != nil { t.Fatal(err) @@ -205,10 +213,20 @@ func TestInOrderDelivery(t *testing.T) { req := &multiraft.RaftMessageRequest{ GroupID: 1, Message: raftpb.Message{ - To: uint64(raftNodeID), + To: uint64(nodeID), From: uint64(clientNodeID), Commit: uint64(i), }, + ToReplica: proto.ReplicaDescriptor{ + NodeID: nodeID, + StoreID: proto.StoreID(nodeID), + ReplicaID: proto.ReplicaID(nodeID), + }, + FromReplica: proto.ReplicaDescriptor{ + NodeID: clientNodeID, + StoreID: proto.StoreID(clientNodeID), + ReplicaID: proto.ReplicaID(clientNodeID), + }, } if err := clientTransport.Send(req); err != nil { t.Errorf("failed to send message %d: %s", i, err) diff --git a/storage/allocator.go b/storage/allocator.go index 2d966ec49e35..37d959e7e14c 100644 --- a/storage/allocator.go +++ b/storage/allocator.go @@ -117,7 +117,7 @@ func MakeAllocator(storePool *StorePool, options RebalancingOptions) Allocator { // getUsedNodes returns a set of node IDs which are already being used // to store replicas. -func getUsedNodes(existing []proto.Replica) map[proto.NodeID]struct{} { +func getUsedNodes(existing []proto.ReplicaDescriptor) map[proto.NodeID]struct{} { usedNodes := map[proto.NodeID]struct{}{} for _, replica := range existing { usedNodes[replica.NodeID] = struct{}{} @@ -169,7 +169,7 @@ func (a *Allocator) ComputeAction(zone config.ZoneConfig, desc *proto.RangeDescr // filter the results. The function will be passed the storeDesc and the used // and new counts. It returns a bool indicating inclusion or exclusion from the // set of stores being considered. -func (a *Allocator) AllocateTarget(required proto.Attributes, existing []proto.Replica, relaxConstraints bool, +func (a *Allocator) AllocateTarget(required proto.Attributes, existing []proto.ReplicaDescriptor, relaxConstraints bool, filter func(storeDesc *proto.StoreDescriptor, count, used *stat) bool) (*proto.StoreDescriptor, error) { // Because more redundancy is better than less, if relaxConstraints, the // matching here is lenient, and tries to find a target by relaxing an @@ -218,14 +218,14 @@ func (a *Allocator) AllocateTarget(required proto.Attributes, existing []proto.R // the zone config associated with the provided replicas. This will allow it to // make correct decisions in the case of ranges with heterogeneous replica // requirements (i.e. multiple data centers). -func (a Allocator) RemoveTarget(existing []proto.Replica) (proto.Replica, error) { +func (a Allocator) RemoveTarget(existing []proto.ReplicaDescriptor) (proto.ReplicaDescriptor, error) { if len(existing) == 0 { - return proto.Replica{}, util.Errorf("must supply at least one replica to allocator.RemoveTarget()") + return proto.ReplicaDescriptor{}, util.Errorf("must supply at least one replica to allocator.RemoveTarget()") } // Retrieve store descriptors for the provided replicas from the StorePool. type replStore struct { - repl proto.Replica + repl proto.ReplicaDescriptor store *proto.StoreDescriptor } replStores := make([]replStore, len(existing)) @@ -275,7 +275,7 @@ func (a Allocator) RemoveTarget(existing []proto.Replica) (proto.Replica, error) // is perfectly fine, as other stores in the cluster will also be // doing their probabilistic best to rebalance. This helps prevent // a stampeding herd targeting an abnormally under-utilized store. -func (a Allocator) RebalanceTarget(required proto.Attributes, existing []proto.Replica) *proto.StoreDescriptor { +func (a Allocator) RebalanceTarget(required proto.Attributes, existing []proto.ReplicaDescriptor) *proto.StoreDescriptor { filter := func(s *proto.StoreDescriptor, count, used *stat) bool { // In clusters with very low disk usage, a store is eligible to be a // rebalancing target if the number of ranges on that store is below @@ -364,7 +364,7 @@ func (a Allocator) ShouldRebalance(storeID proto.StoreID) bool { // replicas. If the supplied filter is nil, it is ignored. Returns the // list of matching descriptors, and the store list matching the // required attributes. -func (a Allocator) selectRandom(count int, required proto.Attributes, existing []proto.Replica) ([]*proto.StoreDescriptor, *StoreList) { +func (a Allocator) selectRandom(count int, required proto.Attributes, existing []proto.ReplicaDescriptor) ([]*proto.StoreDescriptor, *StoreList) { var descs []*proto.StoreDescriptor sl := a.storePool.getStoreList(required, a.options.Deterministic) used := getUsedNodes(existing) diff --git a/storage/allocator_test.go b/storage/allocator_test.go index 3da705d9be65..4f73488ac305 100644 --- a/storage/allocator_test.go +++ b/storage/allocator_test.go @@ -199,7 +199,7 @@ func TestAllocatorSimpleRetrieval(t *testing.T) { stopper, g, _, a := createTestAllocator() defer stopper.Stop() gossiputil.NewStoreGossiper(g).GossipStores(singleStore, t) - result, err := a.AllocateTarget(simpleZoneConfig.ReplicaAttrs[0], []proto.Replica{}, false, nil) + result, err := a.AllocateTarget(simpleZoneConfig.ReplicaAttrs[0], []proto.ReplicaDescriptor{}, false, nil) if err != nil { t.Errorf("Unable to perform allocation: %v", err) } @@ -212,7 +212,7 @@ func TestAllocatorNoAvailableDisks(t *testing.T) { defer leaktest.AfterTest(t) stopper, _, _, a := createTestAllocator() defer stopper.Stop() - result, err := a.AllocateTarget(simpleZoneConfig.ReplicaAttrs[0], []proto.Replica{}, false, nil) + result, err := a.AllocateTarget(simpleZoneConfig.ReplicaAttrs[0], []proto.ReplicaDescriptor{}, false, nil) if result != nil { t.Errorf("expected nil result: %+v", result) } @@ -226,14 +226,14 @@ func TestAllocatorThreeDisksSameDC(t *testing.T) { stopper, g, _, a := createTestAllocator() defer stopper.Stop() gossiputil.NewStoreGossiper(g).GossipStores(sameDCStores, t) - result1, err := a.AllocateTarget(multiDisksConfig.ReplicaAttrs[0], []proto.Replica{}, false, nil) + result1, err := a.AllocateTarget(multiDisksConfig.ReplicaAttrs[0], []proto.ReplicaDescriptor{}, false, nil) if err != nil { t.Fatalf("Unable to perform allocation: %v", err) } if result1.StoreID != 1 && result1.StoreID != 2 { t.Errorf("Expected store 1 or 2; got %+v", result1) } - exReplicas := []proto.Replica{ + exReplicas := []proto.ReplicaDescriptor{ { NodeID: result1.Node.NodeID, StoreID: result1.StoreID, @@ -249,7 +249,7 @@ func TestAllocatorThreeDisksSameDC(t *testing.T) { if result1.Node.NodeID == result2.Node.NodeID { t.Errorf("Expected node ids to be different %+v vs %+v", result1, result2) } - result3, err := a.AllocateTarget(multiDisksConfig.ReplicaAttrs[2], []proto.Replica{}, false, nil) + result3, err := a.AllocateTarget(multiDisksConfig.ReplicaAttrs[2], []proto.ReplicaDescriptor{}, false, nil) if err != nil { t.Errorf("Unable to perform allocation: %v", err) } @@ -263,11 +263,11 @@ func TestAllocatorTwoDatacenters(t *testing.T) { stopper, g, _, a := createTestAllocator() defer stopper.Stop() gossiputil.NewStoreGossiper(g).GossipStores(multiDCStores, t) - result1, err := a.AllocateTarget(multiDCConfig.ReplicaAttrs[0], []proto.Replica{}, false, nil) + result1, err := a.AllocateTarget(multiDCConfig.ReplicaAttrs[0], []proto.ReplicaDescriptor{}, false, nil) if err != nil { t.Fatalf("Unable to perform allocation: %v", err) } - result2, err := a.AllocateTarget(multiDCConfig.ReplicaAttrs[1], []proto.Replica{}, false, nil) + result2, err := a.AllocateTarget(multiDCConfig.ReplicaAttrs[1], []proto.ReplicaDescriptor{}, false, nil) if err != nil { t.Fatalf("Unable to perform allocation: %v", err) } @@ -275,7 +275,7 @@ func TestAllocatorTwoDatacenters(t *testing.T) { t.Errorf("Expected nodes 1 & 2: %+v vs %+v", result1.Node, result2.Node) } // Verify that no result is forthcoming if we already have a replica. - _, err = a.AllocateTarget(multiDCConfig.ReplicaAttrs[1], []proto.Replica{ + _, err = a.AllocateTarget(multiDCConfig.ReplicaAttrs[1], []proto.ReplicaDescriptor{ { NodeID: result2.Node.NodeID, StoreID: result2.StoreID, @@ -291,7 +291,7 @@ func TestAllocatorExistingReplica(t *testing.T) { stopper, g, _, a := createTestAllocator() defer stopper.Stop() gossiputil.NewStoreGossiper(g).GossipStores(sameDCStores, t) - result, err := a.AllocateTarget(multiDisksConfig.ReplicaAttrs[1], []proto.Replica{ + result, err := a.AllocateTarget(multiDisksConfig.ReplicaAttrs[1], []proto.ReplicaDescriptor{ { NodeID: 2, StoreID: 2, @@ -341,9 +341,9 @@ func TestAllocatorRelaxConstraints(t *testing.T) { {[]string{"b", "hdd", "gpu"}, []int{}, true, 2, false}, } for i, test := range testCases { - var existing []proto.Replica + var existing []proto.ReplicaDescriptor for _, id := range test.existing { - existing = append(existing, proto.Replica{NodeID: proto.NodeID(id), StoreID: proto.StoreID(id)}) + existing = append(existing, proto.ReplicaDescriptor{NodeID: proto.NodeID(id), StoreID: proto.StoreID(id)}) } result, err := a.AllocateTarget(proto.Attributes{Attrs: test.required}, existing, test.relaxConstraints, nil) if haveErr := (err != nil); haveErr != test.expErr { @@ -389,7 +389,7 @@ func TestAllocatorRandomAllocation(t *testing.T) { // store 1 or store 2 will be chosen, as the least loaded of the // three random choices is returned. for i := 0; i < 10; i++ { - result, err := a.AllocateTarget(proto.Attributes{}, []proto.Replica{}, false, nil) + result, err := a.AllocateTarget(proto.Attributes{}, []proto.ReplicaDescriptor{}, false, nil) if err != nil { t.Fatal(err) } @@ -432,7 +432,7 @@ func TestAllocatorRebalance(t *testing.T) { // Every rebalance target must be either stores 1 or 2. for i := 0; i < 10; i++ { - result := a.RebalanceTarget(proto.Attributes{}, []proto.Replica{}) + result := a.RebalanceTarget(proto.Attributes{}, []proto.ReplicaDescriptor{}) if result == nil { t.Fatal("nil result") } @@ -484,7 +484,7 @@ func TestAllocatorRebalanceByCapacity(t *testing.T) { // Every rebalance target must be store 4 (if not nil). for i := 0; i < 10; i++ { - result := a.RebalanceTarget(proto.Attributes{}, []proto.Replica{}) + result := a.RebalanceTarget(proto.Attributes{}, []proto.ReplicaDescriptor{}) if result != nil && result.StoreID != 4 { t.Errorf("expected store 4; got %d", result.StoreID) } @@ -535,7 +535,7 @@ func TestAllocatorRebalanceByCount(t *testing.T) { // Every rebalance target must be store 4 (or nil for case of missing the only option). for i := 0; i < 10; i++ { - result := a.RebalanceTarget(proto.Attributes{}, []proto.Replica{}) + result := a.RebalanceTarget(proto.Attributes{}, []proto.ReplicaDescriptor{}) if result != nil && result.StoreID != 4 { t.Errorf("expected store 4; got %d", result.StoreID) } @@ -559,7 +559,7 @@ func TestAllocatorRemoveTarget(t *testing.T) { defer stopper.Stop() // List of replicas that will be passed to RemoveTarget - replicas := []proto.Replica{ + replicas := []proto.ReplicaDescriptor{ { StoreID: 1, NodeID: 1, @@ -684,7 +684,7 @@ func TestAllocatorComputeAction(t *testing.T) { RangeMaxBytes: 64000, }, desc: proto.RangeDescriptor{ - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { StoreID: 1, NodeID: 1, @@ -722,7 +722,7 @@ func TestAllocatorComputeAction(t *testing.T) { RangeMaxBytes: 64000, }, desc: proto.RangeDescriptor{ - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { StoreID: 1, NodeID: 1, @@ -760,7 +760,7 @@ func TestAllocatorComputeAction(t *testing.T) { RangeMaxBytes: 64000, }, desc: proto.RangeDescriptor{ - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { StoreID: 1, NodeID: 1, @@ -808,7 +808,7 @@ func TestAllocatorComputeAction(t *testing.T) { RangeMaxBytes: 64000, }, desc: proto.RangeDescriptor{ - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { StoreID: 1, NodeID: 1, @@ -847,7 +847,7 @@ func TestAllocatorComputeAction(t *testing.T) { RangeMaxBytes: 64000, }, desc: proto.RangeDescriptor{ - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { StoreID: 1, NodeID: 1, @@ -890,7 +890,7 @@ func TestAllocatorComputeAction(t *testing.T) { RangeMaxBytes: 64000, }, desc: proto.RangeDescriptor{ - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { StoreID: 1, NodeID: 1, @@ -933,7 +933,7 @@ func TestAllocatorComputeAction(t *testing.T) { RangeMaxBytes: 64000, }, desc: proto.RangeDescriptor{ - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { StoreID: 1, NodeID: 1, @@ -981,7 +981,7 @@ func TestAllocatorComputeAction(t *testing.T) { RangeMaxBytes: 64000, }, desc: proto.RangeDescriptor{ - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { StoreID: 10, NodeID: 10, @@ -1019,7 +1019,7 @@ func TestAllocatorComputeAction(t *testing.T) { RangeMaxBytes: 64000, }, desc: proto.RangeDescriptor{ - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { StoreID: 1, NodeID: 1, @@ -1119,7 +1119,7 @@ func Example_rebalancing() { for j := 0; j < len(testStores); j++ { ts := &testStores[j] if alloc.ShouldRebalance(ts.StoreID) { - target := alloc.RebalanceTarget(proto.Attributes{}, []proto.Replica{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}) + target := alloc.RebalanceTarget(proto.Attributes{}, []proto.ReplicaDescriptor{{NodeID: ts.Node.NodeID, StoreID: ts.StoreID}}) if target != nil { testStores[j].rebalance(&testStores[int(target.StoreID)], alloc.randGen.Int63n(1<<20)) } diff --git a/storage/client_merge_test.go b/storage/client_merge_test.go index dc04b5348604..c4755d9a9d0b 100644 --- a/storage/client_merge_test.go +++ b/storage/client_merge_test.go @@ -40,7 +40,7 @@ func adminMergeArgs(key []byte, rangeID proto.RangeID, storeID proto.StoreID) pr RequestHeader: proto.RequestHeader{ Key: key, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, } } diff --git a/storage/client_raft_test.go b/storage/client_raft_test.go index eaffaeca9f25..149a4ac9a44c 100644 --- a/storage/client_raft_test.go +++ b/storage/client_raft_test.go @@ -225,7 +225,7 @@ func TestReplicateRange(t *testing.T) { } if err := rng.ChangeReplicas(proto.ADD_REPLICA, - proto.Replica{ + proto.ReplicaDescriptor{ NodeID: mtc.stores[1].Ident.NodeID, StoreID: mtc.stores[1].Ident.StoreID, }, rng.Desc()); err != nil { @@ -287,7 +287,7 @@ func TestRestoreReplicas(t *testing.T) { } if err := firstRng.ChangeReplicas(proto.ADD_REPLICA, - proto.Replica{ + proto.ReplicaDescriptor{ NodeID: mtc.stores[1].Ident.NodeID, StoreID: mtc.stores[1].Ident.StoreID, }, firstRng.Desc()); err != nil { @@ -385,7 +385,7 @@ func TestFailedReplicaChange(t *testing.T) { } err = rng.ChangeReplicas(proto.ADD_REPLICA, - proto.Replica{ + proto.ReplicaDescriptor{ NodeID: mtc.stores[1].Ident.NodeID, StoreID: mtc.stores[1].Ident.StoreID, }, rng.Desc()) @@ -408,7 +408,7 @@ func TestFailedReplicaChange(t *testing.T) { mtc.manualClock.Increment(10 * storage.DefaultHeartbeatInterval.Nanoseconds()) err = rng.ChangeReplicas(proto.ADD_REPLICA, - proto.Replica{ + proto.ReplicaDescriptor{ NodeID: mtc.stores[1].Ident.NodeID, StoreID: mtc.stores[1].Ident.StoreID, }, rng.Desc()) @@ -473,7 +473,7 @@ func TestReplicateAfterTruncation(t *testing.T) { // Now add the second replica. if err := rng.ChangeReplicas(proto.ADD_REPLICA, - proto.Replica{ + proto.ReplicaDescriptor{ NodeID: mtc.stores[1].Ident.NodeID, StoreID: mtc.stores[1].Ident.StoreID, }, rng.Desc()); err != nil { @@ -633,7 +633,7 @@ func TestStoreRangeDownReplicate(t *testing.T) { // storeIDset is used to compare the replica sets from different views (i.e. // local range descriptors) type storeIDset map[proto.StoreID]struct{} - makeStoreIDset := func(replicas []proto.Replica) storeIDset { + makeStoreIDset := func(replicas []proto.ReplicaDescriptor) storeIDset { idSet := make(storeIDset) for _, r := range replicas { idSet[r.StoreID] = struct{}{} @@ -735,7 +735,7 @@ func TestChangeReplicasDescriptorInvariant(t *testing.T) { addReplica := func(storeNum int, desc *proto.RangeDescriptor) error { return repl.ChangeReplicas(proto.ADD_REPLICA, - proto.Replica{ + proto.ReplicaDescriptor{ NodeID: mtc.stores[storeNum].Ident.NodeID, StoreID: mtc.stores[storeNum].Ident.StoreID, }, @@ -848,7 +848,7 @@ func TestReplicateAddAndRemove(t *testing.T) { } verify := func(expected []int64) { - util.SucceedsWithin(t, time.Second, func() error { + util.SucceedsWithin(t, 3*time.Second, func() error { values := []int64{} for _, eng := range mtc.engines { val, _, err := engine.MVCCGet(eng, proto.Key("a"), mtc.clock.Now(), true, nil) @@ -1092,11 +1092,23 @@ func TestRaftAfterRemoveRange(t *testing.T) { return util.Errorf("range still exists") }) + replica1 := proto.ReplicaDescriptor{ + ReplicaID: proto.ReplicaID(mtc.stores[1].StoreID()), + NodeID: proto.NodeID(mtc.stores[1].StoreID()), + StoreID: mtc.stores[1].StoreID(), + } + replica2 := proto.ReplicaDescriptor{ + ReplicaID: proto.ReplicaID(mtc.stores[2].StoreID()), + NodeID: proto.NodeID(mtc.stores[2].StoreID()), + StoreID: mtc.stores[2].StoreID(), + } if err := mtc.transport.Send(&multiraft.RaftMessageRequest{ - GroupID: proto.RangeID(0), + GroupID: 0, + ToReplica: replica1, + FromReplica: replica2, Message: raftpb.Message{ - From: uint64(mtc.stores[2].RaftNodeID()), - To: uint64(mtc.stores[1].RaftNodeID()), + From: uint64(replica2.ReplicaID), + To: uint64(replica1.ReplicaID), Type: raftpb.MsgHeartbeat, }}); err != nil { t.Fatal(err) @@ -1113,6 +1125,7 @@ func TestRaftAfterRemoveRange(t *testing.T) { // number of repetitions adds an unacceptable amount of test runtime). func TestRaftRemoveRace(t *testing.T) { defer leaktest.AfterTest(t) + t.Skip("TODO(bdarnell): #768") mtc := startMultiTestContext(t, 3) defer mtc.Stop() diff --git a/storage/client_range_gc_test.go b/storage/client_range_gc_test.go index 6b2dce7793ba..28ecacc3dc64 100644 --- a/storage/client_range_gc_test.go +++ b/storage/client_range_gc_test.go @@ -57,6 +57,7 @@ func TestRangeGCQueueDropReplica(t *testing.T) { // removes a range from a store that no longer should have a replica. func TestRangeGCQueueDropReplicaGCOnScan(t *testing.T) { defer leaktest.AfterTest(t) + t.Skip("TODO(bdarnell): #768") mtc := startMultiTestContext(t, 3) defer mtc.Stop() diff --git a/storage/client_range_test.go b/storage/client_range_test.go index 402ff03ac4a8..215763812313 100644 --- a/storage/client_range_test.go +++ b/storage/client_range_test.go @@ -292,7 +292,7 @@ func TestTxnPutOutOfOrder(t *testing.T) { requestHeader := proto.RequestHeader{ Key: proto.Key(key), RangeID: 1, - Replica: proto.Replica{StoreID: store.StoreID()}, + Replica: proto.ReplicaDescriptor{StoreID: store.StoreID()}, UserPriority: &priority, Timestamp: clock.Now(), } @@ -348,7 +348,7 @@ func TestRangeLookupUseReverse(t *testing.T) { Key: proto.KeyMin, EndKey: keys.RangeMetaKey(proto.KeyMax), RangeID: 1, - Replica: proto.Replica{StoreID: store.StoreID()}, + Replica: proto.ReplicaDescriptor{StoreID: store.StoreID()}, }, } util.SucceedsWithin(t, time.Second, func() error { @@ -361,7 +361,7 @@ func TestRangeLookupUseReverse(t *testing.T) { RequestHeader: proto.RequestHeader{ Key: key, RangeID: 1, - Replica: proto.Replica{StoreID: store.StoreID()}, + Replica: proto.ReplicaDescriptor{StoreID: store.StoreID()}, ReadConsistency: proto.INCONSISTENT, }, MaxRanges: maxResults, diff --git a/storage/client_split_test.go b/storage/client_split_test.go index f014249bed6f..7295e035bd55 100644 --- a/storage/client_split_test.go +++ b/storage/client_split_test.go @@ -46,7 +46,7 @@ func adminSplitArgs(key, splitKey []byte, rangeID proto.RangeID, storeID proto.S RequestHeader: proto.RequestHeader{ Key: key, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, SplitKey: splitKey, } diff --git a/storage/client_test.go b/storage/client_test.go index 091b4725146f..3d9c1dd7a0f9 100644 --- a/storage/client_test.go +++ b/storage/client_test.go @@ -386,7 +386,7 @@ func (m *multiTestContext) replicateRange(rangeID proto.RangeID, sourceStoreInde for _, dest := range dests { err = rng.ChangeReplicas(proto.ADD_REPLICA, - proto.Replica{ + proto.ReplicaDescriptor{ NodeID: m.stores[dest].Ident.NodeID, StoreID: m.stores[dest].Ident.StoreID, }, rng.Desc()) @@ -396,7 +396,7 @@ func (m *multiTestContext) replicateRange(rangeID proto.RangeID, sourceStoreInde } // Wait for the replication to complete on all destination nodes. - util.SucceedsWithin(m.t, time.Second, func() error { + util.SucceedsWithin(m.t, 3*time.Second, func() error { for _, dest := range dests { // Use LookupRange(keys) instead of GetRange(rangeID) to ensure that the // snapshot has been transferred and the descriptor initialized. @@ -417,7 +417,7 @@ func (m *multiTestContext) unreplicateRange(rangeID proto.RangeID, source, dest } err = rng.ChangeReplicas(proto.REMOVE_REPLICA, - proto.Replica{ + proto.ReplicaDescriptor{ NodeID: m.idents[dest].NodeID, StoreID: m.idents[dest].StoreID, }, rng.Desc()) @@ -444,7 +444,7 @@ func getArgs(key []byte, rangeID proto.RangeID, storeID proto.StoreID) proto.Get RequestHeader: proto.RequestHeader{ Key: key, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, } } @@ -456,7 +456,7 @@ func putArgs(key, value []byte, rangeID proto.RangeID, storeID proto.StoreID) pr RequestHeader: proto.RequestHeader{ Key: key, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, Value: proto.Value{ Bytes: value, @@ -471,7 +471,7 @@ func incrementArgs(key []byte, inc int64, rangeID proto.RangeID, storeID proto.S RequestHeader: proto.RequestHeader{ Key: key, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, Increment: inc, } @@ -481,7 +481,7 @@ func truncateLogArgs(index uint64, rangeID proto.RangeID, storeID proto.StoreID) return proto.TruncateLogRequest{ RequestHeader: proto.RequestHeader{ RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, Index: index, } diff --git a/storage/engine/rocksdb/cockroach/proto/api.pb.cc b/storage/engine/rocksdb/cockroach/proto/api.pb.cc index 49625692cb08..cdaf03e3c4d9 100644 --- a/storage/engine/rocksdb/cockroach/proto/api.pb.cc +++ b/storage/engine/rocksdb/cockroach/proto/api.pb.cc @@ -1258,211 +1258,212 @@ void protobuf_AddDesc_cockroach_2fproto_2fapi_2eproto() { "ckroach/proto/data.proto\032\034cockroach/prot" "o/errors.proto\032\024gogoproto/gogo.proto\"<\n\013" "ClientCmdID\022\027\n\twall_time\030\001 \001(\003B\004\310\336\037\000\022\024\n\006" - "random\030\002 \001(\003B\004\310\336\037\000\"\233\003\n\rRequestHeader\0223\n\t" + "random\030\002 \001(\003B\004\310\336\037\000\"\245\003\n\rRequestHeader\0223\n\t" "timestamp\030\001 \001(\0132\032.cockroach.proto.Timest" "ampB\004\310\336\037\000\022;\n\006cmd_id\030\002 \001(\0132\034.cockroach.pr" "oto.ClientCmdIDB\r\310\336\037\000\342\336\037\005CmdID\022\024\n\003key\030\003 " - "\001(\014B\007\372\336\037\003Key\022\030\n\007end_key\030\004 \001(\014B\007\372\336\037\003Key\022/" - "\n\007replica\030\005 \001(\0132\030.cockroach.proto.Replic" - "aB\004\310\336\037\000\022,\n\010range_id\030\006 \001(\003B\032\310\336\037\000\342\336\037\007Range" - "ID\372\336\037\007RangeID\022\030\n\ruser_priority\030\007 \001(\005:\0011\022" - ")\n\003txn\030\010 \001(\0132\034.cockroach.proto.Transacti" - "on\022D\n\020read_consistency\030\t \001(\0162$.cockroach" - ".proto.ReadConsistencyTypeB\004\310\336\037\000\"\227\001\n\016Res" - "ponseHeader\022%\n\005error\030\001 \001(\0132\026.cockroach.p" - "roto.Error\0223\n\ttimestamp\030\002 \001(\0132\032.cockroac" - "h.proto.TimestampB\004\310\336\037\000\022)\n\003txn\030\003 \001(\0132\034.c" - "ockroach.proto.Transaction\"F\n\nGetRequest" - "\0228\n\006header\030\001 \001(\0132\036.cockroach.proto.Reque" - "stHeaderB\010\310\336\037\000\320\336\037\001\"o\n\013GetResponse\0229\n\006hea" + "\001(\014B\007\372\336\037\003Key\022\030\n\007end_key\030\004 \001(\014B\007\372\336\037\003Key\0229" + "\n\007replica\030\005 \001(\0132\".cockroach.proto.Replic" + "aDescriptorB\004\310\336\037\000\022,\n\010range_id\030\006 \001(\003B\032\310\336\037" + "\000\342\336\037\007RangeID\372\336\037\007RangeID\022\030\n\ruser_priority" + "\030\007 \001(\005:\0011\022)\n\003txn\030\010 \001(\0132\034.cockroach.proto" + ".Transaction\022D\n\020read_consistency\030\t \001(\0162$" + ".cockroach.proto.ReadConsistencyTypeB\004\310\336" + "\037\000\"\227\001\n\016ResponseHeader\022%\n\005error\030\001 \001(\0132\026.c" + "ockroach.proto.Error\0223\n\ttimestamp\030\002 \001(\0132" + "\032.cockroach.proto.TimestampB\004\310\336\037\000\022)\n\003txn" + "\030\003 \001(\0132\034.cockroach.proto.Transaction\"F\n\n" + "GetRequest\0228\n\006header\030\001 \001(\0132\036.cockroach.p" + "roto.RequestHeaderB\010\310\336\037\000\320\336\037\001\"o\n\013GetRespo" + "nse\0229\n\006header\030\001 \001(\0132\037.cockroach.proto.Re" + "sponseHeaderB\010\310\336\037\000\320\336\037\001\022%\n\005value\030\002 \001(\0132\026." + "cockroach.proto.Value\"s\n\nPutRequest\0228\n\006h" + "eader\030\001 \001(\0132\036.cockroach.proto.RequestHea" + "derB\010\310\336\037\000\320\336\037\001\022+\n\005value\030\002 \001(\0132\026.cockroach" + ".proto.ValueB\004\310\336\037\000\"H\n\013PutResponse\0229\n\006hea" "der\030\001 \001(\0132\037.cockroach.proto.ResponseHead" - "erB\010\310\336\037\000\320\336\037\001\022%\n\005value\030\002 \001(\0132\026.cockroach." - "proto.Value\"s\n\nPutRequest\0228\n\006header\030\001 \001(" - "\0132\036.cockroach.proto.RequestHeaderB\010\310\336\037\000\320" - "\336\037\001\022+\n\005value\030\002 \001(\0132\026.cockroach.proto.Val" - "ueB\004\310\336\037\000\"H\n\013PutResponse\0229\n\006header\030\001 \001(\0132" - "\037.cockroach.proto.ResponseHeaderB\010\310\336\037\000\320\336" - "\037\001\"\251\001\n\025ConditionalPutRequest\0228\n\006header\030\001" - " \001(\0132\036.cockroach.proto.RequestHeaderB\010\310\336" - "\037\000\320\336\037\001\022+\n\005value\030\002 \001(\0132\026.cockroach.proto." - "ValueB\004\310\336\037\000\022)\n\texp_value\030\003 \001(\0132\026.cockroa" - "ch.proto.Value\"S\n\026ConditionalPutResponse" - "\0229\n\006header\030\001 \001(\0132\037.cockroach.proto.Respo" - "nseHeaderB\010\310\336\037\000\320\336\037\001\"e\n\020IncrementRequest\022" - "8\n\006header\030\001 \001(\0132\036.cockroach.proto.Reques" - "tHeaderB\010\310\336\037\000\320\336\037\001\022\027\n\tincrement\030\002 \001(\003B\004\310\336" - "\037\000\"g\n\021IncrementResponse\0229\n\006header\030\001 \001(\0132" - "\037.cockroach.proto.ResponseHeaderB\010\310\336\037\000\320\336" - "\037\001\022\027\n\tnew_value\030\002 \001(\003B\004\310\336\037\000\"I\n\rDeleteReq" - "uest\0228\n\006header\030\001 \001(\0132\036.cockroach.proto.R" - "equestHeaderB\010\310\336\037\000\320\336\037\001\"K\n\016DeleteResponse" - "\0229\n\006header\030\001 \001(\0132\037.cockroach.proto.Respo" - "nseHeaderB\010\310\336\037\000\320\336\037\001\"s\n\022DeleteRangeReques" - "t\0228\n\006header\030\001 \001(\0132\036.cockroach.proto.Requ" - "estHeaderB\010\310\336\037\000\320\336\037\001\022#\n\025max_entries_to_de" - "lete\030\002 \001(\003B\004\310\336\037\000\"k\n\023DeleteRangeResponse\022" - "9\n\006header\030\001 \001(\0132\037.cockroach.proto.Respon" - "seHeaderB\010\310\336\037\000\320\336\037\001\022\031\n\013num_deleted\030\002 \001(\003B" - "\004\310\336\037\000\"b\n\013ScanRequest\0228\n\006header\030\001 \001(\0132\036.c" - "ockroach.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022\031" - "\n\013max_results\030\002 \001(\003B\004\310\336\037\000\"x\n\014ScanRespons" - "e\0229\n\006header\030\001 \001(\0132\037.cockroach.proto.Resp" - "onseHeaderB\010\310\336\037\000\320\336\037\001\022-\n\004rows\030\002 \003(\0132\031.coc" - "kroach.proto.KeyValueB\004\310\336\037\000\"i\n\022ReverseSc" - "anRequest\0228\n\006header\030\001 \001(\0132\036.cockroach.pr" - "oto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022\031\n\013max_resul" - "ts\030\002 \001(\003B\004\310\336\037\000\"\177\n\023ReverseScanResponse\0229\n" - "\006header\030\001 \001(\0132\037.cockroach.proto.Response" - "HeaderB\010\310\336\037\000\320\336\037\001\022-\n\004rows\030\002 \003(\0132\031.cockroa" - "ch.proto.KeyValueB\004\310\336\037\000\"\340\001\n\025EndTransacti" - "onRequest\0228\n\006header\030\001 \001(\0132\036.cockroach.pr" - "oto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022\024\n\006commit\030\002 " - "\001(\010B\004\310\336\037\000\022G\n\027internal_commit_trigger\030\003 \001" - "(\0132&.cockroach.proto.InternalCommitTrigg" - "er\022.\n\007intents\030\004 \003(\0132\027.cockroach.proto.In" - "tentB\004\310\336\037\000\"\211\001\n\026EndTransactionResponse\0229\n" - "\006header\030\001 \001(\0132\037.cockroach.proto.Response" - "HeaderB\010\310\336\037\000\320\336\037\001\022\031\n\013commit_wait\030\002 \001(\003B\004\310" - "\336\037\000\022\031\n\010resolved\030\003 \003(\014B\007\372\336\037\003Key\"i\n\021AdminS" - "plitRequest\0228\n\006header\030\001 \001(\0132\036.cockroach." - "proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022\032\n\tsplit_k" - "ey\030\002 \001(\014B\007\372\336\037\003Key\"O\n\022AdminSplitResponse\022" - "9\n\006header\030\001 \001(\0132\037.cockroach.proto.Respon" - "seHeaderB\010\310\336\037\000\320\336\037\001\"M\n\021AdminMergeRequest\022" - "8\n\006header\030\001 \001(\0132\036.cockroach.proto.Reques" - "tHeaderB\010\310\336\037\000\320\336\037\001\"O\n\022AdminMergeResponse\022" - "9\n\006header\030\001 \001(\0132\037.cockroach.proto.Respon" - "seHeaderB\010\310\336\037\000\320\336\037\001\"\237\001\n\022RangeLookupReques" - "t\0228\n\006header\030\001 \001(\0132\036.cockroach.proto.Requ" - "estHeaderB\010\310\336\037\000\320\336\037\001\022\030\n\nmax_ranges\030\002 \001(\005B" - "\004\310\336\037\000\022\036\n\020consider_intents\030\003 \001(\010B\004\310\336\037\000\022\025\n" - "\007reverse\030\004 \001(\010B\004\310\336\037\000\"\210\001\n\023RangeLookupResp" - "onse\0229\n\006header\030\001 \001(\0132\037.cockroach.proto.R" - "esponseHeaderB\010\310\336\037\000\320\336\037\001\0226\n\006ranges\030\002 \003(\0132" - " .cockroach.proto.RangeDescriptorB\004\310\336\037\000\"" - "O\n\023HeartbeatTxnRequest\0228\n\006header\030\001 \001(\0132\036" - ".cockroach.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001" - "\"Q\n\024HeartbeatTxnResponse\0229\n\006header\030\001 \001(\013" - "2\037.cockroach.proto.ResponseHeaderB\010\310\336\037\000\320" - "\336\037\001\"\215\002\n\tGCRequest\0228\n\006header\030\001 \001(\0132\036.cock" - "roach.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022<\n\007g" - "c_meta\030\002 \001(\0132\033.cockroach.proto.GCMetadat" - "aB\016\310\336\037\000\342\336\037\006GCMeta\0224\n\004keys\030\003 \003(\0132 .cockro" - "ach.proto.GCRequest.GCKeyB\004\310\336\037\000\032R\n\005GCKey" - "\022\024\n\003key\030\001 \001(\014B\007\372\336\037\003Key\0223\n\ttimestamp\030\002 \001(" - "\0132\032.cockroach.proto.TimestampB\004\310\336\037\000\"G\n\nG" - "CResponse\0229\n\006header\030\001 \001(\0132\037.cockroach.pr" - "oto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\"\232\002\n\016PushTxn" - "Request\0228\n\006header\030\001 \001(\0132\036.cockroach.prot" - "o.RequestHeaderB\010\310\336\037\000\320\336\037\001\0220\n\npusher_txn\030" - "\002 \001(\0132\034.cockroach.proto.Transaction\0226\n\np" - "ushee_txn\030\003 \001(\0132\034.cockroach.proto.Transa" - "ctionB\004\310\336\037\000\022-\n\003now\030\004 \001(\0132\032.cockroach.pro" - "to.TimestampB\004\310\336\037\000\0225\n\tpush_type\030\005 \001(\0162\034." - "cockroach.proto.PushTxnTypeB\004\310\336\037\000\"~\n\017Pus" - "hTxnResponse\0229\n\006header\030\001 \001(\0132\037.cockroach" - ".proto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\0220\n\npushe" - "e_txn\030\002 \001(\0132\034.cockroach.proto.Transactio" - "n\"\210\001\n\024ResolveIntentRequest\0228\n\006header\030\001 \001" - "(\0132\036.cockroach.proto.RequestHeaderB\010\310\336\037\000" - "\320\336\037\001\0226\n\nintent_txn\030\002 \001(\0132\034.cockroach.pro" - "to.TransactionB\004\310\336\037\000\"R\n\025ResolveIntentRes" - "ponse\0229\n\006header\030\001 \001(\0132\037.cockroach.proto." - "ResponseHeaderB\010\310\336\037\000\320\336\037\001\"\215\001\n\031ResolveInte" - "ntRangeRequest\0228\n\006header\030\001 \001(\0132\036.cockroa" - "ch.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\0226\n\ninte" - "nt_txn\030\002 \001(\0132\034.cockroach.proto.Transacti" - "onB\004\310\336\037\000\"I\n\014NoopResponse\0229\n\006header\030\001 \001(\013" - "2\037.cockroach.proto.ResponseHeaderB\010\310\336\037\000\320" - "\336\037\001\"G\n\013NoopRequest\0228\n\006header\030\001 \001(\0132\036.coc" - "kroach.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\"W\n\032" - "ResolveIntentRangeResponse\0229\n\006header\030\001 \001" - "(\0132\037.cockroach.proto.ResponseHeaderB\010\310\336\037" - "\000\320\336\037\001\"u\n\014MergeRequest\0228\n\006header\030\001 \001(\0132\036." + "erB\010\310\336\037\000\320\336\037\001\"\251\001\n\025ConditionalPutRequest\0228" + "\n\006header\030\001 \001(\0132\036.cockroach.proto.Request" + "HeaderB\010\310\336\037\000\320\336\037\001\022+\n\005value\030\002 \001(\0132\026.cockro" + "ach.proto.ValueB\004\310\336\037\000\022)\n\texp_value\030\003 \001(\013" + "2\026.cockroach.proto.Value\"S\n\026ConditionalP" + "utResponse\0229\n\006header\030\001 \001(\0132\037.cockroach.p" + "roto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\"e\n\020Increme" + "ntRequest\0228\n\006header\030\001 \001(\0132\036.cockroach.pr" + "oto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022\027\n\tincrement" + "\030\002 \001(\003B\004\310\336\037\000\"g\n\021IncrementResponse\0229\n\006hea" + "der\030\001 \001(\0132\037.cockroach.proto.ResponseHead" + "erB\010\310\336\037\000\320\336\037\001\022\027\n\tnew_value\030\002 \001(\003B\004\310\336\037\000\"I\n" + "\rDeleteRequest\0228\n\006header\030\001 \001(\0132\036.cockroa" + "ch.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\"K\n\016Dele" + "teResponse\0229\n\006header\030\001 \001(\0132\037.cockroach.p" + "roto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\"s\n\022DeleteR" + "angeRequest\0228\n\006header\030\001 \001(\0132\036.cockroach." + "proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022#\n\025max_ent" + "ries_to_delete\030\002 \001(\003B\004\310\336\037\000\"k\n\023DeleteRang" + "eResponse\0229\n\006header\030\001 \001(\0132\037.cockroach.pr" + "oto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\022\031\n\013num_dele" + "ted\030\002 \001(\003B\004\310\336\037\000\"b\n\013ScanRequest\0228\n\006header" + "\030\001 \001(\0132\036.cockroach.proto.RequestHeaderB\010" + "\310\336\037\000\320\336\037\001\022\031\n\013max_results\030\002 \001(\003B\004\310\336\037\000\"x\n\014S" + "canResponse\0229\n\006header\030\001 \001(\0132\037.cockroach." + "proto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\022-\n\004rows\030\002" + " \003(\0132\031.cockroach.proto.KeyValueB\004\310\336\037\000\"i\n" + "\022ReverseScanRequest\0228\n\006header\030\001 \001(\0132\036.co" + "ckroach.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022\031\n" + "\013max_results\030\002 \001(\003B\004\310\336\037\000\"\177\n\023ReverseScanR" + "esponse\0229\n\006header\030\001 \001(\0132\037.cockroach.prot" + "o.ResponseHeaderB\010\310\336\037\000\320\336\037\001\022-\n\004rows\030\002 \003(\013" + "2\031.cockroach.proto.KeyValueB\004\310\336\037\000\"\340\001\n\025En" + "dTransactionRequest\0228\n\006header\030\001 \001(\0132\036.co" + "ckroach.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022\024\n" + "\006commit\030\002 \001(\010B\004\310\336\037\000\022G\n\027internal_commit_t" + "rigger\030\003 \001(\0132&.cockroach.proto.InternalC" + "ommitTrigger\022.\n\007intents\030\004 \003(\0132\027.cockroac" + "h.proto.IntentB\004\310\336\037\000\"\211\001\n\026EndTransactionR" + "esponse\0229\n\006header\030\001 \001(\0132\037.cockroach.prot" + "o.ResponseHeaderB\010\310\336\037\000\320\336\037\001\022\031\n\013commit_wai" + "t\030\002 \001(\003B\004\310\336\037\000\022\031\n\010resolved\030\003 \003(\014B\007\372\336\037\003Key" + "\"i\n\021AdminSplitRequest\0228\n\006header\030\001 \001(\0132\036." "cockroach.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022" - "+\n\005value\030\002 \001(\0132\026.cockroach.proto.ValueB\004" - "\310\336\037\000\"J\n\rMergeResponse\0229\n\006header\030\001 \001(\0132\037." + "\032\n\tsplit_key\030\002 \001(\014B\007\372\336\037\003Key\"O\n\022AdminSpli" + "tResponse\0229\n\006header\030\001 \001(\0132\037.cockroach.pr" + "oto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\"M\n\021AdminMer" + "geRequest\0228\n\006header\030\001 \001(\0132\036.cockroach.pr" + "oto.RequestHeaderB\010\310\336\037\000\320\336\037\001\"O\n\022AdminMerg" + "eResponse\0229\n\006header\030\001 \001(\0132\037.cockroach.pr" + "oto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\"\237\001\n\022RangeLo" + "okupRequest\0228\n\006header\030\001 \001(\0132\036.cockroach." + "proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022\030\n\nmax_ran" + "ges\030\002 \001(\005B\004\310\336\037\000\022\036\n\020consider_intents\030\003 \001(" + "\010B\004\310\336\037\000\022\025\n\007reverse\030\004 \001(\010B\004\310\336\037\000\"\210\001\n\023Range" + "LookupResponse\0229\n\006header\030\001 \001(\0132\037.cockroa" + "ch.proto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\0226\n\006ran" + "ges\030\002 \003(\0132 .cockroach.proto.RangeDescrip" + "torB\004\310\336\037\000\"O\n\023HeartbeatTxnRequest\0228\n\006head" + "er\030\001 \001(\0132\036.cockroach.proto.RequestHeader" + "B\010\310\336\037\000\320\336\037\001\"Q\n\024HeartbeatTxnResponse\0229\n\006he" + "ader\030\001 \001(\0132\037.cockroach.proto.ResponseHea" + "derB\010\310\336\037\000\320\336\037\001\"\215\002\n\tGCRequest\0228\n\006header\030\001 " + "\001(\0132\036.cockroach.proto.RequestHeaderB\010\310\336\037" + "\000\320\336\037\001\022<\n\007gc_meta\030\002 \001(\0132\033.cockroach.proto" + ".GCMetadataB\016\310\336\037\000\342\336\037\006GCMeta\0224\n\004keys\030\003 \003(" + "\0132 .cockroach.proto.GCRequest.GCKeyB\004\310\336\037" + "\000\032R\n\005GCKey\022\024\n\003key\030\001 \001(\014B\007\372\336\037\003Key\0223\n\ttime" + "stamp\030\002 \001(\0132\032.cockroach.proto.TimestampB" + "\004\310\336\037\000\"G\n\nGCResponse\0229\n\006header\030\001 \001(\0132\037.co" + "ckroach.proto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\"\232" + "\002\n\016PushTxnRequest\0228\n\006header\030\001 \001(\0132\036.cock" + "roach.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\0220\n\np" + "usher_txn\030\002 \001(\0132\034.cockroach.proto.Transa" + "ction\0226\n\npushee_txn\030\003 \001(\0132\034.cockroach.pr" + "oto.TransactionB\004\310\336\037\000\022-\n\003now\030\004 \001(\0132\032.coc" + "kroach.proto.TimestampB\004\310\336\037\000\0225\n\tpush_typ" + "e\030\005 \001(\0162\034.cockroach.proto.PushTxnTypeB\004\310" + "\336\037\000\"~\n\017PushTxnResponse\0229\n\006header\030\001 \001(\0132\037" + ".cockroach.proto.ResponseHeaderB\010\310\336\037\000\320\336\037" + "\001\0220\n\npushee_txn\030\002 \001(\0132\034.cockroach.proto." + "Transaction\"\210\001\n\024ResolveIntentRequest\0228\n\006" + "header\030\001 \001(\0132\036.cockroach.proto.RequestHe" + "aderB\010\310\336\037\000\320\336\037\001\0226\n\nintent_txn\030\002 \001(\0132\034.coc" + "kroach.proto.TransactionB\004\310\336\037\000\"R\n\025Resolv" + "eIntentResponse\0229\n\006header\030\001 \001(\0132\037.cockro" + "ach.proto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\"\215\001\n\031R" + "esolveIntentRangeRequest\0228\n\006header\030\001 \001(\013" + "2\036.cockroach.proto.RequestHeaderB\010\310\336\037\000\320\336" + "\037\001\0226\n\nintent_txn\030\002 \001(\0132\034.cockroach.proto" + ".TransactionB\004\310\336\037\000\"I\n\014NoopResponse\0229\n\006he" + "ader\030\001 \001(\0132\037.cockroach.proto.ResponseHea" + "derB\010\310\336\037\000\320\336\037\001\"G\n\013NoopRequest\0228\n\006header\030\001" + " \001(\0132\036.cockroach.proto.RequestHeaderB\010\310\336" + "\037\000\320\336\037\001\"W\n\032ResolveIntentRangeResponse\0229\n\006" + "header\030\001 \001(\0132\037.cockroach.proto.ResponseH" + "eaderB\010\310\336\037\000\320\336\037\001\"u\n\014MergeRequest\0228\n\006heade" + "r\030\001 \001(\0132\036.cockroach.proto.RequestHeaderB" + "\010\310\336\037\000\320\336\037\001\022+\n\005value\030\002 \001(\0132\026.cockroach.pro" + "to.ValueB\004\310\336\037\000\"J\n\rMergeResponse\0229\n\006heade" + "r\030\001 \001(\0132\037.cockroach.proto.ResponseHeader" + "B\010\310\336\037\000\320\336\037\001\"c\n\022TruncateLogRequest\0228\n\006head" + "er\030\001 \001(\0132\036.cockroach.proto.RequestHeader" + "B\010\310\336\037\000\320\336\037\001\022\023\n\005index\030\002 \001(\004B\004\310\336\037\000\"P\n\023Trunc" + "ateLogResponse\0229\n\006header\030\001 \001(\0132\037.cockroa" + "ch.proto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\"{\n\022Lea" + "derLeaseRequest\0228\n\006header\030\001 \001(\0132\036.cockro" + "ach.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022+\n\005lea" + "se\030\002 \001(\0132\026.cockroach.proto.LeaseB\004\310\336\037\000\"P" + "\n\023LeaderLeaseResponse\0229\n\006header\030\001 \001(\0132\037." "cockroach.proto.ResponseHeaderB\010\310\336\037\000\320\336\037\001" - "\"c\n\022TruncateLogRequest\0228\n\006header\030\001 \001(\0132\036" - ".cockroach.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001" - "\022\023\n\005index\030\002 \001(\004B\004\310\336\037\000\"P\n\023TruncateLogResp" - "onse\0229\n\006header\030\001 \001(\0132\037.cockroach.proto.R" - "esponseHeaderB\010\310\336\037\000\320\336\037\001\"{\n\022LeaderLeaseRe" - "quest\0228\n\006header\030\001 \001(\0132\036.cockroach.proto." - "RequestHeaderB\010\310\336\037\000\320\336\037\001\022+\n\005lease\030\002 \001(\0132\026" - ".cockroach.proto.LeaseB\004\310\336\037\000\"P\n\023LeaderLe" - "aseResponse\0229\n\006header\030\001 \001(\0132\037.cockroach." - "proto.ResponseHeaderB\010\310\336\037\000\320\336\037\001\"\220\t\n\014Reque" - "stUnion\022(\n\003get\030\001 \001(\0132\033.cockroach.proto.G" - "etRequest\022(\n\003put\030\002 \001(\0132\033.cockroach.proto" - ".PutRequest\022\?\n\017conditional_put\030\003 \001(\0132&.c" - "ockroach.proto.ConditionalPutRequest\0224\n\t" - "increment\030\004 \001(\0132!.cockroach.proto.Increm" - "entRequest\022.\n\006delete\030\005 \001(\0132\036.cockroach.p" - "roto.DeleteRequest\0229\n\014delete_range\030\006 \001(\013" - "2#.cockroach.proto.DeleteRangeRequest\022*\n" - "\004scan\030\007 \001(\0132\034.cockroach.proto.ScanReques" - "t\022\?\n\017end_transaction\030\010 \001(\0132&.cockroach.p" - "roto.EndTransactionRequest\0227\n\013admin_spli" - "t\030\t \001(\0132\".cockroach.proto.AdminSplitRequ" - "est\0227\n\013admin_merge\030\n \001(\0132\".cockroach.pro" - "to.AdminMergeRequest\022;\n\rheartbeat_txn\030\013 " - "\001(\0132$.cockroach.proto.HeartbeatTxnReques" - "t\022&\n\002gc\030\014 \001(\0132\032.cockroach.proto.GCReques" - "t\0221\n\010push_txn\030\r \001(\0132\037.cockroach.proto.Pu" - "shTxnRequest\0229\n\014range_lookup\030\016 \001(\0132#.coc" - "kroach.proto.RangeLookupRequest\022=\n\016resol" - "ve_intent\030\017 \001(\0132%.cockroach.proto.Resolv" - "eIntentRequest\022H\n\024resolve_intent_range\030\020" - " \001(\0132*.cockroach.proto.ResolveIntentRang" - "eRequest\022,\n\005merge\030\021 \001(\0132\035.cockroach.prot" - "o.MergeRequest\0229\n\014truncate_log\030\022 \001(\0132#.c" - "ockroach.proto.TruncateLogRequest\0229\n\014lea" - "der_lease\030\023 \001(\0132#.cockroach.proto.Leader" - "LeaseRequest\0229\n\014reverse_scan\030\024 \001(\0132#.coc" - "kroach.proto.ReverseScanRequest\022*\n\004noop\030" - "\025 \001(\0132\034.cockroach.proto.NoopRequest:\004\310\240\037" - "\001\"\246\t\n\rResponseUnion\022)\n\003get\030\001 \001(\0132\034.cockr" - "oach.proto.GetResponse\022)\n\003put\030\002 \001(\0132\034.co" - "ckroach.proto.PutResponse\022@\n\017conditional" - "_put\030\003 \001(\0132\'.cockroach.proto.Conditional" - "PutResponse\0225\n\tincrement\030\004 \001(\0132\".cockroa" - "ch.proto.IncrementResponse\022/\n\006delete\030\005 \001" - "(\0132\037.cockroach.proto.DeleteResponse\022:\n\014d" - "elete_range\030\006 \001(\0132$.cockroach.proto.Dele" - "teRangeResponse\022+\n\004scan\030\007 \001(\0132\035.cockroac" - "h.proto.ScanResponse\022@\n\017end_transaction\030" - "\010 \001(\0132\'.cockroach.proto.EndTransactionRe" - "sponse\0228\n\013admin_split\030\t \001(\0132#.cockroach." - "proto.AdminSplitResponse\0228\n\013admin_merge\030" - "\n \001(\0132#.cockroach.proto.AdminMergeRespon" - "se\022<\n\rheartbeat_txn\030\013 \001(\0132%.cockroach.pr" - "oto.HeartbeatTxnResponse\022\'\n\002gc\030\014 \001(\0132\033.c" - "ockroach.proto.GCResponse\0222\n\010push_txn\030\r " - "\001(\0132 .cockroach.proto.PushTxnResponse\022:\n" - "\014range_lookup\030\016 \001(\0132$.cockroach.proto.Ra" - "ngeLookupResponse\022>\n\016resolve_intent\030\017 \001(" - "\0132&.cockroach.proto.ResolveIntentRespons" - "e\022I\n\024resolve_intent_range\030\020 \001(\0132+.cockro" - "ach.proto.ResolveIntentRangeResponse\022-\n\005" - "merge\030\021 \001(\0132\036.cockroach.proto.MergeRespo" - "nse\022:\n\014truncate_log\030\022 \001(\0132$.cockroach.pr" - "oto.TruncateLogResponse\022:\n\014leader_lease\030" - "\023 \001(\0132$.cockroach.proto.LeaderLeaseRespo" - "nse\022:\n\014reverse_scan\030\024 \001(\0132$.cockroach.pr" - "oto.ReverseScanResponse\022+\n\004noop\030\025 \001(\0132\035." - "cockroach.proto.NoopResponse:\004\310\240\037\001\"\205\001\n\014B" - "atchRequest\0228\n\006header\030\001 \001(\0132\036.cockroach." - "proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\0225\n\010request" - "s\030\002 \003(\0132\035.cockroach.proto.RequestUnionB\004" - "\310\336\037\000:\004\230\240\037\000\"\203\001\n\rBatchResponse\0229\n\006header\030\001" - " \001(\0132\037.cockroach.proto.ResponseHeaderB\010\310" - "\336\037\000\320\336\037\001\0227\n\tresponses\030\002 \003(\0132\036.cockroach.p" - "roto.ResponseUnionB\004\310\336\037\000*L\n\023ReadConsiste" - "ncyType\022\016\n\nCONSISTENT\020\000\022\r\n\tCONSENSUS\020\001\022\020" - "\n\014INCONSISTENT\020\002\032\004\210\243\036\000*G\n\013PushTxnType\022\022\n" - "\016PUSH_TIMESTAMP\020\000\022\r\n\tABORT_TXN\020\001\022\017\n\013CLEA" - "NUP_TXN\020\002\032\004\210\243\036\000B\027Z\005proto\340\342\036\001\310\342\036\001\320\342\036\001\220\343\036\000", 8400); + "\"\220\t\n\014RequestUnion\022(\n\003get\030\001 \001(\0132\033.cockroa" + "ch.proto.GetRequest\022(\n\003put\030\002 \001(\0132\033.cockr" + "oach.proto.PutRequest\022\?\n\017conditional_put" + "\030\003 \001(\0132&.cockroach.proto.ConditionalPutR" + "equest\0224\n\tincrement\030\004 \001(\0132!.cockroach.pr" + "oto.IncrementRequest\022.\n\006delete\030\005 \001(\0132\036.c" + "ockroach.proto.DeleteRequest\0229\n\014delete_r" + "ange\030\006 \001(\0132#.cockroach.proto.DeleteRange" + "Request\022*\n\004scan\030\007 \001(\0132\034.cockroach.proto." + "ScanRequest\022\?\n\017end_transaction\030\010 \001(\0132&.c" + "ockroach.proto.EndTransactionRequest\0227\n\013" + "admin_split\030\t \001(\0132\".cockroach.proto.Admi" + "nSplitRequest\0227\n\013admin_merge\030\n \001(\0132\".coc" + "kroach.proto.AdminMergeRequest\022;\n\rheartb" + "eat_txn\030\013 \001(\0132$.cockroach.proto.Heartbea" + "tTxnRequest\022&\n\002gc\030\014 \001(\0132\032.cockroach.prot" + "o.GCRequest\0221\n\010push_txn\030\r \001(\0132\037.cockroac" + "h.proto.PushTxnRequest\0229\n\014range_lookup\030\016" + " \001(\0132#.cockroach.proto.RangeLookupReques" + "t\022=\n\016resolve_intent\030\017 \001(\0132%.cockroach.pr" + "oto.ResolveIntentRequest\022H\n\024resolve_inte" + "nt_range\030\020 \001(\0132*.cockroach.proto.Resolve" + "IntentRangeRequest\022,\n\005merge\030\021 \001(\0132\035.cock" + "roach.proto.MergeRequest\0229\n\014truncate_log" + "\030\022 \001(\0132#.cockroach.proto.TruncateLogRequ" + "est\0229\n\014leader_lease\030\023 \001(\0132#.cockroach.pr" + "oto.LeaderLeaseRequest\0229\n\014reverse_scan\030\024" + " \001(\0132#.cockroach.proto.ReverseScanReques" + "t\022*\n\004noop\030\025 \001(\0132\034.cockroach.proto.NoopRe" + "quest:\004\310\240\037\001\"\246\t\n\rResponseUnion\022)\n\003get\030\001 \001" + "(\0132\034.cockroach.proto.GetResponse\022)\n\003put\030" + "\002 \001(\0132\034.cockroach.proto.PutResponse\022@\n\017c" + "onditional_put\030\003 \001(\0132\'.cockroach.proto.C" + "onditionalPutResponse\0225\n\tincrement\030\004 \001(\013" + "2\".cockroach.proto.IncrementResponse\022/\n\006" + "delete\030\005 \001(\0132\037.cockroach.proto.DeleteRes" + "ponse\022:\n\014delete_range\030\006 \001(\0132$.cockroach." + "proto.DeleteRangeResponse\022+\n\004scan\030\007 \001(\0132" + "\035.cockroach.proto.ScanResponse\022@\n\017end_tr" + "ansaction\030\010 \001(\0132\'.cockroach.proto.EndTra" + "nsactionResponse\0228\n\013admin_split\030\t \001(\0132#." + "cockroach.proto.AdminSplitResponse\0228\n\013ad" + "min_merge\030\n \001(\0132#.cockroach.proto.AdminM" + "ergeResponse\022<\n\rheartbeat_txn\030\013 \001(\0132%.co" + "ckroach.proto.HeartbeatTxnResponse\022\'\n\002gc" + "\030\014 \001(\0132\033.cockroach.proto.GCResponse\0222\n\010p" + "ush_txn\030\r \001(\0132 .cockroach.proto.PushTxnR" + "esponse\022:\n\014range_lookup\030\016 \001(\0132$.cockroac" + "h.proto.RangeLookupResponse\022>\n\016resolve_i" + "ntent\030\017 \001(\0132&.cockroach.proto.ResolveInt" + "entResponse\022I\n\024resolve_intent_range\030\020 \001(" + "\0132+.cockroach.proto.ResolveIntentRangeRe" + "sponse\022-\n\005merge\030\021 \001(\0132\036.cockroach.proto." + "MergeResponse\022:\n\014truncate_log\030\022 \001(\0132$.co" + "ckroach.proto.TruncateLogResponse\022:\n\014lea" + "der_lease\030\023 \001(\0132$.cockroach.proto.Leader" + "LeaseResponse\022:\n\014reverse_scan\030\024 \001(\0132$.co" + "ckroach.proto.ReverseScanResponse\022+\n\004noo" + "p\030\025 \001(\0132\035.cockroach.proto.NoopResponse:\004" + "\310\240\037\001\"\205\001\n\014BatchRequest\0228\n\006header\030\001 \001(\0132\036." + "cockroach.proto.RequestHeaderB\010\310\336\037\000\320\336\037\001\022" + "5\n\010requests\030\002 \003(\0132\035.cockroach.proto.Requ" + "estUnionB\004\310\336\037\000:\004\230\240\037\000\"\203\001\n\rBatchResponse\0229" + "\n\006header\030\001 \001(\0132\037.cockroach.proto.Respons" + "eHeaderB\010\310\336\037\000\320\336\037\001\0227\n\tresponses\030\002 \003(\0132\036.c" + "ockroach.proto.ResponseUnionB\004\310\336\037\000*L\n\023Re" + "adConsistencyType\022\016\n\nCONSISTENT\020\000\022\r\n\tCON" + "SENSUS\020\001\022\020\n\014INCONSISTENT\020\002\032\004\210\243\036\000*G\n\013Push" + "TxnType\022\022\n\016PUSH_TIMESTAMP\020\000\022\r\n\tABORT_TXN" + "\020\001\022\017\n\013CLEANUP_TXN\020\002\032\004\210\243\036\000B\027Z\005proto\340\342\036\001\310\342" + "\036\001\320\342\036\001\220\343\036\000", 8410); ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile( "cockroach/proto/api.proto", &protobuf_RegisterTypes); ClientCmdID::default_instance_ = new ClientCmdID(); @@ -1973,7 +1974,7 @@ RequestHeader::RequestHeader() void RequestHeader::InitAsDefaultInstance() { timestamp_ = const_cast< ::cockroach::proto::Timestamp*>(&::cockroach::proto::Timestamp::default_instance()); cmd_id_ = const_cast< ::cockroach::proto::ClientCmdID*>(&::cockroach::proto::ClientCmdID::default_instance()); - replica_ = const_cast< ::cockroach::proto::Replica*>(&::cockroach::proto::Replica::default_instance()); + replica_ = const_cast< ::cockroach::proto::ReplicaDescriptor*>(&::cockroach::proto::ReplicaDescriptor::default_instance()); txn_ = const_cast< ::cockroach::proto::Transaction*>(&::cockroach::proto::Transaction::default_instance()); } @@ -2056,7 +2057,7 @@ void RequestHeader::Clear() { end_key_.ClearToEmptyNoArena(&::google::protobuf::internal::GetEmptyStringAlreadyInited()); } if (has_replica()) { - if (replica_ != NULL) replica_->::cockroach::proto::Replica::Clear(); + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); } range_id_ = GOOGLE_LONGLONG(0); user_priority_ = 1; @@ -2132,7 +2133,7 @@ bool RequestHeader::MergePartialFromCodedStream( break; } - // optional .cockroach.proto.Replica replica = 5; + // optional .cockroach.proto.ReplicaDescriptor replica = 5; case 5: { if (tag == 42) { parse_replica: @@ -2257,7 +2258,7 @@ void RequestHeader::SerializeWithCachedSizes( 4, this->end_key(), output); } - // optional .cockroach.proto.Replica replica = 5; + // optional .cockroach.proto.ReplicaDescriptor replica = 5; if (has_replica()) { ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( 5, *this->replica_, output); @@ -2323,7 +2324,7 @@ ::google::protobuf::uint8* RequestHeader::SerializeWithCachedSizesToArray( 4, this->end_key(), target); } - // optional .cockroach.proto.Replica replica = 5; + // optional .cockroach.proto.ReplicaDescriptor replica = 5; if (has_replica()) { target = ::google::protobuf::internal::WireFormatLite:: WriteMessageNoVirtualToArray( @@ -2393,7 +2394,7 @@ int RequestHeader::ByteSize() const { this->end_key()); } - // optional .cockroach.proto.Replica replica = 5; + // optional .cockroach.proto.ReplicaDescriptor replica = 5; if (has_replica()) { total_size += 1 + ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual( @@ -2469,7 +2470,7 @@ void RequestHeader::MergeFrom(const RequestHeader& from) { end_key_.AssignWithDefault(&::google::protobuf::internal::GetEmptyStringAlreadyInited(), from.end_key_); } if (from.has_replica()) { - mutable_replica()->::cockroach::proto::Replica::MergeFrom(from.replica()); + mutable_replica()->::cockroach::proto::ReplicaDescriptor::MergeFrom(from.replica()); } if (from.has_range_id()) { set_range_id(from.range_id()); @@ -2730,7 +2731,7 @@ void RequestHeader::clear_end_key() { // @@protoc_insertion_point(field_set_allocated:cockroach.proto.RequestHeader.end_key) } -// optional .cockroach.proto.Replica replica = 5; +// optional .cockroach.proto.ReplicaDescriptor replica = 5; bool RequestHeader::has_replica() const { return (_has_bits_[0] & 0x00000010u) != 0; } @@ -2741,28 +2742,28 @@ void RequestHeader::clear_has_replica() { _has_bits_[0] &= ~0x00000010u; } void RequestHeader::clear_replica() { - if (replica_ != NULL) replica_->::cockroach::proto::Replica::Clear(); + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); clear_has_replica(); } - const ::cockroach::proto::Replica& RequestHeader::replica() const { + const ::cockroach::proto::ReplicaDescriptor& RequestHeader::replica() const { // @@protoc_insertion_point(field_get:cockroach.proto.RequestHeader.replica) return replica_ != NULL ? *replica_ : *default_instance_->replica_; } - ::cockroach::proto::Replica* RequestHeader::mutable_replica() { + ::cockroach::proto::ReplicaDescriptor* RequestHeader::mutable_replica() { set_has_replica(); if (replica_ == NULL) { - replica_ = new ::cockroach::proto::Replica; + replica_ = new ::cockroach::proto::ReplicaDescriptor; } // @@protoc_insertion_point(field_mutable:cockroach.proto.RequestHeader.replica) return replica_; } - ::cockroach::proto::Replica* RequestHeader::release_replica() { + ::cockroach::proto::ReplicaDescriptor* RequestHeader::release_replica() { clear_has_replica(); - ::cockroach::proto::Replica* temp = replica_; + ::cockroach::proto::ReplicaDescriptor* temp = replica_; replica_ = NULL; return temp; } - void RequestHeader::set_allocated_replica(::cockroach::proto::Replica* replica) { + void RequestHeader::set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica) { delete replica_; replica_ = replica; if (replica) { diff --git a/storage/engine/rocksdb/cockroach/proto/api.pb.h b/storage/engine/rocksdb/cockroach/proto/api.pb.h index c806ae1adbe7..b74f877a6d6b 100644 --- a/storage/engine/rocksdb/cockroach/proto/api.pb.h +++ b/storage/engine/rocksdb/cockroach/proto/api.pb.h @@ -340,14 +340,14 @@ class RequestHeader : public ::google::protobuf::Message { ::std::string* release_end_key(); void set_allocated_end_key(::std::string* end_key); - // optional .cockroach.proto.Replica replica = 5; + // optional .cockroach.proto.ReplicaDescriptor replica = 5; bool has_replica() const; void clear_replica(); static const int kReplicaFieldNumber = 5; - const ::cockroach::proto::Replica& replica() const; - ::cockroach::proto::Replica* mutable_replica(); - ::cockroach::proto::Replica* release_replica(); - void set_allocated_replica(::cockroach::proto::Replica* replica); + const ::cockroach::proto::ReplicaDescriptor& replica() const; + ::cockroach::proto::ReplicaDescriptor* mutable_replica(); + ::cockroach::proto::ReplicaDescriptor* release_replica(); + void set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica); // optional int64 range_id = 6; bool has_range_id() const; @@ -407,7 +407,7 @@ class RequestHeader : public ::google::protobuf::Message { ::cockroach::proto::ClientCmdID* cmd_id_; ::google::protobuf::internal::ArenaStringPtr key_; ::google::protobuf::internal::ArenaStringPtr end_key_; - ::cockroach::proto::Replica* replica_; + ::cockroach::proto::ReplicaDescriptor* replica_; ::google::protobuf::int64 range_id_; ::cockroach::proto::Transaction* txn_; ::google::protobuf::int32 user_priority_; @@ -5981,7 +5981,7 @@ inline void RequestHeader::set_allocated_end_key(::std::string* end_key) { // @@protoc_insertion_point(field_set_allocated:cockroach.proto.RequestHeader.end_key) } -// optional .cockroach.proto.Replica replica = 5; +// optional .cockroach.proto.ReplicaDescriptor replica = 5; inline bool RequestHeader::has_replica() const { return (_has_bits_[0] & 0x00000010u) != 0; } @@ -5992,28 +5992,28 @@ inline void RequestHeader::clear_has_replica() { _has_bits_[0] &= ~0x00000010u; } inline void RequestHeader::clear_replica() { - if (replica_ != NULL) replica_->::cockroach::proto::Replica::Clear(); + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); clear_has_replica(); } -inline const ::cockroach::proto::Replica& RequestHeader::replica() const { +inline const ::cockroach::proto::ReplicaDescriptor& RequestHeader::replica() const { // @@protoc_insertion_point(field_get:cockroach.proto.RequestHeader.replica) return replica_ != NULL ? *replica_ : *default_instance_->replica_; } -inline ::cockroach::proto::Replica* RequestHeader::mutable_replica() { +inline ::cockroach::proto::ReplicaDescriptor* RequestHeader::mutable_replica() { set_has_replica(); if (replica_ == NULL) { - replica_ = new ::cockroach::proto::Replica; + replica_ = new ::cockroach::proto::ReplicaDescriptor; } // @@protoc_insertion_point(field_mutable:cockroach.proto.RequestHeader.replica) return replica_; } -inline ::cockroach::proto::Replica* RequestHeader::release_replica() { +inline ::cockroach::proto::ReplicaDescriptor* RequestHeader::release_replica() { clear_has_replica(); - ::cockroach::proto::Replica* temp = replica_; + ::cockroach::proto::ReplicaDescriptor* temp = replica_; replica_ = NULL; return temp; } -inline void RequestHeader::set_allocated_replica(::cockroach::proto::Replica* replica) { +inline void RequestHeader::set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica) { delete replica_; replica_ = replica; if (replica) { diff --git a/storage/engine/rocksdb/cockroach/proto/data.pb.cc b/storage/engine/rocksdb/cockroach/proto/data.pb.cc index 92c74ad069ee..81b5612f50b9 100644 --- a/storage/engine/rocksdb/cockroach/proto/data.pb.cc +++ b/storage/engine/rocksdb/cockroach/proto/data.pb.cc @@ -294,7 +294,7 @@ void protobuf_AssignDesc_cockroach_2fproto_2fdata_2eproto() { static const int Lease_offsets_[3] = { GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Lease, start_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Lease, expiration_), - GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Lease, raft_node_id_), + GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Lease, replica_), }; Lease_reflection_ = ::google::protobuf::internal::GeneratedMessageReflection::NewGeneratedMessageReflection( @@ -453,55 +453,56 @@ void protobuf_AddDesc_cockroach_2fproto_2fdata_2eproto() { "\000\"\213\001\n\014MergeTrigger\022<\n\014updated_desc\030\001 \001(\013" "2 .cockroach.proto.RangeDescriptorB\004\310\336\037\000" "\022=\n\021subsumed_range_id\030\002 \001(\003B\"\310\336\037\000\342\336\037\017Sub" - "sumedRangeID\372\336\037\007RangeID\"\327\002\n\025ChangeReplic" + "sumedRangeID\372\336\037\007RangeID\"\353\002\n\025ChangeReplic" "asTrigger\022)\n\007node_id\030\001 \001(\005B\030\310\336\037\000\342\336\037\006Node" "ID\372\336\037\006NodeID\022,\n\010store_id\030\002 \001(\005B\032\310\336\037\000\342\336\037\007" "StoreID\372\336\037\007StoreID\022=\n\013change_type\030\003 \001(\0162" "\".cockroach.proto.ReplicaChangeTypeB\004\310\336\037" - "\000\022/\n\007replica\030\004 \001(\0132\030.cockroach.proto.Rep" - "licaB\004\310\336\037\000\0228\n\020updated_replicas\030\005 \003(\0132\030.c" - "ockroach.proto.ReplicaB\004\310\336\037\000\022;\n\017next_rep" - "lica_id\030\006 \001(\005B\"\310\336\037\000\342\336\037\rNextReplicaID\372\336\037\t" - "ReplicaID\"C\n\023ModifiedSpanTrigger\022,\n\016syst" - "em_db_span\030\001 \001(\010B\024\310\336\037\000\342\336\037\014SystemDBSpan\"\221" - "\002\n\025InternalCommitTrigger\0224\n\rsplit_trigge" - "r\030\001 \001(\0132\035.cockroach.proto.SplitTrigger\0224" - "\n\rmerge_trigger\030\002 \001(\0132\035.cockroach.proto." - "MergeTrigger\022G\n\027change_replicas_trigger\030" - "\003 \001(\0132&.cockroach.proto.ChangeReplicasTr" - "igger\022C\n\025modified_span_trigger\030\004 \001(\0132$.c" - "ockroach.proto.ModifiedSpanTrigger\"\035\n\010No" - "deList\022\021\n\005nodes\030\001 \003(\005B\002\020\001\"\234\004\n\013Transactio" - "n\022\022\n\004name\030\001 \001(\tB\004\310\336\037\000\022\024\n\003key\030\002 \001(\014B\007\372\336\037\003" - "Key\022\022\n\002id\030\003 \001(\014B\006\342\336\037\002ID\022\026\n\010priority\030\004 \001(" - "\005B\004\310\336\037\000\0227\n\tisolation\030\005 \001(\0162\036.cockroach.p" - "roto.IsolationTypeB\004\310\336\037\000\0228\n\006status\030\006 \001(\016" - "2\".cockroach.proto.TransactionStatusB\004\310\336" - "\037\000\022\023\n\005epoch\030\007 \001(\005B\004\310\336\037\000\0222\n\016last_heartbea" - "t\030\010 \001(\0132\032.cockroach.proto.Timestamp\0223\n\tt" - "imestamp\030\t \001(\0132\032.cockroach.proto.Timesta" - "mpB\004\310\336\037\000\0228\n\016orig_timestamp\030\n \001(\0132\032.cockr" - "oach.proto.TimestampB\004\310\336\037\000\0227\n\rmax_timest" - "amp\030\013 \001(\0132\032.cockroach.proto.TimestampB\004\310" - "\336\037\000\0226\n\rcertain_nodes\030\014 \001(\0132\031.cockroach.p" - "roto.NodeListB\004\310\336\037\000\022\025\n\007Writing\030\r \001(\010B\004\310\336" - "\037\000:\004\230\240\037\000\"\254\001\n\005Lease\022/\n\005start\030\001 \001(\0132\032.cock" - "roach.proto.TimestampB\004\310\336\037\000\0224\n\nexpiratio" - "n\030\002 \001(\0132\032.cockroach.proto.TimestampB\004\310\336\037" - "\000\0226\n\014raft_node_id\030\003 \001(\004B \310\336\037\000\342\336\037\nRaftNod" - "eID\372\336\037\nRaftNodeID:\004\230\240\037\000\"i\n\006Intent\022\024\n\003key" - "\030\001 \001(\014B\007\372\336\037\003Key\022\030\n\007end_key\030\002 \001(\014B\007\372\336\037\003Ke" - "y\022/\n\003txn\030\003 \001(\0132\034.cockroach.proto.Transac" - "tionB\004\310\336\037\000\"H\n\nGCMetadata\022\035\n\017last_scan_na" - "nos\030\001 \001(\003B\004\310\336\037\000\022\033\n\023oldest_intent_nanos\030\002" - " \001(\003*Q\n\tValueType\022\013\n\007UNKNOWN\020\000\022\007\n\003INT\020\001\022" - "\t\n\005FLOAT\020\002\022\t\n\005BYTES\020\003\022\010\n\004TIME\020\004\022\016\n\nTIMES" - "ERIES\020d*>\n\021ReplicaChangeType\022\017\n\013ADD_REPL" - "ICA\020\000\022\022\n\016REMOVE_REPLICA\020\001\032\004\210\243\036\000*5\n\rIsola" - "tionType\022\020\n\014SERIALIZABLE\020\000\022\014\n\010SNAPSHOT\020\001" - "\032\004\210\243\036\000*B\n\021TransactionStatus\022\013\n\007PENDING\020\000" - "\022\r\n\tCOMMITTED\020\001\022\013\n\007ABORTED\020\002\032\004\210\243\036\000B\027Z\005pr" - "oto\340\342\036\001\310\342\036\001\320\342\036\001\220\343\036\000", 2779); + "\000\0229\n\007replica\030\004 \001(\0132\".cockroach.proto.Rep" + "licaDescriptorB\004\310\336\037\000\022B\n\020updated_replicas" + "\030\005 \003(\0132\".cockroach.proto.ReplicaDescript" + "orB\004\310\336\037\000\022;\n\017next_replica_id\030\006 \001(\005B\"\310\336\037\000\342" + "\336\037\rNextReplicaID\372\336\037\tReplicaID\"C\n\023Modifie" + "dSpanTrigger\022,\n\016system_db_span\030\001 \001(\010B\024\310\336" + "\037\000\342\336\037\014SystemDBSpan\"\221\002\n\025InternalCommitTri" + "gger\0224\n\rsplit_trigger\030\001 \001(\0132\035.cockroach." + "proto.SplitTrigger\0224\n\rmerge_trigger\030\002 \001(" + "\0132\035.cockroach.proto.MergeTrigger\022G\n\027chan" + "ge_replicas_trigger\030\003 \001(\0132&.cockroach.pr" + "oto.ChangeReplicasTrigger\022C\n\025modified_sp" + "an_trigger\030\004 \001(\0132$.cockroach.proto.Modif" + "iedSpanTrigger\"\035\n\010NodeList\022\021\n\005nodes\030\001 \003(" + "\005B\002\020\001\"\234\004\n\013Transaction\022\022\n\004name\030\001 \001(\tB\004\310\336\037" + "\000\022\024\n\003key\030\002 \001(\014B\007\372\336\037\003Key\022\022\n\002id\030\003 \001(\014B\006\342\336\037" + "\002ID\022\026\n\010priority\030\004 \001(\005B\004\310\336\037\000\0227\n\tisolation" + "\030\005 \001(\0162\036.cockroach.proto.IsolationTypeB\004" + "\310\336\037\000\0228\n\006status\030\006 \001(\0162\".cockroach.proto.T" + "ransactionStatusB\004\310\336\037\000\022\023\n\005epoch\030\007 \001(\005B\004\310" + "\336\037\000\0222\n\016last_heartbeat\030\010 \001(\0132\032.cockroach." + "proto.Timestamp\0223\n\ttimestamp\030\t \001(\0132\032.coc" + "kroach.proto.TimestampB\004\310\336\037\000\0228\n\016orig_tim" + "estamp\030\n \001(\0132\032.cockroach.proto.Timestamp" + "B\004\310\336\037\000\0227\n\rmax_timestamp\030\013 \001(\0132\032.cockroac" + "h.proto.TimestampB\004\310\336\037\000\0226\n\rcertain_nodes" + "\030\014 \001(\0132\031.cockroach.proto.NodeListB\004\310\336\037\000\022" + "\025\n\007Writing\030\r \001(\010B\004\310\336\037\000:\004\230\240\037\000\"\257\001\n\005Lease\022/" + "\n\005start\030\001 \001(\0132\032.cockroach.proto.Timestam" + "pB\004\310\336\037\000\0224\n\nexpiration\030\002 \001(\0132\032.cockroach." + "proto.TimestampB\004\310\336\037\000\0229\n\007replica\030\003 \001(\0132\"" + ".cockroach.proto.ReplicaDescriptorB\004\310\336\037\000" + ":\004\230\240\037\000\"i\n\006Intent\022\024\n\003key\030\001 \001(\014B\007\372\336\037\003Key\022\030" + "\n\007end_key\030\002 \001(\014B\007\372\336\037\003Key\022/\n\003txn\030\003 \001(\0132\034." + "cockroach.proto.TransactionB\004\310\336\037\000\"H\n\nGCM" + "etadata\022\035\n\017last_scan_nanos\030\001 \001(\003B\004\310\336\037\000\022\033" + "\n\023oldest_intent_nanos\030\002 \001(\003*Q\n\tValueType" + "\022\013\n\007UNKNOWN\020\000\022\007\n\003INT\020\001\022\t\n\005FLOAT\020\002\022\t\n\005BYT" + "ES\020\003\022\010\n\004TIME\020\004\022\016\n\nTIMESERIES\020d*>\n\021Replic" + "aChangeType\022\017\n\013ADD_REPLICA\020\000\022\022\n\016REMOVE_R" + "EPLICA\020\001\032\004\210\243\036\000*5\n\rIsolationType\022\020\n\014SERIA" + "LIZABLE\020\000\022\014\n\010SNAPSHOT\020\001\032\004\210\243\036\000*B\n\021Transac" + "tionStatus\022\013\n\007PENDING\020\000\022\r\n\tCOMMITTED\020\001\022\013" + "\n\007ABORTED\020\002\032\004\210\243\036\000B\027Z\005proto\340\342\036\001\310\342\036\001\320\342\036\001\220\343" + "\036\000", 2802); ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile( "cockroach/proto/data.proto", &protobuf_RegisterTypes); Timestamp::default_instance_ = new Timestamp(); @@ -3446,7 +3447,7 @@ ChangeReplicasTrigger::ChangeReplicasTrigger() } void ChangeReplicasTrigger::InitAsDefaultInstance() { - replica_ = const_cast< ::cockroach::proto::Replica*>(&::cockroach::proto::Replica::default_instance()); + replica_ = const_cast< ::cockroach::proto::ReplicaDescriptor*>(&::cockroach::proto::ReplicaDescriptor::default_instance()); } ChangeReplicasTrigger::ChangeReplicasTrigger(const ChangeReplicasTrigger& from) @@ -3516,7 +3517,7 @@ void ChangeReplicasTrigger::Clear() { ZR_(node_id_, store_id_); ZR_(change_type_, next_replica_id_); if (has_replica()) { - if (replica_ != NULL) replica_->::cockroach::proto::Replica::Clear(); + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); } } @@ -3589,7 +3590,7 @@ bool ChangeReplicasTrigger::MergePartialFromCodedStream( break; } - // optional .cockroach.proto.Replica replica = 4; + // optional .cockroach.proto.ReplicaDescriptor replica = 4; case 4: { if (tag == 34) { parse_replica: @@ -3602,7 +3603,7 @@ bool ChangeReplicasTrigger::MergePartialFromCodedStream( break; } - // repeated .cockroach.proto.Replica updated_replicas = 5; + // repeated .cockroach.proto.ReplicaDescriptor updated_replicas = 5; case 5: { if (tag == 42) { parse_updated_replicas: @@ -3675,13 +3676,13 @@ void ChangeReplicasTrigger::SerializeWithCachedSizes( 3, this->change_type(), output); } - // optional .cockroach.proto.Replica replica = 4; + // optional .cockroach.proto.ReplicaDescriptor replica = 4; if (has_replica()) { ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( 4, *this->replica_, output); } - // repeated .cockroach.proto.Replica updated_replicas = 5; + // repeated .cockroach.proto.ReplicaDescriptor updated_replicas = 5; for (unsigned int i = 0, n = this->updated_replicas_size(); i < n; i++) { ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( 5, this->updated_replicas(i), output); @@ -3718,14 +3719,14 @@ ::google::protobuf::uint8* ChangeReplicasTrigger::SerializeWithCachedSizesToArra 3, this->change_type(), target); } - // optional .cockroach.proto.Replica replica = 4; + // optional .cockroach.proto.ReplicaDescriptor replica = 4; if (has_replica()) { target = ::google::protobuf::internal::WireFormatLite:: WriteMessageNoVirtualToArray( 4, *this->replica_, target); } - // repeated .cockroach.proto.Replica updated_replicas = 5; + // repeated .cockroach.proto.ReplicaDescriptor updated_replicas = 5; for (unsigned int i = 0, n = this->updated_replicas_size(); i < n; i++) { target = ::google::protobuf::internal::WireFormatLite:: WriteMessageNoVirtualToArray( @@ -3769,7 +3770,7 @@ int ChangeReplicasTrigger::ByteSize() const { ::google::protobuf::internal::WireFormatLite::EnumSize(this->change_type()); } - // optional .cockroach.proto.Replica replica = 4; + // optional .cockroach.proto.ReplicaDescriptor replica = 4; if (has_replica()) { total_size += 1 + ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual( @@ -3784,7 +3785,7 @@ int ChangeReplicasTrigger::ByteSize() const { } } - // repeated .cockroach.proto.Replica updated_replicas = 5; + // repeated .cockroach.proto.ReplicaDescriptor updated_replicas = 5; total_size += 1 * this->updated_replicas_size(); for (int i = 0; i < this->updated_replicas_size(); i++) { total_size += @@ -3829,7 +3830,7 @@ void ChangeReplicasTrigger::MergeFrom(const ChangeReplicasTrigger& from) { set_change_type(from.change_type()); } if (from.has_replica()) { - mutable_replica()->::cockroach::proto::Replica::MergeFrom(from.replica()); + mutable_replica()->::cockroach::proto::ReplicaDescriptor::MergeFrom(from.replica()); } if (from.has_next_replica_id()) { set_next_replica_id(from.next_replica_id()); @@ -3957,7 +3958,7 @@ void ChangeReplicasTrigger::clear_change_type() { // @@protoc_insertion_point(field_set:cockroach.proto.ChangeReplicasTrigger.change_type) } -// optional .cockroach.proto.Replica replica = 4; +// optional .cockroach.proto.ReplicaDescriptor replica = 4; bool ChangeReplicasTrigger::has_replica() const { return (_has_bits_[0] & 0x00000008u) != 0; } @@ -3968,28 +3969,28 @@ void ChangeReplicasTrigger::clear_has_replica() { _has_bits_[0] &= ~0x00000008u; } void ChangeReplicasTrigger::clear_replica() { - if (replica_ != NULL) replica_->::cockroach::proto::Replica::Clear(); + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); clear_has_replica(); } - const ::cockroach::proto::Replica& ChangeReplicasTrigger::replica() const { + const ::cockroach::proto::ReplicaDescriptor& ChangeReplicasTrigger::replica() const { // @@protoc_insertion_point(field_get:cockroach.proto.ChangeReplicasTrigger.replica) return replica_ != NULL ? *replica_ : *default_instance_->replica_; } - ::cockroach::proto::Replica* ChangeReplicasTrigger::mutable_replica() { + ::cockroach::proto::ReplicaDescriptor* ChangeReplicasTrigger::mutable_replica() { set_has_replica(); if (replica_ == NULL) { - replica_ = new ::cockroach::proto::Replica; + replica_ = new ::cockroach::proto::ReplicaDescriptor; } // @@protoc_insertion_point(field_mutable:cockroach.proto.ChangeReplicasTrigger.replica) return replica_; } - ::cockroach::proto::Replica* ChangeReplicasTrigger::release_replica() { + ::cockroach::proto::ReplicaDescriptor* ChangeReplicasTrigger::release_replica() { clear_has_replica(); - ::cockroach::proto::Replica* temp = replica_; + ::cockroach::proto::ReplicaDescriptor* temp = replica_; replica_ = NULL; return temp; } - void ChangeReplicasTrigger::set_allocated_replica(::cockroach::proto::Replica* replica) { + void ChangeReplicasTrigger::set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica) { delete replica_; replica_ = replica; if (replica) { @@ -4000,31 +4001,31 @@ void ChangeReplicasTrigger::clear_replica() { // @@protoc_insertion_point(field_set_allocated:cockroach.proto.ChangeReplicasTrigger.replica) } -// repeated .cockroach.proto.Replica updated_replicas = 5; +// repeated .cockroach.proto.ReplicaDescriptor updated_replicas = 5; int ChangeReplicasTrigger::updated_replicas_size() const { return updated_replicas_.size(); } void ChangeReplicasTrigger::clear_updated_replicas() { updated_replicas_.Clear(); } - const ::cockroach::proto::Replica& ChangeReplicasTrigger::updated_replicas(int index) const { + const ::cockroach::proto::ReplicaDescriptor& ChangeReplicasTrigger::updated_replicas(int index) const { // @@protoc_insertion_point(field_get:cockroach.proto.ChangeReplicasTrigger.updated_replicas) return updated_replicas_.Get(index); } - ::cockroach::proto::Replica* ChangeReplicasTrigger::mutable_updated_replicas(int index) { + ::cockroach::proto::ReplicaDescriptor* ChangeReplicasTrigger::mutable_updated_replicas(int index) { // @@protoc_insertion_point(field_mutable:cockroach.proto.ChangeReplicasTrigger.updated_replicas) return updated_replicas_.Mutable(index); } - ::cockroach::proto::Replica* ChangeReplicasTrigger::add_updated_replicas() { + ::cockroach::proto::ReplicaDescriptor* ChangeReplicasTrigger::add_updated_replicas() { // @@protoc_insertion_point(field_add:cockroach.proto.ChangeReplicasTrigger.updated_replicas) return updated_replicas_.Add(); } - const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >& + const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >& ChangeReplicasTrigger::updated_replicas() const { // @@protoc_insertion_point(field_list:cockroach.proto.ChangeReplicasTrigger.updated_replicas) return updated_replicas_; } - ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >* + ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >* ChangeReplicasTrigger::mutable_updated_replicas() { // @@protoc_insertion_point(field_mutable_list:cockroach.proto.ChangeReplicasTrigger.updated_replicas) return &updated_replicas_; @@ -6434,7 +6435,7 @@ void Transaction::clear_writing() { #ifndef _MSC_VER const int Lease::kStartFieldNumber; const int Lease::kExpirationFieldNumber; -const int Lease::kRaftNodeIdFieldNumber; +const int Lease::kReplicaFieldNumber; #endif // !_MSC_VER Lease::Lease() @@ -6446,6 +6447,7 @@ Lease::Lease() void Lease::InitAsDefaultInstance() { start_ = const_cast< ::cockroach::proto::Timestamp*>(&::cockroach::proto::Timestamp::default_instance()); expiration_ = const_cast< ::cockroach::proto::Timestamp*>(&::cockroach::proto::Timestamp::default_instance()); + replica_ = const_cast< ::cockroach::proto::ReplicaDescriptor*>(&::cockroach::proto::ReplicaDescriptor::default_instance()); } Lease::Lease(const Lease& from) @@ -6460,7 +6462,7 @@ void Lease::SharedCtor() { _cached_size_ = 0; start_ = NULL; expiration_ = NULL; - raft_node_id_ = GOOGLE_ULONGLONG(0); + replica_ = NULL; ::memset(_has_bits_, 0, sizeof(_has_bits_)); } @@ -6473,6 +6475,7 @@ void Lease::SharedDtor() { if (this != default_instance_) { delete start_; delete expiration_; + delete replica_; } } @@ -6509,7 +6512,9 @@ void Lease::Clear() { if (has_expiration()) { if (expiration_ != NULL) expiration_->::cockroach::proto::Timestamp::Clear(); } - raft_node_id_ = GOOGLE_ULONGLONG(0); + if (has_replica()) { + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); + } } ::memset(_has_bits_, 0, sizeof(_has_bits_)); if (_internal_metadata_.have_unknown_fields()) { @@ -6548,18 +6553,16 @@ bool Lease::MergePartialFromCodedStream( } else { goto handle_unusual; } - if (input->ExpectTag(24)) goto parse_raft_node_id; + if (input->ExpectTag(26)) goto parse_replica; break; } - // optional uint64 raft_node_id = 3; + // optional .cockroach.proto.ReplicaDescriptor replica = 3; case 3: { - if (tag == 24) { - parse_raft_node_id: - DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< - ::google::protobuf::uint64, ::google::protobuf::internal::WireFormatLite::TYPE_UINT64>( - input, &raft_node_id_))); - set_has_raft_node_id(); + if (tag == 26) { + parse_replica: + DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual( + input, mutable_replica())); } else { goto handle_unusual; } @@ -6604,9 +6607,10 @@ void Lease::SerializeWithCachedSizes( 2, *this->expiration_, output); } - // optional uint64 raft_node_id = 3; - if (has_raft_node_id()) { - ::google::protobuf::internal::WireFormatLite::WriteUInt64(3, this->raft_node_id(), output); + // optional .cockroach.proto.ReplicaDescriptor replica = 3; + if (has_replica()) { + ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( + 3, *this->replica_, output); } if (_internal_metadata_.have_unknown_fields()) { @@ -6633,9 +6637,11 @@ ::google::protobuf::uint8* Lease::SerializeWithCachedSizesToArray( 2, *this->expiration_, target); } - // optional uint64 raft_node_id = 3; - if (has_raft_node_id()) { - target = ::google::protobuf::internal::WireFormatLite::WriteUInt64ToArray(3, this->raft_node_id(), target); + // optional .cockroach.proto.ReplicaDescriptor replica = 3; + if (has_replica()) { + target = ::google::protobuf::internal::WireFormatLite:: + WriteMessageNoVirtualToArray( + 3, *this->replica_, target); } if (_internal_metadata_.have_unknown_fields()) { @@ -6664,11 +6670,11 @@ int Lease::ByteSize() const { *this->expiration_); } - // optional uint64 raft_node_id = 3; - if (has_raft_node_id()) { + // optional .cockroach.proto.ReplicaDescriptor replica = 3; + if (has_replica()) { total_size += 1 + - ::google::protobuf::internal::WireFormatLite::UInt64Size( - this->raft_node_id()); + ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual( + *this->replica_); } } @@ -6704,8 +6710,8 @@ void Lease::MergeFrom(const Lease& from) { if (from.has_expiration()) { mutable_expiration()->::cockroach::proto::Timestamp::MergeFrom(from.expiration()); } - if (from.has_raft_node_id()) { - set_raft_node_id(from.raft_node_id()); + if (from.has_replica()) { + mutable_replica()->::cockroach::proto::ReplicaDescriptor::MergeFrom(from.replica()); } } if (from._internal_metadata_.have_unknown_fields()) { @@ -6737,7 +6743,7 @@ void Lease::Swap(Lease* other) { void Lease::InternalSwap(Lease* other) { std::swap(start_, other->start_); std::swap(expiration_, other->expiration_); - std::swap(raft_node_id_, other->raft_node_id_); + std::swap(replica_, other->replica_); std::swap(_has_bits_[0], other->_has_bits_[0]); _internal_metadata_.Swap(&other->_internal_metadata_); std::swap(_cached_size_, other->_cached_size_); @@ -6840,28 +6846,47 @@ void Lease::clear_expiration() { // @@protoc_insertion_point(field_set_allocated:cockroach.proto.Lease.expiration) } -// optional uint64 raft_node_id = 3; -bool Lease::has_raft_node_id() const { +// optional .cockroach.proto.ReplicaDescriptor replica = 3; +bool Lease::has_replica() const { return (_has_bits_[0] & 0x00000004u) != 0; } -void Lease::set_has_raft_node_id() { +void Lease::set_has_replica() { _has_bits_[0] |= 0x00000004u; } -void Lease::clear_has_raft_node_id() { +void Lease::clear_has_replica() { _has_bits_[0] &= ~0x00000004u; } -void Lease::clear_raft_node_id() { - raft_node_id_ = GOOGLE_ULONGLONG(0); - clear_has_raft_node_id(); +void Lease::clear_replica() { + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); + clear_has_replica(); } - ::google::protobuf::uint64 Lease::raft_node_id() const { - // @@protoc_insertion_point(field_get:cockroach.proto.Lease.raft_node_id) - return raft_node_id_; + const ::cockroach::proto::ReplicaDescriptor& Lease::replica() const { + // @@protoc_insertion_point(field_get:cockroach.proto.Lease.replica) + return replica_ != NULL ? *replica_ : *default_instance_->replica_; +} + ::cockroach::proto::ReplicaDescriptor* Lease::mutable_replica() { + set_has_replica(); + if (replica_ == NULL) { + replica_ = new ::cockroach::proto::ReplicaDescriptor; + } + // @@protoc_insertion_point(field_mutable:cockroach.proto.Lease.replica) + return replica_; } - void Lease::set_raft_node_id(::google::protobuf::uint64 value) { - set_has_raft_node_id(); - raft_node_id_ = value; - // @@protoc_insertion_point(field_set:cockroach.proto.Lease.raft_node_id) + ::cockroach::proto::ReplicaDescriptor* Lease::release_replica() { + clear_has_replica(); + ::cockroach::proto::ReplicaDescriptor* temp = replica_; + replica_ = NULL; + return temp; +} + void Lease::set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica) { + delete replica_; + replica_ = replica; + if (replica) { + set_has_replica(); + } else { + clear_has_replica(); + } + // @@protoc_insertion_point(field_set_allocated:cockroach.proto.Lease.replica) } #endif // PROTOBUF_INLINE_NOT_IN_HEADERS diff --git a/storage/engine/rocksdb/cockroach/proto/data.pb.h b/storage/engine/rocksdb/cockroach/proto/data.pb.h index e38ddb98a679..9bb2b16c2744 100644 --- a/storage/engine/rocksdb/cockroach/proto/data.pb.h +++ b/storage/engine/rocksdb/cockroach/proto/data.pb.h @@ -982,25 +982,25 @@ class ChangeReplicasTrigger : public ::google::protobuf::Message { ::cockroach::proto::ReplicaChangeType change_type() const; void set_change_type(::cockroach::proto::ReplicaChangeType value); - // optional .cockroach.proto.Replica replica = 4; + // optional .cockroach.proto.ReplicaDescriptor replica = 4; bool has_replica() const; void clear_replica(); static const int kReplicaFieldNumber = 4; - const ::cockroach::proto::Replica& replica() const; - ::cockroach::proto::Replica* mutable_replica(); - ::cockroach::proto::Replica* release_replica(); - void set_allocated_replica(::cockroach::proto::Replica* replica); + const ::cockroach::proto::ReplicaDescriptor& replica() const; + ::cockroach::proto::ReplicaDescriptor* mutable_replica(); + ::cockroach::proto::ReplicaDescriptor* release_replica(); + void set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica); - // repeated .cockroach.proto.Replica updated_replicas = 5; + // repeated .cockroach.proto.ReplicaDescriptor updated_replicas = 5; int updated_replicas_size() const; void clear_updated_replicas(); static const int kUpdatedReplicasFieldNumber = 5; - const ::cockroach::proto::Replica& updated_replicas(int index) const; - ::cockroach::proto::Replica* mutable_updated_replicas(int index); - ::cockroach::proto::Replica* add_updated_replicas(); - const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >& + const ::cockroach::proto::ReplicaDescriptor& updated_replicas(int index) const; + ::cockroach::proto::ReplicaDescriptor* mutable_updated_replicas(int index); + ::cockroach::proto::ReplicaDescriptor* add_updated_replicas(); + const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >& updated_replicas() const; - ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >* + ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >* mutable_updated_replicas(); // optional int32 next_replica_id = 6; @@ -1028,10 +1028,10 @@ class ChangeReplicasTrigger : public ::google::protobuf::Message { mutable int _cached_size_; ::google::protobuf::int32 node_id_; ::google::protobuf::int32 store_id_; - ::cockroach::proto::Replica* replica_; + ::cockroach::proto::ReplicaDescriptor* replica_; int change_type_; ::google::protobuf::int32 next_replica_id_; - ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica > updated_replicas_; + ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor > updated_replicas_; friend void protobuf_AddDesc_cockroach_2fproto_2fdata_2eproto(); friend void protobuf_AssignDesc_cockroach_2fproto_2fdata_2eproto(); friend void protobuf_ShutdownFile_cockroach_2fproto_2fdata_2eproto(); @@ -1666,12 +1666,14 @@ class Lease : public ::google::protobuf::Message { ::cockroach::proto::Timestamp* release_expiration(); void set_allocated_expiration(::cockroach::proto::Timestamp* expiration); - // optional uint64 raft_node_id = 3; - bool has_raft_node_id() const; - void clear_raft_node_id(); - static const int kRaftNodeIdFieldNumber = 3; - ::google::protobuf::uint64 raft_node_id() const; - void set_raft_node_id(::google::protobuf::uint64 value); + // optional .cockroach.proto.ReplicaDescriptor replica = 3; + bool has_replica() const; + void clear_replica(); + static const int kReplicaFieldNumber = 3; + const ::cockroach::proto::ReplicaDescriptor& replica() const; + ::cockroach::proto::ReplicaDescriptor* mutable_replica(); + ::cockroach::proto::ReplicaDescriptor* release_replica(); + void set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica); // @@protoc_insertion_point(class_scope:cockroach.proto.Lease) private: @@ -1679,15 +1681,15 @@ class Lease : public ::google::protobuf::Message { inline void clear_has_start(); inline void set_has_expiration(); inline void clear_has_expiration(); - inline void set_has_raft_node_id(); - inline void clear_has_raft_node_id(); + inline void set_has_replica(); + inline void clear_has_replica(); ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; ::google::protobuf::uint32 _has_bits_[1]; mutable int _cached_size_; ::cockroach::proto::Timestamp* start_; ::cockroach::proto::Timestamp* expiration_; - ::google::protobuf::uint64 raft_node_id_; + ::cockroach::proto::ReplicaDescriptor* replica_; friend void protobuf_AddDesc_cockroach_2fproto_2fdata_2eproto(); friend void protobuf_AssignDesc_cockroach_2fproto_2fdata_2eproto(); friend void protobuf_ShutdownFile_cockroach_2fproto_2fdata_2eproto(); @@ -2673,7 +2675,7 @@ inline void ChangeReplicasTrigger::set_change_type(::cockroach::proto::ReplicaCh // @@protoc_insertion_point(field_set:cockroach.proto.ChangeReplicasTrigger.change_type) } -// optional .cockroach.proto.Replica replica = 4; +// optional .cockroach.proto.ReplicaDescriptor replica = 4; inline bool ChangeReplicasTrigger::has_replica() const { return (_has_bits_[0] & 0x00000008u) != 0; } @@ -2684,28 +2686,28 @@ inline void ChangeReplicasTrigger::clear_has_replica() { _has_bits_[0] &= ~0x00000008u; } inline void ChangeReplicasTrigger::clear_replica() { - if (replica_ != NULL) replica_->::cockroach::proto::Replica::Clear(); + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); clear_has_replica(); } -inline const ::cockroach::proto::Replica& ChangeReplicasTrigger::replica() const { +inline const ::cockroach::proto::ReplicaDescriptor& ChangeReplicasTrigger::replica() const { // @@protoc_insertion_point(field_get:cockroach.proto.ChangeReplicasTrigger.replica) return replica_ != NULL ? *replica_ : *default_instance_->replica_; } -inline ::cockroach::proto::Replica* ChangeReplicasTrigger::mutable_replica() { +inline ::cockroach::proto::ReplicaDescriptor* ChangeReplicasTrigger::mutable_replica() { set_has_replica(); if (replica_ == NULL) { - replica_ = new ::cockroach::proto::Replica; + replica_ = new ::cockroach::proto::ReplicaDescriptor; } // @@protoc_insertion_point(field_mutable:cockroach.proto.ChangeReplicasTrigger.replica) return replica_; } -inline ::cockroach::proto::Replica* ChangeReplicasTrigger::release_replica() { +inline ::cockroach::proto::ReplicaDescriptor* ChangeReplicasTrigger::release_replica() { clear_has_replica(); - ::cockroach::proto::Replica* temp = replica_; + ::cockroach::proto::ReplicaDescriptor* temp = replica_; replica_ = NULL; return temp; } -inline void ChangeReplicasTrigger::set_allocated_replica(::cockroach::proto::Replica* replica) { +inline void ChangeReplicasTrigger::set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica) { delete replica_; replica_ = replica; if (replica) { @@ -2716,31 +2718,31 @@ inline void ChangeReplicasTrigger::set_allocated_replica(::cockroach::proto::Rep // @@protoc_insertion_point(field_set_allocated:cockroach.proto.ChangeReplicasTrigger.replica) } -// repeated .cockroach.proto.Replica updated_replicas = 5; +// repeated .cockroach.proto.ReplicaDescriptor updated_replicas = 5; inline int ChangeReplicasTrigger::updated_replicas_size() const { return updated_replicas_.size(); } inline void ChangeReplicasTrigger::clear_updated_replicas() { updated_replicas_.Clear(); } -inline const ::cockroach::proto::Replica& ChangeReplicasTrigger::updated_replicas(int index) const { +inline const ::cockroach::proto::ReplicaDescriptor& ChangeReplicasTrigger::updated_replicas(int index) const { // @@protoc_insertion_point(field_get:cockroach.proto.ChangeReplicasTrigger.updated_replicas) return updated_replicas_.Get(index); } -inline ::cockroach::proto::Replica* ChangeReplicasTrigger::mutable_updated_replicas(int index) { +inline ::cockroach::proto::ReplicaDescriptor* ChangeReplicasTrigger::mutable_updated_replicas(int index) { // @@protoc_insertion_point(field_mutable:cockroach.proto.ChangeReplicasTrigger.updated_replicas) return updated_replicas_.Mutable(index); } -inline ::cockroach::proto::Replica* ChangeReplicasTrigger::add_updated_replicas() { +inline ::cockroach::proto::ReplicaDescriptor* ChangeReplicasTrigger::add_updated_replicas() { // @@protoc_insertion_point(field_add:cockroach.proto.ChangeReplicasTrigger.updated_replicas) return updated_replicas_.Add(); } -inline const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >& +inline const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >& ChangeReplicasTrigger::updated_replicas() const { // @@protoc_insertion_point(field_list:cockroach.proto.ChangeReplicasTrigger.updated_replicas) return updated_replicas_; } -inline ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >* +inline ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >* ChangeReplicasTrigger::mutable_updated_replicas() { // @@protoc_insertion_point(field_mutable_list:cockroach.proto.ChangeReplicasTrigger.updated_replicas) return &updated_replicas_; @@ -3598,28 +3600,47 @@ inline void Lease::set_allocated_expiration(::cockroach::proto::Timestamp* expir // @@protoc_insertion_point(field_set_allocated:cockroach.proto.Lease.expiration) } -// optional uint64 raft_node_id = 3; -inline bool Lease::has_raft_node_id() const { +// optional .cockroach.proto.ReplicaDescriptor replica = 3; +inline bool Lease::has_replica() const { return (_has_bits_[0] & 0x00000004u) != 0; } -inline void Lease::set_has_raft_node_id() { +inline void Lease::set_has_replica() { _has_bits_[0] |= 0x00000004u; } -inline void Lease::clear_has_raft_node_id() { +inline void Lease::clear_has_replica() { _has_bits_[0] &= ~0x00000004u; } -inline void Lease::clear_raft_node_id() { - raft_node_id_ = GOOGLE_ULONGLONG(0); - clear_has_raft_node_id(); +inline void Lease::clear_replica() { + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); + clear_has_replica(); +} +inline const ::cockroach::proto::ReplicaDescriptor& Lease::replica() const { + // @@protoc_insertion_point(field_get:cockroach.proto.Lease.replica) + return replica_ != NULL ? *replica_ : *default_instance_->replica_; +} +inline ::cockroach::proto::ReplicaDescriptor* Lease::mutable_replica() { + set_has_replica(); + if (replica_ == NULL) { + replica_ = new ::cockroach::proto::ReplicaDescriptor; + } + // @@protoc_insertion_point(field_mutable:cockroach.proto.Lease.replica) + return replica_; } -inline ::google::protobuf::uint64 Lease::raft_node_id() const { - // @@protoc_insertion_point(field_get:cockroach.proto.Lease.raft_node_id) - return raft_node_id_; +inline ::cockroach::proto::ReplicaDescriptor* Lease::release_replica() { + clear_has_replica(); + ::cockroach::proto::ReplicaDescriptor* temp = replica_; + replica_ = NULL; + return temp; } -inline void Lease::set_raft_node_id(::google::protobuf::uint64 value) { - set_has_raft_node_id(); - raft_node_id_ = value; - // @@protoc_insertion_point(field_set:cockroach.proto.Lease.raft_node_id) +inline void Lease::set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica) { + delete replica_; + replica_ = replica; + if (replica) { + set_has_replica(); + } else { + clear_has_replica(); + } + // @@protoc_insertion_point(field_set_allocated:cockroach.proto.Lease.replica) } // ------------------------------------------------------------------- diff --git a/storage/engine/rocksdb/cockroach/proto/errors.pb.cc b/storage/engine/rocksdb/cockroach/proto/errors.pb.cc index 8cbf9f6ad2c6..ea8043285e9f 100644 --- a/storage/engine/rocksdb/cockroach/proto/errors.pb.cc +++ b/storage/engine/rocksdb/cockroach/proto/errors.pb.cc @@ -491,79 +491,79 @@ void protobuf_AddDesc_cockroach_2fproto_2ferrors_2eproto() { "\n\034cockroach/proto/errors.proto\022\017cockroac" "h.proto\032\036cockroach/proto/metadata.proto\032" "\032cockroach/proto/data.proto\032\024gogoproto/g" - "ogo.proto\"\223\001\n\016NotLeaderError\022)\n\007replica\030" - "\001 \001(\0132\030.cockroach.proto.Replica\022(\n\006leade" - "r\030\002 \001(\0132\030.cockroach.proto.Replica\022,\n\010ran" - "ge_id\030\003 \001(\003B\032\310\336\037\000\342\336\037\007RangeID\372\336\037\007RangeID\"" - "\026\n\024NodeUnavailableError\"B\n\022RangeNotFound" - "Error\022,\n\010range_id\030\001 \001(\003B\032\310\336\037\000\342\336\037\007RangeID" - "\372\336\037\007RangeID\"\216\001\n\025RangeKeyMismatchError\022\"\n" - "\021request_start_key\030\001 \001(\014B\007\372\336\037\003Key\022 \n\017req" - "uest_end_key\030\002 \001(\014B\007\372\336\037\003Key\022/\n\005range\030\003 \001" - "(\0132 .cockroach.proto.RangeDescriptor\"\363\001\n" - "\"ReadWithinUncertaintyIntervalError\0223\n\tt" - "imestamp\030\001 \001(\0132\032.cockroach.proto.Timesta" - "mpB\004\310\336\037\000\022<\n\022existing_timestamp\030\002 \001(\0132\032.c" - "ockroach.proto.TimestampB\004\310\336\037\000\022)\n\007node_i" - "d\030\003 \001(\005B\030\310\336\037\000\342\336\037\006NodeID\372\336\037\006NodeID\022/\n\003txn" - "\030\004 \001(\0132\034.cockroach.proto.TransactionB\004\310\336" - "\037\000\"J\n\027TransactionAbortedError\022/\n\003txn\030\001 \001" - "(\0132\034.cockroach.proto.TransactionB\004\310\336\037\000\"y" - "\n\024TransactionPushError\022)\n\003txn\030\001 \001(\0132\034.co" - "ckroach.proto.Transaction\0226\n\npushee_txn\030" - "\002 \001(\0132\034.cockroach.proto.TransactionB\004\310\336\037" - "\000\"H\n\025TransactionRetryError\022/\n\003txn\030\001 \001(\0132" - "\034.cockroach.proto.TransactionB\004\310\336\037\000\"\\\n\026T" - "ransactionStatusError\022/\n\003txn\030\001 \001(\0132\034.coc" - "kroach.proto.TransactionB\004\310\336\037\000\022\021\n\003msg\030\002 " - "\001(\tB\004\310\336\037\000\"Z\n\020WriteIntentError\022.\n\007intents" - "\030\001 \003(\0132\027.cockroach.proto.IntentB\004\310\336\037\000\022\026\n" - "\010resolved\030\002 \001(\010B\004\310\336\037\000\"\205\001\n\020WriteTooOldErr" - "or\0223\n\ttimestamp\030\001 \001(\0132\032.cockroach.proto." - "TimestampB\004\310\336\037\000\022<\n\022existing_timestamp\030\002 " - "\001(\0132\032.cockroach.proto.TimestampB\004\310\336\037\000\"\024\n" - "\022OpRequiresTxnError\"q\n\024ConditionFailedEr" - "ror\022,\n\014actual_value\030\001 \001(\0132\026.cockroach.pr" - "oto.Value\022+\n\005index\030\002 \001(\0132\034.cockroach.pro" - "to.ErrPosition\"u\n\022LeaseRejectedError\022/\n\t" - "Requested\030\001 \001(\0132\026.cockroach.proto.LeaseB" - "\004\310\336\037\000\022.\n\010Existing\030\002 \001(\0132\026.cockroach.prot" - "o.LeaseB\004\310\336\037\000\";\n\tSendError\022\025\n\007message\030\001 " - "\001(\tB\004\310\336\037\000\022\027\n\tretryable\030\002 \001(\010B\004\310\336\037\000\"\323\007\n\013E" - "rrorDetail\0223\n\nnot_leader\030\001 \001(\0132\037.cockroa" - "ch.proto.NotLeaderError\022<\n\017range_not_fou" - "nd\030\002 \001(\0132#.cockroach.proto.RangeNotFound" - "Error\022B\n\022range_key_mismatch\030\003 \001(\0132&.cock" - "roach.proto.RangeKeyMismatchError\022]\n rea" - "d_within_uncertainty_interval\030\004 \001(\01323.co" - "ckroach.proto.ReadWithinUncertaintyInter" - "valError\022E\n\023transaction_aborted\030\005 \001(\0132(." - "cockroach.proto.TransactionAbortedError\022" - "\?\n\020transaction_push\030\006 \001(\0132%.cockroach.pr" - "oto.TransactionPushError\022A\n\021transaction_" - "retry\030\007 \001(\0132&.cockroach.proto.Transactio" - "nRetryError\022C\n\022transaction_status\030\010 \001(\0132" - "\'.cockroach.proto.TransactionStatusError" - "\0227\n\014write_intent\030\t \001(\0132!.cockroach.proto" - ".WriteIntentError\0228\n\rwrite_too_old\030\n \001(\013" - "2!.cockroach.proto.WriteTooOldError\022<\n\017o" - "p_requires_txn\030\013 \001(\0132#.cockroach.proto.O" - "pRequiresTxnError\022\?\n\020condition_failed\030\014 " - "\001(\0132%.cockroach.proto.ConditionFailedErr" - "or\022;\n\016lease_rejected\030\r \001(\0132#.cockroach.p" - "roto.LeaseRejectedError\022\?\n\020node_unavaila" - "ble\030\016 \001(\0132%.cockroach.proto.NodeUnavaila" - "bleError\022(\n\004send\030\017 \001(\0132\032.cockroach.proto" - ".SendError:\004\310\240\037\001\"\"\n\013ErrPosition\022\023\n\005index" - "\030\001 \001(\005B\004\310\336\037\000\"\332\001\n\005Error\022\025\n\007message\030\001 \001(\tB" - "\004\310\336\037\000\022\027\n\tretryable\030\002 \001(\010B\004\310\336\037\000\022F\n\023transa" - "ction_restart\030\003 \001(\0162#.cockroach.proto.Tr" - "ansactionRestartB\004\310\336\037\000\022,\n\006detail\030\004 \001(\0132\034" - ".cockroach.proto.ErrorDetail\022+\n\005index\030\005 " - "\001(\0132\034.cockroach.proto.ErrPosition*;\n\022Tra" - "nsactionRestart\022\t\n\005ABORT\020\000\022\013\n\007BACKOFF\020\001\022" - "\r\n\tIMMEDIATE\020\002B\033Z\005proto\330\341\036\000\340\342\036\001\310\342\036\001\320\342\036\001\220" - "\343\036\000", 3003); + "ogo.proto\"\247\001\n\016NotLeaderError\0223\n\007replica\030" + "\001 \001(\0132\".cockroach.proto.ReplicaDescripto" + "r\0222\n\006leader\030\002 \001(\0132\".cockroach.proto.Repl" + "icaDescriptor\022,\n\010range_id\030\003 \001(\003B\032\310\336\037\000\342\336\037" + "\007RangeID\372\336\037\007RangeID\"\026\n\024NodeUnavailableEr" + "ror\"B\n\022RangeNotFoundError\022,\n\010range_id\030\001 " + "\001(\003B\032\310\336\037\000\342\336\037\007RangeID\372\336\037\007RangeID\"\216\001\n\025Rang" + "eKeyMismatchError\022\"\n\021request_start_key\030\001" + " \001(\014B\007\372\336\037\003Key\022 \n\017request_end_key\030\002 \001(\014B\007" + "\372\336\037\003Key\022/\n\005range\030\003 \001(\0132 .cockroach.proto" + ".RangeDescriptor\"\363\001\n\"ReadWithinUncertain" + "tyIntervalError\0223\n\ttimestamp\030\001 \001(\0132\032.coc" + "kroach.proto.TimestampB\004\310\336\037\000\022<\n\022existing" + "_timestamp\030\002 \001(\0132\032.cockroach.proto.Times" + "tampB\004\310\336\037\000\022)\n\007node_id\030\003 \001(\005B\030\310\336\037\000\342\336\037\006Nod" + "eID\372\336\037\006NodeID\022/\n\003txn\030\004 \001(\0132\034.cockroach.p" + "roto.TransactionB\004\310\336\037\000\"J\n\027TransactionAbo" + "rtedError\022/\n\003txn\030\001 \001(\0132\034.cockroach.proto" + ".TransactionB\004\310\336\037\000\"y\n\024TransactionPushErr" + "or\022)\n\003txn\030\001 \001(\0132\034.cockroach.proto.Transa" + "ction\0226\n\npushee_txn\030\002 \001(\0132\034.cockroach.pr" + "oto.TransactionB\004\310\336\037\000\"H\n\025TransactionRetr" + "yError\022/\n\003txn\030\001 \001(\0132\034.cockroach.proto.Tr" + "ansactionB\004\310\336\037\000\"\\\n\026TransactionStatusErro" + "r\022/\n\003txn\030\001 \001(\0132\034.cockroach.proto.Transac" + "tionB\004\310\336\037\000\022\021\n\003msg\030\002 \001(\tB\004\310\336\037\000\"Z\n\020WriteIn" + "tentError\022.\n\007intents\030\001 \003(\0132\027.cockroach.p" + "roto.IntentB\004\310\336\037\000\022\026\n\010resolved\030\002 \001(\010B\004\310\336\037" + "\000\"\205\001\n\020WriteTooOldError\0223\n\ttimestamp\030\001 \001(" + "\0132\032.cockroach.proto.TimestampB\004\310\336\037\000\022<\n\022e" + "xisting_timestamp\030\002 \001(\0132\032.cockroach.prot" + "o.TimestampB\004\310\336\037\000\"\024\n\022OpRequiresTxnError\"" + "q\n\024ConditionFailedError\022,\n\014actual_value\030" + "\001 \001(\0132\026.cockroach.proto.Value\022+\n\005index\030\002" + " \001(\0132\034.cockroach.proto.ErrPosition\"u\n\022Le" + "aseRejectedError\022/\n\tRequested\030\001 \001(\0132\026.co" + "ckroach.proto.LeaseB\004\310\336\037\000\022.\n\010Existing\030\002 " + "\001(\0132\026.cockroach.proto.LeaseB\004\310\336\037\000\";\n\tSen" + "dError\022\025\n\007message\030\001 \001(\tB\004\310\336\037\000\022\027\n\tretryab" + "le\030\002 \001(\010B\004\310\336\037\000\"\323\007\n\013ErrorDetail\0223\n\nnot_le" + "ader\030\001 \001(\0132\037.cockroach.proto.NotLeaderEr" + "ror\022<\n\017range_not_found\030\002 \001(\0132#.cockroach" + ".proto.RangeNotFoundError\022B\n\022range_key_m" + "ismatch\030\003 \001(\0132&.cockroach.proto.RangeKey" + "MismatchError\022]\n read_within_uncertainty" + "_interval\030\004 \001(\01323.cockroach.proto.ReadWi" + "thinUncertaintyIntervalError\022E\n\023transact" + "ion_aborted\030\005 \001(\0132(.cockroach.proto.Tran" + "sactionAbortedError\022\?\n\020transaction_push\030" + "\006 \001(\0132%.cockroach.proto.TransactionPushE" + "rror\022A\n\021transaction_retry\030\007 \001(\0132&.cockro" + "ach.proto.TransactionRetryError\022C\n\022trans" + "action_status\030\010 \001(\0132\'.cockroach.proto.Tr" + "ansactionStatusError\0227\n\014write_intent\030\t \001" + "(\0132!.cockroach.proto.WriteIntentError\0228\n" + "\rwrite_too_old\030\n \001(\0132!.cockroach.proto.W" + "riteTooOldError\022<\n\017op_requires_txn\030\013 \001(\013" + "2#.cockroach.proto.OpRequiresTxnError\022\?\n" + "\020condition_failed\030\014 \001(\0132%.cockroach.prot" + "o.ConditionFailedError\022;\n\016lease_rejected" + "\030\r \001(\0132#.cockroach.proto.LeaseRejectedEr" + "ror\022\?\n\020node_unavailable\030\016 \001(\0132%.cockroac" + "h.proto.NodeUnavailableError\022(\n\004send\030\017 \001" + "(\0132\032.cockroach.proto.SendError:\004\310\240\037\001\"\"\n\013" + "ErrPosition\022\023\n\005index\030\001 \001(\005B\004\310\336\037\000\"\332\001\n\005Err" + "or\022\025\n\007message\030\001 \001(\tB\004\310\336\037\000\022\027\n\tretryable\030\002" + " \001(\010B\004\310\336\037\000\022F\n\023transaction_restart\030\003 \001(\0162" + "#.cockroach.proto.TransactionRestartB\004\310\336" + "\037\000\022,\n\006detail\030\004 \001(\0132\034.cockroach.proto.Err" + "orDetail\022+\n\005index\030\005 \001(\0132\034.cockroach.prot" + "o.ErrPosition*;\n\022TransactionRestart\022\t\n\005A" + "BORT\020\000\022\013\n\007BACKOFF\020\001\022\r\n\tIMMEDIATE\020\002B\033Z\005pr" + "oto\330\341\036\000\340\342\036\001\310\342\036\001\320\342\036\001\220\343\036\000", 3023); ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile( "cockroach/proto/errors.proto", &protobuf_RegisterTypes); NotLeaderError::default_instance_ = new NotLeaderError(); @@ -652,8 +652,8 @@ NotLeaderError::NotLeaderError() } void NotLeaderError::InitAsDefaultInstance() { - replica_ = const_cast< ::cockroach::proto::Replica*>(&::cockroach::proto::Replica::default_instance()); - leader_ = const_cast< ::cockroach::proto::Replica*>(&::cockroach::proto::Replica::default_instance()); + replica_ = const_cast< ::cockroach::proto::ReplicaDescriptor*>(&::cockroach::proto::ReplicaDescriptor::default_instance()); + leader_ = const_cast< ::cockroach::proto::ReplicaDescriptor*>(&::cockroach::proto::ReplicaDescriptor::default_instance()); } NotLeaderError::NotLeaderError(const NotLeaderError& from) @@ -712,10 +712,10 @@ NotLeaderError* NotLeaderError::New(::google::protobuf::Arena* arena) const { void NotLeaderError::Clear() { if (_has_bits_[0 / 32] & 7u) { if (has_replica()) { - if (replica_ != NULL) replica_->::cockroach::proto::Replica::Clear(); + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); } if (has_leader()) { - if (leader_ != NULL) leader_->::cockroach::proto::Replica::Clear(); + if (leader_ != NULL) leader_->::cockroach::proto::ReplicaDescriptor::Clear(); } range_id_ = GOOGLE_LONGLONG(0); } @@ -735,7 +735,7 @@ bool NotLeaderError::MergePartialFromCodedStream( tag = p.first; if (!p.second) goto handle_unusual; switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) { - // optional .cockroach.proto.Replica replica = 1; + // optional .cockroach.proto.ReplicaDescriptor replica = 1; case 1: { if (tag == 10) { DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual( @@ -747,7 +747,7 @@ bool NotLeaderError::MergePartialFromCodedStream( break; } - // optional .cockroach.proto.Replica leader = 2; + // optional .cockroach.proto.ReplicaDescriptor leader = 2; case 2: { if (tag == 18) { parse_leader: @@ -800,13 +800,13 @@ bool NotLeaderError::MergePartialFromCodedStream( void NotLeaderError::SerializeWithCachedSizes( ::google::protobuf::io::CodedOutputStream* output) const { // @@protoc_insertion_point(serialize_start:cockroach.proto.NotLeaderError) - // optional .cockroach.proto.Replica replica = 1; + // optional .cockroach.proto.ReplicaDescriptor replica = 1; if (has_replica()) { ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( 1, *this->replica_, output); } - // optional .cockroach.proto.Replica leader = 2; + // optional .cockroach.proto.ReplicaDescriptor leader = 2; if (has_leader()) { ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( 2, *this->leader_, output); @@ -827,14 +827,14 @@ void NotLeaderError::SerializeWithCachedSizes( ::google::protobuf::uint8* NotLeaderError::SerializeWithCachedSizesToArray( ::google::protobuf::uint8* target) const { // @@protoc_insertion_point(serialize_to_array_start:cockroach.proto.NotLeaderError) - // optional .cockroach.proto.Replica replica = 1; + // optional .cockroach.proto.ReplicaDescriptor replica = 1; if (has_replica()) { target = ::google::protobuf::internal::WireFormatLite:: WriteMessageNoVirtualToArray( 1, *this->replica_, target); } - // optional .cockroach.proto.Replica leader = 2; + // optional .cockroach.proto.ReplicaDescriptor leader = 2; if (has_leader()) { target = ::google::protobuf::internal::WireFormatLite:: WriteMessageNoVirtualToArray( @@ -858,14 +858,14 @@ int NotLeaderError::ByteSize() const { int total_size = 0; if (_has_bits_[0 / 32] & 7) { - // optional .cockroach.proto.Replica replica = 1; + // optional .cockroach.proto.ReplicaDescriptor replica = 1; if (has_replica()) { total_size += 1 + ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual( *this->replica_); } - // optional .cockroach.proto.Replica leader = 2; + // optional .cockroach.proto.ReplicaDescriptor leader = 2; if (has_leader()) { total_size += 1 + ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual( @@ -907,10 +907,10 @@ void NotLeaderError::MergeFrom(const NotLeaderError& from) { if (GOOGLE_PREDICT_FALSE(&from == this)) MergeFromFail(__LINE__); if (from._has_bits_[0 / 32] & (0xffu << (0 % 32))) { if (from.has_replica()) { - mutable_replica()->::cockroach::proto::Replica::MergeFrom(from.replica()); + mutable_replica()->::cockroach::proto::ReplicaDescriptor::MergeFrom(from.replica()); } if (from.has_leader()) { - mutable_leader()->::cockroach::proto::Replica::MergeFrom(from.leader()); + mutable_leader()->::cockroach::proto::ReplicaDescriptor::MergeFrom(from.leader()); } if (from.has_range_id()) { set_range_id(from.range_id()); @@ -962,7 +962,7 @@ ::google::protobuf::Metadata NotLeaderError::GetMetadata() const { #if PROTOBUF_INLINE_NOT_IN_HEADERS // NotLeaderError -// optional .cockroach.proto.Replica replica = 1; +// optional .cockroach.proto.ReplicaDescriptor replica = 1; bool NotLeaderError::has_replica() const { return (_has_bits_[0] & 0x00000001u) != 0; } @@ -973,28 +973,28 @@ void NotLeaderError::clear_has_replica() { _has_bits_[0] &= ~0x00000001u; } void NotLeaderError::clear_replica() { - if (replica_ != NULL) replica_->::cockroach::proto::Replica::Clear(); + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); clear_has_replica(); } - const ::cockroach::proto::Replica& NotLeaderError::replica() const { + const ::cockroach::proto::ReplicaDescriptor& NotLeaderError::replica() const { // @@protoc_insertion_point(field_get:cockroach.proto.NotLeaderError.replica) return replica_ != NULL ? *replica_ : *default_instance_->replica_; } - ::cockroach::proto::Replica* NotLeaderError::mutable_replica() { + ::cockroach::proto::ReplicaDescriptor* NotLeaderError::mutable_replica() { set_has_replica(); if (replica_ == NULL) { - replica_ = new ::cockroach::proto::Replica; + replica_ = new ::cockroach::proto::ReplicaDescriptor; } // @@protoc_insertion_point(field_mutable:cockroach.proto.NotLeaderError.replica) return replica_; } - ::cockroach::proto::Replica* NotLeaderError::release_replica() { + ::cockroach::proto::ReplicaDescriptor* NotLeaderError::release_replica() { clear_has_replica(); - ::cockroach::proto::Replica* temp = replica_; + ::cockroach::proto::ReplicaDescriptor* temp = replica_; replica_ = NULL; return temp; } - void NotLeaderError::set_allocated_replica(::cockroach::proto::Replica* replica) { + void NotLeaderError::set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica) { delete replica_; replica_ = replica; if (replica) { @@ -1005,7 +1005,7 @@ void NotLeaderError::clear_replica() { // @@protoc_insertion_point(field_set_allocated:cockroach.proto.NotLeaderError.replica) } -// optional .cockroach.proto.Replica leader = 2; +// optional .cockroach.proto.ReplicaDescriptor leader = 2; bool NotLeaderError::has_leader() const { return (_has_bits_[0] & 0x00000002u) != 0; } @@ -1016,28 +1016,28 @@ void NotLeaderError::clear_has_leader() { _has_bits_[0] &= ~0x00000002u; } void NotLeaderError::clear_leader() { - if (leader_ != NULL) leader_->::cockroach::proto::Replica::Clear(); + if (leader_ != NULL) leader_->::cockroach::proto::ReplicaDescriptor::Clear(); clear_has_leader(); } - const ::cockroach::proto::Replica& NotLeaderError::leader() const { + const ::cockroach::proto::ReplicaDescriptor& NotLeaderError::leader() const { // @@protoc_insertion_point(field_get:cockroach.proto.NotLeaderError.leader) return leader_ != NULL ? *leader_ : *default_instance_->leader_; } - ::cockroach::proto::Replica* NotLeaderError::mutable_leader() { + ::cockroach::proto::ReplicaDescriptor* NotLeaderError::mutable_leader() { set_has_leader(); if (leader_ == NULL) { - leader_ = new ::cockroach::proto::Replica; + leader_ = new ::cockroach::proto::ReplicaDescriptor; } // @@protoc_insertion_point(field_mutable:cockroach.proto.NotLeaderError.leader) return leader_; } - ::cockroach::proto::Replica* NotLeaderError::release_leader() { + ::cockroach::proto::ReplicaDescriptor* NotLeaderError::release_leader() { clear_has_leader(); - ::cockroach::proto::Replica* temp = leader_; + ::cockroach::proto::ReplicaDescriptor* temp = leader_; leader_ = NULL; return temp; } - void NotLeaderError::set_allocated_leader(::cockroach::proto::Replica* leader) { + void NotLeaderError::set_allocated_leader(::cockroach::proto::ReplicaDescriptor* leader) { delete leader_; leader_ = leader; if (leader) { diff --git a/storage/engine/rocksdb/cockroach/proto/errors.pb.h b/storage/engine/rocksdb/cockroach/proto/errors.pb.h index c9bb93844f91..73a930a82055 100644 --- a/storage/engine/rocksdb/cockroach/proto/errors.pb.h +++ b/storage/engine/rocksdb/cockroach/proto/errors.pb.h @@ -146,23 +146,23 @@ class NotLeaderError : public ::google::protobuf::Message { // accessors ------------------------------------------------------- - // optional .cockroach.proto.Replica replica = 1; + // optional .cockroach.proto.ReplicaDescriptor replica = 1; bool has_replica() const; void clear_replica(); static const int kReplicaFieldNumber = 1; - const ::cockroach::proto::Replica& replica() const; - ::cockroach::proto::Replica* mutable_replica(); - ::cockroach::proto::Replica* release_replica(); - void set_allocated_replica(::cockroach::proto::Replica* replica); + const ::cockroach::proto::ReplicaDescriptor& replica() const; + ::cockroach::proto::ReplicaDescriptor* mutable_replica(); + ::cockroach::proto::ReplicaDescriptor* release_replica(); + void set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica); - // optional .cockroach.proto.Replica leader = 2; + // optional .cockroach.proto.ReplicaDescriptor leader = 2; bool has_leader() const; void clear_leader(); static const int kLeaderFieldNumber = 2; - const ::cockroach::proto::Replica& leader() const; - ::cockroach::proto::Replica* mutable_leader(); - ::cockroach::proto::Replica* release_leader(); - void set_allocated_leader(::cockroach::proto::Replica* leader); + const ::cockroach::proto::ReplicaDescriptor& leader() const; + ::cockroach::proto::ReplicaDescriptor* mutable_leader(); + ::cockroach::proto::ReplicaDescriptor* release_leader(); + void set_allocated_leader(::cockroach::proto::ReplicaDescriptor* leader); // optional int64 range_id = 3; bool has_range_id() const; @@ -183,8 +183,8 @@ class NotLeaderError : public ::google::protobuf::Message { ::google::protobuf::internal::InternalMetadataWithArena _internal_metadata_; ::google::protobuf::uint32 _has_bits_[1]; mutable int _cached_size_; - ::cockroach::proto::Replica* replica_; - ::cockroach::proto::Replica* leader_; + ::cockroach::proto::ReplicaDescriptor* replica_; + ::cockroach::proto::ReplicaDescriptor* leader_; ::google::protobuf::int64 range_id_; friend void protobuf_AddDesc_cockroach_2fproto_2ferrors_2eproto(); friend void protobuf_AssignDesc_cockroach_2fproto_2ferrors_2eproto(); @@ -2086,7 +2086,7 @@ class Error : public ::google::protobuf::Message { #if !PROTOBUF_INLINE_NOT_IN_HEADERS // NotLeaderError -// optional .cockroach.proto.Replica replica = 1; +// optional .cockroach.proto.ReplicaDescriptor replica = 1; inline bool NotLeaderError::has_replica() const { return (_has_bits_[0] & 0x00000001u) != 0; } @@ -2097,28 +2097,28 @@ inline void NotLeaderError::clear_has_replica() { _has_bits_[0] &= ~0x00000001u; } inline void NotLeaderError::clear_replica() { - if (replica_ != NULL) replica_->::cockroach::proto::Replica::Clear(); + if (replica_ != NULL) replica_->::cockroach::proto::ReplicaDescriptor::Clear(); clear_has_replica(); } -inline const ::cockroach::proto::Replica& NotLeaderError::replica() const { +inline const ::cockroach::proto::ReplicaDescriptor& NotLeaderError::replica() const { // @@protoc_insertion_point(field_get:cockroach.proto.NotLeaderError.replica) return replica_ != NULL ? *replica_ : *default_instance_->replica_; } -inline ::cockroach::proto::Replica* NotLeaderError::mutable_replica() { +inline ::cockroach::proto::ReplicaDescriptor* NotLeaderError::mutable_replica() { set_has_replica(); if (replica_ == NULL) { - replica_ = new ::cockroach::proto::Replica; + replica_ = new ::cockroach::proto::ReplicaDescriptor; } // @@protoc_insertion_point(field_mutable:cockroach.proto.NotLeaderError.replica) return replica_; } -inline ::cockroach::proto::Replica* NotLeaderError::release_replica() { +inline ::cockroach::proto::ReplicaDescriptor* NotLeaderError::release_replica() { clear_has_replica(); - ::cockroach::proto::Replica* temp = replica_; + ::cockroach::proto::ReplicaDescriptor* temp = replica_; replica_ = NULL; return temp; } -inline void NotLeaderError::set_allocated_replica(::cockroach::proto::Replica* replica) { +inline void NotLeaderError::set_allocated_replica(::cockroach::proto::ReplicaDescriptor* replica) { delete replica_; replica_ = replica; if (replica) { @@ -2129,7 +2129,7 @@ inline void NotLeaderError::set_allocated_replica(::cockroach::proto::Replica* r // @@protoc_insertion_point(field_set_allocated:cockroach.proto.NotLeaderError.replica) } -// optional .cockroach.proto.Replica leader = 2; +// optional .cockroach.proto.ReplicaDescriptor leader = 2; inline bool NotLeaderError::has_leader() const { return (_has_bits_[0] & 0x00000002u) != 0; } @@ -2140,28 +2140,28 @@ inline void NotLeaderError::clear_has_leader() { _has_bits_[0] &= ~0x00000002u; } inline void NotLeaderError::clear_leader() { - if (leader_ != NULL) leader_->::cockroach::proto::Replica::Clear(); + if (leader_ != NULL) leader_->::cockroach::proto::ReplicaDescriptor::Clear(); clear_has_leader(); } -inline const ::cockroach::proto::Replica& NotLeaderError::leader() const { +inline const ::cockroach::proto::ReplicaDescriptor& NotLeaderError::leader() const { // @@protoc_insertion_point(field_get:cockroach.proto.NotLeaderError.leader) return leader_ != NULL ? *leader_ : *default_instance_->leader_; } -inline ::cockroach::proto::Replica* NotLeaderError::mutable_leader() { +inline ::cockroach::proto::ReplicaDescriptor* NotLeaderError::mutable_leader() { set_has_leader(); if (leader_ == NULL) { - leader_ = new ::cockroach::proto::Replica; + leader_ = new ::cockroach::proto::ReplicaDescriptor; } // @@protoc_insertion_point(field_mutable:cockroach.proto.NotLeaderError.leader) return leader_; } -inline ::cockroach::proto::Replica* NotLeaderError::release_leader() { +inline ::cockroach::proto::ReplicaDescriptor* NotLeaderError::release_leader() { clear_has_leader(); - ::cockroach::proto::Replica* temp = leader_; + ::cockroach::proto::ReplicaDescriptor* temp = leader_; leader_ = NULL; return temp; } -inline void NotLeaderError::set_allocated_leader(::cockroach::proto::Replica* leader) { +inline void NotLeaderError::set_allocated_leader(::cockroach::proto::ReplicaDescriptor* leader) { delete leader_; leader_ = leader; if (leader) { diff --git a/storage/engine/rocksdb/cockroach/proto/internal.pb.cc b/storage/engine/rocksdb/cockroach/proto/internal.pb.cc index a4e0edac6f9e..9385de04b726 100644 --- a/storage/engine/rocksdb/cockroach/proto/internal.pb.cc +++ b/storage/engine/rocksdb/cockroach/proto/internal.pb.cc @@ -52,7 +52,7 @@ void protobuf_AssignDesc_cockroach_2fproto_2finternal_2eproto() { RaftCommand_descriptor_ = file->message_type(0); static const int RaftCommand_offsets_[3] = { GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RaftCommand, range_id_), - GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RaftCommand, origin_node_id_), + GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RaftCommand, origin_replica_), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RaftCommand, cmd_), }; RaftCommand_reflection_ = @@ -206,25 +206,25 @@ void protobuf_AddDesc_cockroach_2fproto_2finternal_2eproto() { "\n\036cockroach/proto/internal.proto\022\017cockro" "ach.proto\032\031cockroach/proto/api.proto\032\036co" "ckroach/proto/metadata.proto\032\024gogoproto/" - "gogo.proto\"\251\001\n\013RaftCommand\022,\n\010range_id\030\001" - " \001(\003B\032\310\336\037\000\342\336\037\007RangeID\372\336\037\007RangeID\022:\n\016orig" - "in_node_id\030\002 \001(\004B\"\310\336\037\000\342\336\037\014OriginNodeID\372\336" - "\037\nRaftNodeID\0220\n\003cmd\030\003 \001(\0132\035.cockroach.pr" - "oto.BatchRequestB\004\310\336\037\000\"\236\001\n\026InternalTimeS" - "eriesData\022#\n\025start_timestamp_nanos\030\001 \001(\003" - "B\004\310\336\037\000\022#\n\025sample_duration_nanos\030\002 \001(\003B\004\310" - "\336\037\000\022:\n\007samples\030\003 \003(\0132).cockroach.proto.I" - "nternalTimeSeriesSample\"r\n\030InternalTimeS" - "eriesSample\022\024\n\006offset\030\001 \001(\005B\004\310\336\037\000\022\023\n\005cou" - "nt\030\006 \001(\rB\004\310\336\037\000\022\021\n\003sum\030\007 \001(\001B\004\310\336\037\000\022\013\n\003max" - "\030\010 \001(\001\022\013\n\003min\030\t \001(\001\"=\n\022RaftTruncatedStat" - "e\022\023\n\005index\030\001 \001(\004B\004\310\336\037\000\022\022\n\004term\030\002 \001(\004B\004\310\336" - "\037\000\"\274\001\n\020RaftSnapshotData\022@\n\020range_descrip" - "tor\030\001 \001(\0132 .cockroach.proto.RangeDescrip" - "torB\004\310\336\037\000\022>\n\002KV\030\002 \003(\0132*.cockroach.proto." - "RaftSnapshotData.KeyValueB\006\342\336\037\002KV\032&\n\010Key" - "Value\022\013\n\003key\030\001 \001(\014\022\r\n\005value\030\002 \001(\014B\027Z\005pro" - "to\340\342\036\001\310\342\036\001\320\342\036\001\220\343\036\000", 858); + "gogo.proto\"\257\001\n\013RaftCommand\022,\n\010range_id\030\001" + " \001(\003B\032\310\336\037\000\342\336\037\007RangeID\372\336\037\007RangeID\022@\n\016orig" + "in_replica\030\002 \001(\0132\".cockroach.proto.Repli" + "caDescriptorB\004\310\336\037\000\0220\n\003cmd\030\003 \001(\0132\035.cockro" + "ach.proto.BatchRequestB\004\310\336\037\000\"\236\001\n\026Interna" + "lTimeSeriesData\022#\n\025start_timestamp_nanos" + "\030\001 \001(\003B\004\310\336\037\000\022#\n\025sample_duration_nanos\030\002 " + "\001(\003B\004\310\336\037\000\022:\n\007samples\030\003 \003(\0132).cockroach.p" + "roto.InternalTimeSeriesSample\"r\n\030Interna" + "lTimeSeriesSample\022\024\n\006offset\030\001 \001(\005B\004\310\336\037\000\022" + "\023\n\005count\030\006 \001(\rB\004\310\336\037\000\022\021\n\003sum\030\007 \001(\001B\004\310\336\037\000\022" + "\013\n\003max\030\010 \001(\001\022\013\n\003min\030\t \001(\001\"=\n\022RaftTruncat" + "edState\022\023\n\005index\030\001 \001(\004B\004\310\336\037\000\022\022\n\004term\030\002 \001" + "(\004B\004\310\336\037\000\"\274\001\n\020RaftSnapshotData\022@\n\020range_d" + "escriptor\030\001 \001(\0132 .cockroach.proto.RangeD" + "escriptorB\004\310\336\037\000\022>\n\002KV\030\002 \003(\0132*.cockroach." + "proto.RaftSnapshotData.KeyValueB\006\342\336\037\002KV\032" + "&\n\010KeyValue\022\013\n\003key\030\001 \001(\014\022\r\n\005value\030\002 \001(\014B" + "\027Z\005proto\340\342\036\001\310\342\036\001\320\342\036\001\220\343\036\000", 864); ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile( "cockroach/proto/internal.proto", &protobuf_RegisterTypes); RaftCommand::default_instance_ = new RaftCommand(); @@ -263,7 +263,7 @@ static void MergeFromFail(int line) { #ifndef _MSC_VER const int RaftCommand::kRangeIdFieldNumber; -const int RaftCommand::kOriginNodeIdFieldNumber; +const int RaftCommand::kOriginReplicaFieldNumber; const int RaftCommand::kCmdFieldNumber; #endif // !_MSC_VER @@ -274,6 +274,7 @@ RaftCommand::RaftCommand() } void RaftCommand::InitAsDefaultInstance() { + origin_replica_ = const_cast< ::cockroach::proto::ReplicaDescriptor*>(&::cockroach::proto::ReplicaDescriptor::default_instance()); cmd_ = const_cast< ::cockroach::proto::BatchRequest*>(&::cockroach::proto::BatchRequest::default_instance()); } @@ -288,7 +289,7 @@ RaftCommand::RaftCommand(const RaftCommand& from) void RaftCommand::SharedCtor() { _cached_size_ = 0; range_id_ = GOOGLE_LONGLONG(0); - origin_node_id_ = GOOGLE_ULONGLONG(0); + origin_replica_ = NULL; cmd_ = NULL; ::memset(_has_bits_, 0, sizeof(_has_bits_)); } @@ -300,6 +301,7 @@ RaftCommand::~RaftCommand() { void RaftCommand::SharedDtor() { if (this != default_instance_) { + delete origin_replica_; delete cmd_; } } @@ -330,24 +332,15 @@ RaftCommand* RaftCommand::New(::google::protobuf::Arena* arena) const { } void RaftCommand::Clear() { -#define ZR_HELPER_(f) reinterpret_cast(\ - &reinterpret_cast(16)->f) - -#define ZR_(first, last) do {\ - ::memset(&first, 0,\ - ZR_HELPER_(last) - ZR_HELPER_(first) + sizeof(last));\ -} while (0) - if (_has_bits_[0 / 32] & 7u) { - ZR_(range_id_, origin_node_id_); + range_id_ = GOOGLE_LONGLONG(0); + if (has_origin_replica()) { + if (origin_replica_ != NULL) origin_replica_->::cockroach::proto::ReplicaDescriptor::Clear(); + } if (has_cmd()) { if (cmd_ != NULL) cmd_->::cockroach::proto::BatchRequest::Clear(); } } - -#undef ZR_HELPER_ -#undef ZR_ - ::memset(_has_bits_, 0, sizeof(_has_bits_)); if (_internal_metadata_.have_unknown_fields()) { mutable_unknown_fields()->Clear(); @@ -374,18 +367,16 @@ bool RaftCommand::MergePartialFromCodedStream( } else { goto handle_unusual; } - if (input->ExpectTag(16)) goto parse_origin_node_id; + if (input->ExpectTag(18)) goto parse_origin_replica; break; } - // optional uint64 origin_node_id = 2; + // optional .cockroach.proto.ReplicaDescriptor origin_replica = 2; case 2: { - if (tag == 16) { - parse_origin_node_id: - DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive< - ::google::protobuf::uint64, ::google::protobuf::internal::WireFormatLite::TYPE_UINT64>( - input, &origin_node_id_))); - set_has_origin_node_id(); + if (tag == 18) { + parse_origin_replica: + DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual( + input, mutable_origin_replica())); } else { goto handle_unusual; } @@ -436,9 +427,10 @@ void RaftCommand::SerializeWithCachedSizes( ::google::protobuf::internal::WireFormatLite::WriteInt64(1, this->range_id(), output); } - // optional uint64 origin_node_id = 2; - if (has_origin_node_id()) { - ::google::protobuf::internal::WireFormatLite::WriteUInt64(2, this->origin_node_id(), output); + // optional .cockroach.proto.ReplicaDescriptor origin_replica = 2; + if (has_origin_replica()) { + ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( + 2, *this->origin_replica_, output); } // optional .cockroach.proto.BatchRequest cmd = 3; @@ -462,9 +454,11 @@ ::google::protobuf::uint8* RaftCommand::SerializeWithCachedSizesToArray( target = ::google::protobuf::internal::WireFormatLite::WriteInt64ToArray(1, this->range_id(), target); } - // optional uint64 origin_node_id = 2; - if (has_origin_node_id()) { - target = ::google::protobuf::internal::WireFormatLite::WriteUInt64ToArray(2, this->origin_node_id(), target); + // optional .cockroach.proto.ReplicaDescriptor origin_replica = 2; + if (has_origin_replica()) { + target = ::google::protobuf::internal::WireFormatLite:: + WriteMessageNoVirtualToArray( + 2, *this->origin_replica_, target); } // optional .cockroach.proto.BatchRequest cmd = 3; @@ -493,11 +487,11 @@ int RaftCommand::ByteSize() const { this->range_id()); } - // optional uint64 origin_node_id = 2; - if (has_origin_node_id()) { + // optional .cockroach.proto.ReplicaDescriptor origin_replica = 2; + if (has_origin_replica()) { total_size += 1 + - ::google::protobuf::internal::WireFormatLite::UInt64Size( - this->origin_node_id()); + ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual( + *this->origin_replica_); } // optional .cockroach.proto.BatchRequest cmd = 3; @@ -537,8 +531,8 @@ void RaftCommand::MergeFrom(const RaftCommand& from) { if (from.has_range_id()) { set_range_id(from.range_id()); } - if (from.has_origin_node_id()) { - set_origin_node_id(from.origin_node_id()); + if (from.has_origin_replica()) { + mutable_origin_replica()->::cockroach::proto::ReplicaDescriptor::MergeFrom(from.origin_replica()); } if (from.has_cmd()) { mutable_cmd()->::cockroach::proto::BatchRequest::MergeFrom(from.cmd()); @@ -572,7 +566,7 @@ void RaftCommand::Swap(RaftCommand* other) { } void RaftCommand::InternalSwap(RaftCommand* other) { std::swap(range_id_, other->range_id_); - std::swap(origin_node_id_, other->origin_node_id_); + std::swap(origin_replica_, other->origin_replica_); std::swap(cmd_, other->cmd_); std::swap(_has_bits_[0], other->_has_bits_[0]); _internal_metadata_.Swap(&other->_internal_metadata_); @@ -614,28 +608,47 @@ void RaftCommand::clear_range_id() { // @@protoc_insertion_point(field_set:cockroach.proto.RaftCommand.range_id) } -// optional uint64 origin_node_id = 2; -bool RaftCommand::has_origin_node_id() const { +// optional .cockroach.proto.ReplicaDescriptor origin_replica = 2; +bool RaftCommand::has_origin_replica() const { return (_has_bits_[0] & 0x00000002u) != 0; } -void RaftCommand::set_has_origin_node_id() { +void RaftCommand::set_has_origin_replica() { _has_bits_[0] |= 0x00000002u; } -void RaftCommand::clear_has_origin_node_id() { +void RaftCommand::clear_has_origin_replica() { _has_bits_[0] &= ~0x00000002u; } -void RaftCommand::clear_origin_node_id() { - origin_node_id_ = GOOGLE_ULONGLONG(0); - clear_has_origin_node_id(); +void RaftCommand::clear_origin_replica() { + if (origin_replica_ != NULL) origin_replica_->::cockroach::proto::ReplicaDescriptor::Clear(); + clear_has_origin_replica(); +} + const ::cockroach::proto::ReplicaDescriptor& RaftCommand::origin_replica() const { + // @@protoc_insertion_point(field_get:cockroach.proto.RaftCommand.origin_replica) + return origin_replica_ != NULL ? *origin_replica_ : *default_instance_->origin_replica_; } - ::google::protobuf::uint64 RaftCommand::origin_node_id() const { - // @@protoc_insertion_point(field_get:cockroach.proto.RaftCommand.origin_node_id) - return origin_node_id_; + ::cockroach::proto::ReplicaDescriptor* RaftCommand::mutable_origin_replica() { + set_has_origin_replica(); + if (origin_replica_ == NULL) { + origin_replica_ = new ::cockroach::proto::ReplicaDescriptor; + } + // @@protoc_insertion_point(field_mutable:cockroach.proto.RaftCommand.origin_replica) + return origin_replica_; } - void RaftCommand::set_origin_node_id(::google::protobuf::uint64 value) { - set_has_origin_node_id(); - origin_node_id_ = value; - // @@protoc_insertion_point(field_set:cockroach.proto.RaftCommand.origin_node_id) + ::cockroach::proto::ReplicaDescriptor* RaftCommand::release_origin_replica() { + clear_has_origin_replica(); + ::cockroach::proto::ReplicaDescriptor* temp = origin_replica_; + origin_replica_ = NULL; + return temp; +} + void RaftCommand::set_allocated_origin_replica(::cockroach::proto::ReplicaDescriptor* origin_replica) { + delete origin_replica_; + origin_replica_ = origin_replica; + if (origin_replica) { + set_has_origin_replica(); + } else { + clear_has_origin_replica(); + } + // @@protoc_insertion_point(field_set_allocated:cockroach.proto.RaftCommand.origin_replica) } // optional .cockroach.proto.BatchRequest cmd = 3; diff --git a/storage/engine/rocksdb/cockroach/proto/internal.pb.h b/storage/engine/rocksdb/cockroach/proto/internal.pb.h index b32c31dc2234..a5440f6a14ff 100644 --- a/storage/engine/rocksdb/cockroach/proto/internal.pb.h +++ b/storage/engine/rocksdb/cockroach/proto/internal.pb.h @@ -120,12 +120,14 @@ class RaftCommand : public ::google::protobuf::Message { ::google::protobuf::int64 range_id() const; void set_range_id(::google::protobuf::int64 value); - // optional uint64 origin_node_id = 2; - bool has_origin_node_id() const; - void clear_origin_node_id(); - static const int kOriginNodeIdFieldNumber = 2; - ::google::protobuf::uint64 origin_node_id() const; - void set_origin_node_id(::google::protobuf::uint64 value); + // optional .cockroach.proto.ReplicaDescriptor origin_replica = 2; + bool has_origin_replica() const; + void clear_origin_replica(); + static const int kOriginReplicaFieldNumber = 2; + const ::cockroach::proto::ReplicaDescriptor& origin_replica() const; + ::cockroach::proto::ReplicaDescriptor* mutable_origin_replica(); + ::cockroach::proto::ReplicaDescriptor* release_origin_replica(); + void set_allocated_origin_replica(::cockroach::proto::ReplicaDescriptor* origin_replica); // optional .cockroach.proto.BatchRequest cmd = 3; bool has_cmd() const; @@ -140,8 +142,8 @@ class RaftCommand : public ::google::protobuf::Message { private: inline void set_has_range_id(); inline void clear_has_range_id(); - inline void set_has_origin_node_id(); - inline void clear_has_origin_node_id(); + inline void set_has_origin_replica(); + inline void clear_has_origin_replica(); inline void set_has_cmd(); inline void clear_has_cmd(); @@ -149,7 +151,7 @@ class RaftCommand : public ::google::protobuf::Message { ::google::protobuf::uint32 _has_bits_[1]; mutable int _cached_size_; ::google::protobuf::int64 range_id_; - ::google::protobuf::uint64 origin_node_id_; + ::cockroach::proto::ReplicaDescriptor* origin_replica_; ::cockroach::proto::BatchRequest* cmd_; friend void protobuf_AddDesc_cockroach_2fproto_2finternal_2eproto(); friend void protobuf_AssignDesc_cockroach_2fproto_2finternal_2eproto(); @@ -745,28 +747,47 @@ inline void RaftCommand::set_range_id(::google::protobuf::int64 value) { // @@protoc_insertion_point(field_set:cockroach.proto.RaftCommand.range_id) } -// optional uint64 origin_node_id = 2; -inline bool RaftCommand::has_origin_node_id() const { +// optional .cockroach.proto.ReplicaDescriptor origin_replica = 2; +inline bool RaftCommand::has_origin_replica() const { return (_has_bits_[0] & 0x00000002u) != 0; } -inline void RaftCommand::set_has_origin_node_id() { +inline void RaftCommand::set_has_origin_replica() { _has_bits_[0] |= 0x00000002u; } -inline void RaftCommand::clear_has_origin_node_id() { +inline void RaftCommand::clear_has_origin_replica() { _has_bits_[0] &= ~0x00000002u; } -inline void RaftCommand::clear_origin_node_id() { - origin_node_id_ = GOOGLE_ULONGLONG(0); - clear_has_origin_node_id(); +inline void RaftCommand::clear_origin_replica() { + if (origin_replica_ != NULL) origin_replica_->::cockroach::proto::ReplicaDescriptor::Clear(); + clear_has_origin_replica(); } -inline ::google::protobuf::uint64 RaftCommand::origin_node_id() const { - // @@protoc_insertion_point(field_get:cockroach.proto.RaftCommand.origin_node_id) - return origin_node_id_; +inline const ::cockroach::proto::ReplicaDescriptor& RaftCommand::origin_replica() const { + // @@protoc_insertion_point(field_get:cockroach.proto.RaftCommand.origin_replica) + return origin_replica_ != NULL ? *origin_replica_ : *default_instance_->origin_replica_; } -inline void RaftCommand::set_origin_node_id(::google::protobuf::uint64 value) { - set_has_origin_node_id(); - origin_node_id_ = value; - // @@protoc_insertion_point(field_set:cockroach.proto.RaftCommand.origin_node_id) +inline ::cockroach::proto::ReplicaDescriptor* RaftCommand::mutable_origin_replica() { + set_has_origin_replica(); + if (origin_replica_ == NULL) { + origin_replica_ = new ::cockroach::proto::ReplicaDescriptor; + } + // @@protoc_insertion_point(field_mutable:cockroach.proto.RaftCommand.origin_replica) + return origin_replica_; +} +inline ::cockroach::proto::ReplicaDescriptor* RaftCommand::release_origin_replica() { + clear_has_origin_replica(); + ::cockroach::proto::ReplicaDescriptor* temp = origin_replica_; + origin_replica_ = NULL; + return temp; +} +inline void RaftCommand::set_allocated_origin_replica(::cockroach::proto::ReplicaDescriptor* origin_replica) { + delete origin_replica_; + origin_replica_ = origin_replica; + if (origin_replica) { + set_has_origin_replica(); + } else { + clear_has_origin_replica(); + } + // @@protoc_insertion_point(field_set_allocated:cockroach.proto.RaftCommand.origin_replica) } // optional .cockroach.proto.BatchRequest cmd = 3; diff --git a/storage/engine/rocksdb/cockroach/proto/metadata.pb.cc b/storage/engine/rocksdb/cockroach/proto/metadata.pb.cc index cadf2844c3d7..a80336e716e1 100644 --- a/storage/engine/rocksdb/cockroach/proto/metadata.pb.cc +++ b/storage/engine/rocksdb/cockroach/proto/metadata.pb.cc @@ -24,9 +24,9 @@ namespace { const ::google::protobuf::Descriptor* Attributes_descriptor_ = NULL; const ::google::protobuf::internal::GeneratedMessageReflection* Attributes_reflection_ = NULL; -const ::google::protobuf::Descriptor* Replica_descriptor_ = NULL; +const ::google::protobuf::Descriptor* ReplicaDescriptor_descriptor_ = NULL; const ::google::protobuf::internal::GeneratedMessageReflection* - Replica_reflection_ = NULL; + ReplicaDescriptor_reflection_ = NULL; const ::google::protobuf::Descriptor* RangeDescriptor_descriptor_ = NULL; const ::google::protobuf::internal::GeneratedMessageReflection* RangeDescriptor_reflection_ = NULL; @@ -70,22 +70,22 @@ void protobuf_AssignDesc_cockroach_2fproto_2fmetadata_2eproto() { sizeof(Attributes), GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Attributes, _internal_metadata_), -1); - Replica_descriptor_ = file->message_type(1); - static const int Replica_offsets_[3] = { - GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Replica, node_id_), - GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Replica, store_id_), - GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Replica, replica_id_), + ReplicaDescriptor_descriptor_ = file->message_type(1); + static const int ReplicaDescriptor_offsets_[3] = { + GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ReplicaDescriptor, node_id_), + GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ReplicaDescriptor, store_id_), + GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ReplicaDescriptor, replica_id_), }; - Replica_reflection_ = + ReplicaDescriptor_reflection_ = ::google::protobuf::internal::GeneratedMessageReflection::NewGeneratedMessageReflection( - Replica_descriptor_, - Replica::default_instance_, - Replica_offsets_, - GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Replica, _has_bits_[0]), + ReplicaDescriptor_descriptor_, + ReplicaDescriptor::default_instance_, + ReplicaDescriptor_offsets_, + GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ReplicaDescriptor, _has_bits_[0]), -1, -1, - sizeof(Replica), - GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Replica, _internal_metadata_), + sizeof(ReplicaDescriptor), + GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ReplicaDescriptor, _internal_metadata_), -1); RangeDescriptor_descriptor_ = file->message_type(2); static const int RangeDescriptor_offsets_[5] = { @@ -207,7 +207,7 @@ void protobuf_RegisterTypes(const ::std::string&) { ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage( Attributes_descriptor_, &Attributes::default_instance()); ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage( - Replica_descriptor_, &Replica::default_instance()); + ReplicaDescriptor_descriptor_, &ReplicaDescriptor::default_instance()); ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage( RangeDescriptor_descriptor_, &RangeDescriptor::default_instance()); ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage( @@ -227,8 +227,8 @@ void protobuf_RegisterTypes(const ::std::string&) { void protobuf_ShutdownFile_cockroach_2fproto_2fmetadata_2eproto() { delete Attributes::default_instance_; delete Attributes_reflection_; - delete Replica::default_instance_; - delete Replica_reflection_; + delete ReplicaDescriptor::default_instance_; + delete ReplicaDescriptor_reflection_; delete RangeDescriptor::default_instance_; delete RangeDescriptor_reflection_; delete RangeTree::default_instance_; @@ -256,38 +256,38 @@ void protobuf_AddDesc_cockroach_2fproto_2fmetadata_2eproto() { "ach.proto\032\024gogoproto/gogo.proto\032$cockroa" "ch/util/unresolved_addr.proto\"8\n\nAttribu" "tes\022$\n\005attrs\030\001 \003(\tB\025\362\336\037\021yaml:\"attrs,flow" - "\":\004\230\240\037\000\"\226\001\n\007Replica\022)\n\007node_id\030\001 \001(\005B\030\310\336" - "\037\000\342\336\037\006NodeID\372\336\037\006NodeID\022,\n\010store_id\030\002 \001(\005" - "B\032\310\336\037\000\342\336\037\007StoreID\372\336\037\007StoreID\0222\n\nreplica_" - "id\030\003 \001(\005B\036\310\336\037\000\342\336\037\tReplicaID\372\336\037\tReplicaID" - "\"\344\001\n\017RangeDescriptor\022,\n\010range_id\030\001 \001(\003B\032" - "\310\336\037\000\342\336\037\007RangeID\372\336\037\007RangeID\022\032\n\tstart_key\030" - "\002 \001(\014B\007\372\336\037\003Key\022\030\n\007end_key\030\003 \001(\014B\007\372\336\037\003Key" - "\0220\n\010replicas\030\004 \003(\0132\030.cockroach.proto.Rep" - "licaB\004\310\336\037\000\022;\n\017next_replica_id\030\005 \001(\005B\"\310\336\037" - "\000\342\336\037\rNextReplicaID\372\336\037\tReplicaID\"&\n\tRange" - "Tree\022\031\n\010root_key\030\001 \001(\014B\007\372\336\037\003Key\"\216\001\n\rRang" - "eTreeNode\022\024\n\003key\030\001 \001(\014B\007\372\336\037\003Key\022\023\n\005black" - "\030\002 \001(\010B\004\310\336\037\000\022\033\n\nparent_key\030\003 \001(\014B\007\372\336\037\003Ke" - "y\022\031\n\010left_key\030\004 \001(\014B\007\372\336\037\003Key\022\032\n\tright_ke" - "y\030\005 \001(\014B\007\372\336\037\003Key\"Z\n\rStoreCapacity\022\026\n\010Cap" - "acity\030\001 \001(\003B\004\310\336\037\000\022\027\n\tAvailable\030\002 \001(\003B\004\310\336" - "\037\000\022\030\n\nRangeCount\030\003 \001(\005B\004\310\336\037\000\"\244\001\n\016NodeDes" - "criptor\022)\n\007node_id\030\001 \001(\005B\030\310\336\037\000\342\336\037\006NodeID" - "\372\336\037\006NodeID\0225\n\007address\030\002 \001(\0132\036.cockroach." - "util.UnresolvedAddrB\004\310\336\037\000\0220\n\005attrs\030\003 \001(\013" - "2\033.cockroach.proto.AttributesB\004\310\336\037\000\"\336\001\n\017" - "StoreDescriptor\022,\n\010store_id\030\001 \001(\005B\032\310\336\037\000\342" - "\336\037\007StoreID\372\336\037\007StoreID\0220\n\005attrs\030\002 \001(\0132\033.c" - "ockroach.proto.AttributesB\004\310\336\037\000\0223\n\004node\030" - "\003 \001(\0132\037.cockroach.proto.NodeDescriptorB\004" - "\310\336\037\000\0226\n\010capacity\030\004 \001(\0132\036.cockroach.proto" - ".StoreCapacityB\004\310\336\037\000B\027Z\005proto\340\342\036\001\310\342\036\001\320\342\036" - "\001\220\343\036\000", 1245); + "\":\004\230\240\037\000\"\240\001\n\021ReplicaDescriptor\022)\n\007node_id" + "\030\001 \001(\005B\030\310\336\037\000\342\336\037\006NodeID\372\336\037\006NodeID\022,\n\010stor" + "e_id\030\002 \001(\005B\032\310\336\037\000\342\336\037\007StoreID\372\336\037\007StoreID\0222" + "\n\nreplica_id\030\003 \001(\005B\036\310\336\037\000\342\336\037\tReplicaID\372\336\037" + "\tReplicaID\"\356\001\n\017RangeDescriptor\022,\n\010range_" + "id\030\001 \001(\003B\032\310\336\037\000\342\336\037\007RangeID\372\336\037\007RangeID\022\032\n\t" + "start_key\030\002 \001(\014B\007\372\336\037\003Key\022\030\n\007end_key\030\003 \001(" + "\014B\007\372\336\037\003Key\022:\n\010replicas\030\004 \003(\0132\".cockroach" + ".proto.ReplicaDescriptorB\004\310\336\037\000\022;\n\017next_r" + "eplica_id\030\005 \001(\005B\"\310\336\037\000\342\336\037\rNextReplicaID\372\336" + "\037\tReplicaID\"&\n\tRangeTree\022\031\n\010root_key\030\001 \001" + "(\014B\007\372\336\037\003Key\"\216\001\n\rRangeTreeNode\022\024\n\003key\030\001 \001" + "(\014B\007\372\336\037\003Key\022\023\n\005black\030\002 \001(\010B\004\310\336\037\000\022\033\n\npare" + "nt_key\030\003 \001(\014B\007\372\336\037\003Key\022\031\n\010left_key\030\004 \001(\014B" + "\007\372\336\037\003Key\022\032\n\tright_key\030\005 \001(\014B\007\372\336\037\003Key\"Z\n\r" + "StoreCapacity\022\026\n\010Capacity\030\001 \001(\003B\004\310\336\037\000\022\027\n" + "\tAvailable\030\002 \001(\003B\004\310\336\037\000\022\030\n\nRangeCount\030\003 \001" + "(\005B\004\310\336\037\000\"\244\001\n\016NodeDescriptor\022)\n\007node_id\030\001" + " \001(\005B\030\310\336\037\000\342\336\037\006NodeID\372\336\037\006NodeID\0225\n\007addres" + "s\030\002 \001(\0132\036.cockroach.util.UnresolvedAddrB" + "\004\310\336\037\000\0220\n\005attrs\030\003 \001(\0132\033.cockroach.proto.A" + "ttributesB\004\310\336\037\000\"\336\001\n\017StoreDescriptor\022,\n\010s" + "tore_id\030\001 \001(\005B\032\310\336\037\000\342\336\037\007StoreID\372\336\037\007StoreI" + "D\0220\n\005attrs\030\002 \001(\0132\033.cockroach.proto.Attri" + "butesB\004\310\336\037\000\0223\n\004node\030\003 \001(\0132\037.cockroach.pr" + "oto.NodeDescriptorB\004\310\336\037\000\0226\n\010capacity\030\004 \001" + "(\0132\036.cockroach.proto.StoreCapacityB\004\310\336\037\000" + "B\027Z\005proto\340\342\036\001\310\342\036\001\320\342\036\001\220\343\036\000", 1265); ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile( "cockroach/proto/metadata.proto", &protobuf_RegisterTypes); Attributes::default_instance_ = new Attributes(); - Replica::default_instance_ = new Replica(); + ReplicaDescriptor::default_instance_ = new ReplicaDescriptor(); RangeDescriptor::default_instance_ = new RangeDescriptor(); RangeTree::default_instance_ = new RangeTree(); RangeTreeNode::default_instance_ = new RangeTreeNode(); @@ -295,7 +295,7 @@ void protobuf_AddDesc_cockroach_2fproto_2fmetadata_2eproto() { NodeDescriptor::default_instance_ = new NodeDescriptor(); StoreDescriptor::default_instance_ = new StoreDescriptor(); Attributes::default_instance_->InitAsDefaultInstance(); - Replica::default_instance_->InitAsDefaultInstance(); + ReplicaDescriptor::default_instance_->InitAsDefaultInstance(); RangeDescriptor::default_instance_->InitAsDefaultInstance(); RangeTree::default_instance_->InitAsDefaultInstance(); RangeTreeNode::default_instance_->InitAsDefaultInstance(); @@ -625,29 +625,29 @@ Attributes::mutable_attrs() { // =================================================================== #ifndef _MSC_VER -const int Replica::kNodeIdFieldNumber; -const int Replica::kStoreIdFieldNumber; -const int Replica::kReplicaIdFieldNumber; +const int ReplicaDescriptor::kNodeIdFieldNumber; +const int ReplicaDescriptor::kStoreIdFieldNumber; +const int ReplicaDescriptor::kReplicaIdFieldNumber; #endif // !_MSC_VER -Replica::Replica() +ReplicaDescriptor::ReplicaDescriptor() : ::google::protobuf::Message(), _internal_metadata_(NULL) { SharedCtor(); - // @@protoc_insertion_point(constructor:cockroach.proto.Replica) + // @@protoc_insertion_point(constructor:cockroach.proto.ReplicaDescriptor) } -void Replica::InitAsDefaultInstance() { +void ReplicaDescriptor::InitAsDefaultInstance() { } -Replica::Replica(const Replica& from) +ReplicaDescriptor::ReplicaDescriptor(const ReplicaDescriptor& from) : ::google::protobuf::Message(), _internal_metadata_(NULL) { SharedCtor(); MergeFrom(from); - // @@protoc_insertion_point(copy_constructor:cockroach.proto.Replica) + // @@protoc_insertion_point(copy_constructor:cockroach.proto.ReplicaDescriptor) } -void Replica::SharedCtor() { +void ReplicaDescriptor::SharedCtor() { _cached_size_ = 0; node_id_ = 0; store_id_ = 0; @@ -655,44 +655,44 @@ void Replica::SharedCtor() { ::memset(_has_bits_, 0, sizeof(_has_bits_)); } -Replica::~Replica() { - // @@protoc_insertion_point(destructor:cockroach.proto.Replica) +ReplicaDescriptor::~ReplicaDescriptor() { + // @@protoc_insertion_point(destructor:cockroach.proto.ReplicaDescriptor) SharedDtor(); } -void Replica::SharedDtor() { +void ReplicaDescriptor::SharedDtor() { if (this != default_instance_) { } } -void Replica::SetCachedSize(int size) const { +void ReplicaDescriptor::SetCachedSize(int size) const { GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN(); _cached_size_ = size; GOOGLE_SAFE_CONCURRENT_WRITES_END(); } -const ::google::protobuf::Descriptor* Replica::descriptor() { +const ::google::protobuf::Descriptor* ReplicaDescriptor::descriptor() { protobuf_AssignDescriptorsOnce(); - return Replica_descriptor_; + return ReplicaDescriptor_descriptor_; } -const Replica& Replica::default_instance() { +const ReplicaDescriptor& ReplicaDescriptor::default_instance() { if (default_instance_ == NULL) protobuf_AddDesc_cockroach_2fproto_2fmetadata_2eproto(); return *default_instance_; } -Replica* Replica::default_instance_ = NULL; +ReplicaDescriptor* ReplicaDescriptor::default_instance_ = NULL; -Replica* Replica::New(::google::protobuf::Arena* arena) const { - Replica* n = new Replica; +ReplicaDescriptor* ReplicaDescriptor::New(::google::protobuf::Arena* arena) const { + ReplicaDescriptor* n = new ReplicaDescriptor; if (arena != NULL) { arena->Own(n); } return n; } -void Replica::Clear() { +void ReplicaDescriptor::Clear() { #define ZR_HELPER_(f) reinterpret_cast(\ - &reinterpret_cast(16)->f) + &reinterpret_cast(16)->f) #define ZR_(first, last) do {\ ::memset(&first, 0,\ @@ -710,11 +710,11 @@ void Replica::Clear() { } } -bool Replica::MergePartialFromCodedStream( +bool ReplicaDescriptor::MergePartialFromCodedStream( ::google::protobuf::io::CodedInputStream* input) { #define DO_(EXPRESSION) if (!(EXPRESSION)) goto failure ::google::protobuf::uint32 tag; - // @@protoc_insertion_point(parse_start:cockroach.proto.Replica) + // @@protoc_insertion_point(parse_start:cockroach.proto.ReplicaDescriptor) for (;;) { ::std::pair< ::google::protobuf::uint32, bool> p = input->ReadTagWithCutoff(127); tag = p.first; @@ -778,17 +778,17 @@ bool Replica::MergePartialFromCodedStream( } } success: - // @@protoc_insertion_point(parse_success:cockroach.proto.Replica) + // @@protoc_insertion_point(parse_success:cockroach.proto.ReplicaDescriptor) return true; failure: - // @@protoc_insertion_point(parse_failure:cockroach.proto.Replica) + // @@protoc_insertion_point(parse_failure:cockroach.proto.ReplicaDescriptor) return false; #undef DO_ } -void Replica::SerializeWithCachedSizes( +void ReplicaDescriptor::SerializeWithCachedSizes( ::google::protobuf::io::CodedOutputStream* output) const { - // @@protoc_insertion_point(serialize_start:cockroach.proto.Replica) + // @@protoc_insertion_point(serialize_start:cockroach.proto.ReplicaDescriptor) // optional int32 node_id = 1; if (has_node_id()) { ::google::protobuf::internal::WireFormatLite::WriteInt32(1, this->node_id(), output); @@ -808,12 +808,12 @@ void Replica::SerializeWithCachedSizes( ::google::protobuf::internal::WireFormat::SerializeUnknownFields( unknown_fields(), output); } - // @@protoc_insertion_point(serialize_end:cockroach.proto.Replica) + // @@protoc_insertion_point(serialize_end:cockroach.proto.ReplicaDescriptor) } -::google::protobuf::uint8* Replica::SerializeWithCachedSizesToArray( +::google::protobuf::uint8* ReplicaDescriptor::SerializeWithCachedSizesToArray( ::google::protobuf::uint8* target) const { - // @@protoc_insertion_point(serialize_to_array_start:cockroach.proto.Replica) + // @@protoc_insertion_point(serialize_to_array_start:cockroach.proto.ReplicaDescriptor) // optional int32 node_id = 1; if (has_node_id()) { target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(1, this->node_id(), target); @@ -833,11 +833,11 @@ ::google::protobuf::uint8* Replica::SerializeWithCachedSizesToArray( target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray( unknown_fields(), target); } - // @@protoc_insertion_point(serialize_to_array_end:cockroach.proto.Replica) + // @@protoc_insertion_point(serialize_to_array_end:cockroach.proto.ReplicaDescriptor) return target; } -int Replica::ByteSize() const { +int ReplicaDescriptor::ByteSize() const { int total_size = 0; if (_has_bits_[0 / 32] & 7) { @@ -874,10 +874,10 @@ int Replica::ByteSize() const { return total_size; } -void Replica::MergeFrom(const ::google::protobuf::Message& from) { +void ReplicaDescriptor::MergeFrom(const ::google::protobuf::Message& from) { if (GOOGLE_PREDICT_FALSE(&from == this)) MergeFromFail(__LINE__); - const Replica* source = - ::google::protobuf::internal::DynamicCastToGenerated( + const ReplicaDescriptor* source = + ::google::protobuf::internal::DynamicCastToGenerated( &from); if (source == NULL) { ::google::protobuf::internal::ReflectionOps::Merge(from, this); @@ -886,7 +886,7 @@ void Replica::MergeFrom(const ::google::protobuf::Message& from) { } } -void Replica::MergeFrom(const Replica& from) { +void ReplicaDescriptor::MergeFrom(const ReplicaDescriptor& from) { if (GOOGLE_PREDICT_FALSE(&from == this)) MergeFromFail(__LINE__); if (from._has_bits_[0 / 32] & (0xffu << (0 % 32))) { if (from.has_node_id()) { @@ -904,28 +904,28 @@ void Replica::MergeFrom(const Replica& from) { } } -void Replica::CopyFrom(const ::google::protobuf::Message& from) { +void ReplicaDescriptor::CopyFrom(const ::google::protobuf::Message& from) { if (&from == this) return; Clear(); MergeFrom(from); } -void Replica::CopyFrom(const Replica& from) { +void ReplicaDescriptor::CopyFrom(const ReplicaDescriptor& from) { if (&from == this) return; Clear(); MergeFrom(from); } -bool Replica::IsInitialized() const { +bool ReplicaDescriptor::IsInitialized() const { return true; } -void Replica::Swap(Replica* other) { +void ReplicaDescriptor::Swap(ReplicaDescriptor* other) { if (other == this) return; InternalSwap(other); } -void Replica::InternalSwap(Replica* other) { +void ReplicaDescriptor::InternalSwap(ReplicaDescriptor* other) { std::swap(node_id_, other->node_id_); std::swap(store_id_, other->store_id_); std::swap(replica_id_, other->replica_id_); @@ -934,87 +934,87 @@ void Replica::InternalSwap(Replica* other) { std::swap(_cached_size_, other->_cached_size_); } -::google::protobuf::Metadata Replica::GetMetadata() const { +::google::protobuf::Metadata ReplicaDescriptor::GetMetadata() const { protobuf_AssignDescriptorsOnce(); ::google::protobuf::Metadata metadata; - metadata.descriptor = Replica_descriptor_; - metadata.reflection = Replica_reflection_; + metadata.descriptor = ReplicaDescriptor_descriptor_; + metadata.reflection = ReplicaDescriptor_reflection_; return metadata; } #if PROTOBUF_INLINE_NOT_IN_HEADERS -// Replica +// ReplicaDescriptor // optional int32 node_id = 1; -bool Replica::has_node_id() const { +bool ReplicaDescriptor::has_node_id() const { return (_has_bits_[0] & 0x00000001u) != 0; } -void Replica::set_has_node_id() { +void ReplicaDescriptor::set_has_node_id() { _has_bits_[0] |= 0x00000001u; } -void Replica::clear_has_node_id() { +void ReplicaDescriptor::clear_has_node_id() { _has_bits_[0] &= ~0x00000001u; } -void Replica::clear_node_id() { +void ReplicaDescriptor::clear_node_id() { node_id_ = 0; clear_has_node_id(); } - ::google::protobuf::int32 Replica::node_id() const { - // @@protoc_insertion_point(field_get:cockroach.proto.Replica.node_id) + ::google::protobuf::int32 ReplicaDescriptor::node_id() const { + // @@protoc_insertion_point(field_get:cockroach.proto.ReplicaDescriptor.node_id) return node_id_; } - void Replica::set_node_id(::google::protobuf::int32 value) { + void ReplicaDescriptor::set_node_id(::google::protobuf::int32 value) { set_has_node_id(); node_id_ = value; - // @@protoc_insertion_point(field_set:cockroach.proto.Replica.node_id) + // @@protoc_insertion_point(field_set:cockroach.proto.ReplicaDescriptor.node_id) } // optional int32 store_id = 2; -bool Replica::has_store_id() const { +bool ReplicaDescriptor::has_store_id() const { return (_has_bits_[0] & 0x00000002u) != 0; } -void Replica::set_has_store_id() { +void ReplicaDescriptor::set_has_store_id() { _has_bits_[0] |= 0x00000002u; } -void Replica::clear_has_store_id() { +void ReplicaDescriptor::clear_has_store_id() { _has_bits_[0] &= ~0x00000002u; } -void Replica::clear_store_id() { +void ReplicaDescriptor::clear_store_id() { store_id_ = 0; clear_has_store_id(); } - ::google::protobuf::int32 Replica::store_id() const { - // @@protoc_insertion_point(field_get:cockroach.proto.Replica.store_id) + ::google::protobuf::int32 ReplicaDescriptor::store_id() const { + // @@protoc_insertion_point(field_get:cockroach.proto.ReplicaDescriptor.store_id) return store_id_; } - void Replica::set_store_id(::google::protobuf::int32 value) { + void ReplicaDescriptor::set_store_id(::google::protobuf::int32 value) { set_has_store_id(); store_id_ = value; - // @@protoc_insertion_point(field_set:cockroach.proto.Replica.store_id) + // @@protoc_insertion_point(field_set:cockroach.proto.ReplicaDescriptor.store_id) } // optional int32 replica_id = 3; -bool Replica::has_replica_id() const { +bool ReplicaDescriptor::has_replica_id() const { return (_has_bits_[0] & 0x00000004u) != 0; } -void Replica::set_has_replica_id() { +void ReplicaDescriptor::set_has_replica_id() { _has_bits_[0] |= 0x00000004u; } -void Replica::clear_has_replica_id() { +void ReplicaDescriptor::clear_has_replica_id() { _has_bits_[0] &= ~0x00000004u; } -void Replica::clear_replica_id() { +void ReplicaDescriptor::clear_replica_id() { replica_id_ = 0; clear_has_replica_id(); } - ::google::protobuf::int32 Replica::replica_id() const { - // @@protoc_insertion_point(field_get:cockroach.proto.Replica.replica_id) + ::google::protobuf::int32 ReplicaDescriptor::replica_id() const { + // @@protoc_insertion_point(field_get:cockroach.proto.ReplicaDescriptor.replica_id) return replica_id_; } - void Replica::set_replica_id(::google::protobuf::int32 value) { + void ReplicaDescriptor::set_replica_id(::google::protobuf::int32 value) { set_has_replica_id(); replica_id_ = value; - // @@protoc_insertion_point(field_set:cockroach.proto.Replica.replica_id) + // @@protoc_insertion_point(field_set:cockroach.proto.ReplicaDescriptor.replica_id) } #endif // PROTOBUF_INLINE_NOT_IN_HEADERS @@ -1161,7 +1161,7 @@ bool RangeDescriptor::MergePartialFromCodedStream( break; } - // repeated .cockroach.proto.Replica replicas = 4; + // repeated .cockroach.proto.ReplicaDescriptor replicas = 4; case 4: { if (tag == 34) { parse_replicas: @@ -1235,7 +1235,7 @@ void RangeDescriptor::SerializeWithCachedSizes( 3, this->end_key(), output); } - // repeated .cockroach.proto.Replica replicas = 4; + // repeated .cockroach.proto.ReplicaDescriptor replicas = 4; for (unsigned int i = 0, n = this->replicas_size(); i < n; i++) { ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray( 4, this->replicas(i), output); @@ -1275,7 +1275,7 @@ ::google::protobuf::uint8* RangeDescriptor::SerializeWithCachedSizesToArray( 3, this->end_key(), target); } - // repeated .cockroach.proto.Replica replicas = 4; + // repeated .cockroach.proto.ReplicaDescriptor replicas = 4; for (unsigned int i = 0, n = this->replicas_size(); i < n; i++) { target = ::google::protobuf::internal::WireFormatLite:: WriteMessageNoVirtualToArray( @@ -1328,7 +1328,7 @@ int RangeDescriptor::ByteSize() const { } } - // repeated .cockroach.proto.Replica replicas = 4; + // repeated .cockroach.proto.ReplicaDescriptor replicas = 4; total_size += 1 * this->replicas_size(); for (int i = 0; i < this->replicas_size(); i++) { total_size += @@ -1556,31 +1556,31 @@ void RangeDescriptor::clear_end_key() { // @@protoc_insertion_point(field_set_allocated:cockroach.proto.RangeDescriptor.end_key) } -// repeated .cockroach.proto.Replica replicas = 4; +// repeated .cockroach.proto.ReplicaDescriptor replicas = 4; int RangeDescriptor::replicas_size() const { return replicas_.size(); } void RangeDescriptor::clear_replicas() { replicas_.Clear(); } - const ::cockroach::proto::Replica& RangeDescriptor::replicas(int index) const { + const ::cockroach::proto::ReplicaDescriptor& RangeDescriptor::replicas(int index) const { // @@protoc_insertion_point(field_get:cockroach.proto.RangeDescriptor.replicas) return replicas_.Get(index); } - ::cockroach::proto::Replica* RangeDescriptor::mutable_replicas(int index) { + ::cockroach::proto::ReplicaDescriptor* RangeDescriptor::mutable_replicas(int index) { // @@protoc_insertion_point(field_mutable:cockroach.proto.RangeDescriptor.replicas) return replicas_.Mutable(index); } - ::cockroach::proto::Replica* RangeDescriptor::add_replicas() { + ::cockroach::proto::ReplicaDescriptor* RangeDescriptor::add_replicas() { // @@protoc_insertion_point(field_add:cockroach.proto.RangeDescriptor.replicas) return replicas_.Add(); } - const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >& + const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >& RangeDescriptor::replicas() const { // @@protoc_insertion_point(field_list:cockroach.proto.RangeDescriptor.replicas) return replicas_; } - ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >* + ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >* RangeDescriptor::mutable_replicas() { // @@protoc_insertion_point(field_mutable_list:cockroach.proto.RangeDescriptor.replicas) return &replicas_; diff --git a/storage/engine/rocksdb/cockroach/proto/metadata.pb.h b/storage/engine/rocksdb/cockroach/proto/metadata.pb.h index 2beaaf4cc76b..f64f6a2a036d 100644 --- a/storage/engine/rocksdb/cockroach/proto/metadata.pb.h +++ b/storage/engine/rocksdb/cockroach/proto/metadata.pb.h @@ -40,7 +40,7 @@ void protobuf_AssignDesc_cockroach_2fproto_2fmetadata_2eproto(); void protobuf_ShutdownFile_cockroach_2fproto_2fmetadata_2eproto(); class Attributes; -class Replica; +class ReplicaDescriptor; class RangeDescriptor; class RangeTree; class RangeTreeNode; @@ -146,14 +146,14 @@ class Attributes : public ::google::protobuf::Message { }; // ------------------------------------------------------------------- -class Replica : public ::google::protobuf::Message { +class ReplicaDescriptor : public ::google::protobuf::Message { public: - Replica(); - virtual ~Replica(); + ReplicaDescriptor(); + virtual ~ReplicaDescriptor(); - Replica(const Replica& from); + ReplicaDescriptor(const ReplicaDescriptor& from); - inline Replica& operator=(const Replica& from) { + inline ReplicaDescriptor& operator=(const ReplicaDescriptor& from) { CopyFrom(from); return *this; } @@ -167,19 +167,19 @@ class Replica : public ::google::protobuf::Message { } static const ::google::protobuf::Descriptor* descriptor(); - static const Replica& default_instance(); + static const ReplicaDescriptor& default_instance(); - void Swap(Replica* other); + void Swap(ReplicaDescriptor* other); // implements Message ---------------------------------------------- - inline Replica* New() const { return New(NULL); } + inline ReplicaDescriptor* New() const { return New(NULL); } - Replica* New(::google::protobuf::Arena* arena) const; + ReplicaDescriptor* New(::google::protobuf::Arena* arena) const; void CopyFrom(const ::google::protobuf::Message& from); void MergeFrom(const ::google::protobuf::Message& from); - void CopyFrom(const Replica& from); - void MergeFrom(const Replica& from); + void CopyFrom(const ReplicaDescriptor& from); + void MergeFrom(const ReplicaDescriptor& from); void Clear(); bool IsInitialized() const; @@ -194,7 +194,7 @@ class Replica : public ::google::protobuf::Message { void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const; - void InternalSwap(Replica* other); + void InternalSwap(ReplicaDescriptor* other); private: inline ::google::protobuf::Arena* GetArenaNoVirtual() const { return _internal_metadata_.arena(); @@ -231,7 +231,7 @@ class Replica : public ::google::protobuf::Message { ::google::protobuf::int32 replica_id() const; void set_replica_id(::google::protobuf::int32 value); - // @@protoc_insertion_point(class_scope:cockroach.proto.Replica) + // @@protoc_insertion_point(class_scope:cockroach.proto.ReplicaDescriptor) private: inline void set_has_node_id(); inline void clear_has_node_id(); @@ -251,7 +251,7 @@ class Replica : public ::google::protobuf::Message { friend void protobuf_ShutdownFile_cockroach_2fproto_2fmetadata_2eproto(); void InitAsDefaultInstance(); - static Replica* default_instance_; + static ReplicaDescriptor* default_instance_; }; // ------------------------------------------------------------------- @@ -350,16 +350,16 @@ class RangeDescriptor : public ::google::protobuf::Message { ::std::string* release_end_key(); void set_allocated_end_key(::std::string* end_key); - // repeated .cockroach.proto.Replica replicas = 4; + // repeated .cockroach.proto.ReplicaDescriptor replicas = 4; int replicas_size() const; void clear_replicas(); static const int kReplicasFieldNumber = 4; - const ::cockroach::proto::Replica& replicas(int index) const; - ::cockroach::proto::Replica* mutable_replicas(int index); - ::cockroach::proto::Replica* add_replicas(); - const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >& + const ::cockroach::proto::ReplicaDescriptor& replicas(int index) const; + ::cockroach::proto::ReplicaDescriptor* mutable_replicas(int index); + ::cockroach::proto::ReplicaDescriptor* add_replicas(); + const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >& replicas() const; - ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >* + ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >* mutable_replicas(); // optional int32 next_replica_id = 5; @@ -386,7 +386,7 @@ class RangeDescriptor : public ::google::protobuf::Message { ::google::protobuf::int64 range_id_; ::google::protobuf::internal::ArenaStringPtr start_key_; ::google::protobuf::internal::ArenaStringPtr end_key_; - ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica > replicas_; + ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor > replicas_; ::google::protobuf::int32 next_replica_id_; friend void protobuf_AddDesc_cockroach_2fproto_2fmetadata_2eproto(); friend void protobuf_AssignDesc_cockroach_2fproto_2fmetadata_2eproto(); @@ -1049,78 +1049,78 @@ Attributes::mutable_attrs() { // ------------------------------------------------------------------- -// Replica +// ReplicaDescriptor // optional int32 node_id = 1; -inline bool Replica::has_node_id() const { +inline bool ReplicaDescriptor::has_node_id() const { return (_has_bits_[0] & 0x00000001u) != 0; } -inline void Replica::set_has_node_id() { +inline void ReplicaDescriptor::set_has_node_id() { _has_bits_[0] |= 0x00000001u; } -inline void Replica::clear_has_node_id() { +inline void ReplicaDescriptor::clear_has_node_id() { _has_bits_[0] &= ~0x00000001u; } -inline void Replica::clear_node_id() { +inline void ReplicaDescriptor::clear_node_id() { node_id_ = 0; clear_has_node_id(); } -inline ::google::protobuf::int32 Replica::node_id() const { - // @@protoc_insertion_point(field_get:cockroach.proto.Replica.node_id) +inline ::google::protobuf::int32 ReplicaDescriptor::node_id() const { + // @@protoc_insertion_point(field_get:cockroach.proto.ReplicaDescriptor.node_id) return node_id_; } -inline void Replica::set_node_id(::google::protobuf::int32 value) { +inline void ReplicaDescriptor::set_node_id(::google::protobuf::int32 value) { set_has_node_id(); node_id_ = value; - // @@protoc_insertion_point(field_set:cockroach.proto.Replica.node_id) + // @@protoc_insertion_point(field_set:cockroach.proto.ReplicaDescriptor.node_id) } // optional int32 store_id = 2; -inline bool Replica::has_store_id() const { +inline bool ReplicaDescriptor::has_store_id() const { return (_has_bits_[0] & 0x00000002u) != 0; } -inline void Replica::set_has_store_id() { +inline void ReplicaDescriptor::set_has_store_id() { _has_bits_[0] |= 0x00000002u; } -inline void Replica::clear_has_store_id() { +inline void ReplicaDescriptor::clear_has_store_id() { _has_bits_[0] &= ~0x00000002u; } -inline void Replica::clear_store_id() { +inline void ReplicaDescriptor::clear_store_id() { store_id_ = 0; clear_has_store_id(); } -inline ::google::protobuf::int32 Replica::store_id() const { - // @@protoc_insertion_point(field_get:cockroach.proto.Replica.store_id) +inline ::google::protobuf::int32 ReplicaDescriptor::store_id() const { + // @@protoc_insertion_point(field_get:cockroach.proto.ReplicaDescriptor.store_id) return store_id_; } -inline void Replica::set_store_id(::google::protobuf::int32 value) { +inline void ReplicaDescriptor::set_store_id(::google::protobuf::int32 value) { set_has_store_id(); store_id_ = value; - // @@protoc_insertion_point(field_set:cockroach.proto.Replica.store_id) + // @@protoc_insertion_point(field_set:cockroach.proto.ReplicaDescriptor.store_id) } // optional int32 replica_id = 3; -inline bool Replica::has_replica_id() const { +inline bool ReplicaDescriptor::has_replica_id() const { return (_has_bits_[0] & 0x00000004u) != 0; } -inline void Replica::set_has_replica_id() { +inline void ReplicaDescriptor::set_has_replica_id() { _has_bits_[0] |= 0x00000004u; } -inline void Replica::clear_has_replica_id() { +inline void ReplicaDescriptor::clear_has_replica_id() { _has_bits_[0] &= ~0x00000004u; } -inline void Replica::clear_replica_id() { +inline void ReplicaDescriptor::clear_replica_id() { replica_id_ = 0; clear_has_replica_id(); } -inline ::google::protobuf::int32 Replica::replica_id() const { - // @@protoc_insertion_point(field_get:cockroach.proto.Replica.replica_id) +inline ::google::protobuf::int32 ReplicaDescriptor::replica_id() const { + // @@protoc_insertion_point(field_get:cockroach.proto.ReplicaDescriptor.replica_id) return replica_id_; } -inline void Replica::set_replica_id(::google::protobuf::int32 value) { +inline void ReplicaDescriptor::set_replica_id(::google::protobuf::int32 value) { set_has_replica_id(); replica_id_ = value; - // @@protoc_insertion_point(field_set:cockroach.proto.Replica.replica_id) + // @@protoc_insertion_point(field_set:cockroach.proto.ReplicaDescriptor.replica_id) } // ------------------------------------------------------------------- @@ -1257,31 +1257,31 @@ inline void RangeDescriptor::set_allocated_end_key(::std::string* end_key) { // @@protoc_insertion_point(field_set_allocated:cockroach.proto.RangeDescriptor.end_key) } -// repeated .cockroach.proto.Replica replicas = 4; +// repeated .cockroach.proto.ReplicaDescriptor replicas = 4; inline int RangeDescriptor::replicas_size() const { return replicas_.size(); } inline void RangeDescriptor::clear_replicas() { replicas_.Clear(); } -inline const ::cockroach::proto::Replica& RangeDescriptor::replicas(int index) const { +inline const ::cockroach::proto::ReplicaDescriptor& RangeDescriptor::replicas(int index) const { // @@protoc_insertion_point(field_get:cockroach.proto.RangeDescriptor.replicas) return replicas_.Get(index); } -inline ::cockroach::proto::Replica* RangeDescriptor::mutable_replicas(int index) { +inline ::cockroach::proto::ReplicaDescriptor* RangeDescriptor::mutable_replicas(int index) { // @@protoc_insertion_point(field_mutable:cockroach.proto.RangeDescriptor.replicas) return replicas_.Mutable(index); } -inline ::cockroach::proto::Replica* RangeDescriptor::add_replicas() { +inline ::cockroach::proto::ReplicaDescriptor* RangeDescriptor::add_replicas() { // @@protoc_insertion_point(field_add:cockroach.proto.RangeDescriptor.replicas) return replicas_.Add(); } -inline const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >& +inline const ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >& RangeDescriptor::replicas() const { // @@protoc_insertion_point(field_list:cockroach.proto.RangeDescriptor.replicas) return replicas_; } -inline ::google::protobuf::RepeatedPtrField< ::cockroach::proto::Replica >* +inline ::google::protobuf::RepeatedPtrField< ::cockroach::proto::ReplicaDescriptor >* RangeDescriptor::mutable_replicas() { // @@protoc_insertion_point(field_mutable_list:cockroach.proto.RangeDescriptor.replicas) return &replicas_; diff --git a/storage/replica.go b/storage/replica.go index e73a6dec20ef..c0cd18d79832 100644 --- a/storage/replica.go +++ b/storage/replica.go @@ -149,7 +149,6 @@ type RangeManager interface { // Accessors for shared state. ClusterID() string StoreID() proto.StoreID - RaftNodeID() proto.RaftNodeID Clock() *hlc.Clock Engine() engine.Engine DB() *client.DB @@ -165,7 +164,7 @@ type RangeManager interface { // Range and replica manipulation methods. LookupReplica(start, end proto.Key) *Replica MergeRange(subsumingRng *Replica, updatedEndKey proto.Key, subsumedRangeID proto.RangeID) error - NewRangeDescriptor(start, end proto.Key, replicas []proto.Replica) (*proto.RangeDescriptor, error) + NewRangeDescriptor(start, end proto.Key, replicas []proto.ReplicaDescriptor) (*proto.RangeDescriptor, error) NewSnapshot() engine.Engine ProposeRaftCommand(cmdIDKey, proto.RaftCommand) <-chan error RemoveReplica(rng *Replica) error @@ -195,10 +194,22 @@ type Replica struct { llMu sync.Mutex // Synchronizes readers' requests for leader lease respCache *ResponseCache // Provides idempotence for retries - sync.RWMutex // Protects the following fields: - cmdQ *CommandQueue // Enforce at most one command is running per key(s) - tsCache *TimestampCache // Most recent timestamps for keys / key ranges - pendingCmds map[cmdIDKey]*pendingCmd + sync.RWMutex // Protects the following fields: + cmdQ *CommandQueue // Enforce at most one command is running per key(s) + tsCache *TimestampCache // Most recent timestamps for keys / key ranges + pendingCmds map[cmdIDKey]*pendingCmd + + // pendingReplica houses a replica that is not yet in the range + // descriptor, since we must be able to look up a replica's + // descriptor in order to add it to the range. It is protected by + // the RWMutex and once it has taken on a non-zero value it must not + // be changed until that operation has completed and it has been + // reset to a zero value. The sync.Cond is signaled whenever + // pendingReplica.value changes to zero. + pendingReplica struct { + *sync.Cond + value proto.ReplicaDescriptor + } truncatedState unsafe.Pointer // *proto.RaftTruancatedState } @@ -211,6 +222,7 @@ func NewReplica(desc *proto.RangeDescriptor, rm RangeManager) (*Replica, error) respCache: NewResponseCache(desc.RangeID), pendingCmds: map[cmdIDKey]*pendingCmd{}, } + r.pendingReplica.Cond = sync.NewCond(r) r.setDescWithoutProcessUpdate(desc) lastIndex, err := r.loadLastIndex() @@ -299,16 +311,14 @@ func (r *Replica) getLease() *proto.Lease { // newNotLeaderError returns a NotLeaderError initialized with the // replica for the holder (if any) of the given lease. -func (r *Replica) newNotLeaderError(l *proto.Lease, originNode proto.RaftNodeID) error { +func (r *Replica) newNotLeaderError(l *proto.Lease, originStoreID proto.StoreID) error { err := &proto.NotLeaderError{} - if l != nil && l.RaftNodeID != 0 { + if l != nil && l.Replica.ReplicaID != 0 { desc := r.Desc() err.RangeID = desc.RangeID - _, originStoreID := proto.DecodeRaftNodeID(originNode) _, err.Replica = desc.FindReplica(originStoreID) - _, storeID := proto.DecodeRaftNodeID(proto.RaftNodeID(l.RaftNodeID)) - _, err.Leader = desc.FindReplica(storeID) + _, err.Leader = desc.FindReplica(l.Replica.StoreID) } return err } @@ -323,6 +333,10 @@ func (r *Replica) requestLeaderLease(timestamp proto.Timestamp) error { // Prepare a Raft command to get a leader lease for this replica. expiration := timestamp.Add(duration, 0) desc := r.Desc() + _, replica := desc.FindReplica(r.rm.StoreID()) + if replica == nil { + return util.Errorf("can't find store %s in descriptor %+v", r.rm.StoreID(), desc) + } args := &proto.LeaderLeaseRequest{ RequestHeader: proto.RequestHeader{ Key: desc.StartKey, @@ -336,7 +350,7 @@ func (r *Replica) requestLeaderLease(timestamp proto.Timestamp) error { Lease: proto.Lease{ Start: timestamp, Expiration: expiration, - RaftNodeID: r.rm.RaftNodeID(), + Replica: *replica, }, } ba := &proto.BatchRequest{} @@ -373,15 +387,13 @@ func (r *Replica) redirectOnOrAcquireLeaderLease(trace *tracer.Trace, timestamp r.llMu.Lock() defer r.llMu.Unlock() - raftNodeID := r.rm.RaftNodeID() - if lease := r.getLease(); lease.Covers(timestamp) { - if lease.OwnedBy(raftNodeID) { + if lease.OwnedBy(r.rm.StoreID()) { // Happy path: We have an active lease, nothing to do. return nil } // If lease is currently held by another, redirect to holder. - return r.newNotLeaderError(lease, raftNodeID) + return r.newNotLeaderError(lease, r.rm.StoreID()) } defer trace.Epoch("request leader lease")() // Otherwise, no active lease: Request renewal. @@ -393,7 +405,7 @@ func (r *Replica) redirectOnOrAcquireLeaderLease(trace *tracer.Trace, timestamp // extra round-trip. if _, ok := err.(*proto.LeaseRejectedError); ok { if lease := r.getLease(); lease.Covers(timestamp) { - return r.newNotLeaderError(lease, raftNodeID) + return r.newNotLeaderError(lease, r.rm.StoreID()) } } return err @@ -451,11 +463,29 @@ func (r *Replica) setCachedTruncatedState(state *proto.RaftTruncatedState) { // GetReplica returns the replica for this range from the range descriptor. // Returns nil if the replica is not found. -func (r *Replica) GetReplica() *proto.Replica { +func (r *Replica) GetReplica() *proto.ReplicaDescriptor { _, replica := r.Desc().FindReplica(r.rm.StoreID()) return replica } +// ReplicaDescriptor returns information about the given member of this replica's range. +func (r *Replica) ReplicaDescriptor(replicaID proto.ReplicaID) (proto.ReplicaDescriptor, error) { + r.RLock() + defer r.RUnlock() + + desc := r.Desc() + for _, repAddress := range desc.Replicas { + if repAddress.ReplicaID == replicaID { + return repAddress, nil + } + } + if r.pendingReplica.value.ReplicaID == replicaID { + return r.pendingReplica.value, nil + } + return proto.ReplicaDescriptor{}, util.Errorf("replica %d not found in range %d", + replicaID, desc.RangeID) +} + // GetMVCCStats returns a copy of the MVCC stats object for this range. func (r *Replica) GetMVCCStats() engine.MVCCStats { return r.stats.GetMVCC() @@ -919,10 +949,17 @@ func (r *Replica) proposeRaftCommand(ctx context.Context, ba *proto.BatchRequest ctx: ctx, done: make(chan proto.ResponseWithError, 1), } + desc := r.Desc() + _, replica := desc.FindReplica(r.rm.StoreID()) + if replica == nil { + errChan := make(chan error, 1) + errChan <- util.Errorf("could not find own replica in descriptor") + return errChan, pendingCmd + } raftCmd := proto.RaftCommand{ - RangeID: r.Desc().RangeID, - OriginNodeID: r.rm.RaftNodeID(), - Cmd: *ba, + RangeID: desc.RangeID, + OriginReplica: *replica, + Cmd: *ba, } cmdID := ba.GetOrCreateCmdID(r.rm.Clock().PhysicalNow()) idKey := makeCmdIDKey(cmdID) @@ -962,7 +999,7 @@ func (r *Replica) processRaftCommand(idKey cmdIDKey, index uint64, raftCmd proto // applyRaftCommand will return "expected" errors, but may also indicate // replica corruption (as of now, signaled by a replicaCorruptionError). // We feed its return through maybeSetCorrupt to act when that happens. - br, err := r.applyRaftCommand(ctx, index, proto.RaftNodeID(raftCmd.OriginNodeID), &raftCmd.Cmd) + br, err := r.applyRaftCommand(ctx, index, raftCmd.OriginReplica, &raftCmd.Cmd) err = r.maybeSetCorrupt(err) execDone() if err != nil { @@ -982,7 +1019,7 @@ func (r *Replica) processRaftCommand(idKey cmdIDKey, index uint64, raftCmd proto // underlying state machine (i.e. the engine). // When certain critical operations fail, a replicaCorruptionError may be // returned and must be handled by the caller. -func (r *Replica) applyRaftCommand(ctx context.Context, index uint64, originNode proto.RaftNodeID, +func (r *Replica) applyRaftCommand(ctx context.Context, index uint64, originReplica proto.ReplicaDescriptor, ba *proto.BatchRequest) (*proto.BatchResponse, error) { if index <= 0 { log.Fatalc(ctx, "raft command index is <= 0") @@ -997,7 +1034,7 @@ func (r *Replica) applyRaftCommand(ctx context.Context, index uint64, originNode // Call the helper, which returns a batch containing data written // during command execution and any associated error. ms := engine.MVCCStats{} - batch, br, intents, rErr := r.applyRaftCommandInBatch(ctx, index, originNode, ba, &ms) + batch, br, intents, rErr := r.applyRaftCommandInBatch(ctx, index, originReplica, ba, &ms) defer batch.Close() // Advance the last applied index and commit the batch. @@ -1029,7 +1066,7 @@ func (r *Replica) applyRaftCommand(ctx context.Context, index uint64, originNode // On the replica on which this command originated, resolve skipped intents // asynchronously - even on failure. - if originNode == r.rm.RaftNodeID() { + if originReplica.StoreID == r.rm.StoreID() { r.handleSkippedIntents(intents) } @@ -1039,7 +1076,7 @@ func (r *Replica) applyRaftCommand(ctx context.Context, index uint64, originNode // applyRaftCommandInBatch executes the command in a batch engine and // returns the batch containing the results. The caller is responsible // for committing the batch, even on error. -func (r *Replica) applyRaftCommandInBatch(ctx context.Context, index uint64, originNode proto.RaftNodeID, +func (r *Replica) applyRaftCommandInBatch(ctx context.Context, index uint64, originReplica proto.ReplicaDescriptor, ba *proto.BatchRequest, ms *engine.MVCCStats) (engine.Engine, *proto.BatchResponse, []intentsWithArg, error) { // Create a new batch for the command to ensure all or nothing semantics. btch := r.rm.Engine().NewBatch() @@ -1082,7 +1119,7 @@ func (r *Replica) applyRaftCommandInBatch(ctx context.Context, index uint64, ori // TODO(tschottdorf): shouldn't be in the loop. Currently is because // we haven't cleaned up the timestamp handling fully. if lease := r.getLease(); args.Method() != proto.LeaderLease && - (!lease.OwnedBy(originNode) || !lease.Covers(args.Header().Timestamp)) { + (!lease.OwnedBy(originReplica.StoreID) || !lease.Covers(args.Header().Timestamp)) { // Verify the leader lease is held, unless this command is trying to // obtain it. Any other Raft command has had the leader lease held // by the replica at proposal time, but this may no longer be the case. @@ -1099,7 +1136,7 @@ func (r *Replica) applyRaftCommandInBatch(ctx context.Context, index uint64, ori // same ClientCmdID and would get the distributed sender stuck in an // infinite loop, retrieving a stale NotLeaderError over and over // again, even when proposing at the correct replica. - return btch, nil, nil, r.newNotLeaderError(lease, originNode) + return btch, nil, nil, r.newNotLeaderError(lease, originReplica.StoreID) } } @@ -1304,7 +1341,7 @@ func (r *Replica) maybeGossipSystemConfigLocked() { return } - if lease := r.getLease(); !lease.OwnedBy(r.rm.RaftNodeID()) || !lease.Covers(r.rm.Clock().Now()) { + if lease := r.getLease(); !lease.OwnedBy(r.rm.StoreID()) || !lease.Covers(r.rm.Clock().Now()) { // Do not gossip when a leader lease is not held. return } diff --git a/storage/replica_command.go b/storage/replica_command.go index 03749eb95c03..22329628227a 100644 --- a/storage/replica_command.go +++ b/storage/replica_command.go @@ -1011,7 +1011,7 @@ func (r *Replica) LeaderLease(batch engine.Engine, ms *engine.MVCCStats, args pr defer r.Unlock() prevLease := r.getLease() - isExtension := prevLease.RaftNodeID == args.Lease.RaftNodeID + isExtension := prevLease.Replica.StoreID == args.Lease.Replica.StoreID effectiveStart := args.Lease.Start // We return this error in "normal" lease-overlap related failures. rErr := &proto.LeaseRejectedError{ @@ -1027,8 +1027,7 @@ func (r *Replica) LeaderLease(batch engine.Engine, ms *engine.MVCCStats, args pr // Verify that requestion replica is part of the current replica set. desc := r.Desc() - _, requestingStore := proto.DecodeRaftNodeID(args.Lease.RaftNodeID) - if idx, _ := desc.FindReplica(requestingStore); idx == -1 { + if idx, _ := desc.FindReplica(args.Lease.Replica.StoreID); idx == -1 { return reply, rErr } @@ -1045,7 +1044,7 @@ func (r *Replica) LeaderLease(batch engine.Engine, ms *engine.MVCCStats, args pr // If no old lease exists or this is our lease, we don't need to add an // extra tick. This allows multiple requests from the same replica to // merge without ticking away from the minimal common start timestamp. - if prevLease.RaftNodeID == 0 || isExtension { + if prevLease.Replica.StoreID == 0 || isExtension { // TODO(tschottdorf) Think about whether it'd be better to go all the // way back to prevLease.Start(), so that whenever the last lease is // the own one, the original start is preserved. @@ -1079,7 +1078,8 @@ func (r *Replica) LeaderLease(batch engine.Engine, ms *engine.MVCCStats, args pr // clock offset to account for any difference in clocks // between the expiration (set by a remote node) and this // node. - if r.getLease().RaftNodeID == r.rm.RaftNodeID() && prevLease.RaftNodeID != r.getLease().RaftNodeID { + if r.getLease().Replica.StoreID == r.rm.StoreID() && + prevLease.Replica.StoreID != r.getLease().Replica.StoreID { r.tsCache.SetLowWater(prevLease.Expiration.Add(int64(r.rm.Clock().MaxOffset()), 0)) log.Infof("range %d: new leader lease %s", rangeID, args.Lease) } @@ -1439,6 +1439,7 @@ func (r *Replica) mergeTrigger(batch engine.Engine, merge *proto.MergeTrigger) e } func (r *Replica) changeReplicasTrigger(change *proto.ChangeReplicasTrigger) error { + defer r.clearPendingChangeReplicas() cpy := *r.Desc() cpy.Replicas = change.UpdatedReplicas cpy.NextReplicaID = change.NextReplicaID @@ -1462,10 +1463,15 @@ func (r *Replica) changeReplicasTrigger(change *proto.ChangeReplicasTrigger) err // // The supplied RangeDescriptor is used as a form of optimistic lock. See the // comment of "AdminSplit" for more information on this pattern. -func (r *Replica) ChangeReplicas(changeType proto.ReplicaChangeType, replica proto.Replica, desc *proto.RangeDescriptor) error { +func (r *Replica) ChangeReplicas(changeType proto.ReplicaChangeType, replica proto.ReplicaDescriptor, desc *proto.RangeDescriptor) error { + r.Lock() + for r.pendingReplica.value.ReplicaID != 0 { + r.pendingReplica.Wait() + } + // Validate the request and prepare the new descriptor. updatedDesc := *desc - updatedDesc.Replicas = append([]proto.Replica{}, desc.Replicas...) + updatedDesc.Replicas = append([]proto.ReplicaDescriptor{}, desc.Replicas...) found := -1 // tracks NodeID && StoreID nodeUsed := false // tracks NodeID only for i, existingRep := range desc.Replicas { @@ -1486,6 +1492,10 @@ func (r *Replica) ChangeReplicas(changeType proto.ReplicaChangeType, replica pro replica.ReplicaID = updatedDesc.NextReplicaID updatedDesc.NextReplicaID++ updatedDesc.Replicas = append(updatedDesc.Replicas, replica) + + // We need to be able to look up replica information before the change + // is official. + r.pendingReplica.value = replica } else if changeType == proto.REMOVE_REPLICA { // If that exact node-store combination does not have the replica, // abort the removal. @@ -1497,6 +1507,8 @@ func (r *Replica) ChangeReplicas(changeType proto.ReplicaChangeType, replica pro updatedDesc.Replicas = updatedDesc.Replicas[:len(updatedDesc.Replicas)-1] } + r.Unlock() + err := r.rm.DB().Txn(func(txn *client.Txn) error { // Important: the range descriptor must be the first thing touched in the transaction // so the transaction record is co-located with the range being modified. @@ -1530,14 +1542,22 @@ func (r *Replica) ChangeReplicas(changeType proto.ReplicaChangeType, replica pro return txn.Run(b) }) if err != nil { + r.clearPendingChangeReplicas() return util.Errorf("change replicas of %d failed: %s", desc.RangeID, err) } return nil } +func (r *Replica) clearPendingChangeReplicas() { + r.Lock() + r.pendingReplica.value = proto.ReplicaDescriptor{} + r.pendingReplica.Broadcast() + r.Unlock() +} + // replicaSetsEqual is used in AdminMerge to ensure that the ranges are // all collocate on the same set of replicas. -func replicaSetsEqual(a, b []proto.Replica) bool { +func replicaSetsEqual(a, b []proto.ReplicaDescriptor) bool { if len(a) != len(b) { return false } diff --git a/storage/replica_raftstorage.go b/storage/replica_raftstorage.go index 5aac99a9f6ec..eca4c36d941a 100644 --- a/storage/replica_raftstorage.go +++ b/storage/replica_raftstorage.go @@ -63,7 +63,7 @@ func (r *Replica) InitialState() (raftpb.HardState, raftpb.ConfState, error) { // For uninitalized ranges, membership is unknown at this point. if found || r.isInitialized() { for _, rep := range desc.Replicas { - cs.Nodes = append(cs.Nodes, uint64(proto.MakeRaftNodeID(rep.NodeID, rep.StoreID))) + cs.Nodes = append(cs.Nodes, uint64(rep.ReplicaID)) } } @@ -293,7 +293,7 @@ func (r *Replica) Snapshot() (raftpb.Snapshot, error) { // Synthesize our raftpb.ConfState from desc. var cs raftpb.ConfState for _, rep := range desc.Replicas { - cs.Nodes = append(cs.Nodes, uint64(proto.MakeRaftNodeID(rep.NodeID, rep.StoreID))) + cs.Nodes = append(cs.Nodes, uint64(rep.ReplicaID)) } term, err := r.Term(appliedIndex) diff --git a/storage/replica_test.go b/storage/replica_test.go index 2cb8bf452505..9ba8545d52ad 100644 --- a/storage/replica_test.go +++ b/storage/replica_test.go @@ -56,10 +56,11 @@ func testRangeDescriptor() *proto.RangeDescriptor { RangeID: 1, StartKey: proto.KeyMin, EndKey: proto.KeyMax, - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { - NodeID: 1, - StoreID: 1, + ReplicaID: 1, + NodeID: 1, + StoreID: 1, }, }, } @@ -213,13 +214,13 @@ func newTransaction(name string, baseKey proto.Key, userPriority int32, isolation, clock.Now(), clock.MaxOffset().Nanoseconds()) } -// CreateReplicaSets creates new proto.Replica protos based on an array of +// CreateReplicaSets creates new proto.ReplicaDescriptor protos based on an array of // StoreIDs to aid in testing. Note that this does not actually produce any // replicas, it just creates the proto. -func createReplicaSets(replicaNumbers []proto.StoreID) []proto.Replica { - result := []proto.Replica{} +func createReplicaSets(replicaNumbers []proto.StoreID) []proto.ReplicaDescriptor { + result := []proto.ReplicaDescriptor{} for _, replicaNumber := range replicaNumbers { - result = append(result, proto.Replica{ + result = append(result, proto.ReplicaDescriptor{ StoreID: replicaNumber, }) } @@ -290,7 +291,7 @@ func TestRangeReadConsistency(t *testing.T) { // Modify range descriptor to include a second replica; leader lease can // only be obtained by Replicas which are part of the range descriptor. This // workaround is sufficient for the purpose of this test. - secondReplica := proto.Replica{ + secondReplica := proto.ReplicaDescriptor{ NodeID: 2, StoreID: 2, ReplicaID: 2, @@ -330,7 +331,11 @@ func TestRangeReadConsistency(t *testing.T) { setLeaderLease(t, tc.rng, &proto.Lease{ Start: start, Expiration: start.Add(10, 0), - RaftNodeID: proto.MakeRaftNodeID(2, 2), // a different node + Replica: proto.ReplicaDescriptor{ // a different node + ReplicaID: 2, + NodeID: 2, + StoreID: 2, + }, }) gArgs.ReadConsistency = proto.CONSISTENT gArgs.Txn = nil @@ -359,7 +364,7 @@ func TestApplyCmdLeaseError(t *testing.T) { // Modify range descriptor to include a second replica; leader lease can // only be obtained by Replicas which are part of the range descriptor. This // workaround is sufficient for the purpose of this test. - secondReplica := proto.Replica{ + secondReplica := proto.ReplicaDescriptor{ NodeID: 2, StoreID: 2, ReplicaID: 2, @@ -381,7 +386,11 @@ func TestApplyCmdLeaseError(t *testing.T) { setLeaderLease(t, tc.rng, &proto.Lease{ Start: start, Expiration: start.Add(10, 0), - RaftNodeID: proto.MakeRaftNodeID(2, 2), // a different node + Replica: proto.ReplicaDescriptor{ // a different node + ReplicaID: 2, + NodeID: 2, + StoreID: 2, + }, }) // Submit a proposal to Raft. @@ -416,7 +425,7 @@ func TestRangeRangeBoundsChecking(t *testing.T) { // range replica and whether it's expired for the given timestamp. func hasLease(rng *Replica, timestamp proto.Timestamp) (bool, bool) { l := rng.getLease() - return l.OwnedBy(rng.rm.RaftNodeID()), !l.Covers(timestamp) + return l.OwnedBy(rng.rm.StoreID()), !l.Covers(timestamp) } func TestRangeLeaderLease(t *testing.T) { @@ -429,7 +438,7 @@ func TestRangeLeaderLease(t *testing.T) { // Modify range descriptor to include a second replica; leader lease can // only be obtained by Replicas which are part of the range descriptor. This // workaround is sufficient for the purpose of this test. - secondReplica := proto.Replica{ + secondReplica := proto.ReplicaDescriptor{ NodeID: 2, StoreID: 2, ReplicaID: 2, @@ -446,7 +455,11 @@ func TestRangeLeaderLease(t *testing.T) { setLeaderLease(t, tc.rng, &proto.Lease{ Start: now.Add(10, 0), Expiration: now.Add(20, 0), - RaftNodeID: proto.MakeRaftNodeID(2, 2), + Replica: proto.ReplicaDescriptor{ + ReplicaID: 2, + NodeID: 2, + StoreID: 2, + }, }) if held, expired := hasLease(tc.rng, tc.clock.Now().Add(15, 0)); held || expired { t.Errorf("expected another replica to have leader lease") @@ -474,7 +487,7 @@ func TestRangeNotLeaderError(t *testing.T) { // Modify range descriptor to include a second replica; leader lease can // only be obtained by Replicas which are part of the range descriptor. This // workaround is sufficient for the purpose of this test. - secondReplica := proto.Replica{ + secondReplica := proto.ReplicaDescriptor{ NodeID: 2, StoreID: 2, ReplicaID: 2, @@ -488,13 +501,17 @@ func TestRangeNotLeaderError(t *testing.T) { setLeaderLease(t, tc.rng, &proto.Lease{ Start: now, Expiration: now.Add(10, 0), - RaftNodeID: proto.MakeRaftNodeID(2, 2), + Replica: proto.ReplicaDescriptor{ + ReplicaID: 2, + NodeID: 2, + StoreID: 2, + }, }) header := proto.RequestHeader{ Key: proto.Key("a"), RangeID: tc.rng.Desc().RangeID, - Replica: proto.Replica{StoreID: tc.store.StoreID()}, + Replica: proto.ReplicaDescriptor{StoreID: tc.store.StoreID()}, Timestamp: now, } testCases := []proto.Request{ @@ -536,7 +553,7 @@ func TestRangeGossipConfigsOnLease(t *testing.T) { // Modify range descriptor to include a second replica; leader lease can // only be obtained by Replicas which are part of the range descriptor. This // workaround is sufficient for the purpose of this test. - secondReplica := proto.Replica{ + secondReplica := proto.ReplicaDescriptor{ NodeID: 2, StoreID: 2, ReplicaID: 2, @@ -579,7 +596,11 @@ func TestRangeGossipConfigsOnLease(t *testing.T) { setLeaderLease(t, tc.rng, &proto.Lease{ Start: now, Expiration: now.Add(10, 0), - RaftNodeID: proto.MakeRaftNodeID(2, 2), + Replica: proto.ReplicaDescriptor{ + ReplicaID: 2, + NodeID: 2, + StoreID: 2, + }, }) // Expire that lease. @@ -590,7 +611,11 @@ func TestRangeGossipConfigsOnLease(t *testing.T) { setLeaderLease(t, tc.rng, &proto.Lease{ Start: now.Add(11, 0), Expiration: now.Add(20, 0), - RaftNodeID: tc.store.RaftNodeID(), + Replica: proto.ReplicaDescriptor{ + ReplicaID: 1, + NodeID: 1, + StoreID: 1, + }, }) if !verifySystem() { t.Errorf("expected gossip of new config") @@ -611,7 +636,7 @@ func TestRangeTSCacheLowWaterOnLease(t *testing.T) { // Modify range descriptor to include a second replica; leader lease can // only be obtained by Replicas which are part of the range descriptor. This // workaround is sufficient for the purpose of this test. - secondReplica := proto.Replica{ + secondReplica := proto.ReplicaDescriptor{ NodeID: 2, StoreID: 2, ReplicaID: 2, @@ -627,28 +652,32 @@ func TestRangeTSCacheLowWaterOnLease(t *testing.T) { baseLowWater := baseRTS.WallTime testCases := []struct { - nodeID proto.RaftNodeID + storeID proto.StoreID start proto.Timestamp expiration proto.Timestamp expLowWater int64 }{ // Grant the lease fresh. - {tc.store.RaftNodeID(), now, now.Add(10, 0), baseLowWater}, + {tc.store.StoreID(), now, now.Add(10, 0), baseLowWater}, // Renew the lease. - {tc.store.RaftNodeID(), now.Add(15, 0), now.Add(30, 0), baseLowWater}, + {tc.store.StoreID(), now.Add(15, 0), now.Add(30, 0), baseLowWater}, // Renew the lease but shorten expiration. - {tc.store.RaftNodeID(), now.Add(16, 0), now.Add(25, 0), baseLowWater}, + {tc.store.StoreID(), now.Add(16, 0), now.Add(25, 0), baseLowWater}, // Lease is held by another. - {proto.MakeRaftNodeID(2, 2), now.Add(29, 0), now.Add(50, 0), baseLowWater}, + {tc.store.StoreID() + 1, now.Add(29, 0), now.Add(50, 0), baseLowWater}, // Lease is regranted to this replica. - {tc.store.RaftNodeID(), now.Add(60, 0), now.Add(70, 0), now.Add(50, 0).WallTime + int64(maxClockOffset) + baseLowWater}, + {tc.store.StoreID(), now.Add(60, 0), now.Add(70, 0), now.Add(50, 0).WallTime + int64(maxClockOffset) + baseLowWater}, } for i, test := range testCases { setLeaderLease(t, tc.rng, &proto.Lease{ Start: test.start, Expiration: test.expiration, - RaftNodeID: test.nodeID, + Replica: proto.ReplicaDescriptor{ + ReplicaID: proto.ReplicaID(test.storeID), + NodeID: proto.NodeID(test.storeID), + StoreID: test.storeID, + }, }) // Verify expected low water mark. rTS, wTS := tc.rng.tsCache.GetMax(proto.Key("a"), nil, nil) @@ -673,7 +702,11 @@ func TestRangeLeaderLeaseRejectUnknownRaftNodeID(t *testing.T) { lease := &proto.Lease{ Start: now, Expiration: now.Add(10, 0), - RaftNodeID: proto.MakeRaftNodeID(2, 2), + Replica: proto.ReplicaDescriptor{ + ReplicaID: 2, + NodeID: 2, + StoreID: 2, + }, } args := &proto.BatchRequest{} args.Add(&proto.LeaderLeaseRequest{Lease: *lease}) @@ -827,7 +860,7 @@ func getArgs(key []byte, rangeID proto.RangeID, storeID proto.StoreID) proto.Get RequestHeader: proto.RequestHeader{ Key: key, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, } } @@ -840,7 +873,7 @@ func putArgs(key, value []byte, rangeID proto.RangeID, storeID proto.StoreID) pr Key: key, Timestamp: proto.MinTimestamp, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, Value: proto.Value{ Bytes: value, @@ -854,7 +887,7 @@ func deleteArgs(key proto.Key, rangeID proto.RangeID, storeID proto.StoreID) pro RequestHeader: proto.RequestHeader{ Key: key, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, } } @@ -878,7 +911,7 @@ func incrementArgs(key []byte, inc int64, rangeID proto.RangeID, storeID proto.S RequestHeader: proto.RequestHeader{ Key: key, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, Increment: inc, } @@ -890,7 +923,7 @@ func scanArgs(start, end []byte, rangeID proto.RangeID, storeID proto.StoreID) p Key: start, EndKey: end, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, } } @@ -902,7 +935,7 @@ func endTxnArgs(txn *proto.Transaction, commit bool, rangeID proto.RangeID, stor RequestHeader: proto.RequestHeader{ Key: txn.Key, // not allowed when going through TxnCoordSender, but we're not RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, Txn: txn, }, Commit: commit, @@ -917,7 +950,7 @@ func pushTxnArgs(pusher, pushee *proto.Transaction, pushType proto.PushTxnType, Key: pushee.Key, Timestamp: pusher.Timestamp, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, Now: pusher.Timestamp, PusherTxn: pusher, @@ -932,7 +965,7 @@ func heartbeatArgs(txn *proto.Transaction, rangeID proto.RangeID, storeID proto. RequestHeader: proto.RequestHeader{ Key: txn.Key, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, Txn: txn, }, } @@ -946,7 +979,7 @@ func internalMergeArgs(key []byte, value proto.Value, rangeID proto.RangeID, sto RequestHeader: proto.RequestHeader{ Key: key, RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, Value: value, } @@ -956,7 +989,7 @@ func truncateLogArgs(index uint64, rangeID proto.RangeID, storeID proto.StoreID) return proto.TruncateLogRequest{ RequestHeader: proto.RequestHeader{ RangeID: rangeID, - Replica: proto.Replica{StoreID: storeID}, + Replica: proto.ReplicaDescriptor{StoreID: storeID}, }, Index: index, } @@ -2257,7 +2290,7 @@ func TestRangeResolveIntentRange(t *testing.T) { Key: proto.Key("a"), EndKey: proto.Key("c"), RangeID: tc.rng.Desc().RangeID, - Replica: proto.Replica{StoreID: tc.store.StoreID()}, + Replica: proto.ReplicaDescriptor{StoreID: tc.store.StoreID()}, }, IntentTxn: *txn, } @@ -2284,7 +2317,7 @@ func verifyRangeStats(eng engine.Engine, rangeID proto.RangeID, expMS engine.MVC t.Fatal(err) } if !reflect.DeepEqual(expMS, ms) { - t.Errorf("expected stats %+v; got %+v", expMS, ms) + t.Errorf("expected stats \n %+v;\ngot \n %+v", expMS, ms) } } @@ -2308,7 +2341,7 @@ func TestRangeStatsComputation(t *testing.T) { if _, err := tc.rng.AddCmd(tc.rng.context(), &pArgs); err != nil { t.Fatal(err) } - expMS := engine.MVCCStats{LiveBytes: 42, KeyBytes: 16, ValBytes: 26, IntentBytes: 0, LiveCount: 1, KeyCount: 1, ValCount: 1, IntentCount: 0, SysBytes: 61, SysCount: 1} + expMS := engine.MVCCStats{LiveBytes: 42, KeyBytes: 16, ValBytes: 26, IntentBytes: 0, LiveCount: 1, KeyCount: 1, ValCount: 1, IntentCount: 0, SysBytes: 63, SysCount: 1} verifyRangeStats(tc.engine, tc.rng.Desc().RangeID, expMS, t) // Put a 2nd value transactionally. @@ -2319,7 +2352,7 @@ func TestRangeStatsComputation(t *testing.T) { if _, err := tc.rng.AddCmd(tc.rng.context(), &pArgs); err != nil { t.Fatal(err) } - expMS = engine.MVCCStats{LiveBytes: 136, KeyBytes: 32, ValBytes: 104, IntentBytes: 26, LiveCount: 2, KeyCount: 2, ValCount: 2, IntentCount: 1, SysBytes: 61, SysCount: 1} + expMS = engine.MVCCStats{LiveBytes: 136, KeyBytes: 32, ValBytes: 104, IntentBytes: 26, LiveCount: 2, KeyCount: 2, ValCount: 2, IntentCount: 1, SysBytes: 63, SysCount: 1} verifyRangeStats(tc.engine, tc.rng.Desc().RangeID, expMS, t) // Resolve the 2nd value. @@ -2327,7 +2360,7 @@ func TestRangeStatsComputation(t *testing.T) { RequestHeader: proto.RequestHeader{ Key: pArgs.Key, RangeID: tc.rng.Desc().RangeID, - Replica: proto.Replica{StoreID: tc.store.StoreID()}, + Replica: proto.ReplicaDescriptor{StoreID: tc.store.StoreID()}, }, IntentTxn: *pArgs.Txn, } @@ -2336,7 +2369,7 @@ func TestRangeStatsComputation(t *testing.T) { if _, err := tc.rng.AddCmd(tc.rng.context(), rArgs); err != nil { t.Fatal(err) } - expMS = engine.MVCCStats{LiveBytes: 84, KeyBytes: 32, ValBytes: 52, IntentBytes: 0, LiveCount: 2, KeyCount: 2, ValCount: 2, IntentCount: 0, SysBytes: 61, SysCount: 1} + expMS = engine.MVCCStats{LiveBytes: 84, KeyBytes: 32, ValBytes: 52, IntentBytes: 0, LiveCount: 2, KeyCount: 2, ValCount: 2, IntentCount: 0, SysBytes: 63, SysCount: 1} verifyRangeStats(tc.engine, tc.rng.Desc().RangeID, expMS, t) // Delete the 1st value. @@ -2346,7 +2379,7 @@ func TestRangeStatsComputation(t *testing.T) { if _, err := tc.rng.AddCmd(tc.rng.context(), &dArgs); err != nil { t.Fatal(err) } - expMS = engine.MVCCStats{LiveBytes: 42, KeyBytes: 44, ValBytes: 54, IntentBytes: 0, LiveCount: 1, KeyCount: 2, ValCount: 3, IntentCount: 0, SysBytes: 61, SysCount: 1} + expMS = engine.MVCCStats{LiveBytes: 42, KeyBytes: 44, ValBytes: 54, IntentBytes: 0, LiveCount: 1, KeyCount: 2, ValCount: 3, IntentCount: 0, SysBytes: 63, SysCount: 1} verifyRangeStats(tc.engine, tc.rng.Desc().RangeID, expMS, t) } @@ -2508,7 +2541,7 @@ func TestConditionFailedError(t *testing.T) { Key: key, Timestamp: proto.MinTimestamp, RangeID: 1, - Replica: proto.Replica{StoreID: tc.store.StoreID()}, + Replica: proto.ReplicaDescriptor{StoreID: tc.store.StoreID()}, }, Value: proto.Value{ Bytes: value, @@ -2535,10 +2568,10 @@ func TestReplicaSetsEqual(t *testing.T) { defer leaktest.AfterTest(t) testData := []struct { expected bool - a []proto.Replica - b []proto.Replica + a []proto.ReplicaDescriptor + b []proto.ReplicaDescriptor }{ - {true, []proto.Replica{}, []proto.Replica{}}, + {true, []proto.ReplicaDescriptor{}, []proto.ReplicaDescriptor{}}, {true, createReplicaSets([]proto.StoreID{1}), createReplicaSets([]proto.StoreID{1})}, {true, createReplicaSets([]proto.StoreID{1, 2}), createReplicaSets([]proto.StoreID{1, 2})}, {true, createReplicaSets([]proto.StoreID{1, 2}), createReplicaSets([]proto.StoreID{2, 1})}, @@ -2626,7 +2659,7 @@ func TestChangeReplicasDuplicateError(t *testing.T) { tc.Start(t) defer tc.Stop() - if err := tc.rng.ChangeReplicas(proto.ADD_REPLICA, proto.Replica{ + if err := tc.rng.ChangeReplicas(proto.ADD_REPLICA, proto.ReplicaDescriptor{ NodeID: tc.store.Ident.NodeID, StoreID: 9999, }, tc.rng.Desc()); err == nil || !strings.Contains(err.Error(), @@ -2665,7 +2698,7 @@ func testRangeDanglingMetaIntent(t *testing.T, isReverse bool) { RequestHeader: proto.RequestHeader{ Key: keys.RangeMetaKey(key), RangeID: tc.rng.Desc().RangeID, - Replica: proto.Replica{StoreID: tc.store.StoreID()}, + Replica: proto.ReplicaDescriptor{StoreID: tc.store.StoreID()}, ReadConsistency: proto.INCONSISTENT, }, MaxRanges: 1, @@ -2815,7 +2848,7 @@ func TestRangeLookupUseReverseScan(t *testing.T) { Key: keys.RangeMetaKey(proto.Key("a")), EndKey: keys.RangeMetaKey(proto.Key("z")), RangeID: tc.rng.Desc().RangeID, - Replica: proto.Replica{StoreID: tc.store.StoreID()}, + Replica: proto.ReplicaDescriptor{StoreID: tc.store.StoreID()}, }, IntentTxn: *txn, } @@ -2828,7 +2861,7 @@ func TestRangeLookupUseReverseScan(t *testing.T) { rlArgs := &proto.RangeLookupRequest{ RequestHeader: proto.RequestHeader{ RangeID: tc.rng.Desc().RangeID, - Replica: proto.Replica{StoreID: tc.store.StoreID()}, + Replica: proto.ReplicaDescriptor{StoreID: tc.store.StoreID()}, ReadConsistency: proto.INCONSISTENT, }, MaxRanges: 1, diff --git a/storage/replicate_queue.go b/storage/replicate_queue.go index ec9172d1febe..acd765f5fa62 100644 --- a/storage/replicate_queue.go +++ b/storage/replicate_queue.go @@ -116,7 +116,7 @@ func (rq replicateQueue) process(now proto.Timestamp, repl *Replica, sysCfg *con if err != nil { return err } - newReplica := proto.Replica{ + newReplica := proto.ReplicaDescriptor{ NodeID: newStore.Node.NodeID, StoreID: newStore.StoreID, } @@ -154,7 +154,7 @@ func (rq replicateQueue) process(now proto.Timestamp, repl *Replica, sysCfg *con // without re-queueing this replica. return nil } - rebalanceReplica := proto.Replica{ + rebalanceReplica := proto.ReplicaDescriptor{ NodeID: rebalanceStore.Node.NodeID, StoreID: rebalanceStore.StoreID, } diff --git a/storage/simulation/range.go b/storage/simulation/range.go index 393ae39f1228..66ac36ea48f2 100644 --- a/storage/simulation/range.go +++ b/storage/simulation/range.go @@ -61,7 +61,7 @@ func newRange(rangeID proto.RangeID, allocator storage.Allocator) *Range { // both the range descriptor and the store map. func (r *Range) addReplica(s *Store) { storeID, nodeID := s.getIDs() - r.desc.Replicas = append(r.desc.Replicas, proto.Replica{ + r.desc.Replicas = append(r.desc.Replicas, proto.ReplicaDescriptor{ NodeID: nodeID, StoreID: storeID, }) @@ -93,7 +93,7 @@ func (r *Range) getStores() map[proto.StoreID]*Store { // replicas in the range. func (r *Range) splitRange(originalRange *Range) { stores := originalRange.getStores() - r.desc.Replicas = append([]proto.Replica(nil), originalRange.desc.Replicas...) + r.desc.Replicas = append([]proto.ReplicaDescriptor(nil), originalRange.desc.Replicas...) for storeID, store := range stores { r.replicas[storeID] = replica{ store: store, diff --git a/storage/store.go b/storage/store.go index e5289db3e553..89f9054b46a0 100644 --- a/storage/store.go +++ b/storage/store.go @@ -468,7 +468,7 @@ func (s *Store) Start(stopper *stop.Stopper) error { start := keys.RangeDescriptorKey(proto.KeyMin) end := keys.RangeDescriptorKey(proto.KeyMax) - if s.multiraft, err = multiraft.NewMultiRaft(s.RaftNodeID(), &multiraft.Config{ + if s.multiraft, err = multiraft.NewMultiRaft(s.Ident.NodeID, s.Ident.StoreID, &multiraft.Config{ Transport: s.ctx.Transport, Storage: s, StateMachine: s, @@ -787,7 +787,7 @@ func (s *Store) BootstrapRange(initialValues []proto.KeyValue) error { StartKey: proto.KeyMin, EndKey: proto.KeyMax, NextReplicaID: 2, - Replicas: []proto.Replica{ + Replicas: []proto.ReplicaDescriptor{ { NodeID: 1, StoreID: 1, @@ -857,11 +857,6 @@ func (s *Store) ClusterID() string { return s.Ident.ClusterID } // StoreID accessor. func (s *Store) StoreID() proto.StoreID { return s.Ident.StoreID } -// RaftNodeID accessor. -func (s *Store) RaftNodeID() proto.RaftNodeID { - return proto.MakeRaftNodeID(s.Ident.NodeID, s.Ident.StoreID) -} - // Clock accessor. func (s *Store) Clock() *hlc.Clock { return s.ctx.Clock } @@ -895,7 +890,7 @@ func (s *Store) Tracer() *tracer.Tracer { return s.ctx.Tracer } // NewRangeDescriptor creates a new descriptor based on start and end // keys and the supplied proto.Replicas slice. It allocates new // replica IDs to fill out the supplied replicas. -func (s *Store) NewRangeDescriptor(start, end proto.Key, replicas []proto.Replica) (*proto.RangeDescriptor, error) { +func (s *Store) NewRangeDescriptor(start, end proto.Key, replicas []proto.ReplicaDescriptor) (*proto.RangeDescriptor, error) { id, err := s.rangeIDAlloc.Allocate() if err != nil { return nil, err @@ -904,7 +899,7 @@ func (s *Store) NewRangeDescriptor(start, end proto.Key, replicas []proto.Replic RangeID: proto.RangeID(id), StartKey: start, EndKey: end, - Replicas: append([]proto.Replica(nil), replicas...), + Replicas: append([]proto.ReplicaDescriptor(nil), replicas...), NextReplicaID: proto.ReplicaID(len(replicas) + 1), } for i := range desc.Replicas { @@ -1434,7 +1429,7 @@ func (s *Store) proposeRaftCommandImpl(idKey cmdIDKey, cmd proto.RaftCommand) <- crt := etr.InternalCommitTrigger.ChangeReplicasTrigger return s.multiraft.ChangeGroupMembership(cmd.RangeID, string(idKey), changeTypeInternalToRaft[crt.ChangeType], - proto.MakeRaftNodeID(crt.NodeID, crt.StoreID), + crt.Replica, data) } } @@ -1543,6 +1538,39 @@ func (s *Store) GroupStorage(groupID proto.RangeID) multiraft.WriteableGroupStor return r } +// ReplicaDescriptor implements the multiraft.Storage interface. +func (s *Store) ReplicaDescriptor(groupID proto.RangeID, replicaID proto.ReplicaID) (proto.ReplicaDescriptor, error) { + rep, err := s.GetReplica(groupID) + if err != nil { + return proto.ReplicaDescriptor{}, err + } + return rep.ReplicaDescriptor(replicaID) +} + +// ReplicaIDForStore implements the multiraft.Storage interface. +func (s *Store) ReplicaIDForStore(groupID proto.RangeID, storeID proto.StoreID) (proto.ReplicaID, error) { + r, err := s.GetReplica(groupID) + if err != nil { + return 0, err + } + for _, rep := range r.Desc().Replicas { + if rep.StoreID == storeID { + return rep.ReplicaID, nil + } + } + return 0, util.Errorf("store %s not found as replica of range %d", storeID, groupID) +} + +// ReplicasFromSnapshot implements the multiraft.Storage interface. +func (s *Store) ReplicasFromSnapshot(snap raftpb.Snapshot) ([]proto.ReplicaDescriptor, error) { + // TODO(bdarnell): can we avoid parsing this twice? + var parsedSnap proto.RaftSnapshotData + if err := parsedSnap.Unmarshal(snap.Data); err != nil { + return nil, err + } + return parsedSnap.RangeDescriptor.Replicas, nil +} + // AppliedIndex implements the multiraft.StateMachine interface. func (s *Store) AppliedIndex(groupID proto.RangeID) (uint64, error) { s.mu.RLock() diff --git a/storage/store_pool.go b/storage/store_pool.go index bff87914a409..2ca5bf40dcec 100644 --- a/storage/store_pool.go +++ b/storage/store_pool.go @@ -263,8 +263,8 @@ func (sp *StorePool) getStoreDescriptor(storeID proto.StoreID) *proto.StoreDescr // findDeadReplicas returns any replicas from the supplied slice that are // located on dead stores. -func (sp *StorePool) deadReplicas(repls []proto.Replica) []proto.Replica { - var deadReplicas []proto.Replica +func (sp *StorePool) deadReplicas(repls []proto.ReplicaDescriptor) []proto.ReplicaDescriptor { + var deadReplicas []proto.ReplicaDescriptor for _, repl := range repls { if sp.getStoreDetail(repl.StoreID).dead { deadReplicas = append(deadReplicas, repl) diff --git a/storage/store_pool_test.go b/storage/store_pool_test.go index fdb228c88371..561146678d14 100644 --- a/storage/store_pool_test.go +++ b/storage/store_pool_test.go @@ -324,7 +324,7 @@ func TestStorePoolFindDeadReplicas(t *testing.T) { }, } - replicas := []proto.Replica{ + replicas := []proto.ReplicaDescriptor{ { NodeID: 1, StoreID: 1, diff --git a/storage/store_test.go b/storage/store_test.go index d7d9c2a3a264..d8a7db4cf424 100644 --- a/storage/store_test.go +++ b/storage/store_test.go @@ -449,7 +449,7 @@ func TestStoreExecuteNoop(t *testing.T) { RequestHeader: proto.RequestHeader{ Key: nil, // intentional RangeID: 1, - Replica: proto.Replica{StoreID: store.StoreID()}, + Replica: proto.ReplicaDescriptor{StoreID: store.StoreID()}, }, } ba.Add(&proto.GetRequest{RequestHeader: proto.RequestHeader{Key: proto.Key("a")}}) @@ -670,7 +670,7 @@ func TestStoreRangeIDAllocation(t *testing.T) { // Range IDs should be allocated from ID 2 (first alloc'd range) // to rangeIDAllocCount * 3 + 1. for i := 0; i < rangeIDAllocCount*3; i++ { - replicas := []proto.Replica{{StoreID: store.StoreID()}} + replicas := []proto.ReplicaDescriptor{{StoreID: store.StoreID()}} desc, err := store.NewRangeDescriptor(proto.Key(fmt.Sprintf("%03d", i)), proto.Key(fmt.Sprintf("%03d", i+1)), replicas) if err != nil { t.Fatal(err) @@ -1370,51 +1370,6 @@ func TestStoreScanInconsistentResolvesIntents(t *testing.T) { }) } -func TestRaftNodeID(t *testing.T) { - defer leaktest.AfterTest(t) - cases := []struct { - nodeID proto.NodeID - storeID proto.StoreID - expected proto.RaftNodeID - }{ - {0, 1, 1}, - {1, 1, 0x100000001}, - {2, 3, 0x200000003}, - {math.MaxInt32, math.MaxInt32, 0x7fffffff7fffffff}, - } - for _, c := range cases { - x := proto.MakeRaftNodeID(c.nodeID, c.storeID) - if x != c.expected { - t.Errorf("makeRaftNodeID(%v, %v) returned %v; expected %v", - c.nodeID, c.storeID, x, c.expected) - } - n, s := proto.DecodeRaftNodeID(x) - if n != c.nodeID || s != c.storeID { - t.Errorf("decodeRaftNodeID(%v) returned %v, %v; expected %v, %v", - x, n, s, c.nodeID, c.storeID) - } - } - - panicCases := []struct { - nodeID proto.NodeID - storeID proto.StoreID - }{ - {1, 0}, - {1, -1}, - {-1, 1}, - } - for _, c := range panicCases { - func() { - defer func() { - _ = recover() - }() - x := proto.MakeRaftNodeID(c.nodeID, c.storeID) - t.Errorf("makeRaftNodeID(%v, %v) returned %v; expected panic", - c.nodeID, c.storeID, x) - }() - } -} - // TestStoreBadRequests verifies that ExecuteCmd returns errors for // bad requests that do not pass key verification. //