From d033259d455d3d7958c6f56a3f3a93f9a65121d9 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Fri, 22 Mar 2019 20:42:56 +0800 Subject: [PATCH 01/13] ddl: add check for 'create table partition by range column' (#9527) --- ddl/column.go | 9 ++++ ddl/db_partition_test.go | 81 ++++++++++++++++++++++++----- ddl/ddl.go | 10 ++++ ddl/ddl_api.go | 107 +++++++++++++++++++++++++++++++++++++-- ddl/partition.go | 9 +++- 5 files changed, 197 insertions(+), 19 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 1551a13dd672b..917633b2c1232 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -595,3 +595,12 @@ func findColumnInIndexCols(c *expression.Column, cols []*ast.IndexColName) bool } return false } + +func getColumnInfoByName(tbInfo *model.TableInfo, column string) *model.ColumnInfo { + for _, colInfo := range tbInfo.Cols() { + if colInfo.Name.L == column { + return colInfo + } + } + return nil +} diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 136613f427407..60e07b6dff99a 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/model" tmysql "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/ddl/testutil" "github.com/pingcap/tidb/domain" @@ -115,9 +116,9 @@ func (s *testIntegrationSuite) TestCreateTableWithPartition(c *C) { assertErrorCode(c, tk, sql4, tmysql.ErrPartitionMaxvalue) _, err = tk.Exec(`CREATE TABLE rc ( - a INT NOT NULL, - b INT NOT NULL, - c INT NOT NULL + a INT NOT NULL, + b INT NOT NULL, + c INT NOT NULL ) partition by range columns(a,b,c) ( partition p0 values less than (10,5,1), @@ -217,14 +218,6 @@ func (s *testIntegrationSuite) TestCreateTableWithPartition(c *C) { partition p4 values less than (18446744073709551000 + 10) );`) - // Only range type partition is now supported, range columns partition only implements the parser, so it will not be checked. - // So the following SQL statements can be executed successfully. - tk.MustExec(`create table t29 ( - a decimal - ) - partition by range columns (a) - (partition p0 values less than (0));`) - tk.MustExec("set @@tidb_enable_table_partition = 1") _, err = tk.Exec(`create table t30 ( a int, @@ -232,7 +225,7 @@ func (s *testIntegrationSuite) TestCreateTableWithPartition(c *C) { c varchar(30)) partition by range columns (a, b) (partition p0 values less than (10, 10.0))`) - c.Assert(ddl.ErrUnsupportedPartitionByRangeColumns.Equal(err), IsTrue) + c.Assert(ddl.ErrNotAllowedTypeInPartition.Equal(err), IsTrue) assertErrorCode(c, tk, `create table t31 (a int not null) partition by range( a );`, tmysql.ErrPartitionsMustBeDefined) assertErrorCode(c, tk, `create table t32 (a int not null) partition by range columns( a );`, tmysql.ErrPartitionsMustBeDefined) @@ -304,6 +297,70 @@ create table log_message_1 ( store_id int ) partition by hash( year(hired) ) partitions 4;`) + + tk.MustExec("drop table if exists t") + + type testCase struct { + sql string + err *terror.Error + } + + cases := []testCase{ + { + "create table t (id int) partition by range columns (id);", + ddl.ErrPartitionsMustBeDefined, + }, + { + "create table t (id int) partition by range columns (id) (partition p0 values less than (1, 2));", + ddl.ErrPartitionColumnList, + }, + { + "create table t (a int) partition by range columns (b) (partition p0 values less than (1, 2));", + ddl.ErrFieldNotFoundPart, + }, + { + "create table t (id timestamp) partition by range columns (id) (partition p0 values less than ('2019-01-09 11:23:34'));", + ddl.ErrNotAllowedTypeInPartition, + }, + { + `create table t29 ( + a decimal + ) + partition by range columns (a) + (partition p0 values less than (0));`, + ddl.ErrNotAllowedTypeInPartition, + }, + { + "create table t (id text) partition by range columns (id) (partition p0 values less than ('abc'));", + ddl.ErrNotAllowedTypeInPartition, + }, + { + "create table t (a int, b varchar(64)) partition by range columns (a, b) (" + + "partition p0 values less than (1, 'a')," + + "partition p1 values less than (1, 'a'))", + ddl.ErrRangeNotIncreasing, + }, + { + "create table t (a int, b varchar(64)) partition by range columns (a, b) (" + + "partition p0 values less than (1, 'b')," + + "partition p1 values less than (1, 'a'))", + ddl.ErrRangeNotIncreasing, + }, + { + "create table t (a int, b varchar(64)) partition by range columns (a, b) (" + + "partition p0 values less than (1, maxvalue)," + + "partition p1 values less than (1, 'a'))", + ddl.ErrRangeNotIncreasing, + }, + } + for i, t := range cases { + _, err := tk.Exec(t.sql) + c.Assert(t.err.Equal(err), IsTrue, Commentf("case %d fail, sql = %s", i, t.sql)) + } + + tk.MustExec("create table t1 (a int, b char(3)) partition by range columns (a, b) (" + + "partition p0 values less than (1, 'a')," + + "partition p1 values less than (2, maxvalue))") } func (s *testIntegrationSuite) TestCreateTableWithKeyPartition(c *C) { diff --git a/ddl/ddl.go b/ddl/ddl.go index d3bf39fff24ed..a8e7288827f6b 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -138,6 +138,7 @@ var ( // ErrUnsupportedPartitionByRangeColumns returns for does unsupported partition by range columns. ErrUnsupportedPartitionByRangeColumns = terror.ClassDDL.New(codeUnsupportedPartitionByRangeColumns, "unsupported partition by range columns") + errUnsupportedCreatePartition = terror.ClassDDL.New(codeUnsupportedCreatePartition, "unsupported partition type, treat as normal table") // ErrDupKeyName returns for duplicated key name ErrDupKeyName = terror.ClassDDL.New(codeDupKeyName, "duplicate key name") @@ -219,6 +220,10 @@ var ( ErrAlterOperationNotSupported = terror.ClassDDL.New(codeNotSupportedAlterOperation, mysql.MySQLErrName[mysql.ErrAlterOperationNotSupportedReason]) // ErrWrongObject returns for wrong object. ErrWrongObject = terror.ClassDDL.New(codeErrWrongObject, mysql.MySQLErrName[mysql.ErrWrongObject]) + // ErrFieldNotFoundPart returns an error when 'partition by columns' are not found in table columns. + ErrFieldNotFoundPart = terror.ClassDDL.New(codeFieldNotFoundPart, mysql.MySQLErrName[mysql.ErrFieldNotFoundPart]) + // ErrPartitionColumnList returns "Inconsistency in usage of column lists for partitioning". + ErrPartitionColumnList = terror.ClassDDL.New(codePartitionColumnList, mysql.MySQLErrName[mysql.ErrPartitionColumnList]) ) // DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache. @@ -649,6 +654,7 @@ const ( codeUnsupportedCoalescePartition = 209 codeUnsupportedModifyCharset = 210 codeUnsupportedPartitionByRangeColumns = 211 + codeUnsupportedCreatePartition = 212 codeFileNotFound = 1017 codeErrorOnRename = 1025 @@ -703,6 +709,8 @@ const ( codeUnknownPartition = terror.ErrCode(mysql.ErrUnknownPartition) codeErrGeneratedColumnRefAutoInc = terror.ErrCode(mysql.ErrGeneratedColumnRefAutoInc) codeNotSupportedAlterOperation = terror.ErrCode(mysql.ErrAlterOperationNotSupportedReason) + codeFieldNotFoundPart = terror.ErrCode(mysql.ErrFieldNotFoundPart) + codePartitionColumnList = terror.ErrCode(mysql.ErrPartitionColumnList) ) func init() { @@ -757,6 +765,8 @@ func init() { codeUnknownPartition: mysql.ErrUnknownPartition, codeNotSupportedAlterOperation: mysql.ErrAlterOperationNotSupportedReason, codeErrWrongObject: mysql.ErrWrongObject, + codeFieldNotFoundPart: mysql.ErrFieldNotFoundPart, + codePartitionColumnList: mysql.ErrPartitionColumnList, codeInvalidDefaultValue: mysql.ErrInvalidDefault, codeErrGeneratedColumnRefAutoInc: mysql.ErrGeneratedColumnRefAutoInc, } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 73a127c1b6319..efd9f78f050aa 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/schemautil" @@ -1356,11 +1357,13 @@ func checkPartitionByRangeColumn(ctx sessionctx.Context, tbInfo *model.TableInfo return err } - // TODO: Check CreatePartitionValue. - // Range columns partition key supports multiple data types with integer、datetime、string. - // if err := checkCreatePartitionValue(ctx, tbInfo, pi, cols); err != nil { - // return errors.Trace(err) - // } + if err := checkRangeColumnsPartitionType(tbInfo, pi.Columns); err != nil { + return err + } + + if err := checkRangeColumnsPartitionValue(ctx, tbInfo, pi); err != nil { + return err + } if err := checkNoRangePartitions(len(pi.Definitions)); err != nil { return errors.Trace(err) @@ -1369,6 +1372,100 @@ func checkPartitionByRangeColumn(ctx sessionctx.Context, tbInfo *model.TableInfo return checkAddPartitionTooManyPartitions(uint64(len(pi.Definitions))) } +func checkRangeColumnsPartitionType(tbInfo *model.TableInfo, columns []model.CIStr) error { + for _, col := range columns { + colInfo := getColumnInfoByName(tbInfo, col.L) + if colInfo == nil { + return errors.Trace(ErrFieldNotFoundPart) + } + // The permitted data types are shown in the following list: + // All integer types + // DATE and DATETIME + // CHAR, VARCHAR, BINARY, and VARBINARY + // See https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-columns.html + switch colInfo.FieldType.Tp { + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + case mysql.TypeDate, mysql.TypeDatetime: + case mysql.TypeVarchar, mysql.TypeString: + default: + return ErrNotAllowedTypeInPartition.GenWithStackByArgs(col.O) + } + } + return nil +} + +func checkRangeColumnsPartitionValue(ctx sessionctx.Context, tbInfo *model.TableInfo, pi *model.PartitionInfo) error { + // Range columns partition key supports multiple data types with integer、datetime、string. + defs := pi.Definitions + if len(defs) < 1 { + return errors.Trace(ErrPartitionsMustBeDefined) + } + + curr := &defs[0] + if len(curr.LessThan) != len(pi.Columns) { + return errors.Trace(ErrPartitionColumnList) + } + for i := 1; i < len(defs); i++ { + prev, curr := curr, &defs[i] + succ, err := checkTwoRangeColumns(ctx, curr, prev, pi, tbInfo) + if err != nil { + return err + } + if !succ { + return errors.Trace(ErrRangeNotIncreasing) + } + } + return nil +} + +func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDefinition, pi *model.PartitionInfo, tbInfo *model.TableInfo) (bool, error) { + if len(curr.LessThan) != len(pi.Columns) { + return false, errors.Trace(ErrPartitionColumnList) + } + for i := 0; i < len(pi.Columns); i++ { + // Special handling for MAXVALUE. + if strings.EqualFold(curr.LessThan[i], partitionMaxValue) { + // If current is maxvalue, it certainly >= previous. + return true, nil + } + if strings.EqualFold(prev.LessThan[i], partitionMaxValue) { + // Current is not maxvalue, and previous is maxvalue. + return false, nil + } + + // Current and previous is the same. + if strings.EqualFold(curr.LessThan[i], prev.LessThan[i]) { + continue + } + + // The tuples of column values used to define the partitions are strictly increasing: + // PARTITION p0 VALUES LESS THAN (5,10,'ggg') + // PARTITION p1 VALUES LESS THAN (10,20,'mmm') + // PARTITION p2 VALUES LESS THAN (15,30,'sss') + succ, err := parseAndEvalBoolExpr(ctx, fmt.Sprintf("(%s) > (%s)", curr.LessThan[i], prev.LessThan[i]), tbInfo) + if err != nil { + return false, err + } + + if succ { + return true, nil + } + } + return false, nil +} + +func parseAndEvalBoolExpr(ctx sessionctx.Context, expr string, tbInfo *model.TableInfo) (bool, error) { + e, err := expression.ParseSimpleExprWithTableInfo(ctx, expr, tbInfo) + if err != nil { + return false, err + } + res, _, err1 := e.EvalInt(ctx, chunk.Row{}) + if err1 != nil { + return false, err1 + } + return res > 0, nil +} + func checkCharsetAndCollation(cs string, co string) error { if !charset.ValidCharsetAndCollation(cs, co) { return ErrUnknownCharacterSet.GenWithStackByArgs(cs) diff --git a/ddl/partition.go b/ddl/partition.go index 6853adf572e39..62481a2bdaf53 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -55,6 +55,10 @@ func buildTablePartitionInfo(ctx sessionctx.Context, d *ddl, s *ast.CreateTableS enable = true } } + if !enable { + ctx.GetSessionVars().StmtCtx.AppendWarning(errUnsupportedCreatePartition) + } + pi := &model.PartitionInfo{ Type: s.Partition.Tp, Enable: enable, @@ -66,8 +70,9 @@ func buildTablePartitionInfo(ctx sessionctx.Context, d *ddl, s *ast.CreateTableS pi.Expr = buf.String() } else if s.Partition.ColumnNames != nil { // TODO: Support multiple columns for 'PARTITION BY RANGE COLUMNS'. - if enable && len(s.Partition.ColumnNames) != 1 { - return nil, errors.Trace(ErrUnsupportedPartitionByRangeColumns) + if len(s.Partition.ColumnNames) != 1 { + pi.Enable = false + ctx.GetSessionVars().StmtCtx.AppendWarning(ErrUnsupportedPartitionByRangeColumns) } pi.Columns = make([]model.CIStr, 0, len(s.Partition.ColumnNames)) for _, cn := range s.Partition.ColumnNames { From 655833380e98ec0b5be0016727c1498aca16db43 Mon Sep 17 00:00:00 2001 From: Lynn Date: Sat, 23 Mar 2019 10:10:30 +0800 Subject: [PATCH 02/13] domain: make leaktest happy (#9849) --- domain/domain.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/domain/domain.go b/domain/domain.go index 15ce746f4f7dc..2c75b48260262 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -734,7 +734,9 @@ func (do *Domain) LoadPrivilegeLoop(ctx sessionctx.Context) error { duration = 10 * time.Minute } + do.wg.Add(1) go func() { + defer do.wg.Done() defer recoverInDomain("loadPrivilegeInLoop", false) var count int for { @@ -791,7 +793,9 @@ func (do *Domain) LoadBindInfoLoop(ctx sessionctx.Context, parser *parser.Parser } duration := 3 * time.Second + do.wg.Add(1) go func() { + defer do.wg.Done() defer recoverInDomain("loadBindInfoLoop", false) for { select { From 5137c4d215ef7f05830a9bb90c78b8c1856661be Mon Sep 17 00:00:00 2001 From: Song Guo Date: Sat, 23 Mar 2019 10:20:14 +0800 Subject: [PATCH 03/13] types: replace RoundMode by int32 type (#9809) --- types/mydecimal.go | 71 ++++++++++++++++++++++--------- types/mydecimal_benchmark_test.go | 59 +++++++++++++++++++++++++ 2 files changed, 109 insertions(+), 21 deletions(-) create mode 100644 types/mydecimal_benchmark_test.go diff --git a/types/mydecimal.go b/types/mydecimal.go index 4c8a864e21bc9..06ca50c4204fb 100644 --- a/types/mydecimal.go +++ b/types/mydecimal.go @@ -23,7 +23,7 @@ import ( ) // RoundMode is the type for round mode. -type RoundMode string +type RoundMode int32 // constant values. const ( @@ -49,18 +49,52 @@ const ( DivFracIncr = 4 // ModeHalfEven rounds normally. - ModeHalfEven RoundMode = "ModeHalfEven" + ModeHalfEven RoundMode = 5 // Truncate just truncates the decimal. - ModeTruncate RoundMode = "Truncate" + ModeTruncate RoundMode = 10 // Ceiling is not supported now. - modeCeiling RoundMode = "Ceiling" + modeCeiling RoundMode = 0 ) var ( wordBufLen = 9 - powers10 = [10]int32{ten0, ten1, ten2, ten3, ten4, ten5, ten6, ten7, ten8, ten9} - dig2bytes = [10]int{0, 1, 1, 2, 2, 3, 3, 4, 4, 4} - fracMax = [8]int32{ + mod9 = [128]int8{ + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, 2, 3, 4, 5, 6, 7, 8, + 0, 1, + } + div9 = [128]int{ + 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 1, 1, 1, 1, 1, 1, 1, 1, + 2, 2, 2, 2, 2, 2, 2, 2, 2, + 3, 3, 3, 3, 3, 3, 3, 3, 3, + 4, 4, 4, 4, 4, 4, 4, 4, 4, + 5, 5, 5, 5, 5, 5, 5, 5, 5, + 6, 6, 6, 6, 6, 6, 6, 6, 6, + 7, 7, 7, 7, 7, 7, 7, 7, 7, + 8, 8, 8, 8, 8, 8, 8, 8, 8, + 9, 9, 9, 9, 9, 9, 9, 9, 9, + 10, 10, 10, 10, 10, 10, 10, 10, 10, + 11, 11, 11, 11, 11, 11, 11, 11, 11, + 12, 12, 12, 12, 12, 12, 12, 12, 12, + 13, 13, 13, 13, 13, 13, 13, 13, 13, + 14, 14, + } + powers10 = [10]int32{ten0, ten1, ten2, ten3, ten4, ten5, ten6, ten7, ten8, ten9} + dig2bytes = [10]int{0, 1, 1, 2, 2, 3, 3, 4, 4, 4} + fracMax = [8]int32{ 900000000, 990000000, 999000000, @@ -174,6 +208,9 @@ func countTrailingZeroes(i int, word int32) int { } func digitsToWords(digits int) int { + if digits+digitsPerWord-1 >= 0 && digits+digitsPerWord-1 < 128 { + return div9[digits+digitsPerWord-1] + } return (digits + digitsPerWord - 1) / digitsPerWord } @@ -756,16 +793,8 @@ func (d *MyDecimal) Round(to *MyDecimal, frac int, roundMode RoundMode) (err err wordsFrac := digitsToWords(int(d.digitsFrac)) wordsInt := digitsToWords(int(d.digitsInt)) - var roundDigit int32 + roundDigit := int32(roundMode) /* TODO - fix this code as it won't work for CEILING mode */ - switch roundMode { - case modeCeiling: - roundDigit = 0 - case ModeHalfEven: - roundDigit = 5 - case ModeTruncate: - roundDigit = 10 - } if wordsInt+wordsFracTo > wordBufLen { wordsFracTo = wordBufLen - wordsInt @@ -802,9 +831,9 @@ func (d *MyDecimal) Round(to *MyDecimal, frac int, roundMode RoundMode) (err err toIdx := wordsInt + wordsFracTo - 1 if frac == wordsFracTo*digitsPerWord { doInc := false - switch roundDigit { + switch roundMode { // Notice: No support for ceiling mode now. - case 0: + case modeCeiling: // If any word after scale is not zero, do increment. // e.g ceiling 3.0001 to scale 1, gets 3.1 idx := toIdx + (wordsFrac - wordsFracTo) @@ -815,11 +844,11 @@ func (d *MyDecimal) Round(to *MyDecimal, frac int, roundMode RoundMode) (err err } idx-- } - case 5: + case ModeHalfEven: digAfterScale := d.wordBuf[toIdx+1] / digMask // the first digit after scale. // If first digit after scale is 5 and round even, do increment if digit at scale is odd. doInc = (digAfterScale > 5) || (digAfterScale == 5) - case 10: + case ModeTruncate: // Never round, just truncate. doInc = false } @@ -917,7 +946,7 @@ func (d *MyDecimal) Round(to *MyDecimal, frac int, roundMode RoundMode) (err err } } /* Here we check 999.9 -> 1000 case when we need to increase intDigCnt */ - firstDig := to.digitsInt % digitsPerWord + firstDig := mod9[to.digitsInt] if firstDig > 0 && to.wordBuf[toIdx] >= powers10[firstDig] { to.digitsInt++ } diff --git a/types/mydecimal_benchmark_test.go b/types/mydecimal_benchmark_test.go new file mode 100644 index 0000000000000..3c484799f8caa --- /dev/null +++ b/types/mydecimal_benchmark_test.go @@ -0,0 +1,59 @@ +// Copyright 2019 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 types + +import "testing" + +func BenchmarkRound(b *testing.B) { + b.StopTimer() + var roundTo MyDecimal + tests := []struct { + input string + scale int + inputDec MyDecimal + }{ + {input: "123456789.987654321", scale: 1}, + {input: "15.1", scale: 0}, + {input: "15.5", scale: 0}, + {input: "15.9", scale: 0}, + {input: "-15.1", scale: 0}, + {input: "-15.5", scale: 0}, + {input: "-15.9", scale: 0}, + {input: "15.1", scale: 1}, + {input: "-15.1", scale: 1}, + {input: "15.17", scale: 1}, + {input: "15.4", scale: -1}, + {input: "-15.4", scale: -1}, + {input: "5.4", scale: -1}, + {input: ".999", scale: 0}, + {input: "999999999", scale: -9}, + } + + for i := 0; i < len(tests); i++ { + tests[i].inputDec.FromString([]byte(tests[i].input)) + } + + b.StartTimer() + for n := 0; n < b.N; n++ { + for i := 0; i < len(tests); i++ { + tests[i].inputDec.Round(&roundTo, tests[i].scale, ModeHalfEven) + } + for i := 0; i < len(tests); i++ { + tests[i].inputDec.Round(&roundTo, tests[i].scale, ModeTruncate) + } + for i := 0; i < len(tests); i++ { + tests[i].inputDec.Round(&roundTo, tests[i].scale, modeCeiling) + } + } +} From 9251724533e1e82d43d6b7ff5045218db0009e30 Mon Sep 17 00:00:00 2001 From: Du Chuan Date: Sat, 23 Mar 2019 10:43:06 +0800 Subject: [PATCH 04/13] ddl, expression: disable nondeterministic function calls for generated columns (#9239) --- ddl/ddl.go | 3 ++ ddl/ddl_api.go | 19 ++++++++++--- ddl/generated_column.go | 40 ++++++++++++++++++++++++++ executor/ddl_test.go | 45 ++++++++++++++++++++++++++++++ expression/constant_propagation.go | 4 +-- expression/function_traits.go | 42 ++++++++++++++++++++++++++++ 6 files changed, 147 insertions(+), 6 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index a8e7288827f6b..1ef175d30b9ee 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -135,6 +135,8 @@ var ( ErrUnsupportedAddPartition = terror.ClassDDL.New(codeUnsupportedAddPartition, "unsupported add partitions") // ErrUnsupportedCoalescePartition returns for does not support coalesce partitions. ErrUnsupportedCoalescePartition = terror.ClassDDL.New(codeUnsupportedCoalescePartition, "unsupported coalesce partitions") + // ErrGeneratedColumnFunctionIsNotAllowed returns for unsupported functions for generated columns. + ErrGeneratedColumnFunctionIsNotAllowed = terror.ClassDDL.New(codeErrGeneratedColumnFunctionIsNotAllowed, "Expression of generated column '%s' contains a disallowed function.") // ErrUnsupportedPartitionByRangeColumns returns for does unsupported partition by range columns. ErrUnsupportedPartitionByRangeColumns = terror.ClassDDL.New(codeUnsupportedPartitionByRangeColumns, "unsupported partition by range columns") @@ -707,6 +709,7 @@ const ( codeWarnDataTruncated = terror.ErrCode(mysql.WarnDataTruncated) codeCoalesceOnlyOnHashPartition = terror.ErrCode(mysql.ErrCoalesceOnlyOnHashPartition) codeUnknownPartition = terror.ErrCode(mysql.ErrUnknownPartition) + codeErrGeneratedColumnFunctionIsNotAllowed = terror.ErrCode(mysql.ErrGeneratedColumnFunctionIsNotAllowed) codeErrGeneratedColumnRefAutoInc = terror.ErrCode(mysql.ErrGeneratedColumnRefAutoInc) codeNotSupportedAlterOperation = terror.ErrCode(mysql.ErrAlterOperationNotSupportedReason) codeFieldNotFoundPart = terror.ErrCode(mysql.ErrFieldNotFoundPart) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index efd9f78f050aa..1ab6a84ee49d1 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -700,6 +700,13 @@ func checkGeneratedColumn(colDefs []*ast.ColumnDef) error { var exists bool var autoIncrementColumn string for i, colDef := range colDefs { + for _, option := range colDef.Options { + if option.Tp == ast.ColumnOptionGenerated { + if err := checkIllegalFn4GeneratedColumn(colDef.Name.Name.L, option.Expr); err != nil { + return errors.Trace(err) + } + } + } if checkIsAutoIncrementColumn(colDef) { exists, autoIncrementColumn = true, colDef.Name.Name.L } @@ -1840,6 +1847,9 @@ func (d *ddl) AddColumn(ctx sessionctx.Context, ti ast.Ident, spec *ast.AlterTab // generated columns occurring later in table. for _, option := range specNewColumn.Options { if option.Tp == ast.ColumnOptionGenerated { + if err := checkIllegalFn4GeneratedColumn(specNewColumn.Name.Name.L, option.Expr); err != nil { + return errors.Trace(err) + } referableColNames := make(map[string]struct{}, len(t.Cols())) for _, col := range t.Cols() { referableColNames[col.Name.L] = struct{}{} @@ -2188,8 +2198,8 @@ func setColumnComment(ctx sessionctx.Context, col *table.Column, option *ast.Col return errors.Trace(err) } -// setDefaultAndComment is only used in getModifiableColumnJob. -func setDefaultAndComment(ctx sessionctx.Context, col *table.Column, options []*ast.ColumnOption) error { +// processColumnOptions is only used in getModifiableColumnJob. +func processColumnOptions(ctx sessionctx.Context, col *table.Column, options []*ast.ColumnOption) error { if len(options) == 0 { return nil } @@ -2232,6 +2242,7 @@ func setDefaultAndComment(ctx sessionctx.Context, col *table.Column, options []* col.GeneratedExprString = buf.String() col.GeneratedStored = opt.Stored col.Dependences = make(map[string]struct{}) + col.GeneratedExpr = opt.Expr for _, colName := range findColumnNamesInExpr(opt.Expr) { col.Dependences[colName.Name.L] = struct{}{} } @@ -2281,7 +2292,7 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or } // Constraints in the new column means adding new constraints. Errors should thrown, - // which will be done by `setDefaultAndComment` later. + // which will be done by `processColumnOptions` later. if specNewColumn.Tp == nil { // Make sure the column definition is simple field type. return nil, errors.Trace(errUnsupportedModifyColumn) @@ -2322,7 +2333,7 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or if err != nil { return nil, errors.Trace(err) } - if err = setDefaultAndComment(ctx, newCol, specNewColumn.Options); err != nil { + if err = processColumnOptions(ctx, newCol, specNewColumn.Options); err != nil { return nil, errors.Trace(err) } diff --git a/ddl/generated_column.go b/ddl/generated_column.go index fde2552530530..efee4f1744af4 100644 --- a/ddl/generated_column.go +++ b/ddl/generated_column.go @@ -17,6 +17,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/table" ) @@ -104,6 +105,7 @@ func (c *generatedColumnChecker) Leave(inNode ast.Node) (node ast.Node, ok bool) // old and new is valid or not by such rules: // 1. the modification can't change stored status; // 2. if the new is generated, check its refer rules. +// 3. check if the modified expr contains non-deterministic functions func checkModifyGeneratedColumn(originCols []*table.Column, oldCol, newCol *table.Column) error { // rule 1. var stored = [2]bool{false, false} @@ -152,6 +154,44 @@ func checkModifyGeneratedColumn(originCols []*table.Column, oldCol, newCol *tabl return errors.Trace(err) } } + + // rule 3 + if newCol.IsGenerated() { + if err := checkIllegalFn4GeneratedColumn(newCol.Name.L, newCol.GeneratedExpr); err != nil { + return errors.Trace(err) + } + } + return nil +} + +type illegalFunctionChecker struct { + found bool +} + +func (c *illegalFunctionChecker) Enter(inNode ast.Node) (outNode ast.Node, skipChildren bool) { + switch node := inNode.(type) { + case *ast.FuncCallExpr: + if _, found := expression.IllegalFunctions4GeneratedColumns[node.FnName.L]; found { + c.found = true + return inNode, true + } + } + return inNode, false +} + +func (c *illegalFunctionChecker) Leave(inNode ast.Node) (node ast.Node, ok bool) { + return inNode, true +} + +func checkIllegalFn4GeneratedColumn(colName string, expr ast.ExprNode) error { + if expr == nil { + return nil + } + var c illegalFunctionChecker + expr.Accept(&c) + if c.found { + return ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs(colName) + } return nil } diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 0216b50214b65..2cabf98a579b1 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -595,6 +595,51 @@ func (s *testSuite3) TestSetDDLReorgBatchSize(c *C) { res.Check(testkit.Rows("1000")) } +func (s *testSuite3) TestIllegalFunctionCall4GeneratedColumns(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + // Test create an exist database + _, err := tk.Exec("CREATE database test") + c.Assert(err, NotNil) + + _, err = tk.Exec("create table t1 (b double generated always as (rand()) virtual);") + c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error()) + + _, err = tk.Exec("create table t1 (a varchar(64), b varchar(1024) generated always as (load_file(a)) virtual);") + c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error()) + + _, err = tk.Exec("create table t1 (a datetime generated always as (curdate()) virtual);") + c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error()) + + _, err = tk.Exec("create table t1 (a datetime generated always as (current_time()) virtual);") + c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error()) + + _, err = tk.Exec("create table t1 (a datetime generated always as (current_timestamp()) virtual);") + c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error()) + + _, err = tk.Exec("create table t1 (a datetime, b varchar(10) generated always as (localtime()) virtual);") + c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error()) + + _, err = tk.Exec("create table t1 (a varchar(1024) generated always as (uuid()) virtual);") + c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error()) + + _, err = tk.Exec("create table t1 (a varchar(1024), b varchar(1024) generated always as (is_free_lock(a)) virtual);") + c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error()) + + tk.MustExec("create table t1 (a bigint not null primary key auto_increment, b bigint, c bigint as (b + 1));") + + _, err = tk.Exec("alter table t1 add column d varchar(1024) generated always as (database());") + c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("d").Error()) + + tk.MustExec("alter table t1 add column d bigint generated always as (b + 1); ") + + _, err = tk.Exec("alter table t1 modify column d bigint generated always as (connection_id());") + c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("d").Error()) + + _, err = tk.Exec("alter table t1 change column c cc bigint generated always as (connection_id());") + c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("cc").Error()) +} + func (s *testSuite3) TestGeneratedColumnRelatedDDL(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/expression/constant_propagation.go b/expression/constant_propagation.go index 5c2a4c9dc197a..dbc1c98646290 100644 --- a/expression/constant_propagation.go +++ b/expression/constant_propagation.go @@ -117,8 +117,8 @@ func tryToReplaceCond(ctx sessionctx.Context, src *Column, tgt *Column, cond Exp } args[idx] = tgt } else { - subReplaced, isNonDeterminisitic, subExpr := tryToReplaceCond(ctx, src, tgt, expr) - if isNonDeterminisitic { + subReplaced, isNonDeterministic, subExpr := tryToReplaceCond(ctx, src, tgt, expr) + if isNonDeterministic { return false, true, cond } else if subReplaced { replaced = true diff --git a/expression/function_traits.go b/expression/function_traits.go index 988c020b18528..a350d5dc1dcc9 100644 --- a/expression/function_traits.go +++ b/expression/function_traits.go @@ -51,6 +51,48 @@ var DisableFoldFunctions = map[string]struct{}{ ast.Benchmark: {}, } +// IllegalFunctions4GeneratedColumns stores functions that is illegal for generated columns. +// See https://github.com/mysql/mysql-server/blob/5.7/mysql-test/suite/gcol/inc/gcol_blocked_sql_funcs_main.inc for details +var IllegalFunctions4GeneratedColumns = map[string]struct{}{ + ast.ConnectionID: {}, + ast.LoadFile: {}, + ast.LastInsertId: {}, + ast.Rand: {}, + ast.UUID: {}, + ast.UUIDShort: {}, + ast.Curdate: {}, + ast.CurrentDate: {}, + ast.Curtime: {}, + ast.CurrentTime: {}, + ast.CurrentTimestamp: {}, + ast.LocalTime: {}, + ast.LocalTimestamp: {}, + ast.Now: {}, + ast.UnixTimestamp: {}, + ast.UTCDate: {}, + ast.UTCTime: {}, + ast.UTCTimestamp: {}, + ast.Benchmark: {}, + ast.CurrentUser: {}, + ast.Database: {}, + ast.FoundRows: {}, + ast.GetLock: {}, + ast.IsFreeLock: {}, + ast.IsUsedLock: {}, + ast.MasterPosWait: {}, + ast.NameConst: {}, + ast.ReleaseLock: {}, + ast.RowCount: {}, + ast.Schema: {}, + ast.SessionUser: {}, + ast.Sleep: {}, + ast.SystemUser: {}, + ast.User: {}, + ast.Values: {}, + ast.Encrypt: {}, + ast.Version: {}, +} + // DeferredFunctions stores non-deterministic functions, which can be deferred only when the plan cache is enabled. var DeferredFunctions = map[string]struct{}{ ast.Now: {}, From 5ed3553bfb31c5377ff5bfa8ac703b950fa750da Mon Sep 17 00:00:00 2001 From: Keyi Xie Date: Sun, 24 Mar 2019 16:16:46 +0800 Subject: [PATCH 05/13] *: check more errors in test. (#9859) --- kv/buffer_store_test.go | 8 +++-- kv/mem_buffer_test.go | 6 ++-- kv/mock_test.go | 15 ++++++--- kv/union_store_test.go | 50 ++++++++++++++++++---------- kv/utils_test.go | 3 +- sessionctx/variable/varsutil_test.go | 18 ++++++---- 6 files changed, 66 insertions(+), 34 deletions(-) diff --git a/kv/buffer_store_test.go b/kv/buffer_store_test.go index b3aa2a865b6d9..5d7e1d7eef7cb 100644 --- a/kv/buffer_store_test.go +++ b/kv/buffer_store_test.go @@ -47,10 +47,11 @@ func (s testBufferStoreSuite) TestSaveTo(c *C) { c.Check(err, IsNil) buf.Reset() } - bs.Set(Key("novalue"), nil) + err := bs.Set(Key("novalue"), []byte("novalue")) + c.Check(err, IsNil) mutator := NewMemDbBuffer(DefaultTxnMembufCap) - err := bs.SaveTo(mutator) + err = bs.SaveTo(mutator) c.Check(err, IsNil) iter, err := mutator.Iter(nil, nil) @@ -58,6 +59,7 @@ func (s testBufferStoreSuite) TestSaveTo(c *C) { for iter.Valid() { cmp := bytes.Compare(iter.Key(), iter.Value()) c.Check(cmp, Equals, 0) - iter.Next() + err = iter.Next() + c.Check(err, IsNil) } } diff --git a/kv/mem_buffer_test.go b/kv/mem_buffer_test.go index 2f7d2e5ed812b..4032fe8f6cc63 100644 --- a/kv/mem_buffer_test.go +++ b/kv/mem_buffer_test.go @@ -189,7 +189,8 @@ func (s *testKVSuite) TestNewIteratorMin(c *C) { } for _, buffer := range s.bs { for _, kv := range kvs { - buffer.Set([]byte(kv.key), []byte(kv.value)) + err := buffer.Set([]byte(kv.key), []byte(kv.value)) + c.Assert(err, IsNil) } cnt := 0 @@ -197,7 +198,8 @@ func (s *testKVSuite) TestNewIteratorMin(c *C) { c.Assert(err, IsNil) for it.Valid() { cnt++ - it.Next() + err := it.Next() + c.Assert(err, IsNil) } c.Assert(cnt, Equals, 6) diff --git a/kv/mock_test.go b/kv/mock_test.go index 093f0d7bd553e..86e0e91141a3f 100644 --- a/kv/mock_test.go +++ b/kv/mock_test.go @@ -31,7 +31,8 @@ func (s testMockSuite) TestInterface(c *C) { version, err := storage.CurrentVersion() c.Check(err, IsNil) snapshot, err := storage.GetSnapshot(version) - snapshot.BatchGet([]Key{Key("abc"), Key("def")}) + c.Check(err, IsNil) + _, err = snapshot.BatchGet([]Key{Key("abc"), Key("def")}) c.Check(err, IsNil) transaction, err := storage.Begin() @@ -45,10 +46,14 @@ func (s testMockSuite) TestInterface(c *C) { transaction.StartTS() transaction.DelOption(Option(23)) if transaction.IsReadOnly() { - transaction.Get(Key("lock")) - transaction.Set(Key("lock"), []byte{}) - transaction.Iter(Key("lock"), nil) - transaction.IterReverse(Key("lock")) + _, err = transaction.Get(Key("lock")) + c.Check(err, IsNil) + err = transaction.Set(Key("lock"), []byte{}) + c.Check(err, IsNil) + _, err = transaction.Iter(Key("lock"), nil) + c.Check(err, IsNil) + _, err = transaction.IterReverse(Key("lock")) + c.Check(err, IsNil) } transaction.Commit(context.Background()) diff --git a/kv/union_store_test.go b/kv/union_store_test.go index e23d98776c8ea..cbf0f153b77fc 100644 --- a/kv/union_store_test.go +++ b/kv/union_store_test.go @@ -33,11 +33,13 @@ func (s *testUnionStoreSuite) SetUpTest(c *C) { func (s *testUnionStoreSuite) TestGetSet(c *C) { defer testleak.AfterTest(c)() - s.store.Set([]byte("1"), []byte("1")) + err := s.store.Set([]byte("1"), []byte("1")) + c.Assert(err, IsNil) v, err := s.us.Get([]byte("1")) c.Assert(err, IsNil) c.Assert(v, BytesEquals, []byte("1")) - s.us.Set([]byte("1"), []byte("2")) + err = s.us.Set([]byte("1"), []byte("2")) + c.Assert(err, IsNil) v, err = s.us.Get([]byte("1")) c.Assert(err, IsNil) c.Assert(v, BytesEquals, []byte("2")) @@ -45,13 +47,15 @@ func (s *testUnionStoreSuite) TestGetSet(c *C) { func (s *testUnionStoreSuite) TestDelete(c *C) { defer testleak.AfterTest(c)() - s.store.Set([]byte("1"), []byte("1")) - err := s.us.Delete([]byte("1")) + err := s.store.Set([]byte("1"), []byte("1")) + c.Assert(err, IsNil) + err = s.us.Delete([]byte("1")) c.Assert(err, IsNil) _, err = s.us.Get([]byte("1")) c.Assert(IsErrNotFound(err), IsTrue) - s.us.Set([]byte("1"), []byte("2")) + err = s.us.Set([]byte("1"), []byte("2")) + c.Assert(err, IsNil) v, err := s.us.Get([]byte("1")) c.Assert(err, IsNil) c.Assert(v, BytesEquals, []byte("2")) @@ -59,9 +63,12 @@ func (s *testUnionStoreSuite) TestDelete(c *C) { func (s *testUnionStoreSuite) TestSeek(c *C) { defer testleak.AfterTest(c)() - s.store.Set([]byte("1"), []byte("1")) - s.store.Set([]byte("2"), []byte("2")) - s.store.Set([]byte("3"), []byte("3")) + err := s.store.Set([]byte("1"), []byte("1")) + c.Assert(err, IsNil) + err = s.store.Set([]byte("2"), []byte("2")) + c.Assert(err, IsNil) + err = s.store.Set([]byte("3"), []byte("3")) + c.Assert(err, IsNil) iter, err := s.us.Iter(nil, nil) c.Assert(err, IsNil) @@ -71,12 +78,14 @@ func (s *testUnionStoreSuite) TestSeek(c *C) { c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("3")}, [][]byte{[]byte("2"), []byte("3")}) - s.us.Set([]byte("4"), []byte("4")) + err = s.us.Set([]byte("4"), []byte("4")) + c.Assert(err, IsNil) iter, err = s.us.Iter([]byte("2"), nil) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("3"), []byte("4")}, [][]byte{[]byte("2"), []byte("3"), []byte("4")}) - s.us.Delete([]byte("3")) + err = s.us.Delete([]byte("3")) + c.Assert(err, IsNil) iter, err = s.us.Iter([]byte("2"), nil) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("4")}, [][]byte{[]byte("2"), []byte("4")}) @@ -84,9 +93,12 @@ func (s *testUnionStoreSuite) TestSeek(c *C) { func (s *testUnionStoreSuite) TestIterReverse(c *C) { defer testleak.AfterTest(c)() - s.store.Set([]byte("1"), []byte("1")) - s.store.Set([]byte("2"), []byte("2")) - s.store.Set([]byte("3"), []byte("3")) + err := s.store.Set([]byte("1"), []byte("1")) + c.Assert(err, IsNil) + err = s.store.Set([]byte("2"), []byte("2")) + c.Assert(err, IsNil) + err = s.store.Set([]byte("3"), []byte("3")) + c.Assert(err, IsNil) iter, err := s.us.IterReverse(nil) c.Assert(err, IsNil) @@ -96,12 +108,14 @@ func (s *testUnionStoreSuite) TestIterReverse(c *C) { c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("1")}, [][]byte{[]byte("2"), []byte("1")}) - s.us.Set([]byte("0"), []byte("0")) + err = s.us.Set([]byte("0"), []byte("0")) + c.Assert(err, IsNil) iter, err = s.us.IterReverse([]byte("3")) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("1"), []byte("0")}, [][]byte{[]byte("2"), []byte("1"), []byte("0")}) - s.us.Delete([]byte("1")) + err = s.us.Delete([]byte("1")) + c.Assert(err, IsNil) iter, err = s.us.IterReverse([]byte("3")) c.Assert(err, IsNil) checkIterator(c, iter, [][]byte{[]byte("2"), []byte("0")}, [][]byte{[]byte("2"), []byte("0")}) @@ -109,8 +123,10 @@ func (s *testUnionStoreSuite) TestIterReverse(c *C) { func (s *testUnionStoreSuite) TestLazyConditionCheck(c *C) { defer testleak.AfterTest(c)() - s.store.Set([]byte("1"), []byte("1")) - s.store.Set([]byte("2"), []byte("2")) + err := s.store.Set([]byte("1"), []byte("1")) + c.Assert(err, IsNil) + err = s.store.Set([]byte("2"), []byte("2")) + c.Assert(err, IsNil) v, err := s.us.Get([]byte("1")) c.Assert(err, IsNil) diff --git a/kv/utils_test.go b/kv/utils_test.go index c8e02006ec47c..1c189c4a5c57c 100644 --- a/kv/utils_test.go +++ b/kv/utils_test.go @@ -32,7 +32,8 @@ func (s testUtilsSuite) TestIncInt64(c *C) { c.Check(err, IsNil) c.Check(v, Equals, int64(11)) - mb.Set(key, []byte("not int")) + err = mb.Set(key, []byte("not int")) + c.Check(err, IsNil) _, err = IncInt64(mb, key, 1) c.Check(err, NotNil) } diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 580d2506488c8..e94fdffb08463 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -97,25 +97,29 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { v := NewSessionVars() v.GlobalVarsAccessor = NewMockGlobalAccessor() - SetSessionSystemVar(v, "autocommit", types.NewStringDatum("1")) + err := SetSessionSystemVar(v, "autocommit", types.NewStringDatum("1")) + c.Assert(err, IsNil) val, err := GetSessionSystemVar(v, "autocommit") c.Assert(err, IsNil) c.Assert(val, Equals, "1") c.Assert(SetSessionSystemVar(v, "autocommit", types.Datum{}), NotNil) // 0 converts to OFF - SetSessionSystemVar(v, "foreign_key_checks", types.NewStringDatum("0")) + err = SetSessionSystemVar(v, "foreign_key_checks", types.NewStringDatum("0")) + c.Assert(err, IsNil) val, err = GetSessionSystemVar(v, "foreign_key_checks") c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") // 1/ON is not supported (generates a warning and sets to OFF) - SetSessionSystemVar(v, "foreign_key_checks", types.NewStringDatum("1")) + err = SetSessionSystemVar(v, "foreign_key_checks", types.NewStringDatum("1")) + c.Assert(err, IsNil) val, err = GetSessionSystemVar(v, "foreign_key_checks") c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") - SetSessionSystemVar(v, "sql_mode", types.NewStringDatum("strict_trans_tables")) + err = SetSessionSystemVar(v, "sql_mode", types.NewStringDatum("strict_trans_tables")) + c.Assert(err, IsNil) val, err = GetSessionSystemVar(v, "sql_mode") c.Assert(err, IsNil) c.Assert(val, Equals, "STRICT_TRANS_TABLES") @@ -123,8 +127,10 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { SetSessionSystemVar(v, "sql_mode", types.NewStringDatum("")) c.Assert(v.StrictSQLMode, IsFalse) - SetSessionSystemVar(v, "character_set_connection", types.NewStringDatum("utf8")) - SetSessionSystemVar(v, "collation_connection", types.NewStringDatum("utf8_general_ci")) + err = SetSessionSystemVar(v, "character_set_connection", types.NewStringDatum("utf8")) + c.Assert(err, IsNil) + err = SetSessionSystemVar(v, "collation_connection", types.NewStringDatum("utf8_general_ci")) + c.Assert(err, IsNil) charset, collation := v.GetCharsetInfo() c.Assert(charset, Equals, "utf8") c.Assert(collation, Equals, "utf8_general_ci") From d5f0d2bfdd4fa78840f23ac8f934496a4c8ac02a Mon Sep 17 00:00:00 2001 From: WangXiangUSTC Date: Sun, 24 Mar 2019 18:26:15 +0800 Subject: [PATCH 06/13] binlog: revert commit for variable `tidb_log_bin` && add send binlog strategy config (#9864) --- config/config.go | 4 +++- config/config.toml.example | 8 ++++---- config/config_test.go | 4 ++-- executor/set_test.go | 11 +++-------- go.mod | 2 +- go.sum | 4 ++-- session/session.go | 7 ------- sessionctx/binloginfo/binloginfo.go | 11 ----------- sessionctx/variable/sysvar.go | 3 --- sessionctx/variable/varsutil.go | 2 +- tidb-server/main.go | 9 ++++----- 11 files changed, 20 insertions(+), 45 deletions(-) diff --git a/config/config.go b/config/config.go index 2a3361ff90a70..6ab636cff3d01 100644 --- a/config/config.go +++ b/config/config.go @@ -251,13 +251,14 @@ type TiKVClient struct { // Binlog is the config for binlog. type Binlog struct { Enable bool `toml:"enable" json:"enable"` - AutoMode bool `toml:"auto-mode" json:"auto-mode"` WriteTimeout string `toml:"write-timeout" json:"write-timeout"` // If IgnoreError is true, when writing binlog meets error, TiDB would // ignore the error. IgnoreError bool `toml:"ignore-error" json:"ignore-error"` // Use socket file to write binlog, for compatible with kafka version tidb-binlog. BinlogSocket string `toml:"binlog-socket" json:"binlog-socket"` + // The strategy for sending binlog to pump, value can be "range" or "hash" now. + Strategy string `toml:"strategy" json:"strategy"` } // Plugin is the config for plugin @@ -345,6 +346,7 @@ var defaultConf = Config{ }, Binlog: Binlog{ WriteTimeout: "15s", + Strategy: "range", }, } diff --git a/config/config.toml.example b/config/config.toml.example index bcfd24384ac2d..6c5e8bda1d8ce 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -261,12 +261,9 @@ enabled = true capacity = 2048000 [binlog] -# Enable to write binlog. This config will be disabled if auto-mode is true. +# enable to write binlog. enable = false -# If auto-mode is true, will enable binlog according to the system variables 'tidb_log_bin'. -auto-mode = false - # WriteTimeout specifies how long it will wait for writing binlog to pump. write-timeout = "15s" @@ -276,3 +273,6 @@ ignore-error = false # use socket file to write binlog, for compatible with kafka version tidb-binlog. binlog-socket = "" + +# the strategy for sending binlog to pump, value can be "range" or "hash" now. +strategy = "range" \ No newline at end of file diff --git a/config/config_test.go b/config/config_test.go index 137c07572053f..aa3a91496b09c 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -34,8 +34,8 @@ func TestT(t *testing.T) { func (s *testConfigSuite) TestConfig(c *C) { conf := new(Config) conf.Binlog.Enable = true - conf.Binlog.AutoMode = true conf.Binlog.IgnoreError = true + conf.Binlog.Strategy = "hash" conf.TiKVClient.CommitTimeout = "10s" conf.CheckMb4ValueInUtf8 = true configFile := "config.toml" @@ -56,7 +56,7 @@ max-batch-size=128 // Test that the original value will not be clear by load the config file that does not contain the option. c.Assert(conf.Binlog.Enable, Equals, true) - c.Assert(conf.Binlog.AutoMode, Equals, true) + c.Assert(conf.Binlog.Strategy, Equals, "hash") c.Assert(conf.TiKVClient.CommitTimeout, Equals, "41s") c.Assert(conf.TiKVClient.MaxBatchSize, Equals, uint(128)) diff --git a/executor/set_test.go b/executor/set_test.go index 54a6c347627b7..6fdb59fef08ab 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -18,8 +18,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" @@ -239,13 +239,8 @@ func (s *testSuite2) TestSetVar(c *C) { tk.MustExec("set @@sql_log_bin = on") tk.MustQuery(`select @@session.sql_log_bin;`).Check(testkit.Rows("1")) - tk.MustQuery(`select @@global.log_bin;`).Check(testkit.Rows(variable.BoolToIntStr(binloginfo.ShouldEnableBinlog()))) - tk.MustQuery(`select @@log_bin;`).Check(testkit.Rows(variable.BoolToIntStr(binloginfo.ShouldEnableBinlog()))) - - tk.MustExec("set global tidb_log_bin = on") - tk.MustQuery(`select @@global.tidb_log_bin;`).Check(testkit.Rows("1")) - tk.MustExec("set global tidb_log_bin = off") - tk.MustQuery(`select @@global.tidb_log_bin;`).Check(testkit.Rows("0")) + tk.MustQuery(`select @@global.log_bin;`).Check(testkit.Rows(variable.BoolToIntStr(config.GetGlobalConfig().Binlog.Enable))) + tk.MustQuery(`select @@log_bin;`).Check(testkit.Rows(variable.BoolToIntStr(config.GetGlobalConfig().Binlog.Enable))) tk.MustExec("set @@tidb_general_log = 1") tk.MustExec("set @@tidb_general_log = 0") diff --git a/go.mod b/go.mod index 360f37df49dcb..2334fa8c642a6 100644 --- a/go.mod +++ b/go.mod @@ -53,7 +53,7 @@ require ( github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 github.com/pingcap/parser v0.0.0-20190321052000-f9a452f8f24e github.com/pingcap/pd v2.1.0-rc.4+incompatible - github.com/pingcap/tidb-tools v2.1.3-0.20190116051332-34c808eef588+incompatible + github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7 github.com/pquerna/ffjson v0.0.0-20181028064349-e517b90714f7 // indirect github.com/prometheus/client_golang v0.9.0 diff --git a/go.sum b/go.sum index 30c92606cea8b..6341b69fed8c0 100644 --- a/go.sum +++ b/go.sum @@ -123,8 +123,8 @@ github.com/pingcap/parser v0.0.0-20190321052000-f9a452f8f24e h1:Evw2H5BmAGqHTKbb github.com/pingcap/parser v0.0.0-20190321052000-f9a452f8f24e/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v2.1.0-rc.4+incompatible h1:/buwGk04aHO5odk/+O8ZOXGs4qkUjYTJ2UpCJXna8NE= github.com/pingcap/pd v2.1.0-rc.4+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E= -github.com/pingcap/tidb-tools v2.1.3-0.20190116051332-34c808eef588+incompatible h1:e9Gi/LP9181HT3gBfSOeSBA+5JfemuE4aEAhqNgoE4k= -github.com/pingcap/tidb-tools v2.1.3-0.20190116051332-34c808eef588+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= +github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7 h1:wnjdQRhybddDesBVBKyOLUPgDaOFdtqA92pduBgWvVQ= github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= diff --git a/session/session.go b/session/session.go index 0bd30c0545a1f..9f1f6b5e4b00f 100644 --- a/session/session.go +++ b/session/session.go @@ -1361,13 +1361,6 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { return nil, errors.Trace(err) } - // get global system variable tidb_log_bin from mysql.GLOBAL_VARIABLES - tidbLogBin, err := se1.GetGlobalSysVar(variable.TiDBLogBin) - if err != nil { - return nil, errors.Trace(err) - } - variable.SysVars[variable.TiDBLogBin].Value = tidbLogBin - if len(cfg.Plugin.Load) > 0 { plugin.InitWatchLoops(dom.GetEtcdClient()) } diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 0976fb8801ab3..8be106c057222 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -24,11 +24,9 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb-tools/tidb-binlog/node" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" binlog "github.com/pingcap/tipb/go-binlog" log "github.com/sirupsen/logrus" "google.golang.org/grpc" @@ -95,15 +93,6 @@ func SetIgnoreError(on bool) { } } -// ShouldEnableBinlog returns true if Binlog.AutoMode is false and Binlog.Enable is true, or Binlog.AutoMode is true and tidb_log_bin's value is "1" -func ShouldEnableBinlog() bool { - if config.GetGlobalConfig().Binlog.AutoMode { - return variable.SysVars[variable.TiDBLogBin].Value == "1" - } - - return config.GetGlobalConfig().Binlog.Enable -} - // WriteBinlog writes a binlog to Pump. func (info *BinlogInfo) WriteBinlog(clusterID uint64) error { skip := atomic.LoadUint32(&skipBinlog) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 0ba6b3419cb31..d0ffdb6a31ee3 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -388,7 +388,6 @@ var defaultSysVars = []*SysVar{ {ScopeSession, "last_insert_id", ""}, {ScopeNone, "innodb_ft_cache_size", "8000000"}, {ScopeNone, LogBin, "0"}, - {ScopeGlobal, TiDBLogBin, "0"}, {ScopeGlobal, "innodb_disable_sort_file_cache", "OFF"}, {ScopeGlobal, "log_error_verbosity", ""}, {ScopeNone, "performance_schema_hosts_size", "100"}, @@ -749,8 +748,6 @@ const ( SQLLogBin = "sql_log_bin" // LogBin is the name for 'log_bin' system variable. LogBin = "log_bin" - // TiDBLogBin is the name for 'tidb_log_bin' system variable. - TiDBLogBin = "tidb_log_bin" // MaxSortLength is the name for 'max_sort_length' system variable. MaxSortLength = "max_sort_length" // MaxSpRecursionDepth is the name for 'max_sp_recursion_depth' system variable. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index a0d061181b246..db8cce5ba7ed9 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -350,7 +350,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, case WarningCount, ErrorCount: return value, ErrReadOnly.GenWithStackByArgs(name) case GeneralLog, TiDBGeneralLog, AvoidTemporalUpgrade, BigTables, CheckProxyUsers, LogBin, - CoreFile, EndMakersInJSON, SQLLogBin, TiDBLogBin, OfflineMode, PseudoSlaveMode, LowPriorityUpdates, + CoreFile, EndMakersInJSON, SQLLogBin, OfflineMode, PseudoSlaveMode, LowPriorityUpdates, SkipNameResolve, SQLSafeUpdates, TiDBConstraintCheckInPlace: if strings.EqualFold(value, "ON") || value == "1" { return "1", nil diff --git a/tidb-server/main.go b/tidb-server/main.go index c12814fabe10c..fd54e4c070874 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -146,10 +146,9 @@ func main() { setupLog() setupTracing() // Should before createServer and after setup config. printInfo() + setupBinlogClient() setupMetrics() createStoreAndDomain() - // setupBinlogClient should run after bootstrap - setupBinlogClient() createServer() signal.SetupSignalHandler(serverShutdown) runServer() @@ -188,7 +187,7 @@ func createStoreAndDomain() { } func setupBinlogClient() { - if !binloginfo.ShouldEnableBinlog() { + if !cfg.Binlog.Enable { return } @@ -208,7 +207,7 @@ func setupBinlogClient() { } if len(cfg.Binlog.BinlogSocket) == 0 { - client, err = pumpcli.NewPumpsClient(cfg.Path, parseDuration(cfg.Binlog.WriteTimeout), securityOption) + client, err = pumpcli.NewPumpsClient(cfg.Path, cfg.Binlog.Strategy, parseDuration(cfg.Binlog.WriteTimeout), securityOption) } else { client, err = pumpcli.NewLocalPumpsClient(cfg.Path, cfg.Binlog.BinlogSocket, parseDuration(cfg.Binlog.WriteTimeout), securityOption) } @@ -453,7 +452,7 @@ func setGlobalVars() { variable.SysVars[variable.TIDBMemQuotaQuery].Value = strconv.FormatInt(cfg.MemQuotaQuery, 10) variable.SysVars["lower_case_table_names"].Value = strconv.Itoa(cfg.LowerCaseTableNames) - variable.SysVars[variable.LogBin].Value = variable.BoolToIntStr(binloginfo.ShouldEnableBinlog()) + variable.SysVars[variable.LogBin].Value = variable.BoolToIntStr(config.GetGlobalConfig().Binlog.Enable) variable.SysVars[variable.Port].Value = fmt.Sprintf("%d", cfg.Port) variable.SysVars[variable.Socket].Value = cfg.Socket From 630671e41d37ebb04386dd3a684fd5c6d508518e Mon Sep 17 00:00:00 2001 From: winkyao Date: Mon, 25 Mar 2019 00:06:51 +0800 Subject: [PATCH 07/13] privilege: add drop_priv check when truncate table (#9870) --- planner/core/logical_plan_test.go | 2 +- planner/core/planbuilder.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 99f4b468721f1..5b1c29a18e534 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1535,7 +1535,7 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { { sql: "truncate table t", ans: []visitInfo{ - {mysql.DeletePriv, "test", "t", "", nil}, + {mysql.DropPriv, "test", "t", "", nil}, }, }, { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index f2e27f94fb50f..ee1f5e640f29f 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1582,7 +1582,7 @@ func (b *PlanBuilder) buildDDL(node ast.DDLNode) (Plan, error) { } case *ast.TruncateTableStmt: b.visitInfo = append(b.visitInfo, visitInfo{ - privilege: mysql.DeletePriv, + privilege: mysql.DropPriv, db: v.Table.Schema.L, table: v.Table.Name.L, err: nil, From 99defc0c48836edc04ecd40a111b76bb85f9c666 Mon Sep 17 00:00:00 2001 From: "Zhuomin(Charming) Liu" Date: Mon, 25 Mar 2019 13:47:16 +0800 Subject: [PATCH 08/13] types: fix `time_format` is not compatible with MySQL (#9841) --- expression/integration_test.go | 6 ++++++ types/time.go | 5 +++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index d52d49651412e..bad171ee28de2 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -1515,6 +1515,12 @@ func (s *testIntegrationSuite) TestTimeBuiltin(c *C) { result.Check(testkit.Rows("")) result = tk.MustQuery("SELECT TIME_FORMAT(123, '%H:%i:%s %p');") result.Check(testkit.Rows("00:01:23 AM")) + result = tk.MustQuery("SELECT TIME_FORMAT('24:00:00', '%r');") + result.Check(testkit.Rows("12:00:00 AM")) + result = tk.MustQuery("SELECT TIME_FORMAT('25:00:00', '%r');") + result.Check(testkit.Rows("01:00:00 AM")) + result = tk.MustQuery("SELECT TIME_FORMAT('24:00:00', '%l %p');") + result.Check(testkit.Rows("12 AM")) // for date_format result = tk.MustQuery(`SELECT DATE_FORMAT('2017-06-15', '%W %M %e %Y %r %y');`) diff --git a/types/time.go b/types/time.go index 7969a867c496e..cb96bb917edda 100644 --- a/types/time.go +++ b/types/time.go @@ -1921,14 +1921,14 @@ func (t Time) convertDateFormat(b rune, buf *bytes.Buffer) error { fmt.Fprintf(buf, "%d", t.Time.Hour()) case 'h', 'I': t := t.Time.Hour() - if t == 0 || t == 12 { + if t%12 == 0 { fmt.Fprintf(buf, "%02d", 12) } else { fmt.Fprintf(buf, "%02d", t%12) } case 'l': t := t.Time.Hour() - if t == 0 || t == 12 { + if t%12 == 0 { fmt.Fprintf(buf, "%d", 12) } else { fmt.Fprintf(buf, "%d", t%12) @@ -1944,6 +1944,7 @@ func (t Time) convertDateFormat(b rune, buf *bytes.Buffer) error { } case 'r': h := t.Time.Hour() + h %= 24 switch { case h == 0: fmt.Fprintf(buf, "%02d:%02d:%02d AM", 12, t.Time.Minute(), t.Time.Second()) From d39053d89e774943bc6e3302042df8f4471f287e Mon Sep 17 00:00:00 2001 From: wjHuang Date: Mon, 25 Mar 2019 14:52:27 +0800 Subject: [PATCH 09/13] *: fix type of maxint64 (#9834) --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 2334fa8c642a6..f46ff3a30c1a2 100644 --- a/go.mod +++ b/go.mod @@ -51,7 +51,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562 github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190321052000-f9a452f8f24e + github.com/pingcap/parser v0.0.0-20190325012055-cc0fa08f99ca github.com/pingcap/pd v2.1.0-rc.4+incompatible github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7 diff --git a/go.sum b/go.sum index 6341b69fed8c0..d483098e0b9d0 100644 --- a/go.sum +++ b/go.sum @@ -119,8 +119,8 @@ github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562 h1:32oF1/8lVnBR2JV github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562/go.mod h1:QMdbTAXCHzzygQzqcG9uVUgU2fKeSN1GmfMiykdSzzY= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190321052000-f9a452f8f24e h1:Evw2H5BmAGqHTKbbcrGXBuOq9I02w3iVn/e7yHR+zvg= -github.com/pingcap/parser v0.0.0-20190321052000-f9a452f8f24e/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190325012055-cc0fa08f99ca h1:ylsmsndeqq4NUE3EvL+TIvZKTlv8Qrth6CFPxDpm570= +github.com/pingcap/parser v0.0.0-20190325012055-cc0fa08f99ca/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v2.1.0-rc.4+incompatible h1:/buwGk04aHO5odk/+O8ZOXGs4qkUjYTJ2UpCJXna8NE= github.com/pingcap/pd v2.1.0-rc.4+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= From 0313cbbbde77f4a2f0aca0e4b8c167082903ab10 Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 25 Mar 2019 15:18:34 +0800 Subject: [PATCH 10/13] plugin: add audit plugin extension point (#9136) --- cmd/pluginpkg/pluginpkg.go | 15 ++- docs/design/2018-12-10-plugin-framework.md | 2 +- executor/adapter.go | 23 +++- executor/set.go | 11 ++ planner/core/planbuilder.go | 21 +++ planner/optimize.go | 2 + plugin/README.md | 3 + plugin/audit.go | 87 ++++++++++++ plugin/conn_ip_example/conn_ip_example.go | 4 +- .../conn_ip_example/conn_ip_example_test.go | 21 ++- plugin/conn_ip_example/manifest.toml | 2 +- plugin/plugin.go | 89 ++++++++----- plugin/spi.go | 28 ++-- plugin/spi_test.go | 5 +- server/conn.go | 56 ++++++-- server/server.go | 124 ++++++++++++++---- session/session.go | 13 +- sessionctx/stmtctx/stmtctx.go | 7 + sessionctx/variable/session.go | 20 +++ util/sys/linux/sys_linux.go | 40 ++++++ util/sys/linux/sys_other.go | 24 ++++ util/sys/linux/sys_test.go | 29 ++++ 22 files changed, 521 insertions(+), 105 deletions(-) create mode 100644 plugin/README.md create mode 100644 plugin/audit.go create mode 100644 util/sys/linux/sys_linux.go create mode 100644 util/sys/linux/sys_other.go create mode 100644 util/sys/linux/sys_test.go diff --git a/cmd/pluginpkg/pluginpkg.go b/cmd/pluginpkg/pluginpkg.go index e1b1db5a3dba3..335a8f5b93a49 100644 --- a/cmd/pluginpkg/pluginpkg.go +++ b/cmd/pluginpkg/pluginpkg.go @@ -62,9 +62,18 @@ func PluginManifest() *plugin.Manifest { }, {{end}} }, - Validate: {{.validate}}, - OnInit: {{.onInit}}, - OnShutdown: {{.onShutdown}}, + {{if .validate }} + Validate: {{.validate}}, + {{end}} + {{if .onInit }} + OnInit: {{.onInit}}, + {{end}} + {{if .onShutdown }} + OnShutdown: {{.onShutdown}}, + {{end}} + {{if .onFlush }} + OnFlush: {{.onFlush}}, + {{end}} }, {{range .export}} {{.extPoint}}: {{.impl}}, diff --git a/docs/design/2018-12-10-plugin-framework.md b/docs/design/2018-12-10-plugin-framework.md index 59b11d7ed8d2d..8d2de6671a9b5 100644 --- a/docs/design/2018-12-10-plugin-framework.md +++ b/docs/design/2018-12-10-plugin-framework.md @@ -140,7 +140,7 @@ validate = "Validate" onInit = "OnInit" onShutdown = "OnShutdown" export = [ - {extPoint="NotifyEvent", impl="NotifyEvent"} + {extPoint="OnGeneralEvent", impl="OnGeneralEvent"} ] ``` diff --git a/executor/adapter.go b/executor/adapter.go index 32f1c1431c398..6a8443ddbd81d 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/chunk" @@ -129,6 +130,7 @@ func (a *recordSet) NewRecordBatch() *chunk.RecordBatch { func (a *recordSet) Close() error { err := a.executor.Close() a.stmt.LogSlowQuery(a.txnStartTS, a.lastErr == nil) + a.stmt.logAudit() return errors.Trace(err) } @@ -295,6 +297,7 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Co } } a.LogSlowQuery(txnTS, err == nil) + a.logAudit() }() err = e.Next(ctx, chunk.NewRecordBatch(e.newFirstChunk())) @@ -362,8 +365,27 @@ func (a *ExecStmt) buildExecutor(ctx sessionctx.Context) (Executor, error) { // QueryReplacer replaces new line and tab for grep result including query string. var QueryReplacer = strings.NewReplacer("\r", " ", "\n", " ", "\t", " ") +func (a *ExecStmt) logAudit() { + sessVars := a.Ctx.GetSessionVars() + if sessVars.InRestrictedSQL { + return + } + err := plugin.ForeachPlugin(plugin.Audit, func(p *plugin.Plugin) error { + audit := plugin.DeclareAuditManifest(p.Manifest) + if audit.OnGeneralEvent != nil { + cmd := mysql.Command2Str[byte(atomic.LoadUint32(&a.Ctx.GetSessionVars().CommandValue))] + audit.OnGeneralEvent(context.Background(), sessVars, plugin.Log, cmd) + } + return nil + }) + if err != nil { + log.Error("log audit log failure", zap.Error(err)) + } +} + // LogSlowQuery is used to print the slow query in the log files. func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { + sessVars := a.Ctx.GetSessionVars() level := log.GetLevel() if level > zapcore.WarnLevel { return @@ -378,7 +400,6 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) { if maxQueryLen := atomic.LoadUint64(&cfg.Log.QueryLogMaxLen); uint64(len(sql)) > maxQueryLen { sql = fmt.Sprintf("%.*q(len:%d)", maxQueryLen, sql, len(a.Text)) } - sessVars := a.Ctx.GetSessionVars() sql = QueryReplacer.Replace(sql) + sessVars.GetExecuteArgumentsInfo() var tableIDs, indexIDs string diff --git a/executor/set.go b/executor/set.go index a5b2a7f6a7d80..43635cc1cacee 100644 --- a/executor/set.go +++ b/executor/set.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -138,6 +139,16 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e if err != nil { return errors.Trace(err) } + err = plugin.ForeachPlugin(plugin.Audit, func(p *plugin.Plugin) error { + auditPlugin := plugin.DeclareAuditManifest(p.Manifest) + if auditPlugin.OnGlobalVariableEvent != nil { + auditPlugin.OnGlobalVariableEvent(context.Background(), e.ctx.GetSessionVars(), name, svalue) + } + return nil + }) + if err != nil { + return err + } } else { // Set session scope system variable. if sysVar.Scope&variable.ScopeSession == 0 { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index ee1f5e640f29f..1a327bd3bd880 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/parser_driver" @@ -159,6 +160,26 @@ func (b *PlanBuilder) GetVisitInfo() []visitInfo { return b.visitInfo } +// GetDBTableInfo gets the accessed dbs and tables info. +func (b *PlanBuilder) GetDBTableInfo() []stmtctx.TableEntry { + var tables []stmtctx.TableEntry + existsFunc := func(tbls []stmtctx.TableEntry, tbl *stmtctx.TableEntry) bool { + for _, t := range tbls { + if t == *tbl { + return true + } + } + return false + } + for _, v := range b.visitInfo { + tbl := &stmtctx.TableEntry{DB: v.db, Table: v.table} + if !existsFunc(tables, tbl) { + tables = append(tables, *tbl) + } + } + return tables +} + // GetOptFlag gets the optFlag of the PlanBuilder. func (b *PlanBuilder) GetOptFlag() uint64 { return b.optFlag diff --git a/planner/optimize.go b/planner/optimize.go index 0f2cfe3a5ba67..c8962916a110e 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -40,6 +40,8 @@ func Optimize(ctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) ( return nil, err } + ctx.GetSessionVars().StmtCtx.Tables = builder.GetDBTableInfo() + // Check privilege. Maybe it's better to move this to the Preprocess, but // we need the table information to check privilege, which is collected // into the visitInfo in the logical plan builder. diff --git a/plugin/README.md b/plugin/README.md new file mode 100644 index 0000000000000..7735c7cd15efa --- /dev/null +++ b/plugin/README.md @@ -0,0 +1,3 @@ +# The Plugin Framework + +https://github.com/pingcap/tidb/blob/master/docs/design/2018-12-10-plugin-framework.md diff --git a/plugin/audit.go b/plugin/audit.go new file mode 100644 index 0000000000000..8ad556495ac62 --- /dev/null +++ b/plugin/audit.go @@ -0,0 +1,87 @@ +// Copyright 2019 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 plugin + +import ( + "context" + + "github.com/pingcap/parser/auth" + "github.com/pingcap/tidb/sessionctx/variable" +) + +// GeneralEvent presents TiDB generate event. +type GeneralEvent byte + +const ( + // Log presents log event. + Log GeneralEvent = iota + // Error presents error event. + Error + // Result presents result event. + Result + // Status presents status event. + Status +) + +// ConnectionEvent presents TiDB connection event. +type ConnectionEvent byte + +const ( + // Connected presents new connection establish event(finish auth). + Connected ConnectionEvent = iota + // Disconnect presents disconnect event. + Disconnect + // ChangeUser presents change user. + ChangeUser + // PreAuth presents event before start auth. + PreAuth +) + +func (c ConnectionEvent) String() string { + switch c { + case Connected: + return "Connected" + case Disconnect: + return "Disconnect" + case ChangeUser: + return "ChangeUser" + case PreAuth: + return "PreAuth" + } + return "" +} + +// ParseEvent presents events happen around parser. +type ParseEvent byte + +const ( + // PreParse presents event before parse. + PreParse ParseEvent = 1 + iota + // PostParse presents event after parse. + PostParse +) + +// AuditManifest presents a sub-manifest that every audit plugin must provide. +type AuditManifest struct { + Manifest + // OnConnectionEvent will be called when TiDB receive or disconnect from client. + // return error will ignore and close current connection. + OnConnectionEvent func(ctx context.Context, identity *auth.UserIdentity, event ConnectionEvent, info *variable.ConnectionInfo) error + // OnGeneralEvent will be called during TiDB execution. + OnGeneralEvent func(ctx context.Context, sctx *variable.SessionVars, event GeneralEvent, cmd string) + // OnGlobalVariableEvent will be called when Change GlobalVariable. + OnGlobalVariableEvent func(ctx context.Context, sctx *variable.SessionVars, varName, varValue string) + // OnParseEvent will be called around parse logic. + OnParseEvent func(ctx context.Context, sctx *variable.SessionVars, event ParseEvent) error +} diff --git a/plugin/conn_ip_example/conn_ip_example.go b/plugin/conn_ip_example/conn_ip_example.go index 30cafec357fed..ebdc33ca5f114 100644 --- a/plugin/conn_ip_example/conn_ip_example.go +++ b/plugin/conn_ip_example/conn_ip_example.go @@ -40,8 +40,8 @@ func OnShutdown(ctx context.Context, manifest *plugin.Manifest) error { return nil } -// NotifyEvent implements TiDB Audit plugin's NotifyEvent SPI. -func NotifyEvent(ctx context.Context) error { +// OnGeneralEvent implements TiDB Audit plugin's OnGeneralEvent SPI. +func OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugin.GeneralEvent, cmd byte, stmt string) error { fmt.Println("conn_ip_example notifiy called") fmt.Println("variable test: ", variable.GetSysVar("conn_ip_example_test_variable").Value) fmt.Printf("new connection by %s\n", ctx.Value("ip")) diff --git a/plugin/conn_ip_example/conn_ip_example_test.go b/plugin/conn_ip_example/conn_ip_example_test.go index 91d73ee1ce293..70422d8ce5ffb 100644 --- a/plugin/conn_ip_example/conn_ip_example_test.go +++ b/plugin/conn_ip_example/conn_ip_example_test.go @@ -30,17 +30,24 @@ func LoadRunShutdownPluginExample() { PluginVarNames: &pluginVarNames, } - err := plugin.Init(ctx, cfg) + err := plugin.Load(ctx, cfg) if err != nil { panic(err) } - ps := plugin.GetByKind(plugin.Audit) - for _, auditPlugin := range ps { - if auditPlugin.State != plugin.Ready { - continue - } - plugin.DeclareAuditManifest(auditPlugin.Manifest).NotifyEvent(context.Background(), nil) + // load and start TiDB domain. + + err = plugin.Init(ctx, cfg) + if err != nil { + panic(err) + } + + err = plugin.ForeachPlugin(plugin.Audit, func(auditPlugin *plugin.Plugin) error { + plugin.DeclareAuditManifest(auditPlugin.Manifest).OnGeneralEvent(context.Background(), nil, plugin.Log, "QUERY") + return nil + }) + if err != nil { + panic(err) } plugin.Shutdown(context.Background()) diff --git a/plugin/conn_ip_example/manifest.toml b/plugin/conn_ip_example/manifest.toml index 8f1a2c74ba7f8..b57badaf689f0 100644 --- a/plugin/conn_ip_example/manifest.toml +++ b/plugin/conn_ip_example/manifest.toml @@ -11,5 +11,5 @@ validate = "Validate" onInit = "OnInit" onShutdown = "OnShutdown" export = [ - {extPoint="NotifyEvent", impl="NotifyEvent"} + {extPoint="OnGeneralEvent", impl="OnGeneralEvent"} ] diff --git a/plugin/plugin.go b/plugin/plugin.go index dfc09cce4fc7a..ee5a7d68c6de6 100644 --- a/plugin/plugin.go +++ b/plugin/plugin.go @@ -149,9 +149,9 @@ func (p *Plugin) validate(ctx context.Context, tiPlugins *plugins, mode validate return nil } -// Init initializes the plugin and load plugin by config param. -// This method isn't thread-safe and must be called before any other plugin operation. -func Init(ctx context.Context, cfg Config) (err error) { +// Load load plugin by config param. +// This method need be called before domain init to inject global variable info during bootstrap. +func Load(ctx context.Context, cfg Config) (err error) { tiPlugins := &plugins{ plugins: make(map[Kind][]Plugin), versions: make(map[string]uint16), @@ -175,6 +175,7 @@ func Init(ctx context.Context, cfg Config) (err error) { _, dup := tiPlugins.versions[pName] if dup { if cfg.SkipWhenFail { + logutil.Logger(ctx).Warn("duplicate load %s and ignored", zap.String("pluginName", pName)) continue } err = errDuplicatePlugin.GenWithStackByArgs(pluginID) @@ -185,6 +186,7 @@ func Init(ctx context.Context, cfg Config) (err error) { plugin, err = loadOne(cfg.PluginDir, ID(pluginID)) if err != nil { if cfg.SkipWhenFail { + logutil.Logger(ctx).Warn("load plugin failure and ignored", zap.String("pluginID", pluginID), zap.Error(err)) continue } return @@ -197,15 +199,8 @@ func Init(ctx context.Context, cfg Config) (err error) { for i := range tiPlugins.plugins[kind] { if err = tiPlugins.plugins[kind][i].validate(ctx, tiPlugins, initMode); err != nil { if cfg.SkipWhenFail { - tiPlugins.plugins[kind][i].State = Disable - err = nil - continue - } - return - } - p := tiPlugins.plugins[kind][i] - if err = p.OnInit(ctx, p.Manifest); err != nil { - if cfg.SkipWhenFail { + logutil.Logger(ctx).Warn("validate plugin fail and disable plugin", + zap.String("plugin", tiPlugins.plugins[kind][i].Name), zap.Error(err)) tiPlugins.plugins[kind][i].State = Disable err = nil continue @@ -220,7 +215,6 @@ func Init(ctx context.Context, cfg Config) (err error) { } } } - tiPlugins.plugins[kind][i].State = Ready } } pluginGlobal = copyOnWriteContext{tiPlugins: unsafe.Pointer(tiPlugins)} @@ -228,30 +222,43 @@ func Init(ctx context.Context, cfg Config) (err error) { return } -// InitWatchLoops starts etcd watch loops for plugin that need watch. -func InitWatchLoops(etcdClient *clientv3.Client) { - if etcdClient == nil { - return - } +// Init initializes the loaded plugin by config param. +// This method must be called after `Load` but before any other plugin method call, so it call got TiDB domain info. +func Init(ctx context.Context, cfg Config) (err error) { tiPlugins := pluginGlobal.plugins() + if tiPlugins == nil { + return nil + } for kind := range tiPlugins.plugins { for i := range tiPlugins.plugins[kind] { - if tiPlugins.plugins[kind][i].OnFlush == nil { - continue + p := tiPlugins.plugins[kind][i] + if err = p.OnInit(ctx, p.Manifest); err != nil { + if cfg.SkipWhenFail { + logutil.Logger(ctx).Warn("call Plugin OnInit failure, err: %v", + zap.String("plugin", p.Name), zap.Error(err)) + tiPlugins.plugins[kind][i].State = Disable + err = nil + continue + } + return } - const pluginWatchPrefix = "/tidb/plugins/" - ctx, cancel := context.WithCancel(context.Background()) - watcher := &flushWatcher{ - ctx: ctx, - cancel: cancel, - path: pluginWatchPrefix + tiPlugins.plugins[kind][i].Name, - etcd: etcdClient, - manifest: tiPlugins.plugins[kind][i].Manifest, + if p.OnFlush != nil && cfg.EtcdClient != nil { + const pluginWatchPrefix = "/tidb/plugins/" + ctx, cancel := context.WithCancel(context.Background()) + watcher := &flushWatcher{ + ctx: ctx, + cancel: cancel, + path: pluginWatchPrefix + tiPlugins.plugins[kind][i].Name, + etcd: cfg.EtcdClient, + manifest: tiPlugins.plugins[kind][i].Manifest, + } + tiPlugins.plugins[kind][i].flushWatcher = watcher + go util.WithRecovery(watcher.watchLoop, nil) } - tiPlugins.plugins[kind][i].flushWatcher = watcher - go util.WithRecovery(watcher.watchLoop, nil) + tiPlugins.plugins[kind][i].State = Ready } } + return } type flushWatcher struct { @@ -326,6 +333,8 @@ func Shutdown(ctx context.Context) { p.flushWatcher.cancel() } if err := p.OnShutdown(ctx, p.Manifest); err != nil { + logutil.Logger(ctx).Error("call OnShutdown for failure", + zap.String("plugin", p.Name), zap.Error(err)) } } } @@ -349,13 +358,23 @@ func Get(kind Kind, name string) *Plugin { return nil } -// GetByKind finds and returns plugin by kind parameters. -func GetByKind(kind Kind) []Plugin { +// ForeachPlugin loops all ready plugins. +func ForeachPlugin(kind Kind, fn func(plugin *Plugin) error) error { plugins := pluginGlobal.plugins() if plugins == nil { return nil } - return plugins.plugins[kind] + for i := range plugins.plugins[kind] { + p := &plugins.plugins[kind][i] + if p.State != Ready { + continue + } + err := fn(p) + if err != nil { + return err + } + } + return nil } // GetAll finds and returns all plugins. @@ -370,8 +389,8 @@ func GetAll() map[Kind][]Plugin { // NotifyFlush notify plugins to do flush logic. func NotifyFlush(dom *domain.Domain, pluginName string) error { p := getByName(pluginName) - if p == nil || p.Manifest.flushWatcher == nil { - return errors.Errorf("plugin %s doesn't exists or unsupported flush", pluginName) + if p == nil || p.Manifest.flushWatcher == nil || p.State != Ready { + return errors.Errorf("plugin %s doesn't exists or unsupported flush or doesn't start with PD", pluginName) } _, err := dom.GetEtcdClient().KV.Put(context.Background(), p.Manifest.flushWatcher.path, "") if err != nil { diff --git a/plugin/spi.go b/plugin/spi.go index 3a77a57562c9f..8be7f6253f52e 100644 --- a/plugin/spi.go +++ b/plugin/spi.go @@ -18,7 +18,6 @@ import ( "reflect" "unsafe" - "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/sessionctx/variable" ) @@ -40,11 +39,21 @@ type Manifest struct { License string BuildTime string SysVars map[string]*variable.SysVar - Validate func(ctx context.Context, manifest *Manifest) error - OnInit func(ctx context.Context, manifest *Manifest) error - OnShutdown func(ctx context.Context, manifest *Manifest) error - OnFlush func(ctx context.Context, manifest *Manifest) error - flushWatcher *flushWatcher + // Validate defines the validate logic for plugin. + // returns error will stop load plugin process and TiDB startup. + Validate func(ctx context.Context, manifest *Manifest) error + // OnInit defines the plugin init logic. + // it will be called after domain init. + // return error will stop load plugin process and TiDB startup. + OnInit func(ctx context.Context, manifest *Manifest) error + // OnShutDown defines the plugin cleanup logic. + // return error will write log and continue shutdown. + OnShutdown func(ctx context.Context, manifest *Manifest) error + // OnFlush defines flush logic after executed `flush tidb plugins`. + // it will be called after OnInit. + // return error will write log and continue watch following flush. + OnFlush func(ctx context.Context, manifest *Manifest) error + flushWatcher *flushWatcher } // ExportManifest exports a manifest to TiDB as a known format. @@ -54,13 +63,6 @@ func ExportManifest(m interface{}) *Manifest { return (*Manifest)(unsafe.Pointer(v.Pointer())) } -// AuditManifest presents a sub-manifest that every audit plugin must provide. -type AuditManifest struct { - Manifest - NotifyEvent func(ctx context.Context, sctx *variable.SessionVars) error - OnConnectionEvent func(ctx context.Context, u *auth.UserIdentity) error -} - // AuthenticationManifest presents a sub-manifest that every audit plugin must provide. type AuthenticationManifest struct { Manifest diff --git a/plugin/spi_test.go b/plugin/spi_test.go index 98e676acfcc3a..efdd8b53802cb 100644 --- a/plugin/spi_test.go +++ b/plugin/spi_test.go @@ -36,15 +36,14 @@ func TestExportManifest(t *testing.T) { return nil }, }, - NotifyEvent: func(ctx context.Context, sctx *variable.SessionVars) error { + OnGeneralEvent: func(ctx context.Context, sctx *variable.SessionVars, event plugin.GeneralEvent, cmd string) { callRecorder.NotifyEventCalled = true - return nil }, } exported := plugin.ExportManifest(manifest) exported.OnInit(context.Background(), exported) audit := plugin.DeclareAuditManifest(exported) - audit.NotifyEvent(context.Background(), nil) + audit.OnGeneralEvent(context.Background(), nil, plugin.Log, "QUERY") if !callRecorder.NotifyEventCalled || !callRecorder.OnInitCalled { t.Fatalf("export test failure") } diff --git a/server/conn.go b/server/conn.go index cac8f93d2c0d7..dba6ef82a0ffd 100644 --- a/server/conn.go +++ b/server/conn.go @@ -49,7 +49,7 @@ import ( "sync/atomic" "time" - opentracing "github.com/opentracing/opentracing-go" + "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/mysql" @@ -57,6 +57,7 @@ import ( "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/arena" @@ -103,6 +104,9 @@ type clientConn struct { ctx QueryCtx // an interface to execute sql statements. attrs map[string]string // attributes parsed from client handshake response, not used for now. status int32 // dispatching/reading/shutdown/waitshutdown + peerHost string // peer host + peerPort string // peer port + lastCode uint16 // last error code // mu is used for cancelling the execution of current transaction. mu struct { @@ -497,18 +501,13 @@ func (cc *clientConn) openSessionAndDoAuth(authData []byte) error { if err != nil { return errors.Trace(err) } - host := variable.DefHostname hasPassword := "YES" if len(authData) == 0 { hasPassword = "NO" } - if !cc.server.isUnixSocket() { - addr := cc.bufReadConn.RemoteAddr().String() - // Do Auth. - host, _, err = net.SplitHostPort(addr) - if err != nil { - return errors.Trace(errAccessDenied.GenWithStackByArgs(cc.user, addr, hasPassword)) - } + host, err := cc.PeerHost(hasPassword) + if err != nil { + return err } if !cc.ctx.Auth(&auth.UserIdentity{Username: cc.user, Hostname: host}, authData, cc.salt) { return errors.Trace(errAccessDenied.GenWithStackByArgs(cc.user, host, hasPassword)) @@ -523,6 +522,27 @@ func (cc *clientConn) openSessionAndDoAuth(authData []byte) error { return nil } +func (cc *clientConn) PeerHost(hasPassword string) (host string, err error) { + if len(cc.peerHost) > 0 { + return cc.peerHost, nil + } + host = variable.DefHostname + if cc.server.isUnixSocket() { + cc.peerHost = host + return + } + addr := cc.bufReadConn.RemoteAddr().String() + var port string + host, port, err = net.SplitHostPort(addr) + if err != nil { + err = errAccessDenied.GenWithStackByArgs(cc.user, addr, hasPassword) + return + } + cc.peerHost = host + cc.peerPort = port + return +} + // Run reads client query and writes query result to client in for loop, if there is a panic during query handling, // it will be recovered and log the panic error. // This function returns and the connection is closed if there is an IO error or there is a panic. @@ -837,6 +857,7 @@ func (cc *clientConn) writeError(e error) error { m = mysql.NewErrf(mysql.ErrUnknown, "%s", e.Error()) } + cc.lastCode = m.Code data := cc.alloc.AllocWithLen(4, 16+len(m.Message)) data = append(data, mysql.ErrHeader) data = append(data, byte(m.Code), byte(m.Code>>8)) @@ -1292,9 +1313,26 @@ func (cc *clientConn) handleChangeUser(ctx context.Context, data []byte) error { if err != nil { logutil.Logger(ctx).Debug("close old context error", zap.Error(err)) } + err = cc.openSessionAndDoAuth(pass) if err != nil { return errors.Trace(err) } + + err = plugin.ForeachPlugin(plugin.Audit, func(p *plugin.Plugin) error { + authPlugin := plugin.DeclareAuditManifest(p.Manifest) + if authPlugin.OnConnectionEvent != nil { + connInfo := cc.connectInfo() + err = authPlugin.OnConnectionEvent(context.Background(), &auth.UserIdentity{Hostname: connInfo.Host}, plugin.ChangeUser, connInfo) + if err != nil { + return errors.Trace(err) + } + } + return nil + }) + if err != nil { + return err + } + return cc.writeOK() } diff --git a/server/server.go b/server/server.go index 5c11bc668a26d..a114a43c58d7c 100644 --- a/server/server.go +++ b/server/server.go @@ -38,11 +38,13 @@ import ( "math/rand" "net" "net/http" - // For pprof - _ "net/http/pprof" + "os" + "os/user" "sync" "sync/atomic" "time" + // For pprof + _ "net/http/pprof" "github.com/blacktear23/go-proxyprotocol" "github.com/pingcap/errors" @@ -55,14 +57,32 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sys/linux" log "github.com/sirupsen/logrus" "go.uber.org/zap" ) var ( baseConnID uint32 + serverPID int + osUser string + osVersion string ) +func init() { + serverPID = os.Getpid() + currentUser, err := user.Current() + if err != nil { + osUser = "" + } else { + osUser = currentUser.Name + } + osVersion, err = linux.OSVersion() + if err != nil { + osVersion = "" + } +} + var ( errUnknownFieldType = terror.ClassServer.New(codeUnknownFieldType, "unknown field type") errInvalidPayloadLen = terror.ClassServer.New(codeInvalidPayloadLen, "invalid payload length") @@ -311,26 +331,31 @@ func (s *Server) Run() error { break } - for _, p := range plugin.GetByKind(plugin.Audit) { + clientConn := s.newConn(conn) + + err = plugin.ForeachPlugin(plugin.Audit, func(p *plugin.Plugin) error { authPlugin := plugin.DeclareAuditManifest(p.Manifest) if authPlugin.OnConnectionEvent != nil { - host, err := getPeerHost(conn) + host, err := clientConn.PeerHost("") if err != nil { - log.Error(err) - terror.Log(conn.Close()) - continue + log.Info(err) + terror.Log(clientConn.Close()) + return errors.Trace(err) } - - err = authPlugin.OnConnectionEvent(context.Background(), &auth.UserIdentity{Hostname: host}) + err = authPlugin.OnConnectionEvent(context.Background(), &auth.UserIdentity{Hostname: host}, plugin.PreAuth, nil) if err != nil { log.Info(err) - terror.Log(conn.Close()) - continue + terror.Log(clientConn.Close()) + return errors.Trace(err) } } + return nil + }) + if err != nil { + return err } - go s.onConn(conn) + go s.onConn(clientConn) } err := s.listener.Close() terror.Log(errors.Trace(err)) @@ -342,15 +367,6 @@ func (s *Server) Run() error { } } -func getPeerHost(conn net.Conn) (string, error) { - addr := conn.RemoteAddr().String() - host, _, err := net.SplitHostPort(addr) - if err != nil { - return "", errors.Trace(err) - } - return host, nil -} - func (s *Server) shouldStopListener() bool { select { case <-s.stopListenerCh: @@ -384,18 +400,19 @@ func (s *Server) Close() { } // onConn runs in its own goroutine, handles queries from this connection. -func (s *Server) onConn(c net.Conn) { - conn := s.newConn(c) +func (s *Server) onConn(conn *clientConn) { ctx := logutil.WithConnID(context.Background(), conn.connectionID) if err := conn.handshake(ctx); err != nil { // Some keep alive services will send request to TiDB and disconnect immediately. // So we only record metrics. metrics.HandShakeErrorCounter.Inc() - err = c.Close() + err = conn.Close() terror.Log(errors.Trace(err)) return } - logutil.Logger(ctx).Info("new connection", zap.String("remoteAddr", c.RemoteAddr().String())) + + logutil.Logger(ctx).Info("new connection", zap.String("remoteAddr", conn.bufReadConn.RemoteAddr().String())) + defer func() { logutil.Logger(ctx).Info("close connection") }() @@ -405,7 +422,64 @@ func (s *Server) onConn(c net.Conn) { s.rwlock.Unlock() metrics.ConnGauge.Set(float64(connections)) + err := plugin.ForeachPlugin(plugin.Audit, func(p *plugin.Plugin) error { + authPlugin := plugin.DeclareAuditManifest(p.Manifest) + if authPlugin.OnConnectionEvent != nil { + connInfo := conn.connectInfo() + return authPlugin.OnConnectionEvent(context.Background(), conn.ctx.GetSessionVars().User, plugin.Connected, connInfo) + } + return nil + }) + if err != nil { + return + } + + connectedTime := time.Now() conn.Run(ctx) + + err = plugin.ForeachPlugin(plugin.Audit, func(p *plugin.Plugin) error { + authPlugin := plugin.DeclareAuditManifest(p.Manifest) + if authPlugin.OnConnectionEvent != nil { + connInfo := conn.connectInfo() + connInfo.Duration = float64(time.Since(connectedTime)) / float64(time.Millisecond) + err := authPlugin.OnConnectionEvent(context.Background(), conn.ctx.GetSessionVars().User, plugin.Disconnect, connInfo) + if err != nil { + log.Warnf("call Plugin %s OnConnectionEvent(Disconnect) failure, err: %v", authPlugin.Name, err) + } + } + return nil + }) + if err != nil { + return + } +} + +func (cc *clientConn) connectInfo() *variable.ConnectionInfo { + connType := "Socket" + if cc.server.isUnixSocket() { + connType = "UnixSocket" + } else if cc.tlsConn != nil { + connType = "SSL/TLS" + } + connInfo := &variable.ConnectionInfo{ + ConnectionID: cc.connectionID, + ConnectionType: connType, + Host: cc.peerHost, + ClientIP: cc.peerHost, + ClientPort: cc.peerPort, + ServerID: 1, + ServerPort: int(cc.server.cfg.Port), + Duration: 0, + User: cc.user, + ServerOSLoginUser: osUser, + OSVersion: osVersion, + ClientVersion: "", + ServerVersion: mysql.TiDBReleaseVersion, + SSLVersion: "v1.2.0", // for current go version + PID: serverPID, + DB: cc.dbname, + } + return connInfo } // ShowProcessList implements the SessionManager interface. diff --git a/session/session.go b/session/session.go index 9f1f6b5e4b00f..489990bb6f2d9 100644 --- a/session/session.go +++ b/session/session.go @@ -1304,7 +1304,7 @@ func loadSystemTZ(se *session) (string, error) { func BootstrapSession(store kv.Storage) (*domain.Domain, error) { cfg := config.GetGlobalConfig() if len(cfg.Plugin.Load) > 0 { - err := plugin.Init(context.Background(), plugin.Config{ + err := plugin.Load(context.Background(), plugin.Config{ Plugins: strings.Split(cfg.Plugin.Load, ","), PluginDir: cfg.Plugin.Dir, GlobalSysVar: &variable.SysVars, @@ -1344,6 +1344,13 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { } } + if len(cfg.Plugin.Load) > 0 { + err := plugin.Init(context.Background(), plugin.Config{EtcdClient: dom.GetEtcdClient()}) + if err != nil { + return nil, errors.Trace(err) + } + } + se1, err := createSession(store) if err != nil { return nil, errors.Trace(err) @@ -1361,10 +1368,6 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { return nil, errors.Trace(err) } - if len(cfg.Plugin.Load) > 0 { - plugin.InitWatchLoops(dom.GetEtcdClient()) - } - if raw, ok := store.(domain.EtcdBackend); ok { err = raw.StartGCWorker() if err != nil { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index e999a2921b6fe..de6cd9fb1857e 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -122,6 +122,7 @@ type StatementContext struct { normalized string digest string } + Tables []TableEntry } // SQLDigest gets normalized and digest for provided sql. @@ -133,6 +134,12 @@ func (sc *StatementContext) SQLDigest() (normalized, sqlDigest string) { return sc.digestMemo.normalized, sc.digestMemo.digest } +// TableEntry presents table in db. +type TableEntry struct { + DB string + Table string +} + // AddAffectedRows adds affected rows. func (sc *StatementContext) AddAffectedRows(rows uint64) { sc.mu.Lock() diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index bbffa19c577d6..36f0cec8d196f 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -342,6 +342,26 @@ type SessionVars struct { SlowQueryFile string } +// ConnectionInfo present connection used by audit. +type ConnectionInfo struct { + ConnectionID uint32 + ConnectionType string + Host string + ClientIP string + ClientPort string + ServerID int + ServerPort int + Duration float64 + User string + ServerOSLoginUser string + OSVersion string + ClientVersion string + ServerVersion string + SSLVersion string + PID int + DB string +} + // NewSessionVars creates a session vars object. func NewSessionVars() *SessionVars { vars := &SessionVars{ diff --git a/util/sys/linux/sys_linux.go b/util/sys/linux/sys_linux.go new file mode 100644 index 0000000000000..c53bf77b31681 --- /dev/null +++ b/util/sys/linux/sys_linux.go @@ -0,0 +1,40 @@ +// Copyright 2019 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. +// +build linux + +package linux + +import "syscall" + +// OSVersion returns version info of operation system. +// e.g. Linux 4.15.0-45-generic.x86_64 +func OSVersion() (osVersion string, err error) { + var un syscall.Utsname + err = syscall.Uname(&un) + if err != nil { + return + } + charsToString := func(ca []int8) string { + s := make([]byte, len(ca)) + var lens int + for ; lens < len(ca); lens++ { + if ca[lens] == 0 { + break + } + s[lens] = uint8(ca[lens]) + } + return string(s[0:lens]) + } + osVersion = charsToString(un.Sysname[:]) + " " + charsToString(un.Release[:]) + "." + charsToString(un.Machine[:]) + return +} diff --git a/util/sys/linux/sys_other.go b/util/sys/linux/sys_other.go new file mode 100644 index 0000000000000..98b4eae9749f9 --- /dev/null +++ b/util/sys/linux/sys_other.go @@ -0,0 +1,24 @@ +// Copyright 2019 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. +// +build !linux + +package linux + +import "runtime" + +// OSVersion returns version info of operation system. +// for non-linux system will only return os and arch info. +func OSVersion() (osVersion string, err error) { + osVersion = runtime.GOOS + "." + runtime.GOARCH + return +} diff --git a/util/sys/linux/sys_test.go b/util/sys/linux/sys_test.go new file mode 100644 index 0000000000000..b9ccc2a823bec --- /dev/null +++ b/util/sys/linux/sys_test.go @@ -0,0 +1,29 @@ +// Copyright 2019 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 linux_test + +import ( + "testing" + + "github.com/pingcap/tidb/util/sys/linux" +) + +func TestGetOSVersion(t *testing.T) { + osRelease, err := linux.OSVersion() + if err != nil { + t.Fatal(t) + } + if len(osRelease) == 0 { + t.Fatalf("counld not get os version") + } +} From 1154456af7e5824954d25792bfb2cf700a48021d Mon Sep 17 00:00:00 2001 From: Keyi Xie Date: Mon, 25 Mar 2019 15:50:15 +0800 Subject: [PATCH 11/13] domain,owner: unify and normalize the format of the log (#9646) --- domain/domain.go | 90 ++++++++++++++++++++------------------ domain/info.go | 10 +++-- domain/schema_validator.go | 24 +++++----- owner/manager.go | 48 +++++++++++--------- 4 files changed, 94 insertions(+), 78 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 2c75b48260262..f1d0207c26cef 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -47,7 +47,6 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" - log "github.com/sirupsen/logrus" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/keepalive" @@ -96,12 +95,13 @@ func (do *Domain) loadInfoSchema(handle *infoschema.Handle, usedSchemaVersion in // Update self schema version to etcd. defer func() { if err != nil { - log.Info("[ddl] not update self schema version to etcd") + logutil.Logger(context.Background()).Info("cannot update self schema version to etcd") return } err = do.ddl.SchemaSyncer().UpdateSelfVersion(context.Background(), latestSchemaVersion) if err != nil { - log.Infof("[ddl] update self version from %v to %v failed %v", usedSchemaVersion, latestSchemaVersion, err) + logutil.Logger(context.Background()).Info("update self version failed", zap.Int64("usedSchemaVersion", usedSchemaVersion), + zap.Int64("latestSchemaVersion", latestSchemaVersion), zap.Error(err)) } }() @@ -109,11 +109,14 @@ func (do *Domain) loadInfoSchema(handle *infoschema.Handle, usedSchemaVersion in ok, tblIDs, err := do.tryLoadSchemaDiffs(m, usedSchemaVersion, latestSchemaVersion) if err != nil { // We can fall back to full load, don't need to return the error. - log.Errorf("[ddl] failed to load schema diff err %v", err) + logutil.Logger(context.Background()).Error("failed to load schema diff", zap.Error(err)) } if ok { - log.Infof("[ddl] diff load InfoSchema from version %d to %d in %v, tableIDs %v", - usedSchemaVersion, latestSchemaVersion, time.Since(startTime), tblIDs) + logutil.Logger(context.Background()).Info("diff load InfoSchema from version failed", + zap.Int64("usedSchemaVersion", usedSchemaVersion), + zap.Int64("latestSchemaVersion", latestSchemaVersion), + zap.Duration("start time", time.Since(startTime)), + zap.Int64s("tblIDs", tblIDs)) return latestSchemaVersion, tblIDs, fullLoad, nil } @@ -127,8 +130,8 @@ func (do *Domain) loadInfoSchema(handle *infoschema.Handle, usedSchemaVersion in if err != nil { return 0, nil, fullLoad, errors.Trace(err) } - log.Infof("[ddl] full load InfoSchema from version %d to %d, in %v", - usedSchemaVersion, latestSchemaVersion, time.Since(startTime)) + logutil.Logger(context.Background()).Info("full load InfoSchema failed", zap.Int64("usedSchemaVersion", usedSchemaVersion), + zap.Int64("latestSchemaVersion", latestSchemaVersion), zap.Duration("start time", time.Since(startTime))) newISBuilder.Build() return latestSchemaVersion, nil, fullLoad, nil } @@ -329,7 +332,7 @@ func (do *Domain) Reload() error { metrics.LoadSchemaCounter.WithLabelValues("succ").Inc() if fullLoad { - log.Info("[ddl] full load and reset schema validator.") + logutil.Logger(context.Background()).Info("full load and reset schema validator") do.SchemaValidator.Reset() } do.SchemaValidator.Update(ver.Ver, schemaVersion, latestSchemaVersion, changedTableIDs) @@ -339,7 +342,7 @@ func (do *Domain) Reload() error { // Reload interval is lease / 2, if load schema time elapses more than this interval, // some query maybe responded by ErrInfoSchemaExpired error. if sub > (lease/2) && lease > 0 { - log.Warnf("[ddl] loading schema takes a long time %v", sub) + logutil.Logger(context.Background()).Warn("loading schema takes a long time", zap.Duration("take time", sub)) } return nil @@ -405,11 +408,11 @@ func (do *Domain) infoSyncerKeeper() { for { select { case <-do.info.Done(): - log.Info("[ddl] server info syncer need to restart") + logutil.Logger(context.Background()).Info("server info syncer need to restart") if err := do.info.Restart(context.Background()); err != nil { - log.Error(err) + logutil.Logger(context.Background()).Error("server restart failed", zap.Error(err)) } - log.Info("[ddl] server info syncer restarted.") + logutil.Logger(context.Background()).Info("server info syncer restarted") case <-do.exit: return } @@ -430,21 +433,21 @@ func (do *Domain) loadSchemaInLoop(lease time.Duration) { case <-ticker.C: err := do.Reload() if err != nil { - log.Errorf("[ddl] reload schema in loop err %v", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Error("reload schema in loop failed", zap.Error(err)) } case _, ok := <-syncer.GlobalVersionCh(): err := do.Reload() if err != nil { - log.Errorf("[ddl] reload schema in loop err %v", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Error("reload schema in loop failed", zap.Error(err)) } if !ok { - log.Warn("[ddl] reload schema in loop, schema syncer need rewatch") + logutil.Logger(context.Background()).Warn("reload schema in loop, schema syncer need rewatch") // Make sure the rewatch doesn't affect load schema, so we watch the global schema version asynchronously. syncer.WatchGlobalSchemaVer(context.Background()) } case <-syncer.Done(): // The schema syncer stops, we need stop the schema validator to synchronize the schema version. - log.Info("[ddl] reload schema in loop, schema syncer need restart") + logutil.Logger(context.Background()).Info("reload schema in loop, schema syncer need restart") // The etcd is responsible for schema synchronization, we should ensure there is at most two different schema version // in the TiDB cluster, to make the data/schema be consistent. If we lost connection/session to etcd, the cluster // will treats this TiDB as a down instance, and etcd will remove the key of `/tidb/ddl/all_schema_versions/tidb-id`. @@ -454,18 +457,18 @@ func (do *Domain) loadSchemaInLoop(lease time.Duration) { do.SchemaValidator.Stop() err := do.mustRestartSyncer() if err != nil { - log.Errorf("[ddl] reload schema in loop, schema syncer restart err %v", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Error("reload schema in loop, schema syncer restart failed", zap.Error(err)) break } // The schema maybe changed, must reload schema then the schema validator can restart. exitLoop := do.mustReload() if exitLoop { // domain is closed. - log.Errorf("[ddl] domain is closed. exit loadSchemaInLoop") + logutil.Logger(context.Background()).Error("domain is closed, exit loadSchemaInLoop") return } do.SchemaValidator.Restart() - log.Info("[ddl] schema syncer restarted.") + logutil.Logger(context.Background()).Info("schema syncer restarted") case <-do.exit: return } @@ -490,7 +493,7 @@ func (do *Domain) mustRestartSyncer() error { default: } time.Sleep(time.Second) - log.Infof("[ddl] restart the schema syncer failed %v", err) + logutil.Logger(context.Background()).Info("restart the schema syncer failed", zap.Error(err)) } } @@ -500,15 +503,15 @@ func (do *Domain) mustReload() (exitLoop bool) { for { err := do.Reload() if err == nil { - log.Infof("[ddl] mustReload succeed.") + logutil.Logger(context.Background()).Info("mustReload succeed") return false } - log.Infof("[ddl] reload the schema failed: %v", err) + logutil.Logger(context.Background()).Info("reload the schema failed", zap.Error(err)) // If the domain is closed, we returns immediately. select { case <-do.exit: - log.Infof("[ddl] domain is closed.") + logutil.Logger(context.Background()).Info("domain is closed") return true default: } @@ -532,7 +535,7 @@ func (do *Domain) Close() { do.sysSessionPool.Close() do.slowQuery.Close() do.wg.Wait() - log.Info("[domain] close") + logutil.Logger(context.Background()).Info("domain closed") } type ddlCallback struct { @@ -544,11 +547,11 @@ func (c *ddlCallback) OnChanged(err error) error { if err != nil { return err } - log.Infof("[ddl] on DDL change, must reload") + logutil.Logger(context.Background()).Info("performing DDL change, must reload") err = c.do.Reload() if err != nil { - log.Errorf("[ddl] on DDL change reload err %v", err) + logutil.Logger(context.Background()).Error("performing DDL change failed", zap.Error(err)) } return nil @@ -748,7 +751,7 @@ func (do *Domain) LoadPrivilegeLoop(ctx sessionctx.Context) error { case <-time.After(duration): } if !ok { - log.Error("[domain] load privilege loop watch channel closed.") + logutil.Logger(context.Background()).Error("load privilege loop watch channel closed") watchCh = do.etcdClient.Watch(context.Background(), privilegeKey) count++ if count > 10 { @@ -761,9 +764,9 @@ func (do *Domain) LoadPrivilegeLoop(ctx sessionctx.Context) error { err := do.privHandle.Update(ctx) metrics.LoadPrivilegeCounter.WithLabelValues(metrics.RetLabel(err)).Inc() if err != nil { - log.Error("[domain] load privilege fail:", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Error("load privilege failed", zap.Error(err)) } else { - log.Debug("[domain] reload privilege success.") + logutil.Logger(context.Background()).Debug("reload privilege success") } } }() @@ -873,7 +876,7 @@ func (do *Domain) newStatsOwner() owner.Manager { // TODO: Need to do something when err is not nil. err := statsOwner.CampaignOwner(cancelCtx) if err != nil { - log.Warnf("[stats] campaign owner fail: %s", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Warn("campaign owner failed", zap.Error(err)) } return statsOwner } @@ -898,9 +901,9 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) t := time.Now() err := statsHandle.InitStats(do.InfoSchema()) if err != nil { - log.Debug("[stats] init stats info failed: ", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Debug("init stats info failed", zap.Error(err)) } else { - log.Info("[stats] init stats info takes ", time.Since(t)) + logutil.Logger(context.Background()).Info("init stats info time", zap.Duration("take time", time.Since(t))) } defer func() { do.SetStatsUpdating(false) @@ -911,7 +914,7 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) case <-loadTicker.C: err = statsHandle.Update(do.InfoSchema()) if err != nil { - log.Debug("[stats] update stats info fail: ", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Debug("update stats info failed", zap.Error(err)) } case <-do.exit: statsHandle.FlushStats() @@ -920,18 +923,18 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) case t := <-statsHandle.DDLEventCh(): err = statsHandle.HandleDDLEvent(t) if err != nil { - log.Debug("[stats] handle ddl event fail: ", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Debug("handle ddl event failed", zap.Error(err)) } case <-deltaUpdateTicker.C: err = statsHandle.DumpStatsDeltaToKV(statistics.DumpDelta) if err != nil { - log.Debug("[stats] dump stats delta fail: ", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Debug("dump stats delta failed", zap.Error(err)) } statsHandle.UpdateErrorRate(do.InfoSchema()) case <-loadHistogramTicker.C: err = statsHandle.LoadNeededHistograms() if err != nil { - log.Debug("[stats] load histograms fail: ", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Debug("load histograms failed", zap.Error(err)) } case <-loadFeedbackTicker.C: statsHandle.UpdateStatsByLocalFeedback(do.InfoSchema()) @@ -940,12 +943,12 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) } err = statsHandle.HandleUpdateStats(do.InfoSchema()) if err != nil { - log.Debug("[stats] update stats using feedback fail: ", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Debug("update stats using feedback failed", zap.Error(err)) } case <-dumpFeedbackTicker.C: err = statsHandle.DumpStatsFeedbackToKV() if err != nil { - log.Debug("[stats] dump stats feedback fail: ", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Debug("dump stats feedback failed", zap.Error(err)) } case <-gcStatsTicker.C: if !owner.IsOwner() { @@ -953,7 +956,7 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) } err = statsHandle.GCStats(do.InfoSchema(), do.DDL().GetLease()) if err != nil { - log.Debug("[stats] gc stats fail: ", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Debug("GC stats failed", zap.Error(err)) } } } @@ -988,13 +991,13 @@ func (do *Domain) NotifyUpdatePrivilege(ctx sessionctx.Context) { row := do.etcdClient.KV _, err := row.Put(context.Background(), privilegeKey, "") if err != nil { - log.Warn("notify update privilege failed:", err) + logutil.Logger(context.Background()).Warn("notify update privilege failed", zap.Error(err)) } } // update locally _, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, `FLUSH PRIVILEGES`) if err != nil { - log.Errorf("Unable to update privileges: %s", err) + logutil.Logger(context.Background()).Error("unable to update privileges", zap.Error(err)) } } @@ -1004,7 +1007,8 @@ func recoverInDomain(funcName string, quit bool) { return } buf := util.GetStack() - log.Errorf("%s, %v, %s", funcName, r, buf) + logutil.Logger(context.Background()).Error("recover in domain failed", zap.String("funcName", funcName), + zap.Any("error", r), zap.String("buffer", string(buf))) metrics.PanicCounter.WithLabelValues(metrics.LabelDomain).Inc() if quit { // Wait for metrics to be pushed. diff --git a/domain/info.go b/domain/info.go index cfb30b7655047..404aa1088283e 100644 --- a/domain/info.go +++ b/domain/info.go @@ -27,8 +27,9 @@ import ( "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/printer" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) const ( @@ -139,7 +140,7 @@ func (is *InfoSyncer) RemoveServerInfo() { } err := ddl.DeleteKeyFromEtcd(is.serverInfoPath, is.etcdCli, keyOpDefaultRetryCnt, keyOpDefaultTimeout) if err != nil { - log.Errorf("[info-syncer] remove server info failed %v", err) + logutil.Logger(context.Background()).Error("remove server info failed", zap.Error(err)) } } @@ -188,7 +189,7 @@ func getInfo(ctx context.Context, etcdCli *clientv3.Client, key string, retryCnt resp, err = etcdCli.Get(childCtx, key, opts...) cancel() if err != nil { - log.Infof("[info-syncer] get %s failed %v, continue checking.", key, err) + logutil.Logger(context.Background()).Info("get key failed", zap.String("key", key), zap.Error(err)) time.Sleep(200 * time.Millisecond) continue } @@ -196,7 +197,8 @@ func getInfo(ctx context.Context, etcdCli *clientv3.Client, key string, retryCnt info := &ServerInfo{} err = json.Unmarshal(kv.Value, info) if err != nil { - log.Infof("[info-syncer] get %s, json.Unmarshal %v failed %v.", kv.Key, kv.Value, err) + logutil.Logger(context.Background()).Info("get key failed", zap.String("key", string(kv.Key)), zap.ByteString("value", kv.Value), + zap.Error(err)) return nil, errors.Trace(err) } allInfo[info.ID] = info diff --git a/domain/schema_validator.go b/domain/schema_validator.go index f99bbc392d401..53a9bf4977857 100644 --- a/domain/schema_validator.go +++ b/domain/schema_validator.go @@ -14,11 +14,13 @@ package domain import ( + "context" "sync" "time" "github.com/pingcap/tidb/store/tikv/oracle" - log "github.com/sirupsen/logrus" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" ) type checkResult int @@ -83,7 +85,7 @@ func (s *schemaValidator) IsStarted() bool { } func (s *schemaValidator) Stop() { - log.Info("[domain-ddl] the schema validator stops") + logutil.Logger(context.Background()).Info("the schema validator stops") s.mux.Lock() defer s.mux.Unlock() s.isStarted = false @@ -92,7 +94,7 @@ func (s *schemaValidator) Stop() { } func (s *schemaValidator) Restart() { - log.Info("[domain-ddl] the schema validator restarts") + logutil.Logger(context.Background()).Info("the schema validator restarts") s.mux.Lock() defer s.mux.Unlock() s.isStarted = true @@ -111,7 +113,7 @@ func (s *schemaValidator) Update(leaseGrantTS uint64, oldVer, currVer int64, cha defer s.mux.Unlock() if !s.isStarted { - log.Infof("[domain-ddl] the schema validator stopped before updating") + logutil.Logger(context.Background()).Info("the schema validator stopped before updating") return } @@ -123,7 +125,8 @@ func (s *schemaValidator) Update(leaseGrantTS uint64, oldVer, currVer int64, cha // Update the schema deltaItem information. if currVer != oldVer { - log.Debugf("[domain-ddl] update schema validator, old ver %d, curr ver %d, changed IDs %v", oldVer, currVer, changedTableIDs) + logutil.Logger(context.Background()).Debug("update schema validator", zap.Int64("oldVer", oldVer), + zap.Int64("currVer", currVer), zap.Int64s("changedTableIDs", changedTableIDs)) s.enqueue(currVer, changedTableIDs) } } @@ -144,12 +147,13 @@ func hasRelatedTableID(relatedTableIDs, updateTableIDs []int64) bool { // NOTE, this function should be called under lock! func (s *schemaValidator) isRelatedTablesChanged(currVer int64, tableIDs []int64) bool { if len(s.deltaSchemaInfos) == 0 { - log.Infof("[domain-ddl] schema change history is empty, checking %d", currVer) + logutil.Logger(context.Background()).Info("schema change history is empty", zap.Int64("currVer", currVer)) return true } newerDeltas := s.findNewerDeltas(currVer) if len(newerDeltas) == len(s.deltaSchemaInfos) { - log.Infof("[domain-ddl] the schema version %d is much older than the latest version %d", currVer, s.latestSchemaVer) + logutil.Logger(context.Background()).Info("the schema version is much older than the latest version", zap.Int64("currVer", currVer), + zap.Int64("latestSchemaVer", s.latestSchemaVer)) return true } for _, item := range newerDeltas { @@ -174,7 +178,7 @@ func (s *schemaValidator) Check(txnTS uint64, schemaVer int64, relatedTableIDs [ s.mux.RLock() defer s.mux.RUnlock() if !s.isStarted { - log.Infof("[domain-ddl] the schema validator stopped before checking") + logutil.Logger(context.Background()).Info("the schema validator stopped before checking") return ResultUnknown } if s.lease == 0 { @@ -185,8 +189,8 @@ func (s *schemaValidator) Check(txnTS uint64, schemaVer int64, relatedTableIDs [ if schemaVer < s.latestSchemaVer { // The DDL relatedTableIDs is empty. if len(relatedTableIDs) == 0 { - log.Infof("[domain-ddl] the related table ID is empty, current schema version %d, latest schema version %d", - schemaVer, s.latestSchemaVer) + logutil.Logger(context.Background()).Info("the related table ID is empty", zap.Int64("schemaVer", schemaVer), + zap.Int64("latestSchemaVer", s.latestSchemaVer)) return ResultFail } diff --git a/owner/manager.go b/owner/manager.go index 60610433ca63e..0957bc063e3c6 100644 --- a/owner/manager.go +++ b/owner/manager.go @@ -31,7 +31,8 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/util" - log "github.com/sirupsen/logrus" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" "google.golang.org/grpc" ) @@ -78,6 +79,7 @@ type ownerManager struct { key string prompt string logPrefix string + logCtx context.Context etcdCli *clientv3.Client cancel context.CancelFunc elec unsafe.Pointer @@ -85,13 +87,15 @@ type ownerManager struct { // NewOwnerManager creates a new Manager. func NewOwnerManager(etcdCli *clientv3.Client, prompt, id, key string, cancel context.CancelFunc) Manager { + logPrefix := fmt.Sprintf("[%s] %s ownerManager %s", prompt, key, id) return &ownerManager{ etcdCli: etcdCli, id: id, key: key, prompt: prompt, cancel: cancel, - logPrefix: fmt.Sprintf("[%s] %s ownerManager %s", prompt, key, id), + logPrefix: logPrefix, + logCtx: logutil.WithKeyValue(context.Background(), "owner info", logPrefix), } } @@ -155,7 +159,7 @@ func NewSession(ctx context.Context, logPrefix string, etcdCli *clientv3.Client, break } if failedCnt%logIntervalCnt == 0 { - log.Warnf("%s failed to new session to etcd, err %v", logPrefix, err) + logutil.Logger(context.Background()).Warn("failed to new session to etcd", zap.String("ownerInfo", logPrefix), zap.Error(err)) } time.Sleep(newSessionRetryInterval) @@ -189,7 +193,7 @@ func (m *ownerManager) ResignOwner(ctx context.Context) error { return errors.Trace(err) } - log.Warnf("%s Resign ddl owner success!", m.logPrefix) + logutil.Logger(m.logCtx).Warn("resign ddl owner success") return nil } @@ -209,12 +213,13 @@ func (m *ownerManager) campaignLoop(ctx context.Context, etcdSession *concurrenc cancel() if r := recover(); r != nil { buf := util.GetStack() - log.Errorf("[%s] recover panic:%v, %s", m.prompt, r, buf) + logutil.Logger(context.Background()).Error("recover panic", zap.String("prompt", m.prompt), zap.Any("error", r), zap.String("buffer", string(buf))) metrics.PanicCounter.WithLabelValues(metrics.LabelDDLOwner).Inc() } }() logPrefix := m.logPrefix + logCtx := m.logCtx var err error for { if err != nil { @@ -223,11 +228,11 @@ func (m *ownerManager) campaignLoop(ctx context.Context, etcdSession *concurrenc select { case <-etcdSession.Done(): - log.Infof("%s etcd session is done, creates a new one", logPrefix) + logutil.Logger(logCtx).Info("etcd session is done, creates a new one") leaseID := etcdSession.Lease() etcdSession, err = NewSession(ctx, logPrefix, m.etcdCli, NewSessionRetryUnlimited, ManagerSessionTTL) if err != nil { - log.Infof("%s break campaign loop, NewSession err %v", logPrefix, err) + logutil.Logger(logCtx).Info("break campaign loop, NewSession failed", zap.Error(err)) m.revokeSession(logPrefix, leaseID) return } @@ -242,7 +247,7 @@ func (m *ownerManager) campaignLoop(ctx context.Context, etcdSession *concurrenc if terror.ErrorEqual(err, rpctypes.ErrLeaseNotFound) { if etcdSession != nil { err = etcdSession.Close() - log.Infof("%s etcd session encounters the error of lease not found, closes it err %s", logPrefix, err) + logutil.Logger(logCtx).Info("etcd session encounters the error of lease not found, closes it", zap.Error(err)) } continue } @@ -250,11 +255,11 @@ func (m *ownerManager) campaignLoop(ctx context.Context, etcdSession *concurrenc elec := concurrency.NewElection(etcdSession, m.key) err = elec.Campaign(ctx, m.id) if err != nil { - log.Infof("%s failed to campaign, err %v", logPrefix, err) + logutil.Logger(logCtx).Info("failed to campaign", zap.Error(err)) continue } - ownerKey, err := GetOwnerInfo(ctx, elec, logPrefix, m.id) + ownerKey, err := GetOwnerInfo(ctx, logCtx, elec, m.id) if err != nil { continue } @@ -264,7 +269,7 @@ func (m *ownerManager) campaignLoop(ctx context.Context, etcdSession *concurrenc m.RetireOwner() metrics.CampaignOwnerCounter.WithLabelValues(m.prompt, metrics.NoLongerOwner).Inc() - log.Warnf("%s isn't the owner", logPrefix) + logutil.Logger(logCtx).Warn("is not the owner") } } @@ -275,7 +280,7 @@ func (m *ownerManager) revokeSession(logPrefix string, leaseID clientv3.LeaseID) time.Duration(ManagerSessionTTL)*time.Second) _, err := m.etcdCli.Revoke(cancelCtx, leaseID) cancel() - log.Infof("%s break campaign loop, revoke err %v", logPrefix, err) + logutil.Logger(m.logCtx).Info("break campaign loop, revoke err", zap.Error(err)) } // GetOwnerID implements Manager.GetOwnerID interface. @@ -291,17 +296,17 @@ func (m *ownerManager) GetOwnerID(ctx context.Context) (string, error) { } // GetOwnerInfo gets the owner information. -func GetOwnerInfo(ctx context.Context, elec *concurrency.Election, logPrefix, id string) (string, error) { +func GetOwnerInfo(ctx, logCtx context.Context, elec *concurrency.Election, id string) (string, error) { resp, err := elec.Leader(ctx) if err != nil { // If no leader elected currently, it returns ErrElectionNoLeader. - log.Infof("%s failed to get leader, err %v", logPrefix, err) + logutil.Logger(logCtx).Info("failed to get leader", zap.Error(err)) return "", errors.Trace(err) } ownerID := string(resp.Kvs[0].Value) - log.Infof("%s, owner is %v", logPrefix, ownerID) + logutil.Logger(logCtx).Info("get owner", zap.String("ownerID", ownerID)) if ownerID != id { - log.Warnf("%s isn't the owner", logPrefix) + logutil.Logger(logCtx).Warn("is not the owner") return "", errors.New("ownerInfoNotMatch") } @@ -310,26 +315,27 @@ func GetOwnerInfo(ctx context.Context, elec *concurrency.Election, logPrefix, id func (m *ownerManager) watchOwner(ctx context.Context, etcdSession *concurrency.Session, key string) { logPrefix := fmt.Sprintf("[%s] ownerManager %s watch owner key %v", m.prompt, m.id, key) - log.Debugf("%s", logPrefix) + logCtx := logutil.WithKeyValue(context.Background(), "owner info", logPrefix) + logutil.Logger(context.Background()).Debug(logPrefix) watchCh := m.etcdCli.Watch(ctx, key) for { select { case resp, ok := <-watchCh: if !ok { metrics.WatchOwnerCounter.WithLabelValues(m.prompt, metrics.WatcherClosed).Inc() - log.Infof("%s watcher is closed, no owner", logPrefix) + logutil.Logger(logCtx).Info("watcher is closed, no owner") return } if resp.Canceled { metrics.WatchOwnerCounter.WithLabelValues(m.prompt, metrics.Cancelled).Inc() - log.Infof("%s canceled, no owner", logPrefix) + logutil.Logger(logCtx).Info("watch canceled, no owner") return } for _, ev := range resp.Events { if ev.Type == mvccpb.DELETE { metrics.WatchOwnerCounter.WithLabelValues(m.prompt, metrics.Deleted).Inc() - log.Infof("%s failed, owner is deleted", logPrefix) + logutil.Logger(logCtx).Info("watch failed, owner is deleted") return } } @@ -346,7 +352,7 @@ func (m *ownerManager) watchOwner(ctx context.Context, etcdSession *concurrency. func init() { err := setManagerSessionTTL() if err != nil { - log.Warnf("set manager session TTL failed %v", err) + logutil.Logger(context.Background()).Warn("set manager session TTL failed", zap.Error(err)) } } From 5ffa06b6c396513b8c02b51a6f7829298fc7a8b8 Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 25 Mar 2019 16:10:36 +0800 Subject: [PATCH 12/13] *: unify and normalize the format of the log in the pkg of server (#9878) --- cmd/importer/main.go | 17 ++++----- cmd/importer/rand.go | 19 ++++++----- privilege/privileges/privileges.go | 22 +++++++----- server/http_handler.go | 48 ++++++++++++++++++++------ server/http_handler_test.go | 2 +- server/http_status.go | 18 +++++----- server/server.go | 53 ++++++++++++++--------------- server/server_test.go | 9 ++--- sessionctx/binloginfo/binloginfo.go | 10 +++--- 9 files changed, 117 insertions(+), 81 deletions(-) diff --git a/cmd/importer/main.go b/cmd/importer/main.go index 7abaaf1ce59f1..c470448310cf9 100644 --- a/cmd/importer/main.go +++ b/cmd/importer/main.go @@ -18,7 +18,8 @@ import ( "os" "github.com/pingcap/errors" - log "github.com/sirupsen/logrus" + "github.com/pingcap/log" + "go.uber.org/zap" ) func main() { @@ -29,31 +30,31 @@ func main() { case flag.ErrHelp: os.Exit(0) default: - log.Errorf("parse cmd flags err %s\n", err) + log.Error("parse cmd flags", zap.Error(err)) os.Exit(2) } table := newTable() err = parseTableSQL(table, cfg.DDLCfg.TableSQL) if err != nil { - log.Fatal(err) + log.Fatal(err.Error()) } err = parseIndexSQL(table, cfg.DDLCfg.IndexSQL) if err != nil { - log.Fatal(err) + log.Fatal(err.Error()) } dbs, err := createDBs(cfg.DBCfg, cfg.SysCfg.WorkerCount) if err != nil { - log.Fatal(err) + log.Fatal(err.Error()) } defer closeDBs(dbs) if len(cfg.StatsCfg.Path) > 0 { statsInfo, err1 := loadStats(table.tblInfo, cfg.StatsCfg.Path) if err1 != nil { - log.Fatal(err1) + log.Fatal(err1.Error()) } for _, idxInfo := range table.tblInfo.Indices { offset := idxInfo.Columns[0].Offset @@ -75,12 +76,12 @@ func main() { err = execSQL(dbs[0], cfg.DDLCfg.TableSQL) if err != nil { - log.Fatal(err) + log.Fatal(err.Error()) } err = execSQL(dbs[0], cfg.DDLCfg.IndexSQL) if err != nil { - log.Fatal(err) + log.Fatal(err.Error()) } doProcess(table, dbs, cfg.SysCfg.JobCount, cfg.SysCfg.WorkerCount, cfg.SysCfg.Batch) diff --git a/cmd/importer/rand.go b/cmd/importer/rand.go index ef8457da11821..b4f4e5f1a606d 100644 --- a/cmd/importer/rand.go +++ b/cmd/importer/rand.go @@ -18,7 +18,8 @@ import ( "math/rand" "time" - log "github.com/sirupsen/logrus" + "github.com/pingcap/log" + "go.uber.org/zap" ) const ( @@ -80,7 +81,7 @@ func randDate(col *column) string { minTime, err := time.Parse(dateFormat, min) if err != nil { - log.Warnf("randDate err %s", err) + log.Warn("parse min date failed", zap.Error(err)) } if len(max) == 0 { t := minTime.Add(time.Duration(randInt(0, 365)) * 24 * time.Hour) @@ -89,7 +90,7 @@ func randDate(col *column) string { maxTime, err := time.Parse(dateFormat, max) if err != nil { - log.Warnf("randDate err %s", err) + log.Warn("parse max date failed", zap.Error(err)) } days := int(maxTime.Sub(minTime).Hours() / 24) t := minTime.Add(time.Duration(randInt(0, days)) * 24 * time.Hour) @@ -110,11 +111,11 @@ func randTime(col *column) string { minTime, err := time.Parse(timeFormat, min) if err != nil { - log.Warnf("randTime err %s", err) + log.Warn("parse min time failed", zap.Error(err)) } maxTime, err := time.Parse(timeFormat, max) if err != nil { - log.Warnf("randTime err %s", err) + log.Warn("parse max time failed", zap.Error(err)) } seconds := int(maxTime.Sub(minTime).Seconds()) t := minTime.Add(time.Duration(randInt(0, seconds)) * time.Second) @@ -138,7 +139,7 @@ func randTimestamp(col *column) string { minTime, err := time.Parse(dateTimeFormat, min) if err != nil { - log.Warnf("randTimestamp err %s", err) + log.Warn("parse min timestamp failed", zap.Error(err)) } if len(max) == 0 { t := minTime.Add(time.Duration(randInt(0, 365)) * 24 * time.Hour) @@ -147,7 +148,7 @@ func randTimestamp(col *column) string { maxTime, err := time.Parse(dateTimeFormat, max) if err != nil { - log.Warnf("randTimestamp err %s", err) + log.Warn("parse max timestamp failed", zap.Error(err)) } seconds := int(maxTime.Sub(minTime).Seconds()) t := minTime.Add(time.Duration(randInt(0, seconds)) * time.Second) @@ -165,11 +166,11 @@ func randYear(col *column) string { minTime, err := time.Parse(yearFormat, min) if err != nil { - log.Warnf("randYear err %s", err) + log.Warn("parse min year failed", zap.Error(err)) } maxTime, err := time.Parse(yearFormat, max) if err != nil { - log.Warnf("randYear err %s", err) + log.Warn("parse max year failed", zap.Error(err)) } seconds := int(maxTime.Sub(minTime).Seconds()) t := minTime.Add(time.Duration(randInt(0, seconds)) * time.Second) diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index d5ba12d757c9b..a8d8a6cd7d816 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -14,6 +14,7 @@ package privileges import ( + "context" "strings" "github.com/pingcap/parser/auth" @@ -21,7 +22,8 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - log "github.com/sirupsen/logrus" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" ) // SkipWithGrant causes the server to start without using the privilege system at all. @@ -82,12 +84,13 @@ func (p *UserPrivileges) GetEncodedPassword(user, host string) string { mysqlPriv := p.Handle.Get() record := mysqlPriv.connectionVerification(user, host) if record == nil { - log.Errorf("Get user privilege record fail: user %v, host %v", user, host) + logutil.Logger(context.Background()).Error("get user privilege record fail", + zap.String("user", user), zap.String("host", host)) return "" } pwd := record.Password if len(pwd) != 0 && len(pwd) != mysql.PWDHashLen+1 { - log.Errorf("User [%s] password from SystemDB not like a sha1sum", user) + logutil.Logger(context.Background()).Error("user password from system DB not like sha1sum", zap.String("user", user)) return "" } return pwd @@ -95,7 +98,6 @@ func (p *UserPrivileges) GetEncodedPassword(user, host string) string { // ConnectionVerification implements the Manager interface. func (p *UserPrivileges) ConnectionVerification(user, host string, authentication, salt []byte) (u string, h string, success bool) { - if SkipWithGrant { p.user = user p.host = host @@ -106,7 +108,8 @@ func (p *UserPrivileges) ConnectionVerification(user, host string, authenticatio mysqlPriv := p.Handle.Get() record := mysqlPriv.connectionVerification(user, host) if record == nil { - log.Errorf("Get user privilege record fail: user %v, host %v", user, host) + logutil.Logger(context.Background()).Error("get user privilege record fail", + zap.String("user", user), zap.String("host", host)) return } @@ -116,14 +119,15 @@ func (p *UserPrivileges) ConnectionVerification(user, host string, authenticatio // Login a locked account is not allowed. locked := record.AccountLocked if locked { - log.Errorf("Try to login a locked account: user: %v, host: %v", user, host) + logutil.Logger(context.Background()).Error("try to login a locked account", + zap.String("user", user), zap.String("host", host)) success = false return } pwd := record.Password if len(pwd) != 0 && len(pwd) != mysql.PWDHashLen+1 { - log.Errorf("User [%s] password from SystemDB not like a sha1sum", user) + logutil.Logger(context.Background()).Error("user password from system DB not like sha1sum", zap.String("user", user)) return } @@ -141,7 +145,7 @@ func (p *UserPrivileges) ConnectionVerification(user, host string, authenticatio hpwd, err := auth.DecodePassword(pwd) if err != nil { - log.Errorf("Decode password string error %v", err) + logutil.Logger(context.Background()).Error("decode password string failed", zap.Error(err)) return } @@ -195,7 +199,7 @@ func (p *UserPrivileges) ActiveRoles(ctx sessionctx.Context, roleList []*auth.Ro for _, r := range roleList { ok := mysqlPrivilege.FindRole(u, h, r) if !ok { - log.Errorf("Role: %+v doesn't grant for user", r) + logutil.Logger(context.Background()).Error("find role failed", zap.Stringer("role", r)) return false, r.String() } } diff --git a/server/http_handler.go b/server/http_handler.go index e7da07b7cd2c5..6349b1ab5fce0 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -51,8 +51,9 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" - logutil "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/logutil" log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) const ( @@ -100,7 +101,7 @@ func writeData(w http.ResponseWriter, data interface{}) { writeError(w, err) return } - log.Info(string(js)) + logutil.Logger(context.Background()).Info(string(js)) // write response w.Header().Set(headerContentType, contentTypeJSON) w.WriteHeader(http.StatusOK) @@ -147,9 +148,14 @@ func (t *tikvHandlerTool) getMvccByEncodedKey(encodedKey kv.Key) (*kvrpcpb.MvccG }, } kvResp, err := t.store.SendReq(tikv.NewBackoffer(context.Background(), 500), tikvReq, keyLocation.Region, time.Minute) - log.Info(string(encodedKey), keyLocation.Region, string(keyLocation.StartKey), string(keyLocation.EndKey), kvResp, err) - if err != nil { + logutil.Logger(context.Background()).Info("get MVCC by encoded key failed", + zap.Binary("encodeKey", encodedKey), + zap.Reflect("region", keyLocation.Region), + zap.Binary("startKey", keyLocation.StartKey), + zap.Binary("endKey", keyLocation.EndKey), + zap.Reflect("kvResp", kvResp), + zap.Error(err)) return nil, errors.Trace(err) } return kvResp.MvccGetByKey, nil @@ -165,7 +171,7 @@ func (t *tikvHandlerTool) getMvccByStartTs(startTS uint64, startKey, endKey []by for { curRegion, err := t.regionCache.LocateKey(bo, startKey) if err != nil { - log.Error(startTS, startKey, err) + logutil.Logger(context.Background()).Error("get MVCC by startTS failed", zap.Uint64("txnStartTS", startTS), zap.Binary("startKey", startKey), zap.Error(err)) return nil, errors.Trace(err) } @@ -177,19 +183,39 @@ func (t *tikvHandlerTool) getMvccByStartTs(startTS uint64, startKey, endKey []by } tikvReq.Context.Priority = kvrpcpb.CommandPri_Low kvResp, err := t.store.SendReq(bo, tikvReq, curRegion.Region, time.Hour) - log.Info(startTS, string(startKey), curRegion.Region, string(curRegion.StartKey), string(curRegion.EndKey), kvResp) if err != nil { - log.Error(startTS, string(startKey), curRegion.Region, string(curRegion.StartKey), string(curRegion.EndKey), err) + logutil.Logger(context.Background()).Error("get MVCC by startTS failed", + zap.Uint64("txnStartTS", startTS), + zap.Binary("startKey", startKey), + zap.Reflect("region", curRegion.Region), + zap.Binary("curRegion startKey", curRegion.StartKey), + zap.Binary("curRegion endKey", curRegion.EndKey), + zap.Reflect("kvResp", kvResp), + zap.Error(err)) return nil, errors.Trace(err) } data := kvResp.MvccGetByStartTS if err := data.GetRegionError(); err != nil { - log.Warn(startTS, string(startKey), curRegion.Region, string(curRegion.StartKey), string(curRegion.EndKey), err) + logutil.Logger(context.Background()).Warn("get MVCC by startTS failed", + zap.Uint64("txnStartTS", startTS), + zap.Binary("startKey", startKey), + zap.Reflect("region", curRegion.Region), + zap.Binary("curRegion startKey", curRegion.StartKey), + zap.Binary("curRegion endKey", curRegion.EndKey), + zap.Reflect("kvResp", kvResp), + zap.Stringer("error", err)) continue } if len(data.GetError()) > 0 { - log.Error(startTS, string(startKey), curRegion.Region, string(curRegion.StartKey), string(curRegion.EndKey), data.GetError()) + logutil.Logger(context.Background()).Error("get MVCC by startTS failed", + zap.Uint64("txnStartTS", startTS), + zap.Binary("startKey", startKey), + zap.Reflect("region", curRegion.Region), + zap.Binary("curRegion startKey", curRegion.StartKey), + zap.Binary("curRegion endKey", curRegion.EndKey), + zap.Reflect("kvResp", kvResp), + zap.String("error", data.GetError())) return nil, errors.New(data.GetError()) } @@ -379,7 +405,7 @@ func (t *tikvHandlerTool) fetchHotRegion(rw string) (map[uint64]regionMetric, er defer func() { err = resp.Body.Close() if err != nil { - log.Error(err) + logutil.Logger(context.Background()).Error("close body failed", zap.Error(err)) } }() var regionResp storeHotRegionInfos @@ -406,7 +432,7 @@ func (t *tikvHandlerTool) fetchRegionTableIndex(metrics map[uint64]regionMetric) for regionID, regionMetric := range metrics { region, err := t.regionCache.LocateRegionByID(tikv.NewBackoffer(context.Background(), 500), regionID) if err != nil { - log.Error(err) + logutil.Logger(context.Background()).Error("locate region failed", zap.Error(err)) continue } diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 5a61ce7ebdc65..151bbdd802109 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -690,7 +690,7 @@ func (ts *HTTPHandlerTestSuite) TestPprof(c *C) { } time.Sleep(time.Millisecond * 10) } - log.Fatalf("Failed to get profile for %d retries in every 10 ms", retryTime) + zaplog.Fatal("failed to get profile for %d retries in every 10 ms", zap.Int("retryTime", retryTime)) } func (ts *HTTPHandlerTestSuite) TestServerInfo(c *C) { diff --git a/server/http_status.go b/server/http_status.go index 8c1d2bbd4a513..44f64dcb56694 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -16,6 +16,7 @@ package server import ( "archive/zip" "bytes" + "context" "encoding/json" "fmt" "net" @@ -34,10 +35,11 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/printer" "github.com/prometheus/client_golang/prometheus" - log "github.com/sirupsen/logrus" "github.com/tiancaiamao/appdash/traceapp" + "go.uber.org/zap" static "sourcegraph.com/sourcegraph/appdash-data" ) @@ -105,7 +107,7 @@ func (s *Server) startHTTPServer() { router.HandleFunc("/web/trace", traceapp.HandleTiDB).Name("Trace Viewer") sr := router.PathPrefix("/web/trace/").Subrouter() if _, err := traceapp.New(traceapp.NewRouter(sr), baseURL); err != nil { - log.Error(err) + logutil.Logger(context.Background()).Error("new failed", zap.Error(err)) } router.PathPrefix("/static/").Handler(http.StripPrefix("/static", http.FileServer(static.Data))) } @@ -226,7 +228,7 @@ func (s *Server) startHTTPServer() { err = router.Walk(func(route *mux.Route, router *mux.Router, ancestors []*mux.Route) error { pathTemplate, err = route.GetPathTemplate() if err != nil { - log.Error("Get http router path error ", err) + logutil.Logger(context.Background()).Error("get HTTP router path failed", zap.Error(err)) } name := route.GetName() // If the name attribute is not set, GetName returns "". @@ -237,18 +239,18 @@ func (s *Server) startHTTPServer() { return nil }) if err != nil { - log.Error("Generate root error ", err) + logutil.Logger(context.Background()).Error("generate root failed", zap.Error(err)) } httpRouterPage.WriteString("Debug") httpRouterPage.WriteString("") router.HandleFunc("/", func(responseWriter http.ResponseWriter, request *http.Request) { _, err = responseWriter.Write([]byte(httpRouterPage.String())) if err != nil { - log.Error("Http index page error ", err) + logutil.Logger(context.Background()).Error("write HTTP index page failed", zap.Error(err)) } }) - log.Infof("Listening on %v for status and metrics report.", addr) + logutil.Logger(context.Background()).Info("for status and metrics report", zap.String("listening on addr", addr)) s.statusServer = &http.Server{Addr: addr, Handler: CorsHandler{handler: serverMux, cfg: s.cfg}} if len(s.cfg.Security.ClusterSSLCA) != 0 { @@ -258,7 +260,7 @@ func (s *Server) startHTTPServer() { } if err != nil { - log.Info(err) + logutil.Logger(context.Background()).Info("listen failed", zap.Error(err)) } } @@ -280,7 +282,7 @@ func (s *Server) handleStatus(w http.ResponseWriter, req *http.Request) { js, err := json.Marshal(st) if err != nil { w.WriteHeader(http.StatusInternalServerError) - log.Error("Encode json error", err) + logutil.Logger(context.Background()).Error("encode json failed", zap.Error(err)) } else { _, err = w.Write(js) terror.Log(errors.Trace(err)) diff --git a/server/server.go b/server/server.go index a114a43c58d7c..bbe8f1c8e2f52 100644 --- a/server/server.go +++ b/server/server.go @@ -58,7 +58,6 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sys/linux" - log "github.com/sirupsen/logrus" "go.uber.org/zap" ) @@ -147,7 +146,7 @@ func (s *Server) newConn(conn net.Conn) *clientConn { if s.cfg.Performance.TCPKeepAlive { if tcpConn, ok := conn.(*net.TCPConn); ok { if err := tcpConn.SetKeepAlive(true); err != nil { - log.Error("failed to set tcp keep alive option:", err) + logutil.Logger(context.Background()).Error("failed to set tcp keep alive option", zap.Error(err)) } } } @@ -167,11 +166,11 @@ func (s *Server) forwardUnixSocketToTCP() { return // server shutdown has started } if uconn, err := s.socket.Accept(); err == nil { - log.Infof("server socket forwarding from [%s] to [%s]", s.cfg.Socket, addr) + logutil.Logger(context.Background()).Info("server socket forwarding", zap.String("from", s.cfg.Socket), zap.String("to", addr)) go s.handleForwardedConnection(uconn, addr) } else { if s.listener != nil { - log.Errorf("server failed to forward from [%s] to [%s], err: %s", s.cfg.Socket, addr, err) + logutil.Logger(context.Background()).Error("server failed to forward", zap.String("from", s.cfg.Socket), zap.String("to", addr), zap.Error(err)) } } } @@ -182,14 +181,14 @@ func (s *Server) handleForwardedConnection(uconn net.Conn, addr string) { if tconn, err := net.Dial("tcp", addr); err == nil { go func() { if _, err := io.Copy(uconn, tconn); err != nil { - log.Warningf("copy server to socket failed: %s", err) + logutil.Logger(context.Background()).Warn("copy server to socket failed", zap.Error(err)) } }() if _, err := io.Copy(tconn, uconn); err != nil { - log.Warningf("socket forward copy failed: %s", err) + logutil.Logger(context.Background()).Warn("socket forward copy failed", zap.Error(err)) } } else { - log.Warningf("socket forward failed: could not connect to [%s], err: %s", addr, err) + logutil.Logger(context.Background()).Warn("socket forward failed: could not connect", zap.String("addr", addr), zap.Error(err)) } } @@ -215,17 +214,17 @@ func NewServer(cfg *config.Config, driver IDriver) (*Server, error) { if s.cfg.Host != "" && s.cfg.Port != 0 { addr := fmt.Sprintf("%s:%d", s.cfg.Host, s.cfg.Port) if s.listener, err = net.Listen("tcp", addr); err == nil { - log.Infof("Server is running MySQL Protocol at [%s]", addr) + logutil.Logger(context.Background()).Info("server is running MySQL protocol", zap.String("addr", addr)) if cfg.Socket != "" { if s.socket, err = net.Listen("unix", s.cfg.Socket); err == nil { - log.Infof("Server redirecting [%s] to [%s]", s.cfg.Socket, addr) + logutil.Logger(context.Background()).Info("server redirecting", zap.String("from", s.cfg.Socket), zap.String("to", addr)) go s.forwardUnixSocketToTCP() } } } } else if cfg.Socket != "" { if s.listener, err = net.Listen("unix", cfg.Socket); err == nil { - log.Infof("Server is running MySQL Protocol through Socket [%s]", cfg.Socket) + logutil.Logger(context.Background()).Info("server is running MySQL protocol", zap.String("socket", cfg.Socket)) } } else { err = errors.New("Server not configured to listen on either -socket or -host and -port") @@ -235,10 +234,10 @@ func NewServer(cfg *config.Config, driver IDriver) (*Server, error) { pplistener, errProxy := proxyprotocol.NewListener(s.listener, cfg.ProxyProtocol.Networks, int(cfg.ProxyProtocol.HeaderTimeout)) if errProxy != nil { - log.Error("ProxyProtocol Networks parameter invalid") + logutil.Logger(context.Background()).Error("ProxyProtocol networks parameter invalid") return nil, errors.Trace(errProxy) } - log.Infof("Server is running MySQL Protocol (through PROXY Protocol) at [%s]", s.cfg.Host) + logutil.Logger(context.Background()).Info("server is running MySQL protocol (through PROXY protocol)", zap.String("host", s.cfg.Host)) s.listener = pplistener } @@ -254,13 +253,13 @@ func NewServer(cfg *config.Config, driver IDriver) (*Server, error) { func (s *Server) loadTLSCertificates() { defer func() { if s.tlsConfig != nil { - log.Infof("Secure connection is enabled (client verification enabled = %v)", len(variable.SysVars["ssl_ca"].Value) > 0) + logutil.Logger(context.Background()).Info("secure connection is enabled", zap.Bool("client verification enabled", len(variable.SysVars["ssl_ca"].Value) > 0)) variable.SysVars["have_openssl"].Value = "YES" variable.SysVars["have_ssl"].Value = "YES" variable.SysVars["ssl_cert"].Value = s.cfg.Security.SSLCert variable.SysVars["ssl_key"].Value = s.cfg.Security.SSLKey } else { - log.Warn("Secure connection is NOT ENABLED") + logutil.Logger(context.Background()).Warn("secure connection is not enabled") } }() @@ -271,7 +270,7 @@ func (s *Server) loadTLSCertificates() { tlsCert, err := tls.LoadX509KeyPair(s.cfg.Security.SSLCert, s.cfg.Security.SSLKey) if err != nil { - log.Warn(errors.ErrorStack(err)) + logutil.Logger(context.Background()).Warn("load x509 failed", zap.Error(err)) s.tlsConfig = nil return } @@ -282,7 +281,7 @@ func (s *Server) loadTLSCertificates() { if len(s.cfg.Security.SSLCA) > 0 { caCert, err := ioutil.ReadFile(s.cfg.Security.SSLCA) if err != nil { - log.Warn(errors.ErrorStack(err)) + logutil.Logger(context.Background()).Warn("read file failed", zap.Error(err)) } else { certPool = x509.NewCertPool() if certPool.AppendCertsFromPEM(caCert) { @@ -318,11 +317,11 @@ func (s *Server) Run() error { // If we got PROXY protocol error, we should continue accept. if proxyprotocol.IsProxyProtocolError(err) { - log.Errorf("PROXY protocol error: %s", err.Error()) + logutil.Logger(context.Background()).Error("PROXY protocol failed", zap.Error(err)) continue } - log.Errorf("accept error %s", err.Error()) + logutil.Logger(context.Background()).Error("accept failed", zap.Error(err)) return errors.Trace(err) } if s.shouldStopListener() { @@ -338,13 +337,13 @@ func (s *Server) Run() error { if authPlugin.OnConnectionEvent != nil { host, err := clientConn.PeerHost("") if err != nil { - log.Info(err) + logutil.Logger(context.Background()).Error("get peer host failed", zap.Error(err)) terror.Log(clientConn.Close()) return errors.Trace(err) } err = authPlugin.OnConnectionEvent(context.Background(), &auth.UserIdentity{Hostname: host}, plugin.PreAuth, nil) if err != nil { - log.Info(err) + logutil.Logger(context.Background()).Info("do connection event failed", zap.Error(err)) terror.Log(clientConn.Close()) return errors.Trace(err) } @@ -362,7 +361,7 @@ func (s *Server) Run() error { s.listener = nil for { metrics.ServerEventCounter.WithLabelValues(metrics.EventHang).Inc() - log.Errorf("listener stopped, waiting for manual kill.") + logutil.Logger(context.Background()).Error("listener stopped, waiting for manual kill.") time.Sleep(time.Minute) } } @@ -444,7 +443,7 @@ func (s *Server) onConn(conn *clientConn) { connInfo.Duration = float64(time.Since(connectedTime)) / float64(time.Millisecond) err := authPlugin.OnConnectionEvent(context.Background(), conn.ctx.GetSessionVars().User, plugin.Disconnect, connInfo) if err != nil { - log.Warnf("call Plugin %s OnConnectionEvent(Disconnect) failure, err: %v", authPlugin.Name, err) + logutil.Logger(context.Background()).Warn("do connection event failed", zap.String("plugin", authPlugin.Name), zap.Error(err)) } } return nil @@ -501,7 +500,7 @@ func (s *Server) ShowProcessList() map[uint64]util.ProcessInfo { func (s *Server) Kill(connectionID uint64, query bool) { s.rwlock.Lock() defer s.rwlock.Unlock() - log.Infof("[server] Kill connectionID %d, query %t]", connectionID, query) + logutil.Logger(context.Background()).Info("kill", zap.Uint64("connID", connectionID), zap.Bool("query", query)) metrics.ServerEventCounter.WithLabelValues(metrics.EventKill).Inc() conn, ok := s.clients[uint32(connectionID)] @@ -531,7 +530,7 @@ func killConn(conn *clientConn, query bool) { func (s *Server) KillAllConnections() { s.rwlock.Lock() defer s.rwlock.Unlock() - log.Info("[server] kill all connections.") + logutil.Logger(context.Background()).Info("[server] kill all connections.") for _, conn := range s.clients { atomic.StoreInt32(&conn.status, connStatusShutdown) @@ -565,7 +564,7 @@ func (s *Server) TryGracefulDown() { // GracefulDown waits all clients to close. func (s *Server) GracefulDown(ctx context.Context, done chan struct{}) { - log.Info("[server] graceful shutdown.") + logutil.Logger(ctx).Info("[server] graceful shutdown.") metrics.ServerEventCounter.WithLabelValues(metrics.EventGracefulDown).Inc() count := s.ConnectionCount() @@ -578,7 +577,7 @@ func (s *Server) GracefulDown(ctx context.Context, done chan struct{}) { } // Print information for every 30s. if i%30 == 0 { - log.Infof("graceful shutdown...connection count %d\n", count) + logutil.Logger(ctx).Info("graceful shutdown...", zap.Int("conn count", count)) } ticker := time.After(time.Second) select { @@ -604,7 +603,7 @@ func (s *Server) kickIdleConnection() { for _, cc := range conns { err := cc.Close() if err != nil { - log.Errorf("close connection error: %s", err) + logutil.Logger(context.Background()).Error("close connection", zap.Error(err)) } } } diff --git a/server/server_test.go b/server/server_test.go index 3677004324f56..88f3045b17086 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -27,17 +27,18 @@ import ( "github.com/go-sql-driver/mysql" . "github.com/pingcap/check" + "github.com/pingcap/log" tmysql "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/printer" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" ) func TestT(t *testing.T) { CustomVerboseFlag = true logLevel := os.Getenv("log_level") - logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false)) + logutil.InitZapLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false)) TestingT(t) } @@ -1050,7 +1051,7 @@ func waitUntilServerOnline(statusPort uint) { } } if retry == retryTime { - log.Fatalf("Failed to connect db for %d retries in every 10 ms", retryTime) + log.Fatal("failed to connect DB in every 10 ms", zap.Int("retryTime", retryTime)) } // connect http status statusURL := fmt.Sprintf("http://127.0.0.1:%d/status", statusPort) @@ -1064,6 +1065,6 @@ func waitUntilServerOnline(statusPort uint) { time.Sleep(time.Millisecond * 10) } if retry == retryTime { - log.Fatalf("Failed to connect http status for %d retries in every 10 ms", retryTime) + log.Fatal("failed to connect HTTP status in every 10 ms", zap.Int("retryTime", retryTime)) } } diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 8be106c057222..d4c7bb0be8ef1 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -14,6 +14,7 @@ package binloginfo import ( + "context" "regexp" "strings" "sync" @@ -27,8 +28,9 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/logutil" binlog "github.com/pingcap/tipb/go-binlog" - log "github.com/sirupsen/logrus" + "go.uber.org/zap" "google.golang.org/grpc" ) @@ -80,7 +82,7 @@ var ignoreError uint32 // DisableSkipBinlogFlag disable the skipBinlog flag. func DisableSkipBinlogFlag() { atomic.StoreUint32(&skipBinlog, 0) - log.Warn("[binloginfo] disable the skipBinlog flag") + logutil.Logger(context.Background()).Warn("[binloginfo] disable the skipBinlog flag") } // SetIgnoreError sets the ignoreError flag, this function called when TiDB start @@ -108,9 +110,9 @@ func (info *BinlogInfo) WriteBinlog(clusterID uint64) error { // it will retry in PumpsClient if write binlog fail. err := info.Client.WriteBinlog(info.Data) if err != nil { - log.Errorf("write binlog fail %v", errors.ErrorStack(err)) + logutil.Logger(context.Background()).Error("write binlog failed", zap.Error(err)) if atomic.LoadUint32(&ignoreError) == 1 { - log.Error("write binlog fail but error ignored") + logutil.Logger(context.Background()).Error("write binlog fail but error ignored") metrics.CriticalErrorCounter.Add(1) // If error happens once, we'll stop writing binlog. atomic.CompareAndSwapUint32(&skipBinlog, skip, skip+1) From d6490c1cab3a634e2faba004802fab442d153035 Mon Sep 17 00:00:00 2001 From: Keyi Xie Date: Mon, 25 Mar 2019 16:36:14 +0800 Subject: [PATCH 13/13] test: open coverpkg for coverrage, speed up the test, calculate more folder. (#9725) --- Makefile | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/Makefile b/Makefile index 7851603e6476b..48ff0edc15129 100644 --- a/Makefile +++ b/Makefile @@ -13,7 +13,7 @@ export PATH := $(path_to_add):$(PATH) GO := GO111MODULE=on go GOBUILD := CGO_ENABLED=1 $(GO) build $(BUILD_FLAG) -GOTEST := CGO_ENABLED=1 $(GO) test -p 3 +GOTEST := CGO_ENABLED=1 $(GO) test -p 4 OVERALLS := CGO_ENABLED=1 GO111MODULE=on overalls ARCH := "`uname -s`" @@ -128,8 +128,12 @@ ifeq ("$(TRAVIS_COVERAGE)", "1") @echo "Running in TRAVIS_COVERAGE mode." @export log_level=error; \ $(GO) get github.com/go-playground/overalls - $(OVERALLS) -project=github.com/pingcap/tidb -covermode=count -ignore='.git,vendor,cmd,docs,LICENSES,ddl/failtest,ddl/testutil/,executor/esqtest' \ - -concurrency=1 || { $(GOFAIL_DISABLE); exit 1; } + $(OVERALLS) -project=github.com/pingcap/tidb \ + -covermode=count \ + -ignore='.git,vendor,cmd,docs,LICENSES' \ + -concurrency=2 \ + -- -coverpkg=./... \ + || { $(GOFAIL_DISABLE); exit 1; } else @echo "Running in native mode." @export log_level=error; \