From eb3ee25b455bc49ec5ed0a472807a8d3984aea1d Mon Sep 17 00:00:00 2001 From: Yixuan Zhao Date: Sun, 13 Oct 2019 02:03:03 +0800 Subject: [PATCH 1/6] Makefile: add `-trim-path` flag for build command (#12653) --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 0186a59b5cab3..a7db21b00af7e 100644 --- a/Makefile +++ b/Makefile @@ -12,7 +12,7 @@ path_to_add := $(addsuffix /bin,$(subst :,/bin:,$(GOPATH))):$(PWD)/tools/bin export PATH := $(path_to_add):$(PATH) GO := GO111MODULE=on go -GOBUILD := CGO_ENABLED=1 $(GO) build $(BUILD_FLAG) +GOBUILD := CGO_ENABLED=1 $(GO) build $(BUILD_FLAG) -trimpath GOTEST := CGO_ENABLED=1 $(GO) test -p 4 OVERALLS := CGO_ENABLED=1 GO111MODULE=on overalls From 95aac16da2bf0560343af8b6654c216d4aff8b66 Mon Sep 17 00:00:00 2001 From: pingcap-github-bot Date: Sun, 13 Oct 2019 23:43:03 +0800 Subject: [PATCH 2/6] ddl: allow multiple `unique` attributes in a column (#12165) (#12463) --- ddl/db_integration_test.go | 8 ++++++++ ddl/ddl_api.go | 18 ++++++++++++------ 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 039c749cb28b5..52fd0f40a208f 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1590,6 +1590,14 @@ func (s *testIntegrationSuite4) TestAlterColumn(c *C) { _, err = s.tk.Exec("alter table t1 modify column c bigint;") c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "[ddl:1071]Specified key was too long; max key length is 3072 bytes") + + s.tk.MustExec("drop table if exists multi_unique") + s.tk.MustExec("create table multi_unique (a int unique unique)") + s.tk.MustExec("drop table multi_unique") + s.tk.MustExec("create table multi_unique (a int key primary key unique unique)") + s.tk.MustExec("drop table multi_unique") + s.tk.MustExec("create table multi_unique (a int key unique unique key unique)") + s.tk.MustExec("drop table multi_unique") } func (s *testIntegrationSuite) assertWarningExec(c *C, sql string, expectedWarn *terror.Error) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 17853505179ba..91be4e5fda7da 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -507,13 +507,19 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, o case ast.ColumnOptionAutoIncrement: col.Flag |= mysql.AutoIncrementFlag case ast.ColumnOptionPrimaryKey: - constraint := &ast.Constraint{Tp: ast.ConstraintPrimaryKey, Keys: keys} - constraints = append(constraints, constraint) - col.Flag |= mysql.PriKeyFlag + // Check PriKeyFlag first to avoid extra duplicate constraints. + if col.Flag&mysql.PriKeyFlag == 0 { + constraint := &ast.Constraint{Tp: ast.ConstraintPrimaryKey, Keys: keys} + constraints = append(constraints, constraint) + col.Flag |= mysql.PriKeyFlag + } case ast.ColumnOptionUniqKey: - constraint := &ast.Constraint{Tp: ast.ConstraintUniqKey, Name: colDef.Name.Name.O, Keys: keys} - constraints = append(constraints, constraint) - col.Flag |= mysql.UniqueKeyFlag + // Check UniqueFlag first to avoid extra duplicate constraints. + if col.Flag&mysql.UniqueFlag == 0 { + constraint := &ast.Constraint{Tp: ast.ConstraintUniqKey, Keys: keys} + constraints = append(constraints, constraint) + col.Flag |= mysql.UniqueKeyFlag + } case ast.ColumnOptionDefaultValue: hasDefaultValue, err = setDefaultValue(ctx, col, v) if err != nil { From 21b49ac98dbe6a2e82c160af6748ad1fcf975f72 Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Sun, 13 Oct 2019 23:53:03 +0800 Subject: [PATCH 3/6] log: move autocommit varable value into connection info (#12310) (#12566) --- executor/set.go | 8 +++++++- server/conn.go | 18 +++++++++++++++++- 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/executor/set.go b/executor/set.go index cabfc6e3fd1a2..3400e62257a9d 100644 --- a/executor/set.go +++ b/executor/set.go @@ -188,7 +188,13 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e valStr, err = value.ToString() terror.Log(err) } - logutil.Logger(context.Background()).Info("set session var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) + if name != variable.AutoCommit { + logutil.Logger(context.Background()).Info("set session var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) + } else { + // Some applications will set `autocommit` variable before query. + // This will print too many unnecessary log info. + logutil.Logger(context.Background()).Debug("set session var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) + } } if name == variable.TiDBEnableStmtSummary { diff --git a/server/conn.go b/server/conn.go index a65d2bbb97a1e..145e97fa6b655 100644 --- a/server/conn.go +++ b/server/conn.go @@ -155,7 +155,7 @@ type clientConn struct { func (cc *clientConn) String() string { collationStr := mysql.Collations[cc.collation] - return fmt.Sprintf("id:%d, addr:%s status:%d, collation:%s, user:%s", + return fmt.Sprintf("id:%d, addr:%s status:%b, collation:%s, user:%s", cc.connectionID, cc.bufReadConn.RemoteAddr(), cc.ctx.Status(), collationStr, cc.user, ) } @@ -533,6 +533,20 @@ func (cc *clientConn) readOptionalSSLRequestAndHandshakeResponse(ctx context.Con return err } +func (cc *clientConn) SessionStatusToString() string { + status := cc.ctx.Status() + inTxn, autoCommit := 0, 0 + if status&mysql.ServerStatusInTrans > 0 { + inTxn = 1 + } + if status&mysql.ServerStatusAutocommit > 0 { + autoCommit = 1 + } + return fmt.Sprintf("inTxn:%d, autocommit:%d", + inTxn, autoCommit, + ) +} + func (cc *clientConn) openSessionAndDoAuth(authData []byte) error { var tlsStatePtr *tls.ConnectionState if cc.tlsConn != nil { @@ -667,6 +681,8 @@ func (cc *clientConn) Run(ctx context.Context) { } logutil.Logger(ctx).Warn("dispatch error", zap.String("connInfo", cc.String()), + zap.String("command", mysql.Command2Str[data[0]]), + zap.String("status", cc.SessionStatusToString()), zap.String("sql", queryStrForLog(string(data[1:]))), zap.String("err", errStrForLog(err)), ) From 4fdfdf8c6111a1c3ebcd25a29fee547a60b678b1 Mon Sep 17 00:00:00 2001 From: "Zhuomin(Charming) Liu" Date: Mon, 14 Oct 2019 00:02:03 +0800 Subject: [PATCH 4/6] planner: fix some window specific check bug for window function. (#12394) (#12404) --- planner/core/logical_plan_builder.go | 5 ++++- planner/core/logical_plan_test.go | 12 ++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index c7e464a125d33..01ea7053aef84 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3309,7 +3309,10 @@ func (b *PlanBuilder) checkOriginWindowSpecs(funcs []*ast.WindowFuncExpr, orderB if end.Type == ast.Preceding && end.UnBounded { return ErrWindowFrameEndIllegal.GenWithStackByArgs(getWindowName(spec.Name.O)) } - if start.Type == ast.Following && end.Type == ast.Preceding { + if start.Type == ast.Following && (end.Type == ast.Preceding || end.Type == ast.CurrentRow) { + return ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O)) + } + if (start.Type == ast.Following || start.Type == ast.CurrentRow) && end.Type == ast.Preceding { return ErrWindowFrameIllegal.GenWithStackByArgs(getWindowName(spec.Name.O)) } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index eb086694f3e0e..22093750c5fbb 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -2485,6 +2485,18 @@ func (s *testPlanSuite) TestWindowFunction(c *C) { sql: "SELECT NTH_VALUE(fieldA, -1) OVER (w1 PARTITION BY fieldB ORDER BY fieldB , fieldA ) AS 'ntile', fieldA, fieldB FROM ( SELECT a AS fieldA, b AS fieldB FROM t ) as temp WINDOW w1 AS ( ORDER BY fieldB ASC, fieldA DESC )", result: "[planner:1210]Incorrect arguments to nth_value", }, + { + sql: "SELECT SUM(a) OVER w AS 'sum' FROM t WINDOW w AS (ROWS BETWEEN 1 FOLLOWING AND CURRENT ROW )", + result: "[planner:3586]Window 'w': frame start or end is negative, NULL or of non-integral type", + }, + { + sql: "SELECT SUM(a) OVER w AS 'sum' FROM t WINDOW w AS (ROWS BETWEEN CURRENT ROW AND 1 PRECEDING )", + result: "[planner:3586]Window 'w': frame start or end is negative, NULL or of non-integral type", + }, + { + sql: "SELECT SUM(a) OVER w AS 'sum' FROM t WINDOW w AS (ROWS BETWEEN 1 FOLLOWING AND 1 PRECEDING )", + result: "[planner:3586]Window 'w': frame start or end is negative, NULL or of non-integral type", + }, // Test issue 11943 { sql: "SELECT ROW_NUMBER() OVER (partition by b) + a FROM t", From 65968907421a79eb6a0e045055bd5e9d64fa0792 Mon Sep 17 00:00:00 2001 From: Arenatlx Date: Mon, 14 Oct 2019 00:12:33 +0800 Subject: [PATCH 5/6] * : Multiple rows insert in a statement should have consecutive autoID if needed. (#11876) (#12602) --- executor/insert_common.go | 197 +++++++++++++++++++++++++++++++++- executor/insert_test.go | 174 ++++++++++++++++++++++++++++++- meta/autoid/autoid.go | 203 +++++++++++++++++++++--------------- meta/autoid/autoid_test.go | 184 +++++++++++++++++++++++++++----- table/table.go | 15 ++- table/tables/tables.go | 2 +- util/kvencoder/allocator.go | 5 +- 7 files changed, 660 insertions(+), 120 deletions(-) mode change 100644 => 100755 meta/autoid/autoid.go diff --git a/executor/insert_common.go b/executor/insert_common.go index d3dbc9c6951d3..3f79b9a687c5f 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -58,6 +58,12 @@ type InsertValues struct { colDefaultVals []defaultVal evalBuffer chunk.MutRow evalBufferTypes []*types.FieldType + + // Fill the autoID lazily to datum. This is used for being compatible with JDBC using getGeneratedKeys(). + // `insert|replace values` can guarantee consecutive autoID in a batch. + // Other statements like `insert select from` don't guarantee consecutive autoID. + // https://dev.mysql.com/doc/refman/8.0/en/innodb-auto-increment-handling.html + lazyFillAutoID bool } type defaultVal struct { @@ -184,6 +190,8 @@ func (e *InsertValues) insertRows(ctx context.Context, exec func(ctx context.Con batchInsert := sessVars.BatchInsert && !sessVars.InTxn() batchSize := sessVars.DMLBatchSize + e.lazyFillAutoID = true + rows := make([][]types.Datum, 0, len(e.Lists)) for i, list := range e.Lists { e.rowCount++ @@ -193,6 +201,11 @@ func (e *InsertValues) insertRows(ctx context.Context, exec func(ctx context.Con } rows = append(rows, row) if batchInsert && e.rowCount%uint64(batchSize) == 0 { + // Before batch insert, fill the batch allocated autoIDs. + rows, err = e.lazyAdjustAutoIncrementDatum(ctx, rows) + if err != nil { + return err + } if err = exec(ctx, rows); err != nil { return err } @@ -202,6 +215,11 @@ func (e *InsertValues) insertRows(ctx context.Context, exec func(ctx context.Con } } } + // Fill the batch allocated autoIDs. + rows, err = e.lazyAdjustAutoIncrementDatum(ctx, rows) + if err != nil { + return err + } return exec(ctx, rows) } @@ -259,7 +277,7 @@ func (e *InsertValues) evalRow(ctx context.Context, list []expression.Expression row[offset], hasValue[offset] = *val1.Copy(), true e.evalBuffer.SetDatum(offset, val1) } - + // Row may lack of generated column, autoIncrement column, empty column here. return e.fillRow(ctx, row, hasValue) } @@ -413,6 +431,14 @@ func (e *InsertValues) getColDefaultValue(idx int, col *table.Column) (d types.D func (e *InsertValues) fillColValue(ctx context.Context, datum types.Datum, idx int, column *table.Column, hasValue bool) (types.Datum, error) { if mysql.HasAutoIncrementFlag(column.Flag) { + if e.lazyFillAutoID { + // Handle hasValue info in autoIncrement column previously for lazy handle. + if !hasValue { + datum.SetNull() + } + // Store the plain datum of autoIncrement column directly for lazy handle. + return datum, nil + } d, err := e.adjustAutoIncrementDatum(ctx, datum, hasValue, column) if err != nil { return types.Datum{}, err @@ -431,6 +457,10 @@ func (e *InsertValues) fillColValue(ctx context.Context, datum types.Datum, idx // fillRow fills generated columns, auto_increment column and empty column. // For NOT NULL column, it will return error or use zero value based on sql_mode. +// When lazyFillAutoID is true, fill row will lazily handle auto increment datum for lazy batch allocation. +// `insert|replace values` can guarantee consecutive autoID in a batch. +// Other statements like `insert select from` don't guarantee consecutive autoID. +// https://dev.mysql.com/doc/refman/8.0/en/innodb-auto-increment-handling.html func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue []bool) ([]types.Datum, error) { gIdx := 0 for i, c := range e.Table.Cols() { @@ -454,13 +484,172 @@ func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue return nil, err } } + // Handle the bad null error. Cause generated column with `not null` flag will get default value datum in fillColValue + // which should be override by generated expr first, then handle the bad null logic here. + if !e.lazyFillAutoID || (e.lazyFillAutoID && !mysql.HasAutoIncrementFlag(c.Flag)) { + if row[i], err = c.HandleBadNull(row[i], e.ctx.GetSessionVars().StmtCtx); err != nil { + return nil, err + } + } + } + return row, nil +} - // Handle the bad null error. - if row[i], err = c.HandleBadNull(row[i], e.ctx.GetSessionVars().StmtCtx); err != nil { +// isAutoNull can help judge whether a datum is AutoIncrement Null quickly. +// This used to help lazyFillAutoIncrement to find consecutive N datum backwards for batch autoID alloc. +func (e *InsertValues) isAutoNull(ctx context.Context, d types.Datum, col *table.Column) bool { + var err error + var recordID int64 + if !d.IsNull() { + recordID, err = getAutoRecordID(d, &col.FieldType, true) + if err != nil { + return false + } + } + // Use the value if it's not null and not 0. + if recordID != 0 { + return false + } + // Change NULL to auto id. + // Change value 0 to auto id, if NoAutoValueOnZero SQL mode is not set. + if d.IsNull() || e.ctx.GetSessionVars().SQLMode&mysql.ModeNoAutoValueOnZero == 0 { + return true + } + return false +} + +func (e *InsertValues) hasAutoIncrementColumn() (int, bool) { + colIdx := -1 + for i, c := range e.Table.Cols() { + if mysql.HasAutoIncrementFlag(c.Flag) { + colIdx = i + break + } + } + return colIdx, colIdx != -1 +} + +func (e *InsertValues) lazyAdjustAutoIncrementDatumInRetry(ctx context.Context, rows [][]types.Datum, colIdx int) ([][]types.Datum, error) { + // Get the autoIncrement column. + col := e.Table.Cols()[colIdx] + // Consider the colIdx of autoIncrement in row are the same. + length := len(rows) + for i := 0; i < length; i++ { + autoDatum := rows[i][colIdx] + + // autoID can be found in RetryInfo. + retryInfo := e.ctx.GetSessionVars().RetryInfo + if retryInfo.Retrying { + id, err := retryInfo.GetCurrAutoIncrementID() + if err != nil { + return nil, err + } + autoDatum.SetAutoID(id, col.Flag) + + if autoDatum, err = col.HandleBadNull(autoDatum, e.ctx.GetSessionVars().StmtCtx); err != nil { + return nil, err + } + rows[i][colIdx] = autoDatum + } + } + return rows, nil +} + +// lazyAdjustAutoIncrementDatum is quite similar to adjustAutoIncrementDatum +// except it will cache auto increment datum previously for lazy batch allocation of autoID. +func (e *InsertValues) lazyAdjustAutoIncrementDatum(ctx context.Context, rows [][]types.Datum) ([][]types.Datum, error) { + // Not in lazyFillAutoID mode means no need to fill. + if !e.lazyFillAutoID { + return rows, nil + } + // No autoIncrement column means no need to fill. + colIdx, ok := e.hasAutoIncrementColumn() + if !ok { + return rows, nil + } + // autoID can be found in RetryInfo. + retryInfo := e.ctx.GetSessionVars().RetryInfo + if retryInfo.Retrying { + return e.lazyAdjustAutoIncrementDatumInRetry(ctx, rows, colIdx) + } + // Get the autoIncrement column. + col := e.Table.Cols()[colIdx] + // Consider the colIdx of autoIncrement in row are the same. + length := len(rows) + for i := 0; i < length; i++ { + autoDatum := rows[i][colIdx] + + var err error + var recordID int64 + if !autoDatum.IsNull() { + recordID, err = getAutoRecordID(autoDatum, &col.FieldType, true) + if err != nil { + return nil, err + } + } + // Use the value if it's not null and not 0. + if recordID != 0 { + err = e.Table.RebaseAutoID(e.ctx, recordID, true) + if err != nil { + return nil, err + } + e.ctx.GetSessionVars().StmtCtx.InsertID = uint64(recordID) + retryInfo.AddAutoIncrementID(recordID) + rows[i][colIdx] = autoDatum + continue + } + + // Change NULL to auto id. + // Change value 0 to auto id, if NoAutoValueOnZero SQL mode is not set. + if autoDatum.IsNull() || e.ctx.GetSessionVars().SQLMode&mysql.ModeNoAutoValueOnZero == 0 { + // Find consecutive num. + start := i + cnt := 1 + for i+1 < length && e.isAutoNull(ctx, rows[i+1][colIdx], col) { + i++ + cnt++ + } + // Alloc batch N consecutive (min, max] autoIDs. + // max value can be derived from adding one for cnt times. + min, _, err := table.AllocBatchAutoIncrementValue(ctx, e.Table, e.ctx, cnt) + if e.filterErr(err) != nil { + return nil, err + } + // It's compatible with mysql setting the first allocated autoID to lastInsertID. + // Cause autoID may be specified by user, judge only the first row is not suitable. + if e.lastInsertID == 0 { + e.lastInsertID = uint64(min) + 1 + } + // Assign autoIDs to rows. + for j := 0; j < cnt; j++ { + offset := j + start + d := rows[offset][colIdx] + + id := int64(uint64(min) + uint64(j) + 1) + d.SetAutoID(id, col.Flag) + retryInfo.AddAutoIncrementID(id) + + // The value of d is adjusted by auto ID, so we need to cast it again. + d, err := table.CastValue(e.ctx, d, col.ToInfo()) + if err != nil { + return nil, err + } + rows[offset][colIdx] = d + } + continue + } + + autoDatum.SetAutoID(recordID, col.Flag) + retryInfo.AddAutoIncrementID(recordID) + + // the value of d is adjusted by auto ID, so we need to cast it again. + autoDatum, err = table.CastValue(e.ctx, autoDatum, col.ToInfo()) + if err != nil { return nil, err } + rows[i][colIdx] = autoDatum } - return row, nil + return rows, nil } func (e *InsertValues) adjustAutoIncrementDatum(ctx context.Context, d types.Datum, hasValue bool, c *table.Column) (types.Datum, error) { diff --git a/executor/insert_test.go b/executor/insert_test.go index 0ec561cb04a40..1a8e9c7de99eb 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -15,9 +15,11 @@ package executor_test import ( "fmt" + "strings" . "github.com/pingcap/check" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testkit" @@ -298,6 +300,8 @@ func (s *testSuite3) TestInsertWithAutoidSchema(c *C) { tk.MustExec(`create table t5(id int primary key, n float unsigned auto_increment, key I_n(n));`) tk.MustExec(`create table t6(id int primary key, n double auto_increment, key I_n(n));`) tk.MustExec(`create table t7(id int primary key, n double unsigned auto_increment, key I_n(n));`) + // test for inserting multiple values + tk.MustExec(`create table t8(id int primary key auto_increment, n int);`) tests := []struct { insert string @@ -540,11 +544,177 @@ func (s *testSuite3) TestInsertWithAutoidSchema(c *C) { `select * from t7 where id = 3`, testkit.Rows(`3 3`), }, + + // the following is test for insert multiple values. + { + `insert into t8(n) values(1),(2)`, + `select * from t8 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `;`, + `select * from t8 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `;`, + `select last_insert_id();`, + testkit.Rows(`1`), + }, + // test user rebase and auto alloc mixture. + { + `insert into t8 values(null, 3),(-1, -1),(null,4),(null, 5)`, + `select * from t8 where id = 3`, + testkit.Rows(`3 3`), + }, + // -1 won't rebase allocator here cause -1 < base. + { + `;`, + `select * from t8 where id = -1`, + testkit.Rows(`-1 -1`), + }, + { + `;`, + `select * from t8 where id = 4`, + testkit.Rows(`4 4`), + }, + { + `;`, + `select * from t8 where id = 5`, + testkit.Rows(`5 5`), + }, + { + `;`, + `select last_insert_id();`, + testkit.Rows(`3`), + }, + { + `insert into t8 values(null, 6),(10, 7),(null, 8)`, + `select * from t8 where id = 6`, + testkit.Rows(`6 6`), + }, + // 10 will rebase allocator here. + { + `;`, + `select * from t8 where id = 10`, + testkit.Rows(`10 7`), + }, + { + `;`, + `select * from t8 where id = 11`, + testkit.Rows(`11 8`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`6`), + }, + // fix bug for last_insert_id should be first allocated id in insert rows (skip the rebase id). + { + `insert into t8 values(100, 9),(null,10),(null,11)`, + `select * from t8 where id = 100`, + testkit.Rows(`100 9`), + }, + { + `;`, + `select * from t8 where id = 101`, + testkit.Rows(`101 10`), + }, + { + `;`, + `select * from t8 where id = 102`, + testkit.Rows(`102 11`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`101`), + }, + // test with sql_mode: NO_AUTO_VALUE_ON_ZERO. + { + `;`, + `select @@sql_mode`, + testkit.Rows(`ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`), + }, + { + `;`, + "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION,NO_AUTO_VALUE_ON_ZERO`", + nil, + }, + { + `insert into t8 values (0, 12), (null, 13)`, + `select * from t8 where id = 0`, + testkit.Rows(`0 12`), + }, + { + `;`, + `select * from t8 where id = 103`, + testkit.Rows(`103 13`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`103`), + }, + // test without sql_mode: NO_AUTO_VALUE_ON_ZERO. + { + `;`, + "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`", + nil, + }, + // value 0 will be substitute by autoid. + { + `insert into t8 values (0, 14), (null, 15)`, + `select * from t8 where id = 104`, + testkit.Rows(`104 14`), + }, + { + `;`, + `select * from t8 where id = 105`, + testkit.Rows(`105 15`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`104`), + }, + // last test : auto increment allocation can find in retryInfo. + { + `retry : insert into t8 values (null, 16), (null, 17)`, + `select * from t8 where id = 1000`, + testkit.Rows(`1000 16`), + }, + { + `;`, + `select * from t8 where id = 1001`, + testkit.Rows(`1001 17`), + }, + { + `;`, + `select last_insert_id()`, + // this insert doesn't has the last_insert_id, should be same as the last insert case. + testkit.Rows(`104`), + }, } for _, tt := range tests { - tk.MustExec(tt.insert) - tk.MustQuery(tt.query).Check(tt.result) + if strings.HasPrefix(tt.insert, "retry : ") { + // it's the last retry insert case, change the sessionVars. + retryInfo := &variable.RetryInfo{Retrying: true} + retryInfo.AddAutoIncrementID(1000) + retryInfo.AddAutoIncrementID(1001) + tk.Se.GetSessionVars().RetryInfo = retryInfo + tk.MustExec(tt.insert[8:]) + tk.Se.GetSessionVars().RetryInfo = &variable.RetryInfo{} + } else { + tk.MustExec(tt.insert) + } + if tt.query == "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION,NO_AUTO_VALUE_ON_ZERO`" || + tt.query == "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`" { + tk.MustExec(tt.query) + } else { + tk.MustQuery(tt.query).Check(tt.result) + } } } diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go old mode 100644 new mode 100755 index 15329e8d7dc77..abd52dde335d9 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -45,9 +45,10 @@ var errInvalidTableID = terror.ClassAutoid.New(codeInvalidTableID, "invalid Tabl // Allocator is an auto increment id generator. // Just keep id unique actually. type Allocator interface { - // Alloc allocs the next autoID for table with tableID. + // Alloc allocs N consecutive autoID for table with tableID, returning (min, max] of the allocated autoID batch. // It gets a batch of autoIDs at a time. So it does not need to access storage for each call. - Alloc(tableID int64) (int64, error) + // The consecutive feature is used to insert multiple rows in a statement. + Alloc(tableID int64, n uint64) (int64, int64, error) // Rebase rebases the autoID base for table with tableID and the new base value. // If allocIDs is true, it will allocate some IDs and save to the cache. // If allocIDs is false, it will not allocate IDs. @@ -220,12 +221,80 @@ func (alloc *allocator) Rebase(tableID, requiredBase int64, allocIDs bool) error return alloc.rebase4Signed(tableID, requiredBase, allocIDs) } -func (alloc *allocator) alloc4Unsigned(tableID int64) (int64, error) { - if alloc.base == alloc.end { // step +// NextStep return new auto id step according to previous step and consuming time. +func NextStep(curStep int64, consumeDur time.Duration) int64 { + failpoint.Inject("mockAutoIDChange", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(step) + } + }) + + consumeRate := defaultConsumeTime.Seconds() / consumeDur.Seconds() + res := int64(float64(curStep) * consumeRate) + if res < minStep { + return minStep + } else if res > maxStep { + return maxStep + } + return res +} + +// NewAllocator returns a new auto increment id generator on the store. +func NewAllocator(store kv.Storage, dbID int64, isUnsigned bool) Allocator { + return &allocator{ + store: store, + dbID: dbID, + isUnsigned: isUnsigned, + step: step, + lastAllocTime: time.Now(), + } +} + +//codeInvalidTableID is the code of autoid error. +const codeInvalidTableID terror.ErrCode = 1 + +var localSchemaID = int64(math.MaxInt64) + +// GenLocalSchemaID generates a local schema ID. +func GenLocalSchemaID() int64 { + return atomic.AddInt64(&localSchemaID, -1) +} + +// Alloc implements autoid.Allocator Alloc interface. +func (alloc *allocator) Alloc(tableID int64, n uint64) (int64, int64, error) { + if tableID == 0 { + return 0, 0, errInvalidTableID.GenWithStackByArgs("Invalid tableID") + } + if n == 0 { + return 0, 0, nil + } + alloc.mu.Lock() + defer alloc.mu.Unlock() + if alloc.isUnsigned { + return alloc.alloc4Unsigned(tableID, n) + } + return alloc.alloc4Signed(tableID, n) +} + +func (alloc *allocator) alloc4Signed(tableID int64, n uint64) (int64, int64, error) { + n1 := int64(n) + // Condition alloc.base+N1 > alloc.end will overflow when alloc.base + N1 > MaxInt64. So need this. + if math.MaxInt64-alloc.base <= n1 { + return 0, 0, ErrAutoincReadFailed + } + // The local rest is not enough for allocN, skip it. + if alloc.base+n1 > alloc.end { var newBase, newEnd int64 startTime := time.Now() + // Although it may skip a segment here, we still think it is consumed. consumeDur := startTime.Sub(alloc.lastAllocTime) - alloc.step = NextStep(alloc.step, consumeDur) + nextStep := NextStep(alloc.step, consumeDur) + // Make sure nextStep is big enough. + if nextStep <= n1 { + alloc.step = mathutil.MinInt64(n1*2, maxStep) + } else { + alloc.step = nextStep + } err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { m := meta.NewMeta(txn) var err1 error @@ -233,38 +302,53 @@ func (alloc *allocator) alloc4Unsigned(tableID int64) (int64, error) { if err1 != nil { return err1 } - tmpStep := int64(mathutil.MinUint64(math.MaxUint64-uint64(newBase), uint64(alloc.step))) + tmpStep := mathutil.MinInt64(math.MaxInt64-newBase, alloc.step) + // The global rest is not enough for alloc. + if tmpStep < n1 { + return ErrAutoincReadFailed + } newEnd, err1 = m.GenAutoTableID(alloc.dbID, tableID, tmpStep) return err1 }) metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDAlloc, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) if err != nil { - return 0, err + return 0, 0, err } alloc.lastAllocTime = time.Now() - if uint64(newBase) == math.MaxUint64 { - return 0, ErrAutoincReadFailed + if newBase == math.MaxInt64 { + return 0, 0, ErrAutoincReadFailed } alloc.base, alloc.end = newBase, newEnd } - - if uint64(alloc.base)+uint64(1) == math.MaxUint64 { - return 0, ErrAutoincReadFailed - } - alloc.base = int64(uint64(alloc.base) + 1) - logutil.Logger(context.Background()).Debug("alloc unsigned ID", - zap.Uint64("ID", uint64(alloc.base)), + logutil.Logger(context.TODO()).Debug("alloc N signed ID", + zap.Uint64("from ID", uint64(alloc.base)), + zap.Uint64("to ID", uint64(alloc.base+n1)), zap.Int64("table ID", tableID), zap.Int64("database ID", alloc.dbID)) - return alloc.base, nil + min := alloc.base + alloc.base += n1 + return min, alloc.base, nil } -func (alloc *allocator) alloc4Signed(tableID int64) (int64, error) { - if alloc.base == alloc.end { // step +func (alloc *allocator) alloc4Unsigned(tableID int64, n uint64) (int64, int64, error) { + n1 := int64(n) + // Condition alloc.base+n1 > alloc.end will overflow when alloc.base + n1 > MaxInt64. So need this. + if math.MaxUint64-uint64(alloc.base) <= n { + return 0, 0, ErrAutoincReadFailed + } + // The local rest is not enough for alloc, skip it. + if uint64(alloc.base)+n > uint64(alloc.end) { var newBase, newEnd int64 startTime := time.Now() + // Although it may skip a segment here, we still treat it as consumed. consumeDur := startTime.Sub(alloc.lastAllocTime) - alloc.step = NextStep(alloc.step, consumeDur) + nextStep := NextStep(alloc.step, consumeDur) + // Make sure nextStep is big enough. + if nextStep <= n1 { + alloc.step = mathutil.MinInt64(n1*2, maxStep) + } else { + alloc.step = nextStep + } err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { m := meta.NewMeta(txn) var err1 error @@ -272,80 +356,31 @@ func (alloc *allocator) alloc4Signed(tableID int64) (int64, error) { if err1 != nil { return err1 } - tmpStep := mathutil.MinInt64(math.MaxInt64-newBase, alloc.step) + tmpStep := int64(mathutil.MinUint64(math.MaxUint64-uint64(newBase), uint64(alloc.step))) + // The global rest is not enough for alloc. + if tmpStep < n1 { + return ErrAutoincReadFailed + } newEnd, err1 = m.GenAutoTableID(alloc.dbID, tableID, tmpStep) return err1 }) metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDAlloc, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) if err != nil { - return 0, err + return 0, 0, err } alloc.lastAllocTime = time.Now() - if newBase == math.MaxInt64 { - return 0, ErrAutoincReadFailed + if uint64(newBase) == math.MaxUint64 { + return 0, 0, ErrAutoincReadFailed } alloc.base, alloc.end = newBase, newEnd } - - if alloc.base+1 == math.MaxInt64 { - return 0, ErrAutoincReadFailed - } - alloc.base++ - logutil.Logger(context.Background()).Debug("alloc signed ID", - zap.Uint64("ID", uint64(alloc.base)), + logutil.Logger(context.TODO()).Debug("alloc unsigned ID", + zap.Uint64(" from ID", uint64(alloc.base)), + zap.Uint64("to ID", uint64(alloc.base+n1)), zap.Int64("table ID", tableID), zap.Int64("database ID", alloc.dbID)) - return alloc.base, nil -} - -// Alloc implements autoid.Allocator Alloc interface. -func (alloc *allocator) Alloc(tableID int64) (int64, error) { - if tableID == 0 { - return 0, errInvalidTableID.GenWithStack("Invalid tableID") - } - alloc.mu.Lock() - defer alloc.mu.Unlock() - if alloc.isUnsigned { - return alloc.alloc4Unsigned(tableID) - } - return alloc.alloc4Signed(tableID) -} - -// NextStep return new auto id step according to previous step and consuming time. -func NextStep(curStep int64, consumeDur time.Duration) int64 { - failpoint.Inject("mockAutoIDChange", func(val failpoint.Value) { - if val.(bool) { - failpoint.Return(step) - } - }) - - consumeRate := defaultConsumeTime.Seconds() / consumeDur.Seconds() - res := int64(float64(curStep) * consumeRate) - if res < minStep { - return minStep - } else if res > maxStep { - return maxStep - } - return res -} - -// NewAllocator returns a new auto increment id generator on the store. -func NewAllocator(store kv.Storage, dbID int64, isUnsigned bool) Allocator { - return &allocator{ - store: store, - dbID: dbID, - isUnsigned: isUnsigned, - step: step, - lastAllocTime: time.Now(), - } -} - -//autoid error codes. -const codeInvalidTableID terror.ErrCode = 1 - -var localSchemaID = int64(math.MaxInt64) - -// GenLocalSchemaID generates a local schema ID. -func GenLocalSchemaID() int64 { - return atomic.AddInt64(&localSchemaID, -1) + min := alloc.base + // Use uint64 n directly. + alloc.base = int64(uint64(alloc.base) + n) + return min, alloc.base, nil } diff --git a/meta/autoid/autoid_test.go b/meta/autoid/autoid_test.go index 23961eac150e7..9adacbc82d21b 100644 --- a/meta/autoid/autoid_test.go +++ b/meta/autoid/autoid_test.go @@ -16,6 +16,7 @@ package autoid_test import ( "fmt" "math" + "math/rand" "sync" "testing" "time" @@ -60,6 +61,8 @@ func (*testSuite) TestT(c *C) { c.Assert(err, IsNil) err = m.CreateTableOrView(1, &model.TableInfo{ID: 3, Name: model.NewCIStr("t1")}) c.Assert(err, IsNil) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 4, Name: model.NewCIStr("t2")}) + c.Assert(err, IsNil) return nil }) c.Assert(err, IsNil) @@ -70,13 +73,13 @@ func (*testSuite) TestT(c *C) { globalAutoID, err := alloc.NextGlobalAutoID(1) c.Assert(err, IsNil) c.Assert(globalAutoID, Equals, int64(1)) - id, err := alloc.Alloc(1) + _, id, err := alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(1)) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(2)) - _, err = alloc.Alloc(0) + _, _, err = alloc.Alloc(0, 1) c.Assert(err, NotNil) globalAutoID, err = alloc.NextGlobalAutoID(1) c.Assert(err, IsNil) @@ -85,28 +88,28 @@ func (*testSuite) TestT(c *C) { // rebase err = alloc.Rebase(1, int64(1), true) c.Assert(err, IsNil) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(3)) err = alloc.Rebase(1, int64(3), true) c.Assert(err, IsNil) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(4)) err = alloc.Rebase(1, int64(10), true) c.Assert(err, IsNil) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(11)) err = alloc.Rebase(1, int64(3010), true) c.Assert(err, IsNil) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(3011)) alloc = autoid.NewAllocator(store, 1, false) c.Assert(alloc, NotNil) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(autoid.GetStep()+1)) @@ -114,7 +117,7 @@ func (*testSuite) TestT(c *C) { c.Assert(alloc, NotNil) err = alloc.Rebase(2, int64(1), false) c.Assert(err, IsNil) - id, err = alloc.Alloc(2) + _, id, err = alloc.Alloc(2, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(2)) @@ -126,22 +129,65 @@ func (*testSuite) TestT(c *C) { c.Assert(alloc, NotNil) err = alloc.Rebase(3, int64(3000), false) c.Assert(err, IsNil) - id, err = alloc.Alloc(3) + _, id, err = alloc.Alloc(3, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(3211)) err = alloc.Rebase(3, int64(6543), false) c.Assert(err, IsNil) - id, err = alloc.Alloc(3) + _, id, err = alloc.Alloc(3, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(6544)) // Test the MaxInt64 is the upper bound of `alloc` function but not `rebase`. err = alloc.Rebase(3, int64(math.MaxInt64-1), true) c.Assert(err, IsNil) - _, err = alloc.Alloc(3) + _, _, err = alloc.Alloc(3, 1) c.Assert(alloc, NotNil) err = alloc.Rebase(3, int64(math.MaxInt64), true) c.Assert(err, IsNil) + + // alloc N for signed + alloc = autoid.NewAllocator(store, 1, false) + c.Assert(alloc, NotNil) + globalAutoID, err = alloc.NextGlobalAutoID(4) + c.Assert(err, IsNil) + c.Assert(globalAutoID, Equals, int64(1)) + min, max, err := alloc.Alloc(4, 1) + c.Assert(err, IsNil) + c.Assert(max-min, Equals, int64(1)) + c.Assert(min+1, Equals, int64(1)) + + min, max, err = alloc.Alloc(4, 2) + c.Assert(err, IsNil) + c.Assert(max-min, Equals, int64(2)) + c.Assert(min+1, Equals, int64(2)) + c.Assert(max, Equals, int64(3)) + + min, max, err = alloc.Alloc(4, 100) + c.Assert(err, IsNil) + c.Assert(max-min, Equals, int64(100)) + expected := int64(4) + for i := min + 1; i <= max; i++ { + c.Assert(i, Equals, expected) + expected++ + } + + err = alloc.Rebase(4, int64(1000), false) + c.Assert(err, IsNil) + min, max, err = alloc.Alloc(4, 3) + c.Assert(err, IsNil) + c.Assert(max-min, Equals, int64(3)) + c.Assert(min+1, Equals, int64(1001)) + c.Assert(min+2, Equals, int64(1002)) + c.Assert(max, Equals, int64(1003)) + + lastRemainOne := alloc.End() + err = alloc.Rebase(4, alloc.End()-2, false) + c.Assert(err, IsNil) + min, max, err = alloc.Alloc(4, 5) + c.Assert(err, IsNil) + c.Assert(max-min, Equals, int64(5)) + c.Assert(min+1, Greater, lastRemainOne) } func (*testSuite) TestUnsignedAutoid(c *C) { @@ -164,6 +210,8 @@ func (*testSuite) TestUnsignedAutoid(c *C) { c.Assert(err, IsNil) err = m.CreateTableOrView(1, &model.TableInfo{ID: 3, Name: model.NewCIStr("t1")}) c.Assert(err, IsNil) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 4, Name: model.NewCIStr("t2")}) + c.Assert(err, IsNil) return nil }) c.Assert(err, IsNil) @@ -174,13 +222,13 @@ func (*testSuite) TestUnsignedAutoid(c *C) { globalAutoID, err := alloc.NextGlobalAutoID(1) c.Assert(err, IsNil) c.Assert(globalAutoID, Equals, int64(1)) - id, err := alloc.Alloc(1) + _, id, err := alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(1)) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(2)) - _, err = alloc.Alloc(0) + _, _, err = alloc.Alloc(0, 1) c.Assert(err, NotNil) globalAutoID, err = alloc.NextGlobalAutoID(1) c.Assert(err, IsNil) @@ -189,28 +237,28 @@ func (*testSuite) TestUnsignedAutoid(c *C) { // rebase err = alloc.Rebase(1, int64(1), true) c.Assert(err, IsNil) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(3)) err = alloc.Rebase(1, int64(3), true) c.Assert(err, IsNil) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(4)) err = alloc.Rebase(1, int64(10), true) c.Assert(err, IsNil) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(11)) err = alloc.Rebase(1, int64(3010), true) c.Assert(err, IsNil) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(3011)) alloc = autoid.NewAllocator(store, 1, true) c.Assert(alloc, NotNil) - id, err = alloc.Alloc(1) + _, id, err = alloc.Alloc(1, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(autoid.GetStep()+1)) @@ -218,7 +266,7 @@ func (*testSuite) TestUnsignedAutoid(c *C) { c.Assert(alloc, NotNil) err = alloc.Rebase(2, int64(1), false) c.Assert(err, IsNil) - id, err = alloc.Alloc(2) + _, id, err = alloc.Alloc(2, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(2)) @@ -230,12 +278,12 @@ func (*testSuite) TestUnsignedAutoid(c *C) { c.Assert(alloc, NotNil) err = alloc.Rebase(3, int64(3000), false) c.Assert(err, IsNil) - id, err = alloc.Alloc(3) + _, id, err = alloc.Alloc(3, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(3211)) err = alloc.Rebase(3, int64(6543), false) c.Assert(err, IsNil) - id, err = alloc.Alloc(3) + _, id, err = alloc.Alloc(3, 1) c.Assert(err, IsNil) c.Assert(id, Equals, int64(6544)) @@ -244,11 +292,40 @@ func (*testSuite) TestUnsignedAutoid(c *C) { un := int64(n) err = alloc.Rebase(3, un, true) c.Assert(err, IsNil) - _, err = alloc.Alloc(3) + _, _, err = alloc.Alloc(3, 1) c.Assert(err, NotNil) un = int64(n + 1) err = alloc.Rebase(3, un, true) c.Assert(err, IsNil) + + // alloc N for unsigned + alloc = autoid.NewAllocator(store, 1, true) + c.Assert(alloc, NotNil) + globalAutoID, err = alloc.NextGlobalAutoID(4) + c.Assert(err, IsNil) + c.Assert(globalAutoID, Equals, int64(1)) + + min, max, err := alloc.Alloc(4, 2) + c.Assert(err, IsNil) + c.Assert(max-min, Equals, int64(2)) + c.Assert(min+1, Equals, int64(1)) + c.Assert(max, Equals, int64(2)) + + err = alloc.Rebase(4, int64(500), true) + c.Assert(err, IsNil) + min, max, err = alloc.Alloc(4, 2) + c.Assert(err, IsNil) + c.Assert(max-min, Equals, int64(2)) + c.Assert(min+1, Equals, int64(501)) + c.Assert(max, Equals, int64(502)) + + lastRemainOne := alloc.End() + err = alloc.Rebase(4, alloc.End()-2, false) + c.Assert(err, IsNil) + min, max, err = alloc.Alloc(4, 5) + c.Assert(err, IsNil) + c.Assert(max-min, Equals, int64(5)) + c.Assert(min+1, Greater, lastRemainOne) } // TestConcurrentAlloc is used for the test that @@ -283,7 +360,7 @@ func (*testSuite) TestConcurrentAlloc(c *C) { allocIDs := func() { alloc := autoid.NewAllocator(store, dbID, false) for j := 0; j < int(autoid.GetStep())+5; j++ { - id, err1 := alloc.Alloc(tblID) + _, id, err1 := alloc.Alloc(tblID, 1) if err1 != nil { errCh <- err1 break @@ -297,6 +374,30 @@ func (*testSuite) TestConcurrentAlloc(c *C) { } m[id] = struct{}{} mu.Unlock() + + //test Alloc N + N := rand.Uint64() % 100 + min, max, err1 := alloc.Alloc(tblID, N) + if err1 != nil { + errCh <- err1 + break + } + + errFlag := false + mu.Lock() + for i := min + 1; i <= max; i++ { + if _, ok := m[i]; ok { + errCh <- fmt.Errorf("duplicate id:%v", i) + errFlag = true + mu.Unlock() + break + } + m[i] = struct{}{} + } + if errFlag { + break + } + mu.Unlock() } } for i := 0; i < count; i++ { @@ -336,7 +437,7 @@ func (*testSuite) TestRollbackAlloc(c *C) { injectConf.SetCommitError(errors.New("injected")) injectedStore := kv.NewInjectedStore(store, injectConf) alloc := autoid.NewAllocator(injectedStore, 1, false) - _, err = alloc.Alloc(2) + _, _, err = alloc.Alloc(2, 1) c.Assert(err, NotNil) c.Assert(alloc.Base(), Equals, int64(0)) c.Assert(alloc.End(), Equals, int64(0)) @@ -356,3 +457,34 @@ func (*testSuite) TestNextStep(c *C) { nextStep = autoid.NextStep(50000, 10*time.Minute) c.Assert(nextStep, Equals, int64(1000)) } + +func BenchmarkAllocator_Alloc(b *testing.B) { + b.StopTimer() + store, err := mockstore.NewMockTikvStore() + if err != nil { + return + } + defer store.Close() + dbID := int64(1) + tblID := int64(2) + err = kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + err = m.CreateDatabase(&model.DBInfo{ID: dbID, Name: model.NewCIStr("a")}) + if err != nil { + return err + } + err = m.CreateTableOrView(dbID, &model.TableInfo{ID: tblID, Name: model.NewCIStr("t")}) + if err != nil { + return err + } + return nil + }) + if err != nil { + return + } + alloc := autoid.NewAllocator(store, 1, false) + b.StartTimer() + for i := 0; i < b.N; i++ { + alloc.Alloc(2, 1) + } +} diff --git a/table/table.go b/table/table.go index 1a3aad7a81d7e..c06fa60186333 100644 --- a/table/table.go +++ b/table/table.go @@ -173,7 +173,20 @@ func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Conte span1 := span.Tracer().StartSpan("table.AllocAutoIncrementValue", opentracing.ChildOf(span.Context())) defer span1.Finish() } - return t.Allocator(sctx).Alloc(t.Meta().ID) + _, max, err := t.Allocator(sctx).Alloc(t.Meta().ID, uint64(1)) + if err != nil { + return 0, err + } + return max, err +} + +// AllocBatchAutoIncrementValue allocates batch auto_increment value (min and max] for rows. +func AllocBatchAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Context, N int) (int64, int64, error) { + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span1 := span.Tracer().StartSpan("table.AllocBatchAutoIncrementValue", opentracing.ChildOf(span.Context())) + defer span1.Finish() + } + return t.Allocator(sctx).Alloc(t.Meta().ID, uint64(N)) } // PhysicalTable is an abstraction for two kinds of table representation: partition or non-partitioned table. diff --git a/table/tables/tables.go b/table/tables/tables.go index 290f5c599a6e4..3778ed5453d14 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -912,7 +912,7 @@ func GetColDefaultValue(ctx sessionctx.Context, col *table.Column, defaultVals [ // AllocHandle implements table.Table AllocHandle interface. func (t *tableCommon) AllocHandle(ctx sessionctx.Context) (int64, error) { - rowID, err := t.Allocator(ctx).Alloc(t.tableID) + _, rowID, err := t.Allocator(ctx).Alloc(t.tableID, 1) if err != nil { return 0, err } diff --git a/util/kvencoder/allocator.go b/util/kvencoder/allocator.go index 36a80382819f2..2ed59d3dac282 100644 --- a/util/kvencoder/allocator.go +++ b/util/kvencoder/allocator.go @@ -36,8 +36,9 @@ type Allocator struct { } // Alloc allocs a next autoID for table with tableID. -func (alloc *Allocator) Alloc(tableID int64) (int64, error) { - return atomic.AddInt64(&alloc.base, 1), nil +func (alloc *Allocator) Alloc(tableID int64, n uint64) (int64, int64, error) { + min := alloc.base + return min, atomic.AddInt64(&alloc.base, int64(n)), nil } // Reset allow newBase smaller than alloc.base, and will set the alloc.base to newBase. From 6f8e1ab512642fa6b767d8bc294c0c8d455fcf96 Mon Sep 17 00:00:00 2001 From: pingcap-github-bot Date: Mon, 14 Oct 2019 00:23:04 +0800 Subject: [PATCH 6/6] util/admin: fix the error of "invalid list index" when cancel ddl jobs (#12402) (#12671) --- meta/meta.go | 9 +++++++-- util/admin/admin.go | 14 ++++++++++---- util/admin/admin_test.go | 32 ++++++++++++++++++++++++++++++++ 3 files changed, 49 insertions(+), 6 deletions(-) diff --git a/meta/meta.go b/meta/meta.go index e5eead8b8bbd6..4104d1a11929f 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -457,8 +457,13 @@ func (m *Meta) enQueueDDLJob(key []byte, job *model.Job) error { } // EnQueueDDLJob adds a DDL job to the list. -func (m *Meta) EnQueueDDLJob(job *model.Job) error { - return m.enQueueDDLJob(m.jobListKey, job) +func (m *Meta) EnQueueDDLJob(job *model.Job, jobListKeys ...JobListKeyType) error { + listKey := m.jobListKey + if len(jobListKeys) != 0 { + listKey = jobListKeys[0] + } + + return m.enQueueDDLJob(listKey, job) } func (m *Meta) deQueueDDLJob(key []byte) (*model.Job, error) { diff --git a/util/admin/admin.go b/util/admin/admin.go index a580edbb61bae..d974804ee2492 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -119,13 +119,18 @@ func CancelJobs(txn kv.Transaction, ids []int64) ([]error, error) { return nil, nil } - jobs, err := GetDDLJobs(txn) + errs := make([]error, len(ids)) + t := meta.NewMeta(txn) + generalJobs, err := getDDLJobsInQueue(t, meta.DefaultJobListKey) + if err != nil { + return nil, errors.Trace(err) + } + addIdxJobs, err := getDDLJobsInQueue(t, meta.AddIndexJobListKey) if err != nil { return nil, errors.Trace(err) } + jobs := append(generalJobs, addIdxJobs...) - errs := make([]error, len(ids)) - t := meta.NewMeta(txn) for i, id := range ids { found := false for j, job := range jobs { @@ -158,7 +163,8 @@ func CancelJobs(txn kv.Transaction, ids []int64) ([]error, error) { continue } if job.Type == model.ActionAddIndex { - err = t.UpdateDDLJob(int64(j), job, true, meta.AddIndexJobListKey) + offset := int64(j - len(generalJobs)) + err = t.UpdateDDLJob(offset, job, true, meta.AddIndexJobListKey) } else { err = t.UpdateDDLJob(int64(j), job, true) } diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index 97fe10063d5f7..4dc0730ea86f9 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -254,6 +254,38 @@ func (s *testSuite) TestCancelJobs(c *C) { c.Assert(errs[0], NotNil) c.Assert(errs[0].Error(), Equals, "[admin:6]This job:101 is almost finished, can't be cancelled now") + // When both types of jobs exist in the DDL queue, + // we first cancel the job with a larger ID. + job = &model.Job{ + ID: 1000, + SchemaID: 1, + TableID: 2, + Type: model.ActionAddIndex, + } + job1 := &model.Job{ + ID: 1001, + SchemaID: 1, + TableID: 2, + Type: model.ActionAddColumn, + } + job2 := &model.Job{ + ID: 1002, + SchemaID: 1, + TableID: 2, + Type: model.ActionAddIndex, + } + err = t.EnQueueDDLJob(job, meta.AddIndexJobListKey) + c.Assert(err, IsNil) + err = t.EnQueueDDLJob(job1) + c.Assert(err, IsNil) + err = t.EnQueueDDLJob(job2, meta.AddIndexJobListKey) + c.Assert(err, IsNil) + errs, err = CancelJobs(txn, []int64{job1.ID, job.ID, job2.ID}) + c.Assert(err, IsNil) + for _, err := range errs { + c.Assert(err, IsNil) + } + err = txn.Rollback() c.Assert(err, IsNil) }