Skip to content

Commit

Permalink
upgrades: Added an cluster upgrade from v22.1 to v22.2
Browse files Browse the repository at this point in the history
This upgrade attempts to update invalid column IDs in sequence back
references, if any, on a best-effort basis.

The context of the need for such an upgrade can be found in cockroachdb#82576.
The summary there is bugs in prior versions might cause sequence
descriptor corruption where their back references might contain a column
ID 0. We ought to figure out what the actual column ID is and update
such invalid column IDs.
  • Loading branch information
Xiang-Gu committed Aug 26, 2022
1 parent f1f0697 commit b7be578
Show file tree
Hide file tree
Showing 8 changed files with 170 additions and 4 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -294,4 +294,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using
trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https://<ui>/#/debug/tracez
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.
version version 1000022.1-64 set the active cluster version in the format '<major>.<minor>'
version version 1000022.1-66 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -228,6 +228,6 @@
<tr><td><code>trace.opentelemetry.collector</code></td><td>string</td><td><code></code></td><td>address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.</td></tr>
<tr><td><code>trace.span_registry.enabled</code></td><td>boolean</td><td><code>true</code></td><td>if set, ongoing traces can be seen at https://<ui>/#/debug/tracez</td></tr>
<tr><td><code>trace.zipkin.collector</code></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>1000022.1-64</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>1000022.1-66</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
8 changes: 8 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -290,6 +290,10 @@ const (
// GCHintInReplicaState adds GC hint to replica state. When this version is
// enabled, replicas will populate GC hint and update them when necessary.
GCHintInReplicaState
// UpdateInvalidColumnIDsInSequenceBackReferences looks for invalid column
// ids in sequences' back references and attempts a best-effort-based matching
// to update those column IDs.
UpdateInvalidColumnIDsInSequenceBackReferences

// *************************************************
// Step (1): Add new versions here.
Expand Down Expand Up @@ -484,6 +488,10 @@ var rawVersionsSingleton = keyedVersions{
Key: GCHintInReplicaState,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 64},
},
{
Key: UpdateInvalidColumnIDsInSequenceBackReferences,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 66},
},

// *************************************************
// Step (2): Add new versions here.
Expand Down
5 changes: 3 additions & 2 deletions pkg/clusterversion/key_string.go

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

10 changes: 10 additions & 0 deletions pkg/sql/catalog/descpb/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,16 @@ func (ids IDs) Len() int { return len(ids) }
func (ids IDs) Less(i, j int) bool { return ids[i] < ids[j] }
func (ids IDs) Swap(i, j int) { ids[i], ids[j] = ids[j], ids[i] }

// Contains returns whether `ids` contains `targetID`.
func (ids IDs) Contains(targetID ID) bool {
for _, id := range ids {
if id == targetID {
return true
}
}
return false
}

