From 853fe660bb68945ad4d15585c50e0eeecfe277a6 Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Mon, 12 Feb 2018 16:42:07 -0500 Subject: [PATCH 1/2] server: move license check functions to base These license check functions have no dependencies and might as well be accessible everywhere. Release note: None --- pkg/base/license.go | 39 ++++++++++++++++++++++++++++++++ pkg/ccl/utilccl/license_check.go | 6 ++--- pkg/server/admin.go | 7 +----- pkg/server/admin_test.go | 8 +++---- pkg/server/server.go | 16 ------------- pkg/server/updates.go | 3 ++- 6 files changed, 48 insertions(+), 31 deletions(-) create mode 100644 pkg/base/license.go diff --git a/pkg/base/license.go b/pkg/base/license.go new file mode 100644 index 000000000000..0c1ee5dad485 --- /dev/null +++ b/pkg/base/license.go @@ -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 +} diff --git a/pkg/ccl/utilccl/license_check.go b/pkg/ccl/utilccl/license_check.go index 9deb91ba7013..f6e480f14293 100644 --- a/pkg/ccl/utilccl/license_check.go +++ b/pkg/ccl/utilccl/license_check.go @@ -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" @@ -55,8 +55,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( diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 06a1df2ef145..7a5b3f32885c 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -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(), diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index 579537a1f6cd..0c3cf3d95927 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -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 { diff --git a/pkg/server/server.go b/pkg/server/server.go index fded9d4e9d4f..423ea99d2474 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -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. diff --git a/pkg/server/updates.go b/pkg/server/updates.go index ff5852dfc4f9..8eaa46bde687 100644 --- a/pkg/server/updates.go +++ b/pkg/server/updates.go @@ -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" @@ -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 { From 83dcf43cf77497a7622f7f3c7455f59dd44046a7 Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Tue, 13 Feb 2018 11:50:14 -0500 Subject: [PATCH 2/2] partitionccl,sql: add license checks 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 --- pkg/ccl/cliccl/main_test.go | 2 + pkg/ccl/partitionccl/partition.go | 12 ++- pkg/ccl/partitionccl/partition_test.go | 58 ++++++++++ pkg/ccl/partitionccl/zone.go | 20 +++- pkg/ccl/partitionccl/zone_test.go | 7 +- pkg/ccl/utilccl/license_check.go | 9 ++ pkg/sql/backfill.go | 3 +- pkg/sql/create_table.go | 19 +++- pkg/sql/partition_test.go | 143 +++++++++++++++++++++++++ pkg/sql/set_zone_config.go | 10 +- pkg/sql/zone_config.go | 11 +- 11 files changed, 282 insertions(+), 12 deletions(-) create mode 100644 pkg/sql/partition_test.go diff --git a/pkg/ccl/cliccl/main_test.go b/pkg/ccl/cliccl/main_test.go index ee432e629be9..8f58d9f240e8 100644 --- a/pkg/ccl/cliccl/main_test.go +++ b/pkg/ccl/cliccl/main_test.go @@ -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()) } diff --git a/pkg/ccl/partitionccl/partition.go b/pkg/ccl/partitionccl/partition.go index 76ee06eba0c8..690cec7cb592 100644 --- a/pkg/ccl/partitionccl/partition.go +++ b/pkg/ccl/partitionccl/partition.go @@ -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" @@ -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, @@ -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 */) } @@ -422,5 +428,5 @@ func selectPartitionExprsByName( } func init() { - sql.CreatePartitioning = CreatePartitioning + sql.CreatePartitioningCCL = createPartitioning } diff --git a/pkg/ccl/partitionccl/partition_test.go b/pkg/ccl/partitionccl/partition_test.go index 4b80744fd904..0eb6cad883b1 100644 --- a/pkg/ccl/partitionccl/partition_test.go +++ b/pkg/ccl/partitionccl/partition_test.go @@ -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" @@ -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" @@ -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 ''`) +} diff --git a/pkg/ccl/partitionccl/zone.go b/pkg/ccl/partitionccl/zone.go index ae6d74842d48..e49da8875f60 100644 --- a/pkg/ccl/partitionccl/zone.go +++ b/pkg/ccl/partitionccl/zone.go @@ -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 @@ -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) diff --git a/pkg/ccl/partitionccl/zone_test.go b/pkg/ccl/partitionccl/zone_test.go index 4cf0e78ace7a..2f6d0ef26e9a 100644 --- a/pkg/ccl/partitionccl/zone_test.go +++ b/pkg/ccl/partitionccl/zone_test.go @@ -17,6 +17,7 @@ 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" @@ -24,6 +25,7 @@ import ( "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) { @@ -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) } diff --git a/pkg/ccl/utilccl/license_check.go b/pkg/ccl/utilccl/license_check.go index f6e480f14293..e605edc9a489 100644 --- a/pkg/ccl/utilccl/license_check.go +++ b/pkg/ccl/utilccl/license_check.go @@ -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. diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index b097cbd6b814..166946da32f9 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -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", diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 5079e4c352b3..88c842d54003 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -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, diff --git a/pkg/sql/partition_test.go b/pkg/sql/partition_test.go new file mode 100644 index 000000000000..f649eef8ab0c --- /dev/null +++ b/pkg/sql/partition_test.go @@ -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) + } +} diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index 0acda58b326f..f544281f20e0 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -151,8 +151,9 @@ func (n *setZoneConfigNode) startExec(params runParams) error { } } + hasNewSubzones := yamlConfig != nil && index != nil n.run.numAffected, err = writeZoneConfig(params.ctx, params.p.txn, - targetID, table, zone, params.extendedEvalCtx.ExecCfg) + targetID, table, zone, params.extendedEvalCtx.ExecCfg, hasNewSubzones) return err } @@ -169,12 +170,15 @@ func writeZoneConfig( table *sqlbase.TableDescriptor, zone config.ZoneConfig, execCfg *ExecutorConfig, + hasNewSubzones bool, ) (numAffected int, err error) { if len(zone.Subzones) > 0 { - if !execCfg.Settings.Version.IsMinSupported(cluster.VersionPartitioning) { + st := execCfg.Settings + if !st.Version.IsMinSupported(cluster.VersionPartitioning) { return 0, errors.New("cluster version does not support zone configs on indexes or partitions") } - zone.SubzoneSpans, err = GenerateSubzoneSpans(table, zone.Subzones) + zone.SubzoneSpans, err = GenerateSubzoneSpans( + st, execCfg.ClusterID(), table, zone.Subzones, hasNewSubzones) if err != nil { return 0, err } diff --git a/pkg/sql/zone_config.go b/pkg/sql/zone_config.go index 64ecbdeedef9..ffeabf1e9afc 100644 --- a/pkg/sql/zone_config.go +++ b/pkg/sql/zone_config.go @@ -21,9 +21,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/internal/client" "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/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/pkg/errors" ) @@ -142,7 +144,11 @@ func GetZoneConfigInTxn( // in the SubzonzeSpans field of config.ZoneConfig. If no CCL hook is installed, // it returns an error that directs users to use a CCL binary. var GenerateSubzoneSpans = func( - *sqlbase.TableDescriptor, []config.Subzone, + st *cluster.Settings, + clusterID uuid.UUID, + tableDesc *sqlbase.TableDescriptor, + subzones []config.Subzone, + newSubzones bool, ) ([]config.SubzoneSpan, error) { return nil, sqlbase.NewCCLRequiredError(errors.New( "setting zone configs on indexes or partitions requires a CCL binary")) @@ -244,6 +250,7 @@ func deleteRemovedPartitionZoneConfigs( for _, n := range removedNames { zone.DeleteSubzone(uint32(idxDesc.ID), n) } - _, err = writeZoneConfig(ctx, txn, tableDesc.ID, tableDesc, zone, execCfg) + hasNewSubzones := false + _, err = writeZoneConfig(ctx, txn, tableDesc.ID, tableDesc, zone, execCfg, hasNewSubzones) return err }