Skip to content

Commit

Permalink
Merge pull request #130919 from cockroachdb/blathers/backport-release…
Browse files Browse the repository at this point in the history
…-23.2-130622

release-23.2: changefeedccl: protect system.comments and system.zones
  • Loading branch information
stevendanna authored Sep 27, 2024
2 parents 2db2376 + a8543e2 commit 45be842
Show file tree
Hide file tree
Showing 2 changed files with 68 additions and 9 deletions.
22 changes: 15 additions & 7 deletions pkg/ccl/changefeedccl/protected_timestamps.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,24 +42,30 @@ func createProtectedTimestampRecord(
jobsprotectedts.Jobs, targetToProtect)
}

// systemTablesToProtect holds the descriptor IDs of the system tables
// that need to be protected to ensure that a CHANGEFEED can do a
// historical read of a table descriptor.
var systemTablesToProtect = []descpb.ID{
keys.DescriptorTableID,
keys.CommentsTableID,
keys.ZonesTableID,
}

func makeTargetToProtect(targets changefeedbase.Targets) *ptpb.Target {
// NB: We add 1 because we're also going to protect system.descriptors.
// We protect system.descriptors because a changefeed needs all of the history
// of table descriptors to version data.
tablesToProtect := make(descpb.IDs, 0, targets.NumUniqueTables()+1)
tablesToProtect := make(descpb.IDs, 0, targets.NumUniqueTables()+len(systemTablesToProtect))
_ = targets.EachTableID(func(id descpb.ID) error {
tablesToProtect = append(tablesToProtect, id)
return nil
})
tablesToProtect = append(tablesToProtect, keys.DescriptorTableID)
tablesToProtect = append(tablesToProtect, systemTablesToProtect...)
return ptpb.MakeSchemaObjectsTarget(tablesToProtect)
}

func makeSpansToProtect(codec keys.SQLCodec, targets changefeedbase.Targets) []roachpb.Span {
// NB: We add 1 because we're also going to protect system.descriptors.
// We protect system.descriptors because a changefeed needs all of the history
// of table descriptors to version data.
spansToProtect := make([]roachpb.Span, 0, targets.NumUniqueTables()+1)
spansToProtect := make([]roachpb.Span, 0, targets.NumUniqueTables()+len(systemTablesToProtect))
addTablePrefix := func(id uint32) {
tablePrefix := codec.TablePrefix(id)
spansToProtect = append(spansToProtect, roachpb.Span{
Expand All @@ -71,6 +77,8 @@ func makeSpansToProtect(codec keys.SQLCodec, targets changefeedbase.Targets) []r
addTablePrefix(uint32(id))
return nil
})
addTablePrefix(keys.DescriptorTableID)
for _, id := range systemTablesToProtect {
addTablePrefix(uint32(id))
}
return spansToProtect
}
55 changes: 53 additions & 2 deletions pkg/ccl/changefeedccl/protected_timestamps_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader"
"github.com/cockroachdb/cockroach/pkg/sql"
Expand Down Expand Up @@ -449,7 +450,7 @@ func TestPTSRecordProtectsTargetsAndDescriptorTable(t *testing.T) {
defer stopServer()
execCfg := s.ExecutorConfig().(sql.ExecutorConfig)
sqlDB := sqlutils.MakeSQLRunner(db)

sqlDB.Exec(t, `ALTER DATABASE system CONFIGURE ZONE USING gc.ttlseconds = 1`)
sqlDB.Exec(t, "CREATE TABLE foo (a INT, b STRING)")
ts := s.Clock().Now()
ctx := context.Background()
Expand All @@ -466,10 +467,60 @@ func TestPTSRecordProtectsTargetsAndDescriptorTable(t *testing.T) {
return execCfg.ProtectedTimestampProvider.WithTxn(txn).Protect(ctx, ptr)
}))

// The following code was shameless stolen from
// TestShowTenantFingerprintsProtectsTimestamp which almost
// surely copied it from the 2-3 other tests that have
// something similar. We should put this in a helper. We have
// ForceTableGC, but in ad-hoc testing that appeared to bypass
// the PTS record making it useless for this test.
//
// TODO(ssd): Make a helper that does this.
refreshPTSReaderCache := func(asOf hlc.Timestamp, tableName, databaseName string) {
tableID, err := s.QueryTableID(ctx, username.RootUserName(), tableName, databaseName)
require.NoError(t, err)
tableKey := s.Codec().TablePrefix(uint32(tableID))
store, err := s.StorageLayer().GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID())
require.NoError(t, err)
var repl *kvserver.Replica
testutils.SucceedsSoon(t, func() error {
repl = store.LookupReplica(roachpb.RKey(tableKey))
if repl == nil {
return errors.New("could not find replica")
}
return nil
})
ptsReader := store.GetStoreConfig().ProtectedTimestampReader
t.Logf("updating PTS reader cache to %s", asOf)
require.NoError(
t,
spanconfigptsreader.TestingRefreshPTSState(ctx, t, ptsReader, asOf),
)
require.NoError(t, repl.ReadProtectedTimestampsForTesting(ctx))
}
gcTestTableRange := func(tableName, databaseName string) {
row := sqlDB.QueryRow(t, fmt.Sprintf("SELECT range_id FROM [SHOW RANGES FROM TABLE %s.%s]", tableName, databaseName))
var rangeID int64
row.Scan(&rangeID)
refreshPTSReaderCache(s.Clock().Now(), tableName, databaseName)
t.Logf("enqueuing range %d for mvccGC", rangeID)
sqlDB.Exec(t, `SELECT crdb_internal.kv_enqueue_replica($1, 'mvccGC', true)`, rangeID)
}

// Alter foo few times, then force GC at ts-1.
sqlDB.Exec(t, "ALTER TABLE foo ADD COLUMN c STRING")
sqlDB.Exec(t, "ALTER TABLE foo ADD COLUMN d STRING")
require.NoError(t, s.ForceTableGC(ctx, "system", "descriptor", ts.Add(-1, 0)))
time.Sleep(2 * time.Second)
// If you want to GC all system tables:
//
// tabs := systemschema.MakeSystemTables()
// for _, t := range tabs {
// if t.IsPhysicalTable() && !t.IsSequence() {
// gcTestTableRange("system", t.GetName())
// }
// }
gcTestTableRange("system", "descriptor")
gcTestTableRange("system", "zones")
gcTestTableRange("system", "comments")

// We can still fetch table descriptors because of protected timestamp record.
asOf := ts
Expand Down

0 comments on commit 45be842

Please sign in to comment.