Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
77070: sql: add SHOW STATISTICS WITH FORECAST r=michae2 a=michae2

**sql/stats: replace eval.Context with tree.CompareContext**

Most uses of eval.Context in the sql/stats package can actually be
tree.CompareContext instead, so make the replacement.

Release note: None

**sql/stats: bump histogram version to 2**

In 22.2 as of 963deb8 we support multiple histograms for trigram-
indexed strings. Let's bump the histogram version for this change, as we
may want to know whether multiple histograms are possible for a given
row in system.table_statistics.

(I suspect that during upgrades to 22.2 the 22.1 statistics builder will
choke on these statistics, so maybe we should also backport a version
check to 22.1.)

Also update avgRefreshTime to work correctly in multiple-histogram
cases.

Release note: None

**sql/stats: teach histogram.adjustCounts to remove empty buckets**

Sometimes when adjusting counts down we end up with empty buckets in the
histogram. They don't hurt anything, but they take up some memory (and
some brainpower when examining test results). So, teach adjustCounts to
remove them.

Release note: None

**sql/stats: always use non-nil buckets for empty-table histograms**

After 82b5926 I've been using the convention that nil histogram
buckets = no histogram, and non-nil-zero-length histogram buckets =
histogram on empty table. This is mostly useful for testing but is also
important for forecasting histograms.

Fix a spot that wasn't following this convention.

Also, add some empty-table testcases and some other testcases for
histogram.adjustCounts.

Release note: None

**sql/stats: make ordering of SHOW STATISTICS more deterministic**

Make two changes to produce more deterministic SHOW STATISTICS output:

1. Sort column IDs when creating statistics. We already use `FastIntSet`
   in both create_stats.go and statistics_builder.go to ignore ordering
   when gathering statistics by column set, but the column ordering from
   CREATE STATISTICS leaks into `system.table_statistics` and can affect
   SQL on that table, such as SHOW STATISTICS and various internal
   DELETE statements.

2. Order by column IDs and statistic IDs when reading from
   `system.table_statistics` in both SHOW STATISTICS and the stats
   cache. This will ensure SHOW STATISTICS always produces the same
   output, and shows us rows in the same order as the stats cache sees
   them (well, reverse order of the stats cache).

Release note (sql change): Make SHOW STATISTICS output more
deterministic.

**sql/stats: forecast table statistics**

Add function to forecast table statistics based on observed statistics.
These forecasts are based on linear regression models over time. For
each set of columns with statistics, we construct a linear regression
model over time for each statistic (row count, null count, distinct
count, average row size, and histogram). If all models are good fits
then we produce a statistics forecast for the set of columns.

Assists: #79872

Release note: None

**sql: add SHOW STATISTICS WITH FORECAST**

Add a new WITH FORECAST option to SHOW STATISTICS which calculates and
displays forecasted statistics along with the existing table statistics.

Also, forbid injecting forecasted stats.

Assists: #79872

Release note (sql change): Add a new WITH FORECAST option to SHOW
STATISTICS which calculates and displays forecasted statistics along
with the existing table statistics.

85673: storage: Incrementally calculate range key stats in CheckSSTConflicts r=erikgrinaker a=itsbilal

This change updates CheckSSTConflicts to incrementally
calculate stats in the presence of range keys in the
SST being ingested. This avoids expensive stats recomputation
after AddSSTable, as previously we were marking stats as
estimates if an SST with range keys was added.

Fixes #83405.

Release note: None.

85794: sql: add not visible index to optimizer r=wenyihu6 a=wenyihu6

This commit adds the logic of the invisible index feature to the optimizer.
After this commit has been merged, the invisible index feature should be fully
functional with `CREATE INDEX` and `CREATE TABLE`.

Assists: #72576

See also: #85239

Release note (sql change): creating a not visible index using
`CREATE TABLE …(INDEX … NOT VISIBLE)` or
`CREATE INDEX … NOT VISIBLE` is now supported.

85974: cluster-ui: update active execution and sessions details r=xinhaoz a=xinhaoz

Fixes #85968
Closes #85912
Closes #85973

This commit adds new details to the active execution details pages:
full scan (both stmt and txn), priority (txn only), and last retry
reason (txn only). New information is also added to the sessions
table and details pages: transaction count, active duration,
recent txn fingerprint ids (cache size comes from a cluster setting).

