Skip to content

Commit 3bc0992

Browse files
committed
sql/stats: include partial stats in results of statsCache.GetTableStats
We were not including partial stats in the list of table statistics returned by `statsCache.GetTableStats`. This was fine for the optimizer, which currently cannot use partial stats directly, but it was a problem for backup. We'd like to use partial stats directly in the optimizer eventually, so this commit goes ahead and adds them to the results of `GetTableStats`. The optimizer then must filter them out. To streamline this we add some helper functions. Finally, in an act of overzealous refactoring, this commit also changes `MergedStatistics` and `ForecastTableStatistics` to accept partial statistics and full statistics mixed together in the same input list. This simplifies the code that calls these functions. Fixes: #95056 Part of: #93983 Epic: CRDB-19449 Release note: None
1 parent f5cac1e commit 3bc0992

File tree

16 files changed

+239
-149
lines changed

16 files changed

+239
-149
lines changed

pkg/ccl/backupccl/testdata/backup-restore/metadata

Lines changed: 1 addition & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -53,9 +53,6 @@ exec-sql
5353
BACKUP DATABASE db1 INTO 'nodelocal://0/test/'
5454
----
5555

56-
# TODO(ssd): The expectation here is 6 stats rather than 7 because the
57-
# 'partial' stat is not backed up even though it is persisted. I think
58-
# we may want a different API for fetching statistics.
5956
query-sql
6057
SELECT
6158
json_array_length(
@@ -72,4 +69,4 @@ SELECT
7269
-> 'statistics'
7370
)
7471
----
75-
6
72+
7

pkg/sql/distsql_plan_stats.go

