Skip to content

Commit

Permalink
sql: support ALTER INDEX ... VISIBILITY ...
Browse files Browse the repository at this point in the history
This commit adds support to `ALTER INDEX ... VISIBILITY ...`. `Visibility` is
currently not supported with `CREATE INDEX` or `CREATE TABLE`. Users would need
to create the index as it is fully not visible or fully visible first. And they
can change the index visibility to any float [0.0, 1.0] using `ALTER INDEX ...
VISIBILITY ...` Invisibility specifies the invisibility of an index to the
optimizer and can be any float64 between [0.0, 1.0]. An index with invisibility
0.0 means that the index is visible. An index with invisibility 1.0 means that
the index is fully not visible. By default, an index should be visible or
invisibility 0.0. An index with invisibility 1.0 is ignored by the optimizer
unless it is used for constraint check or is explicitly selected with index
hinting. An index with invisibility between (0.0, 1.0) would be made fully not
visible to a corresponding fraction of the queries. By convention, we will
refer any indexes with invisibility == 0.0 as visible, any indexes with
invisibility == 1.0 as fully not visible, and any indexes with index visibility
in-between as partially not visible.

Informs cockroachdb#82363

Release note (sql change): `ALTER INDEX ... VISIBILITY ...` is now supported.
It can change an index visibility to any visibility within [0.0, 1.0].
Visibility 0.0 means the index is not visible to the optimizer, while
visibility 1.0 means the index is fully visible. A value in the range
(0.0, 1.0) means the index will be visibile to the corresponding fraction of
queries.

Co-authored-by: Rebecca Taft <[email protected]>
  • Loading branch information
wenyihu6 and rytaft committed Apr 18, 2023
1 parent 3b8e222 commit 452a2de
Show file tree
Hide file tree
Showing 14 changed files with 152 additions and 29 deletions.
4 changes: 4 additions & 0 deletions docs/generated/sql/bnf/alter_index.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -32,12 +32,16 @@ alter_index_stmt ::=
| 'ALTER' 'INDEX' table_name '@' index_name 'NOT' 'VISIBLE'
| 'ALTER' 'INDEX' table_name '@' index_name 'INVISIBLE'
| 'ALTER' 'INDEX' table_name '@' index_name 'VISIBLE'
| 'ALTER' 'INDEX' table_name '@' index_name 'VISIBILITY' 'FCONST'
| 'ALTER' 'INDEX' index_name 'NOT' 'VISIBLE'
| 'ALTER' 'INDEX' index_name 'INVISIBLE'
| 'ALTER' 'INDEX' index_name 'VISIBLE'
| 'ALTER' 'INDEX' index_name 'VISIBILITY' 'FCONST'
| 'ALTER' 'INDEX' 'IF' 'EXISTS' table_name '@' index_name 'NOT' 'VISIBLE'
| 'ALTER' 'INDEX' 'IF' 'EXISTS' table_name '@' index_name 'INVISIBLE'
| 'ALTER' 'INDEX' 'IF' 'EXISTS' table_name '@' index_name 'VISIBLE'
| 'ALTER' 'INDEX' 'IF' 'EXISTS' table_name '@' index_name 'VISIBILITY' 'FCONST'
| 'ALTER' 'INDEX' 'IF' 'EXISTS' index_name 'NOT' 'VISIBLE'
| 'ALTER' 'INDEX' 'IF' 'EXISTS' index_name 'INVISIBLE'
| 'ALTER' 'INDEX' 'IF' 'EXISTS' index_name 'VISIBLE'
| 'ALTER' 'INDEX' 'IF' 'EXISTS' index_name 'VISIBILITY' 'FCONST'
3 changes: 3 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -1453,6 +1453,7 @@ unreserved_keyword ::=
| 'VIEWCLUSTERSETTING'
| 'VIEWDEBUG'
| 'VISIBLE'
| 'VISIBILITY'
| 'VOLATILE'
| 'VOTERS'
| 'WITHIN'
Expand Down Expand Up @@ -2887,6 +2888,7 @@ alter_index_visible ::=
'NOT' 'VISIBLE'
| 'INVISIBLE'
| 'VISIBLE'
| 'VISIBILITY' 'FCONST'

