Skip to content

Commit

Permalink
*: add resource group name into information_schema.processlist (#40739)
Browse files Browse the repository at this point in the history
close #40724
  • Loading branch information
BornChanger committed Feb 3, 2023
1 parent 15bac9e commit 975d177
Show file tree
Hide file tree
Showing 22 changed files with 192 additions and 148 deletions.
39 changes: 24 additions & 15 deletions ddl/resource_group_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ package ddl_test

import (
"context"
"strconv"
"testing"

"github.com/pingcap/tidb/ddl/internal/callback"
Expand Down Expand Up @@ -89,16 +88,17 @@ func TestResourceGroupBasic(t *testing.T) {
re.Equal(uint64(2000), g.RURate)
re.Equal(int64(-1), g.BurstLimit)

tk.MustExec("alter resource group if exists not_exists RU_PER_SEC=2000")
// Check warning message
res = tk.MustQuery("show warnings")
res.Check(testkit.Rows("Note 8249 Unknown resource group 'not_exists'"))
tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x 2000 0 YES"))

tk.MustQuery("select * from information_schema.resource_groups where group_name = 'x' ").Check(testkit.Rows(strconv.FormatInt(g.ID, 10) + " x 2000"))
tk.MustExec("drop resource group x")
g = testResourceGroupNameFromIS(t, tk.Session(), "x")
re.Nil(g)

tk.MustExec("alter resource group if exists not_exists RU_PER_SEC=2000")
// Check warning message
res = tk.MustQuery("show warnings")
res.Check(testkit.Rows("Note 8249 Unknown resource group 'not_exists'"))

tk.MustExec("create resource group y " +
"CPU='4000m' " +
"IO_READ_BANDWIDTH='1G' " +
Expand Down Expand Up @@ -137,29 +137,38 @@ func TestResourceGroupBasic(t *testing.T) {

// Check information schema table information_schema.resource_groups
tk.MustExec("create resource group x RU_PER_SEC=1000")
g1 := testResourceGroupNameFromIS(t, tk.Session(), "x")
tk.MustQuery("select * from information_schema.resource_groups where group_name = 'x'").Check(testkit.Rows(strconv.FormatInt(g1.ID, 10) + " x 1000"))
tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x 1000 0 NO"))
tk.MustQuery("show create resource group x").Check(testkit.Rows("x CREATE RESOURCE GROUP `x` RU_PER_SEC=1000"))

tk.MustExec("create resource group y RU_PER_SEC=2000")
g2 := testResourceGroupNameFromIS(t, tk.Session(), "y")
tk.MustQuery("select * from information_schema.resource_groups where group_name = 'y'").Check(testkit.Rows(strconv.FormatInt(g2.ID, 10) + " y 2000"))
tk.MustQuery("select * from information_schema.resource_groups where name = 'y'").Check(testkit.Rows("y 2000 0 NO"))
tk.MustQuery("show create resource group y").Check(testkit.Rows("y CREATE RESOURCE GROUP `y` RU_PER_SEC=2000"))

tk.MustExec("alter resource group y RU_PER_SEC=4000")

g2 = testResourceGroupNameFromIS(t, tk.Session(), "y")
tk.MustQuery("select * from information_schema.resource_groups where group_name = 'y'").Check(testkit.Rows(strconv.FormatInt(g2.ID, 10) + " y 4000"))
tk.MustQuery("show create resource group y").Check(testkit.Rows("y CREATE RESOURCE GROUP `y` RU_PER_SEC=4000"))
tk.MustExec("alter resource group y RU_PER_SEC=4000 BURSTABLE")
tk.MustQuery("select * from information_schema.resource_groups where name = 'y'").Check(testkit.Rows("y 4000 0 YES"))
tk.MustQuery("show create resource group y").Check(testkit.Rows("y CREATE RESOURCE GROUP `y` RU_PER_SEC=4000 BURSTABLE"))

tk.MustQuery("select count(*) from information_schema.resource_groups").Check(testkit.Rows("2"))
tk.MustGetErrCode("create user usr_fail resource group nil_group", mysql.ErrResourceGroupNotExists)
tk.MustContainErrMsg("create user usr_fail resource group nil_group", "Unknown resource group 'nil_group'")
tk.MustExec("create user user2")
tk.MustGetErrCode("alter user user2 resource group nil_group", mysql.ErrResourceGroupNotExists)
tk.MustContainErrMsg("alter user user2 resource group nil_group", "Unknown resource group 'nil_group'")

tk.MustExec("create resource group z " +
"CPU='4000m' " +
"IO_READ_BANDWIDTH='1G' " +
"IO_WRITE_BANDWIDTH='300M'")
tk.MustQuery("show create resource group z").Check(testkit.Rows("z CREATE RESOURCE GROUP `z` CPU=\"4000m\" IO_READ_BANDWIDTH=\"1G\" IO_WRITE_BANDWIDTH=\"300M\""))

tk.MustExec("create resource group do_not_delete_rg ru_per_sec=100")
tk.MustExec("create user usr3 resource group do_not_delete_rg")
tk.MustQuery("select user_attributes from mysql.user where user = 'usr3'").Check(testkit.Rows(`{"resource_group": "do_not_delete_rg"}`))
tk.MustContainErrMsg("drop resource group do_not_delete_rg", "user [usr3] depends on the resource group to drop")
tk.MustExec("alter user usr3 resource group `default`")
tk.MustExec("alter user usr3 resource group ``")
tk.MustExec("alter user usr3 resource group `DeFault`")
tk.MustQuery("select user_attributes from mysql.user where user = 'usr3'").Check(testkit.Rows(`{"resource_group": "default"}`))
}

func testResourceGroupNameFromIS(t *testing.T, ctx sessionctx.Context, name string) *model.ResourceGroupInfo {
Expand Down
5 changes: 3 additions & 2 deletions ddl/resourcegroup/group.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,15 @@ import (
"k8s.io/apimachinery/pkg/api/resource"
)

const maxGroupNameLength = 32
// MaxGroupNameLength is max length of the name of a resource group
const MaxGroupNameLength = 32

// NewGroupFromOptions creates a new resource group from the given options.
func NewGroupFromOptions(groupName string, options *model.ResourceGroupSettings) (*rmpb.ResourceGroup, error) {
if options == nil {
return nil, ErrInvalidGroupSettings
}
if len(groupName) > maxGroupNameLength {
if len(groupName) > MaxGroupNameLength {
return nil, ErrTooLongResourceGroupName
}
group := &rmpb.ResourceGroup{
Expand Down
1 change: 0 additions & 1 deletion domain/domain_sysvars.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,6 @@ func (do *Domain) setGlobalResourceControl(enable bool) {
} else {
variable.DisableGlobalResourceControlFunc()
}
logutil.BgLogger().Info("set resource control", zap.Bool("enable", enable))
}

// updatePDClient is used to set the dynamic option into the PD client.
Expand Down
1 change: 1 addition & 0 deletions executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -214,6 +214,7 @@ go_library(
"@com_github_pingcap_kvproto//pkg/encryptionpb",
"@com_github_pingcap_kvproto//pkg/kvrpcpb",
"@com_github_pingcap_kvproto//pkg/metapb",
"@com_github_pingcap_kvproto//pkg/resource_manager",
"@com_github_pingcap_kvproto//pkg/tikvpb",
"@com_github_pingcap_log//:log",
"@com_github_pingcap_sysutil//:sysutil",
Expand Down
41 changes: 31 additions & 10 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/deadlock"
rmpb "github.com/pingcap/kvproto/pkg/resource_manager"
"github.com/pingcap/tidb/ddl/label"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/domain/infosync"
Expand Down Expand Up @@ -185,7 +186,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex
case infoschema.ClusterTableMemoryUsageOpsHistory:
err = e.setDataForClusterMemoryUsageOpsHistory(sctx)
case infoschema.TableResourceGroups:
err = e.setDataFromResourceGroups(sctx)
err = e.setDataFromResourceGroups()
}
if err != nil {
return nil, err
Expand Down Expand Up @@ -3388,17 +3389,37 @@ func (e *memtableRetriever) setDataFromPlacementPolicies(sctx sessionctx.Context
return nil
}

func (e *memtableRetriever) setDataFromResourceGroups(sctx sessionctx.Context) error {
is := sessiontxn.GetTxnManager(sctx).GetTxnInfoSchema()
resourceGroups := is.AllResourceGroups()
func (e *memtableRetriever) setDataFromResourceGroups() error {
resourceGroups, err := infosync.GetAllResourceGroups(context.TODO())
if err != nil {
return errors.Errorf("failed to access resource group manager, error message is %s", err.Error())
}
rows := make([][]types.Datum, 0, len(resourceGroups))
for _, group := range resourceGroups {
row := types.MakeDatums(
group.ID,
group.Name.O,
group.RURate,
)
rows = append(rows, row)
//mode := ""
burstable := "NO"
switch group.Mode {
case rmpb.GroupMode_RUMode:
if group.RUSettings.RU.Settings.BurstLimit < 0 {
burstable = "YES"
}
row := types.MakeDatums(
group.Name,
group.RUSettings.RU.Settings.FillRate,
uint64(group.RUSettings.RU.Tokens),
burstable,
)
rows = append(rows, row)
default:
//mode = "UNKNOWN_MODE"
row := types.MakeDatums(
group.Name,
nil,
nil,
nil,
)
rows = append(rows, row)
}
}
e.rows = rows
return nil
Expand Down
22 changes: 11 additions & 11 deletions executor/simple.go
Original file line number Diff line number Diff line change
Expand Up @@ -1090,24 +1090,23 @@ func (e *SimpleExec) executeCreateUser(ctx context.Context, s *ast.CreateUserStm
userAttributes = append(userAttributes, fmt.Sprintf("\"metadata\": %s", s.CommentOrAttributeOption.Value))
}
}
resourceGroupName := "default"

if s.ResourceGroupNameOption != nil {
if !variable.EnableResourceControl.Load() {
return infoschema.ErrResourceGroupSupportDisabled
}
if s.ResourceGroupNameOption.Type == ast.UserResourceGroupName {
resourceGroupName = s.ResourceGroupNameOption.Value
}

resourceGroupName := strings.ToLower(s.ResourceGroupNameOption.Value)

// check if specified resource group exists
if resourceGroupName != "default" && resourceGroupName != "" {
_, exists := e.is.ResourceGroupByName(model.NewCIStr(resourceGroupName))
if !exists {
return infoschema.ErrResourceGroupNotExists
return infoschema.ErrResourceGroupNotExists.GenWithStackByArgs(resourceGroupName)
}
}
userAttributes = append(userAttributes, fmt.Sprintf("\"resource_group\": \"%s\"", resourceGroupName))
}
userAttributes = append(userAttributes, fmt.Sprintf("\"resource_group\": \"%s\"", resourceGroupName))
// If FAILED_LOGIN_ATTEMPTS and PASSWORD_LOCK_TIME are both specified to 0, a string of 0 length is generated.
// When inserting the attempts into json, an error occurs. This requires special handling.
if PasswordLocking != "" {
Expand Down Expand Up @@ -1904,20 +1903,21 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt)
newAttributes = append(newAttributes, fmt.Sprintf(`"metadata": %s`, s.CommentOrAttributeOption.Value))
}
}
if s.ResourceGroupNameOption != nil && s.ResourceGroupNameOption.Type == ast.UserResourceGroupName {
if s.ResourceGroupNameOption != nil {
if !variable.EnableResourceControl.Load() {
return infoschema.ErrResourceGroupSupportDisabled
}

// check if specified resource group exists
if s.ResourceGroupNameOption.Value != "default" && s.ResourceGroupNameOption.Value != "" {
_, exists := e.is.ResourceGroupByName(model.NewCIStr(s.ResourceGroupNameOption.Value))
resourceGroupName := strings.ToLower(s.ResourceGroupNameOption.Value)
if resourceGroupName != "default" && s.ResourceGroupNameOption.Value != "" {
_, exists := e.is.ResourceGroupByName(model.NewCIStr(resourceGroupName))
if !exists {
return infoschema.ErrResourceGroupNotExists
return infoschema.ErrResourceGroupNotExists.GenWithStackByArgs(resourceGroupName)
}
}

newAttributes = append(newAttributes, fmt.Sprintf(`"resource_group": "%s"`, s.ResourceGroupNameOption.Value))
newAttributes = append(newAttributes, fmt.Sprintf(`"resource_group": "%s"`, resourceGroupName))
}
if passwordLockingStr != "" {
newAttributes = append(newAttributes, passwordLockingStr)
Expand Down
8 changes: 4 additions & 4 deletions executor/simple_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,9 +94,9 @@ func TestUserAttributes(t *testing.T) {
_, err := rootTK.Exec(`CREATE USER testuser2 ATTRIBUTE '{"name": "Tom", age: 19}'`)
rootTK.MustExec(`CREATE USER testuser2`)
require.Error(t, err)
rootTK.MustQuery(`SELECT user_attributes FROM mysql.user WHERE user = 'testuser'`).Check(testkit.Rows(`{"metadata": {"comment": "1234"}, "resource_group": "default"}`))
rootTK.MustQuery(`SELECT user_attributes FROM mysql.user WHERE user = 'testuser1'`).Check(testkit.Rows(`{"metadata": {"age": 19, "name": "Tom"}, "resource_group": "default"}`))
rootTK.MustQuery(`SELECT user_attributes FROM mysql.user WHERE user = 'testuser2'`).Check(testkit.Rows(`{"resource_group": "default"}`))
rootTK.MustQuery(`SELECT user_attributes FROM mysql.user WHERE user = 'testuser'`).Check(testkit.Rows(`{"metadata": {"comment": "1234"}}`))
rootTK.MustQuery(`SELECT user_attributes FROM mysql.user WHERE user = 'testuser1'`).Check(testkit.Rows(`{"metadata": {"age": 19, "name": "Tom"}}`))
rootTK.MustQuery(`SELECT user_attributes FROM mysql.user WHERE user = 'testuser2'`).Check(testkit.Rows(`{}`))
rootTK.MustQueryWithContext(ctx, `SELECT attribute FROM information_schema.user_attributes WHERE user = 'testuser'`).Check(testkit.Rows(`{"comment": "1234"}`))
rootTK.MustQueryWithContext(ctx, `SELECT attribute FROM information_schema.user_attributes WHERE user = 'testuser1'`).Check(testkit.Rows(`{"age": 19, "name": "Tom"}`))
rootTK.MustQueryWithContext(ctx, `SELECT attribute->>"$.age" AS age, attribute->>"$.name" AS name FROM information_schema.user_attributes WHERE user = 'testuser1'`).Check(testkit.Rows(`19 Tom`))
Expand Down Expand Up @@ -127,7 +127,7 @@ func TestUserAttributes(t *testing.T) {
// https://github.com/pingcap/tidb/issues/39207
rootTK.MustExec("create user usr1@'%' identified by 'passord'")
rootTK.MustExec("alter user usr1 comment 'comment1'")
rootTK.MustQuery("select user_attributes from mysql.user where user = 'usr1'").Check(testkit.Rows(`{"metadata": {"comment": "comment1"}, "resource_group": "default"}`))
rootTK.MustQuery("select user_attributes from mysql.user where user = 'usr1'").Check(testkit.Rows(`{"metadata": {"comment": "comment1"}}`))
rootTK.MustExec("set global tidb_enable_resource_control = 'on'")
rootTK.MustExec("CREATE RESOURCE GROUP rg1 ru_per_sec = 100")
rootTK.MustExec("alter user usr1 resource group rg1")
Expand Down
Loading

0 comments on commit 975d177

Please sign in to comment.