Skip to content

Commit

Permalink
Merge #83710
Browse files Browse the repository at this point in the history
83710: gossip: rename a few gossiped key methods r=tbg a=tbg

The "StoreID" and "NodeID" keys were misleading as they didn't actually
gossip the ID (rather, the value is a {Node,Store}Descriptor).

We're about to have another store-scoped key (#79215) and this cleanup
helps with that.

This commit is entirely mechanical IDE renames.

Release note: None


Co-authored-by: Tobias Grieger <[email protected]>
  • Loading branch information
craig[bot] and tbg committed Jul 5, 2022
2 parents 933b684 + 4488445 commit c897707
Show file tree
Hide file tree
Showing 21 changed files with 63 additions and 63 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ func clusterNodeCount(gw gossip.OptionalGossip) (int, error) {
}
var nodes int
err = g.IterateInfos(
gossip.KeyNodeIDPrefix, func(_ string, _ gossip.Info) error {
gossip.KeyNodeDescPrefix, func(_ string, _ gossip.Info) error {
nodes++
return nil
},
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/kvfeed/scanner.go
Original file line number Diff line number Diff line change
Expand Up @@ -298,7 +298,7 @@ func clusterNodeCount(gw gossip.OptionalGossip) int {
return 1
}
var nodes int
_ = g.IterateInfos(gossip.KeyNodeIDPrefix, func(_ string, _ gossip.Info) error {
_ = g.IterateInfos(gossip.KeyNodeDescPrefix, func(_ string, _ gossip.Info) error {
nodes++
return nil
})
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/kvccl/kvtenantccl/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -258,7 +258,7 @@ var gossipSubsHandlers = map[string]func(*Connector, context.Context, string, ro
// Subscribe to the ClusterID update.
gossip.KeyClusterID: (*Connector).updateClusterID,
// Subscribe to all *NodeDescriptor updates.
gossip.MakePrefixPattern(gossip.KeyNodeIDPrefix): (*Connector).updateNodeAddress,
gossip.MakePrefixPattern(gossip.KeyNodeDescPrefix): (*Connector).updateNodeAddress,
// Subscribe to a filtered view of *SystemConfig updates.
gossip.KeyDeprecatedSystemConfig: (*Connector).updateSystemConfig,
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/kvccl/kvtenantccl/connector_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ func gossipEventForNodeDesc(desc *roachpb.NodeDescriptor) *roachpb.GossipSubscri
return &roachpb.GossipSubscriptionEvent{
Key: gossip.MakeNodeIDKey(desc.NodeID),
Content: roachpb.MakeValueFromBytesAndTimestamp(val, hlc.Timestamp{}),
PatternMatched: gossip.MakePrefixPattern(gossip.KeyNodeIDPrefix),
PatternMatched: gossip.MakePrefixPattern(gossip.KeyNodeDescPrefix),
}
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -969,13 +969,13 @@ func parseGossipValues(gossipInfo *gossip.InfoStatus) (string, error) {
return "", errors.Wrapf(err, "failed to parse value for key %q", key)
}
output = append(output, fmt.Sprintf("%q: %v", key, desc))
} else if gossip.IsNodeIDKey(key) {
} else if gossip.IsNodeDescKey(key) {
var desc roachpb.NodeDescriptor
if err := protoutil.Unmarshal(bytes, &desc); err != nil {
return "", errors.Wrapf(err, "failed to parse value for key %q", key)
}
output = append(output, fmt.Sprintf("%q: %+v", key, desc))
} else if strings.HasPrefix(key, gossip.KeyStorePrefix) {
} else if strings.HasPrefix(key, gossip.KeyStoreDescPrefix) {
var desc roachpb.StoreDescriptor
if err := protoutil.Unmarshal(bytes, &desc); err != nil {
return "", errors.Wrapf(err, "failed to parse value for key %q", key)
Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/gossip.go
Original file line number Diff line number Diff line change
Expand Up @@ -218,7 +218,7 @@ func (gossipUtil) hasPeers(expected int) checkGossipFunc {
return func(infos map[string]gossip.Info) error {
count := 0
for k := range infos {
if strings.HasPrefix(k, gossip.KeyNodeIDPrefix) {
if strings.HasPrefix(k, gossip.KeyNodeDescPrefix) {
count++
}
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/gossip/gossip.go
Original file line number Diff line number Diff line change
Expand Up @@ -331,8 +331,8 @@ func New(
// Add ourselves as a SystemConfig watcher.
g.mu.is.registerCallback(KeyDeprecatedSystemConfig, g.updateSystemConfig)
// Add ourselves as a node descriptor watcher.
g.mu.is.registerCallback(MakePrefixPattern(KeyNodeIDPrefix), g.updateNodeAddress)
g.mu.is.registerCallback(MakePrefixPattern(KeyStorePrefix), g.updateStoreMap)
g.mu.is.registerCallback(MakePrefixPattern(KeyNodeDescPrefix), g.updateNodeAddress)
g.mu.is.registerCallback(MakePrefixPattern(KeyStoreDescPrefix), g.updateStoreMap)
// Log gossip connectivity whenever we receive an update.
g.mu.Unlock()

Expand Down Expand Up @@ -741,7 +741,7 @@ func (g *Gossip) maybeCleanupBootstrapAddressesLocked() {

var desc roachpb.NodeDescriptor
if err := g.mu.is.visitInfos(func(key string, i *Info) error {
if strings.HasPrefix(key, KeyNodeIDPrefix) {
if strings.HasPrefix(key, KeyNodeDescPrefix) {
if err := i.Value.GetProto(&desc); err != nil {
return err
}
Expand Down Expand Up @@ -817,7 +817,7 @@ func (g *Gossip) updateNodeAddress(key string, content roachpb.Value) {
// We can't directly compare the node against the empty descriptor because
// the proto has a repeated field and thus isn't comparable.
if desc.NodeID == 0 || desc.Address.IsEmpty() {
nodeID, err := NodeIDFromKey(key, KeyNodeIDPrefix)
nodeID, err := DecodeNodeDescKey(key, KeyNodeDescPrefix)
if err != nil {
log.Health.Errorf(ctx, "unable to update node address for removed node: %s", err)
return
Expand Down
2 changes: 1 addition & 1 deletion pkg/gossip/gossip_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -427,7 +427,7 @@ func TestGossipMostDistant(t *testing.T) {
g.mu.Lock()
var buf bytes.Buffer
_ = g.mu.is.visitInfos(func(key string, i *Info) error {
if i.NodeID != 1 && IsNodeIDKey(key) {
if i.NodeID != 1 && IsNodeDescKey(key) {
fmt.Fprintf(&buf, "n%d: hops=%d\n", i.NodeID, i.Hops)
}
return nil
Expand Down
4 changes: 2 additions & 2 deletions pkg/gossip/infostore.go
Original file line number Diff line number Diff line change
Expand Up @@ -435,7 +435,7 @@ func (is *infoStore) delta(highWaterTimestamps map[roachpb.NodeID]int64) map[str
// propagated regardless of high water stamps.
func (is *infoStore) populateMostDistantMarkers(infos map[string]*Info) {
if err := is.visitInfos(func(key string, i *Info) error {
if IsNodeIDKey(key) {
if IsNodeDescKey(key) {
infos[key] = i
}
return nil
Expand Down Expand Up @@ -467,7 +467,7 @@ func (is *infoStore) mostDistant(
// acquire unreliably high Hops values in some pathological cases such as
// those described in #9819.
if i.NodeID != localNodeID && i.Hops > maxHops &&
IsNodeIDKey(key) && !hasOutgoingConn(i.NodeID) {
IsNodeDescKey(key) && !hasOutgoingConn(i.NodeID) {
maxHops = i.Hops
nodeID = i.NodeID
}
Expand Down
32 changes: 16 additions & 16 deletions pkg/gossip/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,14 +35,14 @@ const (
// Gossip.Connected channel is closed when we see this key.
KeyClusterID = "cluster-id"

// KeyStorePrefix is the key prefix for gossiping stores in the network.
// KeyStoreDescPrefix is the key prefix for gossiping stores in the network.
// The suffix is a store ID and the value is a roachpb.StoreDescriptor.
KeyStorePrefix = "store"
KeyStoreDescPrefix = "store"

// KeyNodeIDPrefix is the key prefix for gossiping node id addresses.
// KeyNodeDescPrefix is the key prefix for gossiping node id addresses.
// The actual key is suffixed with the decimal representation of the
// node id (e.g. 'node:1') and the value is a roachpb.NodeDescriptor.
KeyNodeIDPrefix = "node"
KeyNodeDescPrefix = "node"

// KeyHealthAlertPrefix is the key prefix for gossiping health alerts.
// The value is a proto of type HealthCheckResult.
Expand Down Expand Up @@ -121,18 +121,18 @@ func MakePrefixPattern(prefix string) string {

// MakeNodeIDKey returns the gossip key for node ID info.
func MakeNodeIDKey(nodeID roachpb.NodeID) string {
return MakeKey(KeyNodeIDPrefix, nodeID.String())
return MakeKey(KeyNodeDescPrefix, nodeID.String())
}

// IsNodeIDKey returns true iff the provided key is a valid node ID key.
func IsNodeIDKey(key string) bool {
return strings.HasPrefix(key, KeyNodeIDPrefix+separator)
// IsNodeDescKey returns true iff the provided key is a valid node ID key.
func IsNodeDescKey(key string) bool {
return strings.HasPrefix(key, KeyNodeDescPrefix+separator)
}

// NodeIDFromKey attempts to extract a NodeID from the provided key after
// DecodeNodeDescKey attempts to extract a NodeID from the provided key after
// stripping the provided prefix. Returns an error if the key is not of the
// correct type or is not parsable.
func NodeIDFromKey(key string, prefix string) (roachpb.NodeID, error) {
func DecodeNodeDescKey(key string, prefix string) (roachpb.NodeID, error) {
trimmedKey, err := removePrefixFromKey(key, prefix)
if err != nil {
return 0, err
Expand Down Expand Up @@ -160,16 +160,16 @@ func MakeNodeLivenessKey(nodeID roachpb.NodeID) string {
return MakeKey(KeyNodeLivenessPrefix, nodeID.String())
}

// MakeStoreKey returns the gossip key for the given store.
func MakeStoreKey(storeID roachpb.StoreID) string {
return MakeKey(KeyStorePrefix, storeID.String())
// MakeStoreDescKey returns the gossip key for the given store.
func MakeStoreDescKey(storeID roachpb.StoreID) string {
return MakeKey(KeyStoreDescPrefix, storeID.String())
}

// StoreIDFromKey attempts to extract a StoreID from the provided key after
// DecodeStoreDescKey attempts to extract a StoreID from the provided key after
// stripping the provided prefix. Returns an error if the key is not of the
// correct type or is not parsable.
func StoreIDFromKey(storeKey string) (roachpb.StoreID, error) {
trimmedKey, err := removePrefixFromKey(storeKey, KeyStorePrefix)
func DecodeStoreDescKey(storeKey string) (roachpb.StoreID, error) {
trimmedKey, err := removePrefixFromKey(storeKey, KeyStoreDescPrefix)
if err != nil {
return 0, err
}
Expand Down
28 changes: 14 additions & 14 deletions pkg/gossip/keys_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,18 +30,18 @@ func TestNodeIDFromKey(t *testing.T) {
{MakeNodeIDKey(123), 123, true},
{MakeNodeIDKey(123) + "foo", 0, false},
{"foo" + MakeNodeIDKey(123), 0, false},
{KeyNodeIDPrefix, 0, false},
{KeyNodeIDPrefix + ":", 0, false},
{KeyNodeIDPrefix + ":foo", 0, false},
{KeyNodeDescPrefix, 0, false},
{KeyNodeDescPrefix + ":", 0, false},
{KeyNodeDescPrefix + ":foo", 0, false},
{"123", 0, false},
{MakePrefixPattern(KeyNodeIDPrefix), 0, false},
{MakePrefixPattern(KeyNodeDescPrefix), 0, false},
{MakeNodeLivenessKey(1), 0, false},
{MakeStoreKey(1), 0, false},
{MakeStoreDescKey(1), 0, false},
}

for _, tc := range testCases {
t.Run(tc.key, func(t *testing.T) {
nodeID, err := NodeIDFromKey(tc.key, KeyNodeIDPrefix)
nodeID, err := DecodeNodeDescKey(tc.key, KeyNodeDescPrefix)
if err != nil {
if tc.success {
t.Errorf("expected success, got error: %s", err)
Expand All @@ -63,19 +63,19 @@ func TestStoreIDFromKey(t *testing.T) {
storeID roachpb.StoreID
success bool
}{
{MakeStoreKey(0), 0, true},
{MakeStoreKey(1), 1, true},
{MakeStoreKey(123), 123, true},
{MakeStoreKey(123) + "foo", 0, false},
{"foo" + MakeStoreKey(123), 0, false},
{KeyStorePrefix, 0, false},
{MakeStoreDescKey(0), 0, true},
{MakeStoreDescKey(1), 1, true},
{MakeStoreDescKey(123), 123, true},
{MakeStoreDescKey(123) + "foo", 0, false},
{"foo" + MakeStoreDescKey(123), 0, false},
{KeyStoreDescPrefix, 0, false},
{"123", 0, false},
{MakePrefixPattern(KeyStorePrefix), 0, false},
{MakePrefixPattern(KeyStoreDescPrefix), 0, false},
}

for _, tc := range testCases {
t.Run(tc.key, func(t *testing.T) {
storeID, err := StoreIDFromKey(tc.key)
storeID, err := DecodeStoreDescKey(tc.key)
if err != nil {
if tc.success {
t.Errorf("expected success, got error: %s", err)
Expand Down
10 changes: 5 additions & 5 deletions pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7597,7 +7597,7 @@ func TestAllocatorFullDisks(t *testing.T) {
}, nil)

var wg sync.WaitGroup
g.RegisterCallback(gossip.MakePrefixPattern(gossip.KeyStorePrefix),
g.RegisterCallback(gossip.MakePrefixPattern(gossip.KeyStoreDescPrefix),
func(_ string, _ roachpb.Value) { wg.Done() },
// Redundant callbacks are required by this test.
gossip.Redundant)
Expand Down Expand Up @@ -7640,7 +7640,7 @@ func TestAllocatorFullDisks(t *testing.T) {
mockNodeLiveness.SetNodeStatus(roachpb.NodeID(j), livenesspb.NodeLivenessStatus_DEAD)
}
wg.Add(1)
if err := g.AddInfoProto(gossip.MakeStoreKey(roachpb.StoreID(j)), &ts.StoreDescriptor, 0); err != nil {
if err := g.AddInfoProto(gossip.MakeStoreDescKey(roachpb.StoreID(j)), &ts.StoreDescriptor, 0); err != nil {
t.Fatal(err)
}
}
Expand Down Expand Up @@ -7676,7 +7676,7 @@ func TestAllocatorFullDisks(t *testing.T) {
// Gossip occasionally, as real Stores do when replicas move around.
if j%3 == 2 {
wg.Add(1)
if err := g.AddInfoProto(gossip.MakeStoreKey(roachpb.StoreID(j)), &ts.StoreDescriptor, 0); err != nil {
if err := g.AddInfoProto(gossip.MakeStoreDescKey(roachpb.StoreID(j)), &ts.StoreDescriptor, 0); err != nil {
t.Fatal(err)
}
}
Expand Down Expand Up @@ -8043,7 +8043,7 @@ func exampleRebalancing(
}, nil)

var wg sync.WaitGroup
g.RegisterCallback(gossip.MakePrefixPattern(gossip.KeyStorePrefix),
g.RegisterCallback(gossip.MakePrefixPattern(gossip.KeyStoreDescPrefix),
func(_ string, _ roachpb.Value) { wg.Done() },
// Redundant callbacks are required by this test.
gossip.Redundant)
Expand Down Expand Up @@ -8076,7 +8076,7 @@ func exampleRebalancing(
testStores[j].add(alloc.randGen.Int63n(1<<20), 0)
}
if err := g.AddInfoProto(
gossip.MakeStoreKey(roachpb.StoreID(j)),
gossip.MakeStoreDescKey(roachpb.StoreID(j)),
&testStores[j].StoreDescriptor,
0,
); err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/allocator/storepool/store_pool.go
Original file line number Diff line number Diff line change
Expand Up @@ -401,7 +401,7 @@ func NewStorePool(
// Enable redundant callbacks for the store keys because we use these
// callbacks as a clock to determine when a store was last updated even if it
// hasn't otherwise changed.
storeRegex := gossip.MakePrefixPattern(gossip.KeyStorePrefix)
storeRegex := gossip.MakePrefixPattern(gossip.KeyStoreDescPrefix)
g.RegisterCallback(storeRegex, sp.storeGossipUpdate, gossip.Redundant)

return sp
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2260,7 +2260,7 @@ func TestStoreRangeGossipOnSplits(t *testing.T) {
defer s.Stopper().Stop(ctx)
store, err := s.Stores().GetStore(s.GetFirstStoreID())
require.NoError(t, err)
storeKey := gossip.MakeStoreKey(store.StoreID())
storeKey := gossip.MakeStoreDescKey(store.StoreID())

// Avoid excessive logging on under-replicated ranges due to our many splits.
config.TestingSetupZoneConfigHook(s.Stopper())
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/replicate_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -406,14 +406,14 @@ func newReplicateQueue(store *Store, allocator allocatorimpl.Allocator) *replica
// Register gossip and node liveness callbacks to signal that
// replicas in purgatory might be retried.
if g := store.cfg.Gossip; g != nil { // gossip is nil for some unittests
g.RegisterCallback(gossip.MakePrefixPattern(gossip.KeyStorePrefix), func(key string, _ roachpb.Value) {
g.RegisterCallback(gossip.MakePrefixPattern(gossip.KeyStoreDescPrefix), func(key string, _ roachpb.Value) {
if !rq.store.IsStarted() {
return
}
// Because updates to our store's own descriptor won't affect
// replicas in purgatory, skip updating the purgatory channel
// in this case.
if storeID, err := gossip.StoreIDFromKey(key); err == nil && storeID == rq.store.StoreID() {
if storeID, err := gossip.DecodeStoreDescKey(key); err == nil && storeID == rq.store.StoreID() {
return
}
updateFn()
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -2533,7 +2533,7 @@ func (s *Store) GossipStore(ctx context.Context, useCached bool) error {
syncutil.StoreFloat64(&s.gossipWritesPerSecondVal, storeDesc.Capacity.WritesPerSecond)

// Unique gossip key per store.
gossipStoreKey := gossip.MakeStoreKey(storeDesc.StoreID)
gossipStoreKey := gossip.MakeStoreDescKey(storeDesc.StoreID)
// Gossip store descriptor.
return s.cfg.Gossip.AddInfoProto(gossipStoreKey, storeDesc, gossip.StoreTTL)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -885,7 +885,7 @@ func (n *Node) writeNodeStatus(ctx context.Context, alertTTL time.Duration, must

if result := n.recorder.CheckHealth(ctx, *nodeStatus); len(result.Alerts) != 0 {
var numNodes int
if err := n.storeCfg.Gossip.IterateInfos(gossip.KeyNodeIDPrefix, func(k string, info gossip.Info) error {
if err := n.storeCfg.Gossip.IterateInfos(gossip.KeyNodeDescPrefix, func(k string, info gossip.Info) error {
numNodes++
return nil
}); err != nil {
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -3703,7 +3703,7 @@ func getAllNodeDescriptors(p *planner) ([]roachpb.NodeDescriptor, error) {
return nil, err
}
var descriptors []roachpb.NodeDescriptor
if err := g.IterateInfos(gossip.KeyNodeIDPrefix, func(key string, i gossip.Info) error {
if err := g.IterateInfos(gossip.KeyNodeDescPrefix, func(key string, i gossip.Info) error {
bytes, err := i.Value.GetBytes()
if err != nil {
return errors.NewAssertionErrorWithWrappedErrf(err,
Expand Down Expand Up @@ -3783,7 +3783,7 @@ CREATE TABLE crdb_internal.gossip_nodes (
}

stats := make(map[roachpb.NodeID]nodeStats)
if err := g.IterateInfos(gossip.KeyStorePrefix, func(key string, i gossip.Info) error {
if err := g.IterateInfos(gossip.KeyStoreDescPrefix, func(key string, i gossip.Info) error {
bytes, err := i.Value.GetBytes()
if err != nil {
return errors.NewAssertionErrorWithWrappedErrf(err,
Expand Down Expand Up @@ -4027,7 +4027,7 @@ CREATE TABLE crdb_internal.gossip_alerts (
return errors.NewAssertionErrorWithWrappedErrf(err,
"failed to parse value for key %q", key)
}
nodeID, err := gossip.NodeIDFromKey(key, gossip.KeyNodeHealthAlertPrefix)
nodeID, err := gossip.DecodeNodeDescKey(key, gossip.KeyNodeHealthAlertPrefix)
if err != nil {
return errors.NewAssertionErrorWithWrappedErrf(err,
"failed to parse node ID from key %q", key)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/relocate.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ func (n *relocateNode) Close(ctx context.Context) {

func lookupStoreDesc(storeID roachpb.StoreID, params runParams) (*roachpb.StoreDescriptor, error) {
var storeDesc roachpb.StoreDescriptor
gossipStoreKey := gossip.MakeStoreKey(storeID)
gossipStoreKey := gossip.MakeStoreDescKey(storeID)
g, err := params.extendedEvalCtx.ExecCfg.Gossip.OptionalErr(54250)
if err != nil {
return nil, err
Expand Down
6 changes: 3 additions & 3 deletions pkg/testutils/gossiputil/store_gossiper.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ func NewStoreGossiper(g *gossip.Gossip) *StoreGossiper {
sg.cond = sync.NewCond(&sg.mu)
// Redundant callbacks are required by StoreGossiper. See GossipWithFunction
// which waits for all of the callbacks to be invoked.
g.RegisterCallback(gossip.MakePrefixPattern(gossip.KeyStorePrefix), func(key string, _ roachpb.Value) {
g.RegisterCallback(gossip.MakePrefixPattern(gossip.KeyStoreDescPrefix), func(key string, _ roachpb.Value) {
sg.mu.Lock()
defer sg.mu.Unlock()
delete(sg.storeKeyMap, key)
Expand All @@ -57,7 +57,7 @@ func (sg *StoreGossiper) GossipStores(storeDescs []*roachpb.StoreDescriptor, t *
}
sg.GossipWithFunction(storeIDs, func() {
for i, storeDesc := range storeDescs {
if err := sg.g.AddInfoProto(gossip.MakeStoreKey(storeIDs[i]), storeDesc, 0); err != nil {
if err := sg.g.AddInfoProto(gossip.MakeStoreDescKey(storeIDs[i]), storeDesc, 0); err != nil {
t.Fatal(err)
}
}
Expand All @@ -71,7 +71,7 @@ func (sg *StoreGossiper) GossipWithFunction(storeIDs []roachpb.StoreID, gossipFn
defer sg.mu.Unlock()
sg.storeKeyMap = make(map[string]struct{})
for _, storeID := range storeIDs {
storeKey := gossip.MakeStoreKey(storeID)
storeKey := gossip.MakeStoreDescKey(storeID)
sg.storeKeyMap[storeKey] = struct{}{}
}

Expand Down
Loading

0 comments on commit c897707

Please sign in to comment.