Skip to content

Commit 606cc0d

Browse files
craig[bot]fqazi
andcommitted
Merge #141850
141850: sql/schemachanger: fix unusable secondary indexes during pk swap r=fqazi a=fqazi Previously, the declarative schema changer during ALTER PRIMARY KEY, the declarative schema changer would make the new primary index public and then work on creating replacement secondary indexes. This could lead to broken behaviour where you could be left with either unusable old secondary indexes (causing trouble for DML queries) or potential validation errors. To address this, this patch adjusts the rules so that the new primary index is kept at a validated state, while replacement secondary indexes are constructed. This guarantees that all secondary indexes are usable during a PK swap, and the new primary key and replacement secondary indexes are swapped in an atomic fashion. Additionally, to help support this change the index backfiller has been updated, so that a source index can be specified for backfilling indexes. This allows the index backfiller to use a validated primary index to create the secondary index, without exposing it to read queries. Fixes: #133129 Fixes: #130165 Release note (bug fix): Addressed a bug where secondary indexes could be unusable by DML while a primary key swap was occurring, if the new primary key did not contain columns from the old primary key. Co-authored-by: Faizan Qazi <[email protected]>
2 parents b58f2af + 6cf5326 commit 606cc0d

File tree

142 files changed

+10481
-2832
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

142 files changed

+10481
-2832
lines changed

pkg/ccl/changefeedccl/schemafeed/testdata/alter_primary_key

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -17,29 +17,31 @@ t 2->3: Unknown
1717
t 3->4: Unknown
1818
t 4->5: Unknown
1919
t 5->6: Unknown
20-
t 6->7: PrimaryKeyChange
20+
t 6->7: Unknown
2121
t 7->8: Unknown
2222
t 8->9: Unknown
2323
t 9->10: Unknown
2424
t 10->11: Unknown
25-
t 11->12: Unknown
25+
t 11->12: PrimaryKeyChange
2626
t 12->13: Unknown
27+
t 13->14: Unknown
2728

2829
exec
2930
ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k, j);
3031
----
3132

3233
pop f=1
3334
----
34-
t 13->14: Unknown
3535
t 14->15: Unknown
3636
t 15->16: Unknown
3737
t 16->17: Unknown
3838
t 17->18: Unknown
39-
t 18->19: PrimaryKeyChange
39+
t 18->19: Unknown
4040
t 19->20: Unknown
4141
t 20->21: Unknown
4242
t 21->22: Unknown
4343
t 22->23: Unknown
4444
t 23->24: Unknown
45-
t 24->25: Unknown
45+
t 24->25: PrimaryKeyChange
46+
t 25->26: Unknown
47+
t 26->27: Unknown

pkg/ccl/schemachangerccl/backup_base_generated_test.go

Lines changed: 28 additions & 0 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

pkg/ccl/schemachangerccl/testdata/decomp/partitioning

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -525,6 +525,7 @@ ElementState:
525525
isNotVisible: false
526526
isUnique: false
527527
recreateSourceId: 0
528+
recreateTargetId: 0
528529
sharding: null
529530
sourceIndexId: 0
530531
tableId: 104
@@ -960,6 +961,7 @@ ElementState:
960961
isNotVisible: false
961962
isUnique: true
962963
recreateSourceId: 0
964+
recreateTargetId: 0
963965
sharding: null
964966
sourceIndexId: 0
965967
tableId: 105

