Skip to content

Commit

Permalink
Merge branch 'master' into metabuildctx
Browse files Browse the repository at this point in the history
  • Loading branch information
lcwangchao committed Sep 25, 2024
2 parents 6193d60 + bf455f5 commit 86c612e
Show file tree
Hide file tree
Showing 44 changed files with 11,694 additions and 10,980 deletions.
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1561,6 +1561,11 @@ error = '''
Global Index is needed for index '%-.192s', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption
'''

["ddl:8265"]
error = '''
Auto analyze is not effective for index '%-.192s', need analyze manually
'''

["domain:8027"]
error = '''
Information schema is out of date: schema failed to update in 1 lease, please make sure TiDB can connect to TiKV
Expand Down
6 changes: 3 additions & 3 deletions pkg/ddl/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -406,7 +406,7 @@ func buildTableInfoWithCheck(ctx *metabuild.Context, s *ast.CreateTableStmt, dbC
if err = checkTableInfoValidWithStmt(ctx, tbInfo, s); err != nil {
return nil, err
}
if err = checkTableInfoValidExtra(tbInfo); err != nil {
if err = checkTableInfoValidExtra(ctx, tbInfo); err != nil {
return nil, err
}
return tbInfo, nil
Expand Down Expand Up @@ -512,7 +512,7 @@ func checkGeneratedColumn(ctx *metabuild.Context, schemaName pmodel.CIStr, table
// name length and column count.
// (checkTableInfoValid is also used in repairing objects which don't perform
// these checks. Perhaps the two functions should be merged together regardless?)
func checkTableInfoValidExtra(tbInfo *model.TableInfo) error {
func checkTableInfoValidExtra(ctx sessionctx.Context, tbInfo *model.TableInfo) error {
if err := checkTooLongTable(tbInfo.Name); err != nil {
return err
}
Expand All @@ -532,7 +532,7 @@ func checkTableInfoValidExtra(tbInfo *model.TableInfo) error {
if err := checkColumnsAttributes(tbInfo.Columns); err != nil {
return errors.Trace(err)
}
if err := checkGlobalIndexes(tbInfo); err != nil {
if err := checkGlobalIndexes(ctx, tbInfo); err != nil {
return errors.Trace(err)
}

Expand Down
49 changes: 39 additions & 10 deletions pkg/ddl/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -939,7 +939,7 @@ func checkInvisibleIndexOnPK(tblInfo *model.TableInfo) error {
}

// checkGlobalIndex check if the index is allowed to have global index
func checkGlobalIndex(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error {
func checkGlobalIndex(ctx sessionctx.Context, tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error {
pi := tblInfo.GetPartitionInfo()
isPartitioned := pi != nil && pi.Type != pmodel.PartitionTypeNone
if indexInfo.Global {
Expand All @@ -961,14 +961,15 @@ func checkGlobalIndex(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) erro
if inAllPartitionColumns {
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global Index including all columns in the partitioning expression")
}
validateGlobalIndexWithGeneratedColumns(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexInfo.Name.O, indexInfo.Columns)
}
return nil
}

// checkGlobalIndexes check if global index is supported.
func checkGlobalIndexes(tblInfo *model.TableInfo) error {
func checkGlobalIndexes(ctx sessionctx.Context, tblInfo *model.TableInfo) error {
for _, indexInfo := range tblInfo.Indices {
err := checkGlobalIndex(tblInfo, indexInfo)
err := checkGlobalIndex(ctx, tblInfo, indexInfo)
if err != nil {
return err
}
Expand Down Expand Up @@ -1078,7 +1079,7 @@ func (e *executor) createTableWithInfoJob(
}
}

if err := checkTableInfoValidExtra(tbInfo); err != nil {
if err := checkTableInfoValidExtra(ctx, tbInfo); err != nil {
return nil, err
}

Expand Down Expand Up @@ -1342,17 +1343,22 @@ func (e *executor) CreatePlacementPolicyWithInfo(ctx sessionctx.Context, policy
policy.ID = policyID

job := &model.Job{
Version: model.GetJobVerInUse(),
SchemaName: policy.Name.L,
Type: model.ActionCreatePlacementPolicy,
BinlogInfo: &model.HistoryInfo{},
Args: []any{policy, onExist == OnExistReplace},
InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{
Policy: policy.Name.L,
}},
CDCWriteSource: ctx.GetSessionVars().CDCWriteSource,
SQLMode: ctx.GetSessionVars().SQLMode,
}
err = e.DoDDLJob(ctx, job)

args := &model.PlacementPolicyArgs{
Policy: policy,
ReplaceOnExist: onExist == OnExistReplace,
}
err = e.doDDLJob2(ctx, job, args)
return errors.Trace(err)
}

Expand Down Expand Up @@ -4542,6 +4548,7 @@ func (e *executor) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexN
if indexOption == nil || !indexOption.Global {
return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs("PRIMARY")
}
validateGlobalIndexWithGeneratedColumns(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexName.O, indexColumns)
}
}

Expand Down Expand Up @@ -4702,6 +4709,7 @@ func (e *executor) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast
if !globalIndex {
return dbterror.ErrGlobalIndexNotExplicitlySet.GenWithStackByArgs(indexName.O)
}
validateGlobalIndexWithGeneratedColumns(ctx.GetSessionVars().StmtCtx.ErrCtx(), tblInfo, indexName.O, indexColumns)
} else if globalIndex {
// TODO: remove this restriction
return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("Global IndexOption on index including all columns in the partitioning expression")
Expand Down Expand Up @@ -5151,6 +5159,18 @@ func validateCommentLength(ec errctx.Context, sqlMode mysql.SQLMode, name string
return *comment, nil
}

func validateGlobalIndexWithGeneratedColumns(ec errctx.Context, tblInfo *model.TableInfo, indexName string, indexColumns []*model.IndexColumn) {
// Auto analyze is not effective when a global index contains prefix columns or virtual generated columns.
for _, col := range indexColumns {
colInfo := tblInfo.Columns[col.Offset]
isPrefixCol := col.Length != types.UnspecifiedLength
if colInfo.IsVirtualGenerated() || isPrefixCol {
ec.AppendWarning(dbterror.ErrWarnGlobalIndexNeedManuallyAnalyze.FastGenByArgs(indexName))
return
}
}
}

// BuildAddedPartitionInfo build alter table add partition info
func BuildAddedPartitionInfo(ctx expression.BuildContext, meta *model.TableInfo, spec *ast.AlterTableSpec) (*model.PartitionInfo, error) {
numParts := uint64(0)
Expand Down Expand Up @@ -5967,18 +5987,23 @@ func (e *executor) DropPlacementPolicy(ctx sessionctx.Context, stmt *ast.DropPla
}

job := &model.Job{
Version: model.GetJobVerInUse(),
SchemaID: policy.ID,
SchemaName: policy.Name.L,
Type: model.ActionDropPlacementPolicy,
BinlogInfo: &model.HistoryInfo{},
CDCWriteSource: ctx.GetSessionVars().CDCWriteSource,
Args: []any{policyName},
InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{
Policy: policyName.L,
}},
SQLMode: ctx.GetSessionVars().SQLMode,
}
err = e.DoDDLJob(ctx, job)

args := &model.PlacementPolicyArgs{
PolicyName: policyName,
PolicyID: policy.ID,
}
err = e.doDDLJob2(ctx, job, args)
return errors.Trace(err)
}

Expand All @@ -6005,18 +6030,22 @@ func (e *executor) AlterPlacementPolicy(ctx sessionctx.Context, stmt *ast.AlterP
}

job := &model.Job{
Version: model.GetJobVerInUse(),
SchemaID: policy.ID,
SchemaName: policy.Name.L,
Type: model.ActionAlterPlacementPolicy,
BinlogInfo: &model.HistoryInfo{},
CDCWriteSource: ctx.GetSessionVars().CDCWriteSource,
Args: []any{newPolicyInfo},
InvolvingSchemaInfo: []model.InvolvingSchemaInfo{{
Policy: newPolicyInfo.Name.L,
}},
SQLMode: ctx.GetSessionVars().SQLMode,
}
err = e.DoDDLJob(ctx, job)
args := &model.PlacementPolicyArgs{
Policy: newPolicyInfo,
PolicyID: policy.ID,
}
err = e.doDDLJob2(ctx, job, args)
return errors.Trace(err)
}

Expand Down
21 changes: 13 additions & 8 deletions pkg/ddl/placement_policy.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,12 +34,13 @@ import (
)

func onCreatePlacementPolicy(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, _ error) {
policyInfo := &model.PolicyInfo{}
var orReplace bool
if err := job.DecodeArgs(policyInfo, &orReplace); err != nil {
args, err := model.GetPlacementPolicyArgs(job)
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
policyInfo, orReplace := args.Policy, args.ReplaceOnExist

policyInfo.State = model.StateNone

if err := checkPolicyValidation(policyInfo.PlacementSettings); err != nil {
Expand Down Expand Up @@ -183,7 +184,11 @@ func checkAllTablePlacementPoliciesExistAndCancelNonExistJob(t *meta.Meta, job *
}

func onDropPlacementPolicy(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, _ error) {
policyInfo, err := checkPlacementPolicyExistAndCancelNonExistJob(t, job, job.SchemaID)
args, err := model.GetPlacementPolicyArgs(job)
if err != nil {
return ver, errors.Trace(err)
}
policyInfo, err := checkPlacementPolicyExistAndCancelNonExistJob(t, job, args.PolicyID)
if err != nil {
return ver, errors.Trace(err)
}
Expand Down Expand Up @@ -241,19 +246,19 @@ func onDropPlacementPolicy(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ve
}

func onAlterPlacementPolicy(jobCtx *jobContext, t *meta.Meta, job *model.Job) (ver int64, _ error) {
alterPolicy := &model.PolicyInfo{}
if err := job.DecodeArgs(alterPolicy); err != nil {
args, err := model.GetPlacementPolicyArgs(job)
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}

oldPolicy, err := checkPlacementPolicyExistAndCancelNonExistJob(t, job, job.SchemaID)
oldPolicy, err := checkPlacementPolicyExistAndCancelNonExistJob(t, job, args.PolicyID)
if err != nil {
return ver, errors.Trace(err)
}

newPolicyInfo := *oldPolicy
newPolicyInfo.PlacementSettings = alterPolicy.PlacementSettings
newPolicyInfo.PlacementSettings = args.Policy.PlacementSettings

err = checkPolicyValidation(newPolicyInfo.PlacementSettings)
if err != nil {
Expand Down
8 changes: 6 additions & 2 deletions pkg/ddl/placement_policy_ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,13 +44,17 @@ func testPlacementPolicyInfo(t *testing.T, store kv.Storage, name string, settin

func testCreatePlacementPolicy(t *testing.T, ctx sessionctx.Context, d ddl.ExecutorForTest, policyInfo *model.PolicyInfo) *model.Job {
job := &model.Job{
Version: model.GetJobVerInUse(),
SchemaName: policyInfo.Name.L,
Type: model.ActionCreatePlacementPolicy,
BinlogInfo: &model.HistoryInfo{},
Args: []any{policyInfo},
}
args := &model.PlacementPolicyArgs{
Policy: policyInfo,
}

ctx.SetValue(sessionctx.QueryString, "skip")
err := d.DoDDLJob(ctx, job)
err := d.DoDDLJobWrapper(ctx, ddl.NewJobWrapperWithArgs(job, args, false))
require.NoError(t, err)

v := getSchemaVer(t, ctx)
Expand Down
10 changes: 7 additions & 3 deletions pkg/ddl/resource_group.go
Original file line number Diff line number Diff line change
Expand Up @@ -238,6 +238,8 @@ func SetDirectResourceGroupSettings(groupInfo *model.ResourceGroupInfo, opt *ast
if len(opt.BackgroundOptions) == 0 {
resourceGroupSettings.Background = nil
}
resourceGroupSettings.Background = &model.ResourceGroupBackgroundSettings{}

for _, opt := range opt.BackgroundOptions {
if err := SetDirectResourceGroupBackgroundOption(resourceGroupSettings, opt); err != nil {
return err
Expand Down Expand Up @@ -300,16 +302,18 @@ func SetDirectResourceGroupRunawayOption(resourceGroupSettings *model.ResourceGr

// SetDirectResourceGroupBackgroundOption set background configs of the ResourceGroupSettings.
func SetDirectResourceGroupBackgroundOption(resourceGroupSettings *model.ResourceGroupSettings, opt *ast.ResourceGroupBackgroundOption) error {
if resourceGroupSettings.Background == nil {
resourceGroupSettings.Background = &model.ResourceGroupBackgroundSettings{}
}
switch opt.Type {
case ast.BackgroundOptionTaskNames:
jobTypes, err := parseBackgroundJobTypes(opt.StrValue)
if err != nil {
return err
}
resourceGroupSettings.Background.JobTypes = jobTypes
case ast.BackgroundUtilizationLimit:
if opt.UintValue == 0 || opt.UintValue > 100 {
return errors.Trace(errors.New("invalid background resource utilization limit, the valid range is (0, 100]"))
}
resourceGroupSettings.Background.ResourceUtilLimit = opt.UintValue
default:
return errors.Trace(errors.New("unknown background option type"))
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/ddl/resourcegroup/group.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,8 @@ func NewGroupFromOptions(groupName string, options *model.ResourceGroupSettings)

if options.Background != nil {
group.BackgroundSettings = &rmpb.BackgroundSettings{
JobTypes: options.Background.JobTypes,
JobTypes: options.Background.JobTypes,
UtilizationLimit: options.Background.ResourceUtilLimit,
}
}

Expand Down
2 changes: 2 additions & 0 deletions pkg/errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -1143,6 +1143,8 @@ const (

ErrGlobalIndexNotExplicitlySet = 8264

ErrWarnGlobalIndexNeedManuallyAnalyze = 8265

// Resource group errors.
ErrResourceGroupExists = 8248
ErrResourceGroupNotExists = 8249
Expand Down
2 changes: 2 additions & 0 deletions pkg/errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1174,4 +1174,6 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrBDRRestrictedDDL: mysql.Message("The operation is not allowed while the bdr role of this cluster is set to %s.", nil),

ErrGlobalIndexNotExplicitlySet: mysql.Message("Global Index is needed for index '%-.192s', since the unique index is not including all partitioning columns, and GLOBAL is not given as IndexOption", nil),

ErrWarnGlobalIndexNeedManuallyAnalyze: mysql.Message("Auto analyze is not effective for index '%-.192s', need analyze manually", nil),
}
3 changes: 1 addition & 2 deletions pkg/executor/analyze_col_v2.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,9 +76,8 @@ func (e *AnalyzeColumnsExecV2) analyzeColumnsPushDownV2(gp *gp.Pool) *statistics
isSpecial := false
for _, col := range idx.Columns {
colInfo := e.colsInfo[col.Offset]
isVirtualCol := colInfo.IsGenerated() && !colInfo.GeneratedStored
isPrefixCol := col.Length != types.UnspecifiedLength
if isVirtualCol || isPrefixCol {
if colInfo.IsVirtualGenerated() || isPrefixCol {
isSpecial = true
break
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/analyze_idx.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,8 +89,8 @@ func analyzeIndexPushdown(idxExec *AnalyzeIndexExec) *statistics.AnalyzeResults
Count: cnt,
Snapshot: idxExec.snapshot,
}
if idxExec.idxInfo.MVIndex {
result.ForMVIndex = true
if idxExec.idxInfo.MVIndex || (idxExec.idxInfo.Global && statsVer == statistics.Version2) {
result.ForMVIndexOrGlobalIndex = true
}
return result
}
Expand Down
18 changes: 18 additions & 0 deletions pkg/executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2747,6 +2747,9 @@ func (e *RecommendIndexExec) Next(ctx context.Context, req *chunk.Chunk) error {
if e.Action == "set" {
return indexadvisor.SetOption(e.Ctx(), e.Option, e.Value)
}
if e.Action == "show" {
return e.showOptions(req)
}

if e.Action != "run" {
return fmt.Errorf("unsupported action: %s", e.Action)
Expand Down Expand Up @@ -2775,3 +2778,18 @@ func (e *RecommendIndexExec) Next(ctx context.Context, req *chunk.Chunk) error {
}
return err
}

func (e *RecommendIndexExec) showOptions(req *chunk.Chunk) error {
vals, desc, err := indexadvisor.GetOptions(e.Ctx(), indexadvisor.AllOptions...)
if err != nil {
return err
}
for _, opt := range indexadvisor.AllOptions {
if v, ok := vals[opt]; ok {
req.AppendString(0, opt)
req.AppendString(1, v)
req.AppendString(2, desc[opt])
}
}
return nil
}
12 changes: 11 additions & 1 deletion pkg/executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -3737,7 +3737,17 @@ func (e *memtableRetriever) setDataFromResourceGroups() error {
// convert background settings
bgBuilder := new(strings.Builder)
if setting := group.BackgroundSettings; setting != nil {
fmt.Fprintf(bgBuilder, "TASK_TYPES='%s'", strings.Join(setting.JobTypes, ","))
first := true
if len(setting.JobTypes) > 0 {
fmt.Fprintf(bgBuilder, "TASK_TYPES='%s'", strings.Join(setting.JobTypes, ","))
first = false
}
if setting.UtilizationLimit > 0 {
if !first {
bgBuilder.WriteString(", ")
}
fmt.Fprintf(bgBuilder, "UTILIZATION_LIMIT=%d", setting.UtilizationLimit)
}
}
background := bgBuilder.String()

Expand Down
Loading

0 comments on commit 86c612e

Please sign in to comment.