From ce88d7b7fddfc50f82adf43751d53b1aa2713486 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Wed, 11 Dec 2019 14:38:59 +0800 Subject: [PATCH 1/4] types: fix insert error when convert string to float (#13716) (#14011) --- executor/insert_test.go | 21 +++++++++++++++++++++ executor/write_test.go | 2 +- expression/builtin_cast.go | 8 ++++++++ planner/core/logical_plan_builder.go | 1 - types/convert.go | 2 +- types/convert_test.go | 7 ++----- 6 files changed, 33 insertions(+), 8 deletions(-) diff --git a/executor/insert_test.go b/executor/insert_test.go index 0751d884750f9..d49a37d464f0d 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -770,3 +770,24 @@ func (s *testSuite3) TestJiraIssue5366(c *C) { tk.MustExec(` insert into bug select ifnull(JSON_UNQUOTE(JSON_EXTRACT('[{"amount":2000,"feeAmount":0,"merchantNo":"20190430140319679394","shareBizCode":"20160311162_SECOND"}]', '$[0].merchantNo')),'') merchant_no union SELECT '20180531557' merchant_no;`) tk.MustQuery(`select * from bug`).Sort().Check(testkit.Rows("20180531557", "20190430140319679394")) } + +func (s *testSuite3) TestDMLCast(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (a int, b double)`) + tk.MustExec(`insert into t values (ifnull('',0)+0, 0)`) + tk.MustExec(`insert into t values (0, ifnull('',0)+0)`) + tk.MustQuery(`select * from t`).Check(testkit.Rows("0 0", "0 0")) + _, err := tk.Exec(`insert into t values ('', 0)`) + c.Assert(err, NotNil) + _, err = tk.Exec(`insert into t values (0, '')`) + c.Assert(err, NotNil) + _, err = tk.Exec(`update t set a = ''`) + c.Assert(err, NotNil) + _, err = tk.Exec(`update t set b = ''`) + c.Assert(err, NotNil) + tk.MustExec("update t set a = ifnull('',0)+0") + tk.MustExec("update t set b = ifnull('',0)+0") + tk.MustExec("delete from t where a = ''") + tk.MustQuery(`select * from t`).Check(testkit.Rows()) +} diff --git a/executor/write_test.go b/executor/write_test.go index c2ef457fe1074..305188ac17bc2 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1498,7 +1498,7 @@ func (s *testSuite) TestUpdate(c *C) { tk.MustExec("update t set a = ''") tk.MustQuery("select * from t").Check(testkit.Rows("0000-00-00 00:00:00 1999-12-13 00:00:00")) tk.MustExec("update t set b = ''") - tk.MustQuery("select * from t").Check(testkit.Rows("0000-00-00 00:00:00 ")) + tk.MustQuery("select * from t").Check(testkit.Rows("0000-00-00 00:00:00 0000-00-00 00:00:00")) tk.MustExec("set @@sql_mode=@orig_sql_mode;") tk.MustExec("create view v as select * from t") diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index a2f39e8eccff7..2e6aa0c7a2b64 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -1095,6 +1095,10 @@ func (b *builtinCastStringAsIntSig) evalInt(row chunk.Row) (res int64, isNull bo if len(val) > 1 && val[0] == '-' { // negative number isNegative = true } + sctx := b.ctx.GetSessionVars().StmtCtx + if val == "" && (sctx.InInsertStmt || sctx.InUpdateStmt) { + return 0, false, nil + } var ures uint64 sc := b.ctx.GetSessionVars().StmtCtx @@ -1137,6 +1141,10 @@ func (b *builtinCastStringAsRealSig) evalReal(row chunk.Row) (res float64, isNul if isNull || err != nil { return res, isNull, err } + sctx := b.ctx.GetSessionVars().StmtCtx + if val == "" && (sctx.InInsertStmt || sctx.InUpdateStmt) { + return 0, false, nil + } sc := b.ctx.GetSessionVars().StmtCtx res, err = types.StrToFloat(sc, val) if err != nil { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 5d85092d959b1..a79f29a08fa88 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -2753,7 +2753,6 @@ func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.Tab if err != nil { return nil, nil, err } - newExpr = expression.BuildCastFunction(b.ctx, newExpr, col.GetType()) p = np newList = append(newList, &expression.Assignment{Col: col, Expr: newExpr}) } diff --git a/types/convert.go b/types/convert.go index 2dffeb6a44a2d..344db78681d7f 100644 --- a/types/convert.go +++ b/types/convert.go @@ -618,7 +618,7 @@ func ConvertJSONToDecimal(sc *stmtctx.StatementContext, j json.BinaryJSON) (*MyD // getValidFloatPrefix gets prefix of string which can be successfully parsed as float. func getValidFloatPrefix(sc *stmtctx.StatementContext, s string) (valid string, err error) { - if (sc.InDeleteStmt || sc.InSelectStmt || sc.InUpdateStmt) && s == "" { + if (sc.InDeleteStmt || sc.InSelectStmt) && s == "" { return "0", nil } var ( diff --git a/types/convert_test.go b/types/convert_test.go index 845e5fc4dfde4..664e8fb107ec6 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -469,17 +469,14 @@ func (s *testTypeConvertSuite) TestStrToNum(c *C) { func testSelectUpdateDeleteEmptyStringError(c *C) { testCases := []struct { inSelect bool - inUpdate bool inDelete bool }{ - {true, false, false}, - {false, true, false}, - {false, false, true}, + {true, false}, + {false, true}, } sc := new(stmtctx.StatementContext) for _, tc := range testCases { sc.InSelectStmt = tc.inSelect - sc.InUpdateStmt = tc.inUpdate sc.InDeleteStmt = tc.inDelete str := "" From 3a10bb8c6f75448bd03c97ba93fdc99752295458 Mon Sep 17 00:00:00 2001 From: pingcap-github-bot Date: Wed, 11 Dec 2019 16:15:58 +0800 Subject: [PATCH 2/4] privilege: fix `REVOKE` privilege check incompatibility with MySQL (#13014) (#13306) --- planner/core/logical_plan_test.go | 15 +++++++++-- planner/core/planbuilder.go | 35 ++++++++++++++++++++++++- privilege/privileges/privileges_test.go | 18 +++++++++++++ 3 files changed, 65 insertions(+), 3 deletions(-) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 6f5081e95e470..b19c129a103a2 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1759,9 +1759,20 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { }, }, { - sql: `revoke all privileges on *.* from 'test'@'%'`, + sql: `revoke all privileges on test.* from 'test'@'%'`, ans: []visitInfo{ - {mysql.SuperPriv, "", "", "", nil}, + {mysql.SelectPriv, "test", "", "", nil}, + {mysql.InsertPriv, "test", "", "", nil}, + {mysql.UpdatePriv, "test", "", "", nil}, + {mysql.DeletePriv, "test", "", "", nil}, + {mysql.CreatePriv, "test", "", "", nil}, + {mysql.DropPriv, "test", "", "", nil}, + {mysql.GrantPriv, "test", "", "", nil}, + {mysql.AlterPriv, "test", "", "", nil}, + {mysql.ExecutePriv, "test", "", "", nil}, + {mysql.IndexPriv, "test", "", "", nil}, + {mysql.CreateViewPriv, "test", "", "", nil}, + {mysql.ShowViewPriv, "test", "", "", nil}, }, }, { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 45b29f02683d1..28a2e8985fff4 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1369,7 +1369,7 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err) case *ast.RevokeStmt: - b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", nil) + b.visitInfo = collectVisitInfoFromRevokeStmt(b.ctx, b.visitInfo, raw) case *ast.RevokeRoleStmt: err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err) @@ -1393,6 +1393,39 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { return p, nil } +func collectVisitInfoFromRevokeStmt(sctx sessionctx.Context, vi []visitInfo, stmt *ast.RevokeStmt) []visitInfo { + // To use REVOKE, you must have the GRANT OPTION privilege, + // and you must have the privileges that you are granting. + dbName := stmt.Level.DBName + tableName := stmt.Level.TableName + if dbName == "" { + dbName = sctx.GetSessionVars().CurrentDB + } + vi = appendVisitInfo(vi, mysql.GrantPriv, dbName, tableName, "", nil) + + var allPrivs []mysql.PrivilegeType + for _, item := range stmt.Privs { + if item.Priv == mysql.AllPriv { + switch stmt.Level.Level { + case ast.GrantLevelGlobal: + allPrivs = mysql.AllGlobalPrivs + case ast.GrantLevelDB: + allPrivs = mysql.AllDBPrivs + case ast.GrantLevelTable: + allPrivs = mysql.AllTablePrivs + } + break + } + vi = appendVisitInfo(vi, item.Priv, dbName, tableName, "", nil) + } + + for _, priv := range allPrivs { + vi = appendVisitInfo(vi, priv, dbName, tableName, "", nil) + } + + return vi +} + func collectVisitInfoFromGrantStmt(sctx sessionctx.Context, vi []visitInfo, stmt *ast.GrantStmt) []visitInfo { // To use GRANT, you must have the GRANT OPTION privilege, // and you must have the privileges that you are granting. diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index ac10d3fa96ba4..a36332565b714 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -541,6 +541,24 @@ func (s *testPrivilegeSuite) TestUseDB(c *C) { c.Assert(err, NotNil) } +func (s *testPrivilegeSuite) TestRevokePrivileges(c *C) { + se := newSession(c, s.store, s.dbName) + mustExec(c, se, "CREATE USER 'hasgrant'") + mustExec(c, se, "CREATE USER 'withoutgrant'") + mustExec(c, se, "GRANT ALL ON *.* TO 'hasgrant'") + mustExec(c, se, "GRANT ALL ON mysql.* TO 'withoutgrant'") + // Without grant option + c.Assert(se.Auth(&auth.UserIdentity{Username: "hasgrant", Hostname: "localhost", AuthUsername: "hasgrant", AuthHostname: "%"}, nil, nil), IsTrue) + _, e := se.Execute(context.Background(), "REVOKE SELECT ON mysql.* FROM 'withoutgrant'") + c.Assert(e, NotNil) + // With grant option + se = newSession(c, s.store, s.dbName) + mustExec(c, se, "GRANT ALL ON *.* TO 'hasgrant' WITH GRANT OPTION") + c.Assert(se.Auth(&auth.UserIdentity{Username: "hasgrant", Hostname: "localhost", AuthUsername: "hasgrant", AuthHostname: "%"}, nil, nil), IsTrue) + mustExec(c, se, "REVOKE SELECT ON mysql.* FROM 'withoutgrant'") + mustExec(c, se, "REVOKE ALL ON mysql.* FROM withoutgrant") +} + func (s *testPrivilegeSuite) TestSetGlobal(c *C) { se := newSession(c, s.store, s.dbName) mustExec(c, se, `CREATE USER setglobal_a@localhost`) From b10d1a58c718bce8c3ed364d85781a4fade45e9c Mon Sep 17 00:00:00 2001 From: pingcap-github-bot Date: Thu, 12 Dec 2019 11:05:58 +0800 Subject: [PATCH 3/4] store/tikv: fix a memory leak in the batchClient for the large transactions (#14031) (#14032) --- store/tikv/client_batch.go | 24 ++++++++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/store/tikv/client_batch.go b/store/tikv/client_batch.go index 1f349c70b9a8f..ae14f3522b528 100644 --- a/store/tikv/client_batch.go +++ b/store/tikv/client_batch.go @@ -398,6 +398,22 @@ func (b *batchCommandsEntry) isCanceled() bool { const idleTimeout = 3 * time.Minute +func resetEntries(entries []*batchCommandsEntry) []*batchCommandsEntry { + for i := 0; i < len(entries); i++ { + entries[i] = nil + } + entries = entries[:0] + return entries +} + +func resetRequests(requests []*tikvpb.BatchCommandsRequest_Request) []*tikvpb.BatchCommandsRequest_Request { + for i := 0; i < len(requests); i++ { + requests[i] = nil + } + requests = requests[:0] + return requests +} + func (a *batchConn) batchSendLoop(cfg config.TiKVClient) { defer func() { if r := recover(); r != nil { @@ -416,8 +432,12 @@ func (a *batchConn) batchSendLoop(cfg config.TiKVClient) { var bestBatchWaitSize = cfg.BatchWaitSize for { - entries = entries[:0] - requests = requests[:0] + // NOTE: We can't simply set entries = entries[:0] here. + // The data in the cap part of the slice would reference the prewrite keys whose + // underlying memory is borrowed from memdb. The reference cause GC can't release + // the memdb, leading to serious memory leak problems in the large transaction case. + entries = resetEntries(entries) + requests = resetRequests(requests) requestIDs = requestIDs[:0] a.pendingRequests.Set(float64(len(a.batchCommandsCh))) From c31b745459ccf29d05ff5d83d84b4563d2923039 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Thu, 12 Dec 2019 15:00:29 +0800 Subject: [PATCH 4/4] stats: remove the lower bound of auto analyze ratio (#13995) (#14015) --- sessionctx/variable/varsutil.go | 6 ++++++ statistics/handle/update.go | 11 ++--------- statistics/handle/update_test.go | 22 +++++++++++----------- 3 files changed, 19 insertions(+), 20 deletions(-) diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index fc53a37bfc40c..c32aa82e8cc37 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -519,6 +519,12 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return "", err } return v, nil + case TiDBAutoAnalyzeRatio: + v, err := strconv.ParseFloat(value, 64) + if err != nil || v < 0 { + return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) + } + return value, nil case TxnIsolation, TransactionIsolation: upVal := strings.ToUpper(value) _, exists := TxIsolationNames[upVal] diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 500e5d827ee51..25df50895a312 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -649,13 +649,9 @@ func NeedAnalyzeTable(tbl *statistics.Table, limit time.Duration, autoAnalyzeRat return false, "" } // Tests if current time is within the time period. - return withinTimePeriod(start, end, now), fmt.Sprintf("too many modifications(%v/%v)", tbl.ModifyCount, tbl.Count) + return withinTimePeriod(start, end, now), fmt.Sprintf("too many modifications(%v/%v>%v)", tbl.ModifyCount, tbl.Count, autoAnalyzeRatio) } -const ( - minAutoAnalyzeRatio = 0.3 -) - func (h *Handle) getAutoAnalyzeParameters() map[string]string { sql := fmt.Sprintf("select variable_name, variable_value from mysql.global_variables where variable_name in ('%s', '%s', '%s')", variable.TiDBAutoAnalyzeRatio, variable.TiDBAutoAnalyzeStartTime, variable.TiDBAutoAnalyzeEndTime) @@ -675,10 +671,7 @@ func parseAutoAnalyzeRatio(ratio string) float64 { if err != nil { return variable.DefAutoAnalyzeRatio } - if autoAnalyzeRatio > 0 { - autoAnalyzeRatio = math.Max(autoAnalyzeRatio, minAutoAnalyzeRatio) - } - return autoAnalyzeRatio + return math.Max(autoAnalyzeRatio, 0) } func parseAnalyzePeriod(start, end string) (time.Time, time.Time, error) { diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 5a0a95464307b..21258e1be004c 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -374,7 +374,7 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { testKit.MustExec("create table t (a varchar(20))") handle.AutoAnalyzeMinCnt = 0 - testKit.MustExec("set global tidb_auto_analyze_ratio = 0.6") + testKit.MustExec("set global tidb_auto_analyze_ratio = 0.2") defer func() { handle.AutoAnalyzeMinCnt = 1000 testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") @@ -392,18 +392,18 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { stats := h.GetTableStats(tableInfo) c.Assert(stats.Count, Equals, int64(0)) - _, err = testKit.Exec("insert into t values ('ss')") + _, err = testKit.Exec("insert into t values ('ss'), ('ss'), ('ss'), ('ss'), ('ss')") c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) c.Assert(h.Update(is), IsNil) h.HandleAutoAnalyze(is) c.Assert(h.Update(is), IsNil) stats = h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(1)) + c.Assert(stats.Count, Equals, int64(5)) c.Assert(stats.ModifyCount, Equals, int64(0)) for _, item := range stats.Columns { - // TotColSize = 2(length of 'ss') + 1(size of len byte). - c.Assert(item.TotColSize, Equals, int64(3)) + // TotColSize = 5*(2(length of 'ss') + 1(size of len byte)). + c.Assert(item.TotColSize, Equals, int64(15)) break } @@ -417,7 +417,7 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { h.HandleAutoAnalyze(is) c.Assert(h.Update(is), IsNil) stats = h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(2)) + c.Assert(stats.Count, Equals, int64(6)) c.Assert(stats.ModifyCount, Equals, int64(1)) _, err = testKit.Exec("insert into t values ('fff')") @@ -427,7 +427,7 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { h.HandleAutoAnalyze(is) c.Assert(h.Update(is), IsNil) stats = h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(3)) + c.Assert(stats.Count, Equals, int64(7)) c.Assert(stats.ModifyCount, Equals, int64(0)) _, err = testKit.Exec("insert into t values ('eee')") @@ -437,12 +437,12 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { h.HandleAutoAnalyze(is) c.Assert(h.Update(is), IsNil) stats = h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(4)) + c.Assert(stats.Count, Equals, int64(8)) // Modify count is non-zero means that we do not analyze the table. c.Assert(stats.ModifyCount, Equals, int64(1)) for _, item := range stats.Columns { - // TotColSize = 6, because the table has not been analyzed, and insert statement will add 3(length of 'eee') to TotColSize. - c.Assert(item.TotColSize, Equals, int64(14)) + // TotColSize = 26, because the table has not been analyzed, and insert statement will add 3(length of 'eee') to TotColSize. + c.Assert(item.TotColSize, Equals, int64(26)) break } @@ -456,7 +456,7 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { h.HandleAutoAnalyze(is) c.Assert(h.Update(is), IsNil) stats = h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(4)) + c.Assert(stats.Count, Equals, int64(8)) c.Assert(stats.ModifyCount, Equals, int64(0)) hg, ok := stats.Indices[tableInfo.Indices[0].ID] c.Assert(ok, IsTrue)