From 1f46efe1561c5b0c13fe0fa75bd72a3e573fcad6 Mon Sep 17 00:00:00 2001 From: Ping Yu Date: Mon, 26 Nov 2018 20:25:59 +0800 Subject: [PATCH 01/17] expression: add builtin function JSON_DEPTH (#8347) --- expression/builtin_json.go | 29 ++++++++++++- expression/builtin_json_test.go | 59 +++++++++++++++++++++++++++ expression/distsql_builtin.go | 16 ++++++++ expression/distsql_builtin_test.go | 65 ++++++++++++++++++++++++++++++ types/json/binary_functions.go | 39 ++++++++++++++++++ types/json/binary_test.go | 20 +++++++++ 6 files changed, 227 insertions(+), 1 deletion(-) diff --git a/expression/builtin_json.go b/expression/builtin_json.go index d372bb89430d4..7bc63a1acdf62 100644 --- a/expression/builtin_json.go +++ b/expression/builtin_json.go @@ -61,6 +61,7 @@ var ( _ builtinFunc = &builtinJSONRemoveSig{} _ builtinFunc = &builtinJSONMergeSig{} _ builtinFunc = &builtinJSONContainsSig{} + _ builtinFunc = &builtinJSONDepthSig{} _ builtinFunc = &builtinJSONKeysSig{} _ builtinFunc = &builtinJSONKeys2ArgsSig{} _ builtinFunc = &builtinJSONLengthSig{} @@ -759,8 +760,34 @@ type jsonDepthFunctionClass struct { baseFunctionClass } +type builtinJSONDepthSig struct { + baseBuiltinFunc +} + +func (b *builtinJSONDepthSig) Clone() builtinFunc { + newSig := &builtinJSONDepthSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonDepthFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { - return nil, errFunctionNotExists.GenWithStackByArgs("FUNCTION", "JSON_DEPTH") + if err := c.verifyArgs(args); err != nil { + return nil, errors.Trace(err) + } + + bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETJson) + sig := &builtinJSONDepthSig{bf} + sig.setPbCode(tipb.ScalarFuncSig_JsonDepthSig) + return sig, nil +} + +func (b *builtinJSONDepthSig) evalInt(row chunk.Row) (res int64, isNull bool, err error) { + obj, isNull, err := b.args[0].EvalJSON(b.ctx, row) + if isNull || err != nil { + return res, isNull, errors.Trace(err) + } + + return int64(obj.GetElemDepth()), false, nil } type jsonKeysFunctionClass struct { diff --git a/expression/builtin_json_test.go b/expression/builtin_json_test.go index 1c3b5ed1d7e0b..7b78081803a7e 100644 --- a/expression/builtin_json_test.go +++ b/expression/builtin_json_test.go @@ -565,3 +565,62 @@ func (s *testEvaluatorSuite) TestJSONKeys(c *C) { } } } + +func (s *testEvaluatorSuite) TestJSONDepth(c *C) { + defer testleak.AfterTest(c)() + fc := funcs[ast.JSONDepth] + tbl := []struct { + input []interface{} + expected interface{} + success bool + }{ + // Tests scalar arguments + {[]interface{}{`null`}, 1, true}, + {[]interface{}{`true`}, 1, true}, + {[]interface{}{`false`}, 1, true}, + {[]interface{}{`1`}, 1, true}, + {[]interface{}{`-1`}, 1, true}, + {[]interface{}{`1.1`}, 1, true}, + {[]interface{}{`"1"`}, 1, true}, + // Tests nil arguments + {[]interface{}{nil}, nil, true}, + // Tests depth + {[]interface{}{`{}`}, 1, true}, + {[]interface{}{`[]`}, 1, true}, + {[]interface{}{`[10, 20]`}, 2, true}, + {[]interface{}{`[[], {}]`}, 2, true}, + {[]interface{}{`{"Name": "Homer"}`}, 2, true}, + {[]interface{}{`[10, {"a": 20}]`}, 3, true}, + {[]interface{}{`{"Person": {"Name": "Homer", "Age": 39, "Hobbies": ["Eating", "Sleeping"]} }`}, 4, true}, + {[]interface{}{`{"a":1}`}, 2, true}, + {[]interface{}{`{"a":[1]}`}, 3, true}, + {[]interface{}{`{"b":2, "c":3}`}, 2, true}, + {[]interface{}{`[1]`}, 2, true}, + {[]interface{}{`[1,2]`}, 2, true}, + {[]interface{}{`[1,2,[1,3]]`}, 3, true}, + {[]interface{}{`[1,2,[1,[5,[3]]]]`}, 5, true}, + {[]interface{}{`[1,2,[1,[5,{"a":[2,3]}]]]`}, 6, true}, + {[]interface{}{`[{"a":1}]`}, 3, true}, + {[]interface{}{`[{"a":1,"b":2}]`}, 3, true}, + {[]interface{}{`[{"a":{"a":1},"b":2}]`}, 4, true}, + // Tests non-json + {[]interface{}{`a`}, nil, false}, + } + for _, t := range tbl { + args := types.MakeDatums(t.input...) + f, err := fc.getFunction(s.ctx, s.datumsToConstants(args)) + c.Assert(err, IsNil) + d, err := evalBuiltinFunc(f, chunk.Row{}) + if t.success { + c.Assert(err, IsNil) + + if t.expected == nil { + c.Assert(d.IsNull(), IsTrue) + } else { + c.Assert(d.GetInt64(), Equals, int64(t.expected.(int))) + } + } else { + c.Assert(err, NotNil) + } + } +} diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index c1c61a8db1183..b79b798235faf 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -438,6 +438,8 @@ func getSignatureByPB(ctx sessionctx.Context, sigCode tipb.ScalarFuncSig, tp *ti f = &builtinLikeSig{base} case tipb.ScalarFuncSig_JsonLengthSig: f = &builtinJSONLengthSig{base} + case tipb.ScalarFuncSig_JsonDepthSig: + f = &builtinJSONDepthSig{base} case tipb.ScalarFuncSig_InInt: f = &builtinInIntSig{base} @@ -507,6 +509,8 @@ func PBToExpr(expr *tipb.Expr, tps []*types.FieldType, sc *stmtctx.StatementCont return convertDuration(expr.Val) case tipb.ExprType_MysqlTime: return convertTime(expr.Val, expr.FieldType, sc.TimeZone) + case tipb.ExprType_MysqlJson: + return convertJSON(expr.Val) } if expr.Tp != tipb.ExprType_ScalarFunc { panic("should be a tipb.ExprType_ScalarFunc") @@ -642,3 +646,15 @@ func convertDuration(val []byte) (*Constant, error) { d.SetMysqlDuration(types.Duration{Duration: time.Duration(i), Fsp: types.MaxFsp}) return &Constant{Value: d, RetType: types.NewFieldType(mysql.TypeDuration)}, nil } + +func convertJSON(val []byte) (*Constant, error) { + var d types.Datum + _, d, err := codec.DecodeOne(val) + if err != nil { + return nil, errors.Errorf("invalid json % x", val) + } + if d.Kind() != types.KindMysqlJSON { + return nil, errors.Errorf("invalid Datum.Kind() %d", d.Kind()) + } + return &Constant{Value: d, RetType: types.NewFieldType(mysql.TypeJSON)}, nil +} diff --git a/expression/distsql_builtin_test.go b/expression/distsql_builtin_test.go index 25162e2edcdf2..2bf1fcc6aafd5 100644 --- a/expression/distsql_builtin_test.go +++ b/expression/distsql_builtin_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tipb/go-tipb" @@ -88,10 +89,26 @@ func (s *testEvalSuite) TestEval(c *C) { datumExpr(types.NewDecimalDatum(types.NewDecFromFloatForTest(1.1))), types.NewDecimalDatum(types.NewDecFromFloatForTest(1.1)), }, + // Columns. { columnExpr(0), types.NewIntDatum(100), }, + // Scalar Functions. + { + scalarFunctionExpr(tipb.ScalarFuncSig_JsonDepthSig, + toPBFieldType(newIntFieldType()), + jsonDatumExpr(`true`), + ), + types.NewIntDatum(1), + }, + { + scalarFunctionExpr(tipb.ScalarFuncSig_JsonDepthSig, + toPBFieldType(newIntFieldType()), + jsonDatumExpr(`[10, {"a": 20}]`), + ), + types.NewIntDatum(3), + }, } sc := new(stmtctx.StatementContext) for _, tt := range tests { @@ -152,15 +169,63 @@ func datumExpr(d types.Datum) *tipb.Expr { if err != nil { log.Warnf("err happened when EncodeDecimal in datumExpr:%s", err.Error()) } + case types.KindMysqlJSON: + expr.Tp = tipb.ExprType_MysqlJson + var err error + expr.Val = make([]byte, 0, 1024) + expr.Val, err = codec.EncodeValue(nil, expr.Val, d) + if err != nil { + log.Warnf("err happened when EncodeValue of JSON in datumExpr:%s", err.Error()) + } default: expr.Tp = tipb.ExprType_Null } return expr } +func jsonDatumExpr(s string) *tipb.Expr { + var d types.Datum + j, err := json.ParseBinaryFromString(s) + if err != nil { + log.Warnf("err happened when json.ParseBinaryFromString in jsonDatumExpr:%s", err.Error()) + } + d.SetMysqlJSON(j) + return datumExpr(d) +} + func columnExpr(columnID int64) *tipb.Expr { expr := new(tipb.Expr) expr.Tp = tipb.ExprType_ColumnRef expr.Val = codec.EncodeInt(nil, columnID) return expr } + +// toPBFieldType converts *types.FieldType to *tipb.FieldType. +func toPBFieldType(ft *types.FieldType) *tipb.FieldType { + return &tipb.FieldType{ + Tp: int32(ft.Tp), + Flag: uint32(ft.Flag), + Flen: int32(ft.Flen), + Decimal: int32(ft.Decimal), + Charset: ft.Charset, + Collate: collationToProto(ft.Collate), + } +} + +func newIntFieldType() *types.FieldType { + return &types.FieldType{ + Tp: mysql.TypeLonglong, + Flen: mysql.MaxIntWidth, + Decimal: 0, + Flag: mysql.BinaryFlag, + } +} + +func scalarFunctionExpr(sigCode tipb.ScalarFuncSig, retType *tipb.FieldType, args ...*tipb.Expr) *tipb.Expr { + return &tipb.Expr{ + Tp: tipb.ExprType_ScalarFunc, + Sig: sigCode, + Children: args, + FieldType: retType, + } +} diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index 3719367f042b8..93b4b6221f71b 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -731,3 +731,42 @@ func ContainsBinary(obj, target BinaryJSON) bool { return CompareBinary(obj, target) == 0 } } + +// GetElemDepth for JSON_DEPTH +// Returns the maximum depth of a JSON document +// rules referenced by MySQL JSON_DEPTH function +// [https://dev.mysql.com/doc/refman/5.7/en/json-attribute-functions.html#function_json-depth] +// 1) An empty array, empty object, or scalar value has depth 1. +// 2) A nonempty array containing only elements of depth 1 or nonempty object containing only member values of depth 1 has depth 2. +// 3) Otherwise, a JSON document has depth greater than 2. +// e.g. depth of '{}', '[]', 'true': 1 +// e.g. depth of '[10, 20]', '[[], {}]': 2 +// e.g. depth of '[10, {"a": 20}]': 3 +func (bj BinaryJSON) GetElemDepth() int { + switch bj.TypeCode { + case TypeCodeObject: + len := bj.GetElemCount() + maxDepth := 0 + for i := 0; i < len; i++ { + obj := bj.objectGetVal(i) + depth := obj.GetElemDepth() + if depth > maxDepth { + maxDepth = depth + } + } + return maxDepth + 1 + case TypeCodeArray: + len := bj.GetElemCount() + maxDepth := 0 + for i := 0; i < len; i++ { + obj := bj.arrayGetElem(i) + depth := obj.GetElemDepth() + if depth > maxDepth { + maxDepth = depth + } + } + return maxDepth + 1 + default: + return 1 + } +} diff --git a/types/json/binary_test.go b/types/json/binary_test.go index 8e88ebaf1317f..efa109cbf525c 100644 --- a/types/json/binary_test.go +++ b/types/json/binary_test.go @@ -324,3 +324,23 @@ func (s *testJSONSuite) TestBinaryJSONContains(c *C) { c.Assert(ContainsBinary(obj, target), Equals, tt.expected) } } + +func (s *testJSONSuite) TestBinaryJSONDepth(c *C) { + var tests = []struct { + input string + expected int + }{ + {`{}`, 1}, + {`[]`, 1}, + {`true`, 1}, + {`[10, 20]`, 2}, + {`[[], {}]`, 2}, + {`[10, {"a": 20}]`, 3}, + {`{"Person": {"Name": "Homer", "Age": 39, "Hobbies": ["Eating", "Sleeping"]} }`, 4}, + } + + for _, tt := range tests { + obj := mustParseBinaryFromString(c, tt.input) + c.Assert(obj.GetElemDepth(), Equals, tt.expected) + } +} From 7ab61d85ce2c9deca971b52077a36812ada66d76 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 26 Nov 2018 05:39:56 -0700 Subject: [PATCH 02/17] executor: improve SHOW commands compatibility (#8417) --- executor/errors.go | 6 +++ executor/show.go | 97 +++++++++++++++++++++++++++++-------- executor/show_test.go | 42 ++++++++++++++-- infoschema/tables.go | 7 ++- planner/core/planbuilder.go | 2 +- 5 files changed, 127 insertions(+), 27 deletions(-) diff --git a/executor/errors.go b/executor/errors.go index ad1c7d9b36344..30662d0cc2368 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -45,6 +45,9 @@ var ( ErrCantChangeTxCharacteristics = terror.ClassExecutor.New(mysql.ErrCantChangeTxCharacteristics, mysql.MySQLErrName[mysql.ErrCantChangeTxCharacteristics]) ErrPsManyParam = terror.ClassExecutor.New(mysql.ErrPsManyParam, mysql.MySQLErrName[mysql.ErrPsManyParam]) ErrAdminCheckTable = terror.ClassExecutor.New(mysql.ErrAdminCheckTable, mysql.MySQLErrName[mysql.ErrAdminCheckTable]) + ErrDBaccessDenied = terror.ClassExecutor.New(mysql.ErrDBaccessDenied, mysql.MySQLErrName[mysql.ErrDBaccessDenied]) + ErrTableaccessDenied = terror.ClassExecutor.New(mysql.ErrTableaccessDenied, mysql.MySQLErrName[mysql.ErrTableaccessDenied]) + ErrBadDB = terror.ClassExecutor.New(mysql.ErrBadDB, mysql.MySQLErrName[mysql.ErrBadDB]) ) func init() { @@ -57,6 +60,9 @@ func init() { mysql.ErrCantChangeTxCharacteristics: mysql.ErrCantChangeTxCharacteristics, mysql.ErrPsManyParam: mysql.ErrPsManyParam, mysql.ErrAdminCheckTable: mysql.ErrAdminCheckTable, + mysql.ErrDBaccessDenied: mysql.ErrDBaccessDenied, + mysql.ErrTableaccessDenied: mysql.ErrTableaccessDenied, + mysql.ErrBadDB: mysql.ErrBadDB, } terror.ErrClassToMySQLCodes[terror.ClassExecutor] = tableMySQLErrCodes } diff --git a/executor/show.go b/executor/show.go index 0893d8213d82f..1a16fe07d6337 100644 --- a/executor/show.go +++ b/executor/show.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/format" + "github.com/pingcap/tidb/util/sqlexec" "golang.org/x/net/context" ) @@ -155,14 +156,15 @@ func (e *ShowExec) fetchAll() error { } func (e *ShowExec) fetchShowEngines() error { - e.appendRow([]interface{}{ - "InnoDB", - "DEFAULT", - "Supports transactions, row-level locking, and foreign keys", - "YES", - "YES", - "YES", - }) + sql := `SELECT * FROM information_schema.engines` + rows, _, err := e.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(e.ctx, sql) + + if err != nil { + return errors.Trace(err) + } + for _, row := range rows { + e.result.AppendRow(row) + } return nil } @@ -241,10 +243,13 @@ func (e *ShowExec) fetchShowProcessList() error { } func (e *ShowExec) fetchShowTables() error { + checker := privilege.GetPrivilegeManager(e.ctx) + if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.DBIsVisible(e.DBName.O) { + return e.dbAccessDenied() + } if !e.is.SchemaExists(e.DBName) { - return errors.Errorf("Can not find DB: %s", e.DBName) + return ErrBadDB.GenWithStackByArgs(e.DBName) } - checker := privilege.GetPrivilegeManager(e.ctx) // sort for tables var tableNames []string for _, v := range e.is.SchemaTables(e.DBName) { @@ -269,18 +274,34 @@ func (e *ShowExec) fetchShowTables() error { } func (e *ShowExec) fetchShowTableStatus() error { + checker := privilege.GetPrivilegeManager(e.ctx) + if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.DBIsVisible(e.DBName.O) { + return e.dbAccessDenied() + } if !e.is.SchemaExists(e.DBName) { - return errors.Errorf("Can not find DB: %s", e.DBName) + return ErrBadDB.GenWithStackByArgs(e.DBName) } - // sort for tables - tables := e.is.SchemaTables(e.DBName) - sort.Sort(table.Slice(tables)) + sql := fmt.Sprintf(`SELECT + table_name, engine, version, row_format, table_rows, + avg_row_length, data_length, max_data_length, index_length, + data_free, auto_increment, create_time, update_time, check_time, + table_collation, IFNULL(check_sum,''), create_options, table_comment + FROM information_schema.tables + WHERE table_schema='%s' ORDER BY table_name`, e.DBName) + + rows, _, err := e.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(e.ctx, sql) + + if err != nil { + return errors.Trace(err) + } + + for _, row := range rows { + if checker != nil && !checker.RequestVerification(e.DBName.O, row.GetString(0), "", mysql.AllPrivMask) { + continue + } + e.result.AppendRow(row) - for _, t := range tables { - now := types.CurrentTime(mysql.TypeDatetime) - e.appendRow([]interface{}{t.Meta().Name.O, "InnoDB", 10, "Compact", 100, 100, 100, 100, 100, 100, 100, - model.TSConvert2Time(t.Meta().UpdateTS).String(), now, now, "utf8_general_ci", "", createOptions(t.Meta()), t.Meta().Comment}) } return nil } @@ -294,9 +315,15 @@ func createOptions(tb *model.TableInfo) string { func (e *ShowExec) fetchShowColumns() error { tb, err := e.getTable() + if err != nil { return errors.Trace(err) } + checker := privilege.GetPrivilegeManager(e.ctx) + if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.RequestVerification(e.DBName.O, tb.Meta().Name.O, "", mysql.AllPrivMask) { + return e.tableAccessDenied("SELECT", tb.Meta().Name.O) + } + cols := tb.Cols() for _, col := range cols { if e.Column != nil && e.Column.Name.L != col.Name.L { @@ -344,13 +371,17 @@ func (e *ShowExec) fetchShowColumns() error { return nil } -// TODO: index collation can have values A (ascending) or NULL (not sorted). -// see: https://dev.mysql.com/doc/refman/5.7/en/show-index.html func (e *ShowExec) fetchShowIndex() error { tb, err := e.getTable() if err != nil { return errors.Trace(err) } + + checker := privilege.GetPrivilegeManager(e.ctx) + if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.RequestVerification(e.DBName.O, tb.Meta().Name.O, "", mysql.AllPrivMask) { + return e.tableAccessDenied("SELECT", tb.Meta().Name.O) + } + if tb.Meta().PKIsHandle { var pkCol *table.Column for _, col := range tb.Cols() { @@ -714,6 +745,10 @@ func (e *ShowExec) fetchShowCreateTable() error { // fetchShowCreateDatabase composes show create database result. func (e *ShowExec) fetchShowCreateDatabase() error { + checker := privilege.GetPrivilegeManager(e.ctx) + if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.DBIsVisible(fmt.Sprint(e.DBName)) { + return e.dbAccessDenied() + } db, ok := e.is.SchemaByName(e.DBName) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(e.DBName.O) @@ -843,6 +878,28 @@ func (e *ShowExec) getTable() (table.Table, error) { return tb, nil } +func (e *ShowExec) dbAccessDenied() error { + user := e.ctx.GetSessionVars().User + u := user.Username + h := user.Hostname + if len(user.AuthUsername) > 0 && len(user.AuthHostname) > 0 { + u = user.AuthUsername + h = user.AuthHostname + } + return ErrDBaccessDenied.GenWithStackByArgs(u, h, e.DBName) +} + +func (e *ShowExec) tableAccessDenied(access string, table string) error { + user := e.ctx.GetSessionVars().User + u := user.Username + h := user.Hostname + if len(user.AuthUsername) > 0 && len(user.AuthHostname) > 0 { + u = user.AuthUsername + h = user.AuthHostname + } + return ErrTableaccessDenied.GenWithStackByArgs(access, u, h, table) +} + func (e *ShowExec) appendRow(row []interface{}) { for i, col := range row { if col == nil { diff --git a/executor/show_test.go b/executor/show_test.go index 0a8761d12e273..6dfd6c1d41803 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -16,7 +16,6 @@ package executor_test import ( "fmt" "strconv" - "time" . "github.com/pingcap/check" "github.com/pingcap/errors" @@ -612,14 +611,14 @@ func (s *testSuite) TestShow2(c *C) { is := domain.GetDomain(ctx).InfoSchema() tblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) - create_time := model.TSConvert2Time(tblInfo.Meta().UpdateTS).String() - r := tk.MustQuery("show table status from test like 't'") - timeStr := time.Now().Format("2006-01-02 15:04:05") - r.Check(testkit.Rows(fmt.Sprintf("t InnoDB 10 Compact 100 100 100 100 100 100 100 %s %s %s utf8_general_ci 注释", create_time, timeStr, timeStr))) + create_time := model.TSConvert2Time(tblInfo.Meta().UpdateTS).Format("2006-01-02 15:04:05") // The Hostname is the actual host tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "192.168.0.1", AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + r := tk.MustQuery("show table status from test like 't'") + r.Check(testkit.Rows(fmt.Sprintf("t InnoDB 10 Compact 0 0 0 0 0 0 0 %s utf8mb4_bin 注释", create_time))) + tk.MustQuery("show databases like 'test'").Check(testkit.Rows("test")) tk.MustExec(`grant all on *.* to 'root'@'%'`) @@ -629,6 +628,37 @@ func (s *testSuite) TestShow2(c *C) { tk.MustQuery("show grants for current_user").Check(testkit.Rows(`GRANT ALL PRIVILEGES ON *.* TO 'root'@'%'`)) } +func (s *testSuite) TestUnprivilegedShow(c *C) { + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE DATABASE testshow") + tk.MustExec("USE testshow") + tk.MustExec("CREATE TABLE t1 (a int)") + tk.MustExec("CREATE TABLE t2 (a int)") + + tk.MustExec(`CREATE USER 'lowprivuser'`) // no grants + tk.MustExec(`FLUSH PRIVILEGES`) + + tk.Se.Auth(&auth.UserIdentity{Username: "lowprivuser", Hostname: "192.168.0.1", AuthUsername: "lowprivuser", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + rs, err := tk.Exec("SHOW TABLE STATUS FROM testshow") + c.Assert(err, IsNil) + c.Assert(rs, NotNil) + + tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "192.168.0.1", AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + tk.MustExec("GRANT ALL ON testshow.t1 TO 'lowprivuser'") + tk.MustExec(`FLUSH PRIVILEGES`) + tk.Se.Auth(&auth.UserIdentity{Username: "lowprivuser", Hostname: "192.168.0.1", AuthUsername: "lowprivuser", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + + ctx := tk.Se.(sessionctx.Context) + is := domain.GetDomain(ctx).InfoSchema() + tblInfo, err := is.TableByName(model.NewCIStr("testshow"), model.NewCIStr("t1")) + c.Assert(err, IsNil) + create_time := model.TSConvert2Time(tblInfo.Meta().UpdateTS).Format("2006-01-02 15:04:05") + + tk.MustQuery("show table status from testshow").Check(testkit.Rows(fmt.Sprintf("t1 InnoDB 10 Compact 0 0 0 0 0 0 0 %s utf8mb4_bin ", create_time))) + +} + func (s *testSuite) TestCollation(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -651,6 +681,8 @@ func (s *testSuite) TestShowTableStatus(c *C) { tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a bigint);`) + tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "192.168.0.1", AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + // It's not easy to test the result contents because every time the test runs, "Create_time" changed. tk.MustExec("show table status;") rs, err := tk.Exec("show table status;") diff --git a/infoschema/tables.go b/infoschema/tables.go index 1b91846c4c152..64e9583a7b303 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -891,6 +891,11 @@ func dataForTables(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]types.D Type: createTimeTp, } + createOptions := "" + if table.GetPartitionInfo() != nil { + createOptions = "partitioned" + } + if checker != nil && !checker.RequestVerification(schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { continue } @@ -925,7 +930,7 @@ func dataForTables(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]types.D nil, // CHECK_TIME collation, // TABLE_COLLATION nil, // CHECKSUM - "", // CREATE_OPTIONS + createOptions, // CREATE_OPTIONS table.Comment, // TABLE_COMMENT ) rows = append(rows, record) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index e5b46a915a80f..2a9b494132b03 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1617,7 +1617,7 @@ func buildShowSchema(s *ast.ShowStmt) (schema *expression.Schema) { "Create_options", "Comment"} ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeLonglong, mysql.TypeLonglong, - mysql.TypeVarchar, mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeVarchar, mysql.TypeVarchar, + mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeVarchar} case ast.ShowColumns: names = table.ColDescFieldNames(s.Full) From 8be09da457a21ad613e3b13b6ea7ff7baa354c6e Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Mon, 26 Nov 2018 21:08:02 +0800 Subject: [PATCH 03/17] stats: fix panic when dumping stats (#8448) --- statistics/dump_test.go | 18 ++++++++++++++++++ statistics/histogram.go | 4 ++-- statistics/table.go | 9 +-------- 3 files changed, 21 insertions(+), 10 deletions(-) diff --git a/statistics/dump_test.go b/statistics/dump_test.go index c165cea1669ec..f119b7a1ca9c5 100644 --- a/statistics/dump_test.go +++ b/statistics/dump_test.go @@ -120,3 +120,21 @@ PARTITION BY RANGE ( a ) ( assertTableEqual(c, originTables[i], t) } } + +func (s *testDumpStatsSuite) TestDumpAlteredTable(c *C) { + defer cleanEnv(c, s.store, s.do) + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + h := s.do.StatsHandle() + oriLease := h.Lease + h.Lease = 1 + defer func() { h.Lease = oriLease }() + tk.MustExec("create table t(a int, b int)") + tk.MustExec("analyze table t") + tk.MustExec("alter table t drop column a") + table, err := s.do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + _, err = h.DumpStatsToJSON("test", table.Meta()) + c.Assert(err, IsNil) +} diff --git a/statistics/histogram.go b/statistics/histogram.go index c4c6abdd03ade..4695cbdd45d90 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -411,7 +411,7 @@ func (hg *Histogram) greaterAndEqRowCount(value types.Datum) float64 { // lessRowCount estimates the row count where the column less than value. func (hg *Histogram) lessRowCountWithBktIdx(value types.Datum) (float64, int) { // all the values is null - if hg.Bounds == nil { + if hg.Bounds.NumRows() == 0 { return 0, 0 } index, match := hg.Bounds.LowerBound(0, &value) @@ -735,7 +735,7 @@ func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, mo return float64(c.NullCount), nil } // all the values is null - if c.Histogram.Bounds == nil { + if c.Histogram.Bounds.NumRows() == 0 { return 0.0, nil } if c.NDV > 0 && c.outOfRange(val) { diff --git a/statistics/table.go b/statistics/table.go index 078b4b9dbd5e3..4d65063143ddc 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -177,14 +177,7 @@ func (h *Handle) columnStatsFromStorage(row chunk.Row, table *Table, tableInfo * return errors.Trace(err) } col = &Column{ - Histogram: Histogram{ - ID: histID, - NDV: distinct, - NullCount: nullCount, - tp: &colInfo.FieldType, - LastUpdateVersion: histVer, - TotColSize: totColSize, - }, + Histogram: *NewHistogram(histID, distinct, nullCount, histVer, &colInfo.FieldType, 0, totColSize), Info: colInfo, Count: count + nullCount, ErrorRate: errorRate, From cb64314131a13fa66b56e71c82e7bfb98a2ab721 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Mon, 26 Nov 2018 21:24:44 +0800 Subject: [PATCH 04/17] session: add current DB to CRUCIAL OPERATION log (#8447) --- session/session.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/session/session.go b/session/session.go index 4efb0544576a4..aa5240c4458f8 100644 --- a/session/session.go +++ b/session/session.go @@ -1509,7 +1509,8 @@ func logStmt(node ast.StmtNode, vars *variable.SessionVars) { if ss, ok := node.(ast.SensitiveStmtNode); ok { log.Infof("[CRUCIAL OPERATION] con:%d schema_ver:%d %s (by %s).", vars.ConnectionID, schemaVersion, ss.SecureText(), user) } else { - log.Infof("[CRUCIAL OPERATION] con:%d schema_ver:%d %s (by %s).", vars.ConnectionID, schemaVersion, stmt.Text(), user) + log.Infof("[CRUCIAL OPERATION] con:%d schema_ver:%d cur_db:%s %s (by %s).", vars.ConnectionID, + schemaVersion, vars.CurrentDB, stmt.Text(), user) } default: logQuery(node.Text(), vars) From cd7e27d6ee246b957775731706acfc12ccc6f6db Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 26 Nov 2018 06:57:30 -0700 Subject: [PATCH 05/17] sessionctx: Set foreign_key_checks = OFF (#8358) --- expression/integration_test.go | 8 ++++++++ sessionctx/variable/sysvar.go | 19 ++++++++++--------- sessionctx/variable/varsutil.go | 12 +++++++++++- sessionctx/variable/varsutil_test.go | 10 ++++++++++ 4 files changed, 39 insertions(+), 10 deletions(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index d69982117f27e..ba5da327f80c8 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -3693,6 +3693,14 @@ func (s *testIntegrationSuite) TestValuesInNonInsertStmt(c *C) { res.Check(testkit.Rows(` `)) } +func (s *testIntegrationSuite) TestForeignKeyVar(c *C) { + + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("SET FOREIGN_KEY_CHECKS=1") + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1105 variable 'foreign_key_checks' does not yet support value: 1")) +} + func (s *testIntegrationSuite) TestUserVarMockWindFunc(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test;`) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 942ef270f9c16..07c60956c31ef 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -71,14 +71,15 @@ const ( // Variable errors var ( - UnknownStatusVar = terror.ClassVariable.New(CodeUnknownStatusVar, "unknown status variable") - UnknownSystemVar = terror.ClassVariable.New(CodeUnknownSystemVar, mysql.MySQLErrName[mysql.ErrUnknownSystemVariable]) - ErrIncorrectScope = terror.ClassVariable.New(CodeIncorrectScope, mysql.MySQLErrName[mysql.ErrIncorrectGlobalLocalVar]) - ErrUnknownTimeZone = terror.ClassVariable.New(CodeUnknownTimeZone, mysql.MySQLErrName[mysql.ErrUnknownTimeZone]) - ErrReadOnly = terror.ClassVariable.New(CodeReadOnly, "variable is read only") - ErrWrongValueForVar = terror.ClassVariable.New(CodeWrongValueForVar, mysql.MySQLErrName[mysql.ErrWrongValueForVar]) - ErrWrongTypeForVar = terror.ClassVariable.New(CodeWrongTypeForVar, mysql.MySQLErrName[mysql.ErrWrongTypeForVar]) - ErrTruncatedWrongValue = terror.ClassVariable.New(CodeTruncatedWrongValue, mysql.MySQLErrName[mysql.ErrTruncatedWrongValue]) + UnknownStatusVar = terror.ClassVariable.New(CodeUnknownStatusVar, "unknown status variable") + UnknownSystemVar = terror.ClassVariable.New(CodeUnknownSystemVar, mysql.MySQLErrName[mysql.ErrUnknownSystemVariable]) + ErrIncorrectScope = terror.ClassVariable.New(CodeIncorrectScope, mysql.MySQLErrName[mysql.ErrIncorrectGlobalLocalVar]) + ErrUnknownTimeZone = terror.ClassVariable.New(CodeUnknownTimeZone, mysql.MySQLErrName[mysql.ErrUnknownTimeZone]) + ErrReadOnly = terror.ClassVariable.New(CodeReadOnly, "variable is read only") + ErrWrongValueForVar = terror.ClassVariable.New(CodeWrongValueForVar, mysql.MySQLErrName[mysql.ErrWrongValueForVar]) + ErrWrongTypeForVar = terror.ClassVariable.New(CodeWrongTypeForVar, mysql.MySQLErrName[mysql.ErrWrongTypeForVar]) + ErrTruncatedWrongValue = terror.ClassVariable.New(CodeTruncatedWrongValue, mysql.MySQLErrName[mysql.ErrTruncatedWrongValue]) + ErrUnsupportedValueForVar = terror.ClassVariable.New(CodeUnknownStatusVar, "variable '%s' does not yet support value: %s") ) func init() { @@ -221,7 +222,7 @@ var defaultSysVars = []*SysVar{ {ScopeNone, "innodb_autoinc_lock_mode", "1"}, {ScopeGlobal, "slave_net_timeout", "3600"}, {ScopeGlobal, "key_buffer_size", "8388608"}, - {ScopeGlobal | ScopeSession, ForeignKeyChecks, "1"}, + {ScopeGlobal | ScopeSession, ForeignKeyChecks, "OFF"}, {ScopeGlobal, "host_cache_size", "279"}, {ScopeGlobal, DelayKeyWrite, "ON"}, {ScopeNone, "metadata_locks_cache_size", "1024"}, diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index de81da330c483..28d5671921478 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -236,6 +236,16 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) case FlushTime: return checkUInt64SystemVar(name, value, 0, secondsPerYear, vars) + case ForeignKeyChecks: + if strings.EqualFold(value, "ON") || value == "1" { + // TiDB does not yet support foreign keys. + // For now, resist the change and show a warning. + vars.StmtCtx.AppendWarning(ErrUnsupportedValueForVar.GenWithStackByArgs(name, value)) + return "OFF", nil + } else if strings.EqualFold(value, "OFF") || value == "0" { + return "OFF", nil + } + return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) case GroupConcatMaxLen: // The reasonable range of 'group_concat_max_len' is 4~18446744073709551615(64-bit platforms) // See https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_group_concat_max_len for details @@ -288,7 +298,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, case WarningCount, ErrorCount: return value, ErrReadOnly.GenWithStackByArgs(name) case GeneralLog, TiDBGeneralLog, AvoidTemporalUpgrade, BigTables, CheckProxyUsers, CoreFile, EndMakersInJSON, SQLLogBin, OfflineMode, - PseudoSlaveMode, LowPriorityUpdates, SkipNameResolve, ForeignKeyChecks, SQLSafeUpdates, TiDBConstraintCheckInPlace: + PseudoSlaveMode, LowPriorityUpdates, SkipNameResolve, SQLSafeUpdates, TiDBConstraintCheckInPlace: if strings.EqualFold(value, "ON") || value == "1" { return "1", nil } else if strings.EqualFold(value, "OFF") || value == "0" { diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index dd0fc336e1c85..e109a76987879 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -103,6 +103,16 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { 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")) + val, err = GetSessionSystemVar(v, "foreign_key_checks") + 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")) + val, err = GetSessionSystemVar(v, "foreign_key_checks") + c.Assert(val, Equals, "OFF") + SetSessionSystemVar(v, "sql_mode", types.NewStringDatum("strict_trans_tables")) val, err = GetSessionSystemVar(v, "sql_mode") c.Assert(err, IsNil) From 7a88c33a67e0afd2611824b00f64dbc32a38b9ea Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 26 Nov 2018 07:19:51 -0700 Subject: [PATCH 06/17] executor,planner: Improve SET PASSWORD (#8426) --- executor/simple.go | 19 ++++++++++++++----- executor/simple_test.go | 5 +++-- planner/core/logical_plan_test.go | 4 +--- planner/core/planbuilder.go | 2 +- privilege/privileges/privileges_test.go | 20 ++++++++++++++++++++ 5 files changed, 39 insertions(+), 11 deletions(-) diff --git a/executor/simple.go b/executor/simple.go index 79fd9110f3570..4e4e944052f9d 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/chunk" @@ -287,14 +288,22 @@ func userExists(ctx sessionctx.Context, name string, host string) (bool, error) } func (e *SimpleExec) executeSetPwd(s *ast.SetPwdStmt) error { + var u, h string if s.User == nil { - vars := e.ctx.GetSessionVars() - s.User = vars.User - if s.User == nil { + if e.ctx.GetSessionVars().User == nil { return errors.New("Session error is empty") } + u = e.ctx.GetSessionVars().User.AuthUsername + h = e.ctx.GetSessionVars().User.AuthHostname + } else { + checker := privilege.GetPrivilegeManager(e.ctx) + if checker != nil && !checker.RequestVerification("", "", "", mysql.SuperPriv) { + return ErrDBaccessDenied.GenWithStackByArgs(u, h, "mysql") + } + u = s.User.Username + h = s.User.Hostname } - exists, err := userExists(e.ctx, s.User.Username, s.User.Hostname) + exists, err := userExists(e.ctx, u, h) if err != nil { return errors.Trace(err) } @@ -303,7 +312,7 @@ func (e *SimpleExec) executeSetPwd(s *ast.SetPwdStmt) error { } // update mysql.user - sql := fmt.Sprintf(`UPDATE %s.%s SET password="%s" WHERE User="%s" AND Host="%s";`, mysql.SystemDB, mysql.UserTable, auth.EncodePassword(s.Password), s.User.Username, s.User.Hostname) + sql := fmt.Sprintf(`UPDATE %s.%s SET password="%s" WHERE User="%s" AND Host="%s";`, mysql.SystemDB, mysql.UserTable, auth.EncodePassword(s.Password), u, h) _, _, err = e.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(e.ctx, sql) domain.GetDomain(e.ctx).NotifyUpdatePrivilege(e.ctx) return errors.Trace(err) diff --git a/executor/simple_test.go b/executor/simple_test.go index ffd819d98d845..af0106c90adf2 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -219,15 +219,16 @@ func (s *testSuite) TestSetPwd(c *C) { tk.Se, err = session.CreateSession4Test(s.store) c.Check(err, IsNil) ctx := tk.Se.(sessionctx.Context) - ctx.GetSessionVars().User = &auth.UserIdentity{Username: "testpwd1", Hostname: "localhost"} + ctx.GetSessionVars().User = &auth.UserIdentity{Username: "testpwd1", Hostname: "localhost", AuthUsername: "testpwd1", AuthHostname: "localhost"} // Session user doesn't exist. _, err = tk.Exec(setPwdSQL) c.Check(terror.ErrorEqual(err, executor.ErrPasswordNoMatch), IsTrue, Commentf("err %v", err)) // normal - ctx.GetSessionVars().User = &auth.UserIdentity{Username: "testpwd", Hostname: "localhost"} + ctx.GetSessionVars().User = &auth.UserIdentity{Username: "testpwd", Hostname: "localhost", AuthUsername: "testpwd", AuthHostname: "localhost"} tk.MustExec(setPwdSQL) result = tk.MustQuery(`SELECT Password FROM mysql.User WHERE User="testpwd" and Host="localhost"`) result.Check(testkit.Rows(auth.EncodePassword("pwd"))) + } func (s *testSuite) TestKillStmt(c *C) { diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index c76d55e7dc973..939214e965bd6 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1709,9 +1709,7 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { }, { sql: `set password for 'root'@'%' = 'xxxxx'`, - ans: []visitInfo{ - {mysql.SuperPriv, "", "", ""}, - }, + ans: []visitInfo{}, }, { sql: `show create table test.ttt`, diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 2a9b494132b03..c64de57ec8847 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -961,7 +961,7 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) Plan { b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreateUserPriv, "", "", "") case *ast.GrantStmt: b.visitInfo = collectVisitInfoFromGrantStmt(b.visitInfo, raw) - case *ast.SetPwdStmt, *ast.RevokeStmt: + case *ast.RevokeStmt: b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "") case *ast.KillStmt: // If you have the SUPER privilege, you can kill all threads and statements. diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 48cebbe1bae9f..8b0c7dc1687d0 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -274,6 +274,26 @@ func (s *testPrivilegeSuite) TestDropTablePriv(c *C) { mustExec(c, se, `DROP TABLE todrop;`) } +func (s *testPrivilegeSuite) TestSetPasswdStmt(c *C) { + + se := newSession(c, s.store, s.dbName) + + // high privileged user setting password for other user (passes) + mustExec(c, se, "CREATE USER 'superuser'") + mustExec(c, se, "CREATE USER 'nobodyuser'") + mustExec(c, se, "GRANT ALL ON *.* TO 'superuser'") + mustExec(c, se, "FLUSH PRIVILEGES") + + c.Assert(se.Auth(&auth.UserIdentity{Username: "superuser", Hostname: "localhost", AuthUsername: "superuser", AuthHostname: "%"}, nil, nil), IsTrue) + mustExec(c, se, "SET PASSWORD for 'nobodyuser' = 'newpassword'") + + // low privileged user trying to set password for other user (fails) + c.Assert(se.Auth(&auth.UserIdentity{Username: "nobodyuser", Hostname: "localhost", AuthUsername: "nobodyuser", AuthHostname: "%"}, nil, nil), IsTrue) + _, err := se.Execute(context.Background(), "SET PASSWORD for 'superuser' = 'newpassword'") + c.Assert(err, NotNil) + +} + func (s *testPrivilegeSuite) TestCheckAuthenticate(c *C) { se := newSession(c, s.store, s.dbName) From 5035e48bb528d2c7a68e769bb62edafaa3aeb93f Mon Sep 17 00:00:00 2001 From: Gao Zhiyuan Date: Mon, 26 Nov 2018 10:03:37 -0500 Subject: [PATCH 07/17] executor, sessionctx: add checks for system variable validate_password_xxxx (#8227) --- executor/set_test.go | 9 +++++++++ go.sum | 2 ++ sessionctx/variable/sysvar.go | 7 +++++-- sessionctx/variable/varsutil.go | 2 ++ 4 files changed, 18 insertions(+), 2 deletions(-) diff --git a/executor/set_test.go b/executor/set_test.go index b82cf70eb49b3..f3e6eacbf8a85 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -516,6 +516,15 @@ func (s *testSuite) TestValidateSetVar(c *C) { tk.MustExec("set @@innodb_lock_wait_timeout = 1073741825") tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect innodb_lock_wait_timeout value: '1073741825'")) + tk.MustExec("set @@global.validate_password_number_count=-1") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect validate_password_number_count value: '-1'")) + + tk.MustExec("set @@global.validate_password_length=-1") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect validate_password_length value: '-1'")) + + tk.MustExec("set @@global.validate_password_length=8") + tk.MustQuery("show warnings").Check(testkit.Rows()) + _, err = tk.Exec("set @@tx_isolation=''") c.Assert(terror.ErrorEqual(err, variable.ErrWrongValueForVar), IsTrue, Commentf("err %v", err)) diff --git a/go.sum b/go.sum index f40c13547dbb5..2fd165738ab9c 100644 --- a/go.sum +++ b/go.sum @@ -7,6 +7,8 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03 github.com/Shopify/sarama v1.16.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= github.com/Shopify/toxiproxy v2.1.3+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= +github.com/alapha23/parser v0.0.0-20181126070418-f5119577d517 h1:TfOLSy1ypCKqWjSwpV9zx4yyX5GNoJvsQM3KBReoT6Y= +github.com/alapha23/parser v0.0.0-20181126070418-f5119577d517/go.mod h1:43oaFTPY5wIvNxt3TSa+1SZtJ645w/1AlsDmSioWtuQ= github.com/apache/thrift v0.0.0-20161221203622-b2a4d4ae21c7 h1:CZI8h5fmYwCCvd2RMSsjLqHN6OqABlWJweFKxz4vdEs= github.com/apache/thrift v0.0.0-20161221203622-b2a4d4ae21c7/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/beorn7/perks v0.0.0-20160229213445-3ac7bf7a47d1/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 07c60956c31ef..d702dd66b70f3 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -127,6 +127,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal, "slave_pending_jobs_size_max", "16777216"}, {ScopeNone, "innodb_sync_array_size", "1"}, {ScopeSession, "rand_seed2", ""}, + {ScopeGlobal, "validate_password_check_user_name", "OFF"}, {ScopeGlobal, "validate_password_number_count", "1"}, {ScopeSession, "gtid_next", ""}, {ScopeGlobal | ScopeSession, SQLSelectLimit, "18446744073709551615"}, @@ -270,7 +271,6 @@ var defaultSysVars = []*SysVar{ {ScopeNone, "performance_schema_max_file_classes", "50"}, {ScopeGlobal, "expire_logs_days", "0"}, {ScopeGlobal | ScopeSession, "binlog_rows_query_log_events", "OFF"}, - {ScopeGlobal, "validate_password_policy", "1"}, {ScopeGlobal, "default_password_lifetime", ""}, {ScopeNone, "pid_file", "/usr/local/mysql/data/localhost.pid"}, {ScopeNone, "innodb_undo_tablespaces", "0"}, @@ -597,7 +597,6 @@ var defaultSysVars = []*SysVar{ {ScopeNone, "innodb_ft_max_token_size", "84"}, {ScopeGlobal, "validate_password_length", "8"}, {ScopeGlobal, "ndb_log_binlog_index", ""}, - {ScopeGlobal, "validate_password_mixed_case_count", "1"}, {ScopeGlobal, "innodb_api_bk_commit_interval", "5"}, {ScopeNone, "innodb_undo_directory", "."}, {ScopeNone, "bind_address", "*"}, @@ -775,6 +774,10 @@ const ( SyncBinlog = "sync_binlog" // BlockEncryptionMode is the name for 'block_encryption_mode' system variable. BlockEncryptionMode = "block_encryption_mode" + // ValidatePasswordNumberCount is the name of 'validate_password_number_count' system variable. + ValidatePasswordNumberCount = "validate_password_number_count" + // ValidatePasswordLength is the name of 'validate_password_length' system variable. + ValidatePasswordLength = "validate_password_length" ) // GlobalVarAccessor is the interface for accessing global scope system and status variables. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 28d5671921478..79143d0155965 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -295,6 +295,8 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return "SYSTEM", nil } return value, nil + case ValidatePasswordLength, ValidatePasswordNumberCount: + return checkUInt64SystemVar(name, value, 0, math.MaxUint64, vars) case WarningCount, ErrorCount: return value, ErrReadOnly.GenWithStackByArgs(name) case GeneralLog, TiDBGeneralLog, AvoidTemporalUpgrade, BigTables, CheckProxyUsers, CoreFile, EndMakersInJSON, SQLLogBin, OfflineMode, From a3e25b2f8f286694d8ee35d61d04ec755136811d Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Mon, 26 Nov 2018 23:16:36 +0800 Subject: [PATCH 08/17] ddl: add some test code to mock genGlobalID fail (#8440) --- ddl/ddl.go | 6 ++++++ ddl/fail_db_test.go | 47 +++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 53 insertions(+) diff --git a/ddl/ddl.go b/ddl/ddl.go index 030268e74feef..e8b8adb019e4c 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -431,6 +431,12 @@ func (d *ddl) genGlobalID() (int64, error) { var globalID int64 err := kv.RunInNewTxn(d.store, true, func(txn kv.Transaction) error { var err error + + // gofail: var mockGenGlobalIDFail bool + // if mockGenGlobalIDFail { + // return errors.New("gofail genGlobalID error") + // } + globalID, err = meta.NewMeta(txn).GenGlobalID() return errors.Trace(err) }) diff --git a/ddl/fail_db_test.go b/ddl/fail_db_test.go index a55a64027ee6b..9f62fc988ba08 100644 --- a/ddl/fail_db_test.go +++ b/ddl/fail_db_test.go @@ -15,6 +15,7 @@ package ddl_test import ( "fmt" + "math/rand" "time" gofail "github.com/etcd-io/gofail/runtime" @@ -197,3 +198,49 @@ func (s *testDBSuite) TestAddIndexFailed(c *C) { tk.MustExec("admin check index t idx_b") tk.MustExec("admin check table t") } + +func (s *testDBSuite) TestGenGlobalIDFail(c *C) { + defer gofail.Disable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail") + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists gen_global_id_fail") + tk.MustExec("use gen_global_id_fail") + + sql1 := "create table t1(a bigint PRIMARY KEY, b int)" + sql2 := `create table t2(a bigint PRIMARY KEY, b int) partition by range (a) ( + partition p0 values less than (3440), + partition p1 values less than (61440), + partition p2 values less than (122880), + partition p3 values less than maxvalue)` + sql3 := `truncate table t1` + sql4 := `truncate table t2` + + testcases := []struct { + sql string + table string + mockErr bool + }{ + {sql1, "t1", true}, + {sql2, "t2", true}, + {sql1, "t1", false}, + {sql2, "t2", false}, + {sql3, "t1", true}, + {sql4, "t2", true}, + {sql3, "t1", false}, + {sql4, "t2", false}, + } + + for idx, test := range testcases { + if test.mockErr { + gofail.Enable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail", `return(true)`) + _, err := tk.Exec(test.sql) + c.Assert(err, NotNil, Commentf("the %dth test case '%s' fail", idx, test.sql)) + } else { + gofail.Enable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail", `return(false)`) + tk.MustExec(test.sql) + tk.MustExec(fmt.Sprintf("insert into %s values (%d, 42)", test.table, rand.Intn(65536))) + tk.MustExec(fmt.Sprintf("admin check table %s", test.table)) + } + } + tk.MustExec("admin check table t1") + tk.MustExec("admin check table t2") +} From 0d005a0b56734cf43590fab5d96481b7bf472ff6 Mon Sep 17 00:00:00 2001 From: exialin Date: Tue, 27 Nov 2018 09:11:33 +0800 Subject: [PATCH 09/17] executor: fix overflow error message of `update` (#8457) --- executor/update.go | 4 ++++ executor/write_test.go | 7 +++++++ 2 files changed, 11 insertions(+) diff --git a/executor/update.go b/executor/update.go index d589a05c00570..e220ea3786b31 100644 --- a/executor/update.go +++ b/executor/update.go @@ -190,6 +190,10 @@ func (e *UpdateExec) handleErr(colName model.CIStr, rowIdx int, err error) error return resetErrDataTooLong(colName.O, rowIdx+1, err) } + if types.ErrOverflow.Equal(err) { + return types.ErrWarnDataOutOfRange.GenWithStackByArgs(colName.O, rowIdx+1) + } + return errors.Trace(err) } diff --git a/executor/write_test.go b/executor/write_test.go index c380ee9b89a9b..042d77a2ca306 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1081,6 +1081,13 @@ func (s *testSuite) TestUpdate(c *C) { tk.MustExec(`CREATE TABLE t1 (c1 float)`) tk.MustExec("INSERT INTO t1 SET c1 = 1") tk.MustExec("UPDATE t1 SET c1 = 1.2 WHERE c1=1;") + + // issue 8119 + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (c1 float(1,1));") + tk.MustExec("insert into t values (0.0);") + _, err = tk.Exec("update t set c1 = 2.0;") + c.Assert(types.ErrWarnDataOutOfRange.Equal(err), IsTrue) } func (s *testSuite) TestPartitionedTableUpdate(c *C) { From 0ac0e154d957b519796f1088b0bc9f9ba5c2e639 Mon Sep 17 00:00:00 2001 From: Andrew Date: Tue, 27 Nov 2018 11:00:41 +0800 Subject: [PATCH 10/17] planner, executor: fix `PREPARE FROM @var_name` (#8437) --- executor/prepared_test.go | 14 ++++++++++++++ planner/core/planbuilder.go | 7 +++++-- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 0338145656173..07e54ef4ff356 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -214,6 +214,20 @@ func (s *testSuite) TestPrepared(c *C) { c.Assert(err, IsNil) c.Assert(len(fields), Equals, 0) + // issue 8074 + tk.MustExec("drop table if exists prepare1;") + tk.MustExec("create table prepare1 (a decimal(1))") + tk.MustExec("insert into prepare1 values(1);") + _, err = tk.Exec("prepare stmt FROM @sql1") + c.Assert(err.Error(), Equals, "line 1 column 4 near \"\" (total length 4)") + tk.MustExec("SET @sql = 'update prepare1 set a=5 where a=?';") + _, err = tk.Exec("prepare stmt FROM @sql") + c.Assert(err, IsNil) + tk.MustExec("set @var=1;") + _, err = tk.Exec("execute stmt using @var") + c.Assert(err, IsNil) + tk.MustQuery("select a from prepare1;").Check(testkit.Rows("5")) + // Coverage. exec := &executor.ExecuteExec{} exec.Next(ctx, nil) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index c64de57ec8847..abf67bf9ff21c 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -401,8 +401,11 @@ func (b *PlanBuilder) buildPrepare(x *ast.PrepareStmt) Plan { Name: x.Name, } if x.SQLVar != nil { - // TODO: Prepared statement from variable expression do not work as expected. - // p.SQLText, _ = x.SQLVar.GetValue().(string) + if v, ok := b.ctx.GetSessionVars().Users[x.SQLVar.Name]; ok { + p.SQLText = v + } else { + p.SQLText = "NULL" + } } else { p.SQLText = x.SQLText } From 487bb50390bbe0c712a4b850c20bf7cc12f4d061 Mon Sep 17 00:00:00 2001 From: Andrew Date: Tue, 27 Nov 2018 11:35:52 +0800 Subject: [PATCH 11/17] planner, executor: fix tp.Flen size when union with castIntAsString (#8442) --- executor/executor_test.go | 6 ++++++ planner/core/logical_plan_builder.go | 3 +++ 2 files changed, 9 insertions(+) diff --git a/executor/executor_test.go b/executor/executor_test.go index d289cad4993e9..33deeeeb02a5a 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1102,6 +1102,12 @@ func (s *testSuite) TestUnion(c *C) { for i := 0; i < 4; i++ { tk.MustQuery("SELECT(SELECT 0 AS a FROM dual UNION SELECT 1 AS a FROM dual ORDER BY a ASC LIMIT 1) AS dev").Check(testkit.Rows("0")) } + + // #issue 8231 + tk.MustExec("drop table if exists t1") + tk.MustExec("CREATE TABLE t1 (uid int(1))") + tk.MustExec("INSERT INTO t1 SELECT 150") + tk.MustQuery("SELECT 'a' UNION SELECT uid FROM t1 order by 1 desc;").Check(testkit.Rows("a", "150")) } func (s *testSuite) TestNeighbouringProj(c *C) { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 3448c62786ec2..0b63efe58a945 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -659,6 +659,9 @@ func unionJoinFieldType(a, b *types.FieldType) *types.FieldType { resultTp.Decimal = mathutil.Max(a.Decimal, b.Decimal) // `Flen - Decimal` is the fraction before '.' resultTp.Flen = mathutil.Max(a.Flen-a.Decimal, b.Flen-b.Decimal) + resultTp.Decimal + if resultTp.EvalType() != types.ETInt && (a.EvalType() == types.ETInt || b.EvalType() == types.ETInt) && resultTp.Flen < mysql.MaxIntWidth { + resultTp.Flen = mysql.MaxIntWidth + } resultTp.Charset = a.Charset resultTp.Collate = a.Collate expression.SetBinFlagOrBinStr(b, resultTp) From 1927cd9a3018b5b59eba083ee03a82ec2d358473 Mon Sep 17 00:00:00 2001 From: lysu Date: Tue, 27 Nov 2018 15:37:12 +0800 Subject: [PATCH 12/17] session: release datum memory after transaction finished (#8458) --- session/session.go | 2 ++ session/txn.go | 4 ++++ sessionctx/variable/session.go | 9 +++++++++ 3 files changed, 15 insertions(+) diff --git a/session/session.go b/session/session.go index aa5240c4458f8..921b4c4ee17bd 100644 --- a/session/session.go +++ b/session/session.go @@ -418,6 +418,7 @@ func (s *session) CommitTxn(ctx context.Context) error { if err != nil { label = metrics.LblError } + s.sessionVars.TxnCtx.Cleanup() metrics.TransactionCounter.WithLabelValues(s.getSQLLabel(), label).Inc() return errors.Trace(err) } @@ -435,6 +436,7 @@ func (s *session) RollbackTxn(ctx context.Context) error { } s.cleanRetryInfo() s.txn.changeToInvalid() + s.sessionVars.TxnCtx.Cleanup() s.sessionVars.SetStatusFlag(mysql.ServerStatusInTrans, false) return errors.Trace(err) } diff --git a/session/txn.go b/session/txn.go index 7239de08c8037..56aeb82416b25 100644 --- a/session/txn.go +++ b/session/txn.go @@ -196,6 +196,10 @@ func (st *TxnState) cleanup() { delete(st.mutations, key) } if st.dirtyTableOP != nil { + empty := dirtyTableOperation{} + for i := 0; i < len(st.dirtyTableOP); i++ { + st.dirtyTableOP[i] = empty + } st.dirtyTableOP = st.dirtyTableOP[:0] } } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 060b323aa3bb9..c989c16500943 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -125,6 +125,15 @@ func (tc *TransactionContext) UpdateDeltaForTable(tableID int64, delta int64, co tc.TableDeltaMap[tableID] = item } +// Cleanup clears up transaction info that no longer use. +func (tc *TransactionContext) Cleanup() { + //tc.InfoSchema = nil; we cannot do it now, because some operation like handleFieldList depend on this. + tc.DirtyDB = nil + tc.Binlog = nil + tc.Histroy = nil + tc.TableDeltaMap = nil +} + // ClearDelta clears the delta map. func (tc *TransactionContext) ClearDelta() { tc.TableDeltaMap = nil From cadab30b4a6f404e379e3f7f228e44a8ab24febc Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 27 Nov 2018 16:09:55 +0800 Subject: [PATCH 13/17] domain,session: simplify the session pool of domain (#8456) --- domain/domain.go | 64 +++++++++++++++++++++++++++++++++++++++++--- session/session.go | 7 ++++- session/tidb_test.go | 2 -- 3 files changed, 67 insertions(+), 6 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 9113f6585c2e3..15058ee13534b 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -60,7 +60,7 @@ type Domain struct { info *InfoSyncer m sync.Mutex SchemaValidator SchemaValidator - sysSessionPool *pools.ResourcePool + sysSessionPool *sessionPool exit chan struct{} etcdClient *clientv3.Client wg sync.WaitGroup @@ -526,7 +526,7 @@ func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duratio store: store, SchemaValidator: NewSchemaValidator(ddlLease), exit: make(chan struct{}), - sysSessionPool: pools.NewResourcePool(factory, capacity, capacity, resourceIdleTimeout), + sysSessionPool: newSessionPool(capacity, factory), statsLease: statsLease, infoHandle: infoschema.NewHandle(store), slowQuery: newTopNSlowQueries(30, time.Hour*24*7, 500), @@ -606,8 +606,66 @@ func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.R return nil } +type sessionPool struct { + resources chan pools.Resource + factory pools.Factory + mu struct { + sync.RWMutex + closed bool + } +} + +func newSessionPool(cap int, factory pools.Factory) *sessionPool { + return &sessionPool{ + resources: make(chan pools.Resource, cap), + factory: factory, + } +} + +func (p *sessionPool) Get() (resource pools.Resource, err error) { + var ok bool + select { + case resource, ok = <-p.resources: + if !ok { + err = errors.New("session pool closed") + } + default: + resource, err = p.factory() + } + return +} + +func (p *sessionPool) Put(resource pools.Resource) { + p.mu.RLock() + defer p.mu.RUnlock() + if p.mu.closed { + resource.Close() + return + } + + select { + case p.resources <- resource: + default: + resource.Close() + } +} +func (p *sessionPool) Close() { + p.mu.Lock() + if p.mu.closed { + p.mu.Unlock() + return + } + p.mu.closed = true + close(p.resources) + p.mu.Unlock() + + for r := range p.resources { + r.Close() + } +} + // SysSessionPool returns the system session pool. -func (do *Domain) SysSessionPool() *pools.ResourcePool { +func (do *Domain) SysSessionPool() *sessionPool { return do.sysSessionPool } diff --git a/session/session.go b/session/session.go index 921b4c4ee17bd..a9f6863e60dda 100644 --- a/session/session.go +++ b/session/session.go @@ -579,7 +579,12 @@ func sqlForLog(sql string) string { return executor.QueryReplacer.Replace(sql) } -func (s *session) sysSessionPool() *pools.ResourcePool { +type sessionPool interface { + Get() (pools.Resource, error) + Put(pools.Resource) +} + +func (s *session) sysSessionPool() sessionPool { return domain.GetDomain(s).SysSessionPool() } diff --git a/session/tidb_test.go b/session/tidb_test.go index 6cb95320a7c0c..c539b77044c62 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -143,8 +143,6 @@ func (s *testMainSuite) TestSysSessionPoolGoroutineLeak(c *C) { }(se) } wg.Wait() - se.sysSessionPool().Close() - c.Assert(se.sysSessionPool().IsClosed(), Equals, true) } func newStore(c *C, dbPath string) kv.Storage { From 654964a35b74e7122de6493c5bc784d52d7caa24 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 27 Nov 2018 17:51:47 +0800 Subject: [PATCH 14/17] server: close connection when tidb server closed. (#8446) --- server/server.go | 23 +++++++++++++++++++---- tidb-server/main.go | 2 ++ 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/server/server.go b/server/server.go index 49455fe1d4e5c..5e516493673bb 100644 --- a/server/server.go +++ b/server/server.go @@ -351,17 +351,32 @@ func (s *Server) Kill(connectionID uint64, query bool) { return } + killConn(conn, query) +} + +func killConn(conn *clientConn, query bool) { + if !query { + // Mark the client connection status as WaitShutdown, when the goroutine detect + // this, it will end the dispatch loop and exit. + atomic.StoreInt32(&conn.status, connStatusWaitShutdown) + } + conn.mu.RLock() cancelFunc := conn.mu.cancelFunc conn.mu.RUnlock() if cancelFunc != nil { cancelFunc() } +} - if !query { - // Mark the client connection status as WaitShutdown, when the goroutine detect - // this, it will end the dispatch loop and exit. - atomic.StoreInt32(&conn.status, connStatusWaitShutdown) +// KillAllConnections kills all connections when server is not gracefully shutdown. +func (s *Server) KillAllConnections() { + s.rwlock.Lock() + defer s.rwlock.Unlock() + log.Info("[server] kill all connections.") + + for _, conn := range s.clients { + killConn(conn, false) } } diff --git a/tidb-server/main.go b/tidb-server/main.go index a1062d9fa59a8..2960db0210cd6 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -513,6 +513,8 @@ func closeDomainAndStorage() { func cleanup() { if graceful { svr.GracefulDown() + } else { + svr.KillAllConnections() } closeDomainAndStorage() } From 73c91738d70f10dadcd79cf527742ddc73c82694 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Tue, 27 Nov 2018 04:06:17 -0700 Subject: [PATCH 15/17] executor: privilege check USE command (#8418) --- executor/simple.go | 17 +++++++++++++++++ privilege/privileges/privileges_test.go | 25 +++++++++++++++++++++++++ server/server_test.go | 12 ++++++++---- 3 files changed, 50 insertions(+), 4 deletions(-) diff --git a/executor/simple.go b/executor/simple.go index 4e4e944052f9d..73f6bb395da73 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -84,8 +84,25 @@ func (e *SimpleExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { return errors.Trace(err) } +func (e *SimpleExec) dbAccessDenied(dbname string) error { + user := e.ctx.GetSessionVars().User + u := user.Username + h := user.Hostname + if len(user.AuthUsername) > 0 && len(user.AuthHostname) > 0 { + u = user.AuthUsername + h = user.AuthHostname + } + return ErrDBaccessDenied.GenWithStackByArgs(u, h, dbname) +} + func (e *SimpleExec) executeUse(s *ast.UseStmt) error { dbname := model.NewCIStr(s.DBName) + + checker := privilege.GetPrivilegeManager(e.ctx) + if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.DBIsVisible(fmt.Sprint(dbname)) { + return e.dbAccessDenied(dbname.O) + } + dbinfo, exists := e.is.SchemaByName(dbname) if !exists { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(dbname) diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 8b0c7dc1687d0..22c71ce98c58c 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -323,6 +323,31 @@ func (s *testPrivilegeSuite) TestCheckAuthenticate(c *C) { c.Assert(se.Auth(&auth.UserIdentity{Username: "u4", Hostname: "localhost"}, nil, nil), IsFalse) } +func (s *testPrivilegeSuite) TestUseDb(c *C) { + + se := newSession(c, s.store, s.dbName) + // high privileged user + mustExec(c, se, "CREATE USER 'usesuper'") + mustExec(c, se, "CREATE USER 'usenobody'") + mustExec(c, se, "GRANT ALL ON *.* TO 'usesuper'") + mustExec(c, se, "FLUSH PRIVILEGES") + c.Assert(se.Auth(&auth.UserIdentity{Username: "usesuper", Hostname: "localhost", AuthUsername: "usesuper", AuthHostname: "%"}, nil, nil), IsTrue) + mustExec(c, se, "use mysql") + // low privileged user + c.Assert(se.Auth(&auth.UserIdentity{Username: "usenobody", Hostname: "localhost", AuthUsername: "usenobody", AuthHostname: "%"}, nil, nil), IsTrue) + _, err := se.Execute(context.Background(), "use mysql") + c.Assert(err, NotNil) + + // try again after privilege granted + c.Assert(se.Auth(&auth.UserIdentity{Username: "usesuper", Hostname: "localhost", AuthUsername: "usesuper", AuthHostname: "%"}, nil, nil), IsTrue) + mustExec(c, se, "GRANT SELECT ON mysql.* TO 'usenobody'") + mustExec(c, se, "FLUSH PRIVILEGES") + c.Assert(se.Auth(&auth.UserIdentity{Username: "usenobody", Hostname: "localhost", AuthUsername: "usenobody", AuthHostname: "%"}, nil, nil), IsTrue) + _, err = se.Execute(context.Background(), "use mysql") + c.Assert(err, IsNil) + +} + func (s *testPrivilegeSuite) TestInformationSchema(c *C) { // This test tests no privilege check for INFORMATION_SCHEMA database. diff --git a/server/server_test.go b/server/server_test.go index 7bb5b43635226..e7eed294b7ee2 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -619,13 +619,14 @@ func runTestShowProcessList(c *C) { func runTestAuth(c *C) { runTests(c, nil, func(dbt *DBTest) { dbt.mustExec(`CREATE USER 'authtest'@'%' IDENTIFIED BY '123';`) + dbt.mustExec(`GRANT ALL on test.* to 'authtest'`) dbt.mustExec(`FLUSH PRIVILEGES;`) }) runTests(c, func(config *mysql.Config) { config.User = "authtest" config.Passwd = "123" }, func(dbt *DBTest) { - dbt.mustExec(`USE mysql;`) + dbt.mustExec(`USE information_schema;`) }) db, err := sql.Open("mysql", getDSN(func(config *mysql.Config) { @@ -633,20 +634,21 @@ func runTestAuth(c *C) { config.Passwd = "456" })) c.Assert(err, IsNil) - _, err = db.Query("USE mysql;") + _, err = db.Query("USE information_schema;") c.Assert(err, NotNil, Commentf("Wrong password should be failed")) db.Close() // Test login use IP that not exists in mysql.user. runTests(c, nil, func(dbt *DBTest) { dbt.mustExec(`CREATE USER 'authtest2'@'localhost' IDENTIFIED BY '123';`) + dbt.mustExec(`GRANT ALL on test.* to 'authtest2'@'localhost'`) dbt.mustExec(`FLUSH PRIVILEGES;`) }) runTests(c, func(config *mysql.Config) { config.User = "authtest2" config.Passwd = "123" }, func(dbt *DBTest) { - dbt.mustExec(`USE mysql;`) + dbt.mustExec(`USE information_schema;`) }) } @@ -683,7 +685,9 @@ func runTestIssue3680(c *C) { func runTestIssue3682(c *C) { runTests(c, nil, func(dbt *DBTest) { dbt.mustExec(`CREATE USER 'issue3682'@'%' IDENTIFIED BY '123';`) - dbt.mustExec(`FLUSH PRIVILEGES;`) + dbt.mustExec(`GRANT ALL on test.* to 'issue3682'`) + dbt.mustExec(`GRANT ALL on mysql.* to 'issue3682'`) + dbt.mustExec(`FLUSH PRIVILEGES`) }) runTests(c, func(config *mysql.Config) { config.User = "issue3682" From e2e8d678199508bc9290b9afbc332036b9971861 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Tue, 27 Nov 2018 20:33:12 +0800 Subject: [PATCH 16/17] table, executor: set a real enum as the default enum value (#8469) --- executor/write_test.go | 8 ++++++++ table/column.go | 6 +++++- table/column_test.go | 2 +- types/datum.go | 6 ++++++ 4 files changed, 20 insertions(+), 2 deletions(-) diff --git a/executor/write_test.go b/executor/write_test.go index 042d77a2ca306..70875e530b820 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -2115,3 +2115,11 @@ func (s *testSuite) TestDeferConstraintCheckForInsert(c *C) { tk.MustExec(`commit;`) tk.MustQuery(`select * from t;`).Check(testkit.Rows("2")) } + +func (s *testSuite) TestDefEnumInsert(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table test (id int, prescription_type enum('a','b','c','d','e','f') NOT NULL, primary key(id));") + tk.MustExec("insert into test (id) values (1)") + tk.MustQuery("select prescription_type from test").Check(testkit.Rows("a")) +} diff --git a/table/column.go b/table/column.go index 67f6d35b6041b..0f664da51a3f7 100644 --- a/table/column.go +++ b/table/column.go @@ -358,7 +358,11 @@ func getColDefaultValueFromNil(ctx sessionctx.Context, col *model.ColumnInfo) (t if col.Tp == mysql.TypeEnum { // For enum type, if no default value and not null is set, // the default value is the first element of the enum list - return types.NewDatum(col.FieldType.Elems[0]), nil + defEnum, err := types.ParseEnumValue(col.FieldType.Elems, 1) + if err != nil { + return types.Datum{}, err + } + return types.NewMysqlEnumDatum(defEnum), nil } if mysql.HasAutoIncrementFlag(col.Flag) { // Auto increment column doesn't has default value and we should not return error. diff --git a/table/column_test.go b/table/column_test.go index 3d34a1436892f..e197b58747834 100644 --- a/table/column_test.go +++ b/table/column_test.go @@ -303,7 +303,7 @@ func (t *testTableSuite) TestGetDefaultValue(c *C) { }, }, false, - types.NewStringDatum("abc"), + types.NewMysqlEnumDatum(types.Enum{Name: "abc", Value: 1}), nil, }, { diff --git a/types/datum.go b/types/datum.go index 20f70a50ad19a..86d94792f2762 100644 --- a/types/datum.go +++ b/types/datum.go @@ -1757,6 +1757,12 @@ func NewMysqlBitDatum(b BinaryLiteral) (d Datum) { return d } +// NewMysqlEnumDatum creates a new MysqlEnum Datum for a Enum value. +func NewMysqlEnumDatum(e Enum) (d Datum) { + d.SetMysqlEnum(e) + return d +} + // MakeDatums creates datum slice from interfaces. func MakeDatums(args ...interface{}) []Datum { datums := make([]Datum, len(args)) From e5dc251dd3e6826f8c23bef645ea2a3b10a970f7 Mon Sep 17 00:00:00 2001 From: ciscoxll Date: Tue, 27 Nov 2018 22:14:37 +0800 Subject: [PATCH 17/17] ddl: fix concurrent access to `do.infoHandle` has a data race error (#8287) --- ddl/fail_db_test.go | 14 ++++++++------ domain/domain.go | 5 ----- 2 files changed, 8 insertions(+), 11 deletions(-) diff --git a/ddl/fail_db_test.go b/ddl/fail_db_test.go index 9f62fc988ba08..85d3eaaf8e6db 100644 --- a/ddl/fail_db_test.go +++ b/ddl/fail_db_test.go @@ -92,8 +92,9 @@ func (s *testFailDBSuite) TestHalfwayCancelOperations(c *C) { c.Assert(row.Len(), Equals, 1) c.Assert(row.GetInt64(0), DeepEquals, int64(1)) c.Assert(rs[0].Close(), IsNil) - // Reload schema. - s.dom.ResetHandle(s.store) + // Execute ddl statement reload schema. + _, err = s.se.Execute(context.Background(), "alter table t comment 'test1'") + c.Assert(err, IsNil) err = s.dom.DDL().(ddl.DDLForTest).GetHook().OnChanged(nil) c.Assert(err, IsNil) s.se, err = session.CreateSession4Test(s.store) @@ -105,8 +106,8 @@ func (s *testFailDBSuite) TestHalfwayCancelOperations(c *C) { c.Assert(err, IsNil) // test for renaming table - gofail.Enable("github.com/pingcap/tidb/ddl/errRenameTable", `return(true)`) - defer gofail.Disable("github.com/pingcap/tidb/ddl/errRenameTable") + gofail.Enable("github.com/pingcap/tidb/ddl/renameTableErr", `return(true)`) + defer gofail.Disable("github.com/pingcap/tidb/ddl/renameTableErr") _, err = s.se.Execute(context.Background(), "create table tx(a int)") c.Assert(err, IsNil) _, err = s.se.Execute(context.Background(), "insert into tx values(1)") @@ -124,8 +125,9 @@ func (s *testFailDBSuite) TestHalfwayCancelOperations(c *C) { c.Assert(row.Len(), Equals, 1) c.Assert(row.GetInt64(0), DeepEquals, int64(1)) c.Assert(rs[0].Close(), IsNil) - // Reload schema. - s.dom.ResetHandle(s.store) + // Execute ddl statement reload schema. + _, err = s.se.Execute(context.Background(), "alter table tx comment 'tx'") + c.Assert(err, IsNil) err = s.dom.DDL().(ddl.DDLForTest).GetHook().OnChanged(nil) c.Assert(err, IsNil) s.se, err = session.CreateSession4Test(s.store) diff --git a/domain/domain.go b/domain/domain.go index 15058ee13534b..00ffba1fa52ca 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -533,11 +533,6 @@ func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duratio } } -// ResetHandle resets the domain's infoschema handle. It is used for testing. -func (do *Domain) ResetHandle(store kv.Storage) { - do.infoHandle = infoschema.NewHandle(store) -} - // Init initializes a domain. func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.Resource, error)) error { if ebd, ok := do.store.(EtcdBackend); ok {