Skip to content

Commit

Permalink
ddl, I_S: support runaway attribute in resource group (#43877)
Browse files Browse the repository at this point in the history
ref #43691
  • Loading branch information
CabinfeverB authored Jun 6, 2023
1 parent fcd20a3 commit a26691c
Show file tree
Hide file tree
Showing 12 changed files with 252 additions and 110 deletions.
54 changes: 45 additions & 9 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -3058,34 +3058,70 @@ func SetDirectPlacementOpt(placementSettings *model.PlacementSettings, placement
}

// SetDirectResourceGroupUnit tries to set the ResourceGroupSettings.
func SetDirectResourceGroupUnit(resourceGroupSettings *model.ResourceGroupSettings, typ ast.ResourceUnitType, stringVal string, uintVal uint64, boolValue bool) error {
switch typ {
func SetDirectResourceGroupUnit(resourceGroupSettings *model.ResourceGroupSettings, opt *ast.ResourceGroupOption) error {
switch opt.Tp {
case ast.ResourceRURate:
resourceGroupSettings.RURate = uintVal
resourceGroupSettings.RURate = opt.UintValue
case ast.ResourcePriority:
resourceGroupSettings.Priority = uintVal
resourceGroupSettings.Priority = opt.UintValue
case ast.ResourceUnitCPU:
resourceGroupSettings.CPULimiter = stringVal
resourceGroupSettings.CPULimiter = opt.StrValue
case ast.ResourceUnitIOReadBandwidth:
resourceGroupSettings.IOReadBandwidth = stringVal
resourceGroupSettings.IOReadBandwidth = opt.StrValue
case ast.ResourceUnitIOWriteBandwidth:
resourceGroupSettings.IOWriteBandwidth = stringVal
resourceGroupSettings.IOWriteBandwidth = opt.StrValue
case ast.ResourceBurstableOpiton:
// Some about BurstLimit(b):
// - If b == 0, that means the limiter is unlimited capacity. default use in resource controller (burst with a rate within a unlimited capacity).
// - If b < 0, that means the limiter is unlimited capacity and fillrate(r) is ignored, can be seen as r == Inf (burst with a inf rate within a unlimited capacity).
// - If b > 0, that means the limiter is limited capacity. (current not used).
limit := int64(0)
if boolValue {
if opt.BoolValue {
limit = -1
}
resourceGroupSettings.BurstLimit = limit
case ast.ResourceGroupRunaway:
for _, opt := range opt.ResourceGroupRunawayOptionList {
err := SetDirectResourceGroupRunawayOption(resourceGroupSettings, opt.Tp, opt.StrValue, opt.IntValue)
if err != nil {
return err
}
}
default:
return errors.Trace(errors.New("unknown resource unit type"))
}
return nil
}

// SetDirectResourceGroupRunawayOption tries to set runaway part of the ResourceGroupSettings.
func SetDirectResourceGroupRunawayOption(resourceGroupSettings *model.ResourceGroupSettings, typ ast.RunawayOptionType, stringVal string, intVal int32) error {
if resourceGroupSettings.Runaway == nil {
resourceGroupSettings.Runaway = &model.ResourceGroupRunawaySettings{}
}
settings := resourceGroupSettings.Runaway
switch typ {
case ast.RunawayRule:
// because execute time won't be too long, we use `time` pkg which does not support to parse unit 'd'.
dur, err := time.ParseDuration(stringVal)
if err != nil {
return err
}
settings.ExecElapsedTimeMs = uint64(dur.Milliseconds())
case ast.RunawayAction:
settings.Action = model.RunawayActionType(intVal)
case ast.RunawayWatch:
settings.WatchType = model.RunawayWatchType(intVal)
dur, err := time.ParseDuration(stringVal)
if err != nil {
return err
}
settings.WatchDurationMs = uint64(dur.Milliseconds())
default:
return errors.Trace(errors.New("unknown runaway option type"))
}
return nil
}

// handleTableOptions updates tableInfo according to table options.
func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) error {
var ttlOptionsHandled bool
Expand Down Expand Up @@ -8078,7 +8114,7 @@ func (d *ddl) DropResourceGroup(ctx sessionctx.Context, stmt *ast.DropResourceGr
func buildResourceGroup(oldGroup *model.ResourceGroupInfo, options []*ast.ResourceGroupOption) (*model.ResourceGroupInfo, error) {
groupInfo := &model.ResourceGroupInfo{Name: oldGroup.Name, ID: oldGroup.ID, ResourceGroupSettings: model.NewResourceGroupSettings()}
for _, opt := range options {
err := SetDirectResourceGroupUnit(groupInfo.ResourceGroupSettings, opt.Tp, opt.StrValue, opt.UintValue, opt.BoolValue)
err := SetDirectResourceGroupUnit(groupInfo.ResourceGroupSettings, opt)
if err != nil {
return nil, err
}
Expand Down
4 changes: 4 additions & 0 deletions ddl/resourcegroup/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,4 +31,8 @@ var (
ErrUnknownResourceGroupMode = errors.New("unknown resource group mode")
// ErrDroppingInternalResourceGroup is from group.go
ErrDroppingInternalResourceGroup = errors.New("can't drop reserved resource group")
// ErrInvalidResourceGroupRunawayExecElapsedTime is from group.go.
ErrInvalidResourceGroupRunawayExecElapsedTime = errors.New("invalid exec elapsed time")
// ErrUnknownResourceGroupRunawayAction is from group.go.
ErrUnknownResourceGroupRunawayAction = errors.New("unknown resource group runaway action")
)
23 changes: 22 additions & 1 deletion ddl/resourcegroup/group.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,28 @@ func NewGroupFromOptions(groupName string, options *model.ResourceGroupSettings)
group := &rmpb.ResourceGroup{
Name: groupName,
}

group.Priority = uint32(options.Priority)
if options.Runaway != nil {
runaway := &rmpb.RunawaySettings{
Rule: &rmpb.RunawayRule{},
}
if options.Runaway.ExecElapsedTimeMs == 0 {
return nil, ErrInvalidResourceGroupRunawayExecElapsedTime
}
runaway.Rule.ExecElapsedTimeMs = options.Runaway.ExecElapsedTimeMs
if options.Runaway.Action == 0 {
return nil, ErrUnknownResourceGroupRunawayAction
}
runaway.Action = rmpb.RunawayAction(options.Runaway.Action)
if options.Runaway.WatchDurationMs > 0 {
runaway.Watch = &rmpb.RunawayWatch{}
runaway.Watch.Type = rmpb.RunawayWatchType(options.Runaway.WatchType)
runaway.Watch.LastingDurationMs = options.Runaway.WatchDurationMs
}
group.RunawaySettings = runaway
}

if options.RURate > 0 {
group.Mode = rmpb.GroupMode_RUMode
group.RUSettings = &rmpb.GroupRequestUnitSettings{
Expand All @@ -44,7 +66,6 @@ func NewGroupFromOptions(groupName string, options *model.ResourceGroupSettings)
},
},
}
group.Priority = uint32(options.Priority)
if len(options.CPULimiter) > 0 || len(options.IOReadBandwidth) > 0 || len(options.IOWriteBandwidth) > 0 {
return nil, ErrInvalidResourceGroupDuplicatedMode
}
Expand Down
59 changes: 41 additions & 18 deletions ddl/resourcegrouptest/resource_group_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"fmt"
"sync/atomic"
"testing"
"time"

"github.com/pingcap/failpoint"
rmpb "github.com/pingcap/kvproto/pkg/resource_manager"
Expand Down Expand Up @@ -60,9 +61,9 @@ func TestResourceGroupBasic(t *testing.T) {
tk.MustExec("set global tidb_enable_resource_control = 'on'")

// test default resource group.
tk.MustQuery("select * from information_schema.resource_groups where name = 'default'").Check(testkit.Rows("default UNLIMITED MEDIUM YES"))
tk.MustQuery("select * from information_schema.resource_groups where name = 'default'").Check(testkit.Rows("default UNLIMITED MEDIUM YES <nil>"))
tk.MustExec("alter resource group `default` RU_PER_SEC=1000 PRIORITY=LOW")
tk.MustQuery("select * from information_schema.resource_groups where name = 'default'").Check(testkit.Rows("default 1000 LOW NO"))
tk.MustQuery("select * from information_schema.resource_groups where name = 'default'").Check(testkit.Rows("default 1000 LOW NO <nil>"))
tk.MustContainErrMsg("drop resource group `default`", "can't drop reserved resource group")

tk.MustExec("create resource group x RU_PER_SEC=1000")
Expand All @@ -71,6 +72,7 @@ func TestResourceGroupBasic(t *testing.T) {
require.Equal(t, "x", groupInfo.Name.L)
require.Equal(t, groupID.Load(), groupInfo.ID)
require.Equal(t, uint64(1000), groupInfo.RURate)
require.Nil(t, groupInfo.Runaway)
}
// Check the group is correctly reloaded in the information schema.
g := testResourceGroupNameFromIS(t, tk.Session(), "x")
Expand All @@ -93,12 +95,16 @@ func TestResourceGroupBasic(t *testing.T) {

tk.MustGetErrCode("create resource group x RU_PER_SEC=1000 ", mysql.ErrResourceGroupExists)

tk.MustExec("alter resource group x RU_PER_SEC=2000 BURSTABLE")
tk.MustExec("alter resource group x RU_PER_SEC=2000 BURSTABLE QUERY_LIMIT=(EXEC_ELAPSED='15s' ACTION COOLDOWN WATCH SIMILAR DURATION '10m')")
g = testResourceGroupNameFromIS(t, tk.Session(), "x")
re.Equal(uint64(2000), g.RURate)
re.Equal(int64(-1), g.BurstLimit)
re.Equal(uint64(time.Second*15/time.Millisecond), g.Runaway.ExecElapsedTimeMs)
re.Equal(model.RunawayActionCooldown, g.Runaway.Action)
re.Equal(model.WatchSimilar, g.Runaway.WatchType)
re.Equal(uint64(time.Minute*10/time.Millisecond), g.Runaway.WatchDurationMs)

tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x 2000 MEDIUM YES"))
tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x 2000 MEDIUM YES EXEC_ELAPSED=15s, ACTION=COOLDOWN, WATCH=SIMILAR[10m0s]"))

tk.MustExec("drop resource group x")
g = testResourceGroupNameFromIS(t, tk.Session(), "x")
Expand All @@ -119,16 +125,20 @@ func TestResourceGroupBasic(t *testing.T) {
}
g = testResourceGroupNameFromIS(t, tk.Session(), "y")
checkFunc(g)
tk.MustExec("alter resource group y BURSTABLE RU_PER_SEC=5000")
tk.MustExec("alter resource group y BURSTABLE RU_PER_SEC=5000 QUERY_LIMIT=(EXEC_ELAPSED='15s' ACTION KILL)")
checkFunc = func(groupInfo *model.ResourceGroupInfo) {
re.Equal(true, groupInfo.ID != 0)
re.Equal("y", groupInfo.Name.L)
re.Equal(groupID.Load(), groupInfo.ID)
re.Equal(uint64(5000), groupInfo.RURate)
re.Equal(int64(-1), groupInfo.BurstLimit)
re.Equal(uint64(time.Second*15/time.Millisecond), groupInfo.Runaway.ExecElapsedTimeMs)
re.Equal(model.RunawayActionKill, groupInfo.Runaway.Action)
re.Equal(uint64(0), groupInfo.Runaway.WatchDurationMs)
}
g = testResourceGroupNameFromIS(t, tk.Session(), "y")
checkFunc(g)
tk.MustQuery("select * from information_schema.resource_groups where name = 'y'").Check(testkit.Rows("y 5000 MEDIUM YES EXEC_ELAPSED=15s, ACTION=KILL"))
tk.MustExec("drop resource group y")
g = testResourceGroupNameFromIS(t, tk.Session(), "y")
re.Nil(g)
Expand All @@ -141,32 +151,45 @@ func TestResourceGroupBasic(t *testing.T) {
tk.MustContainErrMsg("create resource group x ru_per_sec=1000, burstable, burstable", "Dupliated options specified")
tk.MustGetErrCode("create resource group x burstable, ru_per_sec=1000, burstable", mysql.ErrParse)
tk.MustContainErrMsg("create resource group x burstable, ru_per_sec=1000, burstable", "Dupliated options specified")
tk.MustContainErrMsg("create resource group x ru_per_sec=1000 burstable QUERY_LIMIT=(EXEC_ELAPSED='15s' action kill action cooldown)", "Dupliated runaway options specified")
tk.MustContainErrMsg("create resource group x ru_per_sec=1000 QUERY_LIMIT=(EXEC_ELAPSED='15s') burstable priority=Low, QUERY_LIMIT=(EXEC_ELAPSED='15s')", "Dupliated options specified")
tk.MustContainErrMsg("create resource group x ru_per_sec=1000 QUERY_LIMIT=(EXEC_ELAPSED='15s') QUERY_LIMIT=(EXEC_ELAPSED='15s')", "Dupliated options specified")
tk.MustContainErrMsg("create resource group x ru_per_sec=1000 QUERY_LIMIT=(action kill)", "invalid exec elapsed time")
tk.MustGetErrCode("create resource group x ru_per_sec=1000 QUERY_LIMIT=(EXEC_ELAPSED='15s' action kil)", mysql.ErrParse)
tk.MustContainErrMsg("create resource group x ru_per_sec=1000 QUERY_LIMIT=(EXEC_ELAPSED='15s')", "unknown resource group runaway action")
tk.MustGetErrCode("create resource group x ru_per_sec=1000 EXEC_ELAPSED='15s' action kill", mysql.ErrParse)
tk.MustContainErrMsg("create resource group x ru_per_sec=1000 QUERY_LIMIT=(EXEC_ELAPSED='15d' action kill)", "unknown unit \"d\"")
groups, err := infosync.ListResourceGroups(context.TODO())
re.Equal(1, len(groups))
re.NoError(err)

// Check information schema table information_schema.resource_groups
tk.MustExec("create resource group x RU_PER_SEC=1000 PRIORITY=LOW")
tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x 1000 LOW NO"))
tk.MustExec("alter resource group x RU_PER_SEC=2000 BURSTABLE")
tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x 2000 MEDIUM YES"))
tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x 1000 LOW NO <nil>"))
tk.MustExec("alter resource group x RU_PER_SEC=2000 BURSTABLE QUERY_LIMIT=(EXEC_ELAPSED='15s' action kill)")
tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x 2000 MEDIUM YES EXEC_ELAPSED=15s, ACTION=KILL"))
tk.MustQuery("show create resource group x").Check(testkit.Rows("x CREATE RESOURCE GROUP `x` RU_PER_SEC=2000, PRIORITY=MEDIUM, BURSTABLE, QUERY_LIMIT=(EXEC_ELAPSED=\"15s\" ACTION=KILL)"))
tk.MustExec("CREATE RESOURCE GROUP `x_new` RU_PER_SEC=2000 PRIORITY=MEDIUM BURSTABLE QUERY_LIMIT=(EXEC_ELAPSED=\"15s\" ACTION=KILL)")
tk.MustQuery("select * from information_schema.resource_groups where name = 'x_new'").Check(testkit.Rows("x_new 2000 MEDIUM YES EXEC_ELAPSED=15s, ACTION=KILL"))
tk.MustExec("alter resource group x BURSTABLE RU_PER_SEC=3000")
tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x 3000 MEDIUM YES"))
tk.MustQuery("show create resource group x").Check(testkit.Rows("x CREATE RESOURCE GROUP `x` RU_PER_SEC=3000 PRIORITY=MEDIUM BURSTABLE"))
tk.MustQuery("select * from information_schema.resource_groups where name = 'x'").Check(testkit.Rows("x 3000 MEDIUM YES <nil>"))
tk.MustQuery("show create resource group x").Check(testkit.Rows("x CREATE RESOURCE GROUP `x` RU_PER_SEC=3000, PRIORITY=MEDIUM, BURSTABLE"))

tk.MustExec("create resource group y BURSTABLE RU_PER_SEC=2000")
tk.MustQuery("select * from information_schema.resource_groups where name = 'y'").Check(testkit.Rows("y 2000 MEDIUM YES"))
tk.MustQuery("show create resource group y").Check(testkit.Rows("y CREATE RESOURCE GROUP `y` RU_PER_SEC=2000 PRIORITY=MEDIUM BURSTABLE"))
tk.MustExec("create resource group y BURSTABLE RU_PER_SEC=2000 QUERY_LIMIT=(EXEC_ELAPSED='1s' action COOLDOWN WATCH EXACT duration '1h')")
tk.MustQuery("select * from information_schema.resource_groups where name = 'y'").Check(testkit.Rows("y 2000 MEDIUM YES EXEC_ELAPSED=1s, ACTION=COOLDOWN, WATCH=EXACT[1h0m0s]"))
tk.MustQuery("show create resource group y").Check(testkit.Rows("y CREATE RESOURCE GROUP `y` RU_PER_SEC=2000, PRIORITY=MEDIUM, BURSTABLE, QUERY_LIMIT=(EXEC_ELAPSED=\"1s\" ACTION=COOLDOWN WATCH=EXACT DURATION=\"1h0m0s\")"))
tk.MustExec("CREATE RESOURCE GROUP `y_new` RU_PER_SEC=2000 PRIORITY=MEDIUM, BURSTABLE, QUERY_LIMIT=(EXEC_ELAPSED=\"1s\" ACTION=COOLDOWN WATCH=EXACT DURATION=\"1h0m0s\")")
tk.MustQuery("select * from information_schema.resource_groups where name = 'y_new'").Check(testkit.Rows("y_new 2000 MEDIUM YES EXEC_ELAPSED=1s, ACTION=COOLDOWN, WATCH=EXACT[1h0m0s]"))

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 MEDIUM YES"))
tk.MustQuery("show create resource group y").Check(testkit.Rows("y CREATE RESOURCE GROUP `y` RU_PER_SEC=4000 PRIORITY=MEDIUM BURSTABLE"))
tk.MustQuery("select * from information_schema.resource_groups where name = 'y'").Check(testkit.Rows("y 4000 MEDIUM YES <nil>"))
tk.MustQuery("show create resource group y").Check(testkit.Rows("y CREATE RESOURCE GROUP `y` RU_PER_SEC=4000, PRIORITY=MEDIUM, BURSTABLE"))

tk.MustExec("alter resource group y RU_PER_SEC=4000 PRIORITY=HIGH BURSTABLE")
tk.MustQuery("select * from information_schema.resource_groups where name = 'y'").Check(testkit.Rows("y 4000 HIGH YES"))
tk.MustQuery("show create resource group y").Check(testkit.Rows("y CREATE RESOURCE GROUP `y` RU_PER_SEC=4000 PRIORITY=HIGH BURSTABLE"))
tk.MustQuery("select * from information_schema.resource_groups where name = 'y'").Check(testkit.Rows("y 4000 HIGH YES <nil>"))
tk.MustQuery("show create resource group y").Check(testkit.Rows("y CREATE RESOURCE GROUP `y` RU_PER_SEC=4000, PRIORITY=HIGH, BURSTABLE"))

tk.MustQuery("select count(*) from information_schema.resource_groups").Check(testkit.Rows("3"))
tk.MustQuery("select count(*) from information_schema.resource_groups").Check(testkit.Rows("5"))
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")
Expand Down
36 changes: 33 additions & 3 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -3235,6 +3235,13 @@ func (e *memtableRetriever) setDataFromPlacementPolicies(sctx sessionctx.Context
return nil
}

// used in resource_groups
const (
burstableStr = "YES"
burstdisableStr = "NO"
unlimitedFillRate = "UNLIMITED"
)

func (e *memtableRetriever) setDataFromResourceGroups() error {
resourceGroups, err := infosync.ListResourceGroups(context.TODO())
if err != nil {
Expand All @@ -3243,24 +3250,46 @@ func (e *memtableRetriever) setDataFromResourceGroups() error {
rows := make([][]types.Datum, 0, len(resourceGroups))
for _, group := range resourceGroups {
//mode := ""
burstable := "NO"
burstable := burstdisableStr
priority := model.PriorityValueToName(uint64(group.Priority))
fillrate := "UNLIMITED"
fillrate := unlimitedFillRate
isDefaultInReservedSetting := group.Name == "default" && group.RUSettings.RU.Settings.FillRate == math.MaxInt32
if !isDefaultInReservedSetting {
fillrate = strconv.FormatUint(group.RUSettings.RU.Settings.FillRate, 10)
}
// convert runaway settings
queryLimit := ""
if setting := group.RunawaySettings; setting != nil {
runawayRule, runawayAction, runawayWatch := "", "", ""
if setting.Rule == nil {
return errors.Errorf("unexpected runaway config in resource group")
}
dur := time.Duration(setting.Rule.ExecElapsedTimeMs) * time.Millisecond
runawayRule = fmt.Sprintf("%s=%s", "EXEC_ELAPSED", dur.String())
runawayAction = fmt.Sprintf("%s=%s", "ACTION", model.RunawayActionType(setting.Action).String())
if setting.Watch != nil {
dur := time.Duration(setting.Watch.LastingDurationMs) * time.Millisecond
runawayWatch = fmt.Sprintf("%s=%s[%s]", "WATCH", model.RunawayWatchType(setting.Watch.Type).String(), dur.String())
queryLimit = fmt.Sprintf("%s, %s, %s", runawayRule, runawayAction, runawayWatch)
} else {
queryLimit = fmt.Sprintf("%s, %s", runawayRule, runawayAction)
}
}
switch group.Mode {
case rmpb.GroupMode_RUMode:
if group.RUSettings.RU.Settings.BurstLimit < 0 {
burstable = "YES"
burstable = burstableStr
}
row := types.MakeDatums(
group.Name,
fillrate,
priority,
burstable,
queryLimit,
)
if len(queryLimit) == 0 {
row[4].SetNull()
}
rows = append(rows, row)
default:
//mode = "UNKNOWN_MODE"
Expand All @@ -3269,6 +3298,7 @@ func (e *memtableRetriever) setDataFromResourceGroups() error {
nil,
nil,
nil,
nil,
)
rows = append(rows, row)
}
Expand Down
1 change: 1 addition & 0 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -1601,6 +1601,7 @@ var tableResourceGroupsCols = []columnInfo{
{name: "RU_PER_SEC", tp: mysql.TypeVarchar, size: 21},
{name: "PRIORITY", tp: mysql.TypeVarchar, size: 6},
{name: "BURSTABLE", tp: mysql.TypeVarchar, size: 3},
{name: "QUERY_LIMIT", tp: mysql.TypeVarchar, size: 256},
}

// GetShardingInfo returns a nil or description string for the sharding information of given TableInfo.
Expand Down
22 changes: 20 additions & 2 deletions parser/ast/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -2203,13 +2203,13 @@ func (n *ResourceGroupRunawayOption) Restore(ctx *format.RestoreCtx) error {
fn := func() error {
switch n.Tp {
case RunawayRule:
ctx.WriteKeyWord("EXEC_ELAPSED_IN_SEC ")
ctx.WriteKeyWord("EXEC_ELAPSED ")
ctx.WritePlain("= ")
ctx.WriteString(n.StrValue)
case RunawayAction:
ctx.WriteKeyWord("ACTION ")
ctx.WritePlain("= ")
ctx.WriteKeyWord(model.RunawayActionValueToName(n.IntValue))
ctx.WriteKeyWord(model.RunawayActionType(n.IntValue).String())
case RunawayWatch:
ctx.WriteKeyWord("WATCH ")
ctx.WritePlain("= ")
Expand Down Expand Up @@ -4520,6 +4520,24 @@ func (n *AlterPlacementPolicyStmt) Accept(v Visitor) (Node, bool) {
return v.Leave(n)
}

func CheckAppend(ops []*ResourceGroupOption, newOp *ResourceGroupOption) bool {
for _, op := range ops {
if op.Tp == newOp.Tp {
return false
}
}
return true
}

func CheckRunawayAppend(ops []*ResourceGroupRunawayOption, newOp *ResourceGroupRunawayOption) bool {
for _, op := range ops {
if op.Tp == newOp.Tp {
return false
}
}
return true
}

// AlterResourceGroupStmt is a statement to alter placement policy option.
type AlterResourceGroupStmt struct {
ddlNode
Expand Down
Loading

0 comments on commit a26691c

Please sign in to comment.