Skip to content

Commit

Permalink
Merge #69172 #69515
Browse files Browse the repository at this point in the history
69172: kvserver: abstract away system config span usage in kv       r=irfansharif a=irfansharif

Part of #67679. We'll hide `config.SystemConfig` behind the
`spanconfig.StoreReader` interface, and use that instead in the various
queues that need access to the system config span. In future PRs we'll
introduce a data structure that maintains a mapping between spans and
configs that implements this same interface, except it will be powered
by a view over `system.span_configurations` that was introduced in
\#69047.

When we do make that switch, i.e. have KV consult this new thing for
splits, merges, GC and replication, instead of the gossip backed system
config span, ideally it'd be as easy as swapping the source. This PR
helps pave the way for just that.

In #66348 we described how `zonepb.ZoneConfigs` going forward were going
to be an exclusively SQL-level construct. Consequently we purge[*] all
usages of it in KV, storing on each replica a `roachpb.SpanConfig`
instead.

[*]: The only remaining use is what powers our replication reports,
which does not extend well to multi-tenancy in its current form and
needs replacing.

Release note: None

Release justification: low risk, high benefit changes to existing functionality

69515: kvserver: minor test improvement r=andreimatei a=andreimatei

Remove an unncecessary clock.

Release note: None
Release justification: test only

Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
  • Loading branch information
