Skip to content

Commit

Permalink
sql: use deprecated getters for sys config
Browse files Browse the repository at this point in the history
The system config is one main deprecated usage of Gossip. Give it a
special getter to make this easier to see.

Release note: None
  • Loading branch information
tbg committed May 7, 2020
1 parent 0310c63 commit c37f28f
Show file tree
Hide file tree
Showing 7 changed files with 32 additions and 15 deletions.
22 changes: 22 additions & 0 deletions pkg/gossip/gossip.go
Original file line number Diff line number Diff line change
Expand Up @@ -1649,6 +1649,28 @@ func (dg DeprecatedGossip) Deprecated(issueNo int) *Gossip {
return g
}

// DeprecatedSystemConfig calls GetSystemConfig on the wrapped Gossip instance.
//
// Use of Gossip from within the SQL layer is **deprecated**. Please do not
// introduce new uses of it.
func (dg DeprecatedGossip) DeprecatedSystemConfig(issueNo int) *config.SystemConfig {
g := dg.Deprecated(issueNo)
if g == nil {
return nil // a few unit tests
}
return g.GetSystemConfig()
}

// DeprecatedRegisterSystemConfigChannel calls RegisterSystemConfigChannel on
// the wrapped Gossip instance.
//
// Use of Gossip from within the SQL layer is **deprecated**. Please do not
// introduce new uses of it.
func (dg DeprecatedGossip) DeprecatedRegisterSystemConfigChannel(issueNo int) <-chan struct{} {
g := dg.Deprecated(issueNo)
return g.RegisterSystemConfigChannel()
}

// OptionalErr returns the Gossip instance if the wrapper was set up to allow
// it. Otherwise, it returns an error referring to the optionally passed in
// issues.
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -535,7 +535,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
)
execCfg.StmtDiagnosticsRecorder = stmtDiagnosticsRegistry

leaseMgr.RefreshLeases(cfg.stopper, cfg.db, cfg.gossip.Deprecated(47150))
leaseMgr.RefreshLeases(cfg.stopper, cfg.db, cfg.gossip)
leaseMgr.PeriodicallyRefreshSomeLeases()

temporaryObjectCleaner := sql.NewTemporaryObjectCleaner(
Expand Down
5 changes: 2 additions & 3 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -318,13 +318,12 @@ func makeMetrics(internal bool) Metrics {

// Start starts the Server's background processing.
func (s *Server) Start(ctx context.Context, stopper *stop.Stopper) {
g := s.cfg.Gossip.Deprecated(47150)
gossipUpdateC := g.RegisterSystemConfigChannel()
gossipUpdateC := s.cfg.Gossip.DeprecatedRegisterSystemConfigChannel(47150)
stopper.RunWorker(ctx, func(ctx context.Context) {
for {
select {
case <-gossipUpdateC:
sysCfg := g.GetSystemConfig()
sysCfg := s.cfg.Gossip.DeprecatedSystemConfig(47150)
s.dbCache.updateSystemConfig(sysCfg)
case <-stopper.ShouldStop():
return
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/gcjob/gc_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ func (r schemaChangeGCResumer) Resume(
// TTL whenever we get an update on one of the tables/indexes (or the db)
// that this job is responsible for, and computing the earliest deadline
// from our set of cached TTL values.
cfg := execCfg.Gossip.Deprecated(47150).GetSystemConfig()
cfg := execCfg.Gossip.DeprecatedSystemConfig(47150)
zoneConfigUpdated := false
zoneCfgFilter.ForModified(cfg, func(kv roachpb.KeyValue) {
zoneConfigUpdated = true
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/gcjob/refresh_statuses.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ func updateStatusForGCElements(
progress *jobspb.SchemaChangeGCProgress,
) (expired bool, timeToNextTrigger time.Time) {
defTTL := execCfg.DefaultZoneConfig.GC.TTLSeconds
cfg := execCfg.Gossip.Deprecated(47150).GetSystemConfig()
cfg := execCfg.Gossip.DeprecatedSystemConfig(47150)
protectedtsCache := execCfg.ProtectedTimestampProvider

earliestDeadline := timeutil.Unix(0, int64(math.MaxInt64))
Expand Down Expand Up @@ -275,6 +275,6 @@ func setupConfigWatcher(
) (gossip.SystemConfigDeltaFilter, <-chan struct{}) {
k := execCfg.Codec.IndexPrefix(uint32(keys.ZonesTableID), uint32(keys.ZonesTablePrimaryIndexID))
zoneCfgFilter := gossip.MakeSystemConfigDeltaFilter(k)
gossipUpdateC := execCfg.Gossip.Deprecated(47150).RegisterSystemConfigChannel()
gossipUpdateC := execCfg.Gossip.DeprecatedRegisterSystemConfigChannel(47150)
return zoneCfgFilter, gossipUpdateC
}
6 changes: 3 additions & 3 deletions pkg/sql/lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -1773,16 +1773,16 @@ func (m *LeaseManager) findTableState(tableID sqlbase.ID, create bool) *tableSta

// RefreshLeases starts a goroutine that refreshes the lease manager
// leases for tables received in the latest system configuration via gossip.
func (m *LeaseManager) RefreshLeases(s *stop.Stopper, db *kv.DB, g *gossip.Gossip) {
func (m *LeaseManager) RefreshLeases(s *stop.Stopper, db *kv.DB, gw gossip.DeprecatedGossip) {
ctx := context.TODO()
s.RunWorker(ctx, func(ctx context.Context) {
descKeyPrefix := m.codec.TablePrefix(uint32(sqlbase.DescriptorTable.ID))
cfgFilter := gossip.MakeSystemConfigDeltaFilter(descKeyPrefix)
gossipUpdateC := g.RegisterSystemConfigChannel()
gossipUpdateC := gw.DeprecatedRegisterSystemConfigChannel(47150)
for {
select {
case <-gossipUpdateC:
cfg := g.GetSystemConfig()
cfg := gw.DeprecatedSystemConfig(47150)
if m.testingKnobs.GossipUpdateEvent != nil {
if err := m.testingKnobs.GossipUpdateEvent(cfg); err != nil {
break
Expand Down
6 changes: 1 addition & 5 deletions pkg/sql/opt_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -74,11 +74,7 @@ func (oc *optCatalog) reset() {
oc.dataSources = make(map[*sqlbase.ImmutableTableDescriptor]cat.DataSource)
}

g := oc.planner.execCfg.Gossip.Deprecated(47150)
// Gossip can be nil in testing scenarios.
if g != nil {
oc.cfg = g.GetSystemConfig()
}
oc.cfg = oc.planner.execCfg.Gossip.DeprecatedSystemConfig(47150)
}

// optSchema is a wrapper around sqlbase.DatabaseDescriptor that implements the
Expand Down

0 comments on commit c37f28f

Please sign in to comment.