Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
62498: utilccl,kvccl: improve performance when checking enterprise features r=tbg a=erikgrinaker

**utilccl: cache license decoding**

Previously, the `utilccl` package would decode the license from the the
base64-encoded Protobuf representation in settings every time it was
needed, which was sufficient for its uses. However, recently there's
been a need to check whether enterprise features are enabled in hot
paths (e.g. with follower reads as seen in #62447), making the decoding
cost too great.

This patch adds `cluster.Settings.Cache` as a shared cache, and uses it
to cache decoded licenses with a private key type.

**utilccl,kvccl: add IsEnterpriseEnabled for faster license checks**

`utilccl.CheckEnterpriseEnabled()` is used to check whether a valid
enterprise license exists for a given feature. If no valid license is
found, it returns an error with specific details.

However, `kvccl` used this function in follower read hot paths, and
instantiating an error when follower reads are unavailable could have
significant overhead -- see e.g. #62447.

This patch adds `IsEnterpriseEnabled()`, which has the same behavior as
`CheckEnterpriseEnabled()` but returns a boolean instead. This is
significantly faster since we can avoid instantiating a custom error
each time. `kvccl` is also updated to use this in hot paths.

Resolves #62489.

Release note: None

62642: colserde: fix the edge case with nulls handling r=yuzefovich a=yuzefovich

When serializing the data of Bool, Bytes, Int, and Float types when they
don't have any nulls in the vector, we don't explicit specify the null
bitmap. Previously, when deserializing such vectors with no nulls we
would simply call `UnsetNulls` on the `coldata.Nulls` object that is
currently present. However, it is possible that already present nulls
object cannot support the desired batch length. This could lead to index
out of bounds accesses. Note that in the vast majority of cases this
likely doesn't happen in practice because we check `MaybeHasNulls`, and
that would return `false` making us omit the null checking code.

Fixes: #62636.

Release note (bug fix): Previously, CockroachDB could encounter an
internal error in rare circumstances when executing queries via the
vectorized engine that operate on columns of BOOL, BYTES, INT, and FLOAT
types that have a mix of NULL and non-NULL values.

62740: workload: add idle-conns flag for adding idle connections to tpcc r=rafiss a=RichardJCai

workload: add idle-conns flag for adding idle connections to tpcc

Release note: None

#62526

62814: tenantrate: add "test" that reports IOPS estimations r=RaduBerinde a=RaduBerinde

This change adds a "test" facility which takes the description of a
uniform workload (read percentage, read size, write size) and prints
out an estimation of the sustained IOPS and burst IO. This will allow
a better understanding of how changes to the settings or the mechanism
translate into IOPS changes.

Release note: None

62833: kvserver: deflake TestFollowerReadsWithStaleDescriptor r=aayushshah15 a=aayushshah15

A preceding change (#62696) introduced a flakey update to this test.
Prior to that change, this test was using 2 voting replicas but that
change tried to make it use 1 voter and 1 non-voter instead (as a litmus
test for the new syntax added in #62696).

The test rebalances a replica away from a node and ensures that a
historical read sent immediately afterwards gets re-routed to the
leaseholder replica, since the receiving store had its replica
destroyed. However, when we're using a non-voter in this test, that
non-voter may not have learned about this replication change by the time
it receives this historical query and that fails the assertion.

This commit re-organizes the test and fixes the flake.

Release note: None

62862: testutils: add skip.UnderBazelWithIssue r=rickystewart a=stevendanna

This is to skip individual tests under bazel. This seems a bit more
fine-grained than the broken_in_bazel tag in the bazel configuration
but also allows us to avoid skipping tests that work outside of bazel
in our main test suite.

Release note: None

62877: Added CACHE to SEQUENCE syntax diagrams r=ericharmeling a=ericharmeling

Follow-up of #56954.

Release justification: non-production code changes

Release note: None

62889: colexecerror: catch panics from packages in sql/sem folder r=yuzefovich a=yuzefovich

Previously, we would only catch panics from `sql/sem/tree` package.
Recently sqlsmith encountered a crash because of a panic in
`sql/sem/builtins` package, and I believe it is reasonable to catch
panics from that package as well as from `sql/sem/transform`, so we will
now be catching based on `sql/sem` prefix.

Addresses: #62846.

Release note: None

62898: build: install essential build tools in teamcity build agents r=jlinder a=rickystewart

In #62815, we migrated from an alternative way of installing golang, the
`longsleep/golang-backports` deb repo, to the currently recommended
install method found at https://golang.org/doc/install -- namely, we
download a tarball and then just unzip it in the right spot. This
works perfectly, *except* that the deb package had a dependency on build
tools like `gcc` and `make`, and certain build configurations had come
to depend on their global installation (namely, all those that don't use
`builder.sh` to run a build). This resulted in a couple of failures
being reported:

* https://teamcity.cockroachdb.com/buildConfiguration/Cockroach_ExampleORMs/2834741
* https://teamcity.cockroachdb.com/buildConfiguration/Cockroach_UnitTests_Acceptance/2834732

We just install [`build-essential`](https://packages.ubuntu.com/xenial/build-essential)
here, which is the easiest way to get all of that stuff.

Release note: None

Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
Co-authored-by: richardjcai <caioftherichard@gmail.com>
Co-authored-by: Radu Berinde <radu@cockroachlabs.com>
Co-authored-by: Aayush Shah <aayush.shah15@gmail.com>
Co-authored-by: Steven Danna <danna@cockroachlabs.com>
Co-authored-by: Eric Harmeling <eric.harmeling@cockroachlabs.com>
Co-authored-by: Ricky Stewart <ricky@cockroachlabs.com>
  • Loading branch information
9 people committed Mar 31, 2021
10 parents 17dd168 + a7c1f37 + 60f03a7 + fbb2898 + e0efca8 + 04c09fe + 4648747 + db1929f + 830b022 + bb3d37c commit 36f99c3
Show file tree
Hide file tree
Showing 29 changed files with 440 additions and 117 deletions.
1 change: 1 addition & 0 deletions build/packer/teamcity-agent.sh
Expand Up @@ -31,6 +31,7 @@ apt-get update --yes
apt-get install --yes sudo

apt-get install --yes \
build-essential \
curl \
docker-ce \
docker-compose \
Expand Down
4 changes: 2 additions & 2 deletions docs/generated/sql/bnf/alter_sequence_options_stmt.bnf
@@ -1,3 +1,3 @@
alter_sequence_options_stmt ::=
'ALTER' 'SEQUENCE' sequence_name ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) )* )
| 'ALTER' 'SEQUENCE' 'IF' 'EXISTS' sequence_name ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) )* )
'ALTER' 'SEQUENCE' sequence_name ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'CACHE' integer | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'CACHE' integer | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) )* )
| 'ALTER' 'SEQUENCE' 'IF' 'EXISTS' sequence_name ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'CACHE' integer | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'CACHE' integer | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) )* )
4 changes: 2 additions & 2 deletions docs/generated/sql/bnf/create_sequence_stmt.bnf
@@ -1,3 +1,3 @@
create_sequence_stmt ::=
'CREATE' opt_temp 'SEQUENCE' sequence_name ( ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) )* ) | )
| 'CREATE' opt_temp 'SEQUENCE' 'IF' 'NOT' 'EXISTS' sequence_name ( ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) )* ) | )
'CREATE' opt_temp 'SEQUENCE' sequence_name ( ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'CACHE' integer | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'CACHE' integer | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) )* ) | )
| 'CREATE' opt_temp 'SEQUENCE' 'IF' 'NOT' 'EXISTS' sequence_name ( ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'CACHE' integer | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) ( ( ( 'NO' 'CYCLE' | 'OWNED' 'BY' 'NONE' | 'OWNED' 'BY' column_name | 'CACHE' integer | 'INCREMENT' integer | 'INCREMENT' 'BY' integer | 'MINVALUE' integer | 'NO' 'MINVALUE' | 'MAXVALUE' integer | 'NO' 'MAXVALUE' | 'START' integer | 'START' 'WITH' integer | 'VIRTUAL' ) ) )* ) | )
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Expand Up @@ -2560,6 +2560,7 @@ sequence_option_elem ::=
'NO' 'CYCLE'
| 'OWNED' 'BY' 'NONE'
| 'OWNED' 'BY' column_path
| 'CACHE' signed_iconst64
| 'INCREMENT' signed_iconst64
| 'INCREMENT' 'BY' signed_iconst64
| 'MINVALUE' signed_iconst64
Expand Down
12 changes: 11 additions & 1 deletion pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go
Expand Up @@ -77,16 +77,26 @@ func getGlobalReadsLead(clock *hlc.Clock) time.Duration {
return clock.MaxOffset()
}

