Skip to content

Commit

Permalink
*: add warnings to slow log and slow query table (#39884)
Browse files Browse the repository at this point in the history
ref #39893
  • Loading branch information
time-and-fate committed Jan 5, 2023
1 parent 7e64d04 commit 669ba5c
Show file tree
Hide file tree
Showing 10 changed files with 197 additions and 3 deletions.
28 changes: 28 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -1554,6 +1554,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) {
IsWriteCacheTable: stmtCtx.WaitLockLeaseTime > 0,
StatsLoadStatus: convertStatusIntoString(a.Ctx, stmtCtx.StatsLoadStatus),
IsSyncStatsFailed: stmtCtx.IsSyncStatsFailed,
Warnings: collectWarningsForSlowLog(stmtCtx),
}
failpoint.Inject("assertSyncStatsFailed", func(val failpoint.Value) {
if val.(bool) {
Expand Down Expand Up @@ -1609,6 +1610,33 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) {
}
}

func extractMsgFromSQLWarn(SQLWarn *stmtctx.SQLWarn) string {
// Currently, this function is only used in collectWarningsForSlowLog.
// collectWarningsForSlowLog can make sure SQLWarn is not nil so no need to add a nil check here.
warn := errors.Cause(SQLWarn.Err)
if x, ok := warn.(*terror.Error); ok && x != nil {
sqlErr := terror.ToSQLError(x)
return sqlErr.Message
}
return warn.Error()
}

func collectWarningsForSlowLog(stmtCtx *stmtctx.StatementContext) []variable.JSONSQLWarnForSlowLog {
warnings := stmtCtx.GetWarnings()
extraWarnings := stmtCtx.GetExtraWarnings()
res := make([]variable.JSONSQLWarnForSlowLog, len(warnings)+len(extraWarnings))
for i := range warnings {
res[i].Level = warnings[i].Level
res[i].Message = extractMsgFromSQLWarn(&warnings[i])
}
for i := range extraWarnings {
res[len(warnings)+i].Level = extraWarnings[i].Level
res[len(warnings)+i].Message = extractMsgFromSQLWarn(&extraWarnings[i])
res[len(warnings)+i].IsExtra = true
}
return res
}

// GetResultRowsCount gets the count of the statement result rows.
func GetResultRowsCount(stmtCtx *stmtctx.StatementContext, p plannercore.Plan) int64 {
runtimeStatsColl := stmtCtx.RuntimeStatsColl
Expand Down
3 changes: 3 additions & 0 deletions executor/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,17 +33,20 @@ var prepareMergeSuiteData testdata.TestData
var aggMergeSuiteData testdata.TestData
var executorSuiteData testdata.TestData
var pointGetSuiteData testdata.TestData
var slowQuerySuiteData testdata.TestData

func TestMain(m *testing.M) {
testsetup.SetupForCommonTest()
testDataMap.LoadTestSuiteData("testdata", "agg_suite")
testDataMap.LoadTestSuiteData("testdata", "executor_suite")
testDataMap.LoadTestSuiteData("testdata", "prepare_suite")
testDataMap.LoadTestSuiteData("testdata", "point_get_suite")
testDataMap.LoadTestSuiteData("testdata", "slow_query_suite")
aggMergeSuiteData = testDataMap["agg_suite"]
executorSuiteData = testDataMap["executor_suite"]
prepareMergeSuiteData = testDataMap["prepare_suite"]
pointGetSuiteData = testDataMap["point_get_suite"]
slowQuerySuiteData = testDataMap["slow_query_suite"]

autoid.SetStep(5000)
config.UpdateGlobal(func(conf *config.Config) {
Expand Down
5 changes: 4 additions & 1 deletion executor/slow_query.go
Original file line number Diff line number Diff line change
Expand Up @@ -622,6 +622,9 @@ func (e *slowQueryRetriever) parseLog(ctx context.Context, sctx sessionctx.Conte
valid = e.setColumnValue(sctx, row, tz, variable.SlowLogHostStr, host, e.checker, fileLine)
} else if strings.HasPrefix(line, variable.SlowLogCopBackoffPrefix) {
valid = e.setColumnValue(sctx, row, tz, variable.SlowLogBackoffDetail, line, e.checker, fileLine)
} else if strings.HasPrefix(line, variable.SlowLogWarnings) {
line = line[len(variable.SlowLogWarnings+variable.SlowLogSpaceMarkStr):]
valid = e.setColumnValue(sctx, row, tz, variable.SlowLogWarnings, line, e.checker, fileLine)
} else {
fields, values := splitByColon(line)
for i := 0; i < len(fields); i++ {
Expand Down Expand Up @@ -781,7 +784,7 @@ func getColumnValueFactoryByName(sctx sessionctx.Context, colName string, column
}, nil
case variable.SlowLogUserStr, variable.SlowLogHostStr, execdetails.BackoffTypesStr, variable.SlowLogDBStr, variable.SlowLogIndexNamesStr, variable.SlowLogDigestStr,
variable.SlowLogStatsInfoStr, variable.SlowLogCopProcAddr, variable.SlowLogCopWaitAddr, variable.SlowLogPlanDigest,
variable.SlowLogPrevStmt, variable.SlowLogQuerySQLStr:
variable.SlowLogPrevStmt, variable.SlowLogQuerySQLStr, variable.SlowLogWarnings:
return func(row []types.Datum, value string, tz *time.Location, checker *slowLogChecker) (valid bool, err error) {
row[columnIdx] = types.NewStringDatum(value)
return true, nil
Expand Down
59 changes: 59 additions & 0 deletions executor/slow_query_sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/testkit/testdata"
"github.com/pingcap/tidb/util/logutil"
Expand Down Expand Up @@ -283,3 +284,61 @@ func TestIssue37066(t *testing.T) {
}
}
}

func TestWarningsInSlowQuery(t *testing.T) {
// Prepare the slow log
originCfg := config.GetGlobalConfig()
newCfg := *originCfg
f, err := os.CreateTemp("", "tidb-slow-*.log")
require.NoError(t, err)
newCfg.Log.SlowQueryFile = f.Name()
config.StoreGlobalConfig(&newCfg)
defer func() {
config.StoreGlobalConfig(originCfg)
require.NoError(t, f.Close())
require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile))
}()
require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig()))
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name()))
tk.MustExec("set tidb_slow_log_threshold=0;")
defer func() {
tk.MustExec("set tidb_slow_log_threshold=300;")
}()

tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, c int, d int, e int, f int, g int, h set('11', '22', '33')," +
"primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g))")
tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"})
require.NoError(t, err)
tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true}

