Skip to content

Commit

Permalink
expression: create a new context in ColumnInfos2ColumnsAndNames to …
Browse files Browse the repository at this point in the history
…ignore truncate error (pingcap#52468)

close pingcap#52366
  • Loading branch information
lcwangchao authored and 3AceShowHand committed Apr 16, 2024
1 parent eaa6926 commit 41273be
Show file tree
Hide file tree
Showing 3 changed files with 81 additions and 17 deletions.
7 changes: 0 additions & 7 deletions pkg/executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
59 changes: 49 additions & 10 deletions pkg/expression/expression.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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))
Expand All @@ -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)
Expand Down
32 changes: 32 additions & 0 deletions pkg/expression/expression_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
}

0 comments on commit 41273be

Please sign in to comment.