expression: create a new context in ColumnInfos2ColumnsAndNames to ignore truncate error (#52468)
close pingcap/tidb#52366
This commit is contained in:
@ -931,13 +931,6 @@ func (e *CheckTableExec) Next(ctx context.Context, _ *chunk.Chunk) error {
|
||||
}
|
||||
defer func() { e.done = true }()
|
||||
|
||||
// See the comment of `ColumnInfos2ColumnsAndNames`. It's fixing #42341
|
||||
originalTypeFlags := e.Ctx().GetSessionVars().StmtCtx.TypeFlags()
|
||||
defer func() {
|
||||
e.Ctx().GetSessionVars().StmtCtx.SetTypeFlags(originalTypeFlags)
|
||||
}()
|
||||
e.Ctx().GetSessionVars().StmtCtx.SetTypeFlags(originalTypeFlags.WithIgnoreTruncateErr(true))
|
||||
|
||||
idxNames := make([]string, 0, len(e.indexInfos))
|
||||
for _, idx := range e.indexInfos {
|
||||
if idx.MVIndex {
|
||||
|
||||
@ -19,6 +19,7 @@ import (
|
||||
"fmt"
|
||||
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/tidb/pkg/errctx"
|
||||
"github.com/pingcap/tidb/pkg/parser/ast"
|
||||
"github.com/pingcap/tidb/pkg/parser/model"
|
||||
"github.com/pingcap/tidb/pkg/parser/mysql"
|
||||
@ -1005,9 +1006,48 @@ func TableInfo2SchemaAndNames(ctx BuildContext, dbName model.CIStr, tbl *model.T
|
||||
return schema, names, nil
|
||||
}
|
||||
|
||||
type ignoreTruncateExprCtx struct {
|
||||
BuildContext
|
||||
EvalContext
|
||||
tc types.Context
|
||||
ec errctx.Context
|
||||
}
|
||||
|
||||
// ignoreTruncate returns a new BuildContext that ignores the truncate error.
|
||||
func ignoreTruncate(ctx BuildContext) BuildContext {
|
||||
evalCtx := ctx.GetEvalCtx()
|
||||
tc, ec := evalCtx.TypeCtx(), evalCtx.ErrCtx()
|
||||
if tc.Flags().IgnoreTruncateErr() && ec.LevelForGroup(errctx.ErrGroupTruncate) == errctx.LevelIgnore {
|
||||
return ctx
|
||||
}
|
||||
|
||||
tc = tc.WithFlags(tc.Flags().WithIgnoreTruncateErr(true))
|
||||
ec = ec.WithErrGroupLevel(errctx.ErrGroupTruncate, errctx.LevelIgnore)
|
||||
|
||||
return &ignoreTruncateExprCtx{
|
||||
BuildContext: ctx,
|
||||
EvalContext: evalCtx,
|
||||
tc: tc,
|
||||
ec: ec,
|
||||
}
|
||||
}
|
||||
|
||||
// GetEvalCtx implements the BuildContext.EvalCtx().
|
||||
func (ctx *ignoreTruncateExprCtx) GetEvalCtx() EvalContext {
|
||||
return ctx
|
||||
}
|
||||
|
||||
// TypeCtx implements the EvalContext.TypeCtx().
|
||||
func (ctx *ignoreTruncateExprCtx) TypeCtx() types.Context {
|
||||
return ctx.tc
|
||||
}
|
||||
|
||||
// ErrCtx implements the EvalContext.ErrCtx().
|
||||
func (ctx *ignoreTruncateExprCtx) ErrCtx() errctx.Context {
|
||||
return ctx.ec
|
||||
}
|
||||
|
||||
// ColumnInfos2ColumnsAndNames converts the ColumnInfo to the *Column and NameSlice.
|
||||
// This function is **unsafe** to be called concurrently, unless the `IgnoreTruncate` has been set to `true`. The only
|
||||
// known case which will call this function concurrently is `CheckTableExec`. Ref #18408 and #42341.
|
||||
func ColumnInfos2ColumnsAndNames(ctx BuildContext, dbName, tblName model.CIStr, colInfos []*model.ColumnInfo, tblInfo *model.TableInfo) ([]*Column, types.NameSlice, error) {
|
||||
columns := make([]*Column, 0, len(colInfos))
|
||||
names := make([]*types.FieldName, 0, len(colInfos))
|
||||
@ -1031,17 +1071,16 @@ func ColumnInfos2ColumnsAndNames(ctx BuildContext, dbName, tblName model.CIStr,
|
||||
}
|
||||
// Resolve virtual generated column.
|
||||
mockSchema := NewSchema(columns...)
|
||||
// Ignore redundant warning here.
|
||||
flags := ctx.GetSessionVars().StmtCtx.TypeFlags()
|
||||
if !flags.IgnoreTruncateErr() {
|
||||
defer func() {
|
||||
ctx.GetSessionVars().StmtCtx.SetTypeFlags(flags)
|
||||
}()
|
||||
ctx.GetSessionVars().StmtCtx.SetTypeFlags(flags.WithIgnoreTruncateErr(true))
|
||||
}
|
||||
|
||||
truncateIgnored := false
|
||||
for i, col := range colInfos {
|
||||
if col.IsVirtualGenerated() {
|
||||
if !truncateIgnored {
|
||||
// Ignore redundant warning here.
|
||||
ctx = ignoreTruncate(ctx)
|
||||
truncateIgnored = true
|
||||
}
|
||||
|
||||
expr, err := generatedexpr.ParseExpression(col.GeneratedExprString)
|
||||
if err != nil {
|
||||
return nil, nil, errors.Trace(err)
|
||||
|
||||
@ -17,6 +17,7 @@ package expression
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/pingcap/tidb/pkg/errctx"
|
||||
"github.com/pingcap/tidb/pkg/parser/ast"
|
||||
"github.com/pingcap/tidb/pkg/parser/model"
|
||||
"github.com/pingcap/tidb/pkg/parser/mysql"
|
||||
@ -295,3 +296,34 @@ func TestExpressionMemeoryUsage(t *testing.T) {
|
||||
c4 := Constant{Value: types.NewStringDatum("11")}
|
||||
require.Greater(t, c4.MemoryUsage(), c3.MemoryUsage())
|
||||
}
|
||||
|
||||
func TestIgnoreTruncateExprCtx(t *testing.T) {
|
||||
ctx := createContext(t)
|
||||
ctx.GetSessionVars().StmtCtx.SetTypeFlags(types.StrictFlags)
|
||||
evalCtx := ctx.GetEvalCtx()
|
||||
tc, ec := evalCtx.TypeCtx(), evalCtx.ErrCtx()
|
||||
require.True(t, !tc.Flags().IgnoreTruncateErr() && !tc.Flags().TruncateAsWarning())
|
||||
require.Equal(t, errctx.LevelError, ec.LevelForGroup(errctx.ErrGroupTruncate))
|
||||
|
||||
// new ctx will ignore truncate error
|
||||
newEvalCtx := ignoreTruncate(ctx).GetEvalCtx()
|
||||
tc, ec = newEvalCtx.TypeCtx(), newEvalCtx.ErrCtx()
|
||||
require.True(t, tc.Flags().IgnoreTruncateErr() && !tc.Flags().TruncateAsWarning())
|
||||
require.Equal(t, errctx.LevelIgnore, ec.LevelForGroup(errctx.ErrGroupTruncate))
|
||||
|
||||
// old eval ctx will not change
|
||||
tc, ec = evalCtx.TypeCtx(), evalCtx.ErrCtx()
|
||||
require.True(t, !tc.Flags().IgnoreTruncateErr() && !tc.Flags().TruncateAsWarning())
|
||||
require.Equal(t, errctx.LevelError, ec.LevelForGroup(errctx.ErrGroupTruncate))
|
||||
|
||||
// old build ctx will not change
|
||||
evalCtx = ctx.GetEvalCtx()
|
||||
tc, ec = evalCtx.TypeCtx(), evalCtx.ErrCtx()
|
||||
require.True(t, !tc.Flags().IgnoreTruncateErr() && !tc.Flags().TruncateAsWarning())
|
||||
require.Equal(t, errctx.LevelError, ec.LevelForGroup(errctx.ErrGroupTruncate))
|
||||
|
||||
// truncate ignored ctx will not create new ctx
|
||||
ctx.GetSessionVars().StmtCtx.SetTypeFlags(types.StrictFlags.WithIgnoreTruncateErr(true))
|
||||
newCtx := ignoreTruncate(ctx)
|
||||
require.Same(t, ctx, newCtx)
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user