Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
68652: kvserver: count draining nodes as live when computing quorum r=aayushshah15 a=aayushshah15

Similar to #67714

Draining nodes were considered non-live by the allocator when it made the
determination of whether a range could achieve quorum. This meant that, for
instance, on a cluster with a replication factor of 5, if we had 3 or more
nodes marked draining, we (with a high likelihood) wouldn't be able to
decommission _any other_ nodes from the cluster.

Furthermore, due to the same reason as above the system also would incorrectly
decide to not rebalance ranges that had more than a quorum of replicas on
draining nodes.

This patch fixes this problem by considering replicas on draining nodes as live
for the purposes of determining whether a range has quorum. This likely fixes a
subset of "stuck decommissioning" issues we've seen in the wild.

Follows from cockroachlabs/support#1105

Release justification: bug fix

Release note(bug fix): Previously, draining a quorum of nodes (i.e. >=2 if
replication factor is 3, >=3 if replication factor is 5, etc) would block the
subsequent decommissioning of any other nodes in the cluster. This patch fixes
this bug. Now, if the lowest replication factor of some zone in the cluster is
RF, operators should be able to safely drain up to RF-1 nodes simultaneously.

69115: mt_start_sql: enable enterprise features for multitenant sql servers r=JeffSwenson a=JeffSwenson

Enterprise features are controlled by the enterprise.license setting.
Currently this setting applies only to the host tenant cluster. This
change enables enterprise features for all tenant clusters. Enabling
enterprise features for all tenants is reasonable, because multi
tenant deployments are an enterprise feature.

Release note: None

69571: jobs: don't reset next_run when resuming active schedules r=pbardea a=pbardea

Consider an active schedule that was created with a specific first_run
time. The first_run would populate the next_run_time on the schedule.
The resumption of this schedule before it executed would re-evaluate the
next runtime based off the schedule's recurrence.

This commit changes the scheduling system to only recompute the next run
time on paused schedules.

Release justification: bug fix
Release note (bug fix): Fix a bug where resuming an active schedule
would always reset its next run time. This was sometimes undesirable
with schedules that had a first_run option specified.

69696: kvserver: stop transferring leases to replicas that may need snapshots r=aayushshah15 a=aayushshah15

This commit disallows the `replicateQueue` from initiating lease
transfers to replicas that may be in need of a raft snapshot. Note that
the `StoreRebalancer` already has a stronger form of this check since it
disallows lease transfers to replicas that are lagging behind the raft
leader (which includes the set of replicas that need a snapshot).

In cases where the raft leader is not the leaseholder, we disallow the
replicateQueue from any sort of lease transfer until leaseholdership and
leadership are collocated. We rely on calls to
`maybeTransferRaftLeadershipToLeaseholderLocked()` (called on every raft
tick) to make sure that such periods of leadership / leaseholdership
misalignment are ephemeral and rare.

Alternative to #63507

Release justification: bug fix

Resolves #61604

Release note (bug fix): Fixes a bug that can cause prolonged
unavailability due to lease transfer to a replica that may be in need of
a raft snapshot.

69727: kv: deflake TestPriorityRatchetOnAbortOrPush r=nvanbenschoten a=nvanbenschoten

Fixes #68584.

The test was flaky for the reasons described in #68584. There doesn't appear to
be an easy way to fix this behavior, and it's not clear how valuable doing so
even is given how little we rely on transaction priorities anymore, so the
commit just deflakes the test by rejecting them.

Release justification: deflaking a test.

69728: clusterversion,kv: remove old cluster versions corresponding to 20.2 and 21.1 r=nvanbenschoten a=nvanbenschoten

Fixes most of #66544.

This pull request removes (almost) all old cluster versions corresponding to 20.2 and 21.1 which fall under the KV group's responsibility.

The PR leaves one remaining "combo version" of `TruncatedAndRangeAppliedStateMigration` plus `PostTruncatedAndRangeAppliedStateMigration`, which we'll want to remove before closing #66544. I've left this for @irfansharif both because its removal is a little more involved than the others and because I figured that he deserves to reap the fruit of his labor and get to delete the code related to the replicated truncated state and missing RangeAppliedStateKeys. Making that possible was a hard-earned win.

