Skip to content

Commit

Permalink
Merge pull request #22641 from benesch/license-check
Browse files Browse the repository at this point in the history
partitionccl,sql: add license checks
  • Loading branch information
benesch authored Feb 13, 2018
2 parents 0cddbfd + 83dcf43 commit c02d506
Show file tree
Hide file tree
Showing 16 changed files with 330 additions and 43 deletions.
39 changes: 39 additions & 0 deletions pkg/base/license.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.

package base

import (
"errors"

"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)

// 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 function is overridden by an init hook in CCL builds.
var CheckEnterpriseEnabled = func(_ *cluster.Settings, _ uuid.UUID, org, feature string) error {
return errors.New("OSS binaries do not include enterprise features")
}

// LicenseType returns what type of license the cluster is running with, or
// "OSS" if it is an OSS build.
//
// This function is overridden by an init hook in CCL builds.
var LicenseType = func(st *cluster.Settings) (string, error) {
return "OSS", nil
}
2 changes: 2 additions & 0 deletions pkg/ccl/cliccl/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,11 +12,13 @@ import (
"os"
"testing"

"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
)

func TestMain(m *testing.M) {
defer utilccl.TestingEnableEnterprise()()
serverutils.InitTestServerFactory(server.TestServerFactory)
os.Exit(m.Run())
}
Expand Down
12 changes: 9 additions & 3 deletions pkg/ccl/partitionccl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (
"fmt"
"strings"

"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
Expand Down Expand Up @@ -201,9 +202,9 @@ func createPartitioningImpl(
return partDesc, nil
}

// CreatePartitioning constructs the partitioning descriptor for an index that
// createPartitioning constructs the partitioning descriptor for an index that
// is partitioned into ranges, each addressable by zone configs.
func CreatePartitioning(
func createPartitioning(
ctx context.Context,
st *cluster.Settings,
evalCtx *tree.EvalContext,
Expand All @@ -216,6 +217,11 @@ func CreatePartitioning(
errors.New("cluster version does not support partitioning")
}

org := sql.ClusterOrganization.Get(&st.SV)
if err := utilccl.CheckEnterpriseEnabled(st, evalCtx.ClusterID, org, "partitions"); err != nil {
return sqlbase.PartitioningDescriptor{}, err
}

return createPartitioningImpl(
ctx, evalCtx, tableDesc, indexDesc, partBy, 0 /* colOffset */)
}
Expand Down Expand Up @@ -422,5 +428,5 @@ func selectPartitionExprsByName(
}

func init() {
sql.CreatePartitioning = CreatePartitioning
sql.CreatePartitioningCCL = createPartitioning
}
58 changes: 58 additions & 0 deletions pkg/ccl/partitionccl/partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/ccl/importccl"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand All @@ -28,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/pkg/errors"
Expand Down Expand Up @@ -1319,3 +1321,59 @@ func TestRepartitioning(t *testing.T) {
})
}
}

func TestRemovePartitioningExpiredLicense(t *testing.T) {
defer leaktest.AfterTest(t)()
defer utilccl.TestingEnableEnterprise()()

ctx := context.Background()
s, sqlDBRaw, _ := serverutils.StartServer(t, base.TestServerArgs{
UseDatabase: "d",
})
defer s.Stopper().Stop(ctx)

// Create a partitioned table and index.
sqlDB := sqlutils.MakeSQLRunner(sqlDBRaw)
sqlDB.Exec(t, `CREATE DATABASE d`)
sqlDB.Exec(t, `CREATE TABLE t (a INT PRIMARY KEY) PARTITION BY LIST (a) (
PARTITION p1 VALUES IN (1)
)`)
sqlDB.Exec(t, `CREATE INDEX i ON t (a) PARTITION BY RANGE (a) (
PARTITION p34 VALUES FROM (3) TO (4)
)`)
sqlDB.Exec(t, `ALTER PARTITION p1 OF TABLE t EXPERIMENTAL CONFIGURE ZONE ''`)
sqlDB.Exec(t, `ALTER PARTITION p34 OF TABLE t EXPERIMENTAL CONFIGURE ZONE ''`)
sqlDB.Exec(t, `ALTER INDEX t@primary EXPERIMENTAL CONFIGURE ZONE ''`)
sqlDB.Exec(t, `ALTER INDEX t@i EXPERIMENTAL CONFIGURE ZONE ''`)

// Remove the enterprise license.
defer utilccl.TestingDisableEnterprise()()

expectLicenseErr := func(q string) {
t.Helper()
const expErr = "use of partitions requires an enterprise license"
if _, err := sqlDB.DB.Exec(q); !testutils.IsError(err, expErr) {
t.Fatalf("expected error %q, but got %+v", expErr, err)
}
}

// Partitions and zone configs cannot be modified without a valid license.
expectLicenseErr(`ALTER TABLE t PARTITION BY LIST (a) (PARTITION p2 VALUES IN (2))`)
expectLicenseErr(`ALTER INDEX t@i PARTITION BY RANGE (a) (PARTITION p45 VALUES FROM (4) TO (5))`)
expectLicenseErr(`ALTER PARTITION p1 OF TABLE t EXPERIMENTAL CONFIGURE ZONE ''`)
expectLicenseErr(`ALTER PARTITION p34 OF TABLE t EXPERIMENTAL CONFIGURE ZONE ''`)
expectLicenseErr(`ALTER INDEX t@primary EXPERIMENTAL CONFIGURE ZONE ''`)
expectLicenseErr(`ALTER INDEX t@i EXPERIMENTAL CONFIGURE ZONE ''`)

// But they can be removed.
sqlDB.Exec(t, `ALTER TABLE t PARTITION BY NOTHING`)
sqlDB.Exec(t, `ALTER INDEX t@i PARTITION BY NOTHING`)
sqlDB.Exec(t, `ALTER INDEX t@primary EXPERIMENTAL CONFIGURE ZONE NULL`)
sqlDB.Exec(t, `ALTER INDEX t@i EXPERIMENTAL CONFIGURE ZONE NULL`)

// Once removed, they cannot be added back.
expectLicenseErr(`ALTER TABLE t PARTITION BY LIST (a) (PARTITION p2 VALUES IN (2))`)
expectLicenseErr(`ALTER INDEX t@i PARTITION BY RANGE (a) (PARTITION p45 VALUES FROM (4) TO (5))`)
expectLicenseErr(`ALTER INDEX t@primary EXPERIMENTAL CONFIGURE ZONE ''`)
expectLicenseErr(`ALTER INDEX t@i EXPERIMENTAL CONFIGURE ZONE ''`)
}
20 changes: 19 additions & 1 deletion pkg/ccl/partitionccl/zone.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,13 +11,16 @@ package partitionccl
import (
"bytes"

"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl/intervalccl"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)

