-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
Copy pathcreate_stats.go
812 lines (724 loc) · 27.9 KB
/
create_stats.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
// Copyright 2017 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package sql
import (
"context"
"fmt"
"github.com/cockroachdb/cockroach/pkg/featureflag"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)
// createStatsPostEvents controls the cluster setting for logging
// automatic table statistics collection to the event log.
var createStatsPostEvents = settings.RegisterBoolSetting(
settings.ApplicationLevel,
"sql.stats.post_events.enabled",
"if set, an event is logged for every CREATE STATISTICS job",
false,
settings.WithPublic)
// featureStatsEnabled is used to enable and disable the CREATE STATISTICS and
// ANALYZE features.
var featureStatsEnabled = settings.RegisterBoolSetting(
settings.ApplicationLevel,
"feature.stats.enabled",
"set to true to enable CREATE STATISTICS/ANALYZE, false to disable; default is true",
featureflag.FeatureFlagEnabledDefault,
settings.WithPublic)
var statsOnVirtualCols = settings.RegisterBoolSetting(
settings.ApplicationLevel,
"sql.stats.virtual_computed_columns.enabled",
"set to true to collect table statistics on virtual computed columns",
true,
settings.WithPublic)
const nonIndexColHistogramBuckets = 2
// StubTableStats generates "stub" statistics for a table which are missing
// statistics on virtual computed columns, multi-column stats, and histograms,
// and have 0 for all values.
func StubTableStats(
desc catalog.TableDescriptor, name string,
) ([]*stats.TableStatisticProto, error) {
colStats, err := createStatsDefaultColumns(
context.Background(), desc, false /* virtColEnabled */, false, /* multiColEnabled */
nonIndexColHistogramBuckets, nil, /* evalCtx */
)
if err != nil {
return nil, err
}
statistics := make([]*stats.TableStatisticProto, len(colStats))
for i, colStat := range colStats {
statistics[i] = &stats.TableStatisticProto{
TableID: desc.GetID(),
Name: name,
ColumnIDs: colStat.ColumnIDs,
}
}
return statistics, nil
}
// createStatsNode is a planNode implemented in terms of a function. The
// runJob function starts a Job during Start, and the remainder of the
// CREATE STATISTICS planning and execution is performed within the jobs
// framework.
type createStatsNode struct {
tree.CreateStats
// p is the "outer planner" from planning the CREATE STATISTICS
// statement. When we startExec the createStatsNode, it creates a job which
// has a second planner (the JobExecContext). When the job resumes, it does
// its work using a retrying internal transaction for which we create a third
// "inner planner".
p *planner
// runAsJob is true by default, and causes the code below to be executed,
// which sets up a job and waits for it.
//
// If it is false, the flow for create statistics is planned directly; this
// is used when the statement is under EXPLAIN or EXPLAIN ANALYZE.
runAsJob bool
}
func (n *createStatsNode) startExec(params runParams) error {
telemetry.Inc(sqltelemetry.SchemaChangeCreateCounter("stats"))
return n.runJob(params.ctx)
}
func (n *createStatsNode) Next(params runParams) (bool, error) {
return false, nil
}
func (*createStatsNode) Close(context.Context) {}
func (*createStatsNode) Values() tree.Datums { return nil }
// runJob starts a CreateStats job synchronously to plan and execute
// statistics creation and then waits for the job to complete.
func (n *createStatsNode) runJob(ctx context.Context) error {
record, err := n.makeJobRecord(ctx)
if err != nil {
return err
}
if n.Name != jobspb.AutoStatsName {
telemetry.Inc(sqltelemetry.CreateStatisticsUseCounter)
}
var job *jobs.StartableJob
jobID := n.p.ExecCfg().JobRegistry.MakeJobID()
if err := n.p.ExecCfg().InternalDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) (err error) {
if n.Name == jobspb.AutoStatsName {
// Don't start the job if there is already a CREATE STATISTICS job running.
// (To handle race conditions we check this again after the job starts,
// but this check is used to prevent creating a large number of jobs that
// immediately fail).
if err := checkRunningJobsInTxn(ctx, jobspb.InvalidJobID, txn); err != nil {
return err
}
}
return n.p.ExecCfg().JobRegistry.CreateStartableJobWithTxn(ctx, &job, jobID, txn, *record)
}); err != nil {
if job != nil {
if cleanupErr := job.CleanupOnRollback(ctx); cleanupErr != nil {
log.Warningf(ctx, "failed to cleanup StartableJob: %v", cleanupErr)
}
}
return err
}
if err := job.Start(ctx); err != nil {
return err
}
if err = job.AwaitCompletion(ctx); err != nil {
if errors.Is(err, stats.ConcurrentCreateStatsError) {
// Delete the job so users don't see it and get confused by the error.
if delErr := n.p.ExecCfg().JobRegistry.DeleteTerminalJobByID(ctx, job.ID()); delErr != nil {
log.Warningf(ctx, "failed to delete job: %v", delErr)
}
}
}
return err
}
// makeJobRecord creates a CreateStats job record which can be used to plan and
// execute statistics creation.
func (n *createStatsNode) makeJobRecord(ctx context.Context) (*jobs.Record, error) {
var tableDesc catalog.TableDescriptor
var fqTableName string
var err error
switch t := n.Table.(type) {
case *tree.UnresolvedObjectName:
tableDesc, err = n.p.ResolveExistingObjectEx(ctx, t, true /*required*/, tree.ResolveRequireTableDesc)
if err != nil {
return nil, err
}
fqTableName = n.p.ResolvedName(t).FQString()
case *tree.TableRef:
tableDesc, err = n.p.byIDGetterBuilder().WithoutNonPublic().Get().Table(ctx, descpb.ID(t.TableID))
if err != nil {
return nil, err
}
fqName, err := n.p.getQualifiedTableName(ctx, tableDesc)
if err != nil {
return nil, err
}
fqTableName = fqName.FQString()
}
if tableDesc.IsVirtualTable() {
return nil, pgerror.New(
pgcode.WrongObjectType, "cannot create statistics on virtual tables",
)
}
if tableDesc.IsView() {
return nil, pgerror.New(
pgcode.WrongObjectType, "cannot create statistics on views",
)
}
if stats.DisallowedOnSystemTable(tableDesc.GetID()) {
return nil, pgerror.Newf(
pgcode.WrongObjectType, "cannot create statistics on system.%s", tableDesc.GetName(),
)
}
if n.Options.UsingExtremes && !n.p.SessionData().EnableCreateStatsUsingExtremes {
return nil, pgerror.New(pgcode.FeatureNotSupported,
"creating partial statistics at extremes is not yet supported",
)
}
if n.Options.Where != nil {
return nil, pgerror.New(pgcode.FeatureNotSupported,
"creating partial statistics with a WHERE clause is not yet supported",
)
}
if n.Options.AsOf.Expr == nil {
return nil, pgerror.New(pgcode.Syntax,
"creating partial statistics requires an AS OF SYSTEM TIME clause such as AS OF SYSTEM TIME '-0.001s'",
)
}
if err := n.p.CheckPrivilege(ctx, tableDesc, privilege.SELECT); err != nil {
return nil, err
}
var colStats []jobspb.CreateStatsDetails_ColStat
var deleteOtherStats bool
if len(n.ColumnNames) == 0 {
virtColEnabled := statsOnVirtualCols.Get(n.p.ExecCfg().SV())
// Disable multi-column stats and deleting stats if partial statistics at
// the extremes are requested.
// TODO(faizaanmadhani): Add support for multi-column stats.
var multiColEnabled bool
if !n.Options.UsingExtremes {
multiColEnabled = stats.MultiColumnStatisticsClusterMode.Get(n.p.ExecCfg().SV())
deleteOtherStats = true
}
defaultHistogramBuckets := stats.GetDefaultHistogramBuckets(n.p.ExecCfg().SV(), tableDesc)
if colStats, err = createStatsDefaultColumns(
ctx, tableDesc, virtColEnabled, multiColEnabled, defaultHistogramBuckets, n.p.EvalContext(),
); err != nil {
return nil, err
}
} else {
columns, err := catalog.MustFindPublicColumnsByNameList(tableDesc, n.ColumnNames)
if err != nil {
return nil, err
}
columnIDs := make([]descpb.ColumnID, len(columns))
for i := range columns {
if columns[i].IsVirtual() && !statsOnVirtualCols.Get(n.p.ExecCfg().SV()) {
err := pgerror.Newf(
pgcode.InvalidColumnReference,
"cannot create statistics on virtual column %q",
columns[i].ColName(),
)
return nil, errors.WithHint(err,
"set cluster setting sql.stats.virtual_computed_columns.enabled to collect statistics "+
"on virtual columns",
)
}
columnIDs[i] = columns[i].GetID()
}
col, err := catalog.MustFindColumnByID(tableDesc, columnIDs[0])
if err != nil {
return nil, err
}
// Sort columnIDs to make equivalent column sets equal when using SHOW
// STATISTICS or other SQL on table_statistics.
_ = stats.MakeSortedColStatKey(columnIDs)
isInvIndex := colinfo.ColumnTypeIsOnlyInvertedIndexable(col.GetType())
defaultHistogramBuckets := stats.GetDefaultHistogramBuckets(n.p.ExecCfg().SV(), tableDesc)
colStats = []jobspb.CreateStatsDetails_ColStat{{
ColumnIDs: columnIDs,
// By default, create histograms on all explicitly requested column stats
// with a single column that doesn't use an inverted index.
HasHistogram: len(columnIDs) == 1 && !isInvIndex,
HistogramMaxBuckets: defaultHistogramBuckets,
}}
// Make histograms for inverted index column types.
if len(columnIDs) == 1 && isInvIndex {
colStats = append(colStats, jobspb.CreateStatsDetails_ColStat{
ColumnIDs: columnIDs,
HasHistogram: true,
Inverted: true,
HistogramMaxBuckets: defaultHistogramBuckets,
})
}
}
// Evaluate the AS OF time, if any.
var asOfTimestamp *hlc.Timestamp
if n.Options.AsOf.Expr != nil {
asOf, err := n.p.EvalAsOfTimestamp(ctx, n.Options.AsOf)
if err != nil {
return nil, err
}
asOfTimestamp = &asOf.Timestamp
}
// Create a job to run statistics creation.
statement := tree.AsStringWithFQNames(n, n.p.EvalContext().Annotations)
eventLogStatement := statement
var description string
if n.Name == jobspb.AutoStatsName {
// Use a user-friendly description for automatic statistics.
description = fmt.Sprintf("Table statistics refresh for %s", fqTableName)
} else {
// This must be a user query, so use the statement (for consistency with
// other jobs triggered by statements).
description = statement
statement = ""
}
return &jobs.Record{
Description: description,
Statements: []string{statement},
Username: n.p.User(),
Details: jobspb.CreateStatsDetails{
Name: string(n.Name),
FQTableName: fqTableName,
Table: *tableDesc.TableDesc(),
ColumnStats: colStats,
Statement: eventLogStatement,
AsOf: asOfTimestamp,
MaxFractionIdle: n.Options.Throttling,
DeleteOtherStats: deleteOtherStats,
UsingExtremes: n.Options.UsingExtremes,
},
Progress: jobspb.CreateStatsProgress{},
}, nil
}
// maxNonIndexCols is the maximum number of non-index columns that we will use
// when choosing a default set of column statistics.
const maxNonIndexCols = 100
// createStatsDefaultColumns creates column statistics on a default set of
// column lists when no columns were specified by the caller.
//
// To determine a useful set of default column statistics, we rely on
// information provided by the schema. In particular, the presence of an index
// on a particular set of columns indicates that the workload likely contains
// queries that involve those columns (e.g., for filters), and it would be
// useful to have statistics on prefixes of those columns. For example, if a
// table abc contains indexes on (a ASC, b ASC) and (b ASC, c ASC), we will
// collect statistics on a, {a, b}, b, and {b, c}. (But if multiColEnabled is
// false, we will only collect stats on a and b). Columns in partial index
// predicate expressions are also likely to appear in query filters, so stats
// are collected for those columns as well.
//
// In addition to the index columns, we collect stats on up to maxNonIndexCols
// other columns from the table. We only collect histograms for index columns,
// plus any other boolean or enum columns (where the "histogram" is tiny).
func createStatsDefaultColumns(
ctx context.Context,
desc catalog.TableDescriptor,
virtColEnabled, multiColEnabled bool,
defaultHistogramBuckets uint32,
evalCtx *eval.Context,
) ([]jobspb.CreateStatsDetails_ColStat, error) {
colStats := make([]jobspb.CreateStatsDetails_ColStat, 0, len(desc.ActiveIndexes()))
requestedStats := make(map[string]struct{})
// CREATE STATISTICS only runs as a fully-distributed plan. If statistics on
// virtual computed columns are enabled, we must check whether each virtual
// computed column expression is safe to distribute. Virtual computed columns
// with expressions *not* safe to distribute will be skipped, even if
// sql.stats.virtual_computed_columns.enabled is true.
// TODO(michae2): Add the ability to run CREATE STATISTICS locally if a
// local-only virtual computed column expression is needed.
cannotDistribute := make([]bool, len(desc.PublicColumns()))
if virtColEnabled {
semaCtx := tree.MakeSemaContext(evalCtx.Planner)
exprs, _, err := schemaexpr.MakeComputedExprs(
ctx,
desc.PublicColumns(),
desc.PublicColumns(),
desc,
tree.NewUnqualifiedTableName(tree.Name(desc.GetName())),
evalCtx,
&semaCtx,
)
if err != nil {
return nil, err
}
var distSQLVisitor distSQLExprCheckVisitor
for i, col := range desc.PublicColumns() {
cannotDistribute[i] = col.IsVirtual() && checkExprForDistSQL(exprs[i], &distSQLVisitor) != nil
}
}
isUnsupportedVirtual := func(col catalog.Column) bool {
return col.IsVirtual() && (!virtColEnabled || cannotDistribute[col.Ordinal()])
}
// sortAndTrackStatsExists adds the given column IDs as a set to the
// requestedStats set. If the columnIDs were already in the set, it returns
// true. As a side-effect sortAndTrackStatsExists also sorts colIDs. NOTE:
// This assumes that ordering is not significant for multi-column stats.
sortAndTrackStatsExists := func(colIDs []descpb.ColumnID) bool {
key := stats.MakeSortedColStatKey(colIDs)
if _, ok := requestedStats[key]; ok {
return true
}
requestedStats[key] = struct{}{}
return false
}
// addIndexColumnStatsIfNotExists appends column stats for the given column
// ID if they have not already been added. Histogram stats are collected for
// every indexed column.
addIndexColumnStatsIfNotExists := func(colID descpb.ColumnID, isInverted bool) error {
col, err := catalog.MustFindColumnByID(desc, colID)
if err != nil {
return err
}
// There shouldn't be any non-public columns, but defensively skip over them
// if there are.
if !col.Public() {
return nil
}
// Skip unsupported virtual computed columns.
if isUnsupportedVirtual(col) {
return nil
}
colIDs := []descpb.ColumnID{colID}
// Check for existing stats and remember the requested stats.
if ok := sortAndTrackStatsExists(colIDs); ok {
return nil
}
colStat := jobspb.CreateStatsDetails_ColStat{
ColumnIDs: colIDs,
HasHistogram: !isInverted,
HistogramMaxBuckets: defaultHistogramBuckets,
}
colStats = append(colStats, colStat)
// Generate histograms for inverted indexes. The above colStat append is
// still needed for a basic sketch of the column. The following colStat
// is needed for the sampling and sketch of the inverted index keys of
// the column.
if isInverted {
colStat.Inverted = true
colStat.HasHistogram = true
colStats = append(colStats, colStat)
}
return nil
}
// Add column stats for the primary key.
primaryIdx := desc.GetPrimaryIndex()
for i := 0; i < primaryIdx.NumKeyColumns(); i++ {
// Generate stats for each column in the primary key.
err := addIndexColumnStatsIfNotExists(primaryIdx.GetKeyColumnID(i), false /* isInverted */)
if err != nil {
return nil, err
}
// Only collect multi-column stats if enabled.
if i == 0 || !multiColEnabled {
continue
}
colIDs := make([]descpb.ColumnID, 0, i+1)
for j := 0; j <= i; j++ {
col, err := catalog.MustFindColumnByID(desc, desc.GetPrimaryIndex().GetKeyColumnID(j))
if err != nil {
return nil, err
}
// There shouldn't be any non-public columns, but defensively skip over
// them if there are.
if !col.Public() {
continue
}
// Skip unsupported virtual computed columns.
if isUnsupportedVirtual(col) {
continue
}
colIDs = append(colIDs, col.GetID())
}
// Do not attempt to create multi-column stats with < 2 columns. This can
// happen when an index contains only virtual computed columns.
if len(colIDs) < 2 {
continue
}
// Remember the requested stats so we don't request duplicates.
_ = sortAndTrackStatsExists(colIDs)
// Only generate non-histogram multi-column stats.
colStats = append(colStats, jobspb.CreateStatsDetails_ColStat{
ColumnIDs: colIDs,
HasHistogram: false,
})
}
// Add column stats for each secondary index.
for _, idx := range desc.PublicNonPrimaryIndexes() {
for j, n := 0, idx.NumKeyColumns(); j < n; j++ {
colID := idx.GetKeyColumnID(j)
isInverted := idx.GetType() == descpb.IndexDescriptor_INVERTED && colID == idx.InvertedColumnID()
// Generate stats for each indexed column.
if err := addIndexColumnStatsIfNotExists(colID, isInverted); err != nil {
return nil, err
}
// Only collect multi-column stats if enabled.
if j == 0 || !multiColEnabled {
continue
}
colIDs := make([]descpb.ColumnID, 0, j+1)
for k := 0; k <= j; k++ {
col, err := catalog.MustFindColumnByID(desc, idx.GetKeyColumnID(k))
if err != nil {
return nil, err
}
// There shouldn't be any non-public columns, but defensively skip them
// if there are.
if !col.Public() {
continue
}
// Skip unsupported virtual computed columns.
if isUnsupportedVirtual(col) {
continue
}
colIDs = append(colIDs, col.GetID())
}
// Do not attempt to create multi-column stats with < 2 columns. This can
// happen when an index contains only virtual computed columns.
if len(colIDs) < 2 {
continue
}
// Check for existing stats and remember the requested stats.
if ok := sortAndTrackStatsExists(colIDs); ok {
continue
}
// Only generate non-histogram multi-column stats.
colStats = append(colStats, jobspb.CreateStatsDetails_ColStat{
ColumnIDs: colIDs,
HasHistogram: false,
})
}
// Add columns referenced in partial index predicate expressions.
if idx.IsPartial() {
expr, err := parser.ParseExpr(idx.GetPredicate())
if err != nil {
return nil, err
}
// Extract the IDs of columns referenced in the predicate.
colIDs, err := schemaexpr.ExtractColumnIDs(desc, expr)
if err != nil {
return nil, err
}
// Generate stats for each column individually.
for _, colID := range colIDs.Ordered() {
col, err := catalog.MustFindColumnByID(desc, colID)
if err != nil {
return nil, err
}
isInverted := colinfo.ColumnTypeIsOnlyInvertedIndexable(col.GetType())
if err := addIndexColumnStatsIfNotExists(colID, isInverted); err != nil {
return nil, err
}
}
}
}
// Add all remaining columns in the table, up to maxNonIndexCols.
nonIdxCols := 0
for i := 0; i < len(desc.PublicColumns()) && nonIdxCols < maxNonIndexCols; i++ {
col := desc.PublicColumns()[i]
// Skip unsupported virtual computed columns.
if isUnsupportedVirtual(col) {
continue
}
colIDs := []descpb.ColumnID{col.GetID()}
// Check for existing stats.
if ok := sortAndTrackStatsExists(colIDs); ok {
continue
}
// Non-index columns have very small histograms since it's not worth the
// overhead of storing large histograms for these columns. Since bool and
// enum types only have a few values anyway, include all possible values
// for those types, up to DefaultHistogramBuckets.
maxHistBuckets := uint32(nonIndexColHistogramBuckets)
if col.GetType().Family() == types.BoolFamily || col.GetType().Family() == types.EnumFamily {
maxHistBuckets = defaultHistogramBuckets
}
colStats = append(colStats, jobspb.CreateStatsDetails_ColStat{
ColumnIDs: colIDs,
HasHistogram: !colinfo.ColumnTypeIsOnlyInvertedIndexable(col.GetType()),
HistogramMaxBuckets: maxHistBuckets,
})
nonIdxCols++
}
return colStats, nil
}
// createStatsResumer implements the jobs.Resumer interface for CreateStats
// jobs. A new instance is created for each job.
type createStatsResumer struct {
job *jobs.Job
tableID descpb.ID
}
var _ jobs.Resumer = &createStatsResumer{}
// Resume is part of the jobs.Resumer interface.
func (r *createStatsResumer) Resume(ctx context.Context, execCtx interface{}) error {
// jobsPlanner is a second planner distinct from the "outer planner" in the
// createStatsNode. It comes from the jobs system and does not have an
// associated txn.
jobsPlanner := execCtx.(JobExecContext)
details := r.job.Details().(jobspb.CreateStatsDetails)
if details.Name == jobspb.AutoStatsName {
// We want to make sure that an automatic CREATE STATISTICS job only runs if
// there are no other CREATE STATISTICS jobs running, automatic or manual.
if err := checkRunningJobs(ctx, r.job, jobsPlanner); err != nil {
return err
}
}
r.tableID = details.Table.ID
if err := jobsPlanner.ExecCfg().InternalDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error {
// We create a third "inner planner" associated with this txn in order to
// have (a) use of the txn during type checking of any virtual computed
// column expressions, and (b) use of the txn during processor setup during
// the execution of the flow.
innerPlanner, cleanup := NewInternalPlanner(
"create-stats-resume-job",
txn.KV(),
jobsPlanner.User(),
&MemoryMetrics{},
jobsPlanner.ExecCfg(),
jobsPlanner.SessionData(),
)
defer cleanup()
innerP := innerPlanner.(*planner)
innerEvalCtx := innerP.ExtendedEvalContext()
if details.AsOf != nil {
innerP.ExtendedEvalContext().AsOfSystemTime = &eval.AsOfSystemTime{Timestamp: *details.AsOf}
innerP.ExtendedEvalContext().SetTxnTimestamp(details.AsOf.GoTime())
if err := txn.KV().SetFixedTimestamp(ctx, *details.AsOf); err != nil {
return err
}
}
dsp := innerP.DistSQLPlanner()
planCtx := dsp.NewPlanningCtx(ctx, innerEvalCtx, innerP, txn.KV(), FullDistribution)
// CREATE STATS flow doesn't produce any rows and only emits the
// metadata, so we can use a nil rowContainerHelper.
resultWriter := NewRowResultWriter(nil /* rowContainer */)
if err := dsp.planAndRunCreateStats(
ctx, innerEvalCtx, planCtx, innerP.SemaCtx(), txn.KV(), r.job, resultWriter,
); err != nil {
// Check if this was a context canceled error and restart if it was.
if grpcutil.IsContextCanceled(err) {
return jobs.MarkAsRetryJobError(err)
}
// If the job was canceled, any of the distsql processors could have been
// the first to encounter the .Progress error. This error's string is sent
// through distsql back here, so we can't examine the err type in this case
// to see if it's a jobs.InvalidStatusError. Instead, attempt to update the
// job progress to coerce out the correct error type. If the update succeeds
// then return the original error, otherwise return this error instead so
// it can be cleaned up at a higher level.
if jobErr := r.job.NoTxn().FractionProgressed(ctx, func(
ctx context.Context, _ jobspb.ProgressDetails,
) float32 {
// The job failed so the progress value here doesn't really matter.
return 0
},
); jobErr != nil {
return jobErr
}
return err
}
return nil
}); err != nil {
return err
}
evalCtx := jobsPlanner.ExtendedEvalContext()
// Record this statistics creation in the event log.
if !createStatsPostEvents.Get(&evalCtx.Settings.SV) {
return nil
}
// TODO(rytaft): This creates a new transaction for the CREATE STATISTICS
// event. It must be different from the CREATE STATISTICS transaction,
// because that transaction must be read-only. In the future we may want
// to use the transaction that inserted the new stats into the
// system.table_statistics table, but that would require calling
// logEvent() from the distsqlrun package.
//
// TODO(knz): figure out why this is not triggered for a regular
// CREATE STATISTICS statement.
// See: https://github.com/cockroachdb/cockroach/issues/57739
return evalCtx.ExecCfg.InternalDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error {
return logEventInternalForSQLStatements(ctx,
evalCtx.ExecCfg, txn,
0, /* depth: use event_log=2 for vmodule filtering */
eventLogOptions{dst: LogEverywhere},
eventpb.CommonSQLEventDetails{
Statement: redact.Sprint(details.Statement),
Tag: "CREATE STATISTICS",
User: evalCtx.SessionData().User().Normalized(),
ApplicationName: evalCtx.SessionData().ApplicationName,
PlaceholderValues: []string{}, /* no placeholders known at this point */
DescriptorID: uint32(details.Table.ID),
},
&eventpb.CreateStatistics{
TableName: details.FQTableName,
},
)
})
}
// checkRunningJobs checks whether there are any other CreateStats jobs in the
// pending, running, or paused status that started earlier than this one. If
// there are, checkRunningJobs returns an error. If job is nil, checkRunningJobs
// just checks if there are any pending, running, or paused CreateStats jobs.
func checkRunningJobs(ctx context.Context, job *jobs.Job, p JobExecContext) error {
jobID := jobspb.InvalidJobID
if job != nil {
jobID = job.ID()
}
return p.ExecCfg().InternalDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) (err error) {
return checkRunningJobsInTxn(ctx, jobID, txn)
})
}
// checkRunningJobsInTxn checks whether there are any other CreateStats jobs in
// the pending, running, or paused status that started earlier than this one. If
// there are, checkRunningJobsInTxn returns an error. If jobID is
// jobspb.InvalidJobID, checkRunningJobsInTxn just checks if there are any pending,
// running, or paused CreateStats jobs.
func checkRunningJobsInTxn(ctx context.Context, jobID jobspb.JobID, txn isql.Txn) error {
exists, err := jobs.RunningJobExists(ctx, jobID, txn,
jobspb.TypeCreateStats, jobspb.TypeAutoCreateStats,
)
if err != nil {
return err
}
if exists {
return stats.ConcurrentCreateStatsError
}
return nil
}
// OnFailOrCancel is part of the jobs.Resumer interface.
func (r *createStatsResumer) OnFailOrCancel(context.Context, interface{}, error) error { return nil }
// CollectProfile is part of the jobs.Resumer interface.
func (r *createStatsResumer) CollectProfile(context.Context, interface{}) error { return nil }
func init() {
createResumerFn := func(job *jobs.Job, settings *cluster.Settings) jobs.Resumer {
return &createStatsResumer{job: job}
}
jobs.RegisterConstructor(jobspb.TypeCreateStats, createResumerFn, jobs.UsesTenantCostControl)
jobs.RegisterConstructor(jobspb.TypeAutoCreateStats, createResumerFn, jobs.UsesTenantCostControl)
}