// FormatVersion is a custom type for TableDescriptor versions of the sql to
// key:value mapping.
//go:generate stringer -type=FormatVersion
Expand Down
1 change: 1 addition & 0 deletions pkg/upgrade/upgrades/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ go_library(
"system_external_connections.go",
"system_privileges.go",
"system_users_role_id_migration.go",
"update_invalid_column_ids_in_sequence_back_references.go",
"upgrade_sequence_to_be_referenced_by_ID.go",
"upgrades.go",
"wait_for_del_range_in_gc_job.go",
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,141 @@
// 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 upgrades

import (
"context"
"reflect"
"sort"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/upgrade"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)

func updateInvalidColumnIDsInSequenceBackReferences(
ctx context.Context, _ clusterversion.ClusterVersion, d upgrade.TenantDeps, _ *jobs.Job,
) error {
// Scan all sequences and repair those with 0-valued column IDs in their back references,
// one transaction for each repair.
var lastSeqID descpb.ID

for {
var currSeqID descpb.ID
var done bool
if err := d.CollectionFactory.TxnWithExecutor(ctx, d.DB, d.SessionData, func(
ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor,
) (err error) {
currSeqID = lastSeqID
for {
done, currSeqID, err = findNextTableToUpgrade(ctx, ie, currSeqID,
func(table *descpb.TableDescriptor) bool {
return table.IsSequence()
})
if err != nil || done {
return err
}

// Sequence `nextIdToUpgrade` might contain back reference with invalid column IDs. If so, we need to
// update them with valid column IDs.
hasUpgrade, err := maybeUpdateInvalidColumnIdsInSequenceBackReferences(ctx, txn, currSeqID, descriptors)
if err != nil {
return err
}
if hasUpgrade {
return nil
}
}
}); err != nil {
return err
}

// Break out of the loop if we upgraded all sequences.
if done {
break
}
lastSeqID = currSeqID
}

return nil
}

// maybeUpdateInvalidColumnIdsInSequenceBackReferences looks at sequence, identified
// by `idToUpgrade`, and upgrade invalid column IDs in its back references, if any.
//
// The upgrade works by reconstructing the referenced column IDs in each back reference
// in the sequence's `DependedOnBy` field. Reconstruction is possible with the help of
// the `UsesSequenceIDs` field of each column in the referencing table.
//
// A canonical example to create a corrupt sequence descriptor with invalid column IDs is
// to run the following in v21.1:
// `CREATE SEQUENCE s; CREATE TABLE t (i INT PRIMARY KEY);`
// followed by
// `ALTER TABLE t ADD COLUMN j INT DEFAULT nextval('s'), ADD COLUMN k INT DEFAULT nextval('s')`
// which erroneously added the back reference in sequence `s` before allocating a column ID
// to the newly added column, causing the reference in `s.DependedOnBy` to be
// `ref.ColumnIDs = [0, 0]` while it should instead be `ref.ColumnIDs = [2, 3]`.
// This upgrade logic will look at the `UsesSequenceIds` field of each column (`i`, `j`, and `k`)
// in `t`. In this case, exactly two columns (`j` and `k`) will have `UsesSequenceIds` that
// contains id of `s`. We thus reconstruct and update `ref.ColumnIDs` to be `[2, 3]`.
func maybeUpdateInvalidColumnIdsInSequenceBackReferences(
ctx context.Context, txn *kv.Txn, idToUpgrade descpb.ID, descriptors *descs.Collection,
) (hasUpgraded bool, err error) {
// Get the sequence descriptor that we are going to upgrade.
seqDesc, err := descriptors.GetMutableTableByID(ctx, txn, idToUpgrade, tree.ObjectLookupFlagsWithRequired())
if err != nil {
return false, err
}
if !seqDesc.IsSequence() {
return false, errors.AssertionFailedf("input id to upgrade %v is is not a sequence", idToUpgrade)
}

for i, ref := range seqDesc.DependedOnBy {
// Re-construct the expected column IDs in `ref` and update
// `ref.ColumnIDs` if the actual value is not equal to the
// expected value.
expectedColumnIDsInRef := make([]descpb.ColumnID, 0)
tableDesc, err := descriptors.GetMutableTableByID(ctx, txn, ref.ID, tree.ObjectLookupFlagsWithRequired())
if err != nil {
return false, err
}
for _, col := range tableDesc.GetColumns() {
if descpb.IDs(col.UsesSequenceIds).Contains(seqDesc.ID) {
expectedColumnIDsInRef = append(expectedColumnIDsInRef, col.ID)
}
}
sort.Slice(expectedColumnIDsInRef, func(i, j int) bool {
return expectedColumnIDsInRef[i] < expectedColumnIDsInRef[j]
})

if !reflect.DeepEqual(ref.ColumnIDs, expectedColumnIDsInRef) {
seqDesc.DependedOnBy[i].ColumnIDs = expectedColumnIDsInRef
hasUpgraded = true
}
}

if hasUpgraded {
// Write the updated sequence descriptor to storage.
log.Infof(ctx, "updated invalid column IDs in back references in sequence %v (%v)",
seqDesc.Name, idToUpgrade)
if err = descriptors.WriteDesc(ctx, false, seqDesc, txn); err != nil {
return false, err
}
}

return hasUpgraded, err
}
5 changes: 5 additions & 0 deletions pkg/upgrade/upgrades/upgrades.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,6 +154,11 @@ var upgrades = []upgrade.Upgrade{
NoPrecondition,
waitForAllSchemaChanges,
),
upgrade.NewTenantUpgrade("update invalid column IDs in sequence back references",
toCV(clusterversion.UpdateInvalidColumnIDsInSequenceBackReferences),
NoPrecondition,
updateInvalidColumnIDsInSequenceBackReferences,
),
}

func init() {
Expand Down

0 comments on commit b7be578

Please sign in to comment.