Skip to content

Commit

Permalink
*: fix system config for tenant
Browse files Browse the repository at this point in the history
Follow-on from #76279.

Fixes #75864.

This commit adds a mechanism to combine the system config data of the tenant
with the data provided over the GossipSubscription from the system tenant.
It then plumbs a version into the zone config methods. In the mixed version
state, the tenant uses the existing override from the system tenant. After
the span config infrastructure has been activated, the tenant uses the
overrides they've set. This affects, realistically, just the GC job, and,
to a lesser extent, the optimizer.

Release note: None
  • Loading branch information
ajwerner committed Apr 11, 2022
1 parent 5c1ae72 commit ff63efc
Show file tree
Hide file tree
Showing 31 changed files with 592 additions and 177 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1739,7 +1739,7 @@ func TestBackupRestoreControlJob(t *testing.T) {
if err != nil {
t.Fatal(err)
}
last := config.SystemTenantObjectID(v.ValueInt())
last := config.ObjectID(v.ValueInt())
zoneConfig := zonepb.DefaultZoneConfig()
zoneConfig.RangeMaxBytes = proto.Int64(5000)
config.TestingSetZoneConfig(last+1, zoneConfig)
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/kvccl/kvtenantccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ go_test(
"//pkg/config",
"//pkg/gossip",
"//pkg/jobs",
"//pkg/keys",
"//pkg/kv/kvclient/kvtenant",
"//pkg/kv/kvserver",
"//pkg/kv/kvserver/kvserverbase",
Expand Down Expand Up @@ -86,6 +87,7 @@ go_test(
"//pkg/util/stop",
"//pkg/util/tracing",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
Expand Down
115 changes: 114 additions & 1 deletion pkg/ccl/kvccl/kvtenantccl/tenant_upgrade_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,24 +17,29 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/migration"
"github.com/cockroachdb/cockroach/pkg/migration/migrations"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slinstance"
"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/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)

// TestTenantUpgrade exercises the case where a system tenant is in a
// non-finalized version state and creates a tenant. The test ensures
// that that newly created tenant begins in that same version.
// that the newly created tenant begins in that same version.
//
// The first subtest creates the tenant in the mixed version state,
// then upgrades the system tenant, then upgrades the secondary tenant,
Expand Down Expand Up @@ -358,3 +363,111 @@ func TestTenantUpgradeFailure(t *testing.T) {
tenantInfo.v2onMigrationStopper.Stop(ctx)
})
}

// TestTenantSystemConfigUpgrade ensures that the tenant GC job uses the
// appropriate view of the GC TTL.
func TestTenantSystemConfigUpgrade(t *testing.T) {
defer leaktest.AfterTest(t)()
ctx := context.Background()
settings := cluster.MakeTestingClusterSettingsWithVersions(
clusterversion.TestingBinaryVersion,
clusterversion.TestingBinaryMinSupportedVersion,
false, // initializeVersion
)
// Initialize the version to the BinaryMinSupportedVersion.
require.NoError(t, clusterversion.Initialize(ctx,
clusterversion.TestingBinaryMinSupportedVersion, &settings.SV))
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Settings: settings,
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
DisableAutomaticVersionUpgrade: make(chan struct{}),
BinaryVersionOverride: clusterversion.TestingBinaryMinSupportedVersion,
},
},
},
})
hostDB := sqlutils.MakeSQLRunner(tc.ServerConn(0))
hostDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.target_duration = '20ms'`)
hostDB.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '20ms'`)
defer tc.Stopper().Stop(ctx)
connectToTenant := func(t *testing.T, addr string) (_ *gosql.DB, cleanup func()) {
pgURL, cleanupPGUrl := sqlutils.PGUrl(t, addr, "Tenant", url.User(security.RootUser))
tenantDB, err := gosql.Open("postgres", pgURL.String())
require.NoError(t, err)
return tenantDB, func() {
tenantDB.Close()
cleanupPGUrl()
}
}
mkTenant := func(t *testing.T, id uint64) (
tenant serverutils.TestTenantInterface,
) {
settings := cluster.MakeTestingClusterSettingsWithVersions(
clusterversion.TestingBinaryVersion,
clusterversion.TestingBinaryMinSupportedVersion,
false, // initializeVersion
)
// Initialize the version to the minimum it could be.
require.NoError(t, clusterversion.Initialize(ctx,
clusterversion.TestingBinaryMinSupportedVersion, &settings.SV))
tenantArgs := base.TestTenantArgs{
TenantID: roachpb.MakeTenantID(id),
TestingKnobs: base.TestingKnobs{},
Settings: settings,
}
tenant, err := tc.Server(0).StartTenant(ctx, tenantArgs)
require.NoError(t, err)
return tenant
}
const tenantID = 10
codec := keys.MakeSQLCodec(roachpb.MakeTenantID(tenantID))
tenant := mkTenant(t, tenantID)
tenantSQL, cleanup := connectToTenant(t, tenant.SQLAddr())
defer cleanup()
tenantDB := sqlutils.MakeSQLRunner(tenantSQL)
tenantDB.CheckQueryResults(t, "SHOW CLUSTER SETTING version", [][]string{{"21.2"}})
tenantDB.Exec(t, "CREATE TABLE foo ()")
fooID := sqlutils.QueryTableID(t, tenantSQL, "defaultdb", "public", "foo")
tenantP := tenant.SystemConfigProvider()
ch, _ := tenantP.RegisterSystemConfigChannel()