pkg/ccl/testccl/sqlccl/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,6 @@ go_test(
4343
"//pkg/sql",
4444
"//pkg/sql/catalog/lease",
4545
"//pkg/sql/catalog/replication",
46-
"//pkg/sql/colexecerror",
4746
"//pkg/sql/gcjob",
4847
"//pkg/sql/isql",
4948
"//pkg/sql/lexbase",

pkg/ccl/testccl/sqlccl/explain_test.go

Lines changed: 0 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,6 @@ import (
1313

1414
"github.com/cockroachdb/cockroach/pkg/base"
1515
"github.com/cockroachdb/cockroach/pkg/internal/sqlsmith"
16-
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
1716
"github.com/cockroachdb/cockroach/pkg/sql/lexbase"
1817
"github.com/cockroachdb/cockroach/pkg/sql/sqltestutils"
1918
"github.com/cockroachdb/cockroach/pkg/sql/tests"
@@ -139,11 +138,6 @@ func TestExplainGist(t *testing.T) {
139138
skip.UnderDeadlock(t, "the test is too slow")
140139
skip.UnderRace(t, "the test is too slow")
141140

142-
// Use the release-build panic-catching behavior instead of the
143-
// crdb_test-build behavior. This is needed so that some known bugs like
144-
// #119045 and #133129 don't result in a test failure.
145-
defer colexecerror.ProductionBehaviorForTests()()
146-
147141
ctx := context.Background()
148142
rng, _ := randutil.NewTestRand()
149143

pkg/sql/backfill.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1045,7 +1045,7 @@ func (sc *SchemaChanger) distIndexBackfill(
10451045
)
10461046
indexBatchSize := indexBackfillBatchSize.Get(&sc.execCfg.Settings.SV)
10471047
chunkSize := sc.getChunkSize(indexBatchSize)
1048-
spec, err := initIndexBackfillerSpec(*tableDesc.TableDesc(), writeAsOf, readAsOf, writeAtRequestTimestamp, chunkSize, addedIndexes)
1048+
spec, err := initIndexBackfillerSpec(*tableDesc.TableDesc(), writeAsOf, readAsOf, writeAtRequestTimestamp, chunkSize, addedIndexes, 0 /* sourceIndexID*/)
10491049
if err != nil {
10501050
return err
10511051
}

pkg/sql/backfill/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ go_library(
2626
"//pkg/sql/execinfra",
2727
"//pkg/sql/execinfrapb",
2828
"//pkg/sql/isql",
29+
"//pkg/sql/parser",
2930
"//pkg/sql/pgwire/pgcode",
3031
"//pkg/sql/pgwire/pgerror",
3132
"//pkg/sql/row",

pkg/sql/backfill/backfill.go

Lines changed: 18 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -480,6 +480,10 @@ type IndexBackfiller struct {
480480
// backfilled.
481481
indexesToEncode []catalog.Index
482482

483+
// sourceIndex the primary index that should be used to execute this
484+
// backfill.
485+
sourceIndex catalog.Index
486+
483487
// keyPrefixes is a slice of key prefixes for each index in indexesToEncode.
484488
// indexesToEncode and keyPrefixes should both have the same ordering.
485489
keyPrefixes [][]byte
@@ -513,7 +517,7 @@ func (ib *IndexBackfiller) InitForLocalUse(
513517
) error {
514518

515519
// Initialize ib.added.
516-
ib.initIndexes(evalCtx.Codec, desc, nil /* allowList */)
520+
ib.initIndexes(evalCtx.Codec, desc, nil /* allowList */, 0 /*sourceIndex*/)
517521

518522
// Initialize ib.cols and ib.colIdxMap.
519523
if err := ib.initCols(desc); err != nil {
@@ -649,14 +653,15 @@ func (ib *IndexBackfiller) InitForDistributedUse(
649653
flowCtx *execinfra.FlowCtx,
650654
desc catalog.TableDescriptor,
651655
allowList []catid.IndexID,
656+
sourceIndexID catid.IndexID,
652657
mon *mon.BytesMonitor,
653658
) error {
654659
// We'll be modifying the eval.Context in BuildIndexEntriesChunk, so we need
655660
// to make a copy.
656661
evalCtx := flowCtx.NewEvalCtx()
657662

658663
// Initialize ib.added.
659-
ib.initIndexes(evalCtx.Codec, desc, allowList)
664+
ib.initIndexes(evalCtx.Codec, desc, allowList, sourceIndexID)
660665

661666
// Initialize ib.indexBackfillerCols.
662667
if err := ib.initCols(desc); err != nil {
@@ -734,7 +739,7 @@ func (ib *IndexBackfiller) ShrinkBoundAccount(ctx context.Context, shrinkBy int6
734739
// populates the cols and colIdxMap fields.
735740
func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) (err error) {
736741
ib.indexBackfillerCols, err = makeIndexBackfillColumns(
737-
desc.DeletableColumns(), desc.GetPrimaryIndex(), ib.added,
742+
desc, desc.DeletableColumns(), ib.sourceIndex, ib.added,
738743
)
739744
return err
740745
}
@@ -746,7 +751,10 @@ func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) (err error) {
746751
// If `allowList` is non-nil, we only add those in this list.
747752
// If `allowList` is nil, we add all adding index mutations.
748753
func (ib *IndexBackfiller) initIndexes(
749-
codec keys.SQLCodec, desc catalog.TableDescriptor, allowList []catid.IndexID,
754+
codec keys.SQLCodec,
755+
desc catalog.TableDescriptor,
756+
allowList []catid.IndexID,
757+
sourceIndexID catid.IndexID,
750758
) {
751759
var allowListAsSet catid.IndexSet
752760
if len(allowList) > 0 {
@@ -755,6 +763,11 @@ func (ib *IndexBackfiller) initIndexes(
755763

756764
mutations := desc.AllMutations()
757765
mutationID := mutations[0].MutationID()
766+
if sourceIndexID != 0 {
767+
ib.sourceIndex = catalog.FindIndexByID(desc, sourceIndexID)
768+
} else {
769+
ib.sourceIndex = desc.GetPrimaryIndex()
770+
}
758771
ib.keyPrefixes = make([][]byte, 0, len(ib.added))
759772
// Mutations in the same transaction have the same ID. Loop through the
760773
// mutations and collect all index mutations.
@@ -860,7 +873,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
860873
// read or used
861874
var spec fetchpb.IndexFetchSpec
862875
if err := rowenc.InitIndexFetchSpec(
863-
&spec, ib.evalCtx.Codec, tableDesc, tableDesc.GetPrimaryIndex(), fetcherCols,
876+
&spec, ib.evalCtx.Codec, tableDesc, ib.sourceIndex, fetcherCols,
864877
); err != nil {
865878
return nil, nil, memUsedPerChunk, err
866879
}

pkg/sql/backfill/index_backfiller_cols.go

Lines changed: 73 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,8 @@ package backfill
88
import (
99
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
1010
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb"
11+
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
12+
"github.com/cockroachdb/cockroach/pkg/sql/parser"
1113
"github.com/cockroachdb/cockroach/pkg/util/intsets"
1214
"github.com/cockroachdb/errors"
1315
)
@@ -52,7 +54,10 @@ type indexBackfillerCols struct {
5254
// requirements due to references in expressions. That needs to be added after
5355
// constructing this information.
5456
func makeIndexBackfillColumns(
55-
deletableColumns []catalog.Column, sourcePrimaryIndex catalog.Index, addedIndexes []catalog.Index,
57+
tableDesc catalog.TableDescriptor,
58+
deletableColumns []catalog.Column,
59+
sourcePrimaryIndex catalog.Index,
60+
addedIndexes []catalog.Index,
5661
) (indexBackfillerCols, error) {
5762

5863
// We will need to evaluate default or computed expressions for
@@ -71,6 +76,14 @@ func makeIndexBackfillColumns(
7176
for _, idx := range addedIndexes {
7277
allIndexColumns.UnionWith(indexColumns(idx))
7378
}
79+
if tableDesc != nil {
80+
var err error
81+
// Add any dependent columns needed to evaluate expressions.
82+
allIndexColumns, err = addIndexColumnsFromExpressions(allIndexColumns, tableDesc, addedIndexes)
83+
if err != nil {
84+
return indexBackfillerCols{}, err
85+
}
86+
}
7487
for _, column := range deletableColumns {
7588
if !column.Public() &&
7689
// Include columns we are adding, in case we are adding them to a
@@ -86,6 +99,23 @@ func makeIndexBackfillColumns(
8699
allIndexColumns.Contains(column.GetID())) {
87100
continue
88101
}
102+
// Public columns that are not in the source primary index and not
103+
// needed by the secondary index can be skipped. If there are virtual
104+
// columns that are needed by the secondary index then pick those up.
105+
if column.Public() && !primaryColumns.Contains(column.GetID()) {
106+
// If a non-virtual column requested by the secondary index is missing
107+
// we are going to error out.
108+
if allIndexColumns.Contains(column.GetID()) && !column.IsVirtual() {
109+
return indexBackfillerCols{}, errors.AssertionFailedf(
110+
"column %s is public but not in the source primary index",
111+
column.GetName(),
112+
)
113+
}
114+
// If the column is not needed by the secondary index then we can skip it.
115+
if !allIndexColumns.Contains(column.GetID()) {
116+
continue
117+
}
118+
}
89119
if column.IsComputed() && column.IsVirtual() {
90120
computedVirtual.Add(column.GetID())
91121
ib.computedCols = append(ib.computedCols, column)
@@ -183,3 +213,45 @@ func indexColumns(idx catalog.Index) (s catalog.TableColSet) {
183213
s.UnionWith(idx.CollectSecondaryStoredColumnIDs())
184214
return s
185215
}
216+
217+
// addIndexColumnsFromExpressions takes a set of columns stored in an index,
218+
// and computes any dependent columns needed for computed expressions or partial
219+
// indexes.
220+
func addIndexColumnsFromExpressions(
221+
s catalog.TableColSet, table catalog.TableDescriptor, addIndexes []catalog.Index,
222+
) (catalog.TableColSet, error) {
223+
addReferencesFromExpression := func(expression string) error {
224+
expr, err := parser.ParseExpr(expression)
225+
if err != nil {
226+
return err
227+
}
228+
referencedColumns, err := schemaexpr.ExtractColumnIDs(table, expr)
229+
if err != nil {
230+
return err
231+
}
232+
for _, colID := range referencedColumns.Ordered() {
233+
s.Add(colID)
234+
}
235+
return nil
236+
}
237+
// First get any columns needed to compute virtual expressions.
238+
for _, colID := range s.Ordered() {
239+
column := catalog.FindColumnByID(table, colID)
240+
if !column.IsVirtual() || !column.IsComputed() {
241+
continue
242+
}
243+
if err := addReferencesFromExpression(column.GetComputeExpr()); err != nil {
244+
return s, err
245+
}
246+
}
247+
// Next get any expressions needed to evaluate index predicates.
248+
for _, idx := range addIndexes {
249+
if len(idx.GetPredicate()) == 0 {
250+
continue
251+
}
252+
if err := addReferencesFromExpression(idx.GetPredicate()); err != nil {
253+
return s, err
254+
}
255+
}
256+
return s, nil
257+
}

pkg/sql/backfill/index_backfiller_cols_test.go

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -113,9 +113,8 @@ func TestIndexBackfillerColumns(t *testing.T) {
113113
keyCols: colIDs{1},
114114
},
115115
},
116-
expCols: colIDs{1, 2, 3},
117-
expComputed: colIDs{3},
118-
expNeeded: colIDs{1},
116+
expCols: colIDs{1, 2},
117+
expNeeded: colIDs{1},
119118
},
120119
{
121120
name: "one virtual, one computed mutation column in primary",
@@ -333,7 +332,7 @@ func TestIndexBackfillerColumns(t *testing.T) {
333332
} {
334333
t.Run(tc.name, func(t *testing.T) {
335334
out, err := makeIndexBackfillColumns(
336-
asColumnSlice(tc.cols), tc.src, asIndexSlice(tc.toEncode),
335+
nil, asColumnSlice(tc.cols), tc.src, asIndexSlice(tc.toEncode),
337336
)
338337
if tc.expErr != "" {
339338
require.Regexp(t, tc.expErr, err)
@@ -412,15 +411,15 @@ func TestInitIndexesAllowList(t *testing.T) {
412411
t.Run("nil allowList", func(t *testing.T) {
413412
// A nil allowList means no filtering.
414413
ib := &IndexBackfiller{}
415-
ib.initIndexes(keys.SystemSQLCodec, desc, nil /* allowList */)
414+
ib.initIndexes(keys.SystemSQLCodec, desc, nil /* allowList */, 0 /* sourceIndexID */)
416415
require.Equal(t, 2, len(ib.added))
417416
require.Equal(t, catid.IndexID(2), ib.added[0].GetID())
418417
require.Equal(t, catid.IndexID(3), ib.added[1].GetID())
419418
})
420419

421420
t.Run("non-nil allowList", func(t *testing.T) {
422421
ib := &IndexBackfiller{}
423-
ib.initIndexes(keys.SystemSQLCodec, desc, []catid.IndexID{3} /* allowList */)
422+
ib.initIndexes(keys.SystemSQLCodec, desc, []catid.IndexID{3} /* allowList */, 0 /* sourceIndexID */)
424423
require.Equal(t, 1, len(ib.added))
425424
require.Equal(t, catid.IndexID(3), ib.added[0].GetID())
426425
})

0 commit comments

Comments
 (0)