From 4fb65fb7b8d3561309525ab3c4c7d99fcc1b0011 Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Fri, 19 Apr 2024 17:36:53 -0400 Subject: [PATCH 1/6] sql/delegate: don't include external connections in SHOW SYSTEM GRANTS Release note (bug fix): Privileges granted for external connections were incorrectly showing up in SHOW SYSTEM GRANTS, but were not useful since there is no associated object name. Now they do not appear there. Instead, the SHOW GRANTS ON EXTERNAL CONNECTION should be used. --- pkg/sql/delegate/show_grants.go | 1 + .../logic_test/grant_revoke_with_grant_option | 30 +++++++++++++++++++ pkg/sql/logictest/testdata/logic_test/role | 18 ----------- 3 files changed, 31 insertions(+), 18 deletions(-) diff --git a/pkg/sql/delegate/show_grants.go b/pkg/sql/delegate/show_grants.go index 6c52cc266da2..3fc7eac9ecb1 100644 --- a/pkg/sql/delegate/show_grants.go +++ b/pkg/sql/delegate/show_grants.go @@ -88,6 +88,7 @@ SELECT a.username AS grantee, FROM ( SELECT username, unnest(privileges) AS privilege FROM crdb_internal.kv_system_privileges + WHERE path LIKE '/global%' ) AS a` const externalConnectionPrivilegeQuery = ` SELECT * diff --git a/pkg/sql/logictest/testdata/logic_test/grant_revoke_with_grant_option b/pkg/sql/logictest/testdata/logic_test/grant_revoke_with_grant_option index b765463e0750..0236ce623ae8 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_revoke_with_grant_option +++ b/pkg/sql/logictest/testdata/logic_test/grant_revoke_with_grant_option @@ -677,3 +677,33 @@ test public owner_grant_option admin ALL test public owner_grant_option other_owner ALL true test public owner_grant_option owner_grant_option_child SELECT false test public owner_grant_option root ALL true + +statement ok +CREATE USER roach; +CREATE TYPE mood AS enum ('sad','happy'); +GRANT USAGE ON TYPE mood TO roach; +CREATE SEQUENCE test_sequence; +GRANT SELECT ON SEQUENCE test_sequence TO roach; +CREATE EXTERNAL CONNECTION connection1 AS 'nodelocal://1/foo'; +GRANT USAGE ON EXTERNAL CONNECTION connection1 TO roach WITH GRANT OPTION; +GRANT SYSTEM VIEWCLUSTERSETTING TO roach WITH GRANT OPTION; +GRANT SYSTEM VIEWACTIVITY TO roach; + +# The purpose of this test is to verify the object_type column. +query TTTTTTB colnames,rowsort +SHOW GRANTS FOR roach +---- +database_name schema_name object_name object_type grantee privilege_type is_grantable +NULL NULL connection1 external_connection roach USAGE true +test public mood type roach USAGE false +test public test_sequence sequence roach SELECT false + +# Verify that only system grants appear in SHOW SYSTEM GRANTS. Previously, +# there was a bug that would cause external connection privileges to appear +# also, since those privileges are also implemented with synthetic privileges. +query TTB colnames,rowsort +SHOW SYSTEM GRANTS FOR roach +---- +grantee privilege_type is_grantable +roach VIEWACTIVITY false +roach VIEWCLUSTERSETTING true diff --git a/pkg/sql/logictest/testdata/logic_test/role b/pkg/sql/logictest/testdata/logic_test/role index 0500fe88a5bd..c596ccdc83da 100644 --- a/pkg/sql/logictest/testdata/logic_test/role +++ b/pkg/sql/logictest/testdata/logic_test/role @@ -1881,21 +1881,3 @@ statement error SUBJECT role option is only supported after v24.1 upgrade is fin ALTER ROLE testuser SUBJECT 'foo' subtest end - -# The purpose of this test is to verify the object_type column -statement ok -create user roach; -create type mood as enum ('sad','happy'); -grant usage on type mood to roach; -create sequence test_sequence; -grant usage on sequence test_sequence to roach; -CREATE EXTERNAL CONNECTION connection1 AS 'nodelocal://1/foo'; -grant usage on EXTERNAL CONNECTION connection1 to roach; - -query TTTTTTB colnames,rowsort -show grants for roach ----- -database_name schema_name object_name object_type grantee privilege_type is_grantable -NULL NULL connection1 external_connection roach USAGE false -test public mood type roach USAGE false -test public test_sequence sequence roach USAGE false From 47ca87d972c6ca696d77d336d4221eacb87bbb78 Mon Sep 17 00:00:00 2001 From: Andrew Baptist Date: Fri, 19 Apr 2024 10:00:32 -0400 Subject: [PATCH 2/6] revert test change from 124aaa3b00349f4240c3a078301686c2d8883eda The original attempt to fix this test introduced timing issues which made this test flakey. This commit reverts all the changes to TestStoreRangeSplitAndMergeWithGlobalReads. Epic: none Fixes: #119230 Release note: None --- pkg/kv/kvserver/client_split_test.go | 32 +++++++--------------------- 1 file changed, 8 insertions(+), 24 deletions(-) diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index 53ab78f3f48b..288186f24be8 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -4042,17 +4042,10 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { } return nil } - // Set global reads. - zoneConfig := zonepb.DefaultZoneConfig() - zoneConfig.GlobalReads = proto.Bool(true) ctx := context.Background() s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ - Server: &server.TestingKnobs{ - DefaultZoneConfigOverride: &zoneConfig, - }, - Store: &kvserver.StoreTestingKnobs{ DisableMergeQueue: true, TestingResponseFilter: respFilter, @@ -4082,6 +4075,11 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { // response filter. clockPtr.Store(s.Clock()) + // Set global reads. + zoneConfig := zonepb.DefaultZoneConfig() + zoneConfig.GlobalReads = proto.Bool(true) + config.TestingSetZoneConfig(config.ObjectID(descID), zoneConfig) + // Perform a write to the system config span being watched by // the SystemConfigProvider. tdb.Exec(t, "CREATE TABLE foo ()") @@ -4093,19 +4091,6 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { return nil }) - // The commit wait count is 1 due to the split above since global reads are - // set for the default config. - var splitCount = int64(1) - testutils.SucceedsSoon(t, func() error { - if splitCount != store.Metrics().CommitWaitsBeforeCommitTrigger.Count() { - return errors.Errorf("commit wait count is %d", store.Metrics().CommitWaitsBeforeCommitTrigger.Count()) - } - if splitCount != atomic.LoadInt64(&splits) { - return errors.Errorf("num splits is %d", atomic.LoadInt64(&splits)) - } - return nil - }) - // Write to the range, which has the effect of bumping the closed timestamp. pArgs := putArgs(descKey, []byte("foo")) _, pErr = kv.SendWrapped(ctx, store.TestSender(), pArgs) @@ -4116,9 +4101,8 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { splitArgs = adminSplitArgs(splitKey) _, pErr = kv.SendWrapped(ctx, store.TestSender(), splitArgs) require.Nil(t, pErr) - splitCount++ - require.Equal(t, splitCount, store.Metrics().CommitWaitsBeforeCommitTrigger.Count()) - require.Equal(t, splitCount, atomic.LoadInt64(&splits)) + require.Equal(t, int64(1), store.Metrics().CommitWaitsBeforeCommitTrigger.Count()) + require.Equal(t, int64(1), atomic.LoadInt64(&splits)) repl := store.LookupReplica(roachpb.RKey(splitKey)) require.Equal(t, splitKey, repl.Desc().StartKey.AsRawKey()) @@ -4127,7 +4111,7 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { mergeArgs := adminMergeArgs(descKey) _, pErr = kv.SendWrapped(ctx, store.TestSender(), mergeArgs) require.Nil(t, pErr) - require.Equal(t, splitCount+1, store.Metrics().CommitWaitsBeforeCommitTrigger.Count()) + require.Equal(t, int64(2), store.Metrics().CommitWaitsBeforeCommitTrigger.Count()) require.Equal(t, int64(1), atomic.LoadInt64(&merges)) repl = store.LookupReplica(roachpb.RKey(splitKey)) From a47ad19538bb9b3d6c368e9e8e78de8958e3e916 Mon Sep 17 00:00:00 2001 From: Andrew Baptist Date: Fri, 19 Apr 2024 14:43:12 -0400 Subject: [PATCH 3/6] kvserver: allow wrapping the exising reader This commit allows the testing StoreKVSubscriberOverride to be made more flexible. Specifically it passes the original to the test to allow wrapping it rather than just creating a brand new one. The main benefit of this change is to allow overriding of a single span rather than some of the other ways of injecting spans in. Epic: none Release note: None --- pkg/kv/kvserver/client_spanconfigs_test.go | 4 +- pkg/server/server.go | 68 +++++++++++----------- pkg/spanconfig/testing_knobs.go | 2 +- 3 files changed, 37 insertions(+), 37 deletions(-) diff --git a/pkg/kv/kvserver/client_spanconfigs_test.go b/pkg/kv/kvserver/client_spanconfigs_test.go index f969d15ac5f4..0beab9683bd6 100644 --- a/pkg/kv/kvserver/client_spanconfigs_test.go +++ b/pkg/kv/kvserver/client_spanconfigs_test.go @@ -55,7 +55,7 @@ func TestSpanConfigUpdateAppliedToReplica(t *testing.T) { DisableGCQueue: true, }, SpanConfig: &spanconfig.TestingKnobs{ - StoreKVSubscriberOverride: mockSubscriber, + StoreKVSubscriberOverride: func(spanconfig.KVSubscriber) spanconfig.KVSubscriber { return mockSubscriber }, }, }, } @@ -120,7 +120,7 @@ func TestFallbackSpanConfigOverride(t *testing.T) { DisableGCQueue: true, }, SpanConfig: &spanconfig.TestingKnobs{ - StoreKVSubscriberOverride: mockSubscriber, + StoreKVSubscriberOverride: func(spanconfig.KVSubscriber) spanconfig.KVSubscriber { return mockSubscriber }, }, }, } diff --git a/pkg/server/server.go b/pkg/server/server.go index 15bb1b7050ac..b7f4db3ed079 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -753,41 +753,41 @@ func NewServer(cfg Config, stopper *stop.Stopper) (serverctl.ServerStartupInterf kvAccessorForTenantRecords spanconfig.KVAccessor } spanConfigKnobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs) + // We use the span configs infra to control whether rangefeeds are + // enabled on a given range. At the moment this only applies to + // system tables (on both host and secondary tenants). We need to + // consider two things: + // - The sql-side reconciliation process runs asynchronously. When + // the config for a given range is requested, we might not yet have + // it, thus falling back to the static config below. + // - Various internal subsystems rely on rangefeeds to function. + // + // Consequently, we configure our static fallback config to actually + // allow rangefeeds. As the sql-side reconciliation process kicks + // off, it'll install the actual configs that we'll later consult. + // For system table ranges we install configs that allow for + // rangefeeds. Until then, we simply allow rangefeeds when a more + // targeted config is not found. + fallbackConf := cfg.DefaultZoneConfig.AsSpanConfig() + fallbackConf.RangefeedEnabled = true + // We do the same for opting out of strict GC enforcement; it + // really only applies to user table ranges + fallbackConf.GCPolicy.IgnoreStrictEnforcement = true + + spanConfig.subscriber = spanconfigkvsubscriber.New( + clock, + rangeFeedFactory, + keys.SpanConfigurationsTableID, + 4<<20, /* 4 MB */ + fallbackConf, + cfg.Settings, + spanconfigstore.NewBoundsReader(tenantCapabilitiesWatcher), + spanConfigKnobs, + nodeRegistry, + ) + if spanConfigKnobs != nil && spanConfigKnobs.StoreKVSubscriberOverride != nil { - spanConfig.subscriber = spanConfigKnobs.StoreKVSubscriberOverride - } else { - // We use the span configs infra to control whether rangefeeds are - // enabled on a given range. At the moment this only applies to - // system tables (on both host and secondary tenants). We need to - // consider two things: - // - The sql-side reconciliation process runs asynchronously. When - // the config for a given range is requested, we might not yet have - // it, thus falling back to the static config below. - // - Various internal subsystems rely on rangefeeds to function. - // - // Consequently, we configure our static fallback config to actually - // allow rangefeeds. As the sql-side reconciliation process kicks - // off, it'll install the actual configs that we'll later consult. - // For system table ranges we install configs that allow for - // rangefeeds. Until then, we simply allow rangefeeds when a more - // targeted config is not found. - fallbackConf := cfg.DefaultZoneConfig.AsSpanConfig() - fallbackConf.RangefeedEnabled = true - // We do the same for opting out of strict GC enforcement; it - // really only applies to user table ranges - fallbackConf.GCPolicy.IgnoreStrictEnforcement = true - - spanConfig.subscriber = spanconfigkvsubscriber.New( - clock, - rangeFeedFactory, - keys.SpanConfigurationsTableID, - 4<<20, /* 4 MB */ - fallbackConf, - cfg.Settings, - spanconfigstore.NewBoundsReader(tenantCapabilitiesWatcher), - spanConfigKnobs, - nodeRegistry, - ) + spanConfig.subscriber = spanConfigKnobs.StoreKVSubscriberOverride(spanConfig.subscriber) } scKVAccessor := spanconfigkvaccessor.New( diff --git a/pkg/spanconfig/testing_knobs.go b/pkg/spanconfig/testing_knobs.go index 8f947c54a563..942a8c351755 100644 --- a/pkg/spanconfig/testing_knobs.go +++ b/pkg/spanconfig/testing_knobs.go @@ -62,7 +62,7 @@ type TestingKnobs struct { // StoreKVSubscriberOverride is used to override the KVSubscriber used when // setting up a new store. - StoreKVSubscriberOverride KVSubscriber + StoreKVSubscriberOverride func(KVSubscriber) KVSubscriber // KVAccessorPaginationInterceptor, if set, is invoked on every pagination // event. From acab5f138e6a7f9690955c116afe3b7c8efb509f Mon Sep 17 00:00:00 2001 From: Andrew Baptist Date: Fri, 19 Apr 2024 14:59:37 -0400 Subject: [PATCH 4/6] kvserver: add WrappedKVSubscriber This commit adds a utility test struct WrappedKVSubscriber which can wrap another KVSubscriber directly and provide targetted overrides. This approach simplifies the KV testing of altered span configs without resorting to modifying them through SQL. Epic: none Release note: None --- .../spanconfigtestutils/BUILD.bazel | 1 + pkg/spanconfig/spanconfigtestutils/utils.go | 81 +++++++++++++++++++ 2 files changed, 82 insertions(+) diff --git a/pkg/spanconfig/spanconfigtestutils/BUILD.bazel b/pkg/spanconfig/spanconfigtestutils/BUILD.bazel index 3fc2a7402687..904c6d7605e9 100644 --- a/pkg/spanconfig/spanconfigtestutils/BUILD.bazel +++ b/pkg/spanconfig/spanconfigtestutils/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "//pkg/sql/isql", "//pkg/util/hlc", "//pkg/util/syncutil", + "//pkg/util/timeutil", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_stretchr_testify//require", "@in_gopkg_yaml_v2//:yaml_v2", diff --git a/pkg/spanconfig/spanconfigtestutils/utils.go b/pkg/spanconfig/spanconfigtestutils/utils.go index 59e3c1398038..e99cc02a9078 100644 --- a/pkg/spanconfig/spanconfigtestutils/utils.go +++ b/pkg/spanconfig/spanconfigtestutils/utils.go @@ -11,6 +11,7 @@ package spanconfigtestutils import ( + "context" "fmt" "reflect" "regexp" @@ -28,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/datadriven" "github.com/stretchr/testify/require" "gopkg.in/yaml.v2" @@ -750,3 +752,82 @@ func ParseProtectionTarget(t testing.TB, input string) *ptpb.Target { } return nil } + +// configOverride is used to override span configs for specific ranges. +type configOverride struct { + key roachpb.Key + config roachpb.SpanConfig +} + +// WrappedKVSubscriber is a KVSubscriber which wraps another KVSubscriber and +// overrides specific SpanConfigs. This test struct should be used in +// StoreKVSubscriberOverride as a more generic and powerful SpanConfig injection +// alternative to change the behavior for specific ranges instead of using any +// of DefaultZoneConfigOverride, DefaultSystemZoneConfigOverride, +// OverrideFallbackConf, ConfReaderInterceptor, UseSystemConfigSpanForQueues, +// SpanConfigUpdateInterceptor or SetSpanConfigInterceptor. By using this struct +// as an alternative to the others it mainly avoids any need to depend on +// rangefeed timing to propagate the config change. +type WrappedKVSubscriber struct { + wrapped spanconfig.KVSubscriber + // Overrides are list of tuples of roachpb.Key and spanconfig. + overrides []configOverride +} + +// NewWrappedKVSubscriber creates a new WrappedKVSubscriber. +func NewWrappedKVSubscriber(wrapped spanconfig.KVSubscriber) *WrappedKVSubscriber { + return &WrappedKVSubscriber{wrapped: wrapped} +} + +// AddOverride adds a new override to the WrappedKVSubscriber. This should only +// be used during construction. +func (w *WrappedKVSubscriber) AddOverride(key roachpb.Key, config roachpb.SpanConfig) { + w.overrides = append(w.overrides, configOverride{key: key, config: config}) +} + +// GetProtectionTimestamps implements spanconfig.KVSubscriber. +func (w *WrappedKVSubscriber) GetProtectionTimestamps( + ctx context.Context, sp roachpb.Span, +) ([]hlc.Timestamp, hlc.Timestamp, error) { + return w.wrapped.GetProtectionTimestamps(ctx, sp) +} + +// LastUpdated always reports that it has been updated to allow +// GetSpanConfigForKey to be used immediately. +func (w *WrappedKVSubscriber) LastUpdated() hlc.Timestamp { + return hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} +} + +// Subscribe implements spanconfig.KVSubscriber. +func (w *WrappedKVSubscriber) Subscribe(f func(context.Context, roachpb.Span)) { + w.wrapped.Subscribe(f) +} + +// ComputeSplitKey implements spanconfig.StoreReader. +func (w *WrappedKVSubscriber) ComputeSplitKey( + ctx context.Context, start roachpb.RKey, end roachpb.RKey, +) (roachpb.RKey, error) { + return w.wrapped.ComputeSplitKey(ctx, start, end) +} + +// GetSpanConfigForKey implements spanconfig.StoreReader. +func (w *WrappedKVSubscriber) GetSpanConfigForKey( + ctx context.Context, key roachpb.RKey, +) (roachpb.SpanConfig, roachpb.Span, error) { + spanConfig, span, err := w.wrapped.GetSpanConfigForKey(ctx, key) + for _, o := range w.overrides { + if key.Equal(o.key) { + return o.config, span, nil + } + } + return spanConfig, span, err +} + +// NeedsSplit implements spanconfig.StoreReader. +func (w *WrappedKVSubscriber) NeedsSplit( + ctx context.Context, start roachpb.RKey, end roachpb.RKey, +) (bool, error) { + return w.wrapped.NeedsSplit(ctx, start, end) +} + +var _ spanconfig.KVSubscriber = &WrappedKVSubscriber{} From a83214e4d96d1ba5c6dce81f94d635aaee7a7f97 Mon Sep 17 00:00:00 2001 From: Andrew Baptist Date: Fri, 19 Apr 2024 15:02:04 -0400 Subject: [PATCH 5/6] kvserver: fix TestStoreRangeSplitAndMergeWithGlobalReads Directly set the span config for the range under test rather than setting the ZoneConfig and waiting for it to propagate. In addition to simplifying the test it also makes it run faster. Fixes: #119230 Epic: none Release note: None --- pkg/kv/kvserver/BUILD.bazel | 1 + pkg/kv/kvserver/client_split_test.go | 43 ++++++++++++++-------------- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 2652d0370f12..c6963dc89c11 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -454,6 +454,7 @@ go_test( "//pkg/spanconfig", "//pkg/spanconfig/spanconfigptsreader", "//pkg/spanconfig/spanconfigstore", + "//pkg/spanconfig/spanconfigtestutils", "//pkg/sql", "//pkg/sql/catalog/bootstrap", "//pkg/sql/catalog/catalogkeys", diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index 288186f24be8..19091f9cf1c4 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -51,6 +51,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" @@ -4007,8 +4008,6 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - skip.WithIssue(t, 119230) - // Detect splits and merges over the global read ranges. Assert that the split // and merge transactions commit with pushed write timestamps, and that the // commit-wait sleep for these transactions is performed before running their @@ -4043,6 +4042,14 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { return nil } + descID := bootstrap.TestingUserDescID(0) + descKey := keys.SystemSQLCodec.TablePrefix(descID) + splitKey := append(descKey, []byte("split")...) + + // Set global reads for the test ranges. + spanConfig := zonepb.DefaultZoneConfig().AsSpanConfig() + spanConfig.GlobalReads = true + ctx := context.Background() s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ Knobs: base.TestingKnobs{ @@ -4050,6 +4057,14 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { DisableMergeQueue: true, TestingResponseFilter: respFilter, }, + SpanConfig: &spanconfig.TestingKnobs{ + StoreKVSubscriberOverride: func(original spanconfig.KVSubscriber) spanconfig.KVSubscriber { + wrapped := spanconfigtestutils.NewWrappedKVSubscriber(original) + wrapped.AddOverride(descKey, spanConfig) + wrapped.AddOverride(splitKey, spanConfig) + return wrapped + }, + }, }, }) @@ -4062,11 +4077,8 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.closed_timestamp_refresh_interval = '20ms'`) store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) require.NoError(t, err) - config.TestingSetupZoneConfigHook(s.Stopper()) // Split off the range for the test. - descID := bootstrap.TestingUserDescID(0) - descKey := keys.SystemSQLCodec.TablePrefix(descID) splitArgs := adminSplitArgs(descKey) _, pErr := kv.SendWrapped(ctx, store.TestSender(), splitArgs) require.Nil(t, pErr) @@ -4075,21 +4087,9 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { // response filter. clockPtr.Store(s.Clock()) - // Set global reads. - zoneConfig := zonepb.DefaultZoneConfig() - zoneConfig.GlobalReads = proto.Bool(true) - config.TestingSetZoneConfig(config.ObjectID(descID), zoneConfig) - - // Perform a write to the system config span being watched by - // the SystemConfigProvider. - tdb.Exec(t, "CREATE TABLE foo ()") - testutils.SucceedsSoon(t, func() error { - repl := store.LookupReplica(roachpb.RKey(descKey)) - if repl.ClosedTimestampPolicy() != roachpb.LEAD_FOR_GLOBAL_READS { - return errors.Errorf("expected LEAD_FOR_GLOBAL_READS policy") - } - return nil - }) + // Verify that the closed timestamp policy is set up. + repl := store.LookupReplica(roachpb.RKey(descKey)) + require.Equal(t, repl.ClosedTimestampPolicy(), roachpb.LEAD_FOR_GLOBAL_READS) // Write to the range, which has the effect of bumping the closed timestamp. pArgs := putArgs(descKey, []byte("foo")) @@ -4097,14 +4097,13 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { require.Nil(t, pErr) // Split the range. Should succeed. - splitKey := append(descKey, []byte("split")...) splitArgs = adminSplitArgs(splitKey) _, pErr = kv.SendWrapped(ctx, store.TestSender(), splitArgs) require.Nil(t, pErr) require.Equal(t, int64(1), store.Metrics().CommitWaitsBeforeCommitTrigger.Count()) require.Equal(t, int64(1), atomic.LoadInt64(&splits)) - repl := store.LookupReplica(roachpb.RKey(splitKey)) + repl = store.LookupReplica(roachpb.RKey(splitKey)) require.Equal(t, splitKey, repl.Desc().StartKey.AsRawKey()) // Merge the range. Should succeed. From d7c87e69cf127cbf449859e8d73bdc049978c21a Mon Sep 17 00:00:00 2001 From: Andrew Baptist Date: Fri, 19 Apr 2024 15:08:38 -0400 Subject: [PATCH 6/6] kvserver: remove dependency on SQL for the test Previously this test required ZoneConfig changes and waited for them to propagate using rangefeeds. We no longer have that dependency as the span config is set directly so we can remove any need for SQL and no longer update the settings for rangefeeds. Epic: none Release note: None --- pkg/kv/kvserver/client_split_test.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/pkg/kv/kvserver/client_split_test.go b/pkg/kv/kvserver/client_split_test.go index 19091f9cf1c4..090ba83b6cfa 100644 --- a/pkg/kv/kvserver/client_split_test.go +++ b/pkg/kv/kvserver/client_split_test.go @@ -4051,7 +4051,8 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { spanConfig.GlobalReads = true ctx := context.Background() - s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ + s := serverutils.StartServerOnly(t, base.TestServerArgs{ + DisableSQLServer: true, Knobs: base.TestingKnobs{ Store: &kvserver.StoreTestingKnobs{ DisableMergeQueue: true, @@ -4071,10 +4072,6 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) { defer s.Stopper().Stop(ctx) // Set the closed_timestamp interval to be short to shorten the test duration // because we need to wait for a checkpoint on the system config. - tdb := sqlutils.MakeSQLRunner(sqlDB) - tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '20ms'`) - tdb.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '20ms'`) - tdb.Exec(t, `SET CLUSTER SETTING kv.rangefeed.closed_timestamp_refresh_interval = '20ms'`) store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) require.NoError(t, err)