Skip to content

Commit

Permalink
session: refine error message desensitization (pingcap#19409)
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored Sep 15, 2020
1 parent fa6baa9 commit 70a567e
Show file tree
Hide file tree
Showing 12 changed files with 113 additions and 26 deletions.
22 changes: 22 additions & 0 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,8 @@ const (
DefStatusHost = "0.0.0.0"
// DefStoreLivenessTimeout is the default value for store liveness timeout.
DefStoreLivenessTimeout = "5s"
// DefTiDBRedactLog is the default value for redact log.
DefTiDBRedactLog = 0
)

// Valid config maps
Expand Down Expand Up @@ -158,6 +160,8 @@ type Config struct {
EnableGlobalIndex bool `toml:"enable-global-index" json:"enable-global-index"`
// DeprecateIntegerDisplayWidth indicates whether deprecating the max display length for integer.
DeprecateIntegerDisplayWidth bool `toml:"deprecate-integer-display-length" json:"deprecate-integer-display-length"`
// EnableRedactLog indicates that whether redact log, 0 is disable. 1 is enable.
EnableRedactLog int32 `toml:"enable-redact-log" json:"enable-redact-log"`
}

// UpdateTempStoragePath is to update the `TempStoragePath` if port/statusPort was changed
Expand Down Expand Up @@ -736,6 +740,7 @@ var defaultConf = Config{
SpilledFileEncryptionMethod: SpilledFileEncryptionMethodPlaintext,
},
DeprecateIntegerDisplayWidth: false,
EnableRedactLog: DefTiDBRedactLog,
}

