Skip to content

Commit

Permalink
refactor executor pkg's warning and note generation
Browse files Browse the repository at this point in the history
Signed-off-by: AilinKid <314806019@qq.com>
  • Loading branch information
AilinKid committed Dec 20, 2023
1 parent d84df2b commit aafa08f
Show file tree
Hide file tree
Showing 16 changed files with 62 additions and 52 deletions.
2 changes: 1 addition & 1 deletion pkg/executor/aggfuncs/func_group_concat.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ func (e *baseGroupConcat4String) handleTruncateError(sctx sessionctx.Context) (e
if !sctx.GetSessionVars().StmtCtx.TypeFlags().TruncateAsWarning() {
return expression.ErrCutValueGroupConcat.GenWithStackByArgs(e.args[0].String())
}
sctx.GetSessionVars().StmtCtx.AppendWarning(expression.ErrCutValueGroupConcat.GenWithStackByArgs(e.args[0].String()))
sctx.GetSessionVars().StmtCtx.AppendWarning(expression.ErrCutValueGroupConcat.FastGenByArgs(e.args[0].String()))
}
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -292,7 +292,7 @@ func warnLockedTableMsg(sessionVars *variable.SessionVars, needAnalyzeTableCnt u
} else {
msg = "skip analyze locked table: %s"
}
sessionVars.StmtCtx.AppendWarning(errors.Errorf(msg, tables))
sessionVars.StmtCtx.AppendWarning(errors.NewNoStackErrorf(msg, tables))
}
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/bind.go
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ func (e *SQLBindExec) setBindingStatus() error {
}
ok, err := domain.GetDomain(e.Ctx()).BindHandle().SetGlobalBindingStatus(e.normdOrigSQL, bindInfo, e.newStatus)
if err == nil && !ok {
warningMess := errors.New("There are no bindings can be set the status. Please check the SQL text")
warningMess := errors.NewNoStackError("There are no bindings can be set the status. Please check the SQL text")
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(warningMess)
}
return err
Expand All @@ -126,7 +126,7 @@ func (e *SQLBindExec) setBindingStatus() error {
func (e *SQLBindExec) setBindingStatusByDigest() error {
ok, err := domain.GetDomain(e.Ctx()).BindHandle().SetGlobalBindingStatusByDigest(e.newStatus, e.sqlDigest)
if err == nil && !ok {
warningMess := errors.New("There are no bindings can be set the status. Please check the SQL text")
warningMess := errors.NewNoStackError("There are no bindings can be set the status. Please check the SQL text")
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(warningMess)
}
return err
Expand Down
6 changes: 3 additions & 3 deletions pkg/executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -2651,7 +2651,7 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown(
if *sampleRate < 0 {
*sampleRate, sampleRateReason = b.getAdjustedSampleRate(task)
if task.PartitionName != "" {
sc.AppendNote(errors.Errorf(
sc.AppendNote(errors.NewNoStackErrorf(
`Analyze use auto adjusted sample rate %f for table %s.%s's partition %s, reason to use this rate is "%s"`,
*sampleRate,
task.DBName,
Expand All @@ -2660,7 +2660,7 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown(
sampleRateReason,
))
} else {
sc.AppendNote(errors.Errorf(
sc.AppendNote(errors.NewNoStackErrorf(
`Analyze use auto adjusted sample rate %f for table %s.%s, reason to use this rate is "%s"`,
*sampleRate,
task.DBName,
Expand Down Expand Up @@ -3409,7 +3409,7 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) e
useTiFlash := useMPP || useTiFlashBatchCop
if useTiFlash {
if _, isTiDBZoneLabelSet := config.GetGlobalConfig().Labels[placement.DCLabelKey]; b.ctx.GetSessionVars().TiFlashReplicaRead != tiflash.AllReplicas && !isTiDBZoneLabelSet {
b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("the variable tiflash_replica_read is ignored, because the entry TiDB[%s] does not set the zone attribute and tiflash_replica_read is '%s'", config.GetGlobalConfig().AdvertiseAddress, tiflash.GetTiFlashReplicaRead(b.ctx.GetSessionVars().TiFlashReplicaRead)))
b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("the variable tiflash_replica_read is ignored, because the entry TiDB[%s] does not set the zone attribute and tiflash_replica_read is '%s'", config.GetGlobalConfig().AdvertiseAddress, tiflash.GetTiFlashReplicaRead(b.ctx.GetSessionVars().TiFlashReplicaRead)))
}
}
if useMPP {
Expand Down
33 changes: 20 additions & 13 deletions pkg/executor/compact_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ func (e *CompactTableTiFlashExec) Next(ctx context.Context, chk *chunk.Chunk) er
func (e *CompactTableTiFlashExec) doCompact(execCtx context.Context) error {
vars := e.Ctx().GetSessionVars()
if e.tableInfo.TiFlashReplica == nil || e.tableInfo.TiFlashReplica.Count == 0 {
vars.StmtCtx.AppendWarning(errors.Errorf("compact skipped: no tiflash replica in the table"))
vars.StmtCtx.AppendWarning(errors.NewNoStackErrorf("compact skipped: no tiflash replica in the table"))
return nil
}

Expand Down Expand Up @@ -257,33 +257,38 @@ func (task *storeCompactTask) compactOnePhysicalTable(physicalTableID int64) (bo
if err != nil {
// Even after backoff, the request is still failed.., or the request is cancelled or timed out
// For example, the store is down. Let's simply don't compact other partitions.
warn := errors.Errorf("compact on store %s failed: %v", task.targetStore.Address, err)
warn := errors.NewNoStackErrorf("compact on store %s failed: %v", task.targetStore.Address, err)
task.parentExec.Ctx().GetSessionVars().StmtCtx.AppendWarning(warn)
newErr := errors.Trace(warn)
task.logFailure(
zap.Int64("physical-table-id", physicalTableID),
zap.Error(err))
return false, warn
return false, newErr
}
if resp.GetError() != nil {
switch resp.GetError().GetError().(type) {
case *kvrpcpb.CompactError_ErrCompactInProgress:
warn := errors.Errorf("compact on store %s failed: table is compacting in progress", task.targetStore.Address)
// warn is light stackless.
warn := errors.NewNoStackErrorf("compact on store %s failed: table is compacting in progress", task.targetStore.Address)
task.parentExec.Ctx().GetSessionVars().StmtCtx.AppendWarning(warn)
err := errors.Trace(warn)
task.logFailure(
zap.Int64("physical-table-id", physicalTableID),
zap.Error(warn))
zap.Error(err))
// TiFlash reported that there are existing compacting for the same table.
// We should stop the whole SQL execution, including compacting requests to other stores, as repeatedly
// compacting the same table is a waste of resource.
return true, warn
return true, err
case *kvrpcpb.CompactError_ErrTooManyPendingTasks:
// The store is already very busy, don't retry and don't compact other partitions.
warn := errors.Errorf("compact on store %s failed: store is too busy", task.targetStore.Address)
// warn is light stackless.
warn := errors.NewNoStackErrorf("compact on store %s failed: store is too busy", task.targetStore.Address)
task.parentExec.Ctx().GetSessionVars().StmtCtx.AppendWarning(warn)
err := errors.Trace(warn)
task.logFailure(
zap.Int64("physical-table-id", physicalTableID),
zap.Error(warn))
return false, warn
zap.Error(err))
return false, err
case *kvrpcpb.CompactError_ErrPhysicalTableNotExist:
// The physical table does not exist, don't retry this partition, but other partitions should still be compacted.
// This may happen when partition or table is dropped during the long compaction.
Expand All @@ -296,12 +301,13 @@ func (task *storeCompactTask) compactOnePhysicalTable(physicalTableID int64) (bo
return false, nil
default:
// Others are unexpected errors, don't retry and don't compact other partitions.
warn := errors.Errorf("compact on store %s failed: internal error (check logs for details)", task.targetStore.Address)
warn := errors.NewNoStackErrorf("compact on store %s failed: internal error (check logs for details)", task.targetStore.Address)
task.parentExec.Ctx().GetSessionVars().StmtCtx.AppendWarning(warn)
err := errors.Trace(warn)
task.logFailure(
zap.Int64("physical-table-id", physicalTableID),
zap.Any("response-error", resp.GetError().GetError()))
return false, warn
return false, err
}
}

Expand All @@ -314,14 +320,15 @@ func (task *storeCompactTask) compactOnePhysicalTable(physicalTableID int64) (bo
if len(lastEndKey) == 0 || bytes.Compare(lastEndKey, startKey) <= 0 {
// The TiFlash server returned an invalid compacted end key.
// This is unexpected...
warn := errors.Errorf("compact on store %s failed: internal error (check logs for details)", task.targetStore.Address)
warn := errors.NewNoStackErrorf("compact on store %s failed: internal error (check logs for details)", task.targetStore.Address)
task.parentExec.Ctx().GetSessionVars().StmtCtx.AppendWarning(warn)
err := errors.Trace(warn)
task.logFailure(
zap.Int64("physical-table-id", physicalTableID),
zap.String("compacted-start-key", hex.EncodeToString(resp.GetCompactedStartKey())),
zap.String("compacted-end-key", hex.EncodeToString(resp.GetCompactedEndKey())),
)
return false, warn
return false, err
}
startKey = lastEndKey
}
Expand Down
14 changes: 8 additions & 6 deletions pkg/executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,12 +119,14 @@ func (e *DDLExec) Next(ctx context.Context, _ *chunk.Chunk) (err error) {
for _, tn := range s.Tables {
nonExistsTables = append(nonExistsTables, ast.Ident{Schema: tn.Schema, Name: tn.Name}.String())
}
err = infoschema.ErrTableDropExists.GenWithStackByArgs(strings.Join(nonExistsTables, ","))
// stackless err once used like note.
err = infoschema.ErrTableDropExists.FastGenByArgs(strings.Join(nonExistsTables, ","))
if s.IfExists {
e.Ctx().GetSessionVars().StmtCtx.AppendNote(err)
return nil
}
return err
// complete and trace stack info.
return errors.Trace(err)
}

// if all tables are local temporary, directly drop those tables.
Expand Down Expand Up @@ -275,12 +277,12 @@ func (e *DDLExec) createSessionTemporaryTable(s *ast.CreateTableStmt) error {

_, exists := e.getLocalTemporaryTable(s.Table.Schema, s.Table.Name)
if exists {
err := infoschema.ErrTableExists.GenWithStackByArgs(ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name})
err := infoschema.ErrTableExists.FastGenByArgs(ast.Ident{Schema: s.Table.Schema, Name: s.Table.Name})
if s.IfNotExists {
e.Ctx().GetSessionVars().StmtCtx.AppendNote(err)
return nil
}
return err
return errors.Trace(err)
}

tbInfo, err := ddl.BuildSessionTemporaryTableInfo(e.Ctx(), is, s, dbInfo.Charset, dbInfo.Collate, dbInfo.PlacementPolicyRef)
Expand Down Expand Up @@ -696,7 +698,7 @@ func (e *DDLExec) getRecoverDBByName(schemaName model.CIStr) (recoverSchemaInfo

func (e *DDLExec) executeLockTables(s *ast.LockTablesStmt) error {
if !config.TableLockEnabled() {
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(exeerrors.ErrFuncNotEnabled.GenWithStackByArgs("LOCK TABLES", "enable-table-lock"))
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(exeerrors.ErrFuncNotEnabled.FastGenByArgs("LOCK TABLES", "enable-table-lock"))
return nil
}

Expand All @@ -711,7 +713,7 @@ func (e *DDLExec) executeLockTables(s *ast.LockTablesStmt) error {

func (e *DDLExec) executeUnlockTables(_ *ast.UnlockTablesStmt) error {
if !config.TableLockEnabled() {
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(exeerrors.ErrFuncNotEnabled.GenWithStackByArgs("UNLOCK TABLES", "enable-table-lock"))
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(exeerrors.ErrFuncNotEnabled.FastGenByArgs("UNLOCK TABLES", "enable-table-lock"))
return nil
}
lockedTables := e.Ctx().GetAllTableLocks()
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/lockstats/lock_stats_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ func (e *LockExec) Next(_ context.Context, _ *chunk.Chunk) error {
return err
}
if msg != "" {
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(errors.New(msg))
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError(msg))
}
} else {
tableWithPartitions, err := populateTableAndPartitionIDs(e.Tables, is)
Expand All @@ -78,7 +78,7 @@ func (e *LockExec) Next(_ context.Context, _ *chunk.Chunk) error {
return err
}
if msg != "" {
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(errors.New(msg))
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError(msg))
}
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/lockstats/unlock_stats_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ func (e *UnlockExec) Next(context.Context, *chunk.Chunk) error {
return err
}
if msg != "" {
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(errors.New(msg))
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError(msg))
}
} else {
tableWithPartitions, err := populateTableAndPartitionIDs(e.Tables, is)
Expand All @@ -72,7 +72,7 @@ func (e *UnlockExec) Next(context.Context, *chunk.Chunk) error {
return err
}
if msg != "" {
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(errors.New(msg))
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackError(msg))
}
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/memtable_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,7 @@ func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.String
address := srv.Address
statusAddr := srv.StatusAddr
if len(statusAddr) == 0 {
sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("%s node %s does not contain status address", typ, address))
sctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("%s node %s does not contain status address", typ, address))
continue
}
wg.Add(1)
Expand Down Expand Up @@ -456,7 +456,7 @@ func (e *clusterLogRetriever) startRetrieving(
address := srv.Address
statusAddr := srv.StatusAddr
if len(statusAddr) == 0 {
sctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("%s node %s does not contain status address", typ, address))
sctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("%s node %s does not contain status address", typ, address))
continue
}
ch := make(chan logStreamResult)
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/plan_replayer.go
Original file line number Diff line number Diff line change
Expand Up @@ -401,7 +401,7 @@ func loadVariables(ctx sessionctx.Context, z *zip.Reader) error {
}
}
if len(unLoadVars) > 0 {
ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("variables set failed:%s", strings.Join(unLoadVars, ",")))
ctx.GetSessionVars().StmtCtx.AppendWarning(errors.NewNoStackErrorf("variables set failed:%s", strings.Join(unLoadVars, ",")))
}
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/revoke.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,7 +203,7 @@ func (e *RevokeExec) revokePriv(internalSession sessionctx.Context, priv *ast.Pr
func (e *RevokeExec) revokeDynamicPriv(internalSession sessionctx.Context, privName string, user, host string) error {
privName = strings.ToUpper(privName)
if !privilege.GetPrivilegeManager(e.Ctx()).IsDynamicPrivilege(privName) { // for MySQL compatibility
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(exeerrors.ErrDynamicPrivilegeNotRegistered.GenWithStackByArgs(privName))
e.Ctx().GetSessionVars().StmtCtx.AppendWarning(exeerrors.ErrDynamicPrivilegeNotRegistered.FastGenByArgs(privName))
}
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnPrivilege)
_, err := internalSession.(sqlexec.SQLExecutor).ExecuteInternal(ctx, "DELETE FROM mysql.global_grants WHERE user = %? AND host = %? AND priv = %?", user, host, privName)
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/set.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,13 +138,13 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres
// The variable is a noop. For compatibility we allow it to still
// be changed, but we append a warning since users might be expecting
// something that's not going to happen.
sessionVars.StmtCtx.AppendWarning(exeerrors.ErrSettingNoopVariable.GenWithStackByArgs(sysVar.Name))
sessionVars.StmtCtx.AppendWarning(exeerrors.ErrSettingNoopVariable.FastGenByArgs(sysVar.Name))
}
if sysVar.HasInstanceScope() && !v.IsGlobal && sessionVars.EnableLegacyInstanceScope {
// For backward compatibility we will change the v.IsGlobal to true,
// and append a warning saying this will not be supported in future.
v.IsGlobal = true
sessionVars.StmtCtx.AppendWarning(exeerrors.ErrInstanceScope.GenWithStackByArgs(sysVar.Name))
sessionVars.StmtCtx.AppendWarning(exeerrors.ErrInstanceScope.FastGenByArgs(sysVar.Name))
}

if v.IsGlobal {
Expand Down

0 comments on commit aafa08f

Please sign in to comment.