From d3c4f16eab5f6b6f4792cfe4f81681b11fe9ef1d Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 17 Jul 2023 15:03:44 -0700 Subject: [PATCH 1/4] sqlnoccltest: extract new test package that doesn't import CCL This allows for the single test in this package (that expects that CCL code is not loaded) to pass when `ccl` package is loaded by `sql` tests. There is currently no enforcement that `sqlnoccltest` doesn't import `ccl`. This adjustment is needed because the following commit introduces a dependency on `ccl` into `sql_test`. The alternative solution would be to adjust a couple of expected error messages in `TestRemovePartitioningOSS` to match the behavior when `ccl` is loaded. Release note: None --- pkg/BUILD.bazel | 2 ++ pkg/sql/BUILD.bazel | 1 - pkg/sql/sqlnoccltest/BUILD.bazel | 27 ++++++++++++++++++++ pkg/sql/sqlnoccltest/main_test.go | 27 ++++++++++++++++++++ pkg/sql/{ => sqlnoccltest}/partition_test.go | 2 +- 5 files changed, 57 insertions(+), 2 deletions(-) create mode 100644 pkg/sql/sqlnoccltest/BUILD.bazel create mode 100644 pkg/sql/sqlnoccltest/main_test.go rename pkg/sql/{ => sqlnoccltest}/partition_test.go (99%) diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 787de18de815..100f0e42a109 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -556,6 +556,7 @@ ALL_TESTS = [ "//pkg/sql/sqlitelogictest/tests/local:local_test", "//pkg/sql/sqlliveness/slinstance:slinstance_test", "//pkg/sql/sqlliveness/slstorage:slstorage_test", + "//pkg/sql/sqlnoccltest:sqlnoccltest_test", "//pkg/sql/sqlstats/insights/integration:integration_test", "//pkg/sql/sqlstats/insights:insights_test", "//pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil:sqlstatsutil_test", @@ -2052,6 +2053,7 @@ GO_TARGETS = [ "//pkg/sql/sqlliveness/slstorage:slstorage_test", "//pkg/sql/sqlliveness/sqllivenesstestutils:sqllivenesstestutils", "//pkg/sql/sqlliveness:sqlliveness", + "//pkg/sql/sqlnoccltest:sqlnoccltest_test", "//pkg/sql/sqlstats/insights/integration:integration_test", "//pkg/sql/sqlstats/insights:insights", "//pkg/sql/sqlstats/insights:insights_test", diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index bc2e2e5b049d..be61e6f0b4ea 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -658,7 +658,6 @@ go_test( "mutation_test.go", "mvcc_backfiller_test.go", "normalization_test.go", - "partition_test.go", "pg_metadata_test.go", "pg_oid_test.go", "pgwire_internal_test.go", diff --git a/pkg/sql/sqlnoccltest/BUILD.bazel b/pkg/sql/sqlnoccltest/BUILD.bazel new file mode 100644 index 000000000000..2dee6658ad45 --- /dev/null +++ b/pkg/sql/sqlnoccltest/BUILD.bazel @@ -0,0 +1,27 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "sqlnoccltest_test", + srcs = [ + "main_test.go", + "partition_test.go", + ], + args = ["-test.timeout=295s"], + deps = [ + "//pkg/config/zonepb", + "//pkg/keys", + "//pkg/security/securityassets", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/sql/catalog/catalogkeys", + "//pkg/sql/catalog/catpb", + "//pkg/sql/catalog/desctestutils", + "//pkg/sql/tests", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/util/encoding", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/protoutil", + ], +) diff --git a/pkg/sql/sqlnoccltest/main_test.go b/pkg/sql/sqlnoccltest/main_test.go new file mode 100644 index 000000000000..440cf9544366 --- /dev/null +++ b/pkg/sql/sqlnoccltest/main_test.go @@ -0,0 +1,27 @@ +// Copyright 2023 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 sqlnoccltest_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security/securityassets" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" +) + +func TestMain(m *testing.M) { + securityassets.SetLoader(securitytest.EmbeddedAssets) + serverutils.InitTestServerFactory(server.TestServerFactory) + os.Exit(m.Run()) +} diff --git a/pkg/sql/partition_test.go b/pkg/sql/sqlnoccltest/partition_test.go similarity index 99% rename from pkg/sql/partition_test.go rename to pkg/sql/sqlnoccltest/partition_test.go index 6da5104d9ae8..ac985d76d4f6 100644 --- a/pkg/sql/partition_test.go +++ b/pkg/sql/sqlnoccltest/partition_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package sql_test +package sqlnoccltest_test import ( "context" From cd10174dcddbc50adb519383136af2e9d01dfa04 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 17 Jul 2023 13:32:39 -0700 Subject: [PATCH 2/4] ts: remove mentions of old deprecated settings This commit removes all mentions of `timeseries.storage.10s_resolution_ttl` and `timeseries.storage.30m_resolution_ttl` cluster settings that have been deprecated long time ago. I believe they were kept around due to the migration, but the permanent upgrades introduced in 399e56b3f56c0dce6c34525bebf58297be320083 seem to have removed the need for that. Release note: None --- docs/tech-notes/version_upgrades.md | 2 -- pkg/settings/registry.go | 1 + pkg/sql/show_test.go | 5 ----- pkg/ts/db.go | 18 ------------------ 4 files changed, 1 insertion(+), 25 deletions(-) diff --git a/docs/tech-notes/version_upgrades.md b/docs/tech-notes/version_upgrades.md index 10903b450a4f..55c3226028fa 100644 --- a/docs/tech-notes/version_upgrades.md +++ b/docs/tech-notes/version_upgrades.md @@ -208,8 +208,6 @@ Here are the remaining uses: - auto-generate a random UUID for `cluster.secret`. - block the node startup if a user/role with name `public` is present in `system.users`. - create the `defaultdb` and `postgres` empty databases. -- copy the values from the old `timeseries.storage.10s_resolution_ttl` and `timeseries.storage.30m_resolution_ttl` settings - to their new setting names. - add the default lat/long entries to `system.locations`. - add the `CREATELOGIN` option to roles that already have the `CREATEROLE` role option. diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index e151a2c976cb..ac47c5ba0ad7 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -172,6 +172,7 @@ var retiredSettings = map[string]struct{}{ "changefeed.replan_flow_frequency": {}, "changefeed.replan_flow_threshold": {}, "jobs.trace.force_dump_mode": {}, + "timeseries.storage.30m_resolution_ttl": {}, } // sqlDefaultSettings is the list of "grandfathered" existing sql.defaults diff --git a/pkg/sql/show_test.go b/pkg/sql/show_test.go index 83874db6be87..669fb9a4b330 100644 --- a/pkg/sql/show_test.go +++ b/pkg/sql/show_test.go @@ -1285,11 +1285,6 @@ func TestLintClusterSettingNames(t *testing.T) { "sql.metrics.statement_details.sample_logical_plans": `sql.metrics.statement_details.sample_logical_plans: use .enabled for booleans`, "sql.trace.log_statement_execute": `sql.trace.log_statement_execute: use .enabled for booleans`, "trace.debug.enable": `trace.debug.enable: use .enabled for booleans`, - // These two settings have been deprecated in favor of a new (better named) setting - // but the old name is still around to support migrations. - // TODO(knz): remove these cases when these settings are retired. - "timeseries.storage.10s_resolution_ttl": `timeseries.storage.10s_resolution_ttl: part "10s_resolution_ttl" has invalid structure`, - "timeseries.storage.30m_resolution_ttl": `timeseries.storage.30m_resolution_ttl: part "30m_resolution_ttl" has invalid structure`, // These use the _timeout suffix to stay consistent with the // corresponding session variables. diff --git a/pkg/ts/db.go b/pkg/ts/db.go index 667cd23e5eb0..dd5ceed48a7b 100644 --- a/pkg/ts/db.go +++ b/pkg/ts/db.go @@ -59,24 +59,6 @@ var Resolution10sStorageTTL = settings.RegisterDurationSetting( resolution10sDefaultRollupThreshold, ).WithPublic() -// deprecatedResolution30StoreDuration is retained for backward compatibility during a version upgrade. -var deprecatedResolution30StoreDuration = func() *settings.DurationSetting { - s := settings.RegisterDurationSetting( - settings.TenantWritable, - "timeseries.storage.30m_resolution_ttl", "replaced by timeseries.storage.resolution_30m.ttl", - resolution30mDefaultPruneThreshold, - ) - s.SetRetired() - return s -}() - -func init() { - // The setting is not used any more, but we need to keep its - // definition for backward compatibility until the next release - // cycle. - _ = deprecatedResolution30StoreDuration -} - // Resolution30mStorageTTL defines the maximum age of data that will be // retained at he 30 minute resolution. Data older than this is subject to // deletion. From 7611a614194cabcbe4c1fcc3291755cdaf068f4e Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Mon, 17 Jul 2023 13:46:54 -0700 Subject: [PATCH 3/4] sql: run TestLintClusterSettingNames on CCL code This commits enables running `TestLintClusterSettingNames` on CCL code as well as fixes some minor existing failures. I examined all of the cluster settings that are recommended to be named differently, and all of them are present on 23.1, so they have been grandfathered in. Release note: None --- .../settings/settings-for-tenants.txt | 6 +++--- docs/generated/settings/settings.html | 6 +++--- .../backupccl/backupdest/backup_destination.go | 2 +- .../changefeedccl/changefeedbase/settings.go | 4 ++-- pkg/ccl/oidcccl/settings.go | 2 +- pkg/ccl/streamingccl/settings.go | 2 +- .../stream_ingestion_frontier_processor.go | 2 +- pkg/sql/BUILD.bazel | 1 + pkg/sql/show_test.go | 18 ++++++++++++++++++ 9 files changed, 31 insertions(+), 12 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 35ad3d1fe227..8bb8253e1412 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -5,7 +5,7 @@ admission.epoch_lifo.epoch_duration duration 100ms the duration of an epoch, for admission.epoch_lifo.queue_delay_threshold_to_switch_to_lifo duration 105ms the queue delay encountered by a (tenant,priority) for switching to epoch-LIFO ordering tenant-rw admission.sql_kv_response.enabled boolean true when true, work performed by the SQL layer when receiving a KV response is subject to admission control tenant-rw admission.sql_sql_response.enabled boolean true when true, work performed by the SQL layer when receiving a DistSQL response is subject to admission control tenant-rw -bulkio.backup.deprecated_full_backup_with_subdir.enabled boolean false when true, a backup command with a user specified subdirectory will create a full backup at the subdirectory if no backup already exists at that subdirectory. tenant-rw +bulkio.backup.deprecated_full_backup_with_subdir.enabled boolean false when true, a backup command with a user specified subdirectory will create a full backup at the subdirectory if no backup already exists at that subdirectory tenant-rw bulkio.backup.file_size byte size 128 MiB target size for individual data files produced during BACKUP tenant-rw bulkio.backup.read_timeout duration 5m0s amount of time after which a read attempt is considered timed out, which causes the backup to fail tenant-rw bulkio.backup.read_with_priority_after duration 1m0s amount of time since the read-as-of time above which a BACKUP should use priority when retrying reads tenant-rw @@ -19,7 +19,7 @@ changefeed.fast_gzip.enabled boolean true use fast gzip implementation tenant-rw changefeed.node_throttle_config string specifies node level throttling configuration for all changefeeeds tenant-rw changefeed.protect_timestamp.max_age duration 96h0m0s fail the changefeed if the protected timestamp age exceeds this threshold; 0 disables expiration tenant-rw changefeed.schema_feed.read_with_priority_after duration 1m0s retry with high priority if we were not able to read descriptors for too long; 0 disables tenant-rw -changefeed.sink_io_workers integer 0 the number of workers used by changefeeds when sending requests to the sink (currently webhook only): <0 disables, 0 assigns a reasonable default, >0 assigns the setting value. tenant-rw +changefeed.sink_io_workers integer 0 the number of workers used by changefeeds when sending requests to the sink (currently webhook only): <0 disables, 0 assigns a reasonable default, >0 assigns the setting value tenant-rw cloudstorage.azure.concurrent_upload_buffers integer 1 controls the number of concurrent buffers that will be used by the Azure client when uploading chunks.Each buffer can buffer up to cloudstorage.write_chunk.size of memory during an upload tenant-rw cloudstorage.http.custom_ca string custom root CA (appended to system's default CAs) for verifying certificates when interacting with HTTPS storage tenant-rw cloudstorage.timeout duration 10m0s the timeout for import/export storage operations tenant-rw @@ -71,7 +71,7 @@ server.oidc_authentication.client_secret string sets OIDC client secret tenant- server.oidc_authentication.enabled boolean false enables or disabled OIDC login for the DB Console tenant-rw server.oidc_authentication.principal_regex string (.+) regular expression to apply to extracted principal (see claim_json_key setting) to translate to SQL user (golang regex format, must include 1 grouping to extract) tenant-rw server.oidc_authentication.provider_url string sets OIDC provider URL ({provider_url}/.well-known/openid-configuration must resolve) tenant-rw -server.oidc_authentication.redirect_url string https://localhost:8080/oidc/v1/callback sets OIDC redirect URL via a URL string or a JSON string containing a required `redirect_urls` key with an object that maps from region keys to URL strings (URLs should point to your load balancer and must route to the path /oidc/v1/callback) tenant-rw +server.oidc_authentication.redirect_url string https://localhost:8080/oidc/v1/callback sets OIDC redirect URL via a URL string or a JSON string containing a required `redirect_urls` key with an object that maps from region keys to URL strings (URLs should point to your load balancer and must route to the path /oidc/v1/callback) tenant-rw server.oidc_authentication.scopes string openid sets OIDC scopes to include with authentication request (space delimited list of strings, required to start with `openid`) tenant-rw server.rangelog.ttl duration 720h0m0s if nonzero, entries in system.rangelog older than this duration are periodically purged tenant-rw server.shutdown.connection_wait duration 0s the maximum amount of time a server waits for all SQL connections to be closed before proceeding with a drain. (note that the --drain-wait parameter for cockroach node drain may need adjustment after changing this setting) tenant-rw diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 1448370f4aac..dc2b29d3bca5 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -11,7 +11,7 @@
admission.kv.tenant_weights.enabled
booleanfalsewhen true, tenant weights are enabled for KV admission controlDedicated/Self-Hosted
admission.sql_kv_response.enabled
booleantruewhen true, work performed by the SQL layer when receiving a KV response is subject to admission controlServerless/Dedicated/Self-Hosted
admission.sql_sql_response.enabled
booleantruewhen true, work performed by the SQL layer when receiving a DistSQL response is subject to admission controlServerless/Dedicated/Self-Hosted -
bulkio.backup.deprecated_full_backup_with_subdir.enabled
booleanfalsewhen true, a backup command with a user specified subdirectory will create a full backup at the subdirectory if no backup already exists at that subdirectory.Serverless/Dedicated/Self-Hosted +
bulkio.backup.deprecated_full_backup_with_subdir.enabled
booleanfalsewhen true, a backup command with a user specified subdirectory will create a full backup at the subdirectory if no backup already exists at that subdirectoryServerless/Dedicated/Self-Hosted
bulkio.backup.file_size
byte size128 MiBtarget size for individual data files produced during BACKUPServerless/Dedicated/Self-Hosted
bulkio.backup.read_timeout
duration5m0samount of time after which a read attempt is considered timed out, which causes the backup to failServerless/Dedicated/Self-Hosted
bulkio.backup.read_with_priority_after
duration1m0samount of time since the read-as-of time above which a BACKUP should use priority when retrying readsServerless/Dedicated/Self-Hosted @@ -25,7 +25,7 @@
changefeed.node_throttle_config
stringspecifies node level throttling configuration for all changefeeedsServerless/Dedicated/Self-Hosted
changefeed.protect_timestamp.max_age
duration96h0m0sfail the changefeed if the protected timestamp age exceeds this threshold; 0 disables expirationServerless/Dedicated/Self-Hosted
changefeed.schema_feed.read_with_priority_after
duration1m0sretry with high priority if we were not able to read descriptors for too long; 0 disablesServerless/Dedicated/Self-Hosted -
changefeed.sink_io_workers
integer0the number of workers used by changefeeds when sending requests to the sink (currently webhook only): <0 disables, 0 assigns a reasonable default, >0 assigns the setting value.Serverless/Dedicated/Self-Hosted +
changefeed.sink_io_workers
integer0the number of workers used by changefeeds when sending requests to the sink (currently webhook only): <0 disables, 0 assigns a reasonable default, >0 assigns the setting valueServerless/Dedicated/Self-Hosted
cloudstorage.azure.concurrent_upload_buffers
integer1controls the number of concurrent buffers that will be used by the Azure client when uploading chunks.Each buffer can buffer up to cloudstorage.write_chunk.size of memory during an uploadServerless/Dedicated/Self-Hosted
cloudstorage.http.custom_ca
stringcustom root CA (appended to system's default CAs) for verifying certificates when interacting with HTTPS storageServerless/Dedicated/Self-Hosted
cloudstorage.timeout
duration10m0sthe timeout for import/export storage operationsServerless/Dedicated/Self-Hosted @@ -101,7 +101,7 @@
server.oidc_authentication.enabled
booleanfalseenables or disabled OIDC login for the DB ConsoleServerless/Dedicated/Self-Hosted
server.oidc_authentication.principal_regex
string(.+)regular expression to apply to extracted principal (see claim_json_key setting) to translate to SQL user (golang regex format, must include 1 grouping to extract)Serverless/Dedicated/Self-Hosted
server.oidc_authentication.provider_url
stringsets OIDC provider URL ({provider_url}/.well-known/openid-configuration must resolve)Serverless/Dedicated/Self-Hosted -
server.oidc_authentication.redirect_url
stringhttps://localhost:8080/oidc/v1/callbacksets OIDC redirect URL via a URL string or a JSON string containing a required `redirect_urls` key with an object that maps from region keys to URL strings (URLs should point to your load balancer and must route to the path /oidc/v1/callback) Serverless/Dedicated/Self-Hosted +
server.oidc_authentication.redirect_url
stringhttps://localhost:8080/oidc/v1/callbacksets OIDC redirect URL via a URL string or a JSON string containing a required `redirect_urls` key with an object that maps from region keys to URL strings (URLs should point to your load balancer and must route to the path /oidc/v1/callback)Serverless/Dedicated/Self-Hosted
server.oidc_authentication.scopes
stringopenidsets OIDC scopes to include with authentication request (space delimited list of strings, required to start with `openid`)Serverless/Dedicated/Self-Hosted
server.rangelog.ttl
duration720h0m0sif nonzero, entries in system.rangelog older than this duration are periodically purgedServerless/Dedicated/Self-Hosted
server.secondary_tenants.redact_trace.enabled
booleantruecontrols if server side traces are redacted for tenant operationsDedicated/Self-Hosted diff --git a/pkg/ccl/backupccl/backupdest/backup_destination.go b/pkg/ccl/backupccl/backupdest/backup_destination.go index b7dbadec3460..2b28ef22daa3 100644 --- a/pkg/ccl/backupccl/backupdest/backup_destination.go +++ b/pkg/ccl/backupccl/backupdest/backup_destination.go @@ -62,7 +62,7 @@ var featureFullBackupUserSubdir = settings.RegisterBoolSetting( settings.TenantWritable, "bulkio.backup.deprecated_full_backup_with_subdir.enabled", "when true, a backup command with a user specified subdirectory will create a full backup at"+ - " the subdirectory if no backup already exists at that subdirectory.", + " the subdirectory if no backup already exists at that subdirectory", false, ).WithPublic() diff --git a/pkg/ccl/changefeedccl/changefeedbase/settings.go b/pkg/ccl/changefeedccl/changefeedbase/settings.go index c5536ee1452d..e17f1a2d1f34 100644 --- a/pkg/ccl/changefeedccl/changefeedbase/settings.go +++ b/pkg/ccl/changefeedccl/changefeedbase/settings.go @@ -74,7 +74,7 @@ var IdleTimeout = settings.RegisterDurationSetting( var FrontierCheckpointFrequency = settings.RegisterDurationSetting( settings.TenantWritable, "changefeed.frontier_checkpoint_frequency", - "controls the frequency with which span level checkpoints will be written; if 0, disabled.", + "controls the frequency with which span level checkpoints will be written; if 0, disabled", 10*time.Minute, settings.NonNegativeDuration, ) @@ -297,7 +297,7 @@ var SinkIOWorkers = settings.RegisterIntSetting( settings.TenantWritable, "changefeed.sink_io_workers", "the number of workers used by changefeeds when sending requests to the sink "+ - "(currently webhook only): <0 disables, 0 assigns a reasonable default, >0 assigns the setting value.", + "(currently webhook only): <0 disables, 0 assigns a reasonable default, >0 assigns the setting value", 0, ).WithPublic() diff --git a/pkg/ccl/oidcccl/settings.go b/pkg/ccl/oidcccl/settings.go index e056144bae6c..463bf4b2ebb8 100644 --- a/pkg/ccl/oidcccl/settings.go +++ b/pkg/ccl/oidcccl/settings.go @@ -182,7 +182,7 @@ var OIDCRedirectURL = func() *settings.StringSetting { OIDCRedirectURLSettingName, "sets OIDC redirect URL via a URL string or a JSON string containing a required "+ "`redirect_urls` key with an object that maps from region keys to URL strings "+ - "(URLs should point to your load balancer and must route to the path /oidc/v1/callback) ", + "(URLs should point to your load balancer and must route to the path /oidc/v1/callback)", "https://localhost:8080/oidc/v1/callback", validateOIDCRedirectURL, ) diff --git a/pkg/ccl/streamingccl/settings.go b/pkg/ccl/streamingccl/settings.go index cdcc62ee44b6..1e91c9f9f010 100644 --- a/pkg/ccl/streamingccl/settings.go +++ b/pkg/ccl/streamingccl/settings.go @@ -58,7 +58,7 @@ var StreamReplicationMinCheckpointFrequency = settings.RegisterDurationSetting( settings.SystemOnly, "stream_replication.min_checkpoint_frequency", "controls minimum frequency the stream replication source cluster sends checkpoints "+ - "to the destination cluster.", + "to the destination cluster", 10*time.Second, settings.NonNegativeDuration, ) diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go index def5d9907466..623250760ed9 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go @@ -41,7 +41,7 @@ import ( var JobCheckpointFrequency = settings.RegisterDurationSetting( settings.TenantWritable, "stream_replication.job_checkpoint_frequency", - "controls the frequency with which partitions update their progress; if 0, disabled.", + "controls the frequency with which partitions update their progress; if 0, disabled", 10*time.Second, settings.NonNegativeDuration, ) diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index be61e6f0b4ea..3ec30ea43a0c 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -728,6 +728,7 @@ go_test( deps = [ "//pkg/base", "//pkg/build/bazel", + "//pkg/ccl", "//pkg/cloud/impl:cloudimpl", "//pkg/clusterversion", "//pkg/col/coldata", diff --git a/pkg/sql/show_test.go b/pkg/sql/show_test.go index 669fb9a4b330..93f6017021c4 100644 --- a/pkg/sql/show_test.go +++ b/pkg/sql/show_test.go @@ -21,6 +21,7 @@ import ( "unicode/utf8" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/ccl" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" @@ -1197,6 +1198,7 @@ func TestShowRedactedActiveStatements(t *testing.T) { func TestLintClusterSettingNames(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + defer ccl.TestingEnableEnterprise()() skip.UnderRace(t, "lint only test") skip.UnderDeadlock(t, "lint only test") @@ -1286,6 +1288,22 @@ func TestLintClusterSettingNames(t *testing.T) { "sql.trace.log_statement_execute": `sql.trace.log_statement_execute: use .enabled for booleans`, "trace.debug.enable": `trace.debug.enable: use .enabled for booleans`, + // These were grandfathered because the test wasn't running on + // the CCL code. + "bulkio.backup.export_request_verbose_tracing": `bulkio.backup.export_request_verbose_tracing: use .enabled for booleans`, + "bulkio.backup.read_timeout": `bulkio.backup.read_timeout: use ".timeout" instead of "_timeout"`, + "bulkio.backup.split_keys_on_timestamps": `bulkio.backup.split_keys_on_timestamps: use .enabled for booleans`, + "bulkio.restore.memory_monitor_ssts": `bulkio.restore.memory_monitor_ssts: use .enabled for booleans`, + "bulkio.restore.use_simple_import_spans": `bulkio.restore.use_simple_import_spans: use .enabled for booleans`, + "changefeed.balance_range_distribution.enable": `changefeed.balance_range_distribution.enable: use .enabled for booleans`, + "changefeed.batch_reduction_retry_enabled": `changefeed.batch_reduction_retry_enabled: use ".enabled" instead of "_enabled"`, + "changefeed.idle_timeout": `changefeed.idle_timeout: use ".timeout" instead of "_timeout"`, + "changefeed.new_pubsub_sink_enabled": `changefeed.new_pubsub_sink_enabled: use ".enabled" instead of "_enabled"`, + "changefeed.new_webhook_sink_enabled": `changefeed.new_webhook_sink_enabled: use ".enabled" instead of "_enabled"`, + "changefeed.permissions.require_external_connection_sink": `changefeed.permissions.require_external_connection_sink: use .enabled for booleans`, + "server.oidc_authentication.autologin": `server.oidc_authentication.autologin: use .enabled for booleans`, + "stream_replication.job_liveness_timeout": `stream_replication.job_liveness_timeout: use ".timeout" instead of "_timeout"`, + // These use the _timeout suffix to stay consistent with the // corresponding session variables. "sql.defaults.statement_timeout": `sql.defaults.statement_timeout: use ".timeout" instead of "_timeout"`, From 0a00e71c6c1e6565239483893dad5cfdeddae7f7 Mon Sep 17 00:00:00 2001 From: Jayant Shrivastava Date: Tue, 18 Jul 2023 14:23:35 -0400 Subject: [PATCH 4/4] changefeedccl: deflake TestChangefeedSingleColumnFamilySchemaChanges Previously, the test `TestChangefeedSingleColumnFamilySchemaChanges` would flake. The reason is that the helper function, `requireErrorSoon`, would poll the test feed once for an error, timing out after 30 seconds. The problem with this is that the testfeed may have an event buffered which may be returned instead of the error. Basically, there is a race between the buffered event an the error. To fix this, `requireErrorSoon` should poll the test feed repeatedly for 30 seconds for errors. Before this change, the test would fail under stress within 150 runs. After this change, it passes 5k runs under stress. Fixes: https://github.com/cockroachdb/cockroach/issues/107072 Epic: None Release note: None --- pkg/ccl/changefeedccl/changefeed_test.go | 30 ++++++++++++++---------- 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 51610afdce16..9daa31cdd43e 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -3139,22 +3139,28 @@ func TestChangefeedOutputTopics(t *testing.T) { cdcTest(t, testFn, feedTestForceSink("kafka")) } +// requireErrorSoon polls for the test feed for an error and asserts that +// the error matches the provided regex. func requireErrorSoon( ctx context.Context, t *testing.T, f cdctest.TestFeed, errRegex *regexp.Regexp, ) { - ctx, cancel := context.WithTimeout(ctx, 30*time.Second) - defer cancel() - done := make(chan struct{}) - go func() { - if _, err := f.Next(); err != nil { - assert.Regexp(t, errRegex, err) - done <- struct{}{} + err := timeutil.RunWithTimeout(ctx, "requireErrorSoon", 30*time.Second, func(ctx context.Context) error { + for { + select { + case <-ctx.Done(): + return ctx.Err() + default: + m, err := f.Next() + if err != nil { + assert.Regexp(t, errRegex, err) + return nil + } + log.Infof(ctx, "waiting for error; skipping test feed message: %s", m.String()) + } } - }() - select { - case <-ctx.Done(): - t.Fatal("timed out waiting for changefeed to fail") - case <-done: + }) + if err != nil { + t.Fatal(err) } }