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