var (
Expand Down Expand Up @@ -978,6 +983,23 @@ var TableLockDelayClean = func() uint64 {
return GetGlobalConfig().DelayCleanTableLock
}

// RedactLogEnabled uses to check whether enabled the log redact.
func RedactLogEnabled() bool {
return atomic.LoadInt32(&GetGlobalConfig().EnableRedactLog) == 1
}

// SetRedactLog uses to set log redact status.
func SetRedactLog(enable bool) {
value := int32(0)
if enable {
value = 1
}
g := GetGlobalConfig()
newConf := *g
newConf.EnableRedactLog = value
StoreGlobalConfig(&newConf)
}

// ToLogConfig converts *Log to *logutil.LogConfig.
func (l *Log) ToLogConfig() *logutil.LogConfig {
return logutil.NewLogConfig(l.Level, l.Format, l.SlowQueryFile, l.File, l.getDisableTimestamp(), func(config *zaplog.Config) { config.DisableErrorVerbose = l.getDisableErrorStack() })
Expand Down
9 changes: 4 additions & 5 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -808,7 +808,7 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, succ bool, hasMoreResults boo
a.LogSlowQuery(txnTS, succ, hasMoreResults)
a.SummaryStmt(succ)
prevStmt := a.GetTextToLog()
if sessVars.EnableLogDesensitization {
if config.RedactLogEnabled() {
sessVars.PrevStmt = FormatSQL(prevStmt, nil)
} else {
pps := types.CloneRow(sessVars.PreparedParams)
Expand Down Expand Up @@ -852,7 +852,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) {
}
var sql stringutil.StringerFunc
normalizedSQL, digest := sessVars.StmtCtx.SQLDigest()
if sessVars.EnableLogDesensitization {
if config.RedactLogEnabled() {
sql = FormatSQL(normalizedSQL, nil)
} else if sensitiveStmt, ok := a.StmtNode.(ast.SensitiveStmtNode); ok {
sql = FormatSQL(sensitiveStmt.SecureText(), nil)
Expand Down Expand Up @@ -1054,9 +1054,8 @@ func (a *ExecStmt) SummaryStmt(succ bool) {
// GetTextToLog return the query text to log.
func (a *ExecStmt) GetTextToLog() string {
var sql string
sessVars := a.Ctx.GetSessionVars()
if sessVars.EnableLogDesensitization {
sql, _ = sessVars.StmtCtx.SQLDigest()
if config.RedactLogEnabled() {
sql, _ = a.Ctx.GetSessionVars().StmtCtx.SQLDigest()
} else if sensitiveStmt, ok := a.StmtNode.(ast.SensitiveStmtNode); ok {
sql = sensitiveStmt.SecureText()
} else {
Expand Down
8 changes: 6 additions & 2 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6283,10 +6283,14 @@ func (s *testSerialSuite) TestKillTableReader(c *C) {
wg.Wait()
}

func (s *testSuite) TestPrevStmtDesensitization(c *C) {
func (s *testSerialSuite) TestPrevStmtDesensitization(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test;")
tk.Se.GetSessionVars().EnableLogDesensitization = true
oriCfg := config.GetGlobalConfig()
defer config.StoreGlobalConfig(oriCfg)
newCfg := *oriCfg
newCfg.EnableRedactLog = 1
config.StoreGlobalConfig(&newCfg)
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int)")
tk.MustExec("begin")
Expand Down
3 changes: 2 additions & 1 deletion kv/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (

"github.com/pingcap/parser/terror"
mysql "github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/util/redact"
)

// TxnRetryableMark is used to uniform the commit error messages which could retry the transaction.
Expand All @@ -41,7 +42,7 @@ var (
// ErrEntryTooLarge is the error when a key value entry is too large.
ErrEntryTooLarge = terror.ClassKV.New(mysql.ErrEntryTooLarge, mysql.MySQLErrName[mysql.ErrEntryTooLarge])
// ErrKeyExists returns when key is already exist.
ErrKeyExists = terror.ClassKV.New(mysql.ErrDupEntry, mysql.MySQLErrName[mysql.ErrDupEntry])
ErrKeyExists = redact.NewRedactError(terror.ClassKV.New(mysql.ErrDupEntry, mysql.MySQLErrName[mysql.ErrDupEntry]), 0, 1)
// ErrNotImplemented returns when a function is not implemented yet.
ErrNotImplemented = terror.ClassKV.New(mysql.ErrNotImplemented, mysql.MySQLErrName[mysql.ErrNotImplemented])
// ErrWriteConflict is the error when the commit meets an write conflict error.
Expand Down
1 change: 0 additions & 1 deletion kv/error_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@ func (s testErrorSuite) TestError(c *C) {
ErrInvalidTxn,
ErrTxnTooLarge,
ErrEntryTooLarge,
ErrKeyExists,
ErrNotImplemented,
ErrWriteConflict,
ErrWriteConflictInTiDB,
Expand Down
2 changes: 1 addition & 1 deletion server/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -1929,7 +1929,7 @@ func (cc getLastStmtInConn) String() string {
return "ListFields " + string(data)
case mysql.ComQuery, mysql.ComStmtPrepare:
sql := string(hack.String(data))
if cc.ctx.GetSessionVars().EnableLogDesensitization {
if config.RedactLogEnabled() {
sql, _ = parser.NormalizeDigest(sql)
}
return queryStrForLog(sql)
Expand Down
3 changes: 2 additions & 1 deletion server/conn_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/config"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -639,7 +640,7 @@ func (cc *clientConn) preparedStmt2String(stmtID uint32) string {
if sv == nil {
return ""
}
if sv.EnableLogDesensitization {
if config.RedactLogEnabled() {
return cc.preparedStmt2StringNoArgs(stmtID)
}
return cc.preparedStmt2StringNoArgs(stmtID) + sv.PreparedParams.String()
Expand Down
7 changes: 3 additions & 4 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -496,7 +496,6 @@ func (s *session) doCommitWithRetry(ctx context.Context) error {
zap.Int64("tidb_retry_limit", s.sessionVars.RetryLimit),
zap.Bool("tidb_disable_txn_auto_retry", s.sessionVars.DisableTxnAutoRetry))
}

}
counter := s.sessionVars.TxnCtx.StatementCount
duration := time.Since(s.GetSessionVars().TxnCtx.CreateTime).Seconds()
Expand Down Expand Up @@ -673,7 +672,7 @@ 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())
if !sessVars.EnableLogDesensitization {
if !config.RedactLogEnabled() {
sql += sessVars.PreparedParams.String()
}
logutil.Logger(ctx).Warn("retrying",
Expand Down Expand Up @@ -2099,7 +2098,7 @@ var builtinGlobalVariable = []string{
variable.TiDBEnableClusteredIndex,
variable.TiDBPartitionPruneMode,
variable.TiDBSlowLogMasking,
variable.TiDBLogDesensitization,
variable.TiDBRedactLog,
variable.TiDBEnableTelemetry,
variable.TiDBShardAllocateStep,
variable.TiDBEnableChangeColumnType,
Expand Down Expand Up @@ -2283,7 +2282,7 @@ func logStmt(execStmt *executor.ExecStmt, vars *variable.SessionVars) {
func logQuery(query string, vars *variable.SessionVars) {
if atomic.LoadUint32(&variable.ProcessGeneralLog) != 0 && !vars.InRestrictedSQL {
query = executor.QueryReplacer.Replace(query)
if !vars.EnableLogDesensitization {
if !config.RedactLogEnabled() {
query = query + vars.PreparedParams.String()
}
logutil.BgLogger().Info("GENERAL_LOG",
Expand Down
8 changes: 2 additions & 6 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -696,9 +696,6 @@ type SessionVars struct {
// EnableClusteredIndex indicates whether to enable clustered index when creating a new table.
EnableClusteredIndex bool

// EnableLogDesensitization indicates that whether desensitization when log query.
EnableLogDesensitization bool

// PresumeKeyNotExists indicates lazy existence checking is enabled.
PresumeKeyNotExists bool

Expand Down Expand Up @@ -836,7 +833,6 @@ func NewSessionVars() *SessionVars {
AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert,
EnableClusteredIndex: DefTiDBEnableClusteredIndex,
EnableParallelApply: DefTiDBEnableParallelApply,
EnableLogDesensitization: DefTiDBLogDesensitization,
ShardAllocateStep: DefTiDBShardAllocateStep,
EnableChangeColumnType: DefTiDBChangeColumnType,
EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn,
Expand Down Expand Up @@ -1459,8 +1455,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error {
s.PartitionPruneMode = PartitionPruneMode(strings.ToLower(strings.TrimSpace(val)))
case TiDBEnableParallelApply:
s.EnableParallelApply = TiDBOptOn(val)
case TiDBSlowLogMasking, TiDBLogDesensitization:
s.EnableLogDesensitization = TiDBOptOn(val)
case TiDBSlowLogMasking, TiDBRedactLog:
config.SetRedactLog(TiDBOptOn(val))
case TiDBShardAllocateStep:
s.ShardAllocateStep = tidbOptInt64(val, DefTiDBShardAllocateStep)
case TiDBEnableChangeColumnType:
Expand Down
2 changes: 1 addition & 1 deletion sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -699,7 +699,7 @@ var defaultSysVars = []*SysVar{
{ScopeGlobal | ScopeSession, TiDBEnableClusteredIndex, BoolToIntStr(DefTiDBEnableClusteredIndex)},
{ScopeGlobal | ScopeSession, TiDBPartitionPruneMode, string(StaticOnly)},
{ScopeGlobal, TiDBSlowLogMasking, BoolToIntStr(DefTiDBSlowLogMasking)},
{ScopeGlobal, TiDBLogDesensitization, BoolToIntStr(DefTiDBLogDesensitization)},
{ScopeGlobal, TiDBRedactLog, strconv.Itoa(config.DefTiDBRedactLog)},
{ScopeGlobal | ScopeSession, TiDBShardAllocateStep, strconv.Itoa(DefTiDBShardAllocateStep)},
{ScopeGlobal, TiDBEnableTelemetry, BoolToIntStr(DefTiDBEnableTelemetry)},
{ScopeGlobal | ScopeSession, TiDBEnableAmendPessimisticTxn, boolToOnOff(DefTiDBEnableAmendPessimisticTxn)},
Expand Down
7 changes: 3 additions & 4 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -434,11 +434,11 @@ const (
TiDBPartitionPruneMode = "tidb_partition_prune_mode"

// TiDBSlowLogMasking indicates that whether masking the query data when log slow query.
// Deprecated: use TiDBLogDesensitization instead.
// Deprecated: use TiDBRedactLog instead.
TiDBSlowLogMasking = "tidb_slow_log_masking"

// TiDBLogDesensitization indicates that whether desensitization when log query.
TiDBLogDesensitization = "tidb_log_desensitization"
// TiDBRedactLog indicates that whether redact log.
TiDBRedactLog = "tidb_redact_log"

// TiDBShardAllocateStep indicates the max size of continuous rowid shard in one transaction.
TiDBShardAllocateStep = "tidb_shard_allocate_step"
Expand Down Expand Up @@ -548,7 +548,6 @@ const (
DefTiDBAllowAutoRandExplicitInsert = false
DefTiDBEnableClusteredIndex = false
DefTiDBSlowLogMasking = false
DefTiDBLogDesensitization = false
DefTiDBShardAllocateStep = math.MaxInt64
DefTiDBEnableTelemetry = true
DefTiDBEnableParallelApply = false
Expand Down
67 changes: 67 additions & 0 deletions util/redact/redact.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
// Copyright 2020 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,
// See the License for the specific language governing permissions and
// limitations under the License.

package redact

import (
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/config"
)

// TError is a alias, use to avoid `Error` method name in conflict with field name.
type TError = terror.Error

type redactError struct {
*TError
redactPositions []int
}

// GenWithStackByArgs generates a new *Error with the same class and code, and new arguments.
func (e *redactError) GenWithStackByArgs(args ...interface{}) error {
redactErrorArg(args, e.redactPositions)
return e.TError.GenWithStackByArgs(args...)
}

// FastGen generates a new *Error with the same class and code, and a new arguments.
func (e *redactError) FastGenByArgs(args ...interface{}) error {
redactErrorArg(args, e.redactPositions)
return e.TError.GenWithStackByArgs(args...)
}

// Equal checks if err is equal to e.
func (e *redactError) Equal(err error) bool {
if redactErr, ok := err.(*redactError); ok {
return e.TError.Equal(redactErr.TError)
}
return e.TError.Equal(err)
}

// Cause implement the Cause interface.
func (e *redactError) Cause() error {
return e.TError
}

func redactErrorArg(args []interface{}, position []int) {
if config.RedactLogEnabled() {
for _, pos := range position {
if len(args) > pos {
args[pos] = "?"
}
}
}
}

// NewRedactError returns a new redact error.
func NewRedactError(err *terror.Error, redactPositions ...int) *redactError {
return &redactError{err, redactPositions}
}

0 comments on commit 70a567e

Please sign in to comment.