Skip to content

Commit

Permalink
Merge #84912
Browse files Browse the repository at this point in the history
84912: sql: populate NotVisible to index definitions r=wenyihu6 a=wenyihu6

This commit populates the field `NotVisible` for `IndexDescriptor`,
`Tree.CreateIndex`, and `Tree.TableIndexDef` at different places. In order to
add some test cases for this commit, I included a test case that checks the
output of `SHOW CREATE` under
`pkg/sql/opt/xform/testdata/rules/not_visible_index` to make sure that
the field has been populated correctly. Creating invisible indexes under
`pkg/sql/opt/testutils/testcat` no longer returns an unimplemented error. But
executing `CREATE INDEX ... NOT VISIBLE` in logictests or in a cluster still
returns an unimplemented error.

See also: #84783

Assists: #72576

Release note: none

Co-authored-by: wenyihu3 <[email protected]>
  • Loading branch information
craig[bot] and wenyihu6 committed Aug 3, 2022
2 parents 5cd0309 + 2524f94 commit 7326bbb
Show file tree
Hide file tree
Showing 14 changed files with 66 additions and 18 deletions.
2 changes: 2 additions & 0 deletions pkg/cmd/cr2pg/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ func main() {
Inverted: def.Inverted,
Columns: def.Columns,
Storing: def.Storing,
// Postgres doesn't support NotVisible Index, so NotVisible is not populated here.
})
case *tree.UniqueConstraintTableDef:
if def.PrimaryKey {
Expand All @@ -103,6 +104,7 @@ func main() {
Inverted: def.Inverted,
Columns: def.Columns,
Storing: def.Storing,
// Postgres doesn't support NotVisible Index, so NotVisible is not populated here.
})
default:
newdefs = append(newdefs, def)
Expand Down
3 changes: 3 additions & 0 deletions pkg/internal/sqlsmith/alter.go
Original file line number Diff line number Diff line change
Expand Up @@ -351,6 +351,9 @@ func makeCreateIndex(s *Smither) (tree.Statement, bool) {
Storing: storing,
Inverted: inverted,
Concurrently: s.coin(),
// TODO(wenyihu6): uncomment the following line after we support not visible
// index.
// NotVisible: s.d6() == 1, // NotVisible index is rare 1/6 chance.
}, true
}

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,7 @@ func (n *alterTableNode) startExec(params runParams) error {
idx := descpb.IndexDescriptor{
Name: string(d.Name),
Unique: true,
NotVisible: d.NotVisible,
StoreColumnNames: d.Storing.ToStrings(),
CreatedAtNanos: params.EvalContext().GetTxnTimestamp(time.Microsecond).UnixNano(),
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/create_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,6 +203,7 @@ func makeIndexDescriptor(
StoreColumnNames: n.Storing.ToStrings(),
CreatedExplicitly: true,
CreatedAtNanos: params.EvalContext().GetTxnTimestamp(time.Microsecond).UnixNano(),
NotVisible: n.NotVisible,
}

columnsToCheckForOpclass := columns
Expand Down
13 changes: 8 additions & 5 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -712,7 +712,7 @@ func addUniqueWithoutIndexTableDef(
if d.NotVisible {
// Theoretically, this should never happen because this is not supported by
// the parser. This is just a safe check.
return pgerror.Newf(pgcode.IntegrityConstraintViolation,
return pgerror.Newf(pgcode.FeatureNotSupported,
"creating a unique constraint using UNIQUE WITH NOT VISIBLE INDEX is not supported",
)
}
Expand Down Expand Up @@ -1810,6 +1810,7 @@ func NewTableDesc(
Name: string(d.Name),
StoreColumnNames: d.Storing.ToStrings(),
Version: indexEncodingVersion,
NotVisible: d.NotVisible,
}
if d.Inverted {
idx.Type = descpb.IndexDescriptor_INVERTED
Expand Down Expand Up @@ -1924,6 +1925,7 @@ func NewTableDesc(
Unique: true,
StoreColumnNames: d.Storing.ToStrings(),
Version: indexEncodingVersion,
NotVisible: d.NotVisible,
}
columns := d.Columns
if d.Sharded != nil {
Expand Down Expand Up @@ -2648,10 +2650,11 @@ func replaceLikeTableOpts(n *tree.CreateTable, params runParams) (tree.TableDefs
continue
}
indexDef := tree.IndexTableDef{
Name: tree.Name(idx.GetName()),
Inverted: idx.GetType() == descpb.IndexDescriptor_INVERTED,
Storing: make(tree.NameList, 0, idx.NumSecondaryStoredColumns()),
Columns: make(tree.IndexElemList, 0, idx.NumKeyColumns()),
Name: tree.Name(idx.GetName()),
Inverted: idx.GetType() == descpb.IndexDescriptor_INVERTED,
Storing: make(tree.NameList, 0, idx.NumSecondaryStoredColumns()),
Columns: make(tree.IndexElemList, 0, idx.NumKeyColumns()),
NotVisible: idx.IsNotVisible(),
}
numColumns := idx.NumKeyColumns()
if idx.IsSharded() {
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/importer/read_import_mysql.go
Original file line number Diff line number Diff line change
Expand Up @@ -495,6 +495,8 @@ func mysqlTableToCockroach(
if raw.Info.Primary {
idxName = tree.Name(tabledesc.PrimaryKeyIndexName(name))
}
// TODO(wenyihu6): support importing mysql CREATE TABLE statement with not
// visible index.
idx := tree.IndexTableDef{Name: idxName, Columns: elems}
if raw.Info.Primary || raw.Info.Unique {
stmt.Defs = append(stmt.Defs, &tree.UniqueConstraintTableDef{IndexTableDef: idx, PrimaryKey: raw.Info.Primary})
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/importer/read_import_pgdump.go
Original file line number Diff line number Diff line change
Expand Up @@ -653,6 +653,7 @@ func readPostgresStmt(
Inverted: stmt.Inverted,
PartitionByIndex: stmt.PartitionByIndex,
StorageParams: stmt.StorageParams,
// Postgres doesn't support NotVisible Index, so NotVisible is not populated here.
}
if stmt.Unique {
idx = &tree.UniqueConstraintTableDef{IndexTableDef: *idx.(*tree.IndexTableDef)}
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/opt/testutils/testcat/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,6 @@ go_library(
"//pkg/sql/types",
"//pkg/sql/vtable",
"//pkg/util",
"//pkg/util/errorutil/unimplemented",
"//pkg/util/treeprinter",
"@com_github_cockroachdb_errors//:errors",
"@com_github_lib_pq//oid",
Expand Down
18 changes: 7 additions & 11 deletions pkg/sql/opt/testutils/testcat/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
)

type indexType int
Expand Down Expand Up @@ -705,17 +704,14 @@ func (tt *Table) addIndexWithVersion(
tt.addUniqueConstraint(def.Name, def.Columns, def.Predicate, false /* withoutIndex */)
}

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

idx := &Index{
IdxName: tt.makeIndexName(def.Name, def.Columns, typ),
Unique: typ != nonUniqueIndex,
Inverted: def.Inverted,
IdxZone: cat.EmptyZone(),
table: tt,
version: version,
IdxName: tt.makeIndexName(def.Name, def.Columns, typ),
Unique: typ != nonUniqueIndex,
Inverted: def.Inverted,
IdxZone: cat.EmptyZone(),
table: tt,
version: version,
NotVisible: def.NotVisible,
}

// Look for name suffixes indicating this is a mutation index.
Expand Down
29 changes: 29 additions & 0 deletions pkg/sql/opt/testutils/testcat/testdata/index
Original file line number Diff line number Diff line change
Expand Up @@ -169,3 +169,32 @@ TABLE xyz
├── crdb_internal_idx_expr_2_inverted_key encodedkey not null [inverted]
├── x int not null
└── WHERE v > 1

exec-ddl
CREATE TABLE t_invisible (k INT PRIMARY KEY, v INT, i INT, INDEX idx_v_visible(v) VISIBLE, INDEX idx_i_invisible(i) NOT VISIBLE)
----

exec-ddl
CREATE INDEX idx_v_invisible ON t_invisible(v) NOT VISIBLE
----

exec-ddl
SHOW CREATE t_invisible
----
TABLE t_invisible
├── k int not null
├── v int
├── i int
├── crdb_internal_mvcc_timestamp decimal [hidden] [system]
├── tableoid oid [hidden] [system]
├── PRIMARY INDEX t_invisible_pkey
│ └── k int not null
├── INDEX idx_v_visible
│ ├── v int
│ └── k int not null
├── INDEX idx_i_invisible NOT VISIBLE
│ ├── i int
│ └── k int not null
└── INDEX idx_v_invisible NOT VISIBLE
├── v int
└── k int not null
2 changes: 2 additions & 0 deletions pkg/sql/randgen/mutator.go
Original file line number Diff line number Diff line change
Expand Up @@ -778,6 +778,7 @@ func postgresCreateTableMutator(
Inverted: def.Inverted,
Columns: newCols,
Storing: def.Storing,
// Postgres doesn't support NotVisible Index, so NotVisible is not populated here.
})
changed = true
}
Expand Down Expand Up @@ -828,6 +829,7 @@ func postgresCreateTableMutator(
Inverted: def.Inverted,
Columns: newCols,
Storing: def.Storing,
// Postgres doesn't support NotVisible Index, so NotVisible is not populated here.
})
changed = true
default:
Expand Down
6 changes: 6 additions & 0 deletions pkg/sql/randgen/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -198,6 +198,12 @@ func RandCreateTableWithColumnIndexNumberGenerator(
IndexTableDef: indexDef,
})
} else {
// Due to parsing issue with creating unique indexes in a CREATE TABLE
// definition, we are only supporting not visible non-unique indexes for
// now. Make non-unique indexes not visible 1/6 of the time.
// TODO(wenyihu6): uncomment the following line after we support not visible
// index.
// indexDef.NotVisible = rng.Intn(6) == 0
defs = append(defs, &indexDef)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ func CreateIndex(b BuildCtx, n *tree.CreateIndex) {
IsUnique: n.Unique,
IsInverted: n.Inverted,
IsConcurrently: n.Concurrently,
IsNotVisible: false, // TODO(wenyihu6): populate not visible property after CREATE
IsNotVisible: n.NotVisible,
}
var relation scpb.Element
var source *scpb.PrimaryIndex
Expand Down
3 changes: 3 additions & 0 deletions pkg/workload/schemachange/operation_generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -918,6 +918,9 @@ func (og *operationGenerator) createIndex(ctx context.Context, tx pgx.Tx) (strin
Unique: og.randIntn(4) == 0, // 25% UNIQUE
Inverted: og.randIntn(10) == 0, // 10% INVERTED
IfNotExists: og.randIntn(2) == 0, // 50% IF NOT EXISTS
// TODO(wenyihu6): uncomment the following line after we support not visible
// index.
// NotVisible: og.randIntn(20) == 0, // 5% NOT VISIBLE
}

regionColumn := ""
Expand Down

0 comments on commit 7326bbb

Please sign in to comment.