Lines changed: 14 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -223,19 +223,22 @@ func (dsp *DistSQLPlanner) createPartialStatsPlan(
223223
// column that is not partial and not forecasted. The first one we find will
224224
// be the latest due to the newest to oldest ordering property of the cache.
225225
for _, t := range tableStats {
226-
// TODO (faizaanmadhani): Ideally, we don't want to verify that
227-
// a statistic is forecasted or merged based on the name because
228-
// someone could create a statistic named __forecast__ or __merged__.
229-
// Update system.table_statistics to add an enum to indicate which
230-
// type of statistic it is.
231-
if len(t.ColumnIDs) == 1 && column.GetID() == t.ColumnIDs[0] && t.PartialPredicate == "" &&
232-
t.Name != jobspb.ForecastStatsName &&
233-
t.Name != jobspb.MergedStatsName {
226+
if len(t.ColumnIDs) == 1 && column.GetID() == t.ColumnIDs[0] &&
227+
!t.IsPartial() && !t.IsMerged() && !t.IsForecast() {
234228
if t.HistogramData == nil || t.HistogramData.ColumnType == nil || len(t.Histogram) == 0 {
235-
return nil, pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "the latest full statistic for column %s has no histogram", column.GetName())
229+
return nil, pgerror.Newf(
230+
pgcode.ObjectNotInPrerequisiteState,
231+
"the latest full statistic for column %s has no histogram",
232+
column.GetName(),
233+
)
236234
}
237-
if colinfo.ColumnTypeIsInvertedIndexable(column.GetType()) && t.HistogramData.ColumnType.Family() == types.BytesFamily {
238-
return nil, pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "the latest full statistic histogram for column %s is an inverted index histogram", column.GetName())
235+
if colinfo.ColumnTypeIsInvertedIndexable(column.GetType()) &&
236+
t.HistogramData.ColumnType.Family() == types.BytesFamily {
237+
return nil, pgerror.Newf(
238+
pgcode.ObjectNotInPrerequisiteState,
239+
"the latest full statistic histogram for column %s is an inverted index histogram",
240+
column.GetName(),
241+
)
239242
}
240243
stat = t
241244
histogram = t.Histogram

pkg/sql/opt/cat/table.go

Lines changed: 14 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -228,8 +228,21 @@ type TableStatistic interface {
228228
// inverted index histograms, this will always return types.Bytes.
229229
HistogramType() *types.T
230230

231-
// IsForecast returns true if this statistic is a forecast.
231+
// IsPartial returns true if this statistic was collected with a where
232+
// clause. (If the where clause was something like "WHERE 1 = 1" or "WHERE
233+
// true" this could technically be a full statistic rather than a partial
234+
// statistic, but this function does not check for this.)
235+
IsPartial() bool
236+
237+
// IsMerged returns true if this statistic was created by merging a partial
238+
// and a full statistic.
239+
IsMerged() bool
240+
241+
// IsForecast returns true if this statistic was created by forecasting.
232242
IsForecast() bool
243+
244+
// IsAuto returns true if this statistic was collected automatically.
245+
IsAuto() bool
233246
}
234247

235248
// HistogramBucket contains the data for a single histogram bucket. Note

pkg/sql/opt/exec/execbuilder/relational.go

Lines changed: 14 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -403,12 +403,12 @@ func (b *Builder) maybeAnnotateWithEstimates(node exec.Node, e memo.RelExpr) {
403403
if scan, ok := e.(*memo.ScanExpr); ok {
404404
tab := b.mem.Metadata().Table(scan.Table)
405405
if tab.StatisticCount() > 0 {
406-
// The first stat is the most recent one.
406+
// The first stat is the most recent full one.
407407
var first int
408-
if !b.evalCtx.SessionData().OptimizerUseForecasts {
409-
for first < tab.StatisticCount() && tab.Statistic(first).IsForecast() {
410-
first++
411-
}
408+
for first < tab.StatisticCount() &&
409+
tab.Statistic(first).IsPartial() ||
410+
(tab.Statistic(first).IsForecast() && !b.evalCtx.SessionData().OptimizerUseForecasts) {
411+
first++
412412
}
413413

414414
if first < tab.StatisticCount() {
@@ -422,10 +422,10 @@ func (b *Builder) maybeAnnotateWithEstimates(node exec.Node, e memo.RelExpr) {
422422
val.Forecast = stat.IsForecast()
423423
if val.Forecast {
424424
val.ForecastAt = stat.CreatedAt()
425-
// Find the first non-forecast stat.
425+
// Find the first non-forecast full stat.
426426
for i := first + 1; i < tab.StatisticCount(); i++ {
427427
nextStat := tab.Statistic(i)
428-
if !nextStat.IsForecast() {
428+
if !nextStat.IsPartial() && !nextStat.IsForecast() {
429429
val.TableStatsCreatedAt = nextStat.CreatedAt()
430430
break
431431
}
@@ -760,8 +760,11 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) {
760760
b.TotalScanRows += stats.RowCount
761761
b.ScanCounts[exec.ScanWithStatsCount]++
762762

763-
// The first stat is the most recent one. Check if it was a forecast.
763+
// The first stat is the most recent full one. Check if it was a forecast.
764764
var first int
765+
for first < tab.StatisticCount() && tab.Statistic(first).IsPartial() {
766+
first++
767+
}
765768
if first < tab.StatisticCount() && tab.Statistic(first).IsForecast() {
766769
if b.evalCtx.SessionData().OptimizerUseForecasts {
767770
b.ScanCounts[exec.ScanWithStatsForecastCount]++
@@ -772,8 +775,9 @@ func (b *Builder) buildScan(scan *memo.ScanExpr) (execPlan, error) {
772775
b.NanosSinceStatsForecasted = nanosSinceStatsForecasted
773776
}
774777
}
775-
// Find the first non-forecast stat.
776-
for first < tab.StatisticCount() && tab.Statistic(first).IsForecast() {
778+
// Find the first non-forecast full stat.
779+
for first < tab.StatisticCount() &&
780+
(tab.Statistic(first).IsPartial() || tab.Statistic(first).IsForecast()) {
777781
first++
778782
}
779783
}

pkg/sql/opt/memo/statistics_builder.go

Lines changed: 8 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -614,12 +614,12 @@ func (sb *statisticsBuilder) makeTableStatistics(tabID opt.TableID) *props.Stati
614614
// Make now and annotate the metadata table with it for next time.
615615
stats = &props.Statistics{}
616616

617-
// Find the most recent statistic. (Stats are ordered with most recent first.)
617+
// Find the most recent full statistic. (Stats are ordered with most recent first.)
618618
var first int
619-
if !sb.evalCtx.SessionData().OptimizerUseForecasts {
620-
for first < tab.StatisticCount() && tab.Statistic(first).IsForecast() {
621-
first++
622-
}
619+
for first < tab.StatisticCount() &&
620+
(tab.Statistic(first).IsPartial() ||
621+
tab.Statistic(first).IsForecast() && !sb.evalCtx.SessionData().OptimizerUseForecasts) {
622+
first++
623623
}
624624

625625
if first >= tab.StatisticCount() {
@@ -639,6 +639,9 @@ func (sb *statisticsBuilder) makeTableStatistics(tabID opt.TableID) *props.Stati
639639
// column set. Stats are ordered with most recent first.
640640
for i := first; i < tab.StatisticCount(); i++ {
641641
stat := tab.Statistic(i)
642+
if stat.IsPartial() {
643+
continue
644+
}
642645
if stat.IsForecast() && !sb.evalCtx.SessionData().OptimizerUseForecasts {
643646
continue
644647
}

pkg/sql/opt/testutils/testcat/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,6 @@ go_library(
2323
deps = [
2424
"//pkg/config/zonepb",
2525
"//pkg/geo/geoindex",
26-
"//pkg/jobs/jobspb",
2726
"//pkg/roachpb",
2827
"//pkg/security/username",
2928
"//pkg/settings/cluster",

pkg/sql/opt/testutils/testcat/test_catalog.go

Lines changed: 16 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@ import (
1818

1919
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
2020
"github.com/cockroachdb/cockroach/pkg/geo/geoindex"
21-
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
2221
"github.com/cockroachdb/cockroach/pkg/roachpb"
2322
"github.com/cockroachdb/cockroach/pkg/security/username"
2423
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
@@ -1294,9 +1293,24 @@ func (ts *TableStat) HistogramType() *types.T {
12941293
return ts.histogramType
12951294
}
12961295

1296+
// IsPartial is part of the cat.TableStatistic interface.
1297+
func (ts *TableStat) IsPartial() bool {
1298+
return ts.js.IsPartial()
1299+
}
1300+
1301+
// IsMerged is part of the cat.TableStatistic interface.
1302+
func (ts *TableStat) IsMerged() bool {
1303+
return ts.js.IsMerged()
1304+
}
1305+
12971306
// IsForecast is part of the cat.TableStatistic interface.
12981307
func (ts *TableStat) IsForecast() bool {
1299-
return ts.js.Name == jobspb.ForecastStatsName
1308+
return ts.js.IsForecast()
1309+
}
1310+
1311+
// IsAuto is part of the cat.TableStatistic interface.
1312+
func (ts *TableStat) IsAuto() bool {
1313+
return ts.js.IsAuto()
13001314
}
13011315

13021316
// TableStats is a slice of TableStat pointers.

pkg/sql/opt_catalog.go

Lines changed: 16 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@ import (
1717

1818
"github.com/cockroachdb/cockroach/pkg/config"
1919
"github.com/cockroachdb/cockroach/pkg/geo/geoindex"
20-
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
2120
"github.com/cockroachdb/cockroach/pkg/keys"
2221
"github.com/cockroachdb/cockroach/pkg/kv"
2322
"github.com/cockroachdb/cockroach/pkg/roachpb"
@@ -1773,9 +1772,24 @@ func (os *optTableStat) HistogramType() *types.T {
17731772
return os.stat.HistogramData.ColumnType
17741773
}
17751774

1775+
// IsPartial is part of the cat.TableStatistic interface.
1776+
func (os *optTableStat) IsPartial() bool {
1777+
return os.stat.IsPartial()
1778+
}
1779+
1780+
// IsMerged is part of the cat.TableStatistic interface.
1781+
func (os *optTableStat) IsMerged() bool {
1782+
return os.stat.IsMerged()
1783+
}
1784+
17761785
// IsForecast is part of the cat.TableStatistic interface.
17771786
func (os *optTableStat) IsForecast() bool {
1778-
return os.stat.Name == jobspb.ForecastStatsName
1787+
return os.stat.IsForecast()
1788+
}
1789+
1790+
// IsAuto is part of the cat.TableStatistic interface.
1791+
func (os *optTableStat) IsAuto() bool {
1792+
return os.stat.IsAuto()
17791793
}
17801794

17811795
// optFamily is a wrapper around descpb.ColumnFamilyDescriptor that keeps a

pkg/sql/show_stats.go

Lines changed: 25 additions & 38 deletions
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,6 @@ import (
1414
"context"
1515
encjson "encoding/json"
1616
"fmt"
17-
"sort"
1817

1918
"github.com/cockroachdb/cockroach/pkg/clusterversion"
2019
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
@@ -194,10 +193,8 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p
194193

195194
_, withMerge := opts[showTableStatsOptMerge]
196195
_, withForecast := opts[showTableStatsOptForecast]
197-
198-
obsFullStats := make([]*stats.TableStatistic, 0, len(rows))
199-
obsPartialStats := make([]*stats.TableStatistic, 0, len(rows))
200196
if withMerge || withForecast {
197+
statsList := make([]*stats.TableStatistic, 0, len(rows))
201198
for _, row := range rows {
202199
// Skip stats on dropped columns.
203200
colIDs := row[columnIDsIdx].(*tree.DArray).Array
@@ -215,48 +212,38 @@ func (p *planner) ShowTableStats(ctx context.Context, n *tree.ShowTableStats) (p
215212
return nil, err
216213
}
217214
}
218-
if obs.PartialPredicate != "" {
219-
obsPartialStats = append(obsPartialStats, obs)
220-
} else {
221-
obsFullStats = append(obsFullStats, obs)
222-
}
215+
statsList = append(statsList, obs)
223216
}
224217

225-
// Reverse the lists to sort by CreatedAt descending.
226-
for i := 0; i < len(obsFullStats)/2; i++ {
227-
j := len(obsFullStats) - i - 1
228-
obsFullStats[i], obsFullStats[j] = obsFullStats[j], obsFullStats[i]
229-
}
230-
for i := 0; i < len(obsPartialStats)/2; i++ {
231-
j := len(obsPartialStats) - i - 1
232-
obsPartialStats[i], obsPartialStats[j] = obsPartialStats[j], obsPartialStats[i]
218+
// Reverse the list to sort by CreatedAt descending.
219+
for i := 0; i < len(statsList)/2; i++ {
220+
j := len(statsList) - i - 1
221+
statsList[i], statsList[j] = statsList[j], statsList[i]
233222
}
234-
}
235223

236-
if withMerge {
237-
merged := stats.MergedStatistics(ctx, obsPartialStats, obsFullStats)
238-
obsFullStats = append(obsFullStats, merged...)
239-
sort.Slice(obsFullStats, func(i, j int) bool {
240-
return obsFullStats[i].CreatedAt.After(obsFullStats[j].CreatedAt)
241-
})
242-
for i := len(merged) - 1; i >= 0; i-- {
243-
mergedRow, err := tableStatisticProtoToRow(&merged[i].TableStatisticProto, partialStatsVerActive)
244-
if err != nil {
245-
return nil, err
224+
if withMerge {
225+
merged := stats.MergedStatistics(ctx, statsList)
226+
statsList = append(merged, statsList...)
227+
// Iterate in reverse order to match the ORDER BY "columnIDs".
228+
for i := len(merged) - 1; i >= 0; i-- {
229+
mergedRow, err := tableStatisticProtoToRow(&merged[i].TableStatisticProto, partialStatsVerActive)
230+
if err != nil {
231+
return nil, err
232+
}
233+
rows = append(rows, mergedRow)
246234
}
247-
rows = append(rows, mergedRow)
248235
}
249-
}
250236

251-
if withForecast {
252-
forecasts := stats.ForecastTableStatistics(ctx, obsFullStats)
253-
// Iterate in reverse order to match the ORDER BY "columnIDs".
254-
for i := len(forecasts) - 1; i >= 0; i-- {
255-
forecastRow, err := tableStatisticProtoToRow(&forecasts[i].TableStatisticProto, partialStatsVerActive)
256-
if err != nil {
257-
return nil, err
237+
if withForecast {
238+
forecasts := stats.ForecastTableStatistics(ctx, statsList)
239+
// Iterate in reverse order to match the ORDER BY "columnIDs".
240+
for i := len(forecasts) - 1; i >= 0; i-- {
241+
forecastRow, err := tableStatisticProtoToRow(&forecasts[i].TableStatisticProto, partialStatsVerActive)
242+
if err != nil {
243+
return nil, err
244+
}
245+
rows = append(rows, forecastRow)
258246
}
259-
rows = append(rows, forecastRow)
260247
}
261248
}
262249

pkg/sql/stats/automatic_stats.go

Lines changed: 7 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -706,7 +706,7 @@ func (r *Refresher) maybeRefreshStats(
706706
// refresh happens at exactly 2x the current average, and the average
707707
// refresh time is calculated from the most recent 4 refreshes. See the
708708
// comment in stats/delete_stats.go.)
709-
maxTimeBetweenRefreshes := stat.CreatedAt.Add(2*avgRefreshTime(tableStats) + r.extraTime)
709+
maxTimeBetweenRefreshes := stat.CreatedAt.Add(2*avgFullRefreshTime(tableStats) + r.extraTime)
710710
if timeutil.Now().After(maxTimeBetweenRefreshes) {
711711
mustRefresh = true
712712
}
@@ -789,21 +789,20 @@ func mostRecentAutomaticStat(tableStats []*TableStatistic) *TableStatistic {
789789
return nil
790790
}
791791

792-
// avgRefreshTime returns the average time between automatic statistics
792+
// avgFullRefreshTime returns the average time between automatic full statistics
793793
// refreshes given a list of tableStats from one table. It does so by finding
794-
// the most recent automatically generated statistic (identified by the name
795-
// AutoStatsName), and then finds all previously generated automatic stats on
796-
// those same columns. The average is calculated as the average time between
797-
// each consecutive stat.
794+
// the most recent automatically generated statistic and then finds all
795+
// previously generated automatic stats on those same columns. The average is
796+
// calculated as the average time between each consecutive stat.
798797
//
799798
// If there are not at least two automatically generated statistics on the same
800799
// columns, the default value defaultAverageTimeBetweenRefreshes is returned.
801-
func avgRefreshTime(tableStats []*TableStatistic) time.Duration {
800+
func avgFullRefreshTime(tableStats []*TableStatistic) time.Duration {
802801
var reference *TableStatistic
803802
var sum time.Duration
804803
var count int
805804
for _, stat := range tableStats {
806-
if stat.Name != jobspb.AutoStatsName {
805+
if !stat.IsAuto() || stat.IsPartial() {
807806
continue
808807
}
809808
if reference == nil {

0 commit comments

Comments
 (0)