Skip to content

Commit 67f33c2

Browse files
craig[bot]fqazimsbutler
committed
153961: catalog/lease: update locked timestamps when updates are received r=fqazi a=fqazi Previously, the lease manager would increase its locked read time stamp was when a check point occurred on the range feed. This was not adequate because this could cause schema changes could be slowed down if the old version of descriptors was held. While the current design does not intentionally reserve, previous versions, a later update will start holding old versions intentionally to serve queries. To address this, this patch has schema changes write the list of descriptors modified to a special key `/Table/Descriptors/2/<First ID in Schema Change`, which will be used by the lease manager range feed to to update timestamps. Additionally, this patch also deflakes and fixes `TestLeaseManagerLockedTimestampBasic`, which was not setup correctly. Informs: #153618 Fixes: #153826 Fixes: #154075 Release note: None 154866: roachtest: deflake c2c/disconnect r=jeffswenson a=msbutler Previously, the c2c/disconnect roachtest was disconnecting two source nodes, instead of a pair of source/destination nodes. This patch fixes this bug. Informs: #152248 Informs: #150474 Release note: none 154977: roachtest: deflake backup/mvcc-range-tombstones r=kev-cao a=msbutler Informs #154841 Release note: none Co-authored-by: Faizan Qazi <[email protected]> Co-authored-by: Michael Butler <[email protected]>
4 parents a03a1a0 + 2c7f831 + 5d3dc54 + 315684c commit 67f33c2

File tree

21 files changed

+424
-45
lines changed

21 files changed

+424
-45
lines changed