3 people committed Aug 30, 2021
3 parents cb6889c + 2e00964 + d59529f commit a0bfc85
Show file tree
Hide file tree
Showing 70 changed files with 1,607 additions and 1,334 deletions.
16 changes: 8 additions & 8 deletions pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/cli/clierrorplus"
"github.com/cockroachdb/cockroach/pkg/cli/syncbench"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
Expand Down Expand Up @@ -672,7 +671,7 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error {
defer stopper.Stop(context.Background())

var rangeID roachpb.RangeID
gcTTLInSeconds := int64((24 * time.Hour).Seconds())
gcTTL := 24 * time.Hour
intentAgeThreshold := gc.IntentAgeThreshold.Default()
intentBatchSize := gc.MaxIntentsPerCleanupBatch.Default()

Expand All @@ -683,10 +682,11 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error {
}
}
if len(args) > 2 {
var err error
if gcTTLInSeconds, err = parsePositiveInt(args[2]); err != nil {
gcTTLInSeconds, err := parsePositiveInt(args[2])
if err != nil {
return errors.Wrapf(err, "unable to parse %v as TTL", args[2])
}
gcTTL = time.Duration(gcTTLInSeconds) * time.Second
}
if len(args) > 1 {
var err error
Expand Down Expand Up @@ -736,14 +736,14 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error {
for _, desc := range descs {
snap := db.NewSnapshot()
defer snap.Close()
policy := zonepb.GCPolicy{TTLSeconds: int32(gcTTLInSeconds)}
now := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()}
thresh := gc.CalculateThreshold(now, policy)
thresh := gc.CalculateThreshold(now, gcTTL)
info, err := gc.Run(
context.Background(),
&desc, snap,
now, thresh, gc.RunOptions{IntentAgeThreshold: intentAgeThreshold, MaxIntentsPerIntentCleanupBatch: intentBatchSize}, policy,
gc.NoopGCer{},
now, thresh,
gc.RunOptions{IntentAgeThreshold: intentAgeThreshold, MaxIntentsPerIntentCleanupBatch: intentBatchSize},
gcTTL, gc.NoopGCer{},
func(_ context.Context, _ []roachpb.Intent) error { return nil },
func(_ context.Context, _ *roachpb.Transaction) error { return nil },
)
Expand Down
12 changes: 12 additions & 0 deletions pkg/config/system.go
Original file line number Diff line number Diff line change
Expand Up @@ -294,6 +294,18 @@ func (s *SystemConfig) GetZoneConfigForKey(key roachpb.RKey) (*zonepb.ZoneConfig
return s.getZoneConfigForKey(DecodeKeyIntoZoneIDAndSuffix(key))
}

// GetSpanConfigForKey looks of the span config for the given key. It's part of
// spanconfig.StoreReader interface.
func (s *SystemConfig) GetSpanConfigForKey(
ctx context.Context, key roachpb.RKey,
) (roachpb.SpanConfig, error) {
zone, err := s.GetZoneConfigForKey(key)
if err != nil {
return roachpb.SpanConfig{}, err
}
return zone.AsSpanConfig(), nil
}

// DecodeKeyIntoZoneIDAndSuffix figures out the zone that the key belongs to.
func DecodeKeyIntoZoneIDAndSuffix(key roachpb.RKey) (id SystemTenantObjectID, keySuffix []byte) {
objectID, keySuffix, ok := DecodeSystemTenantObjectID(key)
Expand Down
7 changes: 4 additions & 3 deletions pkg/config/system_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -544,6 +544,7 @@ func TestComputeSplitKeyTenantBoundaries(t *testing.T) {
func TestGetZoneConfigForKey(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
testCases := []struct {
key roachpb.RKey
expectedID config.SystemTenantObjectID
Expand Down Expand Up @@ -614,11 +615,11 @@ func TestGetZoneConfigForKey(t *testing.T) {
_ *config.SystemConfig, id config.SystemTenantObjectID,
) (*zonepb.ZoneConfig, *zonepb.ZoneConfig, bool, error) {
objectID = id
return &zonepb.ZoneConfig{}, nil, false, nil
return cfg.DefaultZoneConfig, nil, false, nil
}
_, err := cfg.GetZoneConfigForKey(tc.key)
_, err := cfg.GetSpanConfigForKey(ctx, tc.key)
if err != nil {
t.Errorf("#%d: GetZoneConfigForKey(%v) got error: %v", tcNum, tc.key, err)
t.Errorf("#%d: GetSpanConfigForKey(%v) got error: %v", tcNum, tc.key, err)
}
if objectID != tc.expectedID {
t.Errorf("#%d: GetZoneConfigForKey(%v) got %d; want %d", tcNum, tc.key, objectID, tc.expectedID)
Expand Down
53 changes: 1 addition & 52 deletions pkg/config/zonepb/zone.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"context"
"fmt"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/keys"
Expand Down Expand Up @@ -206,20 +205,6 @@ func NewZoneConfig() *ZoneConfig {
}
}

// EmptyCompleteZoneConfig is the zone configuration where
// all fields are set but set to their respective zero values.
func EmptyCompleteZoneConfig() *ZoneConfig {
return &ZoneConfig{
NumReplicas: proto.Int32(0),
NumVoters: proto.Int32(0),
RangeMinBytes: proto.Int64(0),
RangeMaxBytes: proto.Int64(0),
GC: &GCPolicy{TTLSeconds: 0},
InheritedConstraints: true,
InheritedLeasePreferences: true,
}
}

// DefaultZoneConfig is the default zone configuration used when no custom
// config has been specified.
func DefaultZoneConfig() ZoneConfig {
Expand Down Expand Up @@ -977,37 +962,6 @@ func (z ZoneConfig) GetSubzoneForKeySuffix(keySuffix []byte) (*Subzone, int32) {
return nil, -1
}

// GetNumVoters returns the number of voting replicas for the given zone config.
//
// This method will panic if called on a ZoneConfig with an uninitialized
// NumReplicas attribute.
func (z *ZoneConfig) GetNumVoters() int32 {
if z.NumReplicas == nil {
panic("NumReplicas must not be nil")
}
if z.NumVoters != nil && *z.NumVoters != 0 {
return *z.NumVoters
}
return *z.NumReplicas
}

// GetNumNonVoters returns the number of non-voting replicas as defined in the
// zone config.
//
// This method will panic if called on a ZoneConfig with an uninitialized
// NumReplicas attribute.
func (z *ZoneConfig) GetNumNonVoters() int32 {
if z.NumReplicas == nil {
panic("NumReplicas must not be nil")
}
if z.NumVoters != nil && *z.NumVoters != 0 {
return *z.NumReplicas - *z.NumVoters
}
// `num_voters` hasn't been explicitly configured. Every replica should be a
// voting replica.
return 0
}

// SetSubzone installs subzone into the ZoneConfig, overwriting any existing
// subzone with the same IndexID and PartitionName.
func (z *ZoneConfig) SetSubzone(subzone Subzone) {
Expand Down Expand Up @@ -1150,11 +1104,6 @@ func (c *Constraint) GetValue() string {
return c.Value
}

// TTL returns the implies TTL as a time.Duration.
func (m *GCPolicy) TTL() time.Duration {
return time.Duration(m.TTLSeconds) * time.Second
}

// EnsureFullyHydrated returns an assertion error if the zone config is not
// fully hydrated. A fully hydrated zone configuration must have all required
// fields set, which are RangeMaxBytes, RangeMinBytes, GC, and NumReplicas.
Expand Down Expand Up @@ -1201,7 +1150,7 @@ func (z *ZoneConfig) toSpanConfig() (roachpb.SpanConfig, error) {
// Copy over the values.
sc.RangeMinBytes = *z.RangeMinBytes
sc.RangeMaxBytes = *z.RangeMaxBytes
sc.GCTTL = z.GC.TTLSeconds
sc.GCPolicy.TTLSeconds = z.GC.TTLSeconds

// GlobalReads is false by default.
if z.GlobalReads != nil {
Expand Down
54 changes: 33 additions & 21 deletions pkg/config/zonepb/zone_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1183,9 +1183,11 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) {
},
},
expectSpanConfig: roachpb.SpanConfig{
RangeMinBytes: 100000,
RangeMaxBytes: 200000,
GCTTL: 2400,
RangeMinBytes: 100000,
RangeMaxBytes: 200000,
GCPolicy: roachpb.GCPolicy{
TTLSeconds: 2400,
},
GlobalReads: false,
NumVoters: 0,
NumReplicas: 3,
Expand All @@ -1206,9 +1208,11 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) {
},
},
expectSpanConfig: roachpb.SpanConfig{
RangeMinBytes: 100000,
RangeMaxBytes: 200000,
GCTTL: 2400,
RangeMinBytes: 100000,
RangeMaxBytes: 200000,
GCPolicy: roachpb.GCPolicy{
TTLSeconds: 2400,
},
GlobalReads: true,
NumVoters: 0,
NumReplicas: 3,
Expand All @@ -1229,9 +1233,11 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) {
},
},
expectSpanConfig: roachpb.SpanConfig{
RangeMinBytes: 100000,
RangeMaxBytes: 200000,
GCTTL: 2400,
RangeMinBytes: 100000,
RangeMaxBytes: 200000,
GCPolicy: roachpb.GCPolicy{
TTLSeconds: 2400,
},
GlobalReads: false,
NumVoters: 0,
NumReplicas: 3,
Expand Down Expand Up @@ -1287,10 +1293,12 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) {
expectSpanConfig: roachpb.SpanConfig{
RangeMinBytes: 100000,
RangeMaxBytes: 200000,
GCTTL: 2400,
GlobalReads: false,
NumVoters: 0,
NumReplicas: 3,
GCPolicy: roachpb.GCPolicy{
TTLSeconds: 2400,
},
GlobalReads: false,
NumVoters: 0,
NumReplicas: 3,
Constraints: []roachpb.ConstraintsConjunction{
{
NumReplicas: 1,
Expand Down Expand Up @@ -1333,11 +1341,13 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) {
expectSpanConfig: roachpb.SpanConfig{
RangeMinBytes: 100000,
RangeMaxBytes: 200000,
GCTTL: 2400,
GlobalReads: false,
NumVoters: 0,
NumReplicas: 3,
Constraints: []roachpb.ConstraintsConjunction{},
GCPolicy: roachpb.GCPolicy{
TTLSeconds: 2400,
},
GlobalReads: false,
NumVoters: 0,
NumReplicas: 3,
Constraints: []roachpb.ConstraintsConjunction{},
VoterConstraints: []roachpb.ConstraintsConjunction{
{
Constraints: []roachpb.Constraint{
Expand Down Expand Up @@ -1381,9 +1391,11 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) {
},
},
expectSpanConfig: roachpb.SpanConfig{
RangeMinBytes: 100000,
RangeMaxBytes: 200000,
GCTTL: 2400,
RangeMinBytes: 100000,
RangeMaxBytes: 200000,
GCPolicy: roachpb.GCPolicy{
TTLSeconds: 2400,
},
GlobalReads: false,
NumVoters: 0,
NumReplicas: 3,
Expand Down
1 change: 0 additions & 1 deletion pkg/kv/kvclient/kvcoord/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,6 @@ go_test(
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//errutil",
"@com_github_cockroachdb_redact//:redact",
"@com_github_gogo_protobuf//proto",
"@com_github_golang_mock//gomock",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
Expand Down
10 changes: 4 additions & 6 deletions pkg/kv/kvclient/kvcoord/split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
Expand All @@ -32,7 +31,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/errors"
"github.com/gogo/protobuf/proto"
)

// startTestWriter creates a writer which initiates a sequence of
Expand Down Expand Up @@ -176,15 +174,15 @@ func TestRangeSplitsWithConcurrentTxns(t *testing.T) {
func TestRangeSplitsWithWritePressure(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
// Override default zone config.
cfg := zonepb.DefaultZoneConfigRef()
cfg.RangeMaxBytes = proto.Int64(1 << 18)
// Override default span config.
cfg := kvserver.TestingDefaultSpanConfig()
cfg.RangeMaxBytes = 1 << 18

// Manually create the local test cluster so that the split queue
// is not disabled (LocalTestCluster disables it by default).
s := &localtestcluster.LocalTestCluster{
Cfg: kvserver.StoreConfig{
DefaultZoneConfig: cfg,
DefaultSpanConfig: cfg,
},
StoreTestingKnobs: &kvserver.StoreTestingKnobs{
DisableScanner: true,
Expand Down
2 changes: 2 additions & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -152,6 +152,7 @@ go_library(
"//pkg/server/telemetry",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/spanconfig",
"//pkg/sql/sessiondata",
"//pkg/sql/sqlutil",
"//pkg/storage",
Expand Down Expand Up @@ -340,6 +341,7 @@ go_test(
"//pkg/server/serverpb",
"//pkg/server/telemetry",
"//pkg/settings/cluster",
"//pkg/spanconfig",
"//pkg/sql",
"//pkg/sql/catalog/bootstrap",
"//pkg/sql/catalog/catalogkeys",
Expand Down
Loading

0 comments on commit a0bfc85

Please sign in to comment.