Skip to content

Commit

Permalink
Merge #84761 #85059
Browse files Browse the repository at this point in the history
84761: schematelemetry,eventpb: add schema telemetry r=postamar a=postamar

This commit adds:
  - the event definitions and logic for generating them,
  - the scheduling and jobs boilerplate to periodically log them.

Care is taken to redact all strings present in descriptors which might
unintentionally be leaking PIIs.

The event generation logic is tested on the schema of a bootstrapped
test cluster: the test checks that the events match expectations.

Fixes #84284.

Release note (general change): CRDB will now collect schema info if
phoning home is enabled. This schema info is added to the telemetry log
by a built-in scheduled job which runs on a weekly basis by default.
This recurrence can be changed via the sql.schema.telemetry.recurrence
cluster setting.  The schedule can also be paused via PAUSE SCHEDULE
followed by its ID, which can be retrieved by querying
SELECT * FROM [SHOW SCHEDULES] WHERE label = 'sql-schema-telemetry'.

85059: admission,kvserver: improved byte token estimation for writes r=irfansharif,tbg a=sumeerbhola

The existing scheme for byte token estimation simply looked
at the total bytes added to L0 and divided it among the number
of requests. This was because (a) the parameters to provide
better size information for the request were not populated by
kvserver, (b) the basic estimation approach was flawed since
it assumed that regular writes would be roughly equal sized,
and assumed that ingests would tell what fraction went into L0.