This commit also fixes a bug in the sessions overview UI where
the duration for closed sessions was incorrectly calcualted based
on the current time instead of the session end time.

Release note (ui change): the following fields have been added to
the active stmt/txn details pages:
- Full Scan: indicates if the execution contains a full scan
- Last Retry Reason (txn page only): the last recorded reason the
txn was retried
- Priority (txn page only): the txn priority
The following fields have been added to the sessions table and page:
- Transaction  count: the number of txns executed by the session
- Session active duration: the time a session spent executing txns
- Session most recent fingerprint ids

-------------------------------------
Retry reason populated example:
<img width="855" alt="image" src="https://user-images.githubusercontent.com/20136951/184201435-6d585d9b-13a9-4e87-86dd-718f03f9e92a.png">
https://www.loom.com/share/e396d5aa7dda4d5995227154c6b5076b

Co-authored-by: Michael Erickson <[email protected]>
Co-authored-by: Bilal Akhtar <[email protected]>
Co-authored-by: wenyihu3 <[email protected]>
Co-authored-by: Xin Hao Zhang <[email protected]>
  • Loading branch information
5 people committed Aug 13, 2022
5 parents c1b8cbc + f86a743 + 219652e + ef4970f + ac161dd commit 0dd438d
Show file tree
Hide file tree
Showing 45 changed files with 3,397 additions and 231 deletions.
2 changes: 1 addition & 1 deletion docs/generated/sql/bnf/show_stats.bnf
Original file line number Diff line number Diff line change
@@ -1,2 +1,2 @@
show_stats_stmt ::=
'SHOW' 'STATISTICS' 'FOR' 'TABLE' table_name
'SHOW' 'STATISTICS' 'FOR' 'TABLE' table_name opt_with_options
2 changes: 1 addition & 1 deletion docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -877,7 +877,7 @@ show_sessions_stmt ::=
| 'SHOW' 'ALL' opt_cluster 'SESSIONS'

show_stats_stmt ::=
'SHOW' 'STATISTICS' 'FOR' 'TABLE' table_name
'SHOW' 'STATISTICS' 'FOR' 'TABLE' table_name opt_with_options

show_tables_stmt ::=
'SHOW' 'TABLES' 'FROM' name '.' name with_comment
Expand Down
3 changes: 3 additions & 0 deletions pkg/jobs/jobspb/wrap.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,9 @@ const AutoStatsName = "__auto__"
// during import.
const ImportStatsName = "__import__"

// ForecastStatsName is the name to use for statistic forecasts.
const ForecastStatsName = "__forecast__"