// GenerateSubzoneSpans constructs from a TableDescriptor the entries mapping
Expand Down Expand Up @@ -58,9 +61,24 @@ import (
// slice. As space optimizations, all `Key`s and `EndKey`s of `SubzoneSpan` omit
// the common prefix (the encoded table ID) and if `EndKey` is equal to
// `Key.PrefixEnd()` it is omitted.
//
// TODO(benesch): remove the hasNewSubzones parameter when a statement to clear
// all subzones at once is introduced.
func GenerateSubzoneSpans(
tableDesc *sqlbase.TableDescriptor, subzones []config.Subzone,
st *cluster.Settings,
clusterID uuid.UUID,
tableDesc *sqlbase.TableDescriptor,
subzones []config.Subzone,
hasNewSubzones bool,
) ([]config.SubzoneSpan, error) {
// Removing zone configs does not require a valid license.
if hasNewSubzones {
org := sql.ClusterOrganization.Get(&st.SV)
if err := utilccl.CheckEnterpriseEnabled(st, clusterID, org, "partitions"); err != nil {
return nil, err
}
}

a := &sqlbase.DatumAlloc{}

subzoneIndexByIndexID := make(map[sqlbase.IndexID]int32)
Expand Down
7 changes: 6 additions & 1 deletion pkg/ccl/partitionccl/zone_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,15 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)

func TestValidIndexPartitionSetShowZones(t *testing.T) {
Expand Down Expand Up @@ -262,7 +264,10 @@ func TestGenerateSubzoneSpans(t *testing.T) {
if err := test.parse(); err != nil {
t.Fatalf("%+v", err)
}
spans, err := GenerateSubzoneSpans(test.parsed.tableDesc, test.parsed.subzones)
clusterID := uuid.MakeV4()
hasNewSubzones := false
spans, err := GenerateSubzoneSpans(
cluster.NoSettings, clusterID, test.parsed.tableDesc, test.parsed.subzones, hasNewSubzones)
if err != nil {
t.Fatalf("generating subzone spans: %+v", err)
}
Expand Down
15 changes: 12 additions & 3 deletions pkg/ccl/utilccl/license_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,8 @@ package utilccl
import (
"time"

"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/util/timeutil"
Expand Down Expand Up @@ -44,6 +44,15 @@ func TestingEnableEnterprise() func() {
}
}

// TestingDisableEnterprise allows re-enabling the license check in tests.
func TestingDisableEnterprise() func() {
before := testingEnterpriseEnabled
testingEnterpriseEnabled = false
return func() {
testingEnterpriseEnabled = before
}
}

// 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 @@ -55,8 +64,8 @@ func CheckEnterpriseEnabled(st *cluster.Settings, cluster uuid.UUID, org, featur
}

func init() {
server.LicenseCheckFn = CheckEnterpriseEnabled
server.LicenseTypeFn = getLicenseType
base.CheckEnterpriseEnabled = CheckEnterpriseEnabled
base.LicenseType = getLicenseType
}

func checkEnterpriseEnabledAt(
Expand Down
7 changes: 1 addition & 6 deletions pkg/server/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -1015,13 +1015,8 @@ func (s *adminServer) Cluster(
// Check if enterprise features are enabled. We currently test for the
// feature "BACKUP", although enterprise licenses do not yet distinguish
// between different features.
enterpriseEnabled := false
organization := sql.ClusterOrganization.Get(&s.server.st.SV)
if err := LicenseCheckFn(
s.server.st, clusterID, organization, "BACKUP",
); err == nil {
enterpriseEnabled = true
}
enterpriseEnabled := base.CheckEnterpriseEnabled(s.server.st, clusterID, organization, "BACKUP") == nil

return &serverpb.ClusterResponse{
ClusterID: clusterID.String(),
Expand Down
8 changes: 3 additions & 5 deletions pkg/server/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1050,13 +1050,11 @@ func TestClusterAPI(t *testing.T) {
testutils.RunTrueAndFalse(t, "enterpriseOn", func(t *testing.T, enterpriseOn bool) {
// Override server license check.
if enterpriseOn {
oldLicenseCheck := LicenseCheckFn
LicenseCheckFn = func(_ *cluster.Settings, _ uuid.UUID, _, _ string) error {
old := base.CheckEnterpriseEnabled
base.CheckEnterpriseEnabled = func(_ *cluster.Settings, _ uuid.UUID, _, _ string) error {
return nil
}
defer func() {
LicenseCheckFn = oldLicenseCheck
}()
defer func() { base.CheckEnterpriseEnabled = old }()
}

if _, err := db.Exec(`SET CLUSTER SETTING diagnostics.reporting.enabled = $1`, reportingOn); err != nil {
Expand Down
16 changes: 0 additions & 16 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,22 +93,6 @@ var (
"the amount of time subsystems wait for work to finish before shutting down",
10*time.Second,
)

// LicenseCheckFn is used to check if the current cluster has any enterprise
// features enabled. This function is overridden by an init hook in CCL
// builds.
LicenseCheckFn = func(
st *cluster.Settings, cluster uuid.UUID, org, feature string,
) error {
return errors.New("OSS build does not include Enterprise features")
}

// LicenseTypeFn returns what type of license the cluster is running with, or
// "OSS" if it is an OSS build. This function is overridden by an init hook in
// CCL builds.
LicenseTypeFn = func(st *cluster.Settings) (string, error) {
return "OSS", nil
}
)

// Server is the cockroach server node.
Expand Down
3 changes: 2 additions & 1 deletion pkg/server/updates.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/mitchellh/reflectwalk"
"github.com/pkg/errors"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down Expand Up @@ -169,7 +170,7 @@ func addInfoToURL(ctx context.Context, url *url.URL, s *Server, runningTime time
q.Set("internal",
strconv.FormatBool(strings.Contains(sql.ClusterOrganization.Get(&s.st.SV), "Cockroach Labs")))

licenseType, err := LicenseTypeFn(s.st)
licenseType, err := base.LicenseType(s.st)
if err == nil {
q.Set("licensetype", licenseType)
} else {
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -234,7 +234,8 @@ func (sc *SchemaChanger) removeIndexZoneConfigs(
zone.DeleteIndexSubzones(uint32(indexDesc.ID))
}

_, err = writeZoneConfig(ctx, txn, sc.tableID, tableDesc, zone, sc.execCfg)
hasNewSubzones := false
_, err = writeZoneConfig(ctx, txn, sc.tableID, tableDesc, zone, sc.execCfg, hasNewSubzones)
if sqlbase.IsCCLRequiredError(err) {
return sqlbase.NewCCLRequiredError(fmt.Errorf("schema change requires a CCL binary "+
"because table %q has at least one remaining index or partition with a zone config",
Expand Down
19 changes: 18 additions & 1 deletion pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -777,7 +777,24 @@ func (p *planner) finalizeInterleave(

// CreatePartitioning constructs the partitioning descriptor for an index that
// is partitioned into ranges, each addressable by zone configs.
var CreatePartitioning = func(
func CreatePartitioning(
ctx context.Context,
st *cluster.Settings,
evalCtx *tree.EvalContext,
tableDesc *sqlbase.TableDescriptor,
indexDesc *sqlbase.IndexDescriptor,
partBy *tree.PartitionBy,
) (sqlbase.PartitioningDescriptor, error) {
if partBy == nil {
// No CCL necessary if we're looking at PARTITION BY NOTHING.
return sqlbase.PartitioningDescriptor{}, nil
}
return CreatePartitioningCCL(ctx, st, evalCtx, tableDesc, indexDesc, partBy)
}

// CreatePartitioningCCL is the public hook point for the CCL-licensed
// partitioning creation code.
var CreatePartitioningCCL = func(
ctx context.Context,
st *cluster.Settings,
evalCtx *tree.EvalContext,
Expand Down
Loading

0 comments on commit c02d506

Please sign in to comment.