Skip to content

Commit

Permalink
Merge #69169 #69185
Browse files Browse the repository at this point in the history
69169: sql: allow secondary tenants to set/show zone configurations r=irfansharif,ajwerner a=arulajmani

Part of #67679. The zone configurations themselves have no effect right now. See individual commits for details. 



69185: cli,sql/sem/builtins: emit_defaults to false for decode-proto, pb_to_json r=dt a=ajwerner

When `cockroach debug decode-proto` and `crdb_internal.pb_to_json` were first
added, they emitted the default values in the produced json. This turns out to
not be desirable; setting emit defaults to false produces json which round-
trips back to the same proto.

Release note (cli change): `cockroach debug decode-proto` now does not emit
default values by default.

Release note (sql change): `crdb_internal.pb_to_json` now does not emit
default values by default.

Co-authored-by: arulajmani <arulajmani@gmail.com>
Co-authored-by: Andrew Werner <awerner32@gmail.com>
  • Loading branch information
3 people committed Aug 20, 2021
3 parents 2c51f70 + e3be882 + 2753a47 commit 466e292
Show file tree
Hide file tree
Showing 43 changed files with 410 additions and 205 deletions.
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -153,4 +153,4 @@ trace.datadog.project string CockroachDB the project under which traces will be
trace.debug.enable boolean false if set, traces for recent requests can be seen at https://<ui>/debug/requests
trace.lightstep.token string if set, traces go to Lightstep using this token
trace.zipkin.collector string if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time.
version version 21.1-142 set the active cluster version in the format '<major>.<minor>'
version version 21.1-144 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -157,6 +157,6 @@
<tr><td><code>trace.debug.enable</code></td><td>boolean</td><td><code>false</code></td><td>if set, traces for recent requests can be seen at https://<ui>/debug/requests</td></tr>
<tr><td><code>trace.lightstep.token</code></td><td>string</td><td><code></code></td><td>if set, traces go to Lightstep using this token</td></tr>
<tr><td><code>trace.zipkin.collector</code></td><td>string</td><td><code></code></td><td>if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'). Only one tracer can be configured at a time.</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.1-142</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.1-144</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
10 changes: 5 additions & 5 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6975,7 +6975,7 @@ func TestBackupRestoreTenant(t *testing.T) {
restoreDB.CheckQueryResults(t, `select * from system.tenants`, [][]string{})
restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/t10'`)
restoreDB.CheckQueryResults(t,
`select id, active, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info) from system.tenants`,
`select id, active, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`,
[][]string{{`10`, `true`, `{"id": "10", "state": "ACTIVE"}`}},
)

Expand Down Expand Up @@ -7023,7 +7023,7 @@ func TestBackupRestoreTenant(t *testing.T) {

restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/t10'`)
restoreDB.CheckQueryResults(t,
`select id, active, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info) from system.tenants`,
`select id, active, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`,
[][]string{{`10`, `true`, `{"id": "10", "state": "ACTIVE"}`}},
)

Expand All @@ -7050,7 +7050,7 @@ func TestBackupRestoreTenant(t *testing.T) {
restoreDB.CheckQueryResults(t, `select * from system.tenants`, [][]string{})
restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/t10'`)
restoreDB.CheckQueryResults(t,
`select id, active, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info) from system.tenants`,
`select id, active, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`,
[][]string{{`10`, `true`, `{"id": "10", "state": "ACTIVE"}`}},
)
})
Expand All @@ -7065,7 +7065,7 @@ func TestBackupRestoreTenant(t *testing.T) {
restoreDB.CheckQueryResults(t, `select * from system.tenants`, [][]string{})
restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/clusterwide'`)
restoreDB.CheckQueryResults(t,
`select id, active, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info) from system.tenants`,
`select id, active, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`,
[][]string{{`10`, `true`, `{"id": "10", "state": "ACTIVE"}`}},
)

Expand All @@ -7092,7 +7092,7 @@ func TestBackupRestoreTenant(t *testing.T) {
restoreDB.CheckQueryResults(t, `select * from system.tenants`, [][]string{})
restoreDB.Exec(t, `RESTORE FROM 'nodelocal://1/clusterwide'`)
restoreDB.CheckQueryResults(t,
`select id, active, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info) from system.tenants`,
`select id, active, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`,
[][]string{
{`10`, `true`, `{"id": "10", "state": "ACTIVE"}`},
{`11`, `true`, `{"id": "11", "state": "ACTIVE"}`},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,5 +10,5 @@ us-east-1 {us-az1,us-az2,us-az3}
statement error region "bad-region" does not exist
CREATE DATABASE db PRIMARY REGION "bad-region"

statement error operation is unsupported in multi-tenancy mode
statement ok
CREATE DATABASE db PRIMARY REGION "us-east-1"
13 changes: 0 additions & 13 deletions pkg/ccl/logictestccl/testdata/logic_test/tenant_unsupported
Original file line number Diff line number Diff line change
Expand Up @@ -43,19 +43,6 @@ SELECT * FROM crdb_internal.kv_store_status
statement error operation is unsupported in multi-tenancy mode
SELECT * FROM crdb_internal.kv_node_status

# Cannot manipulate zone configurations

# Selecting from crdb_internal.zones is allowed but no data is returned.
query IITTTTTTTTTTTT
SELECT * FROM crdb_internal.zones
----

statement error operation is unsupported in multi-tenancy mode
SHOW ZONE CONFIGURATION FOR TABLE kv

statement error operation is unsupported in multi-tenancy mode
ALTER TABLE kv CONFIGURE ZONE USING num_replicas = 123

# Cannot perform operations that issue Admin requests.

statement error operation is unsupported in multi-tenancy mode
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
# LogicTest: 3node-tenant
# Zone config logic tests that are only meant to work for secondary tenants.

statement ok
CREATE TABLE t();

statement error pq: unimplemented: operation is unsupported in multi-tenancy mode
ALTER TABLE t CONFIGURE ZONE USING num_replicas = 5;

statement ok
SET CLUSTER SETTING sql.zone_configs.experimental_allow_for_secondary_tenant.enabled = true

statement ok
ALTER TABLE t CONFIGURE ZONE USING num_replicas = 5;

query IT
SELECT zone_id, target FROM crdb_internal.zones ORDER BY 1
----
0 RANGE default
53 TABLE test.public.t
2 changes: 1 addition & 1 deletion pkg/ccl/partitionccl/partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1424,7 +1424,7 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (y)

// Get the zone config corresponding to the table.
table := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "t")
kv, err := kvDB.Get(ctx, config.MakeZoneKey(config.SystemTenantObjectID(table.GetID())))
kv, err := kvDB.Get(ctx, config.MakeZoneKey(keys.SystemSQLCodec, table.GetID()))
if err != nil {
t.Fatal(err)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -1557,7 +1557,7 @@ func init() {
f = debugDecodeProtoCmd.Flags()
f.StringVar(&debugDecodeProtoName, "schema", "cockroach.sql.sqlbase.Descriptor",
"fully qualified name of the proto to decode")
f.BoolVar(&debugDecodeProtoEmitDefaults, "emit-defaults", true,
f.BoolVar(&debugDecodeProtoEmitDefaults, "emit-defaults", false,
"encode default values for every field")

f = debugCheckLogConfigCmd.Flags()
Expand Down
6 changes: 5 additions & 1 deletion pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -288,7 +288,6 @@ const (
// AutoSpanConfigReconciliationJob adds the AutoSpanConfigReconciliationJob
// type.
AutoSpanConfigReconciliationJob

// PreventNewInterleavedTables interleaved table creation is completely
// blocked on this version.
PreventNewInterleavedTables
Expand All @@ -297,6 +296,8 @@ const (
EnsureNoInterleavedTables
// DefaultPrivileges default privileges are supported in this version.
DefaultPrivileges
// ZonesTableForSecondaryTenants adds system.zones for all secondary tenants.
ZonesTableForSecondaryTenants
// Step (1): Add new versions here.
)

Expand Down Expand Up @@ -499,6 +500,9 @@ var versionsSingleton = keyedVersions{
{
Key: DefaultPrivileges,
Version: roachpb.Version{Major: 21, Minor: 1, Internal: 142},
}, {
Key: ZonesTableForSecondaryTenants,
Version: roachpb.Version{Major: 21, Minor: 1, Internal: 144},
},
// Step (2): Add new versions here.
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/clusterversion/key_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

11 changes: 6 additions & 5 deletions pkg/config/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,17 +13,18 @@ package config
import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
)

// MakeZoneKeyPrefix returns the key prefix for id's row in the system.zones
// table.
func MakeZoneKeyPrefix(id SystemTenantObjectID) roachpb.Key {
return keys.SystemSQLCodec.ZoneKeyPrefix(uint32(id))
func MakeZoneKeyPrefix(codec keys.SQLCodec, id descpb.ID) roachpb.Key {
return codec.ZoneKeyPrefix(uint32(id))
}

// MakeZoneKey returns the key for id's entry in the system.zones table.
func MakeZoneKey(id SystemTenantObjectID) roachpb.Key {
return keys.SystemSQLCodec.ZoneKey(uint32(id))
// MakeZoneKey returns the key for a given id's entry in the system.zones table.
func MakeZoneKey(codec keys.SQLCodec, id descpb.ID) roachpb.Key {
return codec.ZoneKey(uint32(id))
}

// DecodeSystemTenantObjectID decodes the object ID for the system-tenant from
Expand Down
2 changes: 1 addition & 1 deletion pkg/config/system.go
Original file line number Diff line number Diff line change
Expand Up @@ -537,7 +537,7 @@ func (s *SystemConfig) systemTenantTableBoundarySplitKey(
return nil
}

zoneVal := s.GetValue(MakeZoneKey(id))
zoneVal := s.GetValue(MakeZoneKey(keys.SystemSQLCodec, descpb.ID(id)))
if zoneVal == nil {
continue
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/config/system_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,9 +82,9 @@ func tenant(tenID uint64) roachpb.KeyValue {
return kv(k, nil)
}

func zoneConfig(descID config.SystemTenantObjectID, spans ...zonepb.SubzoneSpan) roachpb.KeyValue {
func zoneConfig(descID descpb.ID, spans ...zonepb.SubzoneSpan) roachpb.KeyValue {
kv := roachpb.KeyValue{
Key: config.MakeZoneKey(descID),
Key: config.MakeZoneKey(keys.SystemSQLCodec, descID),
}
if err := kv.Value.SetProto(&zonepb.ZoneConfig{SubzoneSpans: spans}); err != nil {
panic(err)
Expand Down Expand Up @@ -373,8 +373,8 @@ func TestComputeSplitKeyTableIDs(t *testing.T) {
var subzoneSQL = make([]roachpb.KeyValue, len(userSQL))
copy(subzoneSQL, userSQL)
subzoneSQL = append(subzoneSQL,
zoneConfig(config.SystemTenantObjectID(start+1), subzone("a", ""), subzone("c", "e")),
zoneConfig(config.SystemTenantObjectID(start+5), subzone("b", ""), subzone("c", "d"), subzone("d", "")))
zoneConfig(descpb.ID(start+1), subzone("a", ""), subzone("c", "e")),
zoneConfig(descpb.ID(start+5), subzone("b", ""), subzone("c", "d"), subzone("d", "")))

sort.Sort(roachpb.KeyValueByKey(userSQL))
sort.Sort(roachpb.KeyValueByKey(subzoneSQL))
Expand Down
6 changes: 0 additions & 6 deletions pkg/keys/sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -153,18 +153,12 @@ func (e sqlEncoder) DescIDSequenceKey() roachpb.Key {

// ZoneKeyPrefix returns the key prefix for id's row in the system.zones table.
func (e sqlEncoder) ZoneKeyPrefix(id uint32) roachpb.Key {
if !e.ForSystemTenant() {
panic("zone keys only exist in the system tenant's keyspace")
}
k := e.IndexPrefix(ZonesTableID, ZonesTablePrimaryIndexID)
return encoding.EncodeUvarintAscending(k, uint64(id))
}

// ZoneKey returns the key for id's entry in the system.zones table.
func (e sqlEncoder) ZoneKey(id uint32) roachpb.Key {
if !e.ForSystemTenant() {
panic("zone keys only exist in the system tenant's keyspace")
}
k := e.ZoneKeyPrefix(id)
return MakeFamilyKey(k, uint32(ZonesTableConfigColumnID))
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvclient/kvtenant/connector.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,6 +121,6 @@ func AddressResolver(c Connector) nodedialer.AddressResolver {
var GossipSubscriptionSystemConfigMask = config.MakeSystemConfigMask(
// Tenant SQL processes need just enough of the zone hierarchy to understand
// which zone configurations apply to their keyspace.
config.MakeZoneKey(keys.RootNamespaceID),
config.MakeZoneKey(keys.TenantsRangesID),
config.MakeZoneKey(keys.SystemSQLCodec, keys.RootNamespaceID),
config.MakeZoneKey(keys.SystemSQLCodec, keys.TenantsRangesID),
)
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/reports/constraint_stats_report_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1104,7 +1104,7 @@ func (b *systemConfigBuilder) setDefaultZoneConfig(cfg zonepb.ZoneConfig) error
}

func (b *systemConfigBuilder) addZoneInner(objectName string, id int, cfg zonepb.ZoneConfig) error {
k := config.MakeZoneKey(config.SystemTenantObjectID(id))
k := config.MakeZoneKey(keys.SystemSQLCodec, descpb.ID(id))
var v roachpb.Value
if err := v.SetProto(&cfg); err != nil {
panic(err)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/reports/reporter.go
Original file line number Diff line number Diff line change
Expand Up @@ -489,7 +489,7 @@ func visitDefaultZone(
func getZoneByID(
id config.SystemTenantObjectID, cfg *config.SystemConfig,
) (*zonepb.ZoneConfig, error) {
zoneVal := cfg.GetValue(config.MakeZoneKey(id))
zoneVal := cfg.GetValue(config.MakeZoneKey(keys.SystemSQLCodec, descpb.ID(id)))
if zoneVal == nil {
return nil, nil
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/migration/migrations/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,13 @@ go_library(
"sql_stats.go",
"tenant_usage.go",
"truncated_state.go",
"zones.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/migration/migrations",
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/config/zonepb",
"//pkg/jobs/jobspb",
"//pkg/keys",
"//pkg/kv",
Expand All @@ -31,6 +33,7 @@ go_library(
"//pkg/security",
"//pkg/server/serverpb",
"//pkg/sql/catalog",
"//pkg/sql/catalog/bootstrap",
"//pkg/sql/catalog/catalogkeys",
"//pkg/sql/catalog/catalogkv",
"//pkg/sql/catalog/descpb",
Expand Down
6 changes: 6 additions & 0 deletions pkg/migration/migrations/migrations.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,6 +121,12 @@ var migrations = []migration.Migration{
interleavedTablesRemovedMigration,
interleavedTablesRemovedMigration,
),
migration.NewTenantMigration(
"add system.zones table for secondary tenants",
toCV(clusterversion.ZonesTableForSecondaryTenants),
NoPrecondition,
zonesTableForSecondaryTenants,
),
}

func init() {
Expand Down
50 changes: 50 additions & 0 deletions pkg/migration/migrations/zones.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package migrations

import (
"context"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/migration"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
"github.com/cockroachdb/cockroach/pkg/startupmigrations"
)

// zonesTableForSecondaryTenants adds system.zones to secondary tenants and
// seeds it.
func zonesTableForSecondaryTenants(
ctx context.Context, _ clusterversion.ClusterVersion, d migration.TenantDeps,
) error {
if d.Codec.ForSystemTenant() {
// We don't need to add system.zones to the system tenant as it should
// already be present.
return nil
}
if err := startupmigrations.CreateSystemTable(
ctx, d.DB, d.Codec, d.Settings, systemschema.ZonesTable,
); err != nil {
return err
}
defaultZoneConfig := zonepb.DefaultZoneConfigRef()
defaultSystemZoneConfig := zonepb.DefaultSystemZoneConfigRef()
kvs := bootstrap.InitialZoneConfigKVs(d.Codec, defaultZoneConfig, defaultSystemZoneConfig)
return d.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
b := txn.NewBatch()
for _, kv := range kvs {
b.Put(kv.Key, &kv.Value)
}
return txn.Run(ctx, b)
})
}
4 changes: 2 additions & 2 deletions pkg/sql/alter_primary_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -607,12 +607,12 @@ func maybeCopyPartitioningWhenDeinterleaving(
return nil
}
newPrimaryIndexDesc.Partitioning = *rootIndex.GetPartitioning().DeepCopy().PartitioningDesc()
rootCfg, err := getZoneConfigRaw(ctx, p.txn, p.execCfg.Codec, root.TableID)
rootCfg, err := getZoneConfigRaw(ctx, p.txn, p.execCfg.Codec, p.execCfg.Settings, root.TableID)
if err != nil {
return errors.Wrapf(err, "retrieving zone config for table %s [%d]",
interleaveRoot.GetName(), interleaveRoot.GetID())
}
tableCfg, err := getZoneConfigRaw(ctx, p.txn, p.execCfg.Codec, tableDesc.GetID())
tableCfg, err := getZoneConfigRaw(ctx, p.txn, p.execCfg.Codec, p.execCfg.Settings, tableDesc.GetID())
if err != nil {
return errors.Wrapf(err, "retrieving zone config for table %s [%d]",
tableDesc.GetName(), tableDesc.GetID())
Expand Down
Loading

0 comments on commit 466e292

Please sign in to comment.