var input []string
var output []struct {
SQL string
Result string
}
slowQuerySuiteData.LoadTestCases(t, &input, &output)
for i, test := range input {
comment := fmt.Sprintf("case:%v sql:%s", i, test)
if len(test) < 6 || test[:6] != "select" {
tk.MustExec(test)
} else {
tk.MustQuery(test)
}
res := testdata.ConvertRowsToStrings(
tk.MustQuery("select warnings from information_schema.slow_query " +
`where query = "` + test + `;" ` +
"order by time desc limit 1").Rows(),
)
require.Lenf(t, res, 1, comment)

testdata.OnRecord(func() {
output[i].SQL = test
output[i].Result = res[0]
})
require.Equal(t, output[i].Result, res[0])
}
}
4 changes: 2 additions & 2 deletions executor/slow_query_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,7 @@ select * from t;`
expectRecordString := `2019-04-28 15:24:04.309074,` +
`405888132465033227,root,localhost,0,57,0.12,0.216905,` +
`0,0,0,0,0,0,0,0,0,0,0,0,,0,0,0,0,0,0,0.38,0.021,0,0,0,1,637,0,10,10,10,10,100,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,` +
`0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,65536,0,0,0,0,0,` +
`0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,65536,0,0,0,0,0,,` +
`Cop_backoff_regionMiss_total_times: 200 Cop_backoff_regionMiss_total_time: 0.2 Cop_backoff_regionMiss_max_time: 0.2 Cop_backoff_regionMiss_max_addr: 127.0.0.1 Cop_backoff_regionMiss_avg_time: 0.2 Cop_backoff_regionMiss_p90_time: 0.2 Cop_backoff_rpcPD_total_times: 200 Cop_backoff_rpcPD_total_time: 0.2 Cop_backoff_rpcPD_max_time: 0.2 Cop_backoff_rpcPD_max_addr: 127.0.0.1 Cop_backoff_rpcPD_avg_time: 0.2 Cop_backoff_rpcPD_p90_time: 0.2 Cop_backoff_rpcTiKV_total_times: 200 Cop_backoff_rpcTiKV_total_time: 0.2 Cop_backoff_rpcTiKV_max_time: 0.2 Cop_backoff_rpcTiKV_max_addr: 127.0.0.1 Cop_backoff_rpcTiKV_avg_time: 0.2 Cop_backoff_rpcTiKV_p90_time: 0.2,` +
`0,0,1,0,1,1,0,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` +
`,update t set i = 1;,select * from t;`
Expand All @@ -183,7 +183,7 @@ select * from t;`
expectRecordString = `2019-04-28 15:24:04.309074,` +
`405888132465033227,root,localhost,0,57,0.12,0.216905,` +
`0,0,0,0,0,0,0,0,0,0,0,0,,0,0,0,0,0,0,0.38,0.021,0,0,0,1,637,0,10,10,10,10,100,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,` +
`0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,65536,0,0,0,0,0,` +
`0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,65536,0,0,0,0,0,,` +
`Cop_backoff_regionMiss_total_times: 200 Cop_backoff_regionMiss_total_time: 0.2 Cop_backoff_regionMiss_max_time: 0.2 Cop_backoff_regionMiss_max_addr: 127.0.0.1 Cop_backoff_regionMiss_avg_time: 0.2 Cop_backoff_regionMiss_p90_time: 0.2 Cop_backoff_rpcPD_total_times: 200 Cop_backoff_rpcPD_total_time: 0.2 Cop_backoff_rpcPD_max_time: 0.2 Cop_backoff_rpcPD_max_addr: 127.0.0.1 Cop_backoff_rpcPD_avg_time: 0.2 Cop_backoff_rpcPD_p90_time: 0.2 Cop_backoff_rpcTiKV_total_times: 200 Cop_backoff_rpcTiKV_total_time: 0.2 Cop_backoff_rpcTiKV_max_time: 0.2 Cop_backoff_rpcTiKV_max_addr: 127.0.0.1 Cop_backoff_rpcTiKV_avg_time: 0.2 Cop_backoff_rpcTiKV_p90_time: 0.2,` +
`0,0,1,0,1,1,0,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` +
`,update t set i = 1;,select * from t;`
Expand Down
19 changes: 19 additions & 0 deletions executor/testdata/slow_query_suite_in.json
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
[
{
"name": "TestWarningsInSlowQuery",
"cases": [
"insert into t(a) value (1)",
"select /*+ READ_FROM_STORAGE(TIKV[t]) */ a/0 from t",
"create binding for select * from t where c = 10 using select * from t use index (c_d_e) where c = 10",
"select * from t where c = 10",
"select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where a > 1 order by f",
"select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where f > 1",
"select /*+ READ_FROM_STORAGE(TIKV[t]) */ f from t where f > 1",
"select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where f > 3 and g = 5",
"select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where g = 5 order by f",
"select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where d = 3 order by c, e",
"select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where h = '11,22'",
"select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where a > rand()*100"
]
}
]
55 changes: 55 additions & 0 deletions executor/testdata/slow_query_suite_out.json
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
[
{
"Name": "TestWarningsInSlowQuery",
"Cases": [
{
"SQL": "insert into t(a) value (1)",
"Result": ""
},
{
"SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ a/0 from t",
"Result": "[{\"Level\":\"Warning\",\"Message\":\"Division by 0\"}]"
},
{
"SQL": "create binding for select * from t where c = 10 using select * from t use index (c_d_e) where c = 10",
"Result": ""
},
{
"SQL": "select * from t where c = 10",
"Result": "[{\"Level\":\"Note\",\"Message\":\"Using the bindSQL: SELECT * FROM `test`.`t` USE INDEX (`c_d_e`) WHERE `c` = 10\",\"IsExtra\":true}]"
},
{
"SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where a > 1 order by f",
"Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash)] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),f,f_g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}\",\"IsExtra\":true}]"
},
{
"SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where f > 1",
"Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true}]"
},
{
"SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ f from t where f > 1",
"Result": "[{\"Level\":\"Note\",\"Message\":\"[t(tiflash),f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true}]"
},
{
"SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where f > 3 and g = 5",
"Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),f_g,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true}]"
},
{
"SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where g = 5 order by f",
"Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),f_g,g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}\",\"IsExtra\":true}]"
},
{
"SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where d = 3 order by c, e",
"Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash)] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash),c_d_e] remain after pruning paths for t given Prop{SortItems: [{test.t.c asc} {test.t.e asc}], TaskTp: rootTask}\",\"IsExtra\":true}]"
},
{
"SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where h = '11,22'",
"Result": "[{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash)] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true}]"
},
{
"SQL": "select /*+ READ_FROM_STORAGE(TIKV[t]) */ * from t where a > rand()*100",
"Result": "[{\"Level\":\"Warning\",\"Message\":\"Scalar function 'rand'(signature: Rand, return type: double) is not supported to push down to storage layer now.\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Expression about 'test.t.h' can not be pushed to TiFlash because it contains unsupported calculation of type 'set'.\",\"IsExtra\":true},{\"Level\":\"Warning\",\"Message\":\"Scalar function 'rand'(signature: Rand, return type: double) is not supported to push down to tiflash now.\",\"IsExtra\":true},{\"Level\":\"Note\",\"Message\":\"[t,t(tiflash)] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}\",\"IsExtra\":true}]"
}
]
}
]
1 change: 1 addition & 0 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -890,6 +890,7 @@ var slowQueryCols = []columnInfo{
{name: variable.SlowLogBackoffTotal, tp: mysql.TypeDouble, size: 22},
{name: variable.SlowLogWriteSQLRespTotal, tp: mysql.TypeDouble, size: 22},
{name: variable.SlowLogResultRows, tp: mysql.TypeLonglong, size: 22},
{name: variable.SlowLogWarnings, tp: mysql.TypeLongBlob, size: types.UnspecifiedLength},
{name: variable.SlowLogBackoffDetail, tp: mysql.TypeVarchar, size: 4096},
{name: variable.SlowLogPrepared, tp: mysql.TypeTiny, size: 1},
{name: variable.SlowLogSucc, tp: mysql.TypeTiny, size: 1},
Expand Down
2 changes: 2 additions & 0 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -597,6 +597,7 @@ func TestSlowQuery(t *testing.T) {
"0",
"10",
"",
"",
"0",
"1",
"0",
Expand Down Expand Up @@ -670,6 +671,7 @@ func TestSlowQuery(t *testing.T) {
"0",
"0",
"",
"",
"0",
"1",
"0",
Expand Down
24 changes: 24 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"context"
"crypto/tls"
"encoding/binary"
"encoding/json"
"fmt"
"math"
"math/rand"
Expand Down Expand Up @@ -2735,6 +2736,9 @@ const (
SlowLogBackoffDetail = "Backoff_Detail"
// SlowLogResultRows is the row count of the SQL result.
SlowLogResultRows = "Result_rows"
// SlowLogWarnings is the warnings generated during executing the statement.
// Note that some extra warnings would also be printed through slow log.
SlowLogWarnings = "Warnings"
// SlowLogIsExplicitTxn is used to indicate whether this sql execute in explicit transaction or not.
SlowLogIsExplicitTxn = "IsExplicitTxn"
// SlowLogIsWriteCacheTable is used to indicate whether writing to the cache table need to wait for the read lock to expire.
Expand All @@ -2747,6 +2751,15 @@ const (
// It's controlled by the global variable `tidb_generate_binary_plan`.
var GenerateBinaryPlan atomic2.Bool

// JSONSQLWarnForSlowLog helps to print the SQLWarn through the slow log in JSON format.
type JSONSQLWarnForSlowLog struct {
Level string
Message string
// IsExtra means this SQL Warn is expected to be recorded only under some conditions (like in EXPLAIN) and should
// haven't been recorded as a warning now, but we recorded it anyway to help diagnostics.
IsExtra bool `json:",omitempty"`
}

// SlowQueryLogItems is a collection of items that should be included in the
// slow query log.
type SlowQueryLogItems struct {
Expand Down Expand Up @@ -2786,6 +2799,7 @@ type SlowQueryLogItems struct {
// table -> name -> status
StatsLoadStatus map[string]map[string]string
IsSyncStatsFailed bool
Warnings []JSONSQLWarnForSlowLog
}

// SlowLogFormat uses for formatting slow log.
Expand Down Expand Up @@ -2951,6 +2965,16 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string {
writeSlowLogItem(&buf, SlowLogBackoffTotal, strconv.FormatFloat(logItems.BackoffTotal.Seconds(), 'f', -1, 64))
writeSlowLogItem(&buf, SlowLogWriteSQLRespTotal, strconv.FormatFloat(logItems.WriteSQLRespTotal.Seconds(), 'f', -1, 64))
writeSlowLogItem(&buf, SlowLogResultRows, strconv.FormatInt(logItems.ResultRows, 10))
if len(logItems.Warnings) > 0 {
buf.WriteString(SlowLogRowPrefixStr + SlowLogWarnings + SlowLogSpaceMarkStr)
jsonEncoder := json.NewEncoder(&buf)
jsonEncoder.SetEscapeHTML(false)
// Note that the Encode() will append a '\n' so we don't need to add another.
err := jsonEncoder.Encode(logItems.Warnings)
if err != nil {
buf.WriteString(err.Error())
}
}
writeSlowLogItem(&buf, SlowLogSucc, strconv.FormatBool(logItems.Succ))
writeSlowLogItem(&buf, SlowLogIsExplicitTxn, strconv.FormatBool(logItems.IsExplicitTxn))
writeSlowLogItem(&buf, SlowLogIsSyncStatsFailed, strconv.FormatBool(logItems.IsSyncStatsFailed))
Expand Down

0 comments on commit 669ba5c

Please sign in to comment.