diff --git a/pkg/ccl/logictestccl/testdata/logic_test/tenant_usage b/pkg/ccl/logictestccl/testdata/logic_test/tenant_usage new file mode 100644 index 000000000000..02d50524012b --- /dev/null +++ b/pkg/ccl/logictestccl/testdata/logic_test/tenant_usage @@ -0,0 +1,25 @@ +# LogicTest: !3node-tenant + +query error tenant "13" does not exist +SELECT crdb_internal.update_tenant_resource_limits(13, 1000, 100, 0, now(), 0) + +query I +SELECT crdb_internal.create_tenant(5) +---- +5 + +# TODO(radu): inspect internal tenant_usage state. + +statement ok +SELECT crdb_internal.update_tenant_resource_limits(5, 1000, 100, 0, now(), 0) + +# TODO(radu): inspect internal tenant_usage state. + +# Note this just marks the tenant as dropped but does not call GC. +query I +SELECT crdb_internal.destroy_tenant(5) +---- +5 + +query error tenant "5" is not active +SELECT crdb_internal.update_tenant_resource_limits(5, 1000, 100, 0, now(), 0) diff --git a/pkg/ccl/multitenantccl/tenantcostserver/BUILD.bazel b/pkg/ccl/multitenantccl/tenantcostserver/BUILD.bazel index 099226b57d3d..a9259323844a 100644 --- a/pkg/ccl/multitenantccl/tenantcostserver/BUILD.bazel +++ b/pkg/ccl/multitenantccl/tenantcostserver/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "tenantcostserver", srcs = [ + "configure.go", "server.go", "token_bucket.go", ], @@ -15,6 +16,10 @@ go_library( "//pkg/roachpb:with-mocks", "//pkg/server", "//pkg/sql", + "//pkg/sql/pgwire/pgcode", + "//pkg/sql/pgwire/pgerror", + "//pkg/sql/sem/tree", + "//pkg/sql/sessiondata", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", ], diff --git a/pkg/ccl/multitenantccl/tenantcostserver/configure.go b/pkg/ccl/multitenantccl/tenantcostserver/configure.go new file mode 100644 index 000000000000..5ec28039ee10 --- /dev/null +++ b/pkg/ccl/multitenantccl/tenantcostserver/configure.go @@ -0,0 +1,63 @@ +// Copyright 2021 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package tenantcostserver + +import ( + "context" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" +) + +// ReconfigureTokenBucket updates a tenant's token bucket settings. It is part +// of the TenantUsageServer interface; see that for more details. +func (s *instance) ReconfigureTokenBucket( + ctx context.Context, + txn *kv.Txn, + tenantID roachpb.TenantID, + availableRU float64, + refillRate float64, + maxBurstRU float64, + asOf time.Time, + asOfConsumedRequestUnits float64, +) error { + row, err := s.executor.QueryRowEx( + ctx, "check-tenant", txn, sessiondata.NodeUserSessionDataOverride, + `SELECT active FROM system.tenants WHERE id = $1`, tenantID.ToUint64(), + ) + if err != nil { + return err + } + if row == nil { + return pgerror.Newf(pgcode.UndefinedObject, "tenant %q does not exist", tenantID) + } + if active := *row[0].(*tree.DBool); !active { + return errors.Errorf("tenant %q is not active", tenantID) + } + state, err := readTenantUsageState(ctx, s.executor, txn, tenantID) + if err != nil { + return err + } + state.Seq++ + state.Bucket.Reconfigure( + availableRU, refillRate, maxBurstRU, asOf, asOfConsumedRequestUnits, + timeutil.Now(), state.Consumption.RU, + ) + if err := updateTenantUsageState(ctx, s.executor, txn, tenantID, state); err != nil { + return err + } + return nil +} diff --git a/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket.go b/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket.go index fa3366acbf61..b4b24a4de09a 100644 --- a/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket.go +++ b/pkg/ccl/multitenantccl/tenantcostserver/tenanttokenbucket/tenant_token_bucket.go @@ -42,4 +42,44 @@ func (s *State) Request( return res } -// TODO(radu): add Reconfigure API. +// Reconfigure updates the settings for the token bucket. +// +// Arguments: +// +// - availableRU is the amount of Request Units that the tenant can consume at +// will. Also known as "burst RUs". +// +// - refillRate is the amount of Request Units per second that the tenant +// receives. +// +// - maxBurstRU is the maximum amount of Request Units that can be accumulated +// from the refill rate, or 0 if there is no limit. +// +// - asOf is a timestamp; the reconfiguration request is assumed to be based on +// the consumption at that time. This timestamp is used to compensate for any +// refill that would have happened in the meantime. +// +// - asOfConsumedRequestUnits is the total number of consumed RUs based on +// which the reconfiguration values were calculated (i.e. at the asOf time). +// It is used to adjust availableRU with the consumption that happened in the +// meantime. +// +// - now is the current time. +// +// - currentConsumedRequestUnits is the current total number of consumed RUs. +// +func (s *State) Reconfigure( + availableRU float64, + refillRate float64, + maxBurstRU float64, + asOf time.Time, + asOfConsumedRequestUnits float64, + now time.Time, + currentConsumedRequestUnits float64, +) { + // TODO(radu): adjust available RUs according to asOf and asOfConsumedUnits + // and add tests. + s.RUCurrent = availableRU + s.RURefillRate = refillRate + s.RUBurstLimit = maxBurstRU +} diff --git a/pkg/multitenant/BUILD.bazel b/pkg/multitenant/BUILD.bazel index 3df4384bb444..ad221a731505 100644 --- a/pkg/multitenant/BUILD.bazel +++ b/pkg/multitenant/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/multitenant", visibility = ["//visibility:public"], deps = [ + "//pkg/kv", "//pkg/roachpb:with-mocks", "//pkg/util/stop", ], diff --git a/pkg/multitenant/tenant_usage.go b/pkg/multitenant/tenant_usage.go index 7fca11450716..51a32e8a693b 100644 --- a/pkg/multitenant/tenant_usage.go +++ b/pkg/multitenant/tenant_usage.go @@ -12,7 +12,9 @@ package multitenant import ( "context" + "time" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" ) @@ -27,5 +29,36 @@ type TenantUsageServer interface { ctx context.Context, tenantID roachpb.TenantID, in *roachpb.TokenBucketRequest, ) (*roachpb.TokenBucketResponse, error) - // TODO(radu): add Reconfigure API. + // ReconfigureTokenBucket updates a tenant's token bucket settings. + // + // Arguments: + // + // - availableRU is the amount of Request Units that the tenant can consume at + // will. Also known as "burst RUs". + // + // - refillRate is the amount of Request Units per second that the tenant + // receives. + // + // - maxBurstRU is the maximum amount of Request Units that can be accumulated + // from the refill rate, or 0 if there is no limit. + // + // - asOf is a timestamp; the reconfiguration request is assumed to be based on + // the consumption at that time. This timestamp is used to compensate for any + // refill that would have happened in the meantime. + // + // - asOfConsumedRequestUnits is the total number of consumed RUs based on + // which the reconfiguration values were calculated (i.e. at the asOf time). + // It is used to adjust availableRU with the consumption that happened in the + // meantime. + // + ReconfigureTokenBucket( + ctx context.Context, + txn *kv.Txn, + tenantID roachpb.TenantID, + availableRU float64, + refillRate float64, + maxBurstRU float64, + asOf time.Time, + asOfConsumedRequestUnits float64, + ) error } diff --git a/pkg/server/node.go b/pkg/server/node.go index 0c5a99bacf6f..832e766cacfa 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -1360,3 +1360,17 @@ func (dummyTenantUsageServer) TokenBucketRequest( ) (*roachpb.TokenBucketResponse, error) { return nil, errors.Errorf("tenant usage requires a CCL binary") } + +// TokenBucketRequest is defined in the TenantUsageServer interface. +func (dummyTenantUsageServer) ReconfigureTokenBucket( + ctx context.Context, + txn *kv.Txn, + tenantID roachpb.TenantID, + availableRU float64, + refillRate float64, + maxBurstRU float64, + asOf time.Time, + asOfConsumedRequestUnits float64, +) error { + return errors.Errorf("tenant resource limits require a CCL binary") +} diff --git a/pkg/sql/faketreeeval/evalctx.go b/pkg/sql/faketreeeval/evalctx.go index 0982ef5615b6..34f7340a6a95 100644 --- a/pkg/sql/faketreeeval/evalctx.go +++ b/pkg/sql/faketreeeval/evalctx.go @@ -13,6 +13,7 @@ package faketreeeval import ( "context" + "time" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/parser" @@ -379,3 +380,16 @@ func (c *DummyTenantOperator) DestroyTenant(_ context.Context, _ uint64) error { func (c *DummyTenantOperator) GCTenant(_ context.Context, _ uint64) error { return errors.WithStack(errEvalTenant) } + +// UpdateTenantResourceLimits is part of the tree.TenantOperator interface. +func (c *DummyTenantOperator) UpdateTenantResourceLimits( + _ context.Context, + tenantID uint64, + availableRU float64, + refillRate float64, + maxBurstRU float64, + asOf time.Time, + asOfConsumedRequestUnits float64, +) error { + return errors.WithStack(errEvalTenant) +} diff --git a/pkg/sql/logictest/testdata/logic_test/tenant b/pkg/sql/logictest/testdata/logic_test/tenant index 6d67ee383196..d73130d166f6 100644 --- a/pkg/sql/logictest/testdata/logic_test/tenant +++ b/pkg/sql/logictest/testdata/logic_test/tenant @@ -151,3 +151,6 @@ ORDER BY id ---- id active crdb_internal.pb_to_json 10 true {"id": "10", "state": "ACTIVE"} + +query error tenant resource limits require a CCL binary +SELECT crdb_internal.update_tenant_resource_limits(10, 1000, 100, 0, now(), 0) diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index ae885bf120d0..04bf66804ef1 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -5084,6 +5084,51 @@ may increase either contention or retry errors, or both.`, }, ), + // Used to configure the tenant token bucket. See UpdateTenantResourceLimits. + "crdb_internal.update_tenant_resource_limits": makeBuiltin( + tree.FunctionProperties{ + Category: categoryMultiTenancy, + Undocumented: true, + }, + tree.Overload{ + Types: tree.ArgTypes{ + {"tenant_id", types.Int}, + {"available_request_units", types.Float}, + {"refill_rate", types.Float}, + {"max_burst_request_units", types.Float}, + {"as_of", types.Timestamp}, + {"as_of_consumed_request_units", types.Float}, + }, + ReturnType: tree.FixedReturnType(types.Int), + Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + sTenID := int64(tree.MustBeDInt(args[0])) + if sTenID <= 0 { + return nil, pgerror.New(pgcode.InvalidParameterValue, "tenant ID must be positive") + } + availableRU := float64(tree.MustBeDFloat(args[1])) + refillRate := float64(tree.MustBeDFloat(args[2])) + maxBurstRU := float64(tree.MustBeDFloat(args[3])) + asOf := tree.MustBeDTimestamp(args[4]).Time + asOfConsumed := float64(tree.MustBeDFloat(args[5])) + + if err := ctx.Tenant.UpdateTenantResourceLimits( + ctx.Context, + uint64(sTenID), + availableRU, + refillRate, + maxBurstRU, + asOf, + asOfConsumed, + ); err != nil { + return nil, err + } + return args[0], nil + }, + Info: "Updates resource limits for the tenant with the provided ID. Must be run by the System tenant.", + Volatility: tree.VolatilityVolatile, + }, + ), + "crdb_internal.compact_engine_span": makeBuiltin( tree.FunctionProperties{ Category: categorySystemRepair, diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index 18c039e63ce7..13d9a021f229 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -3295,8 +3295,8 @@ type SequenceOperators interface { } // TenantOperator is capable of interacting with tenant state, allowing SQL -// builtin functions to create and destroy tenants. The methods will return -// errors when run by any tenant other than the system tenant. +// builtin functions to create, configure, and destroy tenants. The methods will +// return errors when run by any tenant other than the system tenant. type TenantOperator interface { // CreateTenant attempts to install a new tenant in the system. It returns // an error if the tenant already exists. The new tenant is created at the @@ -3311,6 +3311,18 @@ type TenantOperator interface { // success it also removes the tenant record. // It returns an error if the tenant does not exist. GCTenant(ctx context.Context, tenantID uint64) error + + // UpdateTenantResourceLimits reconfigures the tenant resource limits. + // See multitenant.TenantUsageServer for more details on the arguments. + UpdateTenantResourceLimits( + ctx context.Context, + tenantID uint64, + availableRU float64, + refillRate float64, + maxBurstRU float64, + asOf time.Time, + asOfConsumedRequestUnits float64, + ) error } // JoinTokenCreator is capable of creating and persisting join tokens, allowing diff --git a/pkg/sql/tenant.go b/pkg/sql/tenant.go index 24e3fbfaecd0..04f9d4ca61ef 100644 --- a/pkg/sql/tenant.go +++ b/pkg/sql/tenant.go @@ -406,3 +406,26 @@ func (p *planner) GCTenant(ctx context.Context, tenID uint64) error { return GCTenantJob(ctx, p.ExecCfg(), p.Txn(), p.User(), tenID) } + +// UpdateTenantResourceLimits implements the tree.TenantOperator interface. +func (p *planner) UpdateTenantResourceLimits( + ctx context.Context, + tenantID uint64, + availableRU float64, + refillRate float64, + maxBurstRU float64, + asOf time.Time, + asOfConsumedRequestUnits float64, +) error { + const op = "update-resource-limits" + if err := rejectIfCantCoordinateMultiTenancy(p.execCfg.Codec, op); err != nil { + return err + } + if err := rejectIfSystemTenant(tenantID, op); err != nil { + return err + } + return p.ExecCfg().TenantUsageServer.ReconfigureTokenBucket( + ctx, p.Txn(), roachpb.MakeTenantID(tenantID), + availableRU, refillRate, maxBurstRU, asOf, asOfConsumedRequestUnits, + ) +}