diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 75c5ae8a83..ef63c250b2 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -91,6 +91,7 @@ go_library( "//pkg/util/logutil", "//pkg/util/memory", "//pkg/util/printer", + "//pkg/util/redact", "//pkg/util/resourcegrouptag", "//pkg/util/sqlexec", "//pkg/util/sqlkiller", diff --git a/pkg/server/conn.go b/pkg/server/conn.go index 21b6837890..3f2713df58 100644 --- a/pkg/server/conn.go +++ b/pkg/server/conn.go @@ -102,6 +102,7 @@ import ( "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tidb/pkg/util/intest" "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/pingcap/tidb/pkg/util/redact" "github.com/pingcap/tidb/pkg/util/resourcegrouptag" tlsutil "github.com/pingcap/tidb/pkg/util/tls" "github.com/pingcap/tidb/pkg/util/topsql" @@ -1159,7 +1160,7 @@ func (cc *clientConn) Run(ctx context.Context) { zap.Stringer("sql", getLastStmtInConn{cc}), zap.String("txn_mode", txnMode), zap.Uint64("timestamp", startTS), - zap.String("err", errStrForLog(err, cc.ctx.GetSessionVars().EnableRedactLog)), + zap.String("err", errStrForLog(err, cc.ctx.GetSessionVars().EnableRedactNew)), ) } err1 := cc.writeError(ctx, err) @@ -1171,19 +1172,22 @@ func (cc *clientConn) Run(ctx context.Context) { } } -func errStrForLog(err error, enableRedactLog bool) string { - if enableRedactLog { +func errStrForLog(err error, redactMode string) string { + if redactMode == "ON" { // currently, only ErrParse is considered when enableRedactLog because it may contain sensitive information like // password or accesskey if parser.ErrParse.Equal(err) { return "fail to parse SQL and can't redact when enable log redaction" } } + var ret string if kv.ErrKeyExists.Equal(err) || parser.ErrParse.Equal(err) || infoschema.ErrTableNotExists.Equal(err) { // Do not log stack for duplicated entry error. - return err.Error() + ret = err.Error() + } else { + ret = errors.ErrorStack(err) } - return errors.ErrorStack(err) + return redact.Redact(redactMode, ret) } func (cc *clientConn) addMetrics(cmd byte, startTime time.Time, err error) { diff --git a/pkg/session/BUILD.bazel b/pkg/session/BUILD.bazel index f10b6d933e..efc94489c1 100644 --- a/pkg/session/BUILD.bazel +++ b/pkg/session/BUILD.bazel @@ -104,6 +104,7 @@ go_library( "//pkg/util/logutil/consistency", "//pkg/util/memory", "//pkg/util/parser", + "//pkg/util/redact", "//pkg/util/sem", "//pkg/util/sli", "//pkg/util/sqlescape", diff --git a/pkg/session/nontransactional.go b/pkg/session/nontransactional.go index 59a81e6aa7..69f084d3bd 100644 --- a/pkg/session/nontransactional.go +++ b/pkg/session/nontransactional.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/pkg/util/dbterror" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/memory" + "github.com/pingcap/tidb/pkg/util/redact" "github.com/pingcap/tidb/pkg/util/sqlexec" "go.uber.org/zap" ) @@ -67,11 +68,8 @@ type statementBuildInfo struct { originalCondition ast.ExprNode } -func (j job) String(redacted bool) string { - if redacted { - return fmt.Sprintf("job id: %d, estimated size: %d", j.jobID, j.jobSize) - } - return fmt.Sprintf("job id: %d, estimated size: %d, sql: %s", j.jobID, j.jobSize, j.sql) +func (j job) String(redacted string) string { + return fmt.Sprintf("job id: %d, estimated size: %d, sql: %s", j.jobID, j.jobSize, redact.Redact(redacted, j.sql)) } // HandleNonTransactionalDML is the entry point for a non-transactional DML statement @@ -122,7 +120,7 @@ func HandleNonTransactionalDML(ctx context.Context, stmt *ast.NonTransactionalDM if stmt.DryRun == ast.DryRunSplitDml { return buildDryRunResults(stmt.DryRun, splitStmts, se.GetSessionVars().BatchSize.MaxChunkSize) } - return buildExecuteResults(ctx, jobs, se.GetSessionVars().BatchSize.MaxChunkSize, se.GetSessionVars().EnableRedactLog) + return buildExecuteResults(ctx, jobs, se.GetSessionVars().BatchSize.MaxChunkSize, se.GetSessionVars().EnableRedactNew) } // we require: @@ -282,7 +280,7 @@ func runJobs(ctx context.Context, jobs []job, stmt *ast.NonTransactionalDMLStmt, failedJobs := make([]string, 0) for _, job := range jobs { if job.err != nil { - failedJobs = append(failedJobs, fmt.Sprintf("job:%s, error: %s", job.String(se.GetSessionVars().EnableRedactLog), job.err.Error())) + failedJobs = append(failedJobs, fmt.Sprintf("job:%s, error: %s", job.String(se.GetSessionVars().EnableRedactNew), job.err.Error())) } } if len(failedJobs) == 0 { @@ -326,7 +324,7 @@ func runJobs(ctx context.Context, jobs []job, stmt *ast.NonTransactionalDMLStmt, return nil, errors.Annotate(jobs[i].err, "Early return: error occurred in the first job. All jobs are canceled") } if jobs[i].err != nil && !se.GetSessionVars().NonTransactionalIgnoreError { - return nil, ErrNonTransactionalJobFailure.GenWithStackByArgs(jobs[i].jobID, len(jobs), jobs[i].start.String(), jobs[i].end.String(), jobs[i].String(se.GetSessionVars().EnableRedactLog), jobs[i].err.Error()) + return nil, ErrNonTransactionalJobFailure.GenWithStackByArgs(jobs[i].jobID, len(jobs), jobs[i].start.String(), jobs[i].end.String(), jobs[i].String(se.GetSessionVars().EnableRedactNew), jobs[i].err.Error()) } } return splitStmts, nil @@ -412,7 +410,7 @@ func doOneJob(ctx context.Context, job *job, totalJobCount int, options statemen job.sql = dmlSQL logutil.Logger(ctx).Info("start a Non-transactional DML", - zap.String("job", job.String(se.GetSessionVars().EnableRedactLog)), zap.Int("totalJobCount", totalJobCount)) + zap.String("job", job.String(se.GetSessionVars().EnableRedactNew)), zap.Int("totalJobCount", totalJobCount)) var dmlSQLInLog string if se.GetSessionVars().EnableRedactLog { dmlSQLInLog = parser.Normalize(dmlSQL) @@ -804,7 +802,7 @@ func buildDryRunResults(dryRunOption int, results []string, maxChunkSize int) (s }, nil } -func buildExecuteResults(ctx context.Context, jobs []job, maxChunkSize int, redactLog bool) (sqlexec.RecordSet, error) { +func buildExecuteResults(ctx context.Context, jobs []job, maxChunkSize int, redactLog string) (sqlexec.RecordSet, error) { failedJobs := make([]job, 0) for _, job := range jobs { if job.err != nil { diff --git a/pkg/session/session.go b/pkg/session/session.go index 1738952114..573af76c2f 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -108,6 +108,7 @@ import ( "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/logutil/consistency" "github.com/pingcap/tidb/pkg/util/memory" + "github.com/pingcap/tidb/pkg/util/redact" "github.com/pingcap/tidb/pkg/util/sem" "github.com/pingcap/tidb/pkg/util/sli" "github.com/pingcap/tidb/pkg/util/sqlescape" @@ -1171,8 +1172,8 @@ func (s *session) retry(ctx context.Context, maxCnt uint) (err error) { // We do not have to log the query every time. // We print the queries at the first try only. sql := sqlForLog(st.GetTextToLog(false)) - if !sessVars.EnableRedactLog { - sql += sessVars.PlanCacheParams.String() + if sessVars.EnableRedactNew != "ON" { + sql += redact.Redact(sessVars.EnableRedactNew, sessVars.PlanCacheParams.String()) } logutil.Logger(ctx).Warn("retrying", zap.Int64("schemaVersion", schemaVersion), @@ -1667,11 +1668,7 @@ func (s *session) Parse(ctx context.Context, sql string) ([]ast.StmtNode, error) // Only print log message when this SQL is from the user. // Mute the warning for internal SQLs. if !s.sessionVars.InRestrictedSQL { - if s.sessionVars.EnableRedactLog { - logutil.Logger(ctx).Debug("parse SQL failed", zap.Error(err), zap.String("SQL", sql)) - } else { - logutil.Logger(ctx).Warn("parse SQL failed", zap.Error(err), zap.String("SQL", sql)) - } + logutil.Logger(ctx).Warn("parse SQL failed", zap.Error(err), zap.String("SQL", redact.Redact(s.sessionVars.EnableRedactNew, sql))) s.sessionVars.StmtCtx.AppendError(err) } return nil, err @@ -1721,11 +1718,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...any) if err != nil { s.rollbackOnError(ctx) logSQL := sql[:min(500, len(sql))] - if s.sessionVars.EnableRedactLog { - logutil.Logger(ctx).Debug("parse SQL failed", zap.Error(err), zap.String("SQL", logSQL)) - } else { - logutil.Logger(ctx).Warn("parse SQL failed", zap.Error(err), zap.String("SQL", logSQL)) - } + logutil.Logger(ctx).Warn("parse SQL failed", zap.Error(err), zap.String("SQL", redact.Redact(s.sessionVars.EnableRedactNew, logSQL))) return nil, util.SyntaxError(err) } durParse := time.Since(parseStartTime) @@ -3941,8 +3934,8 @@ func logGeneralQuery(execStmt *executor.ExecStmt, s *session, isPrepared bool) { } query = executor.QueryReplacer.Replace(query) - if !vars.EnableRedactLog { - query += vars.PlanCacheParams.String() + if vars.EnableRedactNew != "ON" { + query += redact.Redact(vars.EnableRedactNew, vars.PlanCacheParams.String()) } logutil.BgLogger().Info("GENERAL_LOG", zap.Uint64("conn", vars.ConnectionID), diff --git a/pkg/sessionctx/variable/session.go b/pkg/sessionctx/variable/session.go index 6939f18819..ac020e6a39 100644 --- a/pkg/sessionctx/variable/session.go +++ b/pkg/sessionctx/variable/session.go @@ -1204,6 +1204,8 @@ type SessionVars struct { // EnableRedactLog indicates that whether redact log. EnableRedactLog bool + // EnableRedactNew indicates that whether redact log. + EnableRedactNew string // ShardAllocateStep indicates the max size of continuous rowid shard in one transaction. ShardAllocateStep int64 diff --git a/pkg/sessionctx/variable/sysvar.go b/pkg/sessionctx/variable/sysvar.go index 6e6b5015ca..bd592284c6 100644 --- a/pkg/sessionctx/variable/sysvar.go +++ b/pkg/sessionctx/variable/sysvar.go @@ -2153,6 +2153,7 @@ var defaultSysVars = []*SysVar{ }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: DefTiDBRedactLog, Type: TypeEnum, PossibleValues: []string{Off, On, Marker}, SetSession: func(s *SessionVars, val string) error { s.EnableRedactLog = val != Off + s.EnableRedactNew = val errors.RedactLogEnabled.Store(s.EnableRedactLog) return nil }}, diff --git a/pkg/util/redact/BUILD.bazel b/pkg/util/redact/BUILD.bazel new file mode 100644 index 0000000000..17e67c298c --- /dev/null +++ b/pkg/util/redact/BUILD.bazel @@ -0,0 +1,17 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "redact", + srcs = ["redact.go"], + importpath = "github.com/pingcap/tidb/pkg/util/redact", + visibility = ["//visibility:public"], +) + +go_test( + name = "redact_test", + timeout = "short", + srcs = ["redact_test.go"], + embed = [":redact"], + flaky = True, + deps = ["@com_github_stretchr_testify//require"], +) diff --git a/pkg/util/redact/redact.go b/pkg/util/redact/redact.go new file mode 100644 index 0000000000..8f42c64e6c --- /dev/null +++ b/pkg/util/redact/redact.go @@ -0,0 +1,43 @@ +// 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 ( + "strings" +) + +// Redact will redact the input string according to 'mode'. Check 'tidb_redact_log': https://github.com/pingcap/tidb/blob/acf9e3128693a5a13f31027f05f4de41edf8d7b2/pkg/sessionctx/variable/sysvar.go#L2154. +func Redact(mode string, input string) string { + switch mode { + case "MARKER": + b := &strings.Builder{} + b.Grow(len(input)) + _, _ = b.WriteRune('‹') + for _, c := range input { + if c == '‹' || c == '›' { + _, _ = b.WriteRune(c) + _, _ = b.WriteRune(c) + } else { + _, _ = b.WriteRune(c) + } + } + _, _ = b.WriteRune('›') + return b.String() + case "OFF": + return input + default: + return "" + } +} diff --git a/pkg/util/redact/redact_test.go b/pkg/util/redact/redact_test.go new file mode 100644 index 0000000000..4214b618fd --- /dev/null +++ b/pkg/util/redact/redact_test.go @@ -0,0 +1,37 @@ +// 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/stretchr/testify/require" +) + +func TestRedact(t *testing.T) { + for _, c := range []struct { + mode string + input string + output string + }{ + {"OFF", "fxcv", "fxcv"}, + {"OFF", "f‹xcv", "f‹xcv"}, + {"ON", "f‹xcv", ""}, + {"MARKER", "f‹xcv", "‹f‹‹xcv›"}, + {"MARKER", "f›xcv", "‹f››xcv›"}, + } { + require.Equal(t, c.output, Redact(c.mode, c.input)) + } +}