Skip to content

Commit

Permalink
partitionccl,sql: add license checks
Browse files Browse the repository at this point in the history
Move partition-related code behind a license check. Creating partitioned
tables and indices, repartitioning tables and indices, and installing or
modifying zone configs on indexes or partitions now requires a valid
enterprise license. Removing a partitioning scheme or a zone config on
an index or partition does not require a valid license, but it does
require a CCL binary.

Release note: None
  • Loading branch information
benesch committed Feb 13, 2018
1 parent 853fe66 commit 83dcf43
Show file tree
Hide file tree
Showing 11 changed files with 282 additions and 12 deletions.
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 @@ -1296,3 +1298,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
9 changes: 9 additions & 0 deletions pkg/ccl/utilccl/license_check.go
Original file line number Diff line number Diff line change
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 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
143 changes: 143 additions & 0 deletions pkg/sql/partition_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,143 @@
// 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 sql_test

import (
"context"
"testing"

"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"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/protoutil"
)

func TestRemovePartitioningOSS(t *testing.T) {
defer leaktest.AfterTest(t)()

ctx := context.Background()
params, _ := tests.CreateTestServerParams()
s, sqlDBRaw, kvDB := serverutils.StartServer(t, params)
sqlDB := sqlutils.MakeSQLRunner(sqlDBRaw)
defer s.Stopper().Stop(ctx)

const numRows = 100
if err := tests.CreateKVTable(sqlDB.DB, numRows); err != nil {
t.Fatal(err)
}
tableDesc := sqlbase.GetTableDescriptor(kvDB, "t", "kv")
tableKey := sqlbase.MakeDescMetadataKey(tableDesc.ID)

// Hack in partitions. Doing this properly requires a CCL binary.
tableDesc.PrimaryIndex.Partitioning = sqlbase.PartitioningDescriptor{
NumColumns: 1,
Range: []sqlbase.PartitioningDescriptor_Range{{
Name: "p1",
FromInclusive: encoding.EncodeIntValue(nil /* appendTo */, encoding.NoColumnID, 1),
ToExclusive: encoding.EncodeIntValue(nil /* appendTo */, encoding.NoColumnID, 2),
}},
}
tableDesc.Indexes[0].Partitioning = sqlbase.PartitioningDescriptor{
NumColumns: 1,
Range: []sqlbase.PartitioningDescriptor_Range{{
Name: "p2",
FromInclusive: encoding.EncodeIntValue(nil /* appendTo */, encoding.NoColumnID, 1),
ToExclusive: encoding.EncodeIntValue(nil /* appendTo */, encoding.NoColumnID, 2),
}},
}
if err := kvDB.Put(ctx, tableKey, sqlbase.WrapDescriptor(tableDesc)); err != nil {
t.Fatal(err)
}
exp := `CREATE TABLE kv (
k INT NOT NULL,
v INT NULL,
CONSTRAINT "primary" PRIMARY KEY (k ASC),
INDEX foo (v ASC) PARTITION BY RANGE (v) (
PARTITION p2 VALUES FROM (1) TO (2)
),
FAMILY fam_0_k (k),
FAMILY fam_1_v (v)
) PARTITION BY RANGE (k) (
PARTITION p1 VALUES FROM (1) TO (2)
)`
if a := sqlDB.QueryStr(t, "SHOW CREATE TABLE t.kv")[0][1]; exp != a {
t.Fatalf("expected:\n%s\n\ngot:\n%s\n\n", exp, a)
}

// Hack in partition zone configs. This also requires a CCL binary to do
// properly.
zoneConfig := config.ZoneConfig{
Subzones: []config.Subzone{
{
IndexID: uint32(tableDesc.PrimaryIndex.ID),
PartitionName: "p1",
Config: config.DefaultZoneConfig(),
},
{
IndexID: uint32(tableDesc.Indexes[0].ID),
PartitionName: "p2",
Config: config.DefaultZoneConfig(),
},
},
}
zoneConfigBytes, err := protoutil.Marshal(&zoneConfig)
if err != nil {
t.Fatal(err)
}
sqlDB.Exec(t, `INSERT INTO system.zones VALUES ($1, $2)`, tableDesc.ID, zoneConfigBytes)
for _, p := range []string{"p1", "p2"} {
if exists := sqlutils.ZoneConfigExists(t, sqlDB, "t.kv."+p); !exists {
t.Fatalf("zone config for %s does not exist", p)
}
}

expectCCLRequired := func(q string) {
t.Helper()
const expErr = "requires a CCL binary"
if _, err := sqlDB.DB.Exec(q); !testutils.IsError(err, expErr) {
t.Fatalf("expected %q error, but got %+v", expErr, err)
}
}

// TODO(benesch): introduce a "STRIP CCL" command to make it possible to
// remove CCL features from a table using an OSS binary.
expectCCLRequired(`ALTER TABLE t.kv PARTITION BY NOTHING`)
expectCCLRequired(`ALTER INDEX t.kv@foo PARTITION BY NOTHING`)
expectCCLRequired(`ALTER PARTITION p1 OF TABLE t.kv EXPERIMENTAL CONFIGURE ZONE ''`)
expectCCLRequired(`ALTER PARTITION p2 OF TABLE t.kv EXPERIMENTAL CONFIGURE ZONE ''`)

// Odd exception: removing partitioning is, in fact, possible when there are
// no zone configs for the table's indices or partitions.
sqlDB.Exec(t, `DELETE FROM system.zones WHERE id = $1`, tableDesc.ID)
sqlDB.Exec(t, `ALTER TABLE t.kv PARTITION BY NOTHING`)
sqlDB.Exec(t, `ALTER INDEX t.kv@foo PARTITION BY NOTHING`)

exp = `CREATE TABLE kv (
k INT NOT NULL,
v INT NULL,
CONSTRAINT "primary" PRIMARY KEY (k ASC),
INDEX foo (v ASC),
FAMILY fam_0_k (k),
FAMILY fam_1_v (v)
)`
if a := sqlDB.QueryStr(t, "SHOW CREATE TABLE t.kv")[0][1]; exp != a {
t.Fatalf("expected:\n%s\n\ngot:\n%s\n\n", exp, a)
}
}
Loading

0 comments on commit 83dcf43

Please sign in to comment.