pkg/backup/backup_test.go

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -6687,7 +6687,6 @@ INSERT INTO baz.bar VALUES (110, 'a'), (210, 'b'), (310, 'c'), (410, 'd'), (510,
66876687
tenant10.Exec(t, `BACKUP DATABASE baz INTO 'userfile://defaultdb.myfililes/test4' with revision_history`)
66886688
expected = nil
66896689
for _, resume := range []exportResumePoint{
6690-
{mkSpan(id2, "/Tenant/10/Table/3", "/Tenant/10/Table/4"), withoutTS},
66916690
{mkSpan(id2, "/Tenant/10/Table/:id/1", "/Tenant/10/Table/:id/2"), withoutTS},
66926691
{mkSpan(id2, "/Tenant/10/Table/:id/1/210", "/Tenant/10/Table/:id/2"), withoutTS},
66936692
// We have two entries for 210 because of history and super small table

pkg/backup/targets.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -185,7 +185,7 @@ func getAllDescChanges(
185185
startTime, endTime hlc.Timestamp,
186186
priorIDs map[descpb.ID]descpb.ID,
187187
) ([]backuppb.BackupManifest_DescriptorRevision, error) {
188-
startKey := codec.TablePrefix(keys.DescriptorTableID)
188+
startKey := codec.IndexPrefix(keys.DescriptorTableID, keys.DescriptorTablePrimaryKeyIndexID)
189189
endKey := startKey.PrefixEnd()
190190

191191
g := ctxgroup.WithContext(ctx)

pkg/ccl/changefeedccl/schemafeed/schema_feed.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -831,7 +831,7 @@ func (tf *schemaFeed) fetchDescriptorVersions(
831831
}
832832
codec := tf.leaseMgr.Codec()
833833
start := crtime.NowMono()
834-
span := roachpb.Span{Key: codec.TablePrefix(keys.DescriptorTableID)}
834+
span := roachpb.Span{Key: codec.IndexPrefix(keys.DescriptorTableID, keys.DescriptorTablePrimaryKeyIndexID)}
835835
span.EndKey = span.Key.PrefixEnd()
836836

837837
tf.mu.Lock()

pkg/ccl/changefeedccl/schemafeed/schematestutils/schema_test_utils.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -148,7 +148,7 @@ func FetchDescVersionModificationTime(
148148
db := serverutils.OpenDBConn(
149149
t, s.SQLAddr(), dbName, false, s.AppStopper())
150150

151-
tblKey := s.Codec().TablePrefix(keys.DescriptorTableID)
151+
tblKey := s.Codec().IndexPrefix(keys.DescriptorTableID, keys.DescriptorTablePrimaryKeyIndexID)
152152
header := kvpb.RequestHeader{
153153
Key: tblKey,
154154
EndKey: tblKey.PrefixEnd(),

pkg/cmd/roachtest/tests/backup.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -756,7 +756,7 @@ func runBackupMVCCRangeTombstones(
756756
`IMPORT INTO orders CSV DATA ('%s') WITH delimiter='|', detached`,
757757
strings.Join(files, "', '")),
758758
).Scan(&jobID))
759-
waitForState(jobID, jobs.StatePaused, "", 30*time.Minute)
759+
waitForState(jobID, jobs.StatePaused, "", time.Hour)
760760

761761
t.Status("canceling import")
762762
_, err = conn.ExecContext(ctx, fmt.Sprintf(`CANCEL JOB %s`, jobID))

pkg/cmd/roachtest/tests/cluster_to_cluster.go

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -2057,17 +2057,19 @@ func registerClusterReplicationDisconnect(r registry.Registry) {
20572057
var dstNode int
20582058
srcTenantSQL.QueryRow(t, `select split_part(consumer, '[', 1) from crdb_internal.cluster_replication_node_streams order by random() limit 1`).Scan(&dstNode)
20592059

2060+
roachprodDstNode := dstNode + sp.srcNodes
2061+
20602062
disconnectDuration := sp.additionalDuration
20612063
rd.t.L().Printf("Disconnecting Src %d, Dest %d for %.2f minutes", srcNode,
2062-
dstNode, disconnectDuration.Minutes())
2064+
roachprodDstNode, disconnectDuration.Minutes())
20632065

20642066
// Normally, the blackholeFailer is accessed through the failer interface,
20652067
// at least in the failover tests. Because this test shouldn't use all the
20662068
// failer interface calls (e.g. Setup(), and Ready()), we use the
20672069
// blakholeFailer struct directly. In other words, in this test, we
20682070
// shouldn't treat the blackholeFailer as an abstracted api.
20692071
blackholeFailer := &blackholeFailer{t: rd.t, c: rd.c, input: true, output: true}
2070-
blackholeFailer.FailPartial(ctx, srcNode, []int{dstNode})
2072+
blackholeFailer.FailPartial(ctx, srcNode, []int{roachprodDstNode})
20712073

20722074
time.Sleep(disconnectDuration)
20732075
// Calling this will log the latest topology.

pkg/config/system.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -211,9 +211,9 @@ func (s *SystemConfig) GetLargestObjectID(
211211
// Search for the descriptor table entries within the SystemConfig. lowIndex
212212
// (in s.Values) is the first and highIndex one past the last KV pair in the
213213
// descriptor table.
214-
lowBound := keys.SystemSQLCodec.TablePrefix(keys.DescriptorTableID)
214+
lowBound := keys.SystemSQLCodec.IndexPrefix(keys.DescriptorTableID, keys.DescriptorTablePrimaryKeyIndexID)
215215
lowIndex := s.getIndexBound(lowBound)
216-
highBound := keys.SystemSQLCodec.TablePrefix(keys.DescriptorTableID + 1)
216+
highBound := keys.SystemSQLCodec.IndexPrefix(keys.DescriptorTableID+1, keys.DescriptorTablePrimaryKeyIndexID)
217217
highIndex := s.getIndexBound(highBound)
218218
if lowIndex == highIndex {
219219
return 0, fmt.Errorf("descriptor table not found in system config of %d values", len(s.Values))

pkg/keys/constants.go

Lines changed: 8 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -426,9 +426,14 @@ const (
426426
ZonesTableConfigColumnID = 2
427427
ZonesTableConfigColFamID = 2
428428

429-
DescriptorTablePrimaryKeyIndexID = 1
430-
DescriptorTableDescriptorColID = 2
431-
DescriptorTableDescriptorColFamID = 2
429+
DescriptorTablePrimaryKeyIndexID = 1
430+
DescriptorTableDescriptorColID = 2
431+
DescriptorTableDescriptorColFamID = 2
432+
// DescriptorTableDescriptorUpdateIndexID is not a real index. It is a special
433+
// ID used to construct index entries that inform the lease subsystem of
434+
// descriptor updates within a transaction. The value for such an entry is a
435+
// descpb.DescriptorUpdates message.
436+
DescriptorTableDescriptorUpdateIndexID = 2
432437
TenantsTablePrimaryKeyIndexID = 1
433438
SpanConfigurationsTablePrimaryKeyIndexID = 1
434439
CommentsTablePrimaryKeyIndexID = 1

pkg/keys/sql.go

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -209,6 +209,12 @@ func (e sqlEncoder) DescMetadataPrefix() roachpb.Key {
209209
return e.IndexPrefix(DescriptorTableID, DescriptorTablePrimaryKeyIndexID)
210210
}
211211

212+
// DescUpdatePrefix returns the key prefix for all descriptors in the
213+
// system.descriptor table.
214+
func (e sqlEncoder) DescUpdatePrefix() roachpb.Key {
215+
return e.IndexPrefix(DescriptorTableID, DescriptorTableDescriptorUpdateIndexID)
216+
}
217+
212218
// DescMetadataKey returns the key for the descriptor in the system.descriptor
213219
// table.
214220
func (e sqlEncoder) DescMetadataKey(descID uint32) roachpb.Key {
@@ -217,6 +223,14 @@ func (e sqlEncoder) DescMetadataKey(descID uint32) roachpb.Key {
217223
return MakeFamilyKey(k, DescriptorTableDescriptorColFamID)
218224
}
219225

226+
// DescMetadataUpdateKey returns the key for the descriptor in the system.descriptor
227+
// table.
228+
func (e sqlEncoder) DescMetadataUpdateKey(descID uint32) roachpb.Key {
229+
k := e.DescUpdatePrefix()
230+
k = encoding.EncodeUvarintAscending(k, uint64(descID))
231+
return MakeFamilyKey(k, 0)
232+
}
233+
220234
// TenantMetadataKey returns the key for the tenant metadata in the
221235
// system.tenants table.
222236
func (e sqlEncoder) TenantMetadataKey(tenID roachpb.TenantID) roachpb.Key {
@@ -307,6 +321,19 @@ func (d sqlDecoder) DecodeDescMetadataID(key roachpb.Key) (uint32, error) {
307321
return uint32(id), nil
308322
}
309323

324+
// DecodeDescUpdateKey decodes a descriptor ID from a descriptor metadata key.
325+
func (d sqlDecoder) DecodeDescUpdateKey(key roachpb.Key) (bool, error) {
326+
// Extract table and index ID from key.
327+
_, tableID, indexID, err := d.DecodeIndexPrefix(key)
328+
if err != nil {
329+
return false, err
330+
}
331+
if tableID != DescriptorTableID {
332+
return false, errors.Errorf("key is not a descriptor table entry: %v", key)
333+
}
334+
return indexID == DescriptorTableDescriptorUpdateIndexID, nil
335+
}
336+
310337
// DecodeTenantMetadataID decodes a tenant ID from a tenant metadata key.
311338
func (d sqlDecoder) DecodeTenantMetadataID(key roachpb.Key) (roachpb.TenantID, error) {
312339
// Extract table and index ID from key.

pkg/server/diagnostics/reporter.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -449,7 +449,7 @@ func (r *Reporter) populateSQLInfo(uptime int64, sql *diagnosticspb.SQLInstanceI
449449
// type fields. Check out `schematelemetry` package for a better data source for
450450
// collecting redacted schema information.
451451
func (r *Reporter) collectSchemaInfo(ctx context.Context) ([]descpb.TableDescriptor, error) {
452-
startKey := keys.MakeSQLCodec(r.TenantID).TablePrefix(keys.DescriptorTableID)
452+
startKey := keys.MakeSQLCodec(r.TenantID).IndexPrefix(keys.DescriptorTableID, keys.DescriptorTablePrimaryKeyIndexID)
453453
endKey := startKey.PrefixEnd()
454454
kvs, err := r.DB.Scan(ctx, startKey, endKey, 0)
455455
if err != nil {

0 commit comments

Comments
 (0)