From d0f72f975ad70e9ef496f8812c196e4a8c5595f2 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Tue, 9 Nov 2021 14:15:34 -0500 Subject: [PATCH] Revert "sql: add support for create index inside new schema changer" --- pkg/BUILD.bazel | 1 - pkg/ccl/backupccl/restore_job.go | 8 +- .../testdata/logic_test/new_schema_changer | 13 - .../protectedts/ptstorage/BUILD.bazel | 1 - .../protectedts/ptstorage/storage_test.go | 7 - pkg/sql/BUILD.bazel | 1 - pkg/sql/backfill.go | 99 +- pkg/sql/catalog/tabledesc/structured.go | 8 +- pkg/sql/check.go | 3 +- pkg/sql/conn_executor.go | 2 - pkg/sql/create_table.go | 50 +- .../testdata/logic_test/new_schema_changer | 196 - pkg/sql/schema_change_plan_node.go | 2 - pkg/sql/schema_changer_ccl.go | 60 - pkg/sql/schemachanger/scbuild/BUILD.bazel | 2 - pkg/sql/schemachanger/scbuild/builder.go | 21 - pkg/sql/schemachanger/scbuild/index.go | 382 -- pkg/sql/schemachanger/scbuild/table.go | 129 +- .../scbuild/testdata/alter_table | 360 +- .../scbuild/testdata/create_index | 143 - pkg/sql/schemachanger/scdeps/BUILD.bazel | 3 - pkg/sql/schemachanger/scdeps/exec_deps.go | 68 - pkg/sql/schemachanger/scdeps/run_deps.go | 12 +- .../scdeps/sctestdeps/BUILD.bazel | 3 - .../scdeps/sctestdeps/test_deps.go | 84 - .../scdeps/sctestdeps/test_state.go | 8 - pkg/sql/schemachanger/scexec/BUILD.bazel | 3 - pkg/sql/schemachanger/scexec/dependencies.go | 37 - .../schemachanger/scexec/exec_validation.go | 36 +- .../scexec/executor_external_test.go | 111 +- .../scexec/scmutationexec/BUILD.bazel | 2 - .../scexec/scmutationexec/scmutationexec.go | 180 +- pkg/sql/schemachanger/scgraph/BUILD.bazel | 14 +- pkg/sql/schemachanger/scgraph/graph.go | 87 +- pkg/sql/schemachanger/scgraph/graph_test.go | 156 - pkg/sql/schemachanger/scgraph/iteration.go | 2 +- pkg/sql/schemachanger/schemachanger_test.go | 3 +- pkg/sql/schemachanger/scjob/job.go | 2 - pkg/sql/schemachanger/scop/BUILD.bazel | 1 - pkg/sql/schemachanger/scop/mutation.go | 50 +- .../scop/mutation_visitor_generated.go | 18 - pkg/sql/schemachanger/scop/validation.go | 5 +- .../schemachanger/scpb/elements_generated.go | 3 +- pkg/sql/schemachanger/scpb/scpb.pb.go | 4950 +++++------------ pkg/sql/schemachanger/scpb/scpb.proto | 59 +- .../schemachanger/scplan/deprules/helpers.go | 15 + .../schemachanger/scplan/deprules/rules.go | 113 +- .../scplan/deprules/testdata/rules | 71 +- .../schemachanger/scplan/opgen/BUILD.bazel | 2 - .../scplan/opgen/opgen_partitioning.go | 35 - .../scplan/opgen/opgen_primary_index.go | 49 +- .../scplan/opgen/opgen_secondary_index.go | 83 - pkg/sql/schemachanger/scplan/plan.go | 139 +- pkg/sql/schemachanger/scplan/plan_test.go | 4 +- .../schemachanger/scplan/testdata/alter_table | 392 +- .../scplan/testdata/create_index | 120 - .../scplan/testdata/drop_database | 126 +- .../schemachanger/scplan/testdata/drop_schema | 97 +- .../scplan/testdata/drop_sequence | 18 - .../schemachanger/scplan/testdata/drop_table | 37 - .../schemachanger/scplan/testdata/drop_view | 66 +- pkg/sql/schemachanger/screl/attr.go | 12 +- pkg/sql/sqlutil/BUILD.bazel | 1 - pkg/sql/sqlutil/internal_executor.go | 24 - 64 files changed, 2256 insertions(+), 6533 deletions(-) delete mode 100644 pkg/ccl/logictestccl/testdata/logic_test/new_schema_changer delete mode 100644 pkg/sql/schema_changer_ccl.go delete mode 100644 pkg/sql/schemachanger/scbuild/index.go delete mode 100644 pkg/sql/schemachanger/scbuild/testdata/create_index delete mode 100644 pkg/sql/schemachanger/scgraph/graph_test.go delete mode 100644 pkg/sql/schemachanger/scplan/opgen/opgen_partitioning.go delete mode 100644 pkg/sql/schemachanger/scplan/testdata/create_index diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index d58f0b9542d1..2ca6052d9c9b 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -294,7 +294,6 @@ ALL_TESTS = [ "//pkg/sql/schemachanger/rel:rel_test", "//pkg/sql/schemachanger/scbuild:scbuild_test", "//pkg/sql/schemachanger/scexec:scexec_test", - "//pkg/sql/schemachanger/scgraph:scgraph_test", "//pkg/sql/schemachanger/scplan/deprules:deprules_test", "//pkg/sql/schemachanger/scplan:scplan_test", "//pkg/sql/schemachanger/screl:screl_test", diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 3ee20cf086dc..df66f85e4e89 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -44,8 +44,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" @@ -1800,7 +1798,7 @@ func revalidateIndexes( // We don't actually need the 'historical' read the way the schema change does // since our table is offline. - var runner sqlutil.HistoricalInternalExecTxnRunner = func(ctx context.Context, fn sqlutil.InternalExecFn) error { + var runner sql.HistoricalInternalExecTxnRunner = func(ctx context.Context, fn sql.InternalExecFn) error { return execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { ie := job.MakeSessionBoundInternalExecutor(ctx, sql.NewFakeSessionData(execCfg.SV())).(*sql.InternalExecutor) return fn(ctx, txn, ie) @@ -1828,7 +1826,7 @@ func revalidateIndexes( } } if len(forward) > 0 { - if err := sql.ValidateForwardIndexes(ctx, tableDesc.MakePublic(), forward, runner, false, true, sessiondata.InternalExecutorOverride{}); err != nil { + if err := sql.ValidateForwardIndexes(ctx, tableDesc.MakePublic(), forward, runner, false, true); err != nil { if invalid := (sql.InvalidIndexesError{}); errors.As(err, &invalid) { invalidIndexes[tableDesc.ID] = invalid.Indexes } else { @@ -1837,7 +1835,7 @@ func revalidateIndexes( } } if len(inverted) > 0 { - if err := sql.ValidateInvertedIndexes(ctx, execCfg.Codec, tableDesc.MakePublic(), inverted, runner, true, sessiondata.InternalExecutorOverride{}); err != nil { + if err := sql.ValidateInvertedIndexes(ctx, execCfg.Codec, tableDesc.MakePublic(), inverted, runner, true); err != nil { if invalid := (sql.InvalidIndexesError{}); errors.As(err, &invalid) { invalidIndexes[tableDesc.ID] = append(invalidIndexes[tableDesc.ID], invalid.Indexes...) } else { diff --git a/pkg/ccl/logictestccl/testdata/logic_test/new_schema_changer b/pkg/ccl/logictestccl/testdata/logic_test/new_schema_changer deleted file mode 100644 index a22364def64d..000000000000 --- a/pkg/ccl/logictestccl/testdata/logic_test/new_schema_changer +++ /dev/null @@ -1,13 +0,0 @@ -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) - ); - diff --git a/pkg/kv/kvserver/protectedts/ptstorage/BUILD.bazel b/pkg/kv/kvserver/protectedts/ptstorage/BUILD.bazel index e26d9d972a86..b9d9bd3833d0 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/BUILD.bazel +++ b/pkg/kv/kvserver/protectedts/ptstorage/BUILD.bazel @@ -49,7 +49,6 @@ go_test( "//pkg/security/securitytest", "//pkg/server", "//pkg/sql", - "//pkg/sql/catalog", "//pkg/sql/catalog/colinfo", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go index b404f5422262..b981d824571f 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go +++ b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go @@ -30,7 +30,6 @@ 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" @@ -802,9 +801,3 @@ 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") -} diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 9391d765c8b4..7d27653f23ff 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -169,7 +169,6 @@ go_library( "schema_change_cluster_setting.go", "schema_change_plan_node.go", "schema_changer.go", - "schema_changer_ccl.go", "schema_changer_metrics.go", "schema_changer_state.go", "scrub.go", diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index b8b305cbc3db..8b2cbf8b0b6e 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -41,10 +41,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -157,18 +155,26 @@ func (sc *SchemaChanger) makeFixedTimestampRunner(readAsOf hlc.Timestamp) histor return runner } +// InternalExecFn is the type of functions that operates using an internalExecutor. +type InternalExecFn func(ctx context.Context, txn *kv.Txn, ie *InternalExecutor) error + +// HistoricalInternalExecTxnRunner is like historicalTxnRunner except it only +// passes the fn the exported InternalExecutor instead of the whole unexported +// extendedEvalContenxt, so it can be implemented outside pkg/sql. +type HistoricalInternalExecTxnRunner func(ctx context.Context, fn InternalExecFn) error + // makeFixedTimestampRunner creates a HistoricalTxnRunner suitable for use by the helpers. func (sc *SchemaChanger) makeFixedTimestampInternalExecRunner( readAsOf hlc.Timestamp, -) sqlutil.HistoricalInternalExecTxnRunner { - runner := func(ctx context.Context, retryable sqlutil.InternalExecFn) error { +) HistoricalInternalExecTxnRunner { + runner := func(ctx context.Context, retryable InternalExecFn) error { return sc.fixedTimestampTxn(ctx, readAsOf, func( ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ) error { // We need to re-create the evalCtx since the txn may retry. ie := createSchemaChangeEvalCtx( ctx, sc.execCfg, readAsOf, sc.ieFactory, descriptors, - ).InternalExecutor.(sqlutil.InternalExecutor) + ).InternalExecutor.(*InternalExecutor) return retryable(ctx, txn, ie) }) } @@ -1414,12 +1420,12 @@ func (sc *SchemaChanger) validateIndexes(ctx context.Context) error { if len(forwardIndexes) > 0 { grp.GoCtx(func(ctx context.Context) error { - return ValidateForwardIndexes(ctx, tableDesc, forwardIndexes, runHistoricalTxn, true /* withFirstMutationPubic */, false /* gatherAllInvalid */, sessiondata.InternalExecutorOverride{}) + return ValidateForwardIndexes(ctx, tableDesc, forwardIndexes, runHistoricalTxn, true /* withFirstMutationPubic */, false /* gatherAllInvalid */) }) } if len(invertedIndexes) > 0 { grp.GoCtx(func(ctx context.Context) error { - return ValidateInvertedIndexes(ctx, sc.execCfg.Codec, tableDesc, invertedIndexes, runHistoricalTxn, false /* gatherAllInvalid */, sessiondata.InternalExecutorOverride{}) + return ValidateInvertedIndexes(ctx, sc.execCfg.Codec, tableDesc, invertedIndexes, runHistoricalTxn, false /* gatherAllInvalid */) }) } if err := grp.Wait(); err != nil { @@ -1450,9 +1456,8 @@ func ValidateInvertedIndexes( codec keys.SQLCodec, tableDesc catalog.TableDescriptor, indexes []catalog.Index, - runHistoricalTxn sqlutil.HistoricalInternalExecTxnRunner, + runHistoricalTxn HistoricalInternalExecTxnRunner, gatherAllInvalid bool, - execOverride sessiondata.InternalExecutorOverride, ) error { grp := ctxgroup.WithContext(ctx) invalid := make(chan descpb.IndexID, len(indexes)) @@ -1476,7 +1481,7 @@ func ValidateInvertedIndexes( span := tableDesc.IndexSpan(codec, idx.GetID()) key := span.Key endKey := span.EndKey - if err := runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, _ sqlutil.InternalExecutor) error { + if err := runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, _ *InternalExecutor) error { for { kvs, err := txn.Scan(ctx, key, endKey, 1000000) if err != nil { @@ -1540,7 +1545,7 @@ func ValidateInvertedIndexes( colNameOrExpr = fmt.Sprintf("%q", col.ColName()) } - if err := runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + if err := runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, ie *InternalExecutor) error { var stmt string geoConfig := idx.GetGeoConfig() if geoindex.IsEmptyConfig(&geoConfig) { @@ -1560,7 +1565,7 @@ func ValidateInvertedIndexes( stmt = fmt.Sprintf(`%s WHERE %s`, stmt, idx.GetPredicate()) } return ie.WithSyntheticDescriptors([]catalog.Descriptor{tableDesc}, func() error { - row, err := ie.QueryRowEx(ctx, "verify-inverted-idx-count", txn, execOverride, stmt) + row, err := ie.QueryRowEx(ctx, "verify-inverted-idx-count", txn, sessiondata.InternalExecutorOverride{}, stmt) if err != nil { return err } @@ -1593,65 +1598,6 @@ func ValidateInvertedIndexes( return nil } -type indexValidator struct { - db *kv.DB - codec keys.SQLCodec - executor *InternalExecutor -} - -// ValidateForwardIndexes checks that the indexes have entries for all the rows. -func (iv indexValidator) ValidateForwardIndexes( - ctx context.Context, - tableDesc catalog.TableDescriptor, - indexes []catalog.Index, - withFirstMutationPublic bool, - gatherAllInvalid bool, - override sessiondata.InternalExecutorOverride, -) error { - // Set up a new transaction with the current timestamp. - txnRunner := func(ctx context.Context, fn sqlutil.InternalExecFn) error { - validationTxn := iv.db.NewTxn(ctx, "validation") - err := validationTxn.SetFixedTimestamp(ctx, iv.db.Clock().Now()) - if err != nil { - return err - } - return fn(ctx, validationTxn, iv.executor) - } - return ValidateForwardIndexes(ctx, tableDesc, indexes, txnRunner, withFirstMutationPublic, gatherAllInvalid, override) -} - -// ValidateInvertedIndexes checks that the indexes have entries for all the rows. -func (iv indexValidator) ValidateInvertedIndexes( - ctx context.Context, - tableDesc catalog.TableDescriptor, - indexes []catalog.Index, - gatherAllInvalid bool, - override sessiondata.InternalExecutorOverride, -) error { - // Set up a new transaction with the current timestamp. - txnRunner := func(ctx context.Context, fn sqlutil.InternalExecFn) error { - validationTxn := iv.db.NewTxn(ctx, "validation") - err := validationTxn.SetFixedTimestamp(ctx, iv.db.Clock().Now()) - if err != nil { - return err - } - return fn(ctx, validationTxn, iv.executor) - } - return ValidateInvertedIndexes(ctx, iv.codec, tableDesc, indexes, txnRunner, gatherAllInvalid, override) -} - -// MakeIndexValidator creates a IndexValidator interface -// for the new schema changer. -func MakeIndexValidator( - db *kv.DB, codec keys.SQLCodec, executor *InternalExecutor, -) scexec.IndexValidator { - return indexValidator{ - db: db, - codec: codec, - executor: executor, - } -} - // ValidateForwardIndexes checks that the indexes have entries for all the rows. // // This operates over multiple goroutines concurrently and is thus not @@ -1668,10 +1614,9 @@ func ValidateForwardIndexes( ctx context.Context, tableDesc catalog.TableDescriptor, indexes []catalog.Index, - runHistoricalTxn sqlutil.HistoricalInternalExecTxnRunner, + runHistoricalTxn HistoricalInternalExecTxnRunner, withFirstMutationPublic bool, gatherAllInvalid bool, - execOverride sessiondata.InternalExecutorOverride, ) error { grp := ctxgroup.WithContext(ctx) @@ -1733,7 +1678,7 @@ func ValidateForwardIndexes( // Retrieve the row count in the index. var idxLen int64 - if err := runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + if err := runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, ie *InternalExecutor) error { query := fmt.Sprintf(`SELECT count(1) FROM [%d AS t]@[%d]`, desc.GetID(), idx.GetID()) // If the index is a partial index the predicate must be added // as a filter to the query to force scanning the index. @@ -1742,7 +1687,7 @@ func ValidateForwardIndexes( } return ie.WithSyntheticDescriptors([]catalog.Descriptor{desc}, func() error { - row, err := ie.QueryRowEx(ctx, "verify-idx-count", txn, execOverride, query) + row, err := ie.QueryRowEx(ctx, "verify-idx-count", txn, sessiondata.InternalExecutorOverride{}, query) if err != nil { return err } @@ -1825,7 +1770,7 @@ func ValidateForwardIndexes( } // Count the number of rows in the table. - if err := runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + if err := runHistoricalTxn(ctx, func(ctx context.Context, txn *kv.Txn, ie *InternalExecutor) error { var s strings.Builder for _, idx := range indexes { // For partial indexes, count the number of rows in the table @@ -1841,7 +1786,7 @@ func ValidateForwardIndexes( query := fmt.Sprintf(`SELECT count(1)%s FROM [%d AS t]@[%d]`, partialIndexCounts, desc.GetID(), desc.GetPrimaryIndexID()) return ie.WithSyntheticDescriptors([]catalog.Descriptor{desc}, func() error { - cnt, err := ie.QueryRowEx(ctx, "VERIFY INDEX", txn, execOverride, query) + cnt, err := ie.QueryRowEx(ctx, "VERIFY INDEX", txn, sessiondata.InternalExecutorOverride{}, query) if err != nil { return err } diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index cc4643c9e6c9..79854c3347a9 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -141,7 +141,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: // @@ -154,7 +154,9 @@ 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, idx *descpb.IndexDescriptor) (string, error) { +func buildIndexName(tableDesc *Mutable, index catalog.Index) (string, error) { + idx := index.IndexDesc() + // 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) @@ -682,7 +684,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.IndexDesc()) + name, err := buildIndexName(desc, idx) if err != nil { return err } diff --git a/pkg/sql/check.go b/pkg/sql/check.go index cf75d9aa986e..48022f45c76d 100644 --- a/pkg/sql/check.go +++ b/pkg/sql/check.go @@ -26,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" @@ -371,7 +370,7 @@ func validateUniqueConstraint( constraintName string, columnIDs []descpb.ColumnID, pred string, - ie sqlutil.InternalExecutor, + ie *InternalExecutor, txn *kv.Txn, ) error { query, colNames, err := duplicateRowQuery( diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 19b65ebc0783..f632c492da4c 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -2938,8 +2938,6 @@ func (ex *connExecutor) runPreCommitStages(ctx context.Context) error { &ex.extraTxnState.descCollection, ex.server.cfg.JobRegistry, ex.server.cfg.IndexBackfiller, - MakeIndexValidator(ex.planner.Txn().DB(), ex.server.cfg.Codec, ex.planner.execCfg.InternalExecutor), - MakeCCLCallbacks(ex.server.cfg.Settings, ex.planner.EvalContext()), ex.server.cfg.NewSchemaChangerTestingKnobs, scs.stmts, scop.PreCommitPhase, diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index e843c60dcb0f..f566b2a90957 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -40,7 +40,6 @@ 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" @@ -2485,13 +2484,60 @@ func makeShardColumnDesc( } col.Name = tabledesc.GetShardColumnName(colNames, int32(buckets)) if useDatumsToBytes { - col.ComputeExpr = scbuild.MakeHashShardComputeExpr(colNames, buckets) + col.ComputeExpr = 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: diff --git a/pkg/sql/logictest/testdata/logic_test/new_schema_changer b/pkg/sql/logictest/testdata/logic_test/new_schema_changer index 595b7cde0ae9..4706e29f414a 100644 --- a/pkg/sql/logictest/testdata/logic_test/new_schema_changer +++ b/pkg/sql/logictest/testdata/logic_test/new_schema_changer @@ -679,199 +679,3 @@ 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 ok -CREATE INDEX bar ON tIndex ((a+b)) - -statement ok -CREATE INDEX bar2 ON tIndex (abs(b)) - -statement ok -CREATE UNIQUE INDEX bar3 ON tIndex (abs(b)) - -statement ok -CREATE INVERTED INDEX bar4 ON tIndex ((ARRAY[a,b])) - -statement ok -CREATE TABLE tIndx2 (a INT PRIMARY KEY, b INT, INDEX ((a+b))) - -statement ok -CREATE TABLE tIndx3 (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 bar true 1 crdb_internal_idx_expr ASC false false -tindex bar true 2 a ASC false true -tindex bar2 true 1 crdb_internal_idx_expr_1 ASC false false -tindex bar2 true 2 a ASC false true -tindex bar3 false 1 crdb_internal_idx_expr_2 ASC false false -tindex bar3 false 2 a ASC false true -tindex bar4 true 1 crdb_internal_idx_expr_3 ASC false false -tindex bar4 true 2 a ASC false true -tindex foo true 1 b ASC false false -tindex foo true 2 a ASC false true -tindex tindex_pkey false 1 a ASC false false -tindex tindex_pkey false 2 b N/A true false - -statement error duplicate key value violates unique constraint "bar3" -INSERT INTO tIndex VALUES (2,1) - -statement ok -INSERT INTO tIndex VALUES (20000,10000) - -# 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 bar true 1 crdb_internal_idx_expr ASC false false -tindex bar true 2 a ASC false true -tindex bar2 true 1 crdb_internal_idx_expr_1 ASC false false -tindex bar2 true 2 a ASC false true -tindex bar3 false 1 crdb_internal_idx_expr_2 ASC false false -tindex bar3 false 2 a ASC false true -tindex bar4 true 1 crdb_internal_idx_expr_3 ASC false false -tindex bar4 true 2 a ASC false true -tindex foo true 1 b ASC false false -tindex foo true 2 a ASC false true -tindex tindex_pkey false 1 a ASC false false -tindex tindex_pkey 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 tindx_pkey false 1 a ASC false false -tindx tindx_pkey false 2 b N/A true false -tindx tindx_pkey 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 privs_pkey false 1 a ASC false false -privs privs_pkey 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 privs_pkey false 1 a ASC false false -privs privs_pkey 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; diff --git a/pkg/sql/schema_change_plan_node.go b/pkg/sql/schema_change_plan_node.go index f526f42e98b0..bbc09d6cb2c7 100644 --- a/pkg/sql/schema_change_plan_node.go +++ b/pkg/sql/schema_change_plan_node.go @@ -135,8 +135,6 @@ func (s *schemaChangePlanNode) startExec(params runParams) error { p.Descriptors(), p.ExecCfg().JobRegistry, p.ExecCfg().IndexBackfiller, - MakeIndexValidator(p.Txn().DB(), p.EvalContext().Codec, p.ExecCfg().InternalExecutor), - MakeCCLCallbacks(p.ExecCfg().Settings, p.EvalContext()), p.ExecCfg().NewSchemaChangerTestingKnobs, scs.stmts, scop.StatementPhase, diff --git a/pkg/sql/schema_changer_ccl.go b/pkg/sql/schema_changer_ccl.go deleted file mode 100644 index 8a7366ca509a..000000000000 --- a/pkg/sql/schema_changer_ccl.go +++ /dev/null @@ -1,60 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package sql - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" -) - -type schemaChangerCCLCallbacks struct { - settings *cluster.Settings - evalContext *tree.EvalContext -} - -func (s *schemaChangerCCLCallbacks) CreatePartitioning( - ctx context.Context, - tableDesc *tabledesc.Mutable, - indexDesc descpb.IndexDescriptor, - partBy *tree.PartitionBy, - allowedNewColumnNames []tree.Name, - allowImplicitPartitioning bool, -) (newImplicitCols []catalog.Column, newPartitioning descpb.PartitioningDescriptor, err error) { - if s.settings == nil || - s.evalContext == nil { - panic("unimplemented when settings or evalContext are omitted") - } - return CreatePartitioningCCL(ctx, - s.settings, - s.evalContext, - tableDesc, - indexDesc, - partBy, - allowedNewColumnNames, - allowImplicitPartitioning) -} - -// MakeCCLCallbacks makes callbacks needed for the new schema -// changer. -func MakeCCLCallbacks( - settings *cluster.Settings, evalContext *tree.EvalContext, -) scexec.Partitioner { - return &schemaChangerCCLCallbacks{ - settings: settings, - evalContext: evalContext, - } -} diff --git a/pkg/sql/schemachanger/scbuild/BUILD.bazel b/pkg/sql/schemachanger/scbuild/BUILD.bazel index c02b28e359a0..9c787bf8bb8c 100644 --- a/pkg/sql/schemachanger/scbuild/BUILD.bazel +++ b/pkg/sql/schemachanger/scbuild/BUILD.bazel @@ -6,7 +6,6 @@ go_library( "builder.go", "database.go", "dependencies.go", - "index.go", "relation_common.go", "schema.go", "sequence.go", @@ -17,7 +16,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild", visibility = ["//visibility:public"], deps = [ - "//pkg/clusterversion", "//pkg/keys", "//pkg/settings/cluster", "//pkg/sql/catalog", diff --git a/pkg/sql/schemachanger/scbuild/builder.go b/pkg/sql/schemachanger/scbuild/builder.go index 82d3e182ff22..b31d69b706e9 100644 --- a/pkg/sql/schemachanger/scbuild/builder.go +++ b/pkg/sql/schemachanger/scbuild/builder.go @@ -134,33 +134,12 @@ 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, ¬ImplementedError{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( diff --git a/pkg/sql/schemachanger/scbuild/index.go b/pkg/sql/schemachanger/scbuild/index.go deleted file mode 100644 index d3441723a475..000000000000 --- a/pkg/sql/schemachanger/scbuild/index.go +++ /dev/null @@ -1,382 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package scbuild - -import ( - "context" - - "github.com/cockroachdb/cockroach/pkg/clusterversion" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" - "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" - "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/errors" -) - -// makeShardColumnDesc returns a new column descriptor for a hidden computed shard column -// based on all the `colNames`. -func makeShardColumnDesc(colNames []string, buckets int) (*descpb.ColumnDescriptor, error) { - col := &descpb.ColumnDescriptor{ - Hidden: true, - Nullable: false, - Type: types.Int4, - } - col.Name = tabledesc.GetShardColumnName(colNames, int32(buckets)) - col.ComputeExpr = MakeHashShardComputeExpr(colNames, buckets) - return col, nil -} - -// MakeHashShardComputeExpr 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 -} - -// maybeCreateAndAddShardCol adds a new hidden computed shard column (or its mutation) to -// `desc`, if one doesn't already exist for the given index column set and number of shard -// buckets. -func (b *buildContext) maybeCreateAndAddShardCol( - shardBuckets int, desc catalog.TableDescriptor, colNames []string, isNewTable bool, -) (created bool, err error) { - shardColDesc, err := makeShardColumnDesc(colNames, shardBuckets) - if err != nil { - return false, err - } - existingShardCol, err := desc.FindColumnWithName(tree.Name(shardColDesc.Name)) - if err == nil && !existingShardCol.Dropped() { - // TODO(ajwerner): In what ways is existingShardCol allowed to differ from - // the newly made shardCol? Should there be some validation of - // existingShardCol? - if !existingShardCol.IsHidden() { - // The user managed to reverse-engineer our crazy shard column name, so - // we'll return an error here rather than try to be tricky. - return false, pgerror.Newf(pgcode.DuplicateColumn, - "column %s already specified; can't be used for sharding", shardColDesc.Name) - } - return false, nil - } - columnIsUndefined := sqlerrors.IsUndefinedColumnError(err) - if err != nil && !columnIsUndefined { - return false, err - } - if columnIsUndefined || existingShardCol.Dropped() { - if isNewTable { - panic(false) - //desc.AddColumn(shardColDesc) - } else { - shardColDesc.ID = b.nextColumnID(desc) - column := &scpb.Column{ - Column: *shardColDesc, - TableID: desc.GetID(), - FamilyID: desc.GetFamilies()[0].ID, - FamilyName: desc.GetFamilies()[0].Name, - } - b.addNode(scpb.Target_ADD, column) - } - if !shardColDesc.Virtual { - // Replace the primary index - oldPrimaryIndex := primaryIndexElemFromDescriptor(desc.GetPrimaryIndex().IndexDesc(), desc) - newPrimaryIndex := primaryIndexElemFromDescriptor(desc.GetPrimaryIndex().IndexDesc(), desc) - newPrimaryIndex.IndexId = b.nextIndexID(desc) - newPrimaryIndex.IndexName = tabledesc.GenerateUniqueName( - "new_primary_key", - func(name string) bool { - // TODO (lucy): Also check the new indexes specified in the targets. - _, err := desc.FindIndexWithName(name) - return err == nil - }, - ) - newPrimaryIndex.StoringColumnIDs = append(newPrimaryIndex.StoringColumnIDs, shardColDesc.ID) - b.addNode(scpb.Target_DROP, oldPrimaryIndex) - b.addNode(scpb.Target_ADD, newPrimaryIndex) - } - created = true - } - return created, nil -} - -func (b *buildContext) createIndex(ctx context.Context, n *tree.CreateIndex) { - // Look up the table first. - _, table := b.CatalogReader().MayResolveTable(ctx, *n.Table.ToUnresolvedObjectName()) - if table == nil { - panic(sqlerrors.NewUndefinedRelationError(n.Table.ToUnresolvedObjectName())) - } - // Detect if the index already exists. - foundIndex, err := table.FindIndexWithName(n.Name.String()) - if err == nil { - if foundIndex.Dropped() { - panic(pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, - "index %q being dropped, try again later", n.Name.String())) - } - if n.IfNotExists { - return - } - panic(sqlerrors.NewRelationAlreadyExistsError(n.Name.String())) - } - - if table.IsView() && !table.MaterializedView() { - panic(pgerror.Newf(pgcode.WrongObjectType, "%q is not a table or materialized view", &n.Table)) - } - - if table.MaterializedView() { - if n.Sharded != nil { - panic(pgerror.New(pgcode.InvalidObjectDefinition, - "cannot create hash sharded index on materialized view")) - } - } - - if n.PartitionByIndex != nil && table.GetLocalityConfig() != nil { - panic(pgerror.New( - pgcode.FeatureNotSupported, - "cannot define PARTITION BY on a new INDEX in a multi-region database", - )) - } - - // Setup an secondary index node. - secondaryIndex := &scpb.SecondaryIndex{TableID: table.GetID(), - IndexName: n.Name.Normalize(), - Unique: n.Unique, - KeyColumnIDs: make([]descpb.ColumnID, 0, len(n.Columns)), - StoringColumnIDs: make([]descpb.ColumnID, 0, len(n.Storing)), - Inverted: n.Inverted, - Concurrently: n.Concurrently, - KeySuffixColumnIDs: nil, - ShardedDescriptor: nil, - } - colNames := make([]string, 0, len(n.Columns)) - // Setup the column ID. - for _, columnNode := range n.Columns { - // If the column was just added the new schema changer is not supported. - if b.checkIfNewColumnExistsByName(table.GetID(), columnNode.Column) { - panic(¬ImplementedError{ - detail: "column was added in the current transaction.", - n: n, - }) - } - if columnNode.Expr != nil { - if !b.ClusterSettings().Version.IsActive(ctx, clusterversion.ExpressionIndexes) { - panic(pgerror.Newf(pgcode.FeatureNotSupported, - "version %v must be finalized to use expression indexes", - clusterversion.ExpressionIndexes)) - } - // TODO(fqazi): We need to deal with columns added in the same - // transaction here as well. - _, typ, _, err := schemaexpr.DequalifyAndValidateExpr( - ctx, - table, - columnNode.Expr, - types.Any, - "index expression", - semaCtx(b), - tree.VolatilityImmutable, - &n.Table, - ) - if err != nil { - panic(err) - } - // Create a new virtual column and add it to the table - // descriptor. - colName := tabledesc.GenerateUniqueName("crdb_internal_idx_expr", func(name string) bool { - _, err := table.FindColumnWithName(tree.Name(name)) - return err == nil - }) - addCol := &tree.AlterTableAddColumn{ - ColumnDef: &tree.ColumnTableDef{ - Name: tree.Name(colName), - Type: typ, - Hidden: true, - }, - } - addCol.ColumnDef.Computed.Computed = true - addCol.ColumnDef.Computed.Expr = columnNode.Expr - addCol.ColumnDef.Computed.Virtual = true - addCol.ColumnDef.Nullable.Nullability = tree.Null - - // Add a new column element - b.alterTableAddColumn(ctx, table, addCol, &n.Table) - var addColumn *scpb.Column = nil - for _, node := range b.output { - if node.Target.Column != nil && - node.Target.Column.TableID == table.GetID() && - node.Target.Column.Column.Name == colName { - addColumn = node.Target.Column - } - } - - // Set up the index based on the new column. - colNames = append(colNames, colName) - secondaryIndex.KeyColumnIDs = append(secondaryIndex.KeyColumnIDs, addColumn.Column.ID) - } - if columnNode.Expr == nil { - column, err := table.FindColumnWithName(columnNode.Column) - if err != nil { - panic(err) - } - colNames = append(colNames, column.GetName()) - secondaryIndex.KeyColumnIDs = append(secondaryIndex.KeyColumnIDs, column.GetID()) - } - // Convert the key column directions. - switch columnNode.Direction { - case tree.Ascending, tree.DefaultDirection: - secondaryIndex.KeyColumnDirections = append(secondaryIndex.KeyColumnDirections, scpb.SecondaryIndex_ASC) - case tree.Descending: - secondaryIndex.KeyColumnDirections = append(secondaryIndex.KeyColumnDirections, scpb.SecondaryIndex_DESC) - default: - panic(errors.AssertionFailedf("Unknown direction type %s", columnNode.Direction)) - } - } - // Setup the storing columns. - for _, storingNode := range n.Storing { - column, err := table.FindColumnWithName(storingNode) - if err != nil { - panic(err) - } - secondaryIndex.StoringColumnIDs = append(secondaryIndex.StoringColumnIDs, column.GetID()) - } - if n.Sharded != nil { - if n.PartitionByIndex.ContainsPartitions() { - panic(pgerror.New(pgcode.FeatureNotSupported, "sharded indexes don't support partitioning")) - } - if table.IsLocalityRegionalByRow() { - panic(pgerror.New(pgcode.FeatureNotSupported, "hash sharded indexes are not compatible with REGIONAL BY ROW tables")) - } - buckets, err := tabledesc.EvalShardBucketCount(ctx, semaCtx(b), evalCtx(ctx, b), n.Sharded.ShardBuckets) - if err != nil { - panic(err) - } - shardColName := tabledesc.GetShardColumnName(colNames, buckets) - _, err = b.maybeCreateAndAddShardCol(int(buckets), table, colNames, false) - if err != nil { - panic(err) - } - secondaryIndex.ShardedDescriptor = &descpb.ShardedDescriptor{ - IsSharded: true, - Name: shardColName, - ShardBuckets: buckets, - ColumnNames: colNames, - } - } - // Assign the ID here, since we may have added columns - // and made a new primary key above. - secondaryIndex.IndexId = b.nextIndexID(table) - // Convert partitioning information for the execution - // side of things. - if n.PartitionByIndex.ContainsPartitions() { - listPartitions := make([]*scpb.ListPartition, 0, len(n.PartitionByIndex.List)) - for _, partition := range n.PartitionByIndex.List { - exprs := make([]string, 0, len(partition.Exprs)) - for _, expr := range partition.Exprs { - exprs = append(exprs, expr.String()) - } - listPartition := &scpb.ListPartition{ - Name: partition.Name.String(), - Expr: exprs, - } - listPartitions = append(listPartitions, listPartition) - } - rangePartitions := make([]*scpb.RangePartitions, 0, len(n.PartitionByIndex.Range)) - for _, partition := range n.PartitionByIndex.Range { - toExpr := make([]string, 0, len(partition.To)) - for _, expr := range partition.To { - fmtCtx := tree.NewFmtCtx(tree.FmtSimple) - fmtCtx.FormatNode(expr) - toExpr = append(toExpr, fmtCtx.String()) - } - fromExpr := make([]string, 0, len(partition.From)) - for _, expr := range partition.From { - fmtCtx := tree.NewFmtCtx(tree.FmtSimple) - fmtCtx.FormatNode(expr) - fromExpr = append(fromExpr, fmtCtx.String()) - } - rangePartition := &scpb.RangePartitions{ - Name: partition.Name.String(), - To: toExpr, - From: fromExpr, - } - rangePartitions = append(rangePartitions, rangePartition) - } - fields := make([]string, 0, len(n.PartitionByIndex.Fields)) - for _, field := range n.PartitionByIndex.Fields { - fields = append(fields, field.String()) - } - partitioning := &scpb.Partitioning{ - TableID: table.GetID(), - IndexId: secondaryIndex.IndexId, - Fields: fields, - ListPartitions: listPartitions, - RangePartitions: rangePartitions, - } - b.addNode(scpb.Target_ADD, - partitioning) - } - - // KeySuffixColumnIDs is only populated for indexes using the secondary - // index encoding. It is the set difference of the primary key minus the - // index's key. - colIDs := catalog.MakeTableColSet(secondaryIndex.KeyColumnIDs...) - var extraColumnIDs []descpb.ColumnID - for _, primaryColID := range table.GetPrimaryIndex().IndexDesc().KeyColumnIDs { - if !colIDs.Contains(primaryColID) { - extraColumnIDs = append(extraColumnIDs, primaryColID) - colIDs.Add(primaryColID) - } - } - secondaryIndex.KeySuffixColumnIDs = extraColumnIDs - - b.addNode(scpb.Target_ADD, - secondaryIndex) -} diff --git a/pkg/sql/schemachanger/scbuild/table.go b/pkg/sql/schemachanger/scbuild/table.go index 7946f482d037..48942e3d6c4a 100644 --- a/pkg/sql/schemachanger/scbuild/table.go +++ b/pkg/sql/schemachanger/scbuild/table.go @@ -69,68 +69,6 @@ func (b *buildContext) alterTableCmd( } } -func primaryIndexElemFromDescriptor( - indexDesc *descpb.IndexDescriptor, tableDesc catalog.TableDescriptor, -) *scpb.PrimaryIndex { - if len(indexDesc.Partitioning.Range) > 0 || - len(indexDesc.Partitioning.List) > 0 { - panic(notImplementedError{n: nil, detail: "partitioning on new indexes is not supported."}) - } - keyColumnDirs := make([]scpb.PrimaryIndex_Direction, 0, len(indexDesc.KeyColumnDirections)) - for _, dir := range indexDesc.KeyColumnDirections { - switch dir { - case descpb.IndexDescriptor_DESC: - keyColumnDirs = append(keyColumnDirs, scpb.PrimaryIndex_DESC) - case descpb.IndexDescriptor_ASC: - keyColumnDirs = append(keyColumnDirs, scpb.PrimaryIndex_ASC) - default: - panic(errors.AssertionFailedf("Unknown direction type %s", dir)) - } - } - return &scpb.PrimaryIndex{TableID: tableDesc.GetID(), - IndexId: indexDesc.ID, - IndexName: indexDesc.Name, - Unique: indexDesc.Unique, - KeyColumnIDs: indexDesc.KeyColumnIDs, - KeyColumnDirections: keyColumnDirs, - KeySuffixColumnIDs: indexDesc.KeySuffixColumnIDs, - StoringColumnIDs: indexDesc.StoreColumnIDs, - CompositeColumnIDs: indexDesc.CompositeColumnIDs, - Inverted: indexDesc.Type == descpb.IndexDescriptor_INVERTED, - ShardedDescriptor: &indexDesc.Sharded} -} - -func secondaryIndexElemFromDescriptor( - indexDesc *descpb.IndexDescriptor, tableDesc catalog.TableDescriptor, -) *scpb.SecondaryIndex { - if len(indexDesc.Partitioning.Range) > 0 || - len(indexDesc.Partitioning.List) > 0 { - panic(notImplementedError{n: nil, detail: "partitioning on new indexes is not supported."}) - } - keyColumnDirs := make([]scpb.SecondaryIndex_Direction, 0, len(indexDesc.KeyColumnDirections)) - for _, dir := range indexDesc.KeyColumnDirections { - switch dir { - case descpb.IndexDescriptor_DESC: - keyColumnDirs = append(keyColumnDirs, scpb.SecondaryIndex_DESC) - case descpb.IndexDescriptor_ASC: - keyColumnDirs = append(keyColumnDirs, scpb.SecondaryIndex_ASC) - default: - panic(errors.AssertionFailedf("Unknown direction type %s", dir)) - } - } - return &scpb.SecondaryIndex{TableID: tableDesc.GetID(), - IndexId: indexDesc.ID, - IndexName: indexDesc.Name, - Unique: indexDesc.Unique, - KeyColumnIDs: indexDesc.KeyColumnIDs, - KeyColumnDirections: keyColumnDirs, - KeySuffixColumnIDs: indexDesc.KeySuffixColumnIDs, - StoringColumnIDs: indexDesc.StoreColumnIDs, - CompositeColumnIDs: indexDesc.CompositeColumnIDs, - Inverted: indexDesc.Type == descpb.IndexDescriptor_INVERTED, - ShardedDescriptor: &indexDesc.Sharded} -} - func (b *buildContext) alterTableAddColumn( ctx context.Context, table catalog.TableDescriptor, @@ -194,7 +132,7 @@ func (b *buildContext) alterTableAddColumn( familyID = b.findOrAddColumnFamily( table, familyName, d.Family.Create, d.Family.IfNotExists, ) - } else if !d.IsVirtual() { // FIXME: Compute columns should not have families? + } else { // TODO(ajwerner,lucy-zhang): Deal with adding the first column to the // table. fam := table.GetFamilies()[0] @@ -241,15 +179,16 @@ func (b *buildContext) alterTableAddColumn( FamilyID: familyID, FamilyName: familyName, }) - // Computed columns do not exist inside the primary index, - if !col.Virtual { - b.addOrUpdatePrimaryIndexTargetsForAddColumn(table, colID, col.Name) - if idx := cdd.PrimaryKeyOrUniqueIndexDescriptor; idx != nil { - idxID := b.nextIndexID(table) - idx.ID = idxID - secondaryIndex := secondaryIndexElemFromDescriptor(idx, table) - b.addNode(scpb.Target_ADD, secondaryIndex) - } + newPrimaryIdxID := b.addOrUpdatePrimaryIndexTargetsForAddColumn(table, colID, col.Name) + + if idx := cdd.PrimaryKeyOrUniqueIndexDescriptor; idx != nil { + idxID := b.nextIndexID(table) + idx.ID = idxID + b.addNode(scpb.Target_ADD, &scpb.SecondaryIndex{ + TableID: table.GetID(), + Index: *idx, + PrimaryIndex: newPrimaryIdxID, + }) } } @@ -456,8 +395,9 @@ func (b *buildContext) addOrUpdatePrimaryIndexTargetsForAddColumn( if t, ok := n.Element().(*scpb.PrimaryIndex); ok && b.output[i].Target.Direction == scpb.Target_ADD && t.TableID == table.GetID() { - t.StoringColumnIDs = append(t.StoringColumnIDs, colID) - return t.IndexId + t.Index.StoreColumnIDs = append(t.Index.StoreColumnIDs, colID) + t.Index.StoreColumnNames = append(t.Index.StoreColumnNames, colName) + return t.Index.ID } } @@ -481,10 +421,18 @@ func (b *buildContext) addOrUpdatePrimaryIndexTargetsForAddColumn( newIdx.StoreColumnNames = append(newIdx.StoreColumnNames, colName) } - b.addNode(scpb.Target_ADD, primaryIndexElemFromDescriptor(&newIdx, table)) + b.addNode(scpb.Target_ADD, &scpb.PrimaryIndex{ + TableID: table.GetID(), + Index: newIdx, + OtherPrimaryIndexID: table.GetPrimaryIndexID(), + }) // Drop the existing primary index. - b.addNode(scpb.Target_DROP, primaryIndexElemFromDescriptor(table.GetPrimaryIndex().IndexDesc(), table)) + b.addNode(scpb.Target_DROP, &scpb.PrimaryIndex{ + TableID: table.GetID(), + Index: table.GetPrimaryIndex().IndexDescDeepCopy(), + OtherPrimaryIndexID: newIdx.ID, + }) return idxID } @@ -499,10 +447,11 @@ func (b *buildContext) addOrUpdatePrimaryIndexTargetsForDropColumn( if t, ok := n.Element().(*scpb.PrimaryIndex); ok && n.Target.Direction == scpb.Target_ADD && t.TableID == table.GetID() { - for j := range t.StoringColumnIDs { - if t.StoringColumnIDs[j] == colID { - t.StoringColumnIDs = append(t.StoringColumnIDs[:j], t.StoringColumnIDs[j+1:]...) - return t.IndexId + for j := range t.Index.StoreColumnIDs { + if t.Index.StoreColumnIDs[j] == colID { + t.Index.StoreColumnIDs = append(t.Index.StoreColumnIDs[:j], t.Index.StoreColumnIDs[j+1:]...) + t.Index.StoreColumnNames = append(t.Index.StoreColumnNames[:j], t.Index.StoreColumnNames[j+1:]...) + return t.Index.ID } panic("index not found") @@ -538,10 +487,18 @@ func (b *buildContext) addOrUpdatePrimaryIndexTargetsForDropColumn( } } - b.addNode(scpb.Target_ADD, primaryIndexElemFromDescriptor(&newIdx, table)) + b.addNode(scpb.Target_ADD, &scpb.PrimaryIndex{ + TableID: table.GetID(), + Index: newIdx, + OtherPrimaryIndexID: table.GetPrimaryIndexID(), + }) // Drop the existing primary index. - b.addNode(scpb.Target_DROP, primaryIndexElemFromDescriptor(table.GetPrimaryIndex().IndexDesc(), table)) + b.addNode(scpb.Target_DROP, &scpb.PrimaryIndex{ + TableID: table.GetID(), + Index: table.GetPrimaryIndex().IndexDescDeepCopy(), + OtherPrimaryIndexID: idxID, + }) return idxID } @@ -577,12 +534,12 @@ func (b *buildContext) nextIndexID(table catalog.TableDescriptor) descpb.IndexID continue } if ai, ok := n.Element().(*scpb.SecondaryIndex); ok { - if ai.IndexId > maxIdxID { - maxIdxID = ai.IndexId + if ai.Index.ID > maxIdxID { + maxIdxID = ai.Index.ID } } else if ai, ok := n.Element().(*scpb.PrimaryIndex); ok { - if ai.IndexId > maxIdxID { - maxIdxID = ai.IndexId + if ai.Index.ID > maxIdxID { + maxIdxID = ai.Index.ID } } } diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table b/pkg/sql/schemachanger/scbuild/testdata/alter_table index 41495d2731d3..34e25113ee30 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table @@ -21,29 +21,51 @@ ALTER TABLE defaultdb.foo ADD COLUMN j INT - ADD PrimaryIndex:{DescID: 52, Name: new_primary_key, IndexID: 2} state: ABSENT details: - indexId: 2 - indexName: new_primary_key - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} - storingColumnIds: - - 2 + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 1 + keyColumnNames: + - i + name: new_primary_key + partitioning: {} + sharded: {} + storeColumnIds: + - 2 + storeColumnNames: + - j + unique: true + version: 4 + otherPrimaryIndexId: 1 tableId: 52 - unique: true - DROP PrimaryIndex:{DescID: 52, Name: foo_pkey, IndexID: 1} state: PUBLIC details: - indexId: 1 - indexName: foo_pkey - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 1 + keyColumnNames: + - i + name: foo_pkey + partitioning: {} + sharded: {} + unique: true + version: 4 + otherPrimaryIndexId: 2 tableId: 52 - unique: true build ALTER TABLE defaultdb.foo ADD COLUMN j INT DEFAULT 123 @@ -65,29 +87,51 @@ ALTER TABLE defaultdb.foo ADD COLUMN j INT DEFAULT 123 - ADD PrimaryIndex:{DescID: 52, Name: new_primary_key, IndexID: 2} state: ABSENT details: - indexId: 2 - indexName: new_primary_key - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} - storingColumnIds: - - 2 + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 1 + keyColumnNames: + - i + name: new_primary_key + partitioning: {} + sharded: {} + storeColumnIds: + - 2 + storeColumnNames: + - j + unique: true + version: 4 + otherPrimaryIndexId: 1 tableId: 52 - unique: true - DROP PrimaryIndex:{DescID: 52, Name: foo_pkey, IndexID: 1} state: PUBLIC details: - indexId: 1 - indexName: foo_pkey - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 1 + keyColumnNames: + - i + name: foo_pkey + partitioning: {} + sharded: {} + unique: true + version: 4 + otherPrimaryIndexId: 2 tableId: 52 - unique: true build ALTER TABLE defaultdb.foo ADD COLUMN j INT DEFAULT 123; @@ -124,30 +168,53 @@ ALTER TABLE defaultdb.foo ADD COLUMN k INT DEFAULT 456; - ADD PrimaryIndex:{DescID: 52, Name: new_primary_key, IndexID: 2} state: ABSENT details: - indexId: 2 - indexName: new_primary_key - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} - storingColumnIds: - - 2 - - 3 + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 1 + keyColumnNames: + - i + name: new_primary_key + partitioning: {} + sharded: {} + storeColumnIds: + - 2 + - 3 + storeColumnNames: + - j + - k + unique: true + version: 4 + otherPrimaryIndexId: 1 tableId: 52 - unique: true - DROP PrimaryIndex:{DescID: 52, Name: foo_pkey, IndexID: 1} state: PUBLIC details: - indexId: 1 - indexName: foo_pkey - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 1 + keyColumnNames: + - i + name: foo_pkey + partitioning: {} + sharded: {} + unique: true + version: 4 + otherPrimaryIndexId: 2 tableId: 52 - unique: true build ALTER TABLE defaultdb.foo ADD COLUMN a INT AS (i+1) STORED @@ -169,29 +236,51 @@ ALTER TABLE defaultdb.foo ADD COLUMN a INT AS (i+1) STORED - ADD PrimaryIndex:{DescID: 52, Name: new_primary_key, IndexID: 2} state: ABSENT details: - indexId: 2 - indexName: new_primary_key - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} - storingColumnIds: - - 2 + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 1 + keyColumnNames: + - i + name: new_primary_key + partitioning: {} + sharded: {} + storeColumnIds: + - 2 + storeColumnNames: + - a + unique: true + version: 4 + otherPrimaryIndexId: 1 tableId: 52 - unique: true - DROP PrimaryIndex:{DescID: 52, Name: foo_pkey, IndexID: 1} state: PUBLIC details: - indexId: 1 - indexName: foo_pkey - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 1 + keyColumnNames: + - i + name: foo_pkey + partitioning: {} + sharded: {} + unique: true + version: 4 + otherPrimaryIndexId: 2 tableId: 52 - unique: true create-table CREATE TABLE defaultdb.bar (j INT); @@ -230,55 +319,102 @@ ALTER TABLE defaultdb.bar ADD COLUMN b INT; - ADD PrimaryIndex:{DescID: 52, Name: new_primary_key, IndexID: 2} state: ABSENT details: - indexId: 2 - indexName: new_primary_key - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} - storingColumnIds: - - 2 + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 1 + keyColumnNames: + - i + name: new_primary_key + partitioning: {} + sharded: {} + storeColumnIds: + - 2 + storeColumnNames: + - a + unique: true + version: 4 + otherPrimaryIndexId: 1 tableId: 52 - unique: true - ADD PrimaryIndex:{DescID: 53, Name: new_primary_key, IndexID: 2} state: ABSENT details: - indexId: 2 - indexName: new_primary_key - keyColumnDirection: - - ASC - keyColumnIds: - - 2 - shardedDescriptor: {} - storingColumnIds: - - 1 - - 3 + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 2 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 2 + keyColumnNames: + - rowid + name: new_primary_key + partitioning: {} + sharded: {} + storeColumnIds: + - 1 + - 3 + storeColumnNames: + - j + - b + unique: true + version: 4 + otherPrimaryIndexId: 1 tableId: 53 - unique: true - DROP PrimaryIndex:{DescID: 52, Name: foo_pkey, IndexID: 1} state: PUBLIC details: - indexId: 1 - indexName: foo_pkey - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 1 + keyColumnNames: + - i + name: foo_pkey + partitioning: {} + sharded: {} + unique: true + version: 4 + otherPrimaryIndexId: 2 tableId: 52 - unique: true - DROP PrimaryIndex:{DescID: 53, Name: bar_pkey, IndexID: 1} state: PUBLIC details: - indexId: 1 - indexName: bar_pkey - keyColumnDirection: - - ASC - keyColumnIds: - - 2 - shardedDescriptor: {} - storingColumnIds: - - 1 + index: + encodingType: 1 + foreignKey: {} + geoConfig: {} + id: 1 + interleave: {} + keyColumnDirections: + - ASC + keyColumnIds: + - 2 + keyColumnNames: + - rowid + name: bar_pkey + partitioning: {} + sharded: {} + storeColumnIds: + - 1 + storeColumnNames: + - j + unique: true + version: 4 + otherPrimaryIndexId: 2 tableId: 53 - unique: true diff --git a/pkg/sql/schemachanger/scbuild/testdata/create_index b/pkg/sql/schemachanger/scbuild/testdata/create_index deleted file mode 100644 index fbb6721e8284..000000000000 --- a/pkg/sql/schemachanger/scbuild/testdata/create_index +++ /dev/null @@ -1,143 +0,0 @@ -create-table -CREATE TABLE defaultdb.t1 (id INT PRIMARY KEY, name varchar(256), money int) ----- - -build -CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) ----- -- ADD SecondaryIndex:{DescID: 52, Name: id1, IndexID: 2} - state: ABSENT - details: - indexId: 2 - indexName: id1 - keyColumnDirection: - - ASC - - ASC - keyColumnIds: - - 1 - - 2 - storingColumnIds: - - 3 - tableId: 52 - -build -CREATE INVERTED INDEX concurrently id1 on defaultdb.t1(id, name) storing (money) ----- -- ADD SecondaryIndex:{DescID: 52, Name: id1, IndexID: 2} - state: ABSENT - details: - concurrently: true - indexId: 2 - indexName: id1 - inverted: true - keyColumnDirection: - - ASC - - ASC - keyColumnIds: - - 1 - - 2 - storingColumnIds: - - 3 - tableId: 52 - -build -CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) PARTITION BY LIST (id) ( - PARTITION p1 VALUES IN (1) - ) ----- -- ADD Partitioning:{DescID: 52, IndexID: 2} - state: ABSENT - details: - fields: - - id - indexId: 2 - listPartitions: - - expr: - - "1" - name: p1 - rangePartitions: [] - tableId: 52 -- ADD SecondaryIndex:{DescID: 52, Name: id1, IndexID: 2} - state: ABSENT - details: - indexId: 2 - indexName: id1 - keyColumnDirection: - - ASC - - ASC - keyColumnIds: - - 1 - - 2 - storingColumnIds: - - 3 - tableId: 52 - -build -CREATE INDEX id1 on defaultdb.t1(id, name) USING HASH WITH BUCKET_COUNT=8 storing (money); ----- -- ADD Column:{DescID: 52, ColumnID: 4, Name: crdb_internal_id_name_shard_8} - state: ABSENT - details: - column: - computeExpr: mod(fnv32("crdb_internal.datums_to_bytes"(id, name)), 8:::INT8) - hidden: true - id: 4 - name: crdb_internal_id_name_shard_8 - type: - family: IntFamily - oid: 23 - width: 32 - familyName: primary - tableId: 52 -- ADD PrimaryIndex:{DescID: 52, Name: new_primary_key, IndexID: 2} - state: ABSENT - details: - indexId: 2 - indexName: new_primary_key - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} - storingColumnIds: - - 2 - - 3 - - 4 - tableId: 52 - unique: true -- ADD SecondaryIndex:{DescID: 52, Name: id1, IndexID: 3} - state: ABSENT - details: - indexId: 3 - indexName: id1 - keyColumnDirection: - - ASC - - ASC - keyColumnIds: - - 1 - - 2 - shardedDescriptor: - columnNames: - - id - - name - isSharded: true - name: crdb_internal_id_name_shard_8 - shardBuckets: 8 - storingColumnIds: - - 3 - tableId: 52 -- DROP PrimaryIndex:{DescID: 52, Name: t1_pkey, IndexID: 1} - state: PUBLIC - details: - indexId: 1 - indexName: t1_pkey - keyColumnDirection: - - ASC - keyColumnIds: - - 1 - shardedDescriptor: {} - storingColumnIds: - - 2 - - 3 - tableId: 52 - unique: true diff --git a/pkg/sql/schemachanger/scdeps/BUILD.bazel b/pkg/sql/schemachanger/scdeps/BUILD.bazel index a23dba1ccdd4..f3f9f9771aba 100644 --- a/pkg/sql/schemachanger/scdeps/BUILD.bazel +++ b/pkg/sql/schemachanger/scdeps/BUILD.bazel @@ -22,12 +22,9 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/catalog/resolver", - "//pkg/sql/catalog/tabledesc", - "//pkg/sql/parser", "//pkg/sql/schemachanger/scbuild", "//pkg/sql/schemachanger/scexec", "//pkg/sql/schemachanger/scop", - "//pkg/sql/schemachanger/scpb", "//pkg/sql/schemachanger/scrun", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", diff --git a/pkg/sql/schemachanger/scdeps/exec_deps.go b/pkg/sql/schemachanger/scdeps/exec_deps.go index fbefe0763361..0ccf99a49ffc 100644 --- a/pkg/sql/schemachanger/scdeps/exec_deps.go +++ b/pkg/sql/schemachanger/scdeps/exec_deps.go @@ -23,11 +23,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/errors" ) @@ -40,8 +37,6 @@ func NewExecutorDependencies( descsCollection *descs.Collection, jobRegistry *jobs.Registry, indexBackfiller scexec.IndexBackfiller, - indexValidator scexec.IndexValidator, - cclCallbacks scexec.Partitioner, testingKnobs *scexec.NewSchemaChangerTestingKnobs, statements []string, phase scop.Phase, @@ -52,8 +47,6 @@ func NewExecutorDependencies( codec: codec, descsCollection: descsCollection, jobRegistry: jobRegistry, - indexValidator: indexValidator, - partitioner: cclCallbacks, }, indexBackfiller: indexBackfiller, testingKnobs: testingKnobs, @@ -67,8 +60,6 @@ type txnDeps struct { codec keys.SQLCodec descsCollection *descs.Collection jobRegistry *jobs.Registry - indexValidator scexec.IndexValidator - partitioner scexec.Partitioner } var _ scexec.Catalog = (*txnDeps)(nil) @@ -97,61 +88,6 @@ func (d *txnDeps) RemoveSyntheticDescriptor(id descpb.ID) { d.descsCollection.RemoveSyntheticDescriptor(id) } -// AddPartitioning implements the scmutationexec.CatalogReader interface. -func (d *txnDeps) AddPartitioning( - tableDesc *tabledesc.Mutable, - indexDesc *descpb.IndexDescriptor, - partitionFields []string, - listPartition []*scpb.ListPartition, - rangePartition []*scpb.RangePartitions, - allowedNewColumnNames []tree.Name, - allowImplicitPartitioning bool, -) error { - ctx := context.Background() - // Deserialize back into tree based types - partitionBy := &tree.PartitionBy{} - partitionBy.List = make([]tree.ListPartition, 0, len(listPartition)) - partitionBy.Range = make([]tree.RangePartition, 0, len(rangePartition)) - for _, partition := range listPartition { - exprs, err := parser.ParseExprs(partition.Expr) - if err != nil { - return err - } - partitionBy.List = append(partitionBy.List, - tree.ListPartition{ - Name: tree.UnrestrictedName(partition.Name), - Exprs: exprs, - }) - } - for _, partition := range rangePartition { - toExpr, err := parser.ParseExprs(partition.To) - if err != nil { - return err - } - fromExpr, err := parser.ParseExprs(partition.From) - if err != nil { - return err - } - partitionBy.Range = append(partitionBy.Range, - tree.RangePartition{ - Name: tree.UnrestrictedName(partition.Name), - To: toExpr, - From: fromExpr, - }) - } - partitionBy.Fields = make(tree.NameList, 0, len(partitionFields)) - for _, field := range partitionFields { - partitionBy.Fields = append(partitionBy.Fields, tree.Name(field)) - } - // Create the paritioning - newImplicitCols, newPartitioning, err := d.partitioner.CreatePartitioning(ctx, tableDesc, *indexDesc, partitionBy, allowedNewColumnNames, allowImplicitPartitioning) - if err != nil { - return err - } - tabledesc.UpdateIndexPartitioning(indexDesc, false, newImplicitCols, newPartitioning) - return nil -} - // MustReadMutableDescriptor implements the scexec.Catalog interface. func (d *txnDeps) MustReadMutableDescriptor( ctx context.Context, id descpb.ID, @@ -276,10 +212,6 @@ func (d *execDeps) IndexBackfiller() scexec.IndexBackfiller { return d.indexBackfiller } -func (d *execDeps) IndexValidator() scexec.IndexValidator { - return d.indexValidator -} - // IndexSpanSplitter implements the scexec.Dependencies interface. func (d *execDeps) IndexSpanSplitter() scexec.IndexSpanSplitter { return d diff --git a/pkg/sql/schemachanger/scdeps/run_deps.go b/pkg/sql/schemachanger/scdeps/run_deps.go index 3ec9abc725d7..ce1b4562f63a 100644 --- a/pkg/sql/schemachanger/scdeps/run_deps.go +++ b/pkg/sql/schemachanger/scdeps/run_deps.go @@ -76,8 +76,6 @@ func NewJobExecutionDependencies( job *jobs.Job, codec keys.SQLCodec, settings *cluster.Settings, - indexValidator scexec.IndexValidator, - cclCallbacks scexec.Partitioner, testingKnobs *scexec.NewSchemaChangerTestingKnobs, statements []string, ) scrun.SchemaChangeJobExecutionDependencies { @@ -92,8 +90,6 @@ func NewJobExecutionDependencies( settings: settings, testingKnobs: testingKnobs, statements: statements, - indexValidator: indexValidator, - partitioner: cclCallbacks, } } @@ -105,9 +101,6 @@ type jobExecutionDeps struct { jobRegistry *jobs.Registry job *jobs.Job - indexValidator scexec.IndexValidator - partitioner scexec.Partitioner - codec keys.SQLCodec settings *cluster.Settings testingKnobs *scexec.NewSchemaChangerTestingKnobs @@ -136,8 +129,6 @@ func (d *jobExecutionDeps) WithTxnInJob( codec: d.codec, descsCollection: descriptors, jobRegistry: d.jobRegistry, - indexValidator: d.indexValidator, - partitioner: d.partitioner, }, }) }) @@ -171,7 +162,6 @@ func (d *jobExecutionTxnDeps) ExecutorDependencies() scexec.Dependencies { indexBackfiller: d.indexBackfiller, testingKnobs: d.testingKnobs, statements: d.statements, - - phase: scop.PostCommitPhase, + phase: scop.PostCommitPhase, } } diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/BUILD.bazel b/pkg/sql/schemachanger/scdeps/sctestdeps/BUILD.bazel index dccfde0400c6..9c318d5795db 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/BUILD.bazel +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/BUILD.bazel @@ -21,15 +21,12 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/nstree", "//pkg/sql/catalog/schemadesc", - "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", - "//pkg/sql/parser", "//pkg/sql/privilege", "//pkg/sql/schemachanger/scbuild", "//pkg/sql/schemachanger/scexec", "//pkg/sql/schemachanger/scexec/scmutationexec", "//pkg/sql/schemachanger/scop", - "//pkg/sql/schemachanger/scpb", "//pkg/sql/schemachanger/scrun", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go index 642ed5fef5a3..6bd5bb63cc7a 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go @@ -24,15 +24,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" - "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -449,59 +446,6 @@ func (s *TestState) RemoveSyntheticDescriptor(id descpb.ID) { s.syntheticDescriptors.Remove(id) } -// AddPartitioning implements the scmutationexec.CatalogReader interface. -func (s *TestState) AddPartitioning( - tableDesc *tabledesc.Mutable, - _ *descpb.IndexDescriptor, - partitionFields []string, - listPartition []*scpb.ListPartition, - rangePartition []*scpb.RangePartitions, - _ []tree.Name, - _ bool, -) error { - // Deserialize back into tree based types. - partitionBy := &tree.PartitionBy{} - partitionBy.List = make([]tree.ListPartition, 0, len(listPartition)) - partitionBy.Range = make([]tree.RangePartition, 0, len(rangePartition)) - for _, partition := range listPartition { - exprs, err := parser.ParseExprs(partition.Expr) - if err != nil { - return err - } - partitionBy.List = append(partitionBy.List, - tree.ListPartition{ - Name: tree.UnrestrictedName(partition.Name), - Exprs: exprs, - }) - } - for _, partition := range rangePartition { - toExpr, err := parser.ParseExprs(partition.To) - if err != nil { - return err - } - fromExpr, err := parser.ParseExprs(partition.From) - if err != nil { - return err - } - partitionBy.Range = append(partitionBy.Range, - tree.RangePartition{ - Name: tree.UnrestrictedName(partition.Name), - To: toExpr, - From: fromExpr, - }) - } - partitionBy.Fields = make(tree.NameList, 0, len(partitionFields)) - for _, field := range partitionFields { - partitionBy.Fields = append(partitionBy.Fields, tree.Name(field)) - } - // For the purpose of testing we will only track - // these values. - s.partitioningInfo[tableDesc.GetID()] = &testPartitionInfo{ - PartitionBy: *partitionBy, - } - return nil -} - var _ scexec.Catalog = (*TestState)(nil) // MustReadMutableDescriptor implements the scexec.Catalog interface. @@ -759,31 +703,3 @@ func (ju *testJobUpdater) UpdateProgress(progress *jobspb.Progress) { func (s *TestState) ExecutorDependencies() scexec.Dependencies { return s } - -// ValidateForwardIndexes implements the index validator interface. -func (s *TestState) ValidateForwardIndexes( - ctx context.Context, - tableDesc catalog.TableDescriptor, - indexes []catalog.Index, - withFirstMutationPublic bool, - gatherAllInvalid bool, - override sessiondata.InternalExecutorOverride, -) error { - return nil -} - -// ValidateInvertedIndexes implements the index validator interface. -func (s *TestState) ValidateInvertedIndexes( - ctx context.Context, - tableDesc catalog.TableDescriptor, - indexes []catalog.Index, - gatherAllInvalid bool, - override sessiondata.InternalExecutorOverride, -) error { - return nil -} - -// IndexValidator implements the scexec.Dependencies interface. -func (s *TestState) IndexValidator() scexec.IndexValidator { - return s -} diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go index 0fabb3df0c10..52ff86d94f30 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -31,18 +30,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/protoutil" ) -// testPartitionInfo tracks partitioning information -// for testing -type testPartitionInfo struct { - tree.PartitionBy -} - // TestState is a backing struct used to implement all schema changer // dependencies, like scbuild.Dependencies or scexec.Dependencies, for the // purpose of facilitating end-to-end testing of the declarative schema changer. type TestState struct { descriptors, syntheticDescriptors nstree.Map - partitioningInfo map[descpb.ID]*testPartitionInfo namespace map[descpb.NameInfo]descpb.ID currentDatabase string phase scop.Phase diff --git a/pkg/sql/schemachanger/scexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/BUILD.bazel index 75a942a6c68a..ee962281c78c 100644 --- a/pkg/sql/schemachanger/scexec/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/BUILD.bazel @@ -22,8 +22,6 @@ go_library( "//pkg/sql/schemachanger/scexec/descriptorutils", "//pkg/sql/schemachanger/scexec/scmutationexec", "//pkg/sql/schemachanger/scop", - "//pkg/sql/sem/tree", - "//pkg/sql/sessiondata", "//pkg/util/log", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", @@ -59,7 +57,6 @@ go_test( "//pkg/sql/schemachanger/scpb", "//pkg/sql/schemachanger/scplan", "//pkg/sql/sem/tree", - "//pkg/sql/sessiondata", "//pkg/sql/sqlutil", "//pkg/sql/types", "//pkg/testutils/serverutils", diff --git a/pkg/sql/schemachanger/scexec/dependencies.go b/pkg/sql/schemachanger/scexec/dependencies.go index 07d4ecce6683..6e4e3de1af48 100644 --- a/pkg/sql/schemachanger/scexec/dependencies.go +++ b/pkg/sql/schemachanger/scexec/dependencies.go @@ -18,11 +18,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" ) // Dependencies contains all the dependencies required by the executor. @@ -30,7 +27,6 @@ type Dependencies interface { Catalog() Catalog TransactionalJobCreator() TransactionalJobCreator IndexBackfiller() IndexBackfiller - IndexValidator() IndexValidator IndexSpanSplitter() IndexSpanSplitter JobProgressTracker() JobProgressTracker @@ -96,39 +92,6 @@ type IndexBackfiller interface { ) error } -// Partitioner provides an interface that implements CCL exclusive -// callbacks. -type Partitioner interface { - CreatePartitioning( - ctx context.Context, - tableDesc *tabledesc.Mutable, - indexDesc descpb.IndexDescriptor, - partBy *tree.PartitionBy, - allowedNewColumnNames []tree.Name, - allowImplicitPartitioning bool, - ) (newImplicitCols []catalog.Column, newPartitioning descpb.PartitioningDescriptor, err error) -} - -// IndexValidator provides interfaces that allow indexes to be validated. -type IndexValidator interface { - ValidateForwardIndexes( - ctx context.Context, - tableDesc catalog.TableDescriptor, - indexes []catalog.Index, - withFirstMutationPublic bool, - gatherAllInvalid bool, - override sessiondata.InternalExecutorOverride, - ) error - - ValidateInvertedIndexes( - ctx context.Context, - tableDesc catalog.TableDescriptor, - indexes []catalog.Index, - gatherAllInvalid bool, - override sessiondata.InternalExecutorOverride, - ) error -} - // IndexSpanSplitter can try to split an index span in the current transaction // prior to backfilling. type IndexSpanSplitter interface { diff --git a/pkg/sql/schemachanger/scexec/exec_validation.go b/pkg/sql/schemachanger/scexec/exec_validation.go index 32a3f1491692..54f235b96f47 100644 --- a/pkg/sql/schemachanger/scexec/exec_validation.go +++ b/pkg/sql/schemachanger/scexec/exec_validation.go @@ -13,45 +13,11 @@ package scexec import ( "context" - "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/log" ) func executeValidationOps(ctx context.Context, deps Dependencies, execute []scop.Op) error { - for _, op := range execute { - switch op := op.(type) { - case *scop.ValidateUniqueIndex: - desc, err := deps.Catalog().MustReadImmutableDescriptor(ctx, op.TableID) - if err != nil { - return err - } - table, ok := desc.(catalog.TableDescriptor) - if !ok { - return catalog.WrapTableDescRefErr(desc.GetID(), catalog.NewDescriptorTypeError(desc)) - } - index, err := table.FindIndexWithID(op.IndexID) - if err != nil { - return err - } - // Execute the validation operation as a root user. - execOverride := sessiondata.InternalExecutorOverride{ - User: security.RootUserName(), - } - if index.GetType() == descpb.IndexDescriptor_FORWARD { - err = deps.IndexValidator().ValidateForwardIndexes(ctx, table, []catalog.Index{index}, true, false, execOverride) - } else { - err = deps.IndexValidator().ValidateInvertedIndexes(ctx, table, []catalog.Index{index}, false, execOverride) - } - return err - case *scop.ValidateCheckConstraint: - log.Errorf(ctx, "not implemented") - default: - panic("unimplemented") - } - } + log.Errorf(ctx, "not implemented") return nil } diff --git a/pkg/sql/schemachanger/scexec/executor_external_test.go b/pkg/sql/schemachanger/scexec/executor_external_test.go index 829d071684c8..9b5a9c3bc92f 100644 --- a/pkg/sql/schemachanger/scexec/executor_external_test.go +++ b/pkg/sql/schemachanger/scexec/executor_external_test.go @@ -32,7 +32,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -58,12 +57,10 @@ func (ti testInfra) newExecDeps( ti.lm.Codec(), txn, descsCollection, - nil, /* jobRegistry */ - noopBackfiller{}, /* indexBackfiller */ - noopIndexValidator{}, /* indexValidator */ - noopCCLCallbacks{}, /* noopCCLCallbacks */ - nil, /* testingKnobs */ - nil, /* statements */ + nil, /* jobRegistry */ + noopBackfiller{}, /* indexBackfiller */ + nil, /* testingKnobs */ + nil, /* statements */ phase, ) } @@ -158,13 +155,10 @@ CREATE TABLE db.t ( } indexToAdd := descpb.IndexDescriptor{ - ID: 2, - Name: "foo", - Version: descpb.StrictIndexColumnIDGuaranteesVersion, - CreatedExplicitly: true, - KeyColumnIDs: []descpb.ColumnID{1}, - KeyColumnNames: []string{"i"}, - StoreColumnNames: []string{}, + ID: 2, + Name: "foo", + KeyColumnIDs: []descpb.ColumnID{1}, + KeyColumnNames: []string{"i"}, KeyColumnDirections: []descpb.IndexDescriptor_Direction{ descpb.IndexDescriptor_ASC, }, @@ -189,12 +183,8 @@ CREATE TABLE db.t ( ops: func() scop.Ops { return scop.MakeOps( &scop.MakeAddedIndexDeleteOnly{ - TableID: table.ID, - IndexID: indexToAdd.ID, - IndexName: indexToAdd.Name, - KeyColumnIDs: indexToAdd.KeyColumnIDs, - KeyColumnDirections: indexToAdd.KeyColumnDirections, - SecondaryIndex: true, + TableID: table.ID, + Index: indexToAdd, }, ) }, @@ -237,6 +227,7 @@ CREATE TABLE db.t ( // is fixed up. func TestSchemaChanger(t *testing.T) { defer leaktest.AfterTest(t)() + ctx := context.Background() t.Run("add column", func(t *testing.T) { ti := setupTestInfra(t) @@ -259,14 +250,21 @@ func TestSchemaChanger(t *testing.T) { // targetSlice = []*scpb.Target{ scpb.NewTarget(scpb.Target_ADD, &scpb.PrimaryIndex{ - TableID: fooTable.GetID(), - IndexName: "new_primary_key", - IndexId: 2, - KeyColumnIDs: []descpb.ColumnID{1}, - KeyColumnDirections: []scpb.PrimaryIndex_Direction{scpb.PrimaryIndex_ASC}, - StoringColumnIDs: []descpb.ColumnID{2}, - Unique: true, - Inverted: false, + TableID: fooTable.GetID(), + Index: descpb.IndexDescriptor{ + Name: "new_primary_key", + ID: 2, + KeyColumnIDs: []descpb.ColumnID{1}, + KeyColumnNames: []string{"i"}, + KeyColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC}, + StoreColumnIDs: []descpb.ColumnID{2}, + StoreColumnNames: []string{"j"}, + Unique: true, + Type: descpb.IndexDescriptor_FORWARD, + Version: descpb.PrimaryIndexWithStoredColumnsVersion, + EncodingType: descpb.PrimaryIndexEncoding, + }, + OtherPrimaryIndexID: fooTable.GetPrimaryIndexID(), }), scpb.NewTarget(scpb.Target_ADD, &scpb.Column{ TableID: fooTable.GetID(), @@ -281,13 +279,17 @@ func TestSchemaChanger(t *testing.T) { }, }), scpb.NewTarget(scpb.Target_DROP, &scpb.PrimaryIndex{ - TableID: fooTable.GetID(), - IndexName: "primary", - IndexId: 1, - KeyColumnIDs: []descpb.ColumnID{1}, - KeyColumnDirections: []scpb.PrimaryIndex_Direction{scpb.PrimaryIndex_ASC}, - Unique: true, - Inverted: false, + TableID: fooTable.GetID(), + Index: descpb.IndexDescriptor{ + Name: "primary", + ID: 1, + KeyColumnIDs: []descpb.ColumnID{1}, + KeyColumnNames: []string{"i"}, + KeyColumnDirections: []descpb.IndexDescriptor_Direction{descpb.IndexDescriptor_ASC}, + Unique: true, + Type: descpb.IndexDescriptor_FORWARD, + }, + OtherPrimaryIndexID: 2, }), } @@ -417,43 +419,4 @@ func (n noopBackfiller) BackfillIndex( return nil } -type noopIndexValidator struct{} - -func (noopIndexValidator) ValidateForwardIndexes( - ctx context.Context, - tableDesc catalog.TableDescriptor, - indexes []catalog.Index, - withFirstMutationPublic bool, - gatherAllInvalid bool, - override sessiondata.InternalExecutorOverride, -) error { - return nil -} - -func (noopIndexValidator) ValidateInvertedIndexes( - ctx context.Context, - tableDesc catalog.TableDescriptor, - indexes []catalog.Index, - gatherAllInvalid bool, - override sessiondata.InternalExecutorOverride, -) error { - return nil -} - -type noopCCLCallbacks struct { -} - -func (noopCCLCallbacks) CreatePartitioning( - ctx context.Context, - tableDesc *tabledesc.Mutable, - indexDesc descpb.IndexDescriptor, - partBy *tree.PartitionBy, - allowedNewColumnNames []tree.Name, - allowImplicitPartitioning bool, -) (newImplicitCols []catalog.Column, newPartitioning descpb.PartitioningDescriptor, err error) { - return nil, descpb.PartitioningDescriptor{}, nil -} - var _ scexec.IndexBackfiller = noopBackfiller{} -var _ scexec.IndexValidator = noopIndexValidator{} -var _ scexec.Partitioner = noopCCLCallbacks{} diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel index 100c2472d334..842b5051ae5c 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel @@ -19,8 +19,6 @@ go_library( "//pkg/sql/parser", "//pkg/sql/schemachanger/scexec/descriptorutils", "//pkg/sql/schemachanger/scop", - "//pkg/sql/schemachanger/scpb", - "//pkg/sql/sem/tree", "//pkg/util/protoutil", "@com_github_cockroachdb_errors//:errors", ], diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go b/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go index 268952a9b4f9..1feb9105809d 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go @@ -24,8 +24,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/descriptorutils" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" ) @@ -43,17 +41,6 @@ type CatalogReader interface { // RemoveSyntheticDescriptor undoes the effects of AddSyntheticDescriptor. RemoveSyntheticDescriptor(id descpb.ID) - - // AddPartitioning adds partitioning information on an index descriptor. - AddPartitioning( - tableDesc *tabledesc.Mutable, - indexDesc *descpb.IndexDescriptor, - partitionFields []string, - listPartition []*scpb.ListPartition, - rangePartition []*scpb.RangePartitions, - allowedNewColumnNames []tree.Name, - allowImplicitPartitioning bool, - ) error } // MutationVisitorStateUpdater is the interface for updating the visitor state. @@ -452,36 +439,33 @@ func (m *visitor) MakeAddedColumnDeleteOnly( if err != nil { return err } + // TODO(ajwerner): deal with ordering the indexes or sanity checking this // or what-not. if op.Column.ID >= table.NextColumnID { table.NextColumnID = op.Column.ID + 1 } - if !op.Column.IsComputed() || - !op.Column.Virtual { - var foundFamily bool - for i := range table.Families { - fam := &table.Families[i] - if foundFamily = fam.ID == op.FamilyID; foundFamily { - fam.ColumnIDs = append(fam.ColumnIDs, op.Column.ID) - fam.ColumnNames = append(fam.ColumnNames, op.Column.Name) - break - } + var foundFamily bool + for i := range table.Families { + fam := &table.Families[i] + if foundFamily = fam.ID == op.FamilyID; foundFamily { + fam.ColumnIDs = append(fam.ColumnIDs, op.Column.ID) + fam.ColumnNames = append(fam.ColumnNames, op.Column.Name) + break } - // Only create column families for non-computed columns - if !foundFamily { - table.Families = append(table.Families, descpb.ColumnFamilyDescriptor{ - Name: op.FamilyName, - ID: op.FamilyID, - ColumnNames: []string{op.Column.Name}, - ColumnIDs: []descpb.ColumnID{op.Column.ID}, - }) - sort.Slice(table.Families, func(i, j int) bool { - return table.Families[i].ID < table.Families[j].ID - }) - if table.NextFamilyID <= op.FamilyID { - table.NextFamilyID = op.FamilyID + 1 - } + } + if !foundFamily { + table.Families = append(table.Families, descpb.ColumnFamilyDescriptor{ + Name: op.FamilyName, + ID: op.FamilyID, + ColumnNames: []string{op.Column.Name}, + ColumnIDs: []descpb.ColumnID{op.Column.ID}, + }) + sort.Slice(table.Families, func(i, j int) bool { + return table.Families[i].ID < table.Families[j].ID + }) + if table.NextFamilyID <= op.FamilyID { + table.NextFamilyID = op.FamilyID + 1 } } table.AddColumnMutation(&op.Column, descpb.DescriptorMutation_ADD) @@ -511,81 +495,34 @@ func (m *visitor) MakeDroppedPrimaryIndexDeleteAndWriteOnly( if err != nil { return err } - if table.PrimaryIndex.ID != op.IndexID { - return errors.AssertionFailedf("index being dropped (%d) does not match existing primary index (%d).", op.IndexID, table.PrimaryIndex.ID) + + // NOTE: There is no ordering guarantee between operations which might + // touch the primary index. Remove it if it has not already been overwritten. + if table.PrimaryIndex.ID == op.Index.ID { + table.PrimaryIndex = descpb.IndexDescriptor{} } - idx := protoutil.Clone(&table.PrimaryIndex).(*descpb.IndexDescriptor) + + idx := protoutil.Clone(&op.Index).(*descpb.IndexDescriptor) return table.AddIndexMutation(idx, descpb.DescriptorMutation_DROP) } func (m *visitor) MakeAddedIndexDeleteOnly( ctx context.Context, op scop.MakeAddedIndexDeleteOnly, ) error { + table, err := m.checkOutTable(ctx, op.TableID) if err != nil { return err } + // TODO(ajwerner): deal with ordering the indexes or sanity checking this // or what-not. - if op.IndexID >= table.NextIndexID { - table.NextIndexID = op.IndexID + 1 - } - // Resolve column names - colNames := make([]string, 0, len(op.KeyColumnIDs)) - for _, colID := range op.KeyColumnIDs { - column, err := table.FindColumnWithID(colID) - if err != nil { - return err - } - colNames = append(colNames, column.GetName()) - } - storeColNames := make([]string, 0, len(op.StoreColumnIDs)) - for _, colID := range op.StoreColumnIDs { - column, err := table.FindColumnWithID(colID) - if err != nil { - return err - } - storeColNames = append(storeColNames, column.GetName()) - } - // Setup the index descriptor type. - indexType := descpb.IndexDescriptor_FORWARD - if op.Inverted { - indexType = descpb.IndexDescriptor_INVERTED - } - // Setup the encoding type. - encodingType := descpb.PrimaryIndexEncoding - indexVersion := descpb.PrimaryIndexWithStoredColumnsVersion - if op.SecondaryIndex { - encodingType = descpb.SecondaryIndexEncoding - indexVersion = descpb.StrictIndexColumnIDGuaranteesVersion - } - // Create an index descriptor from the the operation. - idx := &descpb.IndexDescriptor{ - Name: op.IndexName, - ID: op.IndexID, - Unique: op.Unique, - Version: indexVersion, - KeyColumnNames: colNames, - KeyColumnIDs: op.KeyColumnIDs, - StoreColumnIDs: op.StoreColumnIDs, - StoreColumnNames: storeColNames, - KeyColumnDirections: op.KeyColumnDirections, - Type: indexType, - KeySuffixColumnIDs: op.KeySuffixColumnIDs, - CompositeColumnIDs: op.CompositeColumnIDs, - CreatedExplicitly: true, - EncodingType: encodingType, - } - if idx.Name == "" { - name, err := tabledesc.BuildIndexName(table, idx) - if err != nil { - return err - } - idx.Name = name - } - if op.ShardedDescriptor != nil { - idx.Sharded = *op.ShardedDescriptor + if op.Index.ID >= table.NextIndexID { + table.NextIndexID = op.Index.ID + 1 } + // Make some adjustments to the index descriptor so that it behaves correctly + // as a secondary index while being added. + idx := protoutil.Clone(&op.Index).(*descpb.IndexDescriptor) return table.AddIndexMutation(idx, descpb.DescriptorMutation_ADD) } @@ -609,28 +546,6 @@ func (m *visitor) AddCheckConstraint(ctx context.Context, op scop.AddCheckConstr return nil } -func (m *visitor) MakeAddedSecondaryIndexPublic( - ctx context.Context, op scop.MakeAddedSecondaryIndexPublic, -) error { - table, err := m.checkOutTable(ctx, op.TableID) - if err != nil { - return err - } - - for idx, idxMutation := range table.GetMutations() { - if idxMutation.GetIndex() != nil && - idxMutation.GetIndex().ID == op.IndexID { - err := table.MakeMutationComplete(idxMutation) - if err != nil { - return err - } - table.Mutations = append(table.Mutations[:idx], table.Mutations[idx+1:]...) - break - } - } - return nil -} - func (m *visitor) MakeAddedPrimaryIndexPublic( ctx context.Context, op scop.MakeAddedPrimaryIndexPublic, ) error { @@ -638,20 +553,15 @@ func (m *visitor) MakeAddedPrimaryIndexPublic( if err != nil { return err } - index, err := table.FindIndexWithID(op.IndexID) - if err != nil { - return err - } - indexDesc := index.IndexDescDeepCopy() if _, err := removeMutation( ctx, table, - descriptorutils.MakeIndexIDMutationSelector(op.IndexID), + descriptorutils.MakeIndexIDMutationSelector(op.Index.ID), descpb.DescriptorMutation_DELETE_AND_WRITE_ONLY, ); err != nil { return err } - table.PrimaryIndex = indexDesc + table.PrimaryIndex = *(protoutil.Clone(&op.Index)).(*descpb.IndexDescriptor) return nil } @@ -707,20 +617,4 @@ func (m *visitor) DropForeignKeyRef(ctx context.Context, op scop.DropForeignKeyR return nil } -func (m *visitor) AddIndexPartitionInfo(ctx context.Context, op scop.AddIndexPartitionInfo) error { - table, err := m.checkOutTable(ctx, op.TableID) - if err != nil { - return err - } - index, err := table.FindIndexWithID(op.IndexID) - if err != nil { - return err - } - return m.cr.AddPartitioning(table, index.IndexDesc(), op.PartitionFields, op.ListPartitions, op.RangePartitions, nil, true) -} - -func (m *visitor) NoOpInfo(_ context.Context, _ scop.NoOpInfo) error { - return nil -} - var _ scop.MutationVisitor = (*visitor)(nil) diff --git a/pkg/sql/schemachanger/scgraph/BUILD.bazel b/pkg/sql/schemachanger/scgraph/BUILD.bazel index a5543d9127d8..09e889293205 100644 --- a/pkg/sql/schemachanger/scgraph/BUILD.bazel +++ b/pkg/sql/schemachanger/scgraph/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "scgraph", @@ -17,15 +17,3 @@ go_library( "@com_github_cockroachdb_errors//:errors", ], ) - -go_test( - name = "scgraph_test", - srcs = ["graph_test.go"], - deps = [ - ":scgraph", - "//pkg/sql/catalog/descpb", - "//pkg/sql/schemachanger/scop", - "//pkg/sql/schemachanger/scpb", - "@com_github_stretchr_testify//require", - ], -) diff --git a/pkg/sql/schemachanger/scgraph/graph.go b/pkg/sql/schemachanger/scgraph/graph.go index 58bccc5e6f30..9bae7247f66d 100644 --- a/pkg/sql/schemachanger/scgraph/graph.go +++ b/pkg/sql/schemachanger/scgraph/graph.go @@ -11,8 +11,6 @@ package scgraph import ( - "container/list" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" @@ -39,15 +37,10 @@ type Graph struct { // from it. A Node may have at most one opEdge from it. nodeOpEdgesFrom map[*scpb.Node]*OpEdge - // nodeDepEdgesFrom maps a Node from its dependencies. - // A Node dependency is another target node which must be - // reached before or concurrently with this node. - nodeDepEdgesFrom map[*scpb.Node][]*DepEdge - - // nodeDepEdgesTo maps a Node to its dependencies. + // nodeDepEdges maps a Node to its dependencies. // A Node dependency is another target node which must be // reached before or concurrently with this node. - nodeDepEdgesTo map[*scpb.Node][]*DepEdge + nodeDepEdges map[*scpb.Node][]*DepEdge // opToNode maps from an operation back to the // opEdge that generated it as an index. @@ -77,12 +70,11 @@ func New(initial scpb.State) (*Graph, error) { return nil, err } g := Graph{ - targetIdxMap: map[*scpb.Target]int{}, - nodeOpEdgesFrom: map[*scpb.Node]*OpEdge{}, - nodeDepEdgesFrom: map[*scpb.Node][]*DepEdge{}, - nodeDepEdgesTo: map[*scpb.Node][]*DepEdge{}, - opToNode: map[scop.Op]*scpb.Node{}, - entities: db, + targetIdxMap: map[*scpb.Target]int{}, + nodeOpEdgesFrom: map[*scpb.Node]*OpEdge{}, + nodeDepEdges: map[*scpb.Node][]*DepEdge{}, + opToNode: map[scop.Op]*scpb.Node{}, + entities: db, } for _, n := range initial { if existing, ok := g.targetIdxMap[n.Target]; ok { @@ -153,21 +145,10 @@ func (g *Graph) GetOpEdgeFrom(n *scpb.Node) (*OpEdge, bool) { // GetDepEdgesFrom returns the unique outgoing op edge from the specified node, // if one exists. func (g *Graph) GetDepEdgesFrom(n *scpb.Node) ([]*DepEdge, bool) { - de, ok := g.nodeDepEdgesFrom[n] + de, ok := g.nodeDepEdges[n] return de, ok } -var _ = (*Graph)(nil).GetDepEdgesFrom - -// GetDepEdgesTo returns the unique outgoing op edge to the specified node, -// if one exists. -func (g *Graph) GetDepEdgesTo(n *scpb.Node) ([]*DepEdge, bool) { - de, ok := g.nodeDepEdgesTo[n] - return de, ok -} - -var _ = (*Graph)(nil).GetDepEdgesTo - // AddOpEdges adds an op edges connecting the nodes for two statuses of a target. func (g *Graph) AddOpEdges( t *scpb.Target, from, to scpb.Status, revertible bool, ops ...scop.Op, @@ -200,8 +181,6 @@ func (g *Graph) GetNodeFromOp(op scop.Op) *scpb.Node { return g.opToNode[op] } -var _ = (*Graph)(nil).GetNodeFromOp - // AddDepEdge adds a dep edge connecting two nodes (specified by their targets // and statuses). func (g *Graph) AddDepEdge( @@ -219,8 +198,7 @@ func (g *Graph) AddDepEdge( return err } g.edges = append(g.edges, de) - g.nodeDepEdgesFrom[de.from] = append(g.nodeDepEdgesFrom[de.from], de) - g.nodeDepEdgesTo[de.to] = append(g.nodeDepEdgesTo[de.to], de) + g.nodeDepEdges[de.from] = append(g.nodeDepEdges[de.from], de) return nil } @@ -271,50 +249,3 @@ func (de *DepEdge) To() *scpb.Node { return de.to } // Name returns the name of the rule which generated this edge. func (de *DepEdge) Name() string { return de.rule } - -// GetNodeRanks fetches ranks of nodes in topological order. -func (g *Graph) GetNodeRanks() map[*scpb.Node]int { - backCycleExists := func(n *scpb.Node, de *DepEdge) bool { - var foundBack bool - _ = g.ForEachDepEdgeFrom(de.To(), func(maybeBack *DepEdge) error { - foundBack = foundBack || maybeBack.To() == n - return nil - }) - return foundBack - } - l := list.New() - marks := make(map[*scpb.Node]bool) - var visit func(n *scpb.Node) - visit = func(n *scpb.Node) { - permanent, marked := marks[n] - if marked && !permanent { - panic("not a dag") - } - if marked && permanent { - return - } - marks[n] = false - _ = g.ForEachDepEdgeFrom(n, func(de *DepEdge) error { - // We want to eliminate cycles caused by swaps. In that - // case, we want to pretend that there is no edge from the - // add to the drop, and, in that way, the drop is ordered first. - if n.Direction == scpb.Target_ADD || !backCycleExists(n, de) { - visit(de.To()) - } - return nil - }) - marks[n] = true - l.PushFront(n) - } - - _ = g.ForEachNode(func(n *scpb.Node) error { - visit(n) - return nil - }) - rank := make(map[*scpb.Node]int, l.Len()) - for i, cur := 0, l.Front(); i < l.Len(); i++ { - rank[cur.Value.(*scpb.Node)] = i - cur = cur.Next() - } - return rank -} diff --git a/pkg/sql/schemachanger/scgraph/graph_test.go b/pkg/sql/schemachanger/scgraph/graph_test.go deleted file mode 100644 index 731434ac6bac..000000000000 --- a/pkg/sql/schemachanger/scgraph/graph_test.go +++ /dev/null @@ -1,156 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package scgraph_test - -import ( - "fmt" - "sort" - "testing" - - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" - "github.com/stretchr/testify/require" -) - -type depEdge struct { - from, to int -} - -func runRankTestForGraph(t *testing.T, addNode []bool, depEdges []depEdge, expectedOrder []int) { - // Setup a state based on if it is a add or drop. - state := make(scpb.State, 0, len(addNode)) - for idx := range addNode { - if addNode[idx] { - state = append(state, &scpb.Node{ - Target: scpb.NewTarget(scpb.Target_ADD, - &scpb.Table{ - TableID: descpb.ID(idx), - }), - Status: scpb.Status_ABSENT, - }) - } else { - state = append(state, &scpb.Node{ - Target: scpb.NewTarget(scpb.Target_DROP, - &scpb.Table{ - TableID: descpb.ID(idx), - }), - Status: scpb.Status_PUBLIC, - }) - } - } - // Setup the nodes first. - graph, err := scgraph.New(state) - require.NoError(t, err) - // Setup op edges for all the nodes. - for idx := range addNode { - if addNode[idx] { - require.NoError(t, graph.AddOpEdges(state[idx].Target, - scpb.Status_ABSENT, - scpb.Status_PUBLIC, - true, - &scop.MakeColumnAbsent{})) - } else { - require.NoError(t, graph.AddOpEdges(state[idx].Target, - scpb.Status_PUBLIC, - scpb.Status_ABSENT, - true, - &scop.MakeColumnAbsent{})) - } - } - // Add the dep edges next. - for _, edge := range depEdges { - require.NoError(t, graph.AddDepEdge( - fmt.Sprintf("%d to %d", edge.from, edge.to), - state[edge.from].Target, - scpb.Status_PUBLIC, - state[edge.to].Target, - scpb.Status_PUBLIC, - )) - } - - // Validates the rank order for nodes. - validateNodeRanks := func(graph *scgraph.Graph, expectedOrder []int) { - rank := graph.GetNodeRanks() - unsortedNodes := make([]*scpb.Node, 0, len(state)) - for _, node := range state { - publicNode, ok := graph.GetNode(node.Target, scpb.Status_PUBLIC) - require.Truef(t, ok, "public node doesn't exist") - unsortedNodes = append(unsortedNodes, publicNode) - } - sort.SliceStable(unsortedNodes, func(i, j int) bool { - return rank[unsortedNodes[i]] > rank[unsortedNodes[j]] - }) - sortedOrder := make([]int, 0, len(unsortedNodes)) - for _, node := range unsortedNodes { - sortedOrder = append(sortedOrder, int(node.Table.TableID)) - } - require.EqualValues(t, expectedOrder, sortedOrder, "ranks are not in expected order") - } - validateNodeRanks(graph, expectedOrder) -} - -// TestPlanGraphSort sanity checks sorting of the graph. -func TestGraphRanks(t *testing.T) { - // We will set up the dependency graph for basic ordering, so that: - // 1) 0 depends on 1 - // 2) 3 depends on 0 - // 3) 2 depends on nothing - t.Run("simple dependency graph", func(t *testing.T) { - runRankTestForGraph(t, - []bool{true, true, true, true}, - []depEdge{ - {0, 1}, - {3, 0}, - }, - []int{1, 0, 2, 3}, - ) - }) - - // We will set up the dependency graph, so that its - // intentionally cyclic, which should panic: - // 1) 0 depends on 1 - // 2) 3 depends on 0 - // 3) 1 depends on 3 - // 4) 3 depends on 1 - t.Run("cyclic graph", func(t *testing.T) { - require.Panicsf(t, func() { - runRankTestForGraph(t, - []bool{true, true, true, true}, - []depEdge{ - {0, 1}, - {3, 0}, - {1, 3}, - {3, 1}, - }, - nil, // Not expecting this to run. - ) - }, - "cyclic graph should always panic") - }) - - // We will set up the dependency graph to have a swap - // 1) 0 (adding) depends on 1 (dropping) - // 2) 1 (dropping) depends on 0 (adding) - // 3) 2 (adding) depends on 0 (adding) - t.Run("dependency graph with a swap", func(t *testing.T) { - runRankTestForGraph(t, - []bool{true, false, true}, - []depEdge{ - {0, 1}, - {1, 0}, - {2, 0}, - }, - []int{1, 0, 2}, // We expect the drop to be ordered first. - ) - }) -} diff --git a/pkg/sql/schemachanger/scgraph/iteration.go b/pkg/sql/schemachanger/scgraph/iteration.go index 7011302611e8..f9318b98dabd 100644 --- a/pkg/sql/schemachanger/scgraph/iteration.go +++ b/pkg/sql/schemachanger/scgraph/iteration.go @@ -59,7 +59,7 @@ type DepEdgeIterator func(de *DepEdge) error // ForEachDepEdgeFrom iterates the dep edges in the graph. func (g *Graph) ForEachDepEdgeFrom(n *scpb.Node, it DepEdgeIterator) error { - edges := g.nodeDepEdgesFrom[n] + edges := g.nodeDepEdges[n] for _, e := range edges { if err := it(e); err != nil { if iterutil.Done(err) { diff --git a/pkg/sql/schemachanger/schemachanger_test.go b/pkg/sql/schemachanger/schemachanger_test.go index 68a7c97a1722..3ddd66225c08 100644 --- a/pkg/sql/schemachanger/schemachanger_test.go +++ b/pkg/sql/schemachanger/schemachanger_test.go @@ -403,8 +403,7 @@ func TestConcurrentOldSchemaChangesCannotStart(t *testing.T) { } { _, err = conn.ExecContext(ctx, stmt) assert.Truef(t, - testutils.IsError(err, `cannot perform a schema change on table "t"`) || - testutils.IsError(err, `cannot perform TRUNCATE on "t" which has indexes being dropped`), + testutils.IsError(err, `cannot perform a schema change on table "t"`), "statement: %s, error: %s", stmt, err, ) } diff --git a/pkg/sql/schemachanger/scjob/job.go b/pkg/sql/schemachanger/scjob/job.go index bca35b60a302..2abb009cc547 100644 --- a/pkg/sql/schemachanger/scjob/job.go +++ b/pkg/sql/schemachanger/scjob/job.go @@ -66,8 +66,6 @@ func (n *newSchemaChangeResumer) Resume(ctx context.Context, execCtxI interface{ n.job, execCfg.Codec, execCfg.Settings, - sql.MakeIndexValidator(execCfg.DB, execCfg.Codec, execCfg.InternalExecutor), - sql.MakeCCLCallbacks(execCfg.Settings, nil), execCfg.NewSchemaChangerTestingKnobs, payload.Statement, ) diff --git a/pkg/sql/schemachanger/scop/BUILD.bazel b/pkg/sql/schemachanger/scop/BUILD.bazel index d66fe8b11da3..cf10b6aafcbd 100644 --- a/pkg/sql/schemachanger/scop/BUILD.bazel +++ b/pkg/sql/schemachanger/scop/BUILD.bazel @@ -19,7 +19,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/sql/catalog/descpb", - "//pkg/sql/schemachanger/scpb", "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/sql/schemachanger/scop/mutation.go b/pkg/sql/schemachanger/scop/mutation.go index bea1da3f0cd2..af341189be6b 100644 --- a/pkg/sql/schemachanger/scop/mutation.go +++ b/pkg/sql/schemachanger/scop/mutation.go @@ -10,10 +10,7 @@ package scop -import ( - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" -) +import "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" //go:generate go run ./generate_visitor.go scop Mutation mutation.go mutation_visitor_generated.go @@ -31,19 +28,7 @@ type MakeAddedIndexDeleteOnly struct { TableID descpb.ID // Index represents the index as it should appear in the mutation. - PrimaryIndex descpb.IndexID - IndexID descpb.IndexID - IndexName string - Unique bool - KeyColumnIDs []descpb.ColumnID - KeyColumnDirections []descpb.IndexDescriptor_Direction - KeySuffixColumnIDs []descpb.ColumnID - StoreColumnIDs []descpb.ColumnID - CompositeColumnIDs []descpb.ColumnID - ShardedDescriptor *descpb.ShardedDescriptor - Inverted bool - Concurrently bool - SecondaryIndex bool + Index descpb.IndexDescriptor } // MakeAddedIndexDeleteAndWriteOnly transitions an index addition mutation from @@ -54,20 +39,12 @@ type MakeAddedIndexDeleteAndWriteOnly struct { IndexID descpb.IndexID } -// MakeAddedSecondaryIndexPublic moves a new primary index from its mutation to -// public. -type MakeAddedSecondaryIndexPublic struct { - mutationOp - TableID descpb.ID - IndexID descpb.IndexID -} - // MakeAddedPrimaryIndexPublic moves a new primary index from its mutation to // public. type MakeAddedPrimaryIndexPublic struct { mutationOp TableID descpb.ID - IndexID descpb.IndexID + Index descpb.IndexDescriptor } // MakeDroppedPrimaryIndexDeleteAndWriteOnly moves a dropped primary index from @@ -76,11 +53,11 @@ type MakeDroppedPrimaryIndexDeleteAndWriteOnly struct { mutationOp TableID descpb.ID - // IndexID is the descriptor as it should be added as part of the mutation. The + // Index is the descriptor as it should be added as part of the mutation. The // primary index of a table has a slightly different encoding than that of // a secondary index. The value here sets it as it should be when adding // the mutation, including the stored columns. - IndexID descpb.IndexID + Index descpb.IndexDescriptor } // CreateGcJobForDescriptor creates a GC job for a given descriptor. @@ -250,20 +227,3 @@ type RemoveSequenceOwnedBy struct { mutationOp TableID descpb.ID } - -// AddIndexPartitionInfo adds partitoning information into -// an index -type AddIndexPartitionInfo struct { - mutationOp - TableID descpb.ID - IndexID descpb.IndexID - PartitionFields []string - ListPartitions []*scpb.ListPartition - RangePartitions []*scpb.RangePartitions -} - -// NoOpInfo a no-op mutation operation, which is allows -// an emit function to conditionally generate operations. -type NoOpInfo struct { - mutationOp -} diff --git a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go index 1f5367132e3d..d7f7d831f43c 100644 --- a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go +++ b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go @@ -24,7 +24,6 @@ type MutationOp interface { type MutationVisitor interface { MakeAddedIndexDeleteOnly(context.Context, MakeAddedIndexDeleteOnly) error MakeAddedIndexDeleteAndWriteOnly(context.Context, MakeAddedIndexDeleteAndWriteOnly) error - MakeAddedSecondaryIndexPublic(context.Context, MakeAddedSecondaryIndexPublic) error MakeAddedPrimaryIndexPublic(context.Context, MakeAddedPrimaryIndexPublic) error MakeDroppedPrimaryIndexDeleteAndWriteOnly(context.Context, MakeDroppedPrimaryIndexDeleteAndWriteOnly) error CreateGcJobForDescriptor(context.Context, CreateGcJobForDescriptor) error @@ -49,8 +48,6 @@ type MutationVisitor interface { AddColumnFamily(context.Context, AddColumnFamily) error DropForeignKeyRef(context.Context, DropForeignKeyRef) error RemoveSequenceOwnedBy(context.Context, RemoveSequenceOwnedBy) error - AddIndexPartitionInfo(context.Context, AddIndexPartitionInfo) error - NoOpInfo(context.Context, NoOpInfo) error } // Visit is part of the MutationOp interface. @@ -63,11 +60,6 @@ func (op MakeAddedIndexDeleteAndWriteOnly) Visit(ctx context.Context, v Mutation return v.MakeAddedIndexDeleteAndWriteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeAddedSecondaryIndexPublic) Visit(ctx context.Context, v MutationVisitor) error { - return v.MakeAddedSecondaryIndexPublic(ctx, op) -} - // Visit is part of the MutationOp interface. func (op MakeAddedPrimaryIndexPublic) Visit(ctx context.Context, v MutationVisitor) error { return v.MakeAddedPrimaryIndexPublic(ctx, op) @@ -187,13 +179,3 @@ func (op DropForeignKeyRef) Visit(ctx context.Context, v MutationVisitor) error func (op RemoveSequenceOwnedBy) Visit(ctx context.Context, v MutationVisitor) error { return v.RemoveSequenceOwnedBy(ctx, op) } - -// Visit is part of the MutationOp interface. -func (op AddIndexPartitionInfo) Visit(ctx context.Context, v MutationVisitor) error { - return v.AddIndexPartitionInfo(ctx, op) -} - -// Visit is part of the MutationOp interface. -func (op NoOpInfo) Visit(ctx context.Context, v MutationVisitor) error { - return v.NoOpInfo(ctx, op) -} diff --git a/pkg/sql/schemachanger/scop/validation.go b/pkg/sql/schemachanger/scop/validation.go index 7d7528377631..827b43d104c3 100644 --- a/pkg/sql/schemachanger/scop/validation.go +++ b/pkg/sql/schemachanger/scop/validation.go @@ -21,8 +21,9 @@ func (validationOp) Type() Type { return ValidationType } // ValidateUniqueIndex validates uniqueness of entries for a unique index. type ValidateUniqueIndex struct { validationOp - TableID descpb.ID - IndexID descpb.IndexID + TableID descpb.ID + PrimaryIndexID descpb.IndexID + IndexID descpb.IndexID } // ValidateCheckConstraint validates a check constraint on a table's columns. diff --git a/pkg/sql/schemachanger/scpb/elements_generated.go b/pkg/sql/schemachanger/scpb/elements_generated.go index d050e6fcb13b..432ecbf05e4b 100755 --- a/pkg/sql/schemachanger/scpb/elements_generated.go +++ b/pkg/sql/schemachanger/scpb/elements_generated.go @@ -29,5 +29,4 @@ func (e RelationDependedOnBy) element() {} func (e SequenceOwnedBy) element() {} func (e Type) element() {} func (e Schema) element() {} -func (e Database) element() {} -func (e Partitioning) element() {} \ No newline at end of file +func (e Database) element() {} \ No newline at end of file diff --git a/pkg/sql/schemachanger/scpb/scpb.pb.go b/pkg/sql/schemachanger/scpb/scpb.pb.go index fd7425622ca1..385c358ce306 100644 --- a/pkg/sql/schemachanger/scpb/scpb.pb.go +++ b/pkg/sql/schemachanger/scpb/scpb.pb.go @@ -99,58 +99,6 @@ func (Target_Direction) EnumDescriptor() ([]byte, []int) { return fileDescriptor_5413c88842564e28, []int{1, 0} } -// The direction of a column in the index. -type PrimaryIndex_Direction int32 - -const ( - PrimaryIndex_ASC PrimaryIndex_Direction = 0 - PrimaryIndex_DESC PrimaryIndex_Direction = 1 -) - -var PrimaryIndex_Direction_name = map[int32]string{ - 0: "ASC", - 1: "DESC", -} - -var PrimaryIndex_Direction_value = map[string]int32{ - "ASC": 0, - "DESC": 1, -} - -func (x PrimaryIndex_Direction) String() string { - return proto.EnumName(PrimaryIndex_Direction_name, int32(x)) -} - -func (PrimaryIndex_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_5413c88842564e28, []int{3, 0} -} - -// The direction of a column in the index. -type SecondaryIndex_Direction int32 - -const ( - SecondaryIndex_ASC SecondaryIndex_Direction = 0 - SecondaryIndex_DESC SecondaryIndex_Direction = 1 -) - -var SecondaryIndex_Direction_name = map[int32]string{ - 0: "ASC", - 1: "DESC", -} - -var SecondaryIndex_Direction_value = map[string]int32{ - "ASC": 0, - "DESC": 1, -} - -func (x SecondaryIndex_Direction) String() string { - return proto.EnumName(SecondaryIndex_Direction_name, int32(x)) -} - -func (SecondaryIndex_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_5413c88842564e28, []int{4, 0} -} - type SequenceDependency_Type int32 const ( @@ -198,7 +146,6 @@ type ElementProto struct { Type *Type `protobuf:"bytes,16,opt,name=type,proto3" json:"type,omitempty"` Schema *Schema `protobuf:"bytes,17,opt,name=schema,proto3" json:"schema,omitempty"` Database *Database `protobuf:"bytes,18,opt,name=database,proto3" json:"database,omitempty"` - Partitioning *Partitioning `protobuf:"bytes,19,opt,name=partitioning,proto3" json:"partitioning,omitempty"` } func (m *ElementProto) Reset() { *m = ElementProto{} } @@ -301,18 +248,9 @@ func (m *Column) XXX_DiscardUnknown() { var xxx_messageInfo_Column proto.InternalMessageInfo type PrimaryIndex struct { - TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` - IndexId github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID `protobuf:"varint,2,opt,name=index_id,json=indexId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID" json:"index_id,omitempty"` - IndexName string `protobuf:"bytes,3,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` - Unique bool `protobuf:"varint,4,opt,name=unique,proto3" json:"unique,omitempty"` - KeyColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,5,rep,packed,name=key_column_ids,json=keyColumnIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"key_column_ids,omitempty"` - KeyColumnDirections []PrimaryIndex_Direction `protobuf:"varint,6,rep,packed,name=key_column_direction,json=keyColumnDirection,proto3,enum=cockroach.sql.schemachanger.scpb.PrimaryIndex_Direction" json:"key_column_direction,omitempty"` - KeySuffixColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,7,rep,packed,name=key_suffix_column_ids,json=keySuffixColumnIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"key_suffix_column_ids,omitempty"` - ShardedDescriptor *descpb.ShardedDescriptor `protobuf:"bytes,8,opt,name=sharded_descriptor,json=shardedDescriptor,proto3" json:"sharded_descriptor,omitempty"` - StoringColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,9,rep,packed,name=storing_column_ids,json=storingColumnIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"storing_column_ids,omitempty"` - CompositeColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,10,rep,packed,name=composite_column_ids,json=compositeColumnIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"composite_column_ids,omitempty"` - Inverted bool `protobuf:"varint,11,opt,name=inverted,proto3" json:"inverted,omitempty"` - Concurrently bool `protobuf:"varint,12,opt,name=concurrently,proto3" json:"concurrently,omitempty"` + TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` + Index descpb.IndexDescriptor `protobuf:"bytes,2,opt,name=index,proto3" json:"index"` + OtherPrimaryIndexID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID `protobuf:"varint,3,opt,name=other_primary_index_id,json=otherPrimaryIndexId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID" json:"other_primary_index_id,omitempty"` } func (m *PrimaryIndex) Reset() { *m = PrimaryIndex{} } @@ -345,18 +283,9 @@ func (m *PrimaryIndex) XXX_DiscardUnknown() { var xxx_messageInfo_PrimaryIndex proto.InternalMessageInfo type SecondaryIndex struct { - TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` - IndexId github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID `protobuf:"varint,2,opt,name=index_id,json=indexId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID" json:"index_id,omitempty"` - IndexName string `protobuf:"bytes,3,opt,name=index_name,json=indexName,proto3" json:"index_name,omitempty"` - Unique bool `protobuf:"varint,4,opt,name=unique,proto3" json:"unique,omitempty"` - KeyColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,5,rep,packed,name=key_column_ids,json=keyColumnIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"key_column_ids,omitempty"` - KeyColumnDirections []SecondaryIndex_Direction `protobuf:"varint,6,rep,packed,name=key_column_direction,json=keyColumnDirection,proto3,enum=cockroach.sql.schemachanger.scpb.SecondaryIndex_Direction" json:"key_column_direction,omitempty"` - KeySuffixColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,7,rep,packed,name=key_suffix_column_ids,json=keySuffixColumnIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"key_suffix_column_ids,omitempty"` - ShardedDescriptor *descpb.ShardedDescriptor `protobuf:"bytes,8,opt,name=sharded_descriptor,json=shardedDescriptor,proto3" json:"sharded_descriptor,omitempty"` - StoringColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,9,rep,packed,name=storing_column_ids,json=storingColumnIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"storing_column_ids,omitempty"` - CompositeColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID `protobuf:"varint,10,rep,packed,name=composite_column_ids,json=compositeColumnIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID" json:"composite_column_ids,omitempty"` - Inverted bool `protobuf:"varint,11,opt,name=inverted,proto3" json:"inverted,omitempty"` - Concurrently bool `protobuf:"varint,12,opt,name=concurrently,proto3" json:"concurrently,omitempty"` + TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` + Index descpb.IndexDescriptor `protobuf:"bytes,2,opt,name=index,proto3" json:"index"` + PrimaryIndex github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID `protobuf:"varint,3,opt,name=primary_index,json=primaryIndex,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID" json:"primary_index,omitempty"` } func (m *SecondaryIndex) Reset() { *m = SecondaryIndex{} } @@ -913,117 +842,9 @@ func (m *Database) XXX_DiscardUnknown() { var xxx_messageInfo_Database proto.InternalMessageInfo -type ListPartition struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - Expr []string `protobuf:"bytes,2,rep,name=expr,proto3" json:"expr,omitempty"` -} - -func (m *ListPartition) Reset() { *m = ListPartition{} } -func (m *ListPartition) String() string { return proto.CompactTextString(m) } -func (*ListPartition) ProtoMessage() {} -func (*ListPartition) Descriptor() ([]byte, []int) { - return fileDescriptor_5413c88842564e28, []int{20} -} -func (m *ListPartition) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *ListPartition) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil -} -func (m *ListPartition) XXX_Merge(src proto.Message) { - xxx_messageInfo_ListPartition.Merge(m, src) -} -func (m *ListPartition) XXX_Size() int { - return m.Size() -} -func (m *ListPartition) XXX_DiscardUnknown() { - xxx_messageInfo_ListPartition.DiscardUnknown(m) -} - -var xxx_messageInfo_ListPartition proto.InternalMessageInfo - -type RangePartitions struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - To []string `protobuf:"bytes,2,rep,name=To,proto3" json:"To,omitempty"` - From []string `protobuf:"bytes,3,rep,name=From,proto3" json:"From,omitempty"` -} - -func (m *RangePartitions) Reset() { *m = RangePartitions{} } -func (m *RangePartitions) String() string { return proto.CompactTextString(m) } -func (*RangePartitions) ProtoMessage() {} -func (*RangePartitions) Descriptor() ([]byte, []int) { - return fileDescriptor_5413c88842564e28, []int{21} -} -func (m *RangePartitions) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *RangePartitions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil -} -func (m *RangePartitions) XXX_Merge(src proto.Message) { - xxx_messageInfo_RangePartitions.Merge(m, src) -} -func (m *RangePartitions) XXX_Size() int { - return m.Size() -} -func (m *RangePartitions) XXX_DiscardUnknown() { - xxx_messageInfo_RangePartitions.DiscardUnknown(m) -} - -var xxx_messageInfo_RangePartitions proto.InternalMessageInfo - -type Partitioning struct { - TableID github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID" json:"table_id,omitempty"` - IndexId github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID `protobuf:"varint,2,opt,name=index_id,json=indexId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID" json:"index_id,omitempty"` - Fields []string `protobuf:"bytes,3,rep,name=fields,proto3" json:"fields,omitempty"` - ListPartitions []*ListPartition `protobuf:"bytes,4,rep,name=list_partitions,json=listPartitions,proto3" json:"list_partitions,omitempty"` - RangePartitions []*RangePartitions `protobuf:"bytes,5,rep,name=range_partitions,json=rangePartitions,proto3" json:"range_partitions,omitempty"` -} - -func (m *Partitioning) Reset() { *m = Partitioning{} } -func (m *Partitioning) String() string { return proto.CompactTextString(m) } -func (*Partitioning) ProtoMessage() {} -func (*Partitioning) Descriptor() ([]byte, []int) { - return fileDescriptor_5413c88842564e28, []int{22} -} -func (m *Partitioning) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *Partitioning) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil -} -func (m *Partitioning) XXX_Merge(src proto.Message) { - xxx_messageInfo_Partitioning.Merge(m, src) -} -func (m *Partitioning) XXX_Size() int { - return m.Size() -} -func (m *Partitioning) XXX_DiscardUnknown() { - xxx_messageInfo_Partitioning.DiscardUnknown(m) -} - -var xxx_messageInfo_Partitioning proto.InternalMessageInfo - func init() { proto.RegisterEnum("cockroach.sql.schemachanger.scpb.Status", Status_name, Status_value) proto.RegisterEnum("cockroach.sql.schemachanger.scpb.Target_Direction", Target_Direction_name, Target_Direction_value) - proto.RegisterEnum("cockroach.sql.schemachanger.scpb.PrimaryIndex_Direction", PrimaryIndex_Direction_name, PrimaryIndex_Direction_value) - proto.RegisterEnum("cockroach.sql.schemachanger.scpb.SecondaryIndex_Direction", SecondaryIndex_Direction_name, SecondaryIndex_Direction_value) proto.RegisterEnum("cockroach.sql.schemachanger.scpb.SequenceDependency_Type", SequenceDependency_Type_name, SequenceDependency_Type_value) proto.RegisterType((*ElementProto)(nil), "cockroach.sql.schemachanger.scpb.ElementProto") proto.RegisterType((*Target)(nil), "cockroach.sql.schemachanger.scpb.Target") @@ -1045,150 +866,119 @@ func init() { proto.RegisterType((*Type)(nil), "cockroach.sql.schemachanger.scpb.Type") proto.RegisterType((*Schema)(nil), "cockroach.sql.schemachanger.scpb.Schema") proto.RegisterType((*Database)(nil), "cockroach.sql.schemachanger.scpb.Database") - proto.RegisterType((*ListPartition)(nil), "cockroach.sql.schemachanger.scpb.ListPartition") - proto.RegisterType((*RangePartitions)(nil), "cockroach.sql.schemachanger.scpb.RangePartitions") - proto.RegisterType((*Partitioning)(nil), "cockroach.sql.schemachanger.scpb.Partitioning") } func init() { proto.RegisterFile("sql/schemachanger/scpb/scpb.proto", fileDescriptor_5413c88842564e28) } var fileDescriptor_5413c88842564e28 = []byte{ - // 2158 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x5a, 0xcf, 0x6f, 0x23, 0x49, - 0xf5, 0x4f, 0x3b, 0x8e, 0x63, 0xbf, 0xf8, 0x47, 0xa7, 0x66, 0xe6, 0xfb, 0x35, 0xab, 0x51, 0x7a, - 0xd6, 0x48, 0xbb, 0xd1, 0x82, 0x1c, 0x98, 0x5d, 0x21, 0x88, 0x84, 0xd8, 0x38, 0xed, 0x88, 0x26, - 0x59, 0x3b, 0x5b, 0x76, 0x26, 0xbb, 0x23, 0xc0, 0xea, 0x74, 0x57, 0x9c, 0xde, 0xd8, 0xdd, 0x4e, - 0x77, 0x7b, 0x32, 0x06, 0x04, 0x07, 0x04, 0x27, 0x7e, 0x9d, 0x38, 0xf3, 0x37, 0x20, 0xf1, 0x2f, - 0xa0, 0x11, 0x5c, 0x86, 0x0b, 0x5a, 0x09, 0xc9, 0x40, 0x8f, 0x04, 0x5c, 0xe1, 0x80, 0xc4, 0x9e, - 0x50, 0x55, 0xf5, 0x4f, 0x7b, 0x20, 0xde, 0xb1, 0x27, 0x5a, 0x0d, 0x73, 0xb1, 0xba, 0x5f, 0xf5, - 0xfb, 0x7c, 0xaa, 0x5e, 0xd5, 0x7b, 0xaf, 0x5e, 0x95, 0xe1, 0x55, 0xe7, 0xa2, 0xb7, 0xe5, 0x68, - 0x67, 0xa4, 0xaf, 0x6a, 0x67, 0xaa, 0xd9, 0x25, 0xf6, 0x96, 0xa3, 0x0d, 0x4e, 0xd8, 0x4f, 0x75, - 0x60, 0x5b, 0xae, 0x85, 0xee, 0x68, 0x96, 0x76, 0x6e, 0x5b, 0xaa, 0x76, 0x56, 0x75, 0x2e, 0x7a, - 0xd5, 0xc4, 0xc7, 0x55, 0xfa, 0xdd, 0x2b, 0x9f, 0xa6, 0x20, 0x9a, 0xea, 0xaa, 0x3d, 0xab, 0xbb, - 0xa5, 0x13, 0x0e, 0xe0, 0xda, 0x43, 0xcd, 0x1d, 0xda, 0x44, 0xe7, 0x30, 0xaf, 0xdc, 0xec, 0x5a, - 0x5d, 0x8b, 0x3d, 0x6e, 0xd1, 0x27, 0x2e, 0xad, 0xfc, 0x32, 0x0f, 0xf9, 0x7a, 0x8f, 0xf4, 0x89, - 0xe9, 0x1e, 0x32, 0xb6, 0xb7, 0x21, 0xa3, 0x59, 0xbd, 0x61, 0xdf, 0x2c, 0x0b, 0x77, 0x84, 0xcd, - 0xb5, 0xbb, 0x9b, 0xd5, 0xab, 0xe8, 0xab, 0xbb, 0xec, 0x7b, 0xec, 0xeb, 0xa1, 0x16, 0x14, 0x06, - 0xb6, 0xd1, 0x57, 0xed, 0x51, 0xc7, 0x30, 0x75, 0xf2, 0xb0, 0x9c, 0x62, 0x40, 0xd5, 0xab, 0x81, - 0x0e, 0xb9, 0x9a, 0x42, 0xb5, 0x70, 0x7e, 0x10, 0x7b, 0x43, 0xef, 0x43, 0xc9, 0x21, 0x9a, 0x65, - 0xea, 0x11, 0xec, 0x32, 0x83, 0xfd, 0xdc, 0xd5, 0xb0, 0xad, 0x40, 0x91, 0x03, 0x17, 0x9d, 0xc4, - 0x3b, 0x22, 0x70, 0xc3, 0x21, 0x17, 0x43, 0x62, 0x6a, 0xa4, 0xa3, 0x93, 0x01, 0x31, 0x75, 0x62, - 0x6a, 0xa3, 0x72, 0x9a, 0xc1, 0xbf, 0x35, 0x0b, 0x3c, 0x57, 0x96, 0x43, 0x5d, 0x8c, 0x9c, 0x29, - 0x19, 0xea, 0xc0, 0xfa, 0xd0, 0x34, 0x2e, 0x86, 0xa4, 0xa3, 0x59, 0xa6, 0xe3, 0xda, 0xaa, 0x61, - 0xba, 0xe5, 0x15, 0x46, 0x72, 0xf7, 0x6a, 0x92, 0x23, 0xa6, 0xba, 0x1b, 0x6a, 0x62, 0x71, 0x38, - 0x21, 0x41, 0x5f, 0x07, 0x51, 0x3b, 0x23, 0xda, 0x79, 0x1c, 0x3f, 0xc3, 0xf0, 0x3f, 0x3f, 0xc3, - 0x1c, 0x52, 0xcd, 0x18, 0x7c, 0x49, 0x4b, 0x0a, 0xd0, 0x1e, 0x64, 0x83, 0x41, 0x95, 0x57, 0x19, - 0xea, 0x1b, 0xb3, 0x9b, 0x06, 0x87, 0xba, 0xe8, 0x04, 0x90, 0x4e, 0x4e, 0xd5, 0x61, 0xcf, 0xed, - 0x90, 0x87, 0x03, 0x9b, 0x38, 0x8e, 0x61, 0x99, 0xe5, 0x2c, 0x43, 0x7c, 0xf3, 0x6a, 0x44, 0x99, - 0xeb, 0xd6, 0x43, 0x55, 0xbc, 0xae, 0x4f, 0x8a, 0xd0, 0x36, 0xa4, 0x1f, 0x18, 0xe4, 0xb2, 0x9c, - 0x63, 0xa8, 0xaf, 0x5d, 0x8d, 0x7a, 0xcf, 0x20, 0x97, 0x98, 0xe9, 0x20, 0x05, 0x56, 0xdd, 0xd1, - 0x80, 0x60, 0x72, 0x5a, 0x06, 0xa6, 0xbe, 0x75, 0xb5, 0x7a, 0x9b, 0x2b, 0x10, 0x9b, 0x8d, 0x35, - 0xd0, 0x47, 0x5f, 0x86, 0x15, 0x57, 0x3d, 0xe9, 0x91, 0xf2, 0x1a, 0x03, 0x7a, 0x7d, 0x06, 0x20, - 0xfa, 0x39, 0xe6, 0x5a, 0xe8, 0x3e, 0x14, 0xac, 0xa1, 0xbb, 0x67, 0xd9, 0xc4, 0xe8, 0x9a, 0xfb, - 0x64, 0x54, 0xce, 0xcf, 0xba, 0x22, 0x9b, 0x43, 0xf7, 0xc4, 0x1a, 0x9a, 0x7a, 0xa4, 0x8b, 0x93, - 0x50, 0xe8, 0x18, 0xf2, 0x86, 0x19, 0x83, 0x2e, 0xcc, 0x6a, 0x7f, 0xc5, 0x9c, 0x44, 0x4e, 0x00, - 0xa1, 0x0f, 0xe0, 0xa6, 0x4d, 0x7a, 0xaa, 0x6b, 0x58, 0xa6, 0xbf, 0xf6, 0xf5, 0xa6, 0x59, 0x1b, - 0x95, 0x8b, 0x8c, 0xe0, 0x0b, 0x57, 0x13, 0xe0, 0xa7, 0x68, 0xe3, 0xa7, 0x62, 0xa2, 0x63, 0x28, - 0x04, 0xcb, 0xaa, 0x79, 0x69, 0x12, 0xbb, 0x5c, 0x9a, 0x75, 0xb5, 0xb7, 0x62, 0x6a, 0x7a, 0x6d, - 0x84, 0x93, 0x38, 0x74, 0xfd, 0xd0, 0x39, 0x2c, 0x8b, 0xb3, 0xae, 0x1f, 0xb6, 0x00, 0x98, 0x0e, - 0x8d, 0x9f, 0xfc, 0x83, 0xf2, 0xfa, 0xac, 0xf1, 0xb3, 0xc5, 0x44, 0xd8, 0xd7, 0xa3, 0x9e, 0xa6, - 0xab, 0xae, 0x7a, 0xa2, 0x3a, 0xa4, 0x8c, 0x66, 0xf5, 0x34, 0xd9, 0xd7, 0xc0, 0xa1, 0x2e, 0xc2, - 0x90, 0x1f, 0xa8, 0xb6, 0x6b, 0x50, 0xbb, 0x19, 0x66, 0xb7, 0x7c, 0x63, 0xe6, 0x30, 0x1c, 0xd3, - 0xc2, 0x09, 0x8c, 0xed, 0xf4, 0xa3, 0x5f, 0x48, 0x42, 0xe5, 0x2f, 0x02, 0x64, 0xda, 0xaa, 0xdd, - 0x25, 0x2e, 0xfa, 0x06, 0x14, 0x08, 0x4f, 0x1f, 0x1d, 0x96, 0x50, 0xfc, 0xac, 0x31, 0x03, 0x4b, - 0x3c, 0xeb, 0xd4, 0xb2, 0x8f, 0xc6, 0xd2, 0xd2, 0xe3, 0xb1, 0x24, 0xe0, 0x3c, 0x89, 0x67, 0xa3, - 0x43, 0xc8, 0xe9, 0x86, 0x4d, 0x34, 0xca, 0xcf, 0xf2, 0x48, 0x71, 0x96, 0x60, 0xc9, 0xfb, 0x56, - 0x95, 0x03, 0x4d, 0x1c, 0x81, 0x54, 0x3e, 0x03, 0xb9, 0x50, 0x8e, 0xd6, 0x60, 0xf5, 0xa8, 0xb1, - 0xdf, 0x68, 0x1e, 0x37, 0xc4, 0x25, 0xb4, 0x0a, 0xcb, 0x3b, 0xb2, 0x2c, 0x0a, 0x28, 0x0b, 0x69, - 0x19, 0x37, 0x0f, 0xc5, 0x54, 0xe5, 0xb7, 0x29, 0xc8, 0xf0, 0xec, 0x86, 0x74, 0xc8, 0x32, 0xb7, - 0xec, 0x18, 0x3a, 0x1b, 0x63, 0xa1, 0xa6, 0x78, 0x63, 0x69, 0x95, 0x79, 0xac, 0x22, 0x7f, 0x34, - 0x96, 0xb6, 0xbb, 0x86, 0x7b, 0x36, 0x3c, 0xa9, 0x6a, 0x56, 0x7f, 0x2b, 0xec, 0xa1, 0x7e, 0x12, - 0x3d, 0x6f, 0x0d, 0xce, 0xbb, 0x5b, 0xd3, 0x99, 0xba, 0xaa, 0xc8, 0x78, 0x95, 0x41, 0x2b, 0x3a, - 0xea, 0x43, 0xee, 0x54, 0xed, 0x1b, 0xbd, 0x11, 0xa5, 0x49, 0x31, 0x9a, 0x43, 0x6f, 0x2c, 0x65, - 0xf7, 0x98, 0x90, 0xf1, 0xbc, 0xfd, 0xac, 0x3c, 0x01, 0x06, 0xce, 0x72, 0x0a, 0x45, 0x47, 0x12, - 0xac, 0xf9, 0x74, 0xa6, 0xda, 0x27, 0x2c, 0xa3, 0xe6, 0x30, 0x70, 0x51, 0x43, 0xed, 0x13, 0x54, - 0x0f, 0x77, 0x03, 0xe9, 0xa7, 0xc7, 0xb0, 0x8b, 0x1e, 0x5d, 0x6b, 0xfe, 0x16, 0x40, 0x26, 0x8e, - 0x66, 0x1b, 0x03, 0xd7, 0xb2, 0x6b, 0x69, 0x3a, 0xa1, 0xc1, 0x96, 0x60, 0x3b, 0xfd, 0x37, 0xba, - 0x6c, 0x7e, 0x9d, 0x83, 0x7c, 0x3c, 0xc5, 0x5f, 0x93, 0x4d, 0xbf, 0x09, 0x59, 0xb6, 0x61, 0x88, - 0x4c, 0xba, 0xfb, 0xd1, 0x58, 0xfa, 0xca, 0x33, 0x43, 0x53, 0x2c, 0x8a, 0xcf, 0x40, 0x15, 0x1d, - 0x7d, 0x16, 0x80, 0xe3, 0x47, 0x36, 0xac, 0x15, 0xbc, 0xb1, 0x94, 0x63, 0x5f, 0x52, 0x33, 0xe2, - 0x9c, 0x11, 0x3c, 0xa2, 0x0a, 0x64, 0x78, 0xe6, 0x66, 0x16, 0xcd, 0xd6, 0xc0, 0x1b, 0x4b, 0x19, - 0x9e, 0xdd, 0xb1, 0xdf, 0x82, 0xbe, 0x05, 0xc5, 0x73, 0x32, 0xea, 0x70, 0xe3, 0x75, 0x0c, 0xdd, - 0x29, 0xaf, 0xdc, 0x59, 0xde, 0x2c, 0xd4, 0xda, 0xde, 0x58, 0xca, 0xef, 0x93, 0x11, 0xb7, 0xb6, - 0x22, 0x3b, 0xf3, 0x2c, 0x87, 0x00, 0x04, 0xe7, 0xcf, 0x43, 0x44, 0xdd, 0x41, 0xdf, 0x17, 0xe0, - 0x66, 0x8c, 0x3c, 0xf2, 0xbe, 0xcc, 0x9d, 0xe5, 0xcd, 0xe2, 0xdd, 0x2f, 0x7e, 0xbc, 0x5d, 0x5c, - 0xe4, 0x83, 0xb5, 0xff, 0xf7, 0xc6, 0xd2, 0x8d, 0xb0, 0xf3, 0xa1, 0xdc, 0xc1, 0xe8, 0x7c, 0x4a, - 0x88, 0x7e, 0x2c, 0xc0, 0x2d, 0xda, 0x0b, 0x67, 0x78, 0x7a, 0x6a, 0x3c, 0x8c, 0x5b, 0x62, 0x95, - 0x59, 0xe2, 0xbe, 0x37, 0x96, 0xd0, 0x3e, 0x19, 0xb5, 0x58, 0xfb, 0x62, 0xed, 0x41, 0xfb, 0x93, - 0xc0, 0xd5, 0x1d, 0x64, 0x02, 0x72, 0xce, 0x54, 0x5b, 0x27, 0x7a, 0x47, 0x0f, 0x17, 0xb9, 0xbf, - 0x6b, 0xd9, 0xfc, 0x0f, 0x3e, 0xd1, 0xe2, 0x0a, 0x31, 0xa7, 0xb8, 0xe5, 0x8d, 0xa5, 0xf5, 0x29, - 0x31, 0x5e, 0x77, 0x26, 0x45, 0xe8, 0x87, 0x02, 0x20, 0xc7, 0xb5, 0x6c, 0xc3, 0xec, 0xc6, 0x07, - 0x9f, 0x63, 0x83, 0x7f, 0xcf, 0x1b, 0x4b, 0x62, 0x8b, 0xb7, 0x2e, 0x76, 0xe8, 0xa2, 0x93, 0x40, - 0xd5, 0x1d, 0xf4, 0x23, 0x01, 0x6e, 0x6a, 0x56, 0x7f, 0x60, 0x39, 0x86, 0x4b, 0xe2, 0x5d, 0x81, - 0x68, 0x1e, 0x76, 0x83, 0xf6, 0x05, 0xcf, 0x83, 0x36, 0x81, 0xab, 0x3b, 0x68, 0x93, 0xfa, 0xf2, - 0x03, 0x62, 0xbb, 0x44, 0x67, 0xbb, 0xaa, 0x6c, 0x2d, 0x4f, 0xc3, 0xa3, 0xe2, 0xcb, 0x70, 0xd8, - 0x8a, 0xde, 0x82, 0xbc, 0x66, 0x99, 0xda, 0xd0, 0xb6, 0x89, 0xe9, 0xf6, 0xf8, 0xe6, 0x29, 0x5b, - 0x13, 0xa9, 0x07, 0xed, 0xc6, 0xe4, 0x38, 0xf1, 0x55, 0x65, 0x23, 0x9e, 0x1d, 0x68, 0x42, 0x68, - 0xed, 0x8a, 0x4b, 0x2c, 0x21, 0xd4, 0x5b, 0xbb, 0xa2, 0xe0, 0x07, 0xb2, 0xdf, 0xe4, 0xa0, 0x98, - 0x2c, 0x2a, 0x5e, 0x86, 0xb2, 0x4f, 0x60, 0x28, 0xfb, 0xc1, 0x7f, 0x0f, 0x65, 0xdb, 0x1f, 0xb7, - 0x72, 0x7c, 0x19, 0xcc, 0x5e, 0x06, 0xb3, 0x17, 0x2f, 0x98, 0xfd, 0x7e, 0x19, 0xd0, 0xf4, 0x11, - 0xc6, 0xf5, 0xed, 0x77, 0xc3, 0x69, 0x88, 0xef, 0x77, 0x03, 0xbb, 0x2d, 0xc4, 0xf6, 0x59, 0xcd, - 0x37, 0x39, 0xea, 0xc3, 0x5a, 0x78, 0xd4, 0x63, 0xe8, 0x2c, 0xc0, 0x15, 0x6a, 0x07, 0xde, 0x58, - 0x82, 0xc0, 0x02, 0x73, 0x0f, 0x0d, 0x02, 0x02, 0x45, 0x47, 0xef, 0xf8, 0x75, 0x64, 0x9a, 0x15, - 0x2e, 0x5f, 0x7a, 0x96, 0xa3, 0xa4, 0x58, 0x69, 0x59, 0x79, 0x1d, 0xd2, 0xf4, 0x2d, 0x59, 0xb5, - 0x64, 0x21, 0x7d, 0xd4, 0xaa, 0xb7, 0x78, 0xd9, 0xd2, 0x3c, 0x6e, 0xb4, 0xc4, 0x94, 0x3f, 0xb1, - 0x7f, 0x48, 0x81, 0x38, 0x79, 0x6c, 0x74, 0x4d, 0xd3, 0x6a, 0x4c, 0xe5, 0xa9, 0x06, 0x65, 0xf1, - 0xf3, 0xcd, 0x42, 0x53, 0xd6, 0x00, 0x20, 0xe6, 0xc8, 0xcb, 0xcc, 0x91, 0xdf, 0xa5, 0x29, 0x6b, - 0xb1, 0xfe, 0x9b, 0x0b, 0xd6, 0x90, 0xe3, 0x5b, 0xf7, 0x57, 0x29, 0x28, 0x4d, 0x1c, 0x9a, 0x5d, - 0x93, 0x71, 0x11, 0xa4, 0x59, 0x7a, 0x4e, 0xb1, 0x6a, 0x8d, 0x3d, 0x53, 0x19, 0x79, 0x38, 0xb0, - 0xfd, 0x0a, 0x8e, 0x3d, 0x4f, 0x58, 0x26, 0xfd, 0xfc, 0x2d, 0x83, 0x6e, 0x43, 0xee, 0x81, 0xda, - 0x33, 0x74, 0x95, 0x46, 0xb4, 0x15, 0x1a, 0xa3, 0x70, 0x24, 0xf0, 0xed, 0xf6, 0x3d, 0xc8, 0x06, - 0xab, 0x7c, 0xd2, 0x1d, 0x85, 0xe7, 0xeb, 0x8e, 0x7e, 0x07, 0x7e, 0xb2, 0x0c, 0xeb, 0x53, 0xa7, - 0x88, 0x2f, 0x66, 0xb8, 0xfb, 0x36, 0x94, 0x86, 0x0e, 0x71, 0x22, 0x63, 0xc6, 0x1d, 0xa4, 0x74, - 0x94, 0x6c, 0x9a, 0x73, 0x8c, 0x93, 0x4c, 0xe8, 0x55, 0xc8, 0xc7, 0x0f, 0x7a, 0x59, 0x10, 0xcc, - 0xe1, 0xb5, 0xd8, 0x69, 0xad, 0x3f, 0x21, 0xbf, 0x4b, 0x41, 0xfa, 0x9e, 0x41, 0x2e, 0xaf, 0x69, - 0x0e, 0x6c, 0xda, 0xaf, 0xd8, 0xc9, 0x64, 0x8a, 0x59, 0x84, 0xc6, 0xa7, 0x7c, 0xfc, 0x74, 0x71, - 0x4e, 0xba, 0x04, 0x07, 0x32, 0x20, 0xc7, 0xdf, 0x9d, 0xa6, 0xe9, 0x4f, 0xc1, 0x3e, 0xf5, 0x44, - 0x39, 0x10, 0xce, 0xc9, 0x16, 0xa1, 0xfb, 0x36, 0xfd, 0xab, 0x00, 0x2b, 0xcc, 0x6e, 0x2f, 0xae, - 0x51, 0xa3, 0x91, 0x16, 0x12, 0xe7, 0xef, 0xa8, 0x0f, 0x85, 0x68, 0x5b, 0x1b, 0x39, 0xda, 0x57, - 0x69, 0x75, 0x42, 0x77, 0xa5, 0x73, 0x8f, 0x3a, 0x1f, 0xc1, 0x2b, 0x3a, 0x52, 0xf9, 0x85, 0x41, - 0x64, 0x5f, 0x46, 0x44, 0xbb, 0x34, 0x37, 0x51, 0x86, 0x02, 0x87, 0x81, 0xeb, 0xa7, 0x69, 0x40, - 0xd3, 0x27, 0xfb, 0xa8, 0x0b, 0x39, 0xcb, 0x36, 0xba, 0x86, 0x19, 0xf5, 0xe0, 0x6b, 0x34, 0xa6, - 0x34, 0x99, 0x70, 0xee, 0x3e, 0x64, 0x39, 0xb8, 0xa2, 0xa3, 0xef, 0x40, 0xd1, 0x27, 0xe2, 0x01, - 0x26, 0x08, 0x26, 0x47, 0xde, 0x58, 0x2a, 0x70, 0x36, 0x1e, 0x83, 0x16, 0x93, 0x57, 0x0a, 0x56, - 0x1c, 0x12, 0x0d, 0x20, 0x6f, 0x07, 0x53, 0x4c, 0x47, 0x9a, 0x66, 0x23, 0x7d, 0xc7, 0x1b, 0x4b, - 0x6b, 0xe1, 0xd4, 0xcf, 0x3d, 0xd8, 0xb5, 0x90, 0x42, 0xd1, 0x69, 0xf9, 0xb8, 0x1e, 0x51, 0x06, - 0x63, 0x5e, 0x89, 0xaa, 0x96, 0x90, 0x77, 0x91, 0xc3, 0x16, 0xed, 0x09, 0xd4, 0x30, 0xdf, 0x67, - 0xa2, 0x7c, 0x1f, 0xa4, 0xb2, 0x34, 0xac, 0x4f, 0x5d, 0xc8, 0xbc, 0x5c, 0x10, 0xff, 0xbb, 0x0b, - 0xe2, 0x9f, 0x02, 0x94, 0x26, 0xee, 0xb6, 0xae, 0x79, 0x93, 0x85, 0x5c, 0x28, 0x5a, 0x97, 0x26, - 0xb1, 0x3b, 0x61, 0xd6, 0x89, 0x0a, 0x80, 0x3c, 0xbb, 0x5e, 0x5b, 0x4c, 0xea, 0xc9, 0x5b, 0x11, - 0x96, 0x5e, 0xf9, 0x87, 0x00, 0x37, 0x9f, 0x76, 0x73, 0x78, 0x4d, 0xe9, 0xcf, 0x04, 0x88, 0x52, - 0x53, 0x7c, 0xc0, 0x0b, 0x4c, 0x7e, 0x31, 0x06, 0x7f, 0xb6, 0x0d, 0xbf, 0x1e, 0x7c, 0xfe, 0x19, - 0xa8, 0xf2, 0x77, 0x01, 0x32, 0xfc, 0x9a, 0x92, 0x86, 0x17, 0x5e, 0xb9, 0x4e, 0x84, 0x17, 0xde, - 0x3c, 0x7f, 0x78, 0xe1, 0xe0, 0x8a, 0x8e, 0xbe, 0x0b, 0x62, 0xf0, 0x77, 0x0c, 0xb7, 0x79, 0xf2, - 0x01, 0xd1, 0x5c, 0xc7, 0xdf, 0x57, 0x60, 0xea, 0x6c, 0xf2, 0x44, 0xdb, 0x9c, 0xbc, 0x53, 0x5c, - 0x95, 0x7f, 0x09, 0x90, 0x0d, 0xae, 0x55, 0xa9, 0x17, 0x05, 0x17, 0xab, 0x13, 0x5e, 0x14, 0x7c, - 0x32, 0xbf, 0x17, 0x05, 0x04, 0x9f, 0x80, 0xb1, 0xbf, 0x0b, 0x85, 0x03, 0xc3, 0x71, 0xc3, 0x9b, - 0x60, 0x74, 0xdb, 0x8f, 0x39, 0x02, 0x3b, 0x13, 0xce, 0x7a, 0x63, 0x29, 0xcd, 0x8e, 0x83, 0x79, - 0xf9, 0x79, 0xdb, 0x2f, 0x3f, 0x69, 0x17, 0xfd, 0x56, 0xba, 0xc1, 0xe7, 0x85, 0xa8, 0xbf, 0x5a, - 0xcf, 0xa1, 0x84, 0x55, 0xb3, 0x4b, 0x42, 0x4c, 0xe7, 0x0a, 0xd0, 0xff, 0x83, 0x54, 0xdb, 0xf2, - 0x21, 0x33, 0xde, 0x58, 0x4a, 0xb5, 0x2d, 0x9c, 0x6a, 0x5b, 0x54, 0x6b, 0xcf, 0xb6, 0xfa, 0x2c, - 0xd9, 0xf8, 0x5a, 0xf4, 0x1d, 0xb3, 0x5f, 0x9f, 0xec, 0x8f, 0xcb, 0x90, 0x8f, 0x5f, 0x63, 0xbf, - 0x20, 0xe7, 0xf3, 0x15, 0xc8, 0x9c, 0x1a, 0xa4, 0xe7, 0x1f, 0x74, 0xe4, 0xf8, 0x89, 0xfb, 0x1e, - 0x93, 0x60, 0xbf, 0x05, 0xf5, 0xa0, 0xd4, 0x33, 0x1c, 0xb7, 0x13, 0xde, 0xdb, 0xf3, 0xda, 0x7f, - 0xa6, 0x3f, 0xb2, 0x24, 0xe6, 0xbc, 0x86, 0xbc, 0xb1, 0x54, 0x4c, 0x88, 0x1c, 0x5c, 0xec, 0x25, - 0xde, 0xd1, 0x05, 0x88, 0x36, 0x05, 0x88, 0xd3, 0xad, 0x30, 0xba, 0x19, 0xfe, 0x86, 0x31, 0xb1, - 0x1e, 0x6a, 0x37, 0x68, 0x59, 0x3a, 0x21, 0xc4, 0x25, 0x3b, 0x29, 0xe0, 0x33, 0xfc, 0xc6, 0xcf, - 0x69, 0x44, 0x72, 0x55, 0x77, 0xe8, 0x24, 0xcf, 0xc3, 0x00, 0x32, 0x3b, 0xb5, 0x56, 0xbd, 0xd1, - 0x16, 0x05, 0xda, 0x20, 0xe3, 0xe6, 0xe1, 0x61, 0x5d, 0x16, 0x53, 0xa8, 0x04, 0x6b, 0x72, 0xfd, - 0xa0, 0xde, 0xae, 0x77, 0x9a, 0x8d, 0x83, 0xf7, 0xc5, 0x65, 0xf4, 0x29, 0xb8, 0xe5, 0x0b, 0x76, - 0x1a, 0x72, 0xe7, 0x18, 0x2b, 0x41, 0x53, 0x1a, 0x15, 0x01, 0x6a, 0x3b, 0xbb, 0xfb, 0x7b, 0xca, - 0xc1, 0x41, 0x5d, 0x16, 0x57, 0x50, 0x01, 0x72, 0xf7, 0x76, 0x0e, 0x14, 0x79, 0xa7, 0x5d, 0x97, - 0xc5, 0x0c, 0x85, 0x6a, 0xbf, 0xd7, 0xe8, 0x04, 0xd8, 0xab, 0x94, 0xf4, 0xf0, 0xa8, 0x76, 0xa0, - 0xec, 0x8a, 0xd9, 0xda, 0x6b, 0x8f, 0xfe, 0xbc, 0xb1, 0xf4, 0xc8, 0xdb, 0x10, 0x1e, 0x7b, 0x1b, - 0xc2, 0x87, 0xde, 0x86, 0xf0, 0x27, 0x6f, 0x43, 0xf8, 0xd9, 0x93, 0x8d, 0xa5, 0xc7, 0x4f, 0x36, - 0x96, 0x3e, 0x7c, 0xb2, 0xb1, 0x74, 0x3f, 0x4d, 0x07, 0x7f, 0x92, 0x61, 0xff, 0x97, 0x78, 0xf3, - 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xe9, 0x7e, 0xa2, 0x2e, 0x02, 0x28, 0x00, 0x00, + // 1698 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0xcd, 0x6f, 0xdb, 0xc8, + 0x15, 0x37, 0x25, 0x5a, 0x1f, 0x4f, 0x92, 0x45, 0x4f, 0xd2, 0x82, 0x0d, 0x0a, 0x29, 0x51, 0x81, + 0x24, 0x48, 0x01, 0xa9, 0x4d, 0x8a, 0x02, 0x35, 0x50, 0x34, 0x96, 0x29, 0xa3, 0xac, 0x1d, 0x49, + 0xa5, 0xec, 0x38, 0x09, 0x5a, 0x08, 0x14, 0x39, 0x91, 0x99, 0x48, 0xa4, 0x4c, 0x52, 0x71, 0x8c, + 0x16, 0xed, 0xa9, 0xd7, 0xa6, 0xa7, 0x9e, 0x7b, 0xee, 0xbd, 0xff, 0x83, 0x81, 0x5e, 0xdc, 0x4b, + 0x11, 0xa0, 0x80, 0xd0, 0x55, 0x80, 0xdd, 0xbd, 0xee, 0x1e, 0x16, 0xd8, 0x3d, 0x2d, 0x66, 0x86, + 0x5f, 0x92, 0xbc, 0x90, 0x10, 0x39, 0x5e, 0x6c, 0x76, 0x2f, 0x06, 0x35, 0xe4, 0xfb, 0xfd, 0xe6, + 0x7d, 0xbf, 0x19, 0xc3, 0x0d, 0xe7, 0xa8, 0x57, 0x71, 0xb4, 0x43, 0xdc, 0x57, 0xb5, 0x43, 0xd5, + 0xec, 0x62, 0xbb, 0xe2, 0x68, 0x83, 0x0e, 0xfd, 0x53, 0x1e, 0xd8, 0x96, 0x6b, 0xa1, 0xeb, 0x9a, + 0xa5, 0x3d, 0xb7, 0x2d, 0x55, 0x3b, 0x2c, 0x3b, 0x47, 0xbd, 0xf2, 0xc4, 0xc7, 0x65, 0xf2, 0xdd, + 0xb5, 0x1f, 0x11, 0x10, 0x4d, 0x75, 0xd5, 0x9e, 0xd5, 0xad, 0xe8, 0x98, 0x01, 0xb8, 0xf6, 0x50, + 0x73, 0x87, 0x36, 0xd6, 0x19, 0xcc, 0xb5, 0xab, 0x5d, 0xab, 0x6b, 0xd1, 0xc7, 0x0a, 0x79, 0x62, + 0xab, 0xa5, 0x51, 0x06, 0xb2, 0xb5, 0x1e, 0xee, 0x63, 0xd3, 0x6d, 0x52, 0xb6, 0xfb, 0x90, 0xd0, + 0xac, 0xde, 0xb0, 0x6f, 0x8a, 0xdc, 0x75, 0xee, 0x76, 0xe6, 0xee, 0xed, 0xf2, 0x3c, 0xfa, 0xf2, + 0x16, 0xfd, 0x5e, 0xf1, 0xe4, 0x50, 0x0b, 0x72, 0x03, 0xdb, 0xe8, 0xab, 0xf6, 0x49, 0xdb, 0x30, + 0x75, 0xfc, 0x52, 0x8c, 0x51, 0xa0, 0xf2, 0x7c, 0xa0, 0x26, 0x13, 0x93, 0x89, 0x94, 0x92, 0x1d, + 0x44, 0x7e, 0xa1, 0xc7, 0x90, 0x77, 0xb0, 0x66, 0x99, 0x7a, 0x08, 0x1b, 0xa7, 0xb0, 0x3f, 0x99, + 0x0f, 0xdb, 0xf2, 0x05, 0x19, 0xf0, 0x9a, 0x33, 0xf1, 0x1b, 0x61, 0xb8, 0xe2, 0xe0, 0xa3, 0x21, + 0x36, 0x35, 0xdc, 0xd6, 0xf1, 0x00, 0x9b, 0x3a, 0x36, 0xb5, 0x13, 0x91, 0xa7, 0xf0, 0x3f, 0x5b, + 0x04, 0x9e, 0x09, 0x4b, 0x81, 0xac, 0x82, 0x9c, 0x99, 0x35, 0xd4, 0x86, 0xf5, 0xa1, 0x69, 0x1c, + 0x0d, 0x71, 0x5b, 0xb3, 0x4c, 0xc7, 0xb5, 0x55, 0xc3, 0x74, 0xc5, 0x55, 0x4a, 0x72, 0x77, 0x3e, + 0xc9, 0x3e, 0x15, 0xdd, 0x0a, 0x24, 0x15, 0x61, 0x38, 0xb5, 0x82, 0x7e, 0x07, 0x82, 0x76, 0x88, + 0xb5, 0xe7, 0x51, 0xfc, 0x04, 0xc5, 0xff, 0xe9, 0x02, 0x3e, 0x24, 0x92, 0x11, 0xf8, 0xbc, 0x36, + 0xb9, 0x80, 0xb6, 0x21, 0xe5, 0x2b, 0x25, 0x26, 0x29, 0xea, 0x9d, 0xc5, 0x4d, 0xa3, 0x04, 0xb2, + 0xa8, 0x03, 0x48, 0xc7, 0x4f, 0xd5, 0x61, 0xcf, 0x6d, 0xe3, 0x97, 0x03, 0x1b, 0x3b, 0x8e, 0x61, + 0x99, 0x62, 0x8a, 0x22, 0xde, 0x9b, 0x8f, 0x28, 0x31, 0xd9, 0x5a, 0x20, 0xaa, 0xac, 0xeb, 0xd3, + 0x4b, 0x68, 0x03, 0xf8, 0x17, 0x06, 0x3e, 0x16, 0xd3, 0x14, 0xf5, 0xe6, 0x7c, 0xd4, 0x87, 0x06, + 0x3e, 0x56, 0xa8, 0x0c, 0x92, 0x21, 0xe9, 0x9e, 0x0c, 0xb0, 0x82, 0x9f, 0x8a, 0x40, 0xc5, 0x2b, + 0xf3, 0xc5, 0xf7, 0x98, 0x00, 0xb6, 0xa9, 0xae, 0xbe, 0x3c, 0xfa, 0x25, 0xac, 0xba, 0x6a, 0xa7, + 0x87, 0xc5, 0x0c, 0x05, 0xba, 0xb5, 0x00, 0x10, 0xf9, 0x5c, 0x61, 0x52, 0xe8, 0x09, 0xe4, 0xac, + 0xa1, 0xbb, 0x6d, 0xd9, 0xd8, 0xe8, 0x9a, 0x3b, 0xf8, 0x44, 0xcc, 0x2e, 0x1a, 0x91, 0x8d, 0xa1, + 0xdb, 0xb1, 0x86, 0xa6, 0x1e, 0xca, 0x2a, 0x93, 0x50, 0xe8, 0x00, 0xb2, 0x86, 0x19, 0x81, 0xce, + 0x2d, 0x6a, 0x7f, 0xd9, 0x9c, 0x46, 0x9e, 0x00, 0x42, 0xcf, 0xe0, 0xaa, 0x8d, 0x7b, 0xaa, 0x6b, + 0x58, 0xa6, 0x17, 0xfb, 0x7a, 0xc3, 0xac, 0x9e, 0x88, 0x6b, 0x94, 0xe0, 0xe7, 0xf3, 0x09, 0x94, + 0x73, 0xa4, 0x95, 0x73, 0x31, 0xd1, 0x01, 0xe4, 0xfc, 0xb0, 0x6a, 0x1c, 0x9b, 0xd8, 0x16, 0xf3, + 0x8b, 0x46, 0x7b, 0x2b, 0x22, 0xa6, 0x57, 0x4f, 0x94, 0x49, 0x1c, 0x12, 0x3f, 0xc4, 0x87, 0xa2, + 0xb0, 0x68, 0xfc, 0xd0, 0x00, 0xa0, 0x32, 0xa4, 0x7e, 0xb2, 0x0f, 0xc4, 0xf5, 0x45, 0xeb, 0x67, + 0x8b, 0x2e, 0x29, 0x9e, 0x1c, 0xc9, 0x34, 0x5d, 0x75, 0xd5, 0x8e, 0xea, 0x60, 0x11, 0x2d, 0x9a, + 0x69, 0x92, 0x27, 0xa1, 0x04, 0xb2, 0x1b, 0xfc, 0xe9, 0x3f, 0x8a, 0x5c, 0xe9, 0x43, 0x0e, 0x12, + 0x7b, 0xaa, 0xdd, 0xc5, 0x2e, 0xfa, 0x3d, 0xe4, 0x30, 0x2b, 0xf5, 0x6d, 0x5a, 0xfc, 0xbd, 0x0a, + 0xbf, 0x40, 0x61, 0x8e, 0x76, 0x88, 0x6a, 0xea, 0x74, 0x54, 0x5c, 0x39, 0x1b, 0x15, 0x39, 0x25, + 0x8b, 0xa3, 0x9d, 0xa3, 0x09, 0x69, 0xdd, 0xb0, 0xb1, 0x46, 0xfc, 0x44, 0x6b, 0xfe, 0xda, 0x22, + 0x85, 0x8d, 0xed, 0xad, 0x2c, 0xf9, 0x92, 0x4a, 0x08, 0x52, 0xfa, 0x31, 0xa4, 0x83, 0x75, 0x94, + 0x81, 0xe4, 0x7e, 0x7d, 0xa7, 0xde, 0x38, 0xa8, 0x0b, 0x2b, 0x28, 0x09, 0xf1, 0x4d, 0x49, 0x12, + 0x38, 0x94, 0x02, 0x5e, 0x52, 0x1a, 0x4d, 0x21, 0x56, 0xfa, 0x77, 0x0c, 0x12, 0xac, 0x13, 0x21, + 0x1d, 0x52, 0x34, 0x85, 0xda, 0x86, 0x4e, 0x75, 0xcc, 0x55, 0xe5, 0xf1, 0xa8, 0x98, 0xa4, 0xd9, + 0x25, 0x4b, 0x5f, 0x8c, 0x8a, 0x1b, 0x5d, 0xc3, 0x3d, 0x1c, 0x76, 0xca, 0x9a, 0xd5, 0xaf, 0x04, + 0x3b, 0xd4, 0x3b, 0xe1, 0x73, 0x65, 0xf0, 0xbc, 0x5b, 0x99, 0xed, 0xaa, 0x65, 0x59, 0x52, 0x92, + 0x14, 0x5a, 0xd6, 0x51, 0x1f, 0xd2, 0x4f, 0xd5, 0xbe, 0xd1, 0x3b, 0x21, 0x34, 0x31, 0x4a, 0xd3, + 0x1c, 0x8f, 0x8a, 0xa9, 0x6d, 0xba, 0x48, 0x79, 0xee, 0xbf, 0x2d, 0x8f, 0x8f, 0xa1, 0xa4, 0x18, + 0x85, 0xac, 0xa3, 0x22, 0x64, 0x3c, 0x3a, 0x53, 0xed, 0x63, 0xda, 0xfd, 0xd2, 0x0a, 0xb0, 0xa5, + 0xba, 0xda, 0xc7, 0xa8, 0x16, 0x74, 0x6e, 0xfe, 0xfc, 0x7a, 0x73, 0xd4, 0x23, 0x71, 0xe1, 0xb5, + 0x6b, 0x09, 0x3b, 0x9a, 0x6d, 0x0c, 0x5c, 0xcb, 0xae, 0xf2, 0xc4, 0xa1, 0x7e, 0xfb, 0xde, 0xe0, + 0x3f, 0x26, 0x61, 0x73, 0x16, 0x83, 0x6c, 0xb4, 0x1d, 0x5f, 0x92, 0x4d, 0xab, 0xb0, 0x1a, 0x9d, + 0x19, 0x6e, 0x7e, 0x85, 0x0a, 0x74, 0x4b, 0x33, 0x1a, 0x30, 0x51, 0xf4, 0x8a, 0x83, 0xef, 0x5b, + 0xee, 0x21, 0xb6, 0xdb, 0x13, 0x63, 0x08, 0xd9, 0x78, 0x9c, 0x6e, 0xfc, 0xc9, 0x78, 0x54, 0xbc, + 0xd2, 0x20, 0x5f, 0x44, 0x35, 0xa4, 0x4a, 0xfc, 0xea, 0xad, 0x95, 0x60, 0x10, 0xca, 0x15, 0x6b, + 0x06, 0x57, 0xf7, 0x4c, 0xfa, 0xcf, 0x18, 0xac, 0x4d, 0x8e, 0x22, 0xdf, 0x20, 0xa3, 0x1e, 0x4e, + 0x0f, 0x75, 0xcc, 0x94, 0x5b, 0x17, 0x61, 0xb3, 0x89, 0x49, 0xcf, 0x33, 0xd6, 0x7f, 0xe3, 0x80, + 0x66, 0x07, 0xab, 0xcb, 0xcb, 0x6c, 0x96, 0x0c, 0x53, 0x99, 0xcd, 0x32, 0x67, 0xb9, 0xcc, 0xf6, + 0x31, 0x94, 0x14, 0xa3, 0xa0, 0x74, 0x99, 0x60, 0x00, 0x0d, 0x82, 0x74, 0x77, 0x3c, 0x2a, 0x82, + 0x6f, 0x81, 0xa5, 0x55, 0x03, 0x9f, 0x40, 0xd6, 0xd1, 0x03, 0xaf, 0xbb, 0xf1, 0xb4, 0x44, 0xff, + 0xe2, 0x6d, 0x06, 0xdc, 0x48, 0xc3, 0x2b, 0xdd, 0x02, 0x9e, 0xfc, 0x9a, 0xac, 0xcf, 0x29, 0xe0, + 0xf7, 0x5b, 0xb5, 0x16, 0x2b, 0xd0, 0x8d, 0x83, 0x7a, 0x4b, 0x88, 0x79, 0x8e, 0xfd, 0x5f, 0x0c, + 0x84, 0xe9, 0x61, 0xf6, 0x92, 0xdc, 0x6a, 0x40, 0x2a, 0xa8, 0x04, 0xcc, 0xab, 0x75, 0xc2, 0x72, + 0x81, 0xd9, 0x9f, 0x34, 0x58, 0xc6, 0xa3, 0x01, 0x40, 0x10, 0x41, 0x8e, 0x18, 0xbf, 0x1e, 0xbf, + 0x9d, 0xab, 0xfe, 0x76, 0x3c, 0x2a, 0xa6, 0x7d, 0xf7, 0x3b, 0x17, 0x12, 0x43, 0x69, 0x3f, 0x86, + 0x1c, 0xcf, 0xba, 0xff, 0x8a, 0x41, 0x7e, 0x6a, 0x94, 0xbf, 0x24, 0xe3, 0x22, 0xe0, 0x69, 0x5f, + 0x8a, 0xd1, 0xbe, 0x44, 0x9f, 0xc9, 0x1a, 0x99, 0xf1, 0xbd, 0x5e, 0x45, 0x9f, 0xa7, 0x2c, 0xc3, + 0xbf, 0x7b, 0xcb, 0xa0, 0x1f, 0x42, 0xfa, 0x85, 0xda, 0x33, 0x74, 0xd5, 0xc5, 0x3a, 0x3d, 0x70, + 0xa5, 0x94, 0x70, 0xc1, 0xb3, 0xdb, 0x9f, 0x21, 0xe5, 0x47, 0xf9, 0x74, 0x3a, 0x72, 0xef, 0x36, + 0x1d, 0xbd, 0x0d, 0xfc, 0x35, 0x0e, 0xeb, 0x33, 0x67, 0x9b, 0xf7, 0xb3, 0xdc, 0xfd, 0x01, 0xf2, + 0x43, 0x07, 0x3b, 0xa1, 0x31, 0xa3, 0x09, 0x92, 0xdf, 0x9f, 0x7c, 0xb5, 0xa4, 0x8e, 0xd3, 0x4c, + 0xe8, 0x06, 0x64, 0xa3, 0xc7, 0x4f, 0x5a, 0x04, 0xd3, 0x4a, 0x26, 0x72, 0x86, 0xf4, 0x1c, 0xf2, + 0x9f, 0x18, 0xf0, 0xe4, 0x58, 0x78, 0x49, 0x3e, 0xb0, 0xc9, 0xbe, 0x22, 0xe7, 0xa5, 0x18, 0xb5, + 0x08, 0xa9, 0x4f, 0xd9, 0xe8, 0x99, 0x67, 0x49, 0xba, 0x09, 0x0e, 0x64, 0x40, 0x9a, 0xfd, 0x76, + 0x1a, 0xa6, 0xe7, 0x82, 0x1d, 0x92, 0x89, 0x92, 0xbf, 0xb8, 0x24, 0x5b, 0x88, 0xee, 0xd9, 0xf4, + 0x23, 0x0e, 0x56, 0xa9, 0xdd, 0xde, 0x5f, 0xa3, 0x86, 0x9a, 0xe6, 0x26, 0x6e, 0x05, 0x50, 0x1f, + 0x72, 0x7a, 0x30, 0x5b, 0x85, 0x89, 0xf6, 0xeb, 0xf1, 0xa8, 0x98, 0x20, 0x43, 0xd7, 0xd2, 0x5a, + 0x67, 0x43, 0x78, 0x59, 0x47, 0x2a, 0xbb, 0xc6, 0x08, 0xed, 0x4b, 0x89, 0xc8, 0x96, 0x96, 0x26, + 0x4a, 0x10, 0xe0, 0xa0, 0x70, 0xbd, 0xe2, 0x01, 0xcd, 0xde, 0x37, 0xa0, 0x2e, 0xa4, 0x2d, 0xdb, + 0xe8, 0x1a, 0x66, 0xb8, 0x83, 0xdf, 0x90, 0x9a, 0xd2, 0xa0, 0x8b, 0x4b, 0xef, 0x21, 0xc5, 0xc0, + 0x65, 0x1d, 0xfd, 0x11, 0xd6, 0x3c, 0x22, 0x56, 0x60, 0xfc, 0x62, 0xb2, 0x3f, 0x1e, 0x15, 0x73, + 0x8c, 0x8d, 0xd5, 0xa0, 0x8b, 0xe9, 0x2b, 0x39, 0x2b, 0x0a, 0x89, 0x06, 0x90, 0xb5, 0x7d, 0x17, + 0x13, 0x4d, 0x79, 0xaa, 0xe9, 0x83, 0xf1, 0xa8, 0x98, 0x09, 0x5c, 0xbf, 0xb4, 0xb2, 0x99, 0x80, + 0x42, 0xd6, 0xd1, 0x5f, 0x38, 0x58, 0x0f, 0x29, 0x7d, 0x9d, 0x57, 0xa9, 0xce, 0x8f, 0xc6, 0xa3, + 0xa2, 0x10, 0xf0, 0x5e, 0xa4, 0xda, 0x82, 0x3d, 0x85, 0x1a, 0xf4, 0xfb, 0x44, 0xd8, 0xef, 0xfd, + 0x56, 0xc6, 0xc3, 0xfa, 0xcc, 0x35, 0xd1, 0x77, 0x01, 0xf1, 0xed, 0x0d, 0x88, 0xcf, 0x38, 0xc8, + 0x4f, 0xdd, 0xb8, 0x5d, 0xf2, 0x90, 0x85, 0x5c, 0x58, 0xb3, 0x8e, 0x4d, 0x6c, 0xb7, 0x83, 0xae, + 0x13, 0x1e, 0x00, 0xb2, 0xf4, 0xd2, 0xef, 0x62, 0x5a, 0x4f, 0xd6, 0x0a, 0xb1, 0xf4, 0xd2, 0xa7, + 0x1c, 0x5c, 0x3d, 0xef, 0x3e, 0xf3, 0x92, 0xda, 0x9f, 0x09, 0x10, 0xb6, 0xa6, 0xa8, 0xc2, 0x17, + 0xd8, 0xfc, 0x22, 0x0c, 0x9e, 0xb7, 0x0d, 0xef, 0x3c, 0xf8, 0xee, 0x3b, 0x50, 0xe9, 0x13, 0x0e, + 0x12, 0xec, 0xf2, 0x94, 0x94, 0x17, 0x76, 0x72, 0x9d, 0x2a, 0x2f, 0xec, 0xf5, 0xf2, 0xe5, 0x85, + 0x81, 0xcb, 0x3a, 0xfa, 0x13, 0x08, 0xfe, 0x3f, 0x89, 0xdc, 0x46, 0xe7, 0x19, 0xd6, 0x5c, 0xc7, + 0x9b, 0x2b, 0x14, 0x92, 0x6c, 0xd2, 0xd4, 0xbb, 0x25, 0x79, 0x67, 0xb8, 0x4a, 0x9f, 0x73, 0x90, + 0xf2, 0x2f, 0x7b, 0x49, 0x16, 0xf9, 0xd7, 0xbd, 0x53, 0x59, 0xe4, 0x7f, 0xb2, 0x7c, 0x16, 0xf9, + 0x04, 0x5f, 0xbf, 0xee, 0x77, 0xfe, 0x4e, 0xfc, 0xed, 0xaa, 0xee, 0xd0, 0x99, 0xbc, 0x6d, 0x00, + 0x48, 0x6c, 0x56, 0x5b, 0xb5, 0xfa, 0x9e, 0xc0, 0x91, 0x17, 0x92, 0xd2, 0x68, 0x36, 0x6b, 0x92, + 0x10, 0x43, 0x79, 0xc8, 0x48, 0xb5, 0xdd, 0xda, 0x5e, 0xad, 0xdd, 0xa8, 0xef, 0x3e, 0x16, 0xe2, + 0xe8, 0x07, 0xf0, 0x3d, 0x6f, 0x61, 0xb3, 0x2e, 0xb5, 0x0f, 0x14, 0xd9, 0x7f, 0xc5, 0xa3, 0x35, + 0x80, 0xea, 0xe6, 0xd6, 0xce, 0xb6, 0xbc, 0xbb, 0x5b, 0x93, 0x84, 0x55, 0x94, 0x83, 0xf4, 0xc3, + 0xcd, 0x5d, 0x59, 0xda, 0xdc, 0xab, 0x49, 0x42, 0x82, 0x40, 0xed, 0x3d, 0xaa, 0xb7, 0x7d, 0xec, + 0x24, 0x21, 0x6d, 0xee, 0x57, 0x77, 0xe5, 0x2d, 0x21, 0x55, 0xbd, 0x79, 0xfa, 0x41, 0x61, 0xe5, + 0x74, 0x5c, 0xe0, 0xce, 0xc6, 0x05, 0xee, 0xf5, 0xb8, 0xc0, 0xfd, 0x7f, 0x5c, 0xe0, 0xfe, 0xf6, + 0xa6, 0xb0, 0x72, 0xf6, 0xa6, 0xb0, 0xf2, 0xfa, 0x4d, 0x61, 0xe5, 0x09, 0x4f, 0xb4, 0xea, 0x24, + 0xe8, 0xbd, 0xfb, 0xbd, 0x2f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xc8, 0xdd, 0x27, 0xd1, 0xf6, 0x1d, + 0x00, 0x00, } func (this *Column) Equal(that interface{}) bool { @@ -1246,62 +1036,10 @@ func (this *PrimaryIndex) Equal(that interface{}) bool { if this.TableID != that1.TableID { return false } - if this.IndexId != that1.IndexId { - return false - } - if this.IndexName != that1.IndexName { - return false - } - if this.Unique != that1.Unique { - return false - } - if len(this.KeyColumnIDs) != len(that1.KeyColumnIDs) { - return false - } - for i := range this.KeyColumnIDs { - if this.KeyColumnIDs[i] != that1.KeyColumnIDs[i] { - return false - } - } - if len(this.KeyColumnDirections) != len(that1.KeyColumnDirections) { - return false - } - for i := range this.KeyColumnDirections { - if this.KeyColumnDirections[i] != that1.KeyColumnDirections[i] { - return false - } - } - if len(this.KeySuffixColumnIDs) != len(that1.KeySuffixColumnIDs) { - return false - } - for i := range this.KeySuffixColumnIDs { - if this.KeySuffixColumnIDs[i] != that1.KeySuffixColumnIDs[i] { - return false - } - } - if !this.ShardedDescriptor.Equal(that1.ShardedDescriptor) { - return false - } - if len(this.StoringColumnIDs) != len(that1.StoringColumnIDs) { - return false - } - for i := range this.StoringColumnIDs { - if this.StoringColumnIDs[i] != that1.StoringColumnIDs[i] { - return false - } - } - if len(this.CompositeColumnIDs) != len(that1.CompositeColumnIDs) { - return false - } - for i := range this.CompositeColumnIDs { - if this.CompositeColumnIDs[i] != that1.CompositeColumnIDs[i] { - return false - } - } - if this.Inverted != that1.Inverted { + if !this.Index.Equal(&that1.Index) { return false } - if this.Concurrently != that1.Concurrently { + if this.OtherPrimaryIndexID != that1.OtherPrimaryIndexID { return false } return true @@ -1328,62 +1066,10 @@ func (this *SecondaryIndex) Equal(that interface{}) bool { if this.TableID != that1.TableID { return false } - if this.IndexId != that1.IndexId { - return false - } - if this.IndexName != that1.IndexName { - return false - } - if this.Unique != that1.Unique { - return false - } - if len(this.KeyColumnIDs) != len(that1.KeyColumnIDs) { - return false - } - for i := range this.KeyColumnIDs { - if this.KeyColumnIDs[i] != that1.KeyColumnIDs[i] { - return false - } - } - if len(this.KeyColumnDirections) != len(that1.KeyColumnDirections) { - return false - } - for i := range this.KeyColumnDirections { - if this.KeyColumnDirections[i] != that1.KeyColumnDirections[i] { - return false - } - } - if len(this.KeySuffixColumnIDs) != len(that1.KeySuffixColumnIDs) { - return false - } - for i := range this.KeySuffixColumnIDs { - if this.KeySuffixColumnIDs[i] != that1.KeySuffixColumnIDs[i] { - return false - } - } - if !this.ShardedDescriptor.Equal(that1.ShardedDescriptor) { - return false - } - if len(this.StoringColumnIDs) != len(that1.StoringColumnIDs) { - return false - } - for i := range this.StoringColumnIDs { - if this.StoringColumnIDs[i] != that1.StoringColumnIDs[i] { - return false - } - } - if len(this.CompositeColumnIDs) != len(that1.CompositeColumnIDs) { - return false - } - for i := range this.CompositeColumnIDs { - if this.CompositeColumnIDs[i] != that1.CompositeColumnIDs[i] { - return false - } - } - if this.Inverted != that1.Inverted { + if !this.Index.Equal(&that1.Index) { return false } - if this.Concurrently != that1.Concurrently { + if this.PrimaryIndex != that1.PrimaryIndex { return false } return true @@ -1777,129 +1463,6 @@ func (this *RelationDependedOnBy) Equal(that interface{}) bool { } return true } -func (this *ListPartition) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*ListPartition) - if !ok { - that2, ok := that.(ListPartition) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.Name != that1.Name { - return false - } - if len(this.Expr) != len(that1.Expr) { - return false - } - for i := range this.Expr { - if this.Expr[i] != that1.Expr[i] { - return false - } - } - return true -} -func (this *RangePartitions) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*RangePartitions) - if !ok { - that2, ok := that.(RangePartitions) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.Name != that1.Name { - return false - } - if len(this.To) != len(that1.To) { - return false - } - for i := range this.To { - if this.To[i] != that1.To[i] { - return false - } - } - if len(this.From) != len(that1.From) { - return false - } - for i := range this.From { - if this.From[i] != that1.From[i] { - return false - } - } - return true -} -func (this *Partitioning) Equal(that interface{}) bool { - if that == nil { - return this == nil - } - - that1, ok := that.(*Partitioning) - if !ok { - that2, ok := that.(Partitioning) - if ok { - that1 = &that2 - } else { - return false - } - } - if that1 == nil { - return this == nil - } else if this == nil { - return false - } - if this.TableID != that1.TableID { - return false - } - if this.IndexId != that1.IndexId { - return false - } - if len(this.Fields) != len(that1.Fields) { - return false - } - for i := range this.Fields { - if this.Fields[i] != that1.Fields[i] { - return false - } - } - if len(this.ListPartitions) != len(that1.ListPartitions) { - return false - } - for i := range this.ListPartitions { - if !this.ListPartitions[i].Equal(that1.ListPartitions[i]) { - return false - } - } - if len(this.RangePartitions) != len(that1.RangePartitions) { - return false - } - for i := range this.RangePartitions { - if !this.RangePartitions[i].Equal(that1.RangePartitions[i]) { - return false - } - } - return true -} func (m *ElementProto) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1920,20 +1483,6 @@ func (m *ElementProto) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.Partitioning != nil { - { - size, err := m.Partitioning.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintScpb(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x1 - i-- - dAtA[i] = 0x9a - } if m.Database != nil { { size, err := m.Database.MarshalToSizedBuffer(dAtA[:i]) @@ -2267,150 +1816,21 @@ func (m *PrimaryIndex) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.Concurrently { - i-- - if m.Concurrently { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } + if m.OtherPrimaryIndexID != 0 { + i = encodeVarintScpb(dAtA, i, uint64(m.OtherPrimaryIndexID)) i-- - dAtA[i] = 0x60 + dAtA[i] = 0x18 } - if m.Inverted { - i-- - if m.Inverted { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x58 - } - if len(m.CompositeColumnIDs) > 0 { - dAtA23 := make([]byte, len(m.CompositeColumnIDs)*10) - var j22 int - for _, num := range m.CompositeColumnIDs { - for num >= 1<<7 { - dAtA23[j22] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j22++ - } - dAtA23[j22] = uint8(num) - j22++ - } - i -= j22 - copy(dAtA[i:], dAtA23[:j22]) - i = encodeVarintScpb(dAtA, i, uint64(j22)) - i-- - dAtA[i] = 0x52 - } - if len(m.StoringColumnIDs) > 0 { - dAtA25 := make([]byte, len(m.StoringColumnIDs)*10) - var j24 int - for _, num := range m.StoringColumnIDs { - for num >= 1<<7 { - dAtA25[j24] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j24++ - } - dAtA25[j24] = uint8(num) - j24++ - } - i -= j24 - copy(dAtA[i:], dAtA25[:j24]) - i = encodeVarintScpb(dAtA, i, uint64(j24)) - i-- - dAtA[i] = 0x4a - } - if m.ShardedDescriptor != nil { - { - size, err := m.ShardedDescriptor.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintScpb(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x42 - } - if len(m.KeySuffixColumnIDs) > 0 { - dAtA28 := make([]byte, len(m.KeySuffixColumnIDs)*10) - var j27 int - for _, num := range m.KeySuffixColumnIDs { - for num >= 1<<7 { - dAtA28[j27] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j27++ - } - dAtA28[j27] = uint8(num) - j27++ - } - i -= j27 - copy(dAtA[i:], dAtA28[:j27]) - i = encodeVarintScpb(dAtA, i, uint64(j27)) - i-- - dAtA[i] = 0x3a - } - if len(m.KeyColumnDirections) > 0 { - dAtA30 := make([]byte, len(m.KeyColumnDirections)*10) - var j29 int - for _, num := range m.KeyColumnDirections { - for num >= 1<<7 { - dAtA30[j29] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j29++ - } - dAtA30[j29] = uint8(num) - j29++ - } - i -= j29 - copy(dAtA[i:], dAtA30[:j29]) - i = encodeVarintScpb(dAtA, i, uint64(j29)) - i-- - dAtA[i] = 0x32 - } - if len(m.KeyColumnIDs) > 0 { - dAtA32 := make([]byte, len(m.KeyColumnIDs)*10) - var j31 int - for _, num := range m.KeyColumnIDs { - for num >= 1<<7 { - dAtA32[j31] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j31++ - } - dAtA32[j31] = uint8(num) - j31++ - } - i -= j31 - copy(dAtA[i:], dAtA32[:j31]) - i = encodeVarintScpb(dAtA, i, uint64(j31)) - i-- - dAtA[i] = 0x2a - } - if m.Unique { - i-- - if m.Unique { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + { + size, err := m.Index.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err } - i-- - dAtA[i] = 0x20 - } - if len(m.IndexName) > 0 { - i -= len(m.IndexName) - copy(dAtA[i:], m.IndexName) - i = encodeVarintScpb(dAtA, i, uint64(len(m.IndexName))) - i-- - dAtA[i] = 0x1a - } - if m.IndexId != 0 { - i = encodeVarintScpb(dAtA, i, uint64(m.IndexId)) - i-- - dAtA[i] = 0x10 + i -= size + i = encodeVarintScpb(dAtA, i, uint64(size)) } + i-- + dAtA[i] = 0x12 if m.TableID != 0 { i = encodeVarintScpb(dAtA, i, uint64(m.TableID)) i-- @@ -2439,150 +1859,21 @@ func (m *SecondaryIndex) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if m.Concurrently { - i-- - if m.Concurrently { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x60 - } - if m.Inverted { - i-- - if m.Inverted { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x58 - } - if len(m.CompositeColumnIDs) > 0 { - dAtA34 := make([]byte, len(m.CompositeColumnIDs)*10) - var j33 int - for _, num := range m.CompositeColumnIDs { - for num >= 1<<7 { - dAtA34[j33] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j33++ - } - dAtA34[j33] = uint8(num) - j33++ - } - i -= j33 - copy(dAtA[i:], dAtA34[:j33]) - i = encodeVarintScpb(dAtA, i, uint64(j33)) - i-- - dAtA[i] = 0x52 - } - if len(m.StoringColumnIDs) > 0 { - dAtA36 := make([]byte, len(m.StoringColumnIDs)*10) - var j35 int - for _, num := range m.StoringColumnIDs { - for num >= 1<<7 { - dAtA36[j35] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j35++ - } - dAtA36[j35] = uint8(num) - j35++ - } - i -= j35 - copy(dAtA[i:], dAtA36[:j35]) - i = encodeVarintScpb(dAtA, i, uint64(j35)) - i-- - dAtA[i] = 0x4a - } - if m.ShardedDescriptor != nil { - { - size, err := m.ShardedDescriptor.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintScpb(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x42 - } - if len(m.KeySuffixColumnIDs) > 0 { - dAtA39 := make([]byte, len(m.KeySuffixColumnIDs)*10) - var j38 int - for _, num := range m.KeySuffixColumnIDs { - for num >= 1<<7 { - dAtA39[j38] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j38++ - } - dAtA39[j38] = uint8(num) - j38++ - } - i -= j38 - copy(dAtA[i:], dAtA39[:j38]) - i = encodeVarintScpb(dAtA, i, uint64(j38)) - i-- - dAtA[i] = 0x3a - } - if len(m.KeyColumnDirections) > 0 { - dAtA41 := make([]byte, len(m.KeyColumnDirections)*10) - var j40 int - for _, num := range m.KeyColumnDirections { - for num >= 1<<7 { - dAtA41[j40] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j40++ - } - dAtA41[j40] = uint8(num) - j40++ - } - i -= j40 - copy(dAtA[i:], dAtA41[:j40]) - i = encodeVarintScpb(dAtA, i, uint64(j40)) - i-- - dAtA[i] = 0x32 - } - if len(m.KeyColumnIDs) > 0 { - dAtA43 := make([]byte, len(m.KeyColumnIDs)*10) - var j42 int - for _, num := range m.KeyColumnIDs { - for num >= 1<<7 { - dAtA43[j42] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 - j42++ - } - dAtA43[j42] = uint8(num) - j42++ - } - i -= j42 - copy(dAtA[i:], dAtA43[:j42]) - i = encodeVarintScpb(dAtA, i, uint64(j42)) + if m.PrimaryIndex != 0 { + i = encodeVarintScpb(dAtA, i, uint64(m.PrimaryIndex)) i-- - dAtA[i] = 0x2a + dAtA[i] = 0x18 } - if m.Unique { - i-- - if m.Unique { - dAtA[i] = 1 - } else { - dAtA[i] = 0 + { + size, err := m.Index.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err } - i-- - dAtA[i] = 0x20 - } - if len(m.IndexName) > 0 { - i -= len(m.IndexName) - copy(dAtA[i:], m.IndexName) - i = encodeVarintScpb(dAtA, i, uint64(len(m.IndexName))) - i-- - dAtA[i] = 0x1a - } - if m.IndexId != 0 { - i = encodeVarintScpb(dAtA, i, uint64(m.IndexId)) - i-- - dAtA[i] = 0x10 + i -= size + i = encodeVarintScpb(dAtA, i, uint64(size)) } + i-- + dAtA[i] = 0x12 if m.TableID != 0 { i = encodeVarintScpb(dAtA, i, uint64(m.TableID)) i-- @@ -2655,20 +1946,20 @@ func (m *UniqueConstraint) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if len(m.ColumnIDs) > 0 { - dAtA45 := make([]byte, len(m.ColumnIDs)*10) - var j44 int + dAtA24 := make([]byte, len(m.ColumnIDs)*10) + var j23 int for _, num := range m.ColumnIDs { for num >= 1<<7 { - dAtA45[j44] = uint8(uint64(num)&0x7f | 0x80) + dAtA24[j23] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j44++ + j23++ } - dAtA45[j44] = uint8(num) - j44++ + dAtA24[j23] = uint8(num) + j23++ } - i -= j44 - copy(dAtA[i:], dAtA45[:j44]) - i = encodeVarintScpb(dAtA, i, uint64(j44)) + i -= j23 + copy(dAtA[i:], dAtA24[:j23]) + i = encodeVarintScpb(dAtA, i, uint64(j23)) i-- dAtA[i] = 0x1a } @@ -2716,20 +2007,20 @@ func (m *CheckConstraint) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x28 } if len(m.ColumnIDs) > 0 { - dAtA47 := make([]byte, len(m.ColumnIDs)*10) - var j46 int + dAtA26 := make([]byte, len(m.ColumnIDs)*10) + var j25 int for _, num := range m.ColumnIDs { for num >= 1<<7 { - dAtA47[j46] = uint8(uint64(num)&0x7f | 0x80) + dAtA26[j25] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j46++ + j25++ } - dAtA47[j46] = uint8(num) - j46++ + dAtA26[j25] = uint8(num) + j25++ } - i -= j46 - copy(dAtA[i:], dAtA47[:j46]) - i = encodeVarintScpb(dAtA, i, uint64(j46)) + i -= j25 + copy(dAtA[i:], dAtA26[:j25]) + i = encodeVarintScpb(dAtA, i, uint64(j25)) i-- dAtA[i] = 0x22 } @@ -2811,20 +2102,20 @@ func (m *DefaultExpression) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x22 } if len(m.UsesSequenceIDs) > 0 { - dAtA49 := make([]byte, len(m.UsesSequenceIDs)*10) - var j48 int + dAtA28 := make([]byte, len(m.UsesSequenceIDs)*10) + var j27 int for _, num := range m.UsesSequenceIDs { for num >= 1<<7 { - dAtA49[j48] = uint8(uint64(num)&0x7f | 0x80) + dAtA28[j27] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j48++ + j27++ } - dAtA49[j48] = uint8(num) - j48++ + dAtA28[j27] = uint8(num) + j27++ } - i -= j48 - copy(dAtA[i:], dAtA49[:j48]) - i = encodeVarintScpb(dAtA, i, uint64(j48)) + i -= j27 + copy(dAtA[i:], dAtA28[:j27]) + i = encodeVarintScpb(dAtA, i, uint64(j27)) i-- dAtA[i] = 0x1a } @@ -2862,38 +2153,38 @@ func (m *View) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if len(m.DependsOn) > 0 { - dAtA51 := make([]byte, len(m.DependsOn)*10) - var j50 int + dAtA30 := make([]byte, len(m.DependsOn)*10) + var j29 int for _, num := range m.DependsOn { for num >= 1<<7 { - dAtA51[j50] = uint8(uint64(num)&0x7f | 0x80) + dAtA30[j29] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j50++ + j29++ } - dAtA51[j50] = uint8(num) - j50++ + dAtA30[j29] = uint8(num) + j29++ } - i -= j50 - copy(dAtA[i:], dAtA51[:j50]) - i = encodeVarintScpb(dAtA, i, uint64(j50)) + i -= j29 + copy(dAtA[i:], dAtA30[:j29]) + i = encodeVarintScpb(dAtA, i, uint64(j29)) i-- dAtA[i] = 0x1a } if len(m.DependedOnBy) > 0 { - dAtA53 := make([]byte, len(m.DependedOnBy)*10) - var j52 int + dAtA32 := make([]byte, len(m.DependedOnBy)*10) + var j31 int for _, num := range m.DependedOnBy { for num >= 1<<7 { - dAtA53[j52] = uint8(uint64(num)&0x7f | 0x80) + dAtA32[j31] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j52++ + j31++ } - dAtA53[j52] = uint8(num) - j52++ + dAtA32[j31] = uint8(num) + j31++ } - i -= j52 - copy(dAtA[i:], dAtA53[:j52]) - i = encodeVarintScpb(dAtA, i, uint64(j52)) + i -= j31 + copy(dAtA[i:], dAtA32[:j31]) + i = encodeVarintScpb(dAtA, i, uint64(j31)) i-- dAtA[i] = 0x12 } @@ -2926,20 +2217,20 @@ func (m *Table) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if len(m.DependedOnBy) > 0 { - dAtA55 := make([]byte, len(m.DependedOnBy)*10) - var j54 int + dAtA34 := make([]byte, len(m.DependedOnBy)*10) + var j33 int for _, num := range m.DependedOnBy { for num >= 1<<7 { - dAtA55[j54] = uint8(uint64(num)&0x7f | 0x80) + dAtA34[j33] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j54++ + j33++ } - dAtA55[j54] = uint8(num) - j54++ + dAtA34[j33] = uint8(num) + j33++ } - i -= j54 - copy(dAtA[i:], dAtA55[:j54]) - i = encodeVarintScpb(dAtA, i, uint64(j54)) + i -= j33 + copy(dAtA[i:], dAtA34[:j33]) + i = encodeVarintScpb(dAtA, i, uint64(j33)) i-- dAtA[i] = 0x12 } @@ -3012,20 +2303,20 @@ func (m *OutboundForeignKey) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x32 } if len(m.ReferenceColumns) > 0 { - dAtA57 := make([]byte, len(m.ReferenceColumns)*10) - var j56 int + dAtA36 := make([]byte, len(m.ReferenceColumns)*10) + var j35 int for _, num := range m.ReferenceColumns { for num >= 1<<7 { - dAtA57[j56] = uint8(uint64(num)&0x7f | 0x80) + dAtA36[j35] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j56++ + j35++ } - dAtA57[j56] = uint8(num) - j56++ + dAtA36[j35] = uint8(num) + j35++ } - i -= j56 - copy(dAtA[i:], dAtA57[:j56]) - i = encodeVarintScpb(dAtA, i, uint64(j56)) + i -= j35 + copy(dAtA[i:], dAtA36[:j35]) + i = encodeVarintScpb(dAtA, i, uint64(j35)) i-- dAtA[i] = 0x2a } @@ -3035,20 +2326,20 @@ func (m *OutboundForeignKey) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x20 } if len(m.OriginColumns) > 0 { - dAtA59 := make([]byte, len(m.OriginColumns)*10) - var j58 int + dAtA38 := make([]byte, len(m.OriginColumns)*10) + var j37 int for _, num := range m.OriginColumns { for num >= 1<<7 { - dAtA59[j58] = uint8(uint64(num)&0x7f | 0x80) + dAtA38[j37] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j58++ + j37++ } - dAtA59[j58] = uint8(num) - j58++ + dAtA38[j37] = uint8(num) + j37++ } - i -= j58 - copy(dAtA[i:], dAtA59[:j58]) - i = encodeVarintScpb(dAtA, i, uint64(j58)) + i -= j37 + copy(dAtA[i:], dAtA38[:j37]) + i = encodeVarintScpb(dAtA, i, uint64(j37)) i-- dAtA[i] = 0x1a } @@ -3088,20 +2379,20 @@ func (m *InboundForeignKey) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x32 } if len(m.ReferenceColumns) > 0 { - dAtA61 := make([]byte, len(m.ReferenceColumns)*10) - var j60 int + dAtA40 := make([]byte, len(m.ReferenceColumns)*10) + var j39 int for _, num := range m.ReferenceColumns { for num >= 1<<7 { - dAtA61[j60] = uint8(uint64(num)&0x7f | 0x80) + dAtA40[j39] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j60++ + j39++ } - dAtA61[j60] = uint8(num) - j60++ + dAtA40[j39] = uint8(num) + j39++ } - i -= j60 - copy(dAtA[i:], dAtA61[:j60]) - i = encodeVarintScpb(dAtA, i, uint64(j60)) + i -= j39 + copy(dAtA[i:], dAtA40[:j39]) + i = encodeVarintScpb(dAtA, i, uint64(j39)) i-- dAtA[i] = 0x2a } @@ -3111,20 +2402,20 @@ func (m *InboundForeignKey) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x20 } if len(m.OriginColumns) > 0 { - dAtA63 := make([]byte, len(m.OriginColumns)*10) - var j62 int + dAtA42 := make([]byte, len(m.OriginColumns)*10) + var j41 int for _, num := range m.OriginColumns { for num >= 1<<7 { - dAtA63[j62] = uint8(uint64(num)&0x7f | 0x80) + dAtA42[j41] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j62++ + j41++ } - dAtA63[j62] = uint8(num) - j62++ + dAtA42[j41] = uint8(num) + j41++ } - i -= j62 - copy(dAtA[i:], dAtA63[:j62]) - i = encodeVarintScpb(dAtA, i, uint64(j62)) + i -= j41 + copy(dAtA[i:], dAtA42[:j41]) + i = encodeVarintScpb(dAtA, i, uint64(j41)) i-- dAtA[i] = 0x1a } @@ -3251,20 +2542,20 @@ func (m *Schema) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if len(m.DependentObjects) > 0 { - dAtA65 := make([]byte, len(m.DependentObjects)*10) - var j64 int + dAtA44 := make([]byte, len(m.DependentObjects)*10) + var j43 int for _, num := range m.DependentObjects { for num >= 1<<7 { - dAtA65[j64] = uint8(uint64(num)&0x7f | 0x80) + dAtA44[j43] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j64++ + j43++ } - dAtA65[j64] = uint8(num) - j64++ + dAtA44[j43] = uint8(num) + j43++ } - i -= j64 - copy(dAtA[i:], dAtA65[:j64]) - i = encodeVarintScpb(dAtA, i, uint64(j64)) + i -= j43 + copy(dAtA[i:], dAtA44[:j43]) + i = encodeVarintScpb(dAtA, i, uint64(j43)) i-- dAtA[i] = 0x12 } @@ -3297,20 +2588,20 @@ func (m *Database) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if len(m.DependentObjects) > 0 { - dAtA67 := make([]byte, len(m.DependentObjects)*10) - var j66 int + dAtA46 := make([]byte, len(m.DependentObjects)*10) + var j45 int for _, num := range m.DependentObjects { for num >= 1<<7 { - dAtA67[j66] = uint8(uint64(num)&0x7f | 0x80) + dAtA46[j45] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j66++ + j45++ } - dAtA67[j66] = uint8(num) - j66++ + dAtA46[j45] = uint8(num) + j45++ } - i -= j66 - copy(dAtA[i:], dAtA67[:j66]) - i = encodeVarintScpb(dAtA, i, uint64(j66)) + i -= j45 + copy(dAtA[i:], dAtA46[:j45]) + i = encodeVarintScpb(dAtA, i, uint64(j45)) i-- dAtA[i] = 0x12 } @@ -3322,178 +2613,21 @@ func (m *Database) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *ListPartition) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err +func encodeVarintScpb(dAtA []byte, offset int, v uint64) int { + offset -= sovScpb(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ } - return dAtA[:n], nil -} - -func (m *ListPartition) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) + dAtA[offset] = uint8(v) + return base } - -func (m *ListPartition) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.Expr) > 0 { - for iNdEx := len(m.Expr) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Expr[iNdEx]) - copy(dAtA[i:], m.Expr[iNdEx]) - i = encodeVarintScpb(dAtA, i, uint64(len(m.Expr[iNdEx]))) - i-- - dAtA[i] = 0x12 - } - } - if len(m.Name) > 0 { - i -= len(m.Name) - copy(dAtA[i:], m.Name) - i = encodeVarintScpb(dAtA, i, uint64(len(m.Name))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *RangePartitions) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *RangePartitions) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *RangePartitions) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.From) > 0 { - for iNdEx := len(m.From) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.From[iNdEx]) - copy(dAtA[i:], m.From[iNdEx]) - i = encodeVarintScpb(dAtA, i, uint64(len(m.From[iNdEx]))) - i-- - dAtA[i] = 0x1a - } - } - if len(m.To) > 0 { - for iNdEx := len(m.To) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.To[iNdEx]) - copy(dAtA[i:], m.To[iNdEx]) - i = encodeVarintScpb(dAtA, i, uint64(len(m.To[iNdEx]))) - i-- - dAtA[i] = 0x12 - } - } - if len(m.Name) > 0 { - i -= len(m.Name) - copy(dAtA[i:], m.Name) - i = encodeVarintScpb(dAtA, i, uint64(len(m.Name))) - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} - -func (m *Partitioning) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *Partitioning) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *Partitioning) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if len(m.RangePartitions) > 0 { - for iNdEx := len(m.RangePartitions) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.RangePartitions[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintScpb(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x2a - } - } - if len(m.ListPartitions) > 0 { - for iNdEx := len(m.ListPartitions) - 1; iNdEx >= 0; iNdEx-- { - { - size, err := m.ListPartitions[iNdEx].MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintScpb(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x22 - } - } - if len(m.Fields) > 0 { - for iNdEx := len(m.Fields) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Fields[iNdEx]) - copy(dAtA[i:], m.Fields[iNdEx]) - i = encodeVarintScpb(dAtA, i, uint64(len(m.Fields[iNdEx]))) - i-- - dAtA[i] = 0x1a - } - } - if m.IndexId != 0 { - i = encodeVarintScpb(dAtA, i, uint64(m.IndexId)) - i-- - dAtA[i] = 0x10 - } - if m.TableID != 0 { - i = encodeVarintScpb(dAtA, i, uint64(m.TableID)) - i-- - dAtA[i] = 0x8 - } - return len(dAtA) - i, nil -} - -func encodeVarintScpb(dAtA []byte, offset int, v uint64) int { - offset -= sovScpb(v) - base := offset - for v >= 1<<7 { - dAtA[offset] = uint8(v&0x7f | 0x80) - v >>= 7 - offset++ - } - dAtA[offset] = uint8(v) - return base -} -func (m *ElementProto) Size() (n int) { - if m == nil { - return 0 - } +func (m *ElementProto) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Column != nil { @@ -3568,10 +2702,6 @@ func (m *ElementProto) Size() (n int) { l = m.Database.Size() n += 2 + l + sovScpb(uint64(l)) } - if m.Partitioning != nil { - l = m.Partitioning.Size() - n += 2 + l + sovScpb(uint64(l)) - } return n } @@ -3619,60 +2749,10 @@ func (m *PrimaryIndex) Size() (n int) { if m.TableID != 0 { n += 1 + sovScpb(uint64(m.TableID)) } - if m.IndexId != 0 { - n += 1 + sovScpb(uint64(m.IndexId)) - } - l = len(m.IndexName) - if l > 0 { - n += 1 + l + sovScpb(uint64(l)) - } - if m.Unique { - n += 2 - } - if len(m.KeyColumnIDs) > 0 { - l = 0 - for _, e := range m.KeyColumnIDs { - l += sovScpb(uint64(e)) - } - n += 1 + sovScpb(uint64(l)) + l - } - if len(m.KeyColumnDirections) > 0 { - l = 0 - for _, e := range m.KeyColumnDirections { - l += sovScpb(uint64(e)) - } - n += 1 + sovScpb(uint64(l)) + l - } - if len(m.KeySuffixColumnIDs) > 0 { - l = 0 - for _, e := range m.KeySuffixColumnIDs { - l += sovScpb(uint64(e)) - } - n += 1 + sovScpb(uint64(l)) + l - } - if m.ShardedDescriptor != nil { - l = m.ShardedDescriptor.Size() - n += 1 + l + sovScpb(uint64(l)) - } - if len(m.StoringColumnIDs) > 0 { - l = 0 - for _, e := range m.StoringColumnIDs { - l += sovScpb(uint64(e)) - } - n += 1 + sovScpb(uint64(l)) + l - } - if len(m.CompositeColumnIDs) > 0 { - l = 0 - for _, e := range m.CompositeColumnIDs { - l += sovScpb(uint64(e)) - } - n += 1 + sovScpb(uint64(l)) + l - } - if m.Inverted { - n += 2 - } - if m.Concurrently { - n += 2 + l = m.Index.Size() + n += 1 + l + sovScpb(uint64(l)) + if m.OtherPrimaryIndexID != 0 { + n += 1 + sovScpb(uint64(m.OtherPrimaryIndexID)) } return n } @@ -3686,60 +2766,10 @@ func (m *SecondaryIndex) Size() (n int) { if m.TableID != 0 { n += 1 + sovScpb(uint64(m.TableID)) } - if m.IndexId != 0 { - n += 1 + sovScpb(uint64(m.IndexId)) - } - l = len(m.IndexName) - if l > 0 { - n += 1 + l + sovScpb(uint64(l)) - } - if m.Unique { - n += 2 - } - if len(m.KeyColumnIDs) > 0 { - l = 0 - for _, e := range m.KeyColumnIDs { - l += sovScpb(uint64(e)) - } - n += 1 + sovScpb(uint64(l)) + l - } - if len(m.KeyColumnDirections) > 0 { - l = 0 - for _, e := range m.KeyColumnDirections { - l += sovScpb(uint64(e)) - } - n += 1 + sovScpb(uint64(l)) + l - } - if len(m.KeySuffixColumnIDs) > 0 { - l = 0 - for _, e := range m.KeySuffixColumnIDs { - l += sovScpb(uint64(e)) - } - n += 1 + sovScpb(uint64(l)) + l - } - if m.ShardedDescriptor != nil { - l = m.ShardedDescriptor.Size() - n += 1 + l + sovScpb(uint64(l)) - } - if len(m.StoringColumnIDs) > 0 { - l = 0 - for _, e := range m.StoringColumnIDs { - l += sovScpb(uint64(e)) - } - n += 1 + sovScpb(uint64(l)) + l - } - if len(m.CompositeColumnIDs) > 0 { - l = 0 - for _, e := range m.CompositeColumnIDs { - l += sovScpb(uint64(e)) - } - n += 1 + sovScpb(uint64(l)) + l - } - if m.Inverted { - n += 2 - } - if m.Concurrently { - n += 2 + l = m.Index.Size() + n += 1 + l + sovScpb(uint64(l)) + if m.PrimaryIndex != 0 { + n += 1 + sovScpb(uint64(m.PrimaryIndex)) } return n } @@ -4061,83 +3091,6 @@ func (m *Database) Size() (n int) { return n } -func (m *ListPartition) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Name) - if l > 0 { - n += 1 + l + sovScpb(uint64(l)) - } - if len(m.Expr) > 0 { - for _, s := range m.Expr { - l = len(s) - n += 1 + l + sovScpb(uint64(l)) - } - } - return n -} - -func (m *RangePartitions) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - l = len(m.Name) - if l > 0 { - n += 1 + l + sovScpb(uint64(l)) - } - if len(m.To) > 0 { - for _, s := range m.To { - l = len(s) - n += 1 + l + sovScpb(uint64(l)) - } - } - if len(m.From) > 0 { - for _, s := range m.From { - l = len(s) - n += 1 + l + sovScpb(uint64(l)) - } - } - return n -} - -func (m *Partitioning) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.TableID != 0 { - n += 1 + sovScpb(uint64(m.TableID)) - } - if m.IndexId != 0 { - n += 1 + sovScpb(uint64(m.IndexId)) - } - if len(m.Fields) > 0 { - for _, s := range m.Fields { - l = len(s) - n += 1 + l + sovScpb(uint64(l)) - } - } - if len(m.ListPartitions) > 0 { - for _, e := range m.ListPartitions { - l = e.Size() - n += 1 + l + sovScpb(uint64(l)) - } - } - if len(m.RangePartitions) > 0 { - for _, e := range m.RangePartitions { - l = e.Size() - n += 1 + l + sovScpb(uint64(l)) - } - } - return n -} - func sovScpb(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -4199,9 +3152,6 @@ func (this *ElementProto) GetValue() interface{} { if this.Database != nil { return this.Database } - if this.Partitioning != nil { - return this.Partitioning - } return nil } @@ -4243,8 +3193,6 @@ func (this *ElementProto) SetValue(value interface{}) bool { this.Schema = vt case *Database: this.Database = vt - case *Partitioning: - this.Partitioning = vt default: return false } @@ -4927,42 +3875,6 @@ func (m *ElementProto) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 19: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Partitioning", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Partitioning == nil { - m.Partitioning = &Partitioning{} - } - if err := m.Partitioning.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipScpb(dAtA[iNdEx:]) @@ -5288,10 +4200,10 @@ func (m *PrimaryIndex) Unmarshal(dAtA []byte) error { } } case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IndexId", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) } - m.IndexId = 0 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -5301,1605 +4213,30 @@ func (m *PrimaryIndex) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.IndexId |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field IndexName", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.IndexName = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Unique", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Unique = bool(v != 0) - case 5: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeyColumnIDs = append(m.KeyColumnIDs, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.KeyColumnIDs) == 0 { - m.KeyColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeyColumnIDs = append(m.KeyColumnIDs, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field KeyColumnIDs", wireType) - } - case 6: - if wireType == 0 { - var v PrimaryIndex_Direction - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= PrimaryIndex_Direction(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeyColumnDirections = append(m.KeyColumnDirections, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - if elementCount != 0 && len(m.KeyColumnDirections) == 0 { - m.KeyColumnDirections = make([]PrimaryIndex_Direction, 0, elementCount) - } - for iNdEx < postIndex { - var v PrimaryIndex_Direction - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= PrimaryIndex_Direction(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeyColumnDirections = append(m.KeyColumnDirections, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field KeyColumnDirections", wireType) - } - case 7: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeySuffixColumnIDs = append(m.KeySuffixColumnIDs, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.KeySuffixColumnIDs) == 0 { - m.KeySuffixColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeySuffixColumnIDs = append(m.KeySuffixColumnIDs, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field KeySuffixColumnIDs", wireType) - } - case 8: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ShardedDescriptor", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.ShardedDescriptor == nil { - m.ShardedDescriptor = &descpb.ShardedDescriptor{} - } - if err := m.ShardedDescriptor.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 9: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.StoringColumnIDs = append(m.StoringColumnIDs, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.StoringColumnIDs) == 0 { - m.StoringColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.StoringColumnIDs = append(m.StoringColumnIDs, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field StoringColumnIDs", wireType) - } - case 10: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.CompositeColumnIDs = append(m.CompositeColumnIDs, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.CompositeColumnIDs) == 0 { - m.CompositeColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.CompositeColumnIDs = append(m.CompositeColumnIDs, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field CompositeColumnIDs", wireType) - } - case 11: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Inverted", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Inverted = bool(v != 0) - case 12: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Concurrently", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Concurrently = bool(v != 0) - default: - iNdEx = preIndex - skippy, err := skipScpb(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthScpb - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *SecondaryIndex) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: SecondaryIndex: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: SecondaryIndex: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) - } - m.TableID = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IndexId", wireType) - } - m.IndexId = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.IndexId |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field IndexName", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.IndexName = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Unique", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Unique = bool(v != 0) - case 5: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeyColumnIDs = append(m.KeyColumnIDs, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.KeyColumnIDs) == 0 { - m.KeyColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeyColumnIDs = append(m.KeyColumnIDs, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field KeyColumnIDs", wireType) - } - case 6: - if wireType == 0 { - var v SecondaryIndex_Direction - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= SecondaryIndex_Direction(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeyColumnDirections = append(m.KeyColumnDirections, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - if elementCount != 0 && len(m.KeyColumnDirections) == 0 { - m.KeyColumnDirections = make([]SecondaryIndex_Direction, 0, elementCount) - } - for iNdEx < postIndex { - var v SecondaryIndex_Direction - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= SecondaryIndex_Direction(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeyColumnDirections = append(m.KeyColumnDirections, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field KeyColumnDirections", wireType) - } - case 7: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeySuffixColumnIDs = append(m.KeySuffixColumnIDs, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.KeySuffixColumnIDs) == 0 { - m.KeySuffixColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.KeySuffixColumnIDs = append(m.KeySuffixColumnIDs, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field KeySuffixColumnIDs", wireType) - } - case 8: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ShardedDescriptor", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.ShardedDescriptor == nil { - m.ShardedDescriptor = &descpb.ShardedDescriptor{} - } - if err := m.ShardedDescriptor.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 9: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.StoringColumnIDs = append(m.StoringColumnIDs, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.StoringColumnIDs) == 0 { - m.StoringColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.StoringColumnIDs = append(m.StoringColumnIDs, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field StoringColumnIDs", wireType) - } - case 10: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.CompositeColumnIDs = append(m.CompositeColumnIDs, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.CompositeColumnIDs) == 0 { - m.CompositeColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.CompositeColumnIDs = append(m.CompositeColumnIDs, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field CompositeColumnIDs", wireType) - } - case 11: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Inverted", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Inverted = bool(v != 0) - case 12: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Concurrently", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.Concurrently = bool(v != 0) - default: - iNdEx = preIndex - skippy, err := skipScpb(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthScpb - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *SequenceDependency) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: SequenceDependency: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: SequenceDependency: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) - } - m.TableID = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ColumnID", wireType) - } - m.ColumnID = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.ColumnID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SequenceID", wireType) - } - m.SequenceID = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.SequenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) - } - m.Type = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.Type |= SequenceDependency_Type(b&0x7F) << shift - if b < 0x80 { - break - } - } - default: - iNdEx = preIndex - skippy, err := skipScpb(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthScpb - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *UniqueConstraint) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: UniqueConstraint: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: UniqueConstraint: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) - } - m.TableID = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IndexID", wireType) - } - m.IndexID = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.IndexID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 3: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.ColumnIDs = append(m.ColumnIDs, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.ColumnIDs) == 0 { - m.ColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.ColumnIDs = append(m.ColumnIDs, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field ColumnIDs", wireType) - } - default: - iNdEx = preIndex - skippy, err := skipScpb(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthScpb - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *CheckConstraint) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: CheckConstraint: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: CheckConstraint: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) - } - m.TableID = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLengthScpb } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLengthScpb } if postIndex > l { return io.ErrUnexpectedEOF } - m.Expr = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.ColumnIDs = append(m.ColumnIDs, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.ColumnIDs) == 0 { - m.ColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.ColumnIDs = append(m.ColumnIDs, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field ColumnIDs", wireType) + if err := m.Index.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err } - case 5: + iNdEx = postIndex + case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Validated", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field OtherPrimaryIndexID", wireType) } - var v int + m.OtherPrimaryIndexID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -6909,12 +4246,11 @@ func (m *CheckConstraint) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.OtherPrimaryIndexID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b&0x7F) << shift if b < 0x80 { break } } - m.Validated = bool(v != 0) default: iNdEx = preIndex skippy, err := skipScpb(dAtA[iNdEx:]) @@ -6936,7 +4272,7 @@ func (m *CheckConstraint) Unmarshal(dAtA []byte) error { } return nil } -func (m *Sequence) Unmarshal(dAtA []byte) error { +func (m *SecondaryIndex) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -6959,17 +4295,17 @@ func (m *Sequence) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Sequence: wiretype end group for non-group") + return fmt.Errorf("proto: SecondaryIndex: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Sequence: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SecondaryIndex: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SequenceID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) } - m.SequenceID = 0 + m.TableID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -6979,7 +4315,59 @@ func (m *Sequence) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.SequenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthScpb + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Index.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryIndex", wireType) + } + m.PrimaryIndex = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.PrimaryIndex |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b&0x7F) << shift if b < 0x80 { break } @@ -7005,7 +4393,7 @@ func (m *Sequence) Unmarshal(dAtA []byte) error { } return nil } -func (m *DefaultExpression) Unmarshal(dAtA []byte) error { +func (m *SequenceDependency) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7028,10 +4416,10 @@ func (m *DefaultExpression) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DefaultExpression: wiretype end group for non-group") + return fmt.Errorf("proto: SequenceDependency: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DefaultExpression: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SequenceDependency: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -7073,86 +4461,29 @@ func (m *DefaultExpression) Unmarshal(dAtA []byte) error { } } case 3: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.UsesSequenceIDs = append(m.UsesSequenceIDs, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SequenceID", wireType) + } + m.SequenceID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb } - if postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.UsesSequenceIDs) == 0 { - m.UsesSequenceIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.UsesSequenceIDs = append(m.UsesSequenceIDs, v) + b := dAtA[iNdEx] + iNdEx++ + m.SequenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field UsesSequenceIDs", wireType) } case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DefaultExpr", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) } - var stringLen uint64 + m.Type = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7162,24 +4493,11 @@ func (m *DefaultExpression) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.Type |= SequenceDependency_Type(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.DefaultExpr = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipScpb(dAtA[iNdEx:]) @@ -7201,7 +4519,7 @@ func (m *DefaultExpression) Unmarshal(dAtA []byte) error { } return nil } -func (m *View) Unmarshal(dAtA []byte) error { +func (m *UniqueConstraint) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7224,10 +4542,10 @@ func (m *View) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: View: wiretype end group for non-group") + return fmt.Errorf("proto: UniqueConstraint: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: View: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UniqueConstraint: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -7250,84 +4568,27 @@ func (m *View) Unmarshal(dAtA []byte) error { } } case 2: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.DependedOnBy = append(m.DependedOnBy, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field IndexID", wireType) + } + m.IndexID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb } - if postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.DependedOnBy) == 0 { - m.DependedOnBy = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.DependedOnBy = append(m.DependedOnBy, v) + b := dAtA[iNdEx] + iNdEx++ + m.IndexID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b&0x7F) << shift + if b < 0x80 { + break } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field DependedOnBy", wireType) } case 3: if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7337,12 +4598,12 @@ func (m *View) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift if b < 0x80 { break } } - m.DependsOn = append(m.DependsOn, v) + m.ColumnIDs = append(m.ColumnIDs, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { @@ -7377,11 +4638,11 @@ func (m *View) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.DependsOn) == 0 { - m.DependsOn = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) + if elementCount != 0 && len(m.ColumnIDs) == 0 { + m.ColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) } for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7391,15 +4652,15 @@ func (m *View) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift if b < 0x80 { break } } - m.DependsOn = append(m.DependsOn, v) + m.ColumnIDs = append(m.ColumnIDs, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field DependsOn", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ColumnIDs", wireType) } default: iNdEx = preIndex @@ -7422,7 +4683,7 @@ func (m *View) Unmarshal(dAtA []byte) error { } return nil } -func (m *Table) Unmarshal(dAtA []byte) error { +func (m *CheckConstraint) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7445,10 +4706,10 @@ func (m *Table) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Table: wiretype end group for non-group") + return fmt.Errorf("proto: CheckConstraint: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Table: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: CheckConstraint: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: @@ -7471,8 +4732,72 @@ func (m *Table) Unmarshal(dAtA []byte) error { } } case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthScpb + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthScpb + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Expr = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7482,12 +4807,12 @@ func (m *Table) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift if b < 0x80 { break } } - m.DependedOnBy = append(m.DependedOnBy, v) + m.ColumnIDs = append(m.ColumnIDs, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { @@ -7522,11 +4847,11 @@ func (m *Table) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.DependedOnBy) == 0 { - m.DependedOnBy = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) + if elementCount != 0 && len(m.ColumnIDs) == 0 { + m.ColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) } for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7536,16 +4861,36 @@ func (m *Table) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift if b < 0x80 { break } } - m.DependedOnBy = append(m.DependedOnBy, v) + m.ColumnIDs = append(m.ColumnIDs, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field DependedOnBy", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ColumnIDs", wireType) + } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Validated", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } } + m.Validated = bool(v != 0) default: iNdEx = preIndex skippy, err := skipScpb(dAtA[iNdEx:]) @@ -7567,7 +4912,7 @@ func (m *Table) Unmarshal(dAtA []byte) error { } return nil } -func (m *TypeReference) Unmarshal(dAtA []byte) error { +func (m *Sequence) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7590,36 +4935,17 @@ func (m *TypeReference) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: TypeReference: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: TypeReference: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Sequence: wiretype end group for non-group") } - switch fieldNum { - case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TypeID", wireType) - } - m.TypeID = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.TypeID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 2: + if fieldNum <= 0 { + return fmt.Errorf("proto: Sequence: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DescID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SequenceID", wireType) } - m.DescID = 0 + m.SequenceID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7629,7 +4955,7 @@ func (m *TypeReference) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.DescID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.SequenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } @@ -7655,7 +4981,7 @@ func (m *TypeReference) Unmarshal(dAtA []byte) error { } return nil } -func (m *OutboundForeignKey) Unmarshal(dAtA []byte) error { +func (m *DefaultExpression) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7678,17 +5004,17 @@ func (m *OutboundForeignKey) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: OutboundForeignKey: wiretype end group for non-group") + return fmt.Errorf("proto: DefaultExpression: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: OutboundForeignKey: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: DefaultExpression: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field OriginID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) } - m.OriginID = 0 + m.TableID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7698,92 +5024,16 @@ func (m *OutboundForeignKey) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.OriginID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - case 3: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.OriginColumns = append(m.OriginColumns, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.OriginColumns) == 0 { - m.OriginColumns = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.OriginColumns = append(m.OriginColumns, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field OriginColumns", wireType) - } - case 4: + case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ReferenceID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ColumnID", wireType) } - m.ReferenceID = 0 + m.ColumnID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7793,14 +5043,14 @@ func (m *OutboundForeignKey) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.ReferenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.ColumnID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift if b < 0x80 { break } } - case 5: + case 3: if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7810,12 +5060,12 @@ func (m *OutboundForeignKey) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - m.ReferenceColumns = append(m.ReferenceColumns, v) + m.UsesSequenceIDs = append(m.UsesSequenceIDs, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { @@ -7850,11 +5100,11 @@ func (m *OutboundForeignKey) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.ReferenceColumns) == 0 { - m.ReferenceColumns = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) + if elementCount != 0 && len(m.UsesSequenceIDs) == 0 { + m.UsesSequenceIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) } for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7864,19 +5114,19 @@ func (m *OutboundForeignKey) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - m.ReferenceColumns = append(m.ReferenceColumns, v) + m.UsesSequenceIDs = append(m.UsesSequenceIDs, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field ReferenceColumns", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field UsesSequenceIDs", wireType) } - case 6: + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DefaultExpr", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -7904,7 +5154,7 @@ func (m *OutboundForeignKey) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) + m.DefaultExpr = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -7927,7 +5177,7 @@ func (m *OutboundForeignKey) Unmarshal(dAtA []byte) error { } return nil } -func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { +func (m *View) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -7950,17 +5200,17 @@ func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: InboundForeignKey: wiretype end group for non-group") + return fmt.Errorf("proto: View: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: InboundForeignKey: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: View: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field OriginID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) } - m.OriginID = 0 + m.TableID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7970,14 +5220,90 @@ func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.OriginID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } + case 2: + if wireType == 0 { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.DependedOnBy = append(m.DependedOnBy, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthScpb + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.DependedOnBy) == 0 { + m.DependedOnBy = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) + } + for iNdEx < postIndex { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.DependedOnBy = append(m.DependedOnBy, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field DependedOnBy", wireType) + } case 3: if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -7987,12 +5313,12 @@ func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - m.OriginColumns = append(m.OriginColumns, v) + m.DependsOn = append(m.DependsOn, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { @@ -8027,11 +5353,11 @@ func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.OriginColumns) == 0 { - m.OriginColumns = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) + if elementCount != 0 && len(m.DependsOn) == 0 { + m.DependsOn = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) } for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8041,21 +5367,71 @@ func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - m.OriginColumns = append(m.OriginColumns, v) + m.DependsOn = append(m.DependsOn, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field OriginColumns", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DependsOn", wireType) + } + default: + iNdEx = preIndex + skippy, err := skipScpb(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthScpb + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Table) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb } - case 4: + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Table: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Table: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field ReferenceID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) } - m.ReferenceID = 0 + m.TableID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8065,14 +5441,14 @@ func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.ReferenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - case 5: + case 2: if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8082,12 +5458,12 @@ func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - m.ReferenceColumns = append(m.ReferenceColumns, v) + m.DependedOnBy = append(m.DependedOnBy, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { @@ -8122,11 +5498,11 @@ func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.ReferenceColumns) == 0 { - m.ReferenceColumns = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) + if elementCount != 0 && len(m.DependedOnBy) == 0 { + m.DependedOnBy = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) } for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8136,48 +5512,16 @@ func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - m.ReferenceColumns = append(m.ReferenceColumns, v) + m.DependedOnBy = append(m.DependedOnBy, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field ReferenceColumns", wireType) - } - case 6: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF + return fmt.Errorf("proto: wrong wireType = %d for field DependedOnBy", wireType) } - m.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipScpb(dAtA[iNdEx:]) @@ -8199,7 +5543,7 @@ func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { } return nil } -func (m *SequenceOwnedBy) Unmarshal(dAtA []byte) error { +func (m *TypeReference) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8222,17 +5566,17 @@ func (m *SequenceOwnedBy) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SequenceOwnedBy: wiretype end group for non-group") + return fmt.Errorf("proto: TypeReference: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SequenceOwnedBy: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: TypeReference: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SequenceID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TypeID", wireType) } - m.SequenceID = 0 + m.TypeID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8242,16 +5586,16 @@ func (m *SequenceOwnedBy) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.SequenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.TypeID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } case 2: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field OwnerTableID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DescID", wireType) } - m.OwnerTableID = 0 + m.DescID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8261,7 +5605,7 @@ func (m *SequenceOwnedBy) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.OwnerTableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.DescID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } @@ -8287,7 +5631,7 @@ func (m *SequenceOwnedBy) Unmarshal(dAtA []byte) error { } return nil } -func (m *RelationDependedOnBy) Unmarshal(dAtA []byte) error { +func (m *OutboundForeignKey) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8310,36 +5654,207 @@ func (m *RelationDependedOnBy) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RelationDependedOnBy: wiretype end group for non-group") + return fmt.Errorf("proto: OutboundForeignKey: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RelationDependedOnBy: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: OutboundForeignKey: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field OriginID", wireType) } - m.TableID = 0 + m.OriginID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.OriginID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType == 0 { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.OriginColumns = append(m.OriginColumns, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthScpb + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.OriginColumns) == 0 { + m.OriginColumns = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) + } + for iNdEx < postIndex { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.OriginColumns = append(m.OriginColumns, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field OriginColumns", wireType) + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ReferenceID", wireType) + } + m.ReferenceID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb } - if iNdEx >= l { + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ReferenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 5: + if wireType == 0 { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ReferenceColumns = append(m.ReferenceColumns, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthScpb + } + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ReferenceColumns) == 0 { + m.ReferenceColumns = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) + } + for iNdEx < postIndex { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ReferenceColumns = append(m.ReferenceColumns, v) } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field ReferenceColumns", wireType) } - case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DependedOnBy", wireType) + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) } - m.DependedOnBy = 0 + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8349,11 +5864,24 @@ func (m *RelationDependedOnBy) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.DependedOnBy |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthScpb + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipScpb(dAtA[iNdEx:]) @@ -8375,7 +5903,7 @@ func (m *RelationDependedOnBy) Unmarshal(dAtA []byte) error { } return nil } -func (m *Type) Unmarshal(dAtA []byte) error { +func (m *InboundForeignKey) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8398,17 +5926,17 @@ func (m *Type) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Type: wiretype end group for non-group") + return fmt.Errorf("proto: InboundForeignKey: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Type: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: InboundForeignKey: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TypeID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field OriginID", wireType) } - m.TypeID = 0 + m.OriginID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8418,66 +5946,92 @@ func (m *Type) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TypeID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.OriginID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - default: - iNdEx = preIndex - skippy, err := skipScpb(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthScpb - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} -func (m *Schema) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break + case 3: + if wireType == 0 { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.OriginColumns = append(m.OriginColumns, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthScpb + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.OriginColumns) == 0 { + m.OriginColumns = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) + } + for iNdEx < postIndex { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.OriginColumns = append(m.OriginColumns, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field OriginColumns", wireType) } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: Schema: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: Schema: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SchemaID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ReferenceID", wireType) } - m.SchemaID = 0 + m.ReferenceID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8487,14 +6041,14 @@ func (m *Schema) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.SchemaID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.ReferenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - case 2: + case 5: if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8504,12 +6058,12 @@ func (m *Schema) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift if b < 0x80 { break } } - m.DependentObjects = append(m.DependentObjects, v) + m.ReferenceColumns = append(m.ReferenceColumns, v) } else if wireType == 2 { var packedLen int for shift := uint(0); ; shift += 7 { @@ -8544,11 +6098,11 @@ func (m *Schema) Unmarshal(dAtA []byte) error { } } elementCount = count - if elementCount != 0 && len(m.DependentObjects) == 0 { - m.DependentObjects = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) + if elementCount != 0 && len(m.ReferenceColumns) == 0 { + m.ReferenceColumns = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID, 0, elementCount) } for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8558,16 +6112,48 @@ func (m *Schema) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ColumnID(b&0x7F) << shift if b < 0x80 { break } } - m.DependentObjects = append(m.DependentObjects, v) + m.ReferenceColumns = append(m.ReferenceColumns, v) } } else { - return fmt.Errorf("proto: wrong wireType = %d for field DependentObjects", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ReferenceColumns", wireType) + } + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthScpb + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthScpb + } + if postIndex > l { + return io.ErrUnexpectedEOF } + m.Name = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipScpb(dAtA[iNdEx:]) @@ -8589,7 +6175,7 @@ func (m *Schema) Unmarshal(dAtA []byte) error { } return nil } -func (m *Database) Unmarshal(dAtA []byte) error { +func (m *SequenceOwnedBy) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8612,17 +6198,17 @@ func (m *Database) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Database: wiretype end group for non-group") + return fmt.Errorf("proto: SequenceOwnedBy: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Database: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: SequenceOwnedBy: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DatabaseID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SequenceID", wireType) } - m.DatabaseID = 0 + m.SequenceID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8632,86 +6218,29 @@ func (m *Database) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.DatabaseID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.SequenceID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } case 2: - if wireType == 0 { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.DependentObjects = append(m.DependentObjects, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLengthScpb + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field OwnerTableID", wireType) + } + m.OwnerTableID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb } - if postIndex > l { + if iNdEx >= l { return io.ErrUnexpectedEOF } - var elementCount int - var count int - for _, integer := range dAtA[iNdEx:postIndex] { - if integer < 128 { - count++ - } - } - elementCount = count - if elementCount != 0 && len(m.DependentObjects) == 0 { - m.DependentObjects = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) - } - for iNdEx < postIndex { - var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.DependentObjects = append(m.DependentObjects, v) + b := dAtA[iNdEx] + iNdEx++ + m.OwnerTableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field DependentObjects", wireType) } default: iNdEx = preIndex @@ -8734,7 +6263,7 @@ func (m *Database) Unmarshal(dAtA []byte) error { } return nil } -func (m *ListPartition) Unmarshal(dAtA []byte) error { +func (m *RelationDependedOnBy) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8757,17 +6286,17 @@ func (m *ListPartition) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: ListPartition: wiretype end group for non-group") + return fmt.Errorf("proto: RelationDependedOnBy: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: ListPartition: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: RelationDependedOnBy: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) } - var stringLen uint64 + m.TableID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8777,29 +6306,16 @@ func (m *ListPartition) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DependedOnBy", wireType) } - var stringLen uint64 + m.DependedOnBy = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8809,24 +6325,11 @@ func (m *ListPartition) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.DependedOnBy |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Expr = append(m.Expr, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipScpb(dAtA[iNdEx:]) @@ -8848,7 +6351,7 @@ func (m *ListPartition) Unmarshal(dAtA []byte) error { } return nil } -func (m *RangePartitions) Unmarshal(dAtA []byte) error { +func (m *Type) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -8871,17 +6374,17 @@ func (m *RangePartitions) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: RangePartitions: wiretype end group for non-group") + return fmt.Errorf("proto: Type: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: RangePartitions: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Type: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TypeID", wireType) } - var stringLen uint64 + m.TypeID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8891,29 +6394,66 @@ func (m *RangePartitions) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.TypeID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthScpb + default: + iNdEx = preIndex + skippy, err := skipScpb(dAtA[iNdEx:]) + if err != nil { + return err } - postIndex := iNdEx + intStringLen - if postIndex < 0 { + if (skippy < 0) || (iNdEx+skippy) < 0 { return ErrInvalidLengthScpb } - if postIndex > l { + if (iNdEx + skippy) > l { return io.ErrUnexpectedEOF } - m.Name = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field To", wireType) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Schema) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb } - var stringLen uint64 + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Schema: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Schema: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SchemaID", wireType) + } + m.SchemaID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -8921,58 +6461,89 @@ func (m *RangePartitions) Unmarshal(dAtA []byte) error { if iNdEx >= l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break + b := dAtA[iNdEx] + iNdEx++ + m.SchemaID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType == 0 { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.DependentObjects = append(m.DependentObjects, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthScpb } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.To = append(m.To, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field From", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthScpb } - if iNdEx >= l { + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } } + elementCount = count + if elementCount != 0 && len(m.DependentObjects) == 0 { + m.DependentObjects = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) + } + for iNdEx < postIndex { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.DependentObjects = append(m.DependentObjects, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field DependentObjects", wireType) } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.From = append(m.From, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipScpb(dAtA[iNdEx:]) @@ -8994,7 +6565,7 @@ func (m *RangePartitions) Unmarshal(dAtA []byte) error { } return nil } -func (m *Partitioning) Unmarshal(dAtA []byte) error { +func (m *Database) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -9017,17 +6588,17 @@ func (m *Partitioning) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: Partitioning: wiretype end group for non-group") + return fmt.Errorf("proto: Database: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: Partitioning: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: Database: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DatabaseID", wireType) } - m.TableID = 0 + m.DatabaseID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowScpb @@ -9037,130 +6608,87 @@ func (m *Partitioning) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TableID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + m.DatabaseID |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift if b < 0x80 { break } } case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field IndexId", wireType) - } - m.IndexId = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.IndexId |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.IndexID(b&0x7F) << shift - if b < 0x80 { - break - } - } - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Fields", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb + if wireType == 0 { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { - return io.ErrUnexpectedEOF + m.DependentObjects = append(m.DependentObjects, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break + if packedLen < 0 { + return ErrInvalidLengthScpb } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Fields = append(m.Fields, string(dAtA[iNdEx:postIndex])) - iNdEx = postIndex - case 4: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ListPartitions", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthScpb } - if iNdEx >= l { + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.ListPartitions = append(m.ListPartitions, &ListPartition{}) - if err := m.ListPartitions[len(m.ListPartitions)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 5: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field RangePartitions", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowScpb + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } } - if iNdEx >= l { - return io.ErrUnexpectedEOF + elementCount = count + if elementCount != 0 && len(m.DependentObjects) == 0 { + m.DependentObjects = make([]github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID, 0, elementCount) } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break + for iNdEx < postIndex { + var v github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowScpb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.DependentObjects = append(m.DependentObjects, v) } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field DependentObjects", wireType) } - if msglen < 0 { - return ErrInvalidLengthScpb - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthScpb - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.RangePartitions = append(m.RangePartitions, &RangePartitions{}) - if err := m.RangePartitions[len(m.RangePartitions)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipScpb(dAtA[iNdEx:]) diff --git a/pkg/sql/schemachanger/scpb/scpb.proto b/pkg/sql/schemachanger/scpb/scpb.proto index c9faa6a849a1..1706c0bc70ba 100644 --- a/pkg/sql/schemachanger/scpb/scpb.proto +++ b/pkg/sql/schemachanger/scpb/scpb.proto @@ -35,7 +35,6 @@ message ElementProto { Type type = 16; Schema schema = 17; Database database = 18; - Partitioning partitioning = 19; } message Target { @@ -70,45 +69,17 @@ message Column { } message PrimaryIndex { - // The direction of a column in the index. - enum Direction { - ASC = 0; - DESC = 1; - } option (gogoproto.equal) = true; uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; - uint32 index_id = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID"]; - string index_name = 3 [(gogoproto.customname) = "IndexName"]; - bool unique = 4 [(gogoproto.customname) = "Unique"]; - repeated uint32 key_column_ids = 5 [(gogoproto.customname) = "KeyColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; - repeated Direction key_column_direction = 6 [(gogoproto.customname) = "KeyColumnDirections"]; - repeated uint32 key_suffix_column_ids = 7 [(gogoproto.customname) = "KeySuffixColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; - cockroach.sql.sqlbase.ShardedDescriptor sharded_descriptor = 8 [(gogoproto.customname) = "ShardedDescriptor"]; - repeated uint32 storing_column_ids = 9 [(gogoproto.customname) = "StoringColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; - repeated uint32 composite_column_ids = 10 [(gogoproto.customname) = "CompositeColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; - bool inverted = 11 [(gogoproto.customname) = "Inverted"]; - bool concurrently = 12 [(gogoproto.customname) = "Concurrently"]; + cockroach.sql.sqlbase.IndexDescriptor index = 2 [(gogoproto.nullable) = false]; + uint32 other_primary_index_id = 3 [(gogoproto.customname) = "OtherPrimaryIndexID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID"]; } message SecondaryIndex { - // The direction of a column in the index. - enum Direction { - ASC = 0; - DESC = 1; - } option (gogoproto.equal) = true; uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; - uint32 index_id = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID"]; - string index_name = 3 [(gogoproto.customname) = "IndexName"]; - bool unique = 4 [(gogoproto.customname) = "Unique"]; - repeated uint32 key_column_ids = 5 [(gogoproto.customname) = "KeyColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; - repeated Direction key_column_direction = 6 [(gogoproto.customname) = "KeyColumnDirections"]; - repeated uint32 key_suffix_column_ids = 7 [(gogoproto.customname) = "KeySuffixColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; - cockroach.sql.sqlbase.ShardedDescriptor sharded_descriptor = 8 [(gogoproto.customname) = "ShardedDescriptor"]; - repeated uint32 storing_column_ids = 9 [(gogoproto.customname) = "StoringColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; - repeated uint32 composite_column_ids = 10 [(gogoproto.customname) = "CompositeColumnIDs", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ColumnID"]; - bool inverted = 11 [(gogoproto.customname) = "Inverted"]; - bool concurrently = 12 [(gogoproto.customname) = "Concurrently"]; + cockroach.sql.sqlbase.IndexDescriptor index = 2 [(gogoproto.nullable) = false]; + uint32 primary_index = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID"]; } message SequenceDependency { @@ -218,25 +189,3 @@ message Database { uint32 database_id = 1 [(gogoproto.customname) = "DatabaseID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; repeated uint32 dependentObjects = 2 [(gogoproto.customname) = "DependentObjects", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; } - -message ListPartition { - option (gogoproto.equal) = true; - string name = 1 [(gogoproto.customname) = "Name"]; - repeated string expr = 2 [(gogoproto.customname) = "Expr"]; -} - -message RangePartitions { - option (gogoproto.equal) = true; - string name = 1 [(gogoproto.customname) = "Name"]; - repeated string To = 2 [(gogoproto.customname) = "To"]; - repeated string From = 3 [(gogoproto.customname) = "From"]; -} - -message Partitioning { - option (gogoproto.equal) = true; - uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; - uint32 index_id = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.IndexID"]; - repeated string fields = 3 [(gogoproto.customname) = "Fields"]; - repeated ListPartition list_partitions = 4 [(gogoproto.customname) = "ListPartitions"]; - repeated RangePartitions range_partitions = 5 [(gogoproto.customname) = "RangePartitions"]; -} diff --git a/pkg/sql/schemachanger/scplan/deprules/helpers.go b/pkg/sql/schemachanger/scplan/deprules/helpers.go index 65cbbbcf14e3..f0f9a7f3525a 100644 --- a/pkg/sql/schemachanger/scplan/deprules/helpers.go +++ b/pkg/sql/schemachanger/scplan/deprules/helpers.go @@ -12,6 +12,21 @@ package deprules import "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" +func indexContainsColumn(idx *descpb.IndexDescriptor, colID descpb.ColumnID) bool { + return columnsContainsID(idx.KeyColumnIDs, colID) || + columnsContainsID(idx.StoreColumnIDs, colID) || + columnsContainsID(idx.KeySuffixColumnIDs, colID) +} + +func columnsContainsID(haystack []descpb.ColumnID, needle descpb.ColumnID) bool { + for _, id := range haystack { + if id == needle { + return true + } + } + return false +} + func idInIDs(objects []descpb.ID, id descpb.ID) bool { for _, other := range objects { if other == id { diff --git a/pkg/sql/schemachanger/scplan/deprules/rules.go b/pkg/sql/schemachanger/scplan/deprules/rules.go index 585f03d7a98e..d8f05c034649 100644 --- a/pkg/sql/schemachanger/scplan/deprules/rules.go +++ b/pkg/sql/schemachanger/scplan/deprules/rules.go @@ -131,17 +131,10 @@ func init() { } func init() { - columnInList := func(targetColumn descpb.ColumnID, columnList descpb.ColumnIDs) bool { - for _, column := range columnList { - if targetColumn == column { - return true - } - } - return false - } + column, columnTarget, columnNode := targetNodeVars("column") index, indexTarget, indexNode := targetNodeVars("index") - var id, status, direction rel.Var = "id", "index-status", "direction" + var id, status, direction rel.Var = "id", "status", "direction" register( "column depends on indexes", columnNode, indexNode, @@ -158,23 +151,16 @@ func init() { rel.Filter( "columnInIndex", column, index, )(func(from *scpb.Column, to scpb.Element) bool { + var idx *descpb.IndexDescriptor switch to := to.(type) { case *scpb.PrimaryIndex: - if columnInList(from.Column.ID, to.KeyColumnIDs) || - columnInList(from.Column.ID, to.StoringColumnIDs) || - columnInList(from.Column.ID, to.KeySuffixColumnIDs) { - return true - } + idx = &to.Index case *scpb.SecondaryIndex: - if columnInList(from.Column.ID, to.KeyColumnIDs) || - columnInList(from.Column.ID, to.StoringColumnIDs) || - columnInList(from.Column.ID, to.KeySuffixColumnIDs) { - return true - } + idx = &to.Index default: panic(errors.AssertionFailedf("unexpected type %T", to)) } - return false + return indexContainsColumn(idx, from.Column.ID) }), direction.Entities(screl.Direction, columnTarget, indexTarget), @@ -184,46 +170,10 @@ func init() { screl.JoinTargetNode(index, indexTarget, indexNode), ), ) - - register( - "index depends on column", - indexNode, columnNode, - screl.MustQuery( - - column.Type((*scpb.Column)(nil)), - index.Type((*scpb.PrimaryIndex)(nil), (*scpb.SecondaryIndex)(nil)), - - id.Entities(screl.DescID, column, index), - - rel.Filter( - "columnInIndex", column, index, - )(func(from *scpb.Column, to scpb.Element) bool { - switch to := to.(type) { - case *scpb.PrimaryIndex: - if columnInList(from.Column.ID, to.KeyColumnIDs) || - columnInList(from.Column.ID, to.StoringColumnIDs) || - columnInList(from.Column.ID, to.KeySuffixColumnIDs) { - return true - } - case *scpb.SecondaryIndex: - if columnInList(from.Column.ID, to.KeyColumnIDs) || - columnInList(from.Column.ID, to.StoringColumnIDs) || - columnInList(from.Column.ID, to.KeySuffixColumnIDs) { - return true - } - default: - panic(errors.AssertionFailedf("unexpected type %T", to)) - } - return false - }), - - joinTargetNode(column, columnTarget, columnNode, add, deleteOnly), - joinTargetNode(index, indexTarget, indexNode, add, deleteOnly), - ), - ) } func init() { + addIdx, addTarget, addNode := targetNodeVars("add-idx") dropIdx, dropTarget, dropNode := targetNodeVars("drop-idx") var id rel.Var = "id" @@ -235,7 +185,7 @@ func init() { rel.Filter( "referenceEachOther", addIdx, dropIdx, )(func(add, drop *scpb.PrimaryIndex) bool { - return add.IndexId != drop.IndexId + return add.OtherPrimaryIndexID == drop.Index.ID }), joinTargetNode(addIdx, addTarget, addNode, @@ -256,53 +206,6 @@ func init() { ) } -func init() { - addIdx, addTarget, addNode := targetNodeVars("add-idx") - partitioning, partitioningTarget, partitioningNode := targetNodeVars("partitioning") - var id rel.Var = "id" - - register( - "partitioning information needs the basic index as created", - partitioningNode, addNode, - screl.MustQuery( - addIdx.Type((*scpb.PrimaryIndex)(nil)), - partitioning.Type((*scpb.Partitioning)(nil)), - id.Entities(screl.DescID, addIdx, partitioning), - id.Entities(screl.IndexID, addIdx, partitioning), - - joinTargetNode(addIdx, addTarget, addNode, - add, deleteOnly), - joinTargetNode(partitioning, partitioningTarget, partitioningNode, - add, public), - ), - ) -} - -func init() { - addIdx, addTarget, addNode := targetNodeVars("add-idx") - partitioning, partitioningTarget, partitioningNode := targetNodeVars("partitioning") - var id rel.Var = "id" - - register( - "index needs partitioning information to be filled", - partitioningNode, addNode, - screl.MustQuery( - addIdx.Type((*scpb.PrimaryIndex)(nil)), - partitioning.Type((*scpb.Partitioning)(nil)), - id.Entities(screl.DescID, addIdx, partitioning), - id.Entities(screl.IndexID, addIdx, partitioning), - - joinTargetNode(addIdx, addTarget, addNode, - add, deleteAndWriteOnly), - joinTargetNode(partitioning, partitioningTarget, partitioningNode, - add, public), - ), - ) -} - -// TODO(ajwerner): What does this even mean? The sequence starts in -// public. - func init() { depNeedsRelationToExitSynthDrop := func(ruleName string, depTypes []interface{}, depDescIDMatch rel.Attr) { // Before any parts of a relation/type can be dropped, the relation diff --git a/pkg/sql/schemachanger/scplan/deprules/testdata/rules b/pkg/sql/schemachanger/scplan/deprules/testdata/rules index 09ad56c32d25..0a4caa06477b 100644 --- a/pkg/sql/schemachanger/scplan/deprules/testdata/rules +++ b/pkg/sql/schemachanger/scplan/deprules/testdata/rules @@ -61,7 +61,7 @@ rules from: column-node to: index-node query: - - $index-status IN [DELETE_AND_WRITE_ONLY, PUBLIC] + - $status IN [DELETE_AND_WRITE_ONLY, PUBLIC] - $direction = ADD - $column[Type] = '*scpb.Column' - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] @@ -70,8 +70,8 @@ rules - columnInIndex(*scpb.Column, scpb.Element)($column, $index) - $column-target[Direction] = $direction - $index-target[Direction] = $direction - - $column-node[Status] = $index-status - - $index-node[Status] = $index-status + - $column-node[Status] = $status + - $index-node[Status] = $status - $column-target[Type] = '*scpb.Target' - $column-target[Element] = $column - $column-node[Type] = '*scpb.Node' @@ -80,27 +80,6 @@ rules - $index-target[Element] = $index - $index-node[Type] = '*scpb.Node' - $index-node[Target] = $index-target -- name: index depends on column - from: index-node - to: column-node - query: - - $column[Type] = '*scpb.Column' - - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex'] - - $column[DescID] = $id - - $index[DescID] = $id - - columnInIndex(*scpb.Column, scpb.Element)($column, $index) - - $column-target[Type] = '*scpb.Target' - - $column-target[Element] = $column - - $column-node[Type] = '*scpb.Node' - - $column-node[Target] = $column-target - - $column-target[Direction] = ADD - - $column-node[Status] = DELETE_ONLY - - $index-target[Type] = '*scpb.Target' - - $index-target[Element] = $index - - $index-node[Type] = '*scpb.Node' - - $index-node[Target] = $index-target - - $index-target[Direction] = ADD - - $index-node[Status] = DELETE_ONLY - name: primary index add depends on drop from: add-idx-node to: drop-idx-node @@ -143,50 +122,6 @@ rules - $drop-idx-node[Target] = $drop-idx-target - $drop-idx-target[Direction] = DROP - $drop-idx-node[Status] = DELETE_AND_WRITE_ONLY -- name: partitioning information needs the basic index as created - from: partitioning-node - to: add-idx-node - query: - - $add-idx[Type] = '*scpb.PrimaryIndex' - - $partitioning[Type] = '*scpb.Partitioning' - - $add-idx[DescID] = $id - - $partitioning[DescID] = $id - - $add-idx[IndexID] = $id - - $partitioning[IndexID] = $id - - $add-idx-target[Type] = '*scpb.Target' - - $add-idx-target[Element] = $add-idx - - $add-idx-node[Type] = '*scpb.Node' - - $add-idx-node[Target] = $add-idx-target - - $add-idx-target[Direction] = ADD - - $add-idx-node[Status] = DELETE_ONLY - - $partitioning-target[Type] = '*scpb.Target' - - $partitioning-target[Element] = $partitioning - - $partitioning-node[Type] = '*scpb.Node' - - $partitioning-node[Target] = $partitioning-target - - $partitioning-target[Direction] = ADD - - $partitioning-node[Status] = PUBLIC -- name: index needs partitioning information to be filled - from: partitioning-node - to: add-idx-node - query: - - $add-idx[Type] = '*scpb.PrimaryIndex' - - $partitioning[Type] = '*scpb.Partitioning' - - $add-idx[DescID] = $id - - $partitioning[DescID] = $id - - $add-idx[IndexID] = $id - - $partitioning[IndexID] = $id - - $add-idx-target[Type] = '*scpb.Target' - - $add-idx-target[Element] = $add-idx - - $add-idx-node[Type] = '*scpb.Node' - - $add-idx-node[Target] = $add-idx-target - - $add-idx-target[Direction] = ADD - - $add-idx-node[Status] = DELETE_AND_WRITE_ONLY - - $partitioning-target[Type] = '*scpb.Target' - - $partitioning-target[Element] = $partitioning - - $partitioning-node[Type] = '*scpb.Node' - - $partitioning-node[Target] = $partitioning-target - - $partitioning-target[Direction] = ADD - - $partitioning-node[Status] = PUBLIC - name: dependency needs relation/type as non-synthetically dropped from: dep-node to: relation-node diff --git a/pkg/sql/schemachanger/scplan/opgen/BUILD.bazel b/pkg/sql/schemachanger/scplan/opgen/BUILD.bazel index 066f256977f3..def17948eb2d 100644 --- a/pkg/sql/schemachanger/scplan/opgen/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/opgen/BUILD.bazel @@ -10,7 +10,6 @@ go_library( "opgen_default_expression.go", "opgen_inbound_foreign_key.go", "opgen_outbound_foreign_key.go", - "opgen_partitioning.go", "opgen_primary_index.go", "opgen_relation_depended_on_by.go", "opgen_schema.go", @@ -27,7 +26,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/opgen", visibility = ["//visibility:public"], deps = [ - "//pkg/sql/catalog/descpb", "//pkg/sql/schemachanger/rel", "//pkg/sql/schemachanger/scgraph", "//pkg/sql/schemachanger/scop", diff --git a/pkg/sql/schemachanger/scplan/opgen/opgen_partitioning.go b/pkg/sql/schemachanger/scplan/opgen/opgen_partitioning.go deleted file mode 100644 index 89413367ac55..000000000000 --- a/pkg/sql/schemachanger/scplan/opgen/opgen_partitioning.go +++ /dev/null @@ -1,35 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package opgen - -import ( - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" -) - -func init() { - opRegistry.register( - (*scpb.Partitioning)(nil), - scpb.Target_ADD, - scpb.Status_ABSENT, - to(scpb.Status_PUBLIC, - minPhase(scop.PreCommitPhase), - emit(func(this *scpb.Partitioning) scop.Op { - return &scop.AddIndexPartitionInfo{ - TableID: this.TableID, - IndexID: this.IndexId, - PartitionFields: this.Fields, - ListPartitions: this.ListPartitions, - RangePartitions: this.RangePartitions, - } - }), - )) -} diff --git a/pkg/sql/schemachanger/scplan/opgen/opgen_primary_index.go b/pkg/sql/schemachanger/scplan/opgen/opgen_primary_index.go index cdfc3a0ea509..7e87c1c15078 100644 --- a/pkg/sql/schemachanger/scplan/opgen/opgen_primary_index.go +++ b/pkg/sql/schemachanger/scplan/opgen/opgen_primary_index.go @@ -11,27 +11,10 @@ package opgen import ( - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" ) -func convertPrimaryIndexColumnDir( - primaryIndex *scpb.PrimaryIndex, -) []descpb.IndexDescriptor_Direction { - // Convert column directions - convertedColumnDirs := make([]descpb.IndexDescriptor_Direction, 0, len(primaryIndex.KeyColumnDirections)) - for _, columnDir := range primaryIndex.KeyColumnDirections { - switch columnDir { - case scpb.PrimaryIndex_DESC: - convertedColumnDirs = append(convertedColumnDirs, descpb.IndexDescriptor_DESC) - case scpb.PrimaryIndex_ASC: - convertedColumnDirs = append(convertedColumnDirs, descpb.IndexDescriptor_ASC) - } - } - return convertedColumnDirs -} - func init() { opRegistry.register( (*scpb.PrimaryIndex)(nil), @@ -41,19 +24,8 @@ func init() { minPhase(scop.PreCommitPhase), emit(func(this *scpb.PrimaryIndex) scop.Op { return &scop.MakeAddedIndexDeleteOnly{ - TableID: this.TableID, - IndexID: this.IndexId, - IndexName: this.IndexName, - Unique: this.Unique, - KeyColumnIDs: this.KeyColumnIDs, - KeyColumnDirections: convertPrimaryIndexColumnDir(this), - KeySuffixColumnIDs: this.KeySuffixColumnIDs, - StoreColumnIDs: this.StoringColumnIDs, - CompositeColumnIDs: this.CompositeColumnIDs, - ShardedDescriptor: this.ShardedDescriptor, - Inverted: this.Inverted, - Concurrently: this.Concurrently, - SecondaryIndex: false, + TableID: this.TableID, + Index: this.Index, } })), to(scpb.Status_DELETE_AND_WRITE_ONLY, @@ -61,14 +33,14 @@ func init() { emit(func(this *scpb.PrimaryIndex) scop.Op { return &scop.MakeAddedIndexDeleteAndWriteOnly{ TableID: this.TableID, - IndexID: this.IndexId, + IndexID: this.Index.ID, } })), to(scpb.Status_BACKFILLED, emit(func(this *scpb.PrimaryIndex) scop.Op { return &scop.BackfillIndex{ TableID: this.TableID, - IndexID: this.IndexId, + IndexID: this.Index.ID, } })), // If this index is unique (which primary indexes should be) and @@ -79,15 +51,16 @@ func init() { to(scpb.Status_VALIDATED, emit(func(this *scpb.PrimaryIndex) scop.Op { return &scop.ValidateUniqueIndex{ - TableID: this.TableID, - IndexID: this.IndexId, + TableID: this.TableID, + PrimaryIndexID: this.OtherPrimaryIndexID, + IndexID: this.Index.ID, } })), to(scpb.Status_PUBLIC, emit(func(this *scpb.PrimaryIndex) scop.Op { return &scop.MakeAddedPrimaryIndexPublic{ TableID: this.TableID, - IndexID: this.IndexId, + Index: this.Index, } })), ) @@ -102,7 +75,7 @@ func init() { // Most of this logic is taken from MakeMutationComplete(). return &scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly{ TableID: this.TableID, - IndexID: this.IndexId, + Index: this.Index, } })), to(scpb.Status_DELETE_ONLY, @@ -112,7 +85,7 @@ func init() { emit(func(this *scpb.PrimaryIndex) scop.Op { return &scop.MakeDroppedIndexDeleteOnly{ TableID: this.TableID, - IndexID: this.IndexId, + IndexID: this.Index.ID, } })), to(scpb.Status_ABSENT, @@ -120,7 +93,7 @@ func init() { emit(func(this *scpb.PrimaryIndex) scop.Op { return &scop.MakeIndexAbsent{ TableID: this.TableID, - IndexID: this.IndexId, + IndexID: this.Index.ID, } })), ) diff --git a/pkg/sql/schemachanger/scplan/opgen/opgen_secondary_index.go b/pkg/sql/schemachanger/scplan/opgen/opgen_secondary_index.go index ca96b883edb9..890d25226b32 100644 --- a/pkg/sql/schemachanger/scplan/opgen/opgen_secondary_index.go +++ b/pkg/sql/schemachanger/scplan/opgen/opgen_secondary_index.go @@ -10,88 +10,5 @@ package opgen -import ( - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" -) - -func convertSecondaryIndexColumnDir( - secondaryIndex *scpb.SecondaryIndex, -) []descpb.IndexDescriptor_Direction { - // Convert column directions - convertedColumnDirs := make([]descpb.IndexDescriptor_Direction, 0, len(secondaryIndex.KeyColumnDirections)) - for _, columnDir := range secondaryIndex.KeyColumnDirections { - switch columnDir { - case scpb.SecondaryIndex_DESC: - convertedColumnDirs = append(convertedColumnDirs, descpb.IndexDescriptor_DESC) - case scpb.SecondaryIndex_ASC: - convertedColumnDirs = append(convertedColumnDirs, descpb.IndexDescriptor_ASC) - } - } - return convertedColumnDirs -} - func init() { - opRegistry.register( - (*scpb.SecondaryIndex)(nil), - scpb.Target_ADD, - scpb.Status_ABSENT, - to(scpb.Status_DELETE_ONLY, - minPhase(scop.PreCommitPhase), - emit(func(this *scpb.SecondaryIndex) scop.Op { - return &scop.MakeAddedIndexDeleteOnly{ - TableID: this.TableID, - IndexID: this.IndexId, - IndexName: this.IndexName, - Unique: this.Unique, - KeyColumnIDs: this.KeyColumnIDs, - KeyColumnDirections: convertSecondaryIndexColumnDir(this), - KeySuffixColumnIDs: this.KeySuffixColumnIDs, - StoreColumnIDs: this.StoringColumnIDs, - CompositeColumnIDs: this.CompositeColumnIDs, - ShardedDescriptor: this.ShardedDescriptor, - Inverted: this.Inverted, - Concurrently: this.Concurrently, - SecondaryIndex: true, - } - })), - to(scpb.Status_DELETE_AND_WRITE_ONLY, - minPhase(scop.PostCommitPhase), - emit(func(this *scpb.SecondaryIndex) scop.Op { - return &scop.MakeAddedIndexDeleteAndWriteOnly{ - TableID: this.TableID, - IndexID: this.IndexId, - } - })), - to(scpb.Status_BACKFILLED, - emit(func(this *scpb.SecondaryIndex) scop.Op { - return &scop.BackfillIndex{ - TableID: this.TableID, - IndexID: this.IndexId, - } - })), - // If this index is unique (which primary indexes should be) and - // there's not already a covering primary index, then we'll need to - // validate that this index indeed is unique. - // - // TODO(ajwerner): Rationalize this and hook up the optimization. - to(scpb.Status_VALIDATED, - emit(func(this *scpb.SecondaryIndex) scop.Op { - if this.Unique { - return &scop.ValidateUniqueIndex{ - TableID: this.TableID, - IndexID: this.IndexId, - } - } - return &scop.NoOpInfo{} - })), - to(scpb.Status_PUBLIC, - emit(func(this *scpb.SecondaryIndex) scop.Op { - return &scop.MakeAddedSecondaryIndexPublic{ - TableID: this.TableID, - IndexID: this.IndexId, - } - })), - ) } diff --git a/pkg/sql/schemachanger/scplan/plan.go b/pkg/sql/schemachanger/scplan/plan.go index ecd050144657..8a7adca00a3f 100644 --- a/pkg/sql/schemachanger/scplan/plan.go +++ b/pkg/sql/schemachanger/scplan/plan.go @@ -11,8 +11,6 @@ package scplan import ( - "sort" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scgraph" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" @@ -103,9 +101,6 @@ func validateStages(stages []Stage) { } func buildStages(init scpb.State, g *scgraph.Graph, params Params) []Stage { - // Fetch the order of the graph, which will be used to - // evaluating edges in topological order. - nodeRanks := g.GetNodeRanks() // TODO(ajwerner): deal with the case where the target status was // fulfilled by something that preceded the initial state. cur := init @@ -156,19 +151,13 @@ func buildStages(init scpb.State, g *scgraph.Graph, params Params) []Stage { if !ok { return Stage{}, false } - sort.SliceStable(edges, - func(i, j int) bool { - // Higher ranked edges should go first. - return nodeRanks[edges[i].To()] > nodeRanks[edges[j].To()] - }) - next := append(cur[:0:0], cur...) isStageRevertible := true var ops []scop.Op for revertible := 1; revertible >= 0; revertible-- { isStageRevertible = revertible == 1 - for _, e := range edges { - for i, ts := range cur { + for i, ts := range cur { + for _, e := range edges { if e.From() == ts && isStageRevertible == e.Revertible() { next[i] = e.To() ops = append(ops, e.Op()...) @@ -233,9 +222,133 @@ func buildStages(init scpb.State, g *scgraph.Graph, params Params) []Stage { if !didSomething { break } + // Sort ops based on graph dependencies. + sortOps(g, s.Ops.Slice()) stages = append(stages, s) cur = s.After } validateStages(stages) return stages } + +// Check if some route exists from curr to the +// target node +func doesPathExistToNode(graph *scgraph.Graph, start *scpb.Node, target *scpb.Node) bool { + nodesToVisit := []*scpb.Node{start} + visitedNodes := map[*scpb.Node]struct{}{} + for len(nodesToVisit) > 0 { + curr := nodesToVisit[0] + if curr == target { + return true + } + nodesToVisit = nodesToVisit[1:] + if _, ok := visitedNodes[curr]; !ok { + visitedNodes[curr] = struct{}{} + edges, ok := graph.GetDepEdgesFrom(curr) + if !ok { + return false + } + // Append all of the nodes to visit + for _, currEdge := range edges { + nodesToVisit = append(nodesToVisit, currEdge.To()) + } + } + } + return false +} + +// sortOps sorts the operations into order based on +// graph dependencies +func sortOps(graph *scgraph.Graph, ops []scop.Op) { + // Original implicit order of the ops will + // be kept to keep ordering of equal values. + implicitOrder := make([]int, 0, len(ops)) + for i := range ops { + implicitOrder = append(implicitOrder, i) + } + // Unfortunately, we are forced to do an inefficient + // bubble sort, since with unrelated dependencies will + // be equal to each other. But, still have a relative order + // across the entire set of nodes. + for i := 0; i < len(ops); i++ { + for j := i + 1; j < len(ops); j++ { + if i == j { + continue + } + if !compareOps(graph, ops[i], ops[j], implicitOrder[i], implicitOrder[j]) && + compareOps(graph, ops[j], ops[i], implicitOrder[j], implicitOrder[i]) { + tmpOrder := implicitOrder[i] + tmp := ops[i] + ops[i] = ops[j] + ops[j] = tmp + implicitOrder[i] = implicitOrder[j] + implicitOrder[j] = tmpOrder + } + } + } + // Sanity: Graph order is sane across all the ops. + for i := 0; i < len(ops); i++ { + for j := i + 1; j < len(ops); j++ { + // Validate that the list is sorted by checking + // 1) i is always less than j. + // 1a) If it is not less than its equal based on the opposite comparison. + // 2) j should always be greater or equal to i, so our comparison + // should never be true. + // Note: We will intentionally ignore the implicit order for the + // validation phase because we will have non-comparable items. + if !compareOps(graph, ops[i], ops[j], 1, 0) && + compareOps(graph, ops[j], ops[i], 1, 0) { + panic(errors.AssertionFailedf("Operators are not completely sorted %d %d, "+ + "not strictly increasing", i, j)) + } else if compareOps(graph, ops[j], ops[i], 1, 0) { + panic(errors.AssertionFailedf("Operators are not completely sorted %d %d", i, j)) + } + } + } +} + +// compareOps compares operations and orders them based on +// followed by the graph dependencies. +func compareOps( + graph *scgraph.Graph, firstOp, + secondOp scop.Op, firstImplicitOrder, secondImplicitOrder int, +) (less bool) { + // Otherwise, lets compare attributes + firstNode := graph.GetNodeFromOp(firstOp) + secondNode := graph.GetNodeFromOp(secondOp) + if firstNode == secondNode { + // Equal, only implicit order determines which is first. + return firstImplicitOrder < secondImplicitOrder + } + firstExists := doesPathExistToNode(graph, firstNode, secondNode) + secondExists := doesPathExistToNode(graph, secondNode, firstNode) + + // If both paths exist, then we care about the direction of nodes, + // otherwise we have a cycle, and we can't sort. + if firstExists && secondExists { + if firstNode.Target.Direction == scpb.Target_DROP { + return false + } else if secondNode.Target.Direction == scpb.Target_DROP { + return true + } else { + panic(errors.AssertionFailedf("A potential cycle exists in plan the graph, without any"+ + "nodes transitioning in opposite directions\n %s\n%s\n", + firstNode, + secondNode)) + } + } + // If a path exists from first to second, then the first node depends + // on the second. + if firstExists { + return false + } + // If a path exists to the second to the first node, then + // the second node depends on the first. + if secondExists { + return true + } + + // Otherwise, no paths exists and the two are equal. + // Only the implicit order determines the first one. + return firstImplicitOrder < secondImplicitOrder +} diff --git a/pkg/sql/schemachanger/scplan/plan_test.go b/pkg/sql/schemachanger/scplan/plan_test.go index d7d69715f8d8..26d62c0fb26f 100644 --- a/pkg/sql/schemachanger/scplan/plan_test.go +++ b/pkg/sql/schemachanger/scplan/plan_test.go @@ -53,7 +53,7 @@ func TestPlanAlterTable(t *testing.T) { tdb := sqlutils.MakeSQLRunner(sqlDB) run := func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { - case "create-view", "create-sequence", "create-table", "create-type", "create-database", "create-schema", "create-index": + case "create-view", "create-sequence", "create-table", "create-type", "create-database", "create-schema": stmts, err := parser.Parse(d.Input) require.NoError(t, err) require.Len(t, stmts, 1) @@ -71,8 +71,6 @@ func TestPlanAlterTable(t *testing.T) { tableName = "" case *tree.CreateSchema: tableName = "" - case *tree.CreateIndex: - tableName = "" default: t.Fatal("not a CREATE TABLE/SEQUENCE/VIEW statement") } diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table b/pkg/sql/schemachanger/scplan/testdata/alter_table index cf0efdb3cc9d..c86e46fb4404 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table @@ -18,17 +18,23 @@ Stage 0 FamilyName: primary TableID: 52 *scop.MakeAddedIndexDeleteOnly - IndexID: 2 - IndexName: new_primary_key - KeyColumnDirections: - - 0 - KeyColumnIDs: - - 1 - ShardedDescriptor: {} - StoreColumnIDs: - - 2 - TableID: 52 - Unique: true + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - j + Unique: true + Version: 4 + TableID: 52 Stage 1 *scop.MakeAddedIndexDeleteAndWriteOnly IndexID: 2 @@ -43,13 +49,40 @@ Stage 2 Stage 3 *scop.ValidateUniqueIndex IndexID: 2 + PrimaryIndexID: 1 TableID: 52 Stage 4 - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 52 *scop.MakeAddedPrimaryIndexPublic - IndexID: 2 + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - j + Unique: true + Version: 4 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: foo_pkey + Unique: true + Version: 4 TableID: 52 *scop.MakeColumnPublic ColumnID: 2 @@ -80,17 +113,23 @@ Stage 0 FamilyName: primary TableID: 52 *scop.MakeAddedIndexDeleteOnly - IndexID: 2 - IndexName: new_primary_key - KeyColumnDirections: - - 0 - KeyColumnIDs: - - 1 - ShardedDescriptor: {} - StoreColumnIDs: - - 2 - TableID: 52 - Unique: true + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - j + Unique: true + Version: 4 + TableID: 52 Stage 1 *scop.MakeAddedIndexDeleteAndWriteOnly IndexID: 2 @@ -105,13 +144,40 @@ Stage 2 Stage 3 *scop.ValidateUniqueIndex IndexID: 2 + PrimaryIndexID: 1 TableID: 52 Stage 4 - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 52 *scop.MakeAddedPrimaryIndexPublic - IndexID: 2 + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - j + Unique: true + Version: 4 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: foo_pkey + Unique: true + Version: 4 TableID: 52 *scop.MakeColumnPublic ColumnID: 2 @@ -142,6 +208,26 @@ Stage 0 width: 64 FamilyName: primary TableID: 52 + *scop.MakeAddedIndexDeleteOnly + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + - 3 + StoreColumnNames: + - j + - k + Unique: true + Version: 4 + TableID: 52 *scop.MakeAddedColumnDeleteOnly Column: DefaultExpr: 456:::INT8 @@ -154,19 +240,6 @@ Stage 0 width: 64 FamilyName: primary TableID: 52 - *scop.MakeAddedIndexDeleteOnly - IndexID: 2 - IndexName: new_primary_key - KeyColumnDirections: - - 0 - KeyColumnIDs: - - 1 - ShardedDescriptor: {} - StoreColumnIDs: - - 2 - - 3 - TableID: 52 - Unique: true Stage 1 *scop.MakeAddedIndexDeleteAndWriteOnly IndexID: 2 @@ -184,13 +257,42 @@ Stage 2 Stage 3 *scop.ValidateUniqueIndex IndexID: 2 + PrimaryIndexID: 1 TableID: 52 Stage 4 - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 52 *scop.MakeAddedPrimaryIndexPublic - IndexID: 2 + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + - 3 + StoreColumnNames: + - j + - k + Unique: true + Version: 4 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: foo_pkey + Unique: true + Version: 4 TableID: 52 *scop.MakeColumnPublic ColumnID: 2 @@ -224,17 +326,23 @@ Stage 0 FamilyName: primary TableID: 52 *scop.MakeAddedIndexDeleteOnly - IndexID: 2 - IndexName: new_primary_key - KeyColumnDirections: - - 0 - KeyColumnIDs: - - 1 - ShardedDescriptor: {} - StoreColumnIDs: - - 2 - TableID: 52 - Unique: true + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - a + Unique: true + Version: 4 + TableID: 52 Stage 1 *scop.MakeAddedIndexDeleteAndWriteOnly IndexID: 2 @@ -249,13 +357,40 @@ Stage 2 Stage 3 *scop.ValidateUniqueIndex IndexID: 2 + PrimaryIndexID: 1 TableID: 52 Stage 4 - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 52 *scop.MakeAddedPrimaryIndexPublic - IndexID: 2 + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - a + Unique: true + Version: 4 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: foo_pkey + Unique: true + Version: 4 TableID: 52 *scop.MakeColumnPublic ColumnID: 2 @@ -291,17 +426,23 @@ Stage 0 FamilyName: primary TableID: 52 *scop.MakeAddedIndexDeleteOnly - IndexID: 2 - IndexName: new_primary_key - KeyColumnDirections: - - 0 - KeyColumnIDs: - - 1 - ShardedDescriptor: {} - StoreColumnIDs: - - 2 - TableID: 52 - Unique: true + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - a + Unique: true + Version: 4 + TableID: 52 *scop.MakeAddedColumnDeleteOnly Column: ID: 3 @@ -314,18 +455,25 @@ Stage 0 FamilyName: primary TableID: 53 *scop.MakeAddedIndexDeleteOnly - IndexID: 2 - IndexName: new_primary_key - KeyColumnDirections: - - 0 - KeyColumnIDs: - - 2 - ShardedDescriptor: {} - StoreColumnIDs: - - 1 - - 3 + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 2 + KeyColumnNames: + - rowid + Name: new_primary_key + StoreColumnIDs: + - 1 + - 3 + StoreColumnNames: + - j + - b + Unique: true + Version: 4 TableID: 53 - Unique: true Stage 1 *scop.MakeAddedIndexDeleteAndWriteOnly IndexID: 2 @@ -349,25 +497,85 @@ Stage 2 Stage 3 *scop.ValidateUniqueIndex IndexID: 2 + PrimaryIndexID: 1 TableID: 52 *scop.ValidateUniqueIndex IndexID: 2 + PrimaryIndexID: 1 TableID: 53 Stage 4 - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 52 *scop.MakeAddedPrimaryIndexPublic - IndexID: 2 + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: new_primary_key + StoreColumnIDs: + - 2 + StoreColumnNames: + - a + Unique: true + Version: 4 + TableID: 52 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 1 + KeyColumnNames: + - i + Name: foo_pkey + Unique: true + Version: 4 TableID: 52 *scop.MakeColumnPublic ColumnID: 2 TableID: 52 - *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly - IndexID: 1 - TableID: 53 *scop.MakeAddedPrimaryIndexPublic - IndexID: 2 + Index: + EncodingType: 1 + ID: 2 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 2 + KeyColumnNames: + - rowid + Name: new_primary_key + StoreColumnIDs: + - 1 + - 3 + StoreColumnNames: + - j + - b + Unique: true + Version: 4 + TableID: 53 + *scop.MakeDroppedPrimaryIndexDeleteAndWriteOnly + Index: + EncodingType: 1 + ID: 1 + KeyColumnDirections: + - 0 + KeyColumnIDs: + - 2 + KeyColumnNames: + - rowid + Name: bar_pkey + StoreColumnIDs: + - 1 + StoreColumnNames: + - j + Unique: true + Version: 4 TableID: 53 *scop.MakeColumnPublic ColumnID: 3 diff --git a/pkg/sql/schemachanger/scplan/testdata/create_index b/pkg/sql/schemachanger/scplan/testdata/create_index deleted file mode 100644 index 5972e344530e..000000000000 --- a/pkg/sql/schemachanger/scplan/testdata/create_index +++ /dev/null @@ -1,120 +0,0 @@ -create-table -CREATE TABLE defaultdb.t1 (id INT PRIMARY KEY, name varchar(256), money int) ----- - -ops -CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) ----- -Stage 0 - *scop.MakeAddedIndexDeleteOnly - IndexID: 2 - IndexName: id1 - KeyColumnDirections: - - 0 - - 0 - KeyColumnIDs: - - 1 - - 2 - SecondaryIndex: true - StoreColumnIDs: - - 3 - TableID: 52 -Stage 1 - *scop.MakeAddedIndexDeleteAndWriteOnly - IndexID: 2 - TableID: 52 -Stage 2 - *scop.BackfillIndex - IndexID: 2 - TableID: 52 -Stage 3 - *scop.NoOpInfo - {} -Stage 4 - *scop.MakeAddedSecondaryIndexPublic - IndexID: 2 - TableID: 52 - -deps -CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) ----- - -ops -CREATE INVERTED INDEX concurrently id1 on defaultdb.t1(id, name) storing (money) ----- -Stage 0 - *scop.MakeAddedIndexDeleteOnly - Concurrently: true - IndexID: 2 - IndexName: id1 - Inverted: true - KeyColumnDirections: - - 0 - - 0 - KeyColumnIDs: - - 1 - - 2 - SecondaryIndex: true - StoreColumnIDs: - - 3 - TableID: 52 -Stage 1 - *scop.MakeAddedIndexDeleteAndWriteOnly - IndexID: 2 - TableID: 52 -Stage 2 - *scop.BackfillIndex - IndexID: 2 - TableID: 52 -Stage 3 - *scop.NoOpInfo - {} -Stage 4 - *scop.MakeAddedSecondaryIndexPublic - IndexID: 2 - TableID: 52 - -deps -CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) ----- - -ops -CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) PARTITION BY LIST (id) ( - PARTITION p1 VALUES IN (1) - ) ----- -Stage 0 - *scop.MakeAddedIndexDeleteOnly - IndexID: 2 - IndexName: id1 - KeyColumnDirections: - - 0 - - 0 - KeyColumnIDs: - - 1 - - 2 - SecondaryIndex: true - StoreColumnIDs: - - 3 - TableID: 52 -Stage 1 - *scop.MakeAddedIndexDeleteAndWriteOnly - IndexID: 2 - TableID: 52 -Stage 2 - *scop.BackfillIndex - IndexID: 2 - TableID: 52 -Stage 3 - *scop.NoOpInfo - {} -Stage 4 - *scop.MakeAddedSecondaryIndexPublic - IndexID: 2 - TableID: 52 - -deps -CREATE INDEX id1 on defaultdb.t1(id, name) storing (money) PARTITION BY LIST (id) ( - PARTITION p1 VALUES IN (1) - ) ----- diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_database b/pkg/sql/schemachanger/scplan/testdata/drop_database index 8162509322bd..01c6c881d169 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_database +++ b/pkg/sql/schemachanger/scplan/testdata/drop_database @@ -85,36 +85,21 @@ Stage 1 (non-revertible) TableID: 57 *scop.UpdateRelationDeps TableID: 57 - *scop.RemoveColumnDefaultExpression - ColumnID: 3 - TableID: 57 - *scop.UpdateRelationDeps - TableID: 57 *scop.MarkDescriptorAsDropped DescID: 54 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 57 - TableID: 54 *scop.RemoveColumnDefaultExpression ColumnID: 1 TableID: 57 *scop.UpdateRelationDeps TableID: 57 - *scop.RemoveColumnDefaultExpression - ColumnID: 1 - TableID: 57 - *scop.UpdateRelationDeps - TableID: 57 - *scop.RemoveColumnDefaultExpression - ColumnID: 2 - TableID: 57 - *scop.UpdateRelationDeps - TableID: 57 *scop.RemoveColumnDefaultExpression ColumnID: 2 TableID: 57 *scop.UpdateRelationDeps TableID: 57 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 57 + TableID: 54 *scop.MarkDescriptorAsDropped DescID: 56 *scop.RemoveColumnDefaultExpression @@ -122,74 +107,59 @@ Stage 1 (non-revertible) TableID: 56 *scop.UpdateRelationDeps TableID: 56 - *scop.RemoveColumnDefaultExpression - ColumnID: 3 - TableID: 56 - *scop.UpdateRelationDeps - TableID: 56 *scop.MarkDescriptorAsDropped DescID: 55 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 56 - TableID: 55 *scop.RemoveRelationDependedOnBy DependedOnBy: 58 TableID: 56 *scop.MarkDescriptorAsDropped - DescID: 62 - *scop.RemoveTypeBackRef - DescID: 64 - TypeID: 62 - *scop.MarkDescriptorAsDropped - DescID: 63 - *scop.RemoveTypeBackRef - DescID: 64 - TypeID: 63 - *scop.MarkDescriptorAsDropped - DescID: 61 + DescID: 58 *scop.RemoveRelationDependedOnBy - DependedOnBy: 64 - TableID: 61 + DependedOnBy: 59 + TableID: 58 *scop.MarkDescriptorAsDropped DescID: 59 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 60 + TableID: 58 *scop.RemoveRelationDependedOnBy DependedOnBy: 60 TableID: 59 + *scop.MarkDescriptorAsDropped + DescID: 60 *scop.RemoveRelationDependedOnBy DependedOnBy: 61 TableID: 59 *scop.MarkDescriptorAsDropped - DescID: 58 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 59 - TableID: 58 + DescID: 61 *scop.RemoveRelationDependedOnBy - DependedOnBy: 60 - TableID: 58 + DependedOnBy: 64 + TableID: 61 *scop.MarkDescriptorAsDropped - DescID: 60 + DescID: 64 + *scop.MarkDescriptorAsDropped + DescID: 62 + *scop.RemoveTypeBackRef + DescID: 64 + TypeID: 62 *scop.MarkDescriptorAsDropped + DescID: 63 + *scop.RemoveTypeBackRef DescID: 64 + TypeID: 63 *scop.RemoveColumnDefaultExpression ColumnID: 1 TableID: 56 *scop.UpdateRelationDeps TableID: 56 - *scop.RemoveColumnDefaultExpression - ColumnID: 1 - TableID: 56 - *scop.UpdateRelationDeps - TableID: 56 - *scop.RemoveColumnDefaultExpression - ColumnID: 2 - TableID: 56 - *scop.UpdateRelationDeps - TableID: 56 *scop.RemoveColumnDefaultExpression ColumnID: 2 TableID: 56 *scop.UpdateRelationDeps TableID: 56 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 56 + TableID: 55 *scop.MarkDescriptorAsDropped DescID: 53 *scop.MarkDescriptorAsDropped @@ -199,14 +169,6 @@ Stage 2 (non-revertible) TableID: 54 *scop.CreateGcJobForDescriptor DescID: 54 - *scop.DrainDescriptorName - TableID: 54 - *scop.CreateGcJobForDescriptor - DescID: 54 - *scop.DrainDescriptorName - TableID: 57 - *scop.CreateGcJobForDescriptor - DescID: 57 *scop.DrainDescriptorName TableID: 57 *scop.CreateGcJobForDescriptor @@ -215,38 +177,10 @@ Stage 2 (non-revertible) TableID: 55 *scop.CreateGcJobForDescriptor DescID: 55 - *scop.DrainDescriptorName - TableID: 55 - *scop.CreateGcJobForDescriptor - DescID: 55 *scop.DrainDescriptorName TableID: 60 *scop.CreateGcJobForDescriptor DescID: 60 - *scop.DrainDescriptorName - TableID: 60 - *scop.CreateGcJobForDescriptor - DescID: 60 - *scop.DrainDescriptorName - TableID: 64 - *scop.CreateGcJobForDescriptor - DescID: 64 - *scop.DrainDescriptorName - TableID: 64 - *scop.CreateGcJobForDescriptor - DescID: 64 - *scop.DrainDescriptorName - TableID: 61 - *scop.CreateGcJobForDescriptor - DescID: 61 - *scop.DrainDescriptorName - TableID: 61 - *scop.CreateGcJobForDescriptor - DescID: 61 - *scop.DrainDescriptorName - TableID: 59 - *scop.CreateGcJobForDescriptor - DescID: 59 *scop.DrainDescriptorName TableID: 59 *scop.CreateGcJobForDescriptor @@ -256,13 +190,13 @@ Stage 2 (non-revertible) *scop.CreateGcJobForDescriptor DescID: 58 *scop.DrainDescriptorName - TableID: 58 + TableID: 64 *scop.CreateGcJobForDescriptor - DescID: 58 + DescID: 64 *scop.DrainDescriptorName - TableID: 56 + TableID: 61 *scop.CreateGcJobForDescriptor - DescID: 56 + DescID: 61 *scop.DrainDescriptorName TableID: 56 *scop.CreateGcJobForDescriptor diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_schema b/pkg/sql/schemachanger/scplan/testdata/drop_schema index 30b55e3cdee2..c4ce0839f5ea 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_schema +++ b/pkg/sql/schemachanger/scplan/testdata/drop_schema @@ -148,74 +148,59 @@ Stage 1 (non-revertible) TableID: 54 *scop.UpdateRelationDeps TableID: 54 - *scop.RemoveColumnDefaultExpression - ColumnID: 3 - TableID: 54 - *scop.UpdateRelationDeps - TableID: 54 *scop.MarkDescriptorAsDropped DescID: 53 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 54 - TableID: 53 *scop.RemoveRelationDependedOnBy DependedOnBy: 55 TableID: 54 *scop.MarkDescriptorAsDropped - DescID: 59 - *scop.RemoveTypeBackRef - DescID: 61 - TypeID: 59 - *scop.MarkDescriptorAsDropped - DescID: 60 - *scop.RemoveTypeBackRef - DescID: 61 - TypeID: 60 - *scop.MarkDescriptorAsDropped - DescID: 58 + DescID: 55 *scop.RemoveRelationDependedOnBy - DependedOnBy: 61 - TableID: 58 + DependedOnBy: 56 + TableID: 55 *scop.MarkDescriptorAsDropped DescID: 56 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 57 + TableID: 55 *scop.RemoveRelationDependedOnBy DependedOnBy: 57 TableID: 56 + *scop.MarkDescriptorAsDropped + DescID: 57 *scop.RemoveRelationDependedOnBy DependedOnBy: 58 TableID: 56 *scop.MarkDescriptorAsDropped - DescID: 55 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 56 - TableID: 55 + DescID: 58 *scop.RemoveRelationDependedOnBy - DependedOnBy: 57 - TableID: 55 + DependedOnBy: 61 + TableID: 58 *scop.MarkDescriptorAsDropped - DescID: 57 + DescID: 61 *scop.MarkDescriptorAsDropped + DescID: 59 + *scop.RemoveTypeBackRef DescID: 61 + TypeID: 59 + *scop.MarkDescriptorAsDropped + DescID: 60 + *scop.RemoveTypeBackRef + DescID: 61 + TypeID: 60 *scop.RemoveColumnDefaultExpression ColumnID: 1 TableID: 54 *scop.UpdateRelationDeps TableID: 54 - *scop.RemoveColumnDefaultExpression - ColumnID: 1 - TableID: 54 - *scop.UpdateRelationDeps - TableID: 54 - *scop.RemoveColumnDefaultExpression - ColumnID: 2 - TableID: 54 - *scop.UpdateRelationDeps - TableID: 54 *scop.RemoveColumnDefaultExpression ColumnID: 2 TableID: 54 *scop.UpdateRelationDeps TableID: 54 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 54 + TableID: 53 *scop.MarkDescriptorAsDropped DescID: 52 Stage 2 (non-revertible) @@ -223,38 +208,10 @@ Stage 2 (non-revertible) TableID: 53 *scop.CreateGcJobForDescriptor DescID: 53 - *scop.DrainDescriptorName - TableID: 53 - *scop.CreateGcJobForDescriptor - DescID: 53 - *scop.DrainDescriptorName - TableID: 57 - *scop.CreateGcJobForDescriptor - DescID: 57 *scop.DrainDescriptorName TableID: 57 *scop.CreateGcJobForDescriptor DescID: 57 - *scop.DrainDescriptorName - TableID: 61 - *scop.CreateGcJobForDescriptor - DescID: 61 - *scop.DrainDescriptorName - TableID: 61 - *scop.CreateGcJobForDescriptor - DescID: 61 - *scop.DrainDescriptorName - TableID: 58 - *scop.CreateGcJobForDescriptor - DescID: 58 - *scop.DrainDescriptorName - TableID: 58 - *scop.CreateGcJobForDescriptor - DescID: 58 - *scop.DrainDescriptorName - TableID: 56 - *scop.CreateGcJobForDescriptor - DescID: 56 *scop.DrainDescriptorName TableID: 56 *scop.CreateGcJobForDescriptor @@ -264,13 +221,13 @@ Stage 2 (non-revertible) *scop.CreateGcJobForDescriptor DescID: 55 *scop.DrainDescriptorName - TableID: 55 + TableID: 61 *scop.CreateGcJobForDescriptor - DescID: 55 + DescID: 61 *scop.DrainDescriptorName - TableID: 54 + TableID: 58 *scop.CreateGcJobForDescriptor - DescID: 54 + DescID: 58 *scop.DrainDescriptorName TableID: 54 *scop.CreateGcJobForDescriptor diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_sequence b/pkg/sql/schemachanger/scplan/testdata/drop_sequence index 992c63fa5921..3f912b763e60 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_sequence +++ b/pkg/sql/schemachanger/scplan/testdata/drop_sequence @@ -16,10 +16,6 @@ Stage 2 (non-revertible) TableID: 52 *scop.CreateGcJobForDescriptor DescID: 52 - *scop.DrainDescriptorName - TableID: 52 - *scop.CreateGcJobForDescriptor - DescID: 52 create-table CREATE TABLE defaultdb.blog_posts (id INT PRIMARY KEY, val int DEFAULT nextval('defaultdb.sq1'), title text) @@ -41,16 +37,6 @@ Stage 1 (non-revertible) TableID: 53 *scop.UpdateRelationDeps TableID: 53 - *scop.RemoveColumnDefaultExpression - ColumnID: 2 - TableID: 53 - *scop.UpdateRelationDeps - TableID: 53 - *scop.RemoveColumnDefaultExpression - ColumnID: 2 - TableID: 54 - *scop.UpdateRelationDeps - TableID: 54 *scop.RemoveColumnDefaultExpression ColumnID: 2 TableID: 54 @@ -63,10 +49,6 @@ Stage 2 (non-revertible) TableID: 52 *scop.CreateGcJobForDescriptor DescID: 52 - *scop.DrainDescriptorName - TableID: 52 - *scop.CreateGcJobForDescriptor - DescID: 52 deps diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_table b/pkg/sql/schemachanger/scplan/testdata/drop_table index ae9b18c6efa5..54d707fea9bf 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_table +++ b/pkg/sql/schemachanger/scplan/testdata/drop_table @@ -72,11 +72,6 @@ Stage 1 (non-revertible) TableID: 55 *scop.UpdateRelationDeps TableID: 55 - *scop.RemoveColumnDefaultExpression - ColumnID: 1 - TableID: 55 - *scop.UpdateRelationDeps - TableID: 55 *scop.MarkDescriptorAsDropped DescID: 56 *scop.RemoveSequenceOwnedBy @@ -86,16 +81,6 @@ Stage 1 (non-revertible) TableID: 55 *scop.UpdateRelationDeps TableID: 55 - *scop.RemoveColumnDefaultExpression - ColumnID: 2 - TableID: 55 - *scop.UpdateRelationDeps - TableID: 55 - *scop.RemoveColumnDefaultExpression - ColumnID: 3 - TableID: 55 - *scop.UpdateRelationDeps - TableID: 55 *scop.RemoveColumnDefaultExpression ColumnID: 3 TableID: 55 @@ -106,16 +91,6 @@ Stage 1 (non-revertible) TableID: 55 *scop.UpdateRelationDeps TableID: 55 - *scop.RemoveColumnDefaultExpression - ColumnID: 4 - TableID: 55 - *scop.UpdateRelationDeps - TableID: 55 - *scop.RemoveColumnDefaultExpression - ColumnID: 5 - TableID: 55 - *scop.UpdateRelationDeps - TableID: 55 *scop.RemoveColumnDefaultExpression ColumnID: 5 TableID: 55 @@ -129,14 +104,6 @@ Stage 2 (non-revertible) TableID: 57 *scop.CreateGcJobForDescriptor DescID: 57 - *scop.DrainDescriptorName - TableID: 57 - *scop.CreateGcJobForDescriptor - DescID: 57 - *scop.DrainDescriptorName - TableID: 56 - *scop.CreateGcJobForDescriptor - DescID: 56 *scop.DrainDescriptorName TableID: 56 *scop.CreateGcJobForDescriptor @@ -145,10 +112,6 @@ Stage 2 (non-revertible) TableID: 55 *scop.CreateGcJobForDescriptor DescID: 55 - *scop.DrainDescriptorName - TableID: 55 - *scop.CreateGcJobForDescriptor - DescID: 55 deps DROP TABLE defaultdb.shipments CASCADE; diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_view b/pkg/sql/schemachanger/scplan/testdata/drop_view index dba268162d3e..48205a5edd42 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_view +++ b/pkg/sql/schemachanger/scplan/testdata/drop_view @@ -23,10 +23,6 @@ Stage 2 (non-revertible) TableID: 53 *scop.CreateGcJobForDescriptor DescID: 53 - *scop.DrainDescriptorName - TableID: 53 - *scop.CreateGcJobForDescriptor - DescID: 53 deps DROP VIEW defaultdb.v1 @@ -70,50 +66,50 @@ Stage 1 (non-revertible) *scop.RemoveRelationDependedOnBy DependedOnBy: 53 TableID: 52 - *scop.RemoveTypeBackRef - DescID: 59 - TypeID: 57 - *scop.RemoveTypeBackRef - DescID: 59 - TypeID: 58 *scop.MarkDescriptorAsDropped - DescID: 56 + DescID: 53 *scop.RemoveRelationDependedOnBy - DependedOnBy: 59 - TableID: 56 + DependedOnBy: 54 + TableID: 53 *scop.MarkDescriptorAsDropped DescID: 54 + *scop.RemoveRelationDependedOnBy + DependedOnBy: 55 + TableID: 53 *scop.RemoveRelationDependedOnBy DependedOnBy: 55 TableID: 54 + *scop.MarkDescriptorAsDropped + DescID: 55 *scop.RemoveRelationDependedOnBy DependedOnBy: 56 TableID: 54 *scop.MarkDescriptorAsDropped - DescID: 53 - *scop.RemoveRelationDependedOnBy - DependedOnBy: 54 - TableID: 53 + DescID: 56 *scop.RemoveRelationDependedOnBy - DependedOnBy: 55 - TableID: 53 - *scop.MarkDescriptorAsDropped - DescID: 55 + DependedOnBy: 59 + TableID: 56 *scop.MarkDescriptorAsDropped DescID: 59 + *scop.RemoveTypeBackRef + DescID: 59 + TypeID: 57 + *scop.RemoveTypeBackRef + DescID: 59 + TypeID: 58 Stage 2 (non-revertible) *scop.DrainDescriptorName TableID: 55 *scop.CreateGcJobForDescriptor DescID: 55 *scop.DrainDescriptorName - TableID: 55 + TableID: 54 *scop.CreateGcJobForDescriptor - DescID: 55 + DescID: 54 *scop.DrainDescriptorName - TableID: 59 + TableID: 53 *scop.CreateGcJobForDescriptor - DescID: 59 + DescID: 53 *scop.DrainDescriptorName TableID: 59 *scop.CreateGcJobForDescriptor @@ -122,26 +118,6 @@ Stage 2 (non-revertible) TableID: 56 *scop.CreateGcJobForDescriptor DescID: 56 - *scop.DrainDescriptorName - TableID: 56 - *scop.CreateGcJobForDescriptor - DescID: 56 - *scop.DrainDescriptorName - TableID: 54 - *scop.CreateGcJobForDescriptor - DescID: 54 - *scop.DrainDescriptorName - TableID: 54 - *scop.CreateGcJobForDescriptor - DescID: 54 - *scop.DrainDescriptorName - TableID: 53 - *scop.CreateGcJobForDescriptor - DescID: 53 - *scop.DrainDescriptorName - TableID: 53 - *scop.CreateGcJobForDescriptor - DescID: 53 deps DROP VIEW defaultdb.v1 CASCADE diff --git a/pkg/sql/schemachanger/screl/attr.go b/pkg/sql/schemachanger/screl/attr.go index b7e62a594376..2e7f5a344fb2 100644 --- a/pkg/sql/schemachanger/screl/attr.go +++ b/pkg/sql/schemachanger/screl/attr.go @@ -98,13 +98,13 @@ var Schema = rel.MustSchema("screl", ), rel.EntityMapping(t((*scpb.PrimaryIndex)(nil)), rel.EntityAttr(DescID, "TableID"), - rel.EntityAttr(IndexID, "IndexId"), - rel.EntityAttr(Name, "IndexName"), + rel.EntityAttr(IndexID, "Index.ID"), + rel.EntityAttr(Name, "Index.Name"), ), rel.EntityMapping(t((*scpb.SecondaryIndex)(nil)), rel.EntityAttr(DescID, "TableID"), - rel.EntityAttr(IndexID, "IndexId"), - rel.EntityAttr(Name, "IndexName"), + rel.EntityAttr(IndexID, "Index.ID"), + rel.EntityAttr(Name, "Index.Name"), ), rel.EntityMapping(t((*scpb.SequenceDependency)(nil)), rel.EntityAttr(DescID, "SequenceID"), @@ -163,10 +163,6 @@ var Schema = rel.MustSchema("screl", rel.EntityMapping(t((*scpb.Database)(nil)), rel.EntityAttr(DescID, "DatabaseID"), ), - rel.EntityMapping(t((*scpb.Partitioning)(nil)), - rel.EntityAttr(DescID, "TableID"), - rel.EntityAttr(IndexID, "IndexId"), - ), ) // JoinTargetNode generates a clause that joins the target and node vars diff --git a/pkg/sql/sqlutil/BUILD.bazel b/pkg/sql/sqlutil/BUILD.bazel index 234ce51cb482..ccf251c5eb7d 100644 --- a/pkg/sql/sqlutil/BUILD.bazel +++ b/pkg/sql/sqlutil/BUILD.bazel @@ -7,7 +7,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv", - "//pkg/sql/catalog", "//pkg/sql/catalog/colinfo", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", diff --git a/pkg/sql/sqlutil/internal_executor.go b/pkg/sql/sqlutil/internal_executor.go index fedcc0d28440..aa0d2f7d5253 100644 --- a/pkg/sql/sqlutil/internal_executor.go +++ b/pkg/sql/sqlutil/internal_executor.go @@ -14,7 +14,6 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/kv" - "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" @@ -143,21 +142,6 @@ type InternalExecutor interface { stmt string, qargs ...interface{}, ) (InternalRows, error) - - // WithSyntheticDescriptors sets the synthetic descriptors before running the - // the provided closure and resets them afterward. Used for queries/statements - // that need to use in-memory synthetic descriptors different from descriptors - // written to disk. These descriptors override all other descriptors on the - // immutable resolution path. - // - // Warning: Not safe for concurrent use from multiple goroutines. This API is - // flawed in that the internal executor is meant to function as a stateless - // wrapper, and creates a new connExecutor and descs.Collection on each query/ - // statement, so these descriptors should really be specified at a per-query/ - // statement level. See #34304. - WithSyntheticDescriptors( - descs []catalog.Descriptor, run func() error, - ) error } // InternalRows is an iterator interface that's exposed by the internal @@ -196,11 +180,3 @@ type InternalRows interface { type SessionBoundInternalExecutorFactory func( context.Context, *sessiondata.SessionData, ) InternalExecutor - -// InternalExecFn is the type of functions that operates using an internalExecutor. -type InternalExecFn func(ctx context.Context, txn *kv.Txn, ie InternalExecutor) error - -// HistoricalInternalExecTxnRunner is like historicalTxnRunner except it only -// passes the fn the exported InternalExecutor instead of the whole unexported -// extendedEvalContenxt, so it can be implemented outside pkg/sql. -type HistoricalInternalExecTxnRunner func(ctx context.Context, fn InternalExecFn) error