The kvserver-side plumbing for improving (a) were done in a
preceding PR (#83937). This one completes that plumbing to pass on
admission.StoreWorkDoneInfo to the admission control package.
In this scheme the {WriteBytes,IngestedBytes} are provided
post-proposal evaluation, and the IngestedBytes is for the
whole LSM. This PR makes changes to the plumbing in the
admission package: specifically, the post-work-done token
adjustments are performed via the granterWithStoreWriteDone
interface and the addition to granterWithIOTokens. The former
also returns the token adjustments to StoreWorkQueue so
that the per-tenant fairness accounting in WorkQueue can be
updated.

The main changes in this PR are in the byte token
estimation logic in the admission package, where the
estimation now uses a linear model y=a.x + b, where
x is the bytes provided in admission.StoreWorkDoneInfo,
and y is the bytes added to L0 via write or ingestion.
If we consider regular writes, one can expect that even
with many different sized workloads concurrently being
active on a node, we should be able to fit a model where
a is roughly 2 and b is tiny -- this is because x is the
bytes written to the raft log and does not include the
subsequent state machine application. Similarly, one can
expect the a term being in the interval [0,1] for ingested
work. The linear model is meant to fix flaw (b) mentioned
earlier. The current linear model fitting in
store_token_estimation.go is very simple and can be
independently improved in the future -- there are code
comments outlining this. Additionally, all the byte token
estimation logic in granter.go has been removed, which
is better from a code readability perspective.

This change was evaluated with a single node that first
saw a kv0 workload that writes 64KB blocks, then
additionally a kv0 workload that writes 4KB blocks, and
finally a third workload that starts doing an index
backfill due to creating an index on the v column in
the kv table.

Here are snippets from a sequence of log statements when
only the first workload (64KB writes) was running:
```
write-model 1.46x+1 B (smoothed 1.50x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 78 KiB
write-model 1.37x+1 B (smoothed 1.36x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 80 KiB
write-model 1.50x+1 B (smoothed 1.43x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 79 KiB
write-model 1.39x+1 B (smoothed 1.30x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 77 KiB
```
Note that the parameter a, in a.x does fluctuate. The
additive value b stays at the minimum of 1 bytes, which
is desirable. There is no change to the starting ingest
model since there are no ingestions.

After both the 4KB and 64KB writes are active the log
statements look like:
```
write-model 1.85x+1 B (smoothed 1.78x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 59 KiB
write-model 1.23x+1 B (smoothed 1.51x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 47 KiB
write-model 1.21x+1 B (smoothed 1.36x+1 B) + ingested-model 0.00x+0 B (smoothed 0.75x+1 B) + at-admission-tokens 40 KiB
```
Note that the b value stays at 1 byte. The tokens consumed
at admission time are evenly divided among requests, so
the value has dropped.

When the index backfill is also running, the sstables are
ingested into L5 and L6, so the x value in the ingested
model is high, but what is ingested into L0 is low, which
means a becomes very small for the ingested-model -- see
the smoothed 0.00x+1 B below. There is choppiness in this
experiment wrt the write model and the at-admission-tokens,
which is caused by a high number of write stalls. This
was not planned for, and is a side-effect of huge Pebble
manifests caused by 64KB keys. So ignore those values in
the following log statements.
```
write-model 1.93x+1 B (smoothed 1.56x+2 B) + ingested-model 0.00x+1 B (smoothed 0.00x+1 B) + at-admission-tokens 120 KiB
write-model 2.34x+1 B (smoothed 1.95x+1 B) + ingested-model 0.00x+1 B (smoothed 0.00x+1 B) + at-admission-tokens 157 KiB
```

Fixes #79092
Informs #82536

Release note: None

Co-authored-by: Marius Posta <marius@cockroachlabs.com>
Co-authored-by: sumeerbhola <sumeer@cockroachlabs.com>
  • Loading branch information
3 people committed Aug 9, 2022
3 parents 283035c + 7370772 + c7396ee commit 9c840c6
Show file tree
Hide file tree
Showing 102 changed files with 3,342 additions and 713 deletions.
52 changes: 52 additions & 0 deletions docs/generated/eventlog.md
Original file line number Diff line number Diff line change
Expand Up @@ -2577,6 +2577,58 @@ contains common SQL event/execution details.
| `FullIndexScan` | Whether the query contains a full secondary index scan of a non-partial index. | no |
| `TxnCounter` | The sequence number of the SQL transaction inside its session. | no |

### `schema_descriptor`

An event of type `schema_descriptor` is an event for schema telemetry, whose purpose is
to take periodic snapshots of the cluster's SQL schema and publish them in
the telemetry log channel. For all intents and purposes, the data in such a
snapshot can be thought of the outer join of certain system tables:
namespace, descriptor, and at some point perhaps zones, etc.

Snapshots are too large to conveniently be published as a single log event,
so instead they're broken down into SchemaDescriptor events which
contain the data in one record of this outer join projection. These events
are prefixed by a header (a SchemaSnapshotMetadata event).


| Field | Description | Sensitive |
|--|--|--|
| `SnapshotID` | SnapshotID is the unique identifier of the snapshot that this event is part of. | no |
| `ParentDatabaseID` | ParentDatabaseID matches the same key column in system.namespace. | no |
| `ParentSchemaID` | ParentSchemaID matches the same key column in system.namespace. | no |
| `Name` | Name matches the same key column in system.namespace. | no |
| `DescID` | DescID matches the 'id' column in system.namespace and system.descriptor. | no |
| `Desc` | Desc matches the 'descriptor' column in system.descriptor. Some contents of the descriptor may be redacted to prevent leaking PII. | no |


#### Common fields

| Field | Description | Sensitive |
|--|--|--|
| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no |
| `EventType` | The type of the event. | no |

### `schema_snapshot_metadata`

An event of type `schema_snapshot_metadata` is an event describing a schema snapshot, which
is a set of SchemaDescriptor messages sharing the same SnapshotID.


| Field | Description | Sensitive |
|--|--|--|
| `SnapshotID` | SnapshotID is the unique identifier of this snapshot. | no |
| `NumRecords` | NumRecords is how many SchemaDescriptor events are in the snapshot. | no |
| `AsOfTimestamp` | AsOfTimestamp is when the snapshot was taken. This is equivalent to the timestamp given in the AS OF SYSTEM TIME clause when querying the namespace and descriptor tables in the system database. Expressed as nanoseconds since the Unix epoch. | no |
| `Errors` | Errors records any errors encountered when post-processing this snapshot, which includes the redaction of any potential PII. | yes |


#### Common fields

| Field | Description | Sensitive |
|--|--|--|
| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no |
| `EventType` | The type of the event. | no |

## Zone config events

Events in this category pertain to zone configuration changes on
Expand Down
3 changes: 2 additions & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -248,6 +248,7 @@ sql.multiple_modifications_of_table.enabled boolean false if true, allow stateme
sql.multiregion.drop_primary_region.enabled boolean true allows dropping the PRIMARY REGION of a database if it is the last region
sql.notices.enabled boolean true enable notices in the server/client protocol being sent
sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled boolean false if enabled, uniqueness checks may be planned for mutations of UUID columns updated with gen_random_uuid(); otherwise, uniqueness is assumed due to near-zero collision probability
sql.schema.telemetry.recurrence string @weekly cron-tab recurrence for SQL schema telemetry job
sql.spatial.experimental_box2d_comparison_operators.enabled boolean false enables the use of certain experimental box2d comparison operators
sql.stats.automatic_collection.enabled boolean true automatic statistics collection mode
sql.stats.automatic_collection.fraction_stale_rows float 0.2 target fraction of stale rows per table that will trigger a statistics refresh
Expand Down Expand Up @@ -284,4 +285,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 22.1-40 set the active cluster version in the format '<major>.<minor>'
version version 22.1-42 set the active cluster version in the format '<major>.<minor>'
3 changes: 2 additions & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -179,6 +179,7 @@
<tr><td><code>sql.multiregion.drop_primary_region.enabled</code></td><td>boolean</td><td><code>true</code></td><td>allows dropping the PRIMARY REGION of a database if it is the last region</td></tr>
<tr><td><code>sql.notices.enabled</code></td><td>boolean</td><td><code>true</code></td><td>enable notices in the server/client protocol being sent</td></tr>
<tr><td><code>sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled</code></td><td>boolean</td><td><code>false</code></td><td>if enabled, uniqueness checks may be planned for mutations of UUID columns updated with gen_random_uuid(); otherwise, uniqueness is assumed due to near-zero collision probability</td></tr>
<tr><td><code>sql.schema.telemetry.recurrence</code></td><td>string</td><td><code>@weekly</code></td><td>cron-tab recurrence for SQL schema telemetry job</td></tr>
<tr><td><code>sql.spatial.experimental_box2d_comparison_operators.enabled</code></td><td>boolean</td><td><code>false</code></td><td>enables the use of certain experimental box2d comparison operators</td></tr>
<tr><td><code>sql.stats.automatic_collection.enabled</code></td><td>boolean</td><td><code>true</code></td><td>automatic statistics collection mode</td></tr>
<tr><td><code>sql.stats.automatic_collection.fraction_stale_rows</code></td><td>float</td><td><code>0.2</code></td><td>target fraction of stale rows per table that will trigger a statistics refresh</td></tr>
Expand Down Expand Up @@ -215,6 +216,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>22.1-40</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>22.1-42</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
2 changes: 2 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -3023,6 +3023,8 @@ SELECT * FROM crdb_internal.check_consistency(true, ‘\x02’, ‘\x04’)</p>
</span></td><td>Volatile</td></tr>
<tr><td><a name="crdb_internal.create_session_revival_token"></a><code>crdb_internal.create_session_revival_token() &rarr; <a href="bytes.html">bytes</a></code></td><td><span class="funcdesc"><p>Generate a token that can be used to create a new session for the current user.</p>
</span></td><td>Volatile</td></tr>
<tr><td><a name="crdb_internal.create_sql_schema_telemetry_job"></a><code>crdb_internal.create_sql_schema_telemetry_job() &rarr; <a href="int.html">int</a></code></td><td><span class="funcdesc"><p>This function is used to create a schema telemetry job instance.</p>
</span></td><td>Volatile</td></tr>
<tr><td><a name="crdb_internal.decode_cluster_setting"></a><code>crdb_internal.decode_cluster_setting(setting: <a href="string.html">string</a>, value: <a href="string.html">string</a>) &rarr; <a href="string.html">string</a></code></td><td><span class="funcdesc"><p>Decodes the given encoded value for a cluster setting.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="crdb_internal.deserialize_session"></a><code>crdb_internal.deserialize_session(session: <a href="bytes.html">bytes</a>) &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>This function deserializes the serialized variables into the current session.</p>
Expand Down
6 changes: 6 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -274,6 +274,7 @@ ALL_TESTS = [
"//pkg/sql/catalog/resolver:resolver_test",
"//pkg/sql/catalog/schemadesc:schemadesc_test",
"//pkg/sql/catalog/schemaexpr:schemaexpr_test",
"//pkg/sql/catalog/schematelemetry:schematelemetry_test",
"//pkg/sql/catalog/seqexpr:seqexpr_disallowed_imports_test",
"//pkg/sql/catalog/seqexpr:seqexpr_test",
"//pkg/sql/catalog/systemschema_test:systemschema_test_test",
Expand Down Expand Up @@ -1315,6 +1316,9 @@ GO_TARGETS = [
"//pkg/sql/catalog/schemadesc:schemadesc_test",
"//pkg/sql/catalog/schemaexpr:schemaexpr",
"//pkg/sql/catalog/schemaexpr:schemaexpr_test",
"//pkg/sql/catalog/schematelemetry/schematelemetrycontroller:schematelemetrycontroller",
"//pkg/sql/catalog/schematelemetry:schematelemetry",
"//pkg/sql/catalog/schematelemetry:schematelemetry_test",
"//pkg/sql/catalog/seqexpr:seqexpr",
"//pkg/sql/catalog/seqexpr:seqexpr_test",
"//pkg/sql/catalog/systemschema:systemschema",
Expand Down Expand Up @@ -2468,6 +2472,8 @@ GET_X_DATA_TARGETS = [
"//pkg/sql/catalog/rewrite:get_x_data",
"//pkg/sql/catalog/schemadesc:get_x_data",
"//pkg/sql/catalog/schemaexpr:get_x_data",
"//pkg/sql/catalog/schematelemetry:get_x_data",
"//pkg/sql/catalog/schematelemetry/schematelemetrycontroller:get_x_data",
"//pkg/sql/catalog/seqexpr:get_x_data",
"//pkg/sql/catalog/systemschema:get_x_data",
"//pkg/sql/catalog/systemschema_test:get_x_data",
Expand Down
1 change: 1 addition & 0 deletions pkg/acceptance/cluster/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ go_library(
"//pkg/config/zonepb",
"//pkg/roachpb",
"//pkg/security",
"//pkg/security/certnames",
"//pkg/security/username",
"//pkg/util/contextutil",
"//pkg/util/log",
Expand Down
9 changes: 5 additions & 4 deletions pkg/acceptance/cluster/certs.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/security/certnames"
"github.com/cockroachdb/cockroach/pkg/security/username"
)

Expand Down Expand Up @@ -53,27 +54,27 @@ func GenerateCerts(ctx context.Context) func() {
maybePanic(os.RemoveAll(certsDir))

maybePanic(security.CreateCAPair(
certsDir, filepath.Join(certsDir, security.EmbeddedCAKey),
certsDir, filepath.Join(certsDir, certnames.EmbeddedCAKey),
keyLen, 96*time.Hour, false, false))

// Root user.
// Scope root user to system tenant and tenant ID 5 which is what we use by default for acceptance
// tests.
userScopes := []roachpb.TenantID{roachpb.SystemTenantID, roachpb.MakeTenantID(5)}
maybePanic(security.CreateClientPair(
certsDir, filepath.Join(certsDir, security.EmbeddedCAKey),
certsDir, filepath.Join(certsDir, certnames.EmbeddedCAKey),
keyLen, 48*time.Hour, false, username.RootUserName(), userScopes, true /* generate pk8 key */))

// Test user.
// Scope test user to system tenant and tenant ID 5 which is what we use by default for acceptance
// tests.
maybePanic(security.CreateClientPair(
certsDir, filepath.Join(certsDir, security.EmbeddedCAKey),
certsDir, filepath.Join(certsDir, certnames.EmbeddedCAKey),
keyLen, 48*time.Hour, false, username.TestUserName(), userScopes, true /* generate pk8 key */))

// Certs for starting a cockroach server. Key size is from cli/cert.go:defaultKeySize.
maybePanic(security.CreateNodePair(
certsDir, filepath.Join(certsDir, security.EmbeddedCAKey),
certsDir, filepath.Join(certsDir, certnames.EmbeddedCAKey),
keyLen, 48*time.Hour, false, []string{"localhost", "cockroach"}))

// Store a copy of the client certificate and private key in a PKCS#12
Expand Down
9 changes: 5 additions & 4 deletions pkg/acceptance/cluster/dockercluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/security/certnames"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/logflags"
Expand Down Expand Up @@ -451,7 +452,7 @@ func (l *DockerCluster) createNodeCerts() {
}
maybePanic(security.CreateNodePair(
certsDir,
filepath.Join(certsDir, security.EmbeddedCAKey),
filepath.Join(certsDir, certnames.EmbeddedCAKey),
keyLen, 48*time.Hour, true /* overwrite */, nodes))
}

Expand Down Expand Up @@ -772,9 +773,9 @@ func (l *DockerCluster) PGUrl(ctx context.Context, i int) string {
certUser := username.RootUser
options := url.Values{}
options.Add("sslmode", "verify-full")
options.Add("sslcert", filepath.Join(certsDir, security.EmbeddedRootCert))
options.Add("sslkey", filepath.Join(certsDir, security.EmbeddedRootKey))
options.Add("sslrootcert", filepath.Join(certsDir, security.EmbeddedCACert))
options.Add("sslcert", filepath.Join(certsDir, certnames.EmbeddedRootCert))
options.Add("sslkey", filepath.Join(certsDir, certnames.EmbeddedRootKey))
options.Add("sslrootcert", filepath.Join(certsDir, certnames.EmbeddedCACert))
pgURL := url.URL{
Scheme: "postgres",
User: url.User(certUser),
Expand Down
1 change: 1 addition & 0 deletions pkg/base/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ type TestingKnobs struct {
JobsTestingKnobs ModuleTestingKnobs
BackupRestore ModuleTestingKnobs
TTL ModuleTestingKnobs
SchemaTelemetry ModuleTestingKnobs
Streaming ModuleTestingKnobs
UpgradeManager ModuleTestingKnobs
IndexUsageStatsKnobs ModuleTestingKnobs
Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/democluster/demo_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -1048,7 +1048,7 @@ This server is running at increased risk of memory-related failures.`,

// generateCerts generates some temporary certificates for cockroach demo.
func (demoCtx *Context) generateCerts(certsDir string) (err error) {
caKeyPath := filepath.Join(certsDir, security.EmbeddedCAKey)
caKeyPath := filepath.Join(certsDir, certnames.EmbeddedCAKey)
// Create a CA-Key.
if err := security.CreateCAPair(
certsDir,
Expand All @@ -1075,7 +1075,7 @@ func (demoCtx *Context) generateCerts(certsDir string) (err error) {
// rootUserScope contains the tenant IDs the root user is allowed to access.
rootUserScope := []roachpb.TenantID{roachpb.SystemTenantID}
if demoCtx.Multitenant {
tenantCAKeyPath := filepath.Join(certsDir, security.EmbeddedTenantCAKey)
tenantCAKeyPath := filepath.Join(certsDir, certnames.EmbeddedTenantCAKey)
// Create a CA key for the tenants.
if err := security.CreateTenantCAPair(
certsDir,
Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cli/clisqlexec"
"github.com/cockroachdb/cockroach/pkg/cli/exit"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/security/certnames"
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server"
Expand Down Expand Up @@ -422,7 +422,7 @@ func (c TestCLI) RunWithCAArgs(origArgs []string) {
if err := func() error {
args := append([]string(nil), origArgs[:1]...)
if c.TestServer != nil {
args = append(args, fmt.Sprintf("--ca-key=%s", filepath.Join(c.certsDir, security.EmbeddedCAKey)))
args = append(args, fmt.Sprintf("--ca-key=%s", filepath.Join(c.certsDir, certnames.EmbeddedCAKey)))
args = append(args, fmt.Sprintf("--certs-dir=%s", c.certsDir))
}
args = append(args, origArgs[1:]...)
Expand Down
7 changes: 7 additions & 0 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -337,6 +337,9 @@ const (
UsersHaveIDs
// SetUserIDNotNull sets the user_id column in system.users to not null.
SetUserIDNotNull
// SQLSchemaTelemetryScheduledJobs adds an automatic schedule for SQL schema
// telemetry logging jobs.
SQLSchemaTelemetryScheduledJobs

// *************************************************
// Step (1): Add new versions here.
Expand Down Expand Up @@ -590,6 +593,10 @@ var versionsSingleton = keyedVersions{
Key: SetUserIDNotNull,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 40},
},
{
Key: SQLSchemaTelemetryScheduledJobs,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 42},
},
// *************************************************
// Step (2): Add new versions here.
// Do not add new versions to a patch release.
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.

Loading

0 comments on commit 9c840c6

Please sign in to comment.