Skip to content

Commit

Permalink
Merge pull request #79820 from ajwerner/backport22.1-76504
Browse files Browse the repository at this point in the history
release-22.1: *: fix system config for tenant
  • Loading branch information
ajwerner authored Apr 12, 2022
2 parents 3641c5a + 2f6b53a commit 391b1bb
Show file tree
Hide file tree
Showing 34 changed files with 594 additions and 180 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 @@ -2018,7 +2018,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 @@ -85,6 +86,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 @@ -16,23 +16,28 @@ 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/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: 0 additions & 1 deletion pkg/ccl/spanconfigccl/spanconfiglimiterccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ go_test(
"//pkg/ccl/utilccl",
"//pkg/config",
"//pkg/config/zonepb",
"//pkg/keys",
"//pkg/roachpb",
"//pkg/security",
"//pkg/security/securitytest",
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/spanconfigccl/spanconfiglimiterccl/drop_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/gcjob"
Expand Down Expand Up @@ -57,14 +56,15 @@ func TestDropTableLowersSpanCount(t *testing.T) {
zoneConfig := zonepb.DefaultZoneConfig()
zoneConfig.GC.TTLSeconds = 1
config.TestingSetupZoneConfigHook(tc.Stopper())
// TODO(irfansharif): Work around for #75864.
config.TestingSetZoneConfig(keys.TenantsRangesID, zoneConfig)

require.NoError(t, err)
defer func() { require.NoError(t, tenantSQLDB.Close()) }()

tenantDB := sqlutils.MakeSQLRunner(tenantSQLDB)

tenantDB.Exec(t, `CREATE TABLE t(k INT PRIMARY KEY)`)
id := sqlutils.QueryTableID(t, tenantSQLDB, "defaultdb", "public", "t")
config.TestingSetZoneConfig(config.ObjectID(id), zoneConfig)

var spanCount int
tenantDB.QueryRow(t, `SELECT span_count FROM system.span_count LIMIT 1`).Scan(&spanCount)
Expand Down
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 391b1bb

Please sign in to comment.