Skip to content

Commit

Permalink
sql: added support for create index inside new schema changer
Browse files Browse the repository at this point in the history
Fixes: #67264

Previously, create index was not supported inside the
new shcema changer. This was inadequate because, to
help support transaction schema changes in the future
we need all schema changes inside the new schema changer.
To address this, this patch adds CREATE INDEX inside
the new schema changer.

Release note: None
  • Loading branch information
fqazi committed Nov 8, 2021
1 parent faf55a2 commit f5762cb
Show file tree
Hide file tree
Showing 33 changed files with 5,483 additions and 1,616 deletions.
13 changes: 13 additions & 0 deletions pkg/ccl/logictestccl/testdata/logic_test/new_schema_changer
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
statement ok
SET experimental_use_new_schema_changer = 'on'

subtest create_index

statement ok
CREATE TABLE defaultdb.t1 (id INT PRIMARY KEY, name varchar(256), money int)

statement ok
CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) PARTITION BY LIST (id) (
PARTITION p1 VALUES IN (1)
);

1 change: 1 addition & 0 deletions pkg/ccl/partitionccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ go_library(
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/rowenc",
"//pkg/sql/schemachanger/scbuild",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"//pkg/util/encoding",
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/partitionccl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
Expand Down Expand Up @@ -611,4 +612,5 @@ func selectPartitionExprsByName(

func init() {
sql.CreatePartitioningCCL = createPartitioning
scbuild.CreatePartitioningCCL = createPartitioning
}
1 change: 1 addition & 0 deletions pkg/kv/kvserver/protectedts/ptstorage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ go_test(
"//pkg/security/securitytest",
"//pkg/server",
"//pkg/sql",
"//pkg/sql/catalog",
"//pkg/sql/catalog/colinfo",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
Expand Down
7 changes: 7 additions & 0 deletions pkg/kv/kvserver/protectedts/ptstorage/storage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
Expand Down Expand Up @@ -801,3 +802,9 @@ func (ie *wrappedInternalExecutor) setErrFunc(f func(statement string) error) {
defer ie.mu.Unlock()
ie.mu.errFunc = f
}

func (ie *wrappedInternalExecutor) WithSyntheticDescriptors(
descs []catalog.Descriptor, run func() error,
) error {
panic("not implemented")
}
8 changes: 3 additions & 5 deletions pkg/sql/catalog/tabledesc/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ func (desc *wrapper) KeysPerRow(indexID descpb.IndexID) (int, error) {
return len(desc.Families), nil
}

// buildIndexName returns an index name that is not equal to any
// BuildIndexName returns an index name that is not equal to any
// of tableDesc's indexes, roughly following Postgres's conventions for naming
// anonymous indexes. For example:
//
Expand All @@ -150,9 +150,7 @@ func (desc *wrapper) KeysPerRow(indexID descpb.IndexID) (int, error) {
// CREATE INDEX ON t ((a + b), c, lower(d))
// => t_expr_c_expr1_idx
//
func buildIndexName(tableDesc *Mutable, index catalog.Index) (string, error) {
idx := index.IndexDesc()

func BuildIndexName(tableDesc *Mutable, idx *descpb.IndexDescriptor) (string, error) {
// An index name has a segment for the table name, each key column, and a
// final word (either "idx" or "key").
segments := make([]string, 0, len(idx.KeyColumnNames)+2)
Expand Down Expand Up @@ -680,7 +678,7 @@ func (desc *Mutable) allocateIndexIDs(columnNames map[string]descpb.ColumnID) er
// Assign names to unnamed indexes.
err := catalog.ForEachDeletableNonPrimaryIndex(desc, func(idx catalog.Index) error {
if len(idx.GetName()) == 0 {
name, err := buildIndexName(desc, idx)
name, err := BuildIndexName(desc, idx.IndexDesc())
if err != nil {
return err
}
Expand Down
50 changes: 2 additions & 48 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild"
"github.com/cockroachdb/cockroach/pkg/sql/sem/builtins"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
Expand Down Expand Up @@ -2702,60 +2703,13 @@ func makeShardColumnDesc(
}
col.Name = tabledesc.GetShardColumnName(colNames, int32(buckets))
if useDatumsToBytes {
col.ComputeExpr = makeHashShardComputeExpr(colNames, buckets)
col.ComputeExpr = scbuild.MakeHashShardComputeExpr(colNames, buckets)
} else {
col.ComputeExpr = makeDeprecatedHashShardComputeExpr(colNames, buckets)
}

return col, nil
}

// makeDeprecatedHashShardComputeExpr creates the serialized computed expression for a hash shard
// column based on the column names and the number of buckets. The expression will be
// of the form:
//
// mod(fnv32(crdb_internal.datums_to_bytes(...)),buckets)
//
func makeHashShardComputeExpr(colNames []string, buckets int) *string {
unresolvedFunc := func(funcName string) tree.ResolvableFunctionReference {
return tree.ResolvableFunctionReference{
FunctionReference: &tree.UnresolvedName{
NumParts: 1,
Parts: tree.NameParts{funcName},
},
}
}
columnItems := func() tree.Exprs {
exprs := make(tree.Exprs, len(colNames))
for i := range exprs {
exprs[i] = &tree.ColumnItem{ColumnName: tree.Name(colNames[i])}
}
return exprs
}
hashedColumnsExpr := func() tree.Expr {
return &tree.FuncExpr{
Func: unresolvedFunc("fnv32"),
Exprs: tree.Exprs{
&tree.FuncExpr{
Func: unresolvedFunc("crdb_internal.datums_to_bytes"),
Exprs: columnItems(),
},
},
}
}
modBuckets := func(expr tree.Expr) tree.Expr {
return &tree.FuncExpr{
Func: unresolvedFunc("mod"),
Exprs: tree.Exprs{
expr,
tree.NewDInt(tree.DInt(buckets)),
},
}
}
res := tree.Serialize(modBuckets(hashedColumnsExpr()))
return &res
}

// makeDeprecatedHashShardComputeExpr creates the serialized computed expression for a hash shard
// column based on the column names and the number of buckets. The expression will be
// of the form:
Expand Down
177 changes: 177 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/new_schema_changer
Original file line number Diff line number Diff line change
Expand Up @@ -679,3 +679,180 @@ CREATE TABLE public.trewrite (
CONSTRAINT new_primary_key PRIMARY KEY (k ASC),
FAMILY fam_0_k_ts (k, ts, c)
)

subtest create-index
statement ok
CREATE TABLE tIndex (
a INT PRIMARY KEY,
b INT,
FAMILY (a),
FAMILY (b)
)

statement ok
INSERT INTO tIndex VALUES (1,1)

user root

statement ok
CREATE INDEX foo ON tIndex (b)

statement error relation \"foo\" already exists
CREATE INDEX foo ON tIndex (a)

statement error column "c" does not exist
CREATE INDEX bar ON tIndex (c)

statement error index \"bar\" contains duplicate column \"b\"
CREATE INDEX bar ON tIndex (b, b);

statement error pgcode 0A000 only simple columns are supported as index elements
CREATE INDEX bar ON tIndex ((a+b))

statement error pgcode 0A000 only simple columns are supported as index elements
CREATE INDEX bar ON tIndex (abs(b))

statement error pgcode 0A000 only simple columns are supported as index elements
CREATE UNIQUE INDEX bar ON tIndex (abs(b))

statement error pgcode 0A000 only simple columns are supported as index elements
CREATE INVERTED INDEX bar ON tIndex ((ARRAY[a,b]))

statement error pgcode 0A000 only simple columns are supported as index elements
CREATE TABLE tIndx2 (a INT PRIMARY KEY, b INT, INDEX ((a+b)))

statement error pgcode 0A000 only simple columns are supported as index elements
CREATE TABLE tIndx2 (a INT PRIMARY KEY, b INT, INVERTED INDEX ((ARRAY[a,b])))

query TTBITTBB colnames
SHOW INDEXES FROM tIndex
----
table_name index_name non_unique seq_in_index column_name direction storing implicit
tindex foo true 1 b ASC false false
tindex foo true 2 a ASC false true
tindex primary false 1 a ASC false false
tindex primary false 2 b N/A true false

statement ok
INSERT INTO tIndex VALUES (2,1)

# FIXME: Disabled until rollback is supported
#statement error pgcode 23505 violates unique constraint "bar"
#CREATE UNIQUE INDEX bar ON tIndex (b)

query TTBITTBB colnames
SHOW INDEXES FROM tIndex
----
table_name index_name non_unique seq_in_index column_name direction storing implicit
tindex foo true 1 b ASC false false
tindex foo true 2 a ASC false true
tindex primary false 1 a ASC false false
tindex primary false 2 b N/A true false

# test for DESC index

statement ok
DROP TABLE tIndex

statement ok
CREATE TABLE tIndx (
a INT PRIMARY KEY,
b INT,
c INT
)

statement ok
INSERT INTO tIndx VALUES (1,1,1), (2,2,2)

statement ok
CREATE INDEX b_desc ON tIndx (b DESC)

statement ok
CREATE INDEX b_asc ON tIndx (b ASC, c DESC)

query TTBITTBB colnames
SHOW INDEXES FROM tIndx
----
table_name index_name non_unique seq_in_index column_name direction storing implicit
tindx b_asc true 1 b ASC false false
tindx b_asc true 2 c DESC false false
tindx b_asc true 3 a ASC false true
tindx b_desc true 1 b DESC false false
tindx b_desc true 2 a ASC false true
tindx primary false 1 a ASC false false
tindx primary false 2 b N/A true false
tindx primary false 3 c N/A true false

statement error pgcode 42P01 relation "foo" does not exist
CREATE INDEX fail ON foo (b DESC)

statement ok
CREATE VIEW vIndx AS SELECT a,b FROM tIndx

statement error pgcode 42809 "vindx" is not a table or materialized view
CREATE INDEX failview ON vIndx (b DESC)

statement ok
CREATE TABLE privs (a INT PRIMARY KEY, b INT)

user testuser

statement error user testuser does not have CREATE privilege on relation privs
CREATE INDEX foo ON privs (b)

user root

query TTBITTBB colnames
SHOW INDEXES FROM privs
----
table_name index_name non_unique seq_in_index column_name direction storing implicit
privs primary false 1 a ASC false false
privs primary false 2 b N/A true false

statement ok
GRANT CREATE ON privs TO testuser

user testuser

statement ok
CREATE INDEX foo ON privs (b)

query TTBITTBB colnames
SHOW INDEXES FROM privs
----
table_name index_name non_unique seq_in_index column_name direction storing implicit
privs foo true 1 b ASC false false
privs foo true 2 a ASC false true
privs primary false 1 a ASC false false
privs primary false 2 b N/A true false


user root

statement ok
SET experimental_enable_hash_sharded_indexes = true;
CREATE TABLE telemetry (
x INT PRIMARY KEY,
y INT,
z JSONB
)


# Test that creating an index on a column which is currently being dropped
# causes an error.
subtest create_index_on_dropping_column

statement ok
CREATE TABLE create_idx_drop_column (c0 INT PRIMARY KEY, c1 INT);

statement ok
begin; ALTER TABLE create_idx_drop_column DROP COLUMN c1;

statement error column "c1" does not exist
CREATE INDEX idx_create_idx_drop_column ON create_idx_drop_column (c1);

statement ok
ROLLBACK;

statement ok
DROP TABLE create_idx_drop_column;
2 changes: 2 additions & 0 deletions pkg/sql/schemachanger/scbuild/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ go_library(
"builder.go",
"database.go",
"dependencies.go",
"index.go",
"relation_common.go",
"schema.go",
"sequence.go",
Expand All @@ -17,6 +18,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//pkg/keys",
"//pkg/clusterversion",
"//pkg/settings/cluster",
"//pkg/sql/catalog",
"//pkg/sql/catalog/descpb",
Expand Down
21 changes: 21 additions & 0 deletions pkg/sql/schemachanger/scbuild/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -134,12 +134,33 @@ func (b *buildContext) build(ctx context.Context, n tree.Statement) (output scpb
b.dropDatabase(ctx, n)
case *tree.AlterTable:
b.alterTable(ctx, n)
case *tree.CreateIndex:
b.createIndex(ctx, n)
default:
return nil, &notImplementedError{n: n}
}
return b.output, nil
}

// checkIfNodeExists checks if an existing node is already there,
// in any direction.
func (b *buildContext) checkIfNewColumnExistsByName(tableID descpb.ID, name tree.Name) bool {
// Check if any existing node matches the new node we are
// trying to add.
for _, node := range b.output {
if node.Status != scpb.Status_ABSENT {
continue
}
column, ok := node.Element().(*scpb.Column)
if ok &&
column.TableID == tableID &&
column.Column.Name == string(name) {
return true
}
}
return false
}

// checkIfNodeExists checks if an existing node is already there,
// in any direction.
func (b *buildContext) checkIfNodeExists(
Expand Down
Loading

0 comments on commit f5762cb

Please sign in to comment.