Skip to content

Commit 38cc7be

Browse files
authored
session: migrate to the new redact function (#51561)
close #51557
1 parent b59ea4a commit 38cc7be

File tree

10 files changed

+126
-29
lines changed

10 files changed

+126
-29
lines changed

pkg/server/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -91,6 +91,7 @@ go_library(
9191
"//pkg/util/logutil",
9292
"//pkg/util/memory",
9393
"//pkg/util/printer",
94+
"//pkg/util/redact",
9495
"//pkg/util/resourcegrouptag",
9596
"//pkg/util/sqlexec",
9697
"//pkg/util/sqlkiller",

pkg/server/conn.go

Lines changed: 9 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -102,6 +102,7 @@ import (
102102
"github.com/pingcap/tidb/pkg/util/hack"
103103
"github.com/pingcap/tidb/pkg/util/intest"
104104
"github.com/pingcap/tidb/pkg/util/logutil"
105+
"github.com/pingcap/tidb/pkg/util/redact"
105106
"github.com/pingcap/tidb/pkg/util/resourcegrouptag"
106107
tlsutil "github.com/pingcap/tidb/pkg/util/tls"
107108
"github.com/pingcap/tidb/pkg/util/topsql"
@@ -1159,7 +1160,7 @@ func (cc *clientConn) Run(ctx context.Context) {
11591160
zap.Stringer("sql", getLastStmtInConn{cc}),
11601161
zap.String("txn_mode", txnMode),
11611162
zap.Uint64("timestamp", startTS),
1162-
zap.String("err", errStrForLog(err, cc.ctx.GetSessionVars().EnableRedactLog)),
1163+
zap.String("err", errStrForLog(err, cc.ctx.GetSessionVars().EnableRedactNew)),
11631164
)
11641165
}
11651166
err1 := cc.writeError(ctx, err)
@@ -1171,19 +1172,22 @@ func (cc *clientConn) Run(ctx context.Context) {
11711172
}
11721173
}
11731174

1174-
func errStrForLog(err error, enableRedactLog bool) string {
1175-
if enableRedactLog {
1175+
func errStrForLog(err error, redactMode string) string {
1176+
if redactMode == "ON" {
11761177
// currently, only ErrParse is considered when enableRedactLog because it may contain sensitive information like
11771178
// password or accesskey
11781179
if parser.ErrParse.Equal(err) {
11791180
return "fail to parse SQL and can't redact when enable log redaction"
11801181
}
11811182
}
1183+
var ret string
11821184
if kv.ErrKeyExists.Equal(err) || parser.ErrParse.Equal(err) || infoschema.ErrTableNotExists.Equal(err) {
11831185
// Do not log stack for duplicated entry error.
1184-
return err.Error()
1186+
ret = err.Error()
1187+
} else {
1188+
ret = errors.ErrorStack(err)
11851189
}
1186-
return errors.ErrorStack(err)
1190+
return redact.Redact(redactMode, ret)
11871191
}
11881192

11891193
func (cc *clientConn) addMetrics(cmd byte, startTime time.Time, err error) {

pkg/session/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -104,6 +104,7 @@ go_library(
104104
"//pkg/util/logutil/consistency",
105105
"//pkg/util/memory",
106106
"//pkg/util/parser",
107+
"//pkg/util/redact",
107108
"//pkg/util/sem",
108109
"//pkg/util/sli",
109110
"//pkg/util/sqlescape",

pkg/session/nontransactional.go

Lines changed: 8 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,7 @@ import (
4242
"github.com/pingcap/tidb/pkg/util/dbterror"
4343
"github.com/pingcap/tidb/pkg/util/logutil"
4444
"github.com/pingcap/tidb/pkg/util/memory"
45+
"github.com/pingcap/tidb/pkg/util/redact"
4546
"github.com/pingcap/tidb/pkg/util/sqlexec"
4647
"go.uber.org/zap"
4748
)
@@ -67,11 +68,8 @@ type statementBuildInfo struct {
6768
originalCondition ast.ExprNode
6869
}
6970

70-
func (j job) String(redacted bool) string {
71-
if redacted {
72-
return fmt.Sprintf("job id: %d, estimated size: %d", j.jobID, j.jobSize)
73-
}
74-
return fmt.Sprintf("job id: %d, estimated size: %d, sql: %s", j.jobID, j.jobSize, j.sql)
71+
func (j job) String(redacted string) string {
72+
return fmt.Sprintf("job id: %d, estimated size: %d, sql: %s", j.jobID, j.jobSize, redact.Redact(redacted, j.sql))
7573
}
7674

7775
// HandleNonTransactionalDML is the entry point for a non-transactional DML statement
@@ -122,7 +120,7 @@ func HandleNonTransactionalDML(ctx context.Context, stmt *ast.NonTransactionalDM
122120
if stmt.DryRun == ast.DryRunSplitDml {
123121
return buildDryRunResults(stmt.DryRun, splitStmts, se.GetSessionVars().BatchSize.MaxChunkSize)
124122
}
125-
return buildExecuteResults(ctx, jobs, se.GetSessionVars().BatchSize.MaxChunkSize, se.GetSessionVars().EnableRedactLog)
123+
return buildExecuteResults(ctx, jobs, se.GetSessionVars().BatchSize.MaxChunkSize, se.GetSessionVars().EnableRedactNew)
126124
}
127125

128126
// we require:
@@ -282,7 +280,7 @@ func runJobs(ctx context.Context, jobs []job, stmt *ast.NonTransactionalDMLStmt,
282280
failedJobs := make([]string, 0)
283281
for _, job := range jobs {
284282
if job.err != nil {
285-
failedJobs = append(failedJobs, fmt.Sprintf("job:%s, error: %s", job.String(se.GetSessionVars().EnableRedactLog), job.err.Error()))
283+
failedJobs = append(failedJobs, fmt.Sprintf("job:%s, error: %s", job.String(se.GetSessionVars().EnableRedactNew), job.err.Error()))
286284
}
287285
}
288286
if len(failedJobs) == 0 {
@@ -326,7 +324,7 @@ func runJobs(ctx context.Context, jobs []job, stmt *ast.NonTransactionalDMLStmt,
326324
return nil, errors.Annotate(jobs[i].err, "Early return: error occurred in the first job. All jobs are canceled")
327325
}
328326
if jobs[i].err != nil && !se.GetSessionVars().NonTransactionalIgnoreError {
329-
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())
327+
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())
330328
}
331329
}
332330
return splitStmts, nil
@@ -412,7 +410,7 @@ func doOneJob(ctx context.Context, job *job, totalJobCount int, options statemen
412410

413411
job.sql = dmlSQL
414412
logutil.Logger(ctx).Info("start a Non-transactional DML",
415-
zap.String("job", job.String(se.GetSessionVars().EnableRedactLog)), zap.Int("totalJobCount", totalJobCount))
413+
zap.String("job", job.String(se.GetSessionVars().EnableRedactNew)), zap.Int("totalJobCount", totalJobCount))
416414
var dmlSQLInLog string
417415
if se.GetSessionVars().EnableRedactLog {
418416
dmlSQLInLog = parser.Normalize(dmlSQL)
@@ -804,7 +802,7 @@ func buildDryRunResults(dryRunOption int, results []string, maxChunkSize int) (s
804802
}, nil
805803
}
806804

807-
func buildExecuteResults(ctx context.Context, jobs []job, maxChunkSize int, redactLog bool) (sqlexec.RecordSet, error) {
805+
func buildExecuteResults(ctx context.Context, jobs []job, maxChunkSize int, redactLog string) (sqlexec.RecordSet, error) {
808806
failedJobs := make([]job, 0)
809807
for _, job := range jobs {
810808
if job.err != nil {

pkg/session/session.go

Lines changed: 7 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -108,6 +108,7 @@ import (
108108
"github.com/pingcap/tidb/pkg/util/logutil"
109109
"github.com/pingcap/tidb/pkg/util/logutil/consistency"
110110
"github.com/pingcap/tidb/pkg/util/memory"
111+
"github.com/pingcap/tidb/pkg/util/redact"
111112
"github.com/pingcap/tidb/pkg/util/sem"
112113
"github.com/pingcap/tidb/pkg/util/sli"
113114
"github.com/pingcap/tidb/pkg/util/sqlescape"
@@ -1171,8 +1172,8 @@ func (s *session) retry(ctx context.Context, maxCnt uint) (err error) {
11711172
// We do not have to log the query every time.
11721173
// We print the queries at the first try only.
11731174
sql := sqlForLog(st.GetTextToLog(false))
1174-
if !sessVars.EnableRedactLog {
1175-
sql += sessVars.PlanCacheParams.String()
1175+
if sessVars.EnableRedactNew != "ON" {
1176+
sql += redact.Redact(sessVars.EnableRedactNew, sessVars.PlanCacheParams.String())
11761177
}
11771178
logutil.Logger(ctx).Warn("retrying",
11781179
zap.Int64("schemaVersion", schemaVersion),
@@ -1667,11 +1668,7 @@ func (s *session) Parse(ctx context.Context, sql string) ([]ast.StmtNode, error)
16671668
// Only print log message when this SQL is from the user.
16681669
// Mute the warning for internal SQLs.
16691670
if !s.sessionVars.InRestrictedSQL {
1670-
if s.sessionVars.EnableRedactLog {
1671-
logutil.Logger(ctx).Debug("parse SQL failed", zap.Error(err), zap.String("SQL", sql))
1672-
} else {
1673-
logutil.Logger(ctx).Warn("parse SQL failed", zap.Error(err), zap.String("SQL", sql))
1674-
}
1671+
logutil.Logger(ctx).Warn("parse SQL failed", zap.Error(err), zap.String("SQL", redact.Redact(s.sessionVars.EnableRedactNew, sql)))
16751672
s.sessionVars.StmtCtx.AppendError(err)
16761673
}
16771674
return nil, err
@@ -1721,11 +1718,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...any)
17211718
if err != nil {
17221719
s.rollbackOnError(ctx)
17231720
logSQL := sql[:min(500, len(sql))]
1724-
if s.sessionVars.EnableRedactLog {
1725-
logutil.Logger(ctx).Debug("parse SQL failed", zap.Error(err), zap.String("SQL", logSQL))
1726-
} else {
1727-
logutil.Logger(ctx).Warn("parse SQL failed", zap.Error(err), zap.String("SQL", logSQL))
1728-
}
1721+
logutil.Logger(ctx).Warn("parse SQL failed", zap.Error(err), zap.String("SQL", redact.Redact(s.sessionVars.EnableRedactNew, logSQL)))
17291722
return nil, util.SyntaxError(err)
17301723
}
17311724
durParse := time.Since(parseStartTime)
@@ -3941,8 +3934,8 @@ func logGeneralQuery(execStmt *executor.ExecStmt, s *session, isPrepared bool) {
39413934
}
39423935

39433936
query = executor.QueryReplacer.Replace(query)
3944-
if !vars.EnableRedactLog {
3945-
query += vars.PlanCacheParams.String()
3937+
if vars.EnableRedactNew != "ON" {
3938+
query += redact.Redact(vars.EnableRedactNew, vars.PlanCacheParams.String())
39463939
}
39473940
logutil.BgLogger().Info("GENERAL_LOG",
39483941
zap.Uint64("conn", vars.ConnectionID),

pkg/sessionctx/variable/session.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1204,6 +1204,8 @@ type SessionVars struct {
12041204

12051205
// EnableRedactLog indicates that whether redact log.
12061206
EnableRedactLog bool
1207+
// EnableRedactNew indicates that whether redact log.
1208+
EnableRedactNew string
12071209

12081210
// ShardAllocateStep indicates the max size of continuous rowid shard in one transaction.
12091211
ShardAllocateStep int64

pkg/sessionctx/variable/sysvar.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2153,6 +2153,7 @@ var defaultSysVars = []*SysVar{
21532153
}},
21542154
{Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: DefTiDBRedactLog, Type: TypeEnum, PossibleValues: []string{Off, On, Marker}, SetSession: func(s *SessionVars, val string) error {
21552155
s.EnableRedactLog = val != Off
2156+
s.EnableRedactNew = val
21562157
errors.RedactLogEnabled.Store(s.EnableRedactLog)
21572158
return nil
21582159
}},

pkg/util/redact/BUILD.bazel

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,17 @@
1+
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
2+
3+
go_library(
4+
name = "redact",
5+
srcs = ["redact.go"],
6+
importpath = "github.com/pingcap/tidb/pkg/util/redact",
7+
visibility = ["//visibility:public"],
8+
)
9+
10+
go_test(
11+
name = "redact_test",
12+
timeout = "short",
13+
srcs = ["redact_test.go"],
14+
embed = [":redact"],
15+
flaky = True,
16+
deps = ["@com_github_stretchr_testify//require"],
17+
)

pkg/util/redact/redact.go

Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,43 @@
1+
// Copyright 2024 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package redact
16+
17+
import (
18+
"strings"
19+
)
20+
21+
// 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.
22+
func Redact(mode string, input string) string {
23+
switch mode {
24+
case "MARKER":
25+
b := &strings.Builder{}
26+
b.Grow(len(input))
27+
_, _ = b.WriteRune('‹')
28+
for _, c := range input {
29+
if c == '‹' || c == '›' {
30+
_, _ = b.WriteRune(c)
31+
_, _ = b.WriteRune(c)
32+
} else {
33+
_, _ = b.WriteRune(c)
34+
}
35+
}
36+
_, _ = b.WriteRune('›')
37+
return b.String()
38+
case "OFF":
39+
return input
40+
default:
41+
return ""
42+
}
43+
}

pkg/util/redact/redact_test.go

Lines changed: 37 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,37 @@
1+
// Copyright 2024 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package redact
16+
17+
import (
18+
"testing"
19+
20+
"github.com/stretchr/testify/require"
21+
)
22+
23+
func TestRedact(t *testing.T) {
24+
for _, c := range []struct {
25+
mode string
26+
input string
27+
output string
28+
}{
29+
{"OFF", "fxcv", "fxcv"},
30+
{"OFF", "f‹xcv", "f‹xcv"},
31+
{"ON", "f‹xcv", ""},
32+
{"MARKER", "f‹xcv", "‹f‹‹xcv›"},
33+
{"MARKER", "f›xcv", "‹f››xcv›"},
34+
} {
35+
require.Equal(t, c.output, Redact(c.mode, c.input))
36+
}
37+
}

0 commit comments

Comments
 (0)