Release justification: cluster version cleanup

Co-authored-by: Aayush Shah <[email protected]>
Co-authored-by: Jeff <[email protected]>
Co-authored-by: Paul Bardea <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
5 people committed Sep 2, 2021
7 parents e931244 + ff2ae38 + b42a9b3 + 3e0bbe1 + b344fb8 + aa71648 + ef4710c commit 2a33514
Show file tree
Hide file tree
Showing 39 changed files with 722 additions and 309 deletions.
2 changes: 2 additions & 0 deletions pkg/ccl/serverccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ go_test(
"//pkg/ccl",
"//pkg/ccl/kvccl",
"//pkg/ccl/utilccl",
"//pkg/ccl/utilccl/licenseccl",
"//pkg/roachpb:with-mocks",
"//pkg/rpc",
"//pkg/security",
Expand All @@ -39,6 +40,7 @@ go_test(
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util",
"//pkg/util/envutil",
"//pkg/util/httputil",
"//pkg/util/leaktest",
"//pkg/util/log",
Expand Down
26 changes: 26 additions & 0 deletions pkg/ccl/serverccl/server_sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,15 @@ import (
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl/licenseccl"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/httputil"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -84,6 +87,29 @@ func TestTenantCannotSetClusterSetting(t *testing.T) {
require.Equal(t, pq.ErrorCode(pgcode.InsufficientPrivilege.String()), pqErr.Code, "err %v has unexpected code", err)
}

func TestTenantCanUseEnterpriseFeatures(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

license, _ := (&licenseccl.License{
Type: licenseccl.License_Enterprise,
}).Encode()

defer utilccl.TestingDisableEnterprise()()
defer envutil.TestSetEnv(t, "COCKROACH_TENANT_LICENSE", license)()

tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{})
defer tc.Stopper().Stop(context.Background())

_, db := serverutils.StartTenant(t, tc.Server(0), base.TestTenantArgs{TenantID: serverutils.TestTenantID(), AllowSettingClusterSettings: false})
defer db.Close()

_, err := db.Exec(`BACKUP INTO 'userfile:///backup'`)
require.NoError(t, err)
_, err = db.Exec(`BACKUP INTO LATEST IN 'userfile:///backup'`)
require.NoError(t, err)
}

func TestTenantUnauthenticatedAccess(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down
3 changes: 3 additions & 0 deletions pkg/ccl/utilccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,13 +13,15 @@ go_library(
"//pkg/base",
"//pkg/ccl/utilccl/licenseccl",
"//pkg/kv/kvclient/kvcoord:with-mocks",
"//pkg/server",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql/catalog/colinfo",
"//pkg/sql/flowinfra",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/types",
"//pkg/util/envutil",
"//pkg/util/grpcutil",
"//pkg/util/timeutil",
"//pkg/util/uuid",
Expand All @@ -41,6 +43,7 @@ go_test(
"//pkg/ccl/utilccl/licenseccl",
"//pkg/settings/cluster",
"//pkg/testutils",
"//pkg/util/envutil",
"//pkg/util/timeutil",
"//pkg/util/uuid",
"@com_github_stretchr_testify//require",
Expand Down
46 changes: 31 additions & 15 deletions pkg/ccl/utilccl/license_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,12 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl/licenseccl"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
Expand All @@ -45,16 +47,13 @@ var enterpriseLicense = func() *settings.StringSetting {
return s
}()

// testingEnterprise determines whether the cluster is enabled
// or disabled for the purposes of testing.
// It should be loaded and stored using atomic as it can race with an
// in progress kv reader during TestingDisableEnterprise /
// TestingEnableEnterprise.
var testingEnterprise int32
// enterpriseStatus determines whether the cluster is enabled
// for enterprise features or if enterprise status depends on the license.
var enterpriseStatus int32 = deferToLicense

const (
testingEnterpriseDisabled = 0
testingEnterpriseEnabled = 1
deferToLicense = 0
enterpriseEnabled = 1
)

// errEnterpriseRequired is returned by check() when the caller does
Expand All @@ -68,22 +67,38 @@ type licenseCacheKey string

// TestingEnableEnterprise allows overriding the license check in tests.
func TestingEnableEnterprise() func() {
before := atomic.LoadInt32(&testingEnterprise)
atomic.StoreInt32(&testingEnterprise, testingEnterpriseEnabled)
before := atomic.LoadInt32(&enterpriseStatus)
atomic.StoreInt32(&enterpriseStatus, enterpriseEnabled)
return func() {
atomic.StoreInt32(&testingEnterprise, before)
atomic.StoreInt32(&enterpriseStatus, before)
}
}

// TestingDisableEnterprise allows re-enabling the license check in tests.
func TestingDisableEnterprise() func() {
before := atomic.LoadInt32(&testingEnterprise)
atomic.StoreInt32(&testingEnterprise, testingEnterpriseDisabled)
before := atomic.LoadInt32(&enterpriseStatus)
atomic.StoreInt32(&enterpriseStatus, deferToLicense)
return func() {
atomic.StoreInt32(&testingEnterprise, before)
atomic.StoreInt32(&enterpriseStatus, before)
}
}

// ApplyTenantLicense verifies the COCKROACH_TENANT_LICENSE environment variable
// and enables enterprise features for the process. This is a bit of a hack and
// should be replaced once it is possible to read the host cluster's
// enterprise.license setting.
func ApplyTenantLicense() error {
license, ok := envutil.EnvString("COCKROACH_TENANT_LICENSE", 0)
if !ok {
return nil
}
if _, err := decode(license); err != nil {
return errors.Wrap(err, "COCKROACH_TENANT_LICENSE encoding is invalid")
}
atomic.StoreInt32(&enterpriseStatus, enterpriseEnabled)
return nil
}

// CheckEnterpriseEnabled returns a non-nil error if the requested enterprise
// feature is not enabled, including information or a link explaining how to
// enable it.
Expand All @@ -108,6 +123,7 @@ func init() {
base.CheckEnterpriseEnabled = CheckEnterpriseEnabled
base.LicenseType = getLicenseType
base.TimeToEnterpriseLicenseExpiry = TimeToEnterpriseLicenseExpiry
server.ApplyTenantLicense = ApplyTenantLicense
}

// TimeToEnterpriseLicenseExpiry returns a Duration from `asOf` until the current
Expand All @@ -128,7 +144,7 @@ func TimeToEnterpriseLicenseExpiry(
func checkEnterpriseEnabledAt(
st *cluster.Settings, at time.Time, cluster uuid.UUID, org, feature string, withDetails bool,
) error {
if atomic.LoadInt32(&testingEnterprise) == testingEnterpriseEnabled {
if atomic.LoadInt32(&enterpriseStatus) == enterpriseEnabled {
return nil
}
license, err := getLicense(st)
Expand Down
38 changes: 38 additions & 0 deletions pkg/ccl/utilccl/license_check_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl/licenseccl"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -188,3 +189,40 @@ func TestTimeToEnterpriseLicenseExpiry(t *testing.T) {
})
}
}

func TestApplyTenantLicenseWithLicense(t *testing.T) {
license, _ := (&licenseccl.License{
Type: licenseccl.License_Enterprise,
}).Encode()

defer TestingDisableEnterprise()()
defer envutil.TestSetEnv(t, "COCKROACH_TENANT_LICENSE", license)()

settings := cluster.MakeClusterSettings()

require.Error(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.False(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.NoError(t, ApplyTenantLicense())
require.NoError(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.True(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
}

func TestApplyTenantLicenseWithoutLicense(t *testing.T) {
defer TestingDisableEnterprise()()

settings := cluster.MakeClusterSettings()
_, ok := envutil.EnvString("COCKROACH_TENANT_LICENSE", 0)
envutil.ClearEnvCache()
require.False(t, ok)

require.Error(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.False(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.NoError(t, ApplyTenantLicense())
require.Error(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.False(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
}

func TestApplyTenantLicenseWithInvalidLicense(t *testing.T) {
defer envutil.TestSetEnv(t, "COCKROACH_TENANT_LICENSE", "THIS IS NOT A VALID LICENSE")()
require.Error(t, ApplyTenantLicense())
}
48 changes: 1 addition & 47 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ type Key int
// You'll then want to backport (i) to the release branch itself (i.e.
// release-20.2). You'll also want to bump binaryMinSupportedVersion. In the
// example above, you'll set it to V20_2. This indicates that the
// minimum binary version required in a cluster with with nodes running
// minimum binary version required in a cluster with nodes running
// v21.1 binaries (including pre-release alphas) is v20.2, i.e. that an
// upgrade into such a binary must start out from at least v20.2 nodes.
//
Expand Down Expand Up @@ -163,14 +163,8 @@ const (
// the 21.1 release. This is because we now support tenants at the
// predecessor binary interacting with a fully upgraded KV cluster.
Start20_2
// NodeMembershipStatus gates the usage of the MembershipStatus enum in the
// Liveness proto. See comment on proto definition for more details.
NodeMembershipStatus
// MinPasswordLength adds the server.user_login.min_password_length setting.
MinPasswordLength
// AbortSpanBytes adds a field to MVCCStats
// (MVCCStats.AbortSpanBytes) that tracks the size of a range's abort span.
AbortSpanBytes
// CreateLoginPrivilege is when CREATELOGIN/NOCREATELOGIN are introduced.
//
// It represents adding authn principal management via CREATELOGIN role
Expand All @@ -186,10 +180,6 @@ const (
//
// Start21_1 demarcates work towards CockroachDB v21.1.
Start21_1
// CPutInline is conditional put support for inline values.
CPutInline
// ReplicaVersions enables the versioning of Replica state.
ReplicaVersions
// replacedTruncatedAndRangeAppliedStateMigration stands in for
// TruncatedAndRangeAppliedStateMigration which was re-introduced after the
// migration job was introduced. This is necessary because the jobs
Expand All @@ -216,18 +206,10 @@ const (
// using the replicated legacy TruncatedState. It's also used in asserting
// that no replicated truncated state representation is found.
PostTruncatedAndRangeAppliedStateMigration
// SeparatedIntents allows the writing of separated intents/locks.
SeparatedIntents
// TracingVerbosityIndependentSemantics marks a change in which trace spans
// are propagated across RPC boundaries independently of their verbosity setting.
// This requires a version gate this violates implicit assumptions in v20.2.
TracingVerbosityIndependentSemantics
// PriorReadSummaries introduces support for the use of read summary objects
// to ship information about reads on a range through lease changes and
// range merges.
PriorReadSummaries
// NonVotingReplicas enables the creation of non-voting replicas.
NonVotingReplicas
// V21_1 is CockroachDB v21.1. It's used for all v21.1.x patch releases.
V21_1

Expand Down Expand Up @@ -345,18 +327,10 @@ var versionsSingleton = keyedVersions{
Key: Start20_2,
Version: roachpb.Version{Major: 20, Minor: 1, Internal: 1},
},
{
Key: NodeMembershipStatus,
Version: roachpb.Version{Major: 20, Minor: 1, Internal: 11},
},
{
Key: MinPasswordLength,
Version: roachpb.Version{Major: 20, Minor: 1, Internal: 13},
},
{
Key: AbortSpanBytes,
Version: roachpb.Version{Major: 20, Minor: 1, Internal: 14},
},
{
Key: CreateLoginPrivilege,
Version: roachpb.Version{Major: 20, Minor: 1, Internal: 20},
Expand All @@ -375,14 +349,6 @@ var versionsSingleton = keyedVersions{
Key: Start21_1,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 2},
},
{
Key: CPutInline,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 10},
},
{
Key: ReplicaVersions,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 12},
},
{
Key: replacedTruncatedAndRangeAppliedStateMigration,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 14},
Expand All @@ -399,22 +365,10 @@ var versionsSingleton = keyedVersions{
Key: PostTruncatedAndRangeAppliedStateMigration,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 24},
},
{
Key: SeparatedIntents,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 26},
},
{
Key: TracingVerbosityIndependentSemantics,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 28},
},
{
Key: PriorReadSummaries,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 44},
},
{
Key: NonVotingReplicas,
Version: roachpb.Version{Major: 20, Minor: 2, Internal: 46},
},
{
// V21_1 is CockroachDB v21.1. It's used for all v21.1.x patch releases.
Key: V21_1,
Expand Down
Loading

0 comments on commit 2a33514

Please sign in to comment.