Skip to content

Commit

Permalink
sql: remove remnant of gossip-based stats cache update
Browse files Browse the repository at this point in the history
This change removes the last remnant of the gossip-based stats cache
invalidation. This code was necessary only during 21.1->21.2 upgrade.

Release note: None
  • Loading branch information
RaduBerinde committed Oct 14, 2021
1 parent 1013cdb commit d07b32d
Show file tree
Hide file tree
Showing 6 changed files with 5 additions and 46 deletions.
3 changes: 0 additions & 3 deletions pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -955,9 +955,6 @@ 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, drainingInfo))
} else if strings.HasPrefix(key, gossip.KeyTableStatAddedPrefix) {
gossipedTime := timeutil.Unix(0, info.OrigStamp)
output = append(output, fmt.Sprintf("%q: %v", key, gossipedTime))
} else if strings.HasPrefix(key, gossip.KeyGossipClientsPrefix) {
output = append(output, fmt.Sprintf("%q: %v", key, string(bytes)))
}
Expand Down
11 changes: 0 additions & 11 deletions pkg/gossip/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,11 +75,6 @@ const (
// draining state.
KeyDistSQLDrainingPrefix = "distsql-draining"

// KeyTableStatAddedPrefix is the prefix for keys that indicate a new table
// statistic was computed. The statistics themselves are not stored in gossip;
// the keys are used to notify nodes to invalidate table statistic caches.
KeyTableStatAddedPrefix = "table-stat-added"

// KeyGossipClientsPrefix is the prefix for keys that indicate which gossip
// client connections a node has open. This is used by other nodes in the
// cluster to build a map of the gossip network.
Expand Down Expand Up @@ -180,12 +175,6 @@ func MakeDistSQLDrainingKey(nodeID roachpb.NodeID) string {
return MakeKey(KeyDistSQLDrainingPrefix, nodeID.String())
}

// MakeTableStatAddedKey returns the gossip key used to notify that a new
// statistic is available for the given table.
func MakeTableStatAddedKey(tableID uint32) string {
return MakeKey(KeyTableStatAddedPrefix, strconv.FormatUint(uint64(tableID), 10 /* base */))
}

// removePrefixFromKey removes the key prefix and separator and returns what's
// left. Returns an error if the key doesn't have this prefix.
func removePrefixFromKey(key, prefix string) (string, error) {
Expand Down
8 changes: 2 additions & 6 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1537,14 +1537,10 @@ func injectTableStats(
}

// Invalidate the local cache synchronously; this guarantees that the next
// statement in the same session won't use a stale cache (whereas the gossip
// update is handled asynchronously).
// statement in the same session won't use a stale cache (the cache would
// normally be updated asynchronously).
params.extendedEvalCtx.ExecCfg.TableStatsCache.InvalidateTableStats(params.ctx, desc.GetID())

// Use Gossip to refresh the caches on other nodes.
if g, ok := params.extendedEvalCtx.ExecCfg.Gossip.Optional(47925); ok {
return stats.GossipTableStatAdded(g, desc.GetID())
}
return nil
}

Expand Down
4 changes: 0 additions & 4 deletions pkg/sql/rowexec/sample_aggregator.go
Original file line number Diff line number Diff line change
Expand Up @@ -516,10 +516,6 @@ func (s *sampleAggregator) writeResults(ctx context.Context) error {
return err
}

if g, ok := s.FlowCtx.Cfg.Gossip.Optional(47925); ok {
// Gossip refresh of the stat caches for this table.
return stats.GossipTableStatAdded(g, s.tableID)
}
return nil
}

Expand Down
1 change: 0 additions & 1 deletion pkg/sql/stats/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/stats",
visibility = ["//visibility:public"],
deps = [
"//pkg/gossip",
"//pkg/jobs/jobspb",
"//pkg/keys",
"//pkg/kv",
Expand Down
24 changes: 3 additions & 21 deletions pkg/sql/stats/new_stat.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ package stats
import (
"context"

"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -52,8 +51,9 @@ func InsertNewStats(
}

// InsertNewStat inserts a new statistic in the system table.
// The caller is responsible for calling GossipTableStatAdded to notify the stat
// caches.
//
// The stats cache will automatically update asynchronously (as well as the
// stats caches on all other nodes).
func InsertNewStat(
ctx context.Context,
executor sqlutil.InternalExecutor,
Expand Down Expand Up @@ -105,21 +105,3 @@ func InsertNewStat(
)
return err
}

// GossipTableStatAdded causes the statistic caches for this table to be
// invalidated.
//
// Note that we no longer use gossip to keep the cache up-to-date, but we still
// send the updates for mixed-version clusters during upgrade.
//
// TODO(radu): remove this in 22.1.
func GossipTableStatAdded(g *gossip.Gossip, tableID descpb.ID) error {
// TODO(radu): perhaps use a TTL here to avoid having a key per table floating
// around forever (we would need the stat cache to evict old entries
// automatically though).
return g.AddInfo(
gossip.MakeTableStatAddedKey(uint32(tableID)),
nil, /* value */
0, /* ttl */
)
}

0 comments on commit d07b32d

Please sign in to comment.