// AutomaticJobTypes is a list of automatic job types that currently exist.
var AutomaticJobTypes = [...]Type{
TypeAutoCreateStats,
Expand Down
5 changes: 4 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_add_sstable.go
Original file line number Diff line number Diff line change
Expand Up @@ -205,7 +205,10 @@ func EvalAddSSTable(
// bounded with a small number on the SST side.
usePrefixSeek = usePrefixSeek || args.MVCCStats.KeyCount < 100
}
statsDelta, err = storage.CheckSSTConflicts(ctx, sst, readWriter, start, end,
desc := cArgs.EvalCtx.Desc()
leftPeekBound, rightPeekBound := rangeTombstonePeekBounds(
args.Key, args.EndKey, desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey())
statsDelta, err = storage.CheckSSTConflicts(ctx, sst, readWriter, start, end, leftPeekBound, rightPeekBound,
args.DisallowShadowing, args.DisallowShadowingBelow, maxIntents, usePrefixSeek)
if err != nil {
return result.Result{}, errors.Wrap(err, "checking for key collisions")
Expand Down
123 changes: 98 additions & 25 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -739,11 +739,16 @@ func TestEvalAddSSTable(t *testing.T) {
},
// MVCC Range tombstone cases.
"DisallowConflicts allows sst range keys": {
noConflict: true,
expectStatsEst: true,
data: kvs{pointKV("a", 6, "d")},
sst: kvs{rangeKV("a", "b", 8, ""), pointKV("a", 7, "a8")},
expect: kvs{rangeKV("a", "b", 8, ""), pointKV("a", 7, "a8"), pointKV("a", 6, "d")},
noConflict: true,
data: kvs{pointKV("a", 6, "d")},
sst: kvs{rangeKV("a", "b", 8, ""), pointKV("a", 7, "a8")},
expect: kvs{rangeKV("a", "b", 8, ""), pointKV("a", 7, "a8"), pointKV("a", 6, "d")},
},
"DisallowConflicts allows fragmented sst range keys": {
noConflict: true,
data: kvs{pointKV("a", 6, "d")},
sst: kvs{rangeKV("a", "b", 8, ""), pointKV("a", 7, "a8"), rangeKV("c", "d", 8, "")},
expect: kvs{rangeKV("a", "b", 8, ""), pointKV("a", 7, "a8"), pointKV("a", 6, "d"), rangeKV("c", "d", 8, "")},
},
"DisallowConflicts disallows sst range keys below engine point key": {
noConflict: true,
Expand All @@ -764,11 +769,76 @@ func TestEvalAddSSTable(t *testing.T) {
expectErr: &roachpb.WriteTooOldError{},
},
"DisallowConflicts allows sst range keys above engine range keys": {
noConflict: true,
expectStatsEst: true,
data: kvs{pointKV("a", 6, "d"), rangeKV("a", "b", 5, "")},
sst: kvs{pointKV("a", 7, "a8"), rangeKV("a", "b", 8, "")},
expect: kvs{rangeKV("a", "b", 8, ""), rangeKV("a", "b", 5, ""), pointKV("a", 7, "a8"), pointKV("a", 6, "d")},
noConflict: true,
data: kvs{pointKV("a", 6, "d"), rangeKV("a", "b", 5, "")},
sst: kvs{pointKV("a", 7, "a8"), rangeKV("a", "b", 8, "")},
expect: kvs{rangeKV("a", "b", 8, ""), rangeKV("a", "b", 5, ""), pointKV("a", 7, "a8"), pointKV("a", 6, "d")},
},
"DisallowConflicts allows fragmented sst range keys above engine range keys": {
noConflict: true,
data: kvs{pointKV("a", 6, "d"), rangeKV("a", "b", 5, "")},
sst: kvs{pointKV("a", 7, "a8"), rangeKV("a", "b", 8, ""), rangeKV("c", "d", 8, "")},
expect: kvs{rangeKV("a", "b", 8, ""), rangeKV("a", "b", 5, ""), pointKV("a", 7, "a8"), pointKV("a", 6, "d"), rangeKV("c", "d", 8, "")},
},
"DisallowConflicts allows fragmented straddling sst range keys": {
noConflict: true,
data: kvs{pointKV("a", 6, "d"), rangeKV("b", "d", 5, "")},
sst: kvs{pointKV("a", 7, "a8"), rangeKV("a", "c", 8, ""), rangeKV("c", "d", 7, "")},
expect: kvs{rangeKV("a", "b", 8, ""), pointKV("a", 7, "a8"), pointKV("a", 6, "d"), rangeKV("b", "c", 8, ""), rangeKV("b", "c", 5, ""), rangeKV("c", "d", 7, ""), rangeKV("c", "d", 5, "")},
},
"DisallowConflicts allows fragmented straddling sst range keys with no points": {
noConflict: true,
data: kvs{rangeKV("b", "d", 5, "")},
sst: kvs{rangeKV("a", "c", 8, ""), rangeKV("c", "d", 7, "")},
expect: kvs{rangeKV("a", "b", 8, ""), rangeKV("b", "c", 8, ""), rangeKV("b", "c", 5, ""), rangeKV("c", "d", 7, ""), rangeKV("c", "d", 5, "")},
},
"DisallowConflicts allows engine range keys contained within sst range keys": {
noConflict: true,
data: kvs{pointKV("a", 6, "d"), rangeKV("b", "d", 5, "")},
sst: kvs{pointKV("a", 7, "a8"), rangeKV("a", "e", 8, "")},
expect: kvs{rangeKV("a", "b", 8, ""), pointKV("a", 7, "a8"), pointKV("a", 6, "d"), rangeKV("b", "d", 8, ""), rangeKV("b", "d", 5, ""), rangeKV("d", "e", 8, "")},
},
"DisallowConflicts does not skip over engine range keys covering no sst points": {
noConflict: true,
data: kvs{pointKV("a", 6, "d"), rangeKV("b", "c", 6, ""), rangeKV("c", "d", 5, "")},
sst: kvs{pointKV("a", 7, "a8"), rangeKV("a", "e", 8, "")},
expect: kvs{rangeKV("a", "b", 8, ""), pointKV("a", 7, "a8"), pointKV("a", 6, "d"), rangeKV("b", "c", 8, ""), rangeKV("b", "c", 6, ""), rangeKV("c", "d", 8, ""), rangeKV("c", "d", 5, ""), rangeKV("d", "e", 8, "")},
},
"DisallowConflicts does not allow conflict with engine range key covering no sst points": {
noConflict: true,
data: kvs{pointKV("a", 6, "d"), rangeKV("b", "c", 9, ""), rangeKV("c", "d", 5, "")},
sst: kvs{pointKV("a", 7, "a8"), rangeKV("a", "e", 8, "")},
expectErr: &roachpb.WriteTooOldError{},
},
"DisallowConflicts allows sst range keys contained within engine range keys": {
noConflict: true,
data: kvs{pointKV("a", 6, "d"), rangeKV("a", "e", 5, "")},
sst: kvs{pointKV("a", 7, "a8"), rangeKV("b", "d", 8, "")},
expect: kvs{rangeKV("a", "b", 5, ""), pointKV("a", 7, "a8"), pointKV("a", 6, "d"), rangeKV("b", "d", 8, ""), rangeKV("b", "d", 5, ""), rangeKV("d", "e", 5, "")},
},
"DisallowConflicts allows sst range key fragmenting engine range keys": {
noConflict: true,
data: kvs{pointKV("a", 6, "d"), rangeKV("a", "c", 5, ""), rangeKV("c", "e", 6, "")},
sst: kvs{pointKV("a", 7, "a8"), rangeKV("b", "d", 8, "")},
expect: kvs{rangeKV("a", "b", 5, ""), pointKV("a", 7, "a8"), pointKV("a", 6, "d"), rangeKV("b", "c", 8, ""), rangeKV("b", "c", 5, ""), rangeKV("c", "d", 8, ""), rangeKV("c", "d", 6, ""), rangeKV("d", "e", 6, "")},
},
"DisallowConflicts calculates stats correctly for merged range keys": {
noConflict: true,
data: kvs{rangeKV("a", "c", 8, ""), pointKV("a", 6, "d"), rangeKV("d", "e", 8, "")},
sst: kvs{pointKV("a", 10, "de"), rangeKV("c", "d", 8, ""), pointKV("f", 10, "de")},
expect: kvs{rangeKV("a", "e", 8, ""), pointKV("a", 10, "de"), pointKV("a", 6, "d"), pointKV("f", 10, "de")},
},
"DisallowConflicts calculates stats correctly for merged range keys 2": {
noConflict: true,
data: kvs{pointKV("a", 6, "d"), rangeKV("c", "d", 8, "")},
sst: kvs{rangeKV("a", "c", 8, ""), rangeKV("d", "e", 8, ""), pointKV("f", 8, "foo")},
expect: kvs{rangeKV("a", "e", 8, ""), pointKV("a", 6, "d"), pointKV("f", 8, "foo")},
},
"DisallowConflicts calculates stats correctly for merged range keys 3": {
noConflict: true,
data: kvs{pointKV("a", 6, "d"), rangeKV("c", "d", 8, ""), rangeKV("e", "f", 8, "")},
sst: kvs{rangeKV("a", "c", 8, ""), rangeKV("d", "e", 8, ""), pointKV("g", 8, "foo")},
expect: kvs{rangeKV("a", "f", 8, ""), pointKV("a", 6, "d"), pointKV("g", 8, "foo")},
},
"DisallowShadowing disallows sst range keys shadowing live keys": {
noShadow: true,
Expand All @@ -783,11 +853,16 @@ func TestEvalAddSSTable(t *testing.T) {
expect: kvs{rangeKV("a", "b", 7, ""), pointKV("a", 8, "a8"), pointKV("a", 6, "d")},
},
"DisallowShadowing allows idempotent range tombstones": {
noShadow: true,
expectStatsEst: true,
data: kvs{rangeKV("a", "b", 7, "")},
sst: kvs{rangeKV("a", "b", 7, "")},
expect: kvs{rangeKV("a", "b", 7, "")},
noShadow: true,
data: kvs{rangeKV("a", "b", 7, "")},
sst: kvs{rangeKV("a", "b", 7, "")},
expect: kvs{rangeKV("a", "b", 7, "")},
},
"DisallowShadowing calculates stats correctly for merged range keys with idempotence": {
noShadow: true,
data: kvs{rangeKV("b", "d", 8, ""), rangeKV("e", "f", 8, "")},
sst: kvs{rangeKV("a", "c", 8, ""), rangeKV("d", "e", 8, "")},
expect: kvs{rangeKV("a", "f", 8, "")},
},
"DisallowShadowingBelow disallows sst range keys shadowing live keys": {
noShadowBelow: 3,
Expand All @@ -802,18 +877,16 @@ func TestEvalAddSSTable(t *testing.T) {
expect: kvs{rangeKV("a", "b", 7, ""), pointKV("a", 8, "a8"), pointKV("a", 6, "d")},
},
"DisallowShadowingBelow allows idempotent range tombstones": {
noShadowBelow: 3,
expectStatsEst: true,
data: kvs{rangeKV("a", "b", 7, "")},
sst: kvs{rangeKV("a", "b", 7, "")},
expect: kvs{rangeKV("a", "b", 7, "")},
noShadowBelow: 3,
data: kvs{rangeKV("a", "b", 7, "")},
sst: kvs{rangeKV("a", "b", 7, "")},
expect: kvs{rangeKV("a", "b", 7, "")},
},
"DisallowConflict with allowed shadowing disallows idempotent range tombstones": {
noConflict: true,
expectStatsEst: true,
data: kvs{rangeKV("a", "b", 7, "")},
sst: kvs{rangeKV("a", "b", 7, "")},
expectErr: "ingested range key collides with an existing one",
noConflict: true,
data: kvs{rangeKV("a", "b", 7, "")},
sst: kvs{rangeKV("a", "b", 7, "")},
expectErr: "ingested range key collides with an existing one",
},
}
testutils.RunTrueAndFalse(t, "IngestAsWrites", func(t *testing.T, ingestAsWrites bool) {
Expand Down
7 changes: 0 additions & 7 deletions pkg/roachpb/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -1748,8 +1748,6 @@ message AddSSTableRequest {
// reader has already observed, changing the value at that timestamp and above
// it. Use with SSTTimestampToRequestTimestamp to guarantee serializability.
//
// MVCC range tombstones are not currently supported with DisallowConflicts.
//
// Added in 22.1, so check the MVCCAddSSTable version gate before using.
//
// TODO(erikgrinaker): It might be possible to avoid this parameter if we
Expand All @@ -1762,8 +1760,6 @@ message AddSSTableRequest {
// writing above keys that have an existing/visible value (but will write
// above tombstones).
//
// MVCC range tombstones are not currently supported with DisallowShadowing.
//
// TODO(erikgrinaker): Consider removing this in 22.1 if all callers have
// been migrated to DisallowShadowingBelow.
bool disallow_shadowing = 3;
Expand All @@ -1785,9 +1781,6 @@ message AddSSTableRequest {
// If this parameter is used, the value of DisallowShadowing is ignored, so
// callers may pass both for forward and backwards compatibility.
//
// MVCC range tombstones are not currently supported with
// DisallowShadowingBelow.
//
// Added in 22.1, so check the MVCCAddSSTable version gate before using.
util.hlc.Timestamp disallow_shadowing_below = 8 [(gogoproto.nullable) = false];

Expand Down
12 changes: 12 additions & 0 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/security/username"
Expand Down Expand Up @@ -1279,6 +1280,17 @@ func injectTableStats(
return err
}

// Check that we're not injecting any forecasted stats.
for i := range jsonStats {
if jsonStats[i].Name == jobspb.ForecastStatsName {
return errors.WithHintf(
pgerror.New(pgcode.InvalidName, "cannot inject forecasted statistics"),
"either remove forecasts from the statement, or rename them from %q to something else",
jobspb.ForecastStatsName,
)
}
}

// First, delete all statistics for the table.
if _ /* rows */, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.Exec(
params.ctx,
Expand Down
6 changes: 0 additions & 6 deletions pkg/sql/create_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -736,12 +736,6 @@ func (n *createIndexNode) startExec(params runParams) error {
)
}

if n.n.NotVisible {
return unimplemented.Newf(
"Not Visible Index",
"creating a not visible index is not supported yet")
}

// Warn against creating a non-partitioned index on a partitioned table,
// which is undesirable in most cases.
// Avoid the warning if we have PARTITION ALL BY as all indexes will implicitly
Expand Down
46 changes: 20 additions & 26 deletions pkg/sql/create_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@ import (
"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"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -289,6 +288,9 @@ func (n *createStatsNode) makeJobRecord(ctx context.Context) (*jobs.Record, erro
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())
colStats = []jobspb.CreateStatsDetails_ColStat{{
ColumnIDs: columnIDs,
Expand Down Expand Up @@ -377,16 +379,17 @@ func createStatsDefaultColumns(

requestedStats := make(map[string]struct{})

// trackStatsIfNotExists adds the given column IDs as a set to the
// requestedStats set. If the columnIDs were not already in the set, it
// returns true.
trackStatsIfNotExists := func(colIDs []descpb.ColumnID) bool {
key := makeColStatKey(colIDs)
// 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 false
return true
}
requestedStats[key] = struct{}{}
return true
return false
}

// addIndexColumnStatsIfNotExists appends column stats for the given column
Expand All @@ -406,15 +409,15 @@ func createStatsDefaultColumns(
return nil
}

colList := []descpb.ColumnID{colID}
colIDs := []descpb.ColumnID{colID}

// Check for existing stats and remember the requested stats.
if !trackStatsIfNotExists(colList) {
if ok := sortAndTrackStatsExists(colIDs); ok {
return nil
}

colStat := jobspb.CreateStatsDetails_ColStat{
ColumnIDs: colList,
ColumnIDs: colIDs,
HasHistogram: !isInverted,
HistogramMaxBuckets: stats.DefaultHistogramBuckets,
}
Expand Down Expand Up @@ -453,7 +456,7 @@ func createStatsDefaultColumns(
}

// Remember the requested stats so we don't request duplicates.
trackStatsIfNotExists(colIDs)
_ = sortAndTrackStatsExists(colIDs)

// Only generate non-histogram multi-column stats.
colStats = append(colStats, jobspb.CreateStatsDetails_ColStat{
Expand Down Expand Up @@ -497,7 +500,7 @@ func createStatsDefaultColumns(
}

// Check for existing stats and remember the requested stats.
if !trackStatsIfNotExists(colIDs) {
if ok := sortAndTrackStatsExists(colIDs); ok {
continue
}

Expand Down Expand Up @@ -545,9 +548,10 @@ func createStatsDefaultColumns(
continue
}

colList := []descpb.ColumnID{col.GetID()}
colIDs := []descpb.ColumnID{col.GetID()}

if !trackStatsIfNotExists(colList) {
// Check for existing stats.
if ok := sortAndTrackStatsExists(colIDs); ok {
continue
}

Expand All @@ -560,7 +564,7 @@ func createStatsDefaultColumns(
maxHistBuckets = stats.DefaultHistogramBuckets
}
colStats = append(colStats, jobspb.CreateStatsDetails_ColStat{
ColumnIDs: colList,
ColumnIDs: colIDs,
HasHistogram: !colinfo.ColumnTypeIsOnlyInvertedIndexable(col.GetType()),
HistogramMaxBuckets: maxHistBuckets,
})
Expand All @@ -570,16 +574,6 @@ func createStatsDefaultColumns(
return colStats, nil
}

// makeColStatKey constructs a unique key representing cols that can be used
// as the key in a map.
func makeColStatKey(cols []descpb.ColumnID) string {
var colSet util.FastIntSet
for _, c := range cols {
colSet.Add(int(c))
}
return colSet.String()
}

// createStatsResumer implements the jobs.Resumer interface for CreateStats
// jobs. A new instance is created for each job.
type createStatsResumer struct {
Expand Down
10 changes: 0 additions & 10 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -1777,11 +1777,6 @@ func NewTableDesc(
return nil, pgerror.Newf(pgcode.DuplicateRelation, "duplicate index name: %q", d.Name)
}
}
if d.NotVisible {
return nil, unimplemented.Newf(
"Not Visible Index",
"creating a not visible index is not supported yet")
}
if err := validateColumnsAreAccessible(&desc, d.Columns); err != nil {
return nil, err
}
Expand Down Expand Up @@ -1886,11 +1881,6 @@ func NewTableDesc(
// We will add the unique constraint below.
break
}
if d.NotVisible {
return nil, unimplemented.Newf(
"Not Visible Index",
"creating a not visible index is not supported yet")
}
// If the index is named, ensure that the name is unique. Unnamed
// indexes will be given a unique auto-generated name later on when
// AllocateIDs is called.
Expand Down
Loading

0 comments on commit 0dd438d

Please sign in to comment.