Skip to content

Commit

Permalink
sql: emit more tracing events from the stats cache
Browse files Browse the repository at this point in the history
The stats cache has various "slow" paths (where we need to query the
system table). These are currently only logged if verbosity is high.

This change switches to `VEvent` in most cases, so that these are
visible during tracing (including in statement diagnostics bundles).
This will allow us to diagnose slow planning times, e.g. due to the
stats cache getting full.

Release justification: low-risk change to existing functionality, high
potential benefit for debugging issues.

Release note: None
  • Loading branch information
RaduBerinde committed Sep 9, 2020
1 parent 1225fe8 commit 322198e
Showing 1 changed file with 15 additions and 18 deletions.
33 changes: 15 additions & 18 deletions pkg/sql/stats/stats_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -188,11 +189,11 @@ func (sc *TableStatisticsCache) lookupStatsLocked(
if e.mustWait {
// We are in the process of grabbing stats for this table. Wait until
// that is complete, at which point e.stats will be populated.
if log.V(1) {
log.Infof(ctx, "waiting for statistics for table %d", tableID)
}
log.VEventf(ctx, 1, "waiting for statistics for table %d", tableID)
e.waitCond.Wait()
log.VEventf(ctx, 1, "finished waiting for statistics for table %d", tableID)
} else {
// This is the expected "fast" path; don't emit an event.
if log.V(2) {
log.Infof(ctx, "statistics for table %d found in cache", tableID)
}
Expand All @@ -211,10 +212,6 @@ func (sc *TableStatisticsCache) lookupStatsLocked(
func (sc *TableStatisticsCache) addCacheEntryLocked(
ctx context.Context, tableID descpb.ID,
) (stats []*TableStatistic, err error) {
if log.V(1) {
log.Infof(ctx, "reading statistics for table %d", tableID)
}

// Add a cache entry that other queries can find and wait on until we have the
// stats.
e := &cacheEntry{
Expand All @@ -228,7 +225,9 @@ func (sc *TableStatisticsCache) addCacheEntryLocked(
sc.mu.Unlock()
defer sc.mu.Lock()

log.VEventf(ctx, 1, "reading statistics for table %d", tableID)
stats, err = sc.getTableStatsFromDB(ctx, tableID)
log.VEventf(ctx, 1, "finished reading statistics for table %d", tableID)
}()

e.mustWait = false
Expand Down Expand Up @@ -258,10 +257,6 @@ func (sc *TableStatisticsCache) refreshCacheEntry(ctx context.Context, tableID d
sc.mu.Lock()
defer sc.mu.Unlock()

if log.V(1) {
log.Infof(ctx, "reading statistics for table %d", tableID)
}

// If the stats don't already exist in the cache, don't bother performing
// the refresh. If e.err is not nil, the stats are in the process of being
// removed from the cache (see addCacheEntryLocked), so don't refresh in this
Expand All @@ -287,7 +282,9 @@ func (sc *TableStatisticsCache) refreshCacheEntry(ctx context.Context, tableID d
sc.mu.Unlock()
defer sc.mu.Lock()

log.VEventf(ctx, 1, "refreshing statistics for table %d", tableID)
stats, err = sc.getTableStatsFromDB(ctx, tableID)
log.VEventf(ctx, 1, "done refreshing statistics for table %d", tableID)
}()
if !e.mustRefreshAgain {
break
Expand All @@ -307,11 +304,13 @@ func (sc *TableStatisticsCache) refreshCacheEntry(ctx context.Context, tableID d
// RefreshTableStats refreshes the cached statistics for the given table ID
// by fetching the new stats from the database.
func (sc *TableStatisticsCache) RefreshTableStats(ctx context.Context, tableID descpb.ID) {
if log.V(1) {
log.Infof(ctx, "refreshing statistics for table %d", tableID)
}
log.VEventf(ctx, 1, "refreshing statistics for table %d", tableID)
ctx, span := tracing.ForkCtxSpan(ctx, "refresh-table-stats")
// Perform an asynchronous refresh of the cache.
go sc.refreshCacheEntry(ctx, tableID)
go func() {
defer tracing.FinishSpan(span)
sc.refreshCacheEntry(ctx, tableID)
}()
}

// InvalidateTableStats invalidates the cached statistics for the given table ID.
Expand All @@ -320,9 +319,7 @@ func (sc *TableStatisticsCache) RefreshTableStats(ctx context.Context, tableID d
// This function is used only when we want to guarantee that the next query
// uses updated stats.
func (sc *TableStatisticsCache) InvalidateTableStats(ctx context.Context, tableID descpb.ID) {
if log.V(1) {
log.Infof(ctx, "evicting statistics for table %d", tableID)
}
log.VEventf(ctx, 1, "evicting statistics for table %d", tableID)
sc.mu.Lock()
defer sc.mu.Unlock()
sc.mu.cache.Del(tableID)
Expand Down

0 comments on commit 322198e

Please sign in to comment.