Skip to content

Commit aab61b6

Browse files
sql: add fullStatisticsID column to system.table_statistics
This commit adds a column to system.table_statistics used by partial statistics to store an id that references the full statistic that the partial statistics was derived from. For full statistics, the value in this column will be NULL. This commit updates a migration/upgrade that just included the partialPredicate column. Epic: CRDB-19449 Release note (sql change): system.table_statistics now contains a column called fullStatisticsID to store an id referencing the full table statistic the partial statistic was derived from.
1 parent a8d99a1 commit aab61b6

19 files changed

Lines changed: 157 additions & 73 deletions

pkg/clusterversion/cockroach_versions.go

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -335,9 +335,10 @@ const (
335335
// system tenant.
336336
V23_1DescIDSequenceForSystemTenant
337337

338-
// V23_1AddPartialStatisticsPredicateCol adds a column to store the predicate
339-
// for a partial statistics collection.
340-
V23_1AddPartialStatisticsPredicateCol
338+
// V23_1AddPartialStatisticsColumns adds two columns: one to store the predicate
339+
// for a partial statistics collection, and another to refer to the full statistic
340+
// it was collected from.
341+
V23_1AddPartialStatisticsColumns
341342

342343
// V23_1_CreateSystemJobInfoTable creates the system.job_info table.
343344
V23_1CreateSystemJobInfoTable
@@ -593,7 +594,7 @@ var rawVersionsSingleton = keyedVersions{
593594
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 6},
594595
},
595596
{
596-
Key: V23_1AddPartialStatisticsPredicateCol,
597+
Key: V23_1AddPartialStatisticsColumns,
597598
Version: roachpb.Version{Major: 22, Minor: 2, Internal: 8},
598599
},
599600
{

pkg/sql/alter_table.go

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1296,7 +1296,7 @@ func insertJSONStatistic(
12961296
name = s.Name
12971297
}
12981298

1299-
if !settings.Version.IsActive(ctx, clusterversion.V23_1AddPartialStatisticsPredicateCol) {
1299+
if !settings.Version.IsActive(ctx, clusterversion.V23_1AddPartialStatisticsColumns) {
13001300

13011301
if s.PartialPredicate != "" {
13021302
return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "statistic for columns %v with collection time %s to insert is partial but cluster version is below 23.1", s.Columns, s.CreatedAt)
@@ -1334,6 +1334,11 @@ func insertJSONStatistic(
13341334
predicateValue = s.PartialPredicate
13351335
}
13361336

1337+
var fullStatisticIDValue interface{}
1338+
if s.FullStatisticID != 0 {
1339+
fullStatisticIDValue = s.FullStatisticID
1340+
}
1341+
13371342
_ /* rows */, err := ie.Exec(
13381343
ctx,
13391344
"insert-stats",
@@ -1348,8 +1353,9 @@ func insertJSONStatistic(
13481353
"nullCount",
13491354
"avgSize",
13501355
histogram,
1351-
"partialPredicate"
1352-
) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10)`,
1356+
"partialPredicate",
1357+
"fullStatisticID"
1358+
) VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11)`,
13531359
tableID,
13541360
name,
13551361
columnIDs,
@@ -1360,6 +1366,7 @@ func insertJSONStatistic(
13601366
s.AvgSize,
13611367
histogram,
13621368
predicateValue,
1369+
fullStatisticIDValue,
13631370
)
13641371
return err
13651372
}

pkg/sql/catalog/systemschema/system.go

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -265,8 +265,9 @@ CREATE TABLE system.table_statistics (
265265
histogram BYTES,
266266
"avgSize" INT8 NOT NULL DEFAULT 0,
267267
"partialPredicate" STRING,
268+
"fullStatisticID" INT8,
268269
CONSTRAINT "primary" PRIMARY KEY ("tableID", "statisticID"),
269-
FAMILY "fam_0_tableID_statisticID_name_columnIDs_createdAt_rowCount_distinctCount_nullCount_histogram" ("tableID", "statisticID", name, "columnIDs", "createdAt", "rowCount", "distinctCount", "nullCount", histogram, "avgSize", "partialPredicate")
270+
FAMILY "fam_0_tableID_statisticID_name_columnIDs_createdAt_rowCount_distinctCount_nullCount_histogram" ("tableID", "statisticID", name, "columnIDs", "createdAt", "rowCount", "distinctCount", "nullCount", histogram, "avgSize", "partialPredicate", "fullStatisticID")
270271
);`
271272

272273
// locations are used to map a locality specified by a node to geographic
@@ -1558,6 +1559,7 @@ var (
15581559
{Name: "histogram", ID: 9, Type: types.Bytes, Nullable: true},
15591560
{Name: "avgSize", ID: 10, Type: types.Int, DefaultExpr: &zeroIntString},
15601561
{Name: "partialPredicate", ID: 11, Type: types.String, Nullable: true},
1562+
{Name: "fullStatisticID", ID: 12, Type: types.Int, Nullable: true},
15611563
},
15621564
[]descpb.ColumnFamilyDescriptor{
15631565
{
@@ -1575,8 +1577,9 @@ var (
15751577
"histogram",
15761578
"avgSize",
15771579
"partialPredicate",
1580+
"fullStatisticID",
15781581
},
1579-
ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11},
1582+
ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12},
15801583
},
15811584
},
15821585
descpb.IndexDescriptor{

pkg/sql/catalog/systemschema_test/testdata/bootstrap

Lines changed: 4 additions & 3 deletions
Large diffs are not rendered by default.

pkg/sql/distsql_plan_stats.go

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -219,17 +219,18 @@ func (dsp *DistSQLPlanner) createPartialStatsPlan(
219219
sb.Init(planCtx.EvalContext(), planCtx.ExtendedEvalCtx.Codec, desc, scan.index)
220220

221221
var histogram []cat.HistogramBucket
222-
// Find the histogram from the newest table statistic for our column
223-
// that is not partial and not forecasted. The first one we find will
224-
// be the latest due to the newest to oldest ordering property of the
225-
// cache.
222+
var stat *stats.TableStatistic
223+
// Find the statistic and histogram from the newest table statistic for our
224+
// column that is not partial and not forecasted. The first one we find will
225+
// be the latest due to the newest to oldest ordering property of the cache.
226226
for _, t := range tableStats {
227227
if len(t.ColumnIDs) == 1 && column.GetID() == t.ColumnIDs[0] && t.PartialPredicate == "" && t.Name != jobspb.ForecastStatsName {
228+
stat = t
228229
histogram = t.Histogram
229230
break
230231
}
231232
}
232-
if len(histogram) == 0 {
233+
if stat == nil || len(histogram) == 0 {
233234
return nil, pgerror.Newf(
234235
pgcode.ObjectNotInPrerequisiteState,
235236
"column %s does not have a prior statistic, "+
@@ -270,6 +271,7 @@ func (dsp *DistSQLPlanner) createPartialStatsPlan(
270271
Columns: make([]uint32, len(reqStat.columns)),
271272
StatName: reqStat.name,
272273
PartialPredicate: extremesPredicate,
274+
FullStatisticID: stat.StatisticID,
273275
}
274276
// For now, this loop should iterate only once, as we only
275277
// handle single-column partial statistics.

pkg/sql/execinfrapb/processors_table_stats.proto

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -54,6 +54,11 @@ message SketchSpec {
5454
// PartialPredicate is a string representing the predicate for a partial statistic,
5555
// and is the empty string for a full table statistic.
5656
optional string partial_predicate = 7 [(gogoproto.nullable) = false];
57+
58+
// FullStatisticID is non-zero for partial statistics and 0 for full
59+
// statistics. It is the statistic id of the full statistic that this partial
60+
// statistic was derived from.
61+
optional uint64 full_statistic_id = 8 [(gogoproto.customname) = "FullStatisticID", (gogoproto.nullable) = false];
5762
}
5863

5964
// SamplerSpec is the specification of a "sampler" processor which

0 commit comments

Comments
 (0)