// checkEnterpriseEnabled checks whether the enterprise feature for follower
// reads is enabled, returning a detailed error if not. It is not suitable for
// use in hot paths since a new error may be instantiated on each call.
func checkEnterpriseEnabled(clusterID uuid.UUID, st *cluster.Settings) error {
org := sql.ClusterOrganization.Get(&st.SV)
return utilccl.CheckEnterpriseEnabled(st, clusterID, org, "follower reads")
}

// isEnterpriseEnabled is faster than checkEnterpriseEnabled, and suitable
// for hot paths.
func isEnterpriseEnabled(clusterID uuid.UUID, st *cluster.Settings) bool {
org := sql.ClusterOrganization.Get(&st.SV)
return utilccl.IsEnterpriseEnabled(st, clusterID, org, "follower reads")
}

func checkFollowerReadsEnabled(clusterID uuid.UUID, st *cluster.Settings) bool {
if !kvserver.FollowerReadsEnabled.Get(&st.SV) {
return false
}
return checkEnterpriseEnabled(clusterID, st) == nil
return isEnterpriseEnabled(clusterID, st)
}

func evalFollowerReadOffset(clusterID uuid.UUID, st *cluster.Settings) (time.Duration, error) {
Expand Down
34 changes: 26 additions & 8 deletions pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go
Expand Up @@ -540,8 +540,7 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) {
n1 := sqlutils.MakeSQLRunner(tc.Conns[0])
n1.Exec(t, `CREATE DATABASE t`)
n1.Exec(t, `CREATE TABLE test (k INT PRIMARY KEY)`)
n1.Exec(t, `ALTER TABLE test EXPERIMENTAL_RELOCATE VOTERS VALUES (ARRAY[1], 1)`)
n1.Exec(t, `ALTER TABLE test EXPERIMENTAL_RELOCATE NON_VOTERS VALUES (ARRAY[2], 1)`)
n1.Exec(t, `ALTER TABLE test EXPERIMENTAL_RELOCATE VOTERS VALUES (ARRAY[1,2], 1)`)
// Speed up closing of timestamps, as we'll in order to be able to use
// follower_read_timestamp().
// Every 0.2s we'll close the timestamp from 0.4s ago. We'll attempt follower reads
Expand Down Expand Up @@ -571,12 +570,31 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) {
require.Equal(t, roachpb.StoreID(1), entry.Lease().Replica.StoreID)
require.Equal(t, []roachpb.ReplicaDescriptor{
{NodeID: 1, StoreID: 1, ReplicaID: 1},
{NodeID: 2, StoreID: 2, ReplicaID: 2, Type: roachpb.ReplicaTypeNonVoter()},
{NodeID: 2, StoreID: 2, ReplicaID: 2},
}, entry.Desc().Replicas().Descriptors())

// Relocate the follower. n2 will no longer have a replica.
n1.Exec(t, `ALTER TABLE test EXPERIMENTAL_RELOCATE VOTERS VALUES (ARRAY[1,3], 1)`)
n1.Exec(t, `ALTER TABLE test EXPERIMENTAL_RELOCATE NON_VOTERS VALUES (ARRAY[], 1)`)
// Remove the follower and add a new non-voter to n3. n2 will no longer have a
// replica.
n1.Exec(t, `ALTER TABLE test EXPERIMENTAL_RELOCATE VOTERS VALUES (ARRAY[1], 1)`)
n1.Exec(t, `ALTER TABLE test EXPERIMENTAL_RELOCATE NON_VOTERS VALUES (ARRAY[3], 1)`)

// Wait until the new non-voter is upreplicated to n3.
testutils.SucceedsSoon(
t, func() error {
return tc.Server(2).GetStores().(*kvserver.Stores).VisitStores(
func(s *kvserver.Store) error {
repl := s.LookupReplica(tablePrefix)
if repl == nil {
return errors.Errorf("no replica found on store %s", s)
}
if !repl.IsInitialized() {
return errors.Errorf("non-voter not initialized")
}
return nil
},
)
},
)

// Execute the query again and assert the cache is updated. This query will
// not be executed as a follower read since it attempts to use n2 which
Expand All @@ -585,15 +603,15 @@ func TestFollowerReadsWithStaleDescriptor(t *testing.T) {
n4.Exec(t, historicalQuery)
// As a sanity check, verify that this was not a follower read.
rec := <-recCh
require.False(t, kv.OnlyFollowerReads(rec), "query was not served through follower reads: %s", rec)
require.False(t, kv.OnlyFollowerReads(rec), "query was served through follower reads: %s", rec)
// Check that the cache was properly updated.
entry = n4Cache.GetCached(ctx, tablePrefix, false /* inverted */)
require.NotNil(t, entry)
require.False(t, entry.Lease().Empty())
require.Equal(t, roachpb.StoreID(1), entry.Lease().Replica.StoreID)
require.Equal(t, []roachpb.ReplicaDescriptor{
{NodeID: 1, StoreID: 1, ReplicaID: 1},
{NodeID: 3, StoreID: 3, ReplicaID: 3},
{NodeID: 3, StoreID: 3, ReplicaID: 3, Type: roachpb.ReplicaTypeNonVoter()},
}, entry.Desc().Replicas().Descriptors())

// Make a note of the follower reads metric on n3. We'll check that it was
Expand Down
107 changes: 75 additions & 32 deletions pkg/ccl/utilccl/license_check.go
Expand Up @@ -57,6 +57,15 @@ const (
testingEnterpriseEnabled = 1
)

// errEnterpriseRequired is returned by check() when the caller does
// not request detailed errors.
var errEnterpriseRequired = pgerror.New(pgcode.CCLValidLicenseRequired,
"a valid enterprise license is required")

// licenseCacheKey is used to cache licenses in cluster.Settings.Cache,
// keeping the entries private.
type licenseCacheKey string

// TestingEnableEnterprise allows overriding the license check in tests.
func TestingEnableEnterprise() func() {
before := atomic.LoadInt32(&testingEnterprise)
Expand All @@ -78,11 +87,21 @@ func TestingDisableEnterprise() func() {
// CheckEnterpriseEnabled returns a non-nil error if the requested enterprise
// feature is not enabled, including information or a link explaining how to
// enable it.
//
// This should not be used in hot paths, since an unavailable feature will
// result in a new error being instantiated for every call -- use
// IsEnterpriseEnabled() instead.
func CheckEnterpriseEnabled(st *cluster.Settings, cluster uuid.UUID, org, feature string) error {
if atomic.LoadInt32(&testingEnterprise) == testingEnterpriseEnabled {
return nil
}
return checkEnterpriseEnabledAt(st, timeutil.Now(), cluster, org, feature)
return checkEnterpriseEnabledAt(st, timeutil.Now(), cluster, org, feature, true /* withDetails */)
}

// IsEnterpriseEnabled returns whether the requested enterprise feature is
// enabled. It is faster than CheckEnterpriseEnabled, since it does not return
// details about why the feature is unavailable, and can therefore be used in
// hot paths.
func IsEnterpriseEnabled(st *cluster.Settings, cluster uuid.UUID, org, feature string) bool {
return checkEnterpriseEnabledAt(
st, timeutil.Now(), cluster, org, feature, false /* withDetails */) == nil
}

func init() {
Expand All @@ -97,47 +116,56 @@ func init() {
func TimeToEnterpriseLicenseExpiry(
ctx context.Context, st *cluster.Settings, asOf time.Time,
) (time.Duration, error) {
var lic *licenseccl.License
// FIXME(tschottdorf): see whether it makes sense to cache the decoded
// license.
if str := enterpriseLicense.Get(&st.SV); str != "" {
var err error
if lic, err = decode(str); err != nil {
return 0, err
}
} else {
return 0, nil
license, err := getLicense(st)
if err != nil || license == nil {
return 0, err
}

expiration := timeutil.Unix(lic.ValidUntilUnixSec, 0)
expiration := timeutil.Unix(license.ValidUntilUnixSec, 0)
return expiration.Sub(asOf), nil
}

func checkEnterpriseEnabledAt(
st *cluster.Settings, at time.Time, cluster uuid.UUID, org, feature string,
st *cluster.Settings, at time.Time, cluster uuid.UUID, org, feature string, withDetails bool,
) error {
var lic *licenseccl.License
// FIXME(tschottdorf): see whether it makes sense to cache the decoded
// license.
if str := enterpriseLicense.Get(&st.SV); str != "" {
var err error
if lic, err = decode(str); err != nil {
return err
}
if atomic.LoadInt32(&testingEnterprise) == testingEnterpriseEnabled {
return nil
}
license, err := getLicense(st)
if err != nil {
return err
}
return check(lic, at, cluster, org, feature)
return check(license, at, cluster, org, feature, withDetails)
}

func getLicenseType(st *cluster.Settings) (string, error) {
// getLicense fetches the license from the given settings, using Settings.Cache
// to cache the decoded license (if any). The returned license must not be
// modified by the caller.
func getLicense(st *cluster.Settings) (*licenseccl.License, error) {
str := enterpriseLicense.Get(&st.SV)
if str == "" {
return "None", nil
return nil, nil
}
lic, err := decode(str)
cacheKey := licenseCacheKey(str)
if cachedLicense, ok := st.Cache.Load(cacheKey); ok {
return cachedLicense.(*licenseccl.License), nil
}
license, err := decode(str)
if err != nil {
return nil, err
}
st.Cache.Store(cacheKey, license)
return license, nil
}

func getLicenseType(st *cluster.Settings) (string, error) {
license, err := getLicense(st)
if err != nil {
return "", err
} else if license == nil {
return "None", nil
}
return lic.Type.String(), nil
return license.Type.String(), nil
}

// decode attempts to read a base64 encoded License.
Expand All @@ -146,12 +174,18 @@ func decode(s string) (*licenseccl.License, error) {
if err != nil {
return nil, pgerror.WithCandidateCode(err, pgcode.Syntax)
}
return lic, err
return lic, nil
}

// check returns an error if the license is empty or not currently valid.
func check(l *licenseccl.License, at time.Time, cluster uuid.UUID, org, feature string) error {
// check returns an error if the license is empty or not currently valid. If
// withDetails is false, a generic error message is returned for performance.
func check(
l *licenseccl.License, at time.Time, cluster uuid.UUID, org, feature string, withDetails bool,
) error {
if l == nil {
if !withDetails {
return errEnterpriseRequired
}
// TODO(dt): link to some stable URL that then redirects to a helpful page
// that explains what to do here.
link := "https://cockroachlabs.com/pricing?cluster="
Expand All @@ -168,6 +202,9 @@ func check(l *licenseccl.License, at time.Time, cluster uuid.UUID, org, feature
// suddenly throwing errors at them.
if l.ValidUntilUnixSec > 0 && l.Type != licenseccl.License_Enterprise {
if expiration := timeutil.Unix(l.ValidUntilUnixSec, 0); at.After(expiration) {
if !withDetails {
return errEnterpriseRequired
}
licensePrefix := redact.SafeString("")
switch l.Type {
case licenseccl.License_NonCommercial:
Expand All @@ -190,6 +227,9 @@ func check(l *licenseccl.License, at time.Time, cluster uuid.UUID, org, feature
if strings.EqualFold(l.OrganizationName, org) {
return nil
}
if !withDetails {
return errEnterpriseRequired
}
return pgerror.Newf(pgcode.CCLValidLicenseRequired,
"license valid only for %q", l.OrganizationName)
}
Expand All @@ -201,6 +241,9 @@ func check(l *licenseccl.License, at time.Time, cluster uuid.UUID, org, feature
}

// no match, so compose an error message.
if !withDetails {
return errEnterpriseRequired
}
var matches bytes.Buffer
for i, c := range l.ClusterID {
if i > 0 {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/utilccl/license_check_test.go
Expand Up @@ -63,7 +63,7 @@ func TestSettingAndCheckingLicense(t *testing.T) {
if err := updater.Set("enterprise.license", tc.lic, "s"); err != nil {
t.Fatal(err)
}
err := checkEnterpriseEnabledAt(st, tc.checkTime, tc.checkCluster, "", "")
err := checkEnterpriseEnabledAt(st, tc.checkTime, tc.checkCluster, "", "", true)
if !testutils.IsError(err, tc.err) {
l, _ := decode(tc.lic)
t.Fatalf("%d: lic %v, update by %T, checked by %s at %s, got %q", i, l, updater, tc.checkCluster, tc.checkTime, err)
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/utilccl/license_test.go
Expand Up @@ -83,7 +83,7 @@ func TestLicense(t *testing.T) {
}
}
if err := check(
lic, tc.checkTime, tc.checkCluster, tc.checkOrg, "",
lic, tc.checkTime, tc.checkCluster, tc.checkOrg, "", true,
); !testutils.IsError(err, tc.err) {
t.Fatalf("%d: lic for %s to %s, checked by %s at %s.\n got %q", i,
tc.grantedTo, tc.expiration, tc.checkCluster, tc.checkTime, err)
Expand All @@ -108,7 +108,7 @@ func TestExpiredLicenseLanguage(t *testing.T) {
Type: licenseccl.License_Evaluation,
ValidUntilUnixSec: 1,
}
err := check(lic, timeutil.Now(), uuid.MakeV4(), "", "RESTORE")
err := check(lic, timeutil.Now(), uuid.MakeV4(), "", "RESTORE", true)
expected := "Use of RESTORE requires an enterprise license. Your evaluation license expired on " +
"January 1, 1970. If you're interested in getting a new license, please contact " +
"subscriptions@cockroachlabs.com and we can help you out."
Expand Down
22 changes: 19 additions & 3 deletions pkg/cmd/skip-test/main.go
Expand Up @@ -45,6 +45,11 @@ var flagUnderRace = flag.Bool(
false,
"if true, only skip under race",
)
var flagUnderBazel = flag.Bool(
"under_bazel",
false,
"if true, only skip under bazel",
)

func main() {
flag.Parse()
Expand Down Expand Up @@ -81,6 +86,10 @@ func main() {
log.Fatalf("expected test to be of format `TestName` or `pkg/to/test:TestName`, found %s", arg)
}

if *flagUnderBazel && *flagUnderRace {
log.Fatal("cannot use both -under_race and -under_bazel")
}

// Check git status is clean.
if err := spawn("git", "diff", "--exit-code"); err != nil {
log.Fatal(errors.Wrap(err, "git state may not be clean, please use `git stash` or commit changes before proceeding."))
Expand Down Expand Up @@ -131,9 +140,11 @@ func main() {
if err := spawn("git", "add", fileName); err != nil {
log.Fatal(errors.Wrapf(err, "failed to add %s to commit", fileName))
}
var underRaceStr string
var modifierStr string
if *flagUnderRace {
underRaceStr = " under race"
modifierStr = " under race"
} else if *flagUnderBazel {
modifierStr = " under bazel"
}
commitMsg := fmt.Sprintf(`%s: skip %s%s
Expand All @@ -146,7 +157,7 @@ Generated by bin/skip-test.
Release justification: non-production code changes
Release note: None
`, pkgPrefix, testName, underRaceStr, issueNum, *flagReason)
`, pkgPrefix, testName, modifierStr, issueNum, *flagReason)
if err := spawn("git", "commit", "-m", commitMsg); err != nil {
log.Fatal(errors.Wrapf(err, "failed to commit %s", fileName))
}
Expand Down Expand Up @@ -208,6 +219,11 @@ func replaceFile(fileName, testName string, issueNum int) {
newLines,
fmt.Sprintf(`skip.UnderRaceWithIssue(t, %d, "%s")`, issueNum, *flagReason),
)
} else if *flagUnderBazel {
newLines = append(
newLines,
fmt.Sprintf(`skip.UnderBazelWithIssue(t, %d, "%s")`, issueNum, *flagReason),
)
} else {
newLines = append(
newLines,
Expand Down
6 changes: 6 additions & 0 deletions pkg/col/coldata/nulls.go
Expand Up @@ -311,6 +311,12 @@ func (n *Nulls) Slice(start int, end int) Nulls {
return s
}

// MaxNumElements returns the maximum number of elements that this Nulls can
// accommodate.
func (n *Nulls) MaxNumElements() int {
return len(n.nulls) * 8
}

// NullBitmap returns the null bitmap.
func (n *Nulls) NullBitmap() []byte {
return n.nulls
Expand Down

0 comments on commit 36f99c3

Please sign in to comment.