From d5313438aaa61a71a5a66c5718963bbb0fd7268b Mon Sep 17 00:00:00 2001 From: Andrew Werner Date: Wed, 18 May 2022 18:22:44 -0400 Subject: [PATCH] sql/backfill: fix bug adding new columns to new index with volatile default Release note (bug fix): In 21.1 a bug was introduced whereby default values were recomputed when populating data in new secondary indexes for columns which were added in the same transaction as the index. This arises, for example in cases like `ALTER TABLE t ADD COLUMN f FLOAT8 UNIQUE DEFAULT (random())`. If the default expression is not volatile, then the recomputation is harmless. If, however, the default expression is volatile, the data in the secondary index will not match the data in the primary index: a corrupt index will have been created. This bug has now been fixed. --- pkg/BUILD.bazel | 1 + pkg/sql/backfill/BUILD.bazel | 14 +- pkg/sql/backfill/backfill.go | 90 +--- pkg/sql/backfill/index_backfiller_cols.go | 177 ++++++++ .../backfill/index_backfiller_cols_test.go | 389 ++++++++++++++++++ pkg/sql/catalog/table_col_set.go | 15 + .../logictest/testdata/logic_test/alter_table | 22 + 7 files changed, 637 insertions(+), 71 deletions(-) create mode 100644 pkg/sql/backfill/index_backfiller_cols.go create mode 100644 pkg/sql/backfill/index_backfiller_cols_test.go diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index b0596d28fc80..42c840f5e9eb 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -226,6 +226,7 @@ ALL_TESTS = [ "//pkg/spanconfig/spanconfigstore:spanconfigstore_test", "//pkg/spanconfig/spanconfigtestutils:spanconfigtestutils_test", "//pkg/spanconfig:spanconfig_test", + "//pkg/sql/backfill:backfill_test", "//pkg/sql/catalog/catalogkeys:catalogkeys_test", "//pkg/sql/catalog/catformat:catformat_test", "//pkg/sql/catalog/catpb:catpb_test", diff --git a/pkg/sql/backfill/BUILD.bazel b/pkg/sql/backfill/BUILD.bazel index ce3f419c9a45..c5cca0c929aa 100644 --- a/pkg/sql/backfill/BUILD.bazel +++ b/pkg/sql/backfill/BUILD.bazel @@ -1,9 +1,10 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "backfill", srcs = [ "backfill.go", + "index_backfiller_cols.go", "mvcc_index_merger.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/backfill", @@ -40,3 +41,14 @@ go_library( "@com_github_cockroachdb_logtags//:logtags", ], ) + +go_test( + name = "backfill_test", + srcs = ["index_backfiller_cols_test.go"], + embed = [":backfill"], + deps = [ + "//pkg/sql/catalog", + "//pkg/sql/catalog/descpb", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index 8df9ebb7af93..80c81a1c3af5 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -33,7 +33,6 @@ import ( "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/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -414,27 +413,14 @@ type muBoundAccount struct { // IndexBackfiller is capable of backfilling all the added index. type IndexBackfiller struct { + indexBackfillerCols + added []catalog.Index - // colIdxMap maps ColumnIDs to indices into desc.Columns and desc.Mutations. - colIdxMap catalog.TableColMap types []*types.T rowVals tree.Datums evalCtx *eval.Context - // cols are all of the writable (PUBLIC and DELETE_AND_WRITE_ONLY) columns in - // the descriptor. - cols []catalog.Column - - // addedCols are the columns in DELETE_AND_WRITE_ONLY being added as part of - // this index which are not computed. - addedCols []catalog.Column - - // computedCols are the columns in this index which are computed and do - // not have concrete values in the source index. This is virtual computed - // columns and stored computed columns which are non-public. - computedCols []catalog.Column - // Map of columns which need to be evaluated to their expressions. colExprs map[descpb.ColumnID]tree.TypedExpr @@ -447,10 +433,6 @@ type IndexBackfiller struct { // backfilled. indexesToEncode []catalog.Index - // valNeededForCol contains the indexes (into cols) of all columns that we - // need to fetch values for. - valNeededForCol util.FastIntSet - alloc tree.DatumAlloc // mon is a memory monitor linked with the IndexBackfiller on creation. @@ -478,11 +460,14 @@ func (ib *IndexBackfiller) InitForLocalUse( desc catalog.TableDescriptor, mon *mon.BytesMonitor, ) error { - // Initialize ib.cols and ib.colIdxMap. - ib.initCols(desc) // Initialize ib.added. - ib.valNeededForCol = ib.initIndexes(desc) + ib.initIndexes(desc) + + // Initialize ib.cols and ib.colIdxMap. + if err := ib.initCols(desc); err != nil { + return err + } predicates, colExprs, referencedColumns, err := constructExprs( ctx, desc, ib.added, ib.cols, ib.addedCols, ib.computedCols, evalCtx, semaCtx, @@ -614,11 +599,14 @@ func (ib *IndexBackfiller) InitForDistributedUse( desc catalog.TableDescriptor, mon *mon.BytesMonitor, ) error { - // Initialize ib.cols and ib.colIdxMap. - ib.initCols(desc) // Initialize ib.added. - ib.valNeededForCol = ib.initIndexes(desc) + ib.initIndexes(desc) + + // Initialize ib.indexBackfillerCols. + if err := ib.initCols(desc); err != nil { + return err + } evalCtx := flowCtx.NewEvalCtx() var predicates map[descpb.IndexID]tree.TypedExpr @@ -689,35 +677,17 @@ func (ib *IndexBackfiller) ShrinkBoundAccount(ctx context.Context, shrinkBy int6 // initCols is a helper to populate column metadata of an IndexBackfiller. It // populates the cols and colIdxMap fields. -func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) { - ib.cols = make([]catalog.Column, 0, len(desc.DeletableColumns())) - for _, column := range desc.DeletableColumns() { - if column.Public() { - if column.IsComputed() && column.IsVirtual() { - ib.computedCols = append(ib.computedCols, column) - } - } else if column.Adding() && column.WriteAndDeleteOnly() { - // If there are ongoing mutations, add columns that are being added and in - // the DELETE_AND_WRITE_ONLY state. - if column.IsComputed() { - ib.computedCols = append(ib.computedCols, column) - } else { - ib.addedCols = append(ib.addedCols, column) - } - } else { - continue - } - // Create a map of each column's ID to its ordinal. - ib.colIdxMap.Set(column.GetID(), len(ib.cols)) - ib.cols = append(ib.cols, column) - } +func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) (err error) { + ib.indexBackfillerCols, err = makeIndexBackfillColumns( + desc.DeletableColumns(), desc.GetPrimaryIndex(), ib.added, + ) + return err } // initIndexes is a helper to populate index metadata of an IndexBackfiller. It // populates the added field. It returns a set of column ordinals that must be // fetched in order to backfill the added indexes. -func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor) util.FastIntSet { - var valNeededForCol util.FastIntSet +func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor) { mutations := desc.AllMutations() mutationID := mutations[0].MutationID() @@ -729,29 +699,9 @@ func (ib *IndexBackfiller) initIndexes(desc catalog.TableDescriptor) util.FastIn } if IndexMutationFilter(m) { idx := m.AsIndex() - colIDs := idx.CollectKeyColumnIDs() - if idx.GetEncodingType() == descpb.PrimaryIndexEncoding { - for _, col := range ib.cols { - if !col.IsVirtual() { - colIDs.Add(col.GetID()) - } - } - } else { - colIDs.UnionWith(idx.CollectSecondaryStoredColumnIDs()) - colIDs.UnionWith(idx.CollectKeySuffixColumnIDs()) - } - ib.added = append(ib.added, idx) - for i := range ib.cols { - id := ib.cols[i].GetID() - if colIDs.Contains(id) && i < len(desc.PublicColumns()) && !ib.cols[i].IsVirtual() { - valNeededForCol.Add(i) - } - } } } - - return valNeededForCol } // init completes the initialization of an IndexBackfiller. diff --git a/pkg/sql/backfill/index_backfiller_cols.go b/pkg/sql/backfill/index_backfiller_cols.go new file mode 100644 index 000000000000..4f31d3ae38fa --- /dev/null +++ b/pkg/sql/backfill/index_backfiller_cols.go @@ -0,0 +1,177 @@ +// Copyright 2022 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 backfill + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/errors" +) + +// indexBackfillerCols holds the interned set of columns used by the +// indexBackfiller and the information about which columns will need to +// be evaluated during the backfill. +type indexBackfillerCols struct { + // colIdxMap maps ColumnIDs to indices into desc.Columns and desc.Mutations. + colIdxMap catalog.TableColMap + + // cols are all writable (PUBLIC and DELETE_AND_WRITE_ONLY) columns in + // the descriptor. + cols []catalog.Column + + // addedCols are the columns in DELETE_AND_WRITE_ONLY being added as part of + // this index which are not computed. The definition of being added is that + // the index being backfilled is a new primary index and the columns do not + // exist in the currently public primary index. Note that this should never + // be populated when running using the legacy schema changer; the legacy + // schema changer rewrites primary indexes in-place using the column + // backfiller before creating any secondary indexes. + addedCols []catalog.Column + + // computedCols are the columns in this index which are computed and do + // not have concrete values in the source index. This is virtual computed + // columns and stored computed columns which are non-public and not part + // of the currently public primary index. + computedCols []catalog.Column + + // valNeededForCol contains the indexes (into cols) of all columns that we + // need to fetch values for. + valNeededForCol util.FastIntSet +} + +// makeIndexBackfillColumns computes the set of writable columns and +// classifies the two subsets of that set which will need to be computed +// during execution of the backfill. Note that all columns will be included +// in the complete cols slice regardless of whether they appear in +// addedIndexes. Appearance in addedIndexes will affect valNeededForColumn. +// Note that the valNeededForCol set will not be populated with +// requirements due to references in expressions. That needs to be added after +// constructing this information. +func makeIndexBackfillColumns( + deletableColumns []catalog.Column, sourcePrimaryIndex catalog.Index, addedIndexes []catalog.Index, +) (indexBackfillerCols, error) { + + // We will need to evaluate default or computed expressions for + // physical columns being added only if this is a primary index backfill + // and only if the added columns are not in the existing public primary + // index. We're going to assume that if there is a new column being added + // to a new primary index that it doesn't appear in any secondary index + // being simultaneously backfilled, because if that column were needed, + // and we're adding it now, we'd have a problem. + + var ib indexBackfillerCols + ib.cols = make([]catalog.Column, 0, len(deletableColumns)) + + var computedVirtual catalog.TableColSet + for _, column := range deletableColumns { + if !column.Public() && + !(column.Adding() && column.WriteAndDeleteOnly()) { + continue + } + if column.IsComputed() && column.IsVirtual() { + computedVirtual.Add(column.GetID()) + ib.computedCols = append(ib.computedCols, column) + } + // Create a map of each column's ID to its ordinal. + ib.colIdxMap.Set(column.GetID(), len(ib.cols)) + ib.cols = append(ib.cols, column) + } + + // Find the adding columns which are being added to new primary indexes. + primaryColumns := indexColumns(sourcePrimaryIndex) + var addedDefaultOrComputed catalog.TableColSet + for _, idx := range addedIndexes { + if idx.GetEncodingType() != descpb.PrimaryIndexEncoding { + if !indexColumns(idx).Difference(computedVirtual).SubsetOf(primaryColumns) { + return indexBackfillerCols{}, errors.AssertionFailedf( + "secondary index for backfill contains physical column not present in " + + "source primary index", + ) + } + } + addedDefaultOrComputed.UnionWith( + indexColumns(idx).Difference(computedVirtual).Difference(primaryColumns), + ) + } + + // Account for these new columns being initially backfilled into new primary + // indexes. + for _, colID := range addedDefaultOrComputed.Ordered() { + ord, ok := ib.colIdxMap.Get(colID) + if !ok { + return indexBackfillerCols{}, errors.AssertionFailedf( + "index being backfilled contains non-writable or dropping column") + } + col := ib.cols[ord] + if col.IsComputed() { + ib.computedCols = append(ib.computedCols, col) + } else { + ib.addedCols = append(ib.addedCols, col) + } + } + + ib.valNeededForCol = makeInitialValNeededForCol(ib, addedIndexes) + return ib, nil +} + +// makeInitialValNeededForCol computes the initial set of columns whose values +// are known to be needed based solely on index membership. More may be needed +// because of references in expressions. +func makeInitialValNeededForCol( + ib indexBackfillerCols, addedIndexes []catalog.Index, +) (valNeededForCol util.FastIntSet) { + // Any columns we're going to eval, we don't need values for ahead of time. + toEval := func() catalog.TableColSet { + columnIDs := func(columns []catalog.Column) (s catalog.TableColSet) { + for _, c := range columns { + s.Add(c.GetID()) + } + return s + } + + // The set of columns we'll evaluate are the addedColumns and the computed + // columns. We don't need values for these columns. + toEval := columnIDs(ib.addedCols) + toEval.UnionWith(columnIDs(ib.computedCols)) + return toEval + }() + + for _, idx := range addedIndexes { + colIDs := idx.CollectKeyColumnIDs() + if idx.GetEncodingType() == descpb.PrimaryIndexEncoding { + for _, col := range ib.cols { + if !col.IsVirtual() { + colIDs.Add(col.GetID()) + } + } + } else { + colIDs.UnionWith(idx.CollectSecondaryStoredColumnIDs()) + colIDs.UnionWith(idx.CollectKeySuffixColumnIDs()) + } + + for i := range ib.cols { + if id := ib.cols[i].GetID(); colIDs.Contains(id) && !toEval.Contains(id) { + valNeededForCol.Add(i) + } + } + } + return valNeededForCol +} + +// indexColumns computes the complete set of column stored in an index. +func indexColumns(idx catalog.Index) (s catalog.TableColSet) { + s.UnionWith(idx.CollectKeyColumnIDs()) + s.UnionWith(idx.CollectKeySuffixColumnIDs()) + s.UnionWith(idx.CollectPrimaryStoredColumnIDs()) + s.UnionWith(idx.CollectSecondaryStoredColumnIDs()) + return s +} diff --git a/pkg/sql/backfill/index_backfiller_cols_test.go b/pkg/sql/backfill/index_backfiller_cols_test.go new file mode 100644 index 000000000000..7cf3cf9b26e2 --- /dev/null +++ b/pkg/sql/backfill/index_backfiller_cols_test.go @@ -0,0 +1,389 @@ +// Copyright 2022 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 backfill + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/stretchr/testify/require" +) + +// Define some shorthand type names. +type ( + cols = []fakeColumn + index = fakeIndex + indexes = []index + colIDs = []descpb.ColumnID +) + +// TestIndexBackfillerColumns exercises the logic in makeIndexBackfillerColumns +// to ensure that it properly classifies columns needed for evaluation in an +// index backfill. +func TestIndexBackfillerColumns(t *testing.T) { + asIndexSlice := func(in indexes) (out []catalog.Index) { + for _, idx := range in { + out = append(out, idx) + } + return out + } + asColumnSlice := func(in cols) (out []catalog.Column) { + for _, c := range in { + out = append(out, c) + } + return out + } + for _, tc := range []struct { + name string + cols cols + src index + toEncode indexes + expCols colIDs + expComputed colIDs + expAdded colIDs + expErr string + expNeeded colIDs + }{ + { + name: "boring two public columns in a secondary index", + cols: cols{ + {id: 1, public: true}, + {id: 2, public: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1}, + primaryValueCols: colIDs{2}, + }, + toEncode: indexes{ + { + keyCols: colIDs{1}, + }, + }, + expCols: colIDs{1, 2}, + expNeeded: colIDs{1}, + }, + { + name: "one virtual, one computed adding mutation column in secondary", + cols: cols{ + {id: 1, public: true}, + {id: 2, writeAndDeleteOnly: true, adding: true}, + {id: 3, public: true, virtual: true, computed: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1, 2}, + }, + toEncode: indexes{ + { + keyCols: colIDs{1, 3}, + }, + { + keyCols: colIDs{3}, + }, + }, + expCols: colIDs{1, 2, 3}, + expComputed: colIDs{3}, + expNeeded: colIDs{1}, + }, + { + name: "one virtual, one computed adding mutation column not used", + cols: cols{ + {id: 1, public: true}, + {id: 2, writeAndDeleteOnly: true, adding: true}, + {id: 3, public: true, virtual: true, computed: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1, 2}, + }, + toEncode: indexes{ + { + keyCols: colIDs{1}, + }, + { + keyCols: colIDs{1}, + }, + }, + expCols: colIDs{1, 2, 3}, + expComputed: colIDs{3}, + expNeeded: colIDs{1}, + }, + { + name: "one virtual, one computed mutation column in primary", + cols: cols{ + {id: 1, public: true}, + {id: 2, writeAndDeleteOnly: true, adding: true}, + {id: 3, public: true, virtual: true, computed: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1, 2}, + }, + toEncode: indexes{ + { + primary: true, + keyCols: colIDs{1, 3}, + }, + }, + expCols: colIDs{1, 2, 3}, + expComputed: colIDs{3}, + expNeeded: colIDs{1, 2}, + }, + { + // This is a weird case which wouldn't actually happen. + name: "one virtual, one computed mutation column in source, not used in new primary", + cols: cols{ + {id: 1, public: true}, + {id: 2, writeAndDeleteOnly: true, adding: true}, + {id: 3, public: true, virtual: true, computed: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1, 2}, + }, + toEncode: indexes{ + { + primary: true, + keyCols: colIDs{1, 3}, + }, + }, + expCols: colIDs{1, 2, 3}, + expComputed: colIDs{3}, + expNeeded: colIDs{1, 2}, + }, + { + // This is the case where we're building a new primary index as part + // of an add column. + name: "one virtual, one new mutation column in source used in new primary", + cols: cols{ + {id: 1, public: true}, + {id: 2, writeAndDeleteOnly: true, adding: true}, + {id: 3, public: true, virtual: true, computed: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1, 3}, + }, + toEncode: indexes{ + { + primary: true, + keyCols: colIDs{1, 2}, + }, + }, + expCols: colIDs{1, 2, 3}, + expAdded: colIDs{2}, + expComputed: colIDs{3}, + expNeeded: colIDs{1}, + }, + { + name: "dropped columns are excluded", + cols: cols{ + {id: 1, public: true}, + {id: 2, writeAndDeleteOnly: true}, + {id: 3, public: true, virtual: true, computed: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1, 3}, + }, + toEncode: indexes{ + { + primary: true, + keyCols: colIDs{1, 3}, + }, + }, + expCols: colIDs{1, 3}, + expComputed: colIDs{3}, + expNeeded: colIDs{1}, + }, + { + // This is the case where we're building a new primary index as part + // of an add column for a computed stored column. + name: "physical adding computed column in primary index", + cols: cols{ + {id: 1, public: true}, + {id: 2, public: true, virtual: true, computed: true}, + {id: 3, adding: true, writeAndDeleteOnly: true, computed: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1, 2}, + }, + toEncode: indexes{ + { + primary: true, + keyCols: colIDs{2, 1, 3}, + }, + }, + expCols: colIDs{1, 2, 3}, + expNeeded: colIDs{1}, + expComputed: colIDs{2, 3}, + }, + { + name: "physical adding computed column in primary index but not adding", + cols: cols{ + {id: 1, public: true}, + {id: 2, public: true, virtual: true, computed: true}, + {id: 3, writeAndDeleteOnly: true, computed: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1, 2}, + }, + toEncode: indexes{ + { + primary: true, + keyCols: colIDs{2, 1, 3}, + }, + }, + expErr: "index being backfilled contains non-writable or dropping column", + }, + { + name: "physical adding computed column in primary index but not writable", + cols: cols{ + {id: 1, public: true}, + {id: 2, public: true, virtual: true, computed: true}, + {id: 3, adding: true, computed: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1, 2}, + }, + toEncode: indexes{ + { + primary: true, + keyCols: colIDs{2, 1, 3}, + }, + }, + expErr: "index being backfilled contains non-writable or dropping column", + }, + { + name: "secondary index with new column", + cols: cols{ + {id: 1, public: true}, + {id: 2, adding: true, writeAndDeleteOnly: true}, + {id: 3, public: true, virtual: true, computed: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1, 3}, + }, + toEncode: indexes{ + { + keyCols: colIDs{1, 2, 3}, + }, + }, + expErr: "secondary index for backfill contains physical column not present in source primary index", + }, + { + name: "secondary index and primary index with new column", + cols: cols{ + {id: 1, public: true}, + {id: 2, adding: true, writeAndDeleteOnly: true}, + {id: 3, public: true, virtual: true, computed: true}, + }, + src: index{ + primary: true, + keyCols: colIDs{1, 3}, + }, + toEncode: indexes{ + { + primary: true, + keyCols: colIDs{1, 2}, + }, + { + keyCols: colIDs{1, 2, 3}, + }, + }, + expErr: "secondary index for backfill contains physical column not present in source primary index", + }, + } { + t.Run(tc.name, func(t *testing.T) { + out, err := makeIndexBackfillColumns( + asColumnSlice(tc.cols), tc.src, asIndexSlice(tc.toEncode), + ) + if tc.expErr != "" { + require.Regexp(t, tc.expErr, err) + return + } + + // Validate that all the columns are classified as expected. + idToCol := make(map[descpb.ColumnID]catalog.Column) + for _, c := range tc.cols { + idToCol[c.id] = c + } + toColumnSlice := func(ids colIDs) (out []catalog.Column) { + for _, id := range ids { + out = append(out, idToCol[id]) + } + return out + } + + require.Equal(t, toColumnSlice(tc.expCols), out.cols) + require.Equal(t, toColumnSlice(tc.expComputed), out.computedCols) + require.Equal(t, toColumnSlice(tc.expAdded), out.addedCols) + var needed catalog.TableColSet + out.valNeededForCol.ForEach(func(i int) { + needed.Add(out.cols[i].GetID()) + }) + require.Equal(t, catalog.MakeTableColSet(tc.expNeeded...).Ordered(), needed.Ordered()) + }) + + } +} + +type fakeColumn struct { + catalog.Column + id descpb.ColumnID + public, adding, writeAndDeleteOnly bool + computed, virtual bool +} + +func (fc fakeColumn) Public() bool { return fc.public } +func (fc fakeColumn) Adding() bool { return fc.adding } +func (fc fakeColumn) WriteAndDeleteOnly() bool { return fc.writeAndDeleteOnly } +func (fc fakeColumn) IsComputed() bool { return fc.computed } +func (fc fakeColumn) IsVirtual() bool { return fc.virtual } + +func (fc fakeColumn) GetID() descpb.ColumnID { + return fc.id +} + +type fakeIndex struct { + catalog.Index + primary bool + keyCols []descpb.ColumnID + keySuffixCols []descpb.ColumnID + secondaryValueCols []descpb.ColumnID + primaryValueCols []descpb.ColumnID +} + +func (fi fakeIndex) CollectKeyColumnIDs() catalog.TableColSet { + return catalog.MakeTableColSet(fi.keyCols...) +} +func (fi fakeIndex) CollectKeySuffixColumnIDs() catalog.TableColSet { + return catalog.MakeTableColSet(fi.keySuffixCols...) +} +func (fi fakeIndex) CollectPrimaryStoredColumnIDs() catalog.TableColSet { + return catalog.MakeTableColSet(fi.primaryValueCols...) +} +func (fi fakeIndex) CollectSecondaryStoredColumnIDs() catalog.TableColSet { + return catalog.MakeTableColSet(fi.secondaryValueCols...) +} + +func (fi fakeIndex) GetEncodingType() descpb.IndexDescriptorEncodingType { + if fi.primary { + return descpb.PrimaryIndexEncoding + } + return descpb.SecondaryIndexEncoding +} diff --git a/pkg/sql/catalog/table_col_set.go b/pkg/sql/catalog/table_col_set.go index 661d3cfc5819..be6d3fc6acb9 100644 --- a/pkg/sql/catalog/table_col_set.go +++ b/pkg/sql/catalog/table_col_set.go @@ -53,6 +53,21 @@ func (s TableColSet) ForEach(f func(col descpb.ColumnID)) { s.set.ForEach(func(i int) { f(descpb.ColumnID(i)) }) } +// SubsetOf returns true if s is a subset of other. +func (s TableColSet) SubsetOf(other TableColSet) bool { + return s.set.SubsetOf(other.set) +} + +// Intersection returns the intersection between s and other. +func (s TableColSet) Intersection(other TableColSet) TableColSet { + return TableColSet{set: s.set.Intersection(other.set)} +} + +// Difference returns the column IDs in s which are not in other. +func (s TableColSet) Difference(other TableColSet) TableColSet { + return TableColSet{set: s.set.Difference(other.set)} +} + // Ordered returns a slice with all the descpb.ColumnIDs in the set, in // increasing order. func (s TableColSet) Ordered() []descpb.ColumnID { diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index 9cff355d9a7f..2c004025a723 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -2486,3 +2486,25 @@ CREATE TABLE public.t5 ( rowid INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(), CONSTRAINT t5_pkey PRIMARY KEY (rowid ASC) ) WITH (sql_stats_automatic_collection_min_stale_rows = 1234, sql_stats_automatic_collection_fraction_stale_rows = 0.15) + +subtest add_volatile_unique_column_81448 + +statement ok +create table t81448 (a int primary key); +insert into t81448 values (1), (2), (3) + +statement ok +alter table t81448 add column b float8 unique default (random()) + +statement ok +create table t81448_b as select b from t81448@t81448_pkey; + +# Make sure the rows in the index are the same as the rows in the table. + +query I +select (select count(*) from t81448@t81448_b_key bk inner join t81448_b cp on (cp.b = bk.b)) +---- +3 + +statement ok +drop table t81448, t81448_b