-
Notifications
You must be signed in to change notification settings - Fork 3.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
{backup,changefeed,streaming}ccl: start populating pts target #74248
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,7 +17,9 @@ import ( | |
"github.com/cockroachdb/cockroach/pkg/keys" | ||
"github.com/cockroachdb/cockroach/pkg/kv" | ||
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" | ||
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" | ||
"github.com/cockroachdb/cockroach/pkg/roachpb" | ||
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" | ||
"github.com/cockroachdb/cockroach/pkg/util/hlc" | ||
"github.com/cockroachdb/cockroach/pkg/util/log" | ||
"github.com/cockroachdb/cockroach/pkg/util/uuid" | ||
|
@@ -64,12 +66,26 @@ func createProtectedTimestampRecord( | |
progress.ProtectedTimestampRecord = uuid.MakeV4() | ||
log.VEventf(ctx, 2, "creating protected timestamp %v at %v", | ||
progress.ProtectedTimestampRecord, resolved) | ||
spansToProtect := makeSpansToProtect(codec, targets) | ||
deprecatedSpansToProtect := makeSpansToProtect(codec, targets) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. we need both targets and spans? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Until we run the migration in #74281 we want to continue protecting spans so that jobs started in a mixed version state do not fail. Once the migration is complete, the The idea is that GC for 22.1 will continue to respect both spans protected by the old subsystem, and targets protected by the new subsystem since this simplifies the migration in a mixed version state. In 22.2 with some elbow grease, we should be able to stop populating the spans field in the record entirely. |
||
targetToProtect := makeTargetToProtect(targets) | ||
rec := jobsprotectedts.MakeRecord( | ||
progress.ProtectedTimestampRecord, int64(jobID), resolved, spansToProtect, jobsprotectedts.Jobs) | ||
progress.ProtectedTimestampRecord, int64(jobID), resolved, deprecatedSpansToProtect, | ||
jobsprotectedts.Jobs, targetToProtect) | ||
return pts.Protect(ctx, txn, rec) | ||
} | ||
|
||
func makeTargetToProtect(targets jobspb.ChangefeedTargets) *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, len(targets)+1) | ||
for t := range targets { | ||
tablesToProtect = append(tablesToProtect, t) | ||
} | ||
tablesToProtect = append(tablesToProtect, keys.DescriptorTableID) | ||
return ptpb.MakeSchemaObjectsTarget(tablesToProtect) | ||
} | ||
|
||
func makeSpansToProtect(codec keys.SQLCodec, targets jobspb.ChangefeedTargets) []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 | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,34 @@ | ||
// Copyright 2021 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 ptpb | ||
|
||
import ( | ||
"github.com/cockroachdb/cockroach/pkg/roachpb" | ||
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" | ||
) | ||
|
||
// MakeClusterTarget returns a target, which when used in a Record, will | ||
// protect the entire keyspace of the cluster. | ||
func MakeClusterTarget() *Target { | ||
return &Target{&Target_Cluster{Cluster: &Target_ClusterTarget{}}} | ||
} | ||
|
||
// MakeTenantsTarget returns a target, which when used in a Record, will | ||
// protect the keyspace of all tenants in ids. | ||
func MakeTenantsTarget(ids []roachpb.TenantID) *Target { | ||
return &Target{&Target_Tenants{Tenants: &Target_TenantsTarget{IDs: ids}}} | ||
} | ||
|
||
// MakeSchemaObjectsTarget returns a target, which when used in a Record, | ||
// will protect the keyspace of all schema objects (database/table). | ||
func MakeSchemaObjectsTarget(ids descpb.IDs) *Target { | ||
return &Target{&Target_SchemaObjects{SchemaObjects: &Target_SchemaObjectsTarget{IDs: ids}}} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Currently BACKUP syntax doesn't allow backing up table 245 and tenant 123, but that is a valid backup job. It looks like however that is not possible with the ptpb.Target? i.e. tenants target and schema object targets are mutually exclusive?
Not blocking, since it's unreachable with current syntax, just wondering.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's not possible today yeah, we could in the future switch the Record to hold a
repeated Target
instead of a single target to accommodate for this.