Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 7 additions & 5 deletions pkg/ccl/changefeedccl/schemafeed/testdata/alter_primary_key
Original file line number Diff line number Diff line change
Expand Up @@ -17,29 +17,31 @@ t 2->3: Unknown
t 3->4: Unknown
t 4->5: Unknown
t 5->6: Unknown
t 6->7: PrimaryKeyChange
t 6->7: Unknown
t 7->8: Unknown
t 8->9: Unknown
t 9->10: Unknown
t 10->11: Unknown
t 11->12: Unknown
t 11->12: PrimaryKeyChange
t 12->13: Unknown
t 13->14: Unknown

exec
ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k, j);
----

pop f=1
----
t 13->14: Unknown
t 14->15: Unknown
t 15->16: Unknown
t 16->17: Unknown
t 17->18: Unknown
t 18->19: PrimaryKeyChange
t 18->19: Unknown
t 19->20: Unknown
t 20->21: Unknown
t 21->22: Unknown
t 22->23: Unknown
t 23->24: Unknown
t 24->25: Unknown
t 24->25: PrimaryKeyChange
t 25->26: Unknown
t 26->27: Unknown
28 changes: 28 additions & 0 deletions pkg/ccl/schemachangerccl/backup_base_generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 2 additions & 0 deletions pkg/ccl/schemachangerccl/testdata/decomp/partitioning
Original file line number Diff line number Diff line change
Expand Up @@ -525,6 +525,7 @@ ElementState:
isNotVisible: false
isUnique: false
recreateSourceId: 0
recreateTargetId: 0
sharding: null
sourceIndexId: 0
tableId: 104
Expand Down Expand Up @@ -960,6 +961,7 @@ ElementState:
isNotVisible: false
isUnique: true
recreateSourceId: 0
recreateTargetId: 0
sharding: null
sourceIndexId: 0
tableId: 105
Expand Down
1 change: 0 additions & 1 deletion pkg/ccl/testccl/sqlccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@ go_test(
"//pkg/sql",
"//pkg/sql/catalog/lease",
"//pkg/sql/catalog/replication",
"//pkg/sql/colexecerror",
"//pkg/sql/gcjob",
"//pkg/sql/isql",
"//pkg/sql/lexbase",
Expand Down
6 changes: 0 additions & 6 deletions pkg/ccl/testccl/sqlccl/explain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/internal/sqlsmith"
"github.com/cockroachdb/cockroach/pkg/sql/colexecerror"
"github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/sqltestutils"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
Expand Down Expand Up @@ -138,11 +137,6 @@ func TestExplainGist(t *testing.T) {
skip.UnderDeadlock(t, "the test is too slow")
skip.UnderRace(t, "the test is too slow")

// Use the release-build panic-catching behavior instead of the
// crdb_test-build behavior. This is needed so that some known bugs like
// #119045 and #133129 don't result in a test failure.
defer colexecerror.ProductionBehaviorForTests()()

ctx := context.Background()
rng, _ := randutil.NewTestRand()

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -1045,7 +1045,7 @@ func (sc *SchemaChanger) distIndexBackfill(
)
indexBatchSize := indexBackfillBatchSize.Get(&sc.execCfg.Settings.SV)
chunkSize := sc.getChunkSize(indexBatchSize)
spec, err := initIndexBackfillerSpec(*tableDesc.TableDesc(), writeAsOf, readAsOf, writeAtRequestTimestamp, chunkSize, addedIndexes)
spec, err := initIndexBackfillerSpec(*tableDesc.TableDesc(), writeAsOf, readAsOf, writeAtRequestTimestamp, chunkSize, addedIndexes, 0 /* sourceIndexID*/)
if err != nil {
return err
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/backfill/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ go_library(
"//pkg/sql/execinfra",
"//pkg/sql/execinfrapb",
"//pkg/sql/isql",
"//pkg/sql/parser",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/row",
Expand Down
23 changes: 18 additions & 5 deletions pkg/sql/backfill/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -479,6 +479,10 @@ type IndexBackfiller struct {
// backfilled.
indexesToEncode []catalog.Index

// sourceIndex the primary index that should be used to execute this
// backfill.
sourceIndex catalog.Index

// keyPrefixes is a slice of key prefixes for each index in indexesToEncode.
// indexesToEncode and keyPrefixes should both have the same ordering.
keyPrefixes [][]byte
Expand Down Expand Up @@ -512,7 +516,7 @@ func (ib *IndexBackfiller) InitForLocalUse(
) error {

// Initialize ib.added.
ib.initIndexes(evalCtx.Codec, desc, nil /* allowList */)
ib.initIndexes(evalCtx.Codec, desc, nil /* allowList */, 0 /*sourceIndex*/)

// Initialize ib.cols and ib.colIdxMap.
if err := ib.initCols(desc); err != nil {
Expand Down Expand Up @@ -648,14 +652,15 @@ func (ib *IndexBackfiller) InitForDistributedUse(
flowCtx *execinfra.FlowCtx,
desc catalog.TableDescriptor,
allowList []catid.IndexID,
sourceIndexID catid.IndexID,
mon *mon.BytesMonitor,
) error {
// We'll be modifying the eval.Context in BuildIndexEntriesChunk, so we need
// to make a copy.
evalCtx := flowCtx.NewEvalCtx()

// Initialize ib.added.
ib.initIndexes(evalCtx.Codec, desc, allowList)
ib.initIndexes(evalCtx.Codec, desc, allowList, sourceIndexID)

// Initialize ib.indexBackfillerCols.
if err := ib.initCols(desc); err != nil {
Expand Down Expand Up @@ -733,7 +738,7 @@ func (ib *IndexBackfiller) ShrinkBoundAccount(ctx context.Context, shrinkBy int6
// populates the cols and colIdxMap fields.
func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) (err error) {
ib.indexBackfillerCols, err = makeIndexBackfillColumns(
desc.DeletableColumns(), desc.GetPrimaryIndex(), ib.added,
desc, desc.DeletableColumns(), ib.sourceIndex, ib.added,
)
return err
}
Expand All @@ -745,7 +750,10 @@ func (ib *IndexBackfiller) initCols(desc catalog.TableDescriptor) (err error) {
// If `allowList` is non-nil, we only add those in this list.
// If `allowList` is nil, we add all adding index mutations.
func (ib *IndexBackfiller) initIndexes(
codec keys.SQLCodec, desc catalog.TableDescriptor, allowList []catid.IndexID,
codec keys.SQLCodec,
desc catalog.TableDescriptor,
allowList []catid.IndexID,
sourceIndexID catid.IndexID,
) {
var allowListAsSet catid.IndexSet
if len(allowList) > 0 {
Expand All @@ -754,6 +762,11 @@ func (ib *IndexBackfiller) initIndexes(

mutations := desc.AllMutations()
mutationID := mutations[0].MutationID()
if sourceIndexID != 0 {
ib.sourceIndex = catalog.FindIndexByID(desc, sourceIndexID)
} else {
ib.sourceIndex = desc.GetPrimaryIndex()
}
ib.keyPrefixes = make([][]byte, 0, len(ib.added))
// Mutations in the same transaction have the same ID. Loop through the
// mutations and collect all index mutations.
Expand Down Expand Up @@ -859,7 +872,7 @@ func (ib *IndexBackfiller) BuildIndexEntriesChunk(
// read or used
var spec fetchpb.IndexFetchSpec
if err := rowenc.InitIndexFetchSpec(
&spec, ib.evalCtx.Codec, tableDesc, tableDesc.GetPrimaryIndex(), fetcherCols,
&spec, ib.evalCtx.Codec, tableDesc, ib.sourceIndex, fetcherCols,
); err != nil {
return nil, nil, memUsedPerChunk, err
}
Expand Down
74 changes: 73 additions & 1 deletion pkg/sql/backfill/index_backfiller_cols.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,8 @@ package backfill
import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/util/intsets"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -52,7 +54,10 @@ type indexBackfillerCols struct {
// 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,
tableDesc catalog.TableDescriptor,
deletableColumns []catalog.Column,
sourcePrimaryIndex catalog.Index,
addedIndexes []catalog.Index,
) (indexBackfillerCols, error) {

// We will need to evaluate default or computed expressions for
Expand All @@ -71,6 +76,14 @@ func makeIndexBackfillColumns(
for _, idx := range addedIndexes {
allIndexColumns.UnionWith(indexColumns(idx))
}
if tableDesc != nil {
var err error
// Add any dependent columns needed to evaluate expressions.
allIndexColumns, err = addIndexColumnsFromExpressions(allIndexColumns, tableDesc, addedIndexes)
if err != nil {
return indexBackfillerCols{}, err
}
}
for _, column := range deletableColumns {
if !column.Public() &&
// Include columns we are adding, in case we are adding them to a
Expand All @@ -86,6 +99,23 @@ func makeIndexBackfillColumns(
allIndexColumns.Contains(column.GetID())) {
continue
}
// Public columns that are not in the source primary index and not
// needed by the secondary index can be skipped. If there are virtual
// columns that are needed by the secondary index then pick those up.
if column.Public() && !primaryColumns.Contains(column.GetID()) {
// If a non-virtual column requested by the secondary index is missing
// we are going to error out.
if allIndexColumns.Contains(column.GetID()) && !column.IsVirtual() {
return indexBackfillerCols{}, errors.AssertionFailedf(
"column %s is public but not in the source primary index",
column.GetName(),
)
}
// If the column is not needed by the secondary index then we can skip it.
if !allIndexColumns.Contains(column.GetID()) {
continue
}
}
if column.IsComputed() && column.IsVirtual() {
computedVirtual.Add(column.GetID())
ib.computedCols = append(ib.computedCols, column)
Expand Down Expand Up @@ -183,3 +213,45 @@ func indexColumns(idx catalog.Index) (s catalog.TableColSet) {
s.UnionWith(idx.CollectSecondaryStoredColumnIDs())
return s
}

// addIndexColumnsFromExpressions takes a set of columns stored in an index,
// and computes any dependent columns needed for computed expressions or partial
// indexes.
func addIndexColumnsFromExpressions(
s catalog.TableColSet, table catalog.TableDescriptor, addIndexes []catalog.Index,
) (catalog.TableColSet, error) {
addReferencesFromExpression := func(expression string) error {
expr, err := parser.ParseExpr(expression)
if err != nil {
return err
}
referencedColumns, err := schemaexpr.ExtractColumnIDs(table, expr)
if err != nil {
return err
}
for _, colID := range referencedColumns.Ordered() {
s.Add(colID)
}
return nil
}
// First get any columns needed to compute virtual expressions.
for _, colID := range s.Ordered() {
column := catalog.FindColumnByID(table, colID)
if !column.IsVirtual() || !column.IsComputed() {
continue
}
if err := addReferencesFromExpression(column.GetComputeExpr()); err != nil {
return s, err
}
}
// Next get any expressions needed to evaluate index predicates.
for _, idx := range addIndexes {
if len(idx.GetPredicate()) == 0 {
continue
}
if err := addReferencesFromExpression(idx.GetPredicate()); err != nil {
return s, err
}
}
return s, nil
}
11 changes: 5 additions & 6 deletions pkg/sql/backfill/index_backfiller_cols_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,9 +113,8 @@ func TestIndexBackfillerColumns(t *testing.T) {
keyCols: colIDs{1},
},
},
expCols: colIDs{1, 2, 3},
expComputed: colIDs{3},
expNeeded: colIDs{1},
expCols: colIDs{1, 2},
expNeeded: colIDs{1},
},
{
name: "one virtual, one computed mutation column in primary",
Expand Down Expand Up @@ -333,7 +332,7 @@ func TestIndexBackfillerColumns(t *testing.T) {
} {
t.Run(tc.name, func(t *testing.T) {
out, err := makeIndexBackfillColumns(
asColumnSlice(tc.cols), tc.src, asIndexSlice(tc.toEncode),
nil, asColumnSlice(tc.cols), tc.src, asIndexSlice(tc.toEncode),
)
if tc.expErr != "" {
require.Regexp(t, tc.expErr, err)
Expand Down Expand Up @@ -412,15 +411,15 @@ func TestInitIndexesAllowList(t *testing.T) {
t.Run("nil allowList", func(t *testing.T) {
// A nil allowList means no filtering.
ib := &IndexBackfiller{}
ib.initIndexes(keys.SystemSQLCodec, desc, nil /* allowList */)
ib.initIndexes(keys.SystemSQLCodec, desc, nil /* allowList */, 0 /* sourceIndexID */)
require.Equal(t, 2, len(ib.added))
require.Equal(t, catid.IndexID(2), ib.added[0].GetID())
require.Equal(t, catid.IndexID(3), ib.added[1].GetID())
})

t.Run("non-nil allowList", func(t *testing.T) {
ib := &IndexBackfiller{}
ib.initIndexes(keys.SystemSQLCodec, desc, []catid.IndexID{3} /* allowList */)
ib.initIndexes(keys.SystemSQLCodec, desc, []catid.IndexID{3} /* allowList */, 0 /* sourceIndexID */)
require.Equal(t, 1, len(ib.added))
require.Equal(t, catid.IndexID(3), ib.added[0].GetID())
})
Expand Down
9 changes: 3 additions & 6 deletions pkg/sql/catalog/tabledesc/validate.go
Original file line number Diff line number Diff line change
Expand Up @@ -1791,13 +1791,10 @@ func (desc *wrapper) validateTableIndexes(
}

// When newPKColIDs is not empty, it means there is an in-progress `ALTER
// PRIMARY KEY`. We don't allow queueing schema changes when there's a
// primary key mutation, so it's safe to make the assumption that `Adding`
// indexes are associated with the new primary key because they are
// rewritten and `Non-adding` indexes should only contain virtual column
// from old primary key.
// PRIMARY KEY`. Certain schema changes will make the new virtual columns
// public earlier than the new primary key, which should be acceptable.
isOldPKCol := !idx.Adding() && curPKColIDs.Contains(colID)
isNewPKCol := idx.Adding() && newPKColIDs.Contains(colID)
isNewPKCol := newPKColIDs.Contains(colID)
if newPKColIDs.Len() > 0 && (isOldPKCol || isNewPKCol) {
continue
}
Expand Down
Loading