Skip to content

Commit a7df4f9

Browse files
authored
*: support tidb_redact_log for explain (#54553) (#55308)
close #54565
1 parent 79b22d9 commit a7df4f9

File tree

78 files changed

+940
-192
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

78 files changed

+940
-192
lines changed

pkg/executor/aggfuncs/func_group_concat.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import (
2121
"sync/atomic"
2222
"unsafe"
2323

24+
"github.com/pingcap/errors"
2425
"github.com/pingcap/tidb/pkg/expression"
2526
"github.com/pingcap/tidb/pkg/planner/util"
2627
"github.com/pingcap/tidb/pkg/types"
@@ -72,9 +73,9 @@ func (e *baseGroupConcat4String) AppendFinalResult2Chunk(_ AggFuncUpdateContext,
7273
func (e *baseGroupConcat4String) handleTruncateError(tc types.Context) (err error) {
7374
if atomic.CompareAndSwapInt32(e.truncated, 0, 1) {
7475
if !tc.Flags().TruncateAsWarning() {
75-
return expression.ErrCutValueGroupConcat.GenWithStackByArgs(e.args[0].String())
76+
return expression.ErrCutValueGroupConcat.GenWithStackByArgs(e.args[0].StringWithCtx(errors.RedactLogDisable))
7677
}
77-
tc.AppendWarning(expression.ErrCutValueGroupConcat.FastGenByArgs(e.args[0].String()))
78+
tc.AppendWarning(expression.ErrCutValueGroupConcat.FastGenByArgs(e.args[0].StringWithCtx(errors.RedactLogDisable)))
7879
}
7980
return nil
8081
}

pkg/executor/importer/import.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -811,7 +811,7 @@ func (p *Plan) initParameters(plan *plannercore.ImportInto) error {
811811
optionMap := make(map[string]any, len(plan.Options))
812812
for _, opt := range plan.Options {
813813
if opt.Value != nil {
814-
val := opt.Value.String()
814+
val := opt.Value.StringWithCtx(errors.RedactLogDisable)
815815
if opt.Name == cloudStorageURIOption {
816816
val = ast.RedactURL(val)
817817
}

pkg/expression/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -111,6 +111,7 @@ go_library(
111111
"//pkg/util/password-validation",
112112
"//pkg/util/plancodec",
113113
"//pkg/util/printer",
114+
"//pkg/util/redact",
114115
"//pkg/util/sem",
115116
"//pkg/util/set",
116117
"//pkg/util/size",

pkg/expression/aggregation/agg_to_pb.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -107,7 +107,7 @@ func AggFuncToPBExpr(ctx expression.PushDownContext, aggFunc *AggFuncDesc, store
107107
for _, arg := range aggFunc.Args {
108108
pbArg := pc.ExprToPB(arg)
109109
if pbArg == nil {
110-
return nil, errors.New(aggFunc.String() + " can't be converted to PB.")
110+
return nil, errors.New(aggFunc.StringWithCtx(errors.RedactLogDisable) + " can't be converted to PB.")
111111
}
112112
children = append(children, pbArg)
113113
}
@@ -121,7 +121,7 @@ func AggFuncToPBExpr(ctx expression.PushDownContext, aggFunc *AggFuncDesc, store
121121
for _, arg := range aggFunc.OrderByItems {
122122
pbArg := expression.SortByItemToPB(ctx.EvalCtx(), client, arg.Expr, arg.Desc)
123123
if pbArg == nil {
124-
return nil, errors.New(aggFunc.String() + " can't be converted to PB.")
124+
return nil, errors.New(aggFunc.StringWithCtx(errors.RedactLogDisable) + " can't be converted to PB.")
125125
}
126126
orderBy = append(orderBy, pbArg)
127127
}

pkg/expression/aggregation/base_func.go

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -72,10 +72,15 @@ func (a *baseFuncDesc) clone() *baseFuncDesc {
7272

7373
// String implements the fmt.Stringer interface.
7474
func (a *baseFuncDesc) String() string {
75+
return a.StringWithCtx(errors.RedactLogDisable)
76+
}
77+
78+
// StringWithCtx returns the string within given context.
79+
func (a *baseFuncDesc) StringWithCtx(redact string) string {
7580
buffer := bytes.NewBufferString(a.Name)
7681
buffer.WriteString("(")
7782
for i, arg := range a.Args {
78-
buffer.WriteString(arg.String())
83+
buffer.WriteString(arg.StringWithCtx(redact))
7984
if i+1 != len(a.Args) {
8085
buffer.WriteString(", ")
8186
}
@@ -149,7 +154,7 @@ func (a *baseFuncDesc) typeInfer4ApproxPercentile(ctx expression.EvalContext) er
149154
}
150155
percent, isNull, err := a.Args[1].EvalInt(ctx, chunk.Row{})
151156
if err != nil {
152-
return fmt.Errorf("APPROX_PERCENTILE: Invalid argument %s", a.Args[1].String())
157+
return fmt.Errorf("APPROX_PERCENTILE: Invalid argument %s", a.Args[1].StringWithCtx(errors.RedactLogDisable))
153158
}
154159
if percent <= 0 || percent > 100 || isNull {
155160
if isNull {

pkg/expression/aggregation/concat.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -104,7 +104,7 @@ func (cf *concatFunction) Update(evalCtx *AggEvaluateContext, sc *stmtctx.Statem
104104
}
105105
evalCtx.Buffer.Truncate(i)
106106
if !cf.truncated {
107-
sc.AppendWarning(expression.ErrCutValueGroupConcat.FastGenByArgs(cf.Args[0].String()))
107+
sc.AppendWarning(expression.ErrCutValueGroupConcat.FastGenByArgs(cf.Args[0].StringWithCtx(errors.RedactLogDisable)))
108108
}
109109
cf.truncated = true
110110
}

pkg/expression/aggregation/descriptor.go

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -61,13 +61,18 @@ func NewAggFuncDescForWindowFunc(ctx expression.BuildContext, desc *WindowFuncDe
6161

6262
// String implements the fmt.Stringer interface.
6363
func (a *AggFuncDesc) String() string {
64+
return a.StringWithCtx(errors.RedactLogDisable)
65+
}
66+
67+
// StringWithCtx returns the string representation within given ctx.
68+
func (a *AggFuncDesc) StringWithCtx(redact string) string {
6469
buffer := bytes.NewBufferString(a.Name)
6570
buffer.WriteString("(")
6671
if a.HasDistinct {
6772
buffer.WriteString("distinct ")
6873
}
6974
for i, arg := range a.Args {
70-
buffer.WriteString(arg.String())
75+
buffer.WriteString(arg.StringWithCtx(redact))
7176
if i+1 != len(a.Args) {
7277
buffer.WriteString(", ")
7378
}
@@ -76,7 +81,7 @@ func (a *AggFuncDesc) String() string {
7681
buffer.WriteString(" order by ")
7782
}
7883
for i, arg := range a.OrderByItems {
79-
buffer.WriteString(arg.String())
84+
buffer.WriteString(arg.StringWithCtx(redact))
8085
if i+1 != len(a.OrderByItems) {
8186
buffer.WriteString(", ")
8287
}

pkg/expression/bench_test.go

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import (
2929
"time"
3030

3131
"github.com/google/uuid"
32+
perrors "github.com/pingcap/errors"
3233
"github.com/pingcap/tidb/pkg/parser/ast"
3334
"github.com/pingcap/tidb/pkg/parser/auth"
3435
"github.com/pingcap/tidb/pkg/parser/charset"
@@ -1375,7 +1376,7 @@ func benchmarkVectorizedEvalOneVec(b *testing.B, vecExprCases vecExprBenchCases)
13751376
for funcName, testCases := range vecExprCases {
13761377
for _, testCase := range testCases {
13771378
expr, _, input, output := genVecExprBenchCase(ctx, funcName, testCase)
1378-
exprName := expr.String()
1379+
exprName := expr.StringWithCtx(perrors.RedactLogDisable)
13791380
if sf, ok := expr.(*ScalarFunction); ok {
13801381
exprName = fmt.Sprintf("%v", reflect.TypeOf(sf.Function))
13811382
tmp := strings.Split(exprName, ".")

pkg/expression/builtin_arithmetic.go

Lines changed: 20 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@ import (
1818
"fmt"
1919
"math"
2020

21+
"github.com/pingcap/errors"
2122
"github.com/pingcap/tidb/pkg/parser/mysql"
2223
"github.com/pingcap/tidb/pkg/parser/terror"
2324
"github.com/pingcap/tidb/pkg/types"
@@ -225,25 +226,25 @@ func (s *builtinArithmeticPlusIntSig) evalInt(ctx EvalContext, row chunk.Row) (v
225226
switch {
226227
case isLHSUnsigned && isRHSUnsigned:
227228
if uint64(a) > math.MaxUint64-uint64(b) {
228-
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
229+
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
229230
}
230231
case isLHSUnsigned && !isRHSUnsigned:
231232
if b < 0 && uint64(-b) > uint64(a) {
232-
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
233+
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
233234
}
234235
if b > 0 && uint64(a) > math.MaxUint64-uint64(b) {
235-
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
236+
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
236237
}
237238
case !isLHSUnsigned && isRHSUnsigned:
238239
if a < 0 && uint64(-a) > uint64(b) {
239-
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
240+
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
240241
}
241242
if a > 0 && uint64(b) > math.MaxUint64-uint64(a) {
242-
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
243+
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
243244
}
244245
case !isLHSUnsigned && !isRHSUnsigned:
245246
if (a > 0 && b > math.MaxInt64-a) || (a < 0 && b < math.MinInt64-a) {
246-
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
247+
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
247248
}
248249
}
249250

@@ -273,7 +274,7 @@ func (s *builtinArithmeticPlusDecimalSig) evalDecimal(ctx EvalContext, row chunk
273274
err = types.DecimalAdd(a, b, c)
274275
if err != nil {
275276
if err == types.ErrOverflow {
276-
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
277+
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
277278
}
278279
return nil, true, err
279280
}
@@ -303,7 +304,7 @@ func (s *builtinArithmeticPlusRealSig) evalReal(ctx EvalContext, row chunk.Row)
303304
return 0, true, nil
304305
}
305306
if !mathutil.IsFinite(a + b) {
306-
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s + %s)", s.args[0].String(), s.args[1].String()))
307+
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s + %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
307308
}
308309
return a + b, false, nil
309310
}
@@ -368,7 +369,7 @@ func (s *builtinArithmeticMinusRealSig) evalReal(ctx EvalContext, row chunk.Row)
368369
return 0, isNull, err
369370
}
370371
if !mathutil.IsFinite(a - b) {
371-
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s - %s)", s.args[0].String(), s.args[1].String()))
372+
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s - %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
372373
}
373374
return a - b, false, nil
374375
}
@@ -396,7 +397,7 @@ func (s *builtinArithmeticMinusDecimalSig) evalDecimal(ctx EvalContext, row chun
396397
err = types.DecimalSub(a, b, c)
397398
if err != nil {
398399
if err == types.ErrOverflow {
399-
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s - %s)", s.args[0].String(), s.args[1].String()))
400+
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s - %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
400401
}
401402
return nil, true, err
402403
}
@@ -434,7 +435,7 @@ func (s *builtinArithmeticMinusIntSig) evalInt(ctx EvalContext, row chunk.Row) (
434435
}
435436
overflow := s.overflowCheck(isLHSUnsigned, isRHSUnsigned, signed, a, b)
436437
if overflow {
437-
return 0, true, types.ErrOverflow.GenWithStackByArgs(errType, fmt.Sprintf("(%s - %s)", s.args[0].String(), s.args[1].String()))
438+
return 0, true, types.ErrOverflow.GenWithStackByArgs(errType, fmt.Sprintf("(%s - %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
438439
}
439440

440441
return a - b, false, nil
@@ -578,7 +579,7 @@ func (s *builtinArithmeticMultiplyRealSig) evalReal(ctx EvalContext, row chunk.R
578579
}
579580
result := a * b
580581
if math.IsInf(result, 0) {
581-
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s * %s)", s.args[0].String(), s.args[1].String()))
582+
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s * %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
582583
}
583584
return result, false, nil
584585
}
@@ -596,7 +597,7 @@ func (s *builtinArithmeticMultiplyDecimalSig) evalDecimal(ctx EvalContext, row c
596597
err = types.DecimalMul(a, b, c)
597598
if err != nil && !terror.ErrorEqual(err, types.ErrTruncated) {
598599
if err == types.ErrOverflow {
599-
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s * %s)", s.args[0].String(), s.args[1].String()))
600+
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s * %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
600601
}
601602
return nil, true, err
602603
}
@@ -616,7 +617,7 @@ func (s *builtinArithmeticMultiplyIntUnsignedSig) evalInt(ctx EvalContext, row c
616617
unsignedB := uint64(b)
617618
result := unsignedA * unsignedB
618619
if unsignedA != 0 && result/unsignedA != unsignedB {
619-
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s * %s)", s.args[0].String(), s.args[1].String()))
620+
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s * %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
620621
}
621622
return int64(result), false, nil
622623
}
@@ -632,7 +633,7 @@ func (s *builtinArithmeticMultiplyIntSig) evalInt(ctx EvalContext, row chunk.Row
632633
}
633634
result := a * b
634635
if (a != 0 && result/a != b) || (result == math.MinInt64 && a == -1) {
635-
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s * %s)", s.args[0].String(), s.args[1].String()))
636+
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s * %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
636637
}
637638
return result, false, nil
638639
}
@@ -697,7 +698,7 @@ func (s *builtinArithmeticDivideRealSig) evalReal(ctx EvalContext, row chunk.Row
697698
}
698699
result := a / b
699700
if math.IsInf(result, 0) {
700-
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s / %s)", s.args[0].String(), s.args[1].String()))
701+
return 0, true, types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s / %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
701702
}
702703
return result, false, nil
703704
}
@@ -726,7 +727,7 @@ func (s *builtinArithmeticDivideDecimalSig) evalDecimal(ctx EvalContext, row chu
726727
err = c.Round(c, s.baseBuiltinFunc.tp.GetDecimal(), types.ModeHalfUp)
727728
}
728729
} else if err == types.ErrOverflow {
729-
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s / %s)", s.args[0].String(), s.args[1].String()))
730+
err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s / %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
730731
}
731732
return c, false, err
732733
}
@@ -857,14 +858,14 @@ func (s *builtinArithmeticIntDivideDecimalSig) evalInt(ctx EvalContext, row chun
857858
ret = int64(0)
858859
return ret, false, nil
859860
}
860-
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s DIV %s)", s.args[0].String(), s.args[1].String()))
861+
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)))
861862
}
862863
ret = int64(val)
863864
} else {
864865
ret, err = c.ToInt()
865866
// err returned by ToInt may be ErrTruncated or ErrOverflow, only handle ErrOverflow, ignore ErrTruncated.
866867
if err == types.ErrOverflow {
867-
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s DIV %s)", s.args[0].String(), s.args[1].String()))
868+
return 0, true, types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s DIV %s)", s.args[0].StringWithCtx(errors.RedactLogDisable), s.args[1].StringWithCtx(errors.RedactLogDisable)))
868869
}
869870
}
870871

0 commit comments

Comments
 (0)