diff --git a/.github/workflows/integration-test-br-compatibility.yml b/.github/workflows/integration-test-br-compatibility.yml index b455799b91afa..5df7168467005 100644 --- a/.github/workflows/integration-test-br-compatibility.yml +++ b/.github/workflows/integration-test-br-compatibility.yml @@ -22,6 +22,9 @@ concurrency: group: ${{ github.ref }}-${{ github.workflow }} cancel-in-progress: true +permissions: + contents: read # to fetch code (actions/checkout) + jobs: check: runs-on: ubuntu-latest diff --git a/.github/workflows/integration-test-compile-br.yml b/.github/workflows/integration-test-compile-br.yml index 17cbd48b25d7b..7b22aa4e24bbe 100644 --- a/.github/workflows/integration-test-compile-br.yml +++ b/.github/workflows/integration-test-compile-br.yml @@ -36,6 +36,9 @@ concurrency: group: ${{ github.ref }}-${{ github.workflow }} cancel-in-progress: true +permissions: + contents: read # to fetch code (actions/checkout) + jobs: compile-windows: if: github.event_name == 'push' || github.event_name == 'pull_request' && github.event.label.name == 'action/run-br-cross-platform-build' diff --git a/.github/workflows/integration-test-dumpling.yml b/.github/workflows/integration-test-dumpling.yml index e1dd6a58d3e6d..14bae6e6b7115 100644 --- a/.github/workflows/integration-test-dumpling.yml +++ b/.github/workflows/integration-test-dumpling.yml @@ -40,6 +40,9 @@ concurrency: group: ${{ github.workflow }}-${{ github.ref }} cancel-in-progress: true +permissions: + contents: read # to fetch code (actions/checkout) + jobs: integration-test: strategy: diff --git a/.github/workflows/misc.yml b/.github/workflows/misc.yml index 78e0f9a46d4a7..94b68e9c95510 100644 --- a/.github/workflows/misc.yml +++ b/.github/workflows/misc.yml @@ -12,8 +12,15 @@ concurrency: group: ${{ github.workflow }}-${{ github.ref }} cancel-in-progress: true +permissions: + contents: read # to fetch code (actions/checkout) + jobs: check: + permissions: + contents: read # to fetch code (actions/checkout) + pull-requests: write # to comment on pull-requests + runs-on: ubuntu-latest steps: - uses: actions/checkout@v3 diff --git a/cmd/explaintest/main.go b/cmd/explaintest/main.go index 32f88dc30a6bc..d5f7f0c98b0a1 100644 --- a/cmd/explaintest/main.go +++ b/cmd/explaintest/main.go @@ -724,6 +724,7 @@ func main() { "set @@tidb_window_concurrency=4", "set @@tidb_projection_concurrency=4", "set @@tidb_distsql_scan_concurrency=15", + "set @@tidb_enable_clustered_index='int_only';", "set @@global.tidb_enable_clustered_index=0;", "set @@global.tidb_mem_quota_query=34359738368", "set @@tidb_mem_quota_query=34359738368", diff --git a/config/config.go b/config/config.go index 4deb0d9c9c0ad..8bb9cf1776920 100644 --- a/config/config.go +++ b/config/config.go @@ -278,6 +278,10 @@ type Config struct { Plugin Plugin `toml:"plugin" json:"plugin"` MaxServerConnections uint32 `toml:"max-server-connections" json:"max-server-connections"` RunDDL bool `toml:"run-ddl" json:"run-ddl"` + // TiDBMaxReuseChunk indicates max cached chunk num + TiDBMaxReuseChunk uint32 `toml:"tidb-max-reuse-chunk" json:"tidb-max-reuse-chunk"` + // TiDBMaxReuseColumn indicates max cached column num + TiDBMaxReuseColumn uint32 `toml:"tidb-max-reuse-column" json:"tidb-max-reuse-column"` } // UpdateTempStoragePath is to update the `TempStoragePath` if port/statusPort was changed @@ -975,6 +979,8 @@ var defaultConf = Config{ NewCollationsEnabledOnFirstBootstrap: true, EnableGlobalKill: true, TrxSummary: DefaultTrxSummary(), + TiDBMaxReuseChunk: 64, + TiDBMaxReuseColumn: 256, } var ( diff --git a/config/config_test.go b/config/config_test.go index 50e3227de049c..9a6d12a284817 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -730,6 +730,8 @@ enable-enum-length-limit = false stores-refresh-interval = 30 enable-forwarding = true enable-global-kill = true +tidb-max-reuse-chunk = 10 +tidb-max-reuse-column = 20 [performance] txn-total-size-limit=2000 tcp-no-delay = false @@ -798,6 +800,8 @@ max_connections = 200 require.True(t, conf.RepairMode) require.Equal(t, uint64(16), conf.TiKVClient.ResolveLockLiteThreshold) require.Equal(t, uint32(200), conf.Instance.MaxConnections) + require.Equal(t, uint32(10), conf.TiDBMaxReuseChunk) + require.Equal(t, uint32(20), conf.TiDBMaxReuseColumn) require.Equal(t, []string{"tiflash"}, conf.IsolationRead.Engines) require.Equal(t, 3080, conf.MaxIndexLength) require.Equal(t, 70, conf.IndexLimit) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index c0e087ef4a046..b10f01c272ef9 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -4068,7 +4068,7 @@ func TestCreateAndAlterIntervalPartition(t *testing.T) { tk.MustQuery("select count(*) from ipt").Check(testkit.Rows("27")) - tk.MustExec("create table idpt (id date primary key, val varchar(255), key (val)) partition by range COLUMNS (id) INTERVAL (1 week) FIRST PARTITION LESS THAN ('2022-02-01') LAST PARTITION LESS THAN ('2022-03-29') NULL PARTITION MAXVALUE PARTITION") + tk.MustExec("create table idpt (id date primary key nonclustered, val varchar(255), key (val)) partition by range COLUMNS (id) INTERVAL (1 week) FIRST PARTITION LESS THAN ('2022-02-01') LAST PARTITION LESS THAN ('2022-03-29') NULL PARTITION MAXVALUE PARTITION") tk.MustQuery("SHOW CREATE TABLE idpt").Check(testkit.Rows( "idpt CREATE TABLE `idpt` (\n" + " `id` date NOT NULL,\n" + @@ -4094,7 +4094,7 @@ func TestCreateAndAlterIntervalPartition(t *testing.T) { // if using a month with 31 days. // But managing partitions with the day-part of 29, 30 or 31 will be troublesome, since once the FIRST is not 31 // both the ALTER TABLE t FIRST PARTITION and MERGE FIRST PARTITION will have issues - tk.MustExec("create table t (id date primary key, val varchar(255), key (val)) partition by range COLUMNS (id) INTERVAL (1 MONTH) FIRST PARTITION LESS THAN ('2022-01-31') LAST PARTITION LESS THAN ('2022-05-31')") + tk.MustExec("create table t (id date primary key nonclustered, val varchar(255), key (val)) partition by range COLUMNS (id) INTERVAL (1 MONTH) FIRST PARTITION LESS THAN ('2022-01-31') LAST PARTITION LESS THAN ('2022-05-31')") tk.MustQuery("show create table t").Check(testkit.Rows( "t CREATE TABLE `t` (\n" + " `id` date NOT NULL,\n" + @@ -4662,4 +4662,82 @@ func TestAlterModifyColumnOnPartitionedTable(t *testing.T) { "34 34💥", "46 46", "57 57")) + tk.MustGetErrCode(`alter table t modify a varchar(20)`, errno.ErrUnsupportedDDLOperation) +} + +func TestAlterModifyColumnOnPartitionedTableFail(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "modColPartFail" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk.MustExec(`create table t (a int unsigned, b varchar(255), key (b)) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20), partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into t values (7, "07"), (8, "08"),(23,"23"),(34,"34💥"),(46,"46"),(57,"57")`) + tk.MustGetErrCode(`alter table t modify a varchar(255)`, errno.ErrUnsupportedDDLOperation) + tk.MustGetErrCode(`alter table t modify a float`, mysql.ErrFieldTypeNotAllowedAsPartitionField) + tk.MustExec(`drop table t`) + tk.MustExec(`create table t (b int unsigned, a varchar(255), key (b)) partition by range columns (a) (partition p0 values less than (""), partition p1 values less than ("11111"), partition pMax values less than (MAXVALUE))`) + tk.MustExec(`insert into t values (7, "07"), (8, "08"),(23,"23"),(34,"34 💥💥Longer than 11111"),(46,"46"),(57,"57")`) + tk.MustExec(`alter table t modify a varchar(50)`) + tk.MustGetErrCode(`alter table t modify a float`, mysql.ErrFieldTypeNotAllowedAsPartitionField) + tk.MustGetErrCode(`alter table t modify a int`, errno.ErrUnsupportedDDLOperation) + tk.MustContainErrMsg(`alter table t modify a varchar(4)`, "[ddl:8200]New column does not match partition definitions: [ddl:1654]Partition column values of incorrect type") + tk.MustGetErrCode(`alter table t modify a varchar(5)`, errno.WarnDataTruncated) + tk.MustExec(`SET SQL_MODE = ''`) + tk.MustExec(`alter table t modify a varchar(5)`) + // fix https://github.com/pingcap/tidb/issues/38669 and update this + tk.MustQuery(`show warnings`).Check(testkit.Rows()) + tk.MustExec(`SET SQL_MODE = DEFAULT`) + tk.MustQuery(`select * from t`).Sort().Check(testkit.Rows(""+ + "23 23", + "34 34 💥💥", + "46 46", + "57 57", + "7 07", + "8 08")) + tStr := "" + + "CREATE TABLE `t` (\n" + + " `b` int(10) unsigned DEFAULT NULL,\n" + + " `a` varchar(5) DEFAULT NULL,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE COLUMNS(`a`)\n" + + "(PARTITION `p0` VALUES LESS THAN (''),\n" + + " PARTITION `p1` VALUES LESS THAN ('11111'),\n" + + " PARTITION `pMax` VALUES LESS THAN (MAXVALUE))" + tk.MustQuery(`show create table t`).Check(testkit.Rows("t " + tStr)) + tk.MustExec(`drop table t`) + tk.MustExec(tStr) + tk.MustExec(`drop table t`) + tk.MustExec("create table t (a int, b varchar(255), key (b)) partition by range (a) (partition `p-300` values less than (-300), partition p0 values less than (0), partition p300 values less than (300))") + tk.MustExec(`insert into t values (-400, "-400"), (-100, "-100"), (0, "0"), (100, "100"), (290, "290")`) + tk.MustContainErrMsg(`alter table t modify a int unsigned`, "[ddl:8200]Unsupported modify column, decreasing length of int may result in truncation and change of partition") + tk.MustContainErrMsg(`alter table t modify a tinyint`, "[ddl:8200]Unsupported modify column, decreasing length of int may result in truncation and change of partition") + tk.MustExec(`set sql_mode = ''`) + tk.MustContainErrMsg(`alter table t modify a tinyint`, "[ddl:8200]Unsupported modify column, decreasing length of int may result in truncation and change of partition") + tk.MustQuery("select * from t partition (`p-300`)").Sort().Check(testkit.Rows("-400 -400")) + tk.MustExec(`set sql_mode = default`) + tk.MustContainErrMsg(`alter table t modify a smallint`, "[ddl:8200]Unsupported modify column, decreasing length of int may result in truncation and change of partition") + tk.MustExec(`alter table t modify a bigint`) + tk.MustExec(`drop table t`) + tk.MustExec("create table t (a int, b varchar(255), key (b)) partition by range columns (a) (partition `p-300` values less than (-300), partition p0 values less than (0), partition p300 values less than (300))") + tk.MustExec(`insert into t values (-400, "-400"), (-100, "-100"), (0, "0"), (100, "100"), (290, "290")`) + tk.MustContainErrMsg(`alter table t modify a int unsigned`, "[ddl:8200]Unsupported modify column: can't change the partitioning column, since it would require reorganize all partitions") + tk.MustContainErrMsg(`alter table t modify a tinyint`, "[ddl:8200]New column does not match partition definitions: [ddl:1654]Partition column values of incorrect type") + tk.MustExec(`set sql_mode = ''`) + tk.MustContainErrMsg(`alter table t modify a tinyint`, "[ddl:8200]New column does not match partition definitions: [ddl:1654]Partition column values of incorrect type") + tk.MustQuery("select * from t partition (`p-300`)").Sort().Check(testkit.Rows("-400 -400")) + tk.MustExec(`set sql_mode = default`) + // OK to decrease, since with RANGE COLUMNS, it will check the partition definition values against the new type + tk.MustExec(`alter table t modify a smallint`) + tk.MustExec(`alter table t modify a bigint`) + + tk.MustExec(`drop table t`) + + tk.MustExec(`create table t (a int, b varchar(255), key (b)) partition by list columns (b) (partition p1 values in ("1", "ab", "12345"), partition p2 values in ("2", "abc", "999999"))`) + tk.MustExec(`insert into t values (1, "1"), (2, "2"), (999999, "999999")`) + tk.MustContainErrMsg(`alter table t modify column b varchar(5)`, "[ddl:8200]New column does not match partition definitions: [ddl:1654]Partition column values of incorrect type") + tk.MustExec(`set sql_mode = ''`) + tk.MustContainErrMsg(`alter table t modify column b varchar(5)`, "[ddl:8200]New column does not match partition definitions: [ddl:1654]Partition column values of incorrect type") + tk.MustExec(`set sql_mode = default`) } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 3ce114b2e78a2..e67c0eb001128 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/format" @@ -2831,23 +2832,30 @@ func checkPartitionByList(ctx sessionctx.Context, tbInfo *model.TableInfo) error return checkListPartitionValue(ctx, tbInfo) } +func isColTypeAllowedAsPartitioningCol(fieldType types.FieldType) bool { + // The permitted data types are shown in the following list: + // All integer types + // DATE and DATETIME + // CHAR, VARCHAR, BINARY, and VARBINARY + // See https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-columns.html + // Note that also TIME is allowed in MySQL. Also see https://bugs.mysql.com/bug.php?id=84362 + switch fieldType.GetType() { + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeDuration: + case mysql.TypeVarchar, mysql.TypeString: + default: + return false + } + return true +} + func checkColumnsPartitionType(tbInfo *model.TableInfo) error { for _, col := range tbInfo.Partition.Columns { colInfo := tbInfo.FindPublicColumnByName(col.L) if colInfo == nil { return errors.Trace(dbterror.ErrFieldNotFoundPart) } - // The permitted data types are shown in the following list: - // All integer types - // DATE and DATETIME - // CHAR, VARCHAR, BINARY, and VARBINARY - // See https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-columns.html - // Note that also TIME is allowed in MySQL. Also see https://bugs.mysql.com/bug.php?id=84362 - switch colInfo.FieldType.GetType() { - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: - case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeDuration: - case mysql.TypeVarchar, mysql.TypeString: - default: + if !isColTypeAllowedAsPartitioningCol(colInfo.FieldType) { return dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(col.O) } } @@ -4604,6 +4612,88 @@ func GetModifiableColumnJob( } } + // Check that the column change does not affect the partitioning column + // It must keep the same type, int [unsigned], [var]char, date[time] + if t.Meta().Partition != nil { + pt, ok := t.(table.PartitionedTable) + if !ok { + // Should never happen! + return nil, dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(newCol.Name.O) + } + isPartitioningColumn := false + for _, name := range pt.GetPartitionColumnNames() { + if strings.EqualFold(name.L, col.Name.L) { + isPartitioningColumn = true + } + } + if isPartitioningColumn { + if !isColTypeAllowedAsPartitioningCol(newCol.FieldType) { + return nil, dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(newCol.Name.O) + } + pi := pt.Meta().GetPartitionInfo() + if len(pi.Columns) == 0 { + // non COLUMNS partitioning, only checks INTs, not their actual range + // There are many edge cases, like when truncating SQL Mode is allowed + // which will change the partitioning expression value resulting in a + // different partition. Better be safe and not allow decreasing of length. + // TODO: Should we allow it in strict mode? Wait for a use case / request. + if newCol.FieldType.GetFlen() < col.FieldType.GetFlen() { + return nil, dbterror.ErrUnsupportedModifyCollation.GenWithStack("Unsupported modify column, decreasing length of int may result in truncation and change of partition") + } + } + // Basically only allow changes of the length/decimals for the column + // Note that enum is not allowed, so elems are not checked + // TODO: support partition by ENUM + if newCol.FieldType.EvalType() != col.FieldType.EvalType() || + newCol.FieldType.GetFlag() != col.FieldType.GetFlag() || + newCol.FieldType.GetCollate() != col.FieldType.GetCollate() || + newCol.FieldType.GetCharset() != col.FieldType.GetCharset() { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't change the partitioning column, since it would require reorganize all partitions") + } + // Generate a new PartitionInfo and validate it together with the new column definition + // Checks if all partition definition values are compatible. + // Similar to what buildRangePartitionDefinitions would do in terms of checks. + + tblInfo := pt.Meta() + newTblInfo := *tblInfo + // Replace col with newCol and see if we can generate a new SHOW CREATE TABLE + // and reparse it and build new partition definitions (which will do additional + // checks columns vs partition definition values + newCols := make([]*model.ColumnInfo, 0, len(newTblInfo.Columns)) + for _, c := range newTblInfo.Columns { + if c.ID == col.ID { + newCols = append(newCols, newCol.ColumnInfo) + continue + } + newCols = append(newCols, c) + } + newTblInfo.Columns = newCols + + var buf bytes.Buffer + AppendPartitionInfo(tblInfo.GetPartitionInfo(), &buf, mysql.ModeNone) + // The parser supports ALTER TABLE ... PARTITION BY ... even if the ddl code does not yet :) + // Ignoring warnings + stmt, _, err := parser.New().ParseSQL("ALTER TABLE t " + buf.String()) + if err != nil { + // Should never happen! + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("cannot parse generated PartitionInfo") + } + at, ok := stmt[0].(*ast.AlterTableStmt) + if !ok || len(at.Specs) != 1 || at.Specs[0].Partition == nil { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("cannot parse generated PartitionInfo") + } + pAst := at.Specs[0].Partition + sv := sctx.GetSessionVars().StmtCtx + oldTruncAsWarn, oldIgnoreTrunc := sv.TruncateAsWarning, sv.IgnoreTruncate + sv.TruncateAsWarning, sv.IgnoreTruncate = false, false + _, err = buildPartitionDefinitionsInfo(sctx, pAst.Definitions, &newTblInfo) + sv.TruncateAsWarning, sv.IgnoreTruncate = oldTruncAsWarn, oldIgnoreTrunc + if err != nil { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStack("New column does not match partition definitions: %s", err.Error()) + } + } + } + // We don't support modifying column from not_auto_increment to auto_increment. if !mysql.HasAutoIncrementFlag(col.GetFlag()) && mysql.HasAutoIncrementFlag(newCol.GetFlag()) { return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs("can't set auto_increment") diff --git a/ddl/index_modify_test.go b/ddl/index_modify_test.go index 00695b3da8f6e..38f091f41bc10 100644 --- a/ddl/index_modify_test.go +++ b/ddl/index_modify_test.go @@ -816,7 +816,7 @@ func TestDropIndexes(t *testing.T) { store := testkit.CreateMockStoreWithSchemaLease(t, indexModifyLease, mockstore.WithDDLChecker()) // drop multiple indexes - createSQL := "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));" + createSQL := "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id) nonclustered, key i1(c1), key i2(c2));" dropIdxSQL := "alter table test_drop_indexes drop index i1, drop index i2;" idxNames := []string{"i1", "i2"} testDropIndexes(t, store, createSQL, dropIdxSQL, idxNames) @@ -826,7 +826,7 @@ func TestDropIndexes(t *testing.T) { idxNames = []string{"primary", "i1"} testDropIndexes(t, store, createSQL, dropIdxSQL, idxNames) - createSQL = "create table test_drop_indexes (uuid varchar(32), c1 int, c2 int, primary key(uuid), unique key i1(c1), key i2(c2));" + createSQL = "create table test_drop_indexes (uuid varchar(32), c1 int, c2 int, primary key(uuid) nonclustered, unique key i1(c1), key i2(c2));" dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i1, drop index i2;" idxNames = []string{"primary", "i1", "i2"} testDropIndexes(t, store, createSQL, dropIdxSQL, idxNames) diff --git a/ddl/partition.go b/ddl/partition.go index c9a00a52fe5ec..5a6a5b561b146 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -2807,6 +2807,54 @@ func hexIfNonPrint(s string) string { return "0x" + hex.EncodeToString([]byte(driver.UnwrapFromSingleQuotes(s))) } +// AppendPartitionInfo is used in SHOW CREATE TABLE as well as generation the SQL syntax +// for the PartitionInfo during validation of various DDL commands +func AppendPartitionInfo(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, sqlMode mysql.SQLMode) { + if partitionInfo == nil { + return + } + // Since MySQL 5.1/5.5 is very old and TiDB aims for 5.7/8.0 compatibility, we will not + // include the /*!50100 or /*!50500 comments for TiDB. + // This also solves the issue with comments within comments that would happen for + // PLACEMENT POLICY options. + if partitionInfo.Type == model.PartitionTypeHash { + defaultPartitionDefinitions := true + for i, def := range partitionInfo.Definitions { + if def.Name.O != fmt.Sprintf("p%d", i) { + defaultPartitionDefinitions = false + break + } + if len(def.Comment) > 0 || def.PlacementPolicyRef != nil { + defaultPartitionDefinitions = false + break + } + } + + if defaultPartitionDefinitions { + fmt.Fprintf(buf, "\nPARTITION BY HASH (%s) PARTITIONS %d", partitionInfo.Expr, partitionInfo.Num) + return + } + } + // this if statement takes care of lists/range columns case + if len(partitionInfo.Columns) > 0 { + // partitionInfo.Type == model.PartitionTypeRange || partitionInfo.Type == model.PartitionTypeList + // Notice that MySQL uses two spaces between LIST and COLUMNS... + fmt.Fprintf(buf, "\nPARTITION BY %s COLUMNS(", partitionInfo.Type.String()) + for i, col := range partitionInfo.Columns { + buf.WriteString(stringutil.Escape(col.O, sqlMode)) + if i < len(partitionInfo.Columns)-1 { + buf.WriteString(",") + } + } + buf.WriteString(")\n(") + } else { + fmt.Fprintf(buf, "\nPARTITION BY %s (%s)\n(", partitionInfo.Type.String(), partitionInfo.Expr) + } + + AppendPartitionDefs(partitionInfo, buf, sqlMode) + buf.WriteString(")") +} + // AppendPartitionDefs generates a list of partition definitions needed for SHOW CREATE TABLE (in executor/show.go) // as well as needed for generating the ADD PARTITION query for INTERVAL partitioning of ALTER TABLE t LAST PARTITION // and generating the CREATE TABLE query from CREATE TABLE ... INTERVAL diff --git a/distsql/select_result.go b/distsql/select_result.go index 0e807b360d0ad..a2d6215987a32 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -311,7 +311,7 @@ func (r *selectResult) readFromDefault(ctx context.Context, chk *chunk.Chunk) er func (r *selectResult) readFromChunk(ctx context.Context, chk *chunk.Chunk) error { if r.respChunkDecoder == nil { r.respChunkDecoder = chunk.NewDecoder( - chunk.NewChunkWithCapacity(r.fieldTypes, 0), + r.ctx.GetSessionVars().GetNewChunk(r.fieldTypes, 0), r.fieldTypes, ) } diff --git a/docs/design/2022-09-28-flashback-to-timestamp.md b/docs/design/2022-09-28-flashback-to-timestamp.md new file mode 100644 index 0000000000000..9f3c2e2ef7fab --- /dev/null +++ b/docs/design/2022-09-28-flashback-to-timestamp.md @@ -0,0 +1,120 @@ +# Proposal: Flashback To Timestamp +- Author(s): [Defined2014](https://github.com/Defined2014) and [JmPotato](https://github.com/JmPotato) +- Tracking Issues: https://github.com/pingcap/tidb/issues/37197 and https://github.com/tikv/tikv/issues/13303 + +## Abstract + +This proposal aims to support `Flashback To Timestamp` and describe what `Flashback To Timestamp` should look like and how to implement it. + +## Background + +Some users want to `Flashback table/database/cluster` to the specified timestamp when there is a problem with the data like deleted some important keys, updated wrong values etc. + +TiDB uses MVCC to store key-values, which means it can easily get historical data at any timestamp. Based on this feature, TiDB already supports `Flashback To Timestamp`, for example, users can read historical data and update it through Snapshot Read which is not only inelegant, but also inefficient. + +Therefore, we propose to use a series of new SQL syntaxes to support this feature, and push down the read-write operations to storage side. + +## Detailed Design + +### Implementation Overview + +In TiKV, a multi-version concurrency control (MVCC) mechanism is introduced to avoid the overhead of introducing locks when data is updated concurrently. Under this mechanism, when TiDB modified data, it doesn't directly operate on the original value, but writes a data with the latest timestamp to cover it. The GC Worker in the background of TiDB will periodically update `tikv_gc_safe_point` and delete the version older than this point. `Flashback To Timestamp` is developmented based on this feature of TiKV. In order to improve execution efficiency and reduce data transmission overhead, TiKV has added two RPC interfaces called `PrepareFlashbackToVersion` and `FlashbackToVersion`. The protobuf related change shown below: + +```protobuf +// Preparing the flashback for a region/key range will "lock" the region +// so that there is no any read, write or schedule operation could be proposed before +// the actual flashback operation. +message PrepareFlashbackToVersionRequest { + Context context = 1; + bytes start_key = 2; + bytes end_key = 3; +} + +message PrepareFlashbackToVersionResponse { + errorpb.Error region_error = 1; + string error = 2; +} + +// Flashback the region to a specific point with the given `version`, please +// make sure the region is "locked" by `PrepareFlashbackToVersionRequest` first, +// otherwise this request will fail. +message FlashbackToVersionRequest { + Context context = 1; + // The TS version which the data should flashback to. + uint64 version = 2; + bytes start_key = 3; + bytes end_key = 4; + // The `start_ts`` and `commit_ts` which the newly written MVCC version will use. + uint64 start_ts = 5; + uint64 commit_ts = 6; +} + +message FlashbackToVersionResponse { + errorpb.Error region_error = 1; + string error = 2; +} +``` + +Then a `Flashback To Timestamp` DDL job can be simply divided into the following steps + +* Save values of some global variables and PD schedule. Those values will be changed during `Flashback`. + +* Pre-checks. After all checks are passed, TiDB will disable GC and closed PD schedule for the cluster. The specific checks are as follows: + * The FlashbackTS is after `tikv_gc_safe_point`. + * The FlashbackTS is before the minimal store resolved TS. + * No related DDL history in flashback time range. + * No running related DDL jobs. + +* TiDB get flashback key ranges and splits them into separate regions to avoid locking unrelated key ranges. Then TiDB send `PrepareFlashbackToVersion` RPC requests to lock regions in TiKV. Once locked, no more read, write and scheduling operations are allowed for those regions. + +* After locked all relevant key ranges, the DDL Owner will update schema version and synchronize it to other TiDBs. When other TiDB applies the `SchemaDiff` of type `Flashback To Timestamp`, it will disconnect all relevant links. + +* Send `FlashbackToVersion` RPC requests to all relevant key ranges with same `commit_ts`. Each region handles its own flashback progress independently. + * Read the old MVCC data and write it again with the given `commit_ts` to pretend it's a new transaction commit. + * Release the Raft proposing lock and resume the lease read. + +* TiDB checks whether all the requests returned successfully, and retries those that failed with same `commit_ts` until the whole flashback is done. + +* After `Flashback To Timestamp` is finished, TiDB will restore all changed global variables and restart PD schedule. At the same time, notify `Stats Handle` to reload statistics from TiKV. + +### New Syntax Overview + +TiDB will support 3 new syntaxes as follows. + +1. Flashback whole cluster except some system tables to the specified timestamp. + +```sql +FLASHBACK CLUSTER TO TIMESTAMP '2022-07-05 08:00:00'; +``` + +2. Flashback some databases to the specified timestamp. + +```sql +FLASHBACK DATABASE [db] TO TIMESTAMP '2022-07-05 08:00:00'; +``` + +3. Flashback some tables to the specified timestamp. + +```sql +FLASHBACK TABLE [table1], [table2] TO TIMESTAMP '2022-08-10 08:00:00'; +``` + +### Limitations and future Work + +1. DDL history exists for the flashback time period is not currently supported, the error message is shown below: + +```sql +mysql> ALTER TABLE t ADD INDEX i(a); +Query OK, 0 rows affected (2.99 sec) + +mysql> FLASHBACK CLUSTER TO TIMESTAMP '2022-10-10 11:53:30'; +ERROR 1105 (HY000): Had ddl history during [2022-10-10 11:53:30 +0800 CST, now), can't do flashback +``` + +2. Compare with the other DDL jobs, `Flashback To Timestamp` job cannot be rollbacked after some regions failure and also needs to resend rpc to all regions when ddl owner crashed. In the future, we will improve those two issues with a new TiKV interface and new distributed processing ddl framework. + +### Alternative Solutions + +1. Read historical data via `As of timestamp` clause and write back with the lastest timestamp. But it's much slower than `Flashback To Timestamp`, the data needs to be read to TiDB first then written back to TiKV. + +2. Use `Reset To Version` interface to delete all historical version. After this operation, the user can't find the deleted version any more and this interface is incompatible with snapshot read. diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index c7957510b1297..787a7ed8429f2 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -150,6 +150,7 @@ go_library( "//telemetry", "//tidb-binlog/node", "//types", + "//types/parser_driver", "//util", "//util/admin", "//util/bitmap", diff --git a/executor/adapter.go b/executor/adapter.go index 45ab87e2cd409..b638b28690c8b 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -611,6 +611,17 @@ func (a *ExecStmt) handleForeignKeyTrigger(ctx context.Context, e Executor, dept return nil } +// handleForeignKeyCascade uses to execute foreign key cascade behaviour, the progress is: +// 1. Build delete/update executor for foreign key on delete/update behaviour. +// a. Construct delete/update AST. We used to try generated SQL string first and then parse the SQL to get AST, +// but we need convert Datum to string, there may be some risks here, since assert_eq(datum_a, parse(datum_a.toString())) may be broken. +// so we chose to construct AST directly. +// b. Build plan by the delete/update AST. +// c. Build executor by the delete/update plan. +// 2. Execute the delete/update executor. +// 3. Close the executor. +// 4. `StmtCommit` to commit the kv change to transaction mem-buffer. +// 5. If the foreign key cascade behaviour has more fk value need to be cascaded, go to step 1. func (a *ExecStmt) handleForeignKeyCascade(ctx context.Context, fkc *FKCascadeExec, depth int) error { if len(fkc.fkValues) == 0 && len(fkc.fkUpdatedValuesMap) == 0 { return nil @@ -829,7 +840,7 @@ func (a *ExecStmt) runPessimisticSelectForUpdate(ctx context.Context, e Executor }() var rows []chunk.Row var err error - req := newFirstChunk(e) + req := tryNewCacheChunk(e) for { err = a.next(ctx, e, req) if err != nil { @@ -876,7 +887,7 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, e Executor) (sqlex } } - err = a.next(ctx, e, newFirstChunk(e)) + err = a.next(ctx, e, tryNewCacheChunk(e)) if err != nil { return nil, err } diff --git a/executor/admin.go b/executor/admin.go index ba219b70b6db3..6e549e246da42 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -111,7 +111,7 @@ func (e *CheckIndexRangeExec) Open(ctx context.Context) error { FieldType: *colTypeForHandle, }) - e.srcChunk = newFirstChunk(e) + e.srcChunk = tryNewCacheChunk(e) dagPB, err := e.buildDAGPB() if err != nil { return err diff --git a/executor/aggregate.go b/executor/aggregate.go index b407ce80a5353..771d928c9bbad 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -323,13 +323,14 @@ func (e *HashAggExec) initForUnparallelExec() { failpoint.Inject("ConsumeRandomPanic", nil) e.memTracker.Consume(hack.DefBucketMemoryUsageForMapStrToSlice*(1< 0 fields := retTypes(e.children[0]) - chk := newFirstChunk(e.children[0]) + chk := tryNewCacheChunk(e.children[0]) columns := e.children[0].Schema().Columns if len(columns) != len(fields) { logutil.BgLogger().Error("schema columns and fields mismatch", @@ -190,7 +190,7 @@ func (e *DeleteExec) deleteMultiTablesByChunk(ctx context.Context) error { colPosInfos := e.tblColPosInfos tblRowMap := make(tableRowMapType) fields := retTypes(e.children[0]) - chk := newFirstChunk(e.children[0]) + chk := tryNewCacheChunk(e.children[0]) memUsageOfChk := int64(0) for { e.memTracker.Consume(-memUsageOfChk) diff --git a/executor/distsql.go b/executor/distsql.go index 6121d5fcaa4cd..182831bc90021 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -866,7 +866,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes } }() retTps := w.idxLookup.getRetTpsByHandle() - chk := chunk.NewChunkWithCapacity(retTps, w.idxLookup.maxChunkSize) + chk := w.idxLookup.ctx.GetSessionVars().GetNewChunk(retTps, w.idxLookup.maxChunkSize) idxID := w.idxLookup.getIndexPlanRootID() if w.idxLookup.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil { if idxID != w.idxLookup.id && w.idxLookup.stats != nil { @@ -1161,7 +1161,7 @@ func (e *IndexLookUpRunTimeStats) Tp() int { } func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, tableReader Executor) error { - chk := newFirstChunk(tableReader) + chk := tryNewCacheChunk(tableReader) tblInfo := w.idxLookup.table.Meta() vals := make([]types.Datum, 0, len(w.idxTblCols)) @@ -1317,7 +1317,7 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er handleCnt := len(task.handles) task.rows = make([]chunk.Row, 0, handleCnt) for { - chk := newFirstChunk(tableReader) + chk := tryNewCacheChunk(tableReader) err = Next(ctx, tableReader, chk) if err != nil { logutil.Logger(ctx).Error("table reader fetch next chunk failed", zap.Error(err)) diff --git a/executor/executor.go b/executor/executor.go index 21523159e7a9a..ec662c00d8a63 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -229,6 +229,12 @@ func newFirstChunk(e Executor) *chunk.Chunk { return chunk.New(base.retFieldTypes, base.initCap, base.maxChunkSize) } +func tryNewCacheChunk(e Executor) *chunk.Chunk { + base := e.base() + s := base.ctx.GetSessionVars() + return s.GetNewChunkWithCapacity(base.retFieldTypes, base.initCap, base.maxChunkSize) +} + // newList creates a new List to buffer current executor's result. func newList(e Executor) *chunk.List { base := e.base() @@ -1387,7 +1393,7 @@ func (e *LimitExec) Open(ctx context.Context) error { if err := e.baseExecutor.Open(ctx); err != nil { return err } - e.childResult = newFirstChunk(e.children[0]) + e.childResult = tryNewCacheChunk(e.children[0]) e.cursor = 0 e.meetFirstBatch = e.begin == 0 return nil @@ -1444,8 +1450,7 @@ func init() { if err != nil { return nil, err } - chk := newFirstChunk(exec) - + chk := tryNewCacheChunk(exec) err = Next(ctx, exec, chk) if err != nil { return nil, err @@ -1520,7 +1525,7 @@ func (e *SelectionExec) Open(ctx context.Context) error { func (e *SelectionExec) open(ctx context.Context) error { e.memTracker = memory.NewTracker(e.id, -1) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) - e.childResult = newFirstChunk(e.children[0]) + e.childResult = tryNewCacheChunk(e.children[0]) e.memTracker.Consume(e.childResult.MemoryUsage()) e.batched = expression.Vectorizable(e.filters) if e.batched { @@ -1700,7 +1705,7 @@ func (e *MaxOneRowExec) Next(ctx context.Context, req *chunk.Chunk) error { return ErrSubqueryMoreThan1Row } - childChunk := newFirstChunk(e.children[0]) + childChunk := tryNewCacheChunk(e.children[0]) err = Next(ctx, e.children[0], childChunk) if err != nil { return err @@ -2134,6 +2139,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { errCount, warnCount := vars.StmtCtx.NumErrorWarnings() vars.SysErrorCount = errCount vars.SysWarningCount = warnCount + vars.ExchangeChunkStatus() vars.StmtCtx = sc vars.PrevFoundInPlanCache = vars.FoundInPlanCache vars.FoundInPlanCache = false @@ -2172,6 +2178,10 @@ func ResetUpdateStmtCtx(sc *stmtctx.StatementContext, stmt *ast.UpdateStmt, vars // expression using rows from a chunk, and then fill this value into the chunk func FillVirtualColumnValue(virtualRetTypes []*types.FieldType, virtualColumnIndex []int, schema *expression.Schema, columns []*model.ColumnInfo, sctx sessionctx.Context, req *chunk.Chunk) error { + if len(virtualColumnIndex) == 0 { + return nil + } + virCols := chunk.NewChunkWithCapacity(virtualRetTypes, req.Capacity()) iter := chunk.NewIterator4Chunk(req) for i, idx := range virtualColumnIndex { diff --git a/executor/executor_test.go b/executor/executor_test.go index fdd917bd73756..ffe2ade09a979 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3972,12 +3972,13 @@ func TestApplyCache(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int);") tk.MustExec("insert into t values (1),(1),(1),(1),(1),(1),(1),(1),(1);") tk.MustExec("analyze table t;") result := tk.MustQuery("explain analyze SELECT count(1) FROM (SELECT (SELECT min(a) FROM t as t2 WHERE t2.a > t1.a) AS a from t as t1) t;") - require.Equal(t, "└─Apply_39", result.Rows()[1][0]) + require.Contains(t, result.Rows()[1][0], "Apply") var ( ind int flag bool @@ -3997,7 +3998,7 @@ func TestApplyCache(t *testing.T) { tk.MustExec("insert into t values (1),(2),(3),(4),(5),(6),(7),(8),(9);") tk.MustExec("analyze table t;") result = tk.MustQuery("explain analyze SELECT count(1) FROM (SELECT (SELECT min(a) FROM t as t2 WHERE t2.a > t1.a) AS a from t as t1) t;") - require.Equal(t, "└─Apply_39", result.Rows()[1][0]) + require.Contains(t, result.Rows()[1][0], "Apply") flag = false value = (result.Rows()[1][5]).(string) for ind = 0; ind < len(value)-5; ind++ { @@ -6182,6 +6183,7 @@ func TestCompileOutOfMemoryQuota(t *testing.T) { require.Contains(t, err.Error(), "Out Of Memory Quota!") } + func TestAutoIncrementInsertMinMax(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -6228,6 +6230,32 @@ func TestAutoIncrementInsertMinMax(t *testing.T) { require.Error(t, err) } +func TestSignalCheckpointForSort(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/SignalCheckpointForSort", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/SignalCheckpointForSort")) + }() + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/util/chunk/SignalCheckpointForSort", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/util/chunk/SignalCheckpointForSort")) + }() + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + defer tk.MustExec("set global tidb_mem_oom_action = DEFAULT") + tk.MustExec("set global tidb_mem_oom_action='CANCEL'") + tk.MustExec("set tidb_mem_quota_query = 100000000") + tk.MustExec("use test") + tk.MustExec("create table t(a int)") + for i := 0; i < 20; i++ { + tk.MustExec(fmt.Sprintf("insert into t values(%d)", i)) + } + tk.Session().GetSessionVars().ConnectionID = 123456 + + err := tk.QueryToErr("select * from t order by a") + require.Contains(t, err.Error(), "Out Of Memory Quota!") +} + func TestSessionRootTrackerDetach(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/executor/explain.go b/executor/explain.go index 288fcc6b16b88..3f9f1eec6704e 100644 --- a/executor/explain.go +++ b/executor/explain.go @@ -120,7 +120,7 @@ func (e *ExplainExec) executeAnalyzeExec(ctx context.Context) (err error) { }).run() } e.executed = true - chk := newFirstChunk(e.analyzeExec) + chk := tryNewCacheChunk(e.analyzeExec) for { err = Next(ctx, e.analyzeExec, chk) if err != nil || chk.NumRows() == 0 { diff --git a/executor/fktest/BUILD.bazel b/executor/fktest/BUILD.bazel index 40237466542e5..dbdae1843edaf 100644 --- a/executor/fktest/BUILD.bazel +++ b/executor/fktest/BUILD.bazel @@ -13,10 +13,14 @@ go_test( "//executor", "//kv", "//meta/autoid", + "//parser/ast", + "//parser/format", "//parser/model", + "//parser/mysql", "//planner/core", "//testkit", "//types", + "//util/sqlexec", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//tikv", "@org_uber_go_goleak//:goleak", diff --git a/executor/fktest/foreign_key_test.go b/executor/fktest/foreign_key_test.go index 64908a77b271a..c704494ffb7f5 100644 --- a/executor/fktest/foreign_key_test.go +++ b/executor/fktest/foreign_key_test.go @@ -16,6 +16,7 @@ package fk_test import ( "bytes" + "context" "fmt" "strconv" "strings" @@ -25,10 +26,14 @@ import ( "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/format" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/sqlexec" "github.com/stretchr/testify/require" ) @@ -1256,44 +1261,57 @@ func TestForeignKeyOnDeleteCascade2(t *testing.T) { tk.MustQuery("select count(*) from t2").Check(testkit.Rows("0")) } -func TestForeignKeyGenerateCascadeSQL(t *testing.T) { - fk := &model.FKInfo{ - Cols: []model.CIStr{model.NewCIStr("c0"), model.NewCIStr("c1")}, - } +func TestForeignKeyGenerateCascadeAST(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") fkValues := [][]types.Datum{ {types.NewDatum(1), types.NewDatum("a")}, {types.NewDatum(2), types.NewDatum("b")}, } - sql, err := executor.GenCascadeDeleteSQL(model.NewCIStr("test"), model.NewCIStr("t"), model.NewCIStr(""), fk, fkValues) - require.NoError(t, err) - require.Equal(t, "DELETE FROM `test`.`t` WHERE (`c0`, `c1`) IN ((1,'a'), (2,'b'))", sql) - - sql, err = executor.GenCascadeDeleteSQL(model.NewCIStr("test"), model.NewCIStr("t"), model.NewCIStr("idx"), fk, fkValues) - require.NoError(t, err) - require.Equal(t, "DELETE FROM `test`.`t` USE INDEX(`idx`) WHERE (`c0`, `c1`) IN ((1,'a'), (2,'b'))", sql) - - sql, err = executor.GenCascadeSetNullSQL(model.NewCIStr("test"), model.NewCIStr("t"), model.NewCIStr(""), fk, fkValues) - require.NoError(t, err) - require.Equal(t, "UPDATE `test`.`t` SET `c0` = NULL, `c1` = NULL WHERE (`c0`, `c1`) IN ((1,'a'), (2,'b'))", sql) - - sql, err = executor.GenCascadeSetNullSQL(model.NewCIStr("test"), model.NewCIStr("t"), model.NewCIStr("idx"), fk, fkValues) - require.NoError(t, err) - require.Equal(t, "UPDATE `test`.`t` USE INDEX(`idx`) SET `c0` = NULL, `c1` = NULL WHERE (`c0`, `c1`) IN ((1,'a'), (2,'b'))", sql) - + cols := []*model.ColumnInfo{ + {ID: 1, Name: model.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeLonglong)}, + {ID: 2, Name: model.NewCIStr("name"), FieldType: *types.NewFieldType(mysql.TypeVarchar)}, + } + restoreFn := func(stmt ast.StmtNode) string { + var sb strings.Builder + fctx := format.NewRestoreCtx(format.DefaultRestoreFlags, &sb) + err := stmt.Restore(fctx) + require.NoError(t, err) + return sb.String() + } + checkStmtFn := func(stmt ast.StmtNode, sql string) { + exec, ok := tk.Session().(sqlexec.RestrictedSQLExecutor) + require.True(t, ok) + expectedStmt, err := exec.ParseWithParams(context.Background(), sql) + require.NoError(t, err) + require.Equal(t, restoreFn(expectedStmt), restoreFn(stmt)) + } + var stmt ast.StmtNode + stmt = executor.GenCascadeDeleteAST(model.NewCIStr("test"), model.NewCIStr("t2"), model.NewCIStr(""), cols, fkValues) + checkStmtFn(stmt, "delete from test.t2 where (a,name) in ((1,'a'), (2,'b'))") + stmt = executor.GenCascadeDeleteAST(model.NewCIStr("test"), model.NewCIStr("t2"), model.NewCIStr("idx"), cols, fkValues) + checkStmtFn(stmt, "delete from test.t2 use index(idx) where (a,name) in ((1,'a'), (2,'b'))") + stmt = executor.GenCascadeSetNullAST(model.NewCIStr("test"), model.NewCIStr("t2"), model.NewCIStr(""), cols, fkValues) + checkStmtFn(stmt, "update test.t2 set a = null, name = null where (a,name) in ((1,'a'), (2,'b'))") + stmt = executor.GenCascadeSetNullAST(model.NewCIStr("test"), model.NewCIStr("t2"), model.NewCIStr("idx"), cols, fkValues) + checkStmtFn(stmt, "update test.t2 use index(idx) set a = null, name = null where (a,name) in ((1,'a'), (2,'b'))") newValue1 := []types.Datum{types.NewDatum(10), types.NewDatum("aa")} couple := &executor.UpdatedValuesCouple{ NewValues: newValue1, OldValuesList: fkValues, } - sql, err = executor.GenCascadeUpdateSQL(model.NewCIStr("test"), model.NewCIStr("t"), model.NewCIStr(""), fk, couple) - require.NoError(t, err) - require.Equal(t, "UPDATE `test`.`t` SET `c0` = 10, `c1` = 'aa' WHERE (`c0`, `c1`) IN ((1,'a'), (2,'b'))", sql) - - newValue2 := []types.Datum{types.NewDatum(nil), types.NewDatum(nil)} - couple.NewValues = newValue2 - sql, err = executor.GenCascadeUpdateSQL(model.NewCIStr("test"), model.NewCIStr("t"), model.NewCIStr("idx"), fk, couple) - require.NoError(t, err) - require.Equal(t, "UPDATE `test`.`t` USE INDEX(`idx`) SET `c0` = NULL, `c1` = NULL WHERE (`c0`, `c1`) IN ((1,'a'), (2,'b'))", sql) + stmt = executor.GenCascadeUpdateAST(model.NewCIStr("test"), model.NewCIStr("t2"), model.NewCIStr(""), cols, couple) + checkStmtFn(stmt, "update test.t2 set a = 10, name = 'aa' where (a,name) in ((1,'a'), (2,'b'))") + stmt = executor.GenCascadeUpdateAST(model.NewCIStr("test"), model.NewCIStr("t2"), model.NewCIStr("idx"), cols, couple) + checkStmtFn(stmt, "update test.t2 use index(idx) set a = 10, name = 'aa' where (a,name) in ((1,'a'), (2,'b'))") + // Test for 1 fk column. + fkValues = [][]types.Datum{{types.NewDatum(1)}, {types.NewDatum(2)}} + cols = []*model.ColumnInfo{{ID: 1, Name: model.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeLonglong)}} + stmt = executor.GenCascadeDeleteAST(model.NewCIStr("test"), model.NewCIStr("t2"), model.NewCIStr(""), cols, fkValues) + checkStmtFn(stmt, "delete from test.t2 where a in (1,2)") + stmt = executor.GenCascadeDeleteAST(model.NewCIStr("test"), model.NewCIStr("t2"), model.NewCIStr("idx"), cols, fkValues) + checkStmtFn(stmt, "delete from test.t2 use index(idx) where a in (1,2)") } func TestForeignKeyOnDeleteSetNull(t *testing.T) { diff --git a/executor/foreign_key.go b/executor/foreign_key.go index 32c27961023d8..62dbb400537e2 100644 --- a/executor/foreign_key.go +++ b/executor/foreign_key.go @@ -15,12 +15,12 @@ package executor import ( - "bytes" "context" "sync/atomic" "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" @@ -29,9 +29,9 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" + driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/set" - "github.com/pingcap/tidb/util/sqlexec" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" ) @@ -71,6 +71,8 @@ type FKCascadeExec struct { referredFK *model.ReferredFKInfo childTable *model.TableInfo fk *model.FKInfo + fkCols []*model.ColumnInfo + fkIdx *model.IndexInfo // On delete statement, fkValues stores the delete foreign key values. // On update statement and the foreign key cascade is `SET NULL`, fkValues stores the old foreign key values. fkValues [][]types.Datum @@ -587,6 +589,8 @@ func (b *executorBuilder) buildFKCascadeExec(tbl table.Table, fkCascade *planner referredFK: fkCascade.ReferredFK, childTable: fkCascade.ChildTable.Meta(), fk: fkCascade.FK, + fkCols: fkCascade.FKCols, + fkIdx: fkCascade.FKIdx, fkUpdatedValuesMap: make(map[string]*UpdatedValuesCouple), }, nil } @@ -637,6 +641,8 @@ func (fkc *FKCascadeExec) buildExecutor(ctx context.Context) (Executor, error) { return e, fkc.b.err } +// maxHandleFKValueInOneCascade uses to limit the max handle fk value in one cascade executor, +// this is to avoid performance issue, see: https://github.com/pingcap/tidb/issues/38631 var maxHandleFKValueInOneCascade = 1024 func (fkc *FKCascadeExec) buildFKCascadePlan(ctx context.Context) (plannercore.Plan, error) { @@ -644,51 +650,36 @@ func (fkc *FKCascadeExec) buildFKCascadePlan(ctx context.Context) (plannercore.P return nil, nil } var indexName model.CIStr - indexForFK := model.FindIndexByColumns(fkc.childTable, fkc.fk.Cols...) - if indexForFK != nil { - indexName = indexForFK.Name + if fkc.fkIdx != nil { + indexName = fkc.fkIdx.Name } - var sqlStr string - var err error + var stmtNode ast.StmtNode switch fkc.tp { case plannercore.FKCascadeOnDelete: fkValues := fkc.fetchOnDeleteOrUpdateFKValues() switch model.ReferOptionType(fkc.fk.OnDelete) { case model.ReferOptionCascade: - sqlStr, err = GenCascadeDeleteSQL(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fk, fkValues) + stmtNode = GenCascadeDeleteAST(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fkCols, fkValues) case model.ReferOptionSetNull: - sqlStr, err = GenCascadeSetNullSQL(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fk, fkValues) + stmtNode = GenCascadeSetNullAST(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fkCols, fkValues) } case plannercore.FKCascadeOnUpdate: switch model.ReferOptionType(fkc.fk.OnUpdate) { case model.ReferOptionCascade: couple := fkc.fetchUpdatedValuesCouple() if couple != nil && len(couple.NewValues) != 0 { - sqlStr, err = GenCascadeUpdateSQL(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fk, couple) + stmtNode = GenCascadeUpdateAST(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fkCols, couple) } case model.ReferOptionSetNull: fkValues := fkc.fetchOnDeleteOrUpdateFKValues() - sqlStr, err = GenCascadeSetNullSQL(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fk, fkValues) + stmtNode = GenCascadeSetNullAST(fkc.referredFK.ChildSchema, fkc.childTable.Name, indexName, fkc.fkCols, fkValues) } } - if err != nil { - return nil, err + if stmtNode == nil { + return nil, errors.Errorf("generate foreign key cascade ast failed, %v", fkc.tp) } - if sqlStr == "" { - return nil, errors.Errorf("generate foreign key cascade sql failed, %v", fkc.tp) - } - sctx := fkc.b.ctx - exec, ok := sctx.(sqlexec.RestrictedSQLExecutor) - if !ok { - return nil, nil - } - stmtNode, err := exec.ParseWithParams(ctx, sqlStr) - if err != nil { - return nil, err - } - ret := &plannercore.PreprocessorReturn{} - err = plannercore.Preprocess(ctx, sctx, stmtNode, plannercore.WithPreprocessorReturn(ret), plannercore.InitTxnContextProvider) + err := plannercore.Preprocess(ctx, sctx, stmtNode) if err != nil { return nil, err } @@ -727,120 +718,96 @@ func (fkc *FKCascadeExec) fetchUpdatedValuesCouple() *UpdatedValuesCouple { return nil } -// GenCascadeDeleteSQL uses to generate cascade delete SQL, export for test. -func GenCascadeDeleteSQL(schema, table, idx model.CIStr, fk *model.FKInfo, fkValues [][]types.Datum) (string, error) { - buf := bytes.NewBuffer(make([]byte, 0, 48+8*len(fkValues))) - buf.WriteString("DELETE FROM `") - buf.WriteString(schema.L) - buf.WriteString("`.`") - buf.WriteString(table.L) - buf.WriteString("`") - if idx.L != "" { - // Add use index to make sure the optimizer will use index instead of full table scan. - buf.WriteString(" USE INDEX(`") - buf.WriteString(idx.L) - buf.WriteString("`)") - } - err := genCascadeSQLWhereCondition(buf, fk, fkValues) - if err != nil { - return "", err +// GenCascadeDeleteAST uses to generate cascade delete ast, export for test. +func GenCascadeDeleteAST(schema, table, idx model.CIStr, cols []*model.ColumnInfo, fkValues [][]types.Datum) *ast.DeleteStmt { + deleteStmt := &ast.DeleteStmt{ + TableRefs: genTableRefsAST(schema, table, idx), + Where: genWhereConditionAst(cols, fkValues), } - return buf.String(), nil + return deleteStmt } -// GenCascadeSetNullSQL uses to generate foreign key `SET NULL` SQL, export for test. -func GenCascadeSetNullSQL(schema, table, idx model.CIStr, fk *model.FKInfo, fkValues [][]types.Datum) (string, error) { - newValues := make([]types.Datum, len(fk.Cols)) - for i := range fk.Cols { +// GenCascadeSetNullAST uses to generate foreign key `SET NULL` ast, export for test. +func GenCascadeSetNullAST(schema, table, idx model.CIStr, cols []*model.ColumnInfo, fkValues [][]types.Datum) *ast.UpdateStmt { + newValues := make([]types.Datum, len(cols)) + for i := range cols { newValues[i] = types.NewDatum(nil) } couple := &UpdatedValuesCouple{ NewValues: newValues, OldValuesList: fkValues, } - return GenCascadeUpdateSQL(schema, table, idx, fk, couple) + return GenCascadeUpdateAST(schema, table, idx, cols, couple) } -// GenCascadeUpdateSQL uses to generate cascade update SQL, export for test. -func GenCascadeUpdateSQL(schema, table, idx model.CIStr, fk *model.FKInfo, couple *UpdatedValuesCouple) (string, error) { - buf := bytes.NewBuffer(nil) - buf.WriteString("UPDATE `") - buf.WriteString(schema.L) - buf.WriteString("`.`") - buf.WriteString(table.L) - buf.WriteString("`") - if idx.L != "" { - // Add use index to make sure the optimizer will use index instead of full table scan. - buf.WriteString(" USE INDEX(`") - buf.WriteString(idx.L) - buf.WriteString("`)") - } - buf.WriteString(" SET ") - for i, col := range fk.Cols { - if i > 0 { - buf.WriteString(", ") - } - buf.WriteString("`" + col.L) - buf.WriteString("` = ") - val, err := genFKValueString(couple.NewValues[i]) - if err != nil { - return "", err +// GenCascadeUpdateAST uses to generate cascade update ast, export for test. +func GenCascadeUpdateAST(schema, table, idx model.CIStr, cols []*model.ColumnInfo, couple *UpdatedValuesCouple) *ast.UpdateStmt { + list := make([]*ast.Assignment, 0, len(cols)) + for i, col := range cols { + v := &driver.ValueExpr{Datum: couple.NewValues[i]} + v.Type = col.FieldType + assignment := &ast.Assignment{ + Column: &ast.ColumnName{Name: col.Name}, + Expr: v, } - buf.WriteString(val) + list = append(list, assignment) } - err := genCascadeSQLWhereCondition(buf, fk, couple.OldValuesList) - if err != nil { - return "", err + updateStmt := &ast.UpdateStmt{ + TableRefs: genTableRefsAST(schema, table, idx), + Where: genWhereConditionAst(cols, couple.OldValuesList), + List: list, } - return buf.String(), nil + return updateStmt } -func genCascadeSQLWhereCondition(buf *bytes.Buffer, fk *model.FKInfo, fkValues [][]types.Datum) error { - buf.WriteString(" WHERE (") - for i, col := range fk.Cols { - if i > 0 { - buf.WriteString(", ") - } - buf.WriteString("`" + col.L + "`") +func genTableRefsAST(schema, table, idx model.CIStr) *ast.TableRefsClause { + tn := &ast.TableName{Schema: schema, Name: table} + if idx.L != "" { + tn.IndexHints = []*ast.IndexHint{{ + IndexNames: []model.CIStr{idx}, + HintType: ast.HintUse, + HintScope: ast.HintForScan, + }} } - buf.WriteString(") IN (") - for i, vs := range fkValues { - if i > 0 { - buf.WriteString(", (") - } else { - buf.WriteString("(") - } - for i := range vs { - val, err := genFKValueString(vs[i]) - if err != nil { - return err - } - if i > 0 { - buf.WriteString(",") - } - buf.WriteString(val) - } - buf.WriteString(")") + join := &ast.Join{Left: &ast.TableSource{Source: tn}} + return &ast.TableRefsClause{TableRefs: join} +} + +func genWhereConditionAst(cols []*model.ColumnInfo, fkValues [][]types.Datum) ast.ExprNode { + if len(cols) > 1 { + return genWhereConditionAstForMultiColumn(cols, fkValues) + } + valueList := make([]ast.ExprNode, 0, len(fkValues)) + for _, fkVals := range fkValues { + v := &driver.ValueExpr{Datum: fkVals[0]} + v.Type = cols[0].FieldType + valueList = append(valueList, v) + } + return &ast.PatternInExpr{ + Expr: &ast.ColumnNameExpr{Name: &ast.ColumnName{Name: cols[0].Name}}, + List: valueList, } - buf.WriteString(")") - return nil } -func genFKValueString(v types.Datum) (string, error) { - switch v.Kind() { - case types.KindNull: - return "NULL", nil - case types.KindMysqlBit: - return v.GetBinaryLiteral().ToBitLiteralString(true), nil +func genWhereConditionAstForMultiColumn(cols []*model.ColumnInfo, fkValues [][]types.Datum) ast.ExprNode { + colValues := make([]ast.ExprNode, len(cols)) + for i := range cols { + col := &ast.ColumnNameExpr{Name: &ast.ColumnName{Name: cols[i].Name}} + colValues[i] = col } - val, err := v.ToString() - if err != nil { - return "", err + valueList := make([]ast.ExprNode, 0, len(fkValues)) + for _, fkVals := range fkValues { + values := make([]ast.ExprNode, len(fkVals)) + for i, v := range fkVals { + val := &driver.ValueExpr{Datum: v} + val.Type = cols[i].FieldType + values[i] = val + } + row := &ast.RowExpr{Values: values} + valueList = append(valueList, row) } - switch v.Kind() { - case types.KindInt64, types.KindUint64, types.KindFloat32, types.KindFloat64, types.KindMysqlDecimal: - return val, nil - default: - return "'" + val + "'", nil + return &ast.PatternInExpr{ + Expr: &ast.RowExpr{Values: colValues}, + List: valueList, } } diff --git a/executor/index_lookup_hash_join.go b/executor/index_lookup_hash_join.go index bdda0ab3535bc..9601dffc77900 100644 --- a/executor/index_lookup_hash_join.go +++ b/executor/index_lookup_hash_join.go @@ -418,7 +418,7 @@ func (e *IndexNestedLoopHashJoin) newInnerWorker(taskCh chan *indexHashJoinTask, innerCtx: e.innerCtx, outerCtx: e.outerCtx, ctx: e.ctx, - executorChk: chunk.NewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize), + executorChk: e.ctx.GetSessionVars().GetNewChunk(e.innerCtx.rowTypes, e.maxChunkSize), indexRanges: copiedRanges, keyOff2IdxOff: e.keyOff2IdxOff, stats: innerStats, diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 62ee5cea0bdd2..92f195985a191 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -226,7 +226,7 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork outerCtx: e.outerCtx, taskCh: taskCh, ctx: e.ctx, - executorChk: chunk.NewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize), + executorChk: e.ctx.GetSessionVars().GetNewChunk(e.innerCtx.rowTypes, e.maxChunkSize), indexRanges: copiedRanges, keyOff2IdxOff: e.keyOff2IdxOff, stats: innerStats, @@ -431,7 +431,7 @@ func (ow *outerWorker) buildTask(ctx context.Context) (*lookUpJoinTask, error) { } maxChunkSize := ow.ctx.GetSessionVars().MaxChunkSize for requiredRows > task.outerResult.Len() { - chk := chunk.NewChunkWithCapacity(ow.outerCtx.rowTypes, maxChunkSize) + chk := ow.ctx.GetSessionVars().GetNewChunk(ow.outerCtx.rowTypes, maxChunkSize) chk = chk.SetRequiredRows(requiredRows, maxChunkSize) err := Next(ctx, ow.executor, chk) if err != nil { @@ -462,7 +462,7 @@ func (ow *outerWorker) buildTask(ctx context.Context) (*lookUpJoinTask, error) { } task.encodedLookUpKeys = make([]*chunk.Chunk, task.outerResult.NumChunks()) for i := range task.encodedLookUpKeys { - task.encodedLookUpKeys[i] = chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeBlob)}, task.outerResult.GetChunk(i).NumRows()) + task.encodedLookUpKeys[i] = ow.ctx.GetSessionVars().GetNewChunk([]*types.FieldType{types.NewFieldType(mysql.TypeBlob)}, task.outerResult.GetChunk(i).NumRows()) } return task, nil } @@ -714,7 +714,7 @@ func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTa break } innerResult.Add(iw.executorChk) - iw.executorChk = newFirstChunk(innerExec) + iw.executorChk = tryNewCacheChunk(innerExec) } task.innerResult = innerResult return nil diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index d37f3f8a1f743..2f251761b71c2 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -352,13 +352,16 @@ func TestIssue23722(t *testing.T) { tk.MustExec("insert into t values (20301,'Charlie',x'7a');") tk.MustQuery("select * from t;").Check(testkit.Rows("20301 Charlie z")) tk.MustQuery("select * from t where c in (select c from t where t.c >= 'a');").Check(testkit.Rows("20301 Charlie z")) + tk.MustQuery("select @@last_sql_use_alloc").Check(testkit.Rows("1")) // Test lookup content exceeds primary key prefix. tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int, b char(10), c varchar(255), primary key (c(5)) clustered);") tk.MustExec("insert into t values (20301,'Charlie','aaaaaaa');") + tk.MustQuery("select @@last_sql_use_alloc").Check(testkit.Rows("1")) tk.MustQuery("select * from t;").Check(testkit.Rows("20301 Charlie aaaaaaa")) tk.MustQuery("select * from t where c in (select c from t where t.c >= 'a');").Check(testkit.Rows("20301 Charlie aaaaaaa")) + tk.MustQuery("select @@last_sql_use_alloc").Check(testkit.Rows("1")) // Test the original case. tk.MustExec("drop table if exists t;") @@ -452,7 +455,9 @@ func TestIssue27893(t *testing.T) { tk.MustExec("insert into t1 values('x')") tk.MustExec("insert into t2 values(1)") tk.MustQuery("select /*+ inl_join(t2) */ count(*) from t1 join t2 on t1.a = t2.a").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_sql_use_alloc").Check(testkit.Rows("1")) tk.MustQuery("select /*+ inl_hash_join(t2) */ count(*) from t1 join t2 on t1.a = t2.a").Check(testkit.Rows("1")) + tk.MustQuery("select @@last_sql_use_alloc").Check(testkit.Rows("1")) } func TestPartitionTableIndexJoinAndIndexReader(t *testing.T) { diff --git a/executor/index_lookup_merge_join.go b/executor/index_lookup_merge_join.go index 369c18716dbc3..1ba2c2940c3fd 100644 --- a/executor/index_lookup_merge_join.go +++ b/executor/index_lookup_merge_join.go @@ -357,7 +357,7 @@ func (omw *outerMergeWorker) buildTask(ctx context.Context) (*lookUpMergeJoinTas requiredRows = omw.maxBatchSize } for requiredRows > 0 { - execChk := newFirstChunk(omw.executor) + execChk := tryNewCacheChunk(omw.executor) err := Next(ctx, omw.executor, execChk) if err != nil { return task, err @@ -706,7 +706,7 @@ func (imw *innerMergeWorker) dedupDatumLookUpKeys(lookUpContents []*indexJoinLoo // fetchNextInnerResult collects a chunk of inner results from inner child executor. func (imw *innerMergeWorker) fetchNextInnerResult(ctx context.Context, task *lookUpMergeJoinTask) (beginRow chunk.Row, err error) { - task.innerResult = chunk.NewChunkWithCapacity(retTypes(imw.innerExec), imw.ctx.GetSessionVars().MaxChunkSize) + task.innerResult = imw.ctx.GetSessionVars().GetNewChunk(retTypes(imw.innerExec), imw.ctx.GetSessionVars().MaxChunkSize) err = Next(ctx, imw.innerExec, task.innerResult) task.innerIter = chunk.NewIterator4Chunk(task.innerResult) beginRow = task.innerIter.Begin() diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index bc29199a2c2b7..82c6ab2f50817 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -508,7 +508,7 @@ func (w *partialTableWorker) syncErr(resultCh chan<- *lookupTableTask, err error func (w *partialTableWorker) fetchHandles(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *lookupTableTask, resultCh chan<- *lookupTableTask, finished <-chan struct{}, handleCols plannercore.HandleCols) (count int64, err error) { - chk := chunk.NewChunkWithCapacity(retTypes(w.tableReader), w.maxChunkSize) + chk := w.sc.GetSessionVars().GetNewChunk(retTypes(w.tableReader), w.maxChunkSize) var basic *execdetails.BasicRuntimeStats if be := w.tableReader.base(); be != nil && be.runtimeStats != nil { basic = be.runtimeStats @@ -817,7 +817,7 @@ func (w *partialIndexWorker) fetchHandles( resultCh chan<- *lookupTableTask, finished <-chan struct{}, handleCols plannercore.HandleCols) (count int64, err error) { - chk := chunk.NewChunkWithCapacity(handleCols.GetFieldsTypes(), w.maxChunkSize) + chk := w.sc.GetSessionVars().GetNewChunk(handleCols.GetFieldsTypes(), w.maxChunkSize) var basicStats *execdetails.BasicRuntimeStats if w.stats != nil { if w.idxID != 0 { @@ -961,7 +961,7 @@ func (w *indexMergeTableScanWorker) executeTask(ctx context.Context, task *looku handleCnt := len(task.handles) task.rows = make([]chunk.Row, 0, handleCnt) for { - chk := newFirstChunk(tableReader) + chk := tryNewCacheChunk(tableReader) err = Next(ctx, tableReader, chk) if err != nil { logutil.Logger(ctx).Error("table reader fetch next chunk failed", zap.Error(err)) diff --git a/executor/index_merge_reader_test.go b/executor/index_merge_reader_test.go index ae53e1e6c86e2..58dfa71814f28 100644 --- a/executor/index_merge_reader_test.go +++ b/executor/index_merge_reader_test.go @@ -496,7 +496,7 @@ func TestPessimisticLockOnPartitionForIndexMerge(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") - tk.MustExec(`create table t1 (c_datetime datetime, c1 int, c2 int, primary key (c_datetime), key(c1), key(c2)) + tk.MustExec(`create table t1 (c_datetime datetime, c1 int, c2 int, primary key (c_datetime) NONCLUSTERED, key(c1), key(c2)) partition by range (to_days(c_datetime)) ( partition p0 values less than (to_days('2020-02-01')), partition p1 values less than (to_days('2020-04-01')), diff --git a/executor/insert_common.go b/executor/insert_common.go index d4e3dab1bf839..5bb7feb2441da 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -460,7 +460,7 @@ func insertRowsFromSelect(ctx context.Context, base insertCommon) error { e := base.insertCommon() selectExec := e.children[0] fields := retTypes(selectExec) - chk := newFirstChunk(selectExec) + chk := tryNewCacheChunk(selectExec) iter := chunk.NewIterator4Chunk(chk) rows := make([][]types.Datum, 0, chk.Capacity()) diff --git a/executor/join.go b/executor/join.go index c97543b4cb3e2..48d3e5d5a56f8 100644 --- a/executor/join.go +++ b/executor/join.go @@ -297,7 +297,7 @@ func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chu if e.finished.Load().(bool) { return } - chk := chunk.NewChunkWithCapacity(e.buildSideExec.base().retFieldTypes, e.ctx.GetSessionVars().MaxChunkSize) + chk := e.ctx.GetSessionVars().GetNewChunk(e.buildSideExec.base().retFieldTypes, e.ctx.GetSessionVars().MaxChunkSize) err = Next(ctx, e.buildSideExec, chk) if err != nil { e.buildFinished <- errors.Trace(err) @@ -1307,8 +1307,8 @@ func (e *NestedLoopApplyExec) Open(ctx context.Context) error { } e.cursor = 0 e.innerRows = e.innerRows[:0] - e.outerChunk = newFirstChunk(e.outerExec) - e.innerChunk = newFirstChunk(e.innerExec) + e.outerChunk = tryNewCacheChunk(e.outerExec) + e.innerChunk = tryNewCacheChunk(e.innerExec) e.innerList = chunk.NewList(retTypes(e.innerExec), e.initCap, e.maxChunkSize) e.memTracker = memory.NewTracker(e.id, -1) diff --git a/executor/join_test.go b/executor/join_test.go index 6e193a74fac30..d5285162bdff4 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -198,9 +198,9 @@ func TestJoin2(t *testing.T) { tk.MustQuery("select /*+ INL_JOIN(t, t1) */ t1.b from t1 join t on t.b=t1.b").Check(testkit.Rows("2", "3")) tk.MustQuery("select /*+ INL_HASH_JOIN(t, t1) */ t1.b from t1 join t on t.b=t1.b").Sort().Check(testkit.Rows("2", "3")) tk.MustQuery("select /*+ INL_MERGE_JOIN(t, t1) */ t1.b from t1 join t on t.b=t1.b").Check(testkit.Rows("2", "3")) - tk.MustQuery("select /*+ INL_JOIN(t1) */ * from t right outer join t1 on t.a=t1.a").Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4", " 4 5")) - tk.MustQuery("select /*+ INL_HASH_JOIN(t1) */ * from t right outer join t1 on t.a=t1.a").Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4", " 4 5")) - tk.MustQuery("select /*+ INL_MERGE_JOIN(t1) */ * from t right outer join t1 on t.a=t1.a").Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4", " 4 5")) + tk.MustQuery("select /*+ INL_JOIN(t1) */ * from t right outer join t1 on t.a=t1.a").Sort().Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4", " 4 5")) + tk.MustQuery("select /*+ INL_HASH_JOIN(t1) */ * from t right outer join t1 on t.a=t1.a").Sort().Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4", " 4 5")) + tk.MustQuery("select /*+ INL_MERGE_JOIN(t1) */ * from t right outer join t1 on t.a=t1.a").Sort().Check(testkit.Rows("1 1 1 2", "1 1 1 3", "1 1 1 4", "3 3 3 4", " 4 5")) tk.MustQuery("select /*+ INL_JOIN(t) */ avg(t.b) from t right outer join t1 on t.a=t1.a").Check(testkit.Rows("1.5000")) tk.MustQuery("select /*+ INL_HASH_JOIN(t) */ avg(t.b) from t right outer join t1 on t.a=t1.a").Check(testkit.Rows("1.5000")) tk.MustQuery("select /*+ INL_MERGE_JOIN(t) */ avg(t.b) from t right outer join t1 on t.a=t1.a").Check(testkit.Rows("1.5000")) @@ -1222,6 +1222,7 @@ func TestIndexLookupJoin(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set @@tidb_init_chunk_size=2") tk.MustExec("DROP TABLE IF EXISTS t") tk.MustExec("CREATE TABLE `t` (`a` int, pk integer auto_increment,`b` char (20),primary key (pk))") @@ -1356,7 +1357,7 @@ func TestIndexLookupJoin(t *testing.T) { tk.MustExec("analyze table s;") tk.MustQuery("desc format = 'brief' select /*+ TIDB_INLJ(s) */ count(*) from t join s use index(idx) on s.a = t.a and s.b < t.b").Check(testkit.Rows( - "HashAgg 1.00 root funcs:count(1)->Column#6", + "StreamAgg 1.00 root funcs:count(1)->Column#6", "└─IndexJoin 64.00 root inner join, inner:IndexReader, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), other cond:lt(test.s.b, test.t.b)", " ├─TableReader(Build) 64.00 root data:Selection", " │ └─Selection 64.00 cop[tikv] not(isnull(test.t.b))", @@ -1369,7 +1370,7 @@ func TestIndexLookupJoin(t *testing.T) { tk.MustQuery("select /*+ TIDB_INLJ(s) */ count(*) from t join s use index(idx) on s.a = t.a and s.b < t.b").Check(testkit.Rows("64")) tk.MustQuery("desc format = 'brief' select /*+ INL_MERGE_JOIN(s) */ count(*) from t join s use index(idx) on s.a = t.a and s.b < t.b").Check(testkit.Rows( - "HashAgg 1.00 root funcs:count(1)->Column#6", + "StreamAgg 1.00 root funcs:count(1)->Column#6", "└─IndexMergeJoin 64.00 root inner join, inner:IndexReader, outer key:test.t.a, inner key:test.s.a, other cond:lt(test.s.b, test.t.b)", " ├─TableReader(Build) 64.00 root data:Selection", " │ └─Selection 64.00 cop[tikv] not(isnull(test.t.b))", @@ -1383,7 +1384,7 @@ func TestIndexLookupJoin(t *testing.T) { tk.MustQuery("select /*+ INL_MERGE_JOIN(s) */ count(*) from t join s use index(idx) on s.a = t.a and s.b < t.b").Check(testkit.Rows("64")) tk.MustQuery("desc format = 'brief' select /*+ INL_HASH_JOIN(s) */ count(*) from t join s use index(idx) on s.a = t.a and s.b < t.b").Check(testkit.Rows( - "HashAgg 1.00 root funcs:count(1)->Column#6", + "StreamAgg 1.00 root funcs:count(1)->Column#6", "└─IndexHashJoin 64.00 root inner join, inner:IndexReader, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), other cond:lt(test.s.b, test.t.b)", " ├─TableReader(Build) 64.00 root data:Selection", " │ └─Selection 64.00 cop[tikv] not(isnull(test.t.b))", diff --git a/executor/joiner.go b/executor/joiner.go index 842135802444f..5fe4d92eba2a2 100644 --- a/executor/joiner.go +++ b/executor/joiner.go @@ -192,7 +192,7 @@ func newJoiner(ctx sessionctx.Context, joinType plannercore.JoinType, return &antiLeftOuterSemiJoiner{base} case plannercore.LeftOuterJoin, plannercore.RightOuterJoin, plannercore.InnerJoin: if len(base.conditions) > 0 { - base.chk = chunk.NewChunkWithCapacity(shallowRowType, ctx.GetSessionVars().MaxChunkSize) + base.chk = ctx.GetSessionVars().GetNewChunk(shallowRowType, ctx.GetSessionVars().MaxChunkSize) } switch joinType { case plannercore.LeftOuterJoin: diff --git a/executor/merge_join.go b/executor/merge_join.go index 233d140ade678..a64a9fa0c33dc 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -77,7 +77,7 @@ type mergeJoinTable struct { func (t *mergeJoinTable) init(exec *MergeJoinExec) { child := exec.children[t.childIndex] - t.childChunk = newFirstChunk(child) + t.childChunk = tryNewCacheChunk(child) t.childChunkIter = chunk.NewIterator4Chunk(t.childChunk) items := make([]expression.Expression, 0, len(t.joinKeys)) diff --git a/executor/parallel_apply.go b/executor/parallel_apply.go index d0aa68af87d2b..2c4499b14818c 100644 --- a/executor/parallel_apply.go +++ b/executor/parallel_apply.go @@ -107,7 +107,7 @@ func (e *ParallelNestedLoopApplyExec) Open(ctx context.Context) error { e.hasMatch = make([]bool, e.concurrency) e.hasNull = make([]bool, e.concurrency) for i := 0; i < e.concurrency; i++ { - e.innerChunk[i] = newFirstChunk(e.innerExecs[i]) + e.innerChunk[i] = tryNewCacheChunk(e.innerExecs[i]) e.innerList[i] = chunk.NewList(retTypes(e.innerExecs[i]), e.initCap, e.maxChunkSize) e.innerList[i].GetMemTracker().SetLabel(memory.LabelForInnerList) e.innerList[i].GetMemTracker().AttachTo(e.memTracker) @@ -206,7 +206,7 @@ func (e *ParallelNestedLoopApplyExec) outerWorker(ctx context.Context) { var err error for { failpoint.Inject("parallelApplyOuterWorkerPanic", nil) - chk := newFirstChunk(e.outerExec) + chk := tryNewCacheChunk(e.outerExec) if err := Next(ctx, e.outerExec, chk); err != nil { e.putResult(nil, err) return diff --git a/executor/pipelined_window.go b/executor/pipelined_window.go index 1a72864bcf225..505cf09f415d7 100644 --- a/executor/pipelined_window.go +++ b/executor/pipelined_window.go @@ -205,7 +205,7 @@ func (e *PipelinedWindowExec) getRowsInPartition(ctx context.Context) (err error func (e *PipelinedWindowExec) fetchChild(ctx context.Context) (EOF bool, err error) { // TODO: reuse chunks - childResult := newFirstChunk(e.children[0]) + childResult := tryNewCacheChunk(e.children[0]) err = Next(ctx, e.children[0], childResult) if err != nil { return false, errors.Trace(err) @@ -217,7 +217,7 @@ func (e *PipelinedWindowExec) fetchChild(ctx context.Context) (EOF bool, err err } // TODO: reuse chunks - resultChk := chunk.New(e.retFieldTypes, 0, numRows) + resultChk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.retFieldTypes, 0, numRows) err = e.copyChk(childResult, resultChk) if err != nil { return false, err diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 5ebc97e9c0dcc..3d6aa75e06604 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -860,7 +860,7 @@ func TestIssue29101(t *testing.T) { c_last varchar(16) DEFAULT NULL, c_credit char(2) DEFAULT NULL, c_discount decimal(4,4) DEFAULT NULL, - PRIMARY KEY (c_w_id,c_d_id,c_id), + PRIMARY KEY (c_w_id,c_d_id,c_id) NONCLUSTERED, KEY idx_customer (c_w_id,c_d_id,c_last,c_first) )`) tk.MustExec(`CREATE TABLE warehouse ( @@ -890,12 +890,12 @@ func TestIssue29101(t *testing.T) { ol_w_id int(11) NOT NULL, ol_number int(11) NOT NULL, ol_i_id int(11) NOT NULL, - PRIMARY KEY (ol_w_id,ol_d_id,ol_o_id,ol_number))`) + PRIMARY KEY (ol_w_id,ol_d_id,ol_o_id,ol_number) NONCLUSTERED)`) tk.MustExec(`CREATE TABLE stock ( s_i_id int(11) NOT NULL, s_w_id int(11) NOT NULL, s_quantity int(11) DEFAULT NULL, - PRIMARY KEY (s_w_id,s_i_id))`) + PRIMARY KEY (s_w_id,s_i_id) NONCLUSTERED)`) tk.MustExec(`prepare s1 from 'SELECT /*+ TIDB_INLJ(order_line,stock) */ COUNT(DISTINCT (s_i_id)) stock_count FROM order_line, stock WHERE ol_w_id = ? AND ol_d_id = ? AND ol_o_id < ? AND ol_o_id >= ? - 20 AND s_w_id = ? AND s_i_id = ol_i_id AND s_quantity < ?'`) tk.MustExec(`set @a=391,@b=1,@c=3058,@d=18`) tk.MustExec(`execute s1 using @a,@b,@c,@c,@a,@d`) diff --git a/executor/projection.go b/executor/projection.go index ac060e7e4a391..27ce1bafc0b8a 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -109,7 +109,7 @@ func (e *ProjectionExec) open(ctx context.Context) error { } if e.isUnparallelExec() { - e.childResult = newFirstChunk(e.children[0]) + e.childResult = tryNewCacheChunk(e.children[0]) e.memTracker.Consume(e.childResult.MemoryUsage()) } diff --git a/executor/select_into.go b/executor/select_into.go index d526edec874f4..556edc58c7cbe 100644 --- a/executor/select_into.go +++ b/executor/select_into.go @@ -60,7 +60,7 @@ func (s *SelectIntoExec) Open(ctx context.Context) error { s.started = true s.dstFile = f s.writer = bufio.NewWriter(s.dstFile) - s.chk = newFirstChunk(s.children[0]) + s.chk = tryNewCacheChunk(s.children[0]) s.lineBuf = make([]byte, 0, 1024) s.fieldBuf = make([]byte, 0, 64) s.escapeBuf = make([]byte, 0, 64) diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 3deb9cb8a9c2d..cefdf622f2b13 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -541,7 +541,7 @@ func TestShow(t *testing.T) { // Test show create table year type tk.MustExec(`drop table if exists t`) - tk.MustExec(`create table t(y year unsigned signed zerofill zerofill, x int, primary key(y));`) + tk.MustExec(`create table t(y year unsigned signed zerofill zerofill, x int, primary key(y) nonclustered);`) tk.MustQuery(`show create table t`).Check(testkit.RowsWithSep("|", "t CREATE TABLE `t` (\n"+ " `y` year(4) NOT NULL,\n"+ diff --git a/executor/set_test.go b/executor/set_test.go index 72190fc4422af..a8d4a0b059246 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -2041,3 +2041,20 @@ func TestSetPlanCacheMemoryMonitor(t *testing.T) { tk.MustExec("set @@global.tidb_enable_prepared_plan_cache_memory_monitor=off;") tk.MustQuery("select @@global.tidb_enable_prepared_plan_cache_memory_monitor").Check(testkit.Rows("0")) } + +func TestSetChunkReuseVariable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_enable_reuse_chunk=ON;") + tk.MustQuery("select @@session.tidb_enable_reuse_chunk").Check(testkit.Rows("1")) + tk.MustExec("set GLOBAL tidb_enable_reuse_chunk=ON;") + tk.MustQuery("select @@global.tidb_enable_reuse_chunk").Check(testkit.Rows("1")) + + tk.MustExec("set @@tidb_enable_reuse_chunk=OFF;") + tk.MustQuery("select @@session.tidb_enable_reuse_chunk").Check(testkit.Rows("0")) + tk.MustExec("set GLOBAL tidb_enable_reuse_chunk=OFF;") + tk.MustQuery("select @@global.tidb_enable_reuse_chunk").Check(testkit.Rows("0")) + + // error value + tk.MustGetErrCode("set @@tidb_enable_reuse_chunk=s;", errno.ErrWrongValueForVar) +} diff --git a/executor/show.go b/executor/show.go index 9aa6b56f353b5..e96e95b21e699 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1219,7 +1219,7 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } // add partition info here. - appendPartitionInfo(tableInfo.Partition, buf, sqlMode) + ddl.AppendPartitionInfo(tableInfo.Partition, buf, sqlMode) return nil } @@ -1352,52 +1352,6 @@ func fetchShowCreateTable4View(ctx sessionctx.Context, tb *model.TableInfo, buf fmt.Fprintf(buf, ") AS %s", tb.View.SelectStmt) } -func appendPartitionInfo(partitionInfo *model.PartitionInfo, buf *bytes.Buffer, sqlMode mysql.SQLMode) { - if partitionInfo == nil { - return - } - // Since MySQL 5.1/5.5 is very old and TiDB aims for 5.7/8.0 compatibility, we will not - // include the /*!50100 or /*!50500 comments for TiDB. - // This also solves the issue with comments within comments that would happen for - // PLACEMENT POLICY options. - if partitionInfo.Type == model.PartitionTypeHash { - defaultPartitionDefinitions := true - for i, def := range partitionInfo.Definitions { - if def.Name.O != fmt.Sprintf("p%d", i) { - defaultPartitionDefinitions = false - break - } - if len(def.Comment) > 0 || def.PlacementPolicyRef != nil { - defaultPartitionDefinitions = false - break - } - } - - if defaultPartitionDefinitions { - fmt.Fprintf(buf, "\nPARTITION BY HASH (%s) PARTITIONS %d", partitionInfo.Expr, partitionInfo.Num) - return - } - } - // this if statement takes care of lists/range columns case - if len(partitionInfo.Columns) > 0 { - // partitionInfo.Type == model.PartitionTypeRange || partitionInfo.Type == model.PartitionTypeList - // Notice that MySQL uses two spaces between LIST and COLUMNS... - fmt.Fprintf(buf, "\nPARTITION BY %s COLUMNS(", partitionInfo.Type.String()) - for i, col := range partitionInfo.Columns { - buf.WriteString(stringutil.Escape(col.O, sqlMode)) - if i < len(partitionInfo.Columns)-1 { - buf.WriteString(",") - } - } - buf.WriteString(")\n(") - } else { - fmt.Fprintf(buf, "\nPARTITION BY %s (%s)\n(", partitionInfo.Type.String(), partitionInfo.Expr) - } - - ddl.AppendPartitionDefs(partitionInfo, buf, sqlMode) - buf.WriteString(")") -} - // ConstructResultOfShowCreateDatabase constructs the result for show create database. func ConstructResultOfShowCreateDatabase(ctx sessionctx.Context, dbInfo *model.DBInfo, ifNotExists bool, buf *bytes.Buffer) (err error) { sqlMode := ctx.GetSessionVars().SQLMode diff --git a/executor/showtest/show_test.go b/executor/showtest/show_test.go index 76eb9be2e8558..cd06bd56b2229 100644 --- a/executor/showtest/show_test.go +++ b/executor/showtest/show_test.go @@ -186,7 +186,7 @@ func TestShowCreateTable(t *testing.T) { "`END_TIME` datetime NOT NULL," + "`USER_TYPE` int(11) DEFAULT NULL," + "`APP_ID` int(11) DEFAULT NULL," + - "PRIMARY KEY (`LOG_ID`,`END_TIME`)," + + "PRIMARY KEY (`LOG_ID`,`END_TIME`) NONCLUSTERED," + "KEY `IDX_EndTime` (`END_TIME`)," + "KEY `IDX_RoundId` (`ROUND_ID`)," + "KEY `IDX_UserId_EndTime` (`USER_ID`,`END_TIME`)" + diff --git a/executor/shuffle.go b/executor/shuffle.go index 2827d0e1cece7..7596d2cef1970 100644 --- a/executor/shuffle.go +++ b/executor/shuffle.go @@ -262,7 +262,7 @@ func (e *ShuffleExec) fetchDataAndSplit(ctx context.Context, dataSourceIndex int workerIndices []int ) results := make([]*chunk.Chunk, len(e.workers)) - chk := newFirstChunk(e.dataSources[dataSourceIndex]) + chk := tryNewCacheChunk(e.dataSources[dataSourceIndex]) defer func() { if r := recover(); r != nil { diff --git a/executor/sort.go b/executor/sort.go index 0574798d12bce..06241993e05f3 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -194,7 +194,7 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { e.rowChunks.GetDiskTracker().SetLabel(memory.LabelForRowChunks) } for { - chk := newFirstChunk(e.children[0]) + chk := tryNewCacheChunk(e.children[0]) err := Next(ctx, e.children[0], chk) if err != nil { return err @@ -226,6 +226,13 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { } } } + failpoint.Inject("SignalCheckpointForSort", func(val failpoint.Value) { + if val.(bool) { + if e.ctx.GetSessionVars().ConnectionID == 123456 { + e.ctx.GetSessionVars().MemTracker.NeedKill.Store(true) + } + } + }) if e.rowChunks.NumRow() > 0 { e.rowChunks.Sort() e.partitionList = append(e.partitionList, e.rowChunks) @@ -427,7 +434,7 @@ func (e *TopNExec) loadChunksUntilTotalLimit(ctx context.Context) error { e.rowChunks.GetMemTracker().AttachTo(e.memTracker) e.rowChunks.GetMemTracker().SetLabel(memory.LabelForRowChunks) for uint64(e.rowChunks.Len()) < e.totalLimit { - srcChk := newFirstChunk(e.children[0]) + srcChk := tryNewCacheChunk(e.children[0]) // adjust required rows by total limit srcChk.SetRequiredRows(int(e.totalLimit-uint64(e.rowChunks.Len())), e.maxChunkSize) err := Next(ctx, e.children[0], srcChk) @@ -453,7 +460,7 @@ func (e *TopNExec) executeTopN(ctx context.Context) error { // The number of rows we loaded may exceeds total limit, remove greatest rows by Pop. heap.Pop(e.chkHeap) } - childRowChk := newFirstChunk(e.children[0]) + childRowChk := tryNewCacheChunk(e.children[0]) for { err := Next(ctx, e.children[0], childRowChk) if err != nil { diff --git a/executor/union_scan.go b/executor/union_scan.go index 9c0483d974f1c..a23cd8b8c7873 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -129,7 +129,7 @@ func (us *UnionScanExec) open(ctx context.Context) error { if err != nil { return err } - us.snapshotChunkBuffer = newFirstChunk(us) + us.snapshotChunkBuffer = tryNewCacheChunk(us) return nil } diff --git a/executor/update.go b/executor/update.go index d9cffabd08355..cf0a6ae2e33f4 100644 --- a/executor/update.go +++ b/executor/update.go @@ -250,7 +250,7 @@ func (e *UpdateExec) updateRows(ctx context.Context) (int, error) { fields := retTypes(e.children[0]) colsInfo := plannercore.GetUpdateColumnsInfo(e.tblID2table, e.tblColPosInfos, len(fields)) globalRowIdx := 0 - chk := newFirstChunk(e.children[0]) + chk := tryNewCacheChunk(e.children[0]) if !e.allAssignmentsAreConstant { e.evalBuffer = chunk.MutRowFromTypes(fields) } diff --git a/executor/window.go b/executor/window.go index 3ce26a03e59cf..ef284344d0c8c 100644 --- a/executor/window.go +++ b/executor/window.go @@ -151,7 +151,7 @@ func (e *WindowExec) consumeGroupRows(groupRows []chunk.Row) (err error) { } func (e *WindowExec) fetchChild(ctx context.Context) (EOF bool, err error) { - childResult := newFirstChunk(e.children[0]) + childResult := tryNewCacheChunk(e.children[0]) err = Next(ctx, e.children[0], childResult) if err != nil { return false, errors.Trace(err) @@ -162,7 +162,7 @@ func (e *WindowExec) fetchChild(ctx context.Context) (EOF bool, err error) { return true, nil } - resultChk := chunk.New(e.retFieldTypes, 0, numRows) + resultChk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.retFieldTypes, 0, numRows) err = e.copyChk(childResult, resultChk) if err != nil { return false, err diff --git a/executor/write_test.go b/executor/write_test.go index fc565183009b7..32aa261c5518d 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -322,7 +322,7 @@ func TestInsert(t *testing.T) { require.EqualError(t, err, "[kv:1062]Duplicate entry 'ch' for key 't.PRIMARY'") tk.MustExec("insert into t(name, b) values(\"测试\", 3)") err = tk.ExecToErr("insert into t(name, b) values(\"测试\", 3)") - require.EqualError(t, err, "[kv:1062]Duplicate entry '测试' for key 't.PRIMARY'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '\xe6\xb5' for key 't.PRIMARY'") } func TestMultiBatch(t *testing.T) { diff --git a/expression/constant_propagation_test.go b/expression/constant_propagation_test.go index 9c10d9ddd982b..d2ebfdef4080d 100644 --- a/expression/constant_propagation_test.go +++ b/expression/constant_propagation_test.go @@ -27,6 +27,7 @@ func TestOuterJoinPropConst(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t1, t2;") tk.MustExec("create table t1(id bigint primary key, a int, b int);") tk.MustExec("create table t2(id bigint primary key, a int, b int);") diff --git a/expression/testdata/expression_suite_out.json b/expression/testdata/expression_suite_out.json index 7047b62ba1156..164ccd7f50311 100644 --- a/expression/testdata/expression_suite_out.json +++ b/expression/testdata/expression_suite_out.json @@ -65,11 +65,11 @@ "Result": [ "HashJoin 4166.67 root right outer join, equal:[eq(test.t1.a, test.t2.a)]", "├─TableReader(Build) 3333.33 root data:Selection", - "│ └─Selection 3333.33 cop[tikv] gt(test.t1.a, 1), not(isnull(test.t1.a))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─Selection 3333.33 cop[tikv] gt(test.t2.a, 1)", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "└─TableReader(Probe) 3333.33 root data:Selection", - " └─Selection 3333.33 cop[tikv] gt(test.t2.a, 1)", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + " └─Selection 3333.33 cop[tikv] gt(test.t1.a, 1), not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ] }, { diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 1c6ab1490f9bb..64b38de1c9db3 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1543,6 +1543,7 @@ func TestVariablesInfo(t *testing.T) { // See session/bootstrap.go:doDMLWorks() for where the exceptions are defined. stmt := tk.MustQuery(`SELECT variable_name, default_value, current_value FROM information_schema.variables_info WHERE current_value != default_value and default_value != '' ORDER BY variable_name`) stmt.Check(testkit.Rows( + "last_sql_use_alloc OFF ON", // for test stability "tidb_enable_auto_analyze ON OFF", // always changed for tests "tidb_enable_collect_execution_info ON OFF", // for test stability "tidb_enable_mutation_checker OFF ON", // for new installs diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index c88edf4470d9b..c7e663f43ff1e 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -167,6 +167,7 @@ func TestEstimation(t *testing.T) { statistics.RatioOfPseudoEstimate.Store(10.0) defer statistics.RatioOfPseudoEstimate.Store(0.7) testKit.MustExec("use test") + testKit.MustExec("set tidb_cost_model_version=2") testKit.MustExec("create table t (a int)") testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") testKit.MustExec("insert into t select * from t") @@ -211,6 +212,7 @@ func constructInsertSQL(i, n int) string { func TestIndexRead(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) testKit := testkit.NewTestKit(t, store) + testKit.MustExec("set tidb_cost_model_version=2") testKit.MustExec("set @@session.tidb_executor_concurrency = 4;") testKit.MustExec("set @@session.tidb_hash_join_concurrency = 5;") testKit.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") @@ -662,6 +664,7 @@ func TestLimitCrossEstimation(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set @@session.tidb_executor_concurrency = 4;") tk.MustExec("set @@session.tidb_hash_join_concurrency = 5;") tk.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") @@ -796,6 +799,7 @@ func TestLimitIndexEstimation(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, key idx_a(a), key idx_b(b))") tk.MustExec("set session tidb_enable_extended_stats = on") @@ -827,7 +831,7 @@ func TestBatchPointGetTablePartition(t *testing.T) { testKit.MustExec("use test") testKit.MustExec("drop table if exists t1,t2,t3,t4,t5,t6") - testKit.MustExec("create table t1(a int, b int, primary key(a,b)) partition by hash(b) partitions 2") + testKit.MustExec("create table t1(a int, b int, primary key(a,b) nonclustered) partition by hash(b) partitions 2") testKit.MustExec("insert into t1 values(1,1),(1,2),(2,1),(2,2)") testKit.MustExec("set @@tidb_partition_prune_mode = 'static'") testKit.MustQuery("explain format = 'brief' select * from t1 where a in (1,2) and b = 1").Check(testkit.Rows( @@ -864,7 +868,7 @@ func TestBatchPointGetTablePartition(t *testing.T) { "1 2", )) - testKit.MustExec("create table t2(a int, b int, primary key(a,b)) partition by range(b) (partition p0 values less than (2), partition p1 values less than maxvalue)") + testKit.MustExec("create table t2(a int, b int, primary key(a,b) nonclustered) partition by range(b) (partition p0 values less than (2), partition p1 values less than maxvalue)") testKit.MustExec("insert into t2 values(1,1),(1,2),(2,1),(2,2)") testKit.MustExec("set @@tidb_partition_prune_mode = 'static'") testKit.MustQuery("explain format = 'brief' select * from t2 where a in (1,2) and b = 1").Check(testkit.Rows( diff --git a/planner/core/foreign_key.go b/planner/core/foreign_key.go index baa271b903443..6ee715380f8ba 100644 --- a/planner/core/foreign_key.go +++ b/planner/core/foreign_key.go @@ -17,6 +17,7 @@ package core import ( "unsafe" + "github.com/pingcap/errors" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -45,6 +46,8 @@ type FKCascade struct { ReferredFK *model.ReferredFKInfo ChildTable table.Table FK *model.FKInfo + FKCols []*model.ColumnInfo + FKIdx *model.IndexInfo } // FKCascadeType indicates in which (delete/update) statements. @@ -299,13 +302,8 @@ func buildOnDeleteOrUpdateFKTrigger(is infoschema.InfoSchema, referredFK *model. } switch fkReferOption { case model.ReferOptionCascade, model.ReferOptionSetNull: - fkCascade := &FKCascade{ - Tp: tp, - ReferredFK: referredFK, - ChildTable: childTable, - FK: fk, - } - return nil, fkCascade, nil + fkCascade, err := buildFKCascade(tp, referredFK, childTable, fk) + return nil, fkCascade, err default: fkCheck, err := buildFKCheckForReferredFK(childTable, fk, referredFK) return fkCheck, nil, err @@ -395,3 +393,34 @@ func buildFKCheck(tbl table.Table, cols []model.CIStr, failedErr error) (*FKChec FailedErr: failedErr, }, nil } + +func buildFKCascade(tp FKCascadeType, referredFK *model.ReferredFKInfo, childTable table.Table, fk *model.FKInfo) (*FKCascade, error) { + cols := make([]*model.ColumnInfo, len(fk.Cols)) + childTableColumns := childTable.Meta().Columns + for i, c := range fk.Cols { + col := model.FindColumnInfo(childTableColumns, c.L) + if col == nil { + return nil, errors.Errorf("foreign key column %s is not found in table %s", c.L, childTable.Meta().Name) + } + cols[i] = col + } + fkCascade := &FKCascade{ + Tp: tp, + ReferredFK: referredFK, + ChildTable: childTable, + FK: fk, + FKCols: cols, + } + if childTable.Meta().PKIsHandle && len(cols) == 1 { + refColInfo := model.FindColumnInfo(childTableColumns, cols[0].Name.L) + if refColInfo != nil && mysql.HasPriKeyFlag(refColInfo.GetFlag()) { + return fkCascade, nil + } + } + indexForFK := model.FindIndexByColumns(childTable.Meta(), fk.Cols...) + if indexForFK == nil { + return nil, errors.Errorf("Missing index for '%s' foreign key columns in the table '%s'", fk.Name, childTable.Meta().Name) + } + fkCascade.FKIdx = indexForFK + return fkCascade, nil +} diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 98f494da8ee54..2dbfac7a1c9ac 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -68,7 +68,7 @@ func TestShowSubquery(t *testing.T) { )) tk.MustQuery("show columns from t where field in (select 'b') and false").Check(testkit.Rows()) tk.MustExec("insert into t values('c', 0, 0)") - tk.MustQuery("show columns from t where field < all (select a from t)").Check(testkit.Rows( + tk.MustQuery("show columns from t where field < all (select a from t)").Sort().Check(testkit.Rows( "a varchar(10) YES ", "b int(11) YES ", )) @@ -471,6 +471,7 @@ func TestVerboseExplain(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec(`set tidb_opt_limit_push_down_threshold=0`) tk.MustExec("drop table if exists t1, t2, t3") tk.MustExec("create table t1(a int, b int)") @@ -4447,9 +4448,9 @@ func TestReorderSimplifiedOuterJoins(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2,t3") - tk.MustExec("create table t1 (pk char(32) primary key, col1 char(32), col2 varchar(40), col3 char(32), key (col1), key (col3), key (col2,col3), key (col1,col3))") - tk.MustExec("create table t2 (pk char(32) primary key, col1 varchar(100))") - tk.MustExec("create table t3 (pk char(32) primary key, keycol varchar(100), pad1 tinyint(1) default null, pad2 varchar(40), key (keycol,pad1,pad2))") + tk.MustExec("create table t1 (pk char(32) primary key nonclustered, col1 char(32), col2 varchar(40), col3 char(32), key (col1), key (col3), key (col2,col3), key (col1,col3))") + tk.MustExec("create table t2 (pk char(32) primary key nonclustered, col1 varchar(100))") + tk.MustExec("create table t3 (pk char(32) primary key nonclustered, keycol varchar(100), pad1 tinyint(1) default null, pad2 varchar(40), key (keycol,pad1,pad2))") var input []string var output []struct { @@ -4967,7 +4968,7 @@ func TestMultiColMaxOneRow(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2") tk.MustExec("create table t1(a int)") - tk.MustExec("create table t2(a int, b int, c int, primary key(a,b))") + tk.MustExec("create table t2(a int, b int, c int, primary key(a,b) nonclustered)") var input []string var output []struct { @@ -6525,7 +6526,7 @@ func TestAggPushToCopForCachedTable(t *testing.T) { oper_no varchar(12) DEFAULT NULL, modify_date datetime DEFAULT NULL, d_c_flag varchar(2) NOT NULL, - PRIMARY KEY (process_code,ctrl_class,d_c_flag));`) + PRIMARY KEY (process_code,ctrl_class,d_c_flag) NONCLUSTERED);`) tk.MustExec("insert into t32157 values ('GDEP0071', '05', '1', '10000', '2016-06-29 00:00:00', 'C')") tk.MustExec("insert into t32157 values ('GDEP0071', '05', '0', '0000', '2016-06-01 00:00:00', 'D')") tk.MustExec("alter table t32157 cache") diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index fffef54373bd8..75350e7d3a1ad 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -78,7 +78,7 @@ func TestRangeColumnPartitionPruningForIn(t *testing.T) { tk.MustExec(`CREATE TABLE t1 ( id bigint(20) NOT NULL AUTO_INCREMENT, dt date, - PRIMARY KEY (id,dt)) + PRIMARY KEY (id,dt) NONCLUSTERED) PARTITION BY RANGE COLUMNS(dt) ( PARTITION p20201125 VALUES LESS THAN ("20201126"), PARTITION p20201126 VALUES LESS THAN ("20201127"), diff --git a/planner/core/plan_cost_ver2.go b/planner/core/plan_cost_ver2.go index 595670e730982..ef4be46bee174 100644 --- a/planner/core/plan_cost_ver2.go +++ b/planner/core/plan_cost_ver2.go @@ -425,6 +425,10 @@ func (p *PhysicalHashAgg) getPlanCostVer2(taskType property.TaskType, option *Pl memFactor := getTaskMemFactorVer2(p, taskType) concurrency := float64(p.ctx.GetSessionVars().HashAggFinalConcurrency()) + if inputRows < 2000 { // prefer to use StreamAgg if no much data to process + inputRows = 2000 + } + aggCost := aggCostVer2(option, inputRows, p.AggFuncs, cpuFactor) groupCost := groupCostVer2(option, inputRows, p.GroupByItems, cpuFactor) hashBuildCost := hashBuildCostVer2(option, outputRows, outputRowSize, p.GroupByItems, cpuFactor, memFactor) diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 54b6d0ad3e381..8f8c5342794ec 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -314,7 +314,7 @@ func TestNormalizedDigest(t *testing.T) { ol_supply_w_id int(11) DEFAULT NULL, ol_quantity int(11) DEFAULT NULL, ol_dist_info char(24) DEFAULT NULL, - PRIMARY KEY ( ol_w_id , ol_d_id , ol_o_id , ol_number ) + PRIMARY KEY ( ol_w_id , ol_d_id , ol_o_id , ol_number ) NONCLUSTERED );`) tk.MustExec(`CREATE TABLE bmsql_district ( d_w_id int(11) NOT NULL, @@ -328,7 +328,7 @@ func TestNormalizedDigest(t *testing.T) { d_city varchar(20) DEFAULT NULL, d_state char(2) DEFAULT NULL, d_zip char(9) DEFAULT NULL, - PRIMARY KEY ( d_w_id , d_id ) + PRIMARY KEY ( d_w_id , d_id ) NONCLUSTERED );`) tk.MustExec(`CREATE TABLE bmsql_stock ( s_w_id int(11) NOT NULL, @@ -348,7 +348,7 @@ func TestNormalizedDigest(t *testing.T) { s_dist_08 char(24) DEFAULT NULL, s_dist_09 char(24) DEFAULT NULL, s_dist_10 char(24) DEFAULT NULL, - PRIMARY KEY ( s_w_id , s_i_id ) + PRIMARY KEY ( s_w_id , s_i_id ) NONCLUSTERED );`) err := failpoint.Enable("github.com/pingcap/tidb/planner/mockRandomPlanID", "return(true)") diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index 27be9babce05a..1ff0346eecf77 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -959,7 +959,7 @@ func TestIssue26638(t *testing.T) { tk.MustQuery("execute stmt1 using @c;").Check(testkit.Rows()) tk.MustQuery("execute stmt2 using @c, @d;").Check(testkit.Rows()) tk.MustExec("drop table if exists t2;") - tk.MustExec("create table t2(a float, b float, c float, primary key(a, b, c));") + tk.MustExec("create table t2(a float, b float, c float, primary key(a, b, c) nonclustered);") tk.MustExec("insert into t2 values(-1, 0, 1), (-1.1, 0, 1.1), (-1.56018e38, -1.96716e38, 9.46347e37), (0, 1, 2);") tk.MustQuery("explain format='brief' select * from t2 where (a, b, c) in ((-1.1, 0, 1.1), (-1.56018e38, -1.96716e38, 9.46347e37));").Check(testkit.Rows("TableDual 0.00 root rows:0")) tk.MustQuery("select * from t2 where (a, b, c) in ((-1.1, 0, 1.1), (-1.56018e38, -1.96716e38, 9.46347e37), (-1, 0, 1));").Check(testkit.Rows("-1 0 1")) @@ -976,7 +976,7 @@ func TestIssue23511(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("CREATE TABLE `t1` (`COL1` bit(11) NOT NULL,PRIMARY KEY (`COL1`));") + tk.MustExec("CREATE TABLE `t1` (`COL1` bit(11) NOT NULL,PRIMARY KEY (`COL1`) NONCLUSTERED);") tk.MustExec("CREATE TABLE `t2` (`COL1` bit(11) NOT NULL);") tk.MustExec("insert into t1 values(b'00000111001'), (b'00000000000');") tk.MustExec("insert into t2 values(b'00000111001');") diff --git a/planner/core/testdata/analyze_suite_out.json b/planner/core/testdata/analyze_suite_out.json index e1fbcba55ee34..d9345a1a5bcce 100644 --- a/planner/core/testdata/analyze_suite_out.json +++ b/planner/core/testdata/analyze_suite_out.json @@ -45,10 +45,9 @@ { "SQL": "explain format = 'brief' select count(*) from t group by a", "Plan": [ - "HashAgg 2.00 root group by:test.t.a, funcs:count(Column#4)->Column#3", - "└─TableReader 2.00 root data:HashAgg", - " └─HashAgg 2.00 cop[tikv] group by:test.t.a, funcs:count(1)->Column#4", - " └─TableFullScan 8.00 cop[tikv] table:t keep order:false" + "HashAgg 2.00 root group by:test.t.a, funcs:count(1)->Column#3", + "└─TableReader 8.00 root data:TableFullScan", + " └─TableFullScan 8.00 cop[tikv] table:t keep order:false" ] } ] @@ -160,13 +159,13 @@ ], "Plan": [ "Limit 1.00 root offset:0, count:1", - "└─IndexJoin 1.00 root left outer semi join, inner:IndexReader, outer key:test.t.a, inner key:test.t.b, equal cond:eq(test.t.a, test.t.b)", - " ├─TopN(Build) 1.00 root test.t.a, offset:0, count:1", - " │ └─IndexReader 1.00 root index:TopN", - " │ └─TopN 1.00 cop[tikv] test.t.a, offset:0, count:1", - " │ └─IndexRangeScan 6.00 cop[tikv] table:t1, index:idx_bc(b, c) range:[-inf,6], keep order:false", - " └─IndexReader(Probe) 1.04 root index:IndexRangeScan", - " └─IndexRangeScan 1.04 cop[tikv] table:t2, index:idx_bc(b, c) range: decided by [eq(test.t.b, test.t.a)], keep order:false" + "└─MergeJoin 1.00 root left outer semi join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader(Build) 25.00 root index:IndexFullScan", + " │ └─IndexFullScan 25.00 cop[tikv] table:t2, index:idx_bc(b, c) keep order:true", + " └─TopN(Probe) 1.00 root test.t.a, offset:0, count:1", + " └─IndexReader 1.00 root index:TopN", + " └─TopN 1.00 cop[tikv] test.t.a, offset:0, count:1", + " └─IndexRangeScan 6.00 cop[tikv] table:t1, index:idx_bc(b, c) range:[-inf,6], keep order:false" ] }, { @@ -433,13 +432,13 @@ { "Name": "TestIndexRead", "Cases": [ - "IndexReader(Index(t.e)[[NULL,+inf]])->HashAgg", + "IndexReader(Index(t.e)[[NULL,+inf]]->StreamAgg)->StreamAgg", "IndexReader(Index(t.e)[[-inf,10]]->StreamAgg)->StreamAgg", "IndexReader(Index(t.e)[[-inf,50]]->StreamAgg)->StreamAgg", - "IndexReader(Index(t.b_c)[[NULL,+inf]]->Sel([gt(test.t.c, 1)])->HashAgg)->HashAgg", + "IndexReader(Index(t.b_c)[[NULL,+inf]]->Sel([gt(test.t.c, 1)])->StreamAgg)->StreamAgg", "IndexLookUp(Index(t.e)[[1,1]], Table(t))->HashAgg", "TableReader(Table(t)->Sel([gt(test.t.e, 1)])->HashAgg)->HashAgg", - "IndexLookUp(Index(t.b)[[-inf,20]], Table(t)->HashAgg)->HashAgg", + "TableReader(Table(t)->Sel([le(test.t.b, 20)])->StreamAgg)->StreamAgg", "TableReader(Table(t)->Sel([le(test.t.b, 30)])->StreamAgg)->StreamAgg", "TableReader(Table(t)->Sel([le(test.t.b, 40)])->StreamAgg)->StreamAgg", "TableReader(Table(t)->Sel([le(test.t.b, 50)])->StreamAgg)->StreamAgg", @@ -449,7 +448,7 @@ "TableReader(Table(t)->Sel([le(test.t.b, 10000000000)]))", "TableReader(Table(t)->Sel([le(test.t.b, 50)]))", "TableReader(Table(t)->Sel([le(test.t.b, 100)])->Limit)->Limit", - "IndexLookUp(Index(t.b)[[-inf,1]]->TopN([test.t.a],0,10), Table(t))->TopN([test.t.a],0,10)", + "TableReader(Table(t)->Sel([le(test.t.b, 1)])->Limit)->Limit", "IndexLookUp(Index(t.b)[[1,1]], Table(t))", "IndexLookUp(Index(t.d)[[-inf,1991-09-05 00:00:00)], Table(t))", "IndexLookUp(Index(t.ts)[[-inf,1991-09-05 00:00:00)], Table(t))", @@ -501,10 +500,10 @@ "SQL": "explain format = 'brief' select * from t where a <= 10000 order by b limit 1", "Plan": [ "TopN 1.00 root test.t.b, offset:0, count:1", - "└─IndexLookUp 1.00 root ", - " ├─IndexRangeScan(Build) 10000.00 cop[tikv] table:t, index:idx_a(a) range:[-inf,10000], keep order:false", - " └─TopN(Probe) 1.00 cop[tikv] test.t.b, offset:0, count:1", - " └─TableRowIDScan 10000.00 cop[tikv] table:t keep order:false" + "└─TableReader 1.00 root data:TopN", + " └─TopN 1.00 cop[tikv] test.t.b, offset:0, count:1", + " └─Selection 10000.00 cop[tikv] le(test.t.a, 10000)", + " └─TableFullScan 1000000.00 cop[tikv] table:t keep order:false" ] }, { diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 6855084993514..66b6afdeb026e 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -468,10 +468,10 @@ "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", "explain format = 'verbose' select count(*) from t2 where a = 0", "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", - "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a", - "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", - "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", - "explain format = 'verbose' select /*+ merge_join(t1) */ count(*) from t1 join t2 on t1.a = t2.a" + "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", + "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", + "explain format = 'verbose' select (2) in (select /*+ read_from_storage(tiflash[t1]) */ count(*) from t1) from (select t.b < (select /*+ read_from_storage(tiflash[t2]) */ t.b from t2 limit 1 ) from t3 t) t", + "explain format = 'verbose' select /*+ merge_join(t1), read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a" ] }, diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index a18436924f455..b6635a1cfac75 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -2485,80 +2485,80 @@ { "SQL": "explain format = 'verbose' select count(*) from t3", "Plan": [ - "StreamAgg_20 1.00 12.68 root funcs:count(Column#9)->Column#4", - "└─TableReader_21 1.00 9.68 root data:StreamAgg_8", - " └─StreamAgg_8 1.00 117.00 cop[tikv] funcs:count(1)->Column#9", - " └─TableFullScan_18 3.00 108.00 cop[tikv] table:t3 keep order:false" + "StreamAgg_20 1.00 99.31 root funcs:count(Column#9)->Column#4", + "└─TableReader_21 1.00 49.41 root data:StreamAgg_8", + " └─StreamAgg_8 1.00 709.52 cop[tikv] funcs:count(1)->Column#9", + " └─TableFullScan_18 3.00 559.82 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t2", "Plan": [ - "StreamAgg_26 1.00 8.18 root funcs:count(Column#7)->Column#4", - "└─TableReader_27 1.00 5.17 root data:StreamAgg_10", - " └─StreamAgg_10 1.00 49.50 batchCop[tiflash] funcs:count(1)->Column#7", - " └─TableFullScan_25 3.00 40.50 batchCop[tiflash] table:t2 keep order:false" + "StreamAgg_26 1.00 99.31 root funcs:count(Column#7)->Column#4", + "└─TableReader_27 1.00 49.41 root data:StreamAgg_10", + " └─StreamAgg_10 1.00 709.52 cop[tikv] funcs:count(1)->Column#7", + " └─TableFullScan_24 3.00 559.82 cop[tikv] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by a", "Plan": [ - "Sort_4 3.00 45.85 root test.t3.a", - "└─TableReader_8 3.00 11.78 root data:TableFullScan_7", - " └─TableFullScan_7 3.00 108.00 cop[tikv] table:t3 keep order:false" + "Sort_4 3.00 309.53 root test.t3.a", + "└─TableReader_8 3.00 62.67 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 559.82 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by b", "Plan": [ - "Sort_4 3.00 45.85 root test.t3.b", - "└─TableReader_8 3.00 11.78 root data:TableFullScan_7", - " └─TableFullScan_7 3.00 108.00 cop[tikv] table:t3 keep order:false" + "Sort_4 3.00 309.53 root test.t3.b", + "└─TableReader_8 3.00 62.67 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 559.82 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by a limit 1", "Plan": [ - "TopN_7 1.00 13.22 root test.t3.a, offset:0, count:1", - "└─TableReader_16 1.00 10.22 root data:TopN_15", - " └─TopN_15 1.00 117.00 cop[tikv] test.t3.a, offset:0, count:1", - " └─TableFullScan_14 3.00 108.00 cop[tikv] table:t3 keep order:false" + "TopN_7 1.00 44.96 root test.t3.a, offset:0, count:1", + "└─TableReader_16 1.00 41.76 root data:TopN_15", + " └─TopN_15 1.00 563.02 cop[tikv] test.t3.a, offset:0, count:1", + " └─TableFullScan_14 3.00 559.82 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by b limit 1", "Plan": [ - "TopN_7 1.00 13.22 root test.t3.b, offset:0, count:1", - "└─TableReader_16 1.00 10.22 root data:TopN_15", - " └─TopN_15 1.00 117.00 cop[tikv] test.t3.b, offset:0, count:1", - " └─TableFullScan_14 3.00 108.00 cop[tikv] table:t3 keep order:false" + "TopN_7 1.00 44.96 root test.t3.b, offset:0, count:1", + "└─TableReader_16 1.00 41.76 root data:TopN_15", + " └─TopN_15 1.00 563.02 cop[tikv] test.t3.b, offset:0, count:1", + " └─TableFullScan_14 3.00 559.82 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t2 group by a", "Plan": [ - "TableReader_44 3.00 4.98 root data:ExchangeSender_43", - "└─ExchangeSender_43 3.00 96.60 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_38 3.00 76.80 mpp[tiflash] Column#4", - " └─HashAgg_36 3.00 57.00 mpp[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", - " └─ExchangeReceiver_22 3.00 48.00 mpp[tiflash] ", - " └─ExchangeSender_21 3.00 45.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", - " └─TableFullScan_20 3.00 45.00 mpp[tiflash] table:t2 keep order:false" + "TableReader_25 3.00 25896.21 root data:ExchangeSender_24", + "└─ExchangeSender_24 3.00 388390.38 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_23 3.00 388390.38 mpp[tiflash] Column#4", + " └─HashAgg_9 3.00 388390.38 mpp[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", + " └─ExchangeReceiver_22 3.00 385507.26 mpp[tiflash] ", + " └─ExchangeSender_21 3.00 385459.26 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", + " └─TableFullScan_20 3.00 385459.26 mpp[tiflash] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t3 where b = 0", "Plan": [ - "StreamAgg_10 1.00 1.33 root funcs:count(1)->Column#4", - "└─IndexReader_15 0.00 1.33 root index:IndexRangeScan_14", + "StreamAgg_10 1.00 0.00 root funcs:count(1)->Column#4", + "└─IndexReader_15 0.00 0.00 root index:IndexRangeScan_14", " └─IndexRangeScan_14 0.00 0.00 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false" ] }, { "SQL": "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", "Plan": [ - "StreamAgg_10 1.00 19.33 root funcs:count(test.t3.a)->Column#4", - "└─IndexLookUp_17 0.00 19.33 root ", + "StreamAgg_10 1.00 0.00 root funcs:count(test.t3.a)->Column#4", + "└─IndexLookUp_17 0.00 0.00 root ", " ├─IndexRangeScan_15(Build) 0.00 0.00 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false", " └─TableRowIDScan_16(Probe) 0.00 0.00 cop[tikv] table:t3 keep order:false" ] @@ -2566,91 +2566,90 @@ { "SQL": "explain format = 'verbose' select count(*) from t2 where a = 0", "Plan": [ - "StreamAgg_12 1.00 4.93 root funcs:count(1)->Column#4", - "└─TableReader_24 0.00 4.93 root data:Selection_23", - " └─Selection_23 0.00 54.00 cop[tiflash] eq(test.t2.a, 0)", - " └─TableFullScan_22 3.00 45.00 cop[tiflash] table:t2 keep order:false" + "StreamAgg_12 1.00 47.30 root funcs:count(1)->Column#4", + "└─TableReader_21 0.00 47.30 root data:Selection_20", + " └─Selection_20 0.00 709.52 cop[tikv] eq(test.t2.a, 0)", + " └─TableFullScan_19 3.00 559.82 cop[tikv] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", "Plan": [ - "StreamAgg_10 1.00 60.22 root funcs:count(1)->Column#7", - "└─HashJoin_40 3.00 51.22 root inner join, equal:[eq(test.t3.a, test.t3.b)]", - " ├─IndexReader_28(Build) 3.00 11.66 root index:IndexFullScan_27", - " │ └─IndexFullScan_27 3.00 130.50 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_26(Probe) 3.00 10.76 root data:Selection_25", - " └─Selection_25 3.00 117.00 cop[tikv] not(isnull(test.t3.a))", - " └─TableFullScan_24 3.00 108.00 cop[tikv] table:t keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a", - "Plan": [ - "StreamAgg_15 1.00 18.93 root funcs:count(1)->Column#7", - "└─TableReader_47 3.00 9.93 root data:ExchangeSender_46", - " └─ExchangeSender_46 3.00 195.38 mpp[tiflash] ExchangeType: PassThrough", - " └─HashJoin_43 3.00 195.38 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_22(Build) 3.00 57.00 mpp[tiflash] ", - " │ └─ExchangeSender_21 3.00 54.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection_20 3.00 54.00 mpp[tiflash] not(isnull(test.t1.a))", - " │ └─TableFullScan_19 3.00 45.00 mpp[tiflash] table:t1 keep order:false", - " └─Selection_24(Probe) 3.00 54.00 mpp[tiflash] not(isnull(test.t2.a))", - " └─TableFullScan_23 3.00 45.00 mpp[tiflash] table:t2 keep order:false" - ] - }, - { - "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 60.60 root funcs:count(1)->Column#10", - "└─HashJoin_65 3.00 51.60 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 130.50 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_39(Probe) 3.00 11.14 root data:ExchangeSender_38", - " └─ExchangeSender_38 3.00 204.38 mpp[tiflash] ExchangeType: PassThrough", - " └─HashJoin_29 3.00 204.38 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_35(Build) 3.00 66.00 mpp[tiflash] ", - " │ └─ExchangeSender_34 3.00 63.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection_33 3.00 63.00 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─TableFullScan_32 3.00 54.00 mpp[tiflash] table:t1 keep order:false", - " └─Selection_37(Probe) 3.00 54.00 mpp[tiflash] not(isnull(test.t2.a))", - " └─TableFullScan_36 3.00 45.00 mpp[tiflash] table:t2 keep order:false" - ] - }, - { - "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 127.40 root CARTESIAN left outer semi join", - "├─Selection_39(Build) 0.80 11.18 root eq(2, Column#18)", - "│ └─StreamAgg_61 1.00 8.18 root funcs:count(Column#32)->Column#18", - "│ └─TableReader_62 1.00 5.17 root data:StreamAgg_45", - "│ └─StreamAgg_45 1.00 49.50 batchCop[tiflash] funcs:count(1)->Column#32", - "│ └─TableFullScan_60 3.00 40.50 batchCop[tiflash] table:t1 keep order:false", - "└─Projection_20(Probe) 3.00 95.82 root 1->Column#28", - " └─Apply_22 3.00 76.02 root CARTESIAN left outer join", - " ├─TableReader_24(Build) 3.00 10.16 root data:TableFullScan_23", - " │ └─TableFullScan_23 3.00 108.00 cop[tikv] table:t keep order:false", - " └─Projection_27(Probe) 3.00 21.95 root 1->Column#26", - " └─Limit_30 3.00 3.35 root offset:0, count:1", - " └─TableReader_38 3.00 3.35 root data:ExchangeSender_37", - " └─ExchangeSender_37 3.00 19.50 mpp[tiflash] ExchangeType: PassThrough", - " └─Limit_36 3.00 19.50 mpp[tiflash] offset:0, count:1", - " └─TableFullScan_35 3.00 19.50 mpp[tiflash] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select /*+ merge_join(t1) */ count(*) from t1 join t2 on t1.a = t2.a", - "Plan": [ - "StreamAgg_14 1.00 59.65 root funcs:count(1)->Column#7", - "└─MergeJoin_32 3.00 50.65 root inner join, left key:test.t1.a, right key:test.t2.a", - " ├─Sort_30(Build) 3.00 20.83 root test.t2.a", - " │ └─TableReader_29 3.00 6.56 root data:Selection_28", - " │ └─Selection_28 3.00 54.00 cop[tiflash] not(isnull(test.t2.a))", - " │ └─TableFullScan_27 3.00 45.00 cop[tiflash] table:t2 keep order:false", - " └─Sort_23(Probe) 3.00 20.83 root test.t1.a", - " └─TableReader_22 3.00 6.56 root data:Selection_21", - " └─Selection_21 3.00 54.00 cop[tiflash] not(isnull(test.t1.a))", - " └─TableFullScan_20 3.00 45.00 cop[tiflash] table:t1 keep order:false" + "StreamAgg_10 1.00 630.77 root funcs:count(1)->Column#7", + "└─HashJoin_40 3.00 481.07 root inner join, equal:[eq(test.t3.a, test.t3.b)]", + " ├─IndexReader_28(Build) 3.00 52.22 root index:IndexFullScan_27", + " │ └─IndexFullScan_27 3.00 593.16 cop[tikv] table:t3, index:c(b) keep order:false", + " └─TableReader_26(Probe) 3.00 59.97 root data:Selection_25", + " └─Selection_25 3.00 709.52 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan_24 3.00 559.82 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", + "Plan": [ + "StreamAgg_15 1.00 51562.56 root funcs:count(1)->Column#7", + "└─TableReader_41 3.00 51412.86 root data:ExchangeSender_40", + " └─ExchangeSender_40 3.00 771087.32 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_37 3.00 771087.32 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_22(Build) 3.00 385610.46 mpp[tiflash] ", + " │ └─ExchangeSender_21 3.00 385466.46 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection_20 3.00 385466.46 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan_19 3.00 385459.26 mpp[tiflash] table:t1 keep order:false", + " └─Selection_24(Probe) 3.00 385466.46 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_23 3.00 385459.26 mpp[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", + "Plan": [ + "StreamAgg_15 1.00 54032.05 root funcs:count(1)->Column#10", + "└─HashJoin_59 3.00 53882.35 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─IndexReader_47(Build) 3.00 52.22 root index:IndexFullScan_46", + " │ └─IndexFullScan_46 3.00 593.16 cop[tikv] table:t3, index:c(b) keep order:false", + " └─TableReader_39(Probe) 3.00 53461.26 root data:ExchangeSender_38", + " └─ExchangeSender_38 3.00 801760.47 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_29 3.00 801760.47 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_35(Build) 3.00 416282.81 mpp[tiflash] ", + " │ └─ExchangeSender_34 3.00 415994.81 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection_33 3.00 415994.81 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan_32 3.00 415980.41 mpp[tiflash] table:t1 keep order:false", + " └─Selection_37(Probe) 3.00 385466.46 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_36 3.00 385459.26 mpp[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select (2) in (select /*+ read_from_storage(tiflash[t1]) */ count(*) from t1) from (select t.b < (select /*+ read_from_storage(tiflash[t2]) */ t.b from t2 limit 1 ) from t3 t) t", + "Plan": [ + "HashJoin_19 3.00 110724.19 root CARTESIAN left outer semi join", + "├─Selection_38(Build) 0.80 24733.71 root eq(2, Column#18)", + "│ └─StreamAgg_45 1.00 24683.81 root funcs:count(1)->Column#18", + "│ └─TableReader_59 3.00 24534.11 root data:TableFullScan_58", + "│ └─TableFullScan_58 3.00 367821.61 cop[tiflash] table:t1 keep order:false", + "└─Projection_20(Probe) 3.00 85919.34 root 1->Column#28", + " └─Apply_22 3.00 85919.34 root CARTESIAN left outer join", + " ├─TableReader_24(Build) 3.00 49.99 root data:TableFullScan_23", + " │ └─TableFullScan_23 3.00 559.82 cop[tikv] table:t keep order:false", + " └─Projection_27(Probe) 3.00 28623.12 root 1->Column#26", + " └─Limit_30 3.00 28623.12 root offset:0, count:1", + " └─TableReader_37 3.00 28623.12 root data:ExchangeSender_36", + " └─ExchangeSender_36 3.00 429293.93 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit_35 3.00 429293.93 mpp[tiflash] offset:0, count:1", + " └─TableFullScan_34 3.00 429293.93 mpp[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ merge_join(t1), read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", + "Plan": [ + "StreamAgg_14 1.00 52064.31 root funcs:count(1)->Column#7", + "└─MergeJoin_26 3.00 51914.61 root inner join, left key:test.t1.a, right key:test.t2.a", + " ├─Sort_24(Build) 3.00 25957.31 root test.t2.a", + " │ └─TableReader_23 3.00 25710.44 root data:Selection_22", + " │ └─Selection_22 3.00 385466.46 cop[tiflash] not(isnull(test.t2.a))", + " │ └─TableFullScan_21 3.00 385459.26 cop[tiflash] table:t2 keep order:false", + " └─Sort_20(Probe) 3.00 25957.31 root test.t1.a", + " └─TableReader_19 3.00 25710.44 root data:Selection_18", + " └─Selection_18 3.00 385466.46 cop[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan_17 3.00 385459.26 cop[tiflash] table:t1 keep order:false" ] } ] diff --git a/planner/funcdep/extract_fd_test.go b/planner/funcdep/extract_fd_test.go index 7e5dca8daf44c..f574eba78a872 100644 --- a/planner/funcdep/extract_fd_test.go +++ b/planner/funcdep/extract_fd_test.go @@ -342,7 +342,7 @@ func TestFDSet_MakeOuterJoin(t *testing.T) { tk.MustExec("set @@session.tidb_enable_new_only_full_group_by_check = 'on';") tk.MustExec("CREATE TABLE X (a INT PRIMARY KEY, b INT, c INT, d INT, e INT)") tk.MustExec("CREATE UNIQUE INDEX uni ON X (b, c)") - tk.MustExec("CREATE TABLE Y (m INT, n INT, p INT, q INT, PRIMARY KEY (m, n))") + tk.MustExec("CREATE TABLE Y (m INT, n INT, p INT, q INT, PRIMARY KEY (m, n) NONCLUSTERED)") tests := []struct { sql string diff --git a/server/conn.go b/server/conn.go index 4912ab030a927..f48e616577830 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1122,6 +1122,7 @@ func (cc *clientConn) Run(ctx context.Context) { startTime := time.Now() err = cc.dispatch(ctx, data) cc.chunkAlloc.Reset() + cc.ctx.GetSessionVars().ClearAlloc() if err != nil { cc.audit(plugin.Error) // tell the plugin API there was a dispatch error if terror.ErrorEqual(err, io.EOF) { @@ -1866,6 +1867,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { sc := cc.ctx.GetSessionVars().StmtCtx prevWarns := sc.GetWarnings() var stmts []ast.StmtNode + cc.ctx.GetSessionVars().SetAlloc(cc.chunkAlloc) if stmts, err = cc.ctx.Parse(ctx, sql); err != nil { return err } @@ -2485,20 +2487,24 @@ func (cc *clientConn) handleChangeUser(ctx context.Context, data []byte) error { if err != nil { return err } - if pluginName != "" { + fakeResp := &handshakeResponse41{ + Auth: pass, + AuthPlugin: pluginName, + Capability: cc.capability, + } + if fakeResp.AuthPlugin != "" { failpoint.Inject("ChangeUserAuthSwitch", func(val failpoint.Value) { failpoint.Return(errors.Errorf("%v", val)) }) - pass, err = cc.checkAuthPlugin(ctx, &handshakeResponse41{ - Auth: pass, - AuthPlugin: pluginName, - Capability: cc.capability, - }) + newpass, err := cc.checkAuthPlugin(ctx, fakeResp) if err != nil { return err } + if len(newpass) > 0 { + fakeResp.Auth = newpass + } } - if err := cc.openSessionAndDoAuth(pass, ""); err != nil { + if err := cc.openSessionAndDoAuth(fakeResp.Auth, fakeResp.AuthPlugin); err != nil { return err } return cc.handleCommonConnectionReset(ctx) diff --git a/server/conn_stmt.go b/server/conn_stmt.go index ef92ea85abaf0..65ad113d6ad5a 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -158,6 +158,10 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e return mysql.NewErrf(mysql.ErrUnknown, "unsupported flag: CursorTypeScrollable", nil) } + if !useCursor { + // not using streaming ,can reuse chunk + cc.ctx.GetSessionVars().SetAlloc(cc.chunkAlloc) + } // skip iteration-count, always 1 pos += 4 @@ -295,6 +299,7 @@ const ( func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err error) { cc.ctx.GetSessionVars().StartTime = time.Now() + cc.ctx.GetSessionVars().ClearAlloc() stmtID, fetchSize, err := parseStmtFetchCmd(data) if err != nil { diff --git a/server/server.go b/server/server.go index 9b2bf6541e8e9..24172f22c8824 100644 --- a/server/server.go +++ b/server/server.go @@ -41,6 +41,7 @@ import ( _ "net/http/pprof" // #nosec G108 "os" "os/user" + "strconv" "sync" "sync/atomic" "time" @@ -250,7 +251,7 @@ func NewServer(cfg *config.Config, driver IDriver) (*Server, error) { } if s.cfg.Host != "" && (s.cfg.Port != 0 || RunInGoTest) { - addr := fmt.Sprintf("%s:%d", s.cfg.Host, s.cfg.Port) + addr := net.JoinHostPort(s.cfg.Host, strconv.Itoa(int(s.cfg.Port))) tcpProto := "tcp" if s.cfg.EnableTCP4Only { tcpProto = "tcp4" diff --git a/session/bench_test.go b/session/bench_test.go index 47cad404d68a5..742f63aa00f01 100644 --- a/session/bench_test.go +++ b/session/bench_test.go @@ -492,6 +492,26 @@ func BenchmarkSort(b *testing.B) { b.StopTimer() } +func BenchmarkSort2(b *testing.B) { + ctx := context.Background() + se, do, st := prepareBenchSession() + defer func() { + se.Close() + do.Close() + st.Close() + }() + prepareSortBenchData(se, "int", "%v", 1000000) + b.ResetTimer() + for i := 0; i < b.N; i++ { + rs, err := se.Execute(ctx, "select * from t order by col") + if err != nil { + b.Fatal(err) + } + readResult(ctx, rs[0], 1000000) + } + b.StopTimer() +} + func BenchmarkJoin(b *testing.B) { ctx := context.Background() se, do, st := prepareBenchSession() diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index 4a92eca674813..53076c82a0d85 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -625,8 +625,8 @@ func TestUpgradeClusteredIndexDefaultValue(t *testing.T) { require.NoError(t, r.Next(context.Background(), req)) require.Equal(t, 1, req.NumRows()) row := req.GetRow(0) - require.Equal(t, "INT_ONLY", row.GetString(0)) - require.Equal(t, "INT_ONLY", row.GetString(1)) + require.Equal(t, "ON", row.GetString(0)) + require.Equal(t, "ON", row.GetString(1)) domV68.Close() } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 9d5fc86387cee..86e3b36dc1f73 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -339,6 +339,9 @@ type StatementContext struct { SavepointName string HasFKCascades bool } + + // useChunkAlloc indicates whether statement use chunk alloc + useChunkAlloc bool } // StmtHints are SessionVars related sql hints. @@ -502,6 +505,21 @@ func (sc *StatementContext) GetResourceGroupTagger() tikvrpc.ResourceGroupTagger } } +// SetUseChunkAlloc set use chunk alloc status +func (sc *StatementContext) SetUseChunkAlloc() { + sc.useChunkAlloc = true +} + +// ClearUseChunkAlloc clear useChunkAlloc status +func (sc *StatementContext) ClearUseChunkAlloc() { + sc.useChunkAlloc = false +} + +// GetUseChunkAllocStatus returns useChunkAlloc status +func (sc *StatementContext) GetUseChunkAllocStatus() bool { + return sc.useChunkAlloc +} + // SetPlanDigest sets the normalized plan and plan digest. func (sc *StatementContext) SetPlanDigest(normalized string, planDigest *parser.Digest) { if planDigest != nil { diff --git a/sessionctx/variable/BUILD.bazel b/sessionctx/variable/BUILD.bazel index 10ecc7d92b030..fa4865079e8bf 100644 --- a/sessionctx/variable/BUILD.bazel +++ b/sessionctx/variable/BUILD.bazel @@ -97,6 +97,7 @@ go_test( "//testkit", "//testkit/testsetup", "//types", + "//util/chunk", "//util/execdetails", "//util/memory", "//util/mock", diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 06a1ac9010823..3e8c9ed0008cc 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1293,6 +1293,72 @@ type SessionVars struct { // OptPrefixIndexSingleScan indicates whether to do some optimizations to avoid double scan for prefix index. // When set to true, `col is (not) null`(`col` is index prefix column) is regarded as index filter rather than table filter. OptPrefixIndexSingleScan bool + + // ChunkPool Several chunks and columns are cached + ChunkPool struct { + Lock sync.Mutex + Alloc chunk.Allocator + } + // EnableReuseCheck indicates request chunk whether use chunk alloc + EnableReuseCheck bool + + // preuseChunkAlloc indicates whether pre statement use chunk alloc + // like select @@last_sql_use_alloc + preUseChunkAlloc bool +} + +// GetNewChunk Attempt to request memory from the chunk pool +// thread safety +func (s *SessionVars) GetNewChunk(fields []*types.FieldType, capacity int) *chunk.Chunk { + //Chunk memory pool is not set + if s.ChunkPool.Alloc == nil { + return chunk.NewChunkWithCapacity(fields, capacity) + } + s.ChunkPool.Lock.Lock() + defer s.ChunkPool.Lock.Unlock() + if s.ChunkPool.Alloc.CheckReuseAllocSize() && (!s.GetUseChunkAlloc()) { + s.StmtCtx.SetUseChunkAlloc() + } + chk := s.ChunkPool.Alloc.Alloc(fields, capacity, capacity) + return chk +} + +// GetNewChunkWithCapacity Attempt to request memory from the chunk pool +// thread safety +func (s *SessionVars) GetNewChunkWithCapacity(fields []*types.FieldType, capacity int, maxCachesize int) *chunk.Chunk { + if s.ChunkPool.Alloc == nil { + return chunk.New(fields, capacity, maxCachesize) + } + s.ChunkPool.Lock.Lock() + defer s.ChunkPool.Lock.Unlock() + if s.ChunkPool.Alloc.CheckReuseAllocSize() && (!s.GetUseChunkAlloc()) { + s.StmtCtx.SetUseChunkAlloc() + } + chk := s.ChunkPool.Alloc.Alloc(fields, capacity, maxCachesize) + return chk +} + +// ExchangeChunkStatus give the status to preUseChunkAlloc +func (s *SessionVars) ExchangeChunkStatus() { + s.preUseChunkAlloc = s.GetUseChunkAlloc() +} + +// GetUseChunkAlloc return useChunkAlloc status +func (s *SessionVars) GetUseChunkAlloc() bool { + return s.StmtCtx.GetUseChunkAllocStatus() +} + +// SetAlloc Attempt to set the buffer pool address +func (s *SessionVars) SetAlloc(alloc chunk.Allocator) { + if !s.EnableReuseCheck { + return + } + s.ChunkPool.Alloc = alloc +} + +// ClearAlloc indicates stop reuse chunk +func (s *SessionVars) ClearAlloc() { + s.ChunkPool.Alloc = nil } // GetPreparedStmtByName returns the prepared statement specified by stmtName. @@ -1587,6 +1653,13 @@ func NewSessionVars(hctx HookContext) *SessionVars { EnableTiFlashReadForWriteStmt: DefTiDBEnableTiFlashReadForWriteStmt, ForeignKeyChecks: DefTiDBForeignKeyChecks, HookContext: hctx, + EnableReuseCheck: DefTiDBEnableReusechunk, + //useChunkAlloc: DefTiDBUseAlloc, + preUseChunkAlloc: DefTiDBUseAlloc, + ChunkPool: struct { + Lock sync.Mutex + Alloc chunk.Allocator + }{Alloc: nil}, } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 91f1394499c97..903ecf1bf4fa8 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -23,10 +23,13 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/parser/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" @@ -422,3 +425,71 @@ func TestHookContext(t *testing.T) { ctx.GetSessionVars().SetSystemVar("testhooksysvar", "test") } + +func TestGetReuseChunk(t *testing.T) { + fieldTypes := []*types.FieldType{ + types.NewFieldTypeBuilder().SetType(mysql.TypeVarchar).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeJSON).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeFloat).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeNewDecimal).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeDouble).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeLonglong).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeDatetime).BuildP(), + } + + sessVars := variable.NewSessionVars(nil) + + // SetAlloc efficient + sessVars.SetAlloc(nil) + require.Nil(t, sessVars.ChunkPool.Alloc) + require.False(t, sessVars.GetUseChunkAlloc()) + // alloc is nil ,Allocate memory from the system + chk1 := sessVars.GetNewChunk(fieldTypes, 10) + require.NotNil(t, chk1) + chk2 := sessVars.GetNewChunkWithCapacity(fieldTypes, 10, 10) + require.NotNil(t, chk2) + + chunkReuseMap := make(map[*chunk.Chunk]struct{}, 14) + columnReuseMap := make(map[*chunk.Column]struct{}, 14) + + alloc := chunk.NewAllocator() + sessVars.EnableReuseCheck = true + sessVars.SetAlloc(alloc) + require.NotNil(t, sessVars.ChunkPool.Alloc) + require.Equal(t, alloc, sessVars.ChunkPool.Alloc) + require.False(t, sessVars.GetUseChunkAlloc()) + + //tries to apply from the cache + initCap := 10 + chk1 = sessVars.GetNewChunk(fieldTypes, initCap) + require.NotNil(t, chk1) + chunkReuseMap[chk1] = struct{}{} + for i := 0; i < chk1.NumCols(); i++ { + columnReuseMap[chk1.Column(i)] = struct{}{} + } + chk2 = sessVars.GetNewChunkWithCapacity(fieldTypes, initCap, initCap) + require.NotNil(t, chk2) + chunkReuseMap[chk2] = struct{}{} + for i := 0; i < chk2.NumCols(); i++ { + columnReuseMap[chk2.Column(i)] = struct{}{} + } + + alloc.Reset() + chkres1 := sessVars.GetNewChunk(fieldTypes, 10) + _, exist := chunkReuseMap[chkres1] + require.True(t, exist) + for i := 0; i < chkres1.NumCols(); i++ { + _, exist := columnReuseMap[chkres1.Column(i)] + require.True(t, exist) + } + chkres2 := sessVars.GetNewChunkWithCapacity(fieldTypes, 10, 10) + require.NotNil(t, chkres2) + _, exist = chunkReuseMap[chkres2] + require.True(t, exist) + for i := 0; i < chkres2.NumCols(); i++ { + _, exist := columnReuseMap[chkres2.Column(i)] + require.True(t, exist) + } + sessVars.ClearAlloc() + require.Nil(t, sessVars.ChunkPool.Alloc) +} diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 0cce50a063e9d..a7acc03734611 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -347,6 +347,9 @@ var defaultSysVars = []*SysVar{ return s.LastPlanReplayerToken, nil }, }, + {Scope: ScopeSession, Name: TiDBUseAlloc, Value: BoolToOnOff(DefTiDBUseAlloc), Type: TypeBool, ReadOnly: true, GetSession: func(s *SessionVars) (string, error) { + return BoolToOnOff(s.preUseChunkAlloc), nil + }}, /* The system variables below have INSTANCE scope */ {Scope: ScopeInstance, Name: TiDBLogFileMaxDays, Value: strconv.Itoa(config.GetGlobalConfig().Log.File.MaxDays), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { maxAge, err := strconv.ParseInt(val, 10, 32) @@ -1661,7 +1664,7 @@ var defaultSysVars = []*SysVar{ s.AllowAutoRandExplicitInsert = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableClusteredIndex, Value: IntOnly, Type: TypeEnum, PossibleValues: []string{Off, On, IntOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableClusteredIndex, Value: On, Type: TypeEnum, PossibleValues: []string{Off, On, IntOnly}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if normalizedValue == IntOnly { vars.StmtCtx.AppendWarning(errWarnDeprecatedSyntax.FastGenByArgs(normalizedValue, fmt.Sprintf("'%s' or '%s'", On, Off))) } @@ -1968,6 +1971,7 @@ var defaultSysVars = []*SysVar{ return nil }, }, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptPrefixIndexSingleScan, Value: BoolToOnOff(DefTiDBOptPrefixIndexSingleScan), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.OptPrefixIndexSingleScan = TiDBOptOn(val) return nil @@ -1989,6 +1993,11 @@ var defaultSysVars = []*SysVar{ s.EnableExternalTSRead = TiDBOptOn(val) return nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableReusechunk, Value: BoolToOnOff(DefTiDBEnableReusechunk), Type: TypeBool, + SetSession: func(s *SessionVars, val string) error { + s.EnableReuseCheck = TiDBOptOn(val) + return nil + }}, } // FeedbackProbability points to the FeedbackProbability in statistics package. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 97dc4bda88456..db6a39e3280a1 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -252,6 +252,9 @@ const ( // TiDBEnableTiFlashReadForWriteStmt indicates whether to enable TiFlash to read for write statements. TiDBEnableTiFlashReadForWriteStmt = "tidb_enable_tiflash_read_for_write_stmt" + + // TiDBUseAlloc indicates whether the last statement used chunk alloc + TiDBUseAlloc = "last_sql_use_alloc" ) // TiDB system variable names that both in session and global scope. @@ -764,6 +767,9 @@ const ( // TiDBEnableExternalTSRead indicates whether to enable read through an external ts TiDBEnableExternalTSRead = "tidb_enable_external_ts_read" + + // TiDBEnableReusechunk indicates whether to enable chunk alloc + TiDBEnableReusechunk = "tidb_enable_reuse_chunk" ) // TiDB vars that have only global scope @@ -1082,6 +1088,8 @@ const ( DefTiDBOptPrefixIndexSingleScan = true DefTiDBExternalTS = 0 DefTiDBEnableExternalTSRead = false + DefTiDBEnableReusechunk = true + DefTiDBUseAlloc = false ) // Process global variables. diff --git a/testkit/BUILD.bazel b/testkit/BUILD.bazel index 2ab1fb67ada3e..39cedb2ce0b53 100644 --- a/testkit/BUILD.bazel +++ b/testkit/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//store/mockstore", "//util", "//util/breakpoint", + "//util/chunk", "//util/gctuner", "//util/sqlexec", "@com_github_pingcap_errors//:errors", diff --git a/testkit/testkit.go b/testkit/testkit.go index 345746e6c7ef2..6952f4c36d484 100644 --- a/testkit/testkit.go +++ b/testkit/testkit.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -48,6 +49,7 @@ type TestKit struct { t testing.TB store kv.Storage session session.Session + alloc chunk.Allocator } // NewTestKit returns a new *TestKit. @@ -57,6 +59,7 @@ func NewTestKit(t testing.TB, store kv.Storage) *TestKit { assert: assert.New(t), t: t, store: store, + alloc: chunk.NewAllocator(), } tk.RefreshSession() @@ -86,6 +89,7 @@ func NewTestKitWithSession(t testing.TB, store kv.Storage, se session.Session) * t: t, store: store, session: se, + alloc: chunk.NewAllocator(), } } @@ -110,6 +114,12 @@ func (tk *TestKit) Session() session.Session { // MustExec executes a sql statement and asserts nil error. func (tk *TestKit) MustExec(sql string, args ...interface{}) { + defer func() { + tk.Session().GetSessionVars().ClearAlloc() + if tk.alloc != nil { + tk.alloc.Reset() + } + }() tk.MustExecWithContext(context.Background(), sql, args...) } @@ -127,6 +137,12 @@ func (tk *TestKit) MustExecWithContext(ctx context.Context, sql string, args ... // MustQuery query the statements and returns result rows. // If expected result is set it asserts the query result equals expected result. func (tk *TestKit) MustQuery(sql string, args ...interface{}) *Result { + defer func() { + tk.Session().GetSessionVars().ClearAlloc() + if tk.alloc != nil { + tk.alloc.Reset() + } + }() return tk.MustQueryWithContext(context.Background(), sql, args...) } @@ -269,6 +285,7 @@ func (tk *TestKit) ExecWithContext(ctx context.Context, sql string, args ...inte } warns := sc.GetWarnings() parserWarns := warns[len(prevWarns):] + tk.Session().GetSessionVars().SetAlloc(tk.alloc) var rs0 sqlexec.RecordSet for i, stmt := range stmts { var rs sqlexec.RecordSet @@ -297,6 +314,7 @@ func (tk *TestKit) ExecWithContext(ctx context.Context, sql string, args ...inte return nil, errors.Trace(err) } params := expression.Args2Expressions4Test(args...) + tk.Session().GetSessionVars().SetAlloc(tk.alloc) rs, err := tk.session.ExecutePreparedStmt(ctx, stmtID, params) if err != nil { return rs, errors.Trace(err) diff --git a/tests/realtikvtest/pessimistictest/pessimistic_test.go b/tests/realtikvtest/pessimistictest/pessimistic_test.go index 60dead95abf1a..5e2da1d200651 100644 --- a/tests/realtikvtest/pessimistictest/pessimistic_test.go +++ b/tests/realtikvtest/pessimistictest/pessimistic_test.go @@ -3039,6 +3039,12 @@ func TestAmendForColumnChange(t *testing.T) { } // Start a pessimistic transaction for partition table, the amend should fail. + if i == 5 { + // alter table t_part modify column c_int bigint(20) default 100 + // Unsupported modify column: can't change the partitioning column, since it would require reorganize all partitions + // Skip this case + continue + } tk.MustExec("begin pessimistic") tk.MustExec(`insert into t_part values(5, "555", "2000-01-05", "2020-01-05", "5.5", "555.555", 5.5)`) tk2.MustExec(colChangeFunc(true, i)) diff --git a/tidb-binlog/driver/example/kafkaReader/pom.xml b/tidb-binlog/driver/example/kafkaReader/pom.xml index d71b0558e6f5a..ef87a96a80477 100644 --- a/tidb-binlog/driver/example/kafkaReader/pom.xml +++ b/tidb-binlog/driver/example/kafkaReader/pom.xml @@ -29,7 +29,7 @@ org.apache.kafka kafka_2.12 - 3.1.1 + 3.2.3 diff --git a/tidb-server/BUILD.bazel b/tidb-server/BUILD.bazel index b1ee1d700d8bc..960f68a2b597d 100644 --- a/tidb-server/BUILD.bazel +++ b/tidb-server/BUILD.bazel @@ -33,6 +33,7 @@ go_library( "//store/mockstore/unistore/metrics", "//tidb-binlog/pump_client", "//util", + "//util/chunk", "//util/cpuprofile", "//util/deadlockhistory", "//util/disk", diff --git a/tidb-server/main.go b/tidb-server/main.go index 056814222775e..a6e80cbbfe4b6 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -58,6 +58,7 @@ import ( uni_metrics "github.com/pingcap/tidb/store/mockstore/unistore/metrics" pumpcli "github.com/pingcap/tidb/tidb-binlog/pump_client" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/cpuprofile" "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/disk" @@ -715,6 +716,7 @@ func setGlobalVars() { deadlockhistory.GlobalDeadlockHistory.Resize(cfg.PessimisticTxn.DeadlockHistoryCapacity) txninfo.Recorder.ResizeSummaries(cfg.TrxSummary.TransactionSummaryCapacity) txninfo.Recorder.SetMinDuration(time.Duration(cfg.TrxSummary.TransactionIDDigestMinDuration) * time.Millisecond) + chunk.InitChunkAllocSize(cfg.TiDBMaxReuseChunk, cfg.TiDBMaxReuseColumn) } func setupLog() { diff --git a/util/chunk/alloc.go b/util/chunk/alloc.go index 4b706e6297431..44fbb126a4989 100644 --- a/util/chunk/alloc.go +++ b/util/chunk/alloc.go @@ -15,6 +15,8 @@ package chunk import ( + "math" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mathutil" ) @@ -24,18 +26,37 @@ import ( // and Alloc() allocates from the pool. type Allocator interface { Alloc(fields []*types.FieldType, capacity, maxChunkSize int) *Chunk + CheckReuseAllocSize() bool Reset() } +var maxFreeChunks = 64 +var maxFreeColumnsPerType = 256 + +// InitChunkAllocSize init the maximum cache size +func InitChunkAllocSize(setMaxFreeChunks, setMaxFreeColumns uint32) { + if setMaxFreeChunks > math.MaxInt32 { + setMaxFreeChunks = math.MaxInt32 + } + if setMaxFreeColumns > math.MaxInt32 { + setMaxFreeColumns = math.MaxInt32 + } + maxFreeChunks = int(setMaxFreeChunks) + maxFreeColumnsPerType = int(setMaxFreeColumns) +} + // NewAllocator creates an Allocator. func NewAllocator() *allocator { - ret := &allocator{} + ret := &allocator{freeChunk: maxFreeChunks} ret.columnAlloc.init() return ret } var _ Allocator = &allocator{} +// MaxCachedLen Maximum cacheable length +var MaxCachedLen = 16 * 1024 + // allocator try to reuse objects. // It uses `poolColumnAllocator` to alloc chunk column objects. // The allocated chunks are recorded in the `allocated` array. @@ -45,6 +66,27 @@ type allocator struct { allocated []*Chunk free []*Chunk columnAlloc poolColumnAllocator + freeChunk int +} + +// columnList keep column +type columnList struct { + freeColumns []*Column + allocColumns []*Column +} + +func (cList *columnList) add(col *Column) { + cList.freeColumns = append(cList.freeColumns, col) +} + +// columnList Len Get the number of elements in the list +func (cList *columnList) Len() int { + return len(cList.freeColumns) + len(cList.allocColumns) +} + +// CheckReuseAllocSize return whether the cache can cache objects +func (a *allocator) CheckReuseAllocSize() bool { + return a.freeChunk > 0 || a.columnAlloc.freeColumnsPerType > 0 } // Alloc implements the Allocator interface. @@ -66,53 +108,80 @@ func (a *allocator) Alloc(fields []*types.FieldType, capacity, maxChunkSize int) chk.columns = append(chk.columns, a.columnAlloc.NewColumn(f, chk.capacity)) } - a.allocated = append(a.allocated, chk) + //avoid OOM + if a.freeChunk > len(a.allocated) { + a.allocated = append(a.allocated, chk) + } + return chk } -const ( - maxFreeChunks = 64 - maxFreeColumnsPerType = 256 -) - // Reset implements the Allocator interface. func (a *allocator) Reset() { - a.free = a.free[:0] for i, chk := range a.allocated { a.allocated[i] = nil - // Decouple chunk into chunk column objects and put them back to the column allocator for reuse. - for _, col := range chk.columns { - a.columnAlloc.put(col) - } - // Reset the chunk and put it to the free list for reuse. chk.resetForReuse() - - if len(a.free) < maxFreeChunks { // Don't cache too much data. + if len(a.free) < a.freeChunk { // Don't cache too much data. a.free = append(a.free, chk) } } a.allocated = a.allocated[:0] + + //column objects and put them to the column allocator for reuse. + for _, pool := range a.columnAlloc.pool { + for _, col := range pool.allocColumns { + if (len(pool.freeColumns) < a.columnAlloc.freeColumnsPerType) && (!col.avoidReusing) && (cap(col.data) < MaxCachedLen) { + col.reset() + pool.freeColumns = append(pool.freeColumns, col) + } + } + pool.allocColumns = pool.allocColumns[:0] + } } var _ ColumnAllocator = &poolColumnAllocator{} type poolColumnAllocator struct { - pool map[int]freeList + pool map[int]*columnList + freeColumnsPerType int } // poolColumnAllocator implements the ColumnAllocator interface. func (alloc *poolColumnAllocator) NewColumn(ft *types.FieldType, count int) *Column { typeSize := getFixedLen(ft) + col := alloc.NewSizeColumn(typeSize, count) + + //column objects and put them back to the allocated column . + alloc.put(col) + return col +} + +// poolColumnAllocator implements the ColumnAllocator interface. +func (alloc *poolColumnAllocator) NewSizeColumn(typeSize int, count int) *Column { l := alloc.pool[typeSize] if l != nil && !l.empty() { col := l.pop() + + if cap(col.data) < count { + col = newColumn(typeSize, count) + } return col } return newColumn(typeSize, count) } +func (cList *columnList) pop() *Column { + if len(cList.freeColumns) == 0 { + return nil + } + col := cList.freeColumns[len(cList.freeColumns)-1] + cList.freeColumns = cList.freeColumns[:len(cList.freeColumns)-1] + return col +} + func (alloc *poolColumnAllocator) init() { - alloc.pool = make(map[int]freeList) + alloc.pool = make(map[int]*columnList) + alloc.freeColumnsPerType = maxFreeColumnsPerType } func (alloc *poolColumnAllocator) put(col *Column) { @@ -120,16 +189,20 @@ func (alloc *poolColumnAllocator) put(col *Column) { return } typeSize := col.typeSize() - if typeSize <= 0 { + if typeSize <= 0 && typeSize != varElemLen { return } l := alloc.pool[typeSize] if l == nil { - l = make(map[*Column]struct{}, 8) + l = &columnList{freeColumns: nil, allocColumns: nil} + l.freeColumns = make([]*Column, 0, alloc.freeColumnsPerType) + l.allocColumns = make([]*Column, 0, alloc.freeColumnsPerType) alloc.pool[typeSize] = l } - l.push(col) + if len(l.allocColumns) < alloc.freeColumnsPerType { + l.push(col) + } } // freeList is defined as a map, rather than a list, because when recycling chunk @@ -137,22 +210,12 @@ func (alloc *poolColumnAllocator) put(col *Column) { // reference to the others. type freeList map[*Column]struct{} -func (l freeList) empty() bool { - return len(l) == 0 +func (cList *columnList) empty() bool { + return len(cList.freeColumns) == 0 } -func (l freeList) pop() *Column { - for k := range l { - delete(l, k) - return k - } - return nil -} - -func (l freeList) push(c *Column) { - if len(l) >= maxFreeColumnsPerType { - // Don't cache too much to save memory. - return +func (cList *columnList) push(col *Column) { + if cap(col.data) < MaxCachedLen { + cList.allocColumns = append(cList.allocColumns, col) } - l[c] = struct{}{} } diff --git a/util/chunk/alloc_test.go b/util/chunk/alloc_test.go index 33b52590eab7c..edad5e3008e77 100644 --- a/util/chunk/alloc_test.go +++ b/util/chunk/alloc_test.go @@ -114,7 +114,7 @@ func TestColumnAllocator(t *testing.T) { // Check max column size. freeList := alloc1.pool[getFixedLen(ft)] require.NotNil(t, freeList) - require.Len(t, freeList, maxFreeColumnsPerType) + require.Equal(t, freeList.Len(), maxFreeColumnsPerType) } func TestNoDuplicateColumnReuse(t *testing.T) { @@ -202,3 +202,71 @@ func TestAvoidColumnReuse(t *testing.T) { require.True(t, col.avoidReusing) } } + +func TestColumnAllocatorLimit(t *testing.T) { + fieldTypes := []*types.FieldType{ + types.NewFieldTypeBuilder().SetType(mysql.TypeVarchar).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeJSON).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeFloat).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeNewDecimal).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeDouble).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeLonglong).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeDatetime).BuildP(), + } + + //set cache size + InitChunkAllocSize(10, 20) + alloc := NewAllocator() + require.True(t, alloc.CheckReuseAllocSize()) + for i := 0; i < maxFreeChunks+10; i++ { + alloc.Alloc(fieldTypes, 5, 10) + } + alloc.Reset() + require.Equal(t, len(alloc.free), 10) + for _, p := range alloc.columnAlloc.pool { + require.True(t, (p.Len() <= 20)) + } + + //Reduce capacity + InitChunkAllocSize(5, 10) + alloc = NewAllocator() + for i := 0; i < maxFreeChunks+10; i++ { + alloc.Alloc(fieldTypes, 5, 10) + } + alloc.Reset() + require.Equal(t, len(alloc.free), 5) + for _, p := range alloc.columnAlloc.pool { + require.True(t, (p.Len() <= 10)) + } + + //increase capacity + InitChunkAllocSize(50, 100) + alloc = NewAllocator() + for i := 0; i < maxFreeChunks+10; i++ { + alloc.Alloc(fieldTypes, 5, 10) + } + alloc.Reset() + require.Equal(t, len(alloc.free), 50) + for _, p := range alloc.columnAlloc.pool { + require.True(t, (p.Len() <= 100)) + } + + //long characters are not cached + alloc = NewAllocator() + rs := alloc.Alloc([]*types.FieldType{types.NewFieldTypeBuilder().SetType(mysql.TypeVarchar).BuildP()}, 1024, 1024) + nu := len(alloc.columnAlloc.pool[varElemLen].allocColumns) + require.Equal(t, nu, 1) + for _, col := range rs.columns { + for i := 0; i < 20480; i++ { + col.data = append(col.data, byte('a')) + } + } + alloc.Reset() + for _, p := range alloc.columnAlloc.pool { + require.True(t, (p.Len() == 0)) + } + + InitChunkAllocSize(0, 0) + alloc = NewAllocator() + require.False(t, alloc.CheckReuseAllocSize()) +} diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index b08dbdf63aeb2..d40a9846c47ab 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -444,6 +444,10 @@ type SortedRowContainer struct { actionSpill *SortAndSpillDiskAction memTracker *memory.Tracker + + // Sort is a time-consuming operation, we need to set a checkpoint to detect + // the outside signal periodically. + timesOfRowCompare uint } // NewSortedRowContainer creates a new SortedRowContainer in memory. @@ -481,8 +485,22 @@ func (c *SortedRowContainer) lessRow(rowI, rowJ Row) bool { return false } +// SignalCheckpointForSort indicates the times of row comparation that a signal detection will be triggered. +const SignalCheckpointForSort uint = 10240 + // keyColumnsLess is the less function for key columns. func (c *SortedRowContainer) keyColumnsLess(i, j int) bool { + if c.timesOfRowCompare >= SignalCheckpointForSort { + // Trigger Consume for checking the NeedKill signal + c.memTracker.Consume(1) + c.timesOfRowCompare = 0 + } + failpoint.Inject("SignalCheckpointForSort", func(val failpoint.Value) { + if val.(bool) { + c.timesOfRowCompare += 1024 + } + }) + c.timesOfRowCompare++ rowI := c.m.records.inMemory.GetRow(c.ptrM.rowPtrs[i]) rowJ := c.m.records.inMemory.GetRow(c.ptrM.rowPtrs[j]) return c.lessRow(rowI, rowJ)