diff --git a/CHANGELOG.md b/CHANGELOG.md index bf4f96510c353..df77268b59263 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,5 @@ # TiDB Changelog -All notable changes to this project will be documented in this file. See also [Release Notes](https://github.com/pingcap/docs/blob/master/releases/rn.md), [TiKV Changelog](https://github.com/tikv/tikv/blob/master/CHANGELOG.md) and [PD Changelog](https://github.com/tikv/pd/blob/master/CHANGELOG.md). +All notable changes to this project will be documented in this file. See also [Release Notes](https://github.com/pingcap/docs/blob/master/releases/release-notes.md), [TiKV Changelog](https://github.com/tikv/tikv/blob/master/CHANGELOG.md) and [PD Changelog](https://github.com/tikv/pd/blob/master/CHANGELOG.md). ## [3.0.4] 2019-10-08 ## New features diff --git a/ddl/backfilling.go b/ddl/backfilling.go index cd8935572f45c..56512eec6ab65 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -159,7 +159,7 @@ func newBackfillWorker(sessCtx sessionctx.Context, worker *worker, id int, t tab sessCtx: sessCtx, taskCh: make(chan *reorgBackfillTask, 1), resultCh: make(chan *backfillResult, 1), - priority: tikvstore.PriorityLow, + priority: kv.PriorityLow, } } diff --git a/ddl/column.go b/ddl/column.go index 805d1d2e73d1e..18c23b4d9c45a 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -828,6 +828,11 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in return w.doModifyColumn(d, t, job, dbInfo, tblInfo, jobParam.newCol, oldCol, jobParam.pos) } + if err = isGeneratedRelatedColumn(tblInfo, jobParam.newCol, oldCol); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + if jobParam.changingCol == nil { changingColPos := &ast.ColumnPosition{Tp: ast.ColumnPositionNone} newColName := model.NewCIStr(genChangingColumnUniqueName(tblInfo, oldCol)) @@ -1256,7 +1261,7 @@ func (w *updateColumnWorker) fetchRowColVals(txn kv.Transaction, taskRange reorg } func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, rawRow []byte) error { - _, err := w.rowDecoder.DecodeAndEvalRowWithMap(w.sessCtx, handle, rawRow, time.UTC, timeutil.SystemLocation(), w.rowMap) + _, err := w.rowDecoder.DecodeTheExistedColumnMap(w.sessCtx, handle, rawRow, time.UTC, w.rowMap) if err != nil { return errors.Trace(errCantDecodeRecord.GenWithStackByArgs("column", err)) } @@ -1294,6 +1299,11 @@ func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, ra }) w.rowMap[w.newColInfo.ID] = newColVal + _, err = w.rowDecoder.EvalRemainedExprColumnMap(w.sessCtx, timeutil.SystemLocation(), w.rowMap) + if err != nil { + return errors.Trace(err) + } + newColumnIDs := make([]int64, 0, len(w.rowMap)) newRow := make([]types.Datum, 0, len(w.rowMap)) for colID, val := range w.rowMap { diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 68ee059f47305..992631b4bd97b 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -1702,6 +1702,20 @@ func (s *testColumnTypeChangeSuite) TestChangingAttributeOfColumnWithFK(c *C) { tk.MustExec("drop table if exists orders, users") } +func (s *testColumnTypeChangeSuite) TestAlterPrimaryKeyToNull(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.Se.GetSessionVars().EnableChangeColumnType = true + + tk.MustExec("drop table if exists t, t1") + tk.MustExec("create table t(a int not null, b int not null, primary key(a, b));") + tk.MustGetErrCode("alter table t modify a bigint null;", mysql.ErrPrimaryCantHaveNull) + tk.MustGetErrCode("alter table t change column a a bigint null;", mysql.ErrPrimaryCantHaveNull) + tk.MustExec("create table t1(a int not null, b int not null, primary key(a));") + tk.MustGetErrCode("alter table t modify a bigint null;", mysql.ErrPrimaryCantHaveNull) + tk.MustGetErrCode("alter table t change column a a bigint null;", mysql.ErrPrimaryCantHaveNull) +} + // Close issue #23202 func (s *testColumnTypeChangeSuite) TestDDLExitWhenCancelMeetPanic(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -1752,3 +1766,34 @@ func (s *testColumnTypeChangeSuite) TestDDLExitWhenCancelMeetPanic(c *C) { c.Assert(job.ErrorCount, Equals, int64(4)) c.Assert(job.Error.Error(), Equals, "[ddl:-1]panic in handling DDL logic and error count beyond the limitation 3, cancelled") } + +// Close issue #24253 +func (s *testColumnTypeChangeSuite) TestChangeIntToBitWillPanicInBackfillIndexes(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + // Enable column change variable. + tk.Se.GetSessionVars().EnableChangeColumnType = true + + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE `t` (" + + " `a` int(11) DEFAULT NULL," + + " `b` varchar(10) DEFAULT NULL," + + " `c` decimal(10,2) DEFAULT NULL," + + " KEY `idx1` (`a`)," + + " UNIQUE KEY `idx2` (`a`)," + + " KEY `idx3` (`a`,`b`)," + + " KEY `idx4` (`a`,`b`,`c`)" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") + tk.MustExec("insert into t values(19,1,1),(17,2,2)") + tk.MustExec("alter table t modify a bit(5) not null") + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" + + " `a` bit(5) NOT NULL,\n" + + " `b` varchar(10) DEFAULT NULL,\n" + + " `c` decimal(10,2) DEFAULT NULL,\n" + + " KEY `idx1` (`a`),\n" + + " UNIQUE KEY `idx2` (`a`),\n" + + " KEY `idx3` (`a`,`b`),\n" + + " KEY `idx4` (`a`,`b`,`c`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustQuery("select * from t").Check(testkit.Rows("\x13 1 1.00", "\x11 2 2.00")) +} diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index de9d8eb504866..7c3a0f9ad970f 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -1056,6 +1056,26 @@ func (s *testStateChangeSuite) TestParallelAlterModifyColumn(c *C) { s.testControlParallelExecSQL(c, sql, sql, f) } +func (s *testStateChangeSuite) TestParallelAddGeneratedColumnAndAlterModifyColumn(c *C) { + _, err := s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 1") + c.Assert(err, IsNil) + defer func() { + _, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0") + c.Assert(err, IsNil) + }() + domain.GetDomain(s.se).GetGlobalVarsCache().Disable() + + sql1 := "ALTER TABLE t ADD COLUMN f INT GENERATED ALWAYS AS(a+1);" + sql2 := "ALTER TABLE t MODIFY COLUMN a tinyint;" + f := func(c *C, err1, err2 error) { + c.Assert(err1, IsNil) + c.Assert(err2.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, oldCol is a dependent column 'a' for generated column") + _, err := s.se.Execute(context.Background(), "select * from t") + c.Assert(err, IsNil) + } + s.testControlParallelExecSQL(c, sql1, sql2, f) +} + func (s *testStateChangeSuite) TestParallelAlterModifyColumnAndAddPK(c *C) { _, err := s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 1") c.Assert(err, IsNil) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 6481fd9d8fe11..b52644c151fd4 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/ddl/testutil" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" tmysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" @@ -583,6 +584,12 @@ create table log_message_1 ( tk.MustExec("drop table if exists t;") tk.MustExec(`create table t(a int) partition by range (a) (partition p0 values less than (18446744073709551615));`) + + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t(a binary) partition by range columns (a) (partition p0 values less than (X'0C'));`) + tk.MustExec(`alter table t add partition (partition p1 values less than (X'0D'), partition p2 values less than (X'0E'));`) + tk.MustExec(`insert into t values (X'0B'), (X'0C'), (X'0D')`) + tk.MustQuery(`select * from t where a < X'0D'`).Check(testkit.Rows("\x0B", "\x0C")) } func (s *testIntegrationSuite1) TestDisableTablePartition(c *C) { @@ -743,6 +750,7 @@ func (s *testIntegrationSuite1) TestCreateTableWithListPartition(c *C) { "create table t (a bigint) partition by list (a) (partition p0 values in (to_seconds('2020-09-28 17:03:38'),to_seconds('2020-09-28 17:03:39')));", "create table t (a datetime) partition by list (to_seconds(a)) (partition p0 values in (to_seconds('2020-09-28 17:03:38'),to_seconds('2020-09-28 17:03:39')));", "create table t (a int, b int generated always as (a+1) virtual) partition by list (b + 1) (partition p0 values in (1));", + "create table t(a binary) partition by list columns (a) (partition p0 values in (X'0C'));", s.generatePartitionTableByNum(ddl.PartitionCountLimit), } @@ -3334,7 +3342,7 @@ func (s *testIntegrationSuite7) TestAddPartitionForTableWithWrongType(c *C) { _, err = tk.Exec("alter table t_char add partition (partition p1 values less than (0x20))") c.Assert(err, NotNil) - c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) + c.Assert(ddl.ErrRangeNotIncreasing.Equal(err), IsTrue) _, err = tk.Exec("alter table t_char add partition (partition p1 values less than (10))") c.Assert(err, NotNil) @@ -3378,3 +3386,31 @@ func (s *testIntegrationSuite7) TestPartitionListWithNewCollation(c *C) { str := tk.MustQuery(`desc select * from t11 where a = 'b';`).Rows()[0][3].(string) c.Assert(strings.Contains(str, "partition:p0"), IsTrue) } + +func (s *testIntegrationSuite7) TestAddTableWithPartition(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists global_partition_table;") + tk.MustGetErrCode("create global temporary table global_partition_table (a int, b int) partition by hash(a) partitions 3 ON COMMIT DELETE ROWS;", errno.ErrPartitionNoTemporary) + tk.MustExec("drop table if exists global_partition_table;") + tk.MustExec("drop table if exists partition_table;") + _, err := tk.Exec("create table partition_table (a int, b int) partition by hash(a) partitions 3;") + c.Assert(err, IsNil) + tk.MustExec("drop table if exists partition_table;") + tk.MustExec("drop table if exists partition_range_table;") + tk.MustGetErrCode(`create global temporary table partition_range_table (c1 smallint(6) not null, c2 char(5) default null) partition by range ( c1 ) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than (30), + partition p3 values less than (MAXVALUE) + ) ON COMMIT DELETE ROWS;`, errno.ErrPartitionNoTemporary) + tk.MustExec("drop table if exists partition_range_table;") + tk.MustExec("drop table if exists partition_list_table;") + tk.MustExec("set @@session.tidb_enable_list_partition = ON") + tk.MustGetErrCode(`create global temporary table partition_list_table (id int) partition by list (id) ( + partition p0 values in (1,2), + partition p1 values in (3,4), + partition p3 values in (5,null) + ) ON COMMIT DELETE ROWS;`, errno.ErrPartitionNoTemporary) + tk.MustExec("drop table if exists partition_list_table;") +} diff --git a/ddl/db_test.go b/ddl/db_test.go index 7970b14989aaf..e865de39d3248 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -77,6 +77,7 @@ var _ = Suite(&testDBSuite5{&testDBSuite{}}) var _ = Suite(&testDBSuite6{&testDBSuite{}}) var _ = Suite(&testDBSuite7{&testDBSuite{}}) var _ = SerialSuites(&testSerialDBSuite{&testDBSuite{}}) +var _ = Suite(&testDBSuite8{&testDBSuite{}}) const defaultBatchSize = 1024 const defaultReorgBatchSize = 256 @@ -145,6 +146,7 @@ type testDBSuite5 struct{ *testDBSuite } type testDBSuite6 struct{ *testDBSuite } type testDBSuite7 struct{ *testDBSuite } type testSerialDBSuite struct{ *testDBSuite } +type testDBSuite8 struct{ *testDBSuite } func testAddIndexWithPK(tk *testkit.TestKit) { tk.MustExec("drop table if exists test_add_index_with_pk") @@ -6700,3 +6702,32 @@ func (s *testSerialDBSuite) TestJsonUnmarshalErrWhenPanicInCancellingPath(c *C) _, err := tk.Exec("alter table test_add_index_after_add_col add unique index cc(c);") c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '0' for key 'cc'") } + +// For Close issue #24288 +// see https://github.com/pingcap/tidb/issues/24288 +func (s *testDBSuite8) TestDdlMaxLimitOfIdentifier(c *C) { + tk := testkit.NewTestKit(c, s.store) + + // create/drop database test + longDbName := strings.Repeat("库", mysql.MaxDatabaseNameLength-1) + tk.MustExec(fmt.Sprintf("create database %s", longDbName)) + defer func() { + tk.MustExec(fmt.Sprintf("drop database %s", longDbName)) + }() + tk.MustExec(fmt.Sprintf("use %s", longDbName)) + + // create/drop table,index test + longTblName := strings.Repeat("表", mysql.MaxTableNameLength-1) + longColName := strings.Repeat("三", mysql.MaxColumnNameLength-1) + longIdxName := strings.Repeat("索", mysql.MaxIndexIdentifierLen-1) + tk.MustExec(fmt.Sprintf("create table %s(f1 int primary key,f2 int, %s varchar(50))", longTblName, longColName)) + tk.MustExec(fmt.Sprintf("create index %s on %s(%s)", longIdxName, longTblName, longColName)) + defer func() { + tk.MustExec(fmt.Sprintf("drop index %s on %s", longIdxName, longTblName)) + tk.MustExec(fmt.Sprintf("drop table %s", longTblName)) + }() + + // alter table + tk.MustExec(fmt.Sprintf("alter table %s change f2 %s int", longTblName, strings.Repeat("二", mysql.MaxColumnNameLength-1))) + +} diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 6520bc77dbba9..b03b4ca66f536 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -25,6 +25,7 @@ import ( "strings" "sync/atomic" "time" + "unicode/utf8" "github.com/cznic/mathutil" "github.com/go-yaml/yaml" @@ -225,21 +226,21 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) } func checkTooLongSchema(schema model.CIStr) error { - if len(schema.L) > mysql.MaxDatabaseNameLength { + if utf8.RuneCountInString(schema.L) > mysql.MaxDatabaseNameLength { return ErrTooLongIdent.GenWithStackByArgs(schema) } return nil } func checkTooLongTable(table model.CIStr) error { - if len(table.L) > mysql.MaxTableNameLength { + if utf8.RuneCountInString(table.L) > mysql.MaxTableNameLength { return ErrTooLongIdent.GenWithStackByArgs(table) } return nil } func checkTooLongIndex(index model.CIStr) error { - if len(index.L) > mysql.MaxIndexIdentifierLen { + if utf8.RuneCountInString(index.L) > mysql.MaxIndexIdentifierLen { return ErrTooLongIdent.GenWithStackByArgs(index) } return nil @@ -689,24 +690,7 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, o } } - processDefaultValue(col, hasDefaultValue, setOnUpdateNow) - - processColumnFlags(col) - - err = checkPriKeyConstraint(col, hasDefaultValue, hasNullFlag, outPriKeyConstraint) - if err != nil { - return nil, nil, errors.Trace(err) - } - err = checkColumnValueConstraint(col, col.Collate) - if err != nil { - return nil, nil, errors.Trace(err) - } - err = checkDefaultValue(ctx, col, hasDefaultValue) - if err != nil { - return nil, nil, errors.Trace(err) - } - err = checkColumnFieldLength(col) - if err != nil { + if err = processAndCheckDefaultValueAndColumn(ctx, col, outPriKeyConstraint, hasDefaultValue, setOnUpdateNow, hasNullFlag); err != nil { return nil, nil, errors.Trace(err) } return col, constraints, nil @@ -1107,7 +1091,7 @@ func checkGeneratedColumn(colDefs []*ast.ColumnDef) error { func checkTooLongColumn(cols []*model.ColumnInfo) error { for _, col := range cols { colName := col.Name.O - if len(colName) > mysql.MaxColumnNameLength { + if utf8.RuneCountInString(colName) > mysql.MaxColumnNameLength { return ErrTooLongIdent.GenWithStackByArgs(colName) } } @@ -1439,11 +1423,6 @@ func buildTableInfo( if isSingleIntPK { tbInfo.PKIsHandle = true } else { - hasBinlog := ctx.GetSessionVars().BinlogClient != nil - if hasBinlog { - msg := mysql.Message("Cannot create clustered index table when the binlog is ON", nil) - return nil, dbterror.ClassDDL.NewStdErr(errno.ErrUnsupportedDDLOperation, msg) - } tbInfo.IsCommonHandle = true tbInfo.CommonHandleVersion = 1 } @@ -1751,6 +1730,14 @@ func buildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCh if err != nil { return nil, errors.Trace(err) } + switch s.TemporaryKeyword { + case ast.TemporaryGlobal: + tbInfo.TempTableType = model.TempTableGlobal + case ast.TemporaryLocal: + tbInfo.TempTableType = model.TempTableLocal + case ast.TemporaryNone: + tbInfo.TempTableType = model.TempTableNone + } if err = setTableAutoRandomBits(ctx, tbInfo, colDefs); err != nil { return nil, errors.Trace(err) @@ -2714,7 +2701,7 @@ func checkAndCreateNewColumn(ctx sessionctx.Context, ti ast.Ident, schema *model if err = checkColumnAttributes(colName, specNewColumn.Tp); err != nil { return nil, errors.Trace(err) } - if len(colName) > mysql.MaxColumnNameLength { + if utf8.RuneCountInString(colName) > mysql.MaxColumnNameLength { return nil, ErrTooLongIdent.GenWithStackByArgs(colName) } @@ -3671,6 +3658,7 @@ func processColumnOptions(ctx sessionctx.Context, col *table.Column, options []* var hasDefaultValue, setOnUpdateNow bool var err error + var hasNullFlag bool for _, opt := range options { switch opt.Tp { case ast.ColumnOptionDefaultValue: @@ -3686,6 +3674,7 @@ func processColumnOptions(ctx sessionctx.Context, col *table.Column, options []* case ast.ColumnOptionNotNull: col.Flag |= mysql.NotNullFlag case ast.ColumnOptionNull: + hasNullFlag = true col.Flag &= ^mysql.NotNullFlag case ast.ColumnOptionAutoIncrement: col.Flag |= mysql.AutoIncrementFlag @@ -3730,14 +3719,30 @@ func processColumnOptions(ctx sessionctx.Context, col *table.Column, options []* } } - processDefaultValue(col, hasDefaultValue, setOnUpdateNow) + if err = processAndCheckDefaultValueAndColumn(ctx, col, nil, hasDefaultValue, setOnUpdateNow, hasNullFlag); err != nil { + return errors.Trace(err) + } + return nil +} + +func processAndCheckDefaultValueAndColumn(ctx sessionctx.Context, col *table.Column, outPriKeyConstraint *ast.Constraint, hasDefaultValue, setOnUpdateNow, hasNullFlag bool) error { + processDefaultValue(col, hasDefaultValue, setOnUpdateNow) processColumnFlags(col) - if hasDefaultValue { - return errors.Trace(checkDefaultValue(ctx, col, true)) + err := checkPriKeyConstraint(col, hasDefaultValue, hasNullFlag, outPriKeyConstraint) + if err != nil { + return errors.Trace(err) + } + if err = checkColumnValueConstraint(col, col.Collate); err != nil { + return errors.Trace(err) + } + if err = checkDefaultValue(ctx, col, hasDefaultValue); err != nil { + return errors.Trace(err) + } + if err = checkColumnFieldLength(col); err != nil { + return errors.Trace(err) } - return nil } @@ -3850,10 +3855,6 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or return nil, errors.Trace(err) } - if err = checkColumnValueConstraint(newCol, newCol.Collate); err != nil { - return nil, errors.Trace(err) - } - if err = checkModifyTypes(ctx, &col.FieldType, &newCol.FieldType, isColumnWithIndex(col.Name.L, t.Meta().Indices)); err != nil { if strings.Contains(err.Error(), "Unsupported modifying collation") { colErrMsg := "Unsupported modifying collation of column '%s' from '%s' to '%s' when index is defined on it." @@ -3862,11 +3863,10 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or return nil, errors.Trace(err) } if ctx.GetSessionVars().EnableChangeColumnType && needChangeColumnData(col.ColumnInfo, newCol.ColumnInfo) { - if newCol.IsGenerated() || col.IsGenerated() { - // TODO: Make it compatible with MySQL error. - msg := fmt.Sprintf("tidb_enable_change_column_type is true, newCol IsGenerated %v, oldCol IsGenerated %v", newCol.IsGenerated(), col.IsGenerated()) - return nil, errUnsupportedModifyColumn.GenWithStackByArgs(msg) - } else if t.Meta().Partition != nil { + if err = isGeneratedRelatedColumn(t.Meta(), newCol.ColumnInfo, col.ColumnInfo); err != nil { + return nil, errors.Trace(err) + } + if t.Meta().Partition != nil { return nil, errUnsupportedModifyColumn.GenWithStackByArgs("tidb_enable_change_column_type is true, table is partition table") } } @@ -3890,10 +3890,6 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or modifyColumnTp = mysql.TypeNull } - if err = checkColumnFieldLength(newCol); err != nil { - return nil, err - } - if err = checkColumnWithIndexConstraint(t.Meta(), col.ColumnInfo, newCol.ColumnInfo); err != nil { return nil, err } @@ -4980,7 +4976,7 @@ func buildHiddenColumnInfo(ctx sessionctx.Context, indexPartSpecifications []*as } idxPart.Length = types.UnspecifiedLength // The index part is an expression, prepare a hidden column for it. - if len(idxPart.Column.Name.L) > mysql.MaxColumnNameLength { + if utf8.RuneCountInString(idxPart.Column.Name.L) > mysql.MaxColumnNameLength { // TODO: Refine the error message. return nil, ErrTooLongIdent.GenWithStackByArgs("hidden column") } @@ -5360,14 +5356,10 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI } func isDroppableColumn(tblInfo *model.TableInfo, colName model.CIStr) error { - // Check whether there are other columns depend on this column or not. - for _, col := range tblInfo.Columns { - for dep := range col.Dependences { - if dep == colName.L { - return errDependentByGeneratedColumn.GenWithStackByArgs(dep) - } - } + if ok, dep := hasDependentByGeneratedColumn(tblInfo, colName); ok { + return errDependentByGeneratedColumn.GenWithStackByArgs(dep) } + if len(tblInfo.Columns) == 1 { return ErrCantRemoveAllFields.GenWithStack("can't drop only column %s in table %s", colName, tblInfo.Name) @@ -5470,7 +5462,7 @@ func checkColumnsTypeAndValuesMatch(ctx sessionctx.Context, meta *model.TableInf } case mysql.TypeString, mysql.TypeVarString: switch vkind { - case types.KindString, types.KindBytes, types.KindNull: + case types.KindString, types.KindBytes, types.KindNull, types.KindBinaryLiteral: default: return ErrWrongTypeColumnValue.GenWithStackByArgs() } diff --git a/ddl/error.go b/ddl/error.go index 852d2218c6275..463c9c405a19e 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -274,4 +274,7 @@ var ( ErrUnknownEngine = dbterror.ClassDDL.NewStd(mysql.ErrUnknownStorageEngine) errExchangePartitionDisabled = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("Exchange Partition is disabled, please set 'tidb_enable_exchange_partition' if you need to need to enable it", nil)) + + // ErrPartitionNoTemporary returns when partition at temporary mode + ErrPartitionNoTemporary = dbterror.ClassDDL.NewStd(mysql.ErrPartitionNoTemporary) ) diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index 78d6ad9decfd6..8ccb7a5adb594 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -524,6 +524,8 @@ func (s *testFailDBSuite) TestModifyColumn(c *C) { c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, newCol IsGenerated false, oldCol IsGenerated true") _, err = tk.Exec("alter table t2 modify column a mediumint generated always as(id+1) stored") c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, newCol IsGenerated true, oldCol IsGenerated false") + _, err = tk.Exec("alter table t2 modify column a mediumint") + c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, oldCol is a dependent column 'a' for generated column") // Test multiple rows of data. tk.MustExec("create table t3(a int not null default 1, b int default 2, c int not null default 0, primary key(c), index idx(b), index idx1(a), index idx2(b, c))") diff --git a/ddl/generated_column.go b/ddl/generated_column.go index 0b33ad2a90b14..657a27ec3db4f 100644 --- a/ddl/generated_column.go +++ b/ddl/generated_column.go @@ -14,6 +14,8 @@ package ddl import ( + "fmt" + "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" @@ -137,6 +139,31 @@ func findColumnNamesInExpr(expr ast.ExprNode) []*ast.ColumnName { return c.cols } +// hasDependentByGeneratedColumn checks whether there are other columns depend on this column or not. +func hasDependentByGeneratedColumn(tblInfo *model.TableInfo, colName model.CIStr) (bool, string) { + for _, col := range tblInfo.Columns { + for dep := range col.Dependences { + if dep == colName.L { + return true, dep + } + } + } + return false, "" +} + +func isGeneratedRelatedColumn(tblInfo *model.TableInfo, newCol, col *model.ColumnInfo) error { + if newCol.IsGenerated() || col.IsGenerated() { + // TODO: Make it compatible with MySQL error. + msg := fmt.Sprintf("tidb_enable_change_column_type is true, newCol IsGenerated %v, oldCol IsGenerated %v", newCol.IsGenerated(), col.IsGenerated()) + return errUnsupportedModifyColumn.GenWithStackByArgs(msg) + } + if ok, dep := hasDependentByGeneratedColumn(tblInfo, col.Name); ok { + msg := fmt.Sprintf("tidb_enable_change_column_type is true, oldCol is a dependent column '%s' for generated column", dep) + return errUnsupportedModifyColumn.GenWithStackByArgs(msg) + } + return nil +} + type generatedColumnChecker struct { cols []*ast.ColumnName } diff --git a/ddl/reorg.go b/ddl/reorg.go index dd24b2934dc4b..fbe42573dbbf7 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -425,7 +424,7 @@ func (dc *ddlCtx) buildDescTableScan(ctx context.Context, startTS uint64, tbl ta SetConcurrency(1).SetDesc(true) builder.Request.NotFillCache = true - builder.Request.Priority = tikvstore.PriorityLow + builder.Request.Priority = kv.PriorityLow kvReq, err := builder.Build() if err != nil { diff --git a/ddl/table.go b/ddl/table.go index 668de3ac41c05..acd209a7bb0da 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -56,6 +56,11 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) job.State = model.JobStateCancelled return ver, errors.Trace(err) } + if tbInfo.Partition != nil && (tbInfo.TempTableType == model.TempTableGlobal || tbInfo.TempTableType == model.TempTableLocal) { + // unsupported ddl, cancel this job. + job.State = model.JobStateCancelled + return ver, errors.Trace(ErrPartitionNoTemporary) + } tbInfo.State = model.StateNone err := checkTableNotExists(d, t, schemaID, tbInfo.Name.L) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index ac53de8a0c4a3..ce577b993d009 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -145,8 +144,8 @@ func (builder *RequestBuilder) SetAnalyzeRequest(ana *tipb.AnalyzeReq) *RequestB builder.Request.Tp = kv.ReqTypeAnalyze builder.Request.Data, builder.err = ana.Marshal() builder.Request.NotFillCache = true - builder.Request.IsolationLevel = tikvstore.RC - builder.Request.Priority = tikvstore.PriorityLow + builder.Request.IsolationLevel = kv.RC + builder.Request.Priority = kv.PriorityLow } return builder @@ -199,24 +198,24 @@ func (builder *RequestBuilder) SetAllowBatchCop(batchCop bool) *RequestBuilder { return builder } -func (builder *RequestBuilder) getIsolationLevel() tikvstore.IsoLevel { +func (builder *RequestBuilder) getIsolationLevel() kv.IsoLevel { switch builder.Tp { case kv.ReqTypeAnalyze: - return tikvstore.RC + return kv.RC } - return tikvstore.SI + return kv.SI } func (builder *RequestBuilder) getKVPriority(sv *variable.SessionVars) int { switch sv.StmtCtx.Priority { case mysql.NoPriority, mysql.DelayedPriority: - return tikvstore.PriorityNormal + return kv.PriorityNormal case mysql.LowPriority: - return tikvstore.PriorityLow + return kv.PriorityLow case mysql.HighPriority: - return tikvstore.PriorityHigh + return kv.PriorityHigh } - return tikvstore.PriorityNormal + return kv.PriorityNormal } // SetFromSessionVars sets the following fields for "kv.Request" from session variables: diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index 5b945a9ea7986..18c1ee8fc24e4 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -533,7 +533,7 @@ func (s *testSuite) TestRequestBuilder5(c *C) { KeepOrder: true, Desc: false, Concurrency: 15, - IsolationLevel: tikvstore.RC, + IsolationLevel: kv.RC, Priority: 1, NotFillCache: true, SyncLog: false, diff --git a/docs/design/2020-06-01-global-kill.md b/docs/design/2020-06-01-global-kill.md index 61cc6b8504368..b1f92c40b7aaa 100644 --- a/docs/design/2020-06-01-global-kill.md +++ b/docs/design/2020-06-01-global-kill.md @@ -1,12 +1,12 @@ # Global Kill - Author(s): [pingyu](https://github.com/pingyu) (Ping Yu) -- Last updated: 2020-10-25 +- Last updated: 2021-05-05 - Discussion at: https://github.com/pingcap/tidb/issues/8854 ## Abstract -This document introduces the design of global connection id, and the global `KILL ` based on it. +This document introduces the design of global connection id, and the global `KILL ` based on it. ## Background @@ -17,57 +17,104 @@ Currently connection ids are local to TiDB instances, which means that a `KILL x To support "Global Kill", we need: 1. Global connection ids, which are unique among all TiDB instances. 2. Redirect `KILL x` to target TiDB instance, on which the connection `x` is running. -3. Support both 32 & 64 bits `connId`, to be compatible with legacy 32 bits clients. In this stage, we only design the 64 bits `connId`, and left a `markup` to distinguish between these two kinds. +3. Support both 32 & 64 bits `connID`. 32 bits `connID` is used on small clusters (number of TiDB instances less than 2048), to be fully compatible with all clients including legacy 32 bits ones, while 64 bits `connID` is used for big clusters. Bit 0 in `connID` is a markup to distinguish between these two kinds. ## Rationale -#### 1. Structure of `connId` -##### 64 bits version +#### 1. Structure of `connID` +##### 32 bits +``` + 31 21 20 1 0 + +--------+------------------+------+ + |serverID| local connID |markup| + | (11b) | (20b) | =0 | + +--------+------------------+------+ +``` + +##### 64 bits ``` 63 62 41 40 1 0 +--+---------------------+--------------------------------------+------+ - | | serverId | local connId |markup| + | | serverID | local connID |markup| |=0| (22b) | (40b) | =1 | +--+---------------------+--------------------------------------+------+ ``` -##### 32 bits version -(To be discussed in another RFC) -``` - 31 1 0 - +-----------------------------+------+ - | ??? |markup| - | ??? | =0 | - +-----------------------------+------+ -``` -#### 2. bit 63 -Bit 63 is always __ZERO__, making `connId` in range of non-negative int64, to be more friendly to exists codes, and some languages don't have primitive type `uint64`. +##### Determine 32 or 64 bits +The key factor is `serverID` (see `serverID` section for detail), which depends on number of TiDB instances in cluster. +- Choose 32 bits when number of TiDB instances __less than 2048__. Otherwise choose 64 bits. +- When 32 bits chosen, upgrade to 64 bits when: 1) Fail to acquire `serverID` _(because of occupied)_ continuously __more than 3 times__ _(which will happen when size of cluster is increasing rapidly)_; 2) All `local connID` in 32 bits `connID` are in used (see `local connID` section for detail). +- When 64 bits chosen, downgrade to 32 bits in a gradually manner when cluster scales down from big to small, as TiDB instances keep `serverID` until next restart or lost connection to PD. + +#### 2. Bit 63 +Bit 63 is always __ZERO__, making `connID` in range of non-negative int64, to be more friendly to exists codes, and some languages don't have primitive type `uint64`. -#### 3. markup +#### 3. Markup - `markup == 0` indicates that the `connID` is just 32 bits long effectively, and high 32 bits should be all zeros. Compatible with legacy 32 bits clients. - `markup == 1` indicates that the `connID` is 64 bits long. Incompatible with legacy 32 bits clients. - `markup == 1` while __high 32 bits are zeros__, indicates that 32 bits truncation happens. See `Compatibility` section. +#### 4. ServerID +- `serverID` is selected RANDOMLY from `serverIDs pool`_(see next)_ by each TiDB instance on startup, and the uniqueness is guaranteed by PD (etcd). `serverID` should be larger or equal to 1, to ensure that high 32 bits of `connID` is always non-zero, and make it possible to detect truncation. + +- `serverIDs pool` is: + - All __UNUSED__ `serverIDs` within [1, 2047] acquired from [`CLUSTER_INFO`](https://docs.pingcap.com/tidb/stable/information-schema-cluster-info#cluster_info) when 32 bits `connID` chosen. + - All `serverIDs` within [2048, 2^22-1] when 64 bits `connID` chosen. + +- On failure (e.g. fail connecting to PD, or all `serverID` are unavailable when 64 bits `connID` chosen), we block any new connection. +- `serverID` is kept by PD with a lease (defaults to 12 hours, long enough to avoid brutally killing any long-run SQL). If TiDB is disconnected to PD longer than half of the lease (defaults to 6 hours), all connections are killed, and no new connection is accepted, to avoid running with stale/incorrect `serverID`. On connection to PD restored, a new `serverID` is acquired before accepting new connection. +- On single TiDB instance without PD, a `serverID` of `1` is assigned. -#### 4. serverId -`serverId` is selected RANDOMLY by each TiDB instance on startup, and the uniqueness is guaranteed by PD(etcd). `serverId` should be larger or equal to 1, to ensure that high 32 bits of `connId` is always non-zero, and make it possible to detect truncation. +#### 5. Local connID +`local connID` is allocated by each TiDB instance on establishing connections: -On failure (e.g. fail connecting to PD, or all `serverId` are unavailable), we block any new connection. +- For 32 bits `connID`, `local connID` is possible to be integer-overflow and/or used up, especially on system being busy and/or with long running SQL. So we use a __lock-free queue__ to maintain available `local connID`, dequeue on client connecting, and enqueue on disconnecting. When `local connID` exhausted, upgrade to 64 bits. -`serverId` is kept by PD with a lease (defaults to 12 hours, long enough to avoid brutally killing any long-run SQL). If TiDB is disconnected to PD longer than half of the lease (defaults to 6 hours), all connections are killed, and no new connection is accepted, to avoid running with stale/incorrect `serverId`. On connection to PD restored, a new `serverId` is acquired before accepting new connection. +- For 64 bits `connID`, allocate `local connID` by __auto-increment__. Besides, flip to zero if integer-overflow, and check `local connID` existed or not by [Server.clients](https://github.com/pingcap/tidb/blob/7e1533392030514440d27ba98001c374cdf8808f/server/server.go#L122) for correctness with trivial cost, as the conflict is very unlikely to happen (It needs more than 3 years to use up 2^40 `local connID` in a 1w TPS instance). At last, return _"Too many connections"_ error if exhausted. -On single TiDB instance without PD, a `serverId` of `1` is assigned. +#### 6. Global kill +On processing `KILL x` command, first extract `serverID` from `x`. Then if `serverID` aims to a remote TiDB instance, get the address from [`CLUSTER_INFO`](https://docs.pingcap.com/tidb/stable/information-schema-cluster-info#cluster_info), and redirect the command to it by "Coprocessor API" provided by the remote TiDB, along with the original user authentication. -#### 5. local connId -`local connId` is allocated by each TiDB instance on establishing connections incrementally. +#### 7. Summary +| | 32 bits | 64 bits | +| ---- | ---- | ---- | +| ServerID pool size | 2^11 | 2^22 - 2^11 | +| ServerID allocation | Random of __Unused__ serverIDs acquired from PD within pool. Retry if unavailable. Upgrade to 64 bits while failed more than 3 times | Random of __All__ serverIDs within pool. Retry if unavailable | +| Local connID pool size | 2^20 | 2^40 | +| Local connID allocation | Using a queue to maintain and allocate available local connID. Upgrade to 64 bits while exhausted | Auto-increment within pool. Flip to zero when overflow. Return "Too many connections" if exhausted | -Integer overflow is ignored at this stage, as `local connId` should be long enough. -#### 6. global kill -On processing `KILL x` command, first extract `serverId` from `x`. Then if `serverId` aims to a remote TiDB instance, get the address from cluster info (see also [`CLUSTER_INFO`](https://docs.pingcap.com/tidb/stable/information-schema-cluster-info#cluster_info)), and redirect the command to it by "Coprocessor API" provided by the remote TiDB, along with the original user authentication. ## Compatibility -- Incompatible with legacy 32 bits clients. (According to some quick tests by now, MySQL client v8.0.19 supports `KILL` a connection with 64 bits `connId`, while `CTRL-C` does not, because it truncates the `connId` to 32 bits). A warning is set prompting that truncation happened, but user cannot see it, because `CTRL-C` is sent by a new connection in an instant. +- 32 bits `connID` is compatible to well-known clients. + +- 64 bits `connID` is __incompatible__ with legacy 32 bits clients. (According to some quick tests by now, MySQL client v8.0.19 supports `KILL` a connection with 64 bits `connID`, while `CTRL-C` does not, as it truncates the `connID` to 32 bits). A warning is set prompting that truncation happened, but user cannot see it, because `CTRL-C` is sent by a new connection in an instant. - [`KILL TIDB`](https://docs.pingcap.com/tidb/v4.0/sql-statement-kill) syntax and [`compatible-kill-query`](https://docs.pingcap.com/tidb/v4.0/tidb-configuration-file#compatible-kill-query) configuration item are deprecated. + +## Test Design + +### Prerequisite +Set `small_cluster_size_threshold` and `local_connid_pool_size` to small numbers (e.g. 4) by variable hacking, for easily switch between 32 and 64 bits `connID`. + +### Scenario A. 32 bits `connID` with small cluster +1. A TiDB without PD, killed by Ctrl+C, and killed by KILL. +2. One TiDB with PD, killed by Ctrl+C, and killed by KILL. +3. Multiple TiDB nodes, killed {local,remote} by {Ctrl-C,KILL}. + +### Scenario B. Upgrade from 32 to 64 bits `connID` +1. Upgrade caused by cluster scaled up from small to big. +2. Upgrade caused by `local connID` used up. + +### Scenario C. 64 bits `connID` with big cluster +1. Multiple TiDB nodes, killed {local,remote} by {Ctrl-C,KILL}. + +### Scenario D. Downgrade from 64 to 32 bits `connID` +1. Downgrade caused by cluster scaled down from big to small. + +### Scenario E. Fault tolerant while disconnected with PD +1. Existing connections are killed after PD lost connection for long time. +2. New connections are not accepted after PD lost connection for long time. +3. New connections are accepted after PD lost connection for long time and then recovered. +4. Connections can be killed after PD lost connection for long time and then recovered. diff --git a/docs/design/2021-03-09-dynamic-privileges.md b/docs/design/2021-03-09-dynamic-privileges.md index 48b324914ce80..7ad0d59d2c54e 100644 --- a/docs/design/2021-03-09-dynamic-privileges.md +++ b/docs/design/2021-03-09-dynamic-privileges.md @@ -1,7 +1,7 @@ # Proposal: - Author(s): [morgo](https://github.com/morgo) -- Last updated: April 12, 2021 +- Last updated: April 25, 2021 - Discussion at: N/A ## Table of Contents @@ -239,7 +239,7 @@ No change | Privilege Name | Description | Notes | | --------------- | --------------- | --------------- | | `RESTRICTED_SYSTEM_VARIABLES_ADMIN` | Allows changing a restricted `GLOBAL` system variable. | Currently in SEM all high risk variables are unloaded. TBD, it might be required in future that they are only visible/settable to those with this privilege and not SUPER. | -| `RESTRICTED_STATUS_VARIABLES_ADMIN` | Allows observing restricted status variables. | i.e. `SHOW GLOBAL STATUS` by default hides some statistics when `SEM` is enabled. | +| `RESTRICTED_STATUS_ADMIN` | Allows observing restricted status variables. | i.e. `SHOW GLOBAL STATUS` by default hides some statistics when `SEM` is enabled. | | `RESTRICTED_CONNECTION_ADMIN` | A special privilege to say that their connections, etc. can’t be killed by SUPER users AND they can kill connections by all other users. Affects `KILL`, `KILL TIDB` commands. | It is intended for the CloudAdmin user in DBaaS. | | `RESTRICTED_USER_ADMIN` | A special privilege to say that their access can’t be changed by `SUPER` users. Statements `DROP USER`, `SET PASSWORD`, `ALTER USER`, `REVOKE` are all limited. | It is intended for the CloudAdmin user in DbaaS. | | `RESTRICTED_TABLES_ADMIN` | A special privilege which means that the SEM hidden table semantic doesn’t apply. | It is intended for the CloudAdmin user in DbaaS. | diff --git a/docs/design/2021-03-09-security-enhanced-mode.md b/docs/design/2021-03-09-security-enhanced-mode.md index 98d30f3624557..e939fec67c154 100644 --- a/docs/design/2021-03-09-security-enhanced-mode.md +++ b/docs/design/2021-03-09-security-enhanced-mode.md @@ -1,7 +1,7 @@ # Proposal: - Author(s): [morgo](https://github.com/morgo) -- Last updated: April 12, 2021 +- Last updated: April 25, 2021 - Discussion at: N/A ## Table of Contents @@ -49,7 +49,7 @@ A boolean option called `EnableEnhancedSecurity` (default `FALSE`) will be added ### System Variables -The following system variables will be hidden: +The following system variables will be hidden unless the user has the `RESTRICTED_SYSTEM_VARIABLES_ADMIN` privilege: * variable.TiDBDDLSlowOprThreshold, * variable.TiDBAllowRemoveAutoInc, @@ -78,13 +78,13 @@ The following system variables will be reset to defaults: ### Status Variables -The following status variables will be hidden: +The following status variables will be hidden unless the user has the `RESTRICTED_STATUS_ADMIN` privilege: * tidb_gc_leader_desc ### Information Schema Tables -The following tables will be hidden: +The following tables will be hidden unless the user has the `RESTRICTED_TABLES_ADMIN` privilege: * cluster_config * cluster_hardware @@ -99,7 +99,7 @@ The following tables will be hidden: * metrics_tables * tidb_hot_regions -The following tables will be modified to hide columns: +The following tables will be modified to hide columns unless the user has the `RESTRICTED_TABLES_ADMIN` privilege: * tikv_store_status * The address, capacity, available, start_ts and uptime columns will return NULL. @@ -110,7 +110,7 @@ The following tables will be modified to hide columns: ### Performance Schema Tables -The following tables will be hidden: +The following tables will be hidden unless the user has the `RESTRICTED_TABLES_ADMIN` privilege: * pd_profile_allocs * pd_profile_block @@ -128,7 +128,7 @@ The following tables will be hidden: ### System (mysql) Tables -The following tables will be hidden: +The following tables will be hidden unless the user has the `RESTRICTED_TABLES_ADMIN` privilege: * expr_pushdown_blacklist * gc_delete_range @@ -137,11 +137,11 @@ The following tables will be hidden: * tidb * global_variables -The remaining system tables will be limited to read-only operations. +The remaining system tables will be limited to read-only operations and can not create new tables. ### Metrics Schema -All tables will be hidden, including the schema itself. +All tables will be hidden, including the schema itself unless the user has the `RESTRICTED_TABLES_ADMIN` privilege. ### Commands diff --git a/docs/design/2021-04-26-lock-view.md b/docs/design/2021-04-26-lock-view.md new file mode 100644 index 0000000000000..56d16e8e86194 --- /dev/null +++ b/docs/design/2021-04-26-lock-view.md @@ -0,0 +1,196 @@ +# TiDB Design Documents + +- Author(s): [longfangsong](https://github.com/longfangsong), [MyonKeminta](http://github.com/MyonKeminta) +- Last updated: May 6, 2021 +- Discussion PR: N/A +- Tracking Issue: https://github.com/pingcap/tidb/issues/24199 + +## Table of Contents + +* [Introduction](#introduction) +* [Motivation or Background](#motivation-or-background) +* [Detailed Design](#detailed-design) +* [Test Design](#test-design) + * [Functional Tests](#functional-tests) + * [Scenario Tests](#scenario-tests) + * [Compatibility Tests](#compatibility-tests) + * [Benchmark Tests](#benchmark-tests) +* [Impacts & Risks](#impacts--risks) +* [Investigation & Alternatives](#investigation--alternatives) +* [Unresolved Questions](#unresolved-questions) + +## Introduction + +This document describes the design of the feature Lock View, which provides tools to analyze problems about transaction's lock waiting, lock contentions and deadlocks. + +## Motivation or Background + +Currently, it's very hard to analyze lock contentions and deadlocks for transactions. One may need to enable general log, try to reproduce the problem, and try to analyze the log to find the cause, which is very difficult and inconvenient. We also found that this way of analyzing is not feasible in some scenarios. It's highly required to provide some better approach to analyze these kinds of problems. + +## Detailed Design + +Several tables will be provided in `information_schema`. Some tables has both local version (fetches data on the current TiDB node) and global version (fetches data among the whole cluster), and the global version's table name has the `"CLUSTER_"` prefix. + +### Table `(CLUSTER_)TIDB_TRX` + +| Field | Type | Comment | +|------------|------------|---------| +|`TRX_ID` | `unsigned bigint` | The transaction ID (aka. start ts) | +|`TRX_STARTED`|`time`| Human readable start time of the transaction | +|`DIGEST`|`text`| The digest of the current executing SQL statement | +|`SQLS` | `text` | A list of all executed SQL statements' digests | +|`STATE`| `enum('Running', 'Lock waiting', 'Committing', 'RollingBack')`| The state of the transaction | +| `WAITING_START_TIME` | `time` | The elapsed time since the start of the current lock waiting (if any) | +| `SCOPE` | `enum('Global', 'Local')` | The scope of the transaction | +| `ISOLATION_LEVEL` | `enum('RR', 'RC')` | | +| `AUTOCOMMIT` | `bool` | | +| `SESSION_ID` | `unsigned bigint` | | +| `USER` | `varchar` | | +| `DB` | `varchar` | | +| `SET_COUNT` | `int` | Modified keys of the current transaction | +| `LOCKED_COUNT` | `int` | Locked keys of the current transaction | +| `MEM_BUFFER_SIZE` | `int` | Size occupied by the transaction's membuffer | + +* Life span of rows: + * Create on first writing or locking operation in a transaction + * Remove after the transaction is done +* Collecting, storing and querying: + * All these information can be collected on TiDB side. Since the amount of concurrent transactions won't be too large, and it doesn't need to be persisted, so it's ok to implement it as a memory table. For querying among the cluster, just register the table under `infoschema/cluster.go` and write the global table name with the local one. + * As the simplest way of implementing, most information can be passed with a similar way like `ProcessInfo`, or even directly passed via the `ProcessInfo` struct. +* Permission: + * `PROCESS` privilege is needed to access the full content of this table. For users without `PROCESS` permission, only transactions started by the current user will be shown, and others will be filtered out, which is similar to the `processlist` table. + +### Table `DATA_LOCK_WAITS` + +| Field | Type | Comment | +|------------|------------|---------| +| `HASH` | `bigint` | The hash of the lock in TiKV's LockManager | +| `KEY` | `varchar` | The key that's being waiting on | +| `TRX_ID` | `unsigned bigint` | The current transaction that's waiting for the lock | +| `SQL_DIGEST` | `text` | The digest of the SQL that's trying to acquire the lock | +| `CURRENT_HOLDING_TRX_ID` | `unsigned bigint` | The transaction that's holding the lock and blocks the current transaction | + +* Life span of rows: + * Created on a lock come into LockManager + * Removed after a lock leave LockManager +* Collecting, storing and querying: + * All these will be collected on TiKV LockManager, and will need a new RPC entry for TiDB to query. LockManager won't store the un-hashed key or SQL_DIGEST for now, so we need to modify it. +* The SQL Digest of the transaction that's currently holding the lock may be helpful, but it's hard to implement under the current architecture. So it won't be included in the first version of the feature. +* Permission: + * `PROCESS` privilege is needed to access this table. + +### Table `(CLUSTER_)DEAD_LOCK` + +| Field | Type | Comment | +|------------|------------|---------| +| `DEADLOCK_ID` | `int` | There needs multiple rows to represent information of a single deadlock event. This field is used to distinguish different events. | +| `OCCUR_TIME` | `time` | The physical time when the deadlock occurs | +| `TRY_LOCK_TRX_ID` | `unsigned bigint` | The transaction ID (start ts) of the transaction that's trying to acquire the lock | +| `CURRENT_SQL_DIGEST` | `text` | The SQL that's being blocked | +| `KEY` | `varchar` | The key that's being locked, but locked by another transaction in the deadlock event | +| `SQLS` | `text` | A list of the digest of SQL statements that the transaction has executed | +| `TRX_HOLDING_LOCK` | `unsigned bigint` | The transaction that's currently holding the lock. There will be another record in the table with the same `DEADLOCK_ID` for that transaction. | + +* Life span of rows: + * Create after TiDB receive a deadlock error + * FIFO,clean the oldest after buffer is full +* Collecting, storing and querying: + * All of these information can be collected on TiDB side. It just need to add the information to the table when receives deadlock error from TiKV. The information of other transactions involved in the deadlock circle needed to be fetched from elsewhere (the `TIDB_TRX` table) when handling the deadlock error. + * Currently there are no much information in the deadlock error (it doesn't has the SQLs and keys' information), which needs to be improved. +* Permission: + * `PROCESS` privilege is needed to access this table. + +### Protocol + +To pass necessary information between TiDB and TiKV to make this feature possible, there needs some additional information carried in the protocol defined in kvproto. + +deadlockpb: + +```diff + message WaitForEntry { + ... ++ bytes key = ...; ++ bytes resource_group_tag = ...; + } + + message DeadlockResponse { + ... ++ repeated WaitiForEntry wait_chain = ...; + } +``` + +kvrpcpb: + +```diff + message Context { + ... ++ bytes resource_group_tag = ...; + } + + message Deadlock { + ... ++ repeated deadlock.WaitForEntry wait_chain = ...; + } + ++ message GetLockWaitInfoRequest { ++ Context context = 1; ++ } ++ ++ message GetLockWaitInfoResponse { ++ errorpb.Error region_error = 1; ++ string error = 2; ++ repeated deadlock.WaitForEntry entries = 3; ++ } +``` + +A field `resource_group_tag` will be added to `Context`. The SQL digest (and maybe more information) will be serialized and carried in this field. This field is expected to be reused by another feature named *Top SQL* which wants to carry SQL digest and plan to most transactional requests. + +A new KV RPC `GetLockWait` will be added to allow getting the lock waiting status from TiKV. This is a store-level (instead of region level) request, like `UnsafeDestroyRange`, and those Green GC related RPCs. The request can carry some filtering options to filter out those information the user don't care about. But the current memory table implementation only allow TiDB to scan the whole table and then filter it. This may need further optimization in the future. + +The locking key and `resource_group_tag` that comes from the `Context` of the pessimistic lock request is added to the deadlock detect request, and the wait chain is added to the deadlock detect response. + +The wait chain will be added to the `Deadlock` error which is returned by the `PessimisticLock` request, so that when deadlock happens, the full wait chain information can be passed to TiDB. + +## Compatibility + +This feature is not expected to be incompatible with other features. During upgrading, when there are different versions of TiDB nodes exists at the same time, it's possible that the `CLUSTER_` prefixed tables may encounter errors. But since this feature is typically used by user manually, this shouldn't be a severe problem. So we don't need to care much about that. + +## Test Design + +### Functional Tests + +* Querying the tables defined above gives correct result. + +### Scenario Tests + +* In a scenario where there's lock contention, this feature helps locating the problem. +* In a scenario where some a SQL is blocked by another transaction, this feature helps locating the problem. +* In a scenario where deadlock happens, this feature helps finding how the deadlock is formed. + +### Compatibility Tests + +- N/A + +### Benchmark Tests + +* The feature shouldn't cause any obvious performance regression (< 2%) on normal scenarios. +* Accessing these tables shouldn't increase latency of concurrent normal queries. + +## Impacts & Risks + +* To be investigated + +## Investigation & Alternatives + +* MySQL provides `data_locks` and `data_lock_waits` tables. +* Oracle provides `v$lock` view. +* CRDB provides `crdb_internal.node_transaction_statistics` that shows rich information for transactions. + +## Unresolved Questions + +* Since lock waiting on TiKV may timeout and retry, it's possible that in a single query to `DATA_LOCK_WAIT` table doesn't shows all (logical) lock waiting. +* Information about internal transactions may not be collected in our first version of implementation. +* Since TiDB need to query transaction information after it receives the deadlock error, the transactions' status may be changed during that time. As a result the information in `(CLUSTER_)DEAD_LOCK` table can't be promised to be accurate and complete. +* Statistics about transaction conflicts is still not enough. +* Historical information of `TIDB_TRX` and `DATA_LOCK_WAITS` is not kept, which possibly makes it still difficult to investigate some kind of problems. +* The SQL digest that's holding lock and blocking the current transaction is hard to retrieve and is not included in the current design. diff --git a/docs/tidb_http_api.md b/docs/tidb_http_api.md index d948146b58b83..220c9479aef10 100644 --- a/docs/tidb_http_api.md +++ b/docs/tidb_http_api.md @@ -289,6 +289,36 @@ timezone.* $curl http://127.0.0.1:10080/mvcc/index/test(p1)/t1/idx/1\?a\=A ``` + If the handle is clustered, also specify the primary key column values in the query string + + ```shell + $curl http://{TiDBIP}:10080/mvcc/index/{db}/{table}/{index}?${c1}={v1}&${c2}=${v2} + ``` + + ```shell + $curl http://127.0.0.1:10080/mvcc/index/test/t/idx\?a\=1.1\&b\=111\&c\=1 + { + "key": "74800000000000003B5F69800000000000000203800000000000000105BFF199999999999A013131310000000000FA", + "region_id": 59, + "value": { + "info": { + "writes": [ + { + "start_ts": 424752858505150464, + "commit_ts": 424752858506461184, + "short_value": "AH0B" + } + ], + "values": [ + { + "start_ts": 424752858505150464, + "value": "AH0B" + } + ] + } + } + } + 1. Scatter regions of the specified table, add a `scatter-range` scheduler for the PD and the range is same as the table range. ```shell diff --git a/errors.toml b/errors.toml index f09e33605293d..458af951629d8 100644 --- a/errors.toml +++ b/errors.toml @@ -261,6 +261,11 @@ error = ''' Duplicate partition name %-.192s ''' +["ddl:1562"] +error = ''' +Cannot create temporary table with partitions +''' + ["ddl:1563"] error = ''' Partition constant is out of partition function domain diff --git a/executor/adapter.go b/executor/adapter.go index 5a65a31e49c7d..5e5b7990f61d9 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -213,7 +213,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec if err != nil { return nil, err } - a.Ctx.GetSessionVars().StmtCtx.Priority = tikvstore.PriorityHigh + a.Ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityHigh // try to reuse point get executor if a.PsStmt.Executor != nil { @@ -730,15 +730,15 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { if stmtPri := stmtCtx.Priority; stmtPri == mysql.NoPriority { switch { case useMaxTS: - stmtCtx.Priority = tikvstore.PriorityHigh + stmtCtx.Priority = kv.PriorityHigh case a.LowerPriority: - stmtCtx.Priority = tikvstore.PriorityLow + stmtCtx.Priority = kv.PriorityLow } } } } if _, ok := a.Plan.(*plannercore.Analyze); ok && ctx.GetSessionVars().InRestrictedSQL { - ctx.GetSessionVars().StmtCtx.Priority = tikvstore.PriorityLow + ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityLow } b := newExecutorBuilder(ctx, a.InfoSchema) @@ -758,7 +758,7 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { a.isPreparedStmt = true a.Plan = executorExec.plan if executorExec.lowerPriority { - ctx.GetSessionVars().StmtCtx.Priority = tikvstore.PriorityLow + ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityLow } e = executorExec.stmtExec } diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 56a7a22f4bdd7..94820a028123d 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -1307,6 +1307,7 @@ func (s *testSuiteAgg) TestIssue20658(c *C) { for _, sql := range sqls { var expected [][]interface{} for _, con := range concurrencies { + comment := Commentf("sql: %s; concurrency: %d", sql, con) tk.MustExec(fmt.Sprintf("set @@tidb_streamagg_concurrency=%d;", con)) if con == 1 { expected = tk.MustQuery(sql).Sort().Rows() @@ -1320,16 +1321,20 @@ func (s *testSuiteAgg) TestIssue20658(c *C) { break } } - c.Assert(ok, Equals, true) + c.Assert(ok, Equals, true, comment) rows := tk.MustQuery(sql).Sort().Rows() - c.Assert(len(rows), Equals, len(expected)) + c.Assert(len(rows), Equals, len(expected), comment) for i := range rows { - v1, err := strconv.ParseFloat(rows[i][0].(string), 64) - c.Assert(err, IsNil) - v2, err := strconv.ParseFloat(expected[i][0].(string), 64) - c.Assert(err, IsNil) - c.Assert(math.Abs(v1-v2), Less, 1e-3) + rowStr, expStr := rows[i][0].(string), expected[i][0].(string) + if rowStr == "" && expStr == "" { + continue + } + v1, err := strconv.ParseFloat(rowStr, 64) + c.Assert(err, IsNil, comment) + v2, err := strconv.ParseFloat(expStr, 64) + c.Assert(err, IsNil, comment) + c.Assert(math.Abs(v1-v2), Less, 1e-3, comment) } } } diff --git a/executor/analyze.go b/executor/analyze.go index a08145aab0702..2d3187842845c 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -158,7 +158,12 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { globalStatsMap[globalStatsID] = globalStatsInfo{result.IsIndex, hg.ID, result.StatsVer} } } - err1 := statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, result.Cms[i], result.TopNs[i], result.Fms[i], result.StatsVer, 1) + var err1 error + if result.StatsVer == statistics.Version3 { + err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, nil, result.TopNs[i], result.Fms[i], result.StatsVer, 1) + } else { + err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, result.Cms[i], result.TopNs[i], result.Fms[i], result.StatsVer, 1) + } if err1 != nil { err = err1 logutil.Logger(ctx).Error("save stats to storage failed", zap.Error(err)) @@ -517,6 +522,33 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { } else { ranges = ranger.FullIntRange(false) } + if colExec.analyzeVer == statistics.Version3 { + count, hists, topns, fmSketches, err := colExec.buildSamplingStats(ranges) + if err != nil { + return []analyzeResult{{Err: err, job: colExec.job}} + } + cLen := len(colExec.analyzePB.ColReq.ColumnsInfo) + colResult := analyzeResult{ + TableID: colExec.tableID, + Hist: hists[:cLen], + TopNs: topns[:cLen], + Fms: fmSketches[:cLen], + job: colExec.job, + StatsVer: colExec.analyzeVer, + Count: count, + } + colGroupResult := analyzeResult{ + TableID: colExec.tableID, + Hist: hists[cLen:], + TopNs: topns[cLen:], + Fms: fmSketches[cLen:], + job: colExec.job, + StatsVer: colExec.analyzeVer, + Count: count, + IsIndex: 1, + } + return []analyzeResult{colResult, colGroupResult} + } collExtStats := colExec.ctx.GetSessionVars().EnableExtendedStats hists, cms, topNs, fms, extStats, err := colExec.buildStats(ranges, collExtStats) if err != nil { @@ -594,6 +626,7 @@ type AnalyzeColumnsExec struct { opts map[ast.AnalyzeOptionType]uint64 job *statistics.AnalyzeJob analyzeVer int + indexes []*model.IndexInfo } func (e *AnalyzeColumnsExec) open(ranges []*ranger.Range) error { @@ -639,6 +672,129 @@ func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectRe return result, nil } +func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range) ( + count int64, + hists []*statistics.Histogram, + topns []*statistics.TopN, + fmSketches []*statistics.FMSketch, + err error, +) { + if err = e.open(ranges); err != nil { + return 0, nil, nil, nil, err + } + defer func() { + if err1 := e.resultHandler.Close(); err1 != nil { + err = err1 + } + }() + l := len(e.analyzePB.ColReq.ColumnsInfo) + len(e.analyzePB.ColReq.ColumnGroups) + rootRowCollector := &statistics.RowSampleCollector{ + NullCount: make([]int64, l), + FMSketches: make([]*statistics.FMSketch, 0, l), + TotalSizes: make([]int64, l), + Samples: make(statistics.WeightedRowSampleHeap, 0, e.analyzePB.ColReq.SampleSize), + MaxSampleSize: int(e.analyzePB.ColReq.SampleSize), + } + for i := 0; i < l; i++ { + rootRowCollector.FMSketches = append(rootRowCollector.FMSketches, statistics.NewFMSketch(maxSketchSize)) + } + sc := e.ctx.GetSessionVars().StmtCtx + for { + data, err1 := e.resultHandler.nextRaw(context.TODO()) + if err1 != nil { + return 0, nil, nil, nil, err1 + } + if data == nil { + break + } + colResp := &tipb.AnalyzeColumnsResp{} + err = colResp.Unmarshal(data) + if err != nil { + return 0, nil, nil, nil, err + } + subCollector := &statistics.RowSampleCollector{ + MaxSampleSize: int(e.analyzePB.ColReq.SampleSize), + } + subCollector.FromProto(colResp.RowCollector) + e.job.Update(subCollector.Count) + rootRowCollector.MergeCollector(subCollector) + } + for _, sample := range rootRowCollector.Samples { + for i := range sample.Columns { + sample.Columns[i], err = tablecodec.DecodeColumnValue(sample.Columns[i].GetBytes(), &e.colsInfo[i].FieldType, sc.TimeZone) + if err != nil { + return 0, nil, nil, nil, err + } + if sample.Columns[i].Kind() == types.KindBytes { + sample.Columns[i].SetBytes(sample.Columns[i].GetBytes()) + } + } + } + hists = make([]*statistics.Histogram, 0, len(e.colsInfo)) + topns = make([]*statistics.TopN, 0, len(e.colsInfo)) + fmSketches = make([]*statistics.FMSketch, 0, len(e.colsInfo)) + for i, col := range e.colsInfo { + sampleItems := make([]*statistics.SampleItem, 0, rootRowCollector.MaxSampleSize) + for _, row := range rootRowCollector.Samples { + if row.Columns[i].IsNull() { + continue + } + sampleItems = append(sampleItems, &statistics.SampleItem{ + Value: row.Columns[i], + }) + } + collector := &statistics.SampleCollector{ + Samples: sampleItems, + NullCount: rootRowCollector.NullCount[i], + Count: rootRowCollector.Count - rootRowCollector.NullCount[i], + FMSketch: rootRowCollector.FMSketches[i], + TotalSize: rootRowCollector.TotalSizes[i], + } + hg, topn, err := statistics.BuildHistAndTopNOnRowSample(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), col.ID, collector, &col.FieldType, true) + if err != nil { + return 0, nil, nil, nil, err + } + hists = append(hists, hg) + topns = append(topns, topn) + fmSketches = append(fmSketches, rootRowCollector.FMSketches[i]) + } + colLen := len(e.colsInfo) + for i, idx := range e.indexes { + sampleItems := make([]*statistics.SampleItem, 0, rootRowCollector.MaxSampleSize) + for _, row := range rootRowCollector.Samples { + if len(idx.Columns) == 1 && row.Columns[idx.Columns[0].Offset].IsNull() { + continue + } + b := make([]byte, 0, 8) + for _, col := range idx.Columns { + b, err = codec.EncodeKey(e.ctx.GetSessionVars().StmtCtx, b, row.Columns[col.Offset]) + if err != nil { + return 0, nil, nil, nil, err + } + } + sampleItems = append(sampleItems, &statistics.SampleItem{ + Value: types.NewBytesDatum(b), + }) + } + collector := &statistics.SampleCollector{ + Samples: sampleItems, + NullCount: rootRowCollector.NullCount[colLen+i], + Count: rootRowCollector.Count - rootRowCollector.NullCount[colLen+i], + FMSketch: rootRowCollector.FMSketches[colLen+i], + TotalSize: rootRowCollector.TotalSizes[colLen+i], + } + hg, topn, err := statistics.BuildHistAndTopNOnRowSample(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), idx.ID, collector, types.NewFieldType(mysql.TypeBlob), false) + if err != nil { + return 0, nil, nil, nil, err + } + hists = append(hists, hg) + topns = append(topns, topn) + fmSketches = append(fmSketches, rootRowCollector.FMSketches[colLen+i]) + } + count = rootRowCollector.Count + return +} + func (e *AnalyzeColumnsExec) buildStats(ranges []*ranger.Range, needExtStats bool) (hists []*statistics.Histogram, cms []*statistics.CMSketch, topNs []*statistics.TopN, fms []*statistics.FMSketch, extStats *statistics.ExtendedStatsColl, err error) { if err = e.open(ranges); err != nil { return nil, nil, nil, nil, nil, err @@ -964,8 +1120,8 @@ func (e *AnalyzeFastExec) activateTxnForRowCount() (rollbackFn func() error, err return nil, errors.Trace(err) } } - txn.SetOption(tikvstore.Priority, tikvstore.PriorityLow) - txn.SetOption(tikvstore.IsolationLevel, tikvstore.RC) + txn.SetOption(tikvstore.Priority, kv.PriorityLow) + txn.SetOption(tikvstore.IsolationLevel, kv.RC) txn.SetOption(tikvstore.NotFillCache, true) return rollbackFn, nil } @@ -1185,8 +1341,8 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { defer e.wg.Done() snapshot := e.ctx.GetStore().GetSnapshot(kv.MaxVersion) snapshot.SetOption(tikvstore.NotFillCache, true) - snapshot.SetOption(tikvstore.IsolationLevel, tikvstore.RC) - snapshot.SetOption(tikvstore.Priority, tikvstore.PriorityLow) + snapshot.SetOption(tikvstore.IsolationLevel, kv.RC) + snapshot.SetOption(tikvstore.Priority, kv.PriorityLow) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { snapshot.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) } @@ -1331,7 +1487,9 @@ func (e *AnalyzeFastExec) runTasks() ([]*statistics.Histogram, []*statistics.CMS // Build collector properties. collector := e.collectors[i] collector.Samples = collector.Samples[:e.sampCursor] - sort.Slice(collector.Samples, func(i, j int) bool { return collector.Samples[i].Handle.Compare(collector.Samples[j].Handle) < 0 }) + sort.Slice(collector.Samples, func(i, j int) bool { + return collector.Samples[i].Handle.Compare(collector.Samples[j].Handle) < 0 + }) collector.CalcTotalSize() // Adjust the row count in case the count of `tblStats` is not accurate and too small. rowCount = mathutil.MaxInt64(rowCount, int64(len(collector.Samples))) diff --git a/executor/builder.go b/executor/builder.go index 51971b5f8e0a9..e82db0d6aaccc 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -953,19 +953,6 @@ func (b *executorBuilder) buildUnionScanExec(v *plannercore.PhysicalUnionScan) E // Note that this function may be called by inner workers of index lookup join concurrently. // Be careful to avoid data race. func (b *executorBuilder) buildUnionScanFromReader(reader Executor, v *plannercore.PhysicalUnionScan) Executor { - // Adjust UnionScan->PartitionTable->Reader - // to PartitionTable->UnionScan->Reader - // The build of UnionScan executor is delay to the nextPartition() function - // because the Reader executor is available there. - if x, ok := reader.(*PartitionTableExecutor); ok { - nextPartitionForReader := x.nextPartition - x.nextPartition = nextPartitionForUnionScan{ - b: b, - us: v, - child: nextPartitionForReader, - } - return x - } // If reader is union, it means a partition table and we should transfer as above. if x, ok := reader.(*UnionExec); ok { for i, child := range x.children { @@ -2040,7 +2027,65 @@ func (b *executorBuilder) buildAnalyzeIndexIncremental(task plannercore.AnalyzeI return analyzeTask } +func (b *executorBuilder) buildAnalyzeSamplingPushdown( + task plannercore.AnalyzeColumnsTask, + opts map[ast.AnalyzeOptionType]uint64, + autoAnalyze string, +) *analyzeTask { + availableIdx := make([]*model.IndexInfo, 0, len(task.Indexes)) + colGroups := make([]*tipb.AnalyzeColumnGroup, 0, len(task.Indexes)) + if len(task.Indexes) > 0 { + for _, idx := range task.Indexes { + availableIdx = append(availableIdx, idx) + colGroup := &tipb.AnalyzeColumnGroup{ + ColumnOffsets: make([]int64, 0, len(idx.Columns)), + } + for _, col := range idx.Columns { + colGroup.ColumnOffsets = append(colGroup.ColumnOffsets, int64(col.Offset)) + } + colGroups = append(colGroups, colGroup) + } + } + + _, offset := timeutil.Zone(b.ctx.GetSessionVars().Location()) + sc := b.ctx.GetSessionVars().StmtCtx + e := &AnalyzeColumnsExec{ + ctx: b.ctx, + tableID: task.TableID, + colsInfo: task.ColsInfo, + handleCols: task.HandleCols, + concurrency: b.ctx.GetSessionVars().DistSQLScanConcurrency(), + analyzePB: &tipb.AnalyzeReq{ + Tp: tipb.AnalyzeType_TypeFullSampling, + Flags: sc.PushDownFlags(), + TimeZoneOffset: offset, + }, + opts: opts, + analyzeVer: task.StatsVersion, + indexes: availableIdx, + } + e.analyzePB.ColReq = &tipb.AnalyzeColumnsReq{ + BucketSize: int64(opts[ast.AnalyzeOptNumBuckets]), + SampleSize: int64(opts[ast.AnalyzeOptNumSamples]), + SketchSize: maxSketchSize, + ColumnsInfo: util.ColumnsToProto(task.ColsInfo, task.TblInfo.PKIsHandle), + ColumnGroups: colGroups, + } + if task.TblInfo != nil { + e.analyzePB.ColReq.PrimaryColumnIds = tables.TryGetCommonPkColumnIds(task.TblInfo) + if task.TblInfo.IsCommonHandle { + e.analyzePB.ColReq.PrimaryPrefixColumnIds = tables.PrimaryPrefixColumnIDs(task.TblInfo) + } + } + b.err = plannercore.SetPBColumnsDefaultValue(b.ctx, e.analyzePB.ColReq.ColumnsInfo, task.ColsInfo) + job := &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze table"} + return &analyzeTask{taskType: colTask, colExec: e, job: job} +} + func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeColumnsTask, opts map[ast.AnalyzeOptionType]uint64, autoAnalyze string) *analyzeTask { + if task.StatsVersion == statistics.Version3 { + return b.buildAnalyzeSamplingPushdown(task, opts, autoAnalyze) + } cols := task.ColsInfo if hasPkHist(task.HandleCols) { colInfo := task.TblInfo.Columns[task.HandleCols.GetCol(0).Index] @@ -2686,31 +2731,6 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E } if v.StoreType == kv.TiFlash { sctx.IsTiFlash.Store(true) - partsExecutor := make([]Executor, 0, len(partitions)) - for _, part := range partitions { - exec, err := buildNoRangeTableReader(b, v) - if err != nil { - b.err = err - return nil - } - exec.ranges = ts.Ranges - nexec, err := nextPartitionForTableReader{exec: exec}.nextPartition(context.Background(), part) - if err != nil { - b.err = err - return nil - } - partsExecutor = append(partsExecutor, nexec) - } - if len(partsExecutor) == 0 { - return &TableDualExec{baseExecutor: *ret.base()} - } - if len(partsExecutor) == 1 { - return partsExecutor[0] - } - return &UnionExec{ - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID(), partsExecutor...), - concurrency: b.ctx.GetSessionVars().UnionConcurrency(), - } } if len(partitions) == 0 { diff --git a/executor/distsql_test.go b/executor/distsql_test.go index 2a3de537c748a..eca6e1d016b40 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -360,7 +360,7 @@ func (s *testSuite3) TestPartitionTableIndexJoinIndexLookUp(c *C) { tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec(`create table t (a int, b int, key(a)) partition by hash(a) partitions 4`) tk.MustExec("create table tnormal (a int, b int, key(a), key(b))") - nRows := 64 + nRows := 512 values := make([]string, 0, nRows) for i := 0; i < nRows; i++ { values = append(values, fmt.Sprintf("(%v, %v)", rand.Intn(nRows), rand.Intn(nRows))) diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index 6b42c0f63cf6f..9515907b8eba6 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -344,7 +344,7 @@ func (s *testSuite5) TestPartitionTableIndexJoinAndIndexReader(c *C) { tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec(`create table t (a int, b int, key(a)) partition by hash(a) partitions 4`) tk.MustExec("create table tnormal (a int, b int, key(a), key(b))") - nRows := 64 + nRows := 512 values := make([]string, 0, nRows) for i := 0; i < nRows; i++ { values = append(values, fmt.Sprintf("(%v, %v)", rand.Intn(nRows), rand.Intn(nRows))) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 33db7c9fdd18b..368d8838a777e 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -53,6 +53,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/pdapi" + "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" @@ -132,7 +133,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableSessionVar: err = e.setDataFromSessionVar(sctx) case infoschema.TableTiDBServersInfo: - err = e.setDataForServersInfo() + err = e.setDataForServersInfo(sctx) case infoschema.TableTiFlashReplica: e.dataForTableTiFlashReplica(sctx, dbs) case infoschema.TableTiKVStoreStatus: @@ -978,6 +979,18 @@ func (e *memtableRetriever) dataForTiKVStoreStatus(ctx sessionctx.Context) (err lastHeartbeatTs := types.NewTime(types.FromGoTime(storeStat.Status.LastHeartbeatTs), mysql.TypeDatetime, types.DefaultFsp) row[17].SetMysqlTime(lastHeartbeatTs) row[18].SetString(storeStat.Status.Uptime, mysql.DefaultCollationName) + if sem.IsEnabled() { + // Patch out IP addresses etc if the user does not have the RESTRICTED_TABLES_ADMIN privilege + checker := privilege.GetPrivilegeManager(ctx) + if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) { + row[1].SetString(strconv.FormatInt(storeStat.Store.ID, 10), mysql.DefaultCollationName) + row[1].SetNull() + row[6].SetNull() + row[7].SetNull() + row[16].SetNull() + row[18].SetNull() + } + } e.rows = append(e.rows, row) } return nil @@ -1120,6 +1133,15 @@ func (e *memtableRetriever) dataForTiDBClusterInfo(ctx sessionctx.Context) error upTimeStr, server.ServerID, ) + if sem.IsEnabled() { + checker := privilege.GetPrivilegeManager(ctx) + if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) { + row[1].SetString(strconv.FormatUint(server.ServerID, 10), mysql.DefaultCollationName) + row[2].SetNull() + row[5].SetNull() + row[6].SetNull() + } + } rows = append(rows, row) } e.rows = rows @@ -1143,7 +1165,7 @@ func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx sessionctx.Context, sc func (e *memtableRetriever) setDataForClusterProcessList(ctx sessionctx.Context) error { e.setDataForProcessList(ctx) - rows, err := infoschema.AppendHostInfoToRows(e.rows) + rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows) if err != nil { return err } @@ -1729,7 +1751,7 @@ func (e *memtableRetriever) setDataForPseudoProfiling(sctx sessionctx.Context) { } } -func (e *memtableRetriever) setDataForServersInfo() error { +func (e *memtableRetriever) setDataForServersInfo(ctx sessionctx.Context) error { serversInfo, err := infosync.GetAllServerInfo(context.Background()) if err != nil { return err @@ -1747,6 +1769,12 @@ func (e *memtableRetriever) setDataForServersInfo() error { info.BinlogStatus, // BINLOG_STATUS stringutil.BuildStringFromLabels(info.Labels), // LABELS ) + if sem.IsEnabled() { + checker := privilege.GetPrivilegeManager(ctx) + if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) { + row[1].SetNull() // clear IP + } + } rows = append(rows, row) } e.rows = rows @@ -1844,7 +1872,7 @@ func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context, switch tableName { case infoschema.ClusterTableStatementsSummary, infoschema.ClusterTableStatementsSummaryHistory: - rows, err := infoschema.AppendHostInfoToRows(e.rows) + rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows) if err != nil { return err } diff --git a/executor/partition_table.go b/executor/partition_table.go index 41dbe64ba7972..e888332fee396 100644 --- a/executor/partition_table.go +++ b/executor/partition_table.go @@ -17,102 +17,10 @@ import ( "context" "fmt" - "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" - plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tipb/go-tipb" ) -// PartitionTableExecutor is a Executor for partitioned table. -// It works by wrap the underlying TableReader/IndexReader/IndexLookUpReader. -type PartitionTableExecutor struct { - baseExecutor - - nextPartition - partitions []table.PhysicalTable - cursor int - curr Executor -} - -type nextPartition interface { - nextPartition(context.Context, table.PhysicalTable) (Executor, error) -} - -// nolint:structcheck -type innerPartitionInfo struct { - isFullPartition bool -} - -type nextPartitionForTableReader struct { - *innerPartitionInfo - rangeBuilders map[int64]kvRangeBuilder - exec *TableReaderExecutor -} - -func (n nextPartitionForTableReader) GetInnerPartitionInfo() *innerPartitionInfo { - return n.innerPartitionInfo -} - -func (n nextPartitionForTableReader) nextPartition(ctx context.Context, tbl table.PhysicalTable) (Executor, error) { - n.exec.table = tbl - n.exec.kvRanges = n.exec.kvRanges[:0] - if n.innerPartitionInfo != nil && !n.isFullPartition { - n.exec.kvRangeBuilder = n.rangeBuilders[tbl.GetPhysicalID()] - } - if err := updateDAGRequestTableID(ctx, n.exec.dagPB, tbl.GetPhysicalID()); err != nil { - return nil, err - } - return n.exec, nil -} - -type nextPartitionForUnionScan struct { - b *executorBuilder - us *plannercore.PhysicalUnionScan - child nextPartition -} - -// nextPartition implements the nextPartition interface. -// For union scan on partitioned table, the executor should be PartitionTable->UnionScan->TableReader rather than -// UnionScan->PartitionTable->TableReader -func (n nextPartitionForUnionScan) nextPartition(ctx context.Context, tbl table.PhysicalTable) (Executor, error) { - childExec, err := n.child.nextPartition(ctx, tbl) - if err != nil { - return nil, err - } - - n.b.err = nil - ret := n.b.buildUnionScanFromReader(childExec, n.us) - return ret, n.b.err -} - -func nextPartitionWithTrace(ctx context.Context, n nextPartition, tbl table.PhysicalTable) (Executor, error) { - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan(fmt.Sprintf("nextPartition %d", tbl.GetPhysicalID()), opentracing.ChildOf(span.Context())) - defer span1.Finish() - ctx = opentracing.ContextWithSpan(ctx, span1) - } - return n.nextPartition(ctx, tbl) -} - -// updateDAGRequestTableID update the table ID in the DAG request to partition ID. -// TiKV only use that table ID for log, but TiFlash use it. -func updateDAGRequestTableID(ctx context.Context, dag *tipb.DAGRequest, partitionID int64) error { - // TiFlash set RootExecutor field and ignore Executors field. - if dag.RootExecutor != nil { - return updateExecutorTableID(ctx, dag.RootExecutor, partitionID, true) - } - for i := 0; i < len(dag.Executors); i++ { - exec := dag.Executors[i] - err := updateExecutorTableID(ctx, exec, partitionID, false) - if err != nil { - return err - } - } - return nil -} - func updateExecutorTableID(ctx context.Context, exec *tipb.Executor, partitionID int64, recursive bool) error { var child *tipb.Executor switch exec.Tp { @@ -149,55 +57,3 @@ func updateExecutorTableID(ctx context.Context, exec *tipb.Executor, partitionID } return nil } - -// Open implements the Executor interface. -func (e *PartitionTableExecutor) Open(ctx context.Context) error { - e.cursor = 0 - e.curr = nil - return nil -} - -// Next implements the Executor interface. -func (e *PartitionTableExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { - chk.Reset() - var err error - for e.cursor < len(e.partitions) { - if e.curr == nil { - n := e.nextPartition - e.curr, err = nextPartitionWithTrace(ctx, n, e.partitions[e.cursor]) - if err != nil { - return err - } - if err := e.curr.Open(ctx); err != nil { - return err - } - } - - err = Next(ctx, e.curr, chk) - if err != nil { - return err - } - - if chk.NumRows() > 0 { - break - } - - err = e.curr.Close() - if err != nil { - return err - } - e.curr = nil - e.cursor++ - } - return nil -} - -// Close implements the Executor interface. -func (e *PartitionTableExecutor) Close() error { - var err error - if e.curr != nil { - err = e.curr.Close() - e.curr = nil - } - return err -} diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index b1ff32e15f729..ee7d807ef4e8f 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -14,10 +14,15 @@ package executor_test import ( + "fmt" + "math/rand" + "strings" + . "github.com/pingcap/check" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" ) @@ -222,6 +227,84 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows()) } +func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_global_stats") + tk.MustExec("use test_global_stats") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash and range and list partition + tk.MustExec("create table thash(a int, b int, key(a)) partition by hash(a) partitions 4") + tk.MustExec(`create table trange(a int, b int, key(a)) partition by range(a) ( + partition p0 values less than (200), + partition p1 values less than (400), + partition p2 values less than (600), + partition p3 values less than (800), + partition p4 values less than (1001))`) + tk.MustExec(`create table tlist(a int, b int, key(a)) partition by list (a) ( + partition p0 values in (0, 1, 2, 3, 4, 5, 6, 7, 8, 9), + partition p0 values in (10, 11, 12, 13, 14, 15, 16, 17, 18, 19), + partition p0 values in (20, 21, 22, 23, 24, 25, 26, 27, 28, 29), + partition p0 values in (30, 31, 32, 33, 34, 35, 36, 37, 38, 39), + partition p0 values in (40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50))`) + + // construct some special data distribution + vals := make([]string, 0, 1000) + listVals := make([]string, 0, 1000) + for i := 0; i < 1000; i++ { + if i < 10 { + // for hash and range partition, 1% of records are in [0, 100) + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(100), rand.Intn(100))) + // for list partition, 1% of records are equal to 0 + listVals = append(listVals, "(0, 0)") + } else { + vals = append(vals, fmt.Sprintf("(%v, %v)", 100+rand.Intn(900), 100+rand.Intn(900))) + listVals = append(listVals, fmt.Sprintf("(%v, %v)", 1+rand.Intn(50), 1+rand.Intn(50))) + } + } + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into tlist values " + strings.Join(listVals, ",")) + + // before analyzing, the planner will choose TableScan to access the 1% of records + c.Assert(tk.HasPlan("select * from thash where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from trange where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from tlist where a<1", "TableFullScan"), IsTrue) + + tk.MustExec("analyze table thash") + tk.MustExec("analyze table trange") + tk.MustExec("analyze table tlist") + + // after analyzing, the planner will use the Index(a) + tk.MustIndexLookup("select * from thash where a<100") + tk.MustIndexLookup("select * from trange where a<100") + tk.MustIndexLookup("select * from tlist where a<1") + + // create SQL bindings + tk.MustExec("create session binding for select * from thash where a<100 using select * from thash ignore index(a) where a<100") + tk.MustExec("create session binding for select * from trange where a<100 using select * from trange ignore index(a) where a<100") + tk.MustExec("create session binding for select * from tlist where a<100 using select * from tlist ignore index(a) where a<100") + + // use TableScan again since the Index(a) is ignored + c.Assert(tk.HasPlan("select * from thash where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from trange where a<100", "TableFullScan"), IsTrue) + c.Assert(tk.HasPlan("select * from tlist where a<1", "TableFullScan"), IsTrue) + + // drop SQL bindings + tk.MustExec("drop session binding for select * from thash where a<100") + tk.MustExec("drop session binding for select * from trange where a<100") + tk.MustExec("drop session binding for select * from tlist where a<100") + + // use Index(a) again + tk.MustIndexLookup("select * from thash where a<100") + tk.MustIndexLookup("select * from trange where a<100") + tk.MustIndexLookup("select * from tlist where a<1") +} + func (s *globalIndexSuite) TestGlobalIndexScan(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists p") diff --git a/executor/point_get.go b/executor/point_get.go index b4ec8d13bfe72..241f52d421344 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -391,7 +391,7 @@ func (e *PointGetExecutor) get(ctx context.Context, key kv.Key) ([]byte, error) } func (e *PointGetExecutor) verifyTxnScope() error { - txnScope := e.txn.GetUnionStore().GetOption(tikvstore.TxnScope).(string) + txnScope := e.txn.GetOption(tikvstore.TxnScope).(string) if txnScope == "" || txnScope == oracle.GlobalTxnScope { return nil } diff --git a/executor/point_get_test.go b/executor/point_get_test.go index 54bce074dfb22..846b6f1628fe1 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -27,9 +27,9 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" - tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" @@ -536,15 +536,15 @@ func (s *testPointGetSuite) TestSelectCheckVisibility(c *C) { c.Assert(expectErr.Equal(err), IsTrue) } // Test point get. - checkSelectResultError("select * from t where a='1'", tikverr.ErrGCTooEarly) + checkSelectResultError("select * from t where a='1'", txndriver.ErrGCTooEarly) // Test batch point get. - checkSelectResultError("select * from t where a in ('1','2')", tikverr.ErrGCTooEarly) + checkSelectResultError("select * from t where a in ('1','2')", txndriver.ErrGCTooEarly) // Test Index look up read. - checkSelectResultError("select * from t where b > 0 ", tikverr.ErrGCTooEarly) + checkSelectResultError("select * from t where b > 0 ", txndriver.ErrGCTooEarly) // Test Index read. - checkSelectResultError("select b from t where b > 0 ", tikverr.ErrGCTooEarly) + checkSelectResultError("select b from t where b > 0 ", txndriver.ErrGCTooEarly) // Test table read. - checkSelectResultError("select * from t", tikverr.ErrGCTooEarly) + checkSelectResultError("select * from t", txndriver.ErrGCTooEarly) } func (s *testPointGetSuite) TestReturnValues(c *C) { diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 7b38a6f6b673e..061e09dcc1315 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -967,7 +967,7 @@ func (s *seqTestSuite) TestBatchInsertDelete(c *C) { atomic.StoreUint64(&kv.TxnTotalSizeLimit, originLimit) }() // Set the limitation to a small value, make it easier to reach the limitation. - atomic.StoreUint64(&kv.TxnTotalSizeLimit, 5000) + atomic.StoreUint64(&kv.TxnTotalSizeLimit, 5500) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/set.go b/executor/set.go index 83dc63d0e9762..1223f973f5e39 100644 --- a/executor/set.go +++ b/executor/set.go @@ -94,44 +94,22 @@ func (e *SetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { continue } - syns := e.getSynonyms(name) - // Set system variable - for _, n := range syns { - err := e.setSysVariable(n, v) - if err != nil { - return err - } + if err := e.setSysVariable(name, v); err != nil { + return err } } return nil } -func (e *SetExecutor) getSynonyms(varName string) []string { - synonyms, ok := variable.SynonymsSysVariables[varName] - if ok { - return synonyms - } - - synonyms = []string{varName} - return synonyms -} - func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) error { sessionVars := e.ctx.GetSessionVars() sysVar := variable.GetSysVar(name) if sysVar == nil { return variable.ErrUnknownSystemVar.GenWithStackByArgs(name) } - if sysVar.Scope == variable.ScopeNone { - return errors.Errorf("Variable '%s' is a read only variable", name) - } var valStr string var err error if v.IsGlobal { - // Set global scope system variable. - if sysVar.Scope&variable.ScopeGlobal == 0 { - return errors.Errorf("Variable '%s' is a SESSION variable and can't be used with SET GLOBAL", name) - } valStr, err = e.getVarValue(v, sysVar) if err != nil { return err @@ -152,10 +130,6 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e } logutil.BgLogger().Info("set global var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) } else { - // Set session scope system variable. - if sysVar.Scope&variable.ScopeSession == 0 { - return errors.Errorf("Variable '%s' is a GLOBAL variable and should be set with SET GLOBAL", name) - } valStr, err = e.getVarValue(v, nil) if err != nil { return err @@ -186,6 +160,10 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e logutil.BgLogger().Debug("set session var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) } + // These are server instance scoped variables, and have special semantics. + // i.e. after SET SESSION, other users sessions will reflect the new value. + // TODO: in future these could be better managed as a post-set hook. + valStrToBoolStr := variable.BoolToOnOff(variable.TiDBOptOn(valStr)) switch name { diff --git a/executor/show.go b/executor/show.go index 0194305f8b913..725c0e0e28ec2 100644 --- a/executor/show.go +++ b/executor/show.go @@ -61,6 +61,7 @@ import ( "github.com/pingcap/tidb/util/format" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/hint" + "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stringutil" @@ -707,10 +708,17 @@ func (e *ShowExec) fetchShowStatus() error { if err != nil { return errors.Trace(err) } + checker := privilege.GetPrivilegeManager(e.ctx) for status, v := range statusVars { if e.GlobalScope && v.Scope == variable.ScopeSession { continue } + // Skip invisible status vars if permission fails. + if sem.IsEnabled() && sem.IsInvisibleStatusVar(status) { + if checker == nil || !checker.RequestDynamicVerification(sessionVars.ActiveRoles, "RESTRICTED_STATUS_ADMIN", false) { + continue + } + } switch v.Value.(type) { case []interface{}, nil: v.Value = fmt.Sprintf("%v", v.Value) diff --git a/executor/slow_query.go b/executor/slow_query.go index c6a5f8872f0e3..9e32cb175e4b8 100755 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -74,7 +74,7 @@ func (e *slowQueryRetriever) retrieve(ctx context.Context, sctx sessionctx.Conte } e.initializeAsyncParsing(ctx, sctx) } - rows, retrieved, err := e.dataForSlowLog(ctx) + rows, retrieved, err := e.dataForSlowLog(ctx, sctx) if err != nil { return nil, err } @@ -193,7 +193,7 @@ func (e *slowQueryRetriever) parseDataForSlowLog(ctx context.Context, sctx sessi e.parseSlowLog(ctx, sctx, reader, ParseSlowLogBatchSize) } -func (e *slowQueryRetriever) dataForSlowLog(ctx context.Context) ([][]types.Datum, bool, error) { +func (e *slowQueryRetriever) dataForSlowLog(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, bool, error) { var ( task slowLogTask ok bool @@ -216,7 +216,7 @@ func (e *slowQueryRetriever) dataForSlowLog(ctx context.Context) ([][]types.Datu continue } if e.table.Name.L == strings.ToLower(infoschema.ClusterTableSlowLog) { - rows, err := infoschema.AppendHostInfoToRows(rows) + rows, err := infoschema.AppendHostInfoToRows(sctx, rows) return rows, false, err } return rows, false, nil diff --git a/executor/window_test.go b/executor/window_test.go index 219113cf38c0c..e530aa03f0549 100644 --- a/executor/window_test.go +++ b/executor/window_test.go @@ -402,3 +402,31 @@ func baseTestSlidingWindowFunctions(tk *testkit.TestKit) { result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") result.Check(testkit.Rows(" 11", " 11", "M 5", "F 5", "F 4", "F 3", "M 2")) } + +func (s *testSuite7) TestIssue24264(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists tbl_2") + tk.MustExec("create table tbl_2 ( col_10 char(65) collate utf8mb4_unicode_ci not null , col_11 bigint not null , col_12 datetime not null , col_13 bigint unsigned default 327695751717730004 , col_14 timestamp default '2010-11-18' not null , primary key idx_5 ( col_11,col_13 ) /*T![clustered_index] clustered */ , unique key idx_6 ( col_10,col_11,col_13 ) , unique key idx_7 ( col_14,col_12,col_13 ) )") + tk.MustExec("insert into tbl_2 values ( 'RmF',-5353757041350034197,'1996-01-22',1866803697729291364,'1996-09-11' )") + tk.MustExec("insert into tbl_2 values ( 'xEOGaB',-6602924241498980347,'2019-02-22',8297270320597030697,'1972-04-04' )") + tk.MustExec("insert into tbl_2 values ( 'dvUztqgTPAhLdzgEsV',3316448219481769821,'2034-09-12',937089564901142512,'2030-12-04' )") + tk.MustExec("insert into tbl_2 values ( 'mNoyfbT',-6027094365061219400,'2035-10-10',1752804734961508175,'1992-08-09' )") + tk.MustExec("insert into tbl_2 values ( 'BDPJMhLYXuKB',6823702503458376955,'2015-04-09',737914379167848827,'2026-04-29' )") + tk.MustExec("insert into tbl_2 values ( 'WPiaVfPstGohvHd',1308183537252932688,'2020-05-03',5364104746649397703,'1979-01-28' )") + tk.MustExec("insert into tbl_2 values ( 'lrm',4642935044097656317,'1973-04-29',149081313305673035,'2013-02-03' )") + tk.MustExec("insert into tbl_2 values ( '',-7361040853169906422,'2024-10-22',6308270832310351889,'1981-02-01' )") + tk.MustExec("insert into tbl_2 values ( 'uDANahGcLwpSssabD',2235074865448210231,'1992-10-10',7140606140672586593,'1992-11-25' )") + tk.MustExec("insert into tbl_2 values ( 'TDH',-1911014243756021618,'2013-01-26',2022218243939205750,'1982-04-04' )") + tk.MustQuery("select lead(col_13,1,NULL) over w from tbl_2 window w as (order by col_13)").Check(testkit.Rows( + "737914379167848827", + "937089564901142512", + "1752804734961508175", + "1866803697729291364", + "2022218243939205750", + "5364104746649397703", + "6308270832310351889", + "7140606140672586593", + "8297270320597030697", + "")) +} diff --git a/expression/aggregation/base_func.go b/expression/aggregation/base_func.go index 6abd5e8cfc0aa..9a5ef95d49bb2 100644 --- a/expression/aggregation/base_func.go +++ b/expression/aggregation/base_func.go @@ -411,6 +411,9 @@ func (a *baseFuncDesc) WrapCastForAggArgs(ctx sessionctx.Context) { if i == 1 && (a.Name == ast.WindowFuncLead || a.Name == ast.WindowFuncLag || a.Name == ast.WindowFuncNthValue) { continue } + if a.Args[i].GetType().Tp == mysql.TypeNull { + continue + } a.Args[i] = castFunc(ctx, a.Args[i]) if a.Name != ast.AggFuncAvg && a.Name != ast.AggFuncSum { continue diff --git a/expression/integration_test.go b/expression/integration_test.go index b2484db6279cf..0f9d442b8de39 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -19,6 +19,7 @@ import ( "encoding/hex" "fmt" "math" + "math/rand" "sort" "strconv" "strings" @@ -9371,3 +9372,51 @@ func (s *testIntegrationSuite) TestRefineArgNullValues(c *C) { "", )) } + +func (s *testIntegrationSuite) TestEnumIndex(c *C) { + defer s.cleanEnv(c) + + elems := []string{"\"a\"", "\"b\"", "\"c\""} + rand.Shuffle(len(elems), func(i, j int) { + elems[i], elems[j] = elems[j], elems[i] + }) + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t,tidx") + tk.MustExec("create table t(e enum(" + strings.Join(elems, ",") + "))") + tk.MustExec("create table tidx(e enum(" + strings.Join(elems, ",") + "), index idx(e))") + + nRows := 50 + values := make([]string, 0, nRows) + for i := 0; i < nRows; i++ { + values = append(values, fmt.Sprintf("(%v)", rand.Intn(len(elems))+1)) + } + tk.MustExec(fmt.Sprintf("insert into t values %v", strings.Join(values, ", "))) + tk.MustExec(fmt.Sprintf("insert into tidx values %v", strings.Join(values, ", "))) + + ops := []string{"=", "!=", ">", ">=", "<", "<="} + testElems := []string{"\"a\"", "\"b\"", "\"c\"", "\"d\"", "\"\"", "1", "2", "3", "4", "0", "-1"} + for i := 0; i < nRows; i++ { + cond := fmt.Sprintf("e" + ops[rand.Intn(len(ops))] + testElems[rand.Intn(len(testElems))]) + result := tk.MustQuery("select * from t where " + cond).Sort().Rows() + tk.MustQuery("select * from tidx where " + cond).Sort().Check(result) + } + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(e enum('d','c','b','a'), a int, index idx(e));") + tk.MustExec("insert into t values(1,1),(2,2),(3,3),(4,4);") + tk.MustQuery("select /*+ use_index(t, idx) */ * from t where e not in ('a','d') and a = 2;").Check( + testkit.Rows("c 2")) + + // issue 24419 + tk.MustExec("use test") + tk.MustExec("drop table if exists t02") + tk.MustExec("CREATE TABLE `t02` ( `COL1` enum('^YSQT0]V@9TFN>^WB6G?NG@S8>VYOM;BSC@64=ZISGS?O[JDFBI5M]QXJYQNSKU>NGAWLXS26LMTZ2YNN`XKIUGKY0IHDWV>E[BJJCABOKH1M^CB5E@DLS7Q88PWZTEAY]1ZQMN5NX[IFIYA983K:E4N77@FINM5HVGQCUCVNF5WLOOOEORAM=_JLMVFURMUASTVDBE','NL3V:J9LM4U5KUCVR;P','M5=T5FLQEZMPZAXH]4G:TSYYYVQ7O@4S6C3N8WPFKSP;SRD6VW@94BBH8XCT','P]I52Y46F?@RMOOF6;FWDTO`7FIT]R:]ELHD[CNLDSHC7FPBYOOJXLZSBV^5C^AAF6J5BCKE4V9==@H=4C]GMZXPNM','ECIQWH>?MK=ARGI0WVJNIBZFCFVJHFIUYJ:2?2WWZBNBWTPFNQPLLBFP9R_','E<_Y9OT@SOPYR72VIJVMBWIVPF@TTBZ@8ZPBZL=LXZF`WM4V2?K>AT','PZ@PR6XN28JL`B','ZOHBSCRMZPOI`IVTSEZAIDAF7DS@1TT20AP9','QLDIOY[Y:JZR@OL__I^@FBO=O_?WOOR:2BE:QJC','BI^TGJ_NEEXYKV1POHTOJQPGCPVR=TYZMGWABUQR07J8U::W4','N`ZN4P@9T[JW;FR6=FA4WP@APNPG[XQVIK4]F]2>EC>JEIOXC``;;?OHP') DEFAULT NULL, `COL2` tinyint DEFAULT NULL, `COL3` time DEFAULT NULL, KEY `U_M_COL4` (`COL1`,`COL2`), KEY `U_M_COL5` (`COL3`,`COL2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") + tk.MustExec("insert into t02(col1, col2) values ('OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A', 39), ('OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A', 51), ('OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A', 55), ('OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A', -30), ('ZOHBSCRMZPOI`IVTSEZAIDAF7DS@1TT20AP9', -30);") + tk.MustQuery("select * from t02 where col1 not in (\"W1Rgd74pbJaGX47h1MPjpr0XSKJNCnwEleJ50Vbpl9EmbHJX6D6BXYKT2UAbl1uDw3ZGeYykhzG6Gld0wKdOiT4Gv5j9upHI0Q7vrXij4N9WNFJvB\", \"N`ZN4P@9T[JW;FR6=FA4WP@APNPG[XQVIK4]F]2>EC>JEIOXC``;;?OHP\") and col2 = -30;").Check( + testkit.Rows( + "OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A -30 ", + "ZOHBSCRMZPOI`IVTSEZAIDAF7DS@1TT20AP9 -30 ")) +} diff --git a/go.mod b/go.mod index 8fa1e66ceb0e4..a3f3c4e465636 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module github.com/pingcap/tidb -replace github.com/pingcap/parser v0.0.0-20210421190254-588138d35e55 => github.com/JmPotato/parser v0.0.0-20210428071746-2dcbd7ce4694 +replace github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde => github.com/JmPotato/parser v0.0.0-20210510024050-df181d49c58e require ( github.com/BurntSushi/toml v0.3.1 @@ -47,9 +47,9 @@ require ( github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20210402093459-65aa336ccbbf + github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 - github.com/pingcap/parser v0.0.0-20210421190254-588138d35e55 + github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 @@ -84,7 +84,7 @@ require ( gopkg.in/natefinch/lumberjack.v2 v2.0.0 gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 // indirect - honnef.co/go/tools v0.1.3 // indirect + honnef.co/go/tools v0.1.4 // indirect modernc.org/mathutil v1.2.2 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 diff --git a/go.sum b/go.sum index f7d0b653922db..c8f96e64d3b06 100644 --- a/go.sum +++ b/go.sum @@ -34,8 +34,8 @@ github.com/HdrHistogram/hdrhistogram-go v0.9.0 h1:dpujRju0R4M/QZzcnR1LH1qm+TVG3U github.com/HdrHistogram/hdrhistogram-go v0.9.0/go.mod h1:nxrse8/Tzg2tg3DZcZjm6qEclQKK70g0KxO61gFFZD4= github.com/Jeffail/gabs/v2 v2.5.1 h1:ANfZYjpMlfTTKebycu4X1AgkVWumFVDYQl7JwOr4mDk= github.com/Jeffail/gabs/v2 v2.5.1/go.mod h1:xCn81vdHKxFUuWWAaD5jCTQDNPBMh5pPs9IJ+NcziBI= -github.com/JmPotato/parser v0.0.0-20210428071746-2dcbd7ce4694 h1:pBJc7xeGu1d7kTKl46Kfu5ZTOTpxsbi8Q7dwx7VmZWM= -github.com/JmPotato/parser v0.0.0-20210428071746-2dcbd7ce4694/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= +github.com/JmPotato/parser v0.0.0-20210510024050-df181d49c58e h1:hdGbDsK1y2B+EclwSvtDmToBF5NaQbLTIk9bVrV5O3c= +github.com/JmPotato/parser v0.0.0-20210510024050-df181d49c58e/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/KyleBanks/depth v1.2.1/go.mod h1:jzSb9d0L43HxTQfT+oSA1EEp2q+ne2uh6XgeJcm8brE= github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= @@ -438,8 +438,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17Xtb github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210402093459-65aa336ccbbf h1:y8ZVU2X20+3XZW2M0/B8YAZ8RhsTnOuneXr1UfIKeNU= -github.com/pingcap/kvproto v0.0.0-20210402093459-65aa336ccbbf/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e h1:oUMZ6X/Kpaoxfejh9/jQ+4UZ5xk9MRYcouWJ0oXRKNE= +github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= @@ -932,8 +932,8 @@ honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.1.3 h1:qTakTkI6ni6LFD5sBwwsdSO+AQqbSIxOauHTTQKZ/7o= -honnef.co/go/tools v0.1.3/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= +honnef.co/go/tools v0.1.4 h1:SadWOkti5uVN1FAMgxn165+Mw00fuQKyk4Gyn/inxNQ= +honnef.co/go/tools v0.1.4/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= modernc.org/mathutil v1.2.2 h1:+yFk8hBprV+4c0U9GjFtL+dV3N8hOJ8JCituQcMShFY= modernc.org/mathutil v1.2.2/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= diff --git a/infoschema/cluster.go b/infoschema/cluster.go index 2bb3998934815..f113e90a0f587 100644 --- a/infoschema/cluster.go +++ b/infoschema/cluster.go @@ -19,8 +19,11 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/privilege" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/sem" ) // Cluster table list, attention: @@ -78,12 +81,18 @@ func isClusterTableByName(dbName, tableName string) bool { } // AppendHostInfoToRows appends host info to the rows. -func AppendHostInfoToRows(rows [][]types.Datum) ([][]types.Datum, error) { +func AppendHostInfoToRows(ctx sessionctx.Context, rows [][]types.Datum) ([][]types.Datum, error) { serverInfo, err := infosync.GetServerInfo() if err != nil { return nil, err } addr := serverInfo.IP + ":" + strconv.FormatUint(uint64(serverInfo.StatusPort), 10) + if sem.IsEnabled() { + checker := privilege.GetPrivilegeManager(ctx) + if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) { + addr = serverInfo.ID + } + } for i := range rows { row := make([]types.Datum, 0, len(rows[i])+1) row = append(row, types.NewStringDatum(addr)) diff --git a/kv/kv.go b/kv/kv.go index 0a81c8e4c69ac..de0ec31fa6d84 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -257,7 +257,7 @@ type Request struct { // sent to multiple storage units concurrently. Concurrency int // IsolationLevel is the isolation level, default is SI. - IsolationLevel tikvstore.IsoLevel + IsolationLevel IsoLevel // Priority is the priority of this KV request, its value may be PriorityNormal/PriorityLow/PriorityHigh. Priority int // memTracker is used to trace and control memory usage in co-processor layer. @@ -440,3 +440,20 @@ type SplittableStore interface { WaitScatterRegionFinish(ctx context.Context, regionID uint64, backOff int) error CheckRegionInScattering(regionID uint64) (bool, error) } + +// Priority value for transaction priority. +const ( + PriorityNormal = iota + PriorityLow + PriorityHigh +) + +// IsoLevel is the transaction's isolation level. +type IsoLevel int + +const ( + // SI stands for 'snapshot isolation'. + SI IsoLevel = iota + // RC stands for 'read committed'. + RC +) diff --git a/kv/mock_test.go b/kv/mock_test.go index 6b8cec5ae9f66..45e45d5941251 100644 --- a/kv/mock_test.go +++ b/kv/mock_test.go @@ -35,7 +35,7 @@ func (s testMockSuite) TestInterface(c *C) { snapshot := storage.GetSnapshot(version) _, err = snapshot.BatchGet(context.Background(), []Key{Key("abc"), Key("def")}) c.Check(err, IsNil) - snapshot.SetOption(tikvstore.Priority, tikvstore.PriorityNormal) + snapshot.SetOption(tikvstore.Priority, PriorityNormal) transaction, err := storage.Begin() c.Check(err, IsNil) diff --git a/meta/autoid/errors.go b/meta/autoid/errors.go index 093e0abbdea53..777de466b7704 100644 --- a/meta/autoid/errors.go +++ b/meta/autoid/errors.go @@ -22,6 +22,7 @@ import ( var ( errInvalidTableID = dbterror.ClassAutoid.NewStd(mysql.ErrInvalidTableID) errInvalidIncrementAndOffset = dbterror.ClassAutoid.NewStd(mysql.ErrInvalidIncrementAndOffset) + errNotImplemented = dbterror.ClassAutoid.NewStd(mysql.ErrNotImplemented) ErrAutoincReadFailed = dbterror.ClassAutoid.NewStd(mysql.ErrAutoincReadFailed) ErrWrongAutoKey = dbterror.ClassAutoid.NewStd(mysql.ErrWrongAutoKey) ErrInvalidAllocatorType = dbterror.ClassAutoid.NewStd(mysql.ErrUnknownAllocatorType) diff --git a/meta/autoid/memid.go b/meta/autoid/memid.go new file mode 100644 index 0000000000000..703ffe1db4bf8 --- /dev/null +++ b/meta/autoid/memid.go @@ -0,0 +1,145 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package autoid + +import ( + "context" + "math" + + "github.com/pingcap/parser/model" +) + +// NewAllocatorFromTempTblInfo creates an in-memory allocator from a temporary table info. +func NewAllocatorFromTempTblInfo(tblInfo *model.TableInfo) Allocator { + hasRowID := !tblInfo.PKIsHandle && !tblInfo.IsCommonHandle + hasAutoIncID := tblInfo.GetAutoIncrementColInfo() != nil + // Temporary tables don't support auto_random and sequence. + if hasRowID || hasAutoIncID { + return &inMemoryAllocator{ + isUnsigned: tblInfo.IsAutoIncColUnsigned(), + allocType: RowIDAllocType, + } + } + return nil +} + +// inMemoryAllocator is typically used for temporary tables. +// Some characteristics: +// - It allocates IDs from memory. +// - It's session-wide and thus won't be visited concurrently. +// - It doesn't support sequence. +// - The metrics are not reported. +type inMemoryAllocator struct { + base int64 + isUnsigned bool + allocType AllocatorType +} + +// Base implements autoid.Allocator Base interface. +func (alloc *inMemoryAllocator) Base() int64 { + return alloc.base +} + +// End implements autoid.Allocator End interface. +func (alloc *inMemoryAllocator) End() int64 { + // It doesn't matter. + return 0 +} + +// GetType implements autoid.Allocator GetType interface. +func (alloc *inMemoryAllocator) GetType() AllocatorType { + return alloc.allocType +} + +// NextGlobalAutoID implements autoid.Allocator NextGlobalAutoID interface. +func (alloc *inMemoryAllocator) NextGlobalAutoID(tableID int64) (int64, error) { + return 0, errNotImplemented.GenWithStackByArgs() +} + +func (alloc *inMemoryAllocator) Alloc(ctx context.Context, tableID int64, n uint64, increment, offset int64) (int64, int64, error) { + if n == 0 { + return 0, 0, nil + } + if alloc.allocType == AutoIncrementType || alloc.allocType == RowIDAllocType { + if !validIncrementAndOffset(increment, offset) { + return 0, 0, errInvalidIncrementAndOffset.GenWithStackByArgs(increment, offset) + } + } + if alloc.isUnsigned { + return alloc.alloc4Unsigned(n, increment, offset) + } + return alloc.alloc4Signed(n, increment, offset) +} + +// Rebase implements autoid.Allocator Rebase interface. +// The requiredBase is the minimum base value after Rebase. +// The real base may be greater than the required base. +func (alloc *inMemoryAllocator) Rebase(tableID, requiredBase int64, allocIDs bool) error { + if alloc.isUnsigned { + if uint64(requiredBase) > uint64(alloc.base) { + alloc.base = requiredBase + } + } else { + if requiredBase > alloc.base { + alloc.base = requiredBase + } + } + return nil +} + +func (alloc *inMemoryAllocator) alloc4Signed(n uint64, increment, offset int64) (int64, int64, error) { + // Check offset rebase if necessary. + if offset-1 > alloc.base { + alloc.base = offset - 1 + } + // CalcNeededBatchSize calculates the total batch size needed. + n1 := CalcNeededBatchSize(alloc.base, int64(n), increment, offset, alloc.isUnsigned) + + // 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 + } + + min := alloc.base + alloc.base += n1 + return min, alloc.base, nil +} + +func (alloc *inMemoryAllocator) alloc4Unsigned(n uint64, increment, offset int64) (int64, int64, error) { + // Check offset rebase if necessary. + if uint64(offset)-1 > uint64(alloc.base) { + alloc.base = int64(uint64(offset) - 1) + } + + // CalcNeededBatchSize calculates the total batch size needed. + n1 := CalcNeededBatchSize(alloc.base, int64(n), increment, offset, alloc.isUnsigned) + + // Condition alloc.base+n1 > alloc.end will overflow when alloc.base + n1 > MaxInt64. So need this. + if math.MaxUint64-uint64(alloc.base) <= uint64(n1) { + return 0, 0, ErrAutoincReadFailed + } + + min := alloc.base + // Use uint64 n directly. + alloc.base = int64(uint64(alloc.base) + uint64(n1)) + return min, alloc.base, nil +} + +func (alloc *inMemoryAllocator) AllocSeqCache(tableID int64) (int64, int64, int64, error) { + return 0, 0, 0, errNotImplemented.GenWithStackByArgs() +} + +func (alloc *inMemoryAllocator) RebaseSeq(tableID, requiredBase int64) (int64, bool, error) { + return 0, false, errNotImplemented.GenWithStackByArgs() +} diff --git a/meta/autoid/memid_test.go b/meta/autoid/memid_test.go new file mode 100644 index 0000000000000..f4b1267ad8f99 --- /dev/null +++ b/meta/autoid/memid_test.go @@ -0,0 +1,106 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package autoid_test + +import ( + "context" + "math" + + . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" + "github.com/pingcap/parser/types" + "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/store/mockstore" +) + +func (*testSuite) TestInMemoryAlloc(c *C) { + store, err := mockstore.NewMockStore() + c.Assert(err, IsNil) + defer func() { + err := store.Close() + c.Assert(err, IsNil) + }() + + columnInfo := &model.ColumnInfo{ + FieldType: types.FieldType{ + Flag: mysql.AutoIncrementFlag, + }, + } + tblInfo := &model.TableInfo{ + Columns: []*model.ColumnInfo{columnInfo}, + } + alloc := autoid.NewAllocatorFromTempTblInfo(tblInfo) + c.Assert(alloc, NotNil) + + // alloc 1 + ctx := context.Background() + _, id, err := alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(1)) + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(2)) + + // alloc N + _, id, err = alloc.Alloc(ctx, 1, 10, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(12)) + + // increment > N + _, id, err = alloc.Alloc(ctx, 1, 1, 10, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(21)) + + // offset + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 30) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(30)) + + // rebase + err = alloc.Rebase(1, int64(40), true) + c.Assert(err, IsNil) + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(41)) + err = alloc.Rebase(1, int64(10), true) + c.Assert(err, IsNil) + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(42)) + + // maxInt64 + err = alloc.Rebase(1, int64(math.MaxInt64-2), true) + c.Assert(err, IsNil) + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(math.MaxInt64-1)) + _, _, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) + + // test unsigned + columnInfo.FieldType.Flag |= mysql.UnsignedFlag + alloc = autoid.NewAllocatorFromTempTblInfo(tblInfo) + c.Assert(alloc, NotNil) + + var n uint64 = math.MaxUint64 - 2 + err = alloc.Rebase(1, int64(n), true) + c.Assert(err, IsNil) + _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(err, IsNil) + c.Assert(id, Equals, int64(n+1)) + _, _, err = alloc.Alloc(ctx, 1, 1, 1, 1) + c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) +} diff --git a/meta/meta.go b/meta/meta.go index 06bafa9dcd546..2682ed5b47d1e 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -94,8 +94,8 @@ type Meta struct { // NewMeta creates a Meta in transaction txn. // If the current Meta needs to handle a job, jobListKey is the type of the job's list. func NewMeta(txn kv.Transaction, jobListKeys ...JobListKeyType) *Meta { - txn.SetOption(tikvstore.Priority, tikvstore.PriorityHigh) - txn.SetOption(tikvstore.SyncLog, true) + txn.SetOption(tikvstore.Priority, kv.PriorityHigh) + txn.SetOption(tikvstore.SyncLog, struct{}{}) t := structure.NewStructure(txn, txn, mMetaPrefix) listKey := DefaultJobListKey if len(jobListKeys) != 0 { @@ -636,13 +636,13 @@ func (m *Meta) getDDLJob(key []byte, index int64) (*model.Job, error) { job := &model.Job{ // For compatibility, if the job is enqueued by old version TiDB and Priority field is omitted, - // set the default priority to tikvstore.PriorityLow. - Priority: tikvstore.PriorityLow, + // set the default priority to kv.PriorityLow. + Priority: kv.PriorityLow, } err = job.Decode(value) // Check if the job.Priority is valid. - if job.Priority < tikvstore.PriorityNormal || job.Priority > tikvstore.PriorityHigh { - job.Priority = tikvstore.PriorityLow + if job.Priority < kv.PriorityNormal || job.Priority > kv.PriorityHigh { + job.Priority = kv.PriorityLow } return job, errors.Trace(err) } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index eea0e9a3d1339..3818486955646 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -856,6 +856,7 @@ type AnalyzeColumnsTask struct { CommonHandleInfo *model.IndexInfo ColsInfo []*model.ColumnInfo TblInfo *model.TableInfo + Indexes []*model.IndexInfo analyzeInfo } diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index a6faf4b0cda08..816c94fa32a66 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3114,6 +3114,35 @@ func (s *testIntegrationSuite) TestReorderSimplifiedOuterJoins(c *C) { } } +// Apply operator may got panic because empty Projection is eliminated. +func (s *testIntegrationSerialSuite) TestIssue23887(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int, b int);") + tk.MustExec("insert into t values(1, 2), (3, 4);") + var input []string + var output []struct { + SQL string + Plan []string + Res []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) + } + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1 (c1 int primary key, c2 int, c3 int, index c2 (c2));") + tk.MustQuery("select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2;").Check(testkit.Rows("1")) +} + func (s *testIntegrationSerialSuite) TestDeleteStmt(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -3187,8 +3216,11 @@ func (s *testIntegrationSerialSuite) TestMppJoinDecimal(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists tt") tk.MustExec("create table t (c1 decimal(8, 5), c2 decimal(9, 5), c3 decimal(9, 4) NOT NULL, c4 decimal(8, 4) NOT NULL, c5 decimal(40, 20))") + tk.MustExec("create table tt (pk int(11) NOT NULL AUTO_INCREMENT primary key,col_varchar_64 varchar(64),col_char_64_not_null char(64) NOT null, col_decimal_30_10_key decimal(30,10), col_tinyint tinyint, col_varchar_key varchar(1), key col_decimal_30_10_key (col_decimal_30_10_key), key col_varchar_key(col_varchar_key));") tk.MustExec("analyze table t") + tk.MustExec("analyze table tt") // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Se) @@ -3196,7 +3228,7 @@ func (s *testIntegrationSerialSuite) TestMppJoinDecimal(c *C) { db, exists := is.SchemaByName(model.NewCIStr("test")) c.Assert(exists, IsTrue) for _, tblInfo := range db.Tables { - if tblInfo.Name.L == "t" { + if tblInfo.Name.L == "t" || tblInfo.Name.L == "tt" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, Available: true, @@ -3524,3 +3556,58 @@ func (s *testIntegrationSuite) TestIssue24095(c *C) { tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) } } + +func (s *testIntegrationSuite) TestConflictReadFromStorage(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(`create table t ( + a int, b int, c varchar(20), + primary key(a), key(b), key(c) + ) partition by range columns(a) ( + partition p0 values less than(6), + partition p1 values less than(11), + partition p2 values less than(16));`) + tk.MustExec(`insert into t values (1,1,"1"), (2,2,"2"), (8,8,"8"), (11,11,"11"), (15,15,"15")`) + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Se) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + c.Assert(exists, IsTrue) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustQuery(`explain select /*+ read_from_storage(tikv[t partition(p0)], tiflash[t partition(p1, p2)]) */ * from t`) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Storage hints are conflict, you can only specify one storage type of table test.t")) + tk.MustQuery(`explain select /*+ read_from_storage(tikv[t], tiflash[t]) */ * from t`) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Storage hints are conflict, you can only specify one storage type of table test.t")) +} + +// TestSequenceAsDataSource is used to test https://github.com/pingcap/tidb/issues/24383. +func (s *testIntegrationSuite) TestSequenceAsDataSource(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("drop sequence if exists s1, s2") + tk.MustExec("create sequence s1") + tk.MustExec("create sequence s2") + + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 37bef8ffbc38d..7ff0e2ac6c6aa 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -610,10 +610,9 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { } } if hintTbl := hintInfo.ifPreferTiFlash(alias); hintTbl != nil { - // 1. `ds.tableInfo.Partition == nil`, which means the hint takes effect in the whole table. - // 2. `ds.preferStoreType != 0`, which means there's a hint hit the both TiKV value and TiFlash value for table. - // If it's satisfied the above two conditions, then we can make sure there are some hints conflicted. - if ds.preferStoreType != 0 && ds.tableInfo.Partition == nil { + // `ds.preferStoreType != 0`, which means there's a hint hit the both TiKV value and TiFlash value for table. + // We can't support read a table from two different storages, even partition table. + if ds.preferStoreType != 0 { errMsg := fmt.Sprintf("Storage hints are conflict, you can only specify one storage type of table %s.%s", alias.dbName.L, alias.tblName.L) warning := ErrInternal.GenWithStack(errMsg) @@ -3593,6 +3592,14 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as return b.BuildDataSourceFromView(ctx, dbName, tableInfo) } + if tableInfo.IsSequence() { + if tn.TableSample != nil { + return nil, expression.ErrInvalidTableSample.GenWithStackByArgs("Unsupported TABLESAMPLE in sequences") + } + // When the source is a Sequence, we convert it to a TableDual, as what most databases do. + return b.buildTableDual(), nil + } + if tableInfo.GetPartitionInfo() != nil { // Use the new partition implementation, clean up the code here when it's full implemented. if !b.ctx.GetSessionVars().UseDynamicPartitionPrune() { diff --git a/planner/core/partition_prune.go b/planner/core/partition_prune.go index e8859a1890375..c10032c0a498a 100644 --- a/planner/core/partition_prune.go +++ b/planner/core/partition_prune.go @@ -37,7 +37,7 @@ func PartitionPruning(ctx sessionctx.Context, tbl table.PartitionedTable, conds case model.PartitionTypeHash: return s.pruneHashPartition(ctx, tbl, partitionNames, conds, columns, names) case model.PartitionTypeRange: - rangeOr, err := s.pruneRangePartition(ctx, pi, tbl, conds, columns, names) + rangeOr, _, err := s.pruneRangePartition(ctx, pi, tbl, conds, columns, names, nil) if err != nil { return nil, err } diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index 45440db71e6f4..dd614239de55d 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -67,6 +67,7 @@ func (s *testPartitionPruneSuit) TestHashPartitionPruner(c *C) { tk.MustExec("create table t6(a int, b int) partition by hash(a) partitions 3;") tk.MustExec("create table t7(a int, b int) partition by hash(a + b) partitions 10;") tk.MustExec("create table t8(a int, b int) partition by hash(a) partitions 6;") + tk.MustExec("create table t9(a bit(1) default null, b int(11) default null) partition by hash(a) partitions 3;") //issue #22619 var input []string var output []struct { @@ -454,3 +455,34 @@ partition by range (a) ( tk.MustQuery("select * from t3 where not (a = 1)").Sort().Check(testkit.Rows("11", "12", "13", "2", "3")) tk.MustQuery("select * from t3 where not (a != 1)").Check(testkit.Rows("1")) } + +//issue 22079 +func (s *testPartitionPruneSuit) TestRangePartitionPredicatePruner(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") + tk.MustExec("drop database if exists test_partition;") + tk.MustExec("create database test_partition") + tk.MustExec("use test_partition") + tk.MustExec("drop table if exists t") + tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.MustExec(`create table t (a int(11) default null) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin + partition by range(a) ( + partition p0 values less than (1), + partition p1 values less than (2), + partition p2 values less than (3), + partition p_max values less than (maxvalue));`) + + var input []string + var output []struct { + SQL string + Result []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) + } +} diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index c74365bb66384..6a8993286e36a 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -1700,3 +1700,55 @@ func (s *testPlanSuite) TestNthPlanHintWithExplain(c *C) { // hold in the future, you may need to modify this. tk.MustQuery("explain format = 'brief' select * from test.tt where a=1 and b=1").Check(testkit.Rows(output[1].Plan...)) } + +func (s *testPlanSuite) TestEnumIndex(c *C) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + s.testData.GetTestCases(c, &input, &output) + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(e enum('c','b','a'), index idx(e))") + tk.MustExec("insert into t values(1),(2),(3);") + + for i, ts := range input { + s.testData.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + } +} + +func (s *testPlanSuite) TestPossibleProperties(c *C) { + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists student, sc") + tk.MustExec("create table student(id int primary key auto_increment, name varchar(4) not null)") + tk.MustExec("create table sc(id int primary key auto_increment, student_id int not null, course_id int not null, score int not null)") + tk.MustExec("insert into student values (1,'s1'), (2,'s2')") + tk.MustExec("insert into sc (student_id, course_id, score) values (1,1,59), (1,2,57), (1,3,76), (2,1,99), (2,2,100), (2,3,100)") + tk.MustQuery("select /*+ stream_agg() */ a.id, avg(b.score) as afs from student a join sc b on a.id = b.student_id where b.score < 60 group by a.id having count(b.course_id) >= 2").Check(testkit.Rows( + "1 58.0000", + )) +} diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index a7cb71136470b..53f63f25fbc18 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -6,7 +6,7 @@ // // http://www.apache.org/licenses/LICENSE-2.0 // -// Unless required by aprettyPrintlicable law or agreed to in writing, software +// Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, // See the License for the specific language governing permissions and // limitations under the License. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 88f8e54e6be3f..6fc98bc522508 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1633,6 +1633,43 @@ func GetPhysicalIDsAndPartitionNames(tblInfo *model.TableInfo, partitionNames [] return ids, names, nil } +func (b *PlanBuilder) buildAnalyzeFullSamplingTask( + as *ast.AnalyzeTableStmt, + taskSlice []AnalyzeColumnsTask, + physicalIDs []int64, + names []string, + tbl *ast.TableName, + version int, +) []AnalyzeColumnsTask { + idxInfos := make([]*model.IndexInfo, 0, len(tbl.TableInfo.Indices)) + for _, idx := range tbl.TableInfo.Indices { + if idx.State != model.StatePublic { + continue + } + idxInfos = append(idxInfos, idx) + } + for i, id := range physicalIDs { + if id == tbl.TableInfo.ID { + id = -1 + } + info := analyzeInfo{ + DBName: tbl.Schema.O, + TableName: tbl.Name.O, + PartitionName: names[i], + TableID: AnalyzeTableID{TableID: tbl.TableInfo.ID, PartitionID: id}, + Incremental: as.Incremental, + StatsVersion: version, + } + taskSlice = append(taskSlice, AnalyzeColumnsTask{ + ColsInfo: tbl.TableInfo.Columns, + analyzeInfo: info, + TblInfo: tbl.TableInfo, + Indexes: idxInfos, + }) + } + return taskSlice +} + func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.AnalyzeOptionType]uint64, version int) (Plan, error) { p := &Analyze{Opts: opts} for _, tbl := range as.TableNames { @@ -1659,6 +1696,10 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A "If you want to switch to version 2 statistics, please first disable query feedback by setting feedback-probability to 0.0 in the config file.")) } } + if version == statistics.Version3 { + p.ColTasks = b.buildAnalyzeFullSamplingTask(as, p.ColTasks, physicalIDs, names, tbl, version) + continue + } for _, idx := range idxInfo { // For prefix common handle. We don't use analyze mixed to handle it with columns. Because the full value // is read by coprocessor, the prefix index would get wrong stats in this case. @@ -1730,6 +1771,10 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.A } b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) } + if version == statistics.Version3 { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 3 would collect all statistics not only the selected indexes")) + return b.buildAnalyzeTable(as, opts, version) + } for _, idxName := range as.IndexNames { if isPrimaryIndex(idxName) { handleCols := BuildHandleColsForAnalyze(b.ctx, tblInfo) @@ -1791,6 +1836,10 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[as } b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) } + if version == statistics.Version3 { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 3 would collect all statistics not only the selected indexes")) + return b.buildAnalyzeTable(as, opts, version) + } for _, idx := range tblInfo.Indices { if idx.State == model.StatePublic { for i, id := range physicalIDs { @@ -1837,7 +1886,7 @@ var analyzeOptionLimit = map[ast.AnalyzeOptionType]uint64{ ast.AnalyzeOptNumTopN: 1024, ast.AnalyzeOptCMSketchWidth: CMSketchSizeLimit, ast.AnalyzeOptCMSketchDepth: CMSketchSizeLimit, - ast.AnalyzeOptNumSamples: 100000, + ast.AnalyzeOptNumSamples: 500000, } var analyzeOptionDefault = map[ast.AnalyzeOptionType]uint64{ diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 686d67b00a9e0..537f1e06468bd 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -17,7 +17,9 @@ import ( "context" "fmt" "math" + "math/rand" "strconv" + "strings" "time" . "github.com/pingcap/check" @@ -31,6 +33,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/hint" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" @@ -1193,3 +1196,145 @@ func (s *testPlanSerialSuite) TestIssue23671(c *C) { tk.MustQuery("execute s1 using @a, @b, @c").Check(testkit.Rows("1 1", "2 2")) tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } + +func (s *testPlanSerialSuite) TestPartitionTable(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + // enable plan cache + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + orgEnable := core.PreparedPlanCacheEnabled() + defer func() { + dom.Close() + err = store.Close() + c.Assert(err, IsNil) + core.SetPreparedPlanCache(orgEnable) + }() + core.SetPreparedPlanCache(true) + tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + c.Assert(err, IsNil) + + // enable partition table dynamic mode + tk.MustExec("create database test_plan_cache") + tk.MustExec("use test_plan_cache") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + type testcase struct { + t1Create string + t2Create string + rowGener func() string + varGener func() string + query string + } + randDateTime := func() string { + return fmt.Sprintf("%v-%v-%v %v:%v:%v", + 1950+rand.Intn(100), 1+rand.Intn(12), 1+rand.Intn(28), // date + rand.Intn(24), rand.Intn(60), rand.Intn(60)) // time + } + randDate := func() string { + return fmt.Sprintf("%v-%v-%v", 1950+rand.Intn(100), 1+rand.Intn(12), 1+rand.Intn(28)) + } + testcases := []testcase{ + { // hash partition + int + "create table t1(a int, b int) partition by hash(a) partitions 20", + "create table t2(a int, b int)", + func() string { return fmt.Sprintf("(%v, %v)", rand.Intn(100000000), rand.Intn(100000000)) }, + func() string { return fmt.Sprintf("%v", rand.Intn(100000000)) }, + `select * from %v where a > ?`, + }, + { // range partition + int + `create table t1(a int, b int) partition by range(a) ( + partition p0 values less than (20000000), + partition p1 values less than (40000000), + partition p2 values less than (60000000), + partition p3 values less than (80000000), + partition p4 values less than (100000000))`, + `create table t2(a int, b int)`, + func() string { return fmt.Sprintf("(%v, %v)", rand.Intn(100000000), rand.Intn(100000000)) }, + func() string { return fmt.Sprintf("%v", rand.Intn(100000000)) }, + `select * from %v where a > ?`, + }, + { // range partition + varchar + `create table t1(a varchar(10), b varchar(10)) partition by range columns(a) ( + partition p0 values less than ('200'), + partition p1 values less than ('400'), + partition p2 values less than ('600'), + partition p3 values less than ('800'), + partition p4 values less than ('9999'))`, + `create table t2(a varchar(10), b varchar(10))`, + func() string { return fmt.Sprintf(`("%v", "%v")`, rand.Intn(1000), rand.Intn(1000)) }, + func() string { return fmt.Sprintf(`"%v"`, rand.Intn(1000)) }, + `select * from %v where a > ?`, + }, + { // range partition + datetime + `create table t1(a datetime, b datetime) partition by range columns(a) ( + partition p0 values less than ('1970-01-01 00:00:00'), + partition p1 values less than ('1990-01-01 00:00:00'), + partition p2 values less than ('2010-01-01 00:00:00'), + partition p3 values less than ('2030-01-01 00:00:00'), + partition p4 values less than ('2060-01-01 00:00:00'))`, + `create table t2(a datetime, b datetime)`, + func() string { return fmt.Sprintf(`("%v", "%v")`, randDateTime(), randDateTime()) }, + func() string { return fmt.Sprintf(`"%v"`, randDateTime()) }, + `select * from %v where a > ?`, + }, + { // range partition + date + `create table t1(a date, b date) partition by range columns(a) ( + partition p0 values less than ('1970-01-01'), + partition p1 values less than ('1990-01-01'), + partition p2 values less than ('2010-01-01'), + partition p3 values less than ('2030-01-01'), + partition p4 values less than ('2060-01-01'))`, + `create table t2(a date, b date)`, + func() string { return fmt.Sprintf(`("%v", "%v")`, randDate(), randDate()) }, + func() string { return fmt.Sprintf(`"%v"`, randDate()) }, + `select * from %v where a > ?`, + }, + { // list partition + int + `create table t1(a int, b int) partition by list(a) ( + partition p0 values in (0, 1, 2, 3, 4), + partition p1 values in (5, 6, 7, 8, 9), + partition p2 values in (10, 11, 12, 13, 14), + partition p3 values in (15, 16, 17, 18, 19))`, + `create table t2(a int, b int)`, + func() string { return fmt.Sprintf("(%v, %v)", rand.Intn(20), rand.Intn(20)) }, + func() string { return fmt.Sprintf("%v", rand.Intn(20)) }, + `select * from %v where a > ?`, + }, + } + for _, tc := range testcases { + // create tables and insert some records + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec(tc.t1Create) + tk.MustExec(tc.t2Create) + vals := make([]string, 0, 2048) + for i := 0; i < 2048; i++ { + vals = append(vals, tc.rowGener()) + } + tk.MustExec(fmt.Sprintf("insert into t1 values %s", strings.Join(vals, ","))) + tk.MustExec(fmt.Sprintf("insert into t2 values %s", strings.Join(vals, ","))) + + // the first query, @last_plan_from_cache should be zero + tk.MustExec(fmt.Sprintf(`prepare stmt1 from "%s"`, fmt.Sprintf(tc.query, "t1"))) + tk.MustExec(fmt.Sprintf(`prepare stmt2 from "%s"`, fmt.Sprintf(tc.query, "t2"))) + tk.MustExec(fmt.Sprintf("set @a=%v", tc.varGener())) + result1 := tk.MustQuery("execute stmt1 using @a").Sort().Rows() + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt2 using @a").Sort().Check(result1) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + for i := 0; i < 100; i++ { + tk.MustExec(fmt.Sprintf("set @a=%v", tc.varGener())) + result1 := tk.MustQuery("execute stmt1 using @a").Sort().Rows() + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt2 using @a").Sort().Check(result1) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + } + } +} diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index d84fede3e5b00..a3719fe4c4b0b 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -564,7 +564,7 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { return } enableNoopFuncs := p.ctx.GetSessionVars().EnableNoopFuncs - if stmt.IsTemporary && !enableNoopFuncs { + if stmt.TemporaryKeyword == ast.TemporaryLocal && !enableNoopFuncs { p.err = expression.ErrFunctionsNoopImpl.GenWithStackByArgs("CREATE TEMPORARY TABLE") return } @@ -676,7 +676,7 @@ func (p *preprocessor) checkDropSequenceGrammar(stmt *ast.DropSequenceStmt) { func (p *preprocessor) checkDropTableGrammar(stmt *ast.DropTableStmt) { p.checkDropTableNames(stmt.Tables) enableNoopFuncs := p.ctx.GetSessionVars().EnableNoopFuncs - if stmt.IsTemporary && !enableNoopFuncs { + if stmt.TemporaryKeyword == ast.TemporaryLocal && !enableNoopFuncs { p.err = expression.ErrFunctionsNoopImpl.GenWithStackByArgs("DROP TEMPORARY TABLE") return } diff --git a/planner/core/property_cols_prune.go b/planner/core/property_cols_prune.go index c8296aa799283..91a9f34fb9017 100644 --- a/planner/core/property_cols_prune.go +++ b/planner/core/property_cols_prune.go @@ -165,13 +165,27 @@ func (p *LogicalProjection) PreparePossibleProperties(schema *expression.Schema, return childProperties } +func clonePossibleProperties(props [][]*expression.Column) [][]*expression.Column { + res := make([][]*expression.Column, len(props)) + for i, prop := range props { + clonedProp := make([]*expression.Column, len(prop)) + for j, col := range prop { + clonedProp[j] = col.Clone().(*expression.Column) + } + res[i] = clonedProp + } + return res +} + // PreparePossibleProperties implements LogicalPlan PreparePossibleProperties interface. func (p *LogicalJoin) PreparePossibleProperties(schema *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column { leftProperties := childrenProperties[0] rightProperties := childrenProperties[1] // TODO: We should consider properties propagation. - p.leftProperties = leftProperties - p.rightProperties = rightProperties + // Clone the Columns in the property before saving them, otherwise the upper Projection may + // modify them and lead to unexpected results. + p.leftProperties = clonePossibleProperties(leftProperties) + p.rightProperties = clonePossibleProperties(rightProperties) if p.JoinType == LeftOuterJoin || p.JoinType == LeftOuterSemiJoin { rightProperties = nil } else if p.JoinType == RightOuterJoin { @@ -200,13 +214,22 @@ func (la *LogicalAggregation) PreparePossibleProperties(schema *expression.Schem return nil } resultProperties := make([][]*expression.Column, 0, len(childProps)) + clonedProperties := make([][]*expression.Column, 0, len(childProps)) groupByCols := la.GetGroupByCols() for _, possibleChildProperty := range childProps { sortColOffsets := getMaxSortPrefix(possibleChildProperty, groupByCols) if len(sortColOffsets) == len(groupByCols) { - resultProperties = append(resultProperties, possibleChildProperty[:len(groupByCols)]) + prop := possibleChildProperty[:len(groupByCols)] + resultProperties = append(resultProperties, prop) + // Clone the Columns in the property before saving them, otherwise the upper Projection may + // modify them and lead to unexpected results. + clonedProp := make([]*expression.Column, len(prop)) + for i, col := range prop { + clonedProp[i] = col.Clone().(*expression.Column) + } + clonedProperties = append(clonedProperties, clonedProp) } } - la.possibleProperties = resultProperties - return la.possibleProperties + la.possibleProperties = clonedProperties + return resultProperties } diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index d070d793e1f8e..4b31853c138d0 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -314,11 +314,13 @@ func (p *LogicalJoin) PruneColumns(parentUsedCols []*expression.Column) error { if err != nil { return err } + addConstOneForEmptyProjection(p.children[0]) err = p.children[1].PruneColumns(rightCols) if err != nil { return err } + addConstOneForEmptyProjection(p.children[1]) p.mergeSchema() if p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { @@ -337,6 +339,7 @@ func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column) error if err != nil { return err } + addConstOneForEmptyProjection(la.children[1]) la.CorCols = extractCorColumnsBySchema4LogicalPlan(la.children[1], la.children[0].Schema()) for _, col := range la.CorCols { @@ -347,6 +350,7 @@ func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column) error if err != nil { return err } + addConstOneForEmptyProjection(la.children[0]) la.mergeSchema() return nil @@ -431,3 +435,25 @@ func (p *LogicalLimit) PruneColumns(parentUsedCols []*expression.Column) error { func (*columnPruner) name() string { return "column_prune" } + +// By add const one, we can avoid empty Projection is eliminated. +// Because in some cases, Projectoin cannot be eliminated even its output is empty. +func addConstOneForEmptyProjection(p LogicalPlan) { + proj, ok := p.(*LogicalProjection) + if !ok { + return + } + if proj.Schema().Len() != 0 { + return + } + + constOne := expression.NewOne() + proj.schema.Append(&expression.Column{ + UniqueID: proj.ctx.GetSessionVars().AllocPlanColumnID(), + RetType: constOne.GetType(), + }) + proj.Exprs = append(proj.Exprs, &expression.Constant{ + Value: constOne.Value, + RetType: constOne.GetType(), + }) +} diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 6121d92e0fb66..57858679b7795 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" @@ -190,14 +191,27 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl } used = append(used, int(idx)) } - } else { - used = []int{FullRange} - break + continue + } + + // issue:#22619 + if col.RetType.Tp == mysql.TypeBit { + // maximum number of partitions is 8192 + if col.RetType.Flen > 0 && col.RetType.Flen < int(math.Log2(ddl.PartitionCountLimit)) { + // all possible hash values + maxUsedPartitions := 1 << col.RetType.Flen + if maxUsedPartitions < numPartitions { + for i := 0; i < maxUsedPartitions; i++ { + used = append(used, i) + } + continue + } + } } - } else { - used = []int{FullRange} - break } + + used = []int{FullRange} + break } } if len(partitionNames) > 0 && len(used) == 1 && used[0] == FullRange { @@ -771,25 +785,26 @@ func intersectionRange(start, end, newStart, newEnd int) (int, int) { } func (s *partitionProcessor) pruneRangePartition(ctx sessionctx.Context, pi *model.PartitionInfo, tbl table.PartitionedTable, conds []expression.Expression, - columns []*expression.Column, names types.NameSlice) (partitionRangeOR, error) { + columns []*expression.Column, names types.NameSlice, condsToBePruned *[]expression.Expression) (partitionRangeOR, []expression.Expression, error) { partExpr, err := tbl.(partitionTable).PartitionExpr() if err != nil { - return nil, err + return nil, nil, err } // Partition by range columns. if len(pi.Columns) > 0 { - return s.pruneRangeColumnsPartition(ctx, conds, pi, partExpr, columns, names) + result, err := s.pruneRangeColumnsPartition(ctx, conds, pi, partExpr, columns, names) + return result, nil, err } // Partition by range. col, fn, mono, err := makePartitionByFnCol(ctx, columns, names, pi.Expr) if err != nil { - return nil, err + return nil, nil, err } result := fullRange(len(pi.Definitions)) if col == nil { - return result, nil + return result, nil, nil } // Extract the partition column, if the column is not null, it's possible to prune. @@ -803,14 +818,42 @@ func (s *partitionProcessor) pruneRangePartition(ctx sessionctx.Context, pi *mod monotonous: mono, } result = partitionRangeForCNFExpr(ctx, conds, &pruner, result) - return result, nil + + if condsToBePruned == nil { + return result, nil, nil + } + // remove useless predicates after partition pruning + newConds := make([]expression.Expression, 0, len(*condsToBePruned)) + for _, cond := range *condsToBePruned { + if dataForPrune, ok := pruner.extractDataForPrune(ctx, cond); ok { + switch dataForPrune.op { + case ast.EQ: + unsigned := mysql.HasUnsignedFlag(pruner.col.RetType.Flag) + start, _ := pruneUseBinarySearch(pruner.lessThan, dataForPrune, unsigned) + // if the type of partition key is Int + if pk, ok := partExpr.Expr.(*expression.Column); ok && pk.RetType.EvalType() == types.ETInt { + // see if can be removed + // see issue #22079: https://github.com/pingcap/tidb/issues/22079 for details + if start > 0 && pruner.lessThan.data[start-1] == dataForPrune.c && (pruner.lessThan.data[start]-1) == dataForPrune.c { + continue + } + } + } + } + newConds = append(newConds, cond) + } + + return result, newConds, nil } func (s *partitionProcessor) processRangePartition(ds *DataSource, pi *model.PartitionInfo) (LogicalPlan, error) { - used, err := s.pruneRangePartition(ds.ctx, pi, ds.table.(table.PartitionedTable), ds.allConds, ds.TblCols, ds.names) + used, prunedConds, err := s.pruneRangePartition(ds.ctx, pi, ds.table.(table.PartitionedTable), ds.allConds, ds.TblCols, ds.names, &ds.pushedDownConds) if err != nil { return nil, err } + if prunedConds != nil { + ds.pushedDownConds = prunedConds + } return s.makeUnionAllChildren(ds, pi, used) } diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 2b8862d8ae3a5..936c4720cbc87 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -371,7 +371,7 @@ func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression for _, cond := range predicates { newFilter := expression.ColumnSubstitute(cond, p.Schema(), p.Exprs) if !expression.HasGetSetVarFunc(newFilter) { - canBePushed = append(canBePushed, expression.ColumnSubstitute(cond, p.Schema(), p.Exprs)) + canBePushed = append(canBePushed, newFilter) } else { canNotBePushed = append(canNotBePushed, cond) } diff --git a/planner/core/task.go b/planner/core/task.go index 68b6c627e9165..205f5eb77b08a 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -652,9 +652,9 @@ func appendExpr(p *PhysicalProjection, expr expression.Expression) *expression.C return col } -// TiFlash join require that join key has exactly the same type, while TiDB only guarantee the join key is the same catalog, -// so if the join key type is not exactly the same, we need add a projection below the join or exchanger if exists. -func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) (*mppTask, *mppTask) { +// TiFlash join require that partition key has exactly the same type, while TiDB only guarantee the partition key is the same catalog, +// so if the partition key type is not exactly the same, we need add a projection below the join or exchanger if exists. +func (p *PhysicalHashJoin) convertPartitionKeysIfNeed(lTask, rTask *mppTask) (*mppTask, *mppTask) { lp := lTask.p if _, ok := lp.(*PhysicalExchangeReceiver); ok { lp = lp.Children()[0].Children()[0] @@ -663,15 +663,15 @@ func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) if _, ok := rp.(*PhysicalExchangeReceiver); ok { rp = rp.Children()[0].Children()[0] } - // to mark if any equal cond needs to convert - lMask := make([]bool, len(p.EqualConditions)) - rMask := make([]bool, len(p.EqualConditions)) - cTypes := make([]*types.FieldType, len(p.EqualConditions)) + // to mark if any partition key needs to convert + lMask := make([]bool, len(lTask.hashCols)) + rMask := make([]bool, len(rTask.hashCols)) + cTypes := make([]*types.FieldType, len(lTask.hashCols)) lChanged := false rChanged := false - for i, eqFunc := range p.EqualConditions { - lKey := eqFunc.GetArgs()[0].(*expression.Column) - rKey := eqFunc.GetArgs()[1].(*expression.Column) + for i := range lTask.hashCols { + lKey := lTask.hashCols[i] + rKey := rTask.hashCols[i] cType, lConvert, rConvert := negotiateCommonType(lKey.RetType, rKey.RetType) if lConvert { lMask[i] = true @@ -696,14 +696,12 @@ func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) rProj = getProj(p.ctx, rp) rp = rProj } - newEqCondition := make([]*expression.ScalarFunction, 0, len(p.EqualConditions)) - newEqCondition = append(newEqCondition, p.EqualConditions...) - p.EqualConditions = newEqCondition - lKeys := make([]*expression.Column, 0, len(p.EqualConditions)) - rKeys := make([]*expression.Column, 0, len(p.EqualConditions)) - for i, eqFunc := range p.EqualConditions { - lKey := eqFunc.GetArgs()[0].(*expression.Column) - rKey := eqFunc.GetArgs()[1].(*expression.Column) + + lPartKeys := make([]*expression.Column, 0, len(rTask.hashCols)) + rPartKeys := make([]*expression.Column, 0, len(lTask.hashCols)) + for i := range lTask.hashCols { + lKey := lTask.hashCols[i] + rKey := rTask.hashCols[i] if lMask[i] { cType := cTypes[i].Clone() cType.Flag = lKey.RetType.Flag @@ -716,12 +714,8 @@ func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) rCast := expression.BuildCastFunction(p.ctx, rKey, cType) rKey = appendExpr(rProj, rCast) } - if lMask[i] || rMask[i] { - eqCond := expression.NewFunctionInternal(p.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), lKey, rKey) - p.EqualConditions[i] = eqCond.(*expression.ScalarFunction) - } - lKeys = append(lKeys, lKey) - rKeys = append(rKeys, rKey) + lPartKeys = append(lPartKeys, lKey) + rPartKeys = append(rPartKeys, rKey) } // if left or right child changes, we need to add enforcer. if lChanged { @@ -730,7 +724,7 @@ func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) nlTask = nlTask.enforceExchangerImpl(&property.PhysicalProperty{ TaskTp: property.MppTaskType, PartitionTp: property.HashType, - PartitionCols: lKeys, + PartitionCols: lPartKeys, }) nlTask.cst = lTask.cst lProj.cost = nlTask.cst @@ -742,7 +736,7 @@ func (p *PhysicalHashJoin) convertJoinKeyForTiFlashIfNeed(lTask, rTask *mppTask) nrTask = nrTask.enforceExchangerImpl(&property.PhysicalProperty{ TaskTp: property.MppTaskType, PartitionTp: property.HashType, - PartitionCols: rKeys, + PartitionCols: rPartKeys, }) nrTask.cst = rTask.cst rProj.cost = nrTask.cst @@ -758,7 +752,11 @@ func (p *PhysicalHashJoin) attach2TaskForMpp(tasks ...task) task { return invalidTask } if p.mppShuffleJoin { - lTask, rTask = p.convertJoinKeyForTiFlashIfNeed(lTask, rTask) + // protection check is case of some bugs + if len(lTask.hashCols) != len(rTask.hashCols) || len(lTask.hashCols) == 0 { + return invalidTask + } + lTask, rTask = p.convertPartitionKeysIfNeed(lTask, rTask) } p.SetChildren(lTask.plan(), rTask.plan()) p.schema = BuildPhysicalJoinSchema(p.JoinType, p) @@ -2028,11 +2026,12 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { StoreType: kv.TiFlash, }.Init(ctx, t.p.SelectBlockOffset()) p.stats = t.p.statsInfo() - p.cost = t.cost() - return &rootTask{ + rt := &rootTask{ p: p, - cst: t.cst / 20, // TODO: This is tricky because mpp doesn't run in a coprocessor way. + cst: t.cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor, } + p.cost = rt.cost() + return rt } func (t *mppTask) needEnforce(prop *property.PhysicalProperty) bool { diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 81dfccf978aaa..3234652e5d000 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -234,7 +234,8 @@ "desc format = 'brief' select * from t t1 join t t2 on t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5", "desc format = 'brief' select * from t t1 join t t2 on t1.c1 + t1.c2 = t2.c2 / t2.c3", "desc format = 'brief' select * from t t1 where exists (select * from t t2 where t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5)", - "desc format = 'brief' select * from t t1 left join t t2 on t1.c1 = t2.c2 join t t3 on t2.c5 = t3.c3 right join t t4 on t3.c3 = t4.c4 " + "desc format = 'brief' select * from t t1 left join t t2 on t1.c1 = t2.c2 join t t3 on t2.c5 = t3.c3 right join t t4 on t3.c3 = t4.c4 ", + "desc format = 'brief' SELECT STRAIGHT_JOIN t1 . col_varchar_64 , t1 . col_char_64_not_null FROM tt AS t1 INNER JOIN( tt AS t2 JOIN tt AS t3 ON(t3 . col_decimal_30_10_key = t2 . col_tinyint)) ON(t3 . col_varchar_64 = t2 . col_varchar_key) WHERE t3 . col_varchar_64 = t1 . col_char_64_not_null GROUP BY 1 , 2" ] }, { @@ -293,5 +294,11 @@ "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10" ] + }, + { + "name": "TestIssue23887", + "cases": [ + "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t" + ] } ] diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 9512f3bbd1fcc..8b67310894d4b 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -102,7 +102,7 @@ { "SQL": "explain format = 'verbose' select count(*) from t2 group by a", "Plan": [ - "TableReader_24 3.00 77.00 root data:ExchangeSender_23", + "TableReader_24 3.00 3.21 root data:ExchangeSender_23", "└─ExchangeSender_23 3.00 77.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection_22 3.00 0.00 batchCop[tiflash] Column#4", " └─HashAgg_8 3.00 77.00 batchCop[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", @@ -152,8 +152,8 @@ { "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a", "Plan": [ - "StreamAgg_12 1.00 20.77 root funcs:count(1)->Column#7", - "└─TableReader_44 3.00 235.38 root data:ExchangeSender_43", + "StreamAgg_12 1.00 18.81 root funcs:count(1)->Column#7", + "└─TableReader_44 3.00 9.81 root data:ExchangeSender_43", " └─ExchangeSender_43 3.00 235.38 cop[tiflash] ExchangeType: PassThrough", " └─HashJoin_40 3.00 235.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─ExchangeReceiver_19(Build) 3.00 77.00 cop[tiflash] ", @@ -167,11 +167,11 @@ { "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", "Plan": [ - "StreamAgg_15 1.00 62.68 root funcs:count(1)->Column#10", - "└─HashJoin_65 3.00 53.68 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + "StreamAgg_15 1.00 60.48 root funcs:count(1)->Column#10", + "└─HashJoin_65 3.00 51.48 root inner join, equal:[eq(test.t1.b, test.t3.b)]", " ├─IndexReader_53(Build) 3.00 11.66 root index:IndexFullScan_52", " │ └─IndexFullScan_52 3.00 150.50 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_39(Probe) 3.00 264.38 root data:ExchangeSender_38", + " └─TableReader_39(Probe) 3.00 11.02 root data:ExchangeSender_38", " └─ExchangeSender_38 3.00 264.38 cop[tiflash] ExchangeType: PassThrough", " └─HashJoin_29 3.00 264.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─ExchangeReceiver_35(Build) 3.00 106.00 cop[tiflash] ", @@ -185,19 +185,21 @@ { "SQL": "explain format = 'verbose' select (2) in (select count(*) from t1) from (select t.b < (select t.b from t2 limit 1 ) from t3 t) t", "Plan": [ - "HashJoin_19 3.00 113.61 root CARTESIAN left outer semi join", - "├─Selection_38(Build) 0.80 11.18 root eq(2, Column#18)", - "│ └─StreamAgg_59 1.00 69.50 root funcs:count(Column#27)->Column#18", - "│ └─TableReader_60 1.00 5.17 root data:StreamAgg_43", - "│ └─StreamAgg_43 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#27", - "│ └─TableFullScan_58 3.00 60.50 batchCop[tiflash] table:t1 keep order:false", - "└─Apply_21(Probe) 3.00 82.03 root CARTESIAN left outer join", - " ├─TableReader_23(Build) 3.00 10.16 root data:TableFullScan_22", - " │ └─TableFullScan_22 3.00 128.00 cop[tikv] table:t keep order:false", - " └─Limit_27(Probe) 1.00 5.36 root offset:0, count:1", - " └─TableReader_33 1.00 5.36 root data:Limit_32", - " └─Limit_32 1.00 56.00 cop[tikv] offset:0, count:1", - " └─TableFullScan_30 1.00 56.00 cop[tikv] table:t2 keep order:false" + "HashJoin_19 3.00 133.41 root CARTESIAN left outer semi join", + "├─Selection_39(Build) 0.80 11.18 root eq(2, Column#18)", + "│ └─StreamAgg_60 1.00 69.50 root funcs:count(Column#32)->Column#18", + "│ └─TableReader_61 1.00 5.17 root data:StreamAgg_44", + "│ └─StreamAgg_44 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#32", + "│ └─TableFullScan_59 3.00 60.50 batchCop[tiflash] table:t1 keep order:false", + "└─Projection_20(Probe) 3.00 101.83 root 1->Column#28", + " └─Apply_22 3.00 82.03 root CARTESIAN left outer join", + " ├─TableReader_24(Build) 3.00 10.16 root data:TableFullScan_23", + " │ └─TableFullScan_23 3.00 128.00 cop[tikv] table:t keep order:false", + " └─Projection_27(Probe) 1.00 23.96 root 1->Column#26", + " └─Limit_28 1.00 5.36 root offset:0, count:1", + " └─TableReader_34 1.00 5.36 root data:Limit_33", + " └─Limit_33 1.00 56.00 cop[tikv] offset:0, count:1", + " └─TableFullScan_31 1.00 56.00 cop[tikv] table:t2 keep order:false" ] }, { @@ -1803,7 +1805,7 @@ "Projection 12500.00 root test.t.c1, test.t.c2, test.t.c1, test.t.c2, test.t.c3", "└─TableReader 12500.00 root data:ExchangeSender", " └─ExchangeSender 12500.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#21, Column#22) eq(Column#15, Column#16)]", + " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#13, Column#14) eq(Column#15, Column#16)]", " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#21, Column#15", " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, Column#13, Column#15, cast(Column#13, decimal(34,8) BINARY)->Column#21", @@ -1822,7 +1824,7 @@ "Projection 7976.02 root test.t.c1, test.t.c2, test.t.c5, Column#7, test.t.c1, test.t.c2, test.t.c3, Column#14", "└─TableReader 7976.02 root data:ExchangeSender", " └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7976.02 batchCop[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(Column#31, Column#32) eq(test.t.c5, Column#33)]", + " └─HashJoin 7976.02 batchCop[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c5, test.t.c1)]", " ├─ExchangeReceiver(Build) 7976.02 batchCop[tiflash] ", " │ └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c1, Column#31, test.t.c5", " │ └─Projection 7976.02 batchCop[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#31", @@ -1867,7 +1869,7 @@ "Projection 12462.54 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", "└─TableReader 12462.54 root data:ExchangeSender", " └─ExchangeSender 12462.54 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12462.54 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1) eq(Column#13, Column#14) eq(Column#15, Column#16) eq(test.t.c3, test.t.c4) eq(test.t.c5, Column#17)]", + " └─HashJoin 12462.54 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1) eq(test.t.c3, test.t.c2) eq(test.t.c1, test.t.c3) eq(test.t.c3, test.t.c4) eq(test.t.c5, test.t.c1)]", " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2, Column#13, Column#15, test.t.c3, test.t.c5", " │ └─Projection 9970.03 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#13, cast(test.t.c1, decimal(10,5))->Column#15", @@ -1886,7 +1888,7 @@ "Projection 12500.00 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", "└─TableReader 12500.00 root data:ExchangeSender", " └─ExchangeSender 12500.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#17, Column#14)]", + " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#13, Column#14)]", " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#17", " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, Column#13, cast(Column#13, decimal(17,9) BINARY)->Column#17", @@ -1904,7 +1906,7 @@ "Projection 7984.01 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", "└─TableReader 7984.01 root data:ExchangeSender", " └─ExchangeSender 7984.01 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7984.01 cop[tiflash] semi join, equal:[eq(test.t.c1, test.t.c2) eq(Column#13, Column#14) eq(Column#15, Column#16) eq(test.t.c4, test.t.c3) eq(Column#17, test.t.c5)]", + " └─HashJoin 7984.01 cop[tiflash] semi join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c3, test.t.c1) eq(test.t.c4, test.t.c3) eq(test.t.c1, test.t.c5)]", " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2, Column#14, Column#16, test.t.c3, test.t.c5", " │ └─Projection 9970.03 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#14, cast(test.t.c1, decimal(10,5))->Column#16", @@ -1920,29 +1922,57 @@ { "SQL": "desc format = 'brief' select * from t t1 left join t t2 on t1.c1 = t2.c2 join t t3 on t2.c5 = t3.c3 right join t t4 on t3.c3 = t4.c4 ", "Plan": [ - "TableReader 19492.21 root data:ExchangeSender", - "└─ExchangeSender 19492.21 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 19492.21 cop[tiflash] right outer join, equal:[eq(test.t.c3, test.t.c4)]", - " ├─Projection(Build) 15593.77 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", - " │ └─HashJoin 15593.77 cop[tiflash] inner join, equal:[eq(test.t.c5, Column#25)]", - " │ ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#25", - " │ │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", - " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t3 keep order:false, stats:pseudo", - " │ └─ExchangeReceiver(Probe) 12475.01 cop[tiflash] ", - " │ └─ExchangeSender 12475.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c5", - " │ └─HashJoin 12475.01 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", - " │ ├─ExchangeReceiver(Build) 9980.01 cop[tiflash] ", - " │ │ └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2", - " │ │ └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - " │ └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c1", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.c1))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", - " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c4", - " └─TableFullScan 10000.00 cop[tiflash] table:t4 keep order:false, stats:pseudo" + "Projection 19492.21 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + "└─TableReader 19492.21 root data:ExchangeSender", + " └─ExchangeSender 19492.21 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 19492.21 cop[tiflash] right outer join, equal:[eq(test.t.c3, test.t.c4)]", + " ├─Projection(Build) 15593.77 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + " │ └─HashJoin 15593.77 cop[tiflash] inner join, equal:[eq(test.t.c5, test.t.c3)]", + " │ ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#25", + " │ │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", + " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t3 keep order:false, stats:pseudo", + " │ └─ExchangeReceiver(Probe) 12475.01 cop[tiflash] ", + " │ └─ExchangeSender 12475.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c5", + " │ └─HashJoin 12475.01 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", + " │ ├─ExchangeReceiver(Build) 9980.01 cop[tiflash] ", + " │ │ └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2", + " │ │ └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", + " │ └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c1", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.c1))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", + " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#27", + " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c4, decimal(40,20))->Column#27", + " └─TableFullScan 10000.00 cop[tiflash] table:t4 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' SELECT STRAIGHT_JOIN t1 . col_varchar_64 , t1 . col_char_64_not_null FROM tt AS t1 INNER JOIN( tt AS t2 JOIN tt AS t3 ON(t3 . col_decimal_30_10_key = t2 . col_tinyint)) ON(t3 . col_varchar_64 = t2 . col_varchar_key) WHERE t3 . col_varchar_64 = t1 . col_char_64_not_null GROUP BY 1 , 2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, funcs:firstrow(test.tt.col_varchar_64)->test.tt.col_varchar_64, funcs:firstrow(test.tt.col_char_64_not_null)->test.tt.col_char_64_not_null", + " └─ExchangeReceiver 15609.38 batchCop[tiflash] ", + " └─ExchangeSender 15609.38 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64, test.tt.col_char_64_not_null", + " └─HashJoin 15609.38 batchCop[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", + " ├─ExchangeReceiver(Build) 10000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#29", + " │ └─Projection 10000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null, cast(test.tt.col_char_64_not_null, varchar(64) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#29", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 batchCop[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_key", + " │ └─Projection 9990.00 batchCop[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_key))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_64))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t3 keep order:false, stats:pseudo" ] } ] @@ -2552,5 +2582,31 @@ ] } ] + }, + { + "Name": "TestIssue23887", + "Cases": [ + { + "SQL": "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t", + "Plan": [ + "HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(2, test.t.b)", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─Projection(Probe) 10000.00 root 1->Column#27", + " └─Apply 10000.00 root CARTESIAN left outer join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 1.00 root 1->Column#25", + " └─Limit 1.00 root offset:0, count:1", + " └─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "1", + "1" + ] + } + ] } ] diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 7a734a9b9a4cc..087b32110e18f 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -168,8 +168,7 @@ "select /*+ hash_join(t1, t2 partition(p0)) */ * from t t1 join t t2 on t1.a = t2.a", "select /*+ use_index_merge(t partition(p0)) */ * from t where t.b = 1 or t.c = \"8\"", "select /*+ use_index_merge(t partition(p0, p1) primary, b) */ * from t where t.a = 1 or t.b = 2", - "select /*+ use_index(t partition(p0) b) */ * from t partition(p0, p1)", - "select /*+ read_from_storage(tikv[t partition(p0)], tiflash[t partition(p1, p2)]) */ * from t" + "select /*+ use_index(t partition(p0) b) */ * from t partition(p0, p1)" ] }, { @@ -285,5 +284,15 @@ "cases": [ "select count(*) from t join (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 1) v on v.id = t.id and v.v1 = t.value;" ] + }, + { + "name": "TestSequenceAsDataSource", + "cases": [ + "select 1 from s1", + "select count(1) from s1", + "select count(*) from s1", + "select sum(1) from s1", + "select count(1) as cnt from s1 union select count(1) as cnt from s2" + ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 47ce50c6ab717..7c735fcb5657c 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -879,19 +879,6 @@ " └─TableFullScan 10000.00 cop[tiflash] table:t, partition:p1 keep order:false, stats:pseudo" ], "Warn": null - }, - { - "SQL": "select /*+ read_from_storage(tikv[t partition(p0)], tiflash[t partition(p1, p2)]) */ * from t", - "Plan": [ - "PartitionUnion 30000.00 root ", - "├─TableReader 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", - "├─TableReader 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tiflash] table:t, partition:p1 keep order:false, stats:pseudo", - "└─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tiflash] table:t, partition:p2 keep order:false, stats:pseudo" - ], - "Warn": null } ] }, @@ -1533,5 +1520,49 @@ ] } ] + }, + { + "Name": "TestSequenceAsDataSource", + "Cases": [ + { + "SQL": "select 1 from s1", + "Plan": [ + "Projection 1.00 root 1->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select count(1) from s1", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select count(*) from s1", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select sum(1) from s1", + "Plan": [ + "StreamAgg 1.00 root funcs:sum(1)->Column#1", + "└─TableDual 1.00 root rows:1" + ] + }, + { + "SQL": "select count(1) as cnt from s1 union select count(1) as cnt from s2", + "Plan": [ + "HashAgg 2.00 root group by:Column#3, funcs:firstrow(Column#3)->Column#3", + "└─Union 2.00 root ", + " ├─StreamAgg 1.00 root funcs:count(1)->Column#1", + " │ └─TableDual 1.00 root rows:1", + " └─StreamAgg 1.00 root funcs:count(1)->Column#2", + " └─TableDual 1.00 root rows:1" + ] + } + ] } ] diff --git a/planner/core/testdata/partition_pruner_in.json b/planner/core/testdata/partition_pruner_in.json index 1f6c197ba61a9..9562379ac4574 100644 --- a/planner/core/testdata/partition_pruner_in.json +++ b/planner/core/testdata/partition_pruner_in.json @@ -28,7 +28,8 @@ "explain format = 'brief' select * from t8 where (a is null) or (a between 5 and 8)", "explain format = 'brief' select * from t8 where a between 5 and 12", "explain format = 'brief' select * from t8 where (a <= 10 and a >= 8) or (a <= 13 and a >= 11) or (a <= 16 and a >= 14)", - "explain format = 'brief' select * from t8 where a < 12 and a > 9" + "explain format = 'brief' select * from t8 where a < 12 and a > 9", + "explain format = 'brief' select * from t9" ] }, { @@ -501,5 +502,17 @@ "Pruner": "t1: p0,p1; t2: p0" } ] + }, + { + "name": "TestRangePartitionPredicatePruner", + "cases": [ + "explain format = 'brief' select * from t where a = 1", + "explain format = 'brief' select * from t where a = 2", + "explain format = 'brief' select * from t where a = 10", + "explain format = 'brief' select * from t where a = -1", + "explain format = 'brief' select * from t where a is NULL", + "explain format = 'brief' select * from t where a = 1.5", + "explain format = 'brief' select * from t where a in (2)" + ] } ] diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/testdata/partition_pruner_out.json index 0d71b5a0d2427..5593ab52bee74 100644 --- a/planner/core/testdata/partition_pruner_out.json +++ b/planner/core/testdata/partition_pruner_out.json @@ -182,7 +182,6 @@ "TableReader 250.00 root partition:p4,p5 data:Selection", "└─Selection 250.00 cop[tikv] ge(test_partition.t8.a, 10), le(test_partition.t8.a, 11)", " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" - ] }, { @@ -191,7 +190,6 @@ "TableReader 260.00 root partition:p0,p1,p2,p5 data:Selection", "└─Selection 260.00 cop[tikv] or(isnull(test_partition.t8.a), and(ge(test_partition.t8.a, 5), le(test_partition.t8.a, 8)))", " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" - ] }, { @@ -200,7 +198,6 @@ "TableReader 250.00 root partition:all data:Selection", "└─Selection 250.00 cop[tikv] ge(test_partition.t8.a, 5), le(test_partition.t8.a, 12)", " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" - ] }, { @@ -216,7 +213,14 @@ "Result": [ "TableReader 250.00 root partition:p4,p5 data:Selection", "└─Selection 250.00 cop[tikv] gt(test_partition.t8.a, 9), lt(test_partition.t8.a, 12)", - " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + " └─TableFullScan 10000.00 cop[tikv] table:t8 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t9", + "Result": [ + "TableReader 10000.00 root partition:p0,p1 data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t9 keep order:false, stats:pseudo" ] } ] @@ -2827,5 +2831,73 @@ ] } ] + }, + { + "Name": "TestRangePartitionPredicatePruner", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where a = 1", + "Result": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a = 2", + "Result": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a = 10", + "Result": [ + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(test_partition.t.a, 10)", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p_max keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a = -1", + "Result": [ + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(test_partition.t.a, -1)", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a is NULL", + "Result": [ + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] isnull(test_partition.t.a)", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a = 1.5", + "Result": [ + "PartitionUnion 32000.00 root ", + "├─TableReader 8000.00 root data:Selection", + "│ └─Selection 8000.00 cop[tikv] eq(cast(test_partition.t.a, decimal(20,0) BINARY), 1.5)", + "│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", + "├─TableReader 8000.00 root data:Selection", + "│ └─Selection 8000.00 cop[tikv] eq(cast(test_partition.t.a, decimal(20,0) BINARY), 1.5)", + "│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo", + "├─TableReader 8000.00 root data:Selection", + "│ └─Selection 8000.00 cop[tikv] eq(cast(test_partition.t.a, decimal(20,0) BINARY), 1.5)", + "│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo", + "└─TableReader 8000.00 root data:Selection", + " └─Selection 8000.00 cop[tikv] eq(cast(test_partition.t.a, decimal(20,0) BINARY), 1.5)", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p_max keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where a in (2)", + "Result": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + ] + } + ] } ] diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index ea257f22a332c..64a5c973e89cd 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -676,5 +676,28 @@ "select a from t2 where t2.a < (select t1.a from t1 where t1.b = t2.b and t1.a is null);", "select a from t2 where t2.a < (select t3.a from t3 where t3.a = t2.a);" ] + }, + { + "name": "TestEnumIndex", + "cases": [ + "select e from t where e = 'b'", + "select e from t where e != 'b'", + "select e from t where e > 'b'", + "select e from t where e >= 'b'", + "select e from t where e < 'b'", + "select e from t where e <= 'b'", + "select e from t where e = 2", + "select e from t where e != 2", + "select e from t where e > 2", + "select e from t where e >= 2", + "select e from t where e < 2", + "select e from t where e <= 2", + + // Out of range + "select e from t where e > ''", + "select e from t where e > 'd'", + "select e from t where e > -1", + "select e from t where e > 5" + ] } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 2b3c55eddd5bf..a0125e601c94a 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -2280,5 +2280,175 @@ "Result": null } ] + }, + { + "Name": "TestEnumIndex", + "Cases": [ + { + "SQL": "select e from t where e = 'b'", + "Plan": [ + "IndexReader 10.00 root index:IndexRangeScan", + "└─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(e) range:[\"b\",\"b\"], keep order:false, stats:pseudo" + ], + "Result": [ + "b" + ] + }, + { + "SQL": "select e from t where e != 'b'", + "Plan": [ + "IndexReader 20.00 root index:IndexRangeScan", + "└─IndexRangeScan 20.00 cop[tikv] table:t, index:idx(e) range:[\"c\",\"c\"], [\"a\",\"a\"], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "c" + ] + }, + { + "SQL": "select e from t where e > 'b'", + "Plan": [ + "IndexReader 10.00 root index:IndexRangeScan", + "└─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(e) range:[\"c\",\"c\"], keep order:false, stats:pseudo" + ], + "Result": [ + "c" + ] + }, + { + "SQL": "select e from t where e >= 'b'", + "Plan": [ + "IndexReader 20.00 root index:IndexRangeScan", + "└─IndexRangeScan 20.00 cop[tikv] table:t, index:idx(e) range:[\"c\",\"c\"], [\"b\",\"b\"], keep order:false, stats:pseudo" + ], + "Result": [ + "b", + "c" + ] + }, + { + "SQL": "select e from t where e < 'b'", + "Plan": [ + "IndexReader 10.00 root index:IndexRangeScan", + "└─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(e) range:[\"a\",\"a\"], keep order:false, stats:pseudo" + ], + "Result": [ + "a" + ] + }, + { + "SQL": "select e from t where e <= 'b'", + "Plan": [ + "IndexReader 20.00 root index:IndexRangeScan", + "└─IndexRangeScan 20.00 cop[tikv] table:t, index:idx(e) range:[\"b\",\"b\"], [\"a\",\"a\"], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "b" + ] + }, + { + "SQL": "select e from t where e = 2", + "Plan": [ + "IndexReader 10.00 root index:IndexRangeScan", + "└─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(e) range:[\"b\",\"b\"], keep order:false, stats:pseudo" + ], + "Result": [ + "b" + ] + }, + { + "SQL": "select e from t where e != 2", + "Plan": [ + "IndexReader 6656.67 root index:IndexRangeScan", + "└─IndexRangeScan 6656.67 cop[tikv] table:t, index:idx(e) range:[-inf,\"b\"), (\"b\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "c" + ] + }, + { + "SQL": "select e from t where e > 2", + "Plan": [ + "IndexReader 3333.33 root index:IndexRangeScan", + "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:(\"b\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "a" + ] + }, + { + "SQL": "select e from t where e >= 2", + "Plan": [ + "IndexReader 3333.33 root index:IndexRangeScan", + "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:[\"b\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "b" + ] + }, + { + "SQL": "select e from t where e < 2", + "Plan": [ + "IndexReader 3323.33 root index:IndexRangeScan", + "└─IndexRangeScan 3323.33 cop[tikv] table:t, index:idx(e) range:[-inf,\"b\"), keep order:false, stats:pseudo" + ], + "Result": [ + "c" + ] + }, + { + "SQL": "select e from t where e <= 2", + "Plan": [ + "IndexReader 3323.33 root index:IndexRangeScan", + "└─IndexRangeScan 3323.33 cop[tikv] table:t, index:idx(e) range:[-inf,\"b\"], keep order:false, stats:pseudo" + ], + "Result": [ + "b", + "c" + ] + }, + { + "SQL": "select e from t where e > ''", + "Plan": [ + "IndexReader 30.00 root index:IndexRangeScan", + "└─IndexRangeScan 30.00 cop[tikv] table:t, index:idx(e) range:[\"c\",\"c\"], [\"b\",\"b\"], [\"a\",\"a\"], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "b", + "c" + ] + }, + { + "SQL": "select e from t where e > 'd'", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Result": null + }, + { + "SQL": "select e from t where e > -1", + "Plan": [ + "IndexReader 3333.33 root index:IndexRangeScan", + "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:[\"\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "b", + "c" + ] + }, + { + "SQL": "select e from t where e > 5", + "Plan": [ + "IndexReader 3333.33 root index:IndexRangeScan", + "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:(\"a\",+inf], keep order:false, stats:pseudo" + ], + "Result": null + } + ] } ] diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index f976e65117138..5b7917e802aac 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -43,7 +43,8 @@ var dynamicPrivs = []string{ "SYSTEM_VARIABLES_ADMIN", "ROLE_ADMIN", "CONNECTION_ADMIN", - "RESTRICTED_TABLES_ADMIN", + "RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled + "RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled. } var dynamicPrivLock sync.Mutex diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 46deaab7ad550..816fe5a59d0bd 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1339,5 +1339,57 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeRestrictedTables(c *C) { mustExec(c, cloudAdminSe, "USE metrics_schema") mustExec(c, cloudAdminSe, "SELECT * FROM metrics_schema.uptime") mustExec(c, cloudAdminSe, "CREATE TABLE mysql.abcd (a int)") +} + +func (s *testPrivilegeSuite) TestSecurityEnhancedModeInfoschema(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE USER uroot1, uroot2, uroot3") + tk.MustExec("GRANT SUPER ON *.* to uroot1 WITH GRANT OPTION") // super not process + tk.MustExec("SET tidb_enable_dynamic_privileges=1") + tk.MustExec("GRANT SUPER, PROCESS, RESTRICTED_TABLES_ADMIN ON *.* to uroot2 WITH GRANT OPTION") + tk.Se.Auth(&auth.UserIdentity{ + Username: "uroot1", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + sem.Enable() + defer sem.Disable() + + // Even though we have super, we still can't read protected information from tidb_servers_info, cluster_* tables + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.tidb_servers_info WHERE ip IS NOT NULL`).Check(testkit.Rows("0")) + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.cluster_info WHERE status_address IS NOT NULL`).Check(testkit.Rows("0")) + // 36 = a UUID. Normally it is an IP address. + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.CLUSTER_STATEMENTS_SUMMARY WHERE length(instance) != 36`).Check(testkit.Rows("0")) + + // That is unless we have the RESTRICTED_TABLES_ADMIN privilege + tk.Se.Auth(&auth.UserIdentity{ + Username: "uroot2", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + + // flip from is NOT NULL etc + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.tidb_servers_info WHERE ip IS NULL`).Check(testkit.Rows("0")) + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.cluster_info WHERE status_address IS NULL`).Check(testkit.Rows("0")) + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.CLUSTER_STATEMENTS_SUMMARY WHERE length(instance) = 36`).Check(testkit.Rows("0")) +} + +func (s *testPrivilegeSuite) TestSecurityEnhancedModeStatusVars(c *C) { + // Without TiKV the status var list does not include tidb_gc_leader_desc + // So we can only test that the dynamic privilege is grantable. + // We will have to use an integration test to run SHOW STATUS LIKE 'tidb_gc_leader_desc' + // and verify if it appears. + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE USER unostatus, ustatus") + tk.MustExec("SET tidb_enable_dynamic_privileges=1") + tk.MustExec("GRANT RESTRICTED_STATUS_ADMIN ON *.* to ustatus") + tk.Se.Auth(&auth.UserIdentity{ + Username: "unostatus", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) } diff --git a/server/conn.go b/server/conn.go index dc785280c144c..29c87bd0dfd86 100644 --- a/server/conn.go +++ b/server/conn.go @@ -76,7 +76,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - tikverr "github.com/pingcap/tidb/store/tikv/error" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/arena" @@ -1569,7 +1569,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { retryable, err = cc.handleStmt(ctx, stmt, parserWarns, i == len(stmts)-1) if err != nil { _, allowTiFlashFallback := cc.ctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] - if allowTiFlashFallback && errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) && retryable { + if allowTiFlashFallback && errors.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) && retryable { // When the TiFlash server seems down, we append a warning to remind the user to check the status of the TiFlash // server and fallback to TiKV. warns := append(parserWarns, stmtctx.SQLWarn{Level: stmtctx.WarnLevelError, Err: err}) @@ -1870,10 +1870,10 @@ func (cc *clientConn) writeChunks(ctx context.Context, rs ResultSet, binary bool failpoint.Inject("fetchNextErr", func(value failpoint.Value) { switch value.(string) { case "firstNext": - failpoint.Return(firstNext, tikverr.ErrTiFlashServerTimeout) + failpoint.Return(firstNext, txndriver.ErrTiFlashServerTimeout) case "secondNext": if !firstNext { - failpoint.Return(firstNext, tikverr.ErrTiFlashServerTimeout) + failpoint.Return(firstNext, txndriver.ErrTiFlashServerTimeout) } } }) diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 63a9376e198dd..242b0df80fc83 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -50,7 +50,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/stmtctx" - tikverr "github.com/pingcap/tidb/store/tikv/error" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" @@ -198,7 +198,7 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e ctx = context.WithValue(ctx, util.ExecDetailsKey, &util.ExecDetails{}) retryable, err := cc.executePreparedStmtAndWriteResult(ctx, stmt, args, useCursor) _, allowTiFlashFallback := cc.ctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] - if allowTiFlashFallback && err != nil && errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) && retryable { + if allowTiFlashFallback && err != nil && errors.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) && retryable { // When the TiFlash server seems down, we append a warning to remind the user to check the status of the TiFlash // server and fallback to TiKV. prevErr := err diff --git a/server/http_handler.go b/server/http_handler.go index 9ac2de71474ae..67babd1f05e8d 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -157,7 +157,7 @@ func (t *tikvHandlerTool) getRegionIDByKey(encodedKey []byte) (uint64, error) { return keyLocation.Region.GetID(), nil } -func (t *tikvHandlerTool) getMvccByHandle(tb table.PhysicalTable, params map[string]string, values url.Values) (*mvccKV, error) { +func (t *tikvHandlerTool) getHandle(tb table.PhysicalTable, params map[string]string, values url.Values) (kv.Handle, error) { var handle kv.Handle if intHandleStr, ok := params[pHandle]; ok { if tb.Meta().IsCommonHandle { @@ -196,16 +196,7 @@ func (t *tikvHandlerTool) getMvccByHandle(tb table.PhysicalTable, params map[str return nil, errors.Trace(err) } } - encodedKey := tablecodec.EncodeRecordKey(tb.RecordPrefix(), handle) - data, err := t.GetMvccByEncodedKey(encodedKey) - if err != nil { - return nil, err - } - regionID, err := t.getRegionIDByKey(encodedKey) - if err != nil { - return nil, err - } - return &mvccKV{Key: strings.ToUpper(hex.EncodeToString(encodedKey)), Value: data, RegionID: regionID}, err + return handle, nil } func (t *tikvHandlerTool) getMvccByStartTs(startTS uint64, startKey, endKey kv.Key) (*mvccKV, error) { @@ -272,7 +263,7 @@ func (t *tikvHandlerTool) getMvccByStartTs(startTS uint64, startKey, endKey kv.K } } -func (t *tikvHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values, idxCols []*model.ColumnInfo, handleStr string) (*mvccKV, error) { +func (t *tikvHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values, idxCols []*model.ColumnInfo, handle kv.Handle) (*mvccKV, error) { sc := new(stmtctx.StatementContext) // HTTP request is not a database session, set timezone to UTC directly here. // See https://github.com/pingcap/tidb/blob/master/docs/tidb_http_api.md for more details. @@ -281,11 +272,7 @@ func (t *tikvHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values, if err != nil { return nil, errors.Trace(err) } - handle, err := strconv.ParseInt(handleStr, 10, 64) - if err != nil { - return nil, errors.Trace(err) - } - encodedKey, _, err := idx.GenIndexKey(sc, idxRow, kv.IntHandle(handle), nil) + encodedKey, _, err := idx.GenIndexKey(sc, idxRow, handle, nil) if err != nil { return nil, errors.Trace(err) } @@ -473,11 +460,10 @@ type mvccTxnHandler struct { } const ( - opMvccGetByHex = "hex" - opMvccGetByKey = "key" - opMvccGetByIdx = "idx" - opMvccGetByTxn = "txn" - opMvccGetByClusteredKey = "cls_key" + opMvccGetByHex = "hex" + opMvccGetByKey = "key" + opMvccGetByIdx = "idx" + opMvccGetByTxn = "txn" ) // ServeHTTP handles request of list a database or table's schemas. @@ -1583,7 +1569,7 @@ func (h mvccTxnHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { switch h.op { case opMvccGetByHex: data, err = h.handleMvccGetByHex(params) - case opMvccGetByIdx, opMvccGetByKey, opMvccGetByClusteredKey: + case opMvccGetByIdx, opMvccGetByKey: if req.URL == nil { err = errors.BadRequestf("Invalid URL") break @@ -1627,12 +1613,15 @@ func extractTableAndPartitionName(str string) (string, string) { func (h mvccTxnHandler) handleMvccGetByIdx(params map[string]string, values url.Values) (interface{}, error) { dbName := params[pDBName] tableName := params[pTableName] - handleStr := params[pHandle] t, err := h.getTable(dbName, tableName) if err != nil { return nil, errors.Trace(err) } + handle, err := h.getHandle(t, params, values) + if err != nil { + return nil, errors.Trace(err) + } var idxCols []*model.ColumnInfo var idx table.Index @@ -1648,7 +1637,7 @@ func (h mvccTxnHandler) handleMvccGetByIdx(params map[string]string, values url. if idx == nil { return nil, errors.NotFoundf("Index %s not found!", params[pIndexName]) } - return h.getMvccByIdxValue(idx, values, idxCols, handleStr) + return h.getMvccByIdxValue(idx, values, idxCols, handle) } func (h mvccTxnHandler) handleMvccGetByKey(params map[string]string, values url.Values) (interface{}, error) { @@ -1658,10 +1647,21 @@ func (h mvccTxnHandler) handleMvccGetByKey(params map[string]string, values url. if err != nil { return nil, errors.Trace(err) } - resp, err := h.getMvccByHandle(tb, params, values) + handle, err := h.getHandle(tb, params, values) + if err != nil { + return nil, err + } + + encodedKey := tablecodec.EncodeRecordKey(tb.RecordPrefix(), handle) + data, err := h.GetMvccByEncodedKey(encodedKey) + if err != nil { + return nil, err + } + regionID, err := h.getRegionIDByKey(encodedKey) if err != nil { return nil, err } + resp := &mvccKV{Key: strings.ToUpper(hex.EncodeToString(encodedKey)), Value: data, RegionID: regionID} if len(values.Get("decode")) == 0 { return resp, nil } diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 9d7fbb15d69f7..a7495b987f24a 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -274,8 +274,8 @@ func (ts *HTTPHandlerTestSuite) TestRegionsAPIForClusterIndex(c *C) { frameCnt++ } } - // Primary index is as the record frame, so frame count is 1. - c.Assert(frameCnt, Equals, 1) + // frameCnt = clustered primary key + secondary index(idx) = 2. + c.Assert(frameCnt, Equals, 2) c.Assert(resp.Body.Close(), IsNil) } } @@ -294,9 +294,11 @@ func (ts *HTTPHandlerTestSuite) TestRangesAPI(c *C) { err = decoder.Decode(&data) c.Assert(err, IsNil) c.Assert(data.TableName, Equals, "t") - c.Assert(len(data.Indices), Equals, 1) + c.Assert(len(data.Indices), Equals, 2) _, ok := data.Indices["PRIMARY"] c.Assert(ok, IsTrue) + _, ok = data.Indices["idx"] + c.Assert(ok, IsTrue) } func (ts *HTTPHandlerTestSuite) regionContainsTable(c *C, regionID uint64, tableID int64) bool { @@ -552,7 +554,7 @@ partition by range (a) err = txn2.Commit() c.Assert(err, IsNil) dbt.mustExec("drop table if exists t") - dbt.mustExec("create table t (a double, b varchar(20), c int, primary key(a,b) clustered)") + dbt.mustExec("create table t (a double, b varchar(20), c int, primary key(a,b) clustered, key idx(c))") dbt.mustExec("insert into t values(1.1,'111',1),(2.2,'222',2)") } @@ -956,6 +958,10 @@ func (ts *HTTPHandlerTestSuite) TestGetIndexMVCC(c *C) { c.Assert(err, IsNil) decodeKeyMvcc(resp.Body, c, true) + resp, err = ts.fetchStatus("/mvcc/index/tidb/t/idx?a=1.1&b=111&c=1") + c.Assert(err, IsNil) + decodeKeyMvcc(resp.Body, c, true) + // tests for wrong key resp, err = ts.fetchStatus("/mvcc/index/tidb/test/idx1/5?a=5&b=1") c.Assert(err, IsNil) diff --git a/server/http_status.go b/server/http_status.go index 034e71fcfd7b2..b385ea0c45890 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -153,10 +153,11 @@ func (s *Server) startHTTPServer() { } // HTTP path for get MVCC info - router.Handle("/mvcc/key/{db}/{table}", mvccTxnHandler{tikvHandlerTool, opMvccGetByClusteredKey}) + router.Handle("/mvcc/key/{db}/{table}", mvccTxnHandler{tikvHandlerTool, opMvccGetByKey}) router.Handle("/mvcc/key/{db}/{table}/{handle}", mvccTxnHandler{tikvHandlerTool, opMvccGetByKey}) router.Handle("/mvcc/txn/{startTS}/{db}/{table}", mvccTxnHandler{tikvHandlerTool, opMvccGetByTxn}) router.Handle("/mvcc/hex/{hexKey}", mvccTxnHandler{tikvHandlerTool, opMvccGetByHex}) + router.Handle("/mvcc/index/{db}/{table}/{index}", mvccTxnHandler{tikvHandlerTool, opMvccGetByIdx}) router.Handle("/mvcc/index/{db}/{table}/{index}/{handle}", mvccTxnHandler{tikvHandlerTool, opMvccGetByIdx}) // HTTP path for generate metric profile. diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index e7e8398845d88..ffa9ba22e5723 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -16,6 +16,7 @@ package session import ( "context" "fmt" + "sync" . "github.com/pingcap/check" "github.com/pingcap/parser" @@ -118,7 +119,7 @@ func globalVarsCount() int64 { func (s *testBootstrapSuite) bootstrapWithOnlyDDLWork(store kv.Storage, c *C) { ss := &session{ store: store, - parser: parser.New(), + parserPool: &sync.Pool{New: func() interface{} { return parser.New() }}, sessionVars: variable.NewSessionVars(), } ss.txn.init() diff --git a/session/schema_amender_test.go b/session/schema_amender_test.go index 91ff6a3e29cc4..ca05f4a74dbff 100644 --- a/session/schema_amender_test.go +++ b/session/schema_amender_test.go @@ -18,6 +18,7 @@ import ( "context" "sort" "strconv" + "sync" . "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/kvrpcpb" @@ -255,7 +256,7 @@ func (s *testSchemaAmenderSuite) TestAmendCollectAndGenMutations(c *C) { defer store.Close() se := &session{ store: store, - parser: parser.New(), + parserPool: &sync.Pool{New: func() interface{} { return parser.New() }}, sessionVars: variable.NewSessionVars(), } startStates := []model.SchemaState{model.StateNone, model.StateDeleteOnly, model.StateWriteOnly, model.StateWriteReorganization} diff --git a/session/session.go b/session/session.go index 2e78d6167d73c..13df91510f61a 100644 --- a/session/session.go +++ b/session/session.go @@ -68,6 +68,7 @@ import ( tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/telemetry" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" @@ -194,7 +195,7 @@ type session struct { store kv.Storage - parser *parser.Parser + parserPool *sync.Pool preparedPlanCache *kvcache.SimpleLRUCache @@ -509,6 +510,27 @@ func (s *session) doCommit(ctx context.Context) error { s.GetSessionVars().TxnCtx.IsExplicit && s.GetSessionVars().GuaranteeLinearizability) } + // Filter out the temporary table key-values. + if tables := s.sessionVars.TxnCtx.GlobalTemporaryTables; tables != nil { + memBuffer := s.txn.GetMemBuffer() + for tid := range tables { + seekKey := tablecodec.EncodeTablePrefix(tid) + endKey := tablecodec.EncodeTablePrefix(tid + 1) + iter, err := memBuffer.Iter(seekKey, endKey) + if err != nil { + return err + } + for iter.Valid() && iter.Key().HasPrefix(seekKey) { + if err = memBuffer.Delete(iter.Key()); err != nil { + return errors.Trace(err) + } + if err = iter.Next(); err != nil { + return errors.Trace(err) + } + } + } + } + return s.txn.Commit(tikvutil.SetSessionID(ctx, s.GetSessionVars().ConnectionID)) } @@ -977,29 +999,42 @@ func (s *session) GetGlobalSysVar(name string) (string, error) { } // SetGlobalSysVar implements GlobalVarAccessor.SetGlobalSysVar interface. -func (s *session) SetGlobalSysVar(name, value string) error { - if name == variable.TiDBSlowLogMasking { - name = variable.TiDBRedactLog +func (s *session) SetGlobalSysVar(name, value string) (err error) { + sv := variable.GetSysVar(name) + if sv == nil { + return variable.ErrUnknownSystemVar.GenWithStackByArgs(name) } + if value, err = sv.Validate(s.sessionVars, value, variable.ScopeGlobal); err != nil { + return err + } + if err = sv.SetGlobalFromHook(s.sessionVars, value, false); err != nil { + return err + } + + return s.updateGlobalSysVar(sv, value) +} + +// SetGlobalSysVarOnly updates the sysvar, but does not call the validation function or update aliases. +// This is helpful to prevent duplicate warnings being appended from aliases, or recursion. +func (s *session) SetGlobalSysVarOnly(name, value string) (err error) { sv := variable.GetSysVar(name) if sv == nil { return variable.ErrUnknownSystemVar.GenWithStackByArgs(name) } - var sVal string - var err error - sVal, err = sv.Validate(s.sessionVars, value, variable.ScopeGlobal) - if err != nil { + if err = sv.SetGlobalFromHook(s.sessionVars, value, true); err != nil { return err } - name = strings.ToLower(name) + return s.updateGlobalSysVar(sv, value) +} + +func (s *session) updateGlobalSysVar(sv *variable.SysVar, value string) error { // update mysql.tidb if required. - if s.varFromTiDBTable(name) { - if err = s.setTiDBTableValue(name, sVal); err != nil { + if s.varFromTiDBTable(sv.Name) { + if err := s.setTiDBTableValue(sv.Name, value); err != nil { return err } } - variable.CheckDeprecationSetSystemVar(s.sessionVars, name) - stmt, err := s.ParseWithParams(context.TODO(), "REPLACE %n.%n VALUES (%?, %?)", mysql.SystemDB, mysql.GlobalVariablesTable, name, sVal) + stmt, err := s.ParseWithParams(context.TODO(), "REPLACE %n.%n VALUES (%?, %?)", mysql.SystemDB, mysql.GlobalVariablesTable, sv.Name, value) if err != nil { return err } @@ -1089,9 +1124,12 @@ func (s *session) ParseSQL(ctx context.Context, sql, charset, collation string) defer span1.Finish() } defer trace.StartRegion(ctx, "ParseSQL").End() - s.parser.SetSQLMode(s.sessionVars.SQLMode) - s.parser.SetParserConfig(s.sessionVars.BuildParserConfig()) - return s.parser.Parse(sql, charset, collation) + + p := s.parserPool.Get().(*parser.Parser) + defer s.parserPool.Put(p) + p.SetSQLMode(s.sessionVars.SQLMode) + p.SetParserConfig(s.sessionVars.BuildParserConfig()) + return p.Parse(sql, charset, collation) } func (s *session) SetProcessInfo(sql string, t time.Time, command byte, maxExecutionTime uint64) { @@ -1714,7 +1752,7 @@ func (s *session) cachedPlanExec(ctx context.Context, s.txn.changeToInvalid() case *plannercore.Update: s.PrepareTSFuture(ctx) - stmtCtx.Priority = tikvstore.PriorityHigh + stmtCtx.Priority = kv.PriorityHigh resultSet, err = runStmt(ctx, s, stmt) case nil: // cache is invalid @@ -2102,7 +2140,7 @@ func CreateSessionWithOpt(store kv.Storage, opt *Opt) (Session, error) { } privilege.BindPrivilegeManager(s, pm) - sessionBindHandle := bindinfo.NewSessionBindHandle(s.parser) + sessionBindHandle := bindinfo.NewSessionBindHandle(parser.New()) s.SetValue(bindinfo.SessionBindInfoKeyType, sessionBindHandle) // Add stats collector, and it will be freed by background stats worker // which periodically updates stats using the collected data. @@ -2341,7 +2379,7 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) { } s := &session{ store: store, - parser: parser.New(), + parserPool: &sync.Pool{New: func() interface{} { return parser.New() }}, sessionVars: variable.NewSessionVars(), ddlOwnerChecker: dom.DDL().OwnerManager(), client: store.GetClient(), @@ -2363,7 +2401,7 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) { s.sessionVars.BinlogClient = binloginfo.GetPumpsClient() s.txn.init() - sessionBindHandle := bindinfo.NewSessionBindHandle(s.parser) + sessionBindHandle := bindinfo.NewSessionBindHandle(parser.New()) s.SetValue(bindinfo.SessionBindInfoKeyType, sessionBindHandle) return s, nil } @@ -2375,7 +2413,7 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) { func CreateSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, error) { s := &session{ store: store, - parser: parser.New(), + parserPool: &sync.Pool{New: func() interface{} { return parser.New() }}, sessionVars: variable.NewSessionVars(), client: store.GetClient(), mppClient: store.GetMPPClient(), @@ -2610,7 +2648,7 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { // `collation_server` is related to `character_set_server`, set `character_set_server` will also set `collation_server`. // We have to make sure we set the `collation_server` with right value. if _, ok := vars.GetSystemVar(varName); !ok || varName == variable.CollationServer { - err = variable.SetSessionSystemVar(s.sessionVars, varName, varVal) + err = vars.SetSystemVar(varName, varVal) if err != nil { return err } @@ -2709,7 +2747,7 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionctx.StalenessTxnOption) error { if s.txn.Valid() { txnID := s.txn.StartTS() - txnScope := s.txn.GetUnionStore().GetOption(tikvstore.TxnScope).(string) + txnScope := s.txn.GetOption(tikvstore.TxnScope).(string) err := s.CommitTxn(ctx) if err != nil { return err diff --git a/session/session_test.go b/session/session_test.go index 9ce875fa07868..84442a8a16956 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -2624,13 +2624,13 @@ func (s *testSessionSuite3) TestSetTransactionIsolationOneShot(c *C) { // Check isolation level is set to read committed. ctx := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { - c.Assert(req.IsolationLevel, Equals, tikvstore.SI) + c.Assert(req.IsolationLevel, Equals, kv.SI) }) tk.Se.Execute(ctx, "select * from t where k = 1") // Check it just take effect for one time. ctx = context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { - c.Assert(req.IsolationLevel, Equals, tikvstore.SI) + c.Assert(req.IsolationLevel, Equals, kv.SI) }) tk.Se.Execute(ctx, "select * from t where k = 1") @@ -3746,7 +3746,7 @@ func (s *testSessionSuite2) TestMemoryUsageAlarmVariable(c *C) { err = tk.ExecToErr("set @@session.tidb_memory_usage_alarm_ratio=-1") c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_memory_usage_alarm_ratio' can't be set to the value of '-1'") err = tk.ExecToErr("set @@global.tidb_memory_usage_alarm_ratio=0.8") - c.Assert(err.Error(), Equals, "Variable 'tidb_memory_usage_alarm_ratio' is a SESSION variable and can't be used with SET GLOBAL") + c.Assert(err.Error(), Equals, "[variable:1228]Variable 'tidb_memory_usage_alarm_ratio' is a SESSION variable and can't be used with SET GLOBAL") } func (s *testSessionSuite2) TestSelectLockInShare(c *C) { @@ -4125,6 +4125,43 @@ func (s *testSessionSerialSuite) TestRemovedSysVars(c *C) { c.Assert(err.Error(), Equals, "[variable:1193]Unknown system variable 'bogus_var'") } +func (s *testSessionSerialSuite) TestCorrectScopeError(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeNone, Name: "sv_none", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal, Name: "sv_global", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeSession, Name: "sv_session", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal | variable.ScopeSession, Name: "sv_both", Value: "acdc"}) + + // check set behavior + + // none + _, err := tk.Exec("SET sv_none='acdc'") + c.Assert(err.Error(), Equals, "[variable:1238]Variable 'sv_none' is a read only variable") + _, err = tk.Exec("SET GLOBAL sv_none='acdc'") + c.Assert(err.Error(), Equals, "[variable:1238]Variable 'sv_none' is a read only variable") + + // global + tk.MustExec("SET GLOBAL sv_global='acdc'") + _, err = tk.Exec("SET sv_global='acdc'") + c.Assert(err.Error(), Equals, "[variable:1229]Variable 'sv_global' is a GLOBAL variable and should be set with SET GLOBAL") + + // session + _, err = tk.Exec("SET GLOBAL sv_session='acdc'") + c.Assert(err.Error(), Equals, "[variable:1228]Variable 'sv_session' is a SESSION variable and can't be used with SET GLOBAL") + tk.MustExec("SET sv_session='acdc'") + + // both + tk.MustExec("SET GLOBAL sv_both='acdc'") + tk.MustExec("SET sv_both='acdc'") + + // unregister + variable.UnregisterSysVar("sv_none") + variable.UnregisterSysVar("sv_global") + variable.UnregisterSysVar("sv_session") + variable.UnregisterSysVar("sv_both") +} + func (s *testSessionSerialSuite) TestTiKVSystemVars(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) @@ -4243,3 +4280,24 @@ func (s *testSessionSerialSuite) TestParseWithParams(c *C) { c.Assert(err, IsNil) c.Assert(sb.String(), Equals, "SELECT 3") } + +func (s *testSessionSuite3) TestGlobalTemporaryTable(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create global temporary table g_tmp (a int primary key, b int, c int, index i_b(b)) on commit delete rows") + tk.MustExec("begin") + tk.MustExec("insert into g_tmp values (3, 3, 3)") + tk.MustExec("insert into g_tmp values (4, 7, 9)") + + // Cover table scan. + tk.MustQuery("select * from g_tmp").Check(testkit.Rows("3 3 3", "4 7 9")) + // Cover index reader. + tk.MustQuery("select b from g_tmp where b > 3").Check(testkit.Rows("7")) + // Cover index lookup. + tk.MustQuery("select c from g_tmp where b = 3").Check(testkit.Rows("3")) + // Cover point get. + tk.MustQuery("select * from g_tmp where a = 3").Check(testkit.Rows("3 3 3")) + tk.MustExec("commit") + + // The global temporary table data is discard after the transaction commit. + tk.MustQuery("select * from g_tmp").Check(testkit.Rows()) +} diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index ca01fe0fc90e3..2dfca57d73f4c 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -269,25 +269,6 @@ func (s *testBinlogSuite) TestBinlog(c *C) { binlog.MutationType_Insert, }) - // Cannot create common clustered index table when binlog client exists. - errMsg := "[ddl:8200]Cannot create clustered index table when the binlog is ON" - tk.MustGetErrMsg("create table local_clustered_index (c1 varchar(255) primary key clustered);", errMsg) - // Create int clustered index table when binlog client exists. - tk.MustExec("create table local_clustered_index (c1 bigint primary key clustered);") - tk.MustQuery("select tidb_pk_type from information_schema.tables where table_name = 'local_clustered_index' and table_schema = 'test';"). - Check(testkit.Rows("CLUSTERED")) - tk.MustExec("drop table if exists local_clustered_index;") - // Test common clustered index tables will not write binlog. - tk.Se.GetSessionVars().BinlogClient = nil - tk.MustExec("create table local_clustered_index (c1 varchar(255) primary key clustered);") - tk.MustQuery("select tidb_pk_type from information_schema.tables where table_name = 'local_clustered_index' and table_schema = 'test';"). - Check(testkit.Rows("CLUSTERED")) - tk.Se.GetSessionVars().BinlogClient = s.client - // This statement should not write binlog. - tk.MustExec(`insert into local_clustered_index values ("aaaaaa")`) - prewriteVal = getLatestBinlogPrewriteValue(c, pump) - c.Assert(len(prewriteVal.Mutations), Equals, 0) - checkBinlogCount(c, pump) pump.mu.Lock() diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 9d57636e7d18f..c4402e25e19a4 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -202,11 +202,11 @@ type StmtCacheKey int const ( // StmtNowTsCacheKey is a variable for now/current_timestamp calculation/cache for one stmt. StmtNowTsCacheKey StmtCacheKey = iota - // StmtNowTsCacheKey is a variable for resolveTs calculation/cache for one stmt. + // StmtResolveTsCacheKey is a variable for resolveTs calculation/cache for one stmt. StmtResolveTsCacheKey ) -// GetFromStmtCache gets the cached value of the given key if it exists, otherwise will store the value. +// GetOrStoreStmtCache gets the cached value of the given key if it exists, otherwise stores the value. func (sc *StatementContext) GetOrStoreStmtCache(key StmtCacheKey, value interface{}) interface{} { if sc.stmtCache == nil { sc.stmtCache = make(map[StmtCacheKey]interface{}) diff --git a/sessionctx/variable/error.go b/sessionctx/variable/error.go index c5e14eeac83a2..c3029447b00b7 100644 --- a/sessionctx/variable/error.go +++ b/sessionctx/variable/error.go @@ -34,6 +34,8 @@ var ( ErrMaxPreparedStmtCountReached = dbterror.ClassVariable.NewStd(mysql.ErrMaxPreparedStmtCountReached) ErrUnsupportedIsolationLevel = dbterror.ClassVariable.NewStd(mysql.ErrUnsupportedIsolationLevel) errUnknownSystemVariable = dbterror.ClassVariable.NewStd(mysql.ErrUnknownSystemVariable) + errGlobalVariable = dbterror.ClassVariable.NewStd(mysql.ErrGlobalVariable) + errLocalVariable = dbterror.ClassVariable.NewStd(mysql.ErrLocalVariable) errValueNotSupportedWhen = dbterror.ClassVariable.NewStdErr(mysql.ErrNotSupportedYet, pmysql.Message("%s = OFF is not supported when %s = ON", nil)) // ErrFunctionsNoopImpl is an error to say the behavior is protected by the tidb_enable_noop_functions sysvar. // This is copied from expression.ErrFunctionsNoopImpl to prevent circular dependencies. diff --git a/sessionctx/variable/mock_globalaccessor.go b/sessionctx/variable/mock_globalaccessor.go index b2b47c8185dbf..525ce9fa9efff 100644 --- a/sessionctx/variable/mock_globalaccessor.go +++ b/sessionctx/variable/mock_globalaccessor.go @@ -35,3 +35,8 @@ func (m *MockGlobalAccessor) GetGlobalSysVar(name string) (string, error) { func (m *MockGlobalAccessor) SetGlobalSysVar(name string, value string) error { panic("not supported") } + +// SetGlobalSysVarOnly implements GlobalVarAccessor.SetGlobalSysVarOnly interface. +func (m *MockGlobalAccessor) SetGlobalSysVarOnly(name string, value string) error { + panic("not supported") +} diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index 4cc78799918f2..a3382fc1159d1 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -26,10 +26,10 @@ var noopSysVars = []*SysVar{ // It is unsafe to pretend that any variation of "read only" is enabled when the server // does not support it. It is possible that these features will be supported in future, // but until then... - {Scope: ScopeGlobal | ScopeSession, Name: TxReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TxReadOnly, Value: Off, Type: TypeBool, Aliases: []string{TransactionReadOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, - {Scope: ScopeGlobal | ScopeSession, Name: TransactionReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TransactionReadOnly, Value: Off, Type: TypeBool, Aliases: []string{TxReadOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, {Scope: ScopeGlobal, Name: OfflineMode, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 6e75bd7ca010c..1554f3c429d65 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -172,6 +172,8 @@ type TransactionContext struct { // TableDeltaMap lock to prevent potential data race tdmLock sync.Mutex + + GlobalTemporaryTables map[int64]struct{} } // GetShard returns the shard prefix for the next `count` rowids. @@ -962,7 +964,7 @@ func NewSessionVars() *SessionVars { OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, RetryLimit: DefTiDBRetryLimit, DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, - DDLReorgPriority: tikvstore.PriorityLow, + DDLReorgPriority: kv.PriorityLow, allowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, preferRangeScan: DefOptPreferRangeScan, CorrelationThreshold: DefOptCorrelationThreshold, @@ -1308,13 +1310,13 @@ func (s *SessionVars) setDDLReorgPriority(val string) { val = strings.ToLower(val) switch val { case "priority_low": - s.DDLReorgPriority = tikvstore.PriorityLow + s.DDLReorgPriority = kv.PriorityLow case "priority_normal": - s.DDLReorgPriority = tikvstore.PriorityNormal + s.DDLReorgPriority = kv.PriorityNormal case "priority_high": - s.DDLReorgPriority = tikvstore.PriorityHigh + s.DDLReorgPriority = kv.PriorityHigh default: - s.DDLReorgPriority = tikvstore.PriorityLow + s.DDLReorgPriority = kv.PriorityLow } } @@ -1375,11 +1377,7 @@ func (s *SessionVars) ClearStmtVars() { // i.e. oN / on / 1 => ON func (s *SessionVars) SetSystemVar(name string, val string) error { sv := GetSysVar(name) - if err := sv.SetSessionFromHook(s, val); err != nil { - return err - } - s.systems[name] = val - return nil + return sv.SetSessionFromHook(s, val) } // GetReadableTxnMode returns the session variable TxnMode but rewrites it to "OPTIMISTIC" when it's empty. diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 0b6b41cb37b86..c17238c9ae9c5 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -106,24 +106,81 @@ type SysVar struct { AllowEmptyAll bool // AllowAutoValue means that the special value "-1" is permitted, even when outside of range. AllowAutoValue bool - // Validation is a callback after the type validation has been performed + // Validation is a callback after the type validation has been performed, but before the Set function Validation func(*SessionVars, string, string, ScopeFlag) (string, error) - // SetSession is called after validation + // SetSession is called after validation but before updating systems[]. It also doubles as an Init function + // and will be called on all variables in builtinGlobalVariable, regardless of their scope. SetSession func(*SessionVars, string) error + // SetGlobal is called after validation + SetGlobal func(*SessionVars, string) error // IsHintUpdatable indicate whether it's updatable via SET_VAR() hint (optional) IsHintUpdatable bool // Hidden means that it still responds to SET but doesn't show up in SHOW VARIABLES Hidden bool + // Aliases is a list of sysvars that should also be updated when this sysvar is updated. + // Updating aliases calls the SET function of the aliases, but does not update their aliases (preventing SET recursion) + Aliases []string } // SetSessionFromHook calls the SetSession func if it exists. func (sv *SysVar) SetSessionFromHook(s *SessionVars, val string) error { if sv.SetSession != nil { - return sv.SetSession(s, val) + if err := sv.SetSession(s, val); err != nil { + return err + } + } + s.systems[sv.Name] = val + + // Call the Set function on all the aliases for this sysVar + // Skipping the validation function, and not calling aliases of + // aliases. By skipping the validation function it means that things + // like duplicate warnings should not appear. + + if sv.Aliases != nil { + for _, aliasName := range sv.Aliases { + aliasSv := GetSysVar(aliasName) + if aliasSv.SetSession != nil { + if err := aliasSv.SetSession(s, val); err != nil { + return err + } + } + s.systems[aliasSv.Name] = val + } + } + return nil +} + +// SetGlobalFromHook calls the SetGlobal func if it exists. +func (sv *SysVar) SetGlobalFromHook(s *SessionVars, val string, skipAliases bool) error { + if sv.SetGlobal != nil { + return sv.SetGlobal(s, val) + } + + // Call the SetGlobalSysVarOnly function on all the aliases for this sysVar + // which skips the validation function and when SetGlobalFromHook is called again + // it will be with skipAliases=true. This helps break recursion because + // most aliases are reciprocal. + + if !skipAliases && sv.Aliases != nil { + for _, aliasName := range sv.Aliases { + if err := s.GlobalVarsAccessor.SetGlobalSysVarOnly(aliasName, val); err != nil { + return err + } + } } return nil } +// HasSessionScope returns true if the scope for the sysVar includes session. +func (sv *SysVar) HasSessionScope() bool { + return sv.Scope&ScopeSession != 0 +} + +// HasGlobalScope returns true if the scope for the sysVar includes global. +func (sv *SysVar) HasGlobalScope() bool { + return sv.Scope&ScopeGlobal != 0 +} + // Validate checks if system variable satisfies specific restriction. func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (string, error) { // Normalize the value and apply validation based on type. @@ -141,10 +198,17 @@ func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (st // validateFromType provides automatic validation based on the SysVar's type func (sv *SysVar) validateFromType(vars *SessionVars, value string, scope ScopeFlag) (string, error) { - // Some sysvars are read-only. Attempting to set should always fail. + // Check that the scope is correct and return the appropriate error message. if sv.ReadOnly || sv.Scope == ScopeNone { - return value, ErrIncorrectScope.GenWithStackByArgs(sv.Name, "read only") + return value, ErrIncorrectScope.FastGenByArgs(sv.Name, "read only") } + if scope == ScopeGlobal && !sv.HasGlobalScope() { + return value, errLocalVariable.FastGenByArgs(sv.Name) + } + if scope == ScopeSession && !sv.HasSessionScope() { + return value, errGlobalVariable.FastGenByArgs(sv.Name) + } + // The string "DEFAULT" is a special keyword in MySQL, which restores // the compiled sysvar value. In which case we can skip further validation. if strings.EqualFold(value, "DEFAULT") { @@ -434,7 +498,6 @@ func init() { for _, v := range noopSysVars { RegisterSysVar(v) } - initSynonymsSysVariables() } var defaultSysVars = []*SysVar{ @@ -543,27 +606,14 @@ var defaultSysVars = []*SysVar{ return checkCharacterSet(normalizedValue, "") }}, {Scope: ScopeNone, Name: VersionComment, Value: "TiDB Server (Apache License 2.0) " + versioninfo.TiDBEdition + " Edition, MySQL 5.7 compatible"}, - {Scope: ScopeGlobal | ScopeSession, Name: TxnIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if normalizedValue == "SERIALIZABLE" || normalizedValue == "READ-UNCOMMITTED" { - if skipIsolationLevelCheck, err := GetSessionSystemVar(vars, TiDBSkipIsolationLevelCheck); err != nil { - return normalizedValue, err - } else if !TiDBOptOn(skipIsolationLevelCheck) { - return normalizedValue, ErrUnsupportedIsolationLevel.GenWithStackByArgs(normalizedValue) - } - } - return normalizedValue, nil + {Scope: ScopeGlobal | ScopeSession, Name: TxnIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, Aliases: []string{TransactionIsolation}, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + // MySQL appends a warning here for tx_isolation is deprecated + // TiDB doesn't currently, but may in future. It is still commonly used by applications + // So it might be noisy to do so. + return checkIsolationLevel(vars, normalizedValue, originalValue, scope) }}, - {Scope: ScopeGlobal | ScopeSession, Name: TransactionIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if normalizedValue == "SERIALIZABLE" || normalizedValue == "READ-UNCOMMITTED" { - returnErr := ErrUnsupportedIsolationLevel.GenWithStackByArgs(normalizedValue) - if skipIsolationLevelCheck, err := GetSessionSystemVar(vars, TiDBSkipIsolationLevelCheck); err != nil { - return normalizedValue, err - } else if !TiDBOptOn(skipIsolationLevelCheck) { - return normalizedValue, returnErr - } - vars.StmtCtx.AppendWarning(returnErr) - } - return normalizedValue, nil + {Scope: ScopeGlobal | ScopeSession, Name: TransactionIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, Aliases: []string{TxnIsolation}, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + return checkIsolationLevel(vars, normalizedValue, originalValue, scope) }}, {Scope: ScopeGlobal | ScopeSession, Name: CollationConnection, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCollation(vars, normalizedValue, originalValue, scope) @@ -785,10 +835,16 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupConcurrency, Value: strconv.Itoa(DefIndexLookupConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.indexLookupConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBIndexLookupConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupJoinConcurrency, Value: strconv.Itoa(DefIndexLookupJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.indexLookupJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBIndexLookupJoinConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexSerialScanConcurrency, Value: strconv.Itoa(DefIndexSerialScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { s.indexSerialScanConcurrency = tidbOptPositiveInt32(val, DefIndexSerialScanConcurrency) @@ -853,26 +909,44 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TIDBMemQuotaHashJoin, Value: strconv.FormatInt(DefTiDBMemQuotaHashJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaHashJoin = tidbOptInt64(val, DefTiDBMemQuotaHashJoin) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaHashJoin, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TIDBMemQuotaMergeJoin, Value: strconv.FormatInt(DefTiDBMemQuotaMergeJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaMergeJoin = tidbOptInt64(val, DefTiDBMemQuotaMergeJoin) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaMergeJoin, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TIDBMemQuotaSort, Value: strconv.FormatInt(DefTiDBMemQuotaSort, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaSort = tidbOptInt64(val, DefTiDBMemQuotaSort) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaSort, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TIDBMemQuotaTopn, Value: strconv.FormatInt(DefTiDBMemQuotaTopn, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaTopn = tidbOptInt64(val, DefTiDBMemQuotaTopn) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaTopn, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupReader, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupReader, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaIndexLookupReader = tidbOptInt64(val, DefTiDBMemQuotaIndexLookupReader) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaIndexLookupReader, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupJoin, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaIndexLookupJoin = tidbOptInt64(val, DefTiDBMemQuotaIndexLookupJoin) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TIDBMemQuotaIndexLookupJoin, TIDBMemQuotaQuery) + return normalizedValue, nil }}, {Scope: ScopeSession, Name: TiDBEnableStreaming, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableStreaming = TiDBOptOn(val) @@ -883,14 +957,7 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeSession, Name: TxnIsolationOneShot, Value: "", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if normalizedValue == "SERIALIZABLE" || normalizedValue == "READ-UNCOMMITTED" { - if skipIsolationLevelCheck, err := GetSessionSystemVar(vars, TiDBSkipIsolationLevelCheck); err != nil { - return normalizedValue, err - } else if !TiDBOptOn(skipIsolationLevelCheck) { - return normalizedValue, ErrUnsupportedIsolationLevel.GenWithStackByArgs(normalizedValue) - } - } - return normalizedValue, nil + return checkIsolationLevel(vars, normalizedValue, originalValue, scope) }, SetSession: func(s *SessionVars, val string) error { s.txnIsolationLevelOneShot.state = oneShotSet s.txnIsolationLevelOneShot.value = val @@ -907,30 +974,52 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashJoinConcurrency, Value: strconv.Itoa(DefTiDBHashJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.hashJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBHashJoinConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBProjectionConcurrency, Value: strconv.Itoa(DefTiDBProjectionConcurrency), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.projectionConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBProjectionConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggPartialConcurrency, Value: strconv.Itoa(DefTiDBHashAggPartialConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.hashAggPartialConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBHashAggPartialConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggFinalConcurrency, Value: strconv.Itoa(DefTiDBHashAggFinalConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.hashAggFinalConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBHashAggFinalConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBWindowConcurrency, Value: strconv.Itoa(DefTiDBWindowConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.windowConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBWindowConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBMergeJoinConcurrency, Value: strconv.Itoa(DefTiDBMergeJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.mergeJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBMergeJoinConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStreamAggConcurrency, Value: strconv.Itoa(DefTiDBStreamAggConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.streamAggConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBStreamAggConcurrency, TiDBExecutorConcurrency) + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableParallelApply, Value: BoolToOnOff(DefTiDBEnableParallelApply), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableParallelApply = TiDBOptOn(val) @@ -1287,11 +1376,11 @@ var defaultSysVars = []*SysVar{ s.PartitionPruneMode.Store(strings.ToLower(strings.TrimSpace(val))) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSlowLogMasking, Value: BoolToOnOff(DefTiDBRedactLog), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { - // TiDBSlowLogMasking is deprecated and a alias of TiDBRedactLog. - return s.SetSystemVar(TiDBRedactLog, val) + {Scope: ScopeGlobal | ScopeSession, Name: TiDBSlowLogMasking, Value: BoolToOnOff(DefTiDBRedactLog), Aliases: []string{TiDBRedactLog}, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + appendDeprecationWarning(vars, TiDBSlowLogMasking, TiDBRedactLog) + return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: BoolToOnOff(DefTiDBRedactLog), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: BoolToOnOff(DefTiDBRedactLog), Aliases: []string{TiDBSlowLogMasking}, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableRedactLog = TiDBOptOn(val) errors.RedactLogEnabled.Store(s.EnableRedactLog) return nil @@ -1317,7 +1406,7 @@ var defaultSysVars = []*SysVar{ s.GuaranteeLinearizability = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Hidden: true, Type: TypeInt, MinValue: 1, MaxValue: 2, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Hidden: true, Type: TypeInt, MinValue: 1, MaxValue: 3, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if normalizedValue == "2" && FeedbackProbability.Load() > 0 { var original string var err error @@ -1367,20 +1456,6 @@ var defaultSysVars = []*SysVar{ // It's initialized in init() in feedback.go to solve import cycle. var FeedbackProbability *atomic2.Float64 -// SynonymsSysVariables is synonyms of system variables. -var SynonymsSysVariables = map[string][]string{} - -func addSynonymsSysVariables(synonyms ...string) { - for _, s := range synonyms { - SynonymsSysVariables[s] = synonyms - } -} - -func initSynonymsSysVariables() { - addSynonymsSysVariables(TxnIsolation, TransactionIsolation) - addSynonymsSysVariables(TxReadOnly, TransactionReadOnly) -} - // SetNamesVariables is the system variable names related to set names statements. var SetNamesVariables = []string{ CharacterSetClient, @@ -1661,4 +1736,6 @@ type GlobalVarAccessor interface { GetGlobalSysVar(name string) (string, error) // SetGlobalSysVar sets the global system variable name to value. SetGlobalSysVar(name string, value string) error + // SetGlobalSysVarOnly sets the global system variable without calling the validation function or updating aliases. + SetGlobalSysVarOnly(name string, value string) error } diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index d954c753e0088..4029072412977 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -15,6 +15,7 @@ package variable import ( "fmt" + "strings" "testing" . "github.com/pingcap/check" @@ -168,3 +169,152 @@ func (*testSysVarSuite) TestEnumValidation(c *C) { c.Assert(err, IsNil) c.Assert(val, Equals, "AUTO") } + +func (*testSysVarSuite) TestSynonyms(c *C) { + sysVar := GetSysVar(TxnIsolation) + c.Assert(sysVar, NotNil) + + vars := NewSessionVars() + + // It does not permit SERIALIZABLE by default. + _, err := sysVar.Validate(vars, "SERIALIZABLE", ScopeSession) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[variable:8048]The isolation level 'SERIALIZABLE' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error") + + // Enable Skip isolation check + c.Assert(GetSysVar(TiDBSkipIsolationLevelCheck).SetSessionFromHook(vars, "ON"), IsNil) + + // Serializable is now permitted. + _, err = sysVar.Validate(vars, "SERIALIZABLE", ScopeSession) + c.Assert(err, IsNil) + + // Currently TiDB returns a warning because of SERIALIZABLE, but in future + // it may also return a warning because TxnIsolation is deprecated. + + warn := vars.StmtCtx.GetWarnings()[0].Err + c.Assert(warn.Error(), Equals, "[variable:8048]The isolation level 'SERIALIZABLE' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error") + + c.Assert(sysVar.SetSessionFromHook(vars, "SERIALIZABLE"), IsNil) + + // When we set TxnIsolation, it also updates TransactionIsolation. + c.Assert(vars.systems[TxnIsolation], Equals, "SERIALIZABLE") + c.Assert(vars.systems[TransactionIsolation], Equals, vars.systems[TxnIsolation]) +} + +func (*testSysVarSuite) TestDeprecation(c *C) { + sysVar := GetSysVar(TiDBIndexLookupConcurrency) + c.Assert(sysVar, NotNil) + + vars := NewSessionVars() + + _, err := sysVar.Validate(vars, "1234", ScopeSession) + c.Assert(err, IsNil) + + // There was no error but there is a deprecation warning. + warn := vars.StmtCtx.GetWarnings()[0].Err + c.Assert(warn.Error(), Equals, "[variable:1287]'tidb_index_lookup_concurrency' is deprecated and will be removed in a future release. Please use tidb_executor_concurrency instead") +} + +func (*testSysVarSuite) TestScope(c *C) { + sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + c.Assert(sv.HasSessionScope(), IsTrue) + c.Assert(sv.HasGlobalScope(), IsTrue) + + sv = SysVar{Scope: ScopeGlobal, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + c.Assert(sv.HasSessionScope(), IsFalse) + c.Assert(sv.HasGlobalScope(), IsTrue) + + sv = SysVar{Scope: ScopeNone, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + c.Assert(sv.HasSessionScope(), IsFalse) + c.Assert(sv.HasGlobalScope(), IsFalse) +} + +func (*testSysVarSuite) TestBuiltInCase(c *C) { + // All Sysvars should have lower case names. + // This tests builtins. + for name := range GetSysVars() { + c.Assert(name, Equals, strings.ToLower(name)) + } +} + +func (*testSysVarSuite) TestSQLSelectLimit(c *C) { + sv := GetSysVar(SQLSelectLimit) + vars := NewSessionVars() + val, err := sv.Validate(vars, "-10", ScopeSession) + c.Assert(err, IsNil) // it has autoconvert out of range. + c.Assert(val, Equals, "0") + + val, err = sv.Validate(vars, "9999", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "9999") + + c.Assert(sv.SetSessionFromHook(vars, "9999"), IsNil) // sets + c.Assert(vars.SelectLimit, Equals, uint64(9999)) +} + +func (*testSysVarSuite) TestSQLModeVar(c *C) { + sv := GetSysVar(SQLModeVar) + vars := NewSessionVars() + val, err := sv.Validate(vars, "strict_trans_tabLES ", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "STRICT_TRANS_TABLES") + + _, err = sv.Validate(vars, "strict_trans_tabLES,nonsense_option", ScopeSession) + c.Assert(err.Error(), Equals, "ERROR 1231 (42000): Variable 'sql_mode' can't be set to the value of 'NONSENSE_OPTION'") + + val, err = sv.Validate(vars, "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", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "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") + + c.Assert(sv.SetSessionFromHook(vars, val), IsNil) // sets to strict from above + c.Assert(vars.StrictSQLMode, IsTrue) + + sqlMode, err := mysql.GetSQLMode(val) + c.Assert(err, IsNil) + c.Assert(vars.SQLMode, Equals, sqlMode) + + // Set it to non strict. + val, err = sv.Validate(vars, "ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION") + + c.Assert(sv.SetSessionFromHook(vars, val), IsNil) // sets to non-strict from above + c.Assert(vars.StrictSQLMode, IsFalse) + sqlMode, err = mysql.GetSQLMode(val) + c.Assert(err, IsNil) + c.Assert(vars.SQLMode, Equals, sqlMode) +} + +func (*testSysVarSuite) TestMaxExecutionTime(c *C) { + sv := GetSysVar(MaxExecutionTime) + vars := NewSessionVars() + + val, err := sv.Validate(vars, "-10", ScopeSession) + c.Assert(err, IsNil) // it has autoconvert out of range. + c.Assert(val, Equals, "0") + + val, err = sv.Validate(vars, "99999", ScopeSession) + c.Assert(err, IsNil) // it has autoconvert out of range. + c.Assert(val, Equals, "99999") + + c.Assert(sv.SetSessionFromHook(vars, "99999"), IsNil) // sets + c.Assert(vars.MaxExecutionTime, Equals, uint64(99999)) +} + +func (*testSysVarSuite) TestCollationServer(c *C) { + sv := GetSysVar(CollationServer) + vars := NewSessionVars() + + val, err := sv.Validate(vars, "LATIN1_bin", ScopeSession) + c.Assert(err, IsNil) + c.Assert(val, Equals, "latin1_bin") // test normalization + + _, err = sv.Validate(vars, "BOGUSCOLLation", ScopeSession) + c.Assert(err.Error(), Equals, "[ddl:1273]Unknown collation: 'BOGUSCOLLation'") + + c.Assert(sv.SetSessionFromHook(vars, "latin1_bin"), IsNil) + c.Assert(vars.systems[CharacterSetServer], Equals, "latin1") // check it also changes charset. + + c.Assert(sv.SetSessionFromHook(vars, "utf8mb4_bin"), IsNil) + c.Assert(vars.systems[CharacterSetServer], Equals, "utf8mb4") // check it also changes charset. +} diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 0204b58a3e13b..45cebbd62e0db 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -111,10 +111,11 @@ func int32ToBoolStr(i int32) string { } func checkCollation(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if _, err := collate.GetCollationByName(normalizedValue); err != nil { + coll, err := collate.GetCollationByName(normalizedValue) + if err != nil { return normalizedValue, errors.Trace(err) } - return normalizedValue, nil + return coll.Name, nil } func checkCharacterSet(normalizedValue string, argName string) (string, error) { @@ -149,6 +150,17 @@ func checkReadOnly(vars *SessionVars, normalizedValue string, originalValue stri return normalizedValue, nil } +func checkIsolationLevel(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + if normalizedValue == "SERIALIZABLE" || normalizedValue == "READ-UNCOMMITTED" { + returnErr := ErrUnsupportedIsolationLevel.GenWithStackByArgs(normalizedValue) + if !TiDBOptOn(vars.systems[TiDBSkipIsolationLevelCheck]) { + return normalizedValue, ErrUnsupportedIsolationLevel.GenWithStackByArgs(normalizedValue) + } + vars.StmtCtx.AppendWarning(returnErr) + } + return normalizedValue, nil +} + // GetSessionSystemVar gets a system variable. // If it is a session only variable, use the default value defined in code. // Returns error if there is no such variable. @@ -284,7 +296,6 @@ func SetSessionSystemVar(vars *SessionVars, name string, value string) error { if err != nil { return err } - CheckDeprecationSetSystemVar(vars, name) return vars.SetSystemVar(name, sVal) } @@ -299,7 +310,6 @@ func SetStmtVar(vars *SessionVars, name string, value string) error { if err != nil { return err } - CheckDeprecationSetSystemVar(vars, name) return vars.SetStmtVar(name, sVal) } @@ -329,18 +339,9 @@ const ( maxChunkSizeLowerBound = 32 ) -// CheckDeprecationSetSystemVar checks if the system variable is deprecated. -func CheckDeprecationSetSystemVar(s *SessionVars, name string) { - switch name { - case TiDBIndexLookupConcurrency, TiDBIndexLookupJoinConcurrency, - TiDBHashJoinConcurrency, TiDBHashAggPartialConcurrency, TiDBHashAggFinalConcurrency, - TiDBProjectionConcurrency, TiDBWindowConcurrency, TiDBMergeJoinConcurrency, TiDBStreamAggConcurrency: - s.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(name, TiDBExecutorConcurrency)) - case TIDBMemQuotaHashJoin, TIDBMemQuotaMergeJoin, - TIDBMemQuotaSort, TIDBMemQuotaTopn, - TIDBMemQuotaIndexLookupReader, TIDBMemQuotaIndexLookupJoin: - s.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(name, TIDBMemQuotaQuery)) - } +// appendDeprecationWarning adds a warning that the item is deprecated. +func appendDeprecationWarning(s *SessionVars, name, replacement string) { + s.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(name, replacement)) } // TiDBOptOn could be used for all tidb session variable options, we use "ON"/1 to turn on those options. diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 3cd546eba7637..5757eaeeac403 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -277,13 +277,8 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(err, IsNil) c.Assert(v.OptimizerSelectivityLevel, Equals, 1) - err = SetSessionSystemVar(v, TiDBDDLReorgWorkerCount, "-1") - c.Assert(terror.ErrorEqual(err, ErrWrongValueForVar), IsTrue) - - max := int64(maxDDLReorgWorkerCount) + 1 - err = SetSessionSystemVar(v, TiDBDDLReorgWorkerCount, strconv.FormatInt(max, 10)) - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, ErrWrongValueForVar), IsTrue) + err = SetSessionSystemVar(v, TiDBDDLReorgWorkerCount, "4") // wrong scope global only + c.Assert(terror.ErrorEqual(err, errGlobalVariable), IsTrue) err = SetSessionSystemVar(v, TiDBRetryLimit, "3") c.Assert(err, IsNil) @@ -505,8 +500,8 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err = SetSessionSystemVar(v, "UnknownVariable", "on") c.Assert(err, ErrorMatches, ".*]Unknown system variable 'UnknownVariable'") - err = SetSessionSystemVar(v, TiDBAnalyzeVersion, "3") - c.Assert(err, ErrorMatches, ".*Variable 'tidb_analyze_version' can't be set to the value of '3'") + err = SetSessionSystemVar(v, TiDBAnalyzeVersion, "4") + c.Assert(err, ErrorMatches, ".*Variable 'tidb_analyze_version' can't be set to the value of '4'") } func (s *testVarsutilSuite) TestSetOverflowBehave(c *C) { @@ -564,9 +559,6 @@ func (s *testVarsutilSuite) TestValidate(c *C) { {TiDBEnableTablePartition, "OFF", false}, {TiDBEnableTablePartition, "AUTO", false}, {TiDBEnableTablePartition, "UN", true}, - {TiDBEnableListTablePartition, "ON", false}, - {TiDBEnableListTablePartition, "OFF", false}, - {TiDBEnableListTablePartition, "list", true}, {TiDBOptCorrelationExpFactor, "a", true}, {TiDBOptCorrelationExpFactor, "-10", true}, {TiDBOptCorrelationThreshold, "a", true}, @@ -602,10 +594,6 @@ func (s *testVarsutilSuite) TestValidate(c *C) { {TiDBTxnMode, "pessimistic", false}, {TiDBTxnMode, "optimistic", false}, {TiDBTxnMode, "", false}, - {TiDBIsolationReadEngines, "", true}, - {TiDBIsolationReadEngines, "tikv", false}, - {TiDBIsolationReadEngines, "TiKV,tiflash", false}, - {TiDBIsolationReadEngines, " tikv, tiflash ", false}, {TiDBShardAllocateStep, "ad", true}, {TiDBShardAllocateStep, "-123", false}, {TiDBShardAllocateStep, "128", false}, @@ -629,6 +617,30 @@ func (s *testVarsutilSuite) TestValidate(c *C) { } } + // Test session scoped vars. + tests = []struct { + key string + value string + error bool + }{ + {TiDBEnableListTablePartition, "ON", false}, + {TiDBEnableListTablePartition, "OFF", false}, + {TiDBEnableListTablePartition, "list", true}, + {TiDBIsolationReadEngines, "", true}, + {TiDBIsolationReadEngines, "tikv", false}, + {TiDBIsolationReadEngines, "TiKV,tiflash", false}, + {TiDBIsolationReadEngines, " tikv, tiflash ", false}, + } + + for _, t := range tests { + _, err := GetSysVar(t.key).Validate(v, t.value, ScopeSession) + if t.error { + c.Assert(err, NotNil, Commentf("%v got err=%v", t, err)) + } else { + c.Assert(err, IsNil, Commentf("%v got err=%v", t, err)) + } + } + } func (s *testVarsutilSuite) TestValidateStmtSummary(c *C) { diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index 6587340c4a05c..f682a1507a4bf 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -15,9 +15,13 @@ package statistics import ( "bytes" + "fmt" "math" "reflect" "sort" + "strings" + + "github.com/pingcap/tidb/sessionctx" "github.com/cznic/mathutil" "github.com/cznic/sortutil" @@ -426,19 +430,27 @@ func EncodeCMSketchWithoutTopN(c *CMSketch) ([]byte, error) { // DecodeCMSketchAndTopN decode a CMSketch from the given byte slice. func DecodeCMSketchAndTopN(data []byte, topNRows []chunk.Row) (*CMSketch, *TopN, error) { - if data == nil { + if data == nil && len(topNRows) == 0 { return nil, nil, nil } + pbTopN := make([]*tipb.CMSketchTopN, 0, len(topNRows)) + for _, row := range topNRows { + data := make([]byte, len(row.GetBytes(0))) + copy(data, row.GetBytes(0)) + pbTopN = append(pbTopN, &tipb.CMSketchTopN{ + Data: data, + Count: row.GetUint64(1), + }) + } + if len(data) == 0 { + return nil, TopNFromProto(pbTopN), nil + } p := &tipb.CMSketch{} err := p.Unmarshal(data) if err != nil { return nil, nil, errors.Trace(err) } - for _, row := range topNRows { - data := make([]byte, len(row.GetBytes(0))) - copy(data, row.GetBytes(0)) - p.TopN = append(p.TopN, &tipb.CMSketchTopN{Data: data, Count: row.GetUint64(1)}) - } + p.TopN = pbTopN cm, topN := CMSketchAndTopNFromProto(p) return cm, topN, nil } @@ -487,6 +499,46 @@ type TopN struct { TopN []TopNMeta } +func (c *TopN) String() string { + if c == nil { + return "EmptyTopN" + } + builder := &strings.Builder{} + fmt.Fprintf(builder, "TopN{length: %v, ", len(c.TopN)) + fmt.Fprint(builder, "[") + for i := 0; i < len(c.TopN); i++ { + fmt.Fprintf(builder, "(%v, %v)", c.TopN[i].Encoded, c.TopN[i].Count) + if i+1 != len(c.TopN) { + fmt.Fprint(builder, ", ") + } + } + fmt.Fprint(builder, "]") + fmt.Fprint(builder, "}") + return builder.String() +} + +// DecodedString returns the value with decoded result. +func (c *TopN) DecodedString(ctx sessionctx.Context, colTypes []byte) (string, error) { + builder := &strings.Builder{} + fmt.Fprintf(builder, "TopN{length: %v, ", len(c.TopN)) + fmt.Fprint(builder, "[") + var tmpDatum types.Datum + for i := 0; i < len(c.TopN); i++ { + tmpDatum.SetBytes(c.TopN[i].Encoded) + valStr, err := ValueToString(ctx.GetSessionVars(), &tmpDatum, len(colTypes), colTypes) + if err != nil { + return "", err + } + fmt.Fprintf(builder, "(%v, %v)", valStr, c.TopN[i].Count) + if i+1 != len(c.TopN) { + fmt.Fprint(builder, ", ") + } + } + fmt.Fprint(builder, "]") + fmt.Fprint(builder, "}") + return builder.String(), nil +} + // Copy makes a copy for current TopN. func (c *TopN) Copy() *TopN { if c == nil { @@ -593,9 +645,9 @@ func (c *TopN) TotalCount() uint64 { // Equal checks whether the two TopN are equal. func (c *TopN) Equal(cc *TopN) bool { - if c == nil && cc == nil { + if c.TotalCount() == 0 && cc.TotalCount() == 0 { return true - } else if c == nil || cc == nil { + } else if c.TotalCount() != cc.TotalCount() { return false } if len(c.TopN) != len(cc.TopN) { @@ -638,7 +690,7 @@ func NewTopN(n int) *TopN { // 1. `*TopN` is the final global-level topN. // 2. `[]TopNMeta` is the left topN value from the partition-level TopNs, but is not placed to global-level TopN. We should put them back to histogram latter. // 3. `[]*Histogram` are the partition-level histograms which just delete some values when we merge the global-level topN. -func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, topNs []*TopN, n uint32, hists []*Histogram, isIndex bool) (*TopN, []TopNMeta, []*Histogram, error) { +func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, version int, topNs []*TopN, n uint32, hists []*Histogram, isIndex bool) (*TopN, []TopNMeta, []*Histogram, error) { if checkEmptyTopNs(topNs) { return nil, nil, hists, nil } @@ -674,7 +726,7 @@ func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, topNs []*TopN, n uin // 1. Check the topN first. // 2. If the topN doesn't contain the value corresponding to encodedVal. We should check the histogram. for j := 0; j < partNum; j++ { - if j == i || topNs[j].findTopN(val.Encoded) != -1 { + if (j == i && version >= 2) || topNs[j].findTopN(val.Encoded) != -1 { continue } // Get the encodedVal from the hists[j] @@ -706,7 +758,7 @@ func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, topNs []*TopN, n uin if count != 0 { counter[encodedVal] += count // Remove the value corresponding to encodedVal from the histogram. - removeVals[j] = append(removeVals[j], TopNMeta{Encoded: val.Encoded, Count: uint64(count)}) + removeVals[j] = append(removeVals[j], TopNMeta{Encoded: datum.GetBytes(), Count: uint64(count)}) } } } @@ -714,7 +766,12 @@ func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, topNs []*TopN, n uin // Remove the value from the Hists. for i := 0; i < partNum; i++ { if len(removeVals[i]) > 0 { - hists[i].RemoveVals(removeVals[i]) + tmp := removeVals[i] + sort.Slice(tmp, func(i, j int) bool { + cmpResult := bytes.Compare(tmp[i].Encoded, tmp[j].Encoded) + return cmpResult < 0 + }) + hists[i].RemoveVals(tmp) } } numTop := len(counter) diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 32c256afe8834..7036242b4e3a7 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -435,7 +435,7 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context, opts map // Because after merging TopN, some numbers will be left. // These remaining topN numbers will be used as a separate bucket for later histogram merging. var popedTopN []statistics.TopNMeta - globalStats.TopN[i], popedTopN, allHg[i], err = statistics.MergePartTopN2GlobalTopN(sc.GetSessionVars().StmtCtx, allTopN[i], uint32(opts[ast.AnalyzeOptNumTopN]), allHg[i], isIndex == 1) + globalStats.TopN[i], popedTopN, allHg[i], err = statistics.MergePartTopN2GlobalTopN(sc.GetSessionVars().StmtCtx, sc.GetSessionVars().AnalyzeVersion, allTopN[i], uint32(opts[ast.AnalyzeOptNumTopN]), allHg[i], isIndex == 1) if err != nil { return } @@ -684,14 +684,17 @@ func (h *Handle) FlushStats() { } func (h *Handle) cmSketchAndTopNFromStorage(reader *statsReader, tblID int64, isIndex, histID int64) (_ *statistics.CMSketch, _ *statistics.TopN, err error) { - rows, _, err := reader.read("select cm_sketch from mysql.stats_histograms where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) - if err != nil || len(rows) == 0 { + topNRows, _, err := reader.read("select HIGH_PRIORITY value, count from mysql.stats_top_n where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) + if err != nil { return nil, nil, err } - topNRows, _, err := reader.read("select HIGH_PRIORITY value, count from mysql.stats_top_n where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) + rows, _, err := reader.read("select cm_sketch from mysql.stats_histograms where table_id = %? and is_index = %? and hist_id = %?", tblID, isIndex, histID) if err != nil { return nil, nil, err } + if len(rows) == 0 { + return statistics.DecodeCMSketchAndTopN(nil, topNRows) + } return statistics.DecodeCMSketchAndTopN(rows[0].GetBytes(0), topNRows) } diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 0a55f03722b75..3bb95e8219478 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -176,7 +176,7 @@ func assertTableEqual(c *C, a *statistics.Table, b *statistics.Table) { c.Assert(a.Columns[i].CMSketch.Equal(b.Columns[i].CMSketch), IsTrue) } // The nil case has been considered in (*TopN).Equal() so we don't need to consider it here. - c.Assert(a.Columns[i].TopN.Equal(b.Columns[i].TopN), IsTrue) + c.Assert(a.Columns[i].TopN.Equal(b.Columns[i].TopN), IsTrue, Commentf("%v, %v", a.Columns[i].TopN, b.Columns[i].TopN)) } c.Assert(len(a.Indices), Equals, len(b.Indices)) for i := range a.Indices { @@ -2784,3 +2784,21 @@ func (s *testStatsSuite) TestStatsCacheUpdateSkip(c *C) { statsTbl2 := h.GetTableStats(tableInfo) c.Assert(statsTbl1, Equals, statsTbl2) } + +func (s *testSerialStatsSuite) TestIssues24349(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("set @@tidb_partition_prune_mode='dynamic'") + testKit.MustExec("set @@tidb_analyze_version=2") + defer testKit.MustExec("set @@tidb_analyze_version=1") + defer testKit.MustExec("set @@tidb_partition_prune_mode='static'") + testKit.MustExec("create table t (a int, b int) partition by hash(a) partitions 3") + testKit.MustExec("insert into t values (0, 3), (0, 3), (0, 3), (0, 2), (1, 1), (1, 2), (1, 2), (1, 2), (1, 3), (1, 4), (2, 1), (2, 1)") + testKit.MustExec("analyze table t with 1 topn, 3 buckets") + testKit.MustQuery("show stats_buckets where partition_name='global'").Check(testkit.Rows( + "test t global a 0 0 2 2 0 2 0", + "test t global b 0 0 3 1 1 2 0", + "test t global b 0 1 10 1 4 4 0", + )) +} diff --git a/statistics/histogram.go b/statistics/histogram.go index e58abddc16867..37a78fe1b92de 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -285,7 +285,7 @@ func HistogramEqual(a, b *Histogram, ignoreID bool) bool { const ( // Version0 is the state that no statistics is actually collected, only the meta info.(the total count and the average col size) Version0 = 0 - // In Version1 + // Version1 maintains the statistics in the following way. // Column stats: CM Sketch is built in TiKV using full data. Histogram is built from samples. TopN is extracted from CM Sketch. // TopN + CM Sketch represent all data. Histogram also represents all data. // Index stats: CM Sketch and Histogram is built in TiKV using full data. TopN is extracted from histogram. Then values covered by TopN is removed from CM Sketch. @@ -293,11 +293,15 @@ const ( // Int PK column stats is always Version1 because it only has histogram built from full data. // Fast analyze is always Version1 currently. Version1 = 1 - // In Version2 + // Version2 maintains the statistics in the following way. // Column stats: CM Sketch is not used. TopN and Histogram are built from samples. TopN + Histogram represent all data. // Index stats: CM SKetch is not used. TopN and Histograms are built in TiKV using full data. NDV is also collected for each bucket in histogram. // Then values covered by TopN is removed from Histogram. TopN + Histogram represent all data. Version2 = 2 + // Version3 is used for testing now. Once it finished, we will fallback the Version3 to Version2. + // The difference between Version2 and Version3 is that we construct the index's statistics based on sampling also. + // The data structure between them are then same. + Version3 = 3 ) // AnalyzeFlag is set when the statistics comes from analyze and has not been modified by feedback. @@ -344,6 +348,7 @@ func (hg *Histogram) BucketToString(bktID, idxCols int) string { } // RemoveVals remove the given values from the histogram. +// This function contains an **ASSUMPTION**: valCntPairs is sorted in ascending order. func (hg *Histogram) RemoveVals(valCntPairs []TopNMeta) { totalSubCnt := int64(0) var cmpResult int @@ -352,6 +357,7 @@ func (hg *Histogram) RemoveVals(valCntPairs []TopNMeta) { // If the current val smaller than current bucket's lower bound, skip it. cmpResult = bytes.Compare(hg.Bounds.Column(0).GetRaw(bktIdx*2), valCntPairs[pairIdx].Encoded) if cmpResult > 0 { + pairIdx++ continue } // If the current val bigger than current bucket's upper bound, break. diff --git a/statistics/integration_test.go b/statistics/integration_test.go index d080a32c539e5..9edef42f17698 100644 --- a/statistics/integration_test.go +++ b/statistics/integration_test.go @@ -326,3 +326,37 @@ func (s *testIntegrationSuite) TestGlobalStats(c *C) { "TableReader 6.00 root partition:all data:TableFullScan", "└─TableFullScan 6.00 cop[tikv] table:t keep order:false")) } + +func (s *testIntegrationSuite) TestNULLOnFullSampling(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;") + tk.MustExec("set @@session.tidb_analyze_version = 3;") + tk.MustExec("create table t(a int, index idx(a))") + tk.MustExec("insert into t values(1), (1), (1), (2), (2), (3), (4), (null), (null), (null)") + var ( + input []string + output [][]string + ) + tk.MustExec("analyze table t with 2 topn") + is := s.do.InfoSchema() + tblT, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + h := s.do.StatsHandle() + c.Assert(h.Update(is), IsNil) + statsTblT := h.GetTableStats(tblT.Meta()) + // Check the null count is 3. + for _, col := range statsTblT.Columns { + c.Assert(col.NullCount, Equals, int64(3)) + } + + s.testData.GetTestCases(c, &input, &output) + // Check the topn and buckets contains no null values. + for i := 0; i < len(input); i++ { + s.testData.OnRecord(func() { + output[i] = s.testData.ConvertRowsToStrings(tk.MustQuery(input[i]).Rows()) + }) + tk.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) + } +} diff --git a/statistics/row_sampler.go b/statistics/row_sampler.go index e78895e19da13..f4f40945af697 100644 --- a/statistics/row_sampler.go +++ b/statistics/row_sampler.go @@ -14,14 +14,18 @@ package statistics import ( + "bytes" "container/heap" "context" "math/rand" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tipb/go-tipb" @@ -285,9 +289,161 @@ func RowSamplesToProto(samples WeightedRowSampleHeap) []*tipb.RowSample { Weight: sample.Weight, } for _, c := range sample.Columns { + if c.IsNull() { + pbRow.Row = append(pbRow.Row, []byte{codec.NilFlag}) + continue + } pbRow.Row = append(pbRow.Row, c.GetBytes()) } rows = append(rows, pbRow) } return rows } + +// BuildHistAndTopNOnRowSample build a histogram and TopN for a column from samples. +func BuildHistAndTopNOnRowSample( + ctx sessionctx.Context, + numBuckets, numTopN int, + id int64, + collector *SampleCollector, + tp *types.FieldType, + isColumn bool, +) (*Histogram, *TopN, error) { + var getComparedBytes func(datum types.Datum) ([]byte, error) + if isColumn { + getComparedBytes = func(datum types.Datum) ([]byte, error) { + return codec.EncodeKey(ctx.GetSessionVars().StmtCtx, nil, datum) + } + } else { + getComparedBytes = func(datum types.Datum) ([]byte, error) { + return datum.GetBytes(), nil + } + } + count := collector.Count + ndv := collector.FMSketch.NDV() + nullCount := collector.NullCount + if ndv > count { + ndv = count + } + if count == 0 || len(collector.Samples) == 0 { + return NewHistogram(id, ndv, nullCount, 0, tp, 0, collector.TotalSize), nil, nil + } + sc := ctx.GetSessionVars().StmtCtx + samples := collector.Samples + samples, err := SortSampleItems(sc, samples) + if err != nil { + return nil, nil, err + } + hg := NewHistogram(id, ndv, nullCount, 0, tp, numBuckets, collector.TotalSize) + + sampleNum := int64(len(samples)) + // As we use samples to build the histogram, the bucket number and repeat should multiply a factor. + sampleFactor := float64(count) / float64(len(samples)) + + // Step1: collect topn from samples + + // the topNList is always sorted by count from more to less + topNList := make([]TopNMeta, 0, numTopN) + cur, err := getComparedBytes(samples[0].Value) + if err != nil { + return nil, nil, errors.Trace(err) + } + curCnt := float64(0) + + // Iterate through the samples + for i := int64(0); i < sampleNum; i++ { + + sampleBytes, err := getComparedBytes(samples[i].Value) + if err != nil { + return nil, nil, errors.Trace(err) + } + // case 1, this value is equal to the last one: current count++ + if bytes.Equal(cur, sampleBytes) { + curCnt += 1 + continue + } + // case 2, meet a different value: counting for the "current" is complete + // case 2-1, now topn is empty: append the "current" count directly + if len(topNList) == 0 { + topNList = append(topNList, TopNMeta{Encoded: cur, Count: uint64(curCnt)}) + cur, curCnt = sampleBytes, 1 + continue + } + // case 2-2, now topn is full, and the "current" count is less than the least count in the topn: no need to insert the "current" + if len(topNList) >= numTopN && uint64(curCnt) <= topNList[len(topNList)-1].Count { + cur, curCnt = sampleBytes, 1 + continue + } + // case 2-3, now topn is not full, or the "current" count is larger than the least count in the topn: need to find a slot to insert the "current" + j := len(topNList) + for ; j > 0; j-- { + if uint64(curCnt) < topNList[j-1].Count { + break + } + } + topNList = append(topNList, TopNMeta{}) + copy(topNList[j+1:], topNList[j:]) + topNList[j] = TopNMeta{Encoded: cur, Count: uint64(curCnt)} + if len(topNList) > numTopN { + topNList = topNList[:numTopN] + } + cur, curCnt = sampleBytes, 1 + } + + // Handle the counting for the last value. Basically equal to the case 2 above. + // now topn is empty: append the "current" count directly + if len(topNList) == 0 { + topNList = append(topNList, TopNMeta{Encoded: cur, Count: uint64(curCnt)}) + } else if len(topNList) < numTopN || uint64(curCnt) > topNList[len(topNList)-1].Count { + // now topn is not full, or the "current" count is larger than the least count in the topn: need to find a slot to insert the "current" + j := len(topNList) + for ; j > 0; j-- { + if uint64(curCnt) < topNList[j-1].Count { + break + } + } + topNList = append(topNList, TopNMeta{}) + copy(topNList[j+1:], topNList[j:]) + topNList[j] = TopNMeta{Encoded: cur, Count: uint64(curCnt)} + if len(topNList) > numTopN { + topNList = topNList[:numTopN] + } + } + + // Step2: exclude topn from samples + for i := int64(0); i < int64(len(samples)); i++ { + sampleBytes, err := getComparedBytes(samples[i].Value) + if err != nil { + return nil, nil, errors.Trace(err) + } + for j := 0; j < len(topNList); j++ { + if bytes.Equal(sampleBytes, topNList[j].Encoded) { + // find the same value in topn: need to skip over this value in samples + copy(samples[i:], samples[uint64(i)+topNList[j].Count:]) + samples = samples[:uint64(len(samples))-topNList[j].Count] + i-- + continue + } + } + } + + for i := 0; i < len(topNList); i++ { + topNList[i].Count *= uint64(sampleFactor) + } + topn := &TopN{TopN: topNList} + + if uint64(count) <= topn.TotalCount() || int(hg.NDV) <= len(topn.TopN) { + // TopN includes all sample data + return hg, topn, nil + } + + // Step3: build histogram with the rest samples + if len(samples) > 0 { + _, err = buildHist(sc, hg, samples, count-int64(topn.TotalCount()), ndv-int64(len(topn.TopN)), int64(numBuckets)) + if err != nil { + return nil, nil, err + } + } + + return hg, topn, nil +} diff --git a/statistics/sample_test.go b/statistics/sample_test.go index 0183a2a57dac2..61ec41b85870a 100644 --- a/statistics/sample_test.go +++ b/statistics/sample_test.go @@ -254,3 +254,60 @@ func (s *testSampleSuite) TestDistributedWeightedSampling(c *C) { } // } } + +func (s *testSampleSuite) TestBuildStatsOnRowSample(c *C) { + ctx := mock.NewContext() + sketch := NewFMSketch(1000) + data := make([]*SampleItem, 0, 8) + for i := 1; i <= 1000; i++ { + d := types.NewIntDatum(int64(i)) + err := sketch.InsertValue(ctx.GetSessionVars().StmtCtx, d) + c.Assert(err, IsNil) + data = append(data, &SampleItem{Value: d}) + } + for i := 1; i < 10; i++ { + d := types.NewIntDatum(int64(2)) + err := sketch.InsertValue(ctx.GetSessionVars().StmtCtx, d) + c.Assert(err, IsNil) + data = append(data, &SampleItem{Value: d}) + } + for i := 1; i < 7; i++ { + d := types.NewIntDatum(int64(4)) + err := sketch.InsertValue(ctx.GetSessionVars().StmtCtx, d) + c.Assert(err, IsNil) + data = append(data, &SampleItem{Value: d}) + } + for i := 1; i < 5; i++ { + d := types.NewIntDatum(int64(7)) + err := sketch.InsertValue(ctx.GetSessionVars().StmtCtx, d) + c.Assert(err, IsNil) + data = append(data, &SampleItem{Value: d}) + } + for i := 1; i < 3; i++ { + d := types.NewIntDatum(int64(11)) + err := sketch.InsertValue(ctx.GetSessionVars().StmtCtx, d) + c.Assert(err, IsNil) + data = append(data, &SampleItem{Value: d}) + } + collector := &SampleCollector{ + Samples: data, + NullCount: 0, + Count: int64(len(data)), + FMSketch: sketch, + TotalSize: int64(len(data)) * 8, + } + tp := types.NewFieldType(mysql.TypeLonglong) + hist, topN, err := BuildHistAndTopNOnRowSample(ctx, 5, 4, 1, collector, tp, true) + c.Assert(err, IsNil, Commentf("%+v", err)) + topNStr, err := topN.DecodedString(ctx, []byte{tp.Tp}) + c.Assert(err, IsNil) + c.Assert(topNStr, Equals, "TopN{length: 4, [(2, 10), (4, 7), (7, 5), (11, 3)]}") + c.Assert(hist.ToString(0), Equals, "column:1 ndv:1000 totColSize:8168\n"+ + "num: 200 lower_bound: 1 upper_bound: 204 repeats: 1 ndv: 0\n"+ + "num: 200 lower_bound: 205 upper_bound: 404 repeats: 1 ndv: 0\n"+ + "num: 200 lower_bound: 405 upper_bound: 604 repeats: 1 ndv: 0\n"+ + "num: 200 lower_bound: 605 upper_bound: 804 repeats: 1 ndv: 0\n"+ + "num: 196 lower_bound: 805 upper_bound: 1000 repeats: 1 ndv: 0", + ) + +} diff --git a/statistics/testdata/integration_suite_in.json b/statistics/testdata/integration_suite_in.json index 61f4badc3bb72..17afe048b3419 100644 --- a/statistics/testdata/integration_suite_in.json +++ b/statistics/testdata/integration_suite_in.json @@ -9,5 +9,12 @@ "explain select * from exp_backoff where a = 1 and b = 1 and c = 1 and d >= 3 and d<= 5", "explain select * from exp_backoff where a = 1 and b = 1 and c = 1 and d >= 3 and d<= 5" ] + }, + { + "name": "TestNULLOnFullSampling", + "cases": [ + "show stats_topn", + "show stats_buckets" + ] } ] diff --git a/statistics/testdata/integration_suite_out.json b/statistics/testdata/integration_suite_out.json index d5f6ff224a282..550d10caba8e1 100644 --- a/statistics/testdata/integration_suite_out.json +++ b/statistics/testdata/integration_suite_out.json @@ -30,5 +30,22 @@ "└─IndexRangeScan_5 0.00 cop[tikv] table:exp_backoff, index:idx(a, b, c, d) range:[1 1 1 3,1 1 1 5], keep order:false" ] ] + }, + { + "Name": "TestNULLOnFullSampling", + "Cases": [ + [ + "test t a 0 1 3", + "test t a 0 2 2", + "test t idx 1 1 3", + "test t idx 1 2 2" + ], + [ + "test t a 0 0 1 1 3 3 0", + "test t a 0 1 2 1 4 4 0", + "test t idx 1 0 1 1 3 3 0", + "test t idx 1 1 2 1 4 4 0" + ] + ] } ] diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 6e3b8d9b6e903..5506c9d497ac2 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -99,7 +100,7 @@ type copTaskAndRPCContext struct { ctx *tikv.RPCContext } -func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { +func buildBatchCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { start := time.Now() const cmdType = tikvrpc.CmdBatchCop rangesLen := ranges.Len() @@ -114,7 +115,7 @@ func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tik }) } - err := tikv.SplitKeyRanges(bo, cache, ranges, appendTask) + err := tikv.SplitKeyRanges(bo.TiKVBackoffer(), cache, ranges, appendTask) if err != nil { return nil, errors.Trace(err) } @@ -124,7 +125,7 @@ func buildBatchCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tik storeTaskMap := make(map[string]*batchCopTask) needRetry := false for _, task := range tasks { - rpcCtx, err := cache.GetTiFlashRPCContext(bo, task.region, false) + rpcCtx, err := cache.GetTiFlashRPCContext(bo.TiKVBackoffer(), task.region, false) if err != nil { return nil, errors.Trace(err) } @@ -176,19 +177,19 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V return copErrorResponse{errors.New("batch coprocessor cannot prove keep order or desc property")} } ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTs) - bo := tikv.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) + bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildBatchCopTasks(bo, c.store.GetRegionCache(), ranges, req.StoreType) if err != nil { return copErrorResponse{err} } it := &batchCopIterator{ - store: c.store.KVStore, + store: c.store.kvStore, req: req, finishCh: make(chan struct{}), vars: vars, memTracker: req.MemTracker, - ClientHelper: tikv.NewClientHelper(c.store.KVStore, util.NewTSSet(5)), + ClientHelper: tikv.NewClientHelper(c.store.kvStore.store, util.NewTSSet(5)), rpcCancel: tikv.NewRPCanceller(), } ctx = context.WithValue(ctx, tikv.RPCCancellerCtxKey{}, it.rpcCancel) @@ -201,7 +202,7 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V type batchCopIterator struct { *tikv.ClientHelper - store *tikv.KVStore + store *kvStore req *kv.Request finishCh chan struct{} @@ -227,7 +228,7 @@ func (b *batchCopIterator) run(ctx context.Context) { // We run workers for every batch cop. for _, task := range b.tasks { b.wg.Add(1) - bo := tikv.NewBackofferWithVars(ctx, copNextMaxBackoff, b.vars) + bo := newBackofferWithVars(ctx, copNextMaxBackoff, b.vars) go b.handleTask(ctx, bo, task) } b.wg.Wait() @@ -297,7 +298,7 @@ func (b *batchCopIterator) Close() error { return nil } -func (b *batchCopIterator) handleTask(ctx context.Context, bo *tikv.Backoffer, task *batchCopTask) { +func (b *batchCopIterator) handleTask(ctx context.Context, bo *backoffer, task *batchCopTask) { tasks := []*batchCopTask{task} for idx := 0; idx < len(tasks); idx++ { ret, err := b.handleTaskOnce(ctx, bo, tasks[idx]) @@ -312,7 +313,7 @@ func (b *batchCopIterator) handleTask(ctx context.Context, bo *tikv.Backoffer, t } // Merge all ranges and request again. -func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *tikv.Backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) { +func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) { var ranges []tikvstore.KeyRange for _, taskCtx := range batchTask.copTasks { taskCtx.task.ranges.Do(func(ran *tikvstore.KeyRange) { @@ -322,7 +323,7 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *tikv.Backo return buildBatchCopTasks(bo, b.store.GetRegionCache(), tikv.NewKeyRanges(ranges), b.req.StoreType) } -func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *tikv.Backoffer, task *batchCopTask) ([]*batchCopTask, error) { +func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *backoffer, task *batchCopTask) ([]*batchCopTask, error) { sender := NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) var regionInfos = make([]*coprocessor.RegionInfo, 0, len(task.copTasks)) for _, task := range task.copTasks { @@ -345,8 +346,8 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *tikv.Backoffe } req := tikvrpc.NewRequest(task.cmdType, &copReq, kvrpcpb.Context{ - IsolationLevel: tikv.IsolationLevelToPB(b.req.IsolationLevel), - Priority: tikv.PriorityToPB(b.req.Priority), + IsolationLevel: isolationLevelToPB(b.req.IsolationLevel), + Priority: priorityToPB(b.req.Priority), NotFillCache: b.req.NotFillCache, RecordTimeStat: true, RecordScanStat: true, @@ -367,7 +368,7 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *tikv.Backoffe return nil, b.handleStreamedBatchCopResponse(ctx, bo, resp.Resp.(*tikvrpc.BatchCopStreamResponse), task) } -func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, bo *tikv.Backoffer, response *tikvrpc.BatchCopStreamResponse, task *batchCopTask) (err error) { +func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, bo *backoffer, response *tikvrpc.BatchCopStreamResponse, task *batchCopTask) (err error) { defer response.Close() resp := response.BatchResponse if resp == nil { @@ -395,12 +396,12 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b } else { logutil.BgLogger().Info("stream unknown error", zap.Error(err)) } - return tikverr.ErrTiFlashServerTimeout + return txndriver.ErrTiFlashServerTimeout } } } -func (b *batchCopIterator) handleBatchCopResponse(bo *tikv.Backoffer, response *coprocessor.BatchResponse, task *batchCopTask) (err error) { +func (b *batchCopIterator) handleBatchCopResponse(bo *backoffer, response *coprocessor.BatchResponse, task *batchCopTask) (err error) { if otherErr := response.GetOtherError(); otherErr != "" { err = errors.Errorf("other error: %s", otherErr) logutil.BgLogger().Warn("other error", diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index 6a48d27fbb16a..139ee087ec290 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -38,7 +38,7 @@ func NewRegionBatchRequestSender(cache *tikv.RegionCache, client tikv.Client) *R } } -func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *tikv.Backoffer, ctxs []copTaskAndRPCContext, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { +func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *backoffer, ctxs []copTaskAndRPCContext, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { // use the first ctx to send request, because every ctx has same address. cancel = func() {} rpcCtx := ctxs[0].ctx @@ -67,7 +67,7 @@ func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *tikv.Backoffer, ctxs return } -func (ss *RegionBatchRequestSender) onSendFail(bo *tikv.Backoffer, ctxs []copTaskAndRPCContext, err error) error { +func (ss *RegionBatchRequestSender) onSendFail(bo *backoffer, ctxs []copTaskAndRPCContext, err error) error { // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { return errors.Trace(err) @@ -78,7 +78,12 @@ func (ss *RegionBatchRequestSender) onSendFail(bo *tikv.Backoffer, ctxs []copTas for _, failedCtx := range ctxs { ctx := failedCtx.ctx if ctx.Meta != nil { - ss.GetRegionCache().OnSendFail(bo, ctx, ss.NeedReloadRegion(ctx), err) + // The reload region param is always true. Because that every time we try, we must + // re-build the range then re-create the batch sender. As a result, the len of "failStores" + // will change. If tiflash's replica is more than two, the "reload region" will always be false. + // Now that the batch cop and mpp has a relative low qps, it's reasonable to reload every time + // when meeting io error. + ss.GetRegionCache().OnSendFail(bo.TiKVBackoffer(), ctx, true, err) } } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index dd1555f0c614d..cb799edb16f70 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" tidbmetrics "github.com/pingcap/tidb/metrics" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" @@ -73,7 +74,7 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa return c.sendBatch(ctx, req, vars) } ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTs) - bo := tikv.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) + bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildCopTasks(bo, c.store.GetRegionCache(), ranges, req) if err != nil { @@ -144,7 +145,7 @@ func (r *copTask) String() string { // rangesPerTask limits the length of the ranges slice sent in one copTask. const rangesPerTask = 25000 -func buildCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, req *kv.Request) ([]*copTask, error) { +func buildCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, req *kv.Request) ([]*copTask, error) { start := time.Now() cmdType := tikvrpc.CmdCop if req.Streaming { @@ -176,7 +177,7 @@ func buildCopTasks(bo *tikv.Backoffer, cache *tikv.RegionCache, ranges *tikv.Key } } - err := tikv.SplitKeyRanges(bo, cache, ranges, appendTask) + err := tikv.SplitKeyRanges(bo.TiKVBackoffer(), cache, ranges, appendTask) if err != nil { return nil, errors.Trace(err) } @@ -273,7 +274,7 @@ type copIteratorWorker struct { respChan chan<- *copResponse finishCh <-chan struct{} vars *tikv.Variables - *tikv.ClientHelper + kvclient *tikv.ClientHelper memTracker *memory.Tracker @@ -402,7 +403,7 @@ func (it *copIterator) open(ctx context.Context, enabledRateLimitAction bool) { respChan: it.respChan, finishCh: it.finishCh, vars: it.vars, - ClientHelper: tikv.NewClientHelper(it.store.KVStore, it.resolvedLocks), + kvclient: tikv.NewClientHelper(it.store.store, it.resolvedLocks), memTracker: it.memTracker, replicaReadSeed: it.replicaReadSeed, actionOnExceed: it.actionOnExceed, @@ -605,12 +606,12 @@ func (it *copIterator) Next(ctx context.Context) (kv.ResultSubset, error) { // Associate each region with an independent backoffer. In this way, when multiple regions are // unavailable, TiDB can execute very quickly without blocking -func chooseBackoffer(ctx context.Context, backoffermap map[uint64]*tikv.Backoffer, task *copTask, worker *copIteratorWorker) *tikv.Backoffer { +func chooseBackoffer(ctx context.Context, backoffermap map[uint64]*backoffer, task *copTask, worker *copIteratorWorker) *backoffer { bo, ok := backoffermap[task.region.GetID()] if ok { return bo } - newbo := tikv.NewBackofferWithVars(ctx, copNextMaxBackoff, worker.vars) + newbo := newBackofferWithVars(ctx, copNextMaxBackoff, worker.vars) backoffermap[task.region.GetID()] = newbo return newbo } @@ -629,7 +630,7 @@ func (worker *copIteratorWorker) handleTask(ctx context.Context, task *copTask, } }() remainTasks := []*copTask{task} - backoffermap := make(map[uint64]*tikv.Backoffer) + backoffermap := make(map[uint64]*backoffer) for len(remainTasks) > 0 { curTask := remainTasks[0] bo := chooseBackoffer(ctx, backoffermap, curTask, worker) @@ -657,7 +658,7 @@ func (worker *copIteratorWorker) handleTask(ctx context.Context, task *copTask, // handleTaskOnce handles single copTask, successful results are send to channel. // If error happened, returns error. If region split or meet lock, returns the remain tasks. -func (worker *copIteratorWorker) handleTaskOnce(bo *tikv.Backoffer, task *copTask, ch chan<- *copResponse) ([]*copTask, error) { +func (worker *copIteratorWorker) handleTaskOnce(bo *backoffer, task *copTask, ch chan<- *copResponse) ([]*copTask, error) { failpoint.Inject("handleTaskOnceError", func(val failpoint.Value) { if val.(bool) { failpoint.Return(nil, errors.New("mock handleTaskOnce error")) @@ -697,8 +698,8 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *tikv.Backoffer, task *copTas } req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, worker.req.ReplicaRead, &worker.replicaReadSeed, kvrpcpb.Context{ - IsolationLevel: tikv.IsolationLevelToPB(worker.req.IsolationLevel), - Priority: tikv.PriorityToPB(worker.req.Priority), + IsolationLevel: isolationLevelToPB(worker.req.IsolationLevel), + Priority: priorityToPB(worker.req.Priority), NotFillCache: worker.req.NotFillCache, RecordTimeStat: true, RecordScanStat: true, @@ -706,8 +707,8 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *tikv.Backoffer, task *copTas }) req.StoreTp = getEndPointType(task.storeType) startTime := time.Now() - if worker.Stats == nil { - worker.Stats = make(map[tikvrpc.CmdType]*tikv.RPCRuntimeStats) + if worker.kvclient.Stats == nil { + worker.kvclient.Stats = make(map[tikvrpc.CmdType]*tikv.RPCRuntimeStats) } if worker.req.IsStaleness { req.EnableStaleRead() @@ -716,7 +717,8 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *tikv.Backoffer, task *copTas if len(worker.req.MatchStoreLabels) > 0 { ops = append(ops, tikv.WithMatchLabels(worker.req.MatchStoreLabels)) } - resp, rpcCtx, storeAddr, err := worker.SendReqCtx(bo, req, task.region, tikv.ReadTimeoutMedium, getEndPointType(task.storeType), task.storeAddr, ops...) + resp, rpcCtx, storeAddr, err := worker.kvclient.SendReqCtx(bo.TiKVBackoffer(), req, task.region, tikv.ReadTimeoutMedium, getEndPointType(task.storeType), task.storeAddr, ops...) + err = txndriver.ToTiDBErr(err) if err != nil { if task.storeType == kv.TiDB { err = worker.handleTiDBSendReqErr(err, task, ch) @@ -746,10 +748,10 @@ const ( minLogKVProcessTime = 100 ) -func (worker *copIteratorWorker) logTimeCopTask(costTime time.Duration, task *copTask, bo *tikv.Backoffer, resp *tikvrpc.Response) { +func (worker *copIteratorWorker) logTimeCopTask(costTime time.Duration, task *copTask, bo *backoffer, resp *tikvrpc.Response) { logStr := fmt.Sprintf("[TIME_COP_PROCESS] resp_time:%s txnStartTS:%d region_id:%d store_addr:%s", costTime, worker.req.StartTs, task.region.GetID(), task.storeAddr) if bo.GetTotalSleep() > minLogBackoffTime { - backoffTypes := strings.Replace(fmt.Sprintf("%v", bo.GetTypes()), " ", ",", -1) + backoffTypes := strings.Replace(fmt.Sprintf("%v", bo.TiKVBackoffer().GetTypes()), " ", ",", -1) logStr += fmt.Sprintf(" backoff_ms:%d backoff_types:%s", bo.GetTotalSleep(), backoffTypes) } var detailV2 *kvrpcpb.ExecDetailsV2 @@ -808,7 +810,7 @@ func appendScanDetail(logStr string, columnFamily string, scanInfo *kvrpcpb.Scan return logStr } -func (worker *copIteratorWorker) handleCopStreamResult(bo *tikv.Backoffer, rpcCtx *tikv.RPCContext, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan<- *copResponse, costTime time.Duration) ([]*copTask, error) { +func (worker *copIteratorWorker) handleCopStreamResult(bo *backoffer, rpcCtx *tikv.RPCContext, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan<- *copResponse, costTime time.Duration) ([]*copTask, error) { defer stream.Close() var resp *coprocessor.Response var lastRange *coprocessor.KeyRange @@ -854,7 +856,7 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *tikv.Backoffer, rpcCt // returns more tasks when that happens, or handles the response if no error. // if we're handling streaming coprocessor response, lastRange is the range of last // successful response, otherwise it's nil. -func (worker *copIteratorWorker) handleCopResponse(bo *tikv.Backoffer, rpcCtx *tikv.RPCContext, resp *copResponse, cacheKey []byte, cacheValue *coprCacheValue, task *copTask, ch chan<- *copResponse, lastRange *coprocessor.KeyRange, costTime time.Duration) ([]*copTask, error) { +func (worker *copIteratorWorker) handleCopResponse(bo *backoffer, rpcCtx *tikv.RPCContext, resp *copResponse, cacheKey []byte, cacheValue *coprCacheValue, task *copTask, ch chan<- *copResponse, lastRange *coprocessor.KeyRange, costTime time.Duration) ([]*copTask, error) { if regionErr := resp.pbResp.GetRegionError(); regionErr != nil { if rpcCtx != nil && task.storeType == kv.TiDB { resp.err = errors.Errorf("error: %v", regionErr) @@ -872,7 +874,8 @@ func (worker *copIteratorWorker) handleCopResponse(bo *tikv.Backoffer, rpcCtx *t if lockErr := resp.pbResp.GetLocked(); lockErr != nil { logutil.BgLogger().Debug("coprocessor encounters", zap.Stringer("lock", lockErr)) - msBeforeExpired, err1 := worker.ResolveLocks(bo, worker.req.StartTs, []*tikv.Lock{tikv.NewLock(lockErr)}) + msBeforeExpired, err1 := worker.kvclient.ResolveLocks(bo.TiKVBackoffer(), worker.req.StartTs, []*tikv.Lock{tikv.NewLock(lockErr)}) + err1 = txndriver.ToTiDBErr(err1) if err1 != nil { return nil, errors.Trace(err1) } @@ -901,8 +904,8 @@ func (worker *copIteratorWorker) handleCopResponse(bo *tikv.Backoffer, rpcCtx *t if resp.detail == nil { resp.detail = new(CopRuntimeStats) } - resp.detail.Stats = worker.Stats - worker.Stats = nil + resp.detail.Stats = worker.kvclient.Stats + worker.kvclient.Stats = nil backoffTimes := bo.GetBackoffTimes() resp.detail.BackoffTime = time.Duration(bo.GetTotalSleep()) * time.Millisecond resp.detail.BackoffSleep = make(map[string]time.Duration, len(backoffTimes)) @@ -980,11 +983,11 @@ type CopRuntimeStats struct { func (worker *copIteratorWorker) handleTiDBSendReqErr(err error, task *copTask, ch chan<- *copResponse) error { errCode := errno.ErrUnknown errMsg := err.Error() - if terror.ErrorEqual(err, tikverr.ErrTiKVServerTimeout) { + if terror.ErrorEqual(err, txndriver.ErrTiKVServerTimeout) { errCode = errno.ErrTiKVServerTimeout errMsg = "TiDB server timeout, address is " + task.storeAddr } - if terror.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) { + if terror.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) { errCode = errno.ErrTiFlashServerTimeout errMsg = "TiDB server timeout, address is " + task.storeAddr } @@ -1010,7 +1013,7 @@ func (worker *copIteratorWorker) handleTiDBSendReqErr(err error, task *copTask, return nil } -func (worker *copIteratorWorker) buildCopTasksFromRemain(bo *tikv.Backoffer, lastRange *coprocessor.KeyRange, task *copTask) ([]*copTask, error) { +func (worker *copIteratorWorker) buildCopTasksFromRemain(bo *backoffer, lastRange *coprocessor.KeyRange, task *copTask) ([]*copTask, error) { remainedRanges := task.ranges if worker.req.Streaming && lastRange != nil { remainedRanges = worker.calculateRemain(task.ranges, lastRange, worker.req.Desc) @@ -1191,3 +1194,26 @@ func (e *rateLimitAction) setEnabled(enabled bool) { func (e *rateLimitAction) isEnabled() bool { return atomic.LoadUint32(&e.enabled) > 0 } + +// priorityToPB converts priority type to wire type. +func priorityToPB(pri int) kvrpcpb.CommandPri { + switch pri { + case kv.PriorityLow: + return kvrpcpb.CommandPri_Low + case kv.PriorityHigh: + return kvrpcpb.CommandPri_High + default: + return kvrpcpb.CommandPri_Normal + } +} + +func isolationLevelToPB(level kv.IsoLevel) kvrpcpb.IsolationLevel { + switch level { + case kv.RC: + return kvrpcpb.IsolationLevel_RC + case kv.SI: + return kvrpcpb.IsolationLevel_SI + default: + return kvrpcpb.IsolationLevel_SI + } +} diff --git a/store/copr/coprocessor_cache.go b/store/copr/coprocessor_cache.go index 377690fc8d543..d4ddf1806d35b 100644 --- a/store/copr/coprocessor_cache.go +++ b/store/copr/coprocessor_cache.go @@ -61,7 +61,7 @@ func newCoprCache(config *config.CoprocessorCache) (*coprCache, error) { return nil, nil } capacityInBytes := int64(config.CapacityMB * 1024.0 * 1024.0) - if capacityInBytes == 0 { + if capacityInBytes <= 0 { return nil, errors.New("Capacity must be > 0 to enable the cache") } maxEntityInBytes := int64(config.AdmissionMaxResultMB * 1024.0 * 1024.0) diff --git a/store/copr/coprocessor_cache_test.go b/store/copr/coprocessor_cache_test.go index 14084f978d20a..f257e161e2920 100644 --- a/store/copr/coprocessor_cache_test.go +++ b/store/copr/coprocessor_cache_test.go @@ -192,3 +192,8 @@ func (s *testCoprocessorSuite) TestGetSet(c *C) { c.Assert(v, NotNil) c.Assert(v.Data, DeepEquals, []byte("bar")) } + +func (s *testCoprocessorSuite) TestIssue24118(c *C) { + _, err := newCoprCache(&config.CoprocessorCache{AdmissionMinProcessMs: 5, AdmissionMaxResultMB: 1, CapacityMB: -1}) + c.Assert(err.Error(), Equals, "Capacity must be > 0 to enable the cache") +} diff --git a/store/copr/coprocessor_test.go b/store/copr/coprocessor_test.go index 73950f80390af..d7a6d52c5b4bb 100644 --- a/store/copr/coprocessor_test.go +++ b/store/copr/coprocessor_test.go @@ -43,7 +43,7 @@ func (s *testCoprocessorSuite) TestBuildTasks(c *C) { cache := tikv.NewRegionCache(pdCli) defer cache.Close() - bo := tikv.NewBackofferWithVars(context.Background(), 3000, nil) + bo := newBackofferWithVars(context.Background(), 3000, nil) req := &kv.Request{} flashReq := &kv.Request{} @@ -212,7 +212,7 @@ func (s *testCoprocessorSuite) TestRebuild(c *C) { pdCli := &tikv.CodecPDClient{Client: mocktikv.NewPDClient(cluster)} cache := tikv.NewRegionCache(pdCli) defer cache.Close() - bo := tikv.NewBackofferWithVars(context.Background(), 3000, nil) + bo := newBackofferWithVars(context.Background(), 3000, nil) req := &kv.Request{} tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "z"), req) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 74fe82627a036..3ea07e744f9b9 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/tidb/kv" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" @@ -36,7 +37,7 @@ import ( // MPPClient servers MPP requests. type MPPClient struct { - store *tikv.KVStore + store *kvStore } // GetAddress returns the network address. @@ -56,7 +57,7 @@ func (c *MPPClient) selectAllTiFlashStore() []kv.MPPTaskMeta { // ConstructMPPTasks receives ScheduleRequest, which are actually collects of kv ranges. We allocates MPPTaskMeta for them and returns. func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasksRequest) ([]kv.MPPTaskMeta, error) { ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTS) - bo := tikv.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) + bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) if req.KeyRanges == nil { return c.selectAllTiFlashStore(), nil } @@ -117,7 +118,7 @@ func (m *mppResponse) RespTime() time.Duration { } type mppIterator struct { - store *tikv.KVStore + store *kvStore tasks []*kv.MPPDispatchRequest finishCh chan struct{} @@ -152,7 +153,7 @@ func (m *mppIterator) run(ctx context.Context) { break } m.wg.Add(1) - bo := tikv.NewBackoffer(ctx, copNextMaxBackoff) + bo := newBackoffer(ctx, copNextMaxBackoff) go m.handleDispatchReq(ctx, bo, task) } m.wg.Wait() @@ -176,7 +177,7 @@ func (m *mppIterator) sendToRespCh(resp *mppResponse) (exit bool) { // TODO:: Consider that which way is better: // - dispatch all tasks at once, and connect tasks at second. // - dispatch tasks and establish connection at the same time. -func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, req *kv.MPPDispatchRequest) { +func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req *kv.MPPDispatchRequest) { defer func() { m.wg.Done() }() @@ -225,7 +226,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, if sender.GetRPCError() != nil { logutil.BgLogger().Error("mpp dispatch meet io error", zap.String("error", sender.GetRPCError().Error())) // we return timeout to trigger tikv's fallback - m.sendError(tikverr.ErrTiFlashServerTimeout) + m.sendError(txndriver.ErrTiFlashServerTimeout) return } } else { @@ -235,7 +236,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, if err != nil { logutil.BgLogger().Error("mpp dispatch meet error", zap.String("error", err.Error())) // we return timeout to trigger tikv's fallback - m.sendError(tikverr.ErrTiFlashServerTimeout) + m.sendError(txndriver.ErrTiFlashServerTimeout) return } @@ -245,10 +246,17 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *tikv.Backoffer, m.sendError(errors.New(realResp.Error.Msg)) return } + if len(realResp.RetryRegions) > 0 { + for _, retry := range realResp.RetryRegions { + id := tikv.NewRegionVerID(retry.Id, retry.RegionEpoch.ConfVer, retry.RegionEpoch.Version) + logutil.BgLogger().Info("invalid region because tiflash detected stale region", zap.String("region id", id.String())) + m.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) + } + } failpoint.Inject("mppNonRootTaskError", func(val failpoint.Value) { if val.(bool) && !req.IsRoot { time.Sleep(1 * time.Second) - m.sendError(tikverr.ErrTiFlashServerTimeout) + m.sendError(txndriver.ErrTiFlashServerTimeout) return } }) @@ -292,7 +300,7 @@ func (m *mppIterator) cancelMppTasks() { } } -func (m *mppIterator) establishMPPConns(bo *tikv.Backoffer, req *kv.MPPDispatchRequest, taskMeta *mpp.TaskMeta) { +func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchRequest, taskMeta *mpp.TaskMeta) { connReq := &mpp.EstablishMPPConnectionRequest{ SenderMeta: taskMeta, ReceiverMeta: &mpp.TaskMeta{ @@ -311,7 +319,7 @@ func (m *mppIterator) establishMPPConns(bo *tikv.Backoffer, req *kv.MPPDispatchR if err != nil { logutil.BgLogger().Error("establish mpp connection meet error", zap.String("error", err.Error())) // we return timeout to trigger tikv's fallback - m.sendError(tikverr.ErrTiFlashServerTimeout) + m.sendError(txndriver.ErrTiFlashServerTimeout) return } @@ -343,7 +351,7 @@ func (m *mppIterator) establishMPPConns(bo *tikv.Backoffer, req *kv.MPPDispatchR logutil.BgLogger().Info("stream unknown error", zap.Error(err)) } } - m.sendError(tikverr.ErrTiFlashServerTimeout) + m.sendError(txndriver.ErrTiFlashServerTimeout) return } } @@ -359,7 +367,7 @@ func (m *mppIterator) Close() error { return nil } -func (m *mppIterator) handleMPPStreamResponse(bo *tikv.Backoffer, response *mpp.MPPDataPacket, req *kv.MPPDispatchRequest) (err error) { +func (m *mppIterator) handleMPPStreamResponse(bo *backoffer, response *mpp.MPPDataPacket, req *kv.MPPDispatchRequest) (err error) { if response.Error != nil { err = errors.Errorf("other error for mpp stream: %s", response.Error.Msg) logutil.BgLogger().Warn("other error", diff --git a/store/copr/store.go b/store/copr/store.go index 2d0d42a38efb1..2cc10ee7bad38 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -14,31 +14,70 @@ package copr import ( + "context" "math/rand" "sync/atomic" + "time" "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/tikvrpc" ) +type kvStore struct { + store *tikv.KVStore +} + +// GetRegionCache returns the region cache instance. +func (s *kvStore) GetRegionCache() *tikv.RegionCache { + return s.store.GetRegionCache() +} + +// CheckVisibility checks if it is safe to read using given ts. +func (s *kvStore) CheckVisibility(startTime uint64) error { + err := s.store.CheckVisibility(startTime) + return txndriver.ToTiDBErr(err) +} + +// GetTiKVClient gets the client instance. +func (s *kvStore) GetTiKVClient() tikv.Client { + client := s.store.GetTiKVClient() + return &tikvClient{c: client} +} + +type tikvClient struct { + c tikv.Client +} + +func (c *tikvClient) Close() error { + err := c.c.Close() + return txndriver.ToTiDBErr(err) +} + +// SendRequest sends Request. +func (c *tikvClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { + res, err := c.c.SendRequest(ctx, addr, req, timeout) + return res, txndriver.ToTiDBErr(err) +} + // Store wraps tikv.KVStore and provides coprocessor utilities. type Store struct { - *tikv.KVStore + *kvStore coprCache *coprCache replicaReadSeed uint32 } // NewStore creates a new store instance. -func NewStore(kvStore *tikv.KVStore, coprCacheConfig *config.CoprocessorCache) (*Store, error) { +func NewStore(s *tikv.KVStore, coprCacheConfig *config.CoprocessorCache) (*Store, error) { coprCache, err := newCoprCache(coprCacheConfig) if err != nil { return nil, errors.Trace(err) } return &Store{ - KVStore: kvStore, + kvStore: &kvStore{store: s}, coprCache: coprCache, replicaReadSeed: rand.Uint32(), }, nil @@ -66,7 +105,7 @@ func (s *Store) GetClient() kv.Client { // GetMPPClient gets a mpp client instance. func (s *Store) GetMPPClient() kv.MPPClient { return &MPPClient{ - store: s.KVStore, + store: s.kvStore, } } @@ -82,3 +121,63 @@ func getEndPointType(t kv.StoreType) tikvrpc.EndpointType { return tikvrpc.TiKV } } + +// backoffer wraps tikv.Backoffer and converts the error which returns by the functions of tikv.Backoffer to tidb error. +type backoffer struct { + b *tikv.Backoffer +} + +// newBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. +func newBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *backoffer { + b := tikv.NewBackofferWithVars(ctx, maxSleep, vars) + return &backoffer{b: b} +} + +func newBackoffer(ctx context.Context, maxSleep int) *backoffer { + b := tikv.NewBackoffer(ctx, maxSleep) + return &backoffer{b: b} +} + +// TiKVBackoffer returns tikv.Backoffer. +func (b *backoffer) TiKVBackoffer() *tikv.Backoffer { + return b.b +} + +// Backoff sleeps a while base on the backoffType and records the error message. +// It returns a retryable error if total sleep time exceeds maxSleep. +func (b *backoffer) Backoff(typ tikv.BackoffType, err error) error { + e := b.b.Backoff(typ, err) + return txndriver.ToTiDBErr(e) +} + +// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message +// and never sleep more than maxSleepMs for each sleep. +func (b *backoffer) BackoffWithMaxSleep(typ tikv.BackoffType, maxSleepMs int, err error) error { + e := b.b.BackoffWithMaxSleep(typ, maxSleepMs, err) + return txndriver.ToTiDBErr(e) +} + +// GetBackoffTimes returns a map contains backoff time count by type. +func (b *backoffer) GetBackoffTimes() map[tikv.BackoffType]int { + return b.b.GetBackoffTimes() +} + +// GetCtx returns the binded context. +func (b *backoffer) GetCtx() context.Context { + return b.b.GetCtx() +} + +// GetVars returns the binded vars. +func (b *backoffer) GetVars() *tikv.Variables { + return b.b.GetVars() +} + +// GetBackoffSleepMS returns a map contains backoff sleep time by type. +func (b *backoffer) GetBackoffSleepMS() map[tikv.BackoffType]int { + return b.b.GetBackoffSleepMS() +} + +// GetTotalSleep returns total sleep time. +func (b *backoffer) GetTotalSleep() int { + return b.b.GetTotalSleep() +} diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index 133595af3d51f..17308d33c6be3 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -262,7 +262,7 @@ func (s *tikvStore) StartGCWorker() error { gcWorker, err := gcworker.NewGCWorker(s, s.pdClient) if err != nil { - return errors.Trace(err) + return txn_driver.ToTiDBErr(err) } gcWorker.Start() s.gcWorker = gcWorker @@ -286,7 +286,8 @@ func (s *tikvStore) Close() error { s.gcWorker.Close() } s.coprStore.Close() - return s.KVStore.Close() + err := s.KVStore.Close() + return txn_driver.ToTiDBErr(err) } // GetMemCache return memory manager of the storage @@ -298,7 +299,7 @@ func (s *tikvStore) GetMemCache() kv.MemManager { func (s *tikvStore) Begin() (kv.Transaction, error) { txn, err := s.KVStore.Begin() if err != nil { - return nil, errors.Trace(err) + return nil, txn_driver.ToTiDBErr(err) } return txn_driver.NewTiKVTxn(txn), err } @@ -323,7 +324,7 @@ func (s *tikvStore) BeginWithOption(option kv.TransactionOption) (kv.Transaction txn, err = s.BeginWithTxnScope(txnScope) } if err != nil { - return nil, errors.Trace(err) + return nil, txn_driver.ToTiDBErr(err) } return txn_driver.NewTiKVTxn(txn), err @@ -338,5 +339,10 @@ func (s *tikvStore) GetSnapshot(ver kv.Version) kv.Snapshot { // CurrentVersion returns current max committed version with the given txnScope (local or global). func (s *tikvStore) CurrentVersion(txnScope string) (kv.Version, error) { ver, err := s.KVStore.CurrentTimestamp(txnScope) - return kv.NewVersion(ver), err + return kv.NewVersion(ver), txn_driver.ToTiDBErr(err) +} + +// ShowStatus returns the specified status of the storage +func (s *tikvStore) ShowStatus(ctx context.Context, key string) (interface{}, error) { + return nil, kv.ErrNotImplemented } diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 31f1fd15445ab..6794a016a1060 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -25,15 +25,39 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/dbterror" "go.uber.org/zap" ) +// tikv error instance +var ( + // ErrTiKVServerTimeout is the error when tikv server is timeout. + ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) + ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerTimeout) + // ErrGCTooEarly is the error that GC life time is shorter than transaction duration + ErrGCTooEarly = dbterror.ClassTiKV.NewStd(errno.ErrGCTooEarly) + // ErrTiKVStaleCommand is the error that the command is stale in tikv. + ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStaleCommand) + // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. + ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) + ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) + // ErrTiKVServerBusy is the error when tikv server is busy. + ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) + // ErrTiFlashServerBusy is the error that tiflash server is busy. + ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerBusy) + // ErrPDServerTimeout is the error when pd server is timeout. + ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) + // ErrRegionUnavailable is the error when region is not available. + ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) +) + func genKeyExistsError(name string, value string, err error) error { if err != nil { logutil.BgLogger().Info("extractKeyExistsErr meets error", zap.Error(err)) @@ -145,17 +169,84 @@ func extractKeyErr(err error) error { notFoundDetail := prettyLockNotFoundKey(e.Retryable) return kv.ErrTxnRetryable.GenWithStackByArgs(e.Retryable + " " + notFoundDetail) } - return toTiDBErr(err) + return ToTiDBErr(err) } -func toTiDBErr(err error) error { +// ToTiDBErr checks and converts a tikv error to a tidb error. +func ToTiDBErr(err error) error { + originErr := err if err == nil { return nil } + err = errors.Cause(err) if tikverr.IsErrNotFound(err) { return kv.ErrNotExist } - return errors.Trace(err) + + if e, ok := err.(*tikverr.ErrWriteConflictInLatch); ok { + return kv.ErrWriteConflictInTiDB.FastGenByArgs(e.StartTS) + } + + if e, ok := err.(*tikverr.ErrTxnTooLarge); ok { + return kv.ErrTxnTooLarge.GenWithStackByArgs(e.Size) + } + + if errors.ErrorEqual(err, tikverr.ErrCannotSetNilValue) { + return kv.ErrCannotSetNilValue + } + + if e, ok := err.(*tikverr.ErrEntryTooLarge); ok { + return kv.ErrEntryTooLarge.GenWithStackByArgs(e.Limit, e.Size) + } + + if errors.ErrorEqual(err, tikverr.ErrInvalidTxn) { + return kv.ErrInvalidTxn + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVServerTimeout) { + return ErrTiKVServerTimeout + } + + if e, ok := err.(*tikverr.ErrPDServerTimeout); ok { + if len(e.Error()) == 0 { + return ErrPDServerTimeout + } + return ErrPDServerTimeout.GenWithStackByArgs(e.Error()) + } + + if errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) { + return ErrTiFlashServerTimeout + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { + return ErrTiKVServerBusy + } + + if errors.ErrorEqual(err, tikverr.ErrTiFlashServerBusy) { + return ErrTiFlashServerBusy + } + + if e, ok := err.(*tikverr.ErrGCTooEarly); ok { + return ErrGCTooEarly.GenWithStackByArgs(e.TxnStartTS, e.GCSafePoint) + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVStaleCommand) { + return ErrTiKVStaleCommand + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVMaxTimestampNotSynced) { + return ErrTiKVMaxTimestampNotSynced + } + + if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { + return ErrResolveLockTimeout + } + + if errors.ErrorEqual(err, tikverr.ErrRegionUnavailable) { + return ErrRegionUnavailable + } + + return errors.Trace(originErr) } func newWriteConflictError(conflict *kvrpcpb.WriteConflict) error { diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index afcb9eaee3f58..1a3f7bf9c3bbc 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv" + tikvstore "github.com/pingcap/tidb/store/tikv/kv" ) type tikvSnapshot struct { @@ -47,7 +48,7 @@ func (s *tikvSnapshot) Get(ctx context.Context, k kv.Key) ([]byte, error) { func (s *tikvSnapshot) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { scanner, err := s.KVSnapshot.Iter(k, upperBound) if err != nil { - return nil, toTiDBErr(err) + return nil, ToTiDBErr(err) } return &tikvScanner{scanner.(*tikv.Scanner)}, err } @@ -56,12 +57,50 @@ func (s *tikvSnapshot) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { func (s *tikvSnapshot) IterReverse(k kv.Key) (kv.Iterator, error) { scanner, err := s.KVSnapshot.IterReverse(k) if err != nil { - return nil, toTiDBErr(err) + return nil, ToTiDBErr(err) } return &tikvScanner{scanner.(*tikv.Scanner)}, err } +func (s *tikvSnapshot) SetOption(opt int, val interface{}) { + switch opt { + case tikvstore.IsolationLevel: + level := getTiKVIsolationLevel(val.(kv.IsoLevel)) + s.KVSnapshot.SetIsolationLevel(level) + case tikvstore.Priority: + s.KVSnapshot.SetPriority(getTiKVPriority(val.(int))) + case tikvstore.NotFillCache: + s.KVSnapshot.SetNotFillCache(val.(bool)) + case tikvstore.SnapshotTS: + s.KVSnapshot.SetSnapshotTS(val.(uint64)) + default: + s.KVSnapshot.SetOption(opt, val) + } +} + func toTiKVKeys(keys []kv.Key) [][]byte { bytesKeys := *(*[][]byte)(unsafe.Pointer(&keys)) return bytesKeys } + +func getTiKVIsolationLevel(level kv.IsoLevel) tikv.IsoLevel { + switch level { + case kv.SI: + return tikv.SI + case kv.RC: + return tikv.RC + default: + return tikv.SI + } +} + +func getTiKVPriority(pri int) tikv.Priority { + switch pri { + case kv.PriorityHigh: + return tikv.PriorityHigh + case kv.PriorityLow: + return tikv.PriorityLow + default: + return tikv.PriorityNormal + } +} diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 6b39c1cca1e24..8595b2fe874ae 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -35,7 +35,7 @@ type tikvTxn struct { // NewTiKVTxn returns a new Transaction. func NewTiKVTxn(txn *tikv.KVTxn) kv.Transaction { - txn.SetOption(tikvstore.KVFilter, TiDBKVFilter{}) + txn.SetKVFilter(TiDBKVFilter{}) entryLimit := atomic.LoadUint64(&kv.TxnEntrySizeLimit) totalLimit := atomic.LoadUint64(&kv.TxnTotalSizeLimit) @@ -75,7 +75,7 @@ func (txn *tikvTxn) GetSnapshot() kv.Snapshot { // The Iterator must be Closed after use. func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := txn.KVTxn.Iter(k, upperBound) - return newKVIterator(it), toTiDBErr(err) + return newKVIterator(it), ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -84,7 +84,7 @@ func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { // TODO: Add lower bound limit func (txn *tikvTxn) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := txn.KVTxn.IterReverse(k) - return newKVIterator(it), toTiDBErr(err) + return newKVIterator(it), ToTiDBErr(err) } // BatchGet gets kv from the memory buffer of statement and transaction, and the kv storage. @@ -101,17 +101,17 @@ func (txn *tikvTxn) BatchGet(ctx context.Context, keys []kv.Key) (map[string][]b func (txn *tikvTxn) Delete(k kv.Key) error { err := txn.KVTxn.Delete(k) - return toTiDBErr(err) + return ToTiDBErr(err) } func (txn *tikvTxn) Get(ctx context.Context, k kv.Key) ([]byte, error) { data, err := txn.KVTxn.Get(ctx, k) - return data, toTiDBErr(err) + return data, ToTiDBErr(err) } func (txn *tikvTxn) Set(k kv.Key, v []byte) error { err := txn.KVTxn.Set(k, v) - return toTiDBErr(err) + return ToTiDBErr(err) } func (txn *tikvTxn) GetMemBuffer() kv.MemBuffer { @@ -129,11 +129,43 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn: txn.KVTxn, binInfo: val.(*binloginfo.BinlogInfo), // val cannot be other type. }) + case tikvstore.SchemaChecker: + txn.SetSchemaLeaseChecker(val.(tikv.SchemaLeaseChecker)) + case tikvstore.IsolationLevel: + level := getTiKVIsolationLevel(val.(kv.IsoLevel)) + txn.KVTxn.GetSnapshot().SetIsolationLevel(level) + case tikvstore.Priority: + txn.KVTxn.SetPriority(getTiKVPriority(val.(int))) + case tikvstore.NotFillCache: + txn.KVTxn.GetSnapshot().SetNotFillCache(val.(bool)) + case tikvstore.SyncLog: + txn.EnableForceSyncLog() + case tikvstore.Pessimistic: + txn.SetPessimistic(val.(bool)) + case tikvstore.SnapshotTS: + txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) + case tikvstore.InfoSchema: + txn.SetSchemaVer(val.(tikv.SchemaVer)) + case tikvstore.CommitHook: + txn.SetCommitCallback(val.(func(string, error))) + case tikvstore.Enable1PC: + txn.SetEnable1PC(val.(bool)) + case tikvstore.TxnScope: + txn.SetScope(val.(string)) default: txn.KVTxn.SetOption(opt, val) } } +func (txn *tikvTxn) GetOption(opt int) interface{} { + switch opt { + case tikvstore.TxnScope: + return txn.KVTxn.GetScope() + default: + return txn.KVTxn.GetOption(opt) + } +} + // SetVars sets variables to the transaction. func (txn *tikvTxn) SetVars(vars interface{}) { if vs, ok := vars.(*tikv.Variables); ok { diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index 31cb1f2a35f06..9db2325a0148f 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -39,17 +39,17 @@ func (m *memBuffer) Delete(k kv.Key) error { func (m *memBuffer) DeleteWithFlags(k kv.Key, ops ...tikvstore.FlagsOp) error { err := m.MemDB.DeleteWithFlags(k, ops...) - return toTiDBErr(err) + return ToTiDBErr(err) } func (m *memBuffer) Get(_ context.Context, key kv.Key) ([]byte, error) { data, err := m.MemDB.Get(key) - return data, toTiDBErr(err) + return data, ToTiDBErr(err) } func (m *memBuffer) GetFlags(key kv.Key) (tikvstore.KeyFlags, error) { data, err := m.MemDB.GetFlags(key) - return data, toTiDBErr(err) + return data, ToTiDBErr(err) } func (m *memBuffer) Staging() kv.StagingHandle { @@ -73,12 +73,12 @@ func (m *memBuffer) InspectStage(handle kv.StagingHandle, f func(kv.Key, tikvsto func (m *memBuffer) Set(key kv.Key, value []byte) error { err := m.MemDB.Set(key, value) - return toTiDBErr(err) + return ToTiDBErr(err) } func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) error { err := m.MemDB.SetWithFlags(key, value, ops...) - return toTiDBErr(err) + return ToTiDBErr(err) } // Iter creates an Iterator positioned on the first entry that k <= entry's key. @@ -87,7 +87,7 @@ func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) er // The Iterator must be Closed after use. func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := m.MemDB.Iter(k, upperBound) - return &tikvIterator{Iterator: it}, toTiDBErr(err) + return &tikvIterator{Iterator: it}, ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -96,7 +96,7 @@ func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { // TODO: Add lower bound limit func (m *memBuffer) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := m.MemDB.IterReverse(k) - return &tikvIterator{Iterator: it}, toTiDBErr(err) + return &tikvIterator{Iterator: it}, ToTiDBErr(err) } // SnapshotIter returns a Iterator for a snapshot of MemBuffer. @@ -121,7 +121,7 @@ func (u *tikvUnionStore) GetMemBuffer() kv.MemBuffer { func (u *tikvUnionStore) Get(ctx context.Context, k kv.Key) ([]byte, error) { data, err := u.KVUnionStore.Get(ctx, k) - return data, toTiDBErr(err) + return data, ToTiDBErr(err) } func (u *tikvUnionStore) HasPresumeKeyNotExists(k kv.Key) bool { @@ -134,7 +134,7 @@ func (u *tikvUnionStore) UnmarkPresumeKeyNotExists(k kv.Key) { func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := u.KVUnionStore.Iter(k, upperBound) - return newKVIterator(it), toTiDBErr(err) + return newKVIterator(it), ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -143,7 +143,7 @@ func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) // TODO: Add lower bound limit func (u *tikvUnionStore) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := u.KVUnionStore.IterReverse(k) - return newKVIterator(it), toTiDBErr(err) + return newKVIterator(it), ToTiDBErr(err) } type tikvGetter struct { @@ -156,7 +156,7 @@ func newKVGetter(getter unionstore.Getter) kv.Getter { func (g *tikvGetter) Get(_ context.Context, k kv.Key) ([]byte, error) { data, err := g.Getter.Get(k) - return data, toTiDBErr(err) + return data, ToTiDBErr(err) } // tikvIterator wraps unionstore.Iterator as kv.Iterator diff --git a/store/mockstore/mockcopr/executor_test.go b/store/mockstore/mockcopr/executor_test.go index 39dd945d23f89..af9ac45beae96 100644 --- a/store/mockstore/mockcopr/executor_test.go +++ b/store/mockstore/mockcopr/executor_test.go @@ -55,7 +55,8 @@ func (s *testExecutorSuite) SetUpSuite(c *C) { s.mvccStore = rpcClient.MvccStore store, err := tikv.NewTestTiKVStore(rpcClient, pdClient, nil, nil, 0) c.Assert(err, IsNil) - s.store = mockstorage.NewMockStorage(store) + s.store, err = mockstorage.NewMockStorage(store) + c.Assert(err, IsNil) session.SetSchemaLease(0) session.DisableStats4Test() s.dom, err = session.BootstrapSession(s.store) diff --git a/store/mockstore/mockstorage/storage.go b/store/mockstore/mockstorage/storage.go index ee2c556b4dfb6..897ff3d550cf8 100644 --- a/store/mockstore/mockstorage/storage.go +++ b/store/mockstore/mockstorage/storage.go @@ -14,6 +14,7 @@ package mockstorage import ( + "context" "crypto/tls" "github.com/pingcap/tidb/kv" @@ -32,17 +33,18 @@ type mockStorage struct { } // NewMockStorage wraps tikv.KVStore as kv.Storage. -func NewMockStorage(tikvStore *tikv.KVStore) kv.Storage { +func NewMockStorage(tikvStore *tikv.KVStore) (kv.Storage, error) { coprConfig := config.DefaultConfig().TiKVClient.CoprCache coprStore, err := copr.NewStore(tikvStore, &coprConfig) if err != nil { - panic(err) + return nil, err } return &mockStorage{ KVStore: tikvStore, Store: coprStore, memCache: kv.NewCacheDB(), - } + }, nil + } func (s *mockStorage) EtcdAddrs() ([]string, error) { @@ -76,6 +78,11 @@ func (s *mockStorage) Begin() (kv.Transaction, error) { return newTiKVTxn(txn, err) } +// ShowStatus returns the specified status of the storage +func (s *mockStorage) ShowStatus(ctx context.Context, key string) (interface{}, error) { + return nil, kv.ErrNotImplemented +} + // BeginWithOption begins a transaction with given option func (s *mockStorage) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { txnScope := option.TxnScope diff --git a/store/mockstore/tikv.go b/store/mockstore/tikv.go index 10378a51e34b2..ff179c4544c59 100644 --- a/store/mockstore/tikv.go +++ b/store/mockstore/tikv.go @@ -35,5 +35,5 @@ func newMockTikvStore(opt *mockOptions) (kv.Storage, error) { if err != nil { return nil, err } - return mockstorage.NewMockStorage(kvstore), nil + return mockstorage.NewMockStorage(kvstore) } diff --git a/store/mockstore/unistore.go b/store/mockstore/unistore.go index c712a979f821c..5476a6f6e7499 100644 --- a/store/mockstore/unistore.go +++ b/store/mockstore/unistore.go @@ -36,5 +36,5 @@ func newUnistore(opts *mockOptions) (kv.Storage, error) { if err != nil { return nil, err } - return mockstorage.NewMockStorage(kvstore), nil + return mockstorage.NewMockStorage(kvstore) } diff --git a/store/mockstore/unistore/cophandler/mpp.go b/store/mockstore/unistore/cophandler/mpp.go index 7f5dfdc9e8707..bed35eebee01e 100644 --- a/store/mockstore/unistore/cophandler/mpp.go +++ b/store/mockstore/unistore/cophandler/mpp.go @@ -206,6 +206,11 @@ func (b *mppExecBuilder) buildMPPJoin(pb *tipb.Join, children []*tipb.Executor) } e.probeKey = probeExpr.(*expression.Column) } + e.comKeyTp = types.AggFieldType([]*types.FieldType{e.probeKey.RetType, e.buildKey.RetType}) + if e.comKeyTp.Tp == mysql.TypeNewDecimal { + e.comKeyTp.Flen = mysql.MaxDecimalWidth + e.comKeyTp.Decimal = mysql.MaxDecimalScale + } return e, nil } diff --git a/store/mockstore/unistore/cophandler/mpp_exec.go b/store/mockstore/unistore/cophandler/mpp_exec.go index 960f3bd976e60..0072456ecf8c3 100644 --- a/store/mockstore/unistore/cophandler/mpp_exec.go +++ b/store/mockstore/unistore/cophandler/mpp_exec.go @@ -343,6 +343,20 @@ type joinExec struct { defaultInner chunk.Row inited bool + // align the types of join keys and build keys + comKeyTp *types.FieldType +} + +func (e *joinExec) getHashKey(keyCol types.Datum) (str string, err error) { + keyCol, err = keyCol.ConvertTo(e.sc, e.comKeyTp) + if err != nil { + return str, errors.Trace(err) + } + str, err = keyCol.ToString() + if err != nil { + return "", errors.Trace(err) + } + return str, nil } func (e *joinExec) buildHashTable() error { @@ -358,7 +372,7 @@ func (e *joinExec) buildHashTable() error { for i := 0; i < rows; i++ { row := chk.GetRow(i) keyCol := row.GetDatum(e.buildKey.Index, e.buildChild.getFieldTypes()[e.buildKey.Index]) - key, err := keyCol.ToString() + key, err := e.getHashKey(keyCol) if err != nil { return errors.Trace(err) } @@ -386,7 +400,7 @@ func (e *joinExec) fetchRows() (bool, error) { for i := 0; i < chkSize; i++ { row := chk.GetRow(i) keyCol := row.GetDatum(e.probeKey.Index, e.probeChild.getFieldTypes()[e.probeKey.Index]) - key, err := keyCol.ToString() + key, err := e.getHashKey(keyCol) if err != nil { return false, errors.Trace(err) } diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index 478d11d16e131..f571ff4fe963f 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -951,36 +951,6 @@ func (svr *Server) RemoveLockObserver(context.Context, *kvrpcpb.RemoveLockObserv return &kvrpcpb.RemoveLockObserverResponse{}, nil } -// VerGet implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerGet(context.Context, *kvrpcpb.VerGetRequest) (*kvrpcpb.VerGetResponse, error) { - panic("unimplemented") -} - -// VerBatchGet implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerBatchGet(context.Context, *kvrpcpb.VerBatchGetRequest) (*kvrpcpb.VerBatchGetResponse, error) { - panic("unimplemented") -} - -// VerMut implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerMut(context.Context, *kvrpcpb.VerMutRequest) (*kvrpcpb.VerMutResponse, error) { - panic("unimplemented") -} - -// VerBatchMut implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerBatchMut(context.Context, *kvrpcpb.VerBatchMutRequest) (*kvrpcpb.VerBatchMutResponse, error) { - panic("unimplemented") -} - -// VerScan implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerScan(context.Context, *kvrpcpb.VerScanRequest) (*kvrpcpb.VerScanResponse, error) { - panic("unimplemented") -} - -// VerDeleteRange implements implements the tikvpb.TikvServer interface. -func (svr *Server) VerDeleteRange(context.Context, *kvrpcpb.VerDeleteRangeRequest) (*kvrpcpb.VerDeleteRangeResponse, error) { - panic("unimplemented") -} - // CheckLeader implements implements the tikvpb.TikvServer interface. func (svr *Server) CheckLeader(context.Context, *kvrpcpb.CheckLeaderRequest) (*kvrpcpb.CheckLeaderResponse, error) { panic("unimplemented") diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index c33bf6d0f6d1f..8703b1861c65d 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -327,7 +327,7 @@ func (c *twoPhaseCommitter) initKeysAndMutations() error { sizeHint := txn.us.GetMemBuffer().Len() c.mutations = newMemBufferMutations(sizeHint, memBuf) c.isPessimistic = txn.IsPessimistic() - filter := txn.getKVFilter() + filter := txn.kvFilter var err error for it := memBuf.IterWithFlags(nil, nil); it.Valid(); err = it.Next() { @@ -425,8 +425,8 @@ func (c *twoPhaseCommitter) initKeysAndMutations() error { metrics.TiKVTxnWriteSizeHistogram.Observe(float64(commitDetail.WriteSize)) c.hasNoNeedCommitKeys = checkCnt > 0 c.lockTTL = txnLockTTL(txn.startTime, size) - c.priority = getTxnPriority(txn) - c.syncLog = getTxnSyncLog(txn) + c.priority = txn.priority.ToPB() + c.syncLog = txn.syncLog c.setDetail(commitDetail) return nil } @@ -821,8 +821,7 @@ func sendTxnHeartBeat(bo *Backoffer, store *KVStore, primary []byte, startTS, tt // checkAsyncCommit checks if async commit protocol is available for current transaction commit, true is returned if possible. func (c *twoPhaseCommitter) checkAsyncCommit() bool { // Disable async commit in local transactions - txnScopeOption := c.txn.us.GetOption(kv.TxnScope) - if txnScopeOption == nil || txnScopeOption.(string) != oracle.GlobalTxnScope { + if c.txn.GetScope() != oracle.GlobalTxnScope { return false } @@ -849,13 +848,11 @@ func (c *twoPhaseCommitter) checkAsyncCommit() bool { // checkOnePC checks if 1PC protocol is available for current transaction. func (c *twoPhaseCommitter) checkOnePC() bool { // Disable 1PC in local transactions - txnScopeOption := c.txn.us.GetOption(kv.TxnScope) - if txnScopeOption == nil || txnScopeOption.(string) != oracle.GlobalTxnScope { + if c.txn.GetScope() != oracle.GlobalTxnScope { return false } - enable1PCOption := c.txn.us.GetOption(kv.Enable1PC) - return c.sessionID > 0 && !c.shouldWriteBinlog() && enable1PCOption != nil && enable1PCOption.(bool) + return c.sessionID > 0 && !c.shouldWriteBinlog() && c.txn.enable1PC } func (c *twoPhaseCommitter) needLinearizability() bool { @@ -1099,7 +1096,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { } else { start = time.Now() logutil.Event(ctx, "start get commit ts") - commitTS, err = c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetUnionStore().GetOption(kv.TxnScope).(string)) + commitTS, err = c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) if err != nil { logutil.Logger(ctx).Warn("2PC get commitTS failed", zap.Error(err), @@ -1114,12 +1111,12 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { if !c.isAsyncCommit() { tryAmend := c.isPessimistic && c.sessionID > 0 && c.txn.schemaAmender != nil if !tryAmend { - _, _, err = c.checkSchemaValid(ctx, commitTS, c.txn.txnInfoSchema, false) + _, _, err = c.checkSchemaValid(ctx, commitTS, c.txn.schemaVer, false) if err != nil { return errors.Trace(err) } } else { - relatedSchemaChange, memAmended, err := c.checkSchemaValid(ctx, commitTS, c.txn.txnInfoSchema, true) + relatedSchemaChange, memAmended, err := c.checkSchemaValid(ctx, commitTS, c.txn.schemaVer, true) if err != nil { return errors.Trace(err) } @@ -1252,7 +1249,8 @@ type SchemaVer interface { SchemaMetaVersion() int64 } -type schemaLeaseChecker interface { +// SchemaLeaseChecker is used to validate schema version is not changed during transaction execution. +type SchemaLeaseChecker interface { // CheckBySchemaVer checks if the schema has changed for the transaction related tables between the startSchemaVer // and the schema version at txnTS, all the related schema changes will be returned. CheckBySchemaVer(txnTS uint64, startSchemaVer SchemaVer) (*RelatedSchemaChange, error) @@ -1367,7 +1365,7 @@ func (c *twoPhaseCommitter) tryAmendTxn(ctx context.Context, startInfoSchema Sch func (c *twoPhaseCommitter) getCommitTS(ctx context.Context, commitDetail *util.CommitDetails) (uint64, error) { start := time.Now() logutil.Event(ctx, "start get commit ts") - commitTS, err := c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetUnionStore().GetOption(kv.TxnScope).(string)) + commitTS, err := c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) if err != nil { logutil.Logger(ctx).Warn("2PC get commitTS failed", zap.Error(err), @@ -1398,8 +1396,7 @@ func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64 err := errors.Errorf("mock check schema valid failure") failpoint.Return(nil, false, err) }) - checker, ok := c.txn.us.GetOption(kv.SchemaChecker).(schemaLeaseChecker) - if !ok { + if c.txn.schemaLeaseChecker == nil { if c.sessionID > 0 { logutil.Logger(ctx).Warn("schemaLeaseChecker is not set for this transaction", zap.Uint64("sessionID", c.sessionID), @@ -1408,7 +1405,7 @@ func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64 } return nil, false, nil } - relatedChanges, err := checker.CheckBySchemaVer(checkTS, startInfoSchema) + relatedChanges, err := c.txn.schemaLeaseChecker.CheckBySchemaVer(checkTS, startInfoSchema) if err != nil { if tryAmend && relatedChanges != nil && relatedChanges.Amendable && c.txn.schemaAmender != nil { memAmended, amendErr := c.tryAmendTxn(ctx, startInfoSchema, relatedChanges) @@ -1431,7 +1428,7 @@ func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64 func (c *twoPhaseCommitter) calculateMaxCommitTS(ctx context.Context) error { // Amend txn with current time first, then we can make sure we have another SafeWindow time to commit currentTS := oracle.EncodeTSO(int64(time.Since(c.txn.startTime)/time.Millisecond)) + c.startTS - _, _, err := c.checkSchemaValid(ctx, currentTS, c.txn.txnInfoSchema, true) + _, _, err := c.checkSchemaValid(ctx, currentTS, c.txn.schemaVer, true) if err != nil { logutil.Logger(ctx).Info("Schema changed for async commit txn", zap.Error(err), @@ -1649,32 +1646,6 @@ func (batchExe *batchExecutor) process(batches []batchMutations) error { return err } -func getTxnPriority(txn *KVTxn) pb.CommandPri { - if pri := txn.us.GetOption(kv.Priority); pri != nil { - return PriorityToPB(pri.(int)) - } - return pb.CommandPri_Normal -} - -func getTxnSyncLog(txn *KVTxn) bool { - if syncOption := txn.us.GetOption(kv.SyncLog); syncOption != nil { - return syncOption.(bool) - } - return false -} - -// PriorityToPB converts priority type to wire type. -func PriorityToPB(pri int) pb.CommandPri { - switch pri { - case kv.PriorityLow: - return pb.CommandPri_Low - case kv.PriorityHigh: - return pb.CommandPri_High - default: - return pb.CommandPri_Normal - } -} - func (c *twoPhaseCommitter) setDetail(d *util.CommitDetails) { atomic.StorePointer(&c.detail, unsafe.Pointer(d)) } diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index 03d7930071fb3..e0115c9e3904a 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -200,7 +200,7 @@ func (t BackoffType) TError() error { case BoTxnLock, BoTxnLockFast, boTxnNotFound: return tikverr.ErrResolveLockTimeout case BoPDRPC: - return tikverr.ErrPDServerTimeout + return tikverr.NewErrPDServerTimeout("") case BoRegionMiss: return tikverr.ErrRegionUnavailable case boTiKVServerBusy: diff --git a/store/tikv/commit.go b/store/tikv/commit.go index 73406ff19f4c1..ce9df6a927355 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/errors" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -100,7 +99,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch } // Update commit ts and retry. - commitTS, err := c.store.getTimestampWithRetry(bo, c.txn.GetUnionStore().GetOption(kv.TxnScope).(string)) + commitTS, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) if err != nil { logutil.Logger(bo.ctx).Warn("2PC get commitTS failed", zap.Error(err), diff --git a/store/tikv/config/config.go b/store/tikv/config/config.go index a8337cdcedf8c..6fe0c018498ea 100644 --- a/store/tikv/config/config.go +++ b/store/tikv/config/config.go @@ -36,6 +36,11 @@ const ( DefStoresRefreshInterval = 60 ) +func init() { + conf := DefaultConfig() + StoreGlobalConfig(&conf) +} + // Config contains configuration options. type Config struct { CommitterConcurrency int diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index e84c4577c4058..10cc2292be7e7 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -25,17 +25,5 @@ const ( CodeLockAcquireFailAndNoWaitSet = 3572 // TiKV/PD/TiFlash errors. - CodePDServerTimeout = 9001 - CodeTiKVServerTimeout = 9002 - CodeTiKVServerBusy = 9003 - CodeResolveLockTimeout = 9004 - CodeRegionUnavailable = 9005 - CodeGCTooEarly = 9006 - CodeTiKVStoreLimit = 9008 - - CodeTiKVStaleCommand = 9010 - CodeTiKVMaxTimestampNotSynced = 9011 - CodeTiFlashServerTimeout = 9012 - CodeTiFlashServerBusy = 9013 ) diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index ffa2baf96358a..a4f213ab09a02 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -14,6 +14,9 @@ package error import ( + "fmt" + "time" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/pdpb" @@ -27,6 +30,26 @@ var ( ErrTiDBShuttingDown = errors.New("tidb server shutting down") // ErrNotExist means the related data not exist. ErrNotExist = errors.New("not exist") + // ErrCannotSetNilValue is the error when sets an empty value. + ErrCannotSetNilValue = errors.New("can not set nil value") + // ErrInvalidTxn is the error when commits or rollbacks in an invalid transaction. + ErrInvalidTxn = errors.New("invalid transaction") + // ErrTiKVServerTimeout is the error when tikv server is timeout. + ErrTiKVServerTimeout = errors.New("tikv server timeout") + // ErrTiFlashServerTimeout is the error when tiflash server is timeout. + ErrTiFlashServerTimeout = errors.New("tiflash server timeout") + // ErrTiKVStaleCommand is the error that the command is stale in tikv. + ErrTiKVStaleCommand = errors.New("tikv stale command") + // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. + ErrTiKVMaxTimestampNotSynced = errors.New("tikv max timestamp not synced") + // ErrResolveLockTimeout is the error that resolve lock timeout. + ErrResolveLockTimeout = errors.New("resolve lock timeout") + // ErrTiKVServerBusy is the error when tikv server is busy. + ErrTiKVServerBusy = errors.New("tikv server busy") + // ErrTiFlashServerBusy is the error that tiflash server is busy. + ErrTiFlashServerBusy = errors.New("tiflash server busy") + // ErrRegionUnavailable is the error when region is not available. + ErrRegionUnavailable = errors.New("region unavailable") ) // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. @@ -34,16 +57,6 @@ const MismatchClusterID = "mismatch cluster id" // error instances. var ( - ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiKVServerTimeout) - ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) - ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(CodeResolveLockTimeout) - ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(CodePDServerTimeout) - ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(CodeRegionUnavailable) - ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(CodeTiKVServerBusy) - ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(CodeTiFlashServerBusy) - ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(CodeTiKVStaleCommand) - ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(CodeTiKVMaxTimestampNotSynced) - ErrGCTooEarly = dbterror.ClassTiKV.NewStd(CodeGCTooEarly) ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(CodeLockWaitTimeout) @@ -124,6 +137,15 @@ func NewErrWriteConfictWithArgs(startTs, conflictTs, conflictCommitTs uint64, ke return &ErrWriteConflict{WriteConflict: &conflict} } +// ErrWriteConflictInLatch is the error when the commit meets an write conflict error when local latch is enabled. +type ErrWriteConflictInLatch struct { + StartTS uint64 +} + +func (e *ErrWriteConflictInLatch) Error() string { + return fmt.Sprintf("write conflict in latch,startTS: %v", e.StartTS) +} + // ErrRetryable wraps *kvrpcpb.Retryable to implement the error interface. type ErrRetryable struct { Retryable string @@ -132,3 +154,46 @@ type ErrRetryable struct { func (k *ErrRetryable) Error() string { return k.Retryable } + +// ErrTxnTooLarge is the error when transaction is too large, lock time reached the maximum value. +type ErrTxnTooLarge struct { + Size int +} + +func (e *ErrTxnTooLarge) Error() string { + return fmt.Sprintf("txn too large, size: %v.", e.Size) +} + +// ErrEntryTooLarge is the error when a key value entry is too large. +type ErrEntryTooLarge struct { + Limit uint64 + Size uint64 +} + +func (e *ErrEntryTooLarge) Error() string { + return fmt.Sprintf("entry size too large, size: %v,limit: %v.", e.Size, e.Limit) +} + +// ErrPDServerTimeout is the error when pd server is timeout. +type ErrPDServerTimeout struct { + msg string +} + +// NewErrPDServerTimeout creates an ErrPDServerTimeout. +func NewErrPDServerTimeout(msg string) error { + return &ErrPDServerTimeout{msg} +} + +func (e *ErrPDServerTimeout) Error() string { + return e.msg +} + +// ErrGCTooEarly is the error that GC life time is shorter than transaction duration +type ErrGCTooEarly struct { + TxnStartTS time.Time + GCSafePoint time.Time +} + +func (e *ErrGCTooEarly) Error() string { + return fmt.Sprintf("GC life time is shorter than transaction duration, transaction starts at %v, GC safe point is %v", e.TxnStartTS, e.GCSafePoint) +} diff --git a/store/tikv/interface.go b/store/tikv/interface.go index 507cce87535f0..edc0bd06c0400 100644 --- a/store/tikv/interface.go +++ b/store/tikv/interface.go @@ -14,7 +14,6 @@ package tikv import ( - "context" "time" "github.com/pingcap/tidb/store/tikv/oracle" @@ -60,6 +59,4 @@ type Storage interface { GetOracle() oracle.Oracle // SupportDeleteRange gets the storage support delete range or not. SupportDeleteRange() (supported bool) - // ShowStatus returns the specified status of the storage - ShowStatus(ctx context.Context, key string) (interface{}, error) } diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 3e13e14dbb54b..da9ba844fa000 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" - tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" @@ -108,13 +107,16 @@ func (s *KVStore) CheckVisibility(startTime uint64) error { diff := time.Since(cachedTime) if diff > (GcSafePointCacheInterval - gcCPUTimeInaccuracyBound) { - return tikverr.ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point") + return tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point") } if startTime < cachedSafePoint { t1 := oracle.GetTimeFromTS(startTime) t2 := oracle.GetTimeFromTS(cachedSafePoint) - return tikverr.ErrGCTooEarly.GenWithStackByArgs(t1, t2) + return &tikverr.ErrGCTooEarly{ + TxnStartTS: t1, + GCSafePoint: t2, + } } return nil @@ -292,7 +294,7 @@ func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, // This may cause duplicate data to be written. failpoint.Inject("mockGetTSErrorInRetry", func(val failpoint.Value) { if val.(bool) && !IsMockCommitErrorEnable() { - err = tikverr.ErrPDServerTimeout.GenWithStackByArgs("mock PD timeout") + err = tikverr.NewErrPDServerTimeout("mock PD timeout") } }) @@ -333,11 +335,6 @@ func (s *KVStore) GetPDClient() pd.Client { return s.pdClient } -// ShowStatus returns the specified status of the storage -func (s *KVStore) ShowStatus(ctx context.Context, key string) (interface{}, error) { - return nil, tidbkv.ErrNotImplemented -} - // SupportDeleteRange gets the storage support delete range or not. func (s *KVStore) SupportDeleteRange() (supported bool) { return !s.mock diff --git a/store/tikv/kv/kv.go b/store/tikv/kv/kv.go index 28c919a95627f..2b7e87ecd2e47 100644 --- a/store/tikv/kv/kv.go +++ b/store/tikv/kv/kv.go @@ -7,16 +7,6 @@ import ( "github.com/pingcap/tidb/store/tikv/util" ) -// IsoLevel is the transaction's isolation level. -type IsoLevel int - -const ( - // SI stands for 'snapshot isolation'. - SI IsoLevel = iota - // RC stands for 'read committed'. - RC -) - // ReturnedValue pairs the Value and AlreadyLocked flag for PessimisticLock return values result. type ReturnedValue struct { Value []byte diff --git a/store/tikv/kv/option.go b/store/tikv/kv/option.go index e9479cf609502..bac9316d41773 100644 --- a/store/tikv/kv/option.go +++ b/store/tikv/kv/option.go @@ -59,11 +59,10 @@ const ( IsStalenessReadOnly // MatchStoreLabels indicates the labels the store should be matched MatchStoreLabels - // KVFilter filters out the key-value pairs in the memBuf that is unnecessary to be committed - KVFilter ) // Priority value for transaction priority. +// TODO: remove after BR update. const ( PriorityNormal = iota PriorityLow diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index 8ee1cd411c13f..05feee6d31adb 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -485,12 +485,8 @@ func (lr *LockResolver) getTxnStatusFromLock(bo *Backoffer, l *Lock, callerStart var currentTS uint64 var err error var status TxnStatus - if l.UseAsyncCommit && !forceSyncCommit { - // Async commit doesn't need the current ts since it uses the minCommitTS. - currentTS = 0 - // Set to 0 so as not to push forward min commit ts. - callerStartTS = 0 - } else if l.TTL == 0 { + + if l.TTL == 0 { // NOTE: l.TTL = 0 is a special protocol!!! // When the pessimistic txn prewrite meets locks of a txn, it should resolve the lock **unconditionally**. // In this case, TiKV use lock TTL = 0 to notify TiDB, and TiDB should resolve the lock! diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 9adfa65dd316a..a73684fdf49c5 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -275,9 +275,10 @@ type RegionCache struct { enableForwarding bool mu struct { - sync.RWMutex // mutex protect cached region - regions map[RegionVerID]*Region // cached regions are organized as regionVerID to region ref mapping - sorted *btree.BTree // cache regions are organized as sorted key to region ref mapping + sync.RWMutex // mutex protect cached region + regions map[RegionVerID]*Region // cached regions are organized as regionVerID to region ref mapping + latestVersions map[uint64]RegionVerID // cache the map from regionID to its latest RegionVerID + sorted *btree.BTree // cache regions are organized as sorted key to region ref mapping } storeMu struct { sync.RWMutex @@ -299,6 +300,7 @@ func NewRegionCache(pdClient pd.Client) *RegionCache { pdClient: pdClient, } c.mu.regions = make(map[RegionVerID]*Region) + c.mu.latestVersions = make(map[uint64]RegionVerID) c.mu.sorted = btree.New(btreeDegree) c.storeMu.stores = make(map[uint64]*Store) c.notifyCheckCh = make(chan struct{}, 1) @@ -976,6 +978,15 @@ func (c *RegionCache) UpdateLeader(regionID RegionVerID, leaderStoreID uint64, c } } +// removeVersionFromCache removes a RegionVerID from cache, tries to cleanup +// both c.mu.regions and c.mu.versions. Note this function is not thread-safe. +func (c *RegionCache) removeVersionFromCache(oldVer RegionVerID, regionID uint64) { + delete(c.mu.regions, oldVer) + if ver, ok := c.mu.latestVersions[regionID]; ok && ver.Equals(oldVer) { + delete(c.mu.latestVersions, regionID) + } +} + // insertRegionToCache tries to insert the Region to cache. // It should be protected by c.mu.Lock(). func (c *RegionCache) insertRegionToCache(cachedRegion *Region) { @@ -995,9 +1006,14 @@ func (c *RegionCache) insertRegionToCache(cachedRegion *Region) { // Don't refresh TiFlash work idx for region. Otherwise, it will always goto a invalid store which // is under transferring regions. store.workTiFlashIdx = atomic.LoadInt32(&oldRegionStore.workTiFlashIdx) - delete(c.mu.regions, oldRegion.VerID()) + c.removeVersionFromCache(oldRegion.VerID(), cachedRegion.VerID().id) } c.mu.regions[cachedRegion.VerID()] = cachedRegion + newVer := cachedRegion.VerID() + latest, ok := c.mu.latestVersions[cachedRegion.VerID().id] + if !ok || latest.GetVer() < newVer.GetVer() || latest.GetConfVer() < newVer.GetConfVer() { + c.mu.latestVersions[cachedRegion.VerID().id] = newVer + } } // searchCachedRegion finds a region from cache by key. Like `getCachedRegion`, @@ -1032,34 +1048,26 @@ func (c *RegionCache) searchCachedRegion(key []byte, isEndKey bool) *Region { // `getCachedRegion`, it should be called with c.mu.RLock(), and the returned // Region should not be used after c.mu is RUnlock(). func (c *RegionCache) getRegionByIDFromCache(regionID uint64) *Region { - var newestRegion *Region ts := time.Now().Unix() - for v, r := range c.mu.regions { - if v.id == regionID { - lastAccess := atomic.LoadInt64(&r.lastAccess) - if ts-lastAccess > RegionCacheTTLSec { - continue - } - if newestRegion == nil { - newestRegion = r - continue - } - nv := newestRegion.VerID() - cv := r.VerID() - if nv.GetConfVer() < cv.GetConfVer() { - newestRegion = r - continue - } - if nv.GetVer() < cv.GetVer() { - newestRegion = r - continue - } - } + ver, ok := c.mu.latestVersions[regionID] + if !ok { + return nil + } + latestRegion, ok := c.mu.regions[ver] + if !ok { + // should not happen + logutil.BgLogger().Warn("region version not found", + zap.Uint64("regionID", regionID), zap.Stringer("version", &ver)) + return nil + } + lastAccess := atomic.LoadInt64(&latestRegion.lastAccess) + if ts-lastAccess > RegionCacheTTLSec { + return nil } - if newestRegion != nil { - atomic.CompareAndSwapInt64(&newestRegion.lastAccess, atomic.LoadInt64(&newestRegion.lastAccess), ts) + if latestRegion != nil { + atomic.CompareAndSwapInt64(&latestRegion.lastAccess, atomic.LoadInt64(&latestRegion.lastAccess), ts) } - return newestRegion + return latestRegion } // TODO: revise it by get store by closure. @@ -1545,6 +1553,11 @@ type RegionVerID struct { ver uint64 } +// NewRegionVerID creates a region ver id, which used for invalidating regions. +func NewRegionVerID(id, confVer, ver uint64) RegionVerID { + return RegionVerID{id, confVer, ver} +} + // GetID returns the id of the region func (r *RegionVerID) GetID() uint64 { return r.id @@ -1565,6 +1578,11 @@ func (r *RegionVerID) String() string { return fmt.Sprintf("{ region id: %v, ver: %v, confVer: %v }", r.id, r.ver, r.confVer) } +// Equals checks whether the RegionVerID equals to another one +func (r *RegionVerID) Equals(another RegionVerID) bool { + return r.id == another.id && r.confVer == another.confVer && r.ver == another.ver +} + // VerID returns the Region's RegionVerID. func (r *Region) VerID() RegionVerID { return RegionVerID{ diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index 592541eb5b9d2..efb2ae9df73ab 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -69,9 +69,25 @@ func (s *testRegionCacheSuite) storeAddr(id uint64) string { func (s *testRegionCacheSuite) checkCache(c *C, len int) { ts := time.Now().Unix() c.Assert(validRegions(s.cache.mu.regions, ts), Equals, len) + c.Assert(validRegionsSearchedByVersions(s.cache.mu.latestVersions, s.cache.mu.regions, ts), Equals, len) c.Assert(validRegionsInBtree(s.cache.mu.sorted, ts), Equals, len) } +func validRegionsSearchedByVersions( + versions map[uint64]RegionVerID, + regions map[RegionVerID]*Region, + ts int64, +) (count int) { + for _, ver := range versions { + region, ok := regions[ver] + if !ok || !region.checkRegionCacheTTL(ts) { + continue + } + count++ + } + return +} + func validRegions(regions map[RegionVerID]*Region, ts int64) (len int) { for _, region := range regions { if !region.checkRegionCacheTTL(ts) { diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 9b501a8a31c31..cad0ed0379e96 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -737,15 +737,3 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed } return false, nil } - -// IsolationLevelToPB converts isolation level to wire type. -func IsolationLevelToPB(level kv.IsoLevel) kvrpcpb.IsolationLevel { - switch level { - case kv.RC: - return kvrpcpb.IsolationLevel_RC - case kv.SI: - return kvrpcpb.IsolationLevel_SI - default: - return kvrpcpb.IsolationLevel_SI - } -} diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index dab843013314b..81e9cc4498a07 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -471,30 +471,6 @@ func (s *mockTikvGrpcServer) ReadIndex(context.Context, *kvrpcpb.ReadIndexReques return nil, errors.New("unreachable") } -func (s *mockTikvGrpcServer) VerGet(context.Context, *kvrpcpb.VerGetRequest) (*kvrpcpb.VerGetResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) VerBatchGet(context.Context, *kvrpcpb.VerBatchGetRequest) (*kvrpcpb.VerBatchGetResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) VerMut(context.Context, *kvrpcpb.VerMutRequest) (*kvrpcpb.VerMutResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) VerBatchMut(context.Context, *kvrpcpb.VerBatchMutRequest) (*kvrpcpb.VerBatchMutResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) VerScan(context.Context, *kvrpcpb.VerScanRequest) (*kvrpcpb.VerScanResponse, error) { - return nil, errors.New("unreachable") -} - -func (s *mockTikvGrpcServer) VerDeleteRange(context.Context, *kvrpcpb.VerDeleteRangeRequest) (*kvrpcpb.VerDeleteRangeResponse, error) { - return nil, errors.New("unreachable") -} - func (s *mockTikvGrpcServer) CheckLeader(context.Context, *kvrpcpb.CheckLeaderRequest) (*kvrpcpb.CheckLeaderResponse, error) { return nil, errors.New("unreachable") } diff --git a/store/tikv/scan.go b/store/tikv/scan.go index cc8d825384a68..19a14b3f73819 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -189,9 +189,9 @@ func (s *Scanner) getData(bo *Backoffer) error { } sreq := &pb.ScanRequest{ Context: &pb.Context{ - Priority: s.snapshot.priority, + Priority: s.snapshot.priority.ToPB(), NotFillCache: s.snapshot.notFillCache, - IsolationLevel: IsolationLevelToPB(s.snapshot.isolationLevel), + IsolationLevel: s.snapshot.isolationLevel.ToPB(), }, StartKey: s.nextStartKey, EndKey: reqEndKey, @@ -207,7 +207,7 @@ func (s *Scanner) getData(bo *Backoffer) error { } s.snapshot.mu.RLock() req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdScan, sreq, s.snapshot.mu.replicaRead, &s.snapshot.replicaReadSeed, pb.Context{ - Priority: s.snapshot.priority, + Priority: s.snapshot.priority.ToPB(), NotFillCache: s.snapshot.notFillCache, TaskId: s.snapshot.mu.taskID, }) diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index a02cc23a5db66..2b9926c7a2b9a 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -26,6 +26,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" tikverr "github.com/pingcap/tidb/store/tikv/error" @@ -44,14 +45,43 @@ const ( maxTimestamp = math.MaxUint64 ) +// Priority is the priority for tikv to execute a command. +type Priority kvrpcpb.CommandPri + +// Priority value for transaction priority. +const ( + PriorityNormal = Priority(kvrpcpb.CommandPri_Normal) + PriorityLow = Priority(kvrpcpb.CommandPri_Low) + PriorityHigh = Priority(kvrpcpb.CommandPri_High) +) + +// ToPB converts priority to wire type. +func (p Priority) ToPB() kvrpcpb.CommandPri { + return kvrpcpb.CommandPri(p) +} + +// IsoLevel is the transaction's isolation level. +type IsoLevel kvrpcpb.IsolationLevel + +const ( + // SI stands for 'snapshot isolation'. + SI IsoLevel = IsoLevel(kvrpcpb.IsolationLevel_SI) + // RC stands for 'read committed'. + RC IsoLevel = IsoLevel(kvrpcpb.IsolationLevel_RC) +) + +// ToPB converts isolation level to wire type. +func (l IsoLevel) ToPB() kvrpcpb.IsolationLevel { + return kvrpcpb.IsolationLevel(l) +} + // KVSnapshot implements the tidbkv.Snapshot interface. type KVSnapshot struct { store *KVStore version uint64 - isolationLevel kv.IsoLevel - priority pb.CommandPri + isolationLevel IsoLevel + priority Priority notFillCache bool - syncLog bool keyOnly bool vars *kv.Variables replicaReadSeed uint32 @@ -77,7 +107,6 @@ type KVSnapshot struct { matchStoreLabels []*metapb.StoreLabel } sampleStep uint32 - txnScope string } // newTiKVSnapshot creates a snapshot of an TiKV store. @@ -90,14 +119,15 @@ func newTiKVSnapshot(store *KVStore, ts uint64, replicaReadSeed uint32) *KVSnaps return &KVSnapshot{ store: store, version: ts, - priority: pb.CommandPri_Normal, + priority: PriorityNormal, vars: kv.DefaultVars, replicaReadSeed: replicaReadSeed, resolvedLocks: util.NewTSSet(5), } } -func (s *KVSnapshot) setSnapshotTS(ts uint64) { +// SetSnapshotTS resets the timestamp for reads. +func (s *KVSnapshot) SetSnapshotTS(ts uint64) { // Sanity check for snapshot version. if ts >= math.MaxInt64 && ts != math.MaxUint64 { err := errors.Errorf("try to get snapshot with a large ts %d", ts) @@ -277,7 +307,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec Keys: pending, Version: s.version, }, s.mu.replicaRead, &s.replicaReadSeed, pb.Context{ - Priority: s.priority, + Priority: s.priority.ToPB(), NotFillCache: s.notFillCache, TaskId: s.mu.taskID, }) @@ -427,7 +457,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, Key: k, Version: s.version, }, s.mu.replicaRead, &s.replicaReadSeed, pb.Context{ - Priority: s.priority, + Priority: s.priority.ToPB(), NotFillCache: s.notFillCache, TaskId: s.mu.taskID, }) @@ -535,18 +565,6 @@ func (s *KVSnapshot) IterReverse(k []byte) (unionstore.Iterator, error) { // value of this option. Only ReplicaRead is supported for snapshot func (s *KVSnapshot) SetOption(opt int, val interface{}) { switch opt { - case kv.IsolationLevel: - s.isolationLevel = val.(kv.IsoLevel) - case kv.Priority: - s.priority = PriorityToPB(val.(int)) - case kv.NotFillCache: - s.notFillCache = val.(bool) - case kv.SyncLog: - s.syncLog = val.(bool) - case kv.KeyOnly: - s.keyOnly = val.(bool) - case kv.SnapshotTS: - s.setSnapshotTS(val.(uint64)) case kv.ReplicaRead: s.mu.Lock() s.mu.replicaRead = val.(kv.ReplicaReadType) @@ -569,8 +587,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { s.mu.Lock() s.mu.matchStoreLabels = val.([]*metapb.StoreLabel) s.mu.Unlock() - case kv.TxnScope: - s.txnScope = val.(string) } } @@ -588,6 +604,27 @@ func (s *KVSnapshot) DelOption(opt int) { } } +// SetNotFillCache indicates whether tikv should skip filling cache when +// loading data. +func (s *KVSnapshot) SetNotFillCache(b bool) { + s.notFillCache = b +} + +// SetKeyOnly indicates if tikv can return only keys. +func (s *KVSnapshot) SetKeyOnly(b bool) { + s.keyOnly = b +} + +// SetIsolationLevel sets the isolation level used to scan data from tikv. +func (s *KVSnapshot) SetIsolationLevel(level IsoLevel) { + s.isolationLevel = level +} + +// SetPriority sets the priority for tikv to execute commands. +func (s *KVSnapshot) SetPriority(pri Priority) { + s.priority = pri +} + // SnapCacheHitCount gets the snapshot cache hit count. Only for test. func (s *KVSnapshot) SnapCacheHitCount() int { return int(atomic.LoadInt64(&s.mu.hitCnt)) diff --git a/store/tikv/split_region.go b/store/tikv/split_region.go index e8d988db328ab..5839aa4d73c96 100644 --- a/store/tikv/split_region.go +++ b/store/tikv/split_region.go @@ -185,7 +185,7 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool if batchResp.err == nil { batchResp.err = err } - if tikverr.ErrPDServerTimeout.Equal(err) { + if _, ok := err.(*tikverr.ErrPDServerTimeout); ok { break } } @@ -219,7 +219,7 @@ func (s *KVStore) scatterRegion(bo *Backoffer, regionID uint64, tableID *int64) if val, err2 := util.MockScatterRegionTimeout.Eval(); err2 == nil { if val.(bool) { - err = tikverr.ErrPDServerTimeout + err = tikverr.NewErrPDServerTimeout("") } } diff --git a/store/tikv/test_probe.go b/store/tikv/test_probe.go index e2b072ef3c7f1..a6a4f8d826655 100644 --- a/store/tikv/test_probe.go +++ b/store/tikv/test_probe.go @@ -14,6 +14,7 @@ package tikv import ( + "bytes" "context" "sync/atomic" "time" @@ -304,6 +305,12 @@ func (c CommitterProbe) BuildPrewriteRequest(regionID, regionConf, regionVersion var batch batchMutations batch.mutations = mutations batch.region = RegionVerID{regionID, regionConf, regionVersion} + for _, key := range mutations.GetKeys() { + if bytes.Equal(key, c.primary()) { + batch.isPrimary = true + break + } + } return c.buildPrewriteRequest(batch, txnSize) } diff --git a/store/tikv/tests/1pc_test.go b/store/tikv/tests/1pc_test.go index 509dab684628a..6ff57700f4e3c 100644 --- a/store/tikv/tests/1pc_test.go +++ b/store/tikv/tests/1pc_test.go @@ -18,7 +18,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/util" @@ -27,7 +26,7 @@ import ( func (s *testAsyncCommitCommon) begin1PC(c *C) tikv.TxnProbe { txn, err := s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.Enable1PC, true) + txn.SetEnable1PC(true) return tikv.TxnProbe{KVTxn: txn} } @@ -254,7 +253,7 @@ func (s *testOnePCSuite) Test1PCWithMultiDC(c *C) { localTxn := s.begin1PC(c) err := localTxn.Set([]byte("a"), []byte("a1")) - localTxn.SetOption(kv.TxnScope, "bj") + localTxn.SetScope("bj") c.Assert(err, IsNil) ctx := context.WithValue(context.Background(), util.SessionID, uint64(1)) err = localTxn.Commit(ctx) @@ -263,7 +262,7 @@ func (s *testOnePCSuite) Test1PCWithMultiDC(c *C) { globalTxn := s.begin1PC(c) err = globalTxn.Set([]byte("b"), []byte("b1")) - globalTxn.SetOption(kv.TxnScope, oracle.GlobalTxnScope) + globalTxn.SetScope(oracle.GlobalTxnScope) c.Assert(err, IsNil) err = globalTxn.Commit(ctx) c.Assert(err, IsNil) diff --git a/store/tikv/tests/2pc_fail_test.go b/store/tikv/tests/2pc_fail_test.go index ed8820a302fe0..d8b35e4b28c53 100644 --- a/store/tikv/tests/2pc_fail_test.go +++ b/store/tikv/tests/2pc_fail_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/kv" + tikverr "github.com/pingcap/tidb/store/tikv/error" ) // TestFailCommitPrimaryRpcErrors tests rpc errors are handled properly when @@ -40,7 +40,7 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRpcErrors(c *C) { // We don't need to call "Rollback" after "Commit" fails. err = t1.Rollback() - c.Assert(err, Equals, kv.ErrInvalidTxn) + c.Assert(err, Equals, tikverr.ErrInvalidTxn) } // TestFailCommitPrimaryRegionError tests RegionError is handled properly when diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 9313e6f0abeb8..6d7d7e89d1a8e 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" - tidbkv "github.com/pingcap/tidb/kv" drivertxn "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" @@ -307,7 +306,8 @@ func (s *testCommitterSuite) TestContextCancelRetryable(c *C) { c.Assert(err, IsNil) err = txn2.Commit(context.Background()) c.Assert(err, NotNil) - c.Assert(tidbkv.ErrWriteConflictInTiDB.Equal(err), IsTrue, Commentf("err: %s", err)) + _, ok := err.(*tikverr.ErrWriteConflictInLatch) + c.Assert(ok, IsTrue, Commentf("err: %s", err)) } func (s *testCommitterSuite) TestContextCancelCausingUndetermined(c *C) { @@ -617,7 +617,7 @@ func (s *testCommitterSuite) TestRejectCommitTS(c *C) { func (s *testCommitterSuite) TestPessimisticPrewriteRequest(c *C) { // This test checks that the isPessimisticLock field is set in the request even when no keys are pessimistic lock. txn := s.begin(c) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) err := txn.Set([]byte("t1"), []byte("v1")) c.Assert(err, IsNil) committer, err := txn.NewCommitter(0) @@ -636,7 +636,7 @@ func (s *testCommitterSuite) TestUnsetPrimaryKey(c *C) { c.Assert(txn.Commit(context.Background()), IsNil) txn = s.begin(c) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) _, _ = txn.GetUnionStore().Get(context.TODO(), key) c.Assert(txn.GetMemBuffer().SetWithFlags(key, key, kv.SetPresumeKeyNotExists), IsNil) lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()} @@ -651,7 +651,7 @@ func (s *testCommitterSuite) TestUnsetPrimaryKey(c *C) { func (s *testCommitterSuite) TestPessimisticLockedKeysDedup(c *C) { txn := s.begin(c) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) lockCtx := &kv.LockCtx{ForUpdateTS: 100, WaitStartTime: time.Now()} err := txn.LockKeys(context.Background(), lockCtx, []byte("abc"), []byte("def")) c.Assert(err, IsNil) @@ -664,7 +664,7 @@ func (s *testCommitterSuite) TestPessimisticLockedKeysDedup(c *C) { func (s *testCommitterSuite) TestPessimisticTTL(c *C) { key := []byte("key") txn := s.begin(c) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) time.Sleep(time.Millisecond * 100) lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()} err := txn.LockKeys(context.Background(), lockCtx, key) @@ -710,7 +710,7 @@ func (s *testCommitterSuite) TestPessimisticLockReturnValues(c *C) { c.Assert(txn.Set(key2, key2), IsNil) c.Assert(txn.Commit(context.Background()), IsNil) txn = s.begin(c) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()} lockCtx.ReturnValues = true lockCtx.Values = map[string]kv.ReturnedValue{} @@ -725,7 +725,7 @@ func (s *testCommitterSuite) TestElapsedTTL(c *C) { key := []byte("key") txn := s.begin(c) txn.SetStartTS(oracle.ComposeTS(oracle.GetPhysical(time.Now().Add(time.Second*10)), 1)) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) time.Sleep(time.Millisecond * 100) lockCtx := &kv.LockCtx{ ForUpdateTS: oracle.ComposeTS(oracle.ExtractPhysical(txn.StartTS())+100, 1), @@ -746,7 +746,7 @@ func (s *testCommitterSuite) TestDeleteYourWriteCauseGhostPrimary(c *C) { // insert k1, k2, k3 and delete k1 txn1 := s.begin(c) - txn1.DelOption(kv.Pessimistic) + txn1.SetPessimistic(false) s.store.ClearTxnLatches() txn1.Get(context.Background(), k1) txn1.GetMemBuffer().SetWithFlags(k1, []byte{0}, kv.SetPresumeKeyNotExists) @@ -771,7 +771,7 @@ func (s *testCommitterSuite) TestDeleteYourWriteCauseGhostPrimary(c *C) { // start txn2 to read k3(prewrite success and primary should be committed) txn2 := s.begin(c) - txn2.DelOption(kv.Pessimistic) + txn2.SetPessimistic(false) s.store.ClearTxnLatches() v, err := txn2.Get(context.Background(), k3) c.Assert(err, IsNil) // should resolve lock and read txn1 k3 result instead of rollback it. @@ -788,7 +788,7 @@ func (s *testCommitterSuite) TestDeleteAllYourWrites(c *C) { // insert k1, k2, k3 and delete k1, k2, k3 txn1 := s.begin(c) - txn1.DelOption(kv.Pessimistic) + txn1.SetPessimistic(false) s.store.ClearTxnLatches() txn1.GetMemBuffer().SetWithFlags(k1, []byte{0}, kv.SetPresumeKeyNotExists) txn1.Delete(k1) @@ -808,7 +808,7 @@ func (s *testCommitterSuite) TestDeleteAllYourWritesWithSFU(c *C) { // insert k1, k2, k2 and delete k1 txn1 := s.begin(c) - txn1.DelOption(kv.Pessimistic) + txn1.SetPessimistic(false) s.store.ClearTxnLatches() txn1.GetMemBuffer().SetWithFlags(k1, []byte{0}, kv.SetPresumeKeyNotExists) txn1.Delete(k1) @@ -832,7 +832,7 @@ func (s *testCommitterSuite) TestDeleteAllYourWritesWithSFU(c *C) { <-ac // start txn2 to read k3 txn2 := s.begin(c) - txn2.DelOption(kv.Pessimistic) + txn2.SetPessimistic(false) s.store.ClearTxnLatches() err = txn2.Set(k3, []byte{33}) c.Assert(err, IsNil) @@ -860,7 +860,7 @@ func (s *testCommitterSuite) TestAcquireFalseTimeoutLock(c *C) { k2 := []byte("k2") txn1 := s.begin(c) - txn1.SetOption(kv.Pessimistic, true) + txn1.SetPessimistic(true) // lock the primary key lockCtx := &kv.LockCtx{ForUpdateTS: txn1.StartTS(), WaitStartTime: time.Now()} err := txn1.LockKeys(context.Background(), lockCtx, k1) @@ -875,7 +875,7 @@ func (s *testCommitterSuite) TestAcquireFalseTimeoutLock(c *C) { // wait until secondary key exceeds its own TTL time.Sleep(time.Duration(atomic.LoadUint64(&tikv.ManagedLockTTL)) * time.Millisecond) txn2 := s.begin(c) - txn2.SetOption(kv.Pessimistic, true) + txn2.SetPessimistic(true) // test no wait lockCtx = &kv.LockCtx{ForUpdateTS: txn2.StartTS(), LockWaitTime: tikv.LockNoWait, WaitStartTime: time.Now()} @@ -921,7 +921,7 @@ func (s *testCommitterSuite) TestPkNotFound(c *C) { k3 := []byte("k3") txn1 := s.begin(c) - txn1.SetOption(kv.Pessimistic, true) + txn1.SetPessimistic(true) // lock the primary key. lockCtx := &kv.LockCtx{ForUpdateTS: txn1.StartTS(), WaitStartTime: time.Now()} err := txn1.LockKeys(ctx, lockCtx, k1) @@ -957,7 +957,7 @@ func (s *testCommitterSuite) TestPkNotFound(c *C) { c.Assert(err, IsNil) c.Assert(status.Action(), Equals, kvrpcpb.Action_LockNotExistDoNothing) txn2 := s.begin(c) - txn2.SetOption(kv.Pessimistic, true) + txn2.SetPessimistic(true) lockCtx = &kv.LockCtx{ForUpdateTS: txn2.StartTS(), WaitStartTime: time.Now()} err = txn2.LockKeys(ctx, lockCtx, k2) c.Assert(err, IsNil) @@ -981,7 +981,7 @@ func (s *testCommitterSuite) TestPkNotFound(c *C) { // After disable fail point, the rollbackIfNotExist flag will be set, and the resolve should succeed. In this // case, the returned action of TxnStatus should be LockNotExistDoNothing, and lock on k3 could be resolved. txn3 := s.begin(c) - txn3.SetOption(kv.Pessimistic, true) + txn3.SetPessimistic(true) lockCtx = &kv.LockCtx{ForUpdateTS: txn3.StartTS(), WaitStartTime: time.Now(), LockWaitTime: tikv.LockNoWait} err = txn3.LockKeys(ctx, lockCtx, k3) c.Assert(err, IsNil) @@ -997,7 +997,7 @@ func (s *testCommitterSuite) TestPessimisticLockPrimary(c *C) { k2 := []byte("b") txn1 := s.begin(c) - txn1.SetOption(kv.Pessimistic, true) + txn1.SetPessimistic(true) // txn1 lock k1 lockCtx := &kv.LockCtx{ForUpdateTS: txn1.StartTS(), WaitStartTime: time.Now()} err := txn1.LockKeys(context.Background(), lockCtx, k1) @@ -1008,7 +1008,7 @@ func (s *testCommitterSuite) TestPessimisticLockPrimary(c *C) { doneCh := make(chan error) go func() { txn2 := s.begin(c) - txn2.SetOption(kv.Pessimistic, true) + txn2.SetPessimistic(true) lockCtx2 := &kv.LockCtx{ForUpdateTS: txn2.StartTS(), WaitStartTime: time.Now(), LockWaitTime: 200} waitErr := txn2.LockKeys(context.Background(), lockCtx2, k1, k2) doneCh <- waitErr @@ -1017,7 +1017,7 @@ func (s *testCommitterSuite) TestPessimisticLockPrimary(c *C) { // txn3 should locks k2 successfully using no wait txn3 := s.begin(c) - txn3.SetOption(kv.Pessimistic, true) + txn3.SetPessimistic(true) lockCtx3 := &kv.LockCtx{ForUpdateTS: txn3.StartTS(), WaitStartTime: time.Now(), LockWaitTime: tikv.LockNoWait} c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/txnNotFoundRetTTL", "return"), IsNil) err = txn3.LockKeys(context.Background(), lockCtx3, k2) @@ -1032,7 +1032,7 @@ func (s *testCommitterSuite) TestResolvePessimisticLock(c *C) { untouchedIndexValue := []byte{0, 0, 0, 0, 0, 0, 0, 1, 49} noValueIndexKey := []byte("t00000001_i000000002") txn := s.begin(c) - txn.SetOption(kv.KVFilter, drivertxn.TiDBKVFilter{}) + txn.SetKVFilter(drivertxn.TiDBKVFilter{}) err := txn.Set(untouchedIndexKey, untouchedIndexValue) c.Assert(err, IsNil) lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now(), LockWaitTime: tikv.LockNoWait} @@ -1108,7 +1108,7 @@ func (s *testCommitterSuite) TestPushPessimisticLock(c *C) { ctx := context.Background() txn1 := s.begin(c) - txn1.SetOption(kv.Pessimistic, true) + txn1.SetPessimistic(true) lockCtx := &kv.LockCtx{ForUpdateTS: txn1.StartTS(), WaitStartTime: time.Now()} err := txn1.LockKeys(context.Background(), lockCtx, k1, k2) c.Assert(err, IsNil) @@ -1160,7 +1160,7 @@ func (s *testCommitterSuite) TestResolveMixed(c *C) { // make the optimistic and pessimistic lock left with primary lock not found txn1 := s.begin(c) - txn1.SetOption(kv.Pessimistic, true) + txn1.SetPessimistic(true) // lock the primary key lockCtx := &kv.LockCtx{ForUpdateTS: txn1.StartTS(), WaitStartTime: time.Now()} err := txn1.LockKeys(context.Background(), lockCtx, pk) @@ -1202,7 +1202,7 @@ func (s *testCommitterSuite) TestResolveMixed(c *C) { // txn2 tries to lock the pessimisticLockKey, the lock should has been resolved in clean whole region resolve txn2 := s.begin(c) - txn2.SetOption(kv.Pessimistic, true) + txn2.SetPessimistic(true) lockCtx = &kv.LockCtx{ForUpdateTS: txn2.StartTS(), WaitStartTime: time.Now(), LockWaitTime: tikv.LockNoWait} err = txn2.LockKeys(context.Background(), lockCtx, pessimisticLockKey) c.Assert(err, IsNil) diff --git a/store/tikv/tests/async_commit_fail_test.go b/store/tikv/tests/async_commit_fail_test.go index 5e519afb51c99..a791f16c54e86 100644 --- a/store/tikv/tests/async_commit_fail_test.go +++ b/store/tikv/tests/async_commit_fail_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/util" @@ -65,7 +64,7 @@ func (s *testAsyncCommitFailSuite) TestFailAsyncCommitPrewriteRpcErrors(c *C) { // We don't need to call "Rollback" after "Commit" fails. err = t1.Rollback() - c.Assert(err, Equals, kv.ErrInvalidTxn) + c.Assert(err, Equals, tikverr.ErrInvalidTxn) // Create a new transaction to check. The previous transaction should actually commit. t2 := s.beginAsyncCommit(c) diff --git a/store/tikv/tests/async_commit_test.go b/store/tikv/tests/async_commit_test.go index 96a35085f46b8..0f4985fa7ab86 100644 --- a/store/tikv/tests/async_commit_test.go +++ b/store/tikv/tests/async_commit_test.go @@ -350,7 +350,7 @@ func (s *testAsyncCommitSuite) TestRepeatableRead(c *C) { sessionID++ ctx := context.WithValue(context.Background(), util.SessionID, sessionID) txn1 := s.beginAsyncCommit(c) - txn1.SetOption(kv.Pessimistic, isPessimistic) + txn1.SetPessimistic(isPessimistic) s.mustGetFromTxn(c, txn1, []byte("k1"), []byte("v1")) txn1.Set([]byte("k1"), []byte("v2")) @@ -409,7 +409,7 @@ func (s *testAsyncCommitSuite) TestAsyncCommitWithMultiDC(c *C) { localTxn := s.beginAsyncCommit(c) err := localTxn.Set([]byte("a"), []byte("a1")) - localTxn.SetOption(kv.TxnScope, "bj") + localTxn.SetScope("bj") c.Assert(err, IsNil) ctx := context.WithValue(context.Background(), util.SessionID, uint64(1)) err = localTxn.Commit(ctx) @@ -418,13 +418,102 @@ func (s *testAsyncCommitSuite) TestAsyncCommitWithMultiDC(c *C) { globalTxn := s.beginAsyncCommit(c) err = globalTxn.Set([]byte("b"), []byte("b1")) - globalTxn.SetOption(kv.TxnScope, oracle.GlobalTxnScope) + globalTxn.SetScope(oracle.GlobalTxnScope) c.Assert(err, IsNil) err = globalTxn.Commit(ctx) c.Assert(err, IsNil) c.Assert(globalTxn.IsAsyncCommit(), IsTrue) } +func (s *testAsyncCommitSuite) TestResolveTxnFallbackFromAsyncCommit(c *C) { + keys := [][]byte{[]byte("k0"), []byte("k1")} + values := [][]byte{[]byte("v00"), []byte("v10")} + initTest := func() tikv.CommitterProbe { + t0 := s.begin(c) + err := t0.Set(keys[0], values[0]) + c.Assert(err, IsNil) + err = t0.Set(keys[1], values[1]) + c.Assert(err, IsNil) + err = t0.Commit(context.Background()) + c.Assert(err, IsNil) + + t1 := s.beginAsyncCommit(c) + err = t1.Set(keys[0], []byte("v01")) + c.Assert(err, IsNil) + err = t1.Set(keys[1], []byte("v11")) + c.Assert(err, IsNil) + + committer, err := t1.NewCommitter(1) + c.Assert(err, IsNil) + committer.SetLockTTL(1) + committer.SetUseAsyncCommit() + return committer + } + prewriteKey := func(committer tikv.CommitterProbe, idx int, fallback bool) { + bo := tikv.NewBackofferWithVars(context.Background(), 5000, nil) + loc, err := s.store.GetRegionCache().LocateKey(bo, keys[idx]) + c.Assert(err, IsNil) + req := committer.BuildPrewriteRequest(loc.Region.GetID(), loc.Region.GetConfVer(), loc.Region.GetVer(), + committer.GetMutations().Slice(idx, idx+1), 1) + if fallback { + req.Req.(*kvrpcpb.PrewriteRequest).MaxCommitTs = 1 + } + resp, err := s.store.SendReq(bo, req, loc.Region, 5000) + c.Assert(err, IsNil) + c.Assert(resp.Resp, NotNil) + } + readKey := func(idx int) { + t2 := s.begin(c) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + val, err := t2.Get(ctx, keys[idx]) + c.Assert(err, IsNil) + c.Assert(val, DeepEquals, values[idx]) + } + + // Case 1: Fallback primary, read primary + committer := initTest() + prewriteKey(committer, 0, true) + prewriteKey(committer, 1, false) + readKey(0) + readKey(1) + + // Case 2: Fallback primary, read secondary + committer = initTest() + prewriteKey(committer, 0, true) + prewriteKey(committer, 1, false) + readKey(1) + readKey(0) + + // Case 3: Fallback secondary, read primary + committer = initTest() + prewriteKey(committer, 0, false) + prewriteKey(committer, 1, true) + readKey(0) + readKey(1) + + // Case 4: Fallback secondary, read secondary + committer = initTest() + prewriteKey(committer, 0, false) + prewriteKey(committer, 1, true) + readKey(1) + readKey(0) + + // Case 5: Fallback both, read primary + committer = initTest() + prewriteKey(committer, 0, true) + prewriteKey(committer, 1, true) + readKey(0) + readKey(1) + + // Case 6: Fallback both, read secondary + committer = initTest() + prewriteKey(committer, 0, true) + prewriteKey(committer, 1, true) + readKey(1) + readKey(0) +} + type mockResolveClient struct { inner tikv.Client onResolveLock func(*kvrpcpb.ResolveLockRequest) (*tikvrpc.Response, error) diff --git a/store/tikv/tests/lock_test.go b/store/tikv/tests/lock_test.go index d6e20157bc7ac..3c6c652d96041 100644 --- a/store/tikv/tests/lock_test.go +++ b/store/tikv/tests/lock_test.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" ) @@ -142,7 +141,7 @@ func (s *testLockSuite) TestScanLockResolveWithSeekKeyOnly(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.KeyOnly, true) + txn.GetSnapshot().SetKeyOnly(true) iter, err := txn.Iter([]byte("a"), nil) c.Assert(err, IsNil) for ch := byte('a'); ch <= byte('z'); ch++ { diff --git a/store/tikv/tests/safepoint_test.go b/store/tikv/tests/safepoint_test.go index 6135dd63bba5c..84249d0104997 100644 --- a/store/tikv/tests/safepoint_test.go +++ b/store/tikv/tests/safepoint_test.go @@ -91,8 +91,9 @@ func (s *testSafePointSuite) TestSafePoint(c *C) { _, geterr2 := txn2.Get(context.TODO(), encodeKey(s.prefix, s08d("key", 0))) c.Assert(geterr2, NotNil) - isFallBehind := terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrGCTooEarly) - isMayFallBehind := terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point")) + + _, isFallBehind := errors.Cause(geterr2).(*tikverr.ErrGCTooEarly) + isMayFallBehind := terror.ErrorEqual(errors.Cause(geterr2), tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point")) isBehind := isFallBehind || isMayFallBehind c.Assert(isBehind, IsTrue) @@ -103,8 +104,8 @@ func (s *testSafePointSuite) TestSafePoint(c *C) { _, seekerr := txn3.Iter(encodeKey(s.prefix, ""), nil) c.Assert(seekerr, NotNil) - isFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrGCTooEarly) - isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point")) + _, isFallBehind = errors.Cause(geterr2).(*tikverr.ErrGCTooEarly) + isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point")) isBehind = isFallBehind || isMayFallBehind c.Assert(isBehind, IsTrue) @@ -116,8 +117,8 @@ func (s *testSafePointSuite) TestSafePoint(c *C) { _, batchgeterr := toTiDBTxn(&txn4).BatchGet(context.Background(), toTiDBKeys(keys)) c.Assert(batchgeterr, NotNil) - isFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrGCTooEarly) - isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.ErrPDServerTimeout.GenWithStackByArgs("start timestamp may fall behind safe point")) + _, isFallBehind = errors.Cause(geterr2).(*tikverr.ErrGCTooEarly) + isMayFallBehind = terror.ErrorEqual(errors.Cause(geterr2), tikverr.NewErrPDServerTimeout("start timestamp may fall behind safe point")) isBehind = isFallBehind || isMayFallBehind c.Assert(isBehind, IsTrue) } diff --git a/store/tikv/tests/scan_test.go b/store/tikv/tests/scan_test.go index 91e15747d3894..c64c6c3c52500 100644 --- a/store/tikv/tests/scan_test.go +++ b/store/tikv/tests/scan_test.go @@ -146,7 +146,7 @@ func (s *testScanSuite) TestScan(c *C) { check(c, scan, upperBound, false) txn3 := s.beginTxn(c) - txn3.SetOption(kv.KeyOnly, true) + txn3.GetSnapshot().SetKeyOnly(true) // Test scan without upper bound scan, err = txn3.Iter(s.recordPrefix, nil) c.Assert(err, IsNil) @@ -157,7 +157,7 @@ func (s *testScanSuite) TestScan(c *C) { check(c, scan, upperBound, true) // Restore KeyOnly to false - txn3.SetOption(kv.KeyOnly, false) + txn3.GetSnapshot().SetKeyOnly(false) scan, err = txn3.Iter(s.recordPrefix, nil) c.Assert(err, IsNil) check(c, scan, rowNum, true) diff --git a/store/tikv/tests/snapshot_fail_test.go b/store/tikv/tests/snapshot_fail_test.go index 572f6cae884e2..1360841bd743a 100644 --- a/store/tikv/tests/snapshot_fail_test.go +++ b/store/tikv/tests/snapshot_fail_test.go @@ -211,7 +211,7 @@ func (s *testSnapshotFailSuite) TestRetryPointGetResolveTS(c *C) { err = txn.Set([]byte("k2"), []byte("v2")) c.Assert(err, IsNil) txn.SetOption(kv.EnableAsyncCommit, false) - txn.SetOption(kv.Enable1PC, false) + txn.SetEnable1PC(false) txn.SetOption(kv.GuaranteeLinearizability, false) // Prewrite the lock without committing it diff --git a/store/tikv/tests/store_test.go b/store/tikv/tests/store_test.go index d10b09defd2ad..659dc6ea8f226 100644 --- a/store/tikv/tests/store_test.go +++ b/store/tikv/tests/store_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/oracle/oracles" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -124,7 +123,7 @@ func (s *testStoreSuite) TestRequestPriority(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) client.priority = pb.CommandPri_High - txn.SetOption(kv.Priority, kv.PriorityHigh) + txn.SetPriority(tikv.PriorityHigh) err = txn.Set([]byte("key"), []byte("value")) c.Assert(err, IsNil) err = txn.Commit(context.Background()) @@ -134,20 +133,20 @@ func (s *testStoreSuite) TestRequestPriority(c *C) { txn, err = s.store.Begin() c.Assert(err, IsNil) client.priority = pb.CommandPri_Low - txn.SetOption(kv.Priority, kv.PriorityLow) + txn.SetPriority(tikv.PriorityLow) _, err = txn.Get(context.TODO(), []byte("key")) c.Assert(err, IsNil) // A counter example. client.priority = pb.CommandPri_Low - txn.SetOption(kv.Priority, kv.PriorityNormal) + txn.SetPriority(tikv.PriorityNormal) _, err = txn.Get(context.TODO(), []byte("key")) // err is translated to "try again later" by backoffer, so doesn't check error value here. c.Assert(err, NotNil) // Cover Seek request. client.priority = pb.CommandPri_High - txn.SetOption(kv.Priority, kv.PriorityHigh) + txn.SetPriority(tikv.PriorityHigh) iter, err := txn.Iter([]byte("key"), nil) c.Assert(err, IsNil) for iter.Valid() { diff --git a/store/tikv/tests/ticlient_slow_test.go b/store/tikv/tests/ticlient_slow_test.go index b37245e72245c..61f6748874d14 100644 --- a/store/tikv/tests/ticlient_slow_test.go +++ b/store/tikv/tests/ticlient_slow_test.go @@ -64,7 +64,7 @@ func (s *testTiclientSuite) TestSplitRegionIn2PC(c *C) { checkKeyRegion(bo, startKey, endKey, Equals) txn := s.beginTxn(c) if m == "pessimistic" { - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) lockCtx := &kv.LockCtx{} lockCtx.ForUpdateTS = txn.StartTS() keys := make([][]byte, 0, preSplitThresholdInTest) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 5133a6deb35f6..4e462653c415c 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" - tidbkv "github.com/pingcap/tidb/kv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -67,14 +66,21 @@ type KVTxn struct { valid bool - // txnInfoSchema is the infoSchema fetched at startTS. - txnInfoSchema SchemaVer + // schemaVer is the infoSchema fetched at startTS. + schemaVer SchemaVer // SchemaAmender is used amend pessimistic txn commit mutations for schema change schemaAmender SchemaAmender // commitCallback is called after current transaction gets committed commitCallback func(info string, err error) - binlog BinlogExecutor + binlog BinlogExecutor + schemaLeaseChecker SchemaLeaseChecker + syncLog bool + priority Priority + isPessimistic bool + enable1PC bool + scope string + kvFilter KVFilter } func newTiKVTxn(store *KVStore, txnScope string) (*KVTxn, error) { @@ -89,7 +95,7 @@ func newTiKVTxn(store *KVStore, txnScope string) (*KVTxn, error) { // newTiKVTxnWithStartTS creates a txn with startTS. func newTiKVTxnWithStartTS(store *KVStore, txnScope string, startTS uint64, replicaReadSeed uint32) (*KVTxn, error) { snapshot := newTiKVSnapshot(store, startTS, replicaReadSeed) - newTiKVTxn := &KVTxn{ + return &KVTxn{ snapshot: snapshot, us: unionstore.NewUnionStore(snapshot), store: store, @@ -97,9 +103,8 @@ func newTiKVTxnWithStartTS(store *KVStore, txnScope string, startTS uint64, repl startTime: time.Now(), valid: true, vars: kv.DefaultVars, - } - newTiKVTxn.SetOption(kv.TxnScope, txnScope) - return newTiKVTxn, nil + scope: txnScope, + }, nil } func newTiKVTxnWithExactStaleness(store *KVStore, txnScope string, prevSec uint64) (*KVTxn, error) { @@ -179,12 +184,8 @@ func (txn *KVTxn) SetOption(opt int, val interface{}) { txn.us.SetOption(opt, val) txn.snapshot.SetOption(opt, val) switch opt { - case kv.InfoSchema: - txn.txnInfoSchema = val.(SchemaVer) case kv.SchemaAmender: txn.schemaAmender = val.(SchemaAmender) - case kv.CommitHook: - txn.commitCallback = val.(func(info string, err error)) } } @@ -198,16 +199,61 @@ func (txn *KVTxn) DelOption(opt int) { txn.us.DelOption(opt) } +// SetSchemaLeaseChecker sets a hook to check schema version. +func (txn *KVTxn) SetSchemaLeaseChecker(checker SchemaLeaseChecker) { + txn.schemaLeaseChecker = checker +} + +// EnableForceSyncLog indicates tikv to always sync log for the transaction. +func (txn *KVTxn) EnableForceSyncLog() { + txn.syncLog = true +} + +// SetPessimistic indicates if the transaction should use pessimictic lock. +func (txn *KVTxn) SetPessimistic(b bool) { + txn.isPessimistic = b +} + +// SetSchemaVer updates schema version to validate transaction. +func (txn *KVTxn) SetSchemaVer(schemaVer SchemaVer) { + txn.schemaVer = schemaVer +} + +// SetPriority sets the priority for both write and read. +func (txn *KVTxn) SetPriority(pri Priority) { + txn.priority = pri + txn.GetSnapshot().SetPriority(pri) +} + +// SetCommitCallback sets up a function that will be called when the transaction +// is finished. +func (txn *KVTxn) SetCommitCallback(f func(string, error)) { + txn.commitCallback = f +} + +// SetEnable1PC indicates if the transaction will try to use 1 phase commit. +func (txn *KVTxn) SetEnable1PC(b bool) { + txn.enable1PC = b +} + +// SetScope sets the geographical scope of the transaction. +func (txn *KVTxn) SetScope(scope string) { + txn.scope = scope +} + +// SetKVFilter sets the filter to ignore key-values in memory buffer. +func (txn *KVTxn) SetKVFilter(filter KVFilter) { + txn.kvFilter = filter +} + // IsPessimistic returns true if it is pessimistic. func (txn *KVTxn) IsPessimistic() bool { - return txn.us.GetOption(kv.Pessimistic) != nil + return txn.isPessimistic } -func (txn *KVTxn) getKVFilter() KVFilter { - if filter := txn.us.GetOption(kv.KVFilter); filter != nil { - return filter.(KVFilter) - } - return nil +// GetScope returns the geographical scope of the transaction. +func (txn *KVTxn) GetScope() string { + return txn.scope } // Commit commits the transaction operations to KV store. @@ -220,7 +266,7 @@ func (txn *KVTxn) Commit(ctx context.Context) error { defer trace.StartRegion(ctx, "CommitTxn").End() if !txn.valid { - return tidbkv.ErrInvalidTxn + return tikverr.ErrInvalidTxn } defer txn.close() @@ -296,7 +342,7 @@ func (txn *KVTxn) Commit(ctx context.Context) error { } defer txn.store.txnLatches.UnLock(lock) if lock.IsStale() { - return tidbkv.ErrWriteConflictInTiDB.FastGenByArgs(txn.startTS) + return &tikverr.ErrWriteConflictInLatch{StartTS: txn.startTS} } err = committer.execute(ctx) if val == nil || sessionID > 0 { @@ -316,7 +362,7 @@ func (txn *KVTxn) close() { // Rollback undoes the transaction operations to KV store. func (txn *KVTxn) Rollback() error { if !txn.valid { - return tidbkv.ErrInvalidTxn + return tikverr.ErrInvalidTxn } start := time.Now() // Clean up pessimistic lock. @@ -379,7 +425,7 @@ func (txn *KVTxn) onCommitted(err error) { } info := TxnInfo{ - TxnScope: txn.GetUnionStore().GetOption(kv.TxnScope).(string), + TxnScope: txn.GetScope(), StartTS: txn.startTS, CommitTS: txn.commitTS, TxnCommitMode: commitMode, diff --git a/store/tikv/unionstore/memdb.go b/store/tikv/unionstore/memdb.go index a008a7cc6e0ab..7bfc11971e4e2 100644 --- a/store/tikv/unionstore/memdb.go +++ b/store/tikv/unionstore/memdb.go @@ -20,7 +20,6 @@ import ( "sync" "unsafe" - tidbkv "github.com/pingcap/tidb/kv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" ) @@ -217,7 +216,7 @@ func (db *MemDB) UpdateFlags(key []byte, ops ...kv.FlagsOp) { // v must NOT be nil or empty, otherwise it returns ErrCannotSetNilValue. func (db *MemDB) Set(key []byte, value []byte) error { if len(value) == 0 { - return tidbkv.ErrCannotSetNilValue + return tikverr.ErrCannotSetNilValue } return db.set(key, value) } @@ -225,7 +224,7 @@ func (db *MemDB) Set(key []byte, value []byte) error { // SetWithFlags put key-value into the last active staging buffer with the given KeyFlags. func (db *MemDB) SetWithFlags(key []byte, value []byte, ops ...kv.FlagsOp) error { if len(value) == 0 { - return tidbkv.ErrCannotSetNilValue + return tikverr.ErrCannotSetNilValue } return db.set(key, value, ops...) } @@ -281,7 +280,10 @@ func (db *MemDB) set(key []byte, value []byte, ops ...kv.FlagsOp) error { if value != nil { if size := uint64(len(key) + len(value)); size > db.entrySizeLimit { - return tidbkv.ErrEntryTooLarge.GenWithStackByArgs(db.entrySizeLimit, size) + return &tikverr.ErrEntryTooLarge{ + Limit: db.entrySizeLimit, + Size: size, + } } } @@ -307,7 +309,7 @@ func (db *MemDB) set(key []byte, value []byte, ops ...kv.FlagsOp) error { db.setValue(x, value) if uint64(db.Size()) > db.bufferSizeLimit { - return tidbkv.ErrTxnTooLarge.GenWithStackByArgs(db.Size()) + return &tikverr.ErrTxnTooLarge{Size: db.Size()} } return nil } diff --git a/table/tables/tables.go b/table/tables/tables.go index e725130834583..8fd3cca9e2657 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -322,6 +322,10 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx sessionctx.Context, sh := memBuffer.Staging() defer memBuffer.Cleanup(sh) + if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { + addTemporaryTableID(sctx, meta.ID) + } + var colIDs, binlogColIDs []int64 var row, binlogOldRow, binlogNewRow []types.Datum numColsCap := len(newData) + 1 // +1 for the extra handle column that we may need to append. @@ -405,7 +409,7 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx sessionctx.Context, } memBuffer.Release(sh) if shouldWriteBinlog(sctx, t.meta) { - if !t.meta.PKIsHandle { + if !t.meta.PKIsHandle && !t.meta.IsCommonHandle { binlogColIDs = append(binlogColIDs, model.ExtraHandleID) binlogOldRow = append(binlogOldRow, types.NewIntDatum(h.IntValue())) binlogNewRow = append(binlogNewRow, types.NewIntDatum(h.IntValue())) @@ -584,6 +588,14 @@ func TryGetCommonPkColumns(tbl table.Table) []*table.Column { return pkCols } +func addTemporaryTableID(sctx sessionctx.Context, id int64) { + txnCtx := sctx.GetSessionVars().TxnCtx + if txnCtx.GlobalTemporaryTables == nil { + txnCtx.GlobalTemporaryTables = make(map[int64]struct{}) + } + txnCtx.GlobalTemporaryTables[id] = struct{}{} +} + // AddRecord implements table.Table AddRecord interface. func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) { txn, err := sctx.Txn(true) @@ -596,6 +608,10 @@ func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts . fn.ApplyOn(&opt) } + if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { + addTemporaryTableID(sctx, meta.ID) + } + var ctx context.Context if opt.Ctx != nil { ctx = opt.Ctx @@ -962,15 +978,16 @@ func DecodeRawRowData(ctx sessionctx.Context, meta *model.TableInfo, h kv.Handle } // GetChangingColVal gets the changing column value when executing "modify/change column" statement. +// For statement like update-where, it will fetch the old row out and insert it into kv again. +// Since update statement can see the writable columns, it is responsible for the casting relative column / get the fault value here. +// old row : a-b-[nil] +// new row : a-b-[a'/default] +// Thus the writable new row is corresponding to Write-Only constraints. func GetChangingColVal(ctx sessionctx.Context, cols []*table.Column, col *table.Column, rowMap map[int64]types.Datum, defaultVals []types.Datum) (_ types.Datum, isDefaultVal bool, err error) { relativeCol := cols[col.ChangeStateInfo.DependencyColumnOffset] idxColumnVal, ok := rowMap[relativeCol.ID] if ok { - // It needs cast values here when filling back column or index values in "modify/change column" statement. - if ctx.GetSessionVars().StmtCtx.IsDDLJobInQueue { - return idxColumnVal, false, nil - } - idxColumnVal, err := table.CastValue(ctx, rowMap[relativeCol.ID], col.ColumnInfo, false, false) + idxColumnVal, err = table.CastValue(ctx, idxColumnVal, col.ColumnInfo, false, false) // TODO: Consider sql_mode and the error msg(encounter this error check whether to rollback). if err != nil { return idxColumnVal, false, errors.Trace(err) @@ -992,6 +1009,11 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type if err != nil { return err } + + if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { + addTemporaryTableID(ctx, meta.ID) + } + // The table has non-public column and this column is doing the operation of "modify/change column". if len(t.Columns) > len(r) && t.Columns[len(r)].ChangeStateInfo != nil { r = append(r, r[t.Columns[len(r)].ChangeStateInfo.DependencyColumnOffset]) @@ -1008,7 +1030,7 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type colIDs = append(colIDs, col.ID) } var binlogRow []types.Datum - if !t.meta.PKIsHandle { + if !t.meta.PKIsHandle && !t.meta.IsCommonHandle { colIDs = append(colIDs, model.ExtraHandleID) binlogRow = make([]types.Datum, 0, len(r)+1) binlogRow = append(binlogRow, r...) @@ -1386,7 +1408,7 @@ func shouldWriteBinlog(ctx sessionctx.Context, tblInfo *model.TableInfo) bool { if ctx.GetSessionVars().BinlogClient == nil { return false } - return !ctx.GetSessionVars().InRestrictedSQL && !tblInfo.IsCommonHandle + return !ctx.GetSessionVars().InRestrictedSQL } func (t *TableCommon) getMutation(ctx sessionctx.Context) *binlog.TableMutation { diff --git a/util/logutil/hex.go b/util/logutil/hex.go index e8ca2faeccefe..1bff4ec591404 100644 --- a/util/logutil/hex.go +++ b/util/logutil/hex.go @@ -6,7 +6,7 @@ // // http://www.apache.org/licenses/LICENSE-2.0 // -// Unless required by aprettyPrintlicable law or agreed to in writing, software +// Unless required by applicable law or agreed to in writing, software // distributed under the License is distributed on an "AS IS" BASIS, // See the License for the specific language governing permissions and // limitations under the License. diff --git a/util/math/math.go b/util/math/math.go index 3a25178326261..b9d5bf3a94477 100644 --- a/util/math/math.go +++ b/util/math/math.go @@ -48,3 +48,8 @@ func StrLenOfInt64Fast(x int64) int { } return size + StrLenOfUint64Fast(uint64(Abs(x))) } + +// Log2 is used for export purposes +func Log2(x float64) float64 { + return math.Log2(x) +} diff --git a/util/ranger/checker.go b/util/ranger/checker.go index 3f702b35d18b6..97a0c782ce3dc 100644 --- a/util/ranger/checker.go +++ b/util/ranger/checker.go @@ -15,7 +15,6 @@ package ranger import ( "github.com/pingcap/parser/ast" - "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" @@ -162,8 +161,5 @@ func (c *conditionChecker) checkColumn(expr expression.Expression) bool { if !ok { return false } - if col.GetType().Tp == mysql.TypeEnum { - return false - } return c.colUniqueID == col.UniqueID } diff --git a/util/ranger/points.go b/util/ranger/points.go index e12233062b8a2..d98b548dcbb7a 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -102,6 +102,9 @@ func (r *pointSorter) Less(i, j int) bool { } func rangePointLess(sc *stmtctx.StatementContext, a, b *point) (bool, error) { + if a.value.Kind() == types.KindMysqlEnum && b.value.Kind() == types.KindMysqlEnum { + return rangePointEnumLess(sc, a, b) + } cmp, err := a.value.CompareDatum(sc, &b.value) if cmp != 0 { return cmp < 0, nil @@ -109,6 +112,14 @@ func rangePointLess(sc *stmtctx.StatementContext, a, b *point) (bool, error) { return rangePointEqualValueLess(a, b), errors.Trace(err) } +func rangePointEnumLess(sc *stmtctx.StatementContext, a, b *point) (bool, error) { + cmp := types.CompareInt64(a.value.GetInt64(), b.value.GetInt64()) + if cmp != 0 { + return cmp < 0, nil + } + return rangePointEqualValueLess(a, b), nil +} + func rangePointEqualValueLess(a, b *point) bool { if a.start && b.start { return !a.excl && b.excl @@ -319,6 +330,10 @@ func (r *builder) buildFormBinOp(expr *expression.ScalarFunction) []*point { return nil } + if ft.Tp == mysql.TypeEnum && ft.EvalType() == types.ETString { + return handleEnumFromBinOp(r.sc, ft, value, op) + } + switch op { case ast.NullEQ: if value.IsNull() { @@ -433,6 +448,50 @@ func handleBoundCol(ft *types.FieldType, val types.Datum, op string) (types.Datu return val, op, true } +func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val types.Datum, op string) []*point { + res := make([]*point, 0, len(ft.Elems)*2) + appendPointFunc := func(d types.Datum) { + res = append(res, &point{value: d, excl: false, start: true}) + res = append(res, &point{value: d, excl: false, start: false}) + } + + tmpEnum := types.Enum{} + for i := range ft.Elems { + tmpEnum.Name = ft.Elems[i] + tmpEnum.Value = uint64(i) + d := types.NewMysqlEnumDatum(tmpEnum) + if v, err := d.CompareDatum(sc, &val); err == nil { + switch op { + case ast.LT: + if v < 0 { + appendPointFunc(d) + } + case ast.LE: + if v <= 0 { + appendPointFunc(d) + } + case ast.GT: + if v > 0 { + appendPointFunc(d) + } + case ast.GE: + if v >= 0 { + appendPointFunc(d) + } + case ast.EQ: + if v == 0 { + appendPointFunc(d) + } + case ast.NE: + if v != 0 { + appendPointFunc(d) + } + } + } + } + return res +} + func (r *builder) buildFromIsTrue(expr *expression.ScalarFunction, isNot int, keepNull bool) []*point { if isNot == 1 { if keepNull { @@ -504,6 +563,13 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool) if dt.Kind() == types.KindString || dt.Kind() == types.KindBinaryLiteral { dt.SetString(dt.GetString(), colCollate) } + if expr.GetArgs()[0].GetType().Tp == mysql.TypeEnum { + dt, err = dt.ConvertTo(r.sc, expr.GetArgs()[0].GetType()) + if err != nil { + // in (..., an impossible value (not valid enum), ...), the range is empty, so skip it. + continue + } + } if expr.GetArgs()[0].GetType().Tp == mysql.TypeYear { dt, err = dt.ConvertToMysqlYear(r.sc, expr.GetArgs()[0].GetType()) if err != nil { diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index a414fb7fec75a..4f1efef7a7d90 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -101,6 +101,16 @@ func convertPoint(sc *stmtctx.StatementContext, point *point, tp *types.FieldTyp // Ignore the types.ErrOverflow when we convert TypeNewDecimal values. // A trimmed valid boundary point value would be returned then. Accordingly, the `excl` of the point // would be adjusted. Impossible ranges would be skipped by the `validInterval` call later. + } else if tp.Tp == mysql.TypeEnum && terror.ErrorEqual(err, types.ErrTruncated) { + // Ignore the types.ErrorTruncated when we convert TypeEnum values. + // We should cover Enum upper overflow, and convert to the biggest value. + if point.value.GetInt64() > 0 { + upperEnum, err := types.ParseEnumValue(tp.Elems, uint64(len(tp.Elems))) + if err != nil { + return nil, err + } + casted.SetMysqlEnum(upperEnum, tp.Collate) + } } else { return point, errors.Trace(err) } diff --git a/util/ranger/testdata/ranger_suite_out.json b/util/ranger/testdata/ranger_suite_out.json index c652916186c16..f56a1567e6043 100644 --- a/util/ranger/testdata/ranger_suite_out.json +++ b/util/ranger/testdata/ranger_suite_out.json @@ -392,13 +392,11 @@ { "SQL": "select * from t;", "Plan": [ - "PartitionUnion_9 10004.00 root ", - "├─TableReader_11 1.00 root data:TableFullScan_10", - "│ └─TableFullScan_10 1.00 cop[tikv] table:t, partition:p0 keep order:false", - "├─TableReader_13 3.00 root data:TableFullScan_12", - "│ └─TableFullScan_12 3.00 cop[tikv] table:t, partition:p1 keep order:false", - "└─TableReader_15 10000.00 root data:TableFullScan_14", - " └─TableFullScan_14 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "PartitionUnion_8 4.00 root ", + "├─TableReader_10 1.00 root data:TableFullScan_9", + "│ └─TableFullScan_9 1.00 cop[tikv] table:t, partition:p0 keep order:false", + "└─TableReader_12 3.00 root data:TableFullScan_11", + " └─TableFullScan_11 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0000 0", @@ -461,14 +459,12 @@ { "SQL": "select * from t where a < 1;", "Plan": [ - "Selection_10 8003.20 root lt(test.t.a, 1)", - "└─PartitionUnion_11 10004.00 root ", - " ├─TableReader_13 1.00 root data:TableFullScan_12", - " │ └─TableFullScan_12 1.00 cop[tikv] table:t, partition:p0 keep order:false", - " ├─TableReader_15 3.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 3.00 cop[tikv] table:t, partition:p1 keep order:false", - " └─TableReader_17 10000.00 root data:TableFullScan_16", - " └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "Selection_9 3.20 root lt(test.t.a, 1)", + "└─PartitionUnion_10 4.00 root ", + " ├─TableReader_12 1.00 root data:TableFullScan_11", + " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", + " └─TableReader_14 3.00 root data:TableFullScan_13", + " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0000 0" @@ -477,14 +473,12 @@ { "SQL": "select * from t where a < 3;", "Plan": [ - "Selection_10 8003.20 root lt(test.t.a, 3)", - "└─PartitionUnion_11 10004.00 root ", - " ├─TableReader_13 1.00 root data:TableFullScan_12", - " │ └─TableFullScan_12 1.00 cop[tikv] table:t, partition:p0 keep order:false", - " ├─TableReader_15 3.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 3.00 cop[tikv] table:t, partition:p1 keep order:false", - " └─TableReader_17 10000.00 root data:TableFullScan_16", - " └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "Selection_9 3.20 root lt(test.t.a, 3)", + "└─PartitionUnion_10 4.00 root ", + " ├─TableReader_12 1.00 root data:TableFullScan_11", + " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", + " └─TableReader_14 3.00 root data:TableFullScan_13", + " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0000 0", @@ -504,14 +498,12 @@ { "SQL": "select * from t where a > 0;", "Plan": [ - "Selection_10 8003.20 root gt(test.t.a, 0)", - "└─PartitionUnion_11 10004.00 root ", - " ├─TableReader_13 1.00 root data:TableFullScan_12", - " │ └─TableFullScan_12 1.00 cop[tikv] table:t, partition:p0 keep order:false", - " ├─TableReader_15 3.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 3.00 cop[tikv] table:t, partition:p1 keep order:false", - " └─TableReader_17 10000.00 root data:TableFullScan_16", - " └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "Selection_9 3.20 root gt(test.t.a, 0)", + "└─PartitionUnion_10 4.00 root ", + " ├─TableReader_12 1.00 root data:TableFullScan_11", + " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", + " └─TableReader_14 3.00 root data:TableFullScan_13", + " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0001 -1", @@ -522,14 +514,12 @@ { "SQL": "select * from t where a > -1;", "Plan": [ - "Selection_10 8003.20 root gt(test.t.a, -1)", - "└─PartitionUnion_11 10004.00 root ", - " ├─TableReader_13 1.00 root data:TableFullScan_12", - " │ └─TableFullScan_12 1.00 cop[tikv] table:t, partition:p0 keep order:false", - " ├─TableReader_15 3.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 3.00 cop[tikv] table:t, partition:p1 keep order:false", - " └─TableReader_17 10000.00 root data:TableFullScan_16", - " └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "Selection_9 3.20 root gt(test.t.a, -1)", + "└─PartitionUnion_10 4.00 root ", + " ├─TableReader_12 1.00 root data:TableFullScan_11", + " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", + " └─TableReader_14 3.00 root data:TableFullScan_13", + " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0000 0", @@ -541,14 +531,12 @@ { "SQL": "select * from t where a > 3;", "Plan": [ - "Selection_10 8003.20 root gt(test.t.a, 3)", - "└─PartitionUnion_11 10004.00 root ", - " ├─TableReader_13 1.00 root data:TableFullScan_12", - " │ └─TableFullScan_12 1.00 cop[tikv] table:t, partition:p0 keep order:false", - " ├─TableReader_15 3.00 root data:TableFullScan_14", - " │ └─TableFullScan_14 3.00 cop[tikv] table:t, partition:p1 keep order:false", - " └─TableReader_17 10000.00 root data:TableFullScan_16", - " └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "Selection_9 3.20 root gt(test.t.a, 3)", + "└─PartitionUnion_10 4.00 root ", + " ├─TableReader_12 1.00 root data:TableFullScan_11", + " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", + " └─TableReader_14 3.00 root data:TableFullScan_13", + " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": null } diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index 111481ad7b40d..3fd63c05fae96 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -57,14 +57,8 @@ func NewRowDecoder(tbl table.Table, cols []*table.Column, decodeColMap map[int64 tps := make([]*types.FieldType, len(cols)) for _, col := range cols { - if col.ChangeStateInfo == nil { - tps[col.Offset] = &col.FieldType - } else { - // Since changing column in the mutRow will be set with relative column's old value in the process of column-type-change, - // we should set fieldType as the relative column does. Otherwise it may get a panic, take change json to int as an example, - // setting json value to a int type column in mutRow will panic because it lacks of offset array. - tps[col.Offset] = &cols[col.ChangeStateInfo.DependencyColumnOffset].FieldType - } + // Even for changing column in column type change, we target field type uniformly. + tps[col.Offset] = &col.FieldType } var pkCols []int64 switch { @@ -119,6 +113,66 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv. } rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) } + return rd.EvalRemainedExprColumnMap(ctx, sysLoc, row) +} + +// BuildFullDecodeColMap builds a map that contains [columnID -> struct{*table.Column, expression.Expression}] from all columns. +func BuildFullDecodeColMap(cols []*table.Column, schema *expression.Schema) map[int64]Column { + decodeColMap := make(map[int64]Column, len(cols)) + for _, col := range cols { + decodeColMap[col.ID] = Column{ + Col: col, + GenExpr: schema.Columns[col.Offset].VirtualExpr, + } + } + return decodeColMap +} + +// CurrentRowWithDefaultVal returns current decoding row with default column values set properly. +// Please make sure calling DecodeAndEvalRowWithMap first. +func (rd *RowDecoder) CurrentRowWithDefaultVal() chunk.Row { + return rd.mutRow.ToRow() +} + +// DecodeTheExistedColumnMap is used by ddl column-type-change first column reorg stage. +// In the function, we only decode the existed column in the row and fill the default value. +// For changing column, we shouldn't cast it here, because we will do a unified cast operation latter. +// For generated column, we didn't cast it here too, because the eval process will depend on the changing column. +func (rd *RowDecoder) DecodeTheExistedColumnMap(ctx sessionctx.Context, handle kv.Handle, b []byte, decodeLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { + var err error + if rowcodec.IsNewFormat(b) { + row, err = tablecodec.DecodeRowWithMapNew(b, rd.colTypes, decodeLoc, row) + } else { + row, err = tablecodec.DecodeRowWithMap(b, rd.colTypes, decodeLoc, row) + } + if err != nil { + return nil, err + } + row, err = tablecodec.DecodeHandleToDatumMap(handle, rd.pkCols, rd.colTypes, decodeLoc, row) + if err != nil { + return nil, err + } + for _, dCol := range rd.colMap { + colInfo := dCol.Col.ColumnInfo + val, ok := row[colInfo.ID] + if ok || dCol.GenExpr != nil || dCol.Col.ChangeStateInfo != nil { + rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) + continue + } + // Get the default value of the column in the generated column expression. + val, err = tables.GetColDefaultValue(ctx, dCol.Col, rd.defaultVals) + if err != nil { + return nil, err + } + rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) + } + // return the existed column map here. + return row, nil +} + +// EvalRemainedExprColumnMap is used by ddl column-type-change first column reorg stage. +// It is always called after DecodeTheExistedColumnMap to finish the generated column evaluation. +func (rd *RowDecoder) EvalRemainedExprColumnMap(ctx sessionctx.Context, sysLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { keys := make([]int, 0, len(rd.colMap)) ids := make(map[int]int, len(rd.colMap)) for k, col := range rd.colMap { @@ -155,23 +209,6 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv. row[int64(ids[id])] = val } + // return the existed and evaluated column map here. return row, nil } - -// BuildFullDecodeColMap builds a map that contains [columnID -> struct{*table.Column, expression.Expression}] from all columns. -func BuildFullDecodeColMap(cols []*table.Column, schema *expression.Schema) map[int64]Column { - decodeColMap := make(map[int64]Column, len(cols)) - for _, col := range cols { - decodeColMap[col.ID] = Column{ - Col: col, - GenExpr: schema.Columns[col.Offset].VirtualExpr, - } - } - return decodeColMap -} - -// CurrentRowWithDefaultVal returns current decoding row with default column values set properly. -// Please make sure calling DecodeAndEvalRowWithMap first. -func (rd *RowDecoder) CurrentRowWithDefaultVal() chunk.Row { - return rd.mutRow.ToRow() -} diff --git a/util/sem/sem.go b/util/sem/sem.go index 0dde650b9d765..8c3d2b456d991 100644 --- a/util/sem/sem.go +++ b/util/sem/sem.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/logutil" ) const ( @@ -56,6 +57,7 @@ const ( tidbProfileMemory = "tidb_profile_memory" tidbProfileMutex = "tidb_profile_mutex" tikvProfileCPU = "tikv_profile_cpu" + tidbGCLeaderDesc = "tidb_gc_leader_desc" restrictedPriv = "RESTRICTED_" ) @@ -68,6 +70,8 @@ var ( func Enable() { atomic.StoreInt32(&semEnabled, 1) variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.On) + // write to log so users understand why some operations are weird. + logutil.BgLogger().Info("tidb-server is operating with security enhanced mode (SEM) enabled") } // Disable disables SEM. This is intended to be used by the test-suite. @@ -116,6 +120,11 @@ func IsInvisibleTable(dbLowerName, tblLowerName string) bool { return false } +// IsInvisibleStatusVar returns true if the status var needs to be hidden +func IsInvisibleStatusVar(varName string) bool { + return varName == tidbGCLeaderDesc +} + // IsRestrictedPrivilege returns true if the privilege shuld not be satisfied by SUPER // As most dynamic privileges are. func IsRestrictedPrivilege(privNameInUpper string) bool { diff --git a/util/sem/sem_test.go b/util/sem/sem_test.go index d22f9e52b5e57..c303d2195c7f4 100644 --- a/util/sem/sem_test.go +++ b/util/sem/sem_test.go @@ -67,3 +67,10 @@ func (s *testSecurity) TestIsRestrictedPrivilege(c *C) { c.Assert(IsRestrictedPrivilege("BACKUP_ADMIN"), IsFalse) c.Assert(IsRestrictedPrivilege("aa"), IsFalse) } + +func (s *testSecurity) TestIsInvisibleStatusVar(c *C) { + c.Assert(IsInvisibleStatusVar(tidbGCLeaderDesc), IsTrue) + c.Assert(IsInvisibleStatusVar("server_id"), IsFalse) + c.Assert(IsInvisibleStatusVar("ddl_schema_version"), IsFalse) + c.Assert(IsInvisibleStatusVar("Ssl_version"), IsFalse) +}