sequence_option_list ::=
( sequence_option_elem ) ( ( sequence_option_elem ) )*
Expand Down Expand Up @@ -4023,6 +4025,7 @@ bare_label_keywords ::=
| 'VIEWDEBUG'
| 'VIRTUAL'
| 'VISIBLE'
| 'VISIBILITY'
| 'VOLATILE'
| 'VOTERS'
| 'WHEN'
Expand Down
61 changes: 56 additions & 5 deletions pkg/sql/opt/exec/execbuilder/testdata/not_visible_index
Original file line number Diff line number Diff line change
Expand Up @@ -1406,7 +1406,8 @@ vectorized: true
spans: [/2 - /2]

statement ok
DROP TABLE t
DROP TABLE t;
RESET optimizer_use_not_visible_indexes

############################################################################
# We should log notices when dropping an invisible index might be different from marking an index invisible.
Expand Down Expand Up @@ -1528,8 +1529,13 @@ statement ok
CREATE INDEX k_idx on t(k) INVISIBLE

statement ok
CREATE INDEX v_idx on t(k);
ALTER INDEX v_idx INVISIBLE
CREATE INDEX k1_idx on t(k);
ALTER INDEX k1_idx INVISIBLE

# Create an index with partial visibility.
statement ok
CREATE INDEX v_idx on t(v);
ALTER INDEX v_idx VISIBILITY 0.5

query TTBITTTBBBF colnames
SELECT * FROM [SHOW INDEX FROM t]
Expand All @@ -1541,5 +1547,50 @@ t t_pkey false 3 v v N/A
t p_idx true 1 p p ASC false false false 0
t k_idx true 1 k k ASC false false false 0
t k_idx true 2 p p ASC false true false 0
t v_idx true 1 k k ASC false false false 0
t v_idx true 2 p p ASC false true false 0
t k1_idx true 1 k k ASC false false false 0
t k1_idx true 2 p p ASC false true false 0
t v_idx true 1 v v ASC false false false 0.5
t v_idx true 2 p p ASC false true false 0.5

# Depending on the random seed v_idx may or may not be visible.
statement ok
SET testing_optimizer_random_seed=6320964980407535655

query T
EXPLAIN SELECT * FROM t WHERE v = 'foo'
----
distribution: local
vectorized: true
·
• index join
│ table: t@t_pkey
└── • scan
missing stats
table: t@v_idx
spans: [/'foo' - /'foo']

statement ok
SET testing_optimizer_random_seed=3164997759865821235

query T
EXPLAIN SELECT * FROM t WHERE v = 'foo'
----
distribution: local
vectorized: true
·
• filter
│ filter: v = 'foo'
└── • scan
missing stats
table: t@t_pkey
spans: FULL SCAN

statement ok
RESET testing_optimizer_random_seed

# When the seed is unset we'll use a non-deterministic random number
# to determine the visibility. Just make sure this doesn't cause an error.
statement ok
SELECT * FROM t WHERE v = 'foo'
1 change: 1 addition & 0 deletions pkg/sql/opt/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -699,6 +699,7 @@ func (md *Metadata) DuplicateTable(
partialIndexPredicates: partialIndexPredicates,
indexPartitionLocalities: tabMeta.indexPartitionLocalities,
checkConstraintsStats: checkConstraintsStats,
notVisibleIndexMap: tabMeta.notVisibleIndexMap,
}
md.tables = append(md.tables, newTabMeta)
regionConfig, ok := md.TableAnnotation(tabID, regionConfigAnnID).(*multiregion.RegionConfig)
Expand Down
42 changes: 42 additions & 0 deletions pkg/sql/opt/table_meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package opt

