Skip to content

Commit

Permalink
Merge branch 'cherrypick-13889' of github.com:SunRunAway/tidb into ch…
Browse files Browse the repository at this point in the history
…errypick-13889
  • Loading branch information
SunRunAway committed Dec 12, 2019
2 parents 649828c + ffa2664 commit 16a2916
Show file tree
Hide file tree
Showing 13 changed files with 139 additions and 33 deletions.
21 changes: 21 additions & 0 deletions executor/insert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())
}
2 changes: 1 addition & 1 deletion executor/write_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 <nil>"))
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")
Expand Down
8 changes: 8 additions & 0 deletions expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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 {
Expand Down
1 change: 0 additions & 1 deletion planner/core/logical_plan_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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})
}
Expand Down
15 changes: 13 additions & 2 deletions planner/core/logical_plan_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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},
},
},
{
Expand Down
35 changes: 34 additions & 1 deletion planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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.
Expand Down
18 changes: 18 additions & 0 deletions privilege/privileges/privileges_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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`)
Expand Down
6 changes: 6 additions & 0 deletions sessionctx/variable/varsutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -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]
Expand Down
11 changes: 2 additions & 9 deletions statistics/handle/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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) {
Expand Down
22 changes: 11 additions & 11 deletions statistics/handle/update_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand All @@ -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
}

Expand All @@ -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')")
Expand All @@ -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')")
Expand All @@ -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
}

Expand All @@ -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)
Expand Down
24 changes: 22 additions & 2 deletions store/tikv/client_batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand All @@ -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)))
Expand Down
2 changes: 1 addition & 1 deletion types/convert.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 (
Expand Down
7 changes: 2 additions & 5 deletions types/convert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 := ""
Expand Down

0 comments on commit 16a2916

Please sign in to comment.