Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
122710: kvserver: fix TestStoreRangeSplitAndMergeWithGlobalReads r=arulajmani a=andrewbaptist

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: cockroachdb#119230

Epic: none

Release note: None

122720: sql/delegate: don't include external connections in SHOW SYSTEM GRANTS r=rafiss a=rafiss

Epic: None
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 syntax should be used.

Co-authored-by: Andrew Baptist <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
3 people committed Apr 22, 2024
3 parents b69335c + d7c87e6 + 4fb65fb commit e215cf4
Show file tree
Hide file tree
Showing 10 changed files with 177 additions and 101 deletions.
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/client_spanconfigs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ func TestSpanConfigUpdateAppliedToReplica(t *testing.T) {
DisableGCQueue: true,
},
SpanConfig: &spanconfig.TestingKnobs{
StoreKVSubscriberOverride: mockSubscriber,
StoreKVSubscriberOverride: func(spanconfig.KVSubscriber) spanconfig.KVSubscriber { return mockSubscriber },
},
},
}
Expand Down Expand Up @@ -120,7 +120,7 @@ func TestFallbackSpanConfigOverride(t *testing.T) {
DisableGCQueue: true,
},
SpanConfig: &spanconfig.TestingKnobs{
StoreKVSubscriberOverride: mockSubscriber,
StoreKVSubscriberOverride: func(spanconfig.KVSubscriber) spanconfig.KVSubscriber { return mockSubscriber },
},
},
}
Expand Down
72 changes: 26 additions & 46 deletions pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -4042,38 +4041,41 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) {
}
return nil
}
// Set global reads.
zoneConfig := zonepb.DefaultZoneConfig()
zoneConfig.GlobalReads = proto.Bool(true)

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{
s := serverutils.StartServerOnly(t, base.TestServerArgs{
DisableSQLServer: true,
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
DefaultZoneConfigOverride: &zoneConfig,
},

Store: &kvserver.StoreTestingKnobs{
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
},
},
},
})

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)
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)
Expand All @@ -4082,52 +4084,30 @@ func TestStoreRangeSplitAndMergeWithGlobalReads(t *testing.T) {
// response filter.
clockPtr.Store(s.Clock())

// 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
})

// 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
})
// 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"))
_, pErr = kv.SendWrapped(ctx, store.TestSender(), pArgs)
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)
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))
repl = store.LookupReplica(roachpb.RKey(splitKey))
require.Equal(t, splitKey, repl.Desc().StartKey.AsRawKey())

// Merge the range. Should succeed.
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))
Expand Down
68 changes: 34 additions & 34 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down
1 change: 1 addition & 0 deletions pkg/spanconfig/spanconfigtestutils/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
81 changes: 81 additions & 0 deletions pkg/spanconfig/spanconfigtestutils/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package spanconfigtestutils

import (
"context"
"fmt"
"reflect"
"regexp"
Expand All @@ -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"
Expand Down Expand Up @@ -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{}
2 changes: 1 addition & 1 deletion pkg/spanconfig/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/delegate/show_grants.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 *
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Loading

0 comments on commit e215cf4

Please sign in to comment.