diff --git a/errno/infoschema.go b/errno/infoschema.go new file mode 100644 index 0000000000000..60eb5358c83b6 --- /dev/null +++ b/errno/infoschema.go @@ -0,0 +1,157 @@ +// Copyright 2021 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 errno + +import ( + "sync" + "time" +) + +// The error summary is protected by a mutex for simplicity. +// It is not expected to be hot unless there are concurrent workloads +// that are generating high error/warning counts, in which case +// the system probably has other issues already. + +// ErrorSummary summarizes errors and warnings +type ErrorSummary struct { + ErrorCount int + WarningCount int + FirstSeen time.Time + LastSeen time.Time +} + +// instanceStatistics provide statistics for a tidb-server instance. +type instanceStatistics struct { + sync.Mutex + global map[uint16]*ErrorSummary + users map[string]map[uint16]*ErrorSummary + hosts map[string]map[uint16]*ErrorSummary +} + +var stats instanceStatistics + +func init() { + FlushStats() +} + +// FlushStats resets errors and warnings across global/users/hosts +func FlushStats() { + stats.Lock() + defer stats.Unlock() + stats.global = make(map[uint16]*ErrorSummary) + stats.users = make(map[string]map[uint16]*ErrorSummary) + stats.hosts = make(map[string]map[uint16]*ErrorSummary) +} + +func copyMap(oldMap map[uint16]*ErrorSummary) map[uint16]*ErrorSummary { + newMap := make(map[uint16]*ErrorSummary, len(oldMap)) + for k, v := range oldMap { + newMap[k] = &ErrorSummary{ + ErrorCount: v.ErrorCount, + WarningCount: v.WarningCount, + FirstSeen: v.FirstSeen, + LastSeen: v.LastSeen, + } + } + return newMap +} + +// GlobalStats summarizes errors and warnings across all users/hosts +func GlobalStats() map[uint16]*ErrorSummary { + stats.Lock() + defer stats.Unlock() + return copyMap(stats.global) +} + +// UserStats summarizes per-user +func UserStats() map[string]map[uint16]*ErrorSummary { + stats.Lock() + defer stats.Unlock() + newMap := make(map[string]map[uint16]*ErrorSummary, len(stats.users)) + for k, v := range stats.users { + newMap[k] = copyMap(v) + } + return newMap +} + +// HostStats summarizes per remote-host +func HostStats() map[string]map[uint16]*ErrorSummary { + stats.Lock() + defer stats.Unlock() + newMap := make(map[string]map[uint16]*ErrorSummary, len(stats.hosts)) + for k, v := range stats.hosts { + newMap[k] = copyMap(v) + } + return newMap +} + +func initCounters(errCode uint16, user, host string) { + seen := time.Now() + stats.Lock() + defer stats.Unlock() + + if _, ok := stats.global[errCode]; !ok { + stats.global[errCode] = &ErrorSummary{FirstSeen: seen} + } + if _, ok := stats.users[user]; !ok { + stats.users[user] = make(map[uint16]*ErrorSummary) + } + if _, ok := stats.users[user][errCode]; !ok { + stats.users[user][errCode] = &ErrorSummary{FirstSeen: seen} + } + if _, ok := stats.hosts[host]; !ok { + stats.hosts[host] = make(map[uint16]*ErrorSummary) + } + if _, ok := stats.hosts[host][errCode]; !ok { + stats.hosts[host][errCode] = &ErrorSummary{FirstSeen: seen} + } +} + +// IncrementError increments the global/user/host statistics for an errCode +func IncrementError(errCode uint16, user, host string) { + seen := time.Now() + initCounters(errCode, user, host) + + stats.Lock() + defer stats.Unlock() + + // Increment counter + update last seen + stats.global[errCode].ErrorCount++ + stats.global[errCode].LastSeen = seen + // Increment counter + update last seen + stats.users[user][errCode].ErrorCount++ + stats.users[user][errCode].LastSeen = seen + // Increment counter + update last seen + stats.hosts[host][errCode].ErrorCount++ + stats.hosts[host][errCode].LastSeen = seen +} + +// IncrementWarning increments the global/user/host statistics for an errCode +func IncrementWarning(errCode uint16, user, host string) { + seen := time.Now() + initCounters(errCode, user, host) + + stats.Lock() + defer stats.Unlock() + + // Increment counter + update last seen + stats.global[errCode].WarningCount++ + stats.global[errCode].LastSeen = seen + // Increment counter + update last seen + stats.users[user][errCode].WarningCount++ + stats.users[user][errCode].LastSeen = seen + // Increment counter + update last seen + stats.hosts[host][errCode].WarningCount++ + stats.hosts[host][errCode].LastSeen = seen +} diff --git a/errno/infoschema_test.go b/errno/infoschema_test.go new file mode 100644 index 0000000000000..1a8d0f0c3984c --- /dev/null +++ b/errno/infoschema_test.go @@ -0,0 +1,89 @@ +// Copyright 2021 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 errno + +import ( + "testing" + + . "github.com/pingcap/check" +) + +func TestT(t *testing.T) { + TestingT(t) +} + +var _ = Suite(&testErrno{}) + +type testErrno struct{} + +func (s *testErrno) TestCopySafety(c *C) { + + IncrementError(123, "user", "host") + IncrementError(321, "user2", "host2") + IncrementWarning(123, "user", "host") + IncrementWarning(999, "user", "host") + IncrementWarning(222, "u", "h") + + globalCopy := GlobalStats() + userCopy := UserStats() + hostCopy := HostStats() + + IncrementError(123, "user", "host") + IncrementError(999, "user2", "host2") + IncrementError(123, "user3", "host") + IncrementWarning(123, "user", "host") + IncrementWarning(222, "u", "h") + IncrementWarning(222, "a", "b") + IncrementWarning(333, "c", "d") + + // global stats + c.Assert(stats.global[123].ErrorCount, Equals, 3) + c.Assert(globalCopy[123].ErrorCount, Equals, 1) + + // user stats + c.Assert(len(stats.users), Equals, 6) + c.Assert(len(userCopy), Equals, 3) + c.Assert(stats.users["user"][123].ErrorCount, Equals, 2) + c.Assert(stats.users["user"][123].WarningCount, Equals, 2) + c.Assert(userCopy["user"][123].ErrorCount, Equals, 1) + c.Assert(userCopy["user"][123].WarningCount, Equals, 1) + + // ensure there is no user3 in userCopy + _, ok := userCopy["user3"] + c.Assert(ok, IsFalse) + _, ok = stats.users["user3"] + c.Assert(ok, IsTrue) + _, ok = userCopy["a"] + c.Assert(ok, IsFalse) + _, ok = stats.users["a"] + c.Assert(ok, IsTrue) + + // host stats + c.Assert(len(stats.hosts), Equals, 5) + c.Assert(len(hostCopy), Equals, 3) + IncrementError(123, "user3", "newhost") + c.Assert(len(stats.hosts), Equals, 6) + c.Assert(len(hostCopy), Equals, 3) + + // ensure there is no newhost in hostCopy + _, ok = hostCopy["newhost"] + c.Assert(ok, IsFalse) + _, ok = stats.hosts["newhost"] + c.Assert(ok, IsTrue) + _, ok = hostCopy["b"] + c.Assert(ok, IsFalse) + _, ok = stats.hosts["b"] + c.Assert(ok, IsTrue) + +} diff --git a/executor/builder.go b/executor/builder.go index a0e559874a388..c40e36797a432 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1536,7 +1536,10 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TableStatementsSummaryHistory), strings.ToLower(infoschema.ClusterTableStatementsSummary), strings.ToLower(infoschema.ClusterTableStatementsSummaryHistory), - strings.ToLower(infoschema.TablePlacementPolicy): + strings.ToLower(infoschema.TablePlacementPolicy), + strings.ToLower(infoschema.TableClientErrorsSummaryGlobal), + strings.ToLower(infoschema.TableClientErrorsSummaryByUser), + strings.ToLower(infoschema.TableClientErrorsSummaryByHost): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), table: v.Table, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 6241ebb9f56e3..7abf159313a9e 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" @@ -142,6 +143,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex err = e.setDataForStatementsSummary(sctx, e.table.Name.O) case infoschema.TablePlacementPolicy: err = e.setDataForPlacementPolicy(sctx) + case infoschema.TableClientErrorsSummaryGlobal, + infoschema.TableClientErrorsSummaryByUser, + infoschema.TableClientErrorsSummaryByHost: + err = e.setDataForClientErrorsSummary(sctx, e.table.Name.O) } if err != nil { return nil, err @@ -1879,6 +1884,82 @@ func (e *memtableRetriever) setDataForPlacementPolicy(ctx sessionctx.Context) er return nil } +func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context, tableName string) error { + // Seeing client errors should require the PROCESS privilege, with the exception of errors for your own user. + // This is similar to information_schema.processlist, which is the closest comparison. + var hasProcessPriv bool + loginUser := ctx.GetSessionVars().User + if pm := privilege.GetPrivilegeManager(ctx); pm != nil { + if pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) { + hasProcessPriv = true + } + } + + var rows [][]types.Datum + switch tableName { + case infoschema.TableClientErrorsSummaryGlobal: + if !hasProcessPriv { + return plannercore.ErrSpecificAccessDenied.GenWithStackByArgs("PROCESS") + } + for code, summary := range errno.GlobalStats() { + firstSeen := types.NewTime(types.FromGoTime(summary.FirstSeen), mysql.TypeTimestamp, types.DefaultFsp) + lastSeen := types.NewTime(types.FromGoTime(summary.LastSeen), mysql.TypeTimestamp, types.DefaultFsp) + row := types.MakeDatums( + int(code), // ERROR_NUMBER + errno.MySQLErrName[code].Raw, // ERROR_MESSAGE + summary.ErrorCount, // ERROR_COUNT + summary.WarningCount, // WARNING_COUNT + firstSeen, // FIRST_SEEN + lastSeen, // LAST_SEEN + ) + rows = append(rows, row) + } + case infoschema.TableClientErrorsSummaryByUser: + for user, agg := range errno.UserStats() { + for code, summary := range agg { + // Allow anyone to see their own errors. + if !hasProcessPriv && loginUser != nil && loginUser.Username != user { + continue + } + firstSeen := types.NewTime(types.FromGoTime(summary.FirstSeen), mysql.TypeTimestamp, types.DefaultFsp) + lastSeen := types.NewTime(types.FromGoTime(summary.LastSeen), mysql.TypeTimestamp, types.DefaultFsp) + row := types.MakeDatums( + user, // USER + int(code), // ERROR_NUMBER + errno.MySQLErrName[code].Raw, // ERROR_MESSAGE + summary.ErrorCount, // ERROR_COUNT + summary.WarningCount, // WARNING_COUNT + firstSeen, // FIRST_SEEN + lastSeen, // LAST_SEEN + ) + rows = append(rows, row) + } + } + case infoschema.TableClientErrorsSummaryByHost: + if !hasProcessPriv { + return plannercore.ErrSpecificAccessDenied.GenWithStackByArgs("PROCESS") + } + for host, agg := range errno.HostStats() { + for code, summary := range agg { + firstSeen := types.NewTime(types.FromGoTime(summary.FirstSeen), mysql.TypeTimestamp, types.DefaultFsp) + lastSeen := types.NewTime(types.FromGoTime(summary.LastSeen), mysql.TypeTimestamp, types.DefaultFsp) + row := types.MakeDatums( + host, // HOST + int(code), // ERROR_NUMBER + errno.MySQLErrName[code].Raw, // ERROR_MESSAGE + summary.ErrorCount, // ERROR_COUNT + summary.WarningCount, // WARNING_COUNT + firstSeen, // FIRST_SEEN + lastSeen, // LAST_SEEN + ) + rows = append(rows, row) + } + } + } + e.rows = rows + return nil +} + type hugeMemTableRetriever struct { dummyCloser table *model.TableInfo diff --git a/executor/simple.go b/executor/simple.go index 29d60b5aaa310..f0d3135d21e6e 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" @@ -1327,6 +1328,8 @@ func (e *SimpleExec) executeFlush(s *ast.FlushStmt) error { return err } } + case ast.FlushClientErrorsSummary: + errno.FlushStats() } return nil } diff --git a/infoschema/tables.go b/infoschema/tables.go index 3aecf2f7d2454..ac70c62f9efe8 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -155,6 +155,12 @@ const ( TableTiFlashSegments = "TIFLASH_SEGMENTS" // TablePlacementPolicy is the string constant of placement policy table. TablePlacementPolicy = "PLACEMENT_POLICY" + // TableClientErrorsSummaryGlobal is the string constant of client errors table. + TableClientErrorsSummaryGlobal = "CLIENT_ERRORS_SUMMARY_GLOBAL" + // TableClientErrorsSummaryByUser is the string constant of client errors table. + TableClientErrorsSummaryByUser = "CLIENT_ERRORS_SUMMARY_BY_USER" + // TableClientErrorsSummaryByHost is the string constant of client errors table. + TableClientErrorsSummaryByHost = "CLIENT_ERRORS_SUMMARY_BY_HOST" ) var tableIDMap = map[string]int64{ @@ -224,6 +230,9 @@ var tableIDMap = map[string]int64{ TableTiFlashTables: autoid.InformationSchemaDBID + 64, TableTiFlashSegments: autoid.InformationSchemaDBID + 65, TablePlacementPolicy: autoid.InformationSchemaDBID + 66, + TableClientErrorsSummaryGlobal: autoid.InformationSchemaDBID + 67, + TableClientErrorsSummaryByUser: autoid.InformationSchemaDBID + 68, + TableClientErrorsSummaryByHost: autoid.InformationSchemaDBID + 69, } type columnInfo struct { @@ -1292,6 +1301,35 @@ var tablePlacementPolicyCols = []columnInfo{ {name: "CONSTRAINTS", tp: mysql.TypeVarchar, size: 1024}, } +var tableClientErrorsSummaryGlobalCols = []columnInfo{ + {name: "ERROR_NUMBER", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, + {name: "ERROR_MESSAGE", tp: mysql.TypeVarchar, size: 1024, flag: mysql.NotNullFlag}, + {name: "ERROR_COUNT", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, + {name: "WARNING_COUNT", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, + {name: "FIRST_SEEN", tp: mysql.TypeTimestamp, size: 26}, + {name: "LAST_SEEN", tp: mysql.TypeTimestamp, size: 26}, +} + +var tableClientErrorsSummaryByUserCols = []columnInfo{ + {name: "USER", tp: mysql.TypeVarchar, size: 64, flag: mysql.NotNullFlag}, + {name: "ERROR_NUMBER", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, + {name: "ERROR_MESSAGE", tp: mysql.TypeVarchar, size: 1024, flag: mysql.NotNullFlag}, + {name: "ERROR_COUNT", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, + {name: "WARNING_COUNT", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, + {name: "FIRST_SEEN", tp: mysql.TypeTimestamp, size: 26}, + {name: "LAST_SEEN", tp: mysql.TypeTimestamp, size: 26}, +} + +var tableClientErrorsSummaryByHostCols = []columnInfo{ + {name: "HOST", tp: mysql.TypeVarchar, size: 255, flag: mysql.NotNullFlag}, + {name: "ERROR_NUMBER", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, + {name: "ERROR_MESSAGE", tp: mysql.TypeVarchar, size: 1024, flag: mysql.NotNullFlag}, + {name: "ERROR_COUNT", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, + {name: "WARNING_COUNT", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, + {name: "FIRST_SEEN", tp: mysql.TypeTimestamp, size: 26}, + {name: "LAST_SEEN", tp: mysql.TypeTimestamp, size: 26}, +} + // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. // The returned description string may be: // - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified. @@ -1658,6 +1696,9 @@ var tableNameToColumns = map[string][]columnInfo{ TableTiFlashTables: tableTableTiFlashTablesCols, TableTiFlashSegments: tableTableTiFlashSegmentsCols, TablePlacementPolicy: tablePlacementPolicyCols, + TableClientErrorsSummaryGlobal: tableClientErrorsSummaryGlobalCols, + TableClientErrorsSummaryByUser: tableClientErrorsSummaryByUserCols, + TableClientErrorsSummaryByHost: tableClientErrorsSummaryByHostCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index cd3f2588f88d8..699a2caa2caed 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" @@ -1474,3 +1475,27 @@ func (s *testTableSuite) TestPlacementPolicy(c *C) { tk.MustQuery("select rule_id, schema_name, table_name, partition_name from information_schema.placement_policy order by partition_name, rule_id").Check(testkit.Rows( "0 test test_placement p0", "1 test test_placement p0", "0 test test_placement p1", "1 test test_placement p1")) } + +func (s *testTableSuite) TestInfoschemaClientErrors(c *C) { + tk := s.newTestKitWithRoot(c) + + tk.MustExec("FLUSH CLIENT_ERRORS_SUMMARY") + + errno.IncrementError(1365, "root", "localhost") + errno.IncrementError(1365, "infoschematest", "localhost") + errno.IncrementError(1365, "root", "localhost") + + tk.MustExec("CREATE USER 'infoschematest'@'localhost'") + c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "infoschematest", Hostname: "localhost"}, nil, nil), IsTrue) + + err := tk.QueryToErr("SELECT * FROM information_schema.client_errors_summary_global") + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation") + + err = tk.QueryToErr("SELECT * FROM information_schema.client_errors_summary_by_host") + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation") + + tk.MustQuery("SELECT error_number, error_count, warning_count FROM information_schema.client_errors_summary_by_user ORDER BY error_number").Check(testkit.Rows("1365 1 0")) + + err = tk.ExecToErr("FLUSH CLIENT_ERRORS_SUMMARY") + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RELOAD privilege(s) for this operation") +} diff --git a/server/conn.go b/server/conn.go index b4a648811f763..04e099b875c96 100644 --- a/server/conn.go +++ b/server/conn.go @@ -52,6 +52,8 @@ import ( "time" "unsafe" + goerr "errors" + "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -62,6 +64,7 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -1096,7 +1099,18 @@ func (cc *clientConn) useDB(ctx context.Context, db string) (err error) { } func (cc *clientConn) flush(ctx context.Context) error { - defer trace.StartRegion(ctx, "FlushClientConn").End() + defer func() { + trace.StartRegion(ctx, "FlushClientConn").End() + if cc.ctx != nil && cc.ctx.WarningCount() > 0 { + for _, err := range cc.ctx.GetWarnings() { + var warn *errors.Error + if ok := goerr.As(err.Err, &warn); ok { + code := uint16(warn.Code()) + errno.IncrementWarning(code, cc.user, cc.peerHost) + } + } + } + }() failpoint.Inject("FakeClientConn", func() { if cc.pkt == nil { failpoint.Return(nil) @@ -1158,6 +1172,7 @@ func (cc *clientConn) writeError(ctx context.Context, e error) error { } cc.lastCode = m.Code + defer errno.IncrementError(m.Code, cc.user, cc.peerHost) data := cc.alloc.AllocWithLen(4, 16+len(m.Message)) data = append(data, mysql.ErrHeader) data = append(data, byte(m.Code), byte(m.Code>>8)) diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 8af42fca5ba96..ae8fe74b87878 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" @@ -197,6 +198,11 @@ func (qd *TiDBDriver) OpenCtx(connID uint64, capability uint32, collation uint8, return tc, nil } +// GetWarnings implements QueryCtx GetWarnings method. +func (tc *TiDBContext) GetWarnings() []stmtctx.SQLWarn { + return tc.GetSessionVars().StmtCtx.GetWarnings() +} + // CurrentDB implements QueryCtx CurrentDB method. func (tc *TiDBContext) CurrentDB() string { return tc.currentDB diff --git a/server/server_test.go b/server/server_test.go index 5c4ae79cac04c..d92aff484071c 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -1862,3 +1862,64 @@ func (cli *testServerClient) waitUntilServerOnline() { log.Fatal("failed to connect HTTP status in every 10 ms", zap.Int("retryTime", retryTime)) } } + +// Client errors are only incremented when using the TiDB Server protocol, +// and not internal SQL statements. Thus, this test is in the server-test suite. +func (cli *testServerClient) runTestInfoschemaClientErrors(t *C) { + cli.runTestsOnNewDB(t, nil, "clientErrors", func(dbt *DBTest) { + + clientErrors := []struct { + stmt string + incrementWarnings bool + incrementErrors bool + errCode int + }{ + { + stmt: "SELECT 0/0", + incrementWarnings: true, + errCode: 1365, // div by zero + }, + { + stmt: "CREATE TABLE test_client_errors2 (a int primary key, b int primary key)", + incrementErrors: true, + errCode: 1068, // multiple pkeys + }, + { + stmt: "gibberish", + incrementErrors: true, + errCode: 1064, // parse error + }, + } + + sources := []string{"client_errors_summary_global", "client_errors_summary_by_user", "client_errors_summary_by_host"} + + for _, test := range clientErrors { + for _, tbl := range sources { + + var errors, warnings int + rows := dbt.mustQuery("SELECT SUM(error_count), SUM(warning_count) FROM information_schema."+tbl+" WHERE error_number = ? GROUP BY error_number", test.errCode) + if rows.Next() { + rows.Scan(&errors, &warnings) + } + + if test.incrementErrors { + errors++ + } + if test.incrementWarnings { + warnings++ + } + + dbt.db.Query(test.stmt) // ignore results and errors (query table) + var newErrors, newWarnings int + rows = dbt.mustQuery("SELECT SUM(error_count), SUM(warning_count) FROM information_schema."+tbl+" WHERE error_number = ? GROUP BY error_number", test.errCode) + if rows.Next() { + rows.Scan(&newErrors, &newWarnings) + } + + dbt.Check(newErrors, Equals, errors) + dbt.Check(newWarnings, Equals, warnings) + } + } + + }) +} diff --git a/server/tidb_test.go b/server/tidb_test.go index cb2668d49e862..2a727d2d30d3a 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -945,6 +945,10 @@ func (ts *tidbTestSuite) TestFieldList(c *C) { c.Assert(cols[0].Name, Equals, columnAsName) } +func (ts *tidbTestSuite) TestClientErrors(c *C) { + ts.runTestInfoschemaClientErrors(c) +} + func (ts *tidbTestSuite) TestSumAvg(c *C) { c.Parallel() ts.runTestSumAvg(c)