hostDB.Exec(t, "SET CLUSTER SETTING version = crdb_internal.node_executable_version()")
hostDB.Exec(t, "ALTER RANGE tenants CONFIGURE ZONE USING gc.ttlseconds = 111")
hostDB.Exec(t,
"ALTER TENANT $1 SET CLUSTER SETTING sql.zone_configs.allow_for_secondary_tenant.enabled = true;",
tenantID)
tenantDB.CheckQueryResultsRetry(
t, "SHOW CLUSTER SETTING sql.zone_configs.allow_for_secondary_tenant.enabled",
[][]string{{"true"}},
)
tenantVersion := func() clusterversion.ClusterVersion {
return tenant.ClusterSettings().Version.ActiveVersionOrEmpty(ctx)
}
checkConfigEqual := func(t *testing.T, exp int32) {
testutils.SucceedsSoon(t, func() error {
cfg := tenantP.GetSystemConfig()
if cfg == nil {
return errors.New("no config")
}
conf, err := tenantP.GetSystemConfig().GetZoneConfigForObject(codec, tenantVersion(), config.ObjectID(fooID))
if err != nil {
return err
}
if conf.GC.TTLSeconds != exp {
return errors.Errorf("got %d, expected %d", conf.GC.TTLSeconds, exp)
}
return nil
})
}
checkConfigEqual(t, 111)
<-ch
hostDB.Exec(t, "ALTER RANGE tenants CONFIGURE ZONE USING gc.ttlseconds = 112")
<-ch
checkConfigEqual(t, 112)
tenantDB.Exec(t, "SET CLUSTER SETTING version = crdb_internal.node_executable_version()")
tenantDB.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds = 113")
<-ch
checkConfigEqual(t, 113)
}
1 change: 1 addition & 0 deletions pkg/config/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/config",
visibility = ["//visibility:public"],
deps = [
"//pkg/clusterversion",
"//pkg/config/zonepb",
"//pkg/keys",
"//pkg/roachpb",
Expand Down
8 changes: 4 additions & 4 deletions pkg/config/keys.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,11 +27,11 @@ 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
// DecodeObjectID decodes the object ID for the system-tenant from
// the front of key. It returns the decoded object ID, the remainder of the key,
// and whether the result is valid (i.e., whether the key was within the system
// tenant's structured key space).
func DecodeSystemTenantObjectID(key roachpb.RKey) (SystemTenantObjectID, []byte, bool) {
rem, id, err := keys.SystemSQLCodec.DecodeTablePrefix(key.AsRawKey())
return SystemTenantObjectID(id), rem, err == nil
func DecodeObjectID(codec keys.SQLCodec, key roachpb.RKey) (ObjectID, []byte, bool) {
rem, id, err := codec.DecodeTablePrefix(key.AsRawKey())
return ObjectID(id), rem, err == nil
}
4 changes: 2 additions & 2 deletions pkg/config/keys_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ func TestDecodeSystemTenantObjectID(t *testing.T) {
key roachpb.RKey
keySuffix []byte
success bool
id config.SystemTenantObjectID
id config.ObjectID
}{
// Before the structured span.
{roachpb.RKeyMin, nil, false, 0},
Expand All @@ -43,7 +43,7 @@ func TestDecodeSystemTenantObjectID(t *testing.T) {
}

for tcNum, tc := range testCases {
id, keySuffix, success := config.DecodeSystemTenantObjectID(tc.key)
id, keySuffix, success := config.DecodeObjectID(keys.SystemSQLCodec, tc.key)
if success != tc.success {
t.Errorf("#%d: expected success=%t", tcNum, tc.success)
continue
Expand Down
Loading

0 comments on commit ff63efc

Please sign in to comment.