Skip to content

Commit

Permalink
*: support tidb_redact_log for explain (#54553) (#55308)
Browse files Browse the repository at this point in the history
close #54565
  • Loading branch information
ti-chi-bot authored Aug 22, 2024
1 parent 79b22d9 commit a7df4f9
Show file tree
Hide file tree
Showing 78 changed files with 940 additions and 192 deletions.
5 changes: 3 additions & 2 deletions pkg/executor/aggfuncs/func_group_concat.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"sync/atomic"
"unsafe"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/planner/util"
"github.com/pingcap/tidb/pkg/types"
Expand Down Expand Up @@ -72,9 +73,9 @@ func (e *baseGroupConcat4String) AppendFinalResult2Chunk(_ AggFuncUpdateContext,
func (e *baseGroupConcat4String) handleTruncateError(tc types.Context) (err error) {
if atomic.CompareAndSwapInt32(e.truncated, 0, 1) {
if !tc.Flags().TruncateAsWarning() {
return expression.ErrCutValueGroupConcat.GenWithStackByArgs(e.args[0].String())
return expression.ErrCutValueGroupConcat.GenWithStackByArgs(e.args[0].StringWithCtx(errors.RedactLogDisable))
}
tc.AppendWarning(expression.ErrCutValueGroupConcat.FastGenByArgs(e.args[0].String()))
tc.AppendWarning(expression.ErrCutValueGroupConcat.FastGenByArgs(e.args[0].StringWithCtx(errors.RedactLogDisable)))
}
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/importer/import.go
Original file line number Diff line number Diff line change
Expand Up @@ -811,7 +811,7 @@ func (p *Plan) initParameters(plan *plannercore.ImportInto) error {
optionMap := make(map[string]any, len(plan.Options))
for _, opt := range plan.Options {
if opt.Value != nil {
val := opt.Value.String()
val := opt.Value.StringWithCtx(errors.RedactLogDisable)
if opt.Name == cloudStorageURIOption {
val = ast.RedactURL(val)
}
Expand Down
1 change: 1 addition & 0 deletions pkg/expression/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,7 @@ go_library(
"//pkg/util/password-validation",
"//pkg/util/plancodec",
"//pkg/util/printer",
"//pkg/util/redact",
"//pkg/util/sem",
"//pkg/util/set",
"//pkg/util/size",
Expand Down
4 changes: 2 additions & 2 deletions pkg/expression/aggregation/agg_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ func AggFuncToPBExpr(ctx expression.PushDownContext, aggFunc *AggFuncDesc, store
for _, arg := range aggFunc.Args {
pbArg := pc.ExprToPB(arg)
if pbArg == nil {
return nil, errors.New(aggFunc.String() + " can't be converted to PB.")
return nil, errors.New(aggFunc.StringWithCtx(errors.RedactLogDisable) + " can't be converted to PB.")
}
children = append(children, pbArg)
}
Expand All @@ -121,7 +121,7 @@ func AggFuncToPBExpr(ctx expression.PushDownContext, aggFunc *AggFuncDesc, store
for _, arg := range aggFunc.OrderByItems {
pbArg := expression.SortByItemToPB(ctx.EvalCtx(), client, arg.Expr, arg.Desc)
if pbArg == nil {
return nil, errors.New(aggFunc.String() + " can't be converted to PB.")
return nil, errors.New(aggFunc.StringWithCtx(errors.RedactLogDisable) + " can't be converted to PB.")
}
orderBy = append(orderBy, pbArg)
}
Expand Down
9 changes: 7 additions & 2 deletions pkg/expression/aggregation/base_func.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,10 +72,15 @@ func (a *baseFuncDesc) clone() *baseFuncDesc {

// String implements the fmt.Stringer interface.
func (a *baseFuncDesc) String() string {
return a.StringWithCtx(errors.RedactLogDisable)
}

// StringWithCtx returns the string within given context.
func (a *baseFuncDesc) StringWithCtx(redact string) string {
buffer := bytes.NewBufferString(a.Name)
buffer.WriteString("(")
for i, arg := range a.Args {
buffer.WriteString(arg.String())
buffer.WriteString(arg.StringWithCtx(redact))
if i+1 != len(a.Args) {
buffer.WriteString(", ")
}
Expand Down Expand Up @@ -149,7 +154,7 @@ func (a *baseFuncDesc) typeInfer4ApproxPercentile(ctx expression.EvalContext) er
}
percent, isNull, err := a.Args[1].EvalInt(ctx, chunk.Row{})
if err != nil {
return fmt.Errorf("APPROX_PERCENTILE: Invalid argument %s", a.Args[1].String())
return fmt.Errorf("APPROX_PERCENTILE: Invalid argument %s", a.Args[1].StringWithCtx(errors.RedactLogDisable))
}
if percent <= 0 || percent > 100 || isNull {
if isNull {
Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/aggregation/concat.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ func (cf *concatFunction) Update(evalCtx *AggEvaluateContext, sc *stmtctx.Statem
}
evalCtx.Buffer.Truncate(i)
if !cf.truncated {
sc.AppendWarning(expression.ErrCutValueGroupConcat.FastGenByArgs(cf.Args[0].String()))
sc.AppendWarning(expression.ErrCutValueGroupConcat.FastGenByArgs(cf.Args[0].StringWithCtx(errors.RedactLogDisable)))
}
cf.truncated = true
}
Expand Down
9 changes: 7 additions & 2 deletions pkg/expression/aggregation/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,13 +61,18 @@ func NewAggFuncDescForWindowFunc(ctx expression.BuildContext, desc *WindowFuncDe

// String implements the fmt.Stringer interface.
func (a *AggFuncDesc) String() string {
return a.StringWithCtx(errors.RedactLogDisable)
}

// StringWithCtx returns the string representation within given ctx.
func (a *AggFuncDesc) StringWithCtx(redact string) string {
buffer := bytes.NewBufferString(a.Name)
buffer.WriteString("(")
if a.HasDistinct {
buffer.WriteString("distinct ")
}
for i, arg := range a.Args {
buffer.WriteString(arg.String())
buffer.WriteString(arg.StringWithCtx(redact))
if i+1 != len(a.Args) {
buffer.WriteString(", ")
}
Expand All @@ -76,7 +81,7 @@ func (a *AggFuncDesc) String() string {
buffer.WriteString(" order by ")
}
for i, arg := range a.OrderByItems {
buffer.WriteString(arg.String())
buffer.WriteString(arg.StringWithCtx(redact))
if i+1 != len(a.OrderByItems) {
buffer.WriteString(", ")
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/expression/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"time"

"github.com/google/uuid"
perrors "github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/parser/auth"
"github.com/pingcap/tidb/pkg/parser/charset"
Expand Down Expand Up @@ -1375,7 +1376,7 @@ func benchmarkVectorizedEvalOneVec(b *testing.B, vecExprCases vecExprBenchCases)
for funcName, testCases := range vecExprCases {
for _, testCase := range testCases {
expr, _, input, output := genVecExprBenchCase(ctx, funcName, testCase)
exprName := expr.String()
exprName := expr.StringWithCtx(perrors.RedactLogDisable)
if sf, ok := expr.(*ScalarFunction); ok {
exprName = fmt.Sprintf("%v", reflect.TypeOf(sf.Function))
tmp := strings.Split(exprName, ".")
Expand Down
39 changes: 20 additions & 19 deletions pkg/expression/builtin_arithmetic.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"fmt"
"math"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/types"
Expand Down Expand Up @@ -225,25 +226,25 @@ func (s *builtinArithmeticPlusIntSig) evalInt(ctx EvalContext, row chunk.Row) (v
switch {
case isLHSUnsigned && isRHSUnsigned:
if uint64(a) > math.MaxUint64-uint64(b) {
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
case isLHSUnsigned && !isRHSUnsigned:
if b < 0 && uint64(-b) > uint64(a) {
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
if b > 0 && uint64(a) > math.MaxUint64-uint64(b) {
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
case !isLHSUnsigned && isRHSUnsigned:
if a < 0 && uint64(-a) > uint64(b) {
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
if a > 0 && uint64(b) > math.MaxUint64-uint64(a) {
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
case !isLHSUnsigned && !isRHSUnsigned:
if (a > 0 && b > math.MaxInt64-a) || (a < 0 && b < math.MinInt64-a) {
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
}

Expand Down Expand Up @@ -273,7 +274,7 @@ func (s *builtinArithmeticPlusDecimalSig) evalDecimal(ctx EvalContext, row chunk
err = types.DecimalAdd(a, b, c)
if err != nil {
if err == types.ErrOverflow {
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
return nil, true, err
}
Expand Down Expand Up @@ -303,7 +304,7 @@ func (s *builtinArithmeticPlusRealSig) evalReal(ctx EvalContext, row chunk.Row)
return 0, true, nil
}
if !mathutil.IsFinite(a + b) {
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
return a + b, false, nil
}
Expand Down Expand Up @@ -368,7 +369,7 @@ func (s *builtinArithmeticMinusRealSig) evalReal(ctx EvalContext, row chunk.Row)
return 0, isNull, err
}
if !mathutil.IsFinite(a - b) {
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s - %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s - %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
return a - b, false, nil
}
Expand Down Expand Up @@ -396,7 +397,7 @@ func (s *builtinArithmeticMinusDecimalSig) evalDecimal(ctx EvalContext, row chun
err = types.DecimalSub(a, b, c)
if err != nil {
if err == types.ErrOverflow {
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s - %s)", s.args[0].String(), s.args[1].String()))
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s - %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
return nil, true, err
}
Expand Down Expand Up @@ -434,7 +435,7 @@ func (s *builtinArithmeticMinusIntSig) evalInt(ctx EvalContext, row chunk.Row) (
}
overflow := s.overflowCheck(isLHSUnsigned, isRHSUnsigned, signed, a, b)
if overflow {
return 0, true, types.ErrOverflow.GenWithStackByArgs(errType, fmt.Sprintf("(%s - %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs(errType, fmt.Sprintf("(%s - %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}

return a - b, false, nil
Expand Down Expand Up @@ -578,7 +579,7 @@ func (s *builtinArithmeticMultiplyRealSig) evalReal(ctx EvalContext, row chunk.R
}
result := a * b
if math.IsInf(result, 0) {
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s * %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s * %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
return result, false, nil
}
Expand All @@ -596,7 +597,7 @@ func (s *builtinArithmeticMultiplyDecimalSig) evalDecimal(ctx EvalContext, row c
err = types.DecimalMul(a, b, c)
if err != nil && !terror.ErrorEqual(err, types.ErrTruncated) {
if err == types.ErrOverflow {
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s * %s)", s.args[0].String(), s.args[1].String()))
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s * %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
return nil, true, err
}
Expand All @@ -616,7 +617,7 @@ func (s *builtinArithmeticMultiplyIntUnsignedSig) evalInt(ctx EvalContext, row c
unsignedB := uint64(b)
result := unsignedA * unsignedB
if unsignedA != 0 && result/unsignedA != unsignedB {
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s * %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s * %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
return int64(result), false, nil
}
Expand All @@ -632,7 +633,7 @@ func (s *builtinArithmeticMultiplyIntSig) evalInt(ctx EvalContext, row chunk.Row
}
result := a * b
if (a != 0 && result/a != b) || (result == math.MinInt64 && a == -1) {
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s * %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s * %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
return result, false, nil
}
Expand Down Expand Up @@ -697,7 +698,7 @@ func (s *builtinArithmeticDivideRealSig) evalReal(ctx EvalContext, row chunk.Row
}
result := a / b
if math.IsInf(result, 0) {
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s / %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s / %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
return result, false, nil
}
Expand Down Expand Up @@ -726,7 +727,7 @@ func (s *builtinArithmeticDivideDecimalSig) evalDecimal(ctx EvalContext, row chu
err = c.Round(c, s.baseBuiltinFunc.tp.GetDecimal(), types.ModeHalfUp)
}
} else if err == types.ErrOverflow {
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s / %s)", s.args[0].String(), s.args[1].String()))
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s / %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
return c, false, err
}
Expand Down Expand Up @@ -857,14 +858,14 @@ func (s *builtinArithmeticIntDivideDecimalSig) evalInt(ctx EvalContext, row chun
ret = int64(0)
return ret, false, nil
}
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s DIV %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s DIV %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
ret = int64(val)
} else {
ret, err = c.ToInt()
// err returned by ToInt may be ErrTruncated or ErrOverflow, only handle ErrOverflow, ignore ErrTruncated.
if err == types.ErrOverflow {
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s DIV %s)", s.args[0].String(), s.args[1].String()))
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s DIV %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
}
}

Expand Down
Loading

0 comments on commit a7df4f9

Please sign in to comment.