From a7df4f9845d5d6a590c5d45dad0dcc9f21aa8765 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 22 Aug 2024 13:45:13 +0800 Subject: [PATCH] *: support tidb_redact_log for explain (#54553) (#55308) close pingcap/tidb#54565 --- pkg/executor/aggfuncs/func_group_concat.go | 5 +- pkg/executor/importer/import.go | 2 +- pkg/expression/BUILD.bazel | 1 + pkg/expression/aggregation/agg_to_pb.go | 4 +- pkg/expression/aggregation/base_func.go | 9 +- pkg/expression/aggregation/concat.go | 2 +- pkg/expression/aggregation/descriptor.go | 9 +- pkg/expression/bench_test.go | 3 +- pkg/expression/builtin_arithmetic.go | 39 +-- pkg/expression/builtin_arithmetic_vec.go | 35 +-- pkg/expression/builtin_cast.go | 3 +- pkg/expression/builtin_cast_test.go | 3 +- pkg/expression/builtin_cast_vec.go | 3 +- pkg/expression/builtin_compare.go | 6 +- pkg/expression/builtin_compare_test.go | 2 +- pkg/expression/builtin_math.go | 3 +- pkg/expression/builtin_math_vec.go | 3 +- pkg/expression/builtin_string.go | 13 +- pkg/expression/column.go | 12 +- pkg/expression/constant.go | 13 +- pkg/expression/constant_test.go | 7 +- pkg/expression/context.go | 7 + pkg/expression/context/context.go | 2 + pkg/expression/contextimpl/sessionctx.go | 5 + pkg/expression/explain.go | 43 ++- pkg/expression/expr_to_pb.go | 2 +- pkg/expression/expression.go | 21 +- pkg/expression/expression_test.go | 7 +- pkg/expression/grouping_sets.go | 22 +- pkg/expression/grouping_sets_test.go | 15 +- pkg/expression/infer_pushdown.go | 4 +- pkg/expression/scalar_function.go | 9 +- pkg/expression/util.go | 2 +- pkg/expression/util_test.go | 2 +- pkg/parser/digester_test.go | 5 + pkg/planner/cardinality/selectivity.go | 2 +- pkg/planner/cardinality/trace.go | 3 +- pkg/planner/cascades/BUILD.bazel | 1 + pkg/planner/cascades/stringer.go | 5 +- pkg/planner/core/BUILD.bazel | 1 + .../testdata/enforce_mpp_suite_out.json | 2 +- pkg/planner/core/casetest/integration_test.go | 11 +- .../testdata/integration_suite_out.json | 128 +++++++++ pkg/planner/core/debugtrace.go | 3 +- pkg/planner/core/exhaust_physical_plans.go | 7 +- pkg/planner/core/explain.go | 68 +++-- pkg/planner/core/expression_rewriter.go | 2 +- pkg/planner/core/handle_cols.go | 13 + pkg/planner/core/indexmerge_test.go | 3 +- pkg/planner/core/logical_plan_builder.go | 4 +- pkg/planner/core/logical_plans_test.go | 22 +- pkg/planner/core/physical_plan_test.go | 3 +- pkg/planner/core/physical_plans.go | 10 +- pkg/planner/core/plan_cache_test.go | 32 ++- pkg/planner/core/point_get_plan.go | 11 +- .../core/rule_aggregation_push_down.go | 7 +- pkg/planner/core/rule_decorrelate.go | 9 +- pkg/planner/core/rule_eliminate_projection.go | 3 +- .../core/rule_generate_column_substitute.go | 5 +- pkg/planner/core/rule_join_elimination.go | 7 +- pkg/planner/core/rule_partition_processor.go | 2 +- pkg/planner/core/rule_predicate_push_down.go | 7 +- pkg/planner/core/rule_topn_push_down.go | 5 +- pkg/planner/core/runtime_filter.go | 5 +- pkg/planner/core/runtime_filter_generator.go | 7 +- pkg/planner/core/stringer.go | 17 +- pkg/planner/core/tests/redact/BUILD.bazel | 18 ++ pkg/planner/core/tests/redact/main_test.go | 34 +++ pkg/planner/core/tests/redact/redact_test.go | 271 ++++++++++++++++++ pkg/planner/util/BUILD.bazel | 1 + pkg/planner/util/byitem.go | 25 +- pkg/table/tables/test/partition/BUILD.bazel | 1 + .../tables/test/partition/partition_test.go | 3 +- pkg/util/ranger/points.go | 4 +- pkg/util/ranger/types.go | 27 +- pkg/util/redact/redact.go | 14 + .../r/executor/jointest/join.result | 2 +- .../r/executor/merge_join.result | 4 +- 78 files changed, 940 insertions(+), 192 deletions(-) create mode 100644 pkg/planner/core/tests/redact/BUILD.bazel create mode 100644 pkg/planner/core/tests/redact/main_test.go create mode 100644 pkg/planner/core/tests/redact/redact_test.go diff --git a/pkg/executor/aggfuncs/func_group_concat.go b/pkg/executor/aggfuncs/func_group_concat.go index f15b9936759be..de7c5ef35c3e1 100644 --- a/pkg/executor/aggfuncs/func_group_concat.go +++ b/pkg/executor/aggfuncs/func_group_concat.go @@ -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" @@ -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 } diff --git a/pkg/executor/importer/import.go b/pkg/executor/importer/import.go index 07889f31bf622..9cbd8b0f99078 100644 --- a/pkg/executor/importer/import.go +++ b/pkg/executor/importer/import.go @@ -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) } diff --git a/pkg/expression/BUILD.bazel b/pkg/expression/BUILD.bazel index e9f86fb4bc341..4fc61f61e4ffd 100644 --- a/pkg/expression/BUILD.bazel +++ b/pkg/expression/BUILD.bazel @@ -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", diff --git a/pkg/expression/aggregation/agg_to_pb.go b/pkg/expression/aggregation/agg_to_pb.go index 36d4912535813..41448d69508b9 100644 --- a/pkg/expression/aggregation/agg_to_pb.go +++ b/pkg/expression/aggregation/agg_to_pb.go @@ -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) } @@ -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) } diff --git a/pkg/expression/aggregation/base_func.go b/pkg/expression/aggregation/base_func.go index 43876b8888d0c..8732a810e63aa 100644 --- a/pkg/expression/aggregation/base_func.go +++ b/pkg/expression/aggregation/base_func.go @@ -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(", ") } @@ -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 { diff --git a/pkg/expression/aggregation/concat.go b/pkg/expression/aggregation/concat.go index e023c6128646d..c490c72416e32 100644 --- a/pkg/expression/aggregation/concat.go +++ b/pkg/expression/aggregation/concat.go @@ -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 } diff --git a/pkg/expression/aggregation/descriptor.go b/pkg/expression/aggregation/descriptor.go index f83dd6c6bd1ae..054e746b84eec 100644 --- a/pkg/expression/aggregation/descriptor.go +++ b/pkg/expression/aggregation/descriptor.go @@ -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(", ") } @@ -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(", ") } diff --git a/pkg/expression/bench_test.go b/pkg/expression/bench_test.go index 1144a68f99a8b..87bfe9d7905df 100644 --- a/pkg/expression/bench_test.go +++ b/pkg/expression/bench_test.go @@ -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" @@ -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, ".") diff --git a/pkg/expression/builtin_arithmetic.go b/pkg/expression/builtin_arithmetic.go index 71415c1638ffc..412bfc28597c1 100644 --- a/pkg/expression/builtin_arithmetic.go +++ b/pkg/expression/builtin_arithmetic.go @@ -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" @@ -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))) } } @@ -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 } @@ -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 } @@ -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 } @@ -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 } @@ -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 @@ -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 } @@ -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 } @@ -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 } @@ -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 } @@ -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 } @@ -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 } @@ -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))) } } diff --git a/pkg/expression/builtin_arithmetic_vec.go b/pkg/expression/builtin_arithmetic_vec.go index f4eaa6d11af6d..61e0ffa6edaaf 100644 --- a/pkg/expression/builtin_arithmetic_vec.go +++ b/pkg/expression/builtin_arithmetic_vec.go @@ -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" @@ -53,7 +54,7 @@ func (b *builtinArithmeticMultiplyRealSig) vecEvalReal(ctx EvalContext, input *c if result.IsNull(i) { continue } - return types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s * %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s * %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } } return nil @@ -106,7 +107,7 @@ func (b *builtinArithmeticDivideDecimalSig) vecEvalDecimal(ctx EvalContext, inpu } } } else if err == types.ErrOverflow { - return types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s / %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s / %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } else { return err } @@ -313,7 +314,7 @@ func (b *builtinArithmeticMinusRealSig) vecEvalReal(ctx EvalContext, input *chun if result.IsNull(i) { continue } - return types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s - %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s - %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } x[i] = x[i] - y[i] } @@ -348,7 +349,7 @@ func (b *builtinArithmeticMinusDecimalSig) vecEvalDecimal(ctx EvalContext, input } if err = types.DecimalSub(&x[i], &y[i], &to); err != nil { if err == types.ErrOverflow { - err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s - %s)", b.args[0].String(), b.args[1].String())) + err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s - %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } return err } @@ -400,7 +401,7 @@ func (b *builtinArithmeticMinusIntSig) vecEvalInt(ctx EvalContext, input *chunk. if result.IsNull(i) { continue } - return types.ErrOverflow.GenWithStackByArgs(errType, fmt.Sprintf("(%s - %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs(errType, fmt.Sprintf("(%s - %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } resulti64s[i] = lh - rh @@ -514,7 +515,7 @@ func (b *builtinArithmeticPlusRealSig) vecEvalReal(ctx EvalContext, input *chunk if result.IsNull(i) { continue } - return types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s + %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s + %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } x[i] = x[i] + y[i] } @@ -550,7 +551,7 @@ func (b *builtinArithmeticMultiplyDecimalSig) vecEvalDecimal(ctx EvalContext, in err = types.DecimalMul(&x[i], &y[i], &to) if err != nil && !terror.ErrorEqual(err, types.ErrTruncated) { if err == types.ErrOverflow { - err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s * %s)", b.args[0].String(), b.args[1].String())) + err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s * %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } return err } @@ -668,7 +669,7 @@ func (b *builtinArithmeticMultiplyIntSig) vecEvalInt(ctx EvalContext, input *chu continue } result.SetNull(i, true) - return types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s * %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s * %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } x[i] = tmp @@ -712,7 +713,7 @@ func (b *builtinArithmeticDivideRealSig) vecEvalReal(ctx EvalContext, input *chu x[i] = x[i] / y[i] if math.IsInf(x[i], 0) { - return types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s / %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("DOUBLE", fmt.Sprintf("(%s / %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } } return nil @@ -898,7 +899,7 @@ func (b *builtinArithmeticPlusIntSig) plusUU(result *chunk.Column, lhi64s, rhi64 if result.IsNull(i) { continue } - return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } resulti64s[i] = lh + rh @@ -914,13 +915,13 @@ func (b *builtinArithmeticPlusIntSig) plusUS(result *chunk.Column, lhi64s, rhi64 if result.IsNull(i) { continue } - return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } if rh > 0 && uint64(lh) > math.MaxUint64-uint64(rh) { if result.IsNull(i) { continue } - return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } resulti64s[i] = lh + rh @@ -936,13 +937,13 @@ func (b *builtinArithmeticPlusIntSig) plusSU(result *chunk.Column, lhi64s, rhi64 if result.IsNull(i) { continue } - return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } if lh > 0 && uint64(rh) > math.MaxUint64-uint64(lh) { if result.IsNull(i) { continue } - return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } resulti64s[i] = lh + rh @@ -957,7 +958,7 @@ func (b *builtinArithmeticPlusIntSig) plusSS(result *chunk.Column, lhi64s, rhi64 if result.IsNull(i) { continue } - return types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s + %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("BIGINT", fmt.Sprintf("(%s + %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } resulti64s[i] = lh + rh @@ -993,7 +994,7 @@ func (b *builtinArithmeticPlusDecimalSig) vecEvalDecimal(ctx EvalContext, input } if err = types.DecimalAdd(&x[i], &y[i], to); err != nil { if err == types.ErrOverflow { - err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s + %s)", b.args[0].String(), b.args[1].String())) + err = types.ErrOverflow.GenWithStackByArgs("DECIMAL", fmt.Sprintf("(%s + %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } return err } @@ -1031,7 +1032,7 @@ func (b *builtinArithmeticMultiplyIntUnsignedSig) vecEvalInt(ctx EvalContext, in if result.IsNull(i) { continue } - return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s * %s)", b.args[0].String(), b.args[1].String())) + return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s * %s)", b.args[0].StringWithCtx(errors.RedactLogDisable), b.args[1].StringWithCtx(errors.RedactLogDisable))) } x[i] = res } diff --git a/pkg/expression/builtin_cast.go b/pkg/expression/builtin_cast.go index 5ff81283b740e..33ee29323f73a 100644 --- a/pkg/expression/builtin_cast.go +++ b/pkg/expression/builtin_cast.go @@ -29,6 +29,7 @@ import ( "strings" gotime "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/model" @@ -1008,7 +1009,7 @@ func (b *builtinCastRealAsDecimalSig) evalDecimal(ctx EvalContext, row chunk.Row if !b.inUnion || val >= 0 { err = res.FromFloat64(val) if types.ErrOverflow.Equal(err) { - warnErr := types.ErrTruncatedWrongVal.GenWithStackByArgs("DECIMAL", b.args[0]) + warnErr := types.ErrTruncatedWrongVal.GenWithStackByArgs("DECIMAL", b.args[0].StringWithCtx(errors.RedactLogDisable)) err = ec.HandleErrorWithAlias(err, err, warnErr) } else if types.ErrTruncated.Equal(err) { // This behavior is consistent with MySQL. diff --git a/pkg/expression/builtin_cast_test.go b/pkg/expression/builtin_cast_test.go index aecb7977018f2..d2c54affc2d1b 100644 --- a/pkg/expression/builtin_cast_test.go +++ b/pkg/expression/builtin_cast_test.go @@ -21,6 +21,7 @@ import ( "testing" "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" @@ -1446,7 +1447,7 @@ func TestWrapWithCastAsString(t *testing.T) { } expr := BuildCastFunction(ctx, &Constant{RetType: types.NewFieldType(mysql.TypeEnum)}, types.NewFieldType(mysql.TypeVarString)) - require.NotContains(t, expr.String(), "to_binary") + require.NotContains(t, expr.StringWithCtx(errors.RedactLogDisable), "to_binary") } func TestWrapWithCastAsJSON(t *testing.T) { diff --git a/pkg/expression/builtin_cast_vec.go b/pkg/expression/builtin_cast_vec.go index bce2383171338..1b178f32956aa 100644 --- a/pkg/expression/builtin_cast_vec.go +++ b/pkg/expression/builtin_cast_vec.go @@ -21,6 +21,7 @@ import ( "strings" gotime "time" + perrors "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -916,7 +917,7 @@ func (b *builtinCastRealAsDecimalSig) vecEvalDecimal(ctx EvalContext, input *chu if !b.inUnion || bufreal[i] >= 0 { if err = resdecimal[i].FromFloat64(bufreal[i]); err != nil { if types.ErrOverflow.Equal(err) { - warnErr := types.ErrTruncatedWrongVal.GenWithStackByArgs("DECIMAL", b.args[0]) + warnErr := types.ErrTruncatedWrongVal.GenWithStackByArgs("DECIMAL", b.args[0].StringWithCtx(perrors.RedactLogDisable)) err = ec.HandleErrorWithAlias(err, err, warnErr) } else if types.ErrTruncated.Equal(err) { // This behavior is consistent with MySQL. diff --git a/pkg/expression/builtin_compare.go b/pkg/expression/builtin_compare.go index 8e5c36707ec8f..43a38d70df374 100644 --- a/pkg/expression/builtin_compare.go +++ b/pkg/expression/builtin_compare.go @@ -1535,7 +1535,7 @@ func allowCmpArgsRefining4PlanCache(ctx BuildContext, args []Expression) (allowR exprType := args[1-conIdx].GetType() exprEvalType := exprType.EvalType() if exprType.GetType() == mysql.TypeYear { - reason := errors.NewNoStackErrorf("'%v' may be converted to INT", args[conIdx].String()) + reason := errors.NewNoStackErrorf("'%v' may be converted to INT", args[conIdx].StringWithCtx(errors.RedactLogDisable)) ctx.SetSkipPlanCache(reason) return true } @@ -1545,7 +1545,7 @@ func allowCmpArgsRefining4PlanCache(ctx BuildContext, args []Expression) (allowR conEvalType := args[conIdx].GetType().EvalType() if exprEvalType == types.ETInt && (conEvalType == types.ETString || conEvalType == types.ETReal || conEvalType == types.ETDecimal) { - reason := errors.NewNoStackErrorf("'%v' may be converted to INT", args[conIdx].String()) + reason := errors.NewNoStackErrorf("'%v' may be converted to INT", args[conIdx].StringWithCtx(errors.RedactLogDisable)) ctx.SetSkipPlanCache(reason) return true } @@ -1555,7 +1555,7 @@ func allowCmpArgsRefining4PlanCache(ctx BuildContext, args []Expression) (allowR // see https://github.com/pingcap/tidb/issues/38361 for more details _, exprIsCon := args[1-conIdx].(*Constant) if !exprIsCon && matchRefineRule3Pattern(conEvalType, exprType) { - reason := errors.Errorf("'%v' may be converted to datetime", args[conIdx].String()) + reason := errors.Errorf("'%v' may be converted to datetime", args[conIdx].StringWithCtx(errors.RedactLogDisable)) ctx.SetSkipPlanCache(reason) return true } diff --git a/pkg/expression/builtin_compare_test.go b/pkg/expression/builtin_compare_test.go index a8493a8d96139..4433198c6cc19 100644 --- a/pkg/expression/builtin_compare_test.go +++ b/pkg/expression/builtin_compare_test.go @@ -73,7 +73,7 @@ func TestCompareFunctionWithRefine(t *testing.T) { for _, test := range tests { f, err := ParseSimpleExpr(ctx, test.exprStr, WithTableInfo("", tblInfo)) require.NoError(t, err) - require.Equal(t, test.result, f.String()) + require.Equal(t, test.result, f.StringWithCtx(errors.RedactLogDisable)) } } diff --git a/pkg/expression/builtin_math.go b/pkg/expression/builtin_math.go index 2937ebbab8e15..a701d7348cda9 100644 --- a/pkg/expression/builtin_math.go +++ b/pkg/expression/builtin_math.go @@ -25,6 +25,7 @@ import ( "strconv" "strings" + perrors "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" @@ -1739,7 +1740,7 @@ func (b *builtinExpSig) evalReal(ctx EvalContext, row chunk.Row) (float64, bool, } exp := math.Exp(val) if math.IsInf(exp, 0) || math.IsNaN(exp) { - s := fmt.Sprintf("exp(%s)", b.args[0].String()) + s := fmt.Sprintf("exp(%s)", b.args[0].StringWithCtx(perrors.RedactLogDisable)) return 0, false, types.ErrOverflow.GenWithStackByArgs("DOUBLE", s) } return exp, false, nil diff --git a/pkg/expression/builtin_math_vec.go b/pkg/expression/builtin_math_vec.go index 8a3e05b66baf5..182404fa43787 100644 --- a/pkg/expression/builtin_math_vec.go +++ b/pkg/expression/builtin_math_vec.go @@ -20,6 +20,7 @@ import ( "math" "strconv" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -290,7 +291,7 @@ func (b *builtinExpSig) vecEvalReal(ctx EvalContext, input *chunk.Chunk, result } exp := math.Exp(f64s[i]) if math.IsInf(exp, 0) || math.IsNaN(exp) { - s := fmt.Sprintf("exp(%s)", b.args[0].String()) + s := fmt.Sprintf("exp(%s)", b.args[0].StringWithCtx(errors.RedactLogDisable)) if err := types.ErrOverflow.GenWithStackByArgs("DOUBLE", s); err != nil { return err } diff --git a/pkg/expression/builtin_string.go b/pkg/expression/builtin_string.go index d001d050d1785..b9b3ee3f6e7cc 100644 --- a/pkg/expression/builtin_string.go +++ b/pkg/expression/builtin_string.go @@ -2311,7 +2311,7 @@ func (c *charFunctionClass) getFunction(ctx BuildContext, args []Expression) (bu // The last argument represents the charset name after "using". if _, ok := args[len(args)-1].(*Constant); !ok { // If we got there, there must be something wrong in other places. - logutil.BgLogger().Warn(fmt.Sprintf("The last argument in char function must be constant, but got %T", args[len(args)-1])) + logutil.BgLogger().Warn(fmt.Sprintf("The last argument in char function must be constant, but got %T", args[len(args)-1].StringWithCtx(errors.RedactLogDisable))) return nil, errIncorrectArgs } charsetName, isNull, err := args[len(args)-1].EvalString(ctx.GetEvalCtx(), chunk.Row{}) @@ -3867,11 +3867,11 @@ func (c *weightStringFunctionClass) verifyArgs(args []Expression) (weightStringP length := 0 if l == 3 { if args[1].GetType().EvalType() != types.ETString { - return weightStringPaddingNone, 0, ErrIncorrectType.GenWithStackByArgs(args[1].String(), c.funcName) + return weightStringPaddingNone, 0, ErrIncorrectType.GenWithStackByArgs(args[1].StringWithCtx(errors.RedactLogDisable), c.funcName) } c1, ok := args[1].(*Constant) if !ok { - return weightStringPaddingNone, 0, ErrIncorrectType.GenWithStackByArgs(args[1].String(), c.funcName) + return weightStringPaddingNone, 0, ErrIncorrectType.GenWithStackByArgs(args[1].StringWithCtx(errors.RedactLogDisable), c.funcName) } switch x := c1.Value.GetString(); x { case "CHAR": @@ -3883,16 +3883,17 @@ func (c *weightStringFunctionClass) verifyArgs(args []Expression) (weightStringP default: return weightStringPaddingNone, 0, ErrIncorrectType.GenWithStackByArgs(x, c.funcName) } + if args[2].GetType().EvalType() != types.ETInt { - return weightStringPaddingNone, 0, ErrIncorrectType.GenWithStackByArgs(args[2].String(), c.funcName) + return weightStringPaddingNone, 0, ErrIncorrectType.GenWithStackByArgs(args[2].StringWithCtx(errors.RedactLogDisable), c.funcName) } c2, ok := args[2].(*Constant) if !ok { - return weightStringPaddingNone, 0, ErrIncorrectType.GenWithStackByArgs(args[1].String(), c.funcName) + return weightStringPaddingNone, 0, ErrIncorrectType.GenWithStackByArgs(args[1].StringWithCtx(errors.RedactLogDisable), c.funcName) } length = int(c2.Value.GetInt64()) if length == 0 { - return weightStringPaddingNone, 0, ErrIncorrectType.GenWithStackByArgs(args[2].String(), c.funcName) + return weightStringPaddingNone, 0, ErrIncorrectType.GenWithStackByArgs(args[2].StringWithCtx(errors.RedactLogDisable), c.funcName) } } return padding, length, nil diff --git a/pkg/expression/column.go b/pkg/expression/column.go index 5920f6b70f20b..9cb639226a550 100644 --- a/pkg/expression/column.go +++ b/pkg/expression/column.go @@ -388,11 +388,21 @@ func (col *Column) VecEvalJSON(ctx EvalContext, input *chunk.Chunk, result *chun const columnPrefix = "Column#" +// StringWithCtx implements Expression interface. +func (col *Column) StringWithCtx(redact string) string { + return col.string(redact) +} + // String implements Stringer interface. func (col *Column) String() string { + return col.string(errors.RedactLogDisable) +} + +func (col *Column) string(redact string) string { if col.IsHidden && col.VirtualExpr != nil { // A hidden column without virtual expression indicates it's a stored type. - return col.VirtualExpr.String() + // a virtual column should be able to be stringified without context. + return col.VirtualExpr.StringWithCtx(redact) } if col.OrigName != "" { return col.OrigName diff --git a/pkg/expression/constant.go b/pkg/expression/constant.go index 18e7e8e2cd97e..fccf3534a55ab 100644 --- a/pkg/expression/constant.go +++ b/pkg/expression/constant.go @@ -18,6 +18,7 @@ import ( "fmt" "unsafe" + perrors "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/types" @@ -142,13 +143,23 @@ func (d *ParamMarker) GetUserVar() types.Datum { // String implements fmt.Stringer interface. func (c *Constant) String() string { + return c.StringWithCtx(perrors.RedactLogDisable) +} + +// StringWithCtx implements Expression interface. +func (c *Constant) StringWithCtx(redact string) string { if c.ParamMarker != nil { dt := c.ParamMarker.GetUserVar() c.Value.SetValue(dt.GetValue(), c.RetType) } else if c.DeferredExpr != nil { return c.DeferredExpr.String() } - return fmt.Sprintf("%v", c.Value.GetValue()) + if redact == perrors.RedactLogDisable { + return fmt.Sprintf("%v", c.Value.GetValue()) + } else if redact == perrors.RedactLogMarker { + return fmt.Sprintf("‹%v›", c.Value.GetValue()) + } + return "?" } // MarshalJSON implements json.Marshaler interface. diff --git a/pkg/expression/constant_test.go b/pkg/expression/constant_test.go index 652779fe9bb46..0e6e16a15e749 100644 --- a/pkg/expression/constant_test.go +++ b/pkg/expression/constant_test.go @@ -21,6 +21,7 @@ import ( "testing" "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" @@ -184,7 +185,7 @@ func TestConstantPropagation(t *testing.T) { newConds := solver.PropagateConstant(ctx, conds) var result []string for _, v := range newConds { - result = append(result, v.String()) + result = append(result, v.StringWithCtx(errors.RedactLogDisable)) } sort.Strings(result) require.Equalf(t, tt.result, strings.Join(result, ", "), "different for expr %s", tt.conditions) @@ -246,7 +247,7 @@ func TestConstantFolding(t *testing.T) { ctx := mock.NewContext() expr := tt.condition(ctx) newConds := FoldConstant(ctx, expr) - require.Equalf(t, tt.result, newConds.String(), "different for expr %s", tt.condition) + require.Equalf(t, tt.result, newConds.StringWithCtx(errors.RedactLogDisable), "different for expr %s", tt.condition) } } @@ -308,7 +309,7 @@ func TestConstantFoldingCharsetConvert(t *testing.T) { } for _, tt := range tests { newConds := FoldConstant(ctx, tt.condition) - require.Equalf(t, tt.result, newConds.String(), "different for expr %s", tt.condition) + require.Equalf(t, tt.result, newConds.StringWithCtx(errors.RedactLogDisable), "different for expr %s", tt.condition) } } diff --git a/pkg/expression/context.go b/pkg/expression/context.go index 00ca3f4fa1bdf..815357db075ae 100644 --- a/pkg/expression/context.go +++ b/pkg/expression/context.go @@ -121,3 +121,10 @@ func (ctx *assertionEvalContext) GetOptionalPropProvider(key OptionalEvalPropKey ) return ctx.EvalContext.GetOptionalPropProvider(key) } + +// StringerWithCtx is the interface for expressions that can be stringified with context. +type StringerWithCtx interface { + // StringWithCtx returns the string representation of the expression with context. + // NOTE: any implementation of `StringWithCtx` should not panic if the context is nil. + StringWithCtx(redact string) string +} diff --git a/pkg/expression/context/context.go b/pkg/expression/context/context.go index 73a817fdf476a..3f70c8bbc5fb7 100644 --- a/pkg/expression/context/context.go +++ b/pkg/expression/context/context.go @@ -51,6 +51,8 @@ type EvalContext interface { CurrentTime() (time.Time, error) // GetMaxAllowedPacket returns the value of the 'max_allowed_packet' system variable. GetMaxAllowedPacket() uint64 + // GetTiDBRedactLog returns the value of the 'tidb_redact_log' system variable. + GetTiDBRedactLog() string // GetDefaultWeekFormatMode returns the value of the 'default_week_format' system variable. GetDefaultWeekFormatMode() string // GetDivPrecisionIncrement returns the specified value of DivPrecisionIncrement. diff --git a/pkg/expression/contextimpl/sessionctx.go b/pkg/expression/contextimpl/sessionctx.go index 5d792f2c9b983..50666876f698c 100644 --- a/pkg/expression/contextimpl/sessionctx.go +++ b/pkg/expression/contextimpl/sessionctx.go @@ -211,6 +211,11 @@ func (ctx *SessionEvalContext) GetMaxAllowedPacket() uint64 { return ctx.sctx.GetSessionVars().MaxAllowedPacket } +// GetTiDBRedactLog returns the value of the 'tidb_redact_log' system variable. +func (ctx *SessionEvalContext) GetTiDBRedactLog() string { + return ctx.sctx.GetSessionVars().EnableRedactLog +} + // GetDefaultWeekFormatMode returns the value of the 'default_week_format' system variable. func (ctx *SessionEvalContext) GetDefaultWeekFormatMode() string { mode, ok := ctx.sctx.GetSessionVars().GetSystemVar(variable.DefaultWeekFormat) diff --git a/pkg/expression/explain.go b/pkg/expression/explain.go index 3813915c20ad7..284c6e80e2911 100644 --- a/pkg/expression/explain.go +++ b/pkg/expression/explain.go @@ -20,10 +20,12 @@ import ( "slices" "strings" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/intest" + "github.com/pingcap/tidb/pkg/util/redact" ) // ExplainInfo implements the Expression interface. @@ -103,7 +105,15 @@ func (col *Column) ColumnExplainInfo(normalized bool) string { } return "?" } - return col.String() + return col.StringWithCtx(errors.RedactLogDisable) +} + +// ColumnExplainInfoNormalized returns the normalized explained info for column. +func (col *Column) ColumnExplainInfoNormalized() string { + if col.OrigName != "" { + return col.OrigName + } + return "?" } // ExplainInfo implements the Expression interface. @@ -123,10 +133,21 @@ func (col *Column) ExplainNormalizedInfo4InList() string { // ExplainInfo implements the Expression interface. func (expr *Constant) ExplainInfo(ctx EvalContext) string { + redact := ctx.GetTiDBRedactLog() + if redact == errors.RedactLogEnable { + return "?" + } dt, err := expr.Eval(ctx, chunk.Row{}) if err != nil { return "not recognized const value" } + if redact == errors.RedactLogMarker { + builder := new(strings.Builder) + builder.WriteString("‹") + builder.WriteString(expr.format(dt)) + builder.WriteString("›") + return builder.String() + } return expr.format(dt) } @@ -152,32 +173,32 @@ func (expr *Constant) format(dt types.Datum) string { } // ExplainExpressionList generates explain information for a list of expressions. -func ExplainExpressionList(exprs []Expression, schema *Schema) string { +func ExplainExpressionList(exprs []Expression, schema *Schema, redactMode string) string { builder := &strings.Builder{} for i, expr := range exprs { switch expr.(type) { case *Column, *CorrelatedColumn: - builder.WriteString(expr.String()) - if expr.String() != schema.Columns[i].String() { + builder.WriteString(expr.StringWithCtx(redactMode)) + if expr.StringWithCtx(redactMode) != schema.Columns[i].StringWithCtx(redactMode) { // simple col projected again with another uniqueID without origin name. builder.WriteString("->") - builder.WriteString(schema.Columns[i].String()) + builder.WriteString(schema.Columns[i].StringWithCtx(redactMode)) } case *Constant: - v := expr.String() + v := expr.StringWithCtx(errors.RedactLogDisable) length := 64 if len(v) < length { - builder.WriteString(v) + redact.WriteRedact(builder, v, redactMode) } else { - builder.WriteString(v[:length]) + redact.WriteRedact(builder, v[:length], redactMode) fmt.Fprintf(builder, "(len:%d)", len(v)) } builder.WriteString("->") - builder.WriteString(schema.Columns[i].String()) + builder.WriteString(schema.Columns[i].StringWithCtx(redactMode)) default: - builder.WriteString(expr.String()) + builder.WriteString(expr.StringWithCtx(redactMode)) builder.WriteString("->") - builder.WriteString(schema.Columns[i].String()) + builder.WriteString(schema.Columns[i].StringWithCtx(redactMode)) } if i+1 < len(exprs) { builder.WriteString(", ") diff --git a/pkg/expression/expr_to_pb.go b/pkg/expression/expr_to_pb.go index 39f1cde72293d..c4a2a2106a1e4 100644 --- a/pkg/expression/expr_to_pb.go +++ b/pkg/expression/expr_to_pb.go @@ -39,7 +39,7 @@ func ExpressionsToPBList(ctx EvalContext, exprs []Expression, client kv.Client) for _, expr := range exprs { v := pc.ExprToPB(expr) if v == nil { - return nil, plannererrors.ErrInternal.GenWithStack("expression %v cannot be pushed down", expr) + return nil, plannererrors.ErrInternal.GenWithStack("expression %v cannot be pushed down", expr.StringWithCtx(errors.RedactLogDisable)) } pbExpr = append(pbExpr, v) } diff --git a/pkg/expression/expression.go b/pkg/expression/expression.go index f8084d2d87f0f..18b305bffd9de 100644 --- a/pkg/expression/expression.go +++ b/pkg/expression/expression.go @@ -17,6 +17,7 @@ package expression import ( goJSON "encoding/json" "fmt" + "strings" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" @@ -250,6 +251,9 @@ type Expression interface { // MemoryUsage return the memory usage of Expression MemoryUsage() int64 + + // StringWithCtx returns the string representation of the expression with context. + StringWithCtx(redact string) string } // CNFExprs stands for a CNF expression. @@ -864,7 +868,7 @@ func SplitDNFItems(onExpr Expression) []Expression { // If the Expression is a non-constant value, it means the result is unknown. func EvaluateExprWithNull(ctx BuildContext, schema *Schema, expr Expression) Expression { if MaybeOverOptimized4PlanCache(ctx, []Expression{expr}) { - ctx.SetSkipPlanCache(errors.NewNoStackError("%v affects null check")) + ctx.SetSkipPlanCache(errors.NewNoStackErrorf("%v affects null check", expr.StringWithCtx(errors.RedactLogDisable))) } if ctx.GetSessionVars().StmtCtx.InNullRejectCheck { expr, _ = evaluateExprWithNullInNullRejectCheck(ctx, schema, expr) @@ -1229,3 +1233,18 @@ func Args2Expressions4Test(args ...any) []Expression { } return exprs } + +// StringifyExpressionsWithCtx turns a slice of expressions into string +func StringifyExpressionsWithCtx(ctx EvalContext, exprs []Expression) string { + var sb strings.Builder + sb.WriteString("[") + for i, expr := range exprs { + sb.WriteString(expr.StringWithCtx(errors.RedactLogDisable)) + + if i != len(exprs)-1 { + sb.WriteString(" ") + } + } + sb.WriteString("]") + return sb.String() +} diff --git a/pkg/expression/expression_test.go b/pkg/expression/expression_test.go index c19d3796480fa..c5dcf633b5a0b 100644 --- a/pkg/expression/expression_test.go +++ b/pkg/expression/expression_test.go @@ -17,6 +17,7 @@ package expression import ( "testing" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -47,7 +48,9 @@ func TestEvaluateExprWithNull(t *testing.T) { require.NoError(t, err) res := EvaluateExprWithNull(ctx, schema, outerIfNull) - require.Equal(t, "ifnull(Column#1, 1)", res.String()) + require.Equal(t, "ifnull(Column#1, 1)", res.StringWithCtx(errors.RedactLogDisable)) + require.Equal(t, "ifnull(Column#1, ?)", res.StringWithCtx(errors.RedactLogEnable)) + require.Equal(t, "ifnull(Column#1, ‹1›)", res.StringWithCtx(errors.RedactLogMarker)) schema.Columns = append(schema.Columns, col1) // ifnull(null, ifnull(null, 1)) res = EvaluateExprWithNull(ctx, schema, outerIfNull) @@ -148,7 +151,7 @@ func TestConstLevel(t *testing.T) { {newFunctionWithMockCtx(ast.Plus, NewOne(), newColumn(1)), ConstNone}, {newFunctionWithMockCtx(ast.Plus, NewOne(), ctxConst), ConstOnlyInContext}, } { - require.Equal(t, c.level, c.exp.ConstLevel(), c.exp.String()) + require.Equal(t, c.level, c.exp.ConstLevel(), c.exp.StringWithCtx(errors.RedactLogDisable)) } } diff --git a/pkg/expression/grouping_sets.go b/pkg/expression/grouping_sets.go index 356bc00fece3c..e42480c6d90ab 100644 --- a/pkg/expression/grouping_sets.go +++ b/pkg/expression/grouping_sets.go @@ -17,6 +17,7 @@ package expression import ( "strings" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/util/intset" @@ -263,13 +264,18 @@ func (gs GroupingSet) Clone() GroupingSet { // String is used to output a string which simply described current grouping set. func (gs GroupingSet) String() string { + return gs.StringWithCtx(errors.RedactLogDisable) +} + +// StringWithCtx is used to output a string which simply described current grouping set. +func (gs GroupingSet) StringWithCtx(redact string) string { var str strings.Builder str.WriteString("{") for i, one := range gs { if i != 0 { str.WriteString(",") } - str.WriteString(one.String()) + str.WriteString(one.StringWithCtx(redact)) } str.WriteString("}") return str.String() @@ -321,13 +327,18 @@ func (gss GroupingSets) AllSetsColIDs() *intset.FastIntSet { // String is used to output a string which simply described current grouping sets. func (gss GroupingSets) String() string { + return gss.StringWithCtx(errors.RedactLogDisable) +} + +// StringWithCtx is used to output a string which simply described current grouping sets. +func (gss GroupingSets) StringWithCtx(redact string) string { var str strings.Builder str.WriteString("[") for i, gs := range gss { if i != 0 { str.WriteString(",") } - str.WriteString(gs.String()) + str.WriteString(gs.StringWithCtx(redact)) } str.WriteString("]") return str.String() @@ -390,13 +401,18 @@ func (g GroupingExprs) Clone() GroupingExprs { // String is used to output a string which simply described current grouping expressions. func (g GroupingExprs) String() string { + return g.StringWithCtx(errors.RedactLogDisable) +} + +// StringWithCtx is used to output a string which simply described current grouping expressions. +func (g GroupingExprs) StringWithCtx(redact string) string { var str strings.Builder str.WriteString("<") for i, one := range g { if i != 0 { str.WriteString(",") } - str.WriteString(one.String()) + str.WriteString(one.StringWithCtx(redact)) } str.WriteString(">") return str.String() diff --git a/pkg/expression/grouping_sets_test.go b/pkg/expression/grouping_sets_test.go index f61f00ccd032c..36609aa8dfcf9 100644 --- a/pkg/expression/grouping_sets_test.go +++ b/pkg/expression/grouping_sets_test.go @@ -17,6 +17,7 @@ package expression import ( "testing" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" @@ -352,9 +353,9 @@ func TestDistinctGroupingSets(t *testing.T) { rawRollupExprs = []Expression{a, b, b, c} deduplicateExprs, pos = DeduplicateGbyExpression(rawRollupExprs) require.Equal(t, len(deduplicateExprs), 3) - require.Equal(t, deduplicateExprs[0].String(), "Column#1") - require.Equal(t, deduplicateExprs[1].String(), "Column#2") - require.Equal(t, deduplicateExprs[2].String(), "Column#3") + require.Equal(t, deduplicateExprs[0].StringWithCtx(errors.RedactLogDisable), "Column#1") + require.Equal(t, deduplicateExprs[1].StringWithCtx(errors.RedactLogDisable), "Column#2") + require.Equal(t, deduplicateExprs[2].StringWithCtx(errors.RedactLogDisable), "Column#3") deduplicateColumns := make([]*Column, 0, len(deduplicateExprs)) for _, one := range deduplicateExprs { deduplicateColumns = append(deduplicateColumns, one.(*Column)) @@ -372,10 +373,10 @@ func TestDistinctGroupingSets(t *testing.T) { // so that why restore gby expression according to their pos is necessary. restoreGbyExpressions := RestoreGbyExpression(deduplicateColumns, pos) require.Equal(t, len(restoreGbyExpressions), 4) - require.Equal(t, restoreGbyExpressions[0].String(), "Column#1") - require.Equal(t, restoreGbyExpressions[1].String(), "Column#2") - require.Equal(t, restoreGbyExpressions[2].String(), "Column#2") - require.Equal(t, restoreGbyExpressions[3].String(), "Column#3") + require.Equal(t, restoreGbyExpressions[0].StringWithCtx(errors.RedactLogDisable), "Column#1") + require.Equal(t, restoreGbyExpressions[1].StringWithCtx(errors.RedactLogDisable), "Column#2") + require.Equal(t, restoreGbyExpressions[2].StringWithCtx(errors.RedactLogDisable), "Column#2") + require.Equal(t, restoreGbyExpressions[3].StringWithCtx(errors.RedactLogDisable), "Column#3") // rollup grouping sets (build grouping sets on the restored gby expression, because all the // complicated expressions have been projected as simple columns at this time). diff --git a/pkg/expression/infer_pushdown.go b/pkg/expression/infer_pushdown.go index 97bd813d02951..5ea8027150382 100644 --- a/pkg/expression/infer_pushdown.go +++ b/pkg/expression/infer_pushdown.go @@ -127,12 +127,12 @@ func canExprPushDown(ctx PushDownContext, expr Expression, storeType kv.StoreTyp if expr.GetType().GetType() == mysql.TypeEnum && canEnumPush { break } - warnErr := errors.NewNoStackError("Expression about '" + expr.String() + "' can not be pushed to TiFlash because it contains unsupported calculation of type '" + types.TypeStr(expr.GetType().GetType()) + "'.") + warnErr := errors.NewNoStackError("Expression about '" + expr.StringWithCtx(errors.RedactLogDisable) + "' can not be pushed to TiFlash because it contains unsupported calculation of type '" + types.TypeStr(expr.GetType().GetType()) + "'.") ctx.AppendWarning(warnErr) return false case mysql.TypeNewDecimal: if !expr.GetType().IsDecimalValid() { - warnErr := errors.NewNoStackError("Expression about '" + expr.String() + "' can not be pushed to TiFlash because it contains invalid decimal('" + strconv.Itoa(expr.GetType().GetFlen()) + "','" + strconv.Itoa(expr.GetType().GetDecimal()) + "').") + warnErr := errors.NewNoStackError("Expression about '" + expr.StringWithCtx(errors.RedactLogDisable) + "' can not be pushed to TiFlash because it contains invalid decimal('" + strconv.Itoa(expr.GetType().GetFlen()) + "','" + strconv.Itoa(expr.GetType().GetDecimal()) + "').") ctx.AppendWarning(warnErr) return false } diff --git a/pkg/expression/scalar_function.go b/pkg/expression/scalar_function.go index 5d44c9b3d05a6..438e8b359a1b0 100644 --- a/pkg/expression/scalar_function.go +++ b/pkg/expression/scalar_function.go @@ -120,18 +120,23 @@ func (sf *ScalarFunction) Vectorized() bool { // String implements fmt.Stringer interface. func (sf *ScalarFunction) String() string { + return sf.StringWithCtx(errors.RedactLogDisable) +} + +// StringWithCtx implements Expression interface. +func (sf *ScalarFunction) StringWithCtx(redact string) string { var buffer bytes.Buffer fmt.Fprintf(&buffer, "%s(", sf.FuncName.L) switch sf.FuncName.L { case ast.Cast: for _, arg := range sf.GetArgs() { - buffer.WriteString(arg.String()) + buffer.WriteString(arg.StringWithCtx(redact)) buffer.WriteString(", ") buffer.WriteString(sf.RetType.String()) } default: for i, arg := range sf.GetArgs() { - buffer.WriteString(arg.String()) + buffer.WriteString(arg.StringWithCtx(redact)) if i+1 != len(sf.GetArgs()) { buffer.WriteString(", ") } diff --git a/pkg/expression/util.go b/pkg/expression/util.go index 7d9eb8d6384f3..c3f92d69feabe 100644 --- a/pkg/expression/util.go +++ b/pkg/expression/util.go @@ -1812,7 +1812,7 @@ func ExprsToStringsForDisplay(exprs []Expression) []string { // so we trim the \" prefix and suffix here. strs[i] = strings.TrimSuffix( strings.TrimPrefix( - strconv.Quote(cond.String()), + strconv.Quote(cond.StringWithCtx(errors.RedactLogDisable)), quote), quote) } diff --git a/pkg/expression/util_test.go b/pkg/expression/util_test.go index 924cf8905c4ee..25186011246d6 100644 --- a/pkg/expression/util_test.go +++ b/pkg/expression/util_test.go @@ -519,8 +519,8 @@ func (m *MockExpr) VecEvalDuration(ctx EvalContext, input *chunk.Chunk, result * func (m *MockExpr) VecEvalJSON(ctx EvalContext, input *chunk.Chunk, result *chunk.Column) error { return nil } - func (m *MockExpr) String() string { return "" } +func (m *MockExpr) StringWithCtx(string) string { return "" } func (m *MockExpr) MarshalJSON() ([]byte, error) { return nil, nil } func (m *MockExpr) Eval(ctx EvalContext, row chunk.Row) (types.Datum, error) { return types.NewDatum(m.i), m.err diff --git a/pkg/parser/digester_test.go b/pkg/parser/digester_test.go index c96454d715a6e..9956027590d63 100644 --- a/pkg/parser/digester_test.go +++ b/pkg/parser/digester_test.go @@ -110,6 +110,11 @@ func TestNormalizeRedact(t *testing.T) { {"select * from t where a in (1)", "select * from `t` where `a` in ( ‹1› )"}, {"select * from t where a in (1, 3)", "select * from `t` where `a` in ( ‹1› , ‹3› )"}, {"select ? from b order by 2", "select ? from `b` order by ‹2›"}, + {"select ? from b order by 2 limit 10 offset 10", "select ? from `b` order by ‹2› limit ‹10› offset ‹10›"}, + {"with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 100 offset 100) select * from cte1;", + "with recursive `cte1` ( `c1` ) as ( select `c1` from `t1` union select `c1` + ‹1› `c1` from `cte1` limit ‹100› offset ‹100› ) select * from `cte1`"}, + {"select *, first_value(v) over (partition by p order by o range between 3 preceding and 0 following) as a from test.first_range", + "select * , `first_value` ( `v` ) `over` ( partition by `p` order by `o` range between ‹3› preceding and ‹0› following ) as `a` from `test` . `first_range`"}, } for _, c := range cases { diff --git a/pkg/planner/cardinality/selectivity.go b/pkg/planner/cardinality/selectivity.go index 2c8710730aed7..dbadb11eabce8 100644 --- a/pkg/planner/cardinality/selectivity.go +++ b/pkg/planner/cardinality/selectivity.go @@ -112,7 +112,7 @@ func Selectivity( sel = 1.0 / pseudoEqualRate } if sc.EnableOptimizerDebugTrace { - debugtrace.RecordAnyValuesWithNames(ctx, "Expression", expr.String(), "Selectivity", sel) + debugtrace.RecordAnyValuesWithNames(ctx, "Expression", expr.StringWithCtx(errors.RedactLogDisable), "Selectivity", sel) } ret *= sel } diff --git a/pkg/planner/cardinality/trace.go b/pkg/planner/cardinality/trace.go index a348e799daa40..dda2ca95baf3f 100644 --- a/pkg/planner/cardinality/trace.go +++ b/pkg/planner/cardinality/trace.go @@ -19,6 +19,7 @@ import ( "encoding/json" "errors" + perrors "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/format" @@ -95,7 +96,7 @@ func exprToString(ctx expression.EvalContext, e expression.Expression) (string, buffer.WriteString(")") return buffer.String(), nil case *expression.Column: - return expr.String(), nil + return expr.StringWithCtx(perrors.RedactLogDisable), nil case *expression.CorrelatedColumn: return "", errors.New("tracing for correlated columns not supported now") case *expression.Constant: diff --git a/pkg/planner/cascades/BUILD.bazel b/pkg/planner/cascades/BUILD.bazel index 25d33ca880bfe..c35a354d35488 100644 --- a/pkg/planner/cascades/BUILD.bazel +++ b/pkg/planner/cascades/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//pkg/util/dbterror/plannererrors", "//pkg/util/ranger", "//pkg/util/set", + "@com_github_pingcap_errors//:errors", ], ) diff --git a/pkg/planner/cascades/stringer.go b/pkg/planner/cascades/stringer.go index de1f12897a7b7..0e9df0091b823 100644 --- a/pkg/planner/cascades/stringer.go +++ b/pkg/planner/cascades/stringer.go @@ -19,6 +19,7 @@ import ( "fmt" "strings" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/planner/memo" ) @@ -66,7 +67,7 @@ func groupToString(g *memo.Group, idMap map[*memo.Group]int) []string { schema := g.Prop.Schema colStrs := make([]string, 0, len(schema.Columns)) for _, col := range schema.Columns { - colStrs = append(colStrs, col.String()) + colStrs = append(colStrs, col.StringWithCtx(errors.RedactLogDisable)) } groupLine := bytes.NewBufferString("") @@ -77,7 +78,7 @@ func groupToString(g *memo.Group, idMap map[*memo.Group]int) []string { for _, key := range schema.Keys { ukColStrs := make([]string, 0, len(key)) for _, col := range key { - ukColStrs = append(ukColStrs, col.String()) + ukColStrs = append(ukColStrs, col.StringWithCtx(errors.RedactLogDisable)) } ukStrs = append(ukStrs, strings.Join(ukColStrs, ",")) } diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index 1b1a04b02d66a..e8ac8686243a2 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -157,6 +157,7 @@ go_library( "//pkg/util/plancache", "//pkg/util/plancodec", "//pkg/util/ranger", + "//pkg/util/redact", "//pkg/util/rowcodec", "//pkg/util/sem", "//pkg/util/set", diff --git a/pkg/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_out.json b/pkg/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_out.json index ca315171c41b6..6bfadc6c157c3 100644 --- a/pkg/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_out.json +++ b/pkg/planner/core/casetest/enforcempp/testdata/enforce_mpp_suite_out.json @@ -647,7 +647,7 @@ { "SQL": "explain select a from t where t.a not in (select a from s where t.a<1); -- 9. non left join has left conditions", "Plan": [ - "MergeJoin_10 8000.00 root anti semi join, left key:test.t.a, right key:test.s.a, left cond:[lt(test.t.a, 1)]", + "MergeJoin_10 8000.00 root anti semi join, left key:test.t.a, right key:test.s.a, left cond:lt(test.t.a, 1)", "├─TableReader_26(Build) 10000.00 root data:TableFullScan_25", "│ └─TableFullScan_25 10000.00 cop[tikv] table:s keep order:true, stats:pseudo", "└─TableReader_23(Probe) 10000.00 root data:TableFullScan_22", diff --git a/pkg/planner/core/casetest/integration_test.go b/pkg/planner/core/casetest/integration_test.go index 9500bfe508b81..fb890c3faa8b7 100644 --- a/pkg/planner/core/casetest/integration_test.go +++ b/pkg/planner/core/casetest/integration_test.go @@ -337,17 +337,24 @@ func TestTiFlashFineGrainedShuffle(t *testing.T) { tbl1.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} var input []string var output []struct { - SQL string - Plan []string + SQL string + Plan []string + Redact []string } integrationSuiteData := GetIntegrationSuiteData() integrationSuiteData.LoadTestCases(t, &input, &output) for i, tt := range input { testdata.OnRecord(func() { output[i].SQL = tt + tk.MustExec("set session tidb_redact_log=off") output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + tk.MustExec("set session tidb_redact_log=on") + output[i].Redact = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) + tk.MustExec("set session tidb_redact_log=off") tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustExec("set session tidb_redact_log=on") + tk.MustQuery(tt).Check(testkit.Rows(output[i].Redact...)) } } diff --git a/pkg/planner/core/casetest/testdata/integration_suite_out.json b/pkg/planner/core/casetest/testdata/integration_suite_out.json index b3464c4aa2706..4879a37b03874 100644 --- a/pkg/planner/core/casetest/testdata/integration_suite_out.json +++ b/pkg/planner/core/casetest/testdata/integration_suite_out.json @@ -1067,6 +1067,16 @@ " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#5->Column#6, stream_count: 8", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#5 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -1084,6 +1094,20 @@ " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#7->Column#8, Column#6->Column#9, stream_count: 8", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#7 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Window 10000.00 mpp[tiflash] rank()->Column#6 over(partition by test.t1.c2), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -1103,6 +1127,22 @@ " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Redact": [ + "Projection 10.00 root Column#7->Column#8, Column#6->Column#9", + "└─TopN 10.00 root Column#7, Column#6, offset:?, count:?", + " └─TableReader 10.00 root MppVersion: 2, data:ExchangeSender", + " └─ExchangeSender 10.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TopN 10.00 mpp[tiflash] Column#7, Column#6, offset:?, count:?", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#7 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Window 10000.00 mpp[tiflash] rank()->Column#6 over(partition by test.t1.c2), stream_count: 8", + " └─Sort 10000.00 mpp[tiflash] test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -1122,6 +1162,22 @@ " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#10", " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8, stream_count: 8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c2 order by test.t1.c2 rows between current row and current row), stream_count: 8", + " └─Sort 2666.67 mpp[tiflash] test.t1.c2, test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, ?)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" ] }, { @@ -1141,6 +1197,22 @@ " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:count(test.t1.c1)->Column#9, funcs:firstrow(test.t1.c1)->Column#10", " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c2, 10)", " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8, stream_count: 8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c1 order by test.t1.c2 rows between current row and current row), stream_count: 8", + " └─Sort 2666.67 mpp[tiflash] test.t1.c1, test.t1.c2, stream_count: 8", + " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c1, funcs:firstrow(test.t1.c2)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c2, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:count(test.t1.c1)->Column#9, funcs:firstrow(test.t1.c1)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c2, ?)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" ] }, { @@ -1160,6 +1232,22 @@ " │ └─TableFullScan 10000.00 mpp[tiflash] table:a pushed down filter:empty, keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.c2))", " └─TableFullScan 10000.00 mpp[tiflash] table:b pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 12487.50 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 mpp[tiflash] Column#8->Column#9, stream_count: 8", + " └─Window 12487.50 mpp[tiflash] row_number()->Column#8 over(partition by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 12487.50 mpp[tiflash] test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 12487.50 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t1.c1, test.t1.c2)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast, Compression: FAST", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t1.c1))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:a pushed down filter:empty, keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t1.c2))", + " └─TableFullScan 10000.00 mpp[tiflash] table:b pushed down filter:empty, keep order:false, stats:pseudo" ] }, { @@ -1174,6 +1262,17 @@ " └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", " └─Selection 3323.33 mpp[tiflash] lt(test.t1.c1, 100)", " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 3323.33 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 3323.33 mpp[tiflash] Column#5->Column#6, stream_count: 8", + " └─Window 3323.33 mpp[tiflash] row_number()->Column#5 over(partition by test.t1.c1 order by test.t1.c1 rows between current row and current row), stream_count: 8", + " └─Sort 3323.33 mpp[tiflash] test.t1.c1, test.t1.c1, stream_count: 8", + " └─ExchangeReceiver 3323.33 mpp[tiflash] stream_count: 8", + " └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", + " └─Selection 3323.33 mpp[tiflash] lt(test.t1.c1, ?)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" ] }, { @@ -1182,6 +1281,11 @@ "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -1195,6 +1299,16 @@ " └─ExchangeReceiver 10000.00 mpp[tiflash] ", " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 10000.00 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 mpp[tiflash] Column#5->Column#6", + " └─Window 10000.00 mpp[tiflash] row_number()->Column#5 over(order by test.t1.c1 rows between current row and current row)", + " └─Sort 10000.00 mpp[tiflash] test.t1.c1", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: PassThrough, Compression: FAST", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -1212,6 +1326,20 @@ " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#11", " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" + ], + "Redact": [ + "TableReader 2666.67 root MppVersion: 2, data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] Column#6->Column#7, Column#4->Column#8", + " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c1 order by test.t1.c2 rows between current row and current row)", + " └─Sort 2666.67 mpp[tiflash] test.t1.c1, test.t1.c2", + " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(test.t1.c1)->test.t1.c1, funcs:firstrow(Column#11)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#11", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, ?)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 pushed down filter:empty, keep order:false, stats:pseudo" ] } ] diff --git a/pkg/planner/core/debugtrace.go b/pkg/planner/core/debugtrace.go index fe7b2e195c2e8..dca228b4c4db1 100644 --- a/pkg/planner/core/debugtrace.go +++ b/pkg/planner/core/debugtrace.go @@ -18,6 +18,7 @@ import ( "strconv" "strings" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" @@ -106,7 +107,7 @@ func DebugTraceReceivedCommand(s PlanContext, cmd byte, stmtNode ast.StmtNode) { execInfo.BinaryParamsInfo = make([]binaryParamInfo, len(binaryParams)) for i, param := range binaryParams { execInfo.BinaryParamsInfo[i].Type = param.GetType().String() - execInfo.BinaryParamsInfo[i].Value = param.String() + execInfo.BinaryParamsInfo[i].Value = param.StringWithCtx(errors.RedactLogDisable) } } } diff --git a/pkg/planner/core/exhaust_physical_plans.go b/pkg/planner/core/exhaust_physical_plans.go index c5f7e6fa6d12c..8e946a2392c4e 100644 --- a/pkg/planner/core/exhaust_physical_plans.go +++ b/pkg/planner/core/exhaust_physical_plans.go @@ -876,7 +876,7 @@ func (p *LogicalJoin) buildIndexJoinInner2TableScan( if i != 0 { buffer.WriteString(" ") } - buffer.WriteString(key.String()) + buffer.WriteString(key.StringWithCtx(errors.RedactLogDisable)) } buffer.WriteString("]") rangeInfo := buffer.String() @@ -1003,13 +1003,16 @@ func (ijHelper *indexJoinBuildHelper) buildRangeDecidedByInformation(idxCols []* } fmt.Fprintf(buffer, "eq(%v, %v)", idxCols[idxOff], outerJoinKeys[keyOff]) } + // It is to build the range info which is used in explain. It is necessary to redact the range info. + ectx := ijHelper.join.SCtx().GetExprCtx().GetEvalCtx() + redact := ectx.GetTiDBRedactLog() for _, access := range ijHelper.chosenAccess { if !isFirst { buffer.WriteString(" ") } else { isFirst = false } - fmt.Fprintf(buffer, "%v", access) + fmt.Fprintf(buffer, "%v", access.StringWithCtx(redact)) } buffer.WriteString("]") return buffer.String() diff --git a/pkg/planner/core/explain.go b/pkg/planner/core/explain.go index 007dc95f615f2..2456abb1da7e5 100644 --- a/pkg/planner/core/explain.go +++ b/pkg/planner/core/explain.go @@ -20,6 +20,7 @@ import ( "strconv" "strings" + perrors "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/expression/aggregation" "github.com/pingcap/tidb/pkg/kv" @@ -76,6 +77,7 @@ func (p *PhysicalIndexScan) ExplainNormalizedInfo() string { // OperatorInfo implements dataAccesser interface. func (p *PhysicalIndexScan) OperatorInfo(normalized bool) string { + redact := p.SCtx().GetSessionVars().EnableRedactLog var buffer strings.Builder if len(p.rangeInfo) > 0 { if !normalized { @@ -94,7 +96,7 @@ func (p *PhysicalIndexScan) OperatorInfo(normalized bool) string { if i != 0 { buffer.WriteString(" ") } - buffer.WriteString(expr.String()) + buffer.WriteString(expr.StringWithCtx(redact)) } buffer.WriteString("], ") } @@ -104,7 +106,7 @@ func (p *PhysicalIndexScan) OperatorInfo(normalized bool) string { } else if !p.isFullScan() { buffer.WriteString("range:") for _, idxRange := range p.Ranges { - buffer.WriteString(idxRange.String()) + buffer.WriteString(idxRange.Redact(redact)) buffer.WriteString(", ") } } @@ -182,6 +184,7 @@ func (p *PhysicalTableScan) ExplainNormalizedInfo() string { // OperatorInfo implements dataAccesser interface. func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { + redact := p.SCtx().GetSessionVars().EnableRedactLog var buffer strings.Builder if len(p.rangeInfo) > 0 { if !normalized { @@ -200,7 +203,7 @@ func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { if i != 0 { buffer.WriteString(" ") } - buffer.WriteString(AccessCondition.String()) + buffer.WriteString(AccessCondition.StringWithCtx(redact)) } buffer.WriteString("], ") } @@ -210,7 +213,7 @@ func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { } else if !p.isFullScan() { buffer.WriteString("range:") for _, idxRange := range p.Ranges { - buffer.WriteString(idxRange.String()) + buffer.WriteString(idxRange.Redact(redact)) buffer.WriteString(", ") } } @@ -374,7 +377,8 @@ func (p *PhysicalSelection) ExplainNormalizedInfo() string { // ExplainInfo implements Plan interface. func (p *PhysicalProjection) ExplainInfo() string { - exprStr := expression.ExplainExpressionList(p.Exprs, p.schema) + enableRedactLog := p.SCtx().GetSessionVars().EnableRedactLog + exprStr := expression.ExplainExpressionList(p.Exprs, p.schema, enableRedactLog) if p.TiFlashFineGrainedShuffleStreamCount > 0 { exprStr += fmt.Sprintf(", stream_count: %d", p.TiFlashFineGrainedShuffleStreamCount) } @@ -383,22 +387,23 @@ func (p *PhysicalProjection) ExplainInfo() string { func (p *PhysicalExpand) explainInfoV2() string { sb := strings.Builder{} + enableRedactLog := p.SCtx().GetSessionVars().EnableRedactLog for i, oneL := range p.LevelExprs { if i == 0 { sb.WriteString("level-projection:") sb.WriteString("[") - sb.WriteString(expression.ExplainExpressionList(oneL, p.schema)) + sb.WriteString(expression.ExplainExpressionList(oneL, p.schema, enableRedactLog)) sb.WriteString("]") } else { sb.WriteString(",[") - sb.WriteString(expression.ExplainExpressionList(oneL, p.schema)) + sb.WriteString(expression.ExplainExpressionList(oneL, p.schema, enableRedactLog)) sb.WriteString("]") } } sb.WriteString("; schema: [") colStrs := make([]string, 0, len(p.schema.Columns)) for _, col := range p.schema.Columns { - colStrs = append(colStrs, col.String()) + colStrs = append(colStrs, col.StringWithCtx(perrors.RedactLogDisable)) } sb.WriteString(strings.Join(colStrs, ",")) sb.WriteString("]") @@ -433,12 +438,18 @@ func (p *PhysicalSort) ExplainInfo() string { // ExplainInfo implements Plan interface. func (p *PhysicalLimit) ExplainInfo() string { + redact := p.SCtx().GetSessionVars().EnableRedactLog buffer := bytes.NewBufferString("") if len(p.GetPartitionBy()) > 0 { buffer = explainPartitionBy(buffer, p.GetPartitionBy(), false) - fmt.Fprintf(buffer, ", offset:%v, count:%v", p.Offset, p.Count) - } else { + fmt.Fprintf(buffer, ", ") + } + if redact == perrors.RedactLogDisable { fmt.Fprintf(buffer, "offset:%v, count:%v", p.Offset, p.Count) + } else if redact == perrors.RedactLogMarker { + fmt.Fprintf(buffer, "offset:‹%v›, count:‹%v›", p.Offset, p.Count) + } else if redact == perrors.RedactLogEnable { + fmt.Fprintf(buffer, "offset:?, count:?") } return buffer.String() } @@ -452,9 +463,9 @@ func (p *PhysicalExpand) ExplainInfo() string { str.WriteString("group set num:") str.WriteString(strconv.FormatInt(int64(len(p.GroupingSets)), 10)) str.WriteString(", groupingID:") - str.WriteString(p.GroupingIDCol.String()) + str.WriteString(p.GroupingIDCol.StringWithCtx(perrors.RedactLogDisable)) str.WriteString(", ") - str.WriteString(p.GroupingSets.String()) + str.WriteString(p.GroupingSets.StringWithCtx(perrors.RedactLogDisable)) return str.String() } @@ -593,7 +604,7 @@ func (p *PhysicalHashJoin) explainInfo(normalized bool) string { return expression.SortedExplainNormalizedExpressionList(exprs) } } - + redact := p.SCtx().GetSessionVars().EnableRedactLog buffer := new(strings.Builder) if len(p.EqualConditions) == 0 { @@ -617,7 +628,7 @@ func (p *PhysicalHashJoin) explainInfo(normalized bool) string { if i != 0 { buffer.WriteString(" ") } - buffer.WriteString(EqualConditions.String()) + buffer.WriteString(EqualConditions.StringWithCtx(redact)) } buffer.WriteString("]") } @@ -632,7 +643,7 @@ func (p *PhysicalHashJoin) explainInfo(normalized bool) string { if i != 0 { buffer.WriteString(" ") } - buffer.WriteString(NAEqualCondition.String()) + buffer.WriteString(NAEqualCondition.StringWithCtx(redact)) } buffer.WriteString("]") } @@ -647,7 +658,7 @@ func (p *PhysicalHashJoin) explainInfo(normalized bool) string { if i != 0 { buffer.WriteString(" ") } - buffer.WriteString(LeftConditions.String()) + buffer.WriteString(LeftConditions.StringWithCtx(redact)) } buffer.WriteString("]") } @@ -704,7 +715,7 @@ func (p *PhysicalMergeJoin) explainInfo(normalized bool) string { if normalized { fmt.Fprintf(buffer, ", left cond:%s", expression.SortedExplainNormalizedExpressionList(p.LeftConditions)) } else { - fmt.Fprintf(buffer, ", left cond:%s", p.LeftConditions) + fmt.Fprintf(buffer, ", left cond:%s", sortedExplainExpressionList(evalCtx, p.LeftConditions)) } } if len(p.RightConditions) > 0 { @@ -752,7 +763,14 @@ func (p *PhysicalTopN) ExplainInfo() string { } buffer = explainByItems(p.SCtx().GetExprCtx().GetEvalCtx(), buffer, p.ByItems) } - fmt.Fprintf(buffer, ", offset:%v, count:%v", p.Offset, p.Count) + switch p.SCtx().GetSessionVars().EnableRedactLog { + case perrors.RedactLogDisable: + fmt.Fprintf(buffer, ", offset:%v, count:%v", p.Offset, p.Count) + case perrors.RedactLogMarker: + fmt.Fprintf(buffer, ", offset:‹%v›, count:‹%v›", p.Offset, p.Count) + case perrors.RedactLogEnable: + fmt.Fprintf(buffer, ", offset:?, count:?") + } return buffer.String() } @@ -793,7 +811,14 @@ func (p *PhysicalWindow) formatFrameBound(buffer *bytes.Buffer, bound *FrameBoun fmt.Fprintf(buffer, "%s", sf.GetArgs()[1].ExplainInfo(evalCtx)) } } else { - fmt.Fprintf(buffer, "%d", bound.Num) + switch p.SCtx().GetSessionVars().EnableRedactLog { + case perrors.RedactLogDisable: + fmt.Fprintf(buffer, "%d", bound.Num) + case perrors.RedactLogMarker: + fmt.Fprintf(buffer, "‹%d›", bound.Num) + case perrors.RedactLogEnable: + fmt.Fprintf(buffer, "?") + } } if bound.Type == ast.Preceding { buffer.WriteString(" preceding") @@ -870,7 +895,7 @@ func formatWindowFuncDescs(buffer *bytes.Buffer, descs []*aggregation.WindowFunc if i != 0 { buffer.WriteString(", ") } - fmt.Fprintf(buffer, "%v->%v", desc, schema.Columns[winFuncStartIdx+i]) + fmt.Fprintf(buffer, "%v->%v", desc.StringWithCtx(perrors.RedactLogDisable), schema.Columns[winFuncStartIdx+i]) } return buffer } @@ -918,7 +943,8 @@ func (p *LogicalAggregation) ExplainInfo() string { // ExplainInfo implements Plan interface. func (p *LogicalProjection) ExplainInfo() string { - return expression.ExplainExpressionList(p.Exprs, p.schema) + enableRedactLog := p.SCtx().GetSessionVars().EnableRedactLog + return expression.ExplainExpressionList(p.Exprs, p.schema, enableRedactLog) } // ExplainInfo implements Plan interface. diff --git a/pkg/planner/core/expression_rewriter.go b/pkg/planner/core/expression_rewriter.go index 78867a88bfd29..043ce7365182b 100644 --- a/pkg/planner/core/expression_rewriter.go +++ b/pkg/planner/core/expression_rewriter.go @@ -1867,7 +1867,7 @@ func (er *expressionRewriter) inToExpression(lLen int, not bool, tp *types.Field if c.GetType().EvalType() == types.ETInt { continue // no need to refine it } - er.sctx.SetSkipPlanCache(errors.NewNoStackErrorf("'%v' may be converted to INT", c.String())) + er.sctx.SetSkipPlanCache(errors.NewNoStackErrorf("'%v' may be converted to INT", c.StringWithCtx(errors.RedactLogDisable))) if err := expression.RemoveMutableConst(er.sctx, []expression.Expression{c}); err != nil { er.err = err return diff --git a/pkg/planner/core/handle_cols.go b/pkg/planner/core/handle_cols.go index 9c36ab5843da8..0563e9c859cd2 100644 --- a/pkg/planner/core/handle_cols.go +++ b/pkg/planner/core/handle_cols.go @@ -18,6 +18,7 @@ import ( "strings" "unsafe" + perrors "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" @@ -50,6 +51,8 @@ type HandleCols interface { IsInt() bool // String implements the fmt.Stringer interface. String() string + // StringWithCtx implements the fmt.Stringer interface. + StringWithCtx(string) string // GetCol gets the column by idx. GetCol(idx int) *expression.Column // NumCols returns the number of columns. @@ -158,6 +161,11 @@ func (cb *CommonHandleCols) NumCols() int { // String implements the kv.HandleCols interface. func (cb *CommonHandleCols) String() string { + return cb.StringWithCtx(perrors.RedactLogDisable) +} + +// StringWithCtx implements the kv.HandleCols interface. +func (cb *CommonHandleCols) StringWithCtx(_ string) string { b := new(strings.Builder) b.WriteByte('[') for i, col := range cb.columns { @@ -272,6 +280,11 @@ func (ib *IntHandleCols) String() string { return ib.col.ColumnExplainInfo(false) } +// StringWithCtx implements the kv.HandleCols interface. +func (ib *IntHandleCols) StringWithCtx(_ string) string { + return ib.col.ColumnExplainInfo(false) +} + // GetCol implements the kv.HandleCols interface. func (ib *IntHandleCols) GetCol(idx int) *expression.Column { if idx != 0 { diff --git a/pkg/planner/core/indexmerge_test.go b/pkg/planner/core/indexmerge_test.go index f75ff1a8ccb7b..7b1a394160967 100644 --- a/pkg/planner/core/indexmerge_test.go +++ b/pkg/planner/core/indexmerge_test.go @@ -18,6 +18,7 @@ import ( "context" "testing" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser" @@ -58,7 +59,7 @@ func getIndexMergePathDigest(paths []*util.AccessPath, startIndex int) string { if j > 0 { idxMergeDisgest += "," } - idxMergeDisgest += path.TableFilters[j].String() + idxMergeDisgest += path.TableFilters[j].StringWithCtx(errors.RedactLogDisable) } idxMergeDisgest += "]}" } diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index d2030e97e45a1..376b17270c920 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -1439,7 +1439,7 @@ func buildExpandFieldName(expr expression.Expression, name *types.FieldName, gen var origTblName, origColName, dbName, colName, tblName model.CIStr if genName != "" { // for case like: gid_, gpos_ - colName = model.NewCIStr(expr.String()) + colName = model.NewCIStr(expr.StringWithCtx(errors.RedactLogDisable)) } else if isCol { // col ref to original col, while its nullability may be changed. origTblName, origColName, dbName = name.OrigTblName, name.OrigColName, name.DBName @@ -1447,7 +1447,7 @@ func buildExpandFieldName(expr expression.Expression, name *types.FieldName, gen tblName = model.NewCIStr("ex_" + name.TblName.O) } else { // Other: complicated expression. - colName = model.NewCIStr("ex_" + expr.String()) + colName = model.NewCIStr("ex_" + expr.StringWithCtx(errors.RedactLogDisable)) } newName := &types.FieldName{ TblName: tblName, diff --git a/pkg/planner/core/logical_plans_test.go b/pkg/planner/core/logical_plans_test.go index 2511b5d8c30b7..d12a53028f972 100644 --- a/pkg/planner/core/logical_plans_test.go +++ b/pkg/planner/core/logical_plans_test.go @@ -21,6 +21,7 @@ import ( "strings" "testing" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/expression" @@ -783,9 +784,9 @@ func checkDataSourceCols(p LogicalPlan, t *testing.T, ans map[int][]string, comm require.Equal(t, len(colList), len(p.Schema().Columns), comment) for i, col := range p.Schema().Columns { testdata.OnRecord(func() { - colList[i] = col.String() + colList[i] = col.StringWithCtx(errors.RedactLogDisable) }) - require.Equal(t, colList[i], col.String(), comment) + require.Equal(t, colList[i], col.StringWithCtx(errors.RedactLogDisable), comment) } } for _, child := range p.Children() { @@ -801,9 +802,9 @@ func checkOrderByItems(p LogicalPlan, t *testing.T, colList *[]string, comment s }) for i, col := range p.ByItems { testdata.OnRecord(func() { - (*colList)[i] = col.String() + (*colList)[i] = col.StringWithCtx(errors.RedactLogDisable) }) - s := col.String() + s := col.StringWithCtx(errors.RedactLogDisable) require.Equal(t, (*colList)[i], s, comment) } } @@ -1022,9 +1023,9 @@ func checkUniqueKeys(p LogicalPlan, t *testing.T, ans map[int][][]string, sql st require.Equal(t, len(keyList[i]), len(p.Schema().Keys[i]), fmt.Sprintf("for %s, %v %v, the number of column doesn't match", sql, p.ID(), keyList[i])) for j := range keyList[i] { testdata.OnRecord(func() { - keyList[i][j] = p.Schema().Keys[i][j].String() + keyList[i][j] = p.Schema().Keys[i][j].StringWithCtx(errors.RedactLogDisable) }) - require.Equal(t, keyList[i][j], p.Schema().Keys[i][j].String(), fmt.Sprintf("for %s, %v %v, column dosen't match", sql, p.ID(), keyList[i])) + require.Equal(t, keyList[i][j], p.Schema().Keys[i][j].StringWithCtx(errors.RedactLogDisable), fmt.Sprintf("for %s, %v %v, column dosen't match", sql, p.ID(), keyList[i])) } } testdata.OnRecord(func() { @@ -2347,11 +2348,14 @@ func TestRollupExpand(t *testing.T) { require.Equal(t, builder.currentBlockExpand.LevelExprs != nil, true) require.Equal(t, len(builder.currentBlockExpand.LevelExprs), 3) // for grouping set {}: gid = '00' = 0 - require.Equal(t, expression.ExplainExpressionList(expand.LevelExprs[0], expand.schema), "test.t.a, ->Column#13, ->Column#14, 0->gid") + require.Equal(t, expression.ExplainExpressionList(expand.LevelExprs[0], expand.Schema(), errors.RedactLogDisable), + "test.t.a, ->Column#13, ->Column#14, 0->gid") // for grouping set {a}: gid = '01' = 1 - require.Equal(t, expression.ExplainExpressionList(expand.LevelExprs[1], expand.schema), "test.t.a, Column#13, ->Column#14, 1->gid") + require.Equal(t, expression.ExplainExpressionList(expand.LevelExprs[1], expand.Schema(), errors.RedactLogDisable), + "test.t.a, Column#13, ->Column#14, 1->gid") // for grouping set {a,b}: gid = '11' = 3 - require.Equal(t, expression.ExplainExpressionList(expand.LevelExprs[2], expand.schema), "test.t.a, Column#13, Column#14, 3->gid") + require.Equal(t, expression.ExplainExpressionList(expand.LevelExprs[2], expand.Schema(), errors.RedactLogDisable), + "test.t.a, Column#13, Column#14, 3->gid") require.Equal(t, expand.Schema().Len(), 4) // source column a should be kept as real. diff --git a/pkg/planner/core/physical_plan_test.go b/pkg/planner/core/physical_plan_test.go index bd1fab87bb6b8..977e5e3820136 100644 --- a/pkg/planner/core/physical_plan_test.go +++ b/pkg/planner/core/physical_plan_test.go @@ -20,6 +20,7 @@ import ( "math" "testing" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/infoschema" @@ -514,5 +515,5 @@ func TestPhysicalTableScanExtractCorrelatedCols(t *testing.T) { // make sure the correlated columns are extracted correctly correlated := ts.ExtractCorrelatedCols() require.Equal(t, 1, len(correlated)) - require.Equal(t, "test.t2.company_no", correlated[0].String()) + require.Equal(t, "test.t2.company_no", correlated[0].StringWithCtx(errors.RedactLogDisable)) } diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index 264b5714dfddc..3ae807b4c015a 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -2693,7 +2693,15 @@ func (p *CTEDefinition) ExplainInfo() string { res = "Non-Recursive CTE" } if p.CTE.HasLimit { - res += fmt.Sprintf(", limit(offset:%v, count:%v)", p.CTE.LimitBeg, p.CTE.LimitEnd-p.CTE.LimitBeg) + offset, count := p.CTE.LimitBeg, p.CTE.LimitEnd-p.CTE.LimitBeg + switch p.SCtx().GetSessionVars().EnableRedactLog { + case errors.RedactLogMarker: + res += fmt.Sprintf(", limit(offset:‹%v›, count:‹%v›)", offset, count) + case errors.RedactLogDisable: + res += fmt.Sprintf(", limit(offset:%v, count:%v)", offset, count) + case errors.RedactLogEnable: + res += ", limit(offset:?, count:?)" + } } return res } diff --git a/pkg/planner/core/plan_cache_test.go b/pkg/planner/core/plan_cache_test.go index bd671073279f9..f38c1cb5df22e 100644 --- a/pkg/planner/core/plan_cache_test.go +++ b/pkg/planner/core/plan_cache_test.go @@ -71,12 +71,28 @@ func TestNonPreparedPlanCachePlanString(t *testing.T) { require.NoError(t, err) return plannercore.ToString(p) } - + defer func() { + tk.MustExec("set session tidb_redact_log=MARKER") + }() require.Equal(t, planString("select a from t where a < 1"), "IndexReader(Index(t.a)[[-inf,1)])") + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + require.Equal(t, planString("select a from t where a < 10"), "IndexReader(Index(t.a)[[-inf,10)])") // range 1 -> 10 + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec("set session tidb_redact_log=MARKER") + require.Equal(t, planString("select a from t where a < 10"), "IndexReader(Index(t.a)[[-inf,10)])") // range 1 -> 10 + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec("set session tidb_redact_log=ON") require.Equal(t, planString("select a from t where a < 10"), "IndexReader(Index(t.a)[[-inf,10)])") // range 1 -> 10 tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) require.Equal(t, planString("select * from t where b < 1"), "TableReader(Table(t)->Sel([lt(test.t.b, 1)]))") + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + require.Equal(t, planString("select * from t where b < 10"), "TableReader(Table(t)->Sel([lt(test.t.b, 10)]))") // filter 1 -> 10 + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec("set session tidb_redact_log=MARKER") + require.Equal(t, planString("select * from t where b < 10"), "TableReader(Table(t)->Sel([lt(test.t.b, 10)]))") // filter 1 -> 10 + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustExec("set session tidb_redact_log=ON") require.Equal(t, planString("select * from t where b < 10"), "TableReader(Table(t)->Sel([lt(test.t.b, 10)]))") // filter 1 -> 10 tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) } @@ -258,12 +274,22 @@ func TestIssue38269(t *testing.T) { tk.MustExec("prepare stmt1 from 'select /*+ inl_join(t2) */ * from t1 join t2 on t1.a = t2.a where t2.b in (?, ?, ?)'") tk.MustExec("set @a = 10, @b = 20, @c = 30, @d = 40, @e = 50, @f = 60") tk.MustExec("execute stmt1 using @a, @b, @c") + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + tk.MustExec("set session tidb_redact_log=MARKER") tk.MustExec("execute stmt1 using @d, @e, @f") tkProcess := tk.Session().ShowProcess() ps := []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) - rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() - require.Contains(t, rows[6][4], "range: decided by [eq(test.t2.a, test.t1.a) in(test.t2.b, 40, 50, 60)]") + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( + "IndexJoin_12 37.46 root inner join, inner:IndexLookUp_11, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "├─TableReader_24(Build) 9990.00 root data:Selection_23", + "│ └─Selection_23 9990.00 cop[tikv] not(isnull(test.t1.a))", + "│ └─TableFullScan_22 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp_11(Probe) 37.46 root ", + " ├─Selection_10(Build) 37.46 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan_8 37.50 cop[tikv] table:t2, index:idx(a, b) range: decided by [eq(test.t2.a, test.t1.a) in(test.t2.b, ‹40›, ‹50›, ‹60›)], keep order:false, stats:pseudo", + " └─TableRowIDScan_9(Probe) 37.46 cop[tikv] table:t2 keep order:false, stats:pseudo")) } func TestIssue38533(t *testing.T) { diff --git a/pkg/planner/core/point_get_plan.go b/pkg/planner/core/point_get_plan.go index 9b0a4962fad70..65da16445b2ca 100644 --- a/pkg/planner/core/point_get_plan.go +++ b/pkg/planner/core/point_get_plan.go @@ -52,6 +52,7 @@ import ( "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/plancodec" + "github.com/pingcap/tidb/pkg/util/redact" "github.com/pingcap/tidb/pkg/util/size" "github.com/pingcap/tidb/pkg/util/stringutil" "github.com/pingcap/tidb/pkg/util/tracing" @@ -192,11 +193,15 @@ func (p *PointGetPlan) OperatorInfo(normalized bool) string { if normalized { buffer.WriteString("handle:?") } else { + redactMode := p.SCtx().GetSessionVars().EnableRedactLog + redactOn := redactMode == errors.RedactLogEnable buffer.WriteString("handle:") - if p.UnsignedHandle { - buffer.WriteString(strconv.FormatUint(uint64(p.Handle.IntValue()), 10)) + if redactOn { + buffer.WriteString("?") + } else if p.UnsignedHandle { + redact.WriteRedact(&buffer, strconv.FormatUint(uint64(p.Handle.IntValue()), 10), redactMode) } else { - buffer.WriteString(p.Handle.String()) + redact.WriteRedact(&buffer, p.Handle.String(), redactMode) } } } diff --git a/pkg/planner/core/rule_aggregation_push_down.go b/pkg/planner/core/rule_aggregation_push_down.go index 4e4e4e0a72ee7..f8df7ef86f65b 100644 --- a/pkg/planner/core/rule_aggregation_push_down.go +++ b/pkg/planner/core/rule_aggregation_push_down.go @@ -19,6 +19,7 @@ import ( "context" "fmt" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/expression/aggregation" "github.com/pingcap/tidb/pkg/parser/ast" @@ -690,7 +691,7 @@ func appendAggPushDownAcrossJoinTraceStep(oldAgg, newAgg *LogicalAggregation, ag if i > 0 { buffer.WriteString(",") } - buffer.WriteString(aggFunc.String()) + buffer.WriteString(aggFunc.StringWithCtx(errors.RedactLogDisable)) } buffer.WriteString("] are decomposable with join") return buffer.String() @@ -715,7 +716,7 @@ func appendAggPushDownAcrossProjTraceStep(agg *LogicalAggregation, proj *Logical if i > 0 { buffer.WriteString(",") } - buffer.WriteString(aggFunc.String()) + buffer.WriteString(aggFunc.StringWithCtx(errors.RedactLogDisable)) } buffer.WriteString("]") return buffer.String() @@ -733,7 +734,7 @@ func appendAggPushDownAcrossUnionTraceStep(union *LogicalUnionAll, agg *LogicalA if i > 0 { buffer.WriteString(",") } - buffer.WriteString(aggFunc.String()) + buffer.WriteString(aggFunc.StringWithCtx(errors.RedactLogDisable)) } fmt.Fprintf(buffer, "] are decomposable with %v_%v", union.TP(), union.ID()) return buffer.String() diff --git a/pkg/planner/core/rule_decorrelate.go b/pkg/planner/core/rule_decorrelate.go index 48450912d1757..5e175bb825ba0 100644 --- a/pkg/planner/core/rule_decorrelate.go +++ b/pkg/planner/core/rule_decorrelate.go @@ -20,6 +20,7 @@ import ( "fmt" "math" + perrors "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/expression/aggregation" "github.com/pingcap/tidb/pkg/parser/ast" @@ -565,21 +566,21 @@ func appendModifyAggTraceStep(outerPlan LogicalPlan, p *LogicalApply, agg *Logic if i > 0 { buffer.WriteString(",") } - buffer.WriteString(col.String()) + buffer.WriteString(col.StringWithCtx(perrors.RedactLogDisable)) } buffer.WriteString("], and functions added [") for i, f := range appendedAggFuncs { if i > 0 { buffer.WriteString(",") } - buffer.WriteString(f.String()) + buffer.WriteString(f.StringWithCtx(perrors.RedactLogDisable)) } fmt.Fprintf(buffer, "], and %v_%v's conditions added [", p.TP(), p.ID()) for i, cond := range eqCondWithCorCol { if i > 0 { buffer.WriteString(",") } - buffer.WriteString(cond.String()) + buffer.WriteString(cond.StringWithCtx(perrors.RedactLogDisable)) } buffer.WriteString("]") return buffer.String() @@ -590,7 +591,7 @@ func appendModifyAggTraceStep(outerPlan LogicalPlan, p *LogicalApply, agg *Logic if i > 0 { buffer.WriteString(",") } - buffer.WriteString(cond.String()) + buffer.WriteString(cond.StringWithCtx(perrors.RedactLogDisable)) } fmt.Fprintf(buffer, "] are correlated to %v_%v and pulled up as %v_%v's join key", outerPlan.TP(), outerPlan.ID(), p.TP(), p.ID()) diff --git a/pkg/planner/core/rule_eliminate_projection.go b/pkg/planner/core/rule_eliminate_projection.go index b54c165ca303b..9e610e3385c1e 100644 --- a/pkg/planner/core/rule_eliminate_projection.go +++ b/pkg/planner/core/rule_eliminate_projection.go @@ -19,6 +19,7 @@ import ( "context" "fmt" + perrors "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" @@ -349,7 +350,7 @@ func appendDupProjEliminateTraceStep(parent, child *LogicalProjection, opt *util if i > 0 { buffer.WriteString(",") } - buffer.WriteString(expr.String()) + buffer.WriteString(expr.StringWithCtx(perrors.RedactLogDisable)) } buffer.WriteString("]") return buffer.String() diff --git a/pkg/planner/core/rule_generate_column_substitute.go b/pkg/planner/core/rule_generate_column_substitute.go index 46f9906fd0935..12ff95710feef 100644 --- a/pkg/planner/core/rule_generate_column_substitute.go +++ b/pkg/planner/core/rule_generate_column_substitute.go @@ -18,6 +18,7 @@ import ( "bytes" "context" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/util" @@ -100,10 +101,10 @@ func appendSubstituteColumnStep(lp LogicalPlan, candidateExpr expression.Express reason := func() string { return "" } action := func() string { buffer := bytes.NewBufferString("expression:") - buffer.WriteString(candidateExpr.String()) + buffer.WriteString(candidateExpr.StringWithCtx(errors.RedactLogDisable)) buffer.WriteString(" substituted by") buffer.WriteString(" column:") - buffer.WriteString(col.String()) + buffer.WriteString(col.StringWithCtx(errors.RedactLogDisable)) return buffer.String() } opt.AppendStepToCurrent(lp.ID(), lp.TP(), reason, action) diff --git a/pkg/planner/core/rule_join_elimination.go b/pkg/planner/core/rule_join_elimination.go index 2a71714a78a48..2edea74dcc723 100644 --- a/pkg/planner/core/rule_join_elimination.go +++ b/pkg/planner/core/rule_join_elimination.go @@ -19,6 +19,7 @@ import ( "context" "fmt" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/planner/util" @@ -264,14 +265,14 @@ func appendOuterJoinEliminateTraceStep(join *LogicalJoin, outerPlan LogicalPlan, if i > 0 { buffer.WriteString(",") } - buffer.WriteString(col.String()) + buffer.WriteString(col.StringWithCtx(errors.RedactLogDisable)) } buffer.WriteString("] are from outer table, and the inner join keys[") for i, key := range innerJoinKeys.Columns { if i > 0 { buffer.WriteString(",") } - buffer.WriteString(key.String()) + buffer.WriteString(key.StringWithCtx(errors.RedactLogDisable)) } buffer.WriteString("] are unique") return buffer.String() @@ -289,7 +290,7 @@ func appendOuterJoinEliminateAggregationTraceStep(join *LogicalJoin, outerPlan L if i > 0 { buffer.WriteString(",") } - buffer.WriteString(col.String()) + buffer.WriteString(col.StringWithCtx(errors.RedactLogDisable)) } buffer.WriteString("] in agg are from outer table, and the agg functions are duplicate agnostic") return buffer.String() diff --git a/pkg/planner/core/rule_partition_processor.go b/pkg/planner/core/rule_partition_processor.go index 017538239f55e..04e15d14d7c13 100644 --- a/pkg/planner/core/rule_partition_processor.go +++ b/pkg/planner/core/rule_partition_processor.go @@ -497,7 +497,7 @@ func (*partitionProcessor) reconstructTableColNames(ds *DataSource) ([]*types.Fi }) continue } - return nil, errors.Trace(fmt.Errorf("information of column %v is not found", colExpr.String())) + return nil, errors.Trace(fmt.Errorf("information of column %v is not found", colExpr.StringWithCtx(errors.RedactLogDisable))) } return names, nil } diff --git a/pkg/planner/core/rule_predicate_push_down.go b/pkg/planner/core/rule_predicate_push_down.go index 2032acbe71854..8af5f4394c6af 100644 --- a/pkg/planner/core/rule_predicate_push_down.go +++ b/pkg/planner/core/rule_predicate_push_down.go @@ -19,6 +19,7 @@ import ( "context" "fmt" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" @@ -860,7 +861,7 @@ func appendTableDualTraceStep(replaced LogicalPlan, dual LogicalPlan, conditions if i > 0 { buffer.WriteString(",") } - buffer.WriteString(cond.String()) + buffer.WriteString(cond.StringWithCtx(errors.RedactLogDisable)) } buffer.WriteString("] are constant false or null") return buffer.String() @@ -882,7 +883,7 @@ func appendSelectionPredicatePushDownTraceStep(p *LogicalSelection, conditions [ if i > 0 { buffer.WriteString(",") } - buffer.WriteString(cond.String()) + buffer.WriteString(cond.StringWithCtx(errors.RedactLogDisable)) } fmt.Fprintf(buffer, "] in %v_%v are pushed down", p.TP(), p.ID()) return buffer.String() @@ -904,7 +905,7 @@ func appendDataSourcePredicatePushDownTraceStep(ds *DataSource, opt *util.Logica if i > 0 { buffer.WriteString(",") } - buffer.WriteString(cond.String()) + buffer.WriteString(cond.StringWithCtx(errors.RedactLogDisable)) } fmt.Fprintf(buffer, "] are pushed down across %v_%v", ds.TP(), ds.ID()) return buffer.String() diff --git a/pkg/planner/core/rule_topn_push_down.go b/pkg/planner/core/rule_topn_push_down.go index 5f4d73b0c3e8c..57ff9e8c815b9 100644 --- a/pkg/planner/core/rule_topn_push_down.go +++ b/pkg/planner/core/rule_topn_push_down.go @@ -19,6 +19,7 @@ import ( "context" "fmt" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/planner/util" ) @@ -258,7 +259,7 @@ func appendTopNPushDownJoinTraceStep(p *LogicalJoin, topN *LogicalTopN, idx int, if i > 0 { buffer.WriteString(",") } - buffer.WriteString(item.String()) + buffer.WriteString(item.StringWithCtx(errors.RedactLogDisable)) } buffer.WriteString("] contained in ") if idx == 0 { @@ -279,7 +280,7 @@ func appendSortPassByItemsTraceStep(sort *LogicalSort, topN *LogicalTopN, opt *u if i > 0 { buffer.WriteString(",") } - buffer.WriteString(item.String()) + buffer.WriteString(item.StringWithCtx(errors.RedactLogDisable)) } fmt.Fprintf(buffer, "] to %v_%v", topN.TP(), topN.ID()) return buffer.String() diff --git a/pkg/planner/core/runtime_filter.go b/pkg/planner/core/runtime_filter.go index 2b705ab8c6efc..13d79083efde5 100644 --- a/pkg/planner/core/runtime_filter.go +++ b/pkg/planner/core/runtime_filter.go @@ -18,6 +18,7 @@ import ( "fmt" "strings" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -220,7 +221,7 @@ func (rf *RuntimeFilter) ToPB(ctx PlanContext, client kv.Client) (*tipb.RuntimeF for _, srcExpr := range rf.srcExprList { srcExprPB := pc.ExprToPB(srcExpr) if srcExprPB == nil { - return nil, plannererrors.ErrInternal.GenWithStack("failed to transform src expr %s to pb in runtime filter", srcExpr.String()) + return nil, plannererrors.ErrInternal.GenWithStack("failed to transform src expr %s to pb in runtime filter", srcExpr.StringWithCtx(errors.RedactLogDisable)) } srcExprListPB = append(srcExprListPB, srcExprPB) } @@ -228,7 +229,7 @@ func (rf *RuntimeFilter) ToPB(ctx PlanContext, client kv.Client) (*tipb.RuntimeF for _, targetExpr := range rf.targetExprList { targetExprPB := pc.ExprToPB(targetExpr) if targetExprPB == nil { - return nil, plannererrors.ErrInternal.GenWithStack("failed to transform target expr %s to pb in runtime filter", targetExpr.String()) + return nil, plannererrors.ErrInternal.GenWithStack("failed to transform target expr %s to pb in runtime filter", targetExpr.StringWithCtx(errors.RedactLogDisable)) } targetExprListPB = append(targetExprListPB, targetExprPB) } diff --git a/pkg/planner/core/runtime_filter_generator.go b/pkg/planner/core/runtime_filter_generator.go index 385cff8ca04d6..aa55c4324fa9d 100644 --- a/pkg/planner/core/runtime_filter_generator.go +++ b/pkg/planner/core/runtime_filter_generator.go @@ -15,6 +15,7 @@ package core import ( + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" @@ -183,7 +184,7 @@ func (*RuntimeFilterGenerator) matchEQPredicate(eqPredicate *expression.ScalarFu // exclude null safe equal predicate if eqPredicate.FuncName.L == ast.NullEQ { logutil.BgLogger().Debug("The runtime filter doesn't support null safe eq predicate", - zap.String("EQPredicate", eqPredicate.String())) + zap.String("EQPredicate", eqPredicate.StringWithCtx(errors.RedactLogDisable))) return false } var targetColumn, srcColumn *expression.Column @@ -200,7 +201,7 @@ func (*RuntimeFilterGenerator) matchEQPredicate(eqPredicate *expression.ScalarFu // todo: cast expr in target column if targetColumn.IsHidden || targetColumn.OrigName == "" { logutil.BgLogger().Debug("Target column does not match RF pattern", - zap.String("EQPredicate", eqPredicate.String()), + zap.String("EQPredicate", eqPredicate.StringWithCtx(errors.RedactLogDisable)), zap.String("TargetColumn", targetColumn.String()), zap.Bool("IsHidden", targetColumn.IsHidden), zap.String("OrigName", targetColumn.OrigName)) @@ -212,7 +213,7 @@ func (*RuntimeFilterGenerator) matchEQPredicate(eqPredicate *expression.ScalarFu srcColumnType == mysql.TypeLongBlob || srcColumnType == mysql.TypeMediumBlob || srcColumnType == mysql.TypeTinyBlob || srcColumn.GetType().Hybrid() || srcColumn.GetType().IsArray() { logutil.BgLogger().Debug("Src column type does not match RF pattern", - zap.String("EQPredicate", eqPredicate.String()), + zap.String("EQPredicate", eqPredicate.StringWithCtx(errors.RedactLogDisable)), zap.String("SrcColumn", srcColumn.String()), zap.String("SrcColumnType", srcColumn.GetType().String())) return false diff --git a/pkg/planner/core/stringer.go b/pkg/planner/core/stringer.go index 0dd0abbf0e556..0d66eb6646f34 100644 --- a/pkg/planner/core/stringer.go +++ b/pkg/planner/core/stringer.go @@ -19,6 +19,7 @@ import ( "fmt" "strings" + perrors "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/util/plancodec" ) @@ -116,8 +117,8 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { str = "LeftHashJoin{" + strings.Join(children, "->") + "}" } for _, eq := range x.EqualConditions { - l := eq.GetArgs()[0].String() - r := eq.GetArgs()[1].String() + l := eq.GetArgs()[0].StringWithCtx(perrors.RedactLogDisable) + r := eq.GetArgs()[1].StringWithCtx(perrors.RedactLogDisable) str += fmt.Sprintf("(%s,%s)", l, r) } case *PhysicalMergeJoin: @@ -145,8 +146,8 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { } str = id + "{" + strings.Join(children, "->") + "}" for i := range x.LeftJoinKeys { - l := x.LeftJoinKeys[i].String() - r := x.RightJoinKeys[i].String() + l := x.LeftJoinKeys[i].StringWithCtx(perrors.RedactLogDisable) + r := x.RightJoinKeys[i].StringWithCtx(perrors.RedactLogDisable) str += fmt.Sprintf("(%s,%s)", l, r) } case *LogicalApply, *PhysicalApply: @@ -186,8 +187,8 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { str = "Join{" + strings.Join(children, "->") + "}" idxs = idxs[:last] for _, eq := range x.EqualConditions { - l := eq.GetArgs()[0].String() - r := eq.GetArgs()[1].String() + l := eq.GetArgs()[0].StringWithCtx(perrors.RedactLogDisable) + r := eq.GetArgs()[1].StringWithCtx(perrors.RedactLogDisable) str += fmt.Sprintf("(%s,%s)", l, r) } case *LogicalUnionAll, *PhysicalUnionAll, *LogicalPartitionUnionAll: @@ -240,7 +241,7 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { case *LogicalAggregation: str = "Aggr(" for i, aggFunc := range x.AggFuncs { - str += aggFunc.String() + str += aggFunc.StringWithCtx(perrors.RedactLogDisable) if i != len(x.AggFuncs)-1 { str += "," } @@ -308,7 +309,7 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { for _, col := range x.ColTasks { var colNames []string if col.HandleCols != nil { - colNames = append(colNames, col.HandleCols.String()) + colNames = append(colNames, col.HandleCols.StringWithCtx(perrors.RedactLogDisable)) } for _, c := range col.ColsInfo { colNames = append(colNames, c.Name.O) diff --git a/pkg/planner/core/tests/redact/BUILD.bazel b/pkg/planner/core/tests/redact/BUILD.bazel new file mode 100644 index 0000000000000..3c5eea0499f1b --- /dev/null +++ b/pkg/planner/core/tests/redact/BUILD.bazel @@ -0,0 +1,18 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "redact_test", + timeout = "short", + srcs = [ + "main_test.go", + "redact_test.go", + ], + flaky = True, + shard_count = 4, + deps = [ + "//pkg/planner/core/internal", + "//pkg/testkit", + "//pkg/testkit/testsetup", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/pkg/planner/core/tests/redact/main_test.go b/pkg/planner/core/tests/redact/main_test.go new file mode 100644 index 0000000000000..beb8cff7e8c54 --- /dev/null +++ b/pkg/planner/core/tests/redact/main_test.go @@ -0,0 +1,34 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package redact + +import ( + "flag" + "testing" + + "github.com/pingcap/tidb/pkg/testkit/testsetup" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testsetup.SetupForCommonTest() + flag.Parse() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon"), + goleak.IgnoreTopFunction("github.com/bazelbuild/rules_go/go/tools/bzltestutil.RegisterTimeoutHandler.func1"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + } + goleak.VerifyTestMain(m, opts...) +} diff --git a/pkg/planner/core/tests/redact/redact_test.go b/pkg/planner/core/tests/redact/redact_test.go new file mode 100644 index 0000000000000..b747c25c11b8f --- /dev/null +++ b/pkg/planner/core/tests/redact/redact_test.go @@ -0,0 +1,271 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package redact + +import ( + "testing" + + "github.com/pingcap/tidb/pkg/planner/core/internal" + "github.com/pingcap/tidb/pkg/testkit" +) + +func TestRedactExplain(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t2(id int, a int, b int, primary key(id, a)) partition by hash(id + a) partitions 10;") + tk.MustExec("create table t1(id int primary key, a int, b int) partition by hash(id) partitions 10;") + tk.MustExec("create table t(a int primary key, b int);") + tk.MustExec(`create table tlist (a int) partition by list (a) ( + partition p0 values in (0, 1, 2), + partition p1 values in (3, 4, 5), + partition p2 values in (6, 7, 8), + partition p3 values in (9, 10, 11))`) + tk.MustExec("create table employee (empid int, deptid int, salary decimal(10,2))") + tk.MustExec("CREATE TABLE person (id INT NOT NULL AUTO_INCREMENT PRIMARY KEY,name VARCHAR(255) NOT NULL,address_info JSON,city_no INT AS (JSON_EXTRACT(address_info, '$.city_no')) VIRTUAL,KEY(city_no));") + + internal.SetTiFlashReplica(t, dom, "test", "employee") + // --------------------------------------------------------------------------- + // tidb_redact_log=MARKER + // --------------------------------------------------------------------------- + tk.MustExec("set session tidb_redact_log=MARKER") + // in multi-value + tk.MustQuery("explain select 1 from t left join tlist on tlist.a=t.a where t.a in (12, 13)"). + Check(testkit.Rows( + "Projection_7 2.50 root ‹1›->Column#5", + "└─HashJoin_9 2.50 root left outer join, equal:[eq(test.t.a, test.tlist.a)]", + " ├─Batch_Point_Get_10(Build) 2.00 root table:t handle:[12 13], keep order:false, desc:false", + " └─TableReader_13(Probe) 20.00 root partition:dual data:Selection_12", + " └─Selection_12 20.00 cop[tikv] in(test.tlist.a, ‹12›, ‹13›), not(isnull(test.tlist.a))", + " └─TableFullScan_11 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo")) + // TableRangeScan + Limit + tk.MustQuery("explain select * from t where a > 1 limit 10 offset 10;"). + Check(testkit.Rows( + "Limit_8 10.00 root offset:‹10›, count:‹10›", + "└─TableReader_12 20.00 root data:Limit_11", + " └─Limit_11 20.00 cop[tikv] offset:‹0›, count:‹20›", + " └─TableRangeScan_10 20.00 cop[tikv] table:t range:(‹1›,+inf], keep order:false, stats:pseudo")) + tk.MustQuery("explain select * from t where a < 1;"). + Check(testkit.Rows( + "TableReader_6 3333.33 root data:TableRangeScan_5", + "└─TableRangeScan_5 3333.33 cop[tikv] table:t range:[-inf,‹1›), keep order:false, stats:pseudo")) + // PointGet + order by + tk.MustQuery("explain select b+1 as vt from t where a = 1 order by vt;"). + Check(testkit.Rows( + "Sort_5 1.00 root Column#3", + "└─Projection_7 1.00 root plus(test.t.b, ‹1›)->Column#3", + " └─Point_Get_8 1.00 root table:t handle:‹1›")) + // expression partition key + tk.MustQuery("explain select *, row_number() over (partition by deptid+1) FROM employee").Check(testkit.Rows( + "TableReader_31 10000.00 root MppVersion: 2, data:ExchangeSender_30", + "└─ExchangeSender_30 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_7 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, Column#7, stream_count: 8", + " └─Window_29 10000.00 mpp[tiflash] row_number()->Column#7 over(partition by Column#6 rows between current row and current row), stream_count: 8", + " └─Sort_14 10000.00 mpp[tiflash] Column#6, stream_count: 8", + " └─ExchangeReceiver_13 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender_12 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#6, collate: binary], stream_count: 8", + " └─Projection_10 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, plus(test.employee.deptid, ‹1›)->Column#6", // <- here + " └─TableFullScan_11 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo")) + tk.MustQuery("explain format = 'brief' select * from tlist where a in (2)").Check(testkit.Rows( + "TableReader 10.00 root partition:p0 data:Selection", + "└─Selection 10.00 cop[tikv] eq(test.tlist.a, ‹2›)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo")) + // CTE + tk.MustQuery("explain with recursive cte(a) as (select 1 union select a + 1 from cte where a < 1000) select * from cte, t limit 100 offset 100;").Check( + testkit.Rows( + "Limit_25 100.00 root offset:‹100›, count:‹100›", + "└─HashJoin_27 200.00 root CARTESIAN inner join", + " ├─CTEFullScan_31(Build) 2.00 root CTE:cte data:CTE_0", + " └─TableReader_33(Probe) 100.00 root data:TableFullScan_32", + " └─TableFullScan_32 100.00 cop[tikv] table:t keep order:false, stats:pseudo", + "CTE_0 2.00 root Recursive CTE", + "├─Projection_16(Seed Part) 1.00 root ‹1›->Column#2", + "│ └─TableDual_17 1.00 root rows:1", + "└─Projection_18(Recursive Part) 0.80 root cast(plus(Column#3, ‹1›), bigint(1) BINARY)->Column#5", + " └─Selection_19 0.80 root lt(Column#3, ‹1000›)", + " └─CTETable_20 1.00 root Scan on CTE_0")) + // virtual generated column + tk.MustQuery("EXPLAIN format = 'brief' SELECT name FROM person where city_no=1").Check(testkit.Rows( + "Projection 10.00 root test.person.name", + "└─Projection 10.00 root test.person.name, test.person.city_no", + " └─IndexLookUp 10.00 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:person, index:city_no(city_no) range:[‹1›,‹1›], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:person keep order:false, stats:pseudo")) + // group by + tk.MustQuery(" explain select 1 from test.t group by 1").Check(testkit.Rows( + "Projection_4 1.00 root ‹1›->Column#3", + "└─HashAgg_9 1.00 root group by:Column#7, funcs:firstrow(Column#8)->Column#6", + " └─TableReader_10 1.00 root data:HashAgg_5", + " └─HashAgg_5 1.00 cop[tikv] group by:‹1›, funcs:firstrow(‹1›)->Column#8", + " └─TableFullScan_8 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) + // --------------------------------------------------------------------------- + // tidb_redact_log=ON + // --------------------------------------------------------------------------- + tk.MustExec("set session tidb_redact_log=ON") + // in multi-value + tk.MustQuery("explain select 1 from t left join tlist on tlist.a=t.a where t.a in (12, 13)"). + Check(testkit.Rows( + "Projection_7 2.50 root ?->Column#5", + "└─HashJoin_9 2.50 root left outer join, equal:[eq(test.t.a, test.tlist.a)]", + " ├─Batch_Point_Get_10(Build) 2.00 root table:t handle:[12 13], keep order:false, desc:false", + " └─TableReader_13(Probe) 20.00 root partition:dual data:Selection_12", + " └─Selection_12 20.00 cop[tikv] in(test.tlist.a, ?, ?), not(isnull(test.tlist.a))", + " └─TableFullScan_11 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo")) + // TableRangeScan + Limit + tk.MustQuery("explain select * from t where a > 1 limit 10 offset 10;"). + Check(testkit.Rows( + "Limit_8 10.00 root offset:?, count:?", + "└─TableReader_12 20.00 root data:Limit_11", + " └─Limit_11 20.00 cop[tikv] offset:?, count:?", + " └─TableRangeScan_10 20.00 cop[tikv] table:t range:(?,+inf], keep order:false, stats:pseudo")) + tk.MustQuery("explain select * from t where a < 1;"). + Check(testkit.Rows( + "TableReader_6 3333.33 root data:TableRangeScan_5", + "└─TableRangeScan_5 3333.33 cop[tikv] table:t range:[-inf,?), keep order:false, stats:pseudo")) + // PointGet + order by + tk.MustQuery("explain select b+1 as vt from t where a = 1 order by vt;"). + Check(testkit.Rows( + "Sort_5 1.00 root Column#3", + "└─Projection_7 1.00 root plus(test.t.b, ?)->Column#3", + " └─Point_Get_8 1.00 root table:t handle:?")) + // expression partition key + tk.MustQuery("explain select *, row_number() over (partition by deptid+1) FROM employee").Check(testkit.Rows( + "TableReader_31 10000.00 root MppVersion: 2, data:ExchangeSender_30", + "└─ExchangeSender_30 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_7 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, Column#7, stream_count: 8", + " └─Window_29 10000.00 mpp[tiflash] row_number()->Column#7 over(partition by Column#6 rows between current row and current row), stream_count: 8", + " └─Sort_14 10000.00 mpp[tiflash] Column#6, stream_count: 8", + " └─ExchangeReceiver_13 10000.00 mpp[tiflash] stream_count: 8", + " └─ExchangeSender_12 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: Column#6, collate: binary], stream_count: 8", + " └─Projection_10 10000.00 mpp[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, plus(test.employee.deptid, ?)->Column#6", // <- here + " └─TableFullScan_11 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo")) + tk.MustQuery("explain format = 'brief' select * from tlist where a in (2)").Check(testkit.Rows( + "TableReader 10.00 root partition:p0 data:Selection", + "└─Selection 10.00 cop[tikv] eq(test.tlist.a, ?)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo")) + // CTE + tk.MustQuery("explain with recursive cte(a) as (select 1 union select a + 1 from cte where a < 1000) select * from cte, t limit 100 offset 100;").Check( + testkit.Rows("Limit_25 100.00 root offset:?, count:?", + "└─HashJoin_27 200.00 root CARTESIAN inner join", + " ├─CTEFullScan_31(Build) 2.00 root CTE:cte data:CTE_0", + " └─TableReader_33(Probe) 100.00 root data:TableFullScan_32", + " └─TableFullScan_32 100.00 cop[tikv] table:t keep order:false, stats:pseudo", + "CTE_0 2.00 root Recursive CTE", + "├─Projection_16(Seed Part) 1.00 root ?->Column#2", + "│ └─TableDual_17 1.00 root rows:1", + "└─Projection_18(Recursive Part) 0.80 root cast(plus(Column#3, ?), bigint(1) BINARY)->Column#5", + " └─Selection_19 0.80 root lt(Column#3, ?)", + " └─CTETable_20 1.00 root Scan on CTE_0")) + // virtual generated column + tk.MustQuery("EXPLAIN format = 'brief' SELECT name FROM person where city_no=1").Check(testkit.Rows( + "Projection 10.00 root test.person.name", + "└─Projection 10.00 root test.person.name, test.person.city_no", + " └─IndexLookUp 10.00 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:person, index:city_no(city_no) range:[?,?], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:person keep order:false, stats:pseudo")) + // group by + tk.MustQuery(" explain select 1 from test.t group by 1").Check(testkit.Rows( + "Projection_4 1.00 root ?->Column#3", + "└─HashAgg_9 1.00 root group by:Column#7, funcs:firstrow(Column#8)->Column#6", + " └─TableReader_10 1.00 root data:HashAgg_5", + " └─HashAgg_5 1.00 cop[tikv] group by:?, funcs:firstrow(?)->Column#8", + " └─TableFullScan_8 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) +} + +func TestRedactForRangeInfo(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set @@tidb_enable_prepared_plan_cache=1`) + tk.MustExec("set @@tidb_enable_collect_execution_info=0") + tk.MustExec(`set @@tidb_opt_advanced_join_hint=0`) + tk.MustExec("use test") + tk.MustExec("create table t1(a int)") + tk.MustExec("create table t2(a int, b int, c int, index idx(a, b))") + tk.MustExec("set session tidb_redact_log=ON") + tk.MustQuery("explain select /*+ inl_join(t2) */ * from t1 join t2 on t1.a = t2.a where t2.b in (10, 20, 30)").Check( + testkit.Rows( + "IndexJoin_12 37.46 root inner join, inner:IndexLookUp_11, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", + "├─TableReader_24(Build) 9990.00 root data:Selection_23", + "│ └─Selection_23 9990.00 cop[tikv] not(isnull(test.t1.a))", + "│ └─TableFullScan_22 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─IndexLookUp_11(Probe) 37.46 root ", + " ├─Selection_10(Build) 37.46 cop[tikv] not(isnull(test.t2.a))", + " │ └─IndexRangeScan_8 37.50 cop[tikv] table:t2, index:idx(a, b) range: decided by [eq(test.t2.a, test.t1.a) in(test.t2.b, ?, ?, ?)], keep order:false, stats:pseudo", + " └─TableRowIDScan_9(Probe) 37.46 cop[tikv] table:t2 keep order:false, stats:pseudo", + )) +} + +func TestJoinNotSupportedByTiFlash(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists table_1") + tk.MustExec("create table table_1(id int not null, bit_col bit(2) not null, datetime_col datetime not null, index idx(id, bit_col, datetime_col))") + tk.MustExec("insert into table_1 values(1,b'1','2020-01-01 00:00:00'),(2,b'0','2020-01-01 00:00:00')") + tk.MustExec("analyze table table_1") + + tk.MustExec("insert into mysql.expr_pushdown_blacklist values('dayofmonth', 'tiflash', '');") + tk.MustExec("admin reload expr_pushdown_blacklist;") + tk.MustExec("set session tidb_redact_log=ON") + tk.MustQuery("explain format = 'brief' select * from table_1 a left join table_1 b on a.id = b.id and dayofmonth(a.datetime_col) > 100").Check(testkit.Rows( + "MergeJoin 2.00 root left outer join, left key:test.table_1.id, right key:test.table_1.id, left cond:gt(dayofmonth(test.table_1.datetime_col), ?)", + "├─IndexReader(Build) 2.00 root index:IndexFullScan", + "│ └─IndexFullScan 2.00 cop[tikv] table:b, index:idx(id, bit_col, datetime_col) keep order:true", + "└─IndexReader(Probe) 2.00 root index:IndexFullScan", + " └─IndexFullScan 2.00 cop[tikv] table:a, index:idx(id, bit_col, datetime_col) keep order:true")) + tk.MustExec("set session tidb_redact_log=MARKER") + tk.MustQuery("explain format = 'brief' select * from table_1 a left join table_1 b on a.id = b.id and dayofmonth(a.datetime_col) > 100").Check(testkit.Rows( + "MergeJoin 2.00 root left outer join, left key:test.table_1.id, right key:test.table_1.id, left cond:gt(dayofmonth(test.table_1.datetime_col), ‹100›)", + "├─IndexReader(Build) 2.00 root index:IndexFullScan", + "│ └─IndexFullScan 2.00 cop[tikv] table:b, index:idx(id, bit_col, datetime_col) keep order:true", + "└─IndexReader(Probe) 2.00 root index:IndexFullScan", + " └─IndexFullScan 2.00 cop[tikv] table:a, index:idx(id, bit_col, datetime_col) keep order:true")) +} + +func TestRedactTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("create table test.first_range(p int not null, o int not null, v int not null, o_datetime datetime not null, o_time time not null);") + tk.MustExec("insert into test.first_range (p, o, v, o_datetime, o_time) values (0, 0, 0, '2023-9-20 11:17:10', '11:17:10');") + tk.MustExec("create table test.first_range_d64(p int not null, o decimal(17,1) not null, v int not null);") + tk.MustExec("insert into test.first_range_d64 (p, o, v) values (0, 0.1, 0), (1, 1.0, 1), (1, 2.1, 2), (1, 4.1, 4), (1, 8.1, 8), (2, 0.0, 0), (2, 3.1, 3), (2, 10.0, 10), (2, 13.1, 13), (2, 15.1, 15), (3, 1.1, 1), (3, 2.9, 3), (3, 5.1, 5), (3, 9.1, 9), (3, 15.0, 15), (3, 20.1, 20), (3, 31.1, 31);") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + // Create virtual tiflash replica info. + internal.SetTiFlashReplica(t, dom, "test", "first_range") + internal.SetTiFlashReplica(t, dom, "test", "first_range_d64") + + tk.MustExec(`set @@tidb_max_tiflash_threads=20`) + tk.MustExec("set session tidb_redact_log=ON") + tk.MustQuery("explain select *, first_value(v) over (partition by p order by o range between 3 preceding and 0 following) as a from test.first_range;").Check(testkit.Rows( + "TableReader_23 10000.00 root MppVersion: 2, data:ExchangeSender_22", + "└─ExchangeSender_22 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Window_21 10000.00 mpp[tiflash] first_value(test.first_range.v)->Column#8 over(partition by test.first_range.p order by test.first_range.o range between ? preceding and ? following), stream_count: 20", + " └─Sort_13 10000.00 mpp[tiflash] test.first_range.p, test.first_range.o, stream_count: 20", + " └─ExchangeReceiver_12 10000.00 mpp[tiflash] stream_count: 20", + " └─ExchangeSender_11 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.first_range.p, collate: binary], stream_count: 20", + " └─TableFullScan_10 10000.00 mpp[tiflash] table:first_range keep order:false, stats:pseudo")) + tk.MustExec("set session tidb_redact_log=MARKER") + tk.MustQuery("explain select *, first_value(v) over (partition by p order by o range between 3 preceding and 0 following) as a from test.first_range;").Check(testkit.Rows( + "TableReader_23 10000.00 root MppVersion: 2, data:ExchangeSender_22", + "└─ExchangeSender_22 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Window_21 10000.00 mpp[tiflash] first_value(test.first_range.v)->Column#8 over(partition by test.first_range.p order by test.first_range.o range between ‹3› preceding and ‹0› following), stream_count: 20", + " └─Sort_13 10000.00 mpp[tiflash] test.first_range.p, test.first_range.o, stream_count: 20", + " └─ExchangeReceiver_12 10000.00 mpp[tiflash] stream_count: 20", + " └─ExchangeSender_11 10000.00 mpp[tiflash] ExchangeType: HashPartition, Compression: FAST, Hash Cols: [name: test.first_range.p, collate: binary], stream_count: 20", + " └─TableFullScan_10 10000.00 mpp[tiflash] table:first_range keep order:false, stats:pseudo")) +} diff --git a/pkg/planner/util/BUILD.bazel b/pkg/planner/util/BUILD.bazel index e1ab66724eb81..f02a966d934d0 100644 --- a/pkg/planner/util/BUILD.bazel +++ b/pkg/planner/util/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/util/ranger", "//pkg/util/size", "//pkg/util/tracing", + "@com_github_pingcap_errors//:errors", ], ) diff --git a/pkg/planner/util/byitem.go b/pkg/planner/util/byitem.go index 3b429d4a0a4ac..f4d9315528ad2 100644 --- a/pkg/planner/util/byitem.go +++ b/pkg/planner/util/byitem.go @@ -16,7 +16,9 @@ package util import ( "fmt" + "strings" + perrors "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/util/size" ) @@ -29,10 +31,15 @@ type ByItems struct { // String implements fmt.Stringer interface. func (by *ByItems) String() string { + return by.StringWithCtx(perrors.RedactLogDisable) +} + +// StringWithCtx implements expression.StringerWithCtx interface. +func (by *ByItems) StringWithCtx(redact string) string { if by.Desc { - return fmt.Sprintf("%s true", by.Expr) + return fmt.Sprintf("%s true", by.Expr.StringWithCtx(redact)) } - return by.Expr.String() + return by.Expr.StringWithCtx(redact) } // Clone makes a copy of ByItems. @@ -57,3 +64,17 @@ func (by *ByItems) MemoryUsage() (sum int64) { } return sum } + +// StringifyByItemsWithCtx is used to print ByItems slice. +func StringifyByItemsWithCtx(byItems []*ByItems) string { + sb := strings.Builder{} + sb.WriteString("[") + for i, item := range byItems { + sb.WriteString(item.StringWithCtx(perrors.RedactLogDisable)) + if i != len(byItems)-1 { + sb.WriteString(" ") + } + } + sb.WriteString("]") + return sb.String() +} diff --git a/pkg/table/tables/test/partition/BUILD.bazel b/pkg/table/tables/test/partition/BUILD.bazel index ca79dd5358c89..3192f9c588ea4 100644 --- a/pkg/table/tables/test/partition/BUILD.bazel +++ b/pkg/table/tables/test/partition/BUILD.bazel @@ -22,6 +22,7 @@ go_test( "//pkg/types", "//pkg/util", "//pkg/util/logutil", + "@com_github_pingcap_errors//:errors", "@com_github_stretchr_testify//require", "@org_uber_go_goleak//:goleak", "@org_uber_go_zap//:zap", diff --git a/pkg/table/tables/test/partition/partition_test.go b/pkg/table/tables/test/partition/partition_test.go index 18eb0a55b66a8..9a688adee4e72 100644 --- a/pkg/table/tables/test/partition/partition_test.go +++ b/pkg/table/tables/test/partition/partition_test.go @@ -23,6 +23,7 @@ import ( "testing" gotime "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" @@ -260,7 +261,7 @@ func TestGeneratePartitionExpr(t *testing.T) { "1", } for i, expr := range pe.UpperBounds { - require.Equal(t, upperBounds[i], expr.String()) + require.Equal(t, upperBounds[i], expr.StringWithCtx(errors.RedactLogDisable)) } } diff --git a/pkg/util/ranger/points.go b/pkg/util/ranger/points.go index 81d46d14c7071..11b8eb5361c13 100644 --- a/pkg/util/ranger/points.go +++ b/pkg/util/ranger/points.go @@ -664,12 +664,12 @@ func (r *builder) buildFromIn( for _, e := range list { v, ok := e.(*expression.Constant) if !ok { - r.err = plannererrors.ErrUnsupportedType.GenWithStack("expr:%v is not constant", e) + r.err = plannererrors.ErrUnsupportedType.GenWithStack("expr:%v is not constant", e.StringWithCtx(errors.RedactLogDisable)) return getFullRange(), hasNull } dt, err := v.Eval(evalCtx, chunk.Row{}) if err != nil { - r.err = plannererrors.ErrUnsupportedType.GenWithStack("expr:%v is not evaluated", e) + r.err = plannererrors.ErrUnsupportedType.GenWithStack("expr:%v is not evaluated", e.StringWithCtx(errors.RedactLogDisable)) return getFullRange(), hasNull } if dt.IsNull() { diff --git a/pkg/util/ranger/types.go b/pkg/util/ranger/types.go index dab82fabc2813..9fd2d85f01549 100644 --- a/pkg/util/ranger/types.go +++ b/pkg/util/ranger/types.go @@ -173,15 +173,38 @@ func HasFullRange(ranges []*Range, unsignedIntHandle bool) bool { return false } +func dealWithRedact(input string, redact string) string { + if input == "-inf" || input == "+inf" { + return input + } + if redact == errors.RedactLogDisable { + return input + } else if redact == errors.RedactLogEnable { + return "?" + } + return fmt.Sprintf("‹%s›", input) +} + // String implements the Stringer interface. +// don't use it in the product. func (ran *Range) String() string { + return ran.string(errors.RedactLogDisable) +} + +// Redact is to print the range with redacting sensitive data. +func (ran *Range) Redact(redact string) string { + return ran.string(redact) +} + +// String implements the Stringer interface. +func (ran *Range) string(redact string) string { lowStrs := make([]string, 0, len(ran.LowVal)) for _, d := range ran.LowVal { - lowStrs = append(lowStrs, formatDatum(d, true)) + lowStrs = append(lowStrs, dealWithRedact(formatDatum(d, true), redact)) } highStrs := make([]string, 0, len(ran.LowVal)) for _, d := range ran.HighVal { - highStrs = append(highStrs, formatDatum(d, false)) + highStrs = append(highStrs, dealWithRedact(formatDatum(d, false), redact)) } l, r := "[", "]" if ran.LowExclude { diff --git a/pkg/util/redact/redact.go b/pkg/util/redact/redact.go index 4cfb5dfd9c1ba..0ca06fe17160d 100644 --- a/pkg/util/redact/redact.go +++ b/pkg/util/redact/redact.go @@ -209,3 +209,17 @@ func Key(key []byte) string { } return strings.ToUpper(hex.EncodeToString(key)) } + +// WriteRedact is to write string with redact into `strings.Builder` +func WriteRedact(build *strings.Builder, v string, redact string) { + if redact == errors.RedactLogMarker { + build.WriteString("‹") + build.WriteString(v) + build.WriteString("›") + return + } else if redact == errors.RedactLogEnable { + build.WriteString("?") + return + } + build.WriteString(v) +} diff --git a/tests/integrationtest/r/executor/jointest/join.result b/tests/integrationtest/r/executor/jointest/join.result index 431e45630ccd6..8e1330524372b 100644 --- a/tests/integrationtest/r/executor/jointest/join.result +++ b/tests/integrationtest/r/executor/jointest/join.result @@ -220,7 +220,7 @@ insert into t1 values(1, 100), (2, 100), (3, 100), (4, 100), (5, 100); insert into t2 select a*100, b*100 from t1; explain format = 'brief' select /*+ TIDB_SMJ(t2) */ * from t1 left outer join t2 on t1.a=t2.a and t1.a!=3 order by t1.a; id estRows task access object operator info -MergeJoin 10000.00 root left outer join, left key:executor__jointest__join.t1.a, right key:executor__jointest__join.t2.a, left cond:[ne(executor__jointest__join.t1.a, 3)] +MergeJoin 10000.00 root left outer join, left key:executor__jointest__join.t1.a, right key:executor__jointest__join.t2.a, left cond:ne(executor__jointest__join.t1.a, 3) ├─TableReader(Build) 6666.67 root data:TableRangeScan │ └─TableRangeScan 6666.67 cop[tikv] table:t2 range:[-inf,3), (3,+inf], keep order:true, stats:pseudo └─TableReader(Probe) 10000.00 root data:TableFullScan diff --git a/tests/integrationtest/r/executor/merge_join.result b/tests/integrationtest/r/executor/merge_join.result index ec528657c2d38..2bc38e95ef84d 100644 --- a/tests/integrationtest/r/executor/merge_join.result +++ b/tests/integrationtest/r/executor/merge_join.result @@ -61,7 +61,7 @@ c1 c2 c1 c2 explain format = 'brief' select /*+ TIDB_SMJ(t) */ * from t left outer join t1 on t.c1 = t1.c1 and t.c1 != 1 order by t1.c1; id estRows task access object operator info Sort 10000.00 root executor__merge_join.t1.c1 -└─MergeJoin 10000.00 root left outer join, left key:executor__merge_join.t.c1, right key:executor__merge_join.t1.c1, left cond:[ne(executor__merge_join.t.c1, 1)] +└─MergeJoin 10000.00 root left outer join, left key:executor__merge_join.t.c1, right key:executor__merge_join.t1.c1, left cond:ne(executor__merge_join.t.c1, 1) ├─Sort(Build) 6656.67 root executor__merge_join.t1.c1 │ └─TableReader 6656.67 root data:Selection │ └─Selection 6656.67 cop[tikv] ne(executor__merge_join.t1.c1, 1), not(isnull(executor__merge_join.t1.c1)) @@ -338,7 +338,7 @@ explain format = 'brief' select /*+ TIDB_SMJ(t) */ * from t left outer join t1 o id estRows task access object operator info Sort 10000.00 root executor__merge_join.t1.c1 └─Shuffle 10000.00 root execution info: concurrency:4, data sources:[TableReader TableReader] - └─MergeJoin 10000.00 root left outer join, left key:executor__merge_join.t.c1, right key:executor__merge_join.t1.c1, left cond:[ne(executor__merge_join.t.c1, 1)] + └─MergeJoin 10000.00 root left outer join, left key:executor__merge_join.t.c1, right key:executor__merge_join.t1.c1, left cond:ne(executor__merge_join.t.c1, 1) ├─Sort(Build) 6656.67 root executor__merge_join.t1.c1 │ └─ShuffleReceiver 6656.67 root │ └─TableReader 6656.67 root data:Selection