import (
"context"
"math/rand"

"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
Expand Down Expand Up @@ -204,6 +205,47 @@ type TableMeta struct {

// anns annotates the table metadata with arbitrary data.
anns [maxTableAnnIDCount]interface{}

// notVisibleIndexMap stores information about index invisibility which maps
// from index ordinal to index invisibility.
notVisibleIndexMap map[cat.IndexOrdinal]bool
}

// IsIndexNotVisible returns true if the given index is not visible, and false
// if it is fully visible. If the index is partially visible (i.e., it has a
// value for invisibility in the range (0.0, 1.0)), IsIndexNotVisible randomly
// chooses to make the index fully not visible (to this query) with probability
// proportional to the invisibility setting for the index. Otherwise, the index
// is fully visible (to this query). IsIndexNotVisible caches the result so that
// it always returns the same value for a given index.
func (tm *TableMeta) IsIndexNotVisible(indexOrd cat.IndexOrdinal, rng *rand.Rand) bool {
if tm.notVisibleIndexMap == nil {
tm.notVisibleIndexMap = make(map[cat.IndexOrdinal]bool)
}
if val, ok := tm.notVisibleIndexMap[indexOrd]; ok {
return val
}
// Otherwise, roll the dice to assign index visibility.
indexInvisibility := tm.Table.Index(indexOrd).GetInvisibility()

// If the index invisibility is 40%, we want to make this index invisible 40%
// of the time (invisible to 40% of the queries).
isNotVisible := false
if indexInvisibility == 1 {
isNotVisible = true
} else if indexInvisibility != 0 {
var r float64
if rng == nil {
r = rand.Float64()
} else {
r = rng.Float64()
}
if r <= indexInvisibility {
isNotVisible = true
}
}
tm.notVisibleIndexMap[indexOrd] = isNotVisible
return isNotVisible
}

// TableAnnotation returns the given annotation that is associated with the
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/xform/groupby_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -528,7 +528,7 @@ func (c *CustomFuncs) GenerateLimitedGroupByScans(
var iter scanIndexIter
var sb indexScanBuilder
sb.Init(c, sp.Table)
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, sp, nil /* filters */, rejectPrimaryIndex|rejectInvertedIndexes)
iter.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, sp, nil /* filters */, rejectPrimaryIndex|rejectInvertedIndexes)
iter.ForEach(func(index cat.Index, filters memo.FiltersExpr, indexCols opt.ColSet, isCovering bool, constProj memo.ProjectionsExpr) {
// The iterator only produces pseudo-partial indexes (the predicate is
// true) because no filters are passed to iter.Init to imply a partial
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/xform/join_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -408,7 +408,7 @@ func (c *CustomFuncs) generateLookupJoinsImpl(
var pkCols opt.ColList
var newScanPrivate *memo.ScanPrivate
var iter scanIndexIter
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, scanPrivate, on, rejectInvertedIndexes)
iter.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, scanPrivate, on, rejectInvertedIndexes)
iter.ForEach(func(index cat.Index, onFilters memo.FiltersExpr, indexCols opt.ColSet, _ bool, _ memo.ProjectionsExpr) {
// Skip indexes that do not cover all virtual projection columns, if
// there are any. This can happen when there are multiple virtual
Expand Down Expand Up @@ -804,7 +804,7 @@ func (c *CustomFuncs) GenerateInvertedJoins(
var optionalFilters memo.FiltersExpr

var iter scanIndexIter
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, scanPrivate, on, rejectNonInvertedIndexes)
iter.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, scanPrivate, on, rejectNonInvertedIndexes)
iter.ForEach(func(index cat.Index, onFilters memo.FiltersExpr, indexCols opt.ColSet, _ bool, _ memo.ProjectionsExpr) {
invertedJoin := memo.InvertedJoinExpr{Input: input}
numPrefixCols := index.NonInvertedPrefixColumnCount()
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/xform/limit_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ func (c *CustomFuncs) GenerateLimitedScans(
// Iterate over all non-inverted, non-partial indexes, looking for those
// that can be limited.
var iter scanIndexIter
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, scanPrivate, nil /* filters */, rejectInvertedIndexes|rejectPartialIndexes)
iter.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, scanPrivate, nil /* filters */, rejectInvertedIndexes|rejectPartialIndexes)
iter.ForEach(func(index cat.Index, filters memo.FiltersExpr, indexCols opt.ColSet, isCovering bool, constProj memo.ProjectionsExpr) {
// The iterator rejects partial indexes because there are no filters to
// imply a partial index predicate. constProj is a projection of
Expand Down Expand Up @@ -286,7 +286,7 @@ func (c *CustomFuncs) GenerateLimitedTopKScans(
var iter scanIndexIter
var sb indexScanBuilder
sb.Init(c, sp.Table)
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, sp, nil /* filters */, rejectPrimaryIndex|rejectInvertedIndexes)
iter.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, sp, nil /* filters */, rejectPrimaryIndex|rejectInvertedIndexes)
iter.ForEach(func(index cat.Index, filters memo.FiltersExpr, indexCols opt.ColSet, isCovering bool, constProj memo.ProjectionsExpr) {
// The iterator only produces pseudo-partial indexes (the predicate is
// true) because no filters are passed to iter.Init to imply a partial
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/opt/xform/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +98,8 @@ type Optimizer struct {
// JoinOrderBuilder adds new join orderings to the memo.
jb JoinOrderBuilder

// rng is used to deterministically perturb costs and/or disable rules.
// rng is used to deterministically perturb costs and/or disable rules and/or
// determine the visibility (for this query) of a partially visible index.
rng *rand.Rand

// scratchSort is used to avoid repeated allocations during sort enforcement.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/xform/scan_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ func (c *CustomFuncs) GenerateIndexScans(
// Iterate over all non-inverted and non-partial secondary indexes.
var pkCols opt.ColSet
var iter scanIndexIter
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, scanPrivate, nil /* filters */, rejectPrimaryIndex|rejectInvertedIndexes)
iter.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, scanPrivate, nil /* filters */, rejectPrimaryIndex|rejectInvertedIndexes)
iter.ForEach(func(index cat.Index, filters memo.FiltersExpr, indexCols opt.ColSet, isCovering bool, constProj memo.ProjectionsExpr) {
// The iterator only produces pseudo-partial indexes (the predicate is
// true) because no filters are passed to iter.Init to imply a partial
Expand Down
13 changes: 6 additions & 7 deletions pkg/sql/opt/xform/scan_index_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/opt/norm"
"github.com/cockroachdb/cockroach/pkg/sql/opt/partialidx"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -51,7 +50,7 @@ const (
// of a Scan operator table.
type scanIndexIter struct {
evalCtx *eval.Context
f *norm.Factory
e *explorer
im *partialidx.Implicator
tabMeta *opt.TableMeta

Expand Down Expand Up @@ -86,7 +85,7 @@ type scanIndexIter struct {
// Init initializes a new scanIndexIter.
func (it *scanIndexIter) Init(
evalCtx *eval.Context,
f *norm.Factory,
e *explorer,
mem *memo.Memo,
im *partialidx.Implicator,
scanPrivate *memo.ScanPrivate,
Expand All @@ -97,7 +96,7 @@ func (it *scanIndexIter) Init(
// reused. Field reuse must be explicit.
*it = scanIndexIter{
evalCtx: evalCtx,
f: f,
e: e,
im: im,
tabMeta: mem.Metadata().TableMeta(scanPrivate.Table),
scanPrivate: scanPrivate,
Expand Down Expand Up @@ -227,7 +226,7 @@ func (it *scanIndexIter) ForEachStartingAfter(ord int, f enumerateIndexFunc) {
} else {
// If we are not forcing any specific index and not visible index feature is
// enabled here, ignore not visible indexes.
if (index.GetInvisibility() != 0.0) && !it.scanPrivate.Flags.DisableNotVisibleIndex &&
if it.tabMeta.IsIndexNotVisible(ord, it.e.o.rng) && !it.scanPrivate.Flags.DisableNotVisibleIndex &&
!it.evalCtx.SessionData().OptimizerUseNotVisibleIndexes {
continue
}
Expand Down Expand Up @@ -374,9 +373,9 @@ func (it *scanIndexIter) buildConstProjectionsFromPredicate(
panic(errors.AssertionFailedf("could not extract constant value for column %d", col))
}

scalar := it.f.ConstructConstVal(val, typ)
scalar := it.e.f.ConstructConstVal(val, typ)

proj = append(proj, it.f.ConstructProjectionsItem(
proj = append(proj, it.e.f.ConstructProjectionsItem(
scalar,
col,
))
Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/opt/xform/select_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ func (c *CustomFuncs) GeneratePartialIndexScans(
// Iterate over all partial indexes.
var pkCols opt.ColSet
var iter scanIndexIter
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, scanPrivate, filters, rejectNonPartialIndexes|rejectInvertedIndexes)
iter.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, scanPrivate, filters, rejectNonPartialIndexes|rejectInvertedIndexes)
iter.ForEach(func(index cat.Index, remainingFilters memo.FiltersExpr, indexCols opt.ColSet, isCovering bool, constProj memo.ProjectionsExpr) {
var sb indexScanBuilder
sb.Init(c, scanPrivate.Table)
Expand Down Expand Up @@ -437,7 +437,7 @@ func (c *CustomFuncs) GenerateConstrainedScans(

// Iterate over all non-inverted indexes.
var iter scanIndexIter
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, scanPrivate, explicitFilters, rejectInvertedIndexes)
iter.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, scanPrivate, explicitFilters, rejectInvertedIndexes)
iter.ForEach(func(index cat.Index, filters memo.FiltersExpr, indexCols opt.ColSet, isCovering bool, constProj memo.ProjectionsExpr) {

// Create a prefix sorter that describes which index partitions are
Expand Down Expand Up @@ -881,7 +881,7 @@ func (c *CustomFuncs) GenerateInvertedIndexScans(

// Iterate over all inverted indexes.
var iter scanIndexIter
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, scanPrivate, filters, rejectNonInvertedIndexes)
iter.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, scanPrivate, filters, rejectNonInvertedIndexes)
iter.ForEach(func(index cat.Index, filters memo.FiltersExpr, indexCols opt.ColSet, _ bool, _ memo.ProjectionsExpr) {
// Check whether the filter can constrain the index.
spanExpr, constraint, remainingFilters, pfState, ok := invertedidx.TryFilterInvertedIndex(
Expand Down Expand Up @@ -1112,7 +1112,7 @@ func (c *CustomFuncs) GenerateZigzagJoins(
// TODO(mgartner): We should consider primary indexes when it has multiple
// columns and only the first is being constrained.
var iter scanIndexIter
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, scanPrivate, filters, rejectPrimaryIndex|rejectInvertedIndexes)
iter.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, scanPrivate, filters, rejectPrimaryIndex|rejectInvertedIndexes)
iter.ForEach(func(leftIndex cat.Index, outerFilters memo.FiltersExpr, leftCols opt.ColSet, _ bool, _ memo.ProjectionsExpr) {
leftFixed := c.indexConstrainedCols(leftIndex, scanPrivate.Table, fixedCols)
// Short-circuit quickly if the first column in the index is not a fixed
Expand All @@ -1122,7 +1122,7 @@ func (c *CustomFuncs) GenerateZigzagJoins(
}

var iter2 scanIndexIter
iter2.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, scanPrivate, outerFilters, rejectPrimaryIndex|rejectInvertedIndexes)
iter2.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, scanPrivate, outerFilters, rejectPrimaryIndex|rejectInvertedIndexes)
iter2.SetOriginalFilters(filters)
iter2.ForEachStartingAfter(leftIndex.Ordinal(), func(rightIndex cat.Index, innerFilters memo.FiltersExpr, rightCols opt.ColSet, _ bool, _ memo.ProjectionsExpr) {
// Check if we have zigzag hints.
Expand Down Expand Up @@ -1451,7 +1451,7 @@ func (c *CustomFuncs) GenerateInvertedIndexZigzagJoins(

// Iterate over all inverted indexes.
var iter scanIndexIter
iter.Init(c.e.evalCtx, c.e.f, c.e.mem, &c.im, scanPrivate, filters, rejectNonInvertedIndexes)
iter.Init(c.e.evalCtx, c.e, c.e.mem, &c.im, scanPrivate, filters, rejectNonInvertedIndexes)
iter.ForEach(func(index cat.Index, filters memo.FiltersExpr, indexCols opt.ColSet, _ bool, _ memo.ProjectionsExpr) {
// Check if we have zigzag hints.
if !scanPrivate.Flags.ZigzagIndexes.Empty() && !scanPrivate.Flags.ZigzagIndexes.Contains(index.Ordinal()) {
Expand Down
Loading

0 comments on commit 452a2de

Please sign in to comment.