diff --git a/docs/generated/sql/bnf/alter_index.bnf b/docs/generated/sql/bnf/alter_index.bnf index ca8ff21d2e4f..3ac858b3ff8b 100644 --- a/docs/generated/sql/bnf/alter_index.bnf +++ b/docs/generated/sql/bnf/alter_index.bnf @@ -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' diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index ce5e87e1b46c..e038258e218c 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1453,6 +1453,7 @@ unreserved_keyword ::= | 'VIEWCLUSTERSETTING' | 'VIEWDEBUG' | 'VISIBLE' + | 'VISIBILITY' | 'VOLATILE' | 'VOTERS' | 'WITHIN' @@ -2887,6 +2888,7 @@ alter_index_visible ::= 'NOT' 'VISIBLE' | 'INVISIBLE' | 'VISIBLE' + | 'VISIBILITY' 'FCONST' sequence_option_list ::= ( sequence_option_elem ) ( ( sequence_option_elem ) )* @@ -4023,6 +4025,7 @@ bare_label_keywords ::= | 'VIEWDEBUG' | 'VIRTUAL' | 'VISIBLE' + | 'VISIBILITY' | 'VOLATILE' | 'VOTERS' | 'WHEN' diff --git a/pkg/sql/opt/exec/execbuilder/testdata/not_visible_index b/pkg/sql/opt/exec/execbuilder/testdata/not_visible_index index 99636c052185..4e983b079604 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/not_visible_index +++ b/pkg/sql/opt/exec/execbuilder/testdata/not_visible_index @@ -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. @@ -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] @@ -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' diff --git a/pkg/sql/opt/metadata.go b/pkg/sql/opt/metadata.go index 853874307c56..8459acc601aa 100644 --- a/pkg/sql/opt/metadata.go +++ b/pkg/sql/opt/metadata.go @@ -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) diff --git a/pkg/sql/opt/table_meta.go b/pkg/sql/opt/table_meta.go index 9421e5d522c8..a630865e6bbb 100644 --- a/pkg/sql/opt/table_meta.go +++ b/pkg/sql/opt/table_meta.go @@ -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" @@ -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 diff --git a/pkg/sql/opt/xform/groupby_funcs.go b/pkg/sql/opt/xform/groupby_funcs.go index 6ed5f5779547..11a47607e435 100644 --- a/pkg/sql/opt/xform/groupby_funcs.go +++ b/pkg/sql/opt/xform/groupby_funcs.go @@ -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 diff --git a/pkg/sql/opt/xform/join_funcs.go b/pkg/sql/opt/xform/join_funcs.go index 8c3dcfaf81c8..96c65d1cec1a 100644 --- a/pkg/sql/opt/xform/join_funcs.go +++ b/pkg/sql/opt/xform/join_funcs.go @@ -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 @@ -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() diff --git a/pkg/sql/opt/xform/limit_funcs.go b/pkg/sql/opt/xform/limit_funcs.go index c825757799ee..f6102bd9e331 100644 --- a/pkg/sql/opt/xform/limit_funcs.go +++ b/pkg/sql/opt/xform/limit_funcs.go @@ -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 @@ -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 diff --git a/pkg/sql/opt/xform/optimizer.go b/pkg/sql/opt/xform/optimizer.go index 32dc7fc2932e..2a26cd4be31e 100644 --- a/pkg/sql/opt/xform/optimizer.go +++ b/pkg/sql/opt/xform/optimizer.go @@ -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. diff --git a/pkg/sql/opt/xform/scan_funcs.go b/pkg/sql/opt/xform/scan_funcs.go index bd4bd15c391a..00a868c9b701 100644 --- a/pkg/sql/opt/xform/scan_funcs.go +++ b/pkg/sql/opt/xform/scan_funcs.go @@ -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 diff --git a/pkg/sql/opt/xform/scan_index_iter.go b/pkg/sql/opt/xform/scan_index_iter.go index 644b9ced9cf6..d76a205aa5ea 100644 --- a/pkg/sql/opt/xform/scan_index_iter.go +++ b/pkg/sql/opt/xform/scan_index_iter.go @@ -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" @@ -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 @@ -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, @@ -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, @@ -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 } @@ -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, )) diff --git a/pkg/sql/opt/xform/select_funcs.go b/pkg/sql/opt/xform/select_funcs.go index b53d9a7acbe8..a94d3426bddf 100644 --- a/pkg/sql/opt/xform/select_funcs.go +++ b/pkg/sql/opt/xform/select_funcs.go @@ -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) @@ -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 @@ -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( @@ -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 @@ -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. @@ -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()) { diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 43431be944f9..60e6d36597b7 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -991,7 +991,7 @@ func (u *sqlSymUnion) showCreateFormatOption() tree.ShowCreateFormatOption { %token UPDATE UPSERT UNSET UNTIL USE USER USERS USING UUID %token VALID VALIDATE VALUE VALUES VARBIT VARCHAR VARIADIC VERIFY_BACKUP_TABLE_DATA VIEW VARYING VIEWACTIVITY VIEWACTIVITYREDACTED VIEWDEBUG -%token VIEWCLUSTERMETADATA VIEWCLUSTERSETTING VIRTUAL VISIBLE INVISIBLE VOLATILE VOTERS +%token VIEWCLUSTERMETADATA VIEWCLUSTERSETTING VIRTUAL VISIBLE INVISIBLE VISIBILITY VOLATILE VOTERS %token WHEN WHERE WINDOW WITH WITHIN WITHOUT WORK WRITE @@ -1498,7 +1498,7 @@ func (u *sqlSymUnion) showCreateFormatOption() tree.ShowCreateFormatOption { %type <*tree.IndexFlags> opt_index_flags %type <*tree.IndexFlags> index_flags_param %type <*tree.IndexFlags> index_flags_param_list -%type a_expr b_expr c_expr d_expr typed_literal alter_index_visible opt_index_visible +%type a_expr b_expr c_expr d_expr typed_literal %type substr_from substr_for %type in_expr %type having_clause @@ -1548,7 +1548,7 @@ func (u *sqlSymUnion) showCreateFormatOption() tree.ShowCreateFormatOption { %type extract_arg %type opt_varying -%type <*tree.NumVal> signed_iconst only_signed_iconst +%type <*tree.NumVal> signed_iconst only_signed_iconst alter_index_visible opt_index_visible %type <*tree.NumVal> signed_fconst only_signed_fconst %type iconst32 %type signed_iconst64 @@ -2213,7 +2213,7 @@ alter_range_stmt: // ALTER INDEX ... UNSPLIT ALL // ALTER INDEX ... SCATTER [ FROM ( ) TO ( ) ] // ALTER INDEX ... RELOCATE [ LEASE | VOTERS | NONVOTERS ] -// ALTER INDEX ... [VISIBLE | NOT VISIBLE | INVISIBLE] +// ALTER INDEX ... [VISIBLE | NOT VISIBLE | INVISIBLE | VISIBILITY ...] // // Zone configurations: // DISCARD @@ -2388,6 +2388,18 @@ alter_index_visible: { $$.val = tree.NewNumVal(constant.MakeFloat64(0.0), "0.0", false /*negative*/) } +| VISIBILITY FCONST + { + visibilityConst, _ := constant.Float64Val($2.numVal().AsConstantValue()) + if visibilityConst < 0.0 || visibilityConst > 1.0 { + sqllex.Error("index visibility must be between 0 and 1") + return 1 + } + invisibilityConst := 1.0 - visibilityConst + invisibilityStr := fmt.Sprintf("%.2f", invisibilityConst) + treeNumVal := tree.NewNumVal(constant.MakeFloat64(invisibilityConst), invisibilityStr, false /*negative*/) + $$.val = treeNumVal + } // Note: even though the ALTER RANGE ... CONFIGURE ZONE syntax only // accepts unrestricted names in the 3rd position, such that we could @@ -16672,6 +16684,7 @@ unreserved_keyword: | VIEWCLUSTERSETTING | VIEWDEBUG | VISIBLE +| VISIBILITY | VOLATILE | VOTERS | WITHIN @@ -17241,6 +17254,7 @@ bare_label_keywords: | VIEWDEBUG | VIRTUAL | VISIBLE +| VISIBILITY | VOLATILE | VOTERS | WHEN diff --git a/pkg/sql/parser/testdata/alter_index b/pkg/sql/parser/testdata/alter_index index f57872a4ce9e..0fa1c58bb655 100644 --- a/pkg/sql/parser/testdata/alter_index +++ b/pkg/sql/parser/testdata/alter_index @@ -423,3 +423,11 @@ ALTER INDEX db.t@i NOT VISIBLE ALTER INDEX db.t@i NOT VISIBLE -- fully parenthesized ALTER INDEX db.t@i NOT VISIBLE -- literals removed ALTER INDEX _._@_ NOT VISIBLE -- identifiers removed + +parse +ALTER INDEX db.t@i VISIBILITY 0.2 +---- +ALTER INDEX db.t@i VISIBILITY 0.20 -- normalized! +ALTER INDEX db.t@i VISIBILITY 0.20 -- fully parenthesized +ALTER INDEX db.t@i VISIBILITY 0.20 -- literals removed +ALTER INDEX _._@_ VISIBILITY 0.20 -- identifiers removed