session: migrate to the new redact function (#51561)
close pingcap/tidb#51557
This commit is contained in:
@ -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",
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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),
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
}},
|
||||
|
||||
17
pkg/util/redact/BUILD.bazel
Normal file
17
pkg/util/redact/BUILD.bazel
Normal file
@ -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"],
|
||||
)
|
||||
43
pkg/util/redact/redact.go
Normal file
43
pkg/util/redact/redact.go
Normal file
@ -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 ""
|
||||
}
|
||||
}
|
||||
37
pkg/util/redact/redact_test.go
Normal file
37
pkg/util/redact/redact_test.go
Normal file
@ -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))
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user