diff --git a/DEPS.bzl b/DEPS.bzl index dbd1edbaf98d6..686aef1f3c028 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2923,8 +2923,8 @@ def go_deps(): name = "com_github_pingcap_kvproto", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/kvproto", - sum = "h1:46ZD6xzQWJ8Jkeal/U7SqkX030Mgs8DAn6QV/9zbqOQ=", - version = "v0.0.0-20221130022225-6c56ac56fe5f", + sum = "h1:v0Z0nC0knwWHn3e9br8EMNfLBB14QDULn142UGjiTMQ=", + version = "v0.0.0-20221213093948-9ccc6beaf0aa", ) go_repository( name = "com_github_pingcap_log", @@ -3523,12 +3523,20 @@ def go_deps(): sum = "h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ=", version = "v0.0.0-20181126055449-889f96f722a2", ) + go_repository( + name = "com_github_tiancaiamao_gp", + build_file_proto_mode = "disable", + importpath = "github.com/tiancaiamao/gp", + sum = "h1:4RNtqw1/tW67qP9fFgfQpTVd7DrfkaAWu4vsC18QmBo=", + version = "v0.0.0-20221221095600-1a473d1f9b4b", + ) + go_repository( name = "com_github_tikv_client_go_v2", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/client-go/v2", - sum = "h1:/glZOHs/K2pkCioDVae+aThUHFYRYQkEgY4NUTgfh+s=", - version = "v2.0.3", + sum = "h1:m6glgBGCIds9QURbk8Mn+8mjLKDcv6nWrNwYh92fydQ=", + version = "v2.0.4-0.20221226080148-018c59dbd837", ) go_repository( name = "com_github_tikv_pd_client", @@ -4432,8 +4440,8 @@ def go_deps(): name = "org_golang_x_oauth2", build_file_proto_mode = "disable_global", importpath = "golang.org/x/oauth2", - sum = "h1:GtQkldQ9m7yvzCL1V+LrYow3Khe0eJH0w7RbX/VbaIU=", - version = "v0.2.0", + sum = "h1:6l90koy8/LaBLmLu8jpHeHexzMwEita0zFfYlggy2F8=", + version = "v0.3.0", ) go_repository( name = "org_golang_x_sync", diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 907b6e94ccde0..59919e2b5ad85 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -533,6 +533,15 @@ func (h *BindHandle) SetBindRecordStatus(originalSQL string, binding *Binding, n return } +// SetBindRecordStatusByDigest set a BindRecord's status to the storage and bind cache. +func (h *BindHandle) SetBindRecordStatusByDigest(newStatus, sqlDigest string) (ok bool, err error) { + oldRecord, err := h.GetBindRecordBySQLDigest(sqlDigest) + if err != nil { + return false, err + } + return h.SetBindRecordStatus(oldRecord.OriginalSQL, nil, newStatus) +} + // GCBindRecord physically removes the deleted bind records in mysql.bind_info. func (h *BindHandle) GCBindRecord() (err error) { h.bindInfo.Lock() diff --git a/br/pkg/gluetidb/glue.go b/br/pkg/gluetidb/glue.go index 3b1675921afd3..19c3ef1c4c460 100644 --- a/br/pkg/gluetidb/glue.go +++ b/br/pkg/gluetidb/glue.go @@ -203,9 +203,32 @@ func (gs *tidbSession) CreatePlacementPolicy(ctx context.Context, policy *model. return d.CreatePlacementPolicyWithInfo(gs.se, policy, ddl.OnExistIgnore) } +// SplitBatchCreateTable provide a way to split batch into small batch when batch size is large than 6 MB. +// The raft entry has limit size of 6 MB, a batch of CreateTables may hit this limitation +// TODO: shall query string be set for each split batch create, it looks does not matter if we set once for all. +func (gs *tidbSession) SplitBatchCreateTable(schema model.CIStr, info []*model.TableInfo, cs ...ddl.CreateTableWithInfoConfigurier) error { + var err error + d := domain.GetDomain(gs.se).DDL() + if err = d.BatchCreateTableWithInfo(gs.se, schema, info, append(cs, ddl.OnExistIgnore)...); kv.ErrEntryTooLarge.Equal(err) { + if len(info) == 1 { + return err + } + mid := len(info) / 2 + err = gs.SplitBatchCreateTable(schema, info[:mid]) + if err != nil { + return err + } + err = gs.SplitBatchCreateTable(schema, info[mid:]) + if err != nil { + return err + } + return nil + } + return err +} + // CreateTables implements glue.BatchCreateTableSession. func (gs *tidbSession) CreateTables(ctx context.Context, tables map[string][]*model.TableInfo, cs ...ddl.CreateTableWithInfoConfigurier) error { - d := domain.GetDomain(gs.se).DDL() var dbName model.CIStr // Disable foreign key check when batch create tables. @@ -233,8 +256,7 @@ func (gs *tidbSession) CreateTables(ctx context.Context, tables map[string][]*mo cloneTables = append(cloneTables, table) } gs.se.SetValue(sessionctx.QueryString, queryBuilder.String()) - err := d.BatchCreateTableWithInfo(gs.se, dbName, cloneTables, append(cs, ddl.OnExistIgnore)...) - if err != nil { + if err := gs.SplitBatchCreateTable(dbName, cloneTables); err != nil { //It is possible to failure when TiDB does not support model.ActionCreateTables. //In this circumstance, BatchCreateTableWithInfo returns errno.ErrInvalidDDLJob, //we fall back to old way that creating table one by one diff --git a/br/pkg/lightning/restore/tidb.go b/br/pkg/lightning/restore/tidb.go index ddc8bb6c5ff19..00d49af646e3c 100644 --- a/br/pkg/lightning/restore/tidb.go +++ b/br/pkg/lightning/restore/tidb.go @@ -95,8 +95,10 @@ func DBFromConfig(ctx context.Context, dsn config.DBStore) (*sql.DB, error) { "tidb_opt_write_row_id": "1", // always set auto-commit to ON "autocommit": "1", - // alway set transaction mode to optimistic + // always set transaction mode to optimistic "tidb_txn_mode": "optimistic", + // disable foreign key checks + "foreign_key_checks": "0", } if dsn.Vars != nil { @@ -143,47 +145,6 @@ func (timgr *TiDBManager) Close() { timgr.db.Close() } -func InitSchema(ctx context.Context, g glue.Glue, database string, tablesSchema map[string]string) error { - logger := log.FromContext(ctx).With(zap.String("db", database)) - sqlExecutor := g.GetSQLExecutor() - - var createDatabase strings.Builder - createDatabase.WriteString("CREATE DATABASE IF NOT EXISTS ") - common.WriteMySQLIdentifier(&createDatabase, database) - err := sqlExecutor.ExecuteWithLog(ctx, createDatabase.String(), "create database", logger) - if err != nil { - return errors.Trace(err) - } - - task := logger.Begin(zap.InfoLevel, "create tables") - var sqlCreateStmts []string -loopCreate: - for tbl, sqlCreateTable := range tablesSchema { - task.Debug("create table", zap.String("schema", sqlCreateTable)) - - sqlCreateStmts, err = createIfNotExistsStmt(g.GetParser(), sqlCreateTable, database, tbl) - if err != nil { - break - } - - // TODO: maybe we should put these createStems into a transaction - for _, s := range sqlCreateStmts { - err = sqlExecutor.ExecuteWithLog( - ctx, - s, - "create table", - logger.With(zap.String("table", common.UniqueTable(database, tbl))), - ) - if err != nil { - break loopCreate - } - } - } - task.End(zap.ErrorLevel, err) - - return errors.Trace(err) -} - func createIfNotExistsStmt(p *parser.Parser, createTable, dbName, tblName string) ([]string, error) { stmts, _, err := p.ParseSQL(createTable) if err != nil { @@ -199,6 +160,9 @@ func createIfNotExistsStmt(p *parser.Parser, createTable, dbName, tblName string case *ast.CreateDatabaseStmt: node.Name = model.NewCIStr(dbName) node.IfNotExists = true + case *ast.DropDatabaseStmt: + node.Name = model.NewCIStr(dbName) + node.IfExists = true case *ast.CreateTableStmt: node.Table.Schema = model.NewCIStr(dbName) node.Table.Name = model.NewCIStr(tblName) diff --git a/br/pkg/lightning/restore/tidb_test.go b/br/pkg/lightning/restore/tidb_test.go index 9b204b2da22b1..a3710d822d2dd 100644 --- a/br/pkg/lightning/restore/tidb_test.go +++ b/br/pkg/lightning/restore/tidb_test.go @@ -165,97 +165,6 @@ func TestCreateTableIfNotExistsStmt(t *testing.T) { `, "m")) } -func TestInitSchema(t *testing.T) { - s := newTiDBSuite(t) - ctx := context.Background() - - s.mockDB. - ExpectExec("CREATE DATABASE IF NOT EXISTS `db`"). - WillReturnResult(sqlmock.NewResult(1, 1)) - s.mockDB. - ExpectExec("\\QCREATE TABLE IF NOT EXISTS `db`.`t1` (`a` INT PRIMARY KEY,`b` VARCHAR(200));\\E"). - WillReturnResult(sqlmock.NewResult(2, 1)) - s.mockDB. - ExpectExec("\\QSET @@SESSION.`FOREIGN_KEY_CHECKS`=0;\\E"). - WillReturnResult(sqlmock.NewResult(0, 0)) - s.mockDB. - ExpectExec("\\QCREATE TABLE IF NOT EXISTS `db`.`t2` (`xx` TEXT) AUTO_INCREMENT = 11203;\\E"). - WillReturnResult(sqlmock.NewResult(2, 1)) - s.mockDB. - ExpectClose() - - s.mockDB.MatchExpectationsInOrder(false) // maps are unordered. - err := InitSchema(ctx, s.tiGlue, "db", map[string]string{ - "t1": "create table t1 (a int primary key, b varchar(200));", - "t2": "/*!40014 SET FOREIGN_KEY_CHECKS=0*/;CREATE TABLE `db`.`t2` (xx TEXT) AUTO_INCREMENT=11203;", - }) - s.mockDB.MatchExpectationsInOrder(true) - require.NoError(t, err) -} - -func TestInitSchemaSyntaxError(t *testing.T) { - s := newTiDBSuite(t) - ctx := context.Background() - - s.mockDB. - ExpectExec("CREATE DATABASE IF NOT EXISTS `db`"). - WillReturnResult(sqlmock.NewResult(1, 1)) - s.mockDB. - ExpectClose() - - err := InitSchema(ctx, s.tiGlue, "db", map[string]string{ - "t1": "create table `t1` with invalid syntax;", - }) - require.Error(t, err) -} - -func TestInitSchemaErrorLost(t *testing.T) { - s := newTiDBSuite(t) - ctx := context.Background() - - s.mockDB. - ExpectExec("CREATE DATABASE IF NOT EXISTS `db`"). - WillReturnResult(sqlmock.NewResult(1, 1)) - - s.mockDB. - ExpectExec("CREATE TABLE IF NOT EXISTS.*"). - WillReturnError(&mysql.MySQLError{ - Number: tmysql.ErrTooBigFieldlength, - Message: "Column length too big", - }) - - s.mockDB. - ExpectClose() - - err := InitSchema(ctx, s.tiGlue, "db", map[string]string{ - "t1": "create table `t1` (a int);", - "t2": "create table t2 (a int primary key, b varchar(200));", - }) - require.Regexp(t, ".*Column length too big.*", err.Error()) -} - -func TestInitSchemaUnsupportedSchemaError(t *testing.T) { - s := newTiDBSuite(t) - ctx := context.Background() - - s.mockDB. - ExpectExec("CREATE DATABASE IF NOT EXISTS `db`"). - WillReturnResult(sqlmock.NewResult(1, 1)) - s.mockDB. - ExpectExec("CREATE TABLE IF NOT EXISTS `db`.`t1`.*"). - WillReturnError(&mysql.MySQLError{ - Number: tmysql.ErrTooBigFieldlength, - Message: "Column length too big", - }) - s.mockDB. - ExpectClose() - - err := InitSchema(ctx, s.tiGlue, "db", map[string]string{ - "t1": "create table `t1` (a VARCHAR(999999999));", - }) - require.Regexp(t, ".*Column length too big.*", err.Error()) -} - func TestDropTable(t *testing.T) { s := newTiDBSuite(t) ctx := context.Background() diff --git a/br/pkg/restore/db.go b/br/pkg/restore/db.go index ae62162c3e890..132e3294f1617 100644 --- a/br/pkg/restore/db.go +++ b/br/pkg/restore/db.go @@ -284,7 +284,7 @@ func (db *DB) tableIDAllocFilter() ddl.AllocTableIDIf { if db.preallocedIDs == nil { return true } - prealloced := db.preallocedIDs.Prealloced(ti.ID) + prealloced := db.preallocedIDs.PreallocedFor(ti) if prealloced { log.Info("reusing table ID", zap.Stringer("table", ti.Name)) } diff --git a/br/pkg/restore/prealloc_table_id/BUILD.bazel b/br/pkg/restore/prealloc_table_id/BUILD.bazel index 8ce80b039178a..cfdb0432fd446 100644 --- a/br/pkg/restore/prealloc_table_id/BUILD.bazel +++ b/br/pkg/restore/prealloc_table_id/BUILD.bazel @@ -5,7 +5,10 @@ go_library( srcs = ["alloc.go"], importpath = "github.com/pingcap/tidb/br/pkg/restore/prealloc_table_id", visibility = ["//visibility:public"], - deps = ["//br/pkg/metautil"], + deps = [ + "//br/pkg/metautil", + "//parser/model", + ], ) go_test( diff --git a/br/pkg/restore/prealloc_table_id/alloc.go b/br/pkg/restore/prealloc_table_id/alloc.go index 9232ed84a8fc8..8554de5e9891b 100644 --- a/br/pkg/restore/prealloc_table_id/alloc.go +++ b/br/pkg/restore/prealloc_table_id/alloc.go @@ -7,6 +7,7 @@ import ( "math" "github.com/pingcap/tidb/br/pkg/metautil" + "github.com/pingcap/tidb/parser/model" ) const ( @@ -48,6 +49,14 @@ func New(tables []*metautil.Table) *PreallocIDs { if t.Info.ID > max && t.Info.ID < insaneTableIDThreshold { max = t.Info.ID } + + if t.Info.Partition != nil && t.Info.Partition.Definitions != nil { + for _, part := range t.Info.Partition.Definitions { + if part.ID > max && part.ID < insaneTableIDThreshold { + max = part.ID + } + } + } } return &PreallocIDs{ end: max + 1, @@ -86,3 +95,17 @@ func (p *PreallocIDs) Alloc(m Allocator) error { func (p *PreallocIDs) Prealloced(tid int64) bool { return p.allocedFrom <= tid && tid < p.end } + +func (p *PreallocIDs) PreallocedFor(ti *model.TableInfo) bool { + if !p.Prealloced(ti.ID) { + return false + } + if ti.Partition != nil && ti.Partition.Definitions != nil { + for _, part := range ti.Partition.Definitions { + if !p.Prealloced(part.ID) { + return false + } + } + } + return true +} diff --git a/br/pkg/restore/prealloc_table_id/alloc_test.go b/br/pkg/restore/prealloc_table_id/alloc_test.go index 8cf6b95fb070e..c1c3f018a2de8 100644 --- a/br/pkg/restore/prealloc_table_id/alloc_test.go +++ b/br/pkg/restore/prealloc_table_id/alloc_test.go @@ -27,6 +27,7 @@ func (t *testAllocator) AdvanceGlobalIDs(n int) (int64, error) { func TestAllocator(t *testing.T) { type Case struct { tableIDs []int64 + partitions map[int64][]int64 hasAllocatedTo int64 successfullyAllocated []int64 shouldAllocatedTo int64 @@ -57,16 +58,41 @@ func TestAllocator(t *testing.T) { successfullyAllocated: []int64{5, 6}, shouldAllocatedTo: 7, }, + { + tableIDs: []int64{1, 2, 5, 6, 7}, + hasAllocatedTo: 6, + successfullyAllocated: []int64{6, 7}, + shouldAllocatedTo: 13, + partitions: map[int64][]int64{ + 7: {8, 9, 10, 11, 12}, + }, + }, + { + tableIDs: []int64{1, 2, 5, 6, 7, 13}, + hasAllocatedTo: 9, + successfullyAllocated: []int64{13}, + shouldAllocatedTo: 14, + partitions: map[int64][]int64{ + 7: {8, 9, 10, 11, 12}, + }, + }, } run := func(t *testing.T, c Case) { tables := make([]*metautil.Table, 0, len(c.tableIDs)) for _, id := range c.tableIDs { - tables = append(tables, &metautil.Table{ + table := metautil.Table{ Info: &model.TableInfo{ - ID: id, + ID: id, + Partition: &model.PartitionInfo{}, }, - }) + } + if c.partitions != nil { + for _, part := range c.partitions[id] { + table.Info.Partition.Definitions = append(table.Info.Partition.Definitions, model.PartitionDefinition{ID: part}) + } + } + tables = append(tables, &table) } ids := prealloctableid.New(tables) @@ -74,9 +100,9 @@ func TestAllocator(t *testing.T) { require.NoError(t, ids.Alloc(&allocator)) allocated := make([]int64, 0, len(c.successfullyAllocated)) - for _, t := range c.tableIDs { - if ids.Prealloced(t) { - allocated = append(allocated, t) + for _, t := range tables { + if ids.PreallocedFor(t.Info) { + allocated = append(allocated, t.Info.ID) } } require.ElementsMatch(t, allocated, c.successfullyAllocated) diff --git a/br/tests/lightning_foreign_key/config.toml b/br/tests/lightning_foreign_key/config.toml new file mode 100644 index 0000000000000..3c85a830bfa22 --- /dev/null +++ b/br/tests/lightning_foreign_key/config.toml @@ -0,0 +1,3 @@ +[tikv-importer] +# Set on-duplicate=error to force using insert statement to write data. +on-duplicate = "error" diff --git a/br/tests/lightning_foreign_key/data/fk.t-schema.sql b/br/tests/lightning_foreign_key/data/fk.t-schema.sql new file mode 100644 index 0000000000000..98f00b9cadca8 --- /dev/null +++ b/br/tests/lightning_foreign_key/data/fk.t-schema.sql @@ -0,0 +1,8 @@ +CREATE TABLE `t` +( + `a` bigint(20) NOT NULL, + `b` bigint(20) DEFAULT NULL, + PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */, + KEY `fk_1` (`b`), + CONSTRAINT `fk_1` FOREIGN KEY (`b`) REFERENCES `test`.`t2` (`a`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; diff --git a/br/tests/lightning_foreign_key/data/fk.t.csv b/br/tests/lightning_foreign_key/data/fk.t.csv new file mode 100644 index 0000000000000..cd0368580a4c8 --- /dev/null +++ b/br/tests/lightning_foreign_key/data/fk.t.csv @@ -0,0 +1,6 @@ +a,b +1,1 +2,2 +3,3 +4,4 +5,5 diff --git a/br/tests/lightning_foreign_key/run.sh b/br/tests/lightning_foreign_key/run.sh new file mode 100644 index 0000000000000..631b55d907102 --- /dev/null +++ b/br/tests/lightning_foreign_key/run.sh @@ -0,0 +1,28 @@ +#!/bin/bash +# +# Copyright 2022 PingCAP, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +set -eu + +# Create existing tables that import data will reference. +run_sql 'CREATE DATABASE IF NOT EXISTS fk;' +run_sql 'CREATE TABLE fk.t2 (a BIGINT PRIMARY KEY);' + +for BACKEND in tidb local; do + run_sql 'DROP TABLE IF EXISTS fk.t;' + run_lightning --backend $BACKEND + run_sql 'SELECT GROUP_CONCAT(a) FROM fk.t ORDER BY a;' + check_contains '1,2,3,4,5' +done diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index 8d8b47237b453..d8e1f186a4028 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -243,6 +243,7 @@ created_on datetime DEFAULT NULL, updated_on datetime DEFAULT NULL, UNIQUE KEY org_employee_position_pk (hotel_id,user_id,position_id) ) ENGINE=InnoDB DEFAULT CHARSET=utf8; +set tidb_cost_model_version=2; explain format = 'brief' SELECT d.id, d.ctx, d.name, d.left_value, d.right_value, d.depth, d.leader_id, d.status, d.created_on, d.updated_on FROM org_department AS d LEFT JOIN org_position AS p ON p.department_id = d.id AND p.status = 1000 LEFT JOIN org_employee_position AS ep ON ep.position_id = p.id AND ep.status = 1000 WHERE (d.ctx = 1 AND (ep.user_id = 62 OR d.id = 20 OR d.id = 20) AND d.status = 1000) GROUP BY d.id ORDER BY d.left_value; id estRows task access object operator info Sort 1.00 root test.org_department.left_value @@ -262,6 +263,7 @@ Sort 1.00 root test.org_department.left_value └─TableReader(Probe) 9.99 root data:Selection └─Selection 9.99 cop[tikv] eq(test.org_employee_position.status, 1000), not(isnull(test.org_employee_position.position_id)) └─TableFullScan 10000.00 cop[tikv] table:ep keep order:false, stats:pseudo +set tidb_cost_model_version=1; create table test.Tab_A (id int primary key,bid int,cid int,name varchar(20),type varchar(20),num int,amt decimal(11,2)); create table test.Tab_B (id int primary key,name varchar(20)); create table test.Tab_C (id int primary key,name varchar(20),amt decimal(11,2)); diff --git a/cmd/explaintest/t/explain_complex.test b/cmd/explaintest/t/explain_complex.test index d9d1b9285f727..187ec571857d8 100644 --- a/cmd/explaintest/t/explain_complex.test +++ b/cmd/explaintest/t/explain_complex.test @@ -174,7 +174,9 @@ CREATE TABLE org_position ( UNIQUE KEY org_employee_position_pk (hotel_id,user_id,position_id) ) ENGINE=InnoDB DEFAULT CHARSET=utf8; +set tidb_cost_model_version=2; explain format = 'brief' SELECT d.id, d.ctx, d.name, d.left_value, d.right_value, d.depth, d.leader_id, d.status, d.created_on, d.updated_on FROM org_department AS d LEFT JOIN org_position AS p ON p.department_id = d.id AND p.status = 1000 LEFT JOIN org_employee_position AS ep ON ep.position_id = p.id AND ep.status = 1000 WHERE (d.ctx = 1 AND (ep.user_id = 62 OR d.id = 20 OR d.id = 20) AND d.status = 1000) GROUP BY d.id ORDER BY d.left_value; +set tidb_cost_model_version=1; create table test.Tab_A (id int primary key,bid int,cid int,name varchar(20),type varchar(20),num int,amt decimal(11,2)); create table test.Tab_B (id int primary key,name varchar(20)); diff --git a/config/config.go b/config/config.go index 2352a17acf0d1..bc25b8c9b9ec3 100644 --- a/config/config.go +++ b/config/config.go @@ -261,6 +261,9 @@ type Config struct { // EnableGlobalKill indicates whether to enable global kill. TrxSummary TrxSummary `toml:"transaction-summary" json:"transaction-summary"` EnableGlobalKill bool `toml:"enable-global-kill" json:"enable-global-kill"` + // InitializeSQLFile is a file that will be executed after first bootstrap only. + // It can be used to set GLOBAL system variable values + InitializeSQLFile string `toml:"initialize-sql-file" json:"initialize-sql-file"` // The following items are deprecated. We need to keep them here temporarily // to support the upgrade process. They can be removed in future. diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index 4528564d2f231..62623e1aca3c4 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -437,3 +437,27 @@ func testNewContext(store kv.Storage) sessionctx.Context { ctx.Store = store return ctx } + +func TestIssue40135(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + + tk.MustExec("CREATE TABLE t40135 ( a tinyint DEFAULT NULL, b varchar(32) DEFAULT 'md') PARTITION BY HASH (a) PARTITIONS 2") + one := true + hook := &ddl.TestDDLCallback{Do: dom} + var checkErr error + hook.OnJobRunBeforeExported = func(job *model.Job) { + if one { + one = false + _, checkErr = tk1.Exec("alter table t40135 change column a aNew SMALLINT NULL DEFAULT '-14996'") + } + } + dom.DDL().SetHook(hook) + tk.MustExec("alter table t40135 modify column a MEDIUMINT NULL DEFAULT '6243108' FIRST") + + require.ErrorContains(t, checkErr, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed") +} diff --git a/ddl/column_modify_test.go b/ddl/column_modify_test.go index 2055f9df9fa9c..1520339debf4a 100644 --- a/ddl/column_modify_test.go +++ b/ddl/column_modify_test.go @@ -289,8 +289,7 @@ func TestDropColumn(t *testing.T) { tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (a int,b int) partition by hash(a) partitions 4;") err := tk.ExecToErr("alter table t1 drop column a") - // TODO: refine the error message to compatible with MySQL - require.EqualError(t, err, "[planner:1054]Unknown column 'a' in 'expression'") + require.EqualError(t, err, "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed") } func TestChangeColumn(t *testing.T) { diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 38aa43e4e7755..ad96365c4668f 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -2246,14 +2246,6 @@ func TestExchangePartitionTableCompatiable(t *testing.T) { "alter table pt8 exchange partition p0 with table nt8;", dbterror.ErrTablesDifferentMetadata, }, - { - // foreign key test - // Partition table doesn't support to add foreign keys in mysql - "create table pt9 (id int not null primary key auto_increment,t_id int not null) partition by hash(id) partitions 1;", - "create table nt9 (id int not null primary key auto_increment, t_id int not null,foreign key fk_id (t_id) references pt5(id));", - "alter table pt9 exchange partition p0 with table nt9;", - dbterror.ErrPartitionExchangeForeignKey, - }, { // Generated column (virtual) "create table pt10 (id int not null, lname varchar(30), fname varchar(100) generated always as (concat(lname,' ')) virtual) partition by hash(id) partitions 1;", @@ -4543,17 +4535,49 @@ func TestAlterModifyColumnOnPartitionedTableRename(t *testing.T) { tk.MustExec("create database " + schemaName) tk.MustExec("use " + schemaName) tk.MustExec(`create table t (a int, b char) partition by range (a) (partition p0 values less than (10))`) - tk.MustContainErrMsg(`alter table t change a c int`, "[planner:1054]Unknown column 'a' in 'expression'") + tk.MustContainErrMsg(`alter table t change a c int`, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed") tk.MustExec(`drop table t`) tk.MustExec(`create table t (a char, b char) partition by range columns (a) (partition p0 values less than ('z'))`) - tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:8200]New column does not match partition definitions: [ddl:1567]partition column name cannot be found") + tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed") tk.MustExec(`drop table t`) tk.MustExec(`create table t (a int, b char) partition by list (a) (partition p0 values in (10))`) - tk.MustContainErrMsg(`alter table t change a c int`, "[planner:1054]Unknown column 'a' in 'expression'") + tk.MustContainErrMsg(`alter table t change a c int`, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed") tk.MustExec(`drop table t`) tk.MustExec(`create table t (a char, b char) partition by list columns (a) (partition p0 values in ('z'))`) - tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:8200]New column does not match partition definitions: [ddl:1567]partition column name cannot be found") + tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed") tk.MustExec(`drop table t`) tk.MustExec(`create table t (a int, b char) partition by hash (a) partitions 3`) - tk.MustContainErrMsg(`alter table t change a c int`, "[planner:1054]Unknown column 'a' in 'expression'") + tk.MustContainErrMsg(`alter table t change a c int`, "[ddl:8200]Unsupported modify column: Column 'a' has a partitioning function dependency and cannot be renamed") +} + +func TestDropPartitionKeyColumn(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database DropPartitionKeyColumn") + defer tk.MustExec("drop database DropPartitionKeyColumn") + tk.MustExec("use DropPartitionKeyColumn") + + tk.MustExec("create table t1 (a tinyint, b char) partition by range (a) ( partition p0 values less than (10) )") + err := tk.ExecToErr("alter table t1 drop column a") + require.Error(t, err) + require.Equal(t, "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error()) + tk.MustExec("alter table t1 drop column b") + + tk.MustExec("create table t2 (a tinyint, b char) partition by range (a-1) ( partition p0 values less than (10) )") + err = tk.ExecToErr("alter table t2 drop column a") + require.Error(t, err) + require.Equal(t, "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error()) + tk.MustExec("alter table t2 drop column b") + + tk.MustExec("create table t3 (a tinyint, b char) partition by hash(a) partitions 4;") + err = tk.ExecToErr("alter table t3 drop column a") + require.Error(t, err) + require.Equal(t, "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error()) + tk.MustExec("alter table t3 drop column b") + + tk.MustExec("create table t4 (a char, b char) partition by list columns (a) ( partition p0 values in ('0'), partition p1 values in ('a'), partition p2 values in ('b'));") + err = tk.ExecToErr("alter table t4 drop column a") + require.Error(t, err) + require.Equal(t, "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed", err.Error()) + tk.MustExec("alter table t4 drop column b") } diff --git a/ddl/db_table_test.go b/ddl/db_table_test.go index 13ae2f2dbb0de..40856d2a270f5 100644 --- a/ddl/db_table_test.go +++ b/ddl/db_table_test.go @@ -871,8 +871,7 @@ func TestDDLWithInvalidTableInfo(t *testing.T) { tk.MustExec("create table t (a bigint, b int, c int generated always as (b+1)) partition by hash(a) partitions 4;") // Test drop partition column. - // TODO: refine the error message to compatible with MySQL - tk.MustGetErrMsg("alter table t drop column a;", "[planner:1054]Unknown column 'a' in 'expression'") + tk.MustGetErrMsg("alter table t drop column a;", "[ddl:3885]Column 'a' has a partitioning function dependency and cannot be dropped or renamed") // Test modify column with invalid expression. tk.MustGetErrMsg("alter table t modify column c int GENERATED ALWAYS AS ((case when (a = 0) then 0when (a > 0) then (b / a) end));", "[parser:1064]You have an error in your SQL syntax; check the manual that corresponds to your TiDB version for the right syntax to use line 1 column 97 near \"then (b / a) end));\" ") // Test add column with invalid expression. diff --git a/ddl/ddl.go b/ddl/ddl.go index 8c4d5235ea7ad..1e1b38eeb77bb 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -439,9 +439,9 @@ func (dc *ddlCtx) setDDLSourceForDiagnosis(job *model.Job) { ctx, exists := dc.jobCtx.jobCtxMap[job.ID] if !exists { ctx = NewJobContext() - ctx.setDDLLabelForDiagnosis(job) dc.jobCtx.jobCtxMap[job.ID] = ctx } + ctx.setDDLLabelForDiagnosis(job) } func (dc *ddlCtx) getResourceGroupTaggerForTopSQL(job *model.Job) tikvrpc.ResourceGroupTagger { @@ -1786,7 +1786,11 @@ func (s *session) execute(ctx context.Context, query string, label string) ([]ch defer func() { metrics.DDLJobTableDuration.WithLabelValues(label + "-" + metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) }() - rs, err := s.Context.(sqlexec.SQLExecutor).ExecuteInternal(kv.WithInternalSourceType(ctx, kv.InternalTxnDDL), query) + + if ctx.Value(kv.RequestSourceKey) == nil { + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnDDL) + } + rs, err := s.Context.(sqlexec.SQLExecutor).ExecuteInternal(ctx, query) if err != nil { return nil, errors.Trace(err) } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 429d3f13425b2..caeb713bdebc8 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1980,7 +1980,7 @@ func addIndexForForeignKey(ctx sessionctx.Context, tbInfo *model.TableInfo) erro if handleCol != nil && len(fk.Cols) == 1 && handleCol.Name.L == fk.Cols[0].L { continue } - if model.FindIndexByColumns(tbInfo, fk.Cols...) != nil { + if model.FindIndexByColumns(tbInfo, tbInfo.Indices, fk.Cols...) != nil { continue } idxName := fk.Name @@ -3264,6 +3264,14 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Alter Table") } } + // set name for anonymous foreign key. + maxForeignKeyID := tb.Meta().MaxForeignKeyID + for _, spec := range validSpecs { + if spec.Tp == ast.AlterTableAddConstraint && spec.Constraint.Tp == ast.ConstraintForeignKey && spec.Constraint.Name == "" { + maxForeignKeyID++ + spec.Constraint.Name = fmt.Sprintf("fk_%d", maxForeignKeyID) + } + } if len(validSpecs) > 1 { sctx.GetSessionVars().StmtCtx.MultiSchemaInfo = model.NewMultiSchemaInfo() @@ -4298,6 +4306,9 @@ func checkIsDroppableColumn(ctx sessionctx.Context, is infoschema.InfoSchema, sc if err = isDroppableColumn(tblInfo, colName); err != nil { return false, errors.Trace(err) } + if err = checkDropColumnWithPartitionConstraint(t, colName); err != nil { + return false, errors.Trace(err) + } // Check the column with foreign key. err = checkDropColumnWithForeignKeyConstraint(is, schema.Name.L, tblInfo, colName.L) if err != nil { @@ -4318,6 +4329,24 @@ func checkIsDroppableColumn(ctx sessionctx.Context, is infoschema.InfoSchema, sc return true, nil } +// checkDropColumnWithPartitionConstraint is used to check the partition constraint of the drop column. +func checkDropColumnWithPartitionConstraint(t table.Table, colName model.CIStr) error { + if t.Meta().Partition == nil { + return nil + } + pt, ok := t.(table.PartitionedTable) + if !ok { + // Should never happen! + return errors.Trace(dbterror.ErrDependentByPartitionFunctional.GenWithStackByArgs(colName.L)) + } + for _, name := range pt.GetPartitionColumnNames() { + if strings.EqualFold(name.L, colName.L) { + return errors.Trace(dbterror.ErrDependentByPartitionFunctional.GenWithStackByArgs(colName.L)) + } + } + return nil +} + func checkVisibleColumnCnt(t table.Table, addCnt, dropCnt int) error { tblInfo := t.Meta() visibleColumCnt := 0 @@ -4698,9 +4727,15 @@ func GetModifiableColumnJob( for _, name := range pt.GetPartitionColumnNames() { if strings.EqualFold(name.L, col.Name.L) { isPartitioningColumn = true + break } } if isPartitioningColumn { + // TODO: update the partitioning columns with new names if column is renamed + // Would be an extension from MySQL which does not support it. + if col.Name.L != newCol.Name.L { + return nil, dbterror.ErrUnsupportedModifyColumn.GenWithStackByArgs(fmt.Sprintf("Column '%s' has a partitioning function dependency and cannot be renamed", col.Name.O)) + } if !isColTypeAllowedAsPartitioningCol(newCol.FieldType) { return nil, dbterror.ErrNotAllowedTypeInPartition.GenWithStackByArgs(newCol.Name.O) } @@ -4744,7 +4779,6 @@ func GetModifiableColumnJob( newTblInfo.Columns = newCols var buf bytes.Buffer - // TODO: update the partitioning columns with new names if column is renamed AppendPartitionInfo(tblInfo.GetPartitionInfo(), &buf, mysql.ModeNone) // The parser supports ALTER TABLE ... PARTITION BY ... even if the ddl code does not yet :) // Ignoring warnings @@ -6570,7 +6604,7 @@ func (d *ddl) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName mode if err != nil { return err } - if model.FindIndexByColumns(t.Meta(), fkInfo.Cols...) == nil { + if model.FindIndexByColumns(t.Meta(), t.Meta().Indices, fkInfo.Cols...) == nil { // Need to auto create index for fk cols if ctx.GetSessionVars().StmtCtx.MultiSchemaInfo == nil { ctx.GetSessionVars().StmtCtx.MultiSchemaInfo = model.NewMultiSchemaInfo() diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 89e515db8e1bc..89f466b1497c5 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -119,7 +119,7 @@ func NewJobContext() *JobContext { cacheSQL: "", cacheNormalizedSQL: "", cacheDigest: nil, - tp: "unknown", + tp: "", } } @@ -761,6 +761,9 @@ func getDDLRequestSource(job *model.Job) string { } func (w *JobContext) setDDLLabelForDiagnosis(job *model.Job) { + if w.tp != "" { + return + } w.tp = getDDLRequestSource(job) w.ddlJobCtx = kv.WithInternalSourceType(w.ddlJobCtx, w.ddlJobSourceType()) } diff --git a/ddl/fktest/foreign_key_test.go b/ddl/fktest/foreign_key_test.go index 13701ad5b610d..df461fa048e5c 100644 --- a/ddl/fktest/foreign_key_test.go +++ b/ddl/fktest/foreign_key_test.go @@ -18,7 +18,9 @@ import ( "bytes" "context" "fmt" + "sync" "testing" + "time" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" @@ -666,6 +668,28 @@ func TestCreateTableWithForeignKeyError(t *testing.T) { create: "create table t2 (id int key, constraint fk foreign key (id) references t1(name5678901234567890123456789012345678901234567890123456789012345));", err: "[ddl:1059]Identifier name 'name5678901234567890123456789012345678901234567890123456789012345' is too long", }, + // Test foreign key with temporary table + { + refer: "create temporary table t1 (id int key);", + create: "create table t2 (id int key, constraint fk foreign key (id) references t1(id));", + err: "[schema:1824]Failed to open the referenced table 't1'", + }, + { + refer: "create table t1 (id int key);", + create: "create temporary table t2 (id int key, constraint fk foreign key (id) references t1(id));", + err: "[schema:1215]Cannot add foreign key constraint", + }, + // Test foreign key with partition table + { + refer: "create table t1 (id int key) partition by hash(id) partitions 3;", + create: "create table t2 (id int key, constraint fk foreign key (id) references t1(id));", + err: "[schema:1506]Foreign key clause is not yet supported in conjunction with partitioning", + }, + { + refer: "create table t1 (id int key);", + create: "create table t2 (id int key, constraint fk foreign key (id) references t1(id)) partition by hash(id) partitions 3;", + err: "[schema:1506]Foreign key clause is not yet supported in conjunction with partitioning", + }, } for _, ca := range cases { tk.MustExec("drop table if exists t2") @@ -1414,6 +1438,40 @@ func TestAlterTableAddForeignKeyError(t *testing.T) { alter: "alter table t2 add constraint name5678901234567890123456789012345678901234567890123456789012345 foreign key (b) references t1(id)", err: "[ddl:1059]Identifier name 'name5678901234567890123456789012345678901234567890123456789012345' is too long", }, + // Test foreign key with temporary table. + { + prepares: []string{ + "create temporary table t1 (id int key);", + "create table t2 (a int, b int unique);", + }, + alter: "alter table t2 add constraint fk foreign key (b) references t1(id)", + err: "[schema:1824]Failed to open the referenced table 't1'", + }, + { + prepares: []string{ + "create table t1 (id int key);", + "create temporary table t2 (a int, b int unique);", + }, + alter: "alter table t2 add constraint fk foreign key (b) references t1(id)", + err: "[ddl:8200]TiDB doesn't support ALTER TABLE for local temporary table", + }, + // Test foreign key with partition table + { + prepares: []string{ + "create table t1 (id int key) partition by hash(id) partitions 3;", + "create table t2 (id int key);", + }, + alter: "alter table t2 add constraint fk foreign key (id) references t1(id)", + err: "[schema:1506]Foreign key clause is not yet supported in conjunction with partitioning", + }, + { + prepares: []string{ + "create table t1 (id int key);", + "create table t2 (id int key) partition by hash(id) partitions 3;;", + }, + alter: "alter table t2 add constraint fk foreign key (id) references t1(id)", + err: "[schema:1506]Foreign key clause is not yet supported in conjunction with partitioning", + }, } for i, ca := range cases { tk.MustExec("drop table if exists t2") @@ -1562,3 +1620,193 @@ func getLatestSchemaDiff(t *testing.T, tk *testkit.TestKit) *model.SchemaDiff { require.NoError(t, err) return diff } + +func TestMultiSchemaAddForeignKey(t *testing.T) { + store, _ := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@foreign_key_checks=1;") + tk.MustExec("use test") + tk.MustExec("create table t1 (id int key);") + tk.MustExec("create table t2 (a int, b int);") + tk.MustExec("alter table t2 add foreign key (a) references t1(id), add foreign key (b) references t1(id)") + tk.MustExec("alter table t2 add column c int, add column d int") + tk.MustExec("alter table t2 add foreign key (c) references t1(id), add foreign key (d) references t1(id), add index(c), add index(d)") + tk.MustExec("drop table t2") + tk.MustExec("create table t2 (a int, b int, index idx1(a), index idx2(b));") + tk.MustGetErrMsg("alter table t2 drop index idx1, drop index idx2, add foreign key (a) references t1(id), add foreign key (b) references t1(id)", + "[ddl:1553]Cannot drop index 'idx1': needed in a foreign key constraint") + tk.MustExec("alter table t2 drop index idx1, drop index idx2") + tk.MustExec("alter table t2 add foreign key (a) references t1(id), add foreign key (b) references t1(id)") + tk.MustQuery("show create table t2").Check(testkit.Rows("t2 CREATE TABLE `t2` (\n" + + " `a` int(11) DEFAULT NULL,\n" + + " `b` int(11) DEFAULT NULL,\n" + + " KEY `fk_1` (`a`),\n" + + " KEY `fk_2` (`b`),\n" + + " CONSTRAINT `fk_1` FOREIGN KEY (`a`) REFERENCES `test`.`t1` (`id`),\n" + + " CONSTRAINT `fk_2` FOREIGN KEY (`b`) REFERENCES `test`.`t1` (`id`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustExec("drop table t2") + tk.MustExec("create table t2 (a int, b int, index idx0(a,b), index idx1(a), index idx2(b));") + tk.MustExec("alter table t2 drop index idx1, add foreign key (a) references t1(id), add foreign key (b) references t1(id)") +} + +func TestAddForeignKeyInBigTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@foreign_key_checks=1;") + tk.MustExec("use test") + tk.MustExec("create table employee (id bigint auto_increment key, pid bigint)") + tk.MustExec("insert into employee (id) values (1),(2),(3),(4),(5),(6),(7),(8)") + for i := 0; i < 14; i++ { + tk.MustExec("insert into employee (pid) select pid from employee") + } + tk.MustExec("update employee set pid=id-1 where id>1") + start := time.Now() + tk.MustExec("alter table employee add foreign key fk_1(pid) references employee(id)") + require.Less(t, time.Since(start), time.Minute) +} + +func TestForeignKeyWithCacheTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@foreign_key_checks=1;") + tk.MustExec("use test") + // Test foreign key refer cache table. + tk.MustExec("create table t1 (id int key);") + tk.MustExec("insert into t1 values (1),(2),(3),(4)") + tk.MustExec("alter table t1 cache;") + tk.MustExec("create table t2 (b int);") + tk.MustExec("alter table t2 add constraint fk foreign key (b) references t1(id) on delete cascade on update cascade") + tk.MustExec("insert into t2 values (1),(2),(3),(4)") + tk.MustGetDBError("insert into t2 values (5)", plannercore.ErrNoReferencedRow2) + tk.MustExec("update t1 set id = id+10 where id=1") + tk.MustExec("delete from t1 where id<10") + tk.MustQuery("select * from t1").Check(testkit.Rows("11")) + tk.MustQuery("select * from t2").Check(testkit.Rows("11")) + tk.MustExec("alter table t1 nocache;") + tk.MustExec("drop table t1,t2;") + + // Test add foreign key on cache table. + tk.MustExec("create table t1 (id int key);") + tk.MustExec("create table t2 (b int);") + tk.MustExec("alter table t2 add constraint fk foreign key (b) references t1(id) on delete cascade on update cascade") + tk.MustExec("alter table t2 cache;") + tk.MustExec("insert into t1 values (1),(2),(3),(4)") + tk.MustExec("insert into t2 values (1),(2),(3),(4)") + tk.MustGetDBError("insert into t2 values (5)", plannercore.ErrNoReferencedRow2) + tk.MustExec("update t1 set id = id+10 where id=1") + tk.MustExec("delete from t1 where id<10") + tk.MustQuery("select * from t1").Check(testkit.Rows("11")) + tk.MustQuery("select * from t2").Check(testkit.Rows("11")) + tk.MustExec("alter table t2 nocache;") + tk.MustExec("drop table t1,t2;") +} + +func TestForeignKeyAndConcurrentDDL(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@foreign_key_checks=1;") + tk.MustExec("use test") + // Test foreign key refer cache table. + tk.MustExec("create table t1 (a int, b int, c int, index(a), index(b), index(c));") + tk.MustExec("create table t2 (a int, b int, c int, index(a), index(b), index(c));") + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("set @@foreign_key_checks=1;") + tk2.MustExec("use test") + passCases := []struct { + prepare []string + ddl1 string + ddl2 string + }{ + { + ddl1: "alter table t2 add constraint fk_1 foreign key (a) references t1(a)", + ddl2: "alter table t2 add constraint fk_2 foreign key (b) references t1(b)", + }, + { + ddl1: "alter table t2 drop foreign key fk_1", + ddl2: "alter table t2 drop foreign key fk_2", + }, + { + prepare: []string{ + "alter table t2 drop index a", + }, + ddl1: "alter table t2 add index(a)", + ddl2: "alter table t2 add constraint fk_1 foreign key (a) references t1(a)", + }, + { + ddl1: "alter table t2 drop index c", + ddl2: "alter table t2 add constraint fk_2 foreign key (b) references t1(b)", + }, + } + for _, ca := range passCases { + var wg sync.WaitGroup + wg.Add(2) + go func() { + defer wg.Done() + tk.MustExec(ca.ddl1) + }() + go func() { + defer wg.Done() + tk2.MustExec(ca.ddl2) + }() + wg.Wait() + } + errorCases := []struct { + prepare []string + ddl1 string + err1 string + ddl2 string + err2 string + }{ + { + ddl1: "alter table t2 add constraint fk foreign key (a) references t1(a)", + err1: "[ddl:1826]Duplicate foreign key constraint name 'fk'", + ddl2: "alter table t2 add constraint fk foreign key (b) references t1(b)", + err2: "[ddl:1826]Duplicate foreign key constraint name 'fk'", + }, + { + prepare: []string{ + "alter table t2 add constraint fk_1 foreign key (a) references t1(a)", + }, + ddl1: "alter table t2 drop foreign key fk_1", + err1: "[schema:1091]Can't DROP 'fk_1'; check that column/key exists", + ddl2: "alter table t2 drop foreign key fk_1", + err2: "[schema:1091]Can't DROP 'fk_1'; check that column/key exists", + }, + { + ddl1: "alter table t2 drop index a", + err1: "[ddl:1553]Cannot drop index 'a': needed in a foreign key constraint", + ddl2: "alter table t2 add constraint fk_1 foreign key (a) references t1(a)", + err2: "[ddl:-1]Failed to add the foreign key constraint. Missing index for 'fk_1' foreign key columns in the table 't2'", + }, + } + tk.MustExec("drop table t1,t2") + tk.MustExec("create table t1 (a int, b int, c int, index(a), index(b), index(c));") + tk.MustExec("create table t2 (a int, b int, c int, index(a), index(b), index(c));") + for i, ca := range errorCases { + for _, sql := range ca.prepare { + tk.MustExec(sql) + } + var wg sync.WaitGroup + var err1, err2 error + wg.Add(2) + go func() { + defer wg.Done() + err1 = tk.ExecToErr(ca.ddl1) + }() + go func() { + defer wg.Done() + err2 = tk2.ExecToErr(ca.ddl2) + }() + wg.Wait() + if (err1 == nil && err2 == nil) || (err1 != nil && err2 != nil) { + require.Failf(t, "both ddl1 and ddl2 execute success, but expect 1 error", fmt.Sprintf("idx: %v, err1: %v, err2: %v", i, err1, err2)) + } + if err1 != nil { + require.Equal(t, ca.err1, err1.Error()) + } + if err2 != nil { + require.Equal(t, ca.err2, err2.Error()) + } + } +} diff --git a/ddl/foreign_key.go b/ddl/foreign_key.go index af9570fbee837..1a06719cb404b 100644 --- a/ddl/foreign_key.go +++ b/ddl/foreign_key.go @@ -266,6 +266,9 @@ func checkTableForeignKey(referTblInfo, tblInfo *model.TableInfo, fkInfo *model. if referTblInfo.TTLInfo != nil { return dbterror.ErrUnsupportedTTLReferencedByFK } + if referTblInfo.GetPartitionInfo() != nil || tblInfo.GetPartitionInfo() != nil { + return infoschema.ErrForeignKeyOnPartitioned + } // check refer columns in parent table. for i := range fkInfo.RefCols { @@ -291,7 +294,7 @@ func checkTableForeignKey(referTblInfo, tblInfo *model.TableInfo, fkInfo *model. } } // check refer columns should have index. - if model.FindIndexByColumns(referTblInfo, fkInfo.RefCols...) == nil { + if model.FindIndexByColumns(referTblInfo, referTblInfo.Indices, fkInfo.RefCols...) == nil { return infoschema.ErrForeignKeyNoIndexInParent.GenWithStackByArgs(fkInfo.Name, fkInfo.RefTable) } return nil @@ -660,7 +663,7 @@ func checkAddForeignKeyValidInOwner(d *ddlCtx, t *meta.Meta, schema string, tbIn return nil } } - if model.FindIndexByColumns(tbInfo, fk.Cols...) == nil { + if model.FindIndexByColumns(tbInfo, tbInfo.Indices, fk.Cols...) == nil { return errors.Errorf("Failed to add the foreign key constraint. Missing index for '%s' foreign key columns in the table '%s'", fk.Name, tbInfo.Name) } return nil @@ -674,7 +677,12 @@ func checkForeignKeyConstrain(w *worker, schema, table string, fkInfo *model.FKI if err != nil { return errors.Trace(err) } - defer w.sessPool.put(sctx) + originValue := sctx.GetSessionVars().OptimizerEnableNAAJ + sctx.GetSessionVars().OptimizerEnableNAAJ = true + defer func() { + sctx.GetSessionVars().OptimizerEnableNAAJ = originValue + w.sessPool.put(sctx) + }() var buf strings.Builder buf.WriteString("select 1 from %n.%n where ") @@ -709,7 +717,7 @@ func checkForeignKeyConstrain(w *worker, schema, table string, fkInfo *model.FKI } buf.WriteString(" from %n.%n ) limit 1") paramsList = append(paramsList, fkInfo.RefSchema.L, fkInfo.RefTable.L) - rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(w.ctx, nil, buf.String(), paramsList...) + rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(w.ctx, []sqlexec.OptionFuncAlias{sqlexec.ExecOptionUseCurSession}, buf.String(), paramsList...) if err != nil { return errors.Trace(err) } diff --git a/ddl/index.go b/ddl/index.go index 273b89e041233..929a86b08321b 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -81,7 +81,12 @@ func buildIndexColumns(ctx sessionctx.Context, columns []*model.ColumnInfo, inde if err := checkIndexColumn(ctx, col, ip.Length); err != nil { return nil, false, err } - mvIndex = mvIndex || col.FieldType.IsArray() + if col.FieldType.IsArray() { + if mvIndex { + return nil, false, dbterror.ErrNotSupportedYet.GenWithStack("'more than one multi-valued key part per index'") + } + mvIndex = true + } indexColLen := ip.Length indexColumnLength, err := getIndexColumnLength(col, ip.Length) if err != nil { @@ -698,15 +703,18 @@ func pickBackfillType(w *worker, job *model.Job) model.ReorgType { return job.ReorgMeta.ReorgTp } if IsEnableFastReorg() { - canUseIngest := canUseIngest(w) - if ingest.LitInitialized && canUseIngest { - job.ReorgMeta.ReorgTp = model.ReorgTypeLitMerge - return model.ReorgTypeLitMerge + var useIngest bool + if ingest.LitInitialized { + useIngest = canUseIngest(w) + if useIngest { + job.ReorgMeta.ReorgTp = model.ReorgTypeLitMerge + return model.ReorgTypeLitMerge + } } // The lightning environment is unavailable, but we can still use the txn-merge backfill. logutil.BgLogger().Info("[ddl] fallback to txn-merge backfill process", zap.Bool("lightning env initialized", ingest.LitInitialized), - zap.Bool("can use ingest", canUseIngest)) + zap.Bool("can use ingest", useIngest)) job.ReorgMeta.ReorgTp = model.ReorgTypeTxnMerge return model.ReorgTypeTxnMerge } diff --git a/ddl/job_table.go b/ddl/job_table.go index 771a83b8f8264..117b3722eccde 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -396,7 +396,8 @@ func updateDDLJob2Table(sctx *session, job *model.Job, updateRawArgs bool) error // getDDLReorgHandle gets DDL reorg handle. func getDDLReorgHandle(sess *session, job *model.Job) (element *meta.Element, startKey, endKey kv.Key, physicalTableID int64, err error) { sql := fmt.Sprintf("select ele_id, ele_type, start_key, end_key, physical_id from mysql.tidb_ddl_reorg where job_id = %d", job.ID) - rows, err := sess.execute(context.Background(), sql, "get_handle") + ctx := kv.WithInternalSourceType(context.Background(), getDDLRequestSource(job)) + rows, err := sess.execute(ctx, sql, "get_handle") if err != nil { return nil, nil, nil, 0, err } diff --git a/ddl/main_test.go b/ddl/main_test.go index 3418d16a23ece..6a8642ae34380 100644 --- a/ddl/main_test.go +++ b/ddl/main_test.go @@ -64,6 +64,8 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + goleak.IgnoreTopFunction("internal/poll.runtime_pollWait"), + goleak.IgnoreTopFunction("net/http.(*persistConn).writeLoop"), } goleak.VerifyTestMain(m, opts...) diff --git a/ddl/metadatalocktest/mdl_test.go b/ddl/metadatalocktest/mdl_test.go index 64bdf77d55707..fd307968cad73 100644 --- a/ddl/metadatalocktest/mdl_test.go +++ b/ddl/metadatalocktest/mdl_test.go @@ -257,6 +257,47 @@ func TestMDLBasicBatchPointGet(t *testing.T) { require.Less(t, ts1, ts2) } +func TestMDLAddForeignKey(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + sv := server.CreateMockServer(t, store) + + sv.SetDomain(dom) + dom.InfoSyncer().SetSessionManager(sv) + defer sv.Close() + + conn1 := server.CreateMockConn(t, sv) + tk := testkit.NewTestKitWithSession(t, store, conn1.Context().Session) + conn2 := server.CreateMockConn(t, sv) + tkDDL := testkit.NewTestKitWithSession(t, store, conn2.Context().Session) + tk.MustExec("use test") + tk.MustExec("set global tidb_enable_metadata_lock=1") + tk.MustExec("create table t1(id int key);") + tk.MustExec("create table t2(id int key);") + + tk.MustExec("begin") + tk.MustExec("insert into t2 values(1);") + + var wg sync.WaitGroup + var ddlErr error + wg.Add(1) + var ts2 time.Time + go func() { + defer wg.Done() + ddlErr = tkDDL.ExecToErr("alter table test.t2 add foreign key (id) references t1(id)") + ts2 = time.Now() + }() + + time.Sleep(2 * time.Second) + + ts1 := time.Now() + tk.MustExec("commit") + + wg.Wait() + require.Error(t, ddlErr) + require.Equal(t, "[ddl:1452]Cannot add or update a child row: a foreign key constraint fails (`test`.`t2`, CONSTRAINT `fk_1` FOREIGN KEY (`id`) REFERENCES `t1` (`id`))", ddlErr.Error()) + require.Less(t, ts1, ts2) +} + func TestMDLRRUpdateSchema(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) sv := server.CreateMockServer(t, store) diff --git a/ddl/multi_schema_change.go b/ddl/multi_schema_change.go index ab306fe546932..a21bc27965c77 100644 --- a/ddl/multi_schema_change.go +++ b/ddl/multi_schema_change.go @@ -261,7 +261,10 @@ func fillMultiSchemaInfo(info *model.MultiSchemaInfo, job *model.Job) (err error case model.ActionRebaseAutoID, model.ActionModifyTableComment, model.ActionModifyTableCharsetAndCollate: case model.ActionAddForeignKey: fkInfo := job.Args[0].(*model.FKInfo) - info.ForeignKeys = append(info.ForeignKeys, fkInfo.Name) + info.AddForeignKeys = append(info.AddForeignKeys, model.AddForeignKeyInfo{ + Name: fkInfo.Name, + Cols: fkInfo.Cols, + }) default: return dbterror.ErrRunMultiSchemaChanges.FastGenByArgs(job.Type.String()) } @@ -323,6 +326,32 @@ func checkOperateSameColAndIdx(info *model.MultiSchemaInfo) error { return checkIndexes(info.AlterIndexes, true) } +func checkOperateDropIndexUseByForeignKey(info *model.MultiSchemaInfo, t table.Table) error { + var remainIndexes, droppingIndexes []*model.IndexInfo + tbInfo := t.Meta() + for _, idx := range tbInfo.Indices { + dropping := false + for _, name := range info.DropIndexes { + if name.L == idx.Name.L { + dropping = true + break + } + } + if dropping { + droppingIndexes = append(droppingIndexes, idx) + } else { + remainIndexes = append(remainIndexes, idx) + } + } + + for _, fk := range info.AddForeignKeys { + if droppingIdx := model.FindIndexByColumns(tbInfo, droppingIndexes, fk.Cols...); droppingIdx != nil && model.FindIndexByColumns(tbInfo, remainIndexes, fk.Cols...) == nil { + return dbterror.ErrDropIndexNeededInForeignKey.GenWithStackByArgs(droppingIdx.Name) + } + } + return nil +} + func checkMultiSchemaInfo(info *model.MultiSchemaInfo, t table.Table) error { err := checkOperateSameColAndIdx(info) if err != nil { @@ -334,6 +363,11 @@ func checkMultiSchemaInfo(info *model.MultiSchemaInfo, t table.Table) error { return err } + err = checkOperateDropIndexUseByForeignKey(info, t) + if err != nil { + return err + } + return checkAddColumnTooManyColumns(len(t.Cols()) + len(info.AddColumns) - len(info.DropColumns)) } diff --git a/distsql/distsql.go b/distsql/distsql.go index 1f18d084eb0b2..3c65205f3d331 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -38,11 +38,11 @@ import ( ) // DispatchMPPTasks dispatches all tasks and returns an iterator. -func DispatchMPPTasks(ctx context.Context, sctx sessionctx.Context, tasks []*kv.MPPDispatchRequest, fieldTypes []*types.FieldType, planIDs []int, rootID int, startTs uint64) (SelectResult, error) { +func DispatchMPPTasks(ctx context.Context, sctx sessionctx.Context, tasks []*kv.MPPDispatchRequest, fieldTypes []*types.FieldType, planIDs []int, rootID int, startTs uint64, mppQueryID kv.MPPQueryID) (SelectResult, error) { ctx = WithSQLKvExecCounterInterceptor(ctx, sctx.GetSessionVars().StmtCtx) _, allowTiFlashFallback := sctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] ctx = SetTiFlashMaxThreadsInContext(ctx, sctx) - resp := sctx.GetMPPClient().DispatchMPPTasks(ctx, sctx.GetSessionVars().KVVars, tasks, allowTiFlashFallback, startTs) + resp := sctx.GetMPPClient().DispatchMPPTasks(ctx, sctx.GetSessionVars().KVVars, tasks, allowTiFlashFallback, startTs, mppQueryID) if resp == nil { return nil, errors.New("client returns nil response") } diff --git a/domain/plan_replayer.go b/domain/plan_replayer.go index 4df05a627f3e2..fc54d30759057 100644 --- a/domain/plan_replayer.go +++ b/domain/plan_replayer.go @@ -174,7 +174,7 @@ type planReplayerHandle struct { } // SendTask send dumpTask in background task handler -func (h *planReplayerHandle) SendTask(task *PlanReplayerDumpTask) { +func (h *planReplayerHandle) SendTask(task *PlanReplayerDumpTask) bool { select { case h.planReplayerTaskDumpHandle.taskCH <- task: // we directly remove the task key if we put task in channel successfully, if the task was failed to dump, @@ -182,11 +182,13 @@ func (h *planReplayerHandle) SendTask(task *PlanReplayerDumpTask) { if !task.IsContinuesCapture { h.planReplayerTaskCollectorHandle.removeTask(task.PlanReplayerTaskKey) } + return true default: // TODO: add metrics here // directly discard the task if the task channel is full in order not to block the query process - logutil.BgLogger().Info("discard one plan replayer dump task", - zap.String("sql digest", task.SQLDigest), zap.String("plan digest", task.PlanDigest)) + logutil.BgLogger().Warn("discard one plan replayer dump task", + zap.String("sql-digest", task.SQLDigest), zap.String("plan-digest", task.PlanDigest)) + return false } } @@ -209,9 +211,13 @@ func (h *planReplayerTaskCollectorHandle) CollectPlanReplayerTask() error { for _, key := range allKeys { unhandled, err := checkUnHandledReplayerTask(h.ctx, h.sctx, key) if err != nil { + logutil.BgLogger().Warn("[plan-replayer-task] collect plan replayer task failed", zap.Error(err)) return err } if unhandled { + logutil.BgLogger().Debug("[plan-replayer-task] collect plan replayer task success", + zap.String("sql-digest", key.SQLDigest), + zap.String("plan-digest", key.PlanDigest)) tasks = append(tasks, key) } } @@ -351,16 +357,36 @@ type planReplayerTaskDumpWorker struct { func (w *planReplayerTaskDumpWorker) run() { for task := range w.taskCH { + w.handleTask(task) + } +} + +func (w *planReplayerTaskDumpWorker) handleTask(task *PlanReplayerDumpTask) { + sqlDigest := task.SQLDigest + planDigest := task.PlanDigest + check := true + occupy := true + handleTask := true + defer func() { + logutil.BgLogger().Debug("[plan-replayer-capture] handle task", + zap.String("sql-digest", sqlDigest), + zap.String("plan-digest", planDigest), + zap.Bool("check", check), + zap.Bool("occupy", occupy), + zap.Bool("handle", handleTask)) + }() + if task.IsContinuesCapture { if w.status.checkTaskKeyFinishedBefore(task) { - continue + check = false + return } - successOccupy := w.status.occupyRunningTaskKey(task) - if !successOccupy { - continue - } - w.HandleTask(task) - w.status.releaseRunningTaskKey(task) } + occupy = w.status.occupyRunningTaskKey(task) + if !occupy { + return + } + handleTask = w.HandleTask(task) + w.status.releaseRunningTaskKey(task) } // HandleTask handled task @@ -373,7 +399,7 @@ func (w *planReplayerTaskDumpWorker) HandleTask(task *PlanReplayerDumpTask) (suc taskKey := task.PlanReplayerTaskKey unhandled, err := checkUnHandledReplayerTask(w.ctx, w.sctx, taskKey) if err != nil { - logutil.BgLogger().Warn("check plan replayer capture task failed", + logutil.BgLogger().Warn("[plan-replayer-capture] check task failed", zap.String("sqlDigest", taskKey.SQLDigest), zap.String("planDigest", taskKey.PlanDigest), zap.Error(err)) @@ -384,9 +410,9 @@ func (w *planReplayerTaskDumpWorker) HandleTask(task *PlanReplayerDumpTask) (suc return true } - file, fileName, err := replayer.GeneratePlanReplayerFile() + file, fileName, err := replayer.GeneratePlanReplayerFile(task.IsContinuesCapture) if err != nil { - logutil.BgLogger().Warn("generate plan replayer capture task file failed", + logutil.BgLogger().Warn("[plan-replayer-capture] generate task file failed", zap.String("sqlDigest", taskKey.SQLDigest), zap.String("planDigest", taskKey.PlanDigest), zap.Error(err)) @@ -409,7 +435,7 @@ func (w *planReplayerTaskDumpWorker) HandleTask(task *PlanReplayerDumpTask) (suc } r, err := handle.GenJSONTableFromStats(schema.Name.String(), tbl.Meta(), stat.(*statistics.Table)) if err != nil { - logutil.BgLogger().Warn("generate plan replayer capture task json stats failed", + logutil.BgLogger().Warn("[plan-replayer-capture] generate task json stats failed", zap.String("sqlDigest", taskKey.SQLDigest), zap.String("planDigest", taskKey.PlanDigest), zap.Error(err)) @@ -421,7 +447,7 @@ func (w *planReplayerTaskDumpWorker) HandleTask(task *PlanReplayerDumpTask) (suc } err = DumpPlanReplayerInfo(w.ctx, w.sctx, task) if err != nil { - logutil.BgLogger().Warn("dump plan replayer capture task result failed", + logutil.BgLogger().Warn("[plan-replayer-capture] dump task result failed", zap.String("sqlDigest", taskKey.SQLDigest), zap.String("planDigest", taskKey.PlanDigest), zap.Error(err)) diff --git a/domain/plan_replayer_dump.go b/domain/plan_replayer_dump.go index 264631b6fbdb3..0dd4945873e58 100644 --- a/domain/plan_replayer_dump.go +++ b/domain/plan_replayer_dump.go @@ -66,6 +66,10 @@ const ( PlanReplayerTaskMetaIsCapture = "isCapture" // PlanReplayerTaskMetaIsContinues indicates whether this task is continues task PlanReplayerTaskMetaIsContinues = "isContinues" + // PlanReplayerTaskMetaSQLDigest indicates the sql digest of this task + PlanReplayerTaskMetaSQLDigest = "sqlDigest" + // PlanReplayerTaskMetaPlanDigest indicates the plan digest of this task + PlanReplayerTaskMetaPlanDigest = "planDigest" ) type tableNamePair struct { @@ -180,25 +184,53 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, execStmts := task.ExecStmts zw := zip.NewWriter(zf) var records []PlanReplayerStatusRecord + sqls := make([]string, 0) + for _, execStmt := range task.ExecStmts { + sqls = append(sqls, execStmt.Text()) + } + if task.IsCapture { + logutil.BgLogger().Info("[plan-replayer-dump] start to dump plan replayer result", + zap.String("sql-digest", task.SQLDigest), + zap.String("plan-digest", task.PlanDigest), + zap.Strings("sql", sqls), + zap.Bool("isContinues", task.IsContinuesCapture)) + } else { + logutil.BgLogger().Info("[plan-replayer-dump] start to dump plan replayer result", + zap.Strings("sqls", sqls)) + } defer func() { + errMsg := "" if err != nil { - logutil.BgLogger().Error("dump plan replayer failed", zap.Error(err)) + if task.IsCapture { + logutil.BgLogger().Info("[plan-replayer-dump] dump file failed", + zap.String("sql-digest", task.SQLDigest), + zap.String("plan-digest", task.PlanDigest), + zap.Strings("sql", sqls), + zap.Bool("isContinues", task.IsContinuesCapture)) + } else { + logutil.BgLogger().Info("[plan-replayer-dump] start to dump plan replayer result", + zap.Strings("sqls", sqls)) + } + errMsg = err.Error() } - err = zw.Close() - if err != nil { - logutil.BgLogger().Error("Closing zip writer failed", zap.Error(err), zap.String("filename", fileName)) + err1 := zw.Close() + if err1 != nil { + logutil.BgLogger().Error("[plan-replayer-dump] Closing zip writer failed", zap.Error(err), zap.String("filename", fileName)) + errMsg = errMsg + "," + err1.Error() } - err = zf.Close() - if err != nil { - logutil.BgLogger().Error("Closing zip file failed", zap.Error(err), zap.String("filename", fileName)) + err2 := zf.Close() + if err2 != nil { + logutil.BgLogger().Error("[plan-replayer-dump] Closing zip file failed", zap.Error(err), zap.String("filename", fileName)) + errMsg = errMsg + "," + err2.Error() + } + if len(errMsg) > 0 { for i, record := range records { - record.FailedReason = err.Error() + record.FailedReason = errMsg records[i] = record } } insertPlanReplayerStatus(ctx, sctx, records) }() - // Dump SQLMeta if err = dumpSQLMeta(zw, task); err != nil { return err @@ -299,6 +331,8 @@ func dumpSQLMeta(zw *zip.Writer, task *PlanReplayerDumpTask) error { varMap[PlanReplayerSQLMetaStartTS] = strconv.FormatUint(task.StartTS, 10) varMap[PlanReplayerTaskMetaIsCapture] = strconv.FormatBool(task.IsCapture) varMap[PlanReplayerTaskMetaIsContinues] = strconv.FormatBool(task.IsContinuesCapture) + varMap[PlanReplayerTaskMetaSQLDigest] = task.SQLDigest + varMap[PlanReplayerTaskMetaPlanDigest] = task.PlanDigest if err := toml.NewEncoder(cf).Encode(varMap); err != nil { return errors.AddStack(err) } diff --git a/errno/errcode.go b/errno/errcode.go index de4cebc55e144..16f7c82989d87 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -915,6 +915,7 @@ const ( ErrFunctionalIndexRowValueIsNotAllowed = 3800 ErrDependentByFunctionalIndex = 3837 ErrCannotConvertString = 3854 + ErrDependentByPartitionFunctional = 3885 ErrInvalidJSONValueForFuncIndex = 3903 ErrJSONValueOutOfRangeForFuncIndex = 3904 ErrFunctionalIndexDataIsTooLong = 3907 diff --git a/errno/errname.go b/errno/errname.go index 93b08fbe0e1a4..b7f02df1c13f5 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -908,6 +908,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrFKIncompatibleColumns: mysql.Message("Referencing column '%s' and referenced column '%s' in foreign key constraint '%s' are incompatible.", nil), ErrFunctionalIndexRowValueIsNotAllowed: mysql.Message("Expression of expression index '%s' cannot refer to a row value", nil), ErrDependentByFunctionalIndex: mysql.Message("Column '%s' has an expression index dependency and cannot be dropped or renamed", nil), + ErrDependentByPartitionFunctional: mysql.Message("Column '%s' has a partitioning function dependency and cannot be dropped or renamed", nil), ErrCannotConvertString: mysql.Message("Cannot convert string '%.64s' from %s to %s", nil), ErrInvalidJSONValueForFuncIndex: mysql.Message("Invalid JSON value for CAST for expression index '%s'", nil), ErrJSONValueOutOfRangeForFuncIndex: mysql.Message("Out of range JSON value for CAST for expression index '%s'", nil), diff --git a/errors.toml b/errors.toml index 32b8d40e902b2..a1af7ba7e3866 100644 --- a/errors.toml +++ b/errors.toml @@ -766,6 +766,11 @@ error = ''' Incorrect usage of %s and %s ''' +["ddl:1235"] +error = ''' +This version of TiDB doesn't yet support '%s' +''' + ["ddl:1246"] error = ''' Converting column '%s' from %s to %s @@ -1171,6 +1176,11 @@ error = ''' Column '%s' has an expression index dependency and cannot be dropped or renamed ''' +["ddl:3885"] +error = ''' +Column '%s' has a partitioning function dependency and cannot be dropped or renamed +''' + ["ddl:4135"] error = ''' Sequence '%-.64s.%-.64s' has run out @@ -1691,6 +1701,21 @@ error = ''' Invalid data type for JSON data in argument %d to function %s; a JSON string or JSON type is required. ''' +["expression:3752"] +error = ''' +Value is out of range for expression index '%s' at row %d +''' + +["expression:3903"] +error = ''' +Invalid JSON value for CAST for expression index '%s' +''' + +["expression:3907"] +error = ''' +Data too long for expression index '%s' +''' + ["expression:8128"] error = ''' Invalid TABLESAMPLE: %s @@ -2406,6 +2431,11 @@ error = ''' Changing schema from '%-.192s' to '%-.192s' is not allowed. ''' +["schema:1506"] +error = ''' +Foreign key clause is not yet supported in conjunction with partitioning +''' + ["schema:1822"] error = ''' Failed to add the foreign key constraint. Missing index for constraint '%s' in the referenced table '%s' diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 46ae254888db6..e1ecaca57456f 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -249,7 +249,7 @@ go_library( go_test( name = "executor_test", - timeout = "moderate", + timeout = "short", srcs = [ "adapter_test.go", "admin_test.go", @@ -332,7 +332,6 @@ go_test( "utils_test.go", "window_test.go", "write_concurrent_test.go", - "write_test.go", ], data = glob(["testdata/**"]), embed = [":executor"], diff --git a/executor/adapter.go b/executor/adapter.go index 5e12cce1ccc69..c087a50e5f5f0 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -18,6 +18,7 @@ import ( "bytes" "context" "fmt" + "math" "runtime/trace" "strconv" "strings" @@ -295,8 +296,12 @@ func (a *ExecStmt) PointGet(ctx context.Context) (*recordSet, error) { } a.Ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityHigh + var pointExecutor *PointGetExecutor + useMaxTS := startTs == math.MaxUint64 + // try to reuse point get executor - if a.PsStmt.Executor != nil { + // We should only use the cached the executor when the startTS is MaxUint64 + if a.PsStmt.Executor != nil && useMaxTS { exec, ok := a.PsStmt.Executor.(*PointGetExecutor) if !ok { logutil.Logger(ctx).Error("invalid executor type, not PointGetExecutor for point get path") @@ -306,17 +311,21 @@ func (a *ExecStmt) PointGet(ctx context.Context) (*recordSet, error) { pointGetPlan := a.PsStmt.PreparedAst.CachedPlan.(*plannercore.PointGetPlan) exec.Init(pointGetPlan) a.PsStmt.Executor = exec + pointExecutor = exec } } - if a.PsStmt.Executor == nil { + + if pointExecutor == nil { b := newExecutorBuilder(a.Ctx, a.InfoSchema, a.Ti) - newExecutor := b.build(a.Plan) + pointExecutor = b.build(a.Plan).(*PointGetExecutor) if b.err != nil { return nil, b.err } - a.PsStmt.Executor = newExecutor + + if useMaxTS { + a.PsStmt.Executor = pointExecutor + } } - pointExecutor := a.PsStmt.Executor.(*PointGetExecutor) if err = pointExecutor.Open(ctx); err != nil { terror.Call(pointExecutor.Close) @@ -1407,6 +1416,10 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, err error, hasMoreResults boo sessVars.DurationParse = 0 // Clean the stale read flag when statement execution finish sessVars.StmtCtx.IsStaleness = false + // Clean the MPP query info + sessVars.StmtCtx.MPPQueryInfo.QueryID.Store(0) + sessVars.StmtCtx.MPPQueryInfo.QueryTS.Store(0) + sessVars.StmtCtx.MPPQueryInfo.AllocatedMPPTaskID.Store(0) if sessVars.StmtCtx.ReadFromTableCache { metrics.ReadFromTableCacheCounter.Inc() diff --git a/executor/aggfuncs/BUILD.bazel b/executor/aggfuncs/BUILD.bazel index 5c01950eef836..a1d4a57dde1f5 100644 --- a/executor/aggfuncs/BUILD.bazel +++ b/executor/aggfuncs/BUILD.bazel @@ -89,7 +89,7 @@ go_test( embed = [":aggfuncs"], flaky = True, race = "on", - shard_count = 10, + shard_count = 20, deps = [ "//expression", "//expression/aggregation", diff --git a/executor/bind.go b/executor/bind.go index c5f254121a9bf..90272e6878620 100644 --- a/executor/bind.go +++ b/executor/bind.go @@ -63,6 +63,8 @@ func (e *SQLBindExec) Next(ctx context.Context, req *chunk.Chunk) error { return e.reloadBindings() case plannercore.OpSetBindingStatus: return e.setBindingStatus() + case plannercore.OpSetBindingStatusByDigest: + return e.setBindingStatusByDigest() default: return errors.Errorf("unsupported SQL bind operation: %v", e.sqlBindOp) } @@ -119,6 +121,15 @@ func (e *SQLBindExec) setBindingStatus() error { return err } +func (e *SQLBindExec) setBindingStatusByDigest() error { + ok, err := domain.GetDomain(e.ctx).BindHandle().SetBindRecordStatusByDigest(e.newStatus, e.sqlDigest) + if err == nil && !ok { + warningMess := errors.New("There are no bindings can be set the status. Please check the SQL text") + e.ctx.GetSessionVars().StmtCtx.AppendWarning(warningMess) + } + return err +} + func (e *SQLBindExec) createSQLBind() error { // For audit log, SQLBindExec execute "explain" statement internally, save and recover stmtctx // is necessary to avoid 'create binding' been recorded as 'explain'. diff --git a/executor/builder.go b/executor/builder.go index d4270397eecd0..c6baaf0932849 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3402,6 +3402,7 @@ func (b *executorBuilder) buildMPPGather(v *plannercore.PhysicalTableReader) Exe is: b.is, originalPlan: v.GetTablePlan(), startTS: startTs, + mppQueryID: kv.MPPQueryID{QueryTs: getMPPQueryTS(b.ctx), LocalQueryID: getMPPQueryID(b.ctx), ServerID: domain.GetDomain(b.ctx).ServerID()}, } return gather } @@ -3409,10 +3410,6 @@ func (b *executorBuilder) buildMPPGather(v *plannercore.PhysicalTableReader) Exe // buildTableReader builds a table reader executor. It first build a no range table reader, // and then update it ranges from table scan plan. func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) Executor { - if v.StoreType != kv.TiKV && b.isStaleness { - b.err = errors.New("stale requests require tikv backend") - return nil - } failpoint.Inject("checkUseMPP", func(val failpoint.Value) { if !b.ctx.GetSessionVars().InRestrictedSQL && val.(bool) != useMPPExecution(b.ctx, v) { if val.(bool) { diff --git a/executor/compiler.go b/executor/compiler.go index cfe71f2ccdf29..821561899f4e7 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -157,6 +157,11 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (_ *ExecS } } } + + if err = sessiontxn.OptimizeWithPlanAndThenWarmUp(c.Ctx, stmt.Plan); err != nil { + return nil, err + } + if c.Ctx.GetSessionVars().IsPlanReplayerCaptureEnabled() && !c.Ctx.GetSessionVars().InRestrictedSQL { startTS, err := sessiontxn.GetTxnManager(c.Ctx).GetStmtReadTS() if err != nil { @@ -168,7 +173,6 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (_ *ExecS checkPlanReplayerCaptureTask(c.Ctx, stmtNode, startTS) } } - return stmt, nil } @@ -181,9 +185,17 @@ func checkPlanReplayerCaptureTask(sctx sessionctx.Context, stmtNode ast.StmtNode if handle == nil { return } + captured := false tasks := handle.GetTasks() _, sqlDigest := sctx.GetSessionVars().StmtCtx.SQLDigest() _, planDigest := getPlanDigest(sctx.GetSessionVars().StmtCtx) + defer func() { + logutil.BgLogger().Debug("[plan-replayer-capture] check capture task", + zap.String("sql-digest", sqlDigest.String()), + zap.String("plan-digest", planDigest.String()), + zap.Int("tasks", len(tasks)), + zap.Bool("captured", captured)) + }() key := replayer.PlanReplayerTaskKey{ SQLDigest: sqlDigest.String(), PlanDigest: planDigest.String(), @@ -191,7 +203,7 @@ func checkPlanReplayerCaptureTask(sctx sessionctx.Context, stmtNode ast.StmtNode for _, task := range tasks { if task.SQLDigest == sqlDigest.String() { if task.PlanDigest == "*" || task.PlanDigest == planDigest.String() { - sendPlanReplayerDumpTask(key, sctx, stmtNode, startTS, false) + captured = sendPlanReplayerDumpTask(key, sctx, stmtNode, startTS, false) return } } @@ -213,16 +225,26 @@ func checkPlanReplayerContinuesCapture(sctx sessionctx.Context, stmtNode ast.Stm SQLDigest: sqlDigest.String(), PlanDigest: planDigest.String(), } + captured := false + defer func() { + logutil.BgLogger().Debug("[plan-replayer-capture] check continues capture task", + zap.String("sql-digest", sqlDigest.String()), + zap.String("plan-digest", planDigest.String()), + zap.Bool("captured", captured)) + }() + existed := sctx.GetSessionVars().CheckPlanReplayerFinishedTaskKey(key) if existed { return } - sendPlanReplayerDumpTask(key, sctx, stmtNode, startTS, true) - sctx.GetSessionVars().AddPlanReplayerFinishedTaskKey(key) + captured = sendPlanReplayerDumpTask(key, sctx, stmtNode, startTS, true) + if captured { + sctx.GetSessionVars().AddPlanReplayerFinishedTaskKey(key) + } } func sendPlanReplayerDumpTask(key replayer.PlanReplayerTaskKey, sctx sessionctx.Context, stmtNode ast.StmtNode, - startTS uint64, isContinuesCapture bool) { + startTS uint64, isContinuesCapture bool) bool { stmtCtx := sctx.GetSessionVars().StmtCtx handle := sctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) dumpTask := &domain.PlanReplayerDumpTask{ @@ -237,7 +259,7 @@ func sendPlanReplayerDumpTask(key replayer.PlanReplayerTaskKey, sctx sessionctx. IsCapture: true, IsContinuesCapture: isContinuesCapture, } - domain.GetDomain(sctx).GetPlanReplayerHandle().SendTask(dumpTask) + return domain.GetDomain(sctx).GetPlanReplayerHandle().SendTask(dumpTask) } // needLowerPriority checks whether it's needed to lower the execution priority diff --git a/executor/delete.go b/executor/delete.go index 3aa0932a07c22..97b3487ffa3f9 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -245,7 +245,8 @@ func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h kv.Handl if err != nil { return err } - err = e.onRemoveRowForFK(ctx, t, data) + tid := t.Meta().ID + err = onRemoveRowForFK(ctx, data, e.fkChecks[tid], e.fkCascades[tid]) if err != nil { return err } @@ -253,8 +254,7 @@ func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h kv.Handl return nil } -func (e *DeleteExec) onRemoveRowForFK(ctx sessionctx.Context, t table.Table, data []types.Datum) error { - fkChecks := e.fkChecks[t.Meta().ID] +func onRemoveRowForFK(ctx sessionctx.Context, data []types.Datum, fkChecks []*FKCheckExec, fkCascades []*FKCascadeExec) error { sc := ctx.GetSessionVars().StmtCtx for _, fkc := range fkChecks { err := fkc.deleteRowNeedToCheck(sc, data) @@ -262,7 +262,6 @@ func (e *DeleteExec) onRemoveRowForFK(ctx sessionctx.Context, t table.Table, dat return err } } - fkCascades := e.fkCascades[t.Meta().ID] for _, fkc := range fkCascades { err := fkc.onDeleteRow(sc, data) if err != nil { diff --git a/executor/executor.go b/executor/executor.go index 2e5b5c4a0280f..9f95e63aaed20 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1933,7 +1933,7 @@ func (e *UnionExec) Close() error { func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars := ctx.GetSessionVars() var sc *stmtctx.StatementContext - if vars.TxnCtx.CouldRetry { + if vars.TxnCtx.CouldRetry || mysql.HasCursorExistsFlag(vars.Status) { // Must construct new statement context object, the retry history need context for every statement. // TODO: Maybe one day we can get rid of transaction retry, then this logic can be deleted. sc = &stmtctx.StatementContext{} diff --git a/executor/executor_required_rows_test.go b/executor/executor_required_rows_test.go index cbca9914b5bc2..c3ac762050d24 100644 --- a/executor/executor_required_rows_test.go +++ b/executor/executor_required_rows_test.go @@ -22,6 +22,7 @@ import ( "testing" "time" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/parser/ast" @@ -211,6 +212,7 @@ func defaultCtx() sessionctx.Context { ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(-1, ctx.GetSessionVars().MemQuotaQuery) ctx.GetSessionVars().StmtCtx.DiskTracker = disk.NewTracker(-1, -1) ctx.GetSessionVars().SnapshotTS = uint64(1) + domain.BindDomain(ctx, domain.NewMockDomain()) return ctx } diff --git a/executor/fktest/BUILD.bazel b/executor/fktest/BUILD.bazel index f245bba152c59..2c9f00dfa0624 100644 --- a/executor/fktest/BUILD.bazel +++ b/executor/fktest/BUILD.bazel @@ -15,6 +15,7 @@ go_test( "//infoschema", "//kv", "//meta/autoid", + "//parser", "//parser/ast", "//parser/auth", "//parser/format", diff --git a/executor/fktest/foreign_key_test.go b/executor/fktest/foreign_key_test.go index a162bd22b96aa..1dc92d6954a2e 100644 --- a/executor/fktest/foreign_key_test.go +++ b/executor/fktest/foreign_key_test.go @@ -21,6 +21,7 @@ import ( "strconv" "strings" "sync" + "sync/atomic" "testing" "time" @@ -28,6 +29,7 @@ import ( "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/format" @@ -2539,3 +2541,195 @@ func TestForeignKeyIssue39732(t *testing.T) { tk.MustExec("execute stmt1 using @a;") tk.MustQuery("select * from t1 order by id").Check(testkit.Rows()) } + +func TestForeignKeyOnReplaceIntoChildTable(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@global.tidb_enable_foreign_key=1") + tk.MustExec("set @@foreign_key_checks=1") + tk.MustExec("use test") + tk.MustExec("create table t_data (id int, a int, b int)") + tk.MustExec("insert into t_data (id, a, b) values (1, 1, 1), (2, 2, 2);") + for _, ca := range foreignKeyTestCase1 { + tk.MustExec("drop table if exists t2;") + tk.MustExec("drop table if exists t1;") + for _, sql := range ca.prepareSQLs { + tk.MustExec(sql) + } + tk.MustExec("replace into t1 (id, a, b) values (1, 1, 1);") + tk.MustExec("replace into t2 (id, a, b) values (1, 1, 1)") + tk.MustGetDBError("replace into t1 (id, a, b) values (1, 2, 3);", plannercore.ErrRowIsReferenced2) + if !ca.notNull { + tk.MustExec("replace into t2 (id, a, b) values (2, null, 1)") + tk.MustExec("replace into t2 (id, a, b) values (3, 1, null)") + tk.MustExec("replace into t2 (id, a, b) values (4, null, null)") + } + tk.MustGetDBError("replace into t2 (id, a, b) values (5, 1, 0);", plannercore.ErrNoReferencedRow2) + tk.MustGetDBError("replace into t2 (id, a, b) values (6, 0, 1);", plannercore.ErrNoReferencedRow2) + tk.MustGetDBError("replace into t2 (id, a, b) values (7, 2, 2);", plannercore.ErrNoReferencedRow2) + // Test replace into from select. + tk.MustExec("delete from t2") + tk.MustExec("replace into t2 (id, a, b) select id, a, b from t_data where t_data.id=1") + tk.MustGetDBError("replace into t2 (id, a, b) select id, a, b from t_data where t_data.id=2", plannercore.ErrNoReferencedRow2) + + // Test in txn + tk.MustExec("delete from t2") + tk.MustExec("begin") + tk.MustExec("delete from t1 where a=1") + tk.MustGetDBError("replace into t2 (id, a, b) values (1, 1, 1)", plannercore.ErrNoReferencedRow2) + tk.MustExec("replace into t1 (id, a, b) values (2, 2, 2)") + tk.MustExec("replace into t2 (id, a, b) values (2, 2, 2)") + tk.MustGetDBError("replace into t1 (id, a, b) values (2, 2, 3);", plannercore.ErrRowIsReferenced2) + tk.MustExec("rollback") + tk.MustQuery("select id, a, b from t1 order by id").Check(testkit.Rows("1 1 1")) + tk.MustQuery("select id, a, b from t2 order by id").Check(testkit.Rows()) + } + + // Case-10: test primary key is handle and contain foreign key column, and foreign key column has default value. + tk.MustExec("drop table if exists t2;") + tk.MustExec("drop table if exists t1;") + tk.MustExec("set @@tidb_enable_clustered_index=0;") + tk.MustExec("create table t1 (id int,a int, primary key(id));") + tk.MustExec("create table t2 (id int key,a int not null default 0, index (a), foreign key fk(a) references t1(id));") + tk.MustExec("replace into t1 values (1, 1);") + tk.MustExec("replace into t2 values (1, 1);") + tk.MustGetDBError("replace into t2 (id) values (10);", plannercore.ErrNoReferencedRow2) + tk.MustGetDBError("replace into t2 values (3, 2);", plannercore.ErrNoReferencedRow2) + + // Case-11: test primary key is handle and contain foreign key column, and foreign key column doesn't have default value. + tk.MustExec("drop table if exists t2;") + tk.MustExec("create table t2 (id int key,a int, index (a), foreign key fk(a) references t1(id));") + tk.MustExec("replace into t2 values (1, 1);") + tk.MustExec("replace into t2 (id) values (10);") + tk.MustGetDBError("replace into t2 values (3, 2);", plannercore.ErrNoReferencedRow2) +} + +func TestForeignKeyOnReplaceInto(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@foreign_key_checks=1") + tk.MustExec("use test") + tk.MustExec("create table t1 (id int key, a int, index (a));") + tk.MustExec("create table t2 (id int key, a int, index (a), constraint fk_1 foreign key (a) references t1(a));") + tk.MustExec("replace into t1 values (1, 1);") + tk.MustExec("replace into t2 values (1, 1);") + tk.MustExec("replace into t2 (id) values (2);") + tk.MustGetDBError("replace into t2 values (1, 2);", plannercore.ErrNoReferencedRow2) + // Test fk check on replace into parent table. + tk.MustGetDBError("replace into t1 values (1, 2);", plannercore.ErrRowIsReferenced2) + // Test fk cascade delete on replace into parent table. + tk.MustExec("alter table t2 drop foreign key fk_1") + tk.MustExec("alter table t2 add constraint fk_1 foreign key (a) references t1(a) on delete cascade") + tk.MustExec("replace into t1 values (1, 2);") + tk.MustQuery("select id, a from t1").Check(testkit.Rows("1 2")) + tk.MustQuery("select * from t2").Check(testkit.Rows("2 ")) + // Test fk cascade delete on replace into parent table. + tk.MustExec("alter table t2 drop foreign key fk_1") + tk.MustExec("alter table t2 add constraint fk_1 foreign key (a) references t1(a) on delete set null") + tk.MustExec("delete from t2") + tk.MustExec("delete from t1") + tk.MustExec("replace into t1 values (1, 1);") + tk.MustExec("replace into t2 values (1, 1);") + tk.MustExec("replace into t1 values (1, 2);") + tk.MustQuery("select id, a from t1").Check(testkit.Rows("1 2")) + tk.MustQuery("select id, a from t2").Check(testkit.Rows("1 ")) + + // Test cascade delete in self table by replace into statement. + tk.MustExec("drop table t1,t2") + tk.MustExec("create table t1 (id int key, name varchar(10), leader int, index(leader), foreign key (leader) references t1(id) ON DELETE CASCADE);") + tk.MustExec("replace into t1 values (1, 'boss', null), (10, 'l1_a', 1), (11, 'l1_b', 1), (12, 'l1_c', 1)") + tk.MustExec("replace into t1 values (100, 'l2_a1', 10), (101, 'l2_a2', 10), (102, 'l2_a3', 10)") + tk.MustExec("replace into t1 values (110, 'l2_b1', 11), (111, 'l2_b2', 11), (112, 'l2_b3', 11)") + tk.MustExec("replace into t1 values (120, 'l2_c1', 12), (121, 'l2_c2', 12), (122, 'l2_c3', 12)") + tk.MustExec("replace into t1 values (1000,'l3_a1', 100)") + tk.MustExec("replace into t1 values (1, 'new-boss', null)") + tk.MustQuery("select id from t1 order by id").Check(testkit.Rows("1")) +} + +func TestForeignKeyLargeTxnErr(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@foreign_key_checks=1") + tk.MustExec("use test") + tk.MustExec("create table t1 (id int auto_increment key, pid int, name varchar(200), index(pid));") + tk.MustExec("insert into t1 (name) values ('abcdefghijklmnopqrstuvwxyz1234567890abcdefghijklmnopqrstuvwxyz1234567890abcdefghijklmnopqrstuvwxyz1234567890abcdefghijklmnopqrstuvwxyz1234567890abcdefghijklmnopqrstuvwxyz1234567890');") + for i := 0; i < 8; i++ { + tk.MustExec("insert into t1 (name) select name from t1;") + } + tk.MustQuery("select count(*) from t1").Check(testkit.Rows("256")) + tk.MustExec("update t1 set pid=1 where id>1") + tk.MustExec("alter table t1 add foreign key (pid) references t1 (id) on update cascade") + originLimit := atomic.LoadUint64(&kv.TxnTotalSizeLimit) + defer func() { + atomic.StoreUint64(&kv.TxnTotalSizeLimit, originLimit) + }() + // Set the limitation to a small value, make it easier to reach the limitation. + atomic.StoreUint64(&kv.TxnTotalSizeLimit, 10240) + tk.MustQuery("select sum(id) from t1").Check(testkit.Rows("32896")) + // foreign key cascade behaviour will cause ErrTxnTooLarge. + tk.MustGetDBError("update t1 set id=id+100000 where id=1", kv.ErrTxnTooLarge) + tk.MustQuery("select sum(id) from t1").Check(testkit.Rows("32896")) + tk.MustGetDBError("update t1 set id=id+100000 where id=1", kv.ErrTxnTooLarge) + tk.MustQuery("select id,pid from t1 where id<3 order by id").Check(testkit.Rows("1 ", "2 1")) + tk.MustExec("set @@foreign_key_checks=0") + tk.MustExec("update t1 set id=id+100000 where id=1") + tk.MustQuery("select id,pid from t1 where id<3 or pid is null order by id").Check(testkit.Rows("2 1", "100001 ")) +} + +func TestForeignKeyAndLockView(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1 (id int key)") + tk.MustExec("create table t2 (id int key, foreign key (id) references t1(id) ON DELETE CASCADE ON UPDATE CASCADE)") + tk.MustExec("insert into t1 values (1)") + tk.MustExec("insert into t2 values (1)") + tk.MustExec("begin pessimistic") + tk.MustExec("set @@foreign_key_checks=0") + tk.MustExec("update t2 set id=2") + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("set @@foreign_key_checks=1") + tk2.MustExec("use test") + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + tk2.MustExec("begin pessimistic") + tk2.MustExec("update t1 set id=2 where id=1") + tk2.MustExec("commit") + }() + time.Sleep(time.Millisecond * 200) + _, digest := parser.NormalizeDigest("update t1 set id=2 where id=1") + tk.MustQuery("select CURRENT_SQL_DIGEST from information_schema.tidb_trx where state='LockWaiting' and db='test'").Check(testkit.Rows(digest.String())) + tk.MustGetErrMsg("update t1 set id=2", "[executor:1213]Deadlock found when trying to get lock; try restarting transaction") + wg.Wait() +} + +func TestForeignKeyAndMemoryTracker(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@foreign_key_checks=1") + tk.MustExec("use test") + tk.MustExec("create table t1 (id int auto_increment key, pid int, name varchar(200), index(pid));") + tk.MustExec("insert into t1 (name) values ('abcdefghijklmnopqrstuvwxyz1234567890abcdefghijklmnopqrstuvwxyz');") + for i := 0; i < 8; i++ { + tk.MustExec("insert into t1 (name) select name from t1;") + } + tk.MustQuery("select count(*) from t1").Check(testkit.Rows("256")) + tk.MustExec("update t1 set pid=1 where id>1") + tk.MustExec("alter table t1 add foreign key (pid) references t1 (id) on update cascade") + tk.MustQuery("select sum(id) from t1").Check(testkit.Rows("32896")) + 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=40960;") + // foreign key cascade behaviour will exceed memory quota. + err := tk.ExecToErr("update t1 set id=id+100000 where id=1") + require.Error(t, err) + require.Contains(t, err.Error(), "Out Of Memory Quota!") + tk.MustQuery("select id,pid from t1 where id = 1").Check(testkit.Rows("1 ")) + tk.MustExec("set @@foreign_key_checks=0") + // After disable foreign_key_checks, following DML will execute successful. + tk.MustExec("update t1 set id=id+100000 where id=1") + tk.MustQuery("select id,pid from t1 where id<3 or pid is null order by id").Check(testkit.Rows("2 1", "100001 ")) +} diff --git a/executor/insert_common.go b/executor/insert_common.go index d321f02a2f4fd..8440242f1dad5 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -388,7 +388,7 @@ func (e *InsertValues) evalRow(ctx context.Context, list []expression.Expression e.evalBuffer.SetDatum(offset, val1) } // Row may lack of generated column, autoIncrement column, empty column here. - return e.fillRow(ctx, row, hasValue) + return e.fillRow(ctx, row, hasValue, rowIdx) } var emptyRow chunk.Row @@ -422,7 +422,7 @@ func (e *InsertValues) fastEvalRow(ctx context.Context, list []expression.Expres offset := e.insertColumns[i].Offset row[offset], hasValue[offset] = val1, true } - return e.fillRow(ctx, row, hasValue) + return e.fillRow(ctx, row, hasValue, rowIdx) } // setValueForRefColumn set some default values for the row to eval the row value with other columns, @@ -562,7 +562,7 @@ func (e *InsertValues) getRow(ctx context.Context, vals []types.Datum) ([]types. hasValue[offset] = true } - return e.fillRow(ctx, row, hasValue) + return e.fillRow(ctx, row, hasValue, 0) } // getColDefaultValue gets the column default value. @@ -647,7 +647,7 @@ func (e *InsertValues) fillColValue(ctx context.Context, datum types.Datum, idx // `insert|replace values` can guarantee consecutive autoID in a batch. // Other statements like `insert select from` don't guarantee consecutive autoID. // https://dev.mysql.com/doc/refman/8.0/en/innodb-auto-increment-handling.html -func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue []bool) ([]types.Datum, error) { +func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue []bool, rowIdx int) ([]types.Datum, error) { gCols := make([]*table.Column, 0) tCols := e.Table.Cols() if e.hasExtraHandle { @@ -693,6 +693,9 @@ func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue for i, gCol := range gCols { colIdx := gCol.ColumnInfo.Offset val, err := e.GenExprs[i].Eval(chunk.MutRowFromDatums(row).ToRow()) + if err != nil && gCol.FieldType.IsArray() { + return nil, completeError(tbl, gCol.Offset, rowIdx, err) + } if e.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) != nil { return nil, err } @@ -708,6 +711,29 @@ func (e *InsertValues) fillRow(ctx context.Context, row []types.Datum, hasValue return row, nil } +func completeError(tbl *model.TableInfo, offset int, rowIdx int, err error) error { + name := "expression_index" + for _, idx := range tbl.Indices { + for _, column := range idx.Columns { + if column.Offset == offset { + name = idx.Name.O + break + } + } + } + + if expression.ErrInvalidJSONForFuncIndex.Equal(err) { + return expression.ErrInvalidJSONForFuncIndex.GenWithStackByArgs(name) + } + if types.ErrOverflow.Equal(err) { + return expression.ErrDataOutOfRangeFuncIndex.GenWithStackByArgs(name, rowIdx+1) + } + if types.ErrDataTooLong.Equal(err) { + return expression.ErrFuncIndexDataIsTooLong.GenWithStackByArgs(name) + } + return err +} + // isAutoNull can help judge whether a datum is AutoIncrement Null quickly. // This used to help lazyFillAutoIncrement to find consecutive N datum backwards for batch autoID alloc. func (e *InsertValues) isAutoNull(ctx context.Context, d types.Datum, col *table.Column) bool { diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index 42526774dbdd5..eee019bd0de47 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -16,6 +16,7 @@ package executor import ( "context" + "time" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -38,6 +39,18 @@ func useMPPExecution(ctx sessionctx.Context, tr *plannercore.PhysicalTableReader return ok } +func getMPPQueryID(ctx sessionctx.Context) uint64 { + mppQueryInfo := &ctx.GetSessionVars().StmtCtx.MPPQueryInfo + mppQueryInfo.QueryID.CompareAndSwap(0, plannercore.AllocMPPQueryID()) + return mppQueryInfo.QueryID.Load() +} + +func getMPPQueryTS(ctx sessionctx.Context) uint64 { + mppQueryInfo := &ctx.GetSessionVars().StmtCtx.MPPQueryInfo + mppQueryInfo.QueryTS.CompareAndSwap(0, uint64(time.Now().UnixNano())) + return mppQueryInfo.QueryTS.Load() +} + // MPPGather dispatch MPP tasks and read data from root tasks. type MPPGather struct { // following fields are construct needed @@ -45,6 +58,7 @@ type MPPGather struct { is infoschema.InfoSchema originalPlan plannercore.PhysicalPlan startTS uint64 + mppQueryID kv.MPPQueryID mppReqs []*kv.MPPDispatchRequest @@ -78,17 +92,19 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { return errors.Trace(err) } logutil.BgLogger().Info("Dispatch mpp task", zap.Uint64("timestamp", mppTask.StartTs), - zap.Int64("ID", mppTask.ID), zap.String("address", mppTask.Meta.GetAddress()), + zap.Int64("ID", mppTask.ID), zap.Uint64("QueryTs", mppTask.MppQueryID.QueryTs), zap.Uint64("LocalQueryId", mppTask.MppQueryID.LocalQueryID), + zap.Uint64("ServerID", mppTask.MppQueryID.ServerID), zap.String("address", mppTask.Meta.GetAddress()), zap.String("plan", plannercore.ToString(pf.ExchangeSender))) req := &kv.MPPDispatchRequest{ - Data: pbData, - Meta: mppTask.Meta, - ID: mppTask.ID, - IsRoot: pf.IsRoot, - Timeout: 10, - SchemaVar: e.is.SchemaMetaVersion(), - StartTs: e.startTS, - State: kv.MppTaskReady, + Data: pbData, + Meta: mppTask.Meta, + ID: mppTask.ID, + IsRoot: pf.IsRoot, + Timeout: 10, + SchemaVar: e.is.SchemaMetaVersion(), + StartTs: e.startTS, + MppQueryID: mppTask.MppQueryID, + State: kv.MppTaskReady, } e.mppReqs = append(e.mppReqs, req) } @@ -109,7 +125,7 @@ func (e *MPPGather) Open(ctx context.Context) (err error) { // TODO: Move the construct tasks logic to planner, so we can see the explain results. sender := e.originalPlan.(*plannercore.PhysicalExchangeSender) planIDs := collectPlanIDS(e.originalPlan, nil) - frags, err := plannercore.GenerateRootMPPTasks(e.ctx, e.startTS, sender, e.is) + frags, err := plannercore.GenerateRootMPPTasks(e.ctx, e.startTS, e.mppQueryID, sender, e.is) if err != nil { return errors.Trace(err) } @@ -124,7 +140,7 @@ func (e *MPPGather) Open(ctx context.Context) (err error) { failpoint.Return(errors.Errorf("The number of tasks is not right, expect %d tasks but actually there are %d tasks", val.(int), len(e.mppReqs))) } }) - e.respIter, err = distsql.DispatchMPPTasks(ctx, e.ctx, e.mppReqs, e.retFieldTypes, planIDs, e.id, e.startTS) + e.respIter, err = distsql.DispatchMPPTasks(ctx, e.ctx, e.mppReqs, e.retFieldTypes, planIDs, e.id, e.startTS, e.mppQueryID) if err != nil { return errors.Trace(err) } diff --git a/executor/plan_replayer.go b/executor/plan_replayer.go index 60a47991b48df..fae6273b3bd5e 100644 --- a/executor/plan_replayer.go +++ b/executor/plan_replayer.go @@ -125,7 +125,7 @@ func (e *PlanReplayerExec) registerCaptureTask(ctx context.Context) error { func (e *PlanReplayerExec) createFile() error { var err error - e.DumpInfo.File, e.DumpInfo.FileName, err = replayer.GeneratePlanReplayerFile() + e.DumpInfo.File, e.DumpInfo.FileName, err = replayer.GeneratePlanReplayerFile(false) if err != nil { return err } diff --git a/executor/point_get_test.go b/executor/point_get_test.go index c615c3a75cb1a..8f13675457481 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -825,3 +825,20 @@ func TestPointGetIssue25167(t *testing.T) { tk.MustExec("insert into t values (1)") tk.MustQuery("select * from t as of timestamp @a where a = 1").Check(testkit.Rows()) } + +func TestPointGetIssue40194(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1(id int primary key, v int)") + tk.MustExec("insert into t1 values(1, 10)") + tk.MustExec("prepare s from 'select * from t1 where id=1'") + tk.MustExec("set @@tidb_enable_plan_replayer_capture=1") + tk.MustQuery("execute s").Check(testkit.Rows("1 10")) + tk.MustQuery("execute s").Check(testkit.Rows("1 10")) + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + tk2.MustExec("update t1 set v=v+1") + tk.MustQuery("execute s").Check(testkit.Rows("1 11")) +} diff --git a/executor/replace.go b/executor/replace.go index 158a620fb300e..bfc70ebc4451c 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -92,6 +92,10 @@ func (e *ReplaceExec) removeRow(ctx context.Context, txn kv.Transaction, handle if err != nil { return false, err } + err = onRemoveRowForFK(e.ctx, oldRow, e.fkChecks, e.fkCascades) + if err != nil { + return false, err + } e.ctx.GetSessionVars().StmtCtx.AddAffectedRows(1) return false, nil } @@ -277,3 +281,18 @@ func (e *ReplaceExec) setMessage() { stmtCtx.SetMessage(msg) } } + +// GetFKChecks implements WithForeignKeyTrigger interface. +func (e *ReplaceExec) GetFKChecks() []*FKCheckExec { + return e.fkChecks +} + +// GetFKCascades implements WithForeignKeyTrigger interface. +func (e *ReplaceExec) GetFKCascades() []*FKCascadeExec { + return e.fkCascades +} + +// HasFKCascades implements WithForeignKeyTrigger interface. +func (e *ReplaceExec) HasFKCascades() bool { + return len(e.fkCascades) > 0 +} diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index 11dae5e570767..5edbed52b4e13 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -420,14 +420,14 @@ func TestPreparedInsert(t *testing.T) { err = counter.Write(pb) require.NoError(t, err) hit := pb.GetCounter().GetValue() - require.Equal(t, float64(1), hit) + require.Equal(t, float64(0), hit) // insert-values-stmt cannot use the plan cache } tk.MustExec(`set @a=3,@b=3; execute stmt_insert using @a, @b;`) if flag { err = counter.Write(pb) require.NoError(t, err) hit := pb.GetCounter().GetValue() - require.Equal(t, float64(2), hit) + require.Equal(t, float64(0), hit) } result := tk.MustQuery("select id, c1 from prepare_test where id = ?", 1) @@ -443,21 +443,21 @@ func TestPreparedInsert(t *testing.T) { err = counter.Write(pb) require.NoError(t, err) hit := pb.GetCounter().GetValue() - require.Equal(t, float64(2), hit) + require.Equal(t, float64(0), hit) } tk.MustExec(`set @a=2; execute stmt_insert_select using @a;`) if flag { err = counter.Write(pb) require.NoError(t, err) hit := pb.GetCounter().GetValue() - require.Equal(t, float64(3), hit) + require.Equal(t, float64(1), hit) } tk.MustExec(`set @a=3; execute stmt_insert_select using @a;`) if flag { err = counter.Write(pb) require.NoError(t, err) hit := pb.GetCounter().GetValue() - require.Equal(t, float64(4), hit) + require.Equal(t, float64(2), hit) } result = tk.MustQuery("select id, c1 from prepare_test where id = ?", 101) diff --git a/executor/simple_test.go b/executor/simple_test.go index 61688519b55d5..a67933bbbca92 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -21,13 +21,13 @@ import ( "testing" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" "github.com/stretchr/testify/require" - tikvutil "github.com/tikv/client-go/v2/util" ) func TestKillStmt(t *testing.T) { @@ -86,7 +86,7 @@ func TestKillStmt(t *testing.T) { func TestUserAttributes(t *testing.T) { store, _ := testkit.CreateMockStoreAndDomain(t) rootTK := testkit.NewTestKit(t, store) - ctx := context.WithValue(context.Background(), tikvutil.RequestSourceKey, tikvutil.RequestSource{RequestSourceInternal: true}) + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnPrivilege) // https://dev.mysql.com/doc/refman/8.0/en/create-user.html#create-user-comments-attributes rootTK.MustExec(`CREATE USER testuser COMMENT '1234'`) diff --git a/executor/tiflashtest/BUILD.bazel b/executor/tiflashtest/BUILD.bazel index 5223fa79cc2d9..c7678e569522d 100644 --- a/executor/tiflashtest/BUILD.bazel +++ b/executor/tiflashtest/BUILD.bazel @@ -16,6 +16,7 @@ go_test( "//executor", "//meta/autoid", "//parser/terror", + "//planner/core", "//store/mockstore", "//store/mockstore/unistore", "//testkit", diff --git a/executor/tiflashtest/tiflash_test.go b/executor/tiflashtest/tiflash_test.go index 222e1bfdaff0b..9898ecd89a8eb 100644 --- a/executor/tiflashtest/tiflash_test.go +++ b/executor/tiflashtest/tiflash_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/parser/terror" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/testkit" @@ -267,14 +268,9 @@ func TestMppExecution(t *testing.T) { tk.MustExec("begin") tk.MustQuery("select count(*) from ( select * from t2 group by a, b) A group by A.b").Check(testkit.Rows("3")) tk.MustQuery("select count(*) from t1 where t1.a+100 > ( select count(*) from t2 where t1.a=t2.a and t1.b=t2.b) group by t1.b").Check(testkit.Rows("4")) - txn, err := tk.Session().Txn(true) - require.NoError(t, err) - ts := txn.StartTS() - taskID := tk.Session().GetSessionVars().AllocMPPTaskID(ts) - require.Equal(t, int64(6), taskID) - tk.MustExec("commit") - taskID = tk.Session().GetSessionVars().AllocMPPTaskID(ts + 1) + taskID := plannercore.AllocMPPTaskID(tk.Session()) require.Equal(t, int64(1), taskID) + tk.MustExec("commit") failpoint.Enable("github.com/pingcap/tidb/executor/checkTotalMPPTasks", `return(3)`) // all the data is related to one store, so there are three tasks. @@ -1043,7 +1039,7 @@ func TestTiFlashPartitionTableBroadcastJoin(t *testing.T) { } } -func TestForbidTiflashDuringStaleRead(t *testing.T) { +func TestTiflashSupportStaleRead(t *testing.T) { store := testkit.CreateMockStore(t, withMockTiFlash(2)) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -1075,8 +1071,8 @@ func TestForbidTiflashDuringStaleRead(t *testing.T) { fmt.Fprintf(resBuff, "%s\n", row) } res = resBuff.String() - require.NotContains(t, res, "tiflash") - require.Contains(t, res, "tikv") + require.Contains(t, res, "tiflash") + require.NotContains(t, res, "tikv") } func TestForbidTiFlashIfExtraPhysTableIDIsNeeded(t *testing.T) { diff --git a/executor/writetest/BUILD.bazel b/executor/writetest/BUILD.bazel new file mode 100644 index 0000000000000..2ddc46c29ad82 --- /dev/null +++ b/executor/writetest/BUILD.bazel @@ -0,0 +1,38 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "writetest_test", + srcs = [ + "main_test.go", + "write_test.go", + ], + flaky = True, + race = "on", + shard_count = 50, + deps = [ + "//config", + "//executor", + "//kv", + "//meta/autoid", + "//parser/model", + "//parser/mysql", + "//planner/core", + "//session", + "//sessionctx", + "//sessionctx/stmtctx", + "//sessionctx/variable", + "//sessiontxn", + "//store/mockstore", + "//table", + "//table/tables", + "//testkit", + "//types", + "//util", + "//util/mock", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//tikv", + "@io_opencensus_go//stats/view", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/executor/writetest/main_test.go b/executor/writetest/main_test.go new file mode 100644 index 0000000000000..075e1f91b488a --- /dev/null +++ b/executor/writetest/main_test.go @@ -0,0 +1,60 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package writetest + +import ( + "fmt" + "testing" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/testkit" + "github.com/tikv/client-go/v2/tikv" + "go.opencensus.io/stats/view" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + autoid.SetStep(5000) + config.UpdateGlobal(func(conf *config.Config) { + conf.Log.SlowThreshold = 30000 // 30s + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + conf.Experimental.AllowsExpressionIndex = true + }) + tikv.EnableFailpoints() + + opts := []goleak.Option{ + goleak.Cleanup(func(_ int) { + view.Stop() + }), + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), + goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"), + } + + goleak.VerifyTestMain(m, opts...) +} + +func fillData(tk *testkit.TestKit, table string) { + tk.MustExec("use test") + tk.MustExec(fmt.Sprintf("create table %s(id int not null default 1, name varchar(255), PRIMARY KEY(id));", table)) + + // insert data + tk.MustExec(fmt.Sprintf("insert INTO %s VALUES (1, \"hello\");", table)) + tk.MustExec(fmt.Sprintf("insert into %s values (2, \"hello\");", table)) +} diff --git a/executor/write_test.go b/executor/writetest/write_test.go similarity index 99% rename from executor/write_test.go rename to executor/writetest/write_test.go index 32aa261c5518d..ebeaaa388e269 100644 --- a/executor/write_test.go +++ b/executor/writetest/write_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package executor_test +package writetest import ( "context" diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index e6257c4dd058c..545abd497a2da 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -420,7 +421,7 @@ func (c *castAsArrayFunctionClass) verifyArgs(args []Expression) error { } if args[0].GetType().EvalType() != types.ETJson { - return types.ErrInvalidJSONData.GenWithStackByArgs("1", "cast_as_array") + return ErrInvalidTypeForJSON.GenWithStackByArgs(1, "cast_as_array") } return nil @@ -467,9 +468,80 @@ func (b *castJSONAsArrayFunctionSig) evalJSON(row chunk.Row) (res types.BinaryJS return types.BinaryJSON{}, false, ErrNotSupportedYet.GenWithStackByArgs("CAST-ing Non-JSON Array type to array") } - // TODO: impl the cast(... as ... array) function + arrayVals := make([]any, 0, len(b.args)) + ft := b.tp.ArrayType() + f := convertJSON2Tp(ft.EvalType()) + if f == nil { + return types.BinaryJSON{}, false, ErrNotSupportedYet.GenWithStackByArgs("CAS-ing JSON to the target type") + } + sc := b.ctx.GetSessionVars().StmtCtx + originalOverflowAsWarning := sc.OverflowAsWarning + originIgnoreTruncate := sc.IgnoreTruncate + originTruncateAsWarning := sc.TruncateAsWarning + sc.OverflowAsWarning = false + sc.IgnoreTruncate = false + sc.TruncateAsWarning = false + defer func() { + sc.OverflowAsWarning = originalOverflowAsWarning + sc.IgnoreTruncate = originIgnoreTruncate + sc.TruncateAsWarning = originTruncateAsWarning + }() + for i := 0; i < val.GetElemCount(); i++ { + item, err := f(sc, val.ArrayGetElem(i), ft) + if err != nil { + return types.BinaryJSON{}, false, err + } + arrayVals = append(arrayVals, item) + } + return types.CreateBinaryJSON(arrayVals), false, nil +} - return types.BinaryJSON{}, false, nil +func convertJSON2Tp(eval types.EvalType) func(*stmtctx.StatementContext, types.BinaryJSON, *types.FieldType) (any, error) { + switch eval { + case types.ETString: + return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) { + if item.TypeCode != types.JSONTypeCodeString { + return nil, ErrInvalidJSONForFuncIndex + } + return types.ProduceStrWithSpecifiedTp(string(item.GetString()), tp, sc, false) + } + case types.ETInt: + return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) { + if item.TypeCode != types.JSONTypeCodeInt64 && item.TypeCode != types.JSONTypeCodeUint64 { + return nil, ErrInvalidJSONForFuncIndex + } + return types.ConvertJSONToInt(sc, item, mysql.HasUnsignedFlag(tp.GetFlag()), tp.GetType()) + } + case types.ETReal, types.ETDecimal: + return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) { + if item.TypeCode != types.JSONTypeCodeInt64 && item.TypeCode != types.JSONTypeCodeUint64 && item.TypeCode != types.JSONTypeCodeFloat64 { + return nil, ErrInvalidJSONForFuncIndex + } + return types.ConvertJSONToFloat(sc, item) + } + case types.ETDatetime: + return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) { + if (tp.GetType() == mysql.TypeDatetime && item.TypeCode != types.JSONTypeCodeDatetime) || (tp.GetType() == mysql.TypeDate && item.TypeCode != types.JSONTypeCodeDate) { + return nil, ErrInvalidJSONForFuncIndex + } + res := item.GetTime() + res.SetType(tp.GetType()) + if tp.GetType() == mysql.TypeDate { + // Truncate hh:mm:ss part if the type is Date. + res.SetCoreTime(types.FromDate(res.Year(), res.Month(), res.Day(), 0, 0, 0, 0)) + } + return res, nil + } + case types.ETDuration: + return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) { + if item.TypeCode != types.JSONTypeCodeDuration { + return nil, ErrInvalidJSONForFuncIndex + } + return item.GetDuration(), nil + } + default: + return nil + } } type castAsJSONFunctionClass struct { diff --git a/expression/builtin_cast_test.go b/expression/builtin_cast_test.go index e670aad1e35ae..8208e0a0af0d6 100644 --- a/expression/builtin_cast_test.go +++ b/expression/builtin_cast_test.go @@ -1619,3 +1619,73 @@ func TestCastBinaryStringAsJSONSig(t *testing.T) { require.Equal(t, tt.resultStr, res.String()) } } + +func TestCastArrayFunc(t *testing.T) { + ctx := createContext(t) + tbl := []struct { + input interface{} + expected interface{} + tp *types.FieldType + success bool + buildFuncSuccess bool + }{ + { + []interface{}{int64(-1), int64(2), int64(3)}, + []interface{}{int64(-1), int64(2), int64(3)}, + types.NewFieldTypeBuilder().SetType(mysql.TypeLonglong).SetCharset(charset.CharsetBin).SetCollate(charset.CollationBin).SetArray(true).BuildP(), + true, + true, + }, + { + []interface{}{int64(-1), int64(2), int64(3)}, + nil, + types.NewFieldTypeBuilder().SetType(mysql.TypeString).SetCharset(charset.CharsetUTF8MB4).SetCollate(charset.CollationUTF8MB4).SetArray(true).BuildP(), + false, + true, + }, + { + []interface{}{"1"}, + nil, + types.NewFieldTypeBuilder().SetType(mysql.TypeLonglong).SetCharset(charset.CharsetBin).SetCollate(charset.CharsetBin).SetArray(true).BuildP(), + false, + true, + }, + { + []interface{}{"1", "2"}, + nil, + types.NewFieldTypeBuilder().SetType(mysql.TypeDouble).SetCharset(charset.CharsetBin).SetCollate(charset.CharsetBin).SetArray(true).BuildP(), + false, + true, + }, + { + []interface{}{int64(-1), 2.1, int64(3)}, + []interface{}{int64(-1), 2.1, int64(3)}, + types.NewFieldTypeBuilder().SetType(mysql.TypeDouble).SetCharset(charset.CharsetBin).SetCollate(charset.CharsetBin).SetArray(true).BuildP(), + true, + true, + }, + } + for _, tt := range tbl { + f, err := BuildCastFunctionWithCheck(ctx, datumsToConstants(types.MakeDatums(types.CreateBinaryJSON(tt.input)))[0], tt.tp) + if tt.buildFuncSuccess { + require.NoError(t, err, tt.input) + } else { + require.Error(t, err, tt.input) + continue + } + + val, isNull, err := f.EvalJSON(ctx, chunk.Row{}) + if tt.success { + require.NoError(t, err, tt.input) + if tt.expected == nil { + require.True(t, isNull, tt.input) + } else { + j1 := types.CreateBinaryJSON(tt.expected) + cmp := types.CompareBinaryJSON(j1, val) + require.Equal(t, 0, cmp, tt.input) + } + } else { + require.Error(t, err, tt.input) + } + } +} diff --git a/expression/errors.go b/expression/errors.go index c56737ec2fae3..62d5b89d547b9 100644 --- a/expression/errors.go +++ b/expression/errors.go @@ -38,6 +38,9 @@ var ( ErrInternal = dbterror.ClassOptimizer.NewStd(mysql.ErrInternal) ErrNoDB = dbterror.ClassOptimizer.NewStd(mysql.ErrNoDB) ErrNotSupportedYet = dbterror.ClassExpression.NewStd(mysql.ErrNotSupportedYet) + ErrInvalidJSONForFuncIndex = dbterror.ClassExpression.NewStd(mysql.ErrInvalidJSONValueForFuncIndex) + ErrDataOutOfRangeFuncIndex = dbterror.ClassExpression.NewStd(mysql.ErrDataOutOfRangeFunctionalIndex) + ErrFuncIndexDataIsTooLong = dbterror.ClassExpression.NewStd(mysql.ErrFunctionalIndexDataIsTooLong) // All the un-exported errors are defined here: errFunctionNotExists = dbterror.ClassExpression.NewStd(mysql.ErrSpDoesNotExist) diff --git a/expression/expression.go b/expression/expression.go index 6d7eb080b29fc..ee4ba67f1f854 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -998,7 +998,7 @@ func ColumnInfos2ColumnsAndNames(ctx sessionctx.Context, dbName, tblName model.C if err != nil { return nil, nil, errors.Trace(err) } - e, err := RewriteAstExpr(ctx, expr, mockSchema, names, false) + e, err := RewriteAstExpr(ctx, expr, mockSchema, names, true) if err != nil { return nil, nil, errors.Trace(err) } diff --git a/expression/multi_valued_index_test.go b/expression/multi_valued_index_test.go index 058d955faa4fb..33557726a1073 100644 --- a/expression/multi_valued_index_test.go +++ b/expression/multi_valued_index_test.go @@ -15,6 +15,7 @@ package expression_test import ( + "fmt" "testing" "github.com/pingcap/tidb/errno" @@ -36,7 +37,7 @@ func TestMultiValuedIndexDDL(t *testing.T) { tk.MustExec("drop table t") tk.MustGetErrCode("CREATE TABLE t(x INT, KEY k ((1 AND CAST(JSON_ARRAY(x) AS UNSIGNED ARRAY))));", errno.ErrNotSupportedYet) tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(cast(f1 as unsigned array) as unsigned array))));", errno.ErrNotSupportedYet) - tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(f1->>'$[*]' as unsigned array))));", errno.ErrInvalidJSONData) + tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(f1->>'$[*]' as unsigned array))));", errno.ErrInvalidTypeForJSON) tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(f1->'$[*]' as year array))));", errno.ErrNotSupportedYet) tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(f1->'$[*]' as json array))));", errno.ErrNotSupportedYet) tk.MustGetErrCode("CREATE TABLE t1 (f1 json, key mvi((cast(f1->'$[*]' as char(10) charset gbk array))));", errno.ErrNotSupportedYet) @@ -44,4 +45,183 @@ func TestMultiValuedIndexDDL(t *testing.T) { tk.MustGetErrCode("create table t(j json, gc json as (cast(j->'$[*]' as unsigned array)));", errno.ErrNotSupportedYet) tk.MustGetErrCode("create view v as select cast('[1,2,3]' as unsigned array);", errno.ErrNotSupportedYet) tk.MustExec("create table t(a json, index idx((cast(a as signed array))));") + + tk.MustExec("drop table t") + tk.MustGetErrCode("create table t(a json, b int, index idx(b, (cast(a as signed array)), (cast(a as signed array))));", errno.ErrNotSupportedYet) + tk.MustExec("create table t(a json, b int);") + tk.MustGetErrCode("create index idx on t (b, (cast(a as signed array)), (cast(a as signed array)))", errno.ErrNotSupportedYet) + tk.MustGetErrCode("alter table t add index idx(b, (cast(a as signed array)), (cast(a as signed array)))", errno.ErrNotSupportedYet) + tk.MustExec("create index idx1 on t (b, (cast(a as signed array)))") + tk.MustExec("alter table t add index idx2(b, (cast(a as signed array)))") + + tk.MustExec("drop table t") + tk.MustExec("create table t(a json, b int, index idx3(b, (cast(a as signed array))));") +} + +func TestMultiValuedIndexDML(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("USE test;") + + mode := []string{`''`, `default`} + + for _, m := range mode { + tk.MustExec(fmt.Sprintf("set @@sql_mode=%s", m)) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as unsigned array))));`) + tk.MustExec(`insert into t values ('[1,2,3]');`) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrDataOutOfRangeFunctionalIndex) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as signed array))));`) + tk.MustExec(`insert into t values ('[1,2,3]');`) + tk.MustExec(`insert into t values ('[-1]');`) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as double array))));`) + tk.MustExec(`insert into t values ('[1,2,3]');`) + tk.MustExec(`insert into t values ('[-1]');`) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('[1.2]');`) + tk.MustExec(`insert into t values ('[1.0]');`) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as decimal(10, 2) array))));`) + tk.MustExec(`insert into t values ('[1,2,3]');`) + tk.MustExec(`insert into t values ('[-1]');`) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('[1.2]');`) + tk.MustExec(`insert into t values ('[1.0]');`) + tk.MustExec(`insert into t values ('[1.1102]');`) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as char(1) array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('["1"]');`) + tk.MustExec(`insert into t values ('["a"]');`) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrFunctionalIndexDataIsTooLong) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as char(2) array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('["1"]');`) + tk.MustExec(`insert into t values ('["a"]');`) + tk.MustExec(`insert into t values ('["汉字"]');`) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as binary(1) array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('["1"]');`) + tk.MustExec(`insert into t values ('["a"]');`) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrFunctionalIndexDataIsTooLong) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as binary(2) array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values ('["1"]');`) + tk.MustExec(`insert into t values ('["a"]');`) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrFunctionalIndexDataIsTooLong) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as date array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values (json_array(cast("2022-02-02" as date)));`) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as time array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values (json_array(cast("11:00:00" as time)));`) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(a json, index idx((cast(a as datetime array))));`) + tk.MustGetErrCode(`insert into t values ('[1,2,3]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[-1]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["1"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["a"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('["汉字"]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.2]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values ('[1.0]');`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("11:00:00" as time)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustGetErrCode(`insert into t values (json_array(cast("2022-02-02" as date)));`, errno.ErrInvalidJSONValueForFuncIndex) + tk.MustExec(`insert into t values (json_array(cast("2022-02-02 11:00:00" as datetime)));`) + tk.MustGetErrCode(`insert into t values (json_array(cast('{"a":1}' as json)));`, errno.ErrInvalidJSONValueForFuncIndex) + } } diff --git a/go.mod b/go.mod index f4d3fb93f9e98..b7375dddbe3dc 100644 --- a/go.mod +++ b/go.mod @@ -70,7 +70,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20220729040631-518f63d66278 github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3 github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20221130022225-6c56ac56fe5f + github.com/pingcap/kvproto v0.0.0-20221213093948-9ccc6beaf0aa github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e @@ -89,7 +89,7 @@ require ( github.com/stretchr/testify v1.8.0 github.com/tdakkota/asciicheck v0.1.1 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.3 + github.com/tikv/client-go/v2 v2.0.4-0.20221226080148-018c59dbd837 github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 github.com/twmb/murmur3 v1.1.3 @@ -111,7 +111,7 @@ require ( go.uber.org/zap v1.23.0 golang.org/x/exp v0.0.0-20221023144134-a1e5550cf13e golang.org/x/net v0.4.0 - golang.org/x/oauth2 v0.2.0 + golang.org/x/oauth2 v0.3.0 golang.org/x/sync v0.1.0 golang.org/x/sys v0.3.0 golang.org/x/term v0.3.0 @@ -219,6 +219,7 @@ require ( github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd // indirect github.com/sirupsen/logrus v1.9.0 // indirect github.com/spaolacci/murmur3 v1.1.0 // indirect + github.com/tiancaiamao/gp v0.0.0-20221221095600-1a473d1f9b4b // indirect github.com/tklauser/go-sysconf v0.3.10 // indirect github.com/tklauser/numcpus v0.4.0 // indirect github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 // indirect diff --git a/go.sum b/go.sum index 8c21c0326306d..2a6c8d4fc308e 100644 --- a/go.sum +++ b/go.sum @@ -781,8 +781,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20221026112947-f8d61344b172/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= -github.com/pingcap/kvproto v0.0.0-20221130022225-6c56ac56fe5f h1:46ZD6xzQWJ8Jkeal/U7SqkX030Mgs8DAn6QV/9zbqOQ= -github.com/pingcap/kvproto v0.0.0-20221130022225-6c56ac56fe5f/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= +github.com/pingcap/kvproto v0.0.0-20221213093948-9ccc6beaf0aa h1:v0Z0nC0knwWHn3e9br8EMNfLBB14QDULn142UGjiTMQ= +github.com/pingcap/kvproto v0.0.0-20221213093948-9ccc6beaf0aa/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= @@ -932,8 +932,10 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tikv/client-go/v2 v2.0.3 h1:/glZOHs/K2pkCioDVae+aThUHFYRYQkEgY4NUTgfh+s= -github.com/tikv/client-go/v2 v2.0.3/go.mod h1:MDT4J9LzgS7Bj1DnEq6Gk/puy6mp8TgUC92zGEVVLLg= +github.com/tiancaiamao/gp v0.0.0-20221221095600-1a473d1f9b4b h1:4RNtqw1/tW67qP9fFgfQpTVd7DrfkaAWu4vsC18QmBo= +github.com/tiancaiamao/gp v0.0.0-20221221095600-1a473d1f9b4b/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= +github.com/tikv/client-go/v2 v2.0.4-0.20221226080148-018c59dbd837 h1:m6glgBGCIds9QURbk8Mn+8mjLKDcv6nWrNwYh92fydQ= +github.com/tikv/client-go/v2 v2.0.4-0.20221226080148-018c59dbd837/go.mod h1:ptS8K+VBrEH2gIS3JxaiFSSLfDDyuS2xcdLozOtBWBw= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 h1:ckPpxKcl75mO2N6a4cJXiZH43hvcHPpqc9dh1TmH1nc= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07/go.mod h1:CipBxPfxPUME+BImx9MUYXCnAVLS3VJUr3mnSJwh40A= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 h1:kl4KhGNsJIbDHS9/4U9yQo1UcPQM0kOMJHn29EoH/Ro= @@ -1210,8 +1212,8 @@ golang.org/x/oauth2 v0.0.0-20210819190943-2bc19b11175f/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20211104180415-d3ed0bb246c8/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20220223155221-ee480838109b/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= golang.org/x/oauth2 v0.0.0-20220309155454-6242fa91716a/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= -golang.org/x/oauth2 v0.2.0 h1:GtQkldQ9m7yvzCL1V+LrYow3Khe0eJH0w7RbX/VbaIU= -golang.org/x/oauth2 v0.2.0/go.mod h1:Cwn6afJ8jrQwYMxQDTpISoXmXW9I6qF6vDeuuoX3Ibs= +golang.org/x/oauth2 v0.3.0 h1:6l90koy8/LaBLmLu8jpHeHexzMwEita0zFfYlggy2F8= +golang.org/x/oauth2 v0.3.0/go.mod h1:rQrIauxkUhJ6CuwEXwymO2/eh4xz2ZWF1nBkcxS+tGk= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= diff --git a/infoschema/cluster_tables_test.go b/infoschema/cluster_tables_test.go index 00892c95cfe9c..73488028a26ec 100644 --- a/infoschema/cluster_tables_test.go +++ b/infoschema/cluster_tables_test.go @@ -1028,3 +1028,34 @@ func TestBindingFromHistoryWithTiFlashBindable(t *testing.T) { planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows() tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with tiflash engine") } + +func TestSetBindingStatusBySQLDigest(t *testing.T) { + s := new(clusterTablesSuite) + s.store, s.dom = testkit.CreateMockStoreAndDomain(t) + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0", nil) + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + tk := s.newTestKitWithRoot(t) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int, a int, key(a))") + sql := "select /*+ ignore_index(t, a) */ * from t where t.a = 1" + tk.MustExec(sql) + planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.cluster_statements_summary where query_sample_text = '%s'", sql)).Rows() + tk.MustExec(fmt.Sprintf("create global binding from history using plan digest '%s'", planDigest[0][0])) + sql = "select * from t where t.a = 1" + tk.MustExec(sql) + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + + sqlDigest := tk.MustQuery("show global bindings").Rows() + tk.MustExec(fmt.Sprintf("set binding disabled for sql digest '%s'", sqlDigest[0][9])) + tk.MustExec(sql) + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + tk.MustExec(fmt.Sprintf("set binding enabled for sql digest '%s'", sqlDigest[0][9])) + tk.MustExec(sql) + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + tk.MustGetErrMsg("set binding enabled for sql digest '2'", "can't find any binding for '2'") +} diff --git a/infoschema/error.go b/infoschema/error.go index a7e4929a35bcc..d89d3e3dc8201 100644 --- a/infoschema/error.go +++ b/infoschema/error.go @@ -64,6 +64,8 @@ var ( ErrKeyNotExists = dbterror.ClassSchema.NewStd(mysql.ErrKeyDoesNotExist) // ErrCannotAddForeign returns for foreign key exists. ErrCannotAddForeign = dbterror.ClassSchema.NewStd(mysql.ErrCannotAddForeign) + // ErrForeignKeyOnPartitioned returns for foreign key on partition table. + ErrForeignKeyOnPartitioned = dbterror.ClassSchema.NewStd(mysql.ErrForeignKeyOnPartitioned) // ErrForeignKeyNotMatch returns for foreign key not match. ErrForeignKeyNotMatch = dbterror.ClassSchema.NewStd(mysql.ErrWrongFkDef) // ErrIndexExists returns for index already exists. diff --git a/kv/mpp.go b/kv/mpp.go index b0752f8186deb..32e9186506067 100644 --- a/kv/mpp.go +++ b/kv/mpp.go @@ -16,7 +16,6 @@ package kv import ( "context" - "sync" "time" "github.com/pingcap/kvproto/pkg/mpp" @@ -28,12 +27,20 @@ type MPPTaskMeta interface { GetAddress() string } +// MPPQueryID means the global unique id of a mpp query. +type MPPQueryID struct { + QueryTs uint64 // timestamp of query execution, used for TiFlash minTSO schedule + LocalQueryID uint64 // unique mpp query id in local tidb memory. + ServerID uint64 +} + // MPPTask means the minimum execution unit of a mpp computation job. type MPPTask struct { - Meta MPPTaskMeta // on which store this task will execute - ID int64 // mppTaskID - StartTs uint64 - TableID int64 // physical table id + Meta MPPTaskMeta // on which store this task will execute + ID int64 // mppTaskID + StartTs uint64 + MppQueryID MPPQueryID + TableID int64 // physical table id PartitionTableIDs []int64 } @@ -41,8 +48,11 @@ type MPPTask struct { // ToPB generates the pb structure. func (t *MPPTask) ToPB() *mpp.TaskMeta { meta := &mpp.TaskMeta{ - StartTs: t.StartTs, - TaskId: t.ID, + StartTs: t.StartTs, + QueryTs: t.MppQueryID.QueryTs, + LocalQueryId: t.MppQueryID.LocalQueryID, + ServerId: t.MppQueryID.ServerID, + TaskId: t.ID, } if t.ID != -1 { meta.Address = t.Meta.GetAddress() @@ -71,20 +81,21 @@ type MPPDispatchRequest struct { IsRoot bool // root task returns data to tidb directly. Timeout uint64 // If task is assigned but doesn't receive a connect request during timeout, the task should be destroyed. // SchemaVer is for any schema-ful storage (like tiflash) to validate schema correctness if necessary. - SchemaVar int64 - StartTs uint64 - ID int64 // identify a single task - State MppTaskStates + SchemaVar int64 + StartTs uint64 + MppQueryID MPPQueryID + ID int64 // identify a single task + State MppTaskStates } // MPPClient accepts and processes mpp requests. type MPPClient interface { // ConstructMPPTasks schedules task for a plan fragment. // TODO:: This interface will be refined after we support more executors. - ConstructMPPTasks(context.Context, *MPPBuildTasksRequest, *sync.Map, time.Duration) ([]MPPTaskMeta, error) + ConstructMPPTasks(context.Context, *MPPBuildTasksRequest, time.Duration) ([]MPPTaskMeta, error) // DispatchMPPTasks dispatches ALL mpp requests at once, and returns an iterator that transfers the data. - DispatchMPPTasks(ctx context.Context, vars interface{}, reqs []*MPPDispatchRequest, needTriggerFallback bool, startTs uint64) Response + DispatchMPPTasks(ctx context.Context, vars interface{}, reqs []*MPPDispatchRequest, needTriggerFallback bool, startTs uint64, mppQueryID MPPQueryID) Response } // MPPBuildTasksRequest request the stores allocation for a mpp plan fragment. diff --git a/kv/txn.go b/kv/txn.go index d7828c7fb3138..035f2aa662eca 100644 --- a/kv/txn.go +++ b/kv/txn.go @@ -195,20 +195,22 @@ func BackOff(attempts uint) int { func setRequestSourceForInnerTxn(ctx context.Context, txn Transaction) { if source := ctx.Value(RequestSourceKey); source != nil { requestSource := source.(RequestSource) - if !requestSource.RequestSourceInternal { - logutil.Logger(ctx).Warn("`RunInNewTxn` should be used by inner txn only") + if requestSource.RequestSourceType != "" { + if !requestSource.RequestSourceInternal { + logutil.Logger(ctx).Warn("`RunInNewTxn` should be used by inner txn only") + } + txn.SetOption(RequestSourceInternal, requestSource.RequestSourceInternal) + txn.SetOption(RequestSourceType, requestSource.RequestSourceType) + return } - txn.SetOption(RequestSourceInternal, requestSource.RequestSourceInternal) - txn.SetOption(RequestSourceType, requestSource.RequestSourceType) + } + // panic in test mode in case there are requests without source in the future. + // log warnings in production mode. + if flag.Lookup("test.v") != nil || flag.Lookup("check.v") != nil { + panic("unexpected no source type context, if you see this error, " + + "the `RequestSourceTypeKey` is missing in your context") } else { - // panic in test mode in case there are requests without source in the future. - // log warnings in production mode. - if flag.Lookup("test.v") != nil || flag.Lookup("check.v") != nil { - panic("unexpected no source type context, if you see this error, " + - "the `RequestSourceTypeKey` is missing in your context") - } else { - logutil.Logger(ctx).Warn("unexpected no source type context, if you see this warning, " + - "the `RequestSourceTypeKey` is missing in the context") - } + logutil.Logger(ctx).Warn("unexpected no source type context, if you see this warning, " + + "the `RequestSourceTypeKey` is missing in the context") } } diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 7677a718e156f..0e2ce93934f7c 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -17000,6 +17000,212 @@ "title": "TTL", "type": "row" }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 19 + }, + "id": 291, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 20 + }, + "hiddenSeries": false, + "id": 289, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "tidb_server_gogc{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "interval": "", + "legendFormat": "{{instance}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "GOGC", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "exponential moving average of CPU Usage", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 20 + }, + "hiddenSeries": false, + "id": 293, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "tidb_rm_ema_cpu_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "interval": "", + "legendFormat": "{{instance}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "EMA CPU Usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "Resource Manager", + "type": "row" + }, { "aliasColors": {}, "bars": false, diff --git a/metrics/metrics.go b/metrics/metrics.go index 0cca442e5eb83..8f303ba58180e 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -124,7 +124,6 @@ func RegisterMetrics() { prometheus.MustRegister(SyncLoadHistogram) prometheus.MustRegister(ReadStatsHistogram) prometheus.MustRegister(JobsGauge) - prometheus.MustRegister(KeepAliveCounter) prometheus.MustRegister(LoadPrivilegeCounter) prometheus.MustRegister(InfoCacheCounters) prometheus.MustRegister(LoadSchemaCounter) @@ -182,6 +181,7 @@ func RegisterMetrics() { prometheus.MustRegister(TokenGauge) prometheus.MustRegister(ConfigStatus) prometheus.MustRegister(TiFlashQueryTotalCounter) + prometheus.MustRegister(TiFlashFailedMPPStoreState) prometheus.MustRegister(SmallTxnWriteDuration) prometheus.MustRegister(TxnWriteThroughput) prometheus.MustRegister(LoadSysVarCacheCounter) @@ -238,6 +238,7 @@ func ToggleSimplifiedMode(simplified bool) { InfoCacheCounters, ReadFromTableCacheCounter, TiFlashQueryTotalCounter, + TiFlashFailedMPPStoreState, CampaignOwnerCounter, NonTransactionalDMLCount, MemoryUsage, diff --git a/metrics/server.go b/metrics/server.go index 116b02eb122b6..830e03b28e986 100644 --- a/metrics/server.go +++ b/metrics/server.go @@ -120,14 +120,6 @@ var ( Help: "Counter of system time jumps backward.", }) - KeepAliveCounter = prometheus.NewCounter( - prometheus.CounterOpts{ - Namespace: "tidb", - Subsystem: "monitor", - Name: "keep_alive_total", - Help: "Counter of TiDB keep alive.", - }) - PlanCacheCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: "tidb", @@ -279,6 +271,14 @@ var ( Help: "Counter of TiFlash queries.", }, []string{LblType, LblResult}) + TiFlashFailedMPPStoreState = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "tiflash_failed_store", + Help: "Statues of failed tiflash mpp store,-1 means detector heartbeat,0 means reachable,1 means abnormal.", + }, []string{LblAddress}) + PDAPIExecutionHistogram = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", diff --git a/parser/ast/misc.go b/parser/ast/misc.go index 048b0d9c65b52..7a0e2fc7a1a50 100644 --- a/parser/ast/misc.go +++ b/parser/ast/misc.go @@ -3768,7 +3768,7 @@ func (n *TableOptimizerHint) Restore(ctx *format.RestoreCtx) error { } table.Restore(ctx) } - case "use_index", "ignore_index", "use_index_merge", "force_index": + case "use_index", "ignore_index", "use_index_merge", "force_index", "keep_order", "no_keep_order": n.Tables[0].Restore(ctx) ctx.WritePlain(" ") for i, index := range n.Indexes { diff --git a/parser/ast/misc_test.go b/parser/ast/misc_test.go index 8042a171874d0..36e8cb1ef002f 100644 --- a/parser/ast/misc_test.go +++ b/parser/ast/misc_test.go @@ -228,6 +228,18 @@ func TestTableOptimizerHintRestore(t *testing.T) { {"IGNORE_INDEX(@sel_1 t1 c1)", "IGNORE_INDEX(@`sel_1` `t1` `c1`)"}, {"IGNORE_INDEX(t1@sel_1 c1)", "IGNORE_INDEX(`t1`@`sel_1` `c1`)"}, {"IGNORE_INDEX(t1@sel_1 partition(p0, p1) c1)", "IGNORE_INDEX(`t1`@`sel_1` PARTITION(`p0`, `p1`) `c1`)"}, + {"KEEP_ORDER(t1 c1)", "KEEP_ORDER(`t1` `c1`)"}, + {"KEEP_ORDER(test.t1 c1)", "KEEP_ORDER(`test`.`t1` `c1`)"}, + {"KEEP_ORDER(@sel_1 t1 c1)", "KEEP_ORDER(@`sel_1` `t1` `c1`)"}, + {"KEEP_ORDER(t1@sel_1 c1)", "KEEP_ORDER(`t1`@`sel_1` `c1`)"}, + {"KEEP_ORDER(test.t1@sel_1 c1)", "KEEP_ORDER(`test`.`t1`@`sel_1` `c1`)"}, + {"KEEP_ORDER(test.t1@sel_1 partition(p0) c1)", "KEEP_ORDER(`test`.`t1`@`sel_1` PARTITION(`p0`) `c1`)"}, + {"NO_KEEP_ORDER(t1 c1)", "NO_KEEP_ORDER(`t1` `c1`)"}, + {"NO_KEEP_ORDER(test.t1 c1)", "NO_KEEP_ORDER(`test`.`t1` `c1`)"}, + {"NO_KEEP_ORDER(@sel_1 t1 c1)", "NO_KEEP_ORDER(@`sel_1` `t1` `c1`)"}, + {"NO_KEEP_ORDER(t1@sel_1 c1)", "NO_KEEP_ORDER(`t1`@`sel_1` `c1`)"}, + {"NO_KEEP_ORDER(test.t1@sel_1 c1)", "NO_KEEP_ORDER(`test`.`t1`@`sel_1` `c1`)"}, + {"NO_KEEP_ORDER(test.t1@sel_1 partition(p0) c1)", "NO_KEEP_ORDER(`test`.`t1`@`sel_1` PARTITION(`p0`) `c1`)"}, {"TIDB_SMJ(`t1`)", "TIDB_SMJ(`t1`)"}, {"TIDB_SMJ(t1)", "TIDB_SMJ(`t1`)"}, {"TIDB_SMJ(t1,t2)", "TIDB_SMJ(`t1`, `t2`)"}, diff --git a/parser/hintparser.go b/parser/hintparser.go index 1b14e7d292e1e..a619d2a1c74f2 100644 --- a/parser/hintparser.go +++ b/parser/hintparser.go @@ -41,18 +41,18 @@ type yyhintXError struct { } const ( - yyhintDefault = 57424 + yyhintDefault = 57426 yyhintEOFCode = 57344 yyhintErrCode = 57345 hintAggToCop = 57379 hintBCJoin = 57394 hintBKA = 57355 hintBNL = 57357 - hintDupsWeedOut = 57420 - hintFalse = 57416 - hintFirstMatch = 57421 - hintForceIndex = 57406 - hintGB = 57419 + hintDupsWeedOut = 57422 + hintFalse = 57418 + hintFirstMatch = 57423 + hintForceIndex = 57408 + hintGB = 57421 hintHashAgg = 57381 hintHashJoin = 57359 hintHashJoinBuild = 57360 @@ -70,12 +70,13 @@ const ( hintJoinOrder = 57352 hintJoinPrefix = 57353 hintJoinSuffix = 57354 - hintLeading = 57408 - hintLimitToCop = 57405 - hintLooseScan = 57422 - hintMB = 57418 + hintKeepOrder = 57400 + hintLeading = 57410 + hintLimitToCop = 57407 + hintLooseScan = 57424 + hintMB = 57420 hintMRR = 57367 - hintMaterialization = 57423 + hintMaterialization = 57425 hintMaxExecutionTime = 57375 hintMemoryQuota = 57388 hintMerge = 57363 @@ -83,175 +84,178 @@ const ( hintMpp2PhaseAgg = 57383 hintNoBKA = 57356 hintNoBNL = 57358 - hintNoDecorrelate = 57410 + hintNoDecorrelate = 57412 hintNoHashJoin = 57362 hintNoICP = 57369 hintNoIndexMerge = 57366 + hintNoKeepOrder = 57401 hintNoMRR = 57368 hintNoMerge = 57364 hintNoRangeOptimization = 57370 hintNoSemijoin = 57374 hintNoSkipScan = 57372 hintNoSwapJoinInputs = 57389 - hintNthPlan = 57404 - hintOLAP = 57411 - hintOLTP = 57412 - hintPartition = 57413 + hintNthPlan = 57406 + hintOLAP = 57413 + hintOLTP = 57414 + hintPartition = 57415 hintQBName = 57378 hintQueryType = 57390 hintReadConsistentReplica = 57391 hintReadFromStorage = 57392 hintResourceGroup = 57377 hintSMJoin = 57393 - hintSemiJoinRewrite = 57409 + hintSemiJoinRewrite = 57411 hintSemijoin = 57373 hintSetVar = 57376 hintShuffleJoin = 57395 hintSingleAtIdentifier = 57349 hintSkipScan = 57371 - hintStraightJoin = 57407 + hintStraightJoin = 57409 hintStreamAgg = 57396 hintStringLit = 57350 hintSwapJoinInputs = 57397 - hintTiFlash = 57415 - hintTiKV = 57414 - hintTimeRange = 57402 - hintTrue = 57417 - hintUseCascades = 57403 + hintTiFlash = 57417 + hintTiKV = 57416 + hintTimeRange = 57404 + hintTrue = 57419 + hintUseCascades = 57405 hintUseIndex = 57399 hintUseIndexMerge = 57398 - hintUsePlanCache = 57400 - hintUseToja = 57401 + hintUsePlanCache = 57402 + hintUseToja = 57403 yyhintMaxDepth = 200 - yyhintTabOfs = -193 + yyhintTabOfs = -197 ) var ( yyhintXLAT = map[int]int{ - 41: 0, // ')' (146x) - 57379: 1, // hintAggToCop (133x) - 57394: 2, // hintBCJoin (133x) - 57355: 3, // hintBKA (133x) - 57357: 4, // hintBNL (133x) - 57406: 5, // hintForceIndex (133x) - 57381: 6, // hintHashAgg (133x) - 57359: 7, // hintHashJoin (133x) - 57360: 8, // hintHashJoinBuild (133x) - 57361: 9, // hintHashJoinProbe (133x) - 57384: 10, // hintIgnoreIndex (133x) - 57380: 11, // hintIgnorePlanCache (133x) - 57365: 12, // hintIndexMerge (133x) - 57385: 13, // hintInlHashJoin (133x) - 57386: 14, // hintInlJoin (133x) - 57387: 15, // hintInlMergeJoin (133x) - 57351: 16, // hintJoinFixedOrder (133x) - 57352: 17, // hintJoinOrder (133x) - 57353: 18, // hintJoinPrefix (133x) - 57354: 19, // hintJoinSuffix (133x) - 57408: 20, // hintLeading (133x) - 57405: 21, // hintLimitToCop (133x) - 57375: 22, // hintMaxExecutionTime (133x) - 57388: 23, // hintMemoryQuota (133x) - 57363: 24, // hintMerge (133x) - 57382: 25, // hintMpp1PhaseAgg (133x) - 57383: 26, // hintMpp2PhaseAgg (133x) - 57367: 27, // hintMRR (133x) - 57356: 28, // hintNoBKA (133x) - 57358: 29, // hintNoBNL (133x) - 57410: 30, // hintNoDecorrelate (133x) - 57362: 31, // hintNoHashJoin (133x) - 57369: 32, // hintNoICP (133x) - 57366: 33, // hintNoIndexMerge (133x) - 57364: 34, // hintNoMerge (133x) - 57368: 35, // hintNoMRR (133x) - 57370: 36, // hintNoRangeOptimization (133x) - 57374: 37, // hintNoSemijoin (133x) - 57372: 38, // hintNoSkipScan (133x) - 57389: 39, // hintNoSwapJoinInputs (133x) - 57404: 40, // hintNthPlan (133x) - 57378: 41, // hintQBName (133x) - 57390: 42, // hintQueryType (133x) - 57391: 43, // hintReadConsistentReplica (133x) - 57392: 44, // hintReadFromStorage (133x) - 57377: 45, // hintResourceGroup (133x) - 57373: 46, // hintSemijoin (133x) - 57409: 47, // hintSemiJoinRewrite (133x) - 57376: 48, // hintSetVar (133x) - 57395: 49, // hintShuffleJoin (133x) - 57371: 50, // hintSkipScan (133x) - 57393: 51, // hintSMJoin (133x) - 57407: 52, // hintStraightJoin (133x) - 57396: 53, // hintStreamAgg (133x) - 57397: 54, // hintSwapJoinInputs (133x) - 57402: 55, // hintTimeRange (133x) - 57403: 56, // hintUseCascades (133x) - 57399: 57, // hintUseIndex (133x) - 57398: 58, // hintUseIndexMerge (133x) - 57400: 59, // hintUsePlanCache (133x) - 57401: 60, // hintUseToja (133x) - 44: 61, // ',' (130x) - 57420: 62, // hintDupsWeedOut (110x) - 57421: 63, // hintFirstMatch (110x) - 57422: 64, // hintLooseScan (110x) - 57423: 65, // hintMaterialization (110x) - 57415: 66, // hintTiFlash (110x) - 57414: 67, // hintTiKV (110x) - 57416: 68, // hintFalse (109x) - 57411: 69, // hintOLAP (109x) - 57412: 70, // hintOLTP (109x) - 57417: 71, // hintTrue (109x) - 57419: 72, // hintGB (108x) - 57418: 73, // hintMB (108x) - 57347: 74, // hintIdentifier (107x) - 57349: 75, // hintSingleAtIdentifier (93x) - 93: 76, // ']' (84x) - 46: 77, // '.' (83x) - 57413: 78, // hintPartition (78x) - 61: 79, // '=' (74x) - 40: 80, // '(' (69x) - 57344: 81, // $end (25x) - 57444: 82, // QueryBlockOpt (20x) - 57436: 83, // Identifier (15x) - 57346: 84, // hintIntLit (8x) - 57350: 85, // hintStringLit (5x) - 57426: 86, // CommaOpt (4x) - 57432: 87, // HintTable (4x) - 57433: 88, // HintTableList (4x) - 91: 89, // '[' (3x) - 57425: 90, // BooleanHintName (2x) - 57427: 91, // HintIndexList (2x) - 57429: 92, // HintStorageType (2x) - 57430: 93, // HintStorageTypeAndTable (2x) - 57434: 94, // HintTableListOpt (2x) - 57439: 95, // JoinOrderOptimizerHintName (2x) - 57440: 96, // NullaryHintName (2x) - 57443: 97, // PartitionListOpt (2x) - 57446: 98, // StorageOptimizerHintOpt (2x) - 57447: 99, // SubqueryOptimizerHintName (2x) - 57450: 100, // SubqueryStrategy (2x) - 57451: 101, // SupportedIndexLevelOptimizerHintName (2x) - 57452: 102, // SupportedTableLevelOptimizerHintName (2x) - 57453: 103, // TableOptimizerHintOpt (2x) - 57455: 104, // UnsupportedIndexLevelOptimizerHintName (2x) - 57456: 105, // UnsupportedTableLevelOptimizerHintName (2x) - 57458: 106, // ViewName (2x) - 57428: 107, // HintQueryType (1x) - 57431: 108, // HintStorageTypeAndTableList (1x) - 57435: 109, // HintTrueOrFalse (1x) - 57437: 110, // IndexNameList (1x) - 57438: 111, // IndexNameListOpt (1x) - 57441: 112, // OptimizerHintList (1x) - 57442: 113, // PartitionList (1x) - 57445: 114, // Start (1x) - 57448: 115, // SubqueryStrategies (1x) - 57449: 116, // SubqueryStrategiesOpt (1x) - 57454: 117, // UnitOfBytes (1x) - 57457: 118, // Value (1x) - 57459: 119, // ViewNameList (1x) - 57424: 120, // $default (0x) - 57345: 121, // error (0x) - 57348: 122, // hintInvalid (0x) + 41: 0, // ')' (148x) + 57379: 1, // hintAggToCop (135x) + 57394: 2, // hintBCJoin (135x) + 57355: 3, // hintBKA (135x) + 57357: 4, // hintBNL (135x) + 57408: 5, // hintForceIndex (135x) + 57381: 6, // hintHashAgg (135x) + 57359: 7, // hintHashJoin (135x) + 57360: 8, // hintHashJoinBuild (135x) + 57361: 9, // hintHashJoinProbe (135x) + 57384: 10, // hintIgnoreIndex (135x) + 57380: 11, // hintIgnorePlanCache (135x) + 57365: 12, // hintIndexMerge (135x) + 57385: 13, // hintInlHashJoin (135x) + 57386: 14, // hintInlJoin (135x) + 57387: 15, // hintInlMergeJoin (135x) + 57351: 16, // hintJoinFixedOrder (135x) + 57352: 17, // hintJoinOrder (135x) + 57353: 18, // hintJoinPrefix (135x) + 57354: 19, // hintJoinSuffix (135x) + 57400: 20, // hintKeepOrder (135x) + 57410: 21, // hintLeading (135x) + 57407: 22, // hintLimitToCop (135x) + 57375: 23, // hintMaxExecutionTime (135x) + 57388: 24, // hintMemoryQuota (135x) + 57363: 25, // hintMerge (135x) + 57382: 26, // hintMpp1PhaseAgg (135x) + 57383: 27, // hintMpp2PhaseAgg (135x) + 57367: 28, // hintMRR (135x) + 57356: 29, // hintNoBKA (135x) + 57358: 30, // hintNoBNL (135x) + 57412: 31, // hintNoDecorrelate (135x) + 57362: 32, // hintNoHashJoin (135x) + 57369: 33, // hintNoICP (135x) + 57366: 34, // hintNoIndexMerge (135x) + 57401: 35, // hintNoKeepOrder (135x) + 57364: 36, // hintNoMerge (135x) + 57368: 37, // hintNoMRR (135x) + 57370: 38, // hintNoRangeOptimization (135x) + 57374: 39, // hintNoSemijoin (135x) + 57372: 40, // hintNoSkipScan (135x) + 57389: 41, // hintNoSwapJoinInputs (135x) + 57406: 42, // hintNthPlan (135x) + 57378: 43, // hintQBName (135x) + 57390: 44, // hintQueryType (135x) + 57391: 45, // hintReadConsistentReplica (135x) + 57392: 46, // hintReadFromStorage (135x) + 57377: 47, // hintResourceGroup (135x) + 57373: 48, // hintSemijoin (135x) + 57411: 49, // hintSemiJoinRewrite (135x) + 57376: 50, // hintSetVar (135x) + 57395: 51, // hintShuffleJoin (135x) + 57371: 52, // hintSkipScan (135x) + 57393: 53, // hintSMJoin (135x) + 57409: 54, // hintStraightJoin (135x) + 57396: 55, // hintStreamAgg (135x) + 57397: 56, // hintSwapJoinInputs (135x) + 57404: 57, // hintTimeRange (135x) + 57405: 58, // hintUseCascades (135x) + 57399: 59, // hintUseIndex (135x) + 57398: 60, // hintUseIndexMerge (135x) + 57402: 61, // hintUsePlanCache (135x) + 57403: 62, // hintUseToja (135x) + 44: 63, // ',' (132x) + 57422: 64, // hintDupsWeedOut (112x) + 57423: 65, // hintFirstMatch (112x) + 57424: 66, // hintLooseScan (112x) + 57425: 67, // hintMaterialization (112x) + 57417: 68, // hintTiFlash (112x) + 57416: 69, // hintTiKV (112x) + 57418: 70, // hintFalse (111x) + 57413: 71, // hintOLAP (111x) + 57414: 72, // hintOLTP (111x) + 57419: 73, // hintTrue (111x) + 57421: 74, // hintGB (110x) + 57420: 75, // hintMB (110x) + 57347: 76, // hintIdentifier (109x) + 57349: 77, // hintSingleAtIdentifier (95x) + 93: 78, // ']' (86x) + 46: 79, // '.' (85x) + 57415: 80, // hintPartition (80x) + 61: 81, // '=' (76x) + 40: 82, // '(' (71x) + 57344: 83, // $end (25x) + 57446: 84, // QueryBlockOpt (20x) + 57438: 85, // Identifier (15x) + 57346: 86, // hintIntLit (8x) + 57350: 87, // hintStringLit (5x) + 57428: 88, // CommaOpt (4x) + 57434: 89, // HintTable (4x) + 57435: 90, // HintTableList (4x) + 91: 91, // '[' (3x) + 57427: 92, // BooleanHintName (2x) + 57429: 93, // HintIndexList (2x) + 57431: 94, // HintStorageType (2x) + 57432: 95, // HintStorageTypeAndTable (2x) + 57436: 96, // HintTableListOpt (2x) + 57441: 97, // JoinOrderOptimizerHintName (2x) + 57442: 98, // NullaryHintName (2x) + 57445: 99, // PartitionListOpt (2x) + 57448: 100, // StorageOptimizerHintOpt (2x) + 57449: 101, // SubqueryOptimizerHintName (2x) + 57452: 102, // SubqueryStrategy (2x) + 57453: 103, // SupportedIndexLevelOptimizerHintName (2x) + 57454: 104, // SupportedTableLevelOptimizerHintName (2x) + 57455: 105, // TableOptimizerHintOpt (2x) + 57457: 106, // UnsupportedIndexLevelOptimizerHintName (2x) + 57458: 107, // UnsupportedTableLevelOptimizerHintName (2x) + 57460: 108, // ViewName (2x) + 57430: 109, // HintQueryType (1x) + 57433: 110, // HintStorageTypeAndTableList (1x) + 57437: 111, // HintTrueOrFalse (1x) + 57439: 112, // IndexNameList (1x) + 57440: 113, // IndexNameListOpt (1x) + 57443: 114, // OptimizerHintList (1x) + 57444: 115, // PartitionList (1x) + 57447: 116, // Start (1x) + 57450: 117, // SubqueryStrategies (1x) + 57451: 118, // SubqueryStrategiesOpt (1x) + 57456: 119, // UnitOfBytes (1x) + 57459: 120, // Value (1x) + 57461: 121, // ViewNameList (1x) + 57426: 122, // $default (0x) + 57345: 123, // error (0x) + 57348: 124, // hintInvalid (0x) } yyhintSymNames = []string{ @@ -275,6 +279,7 @@ var ( "hintJoinOrder", "hintJoinPrefix", "hintJoinSuffix", + "hintKeepOrder", "hintLeading", "hintLimitToCop", "hintMaxExecutionTime", @@ -289,6 +294,7 @@ var ( "hintNoHashJoin", "hintNoICP", "hintNoIndexMerge", + "hintNoKeepOrder", "hintNoMerge", "hintNoMRR", "hintNoRangeOptimization", @@ -382,89 +388,76 @@ var ( yyhintReductions = []struct{ xsym, components int }{ {0, 1}, + {116, 1}, {114, 1}, - {112, 1}, - {112, 3}, - {112, 1}, - {112, 3}, - {103, 4}, - {103, 4}, - {103, 4}, - {103, 4}, - {103, 4}, - {103, 4}, - {103, 5}, - {103, 5}, - {103, 5}, - {103, 6}, - {103, 4}, - {103, 4}, - {103, 6}, - {103, 6}, - {103, 6}, - {103, 5}, - {103, 4}, - {103, 5}, - {98, 5}, - {108, 1}, - {108, 3}, - {93, 4}, - {82, 0}, - {82, 1}, - {86, 0}, - {86, 1}, - {97, 0}, - {97, 4}, - {113, 1}, - {113, 3}, - {94, 1}, - {94, 1}, - {88, 2}, - {88, 3}, - {87, 3}, - {87, 5}, - {119, 3}, - {119, 1}, - {106, 2}, - {106, 1}, - {91, 4}, - {111, 0}, - {111, 1}, + {114, 3}, + {114, 1}, + {114, 3}, + {105, 4}, + {105, 4}, + {105, 4}, + {105, 4}, + {105, 4}, + {105, 4}, + {105, 5}, + {105, 5}, + {105, 5}, + {105, 6}, + {105, 4}, + {105, 4}, + {105, 6}, + {105, 6}, + {105, 6}, + {105, 5}, + {105, 4}, + {105, 5}, + {100, 5}, {110, 1}, {110, 3}, - {116, 0}, - {116, 1}, + {95, 4}, + {84, 0}, + {84, 1}, + {88, 0}, + {88, 1}, + {99, 0}, + {99, 4}, {115, 1}, {115, 3}, + {96, 1}, + {96, 1}, + {90, 2}, + {90, 3}, + {89, 3}, + {89, 5}, + {121, 3}, + {121, 1}, + {108, 2}, + {108, 1}, + {93, 4}, + {113, 0}, + {113, 1}, + {112, 1}, + {112, 3}, + {118, 0}, {118, 1}, - {118, 1}, - {118, 1}, - {117, 1}, {117, 1}, - {109, 1}, - {109, 1}, - {95, 1}, - {95, 1}, - {95, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, + {117, 3}, + {120, 1}, + {120, 1}, + {120, 1}, + {119, 1}, + {119, 1}, + {111, 1}, + {111, 1}, + {97, 1}, + {97, 1}, + {97, 1}, + {107, 1}, + {107, 1}, + {107, 1}, + {107, 1}, + {107, 1}, + {107, 1}, {104, 1}, {104, 1}, {104, 1}, @@ -472,449 +465,471 @@ var ( {104, 1}, {104, 1}, {104, 1}, + {104, 1}, + {104, 1}, + {104, 1}, + {104, 1}, + {104, 1}, + {104, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {103, 1}, + {103, 1}, + {103, 1}, + {103, 1}, + {103, 1}, + {103, 1}, {101, 1}, {101, 1}, - {101, 1}, - {101, 1}, - {99, 1}, - {99, 1}, - {100, 1}, - {100, 1}, - {100, 1}, - {100, 1}, - {90, 1}, - {90, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {107, 1}, - {107, 1}, + {102, 1}, + {102, 1}, + {102, 1}, + {102, 1}, {92, 1}, {92, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {109, 1}, + {109, 1}, + {94, 1}, + {94, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, } yyhintXErrors = map[yyhintXError]string{} - yyhintParseTab = [280][]uint16{ + yyhintParseTab = [284][]uint16{ // 0 - {1: 258, 226, 219, 221, 248, 254, 234, 235, 236, 246, 262, 238, 230, 228, 233, 198, 216, 217, 218, 237, 259, 205, 210, 229, 255, 256, 239, 220, 222, 265, 223, 241, 260, 224, 240, 242, 250, 244, 232, 206, 209, 214, 261, 215, 208, 249, 264, 207, 227, 243, 225, 263, 257, 231, 211, 252, 245, 247, 253, 251, 90: 212, 95: 199, 213, 98: 197, 204, 101: 203, 201, 196, 202, 200, 112: 195, 114: 194}, - {81: 193}, - {1: 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 361, 81: 192, 86: 470}, - {1: 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 81: 191}, - {1: 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 81: 189}, + {1: 264, 230, 223, 225, 252, 260, 238, 239, 240, 250, 268, 242, 234, 232, 237, 202, 220, 221, 222, 253, 241, 265, 209, 214, 233, 261, 262, 243, 224, 226, 271, 227, 245, 266, 254, 228, 244, 246, 256, 248, 236, 210, 213, 218, 267, 219, 212, 255, 270, 211, 231, 247, 229, 269, 263, 235, 215, 258, 249, 251, 259, 257, 92: 216, 97: 203, 217, 100: 201, 208, 103: 207, 205, 200, 206, 204, 114: 199, 116: 198}, + {83: 197}, + {1: 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 369, 83: 196, 88: 478}, + {1: 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 83: 195}, + {1: 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 83: 193}, // 5 - {80: 467}, - {80: 464}, - {80: 461}, - {80: 456}, - {80: 453}, + {82: 475}, + {82: 472}, + {82: 469}, + {82: 464}, + {82: 461}, // 10 - {80: 442}, - {80: 430}, - {80: 426}, - {80: 422}, - {80: 414}, + {82: 450}, + {82: 438}, + {82: 434}, + {82: 430}, + {82: 422}, // 15 - {80: 411}, - {80: 399}, - {80: 392}, - {80: 387}, - {80: 381}, + {82: 419}, + {82: 407}, + {82: 400}, + {82: 395}, + {82: 389}, // 20 - {80: 378}, - {80: 372}, - {80: 266}, - {80: 131}, - {80: 130}, + {82: 386}, + {82: 380}, + {82: 272}, + {82: 135}, + {82: 134}, // 25 - {80: 129}, - {80: 128}, - {80: 127}, - {80: 126}, - {80: 125}, + {82: 133}, + {82: 132}, + {82: 131}, + {82: 130}, + {82: 129}, // 30 - {80: 124}, - {80: 123}, - {80: 122}, - {80: 121}, - {80: 120}, + {82: 128}, + {82: 127}, + {82: 126}, + {82: 125}, + {82: 124}, // 35 - {80: 119}, - {80: 118}, - {80: 117}, - {80: 116}, - {80: 115}, + {82: 123}, + {82: 122}, + {82: 121}, + {82: 120}, + {82: 119}, // 40 - {80: 114}, - {80: 113}, - {80: 112}, - {80: 111}, - {80: 110}, + {82: 118}, + {82: 117}, + {82: 116}, + {82: 115}, + {82: 114}, // 45 - {80: 109}, - {80: 108}, - {80: 107}, - {80: 106}, - {80: 105}, + {82: 113}, + {82: 112}, + {82: 111}, + {82: 110}, + {82: 109}, // 50 - {80: 104}, - {80: 103}, - {80: 102}, - {80: 101}, - {80: 100}, + {82: 108}, + {82: 107}, + {82: 106}, + {82: 105}, + {82: 104}, // 55 - {80: 99}, - {80: 98}, - {80: 97}, - {80: 92}, - {80: 91}, + {82: 103}, + {82: 102}, + {82: 101}, + {82: 100}, + {82: 99}, // 60 - {80: 90}, - {80: 89}, - {80: 88}, - {80: 87}, - {80: 86}, + {82: 94}, + {82: 93}, + {82: 92}, + {82: 91}, + {82: 90}, // 65 - {80: 85}, - {80: 84}, - {80: 83}, - {80: 82}, - {80: 81}, + {82: 89}, + {82: 88}, + {82: 87}, + {82: 86}, + {82: 85}, // 70 - {80: 80}, - {80: 79}, - {80: 78}, - {66: 165, 165, 75: 268, 82: 267}, - {66: 273, 272, 92: 271, 270, 108: 269}, + {82: 84}, + {82: 83}, + {82: 82}, + {82: 81}, + {82: 80}, // 75 - {164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 76: 164, 164, 164, 84: 164}, - {369, 61: 370}, - {168, 61: 168}, - {89: 274}, - {89: 75}, + {68: 169, 169, 77: 274, 84: 273}, + {68: 279, 278, 94: 277, 276, 110: 275}, + {168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 78: 168, 168, 168, 86: 168}, + {377, 63: 378}, + {172, 63: 172}, // 80 - {89: 74}, - {1: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 276, 88: 275}, - {61: 367, 76: 366}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 278, 87: 277}, - {155, 61: 155, 76: 155}, + {91: 280}, + {91: 77}, + {91: 76}, + {1: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 282, 90: 281}, + {63: 375, 78: 374}, // 85 - {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 165, 353, 165, 82: 352}, - {73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73}, - {72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72}, - {71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71}, - {70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 284, 89: 283}, + {159, 63: 159, 78: 159}, + {169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 169, 361, 169, 84: 360}, + {75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75}, + {74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74}, // 90 - {69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69}, - {68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68}, - {67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67}, - {66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66}, - {65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65}, + {73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73}, + {72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72}, + {71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71}, + {70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70}, + {69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69}, // 95 - {64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64}, - {63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63}, - {62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62}, - {61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61}, - {60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60}, + {68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68}, + {67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67}, + {66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66}, + {65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65}, + {64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64}, // 100 - {59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59}, - {58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58}, - {57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57}, - {56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56}, - {55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55}, + {63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63}, + {62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62}, + {61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61}, + {60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60}, + {59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59}, // 105 - {54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54}, - {53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53}, - {52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52}, - {51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51}, - {50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50}, + {58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58}, + {57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57}, + {56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56}, + {55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55}, + {54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54}, // 110 - {49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49}, - {48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48}, - {47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47}, - {46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46}, - {45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45}, + {53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53}, + {52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52}, + {51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51}, + {50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50}, + {49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49}, // 115 - {44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44}, - {43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43}, - {42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42}, - {41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41}, - {40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40}, + {48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48}, + {47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47}, + {46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46}, + {45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45}, + {44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44}, // 120 - {39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39}, - {38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38}, - {37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37}, - {36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36}, - {35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35}, + {43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43}, + {42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42}, + {41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41}, + {40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40}, + {39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39}, // 125 - {34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34}, - {33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33}, - {32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32}, - {31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31}, - {30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30}, + {38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38}, + {37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37}, + {36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36}, + {35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35}, + {34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34}, // 130 - {29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29}, - {28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28}, - {27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27}, - {26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26}, - {25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25}, + {33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33}, + {32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32}, + {31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31}, + {30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30}, + {29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29}, // 135 - {24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24}, - {23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23}, - {22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22}, - {21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21}, - {20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20}, + {28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28}, + {27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27}, + {26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26}, + {25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25}, + {24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24}, // 140 - {19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19}, - {18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18}, - {17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17}, - {16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16}, - {15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15}, + {23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23}, + {22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22}, + {21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21}, + {20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20}, + {19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19}, // 145 - {14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14}, - {13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13}, - {12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12}, - {11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11}, - {10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10}, + {18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18}, + {17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17}, + {16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16}, + {15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15}, + {14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14}, // 150 - {9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9}, - {8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8}, - {7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7}, - {6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6}, - {5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5}, + {13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13}, + {12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12}, + {11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11}, + {10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10}, + {9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9}, // 155 - {4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4}, - {3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3}, - {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2}, - {1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1}, - {161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 76: 161, 78: 356, 97: 365}, + {8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8}, + {7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7}, + {6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6}, + {5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5}, + {4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4}, // 160 - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 354}, - {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 165, 78: 165, 82: 355}, - {161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 76: 161, 78: 356, 97: 357}, - {80: 358}, - {152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 76: 152}, + {3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3}, + {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2}, + {1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1}, + {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 78: 165, 80: 364, 99: 373}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 362}, // 165 - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 360, 113: 359}, - {362, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 361, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 86: 363}, - {159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159}, - {162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 62: 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 85: 162}, - {160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 76: 160}, + {169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 169, 80: 169, 84: 363}, + {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 78: 165, 80: 364, 99: 365}, + {82: 366}, + {156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 78: 156}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 368, 115: 367}, // 170 - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 364}, - {158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158}, - {153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 76: 153}, - {166, 61: 166}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 278, 87: 368}, + {370, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 369, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 88: 371}, + {163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163}, + {166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 64: 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 87: 166}, + {164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 78: 164}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 372}, // 175 - {154, 61: 154, 76: 154}, - {1: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 81: 169}, - {66: 273, 272, 92: 271, 371}, - {167, 61: 167}, - {69: 165, 165, 75: 268, 82: 373}, + {162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162}, + {157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 78: 157}, + {170, 63: 170}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 284, 89: 376}, + {158, 63: 158, 78: 158}, // 180 - {69: 375, 376, 107: 374}, - {377}, - {77}, - {76}, - {1: 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 81: 170}, + {1: 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 83: 173}, + {68: 279, 278, 94: 277, 379}, + {171, 63: 171}, + {71: 169, 169, 77: 274, 84: 381}, + {71: 383, 384, 109: 382}, // 185 - {165, 75: 268, 82: 379}, - {380}, - {1: 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 81: 171}, - {68: 165, 71: 165, 75: 268, 82: 382}, - {68: 385, 71: 384, 109: 383}, + {385}, + {79}, + {78}, + {1: 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 83: 174}, + {169, 77: 274, 84: 387}, // 190 - {386}, - {133}, - {132}, - {1: 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 81: 172}, - {85: 388}, + {388}, + {1: 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 83: 175}, + {70: 169, 73: 169, 77: 274, 84: 390}, + {70: 393, 73: 392, 111: 391}, + {394}, // 195 - {61: 361, 85: 163, 389}, - {85: 390}, - {391}, - {1: 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 81: 173}, - {75: 268, 82: 393, 84: 165}, + {137}, + {136}, + {1: 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 83: 176}, + {87: 396}, + {63: 369, 87: 167, 397}, // 200 - {84: 394}, - {72: 397, 396, 117: 395}, - {398}, - {135}, - {134}, + {87: 398}, + {399}, + {1: 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 83: 177}, + {77: 274, 84: 401, 86: 169}, + {86: 402}, // 205 - {1: 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 81: 174}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 400}, - {401, 61: 402}, - {1: 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 81: 176}, - {165, 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 268, 77: 165, 82: 406, 405, 106: 404, 119: 403}, + {74: 405, 404, 119: 403}, + {406}, + {139}, + {138}, + {1: 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 83: 178}, // 210 - {408, 77: 409}, - {150, 77: 150}, - {165, 75: 268, 77: 165, 82: 407}, - {148, 77: 148}, - {149, 77: 149}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 408}, + {409, 63: 410}, + {1: 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 83: 180}, + {169, 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 274, 79: 169, 84: 414, 413, 108: 412, 121: 411}, + {416, 79: 417}, // 215 - {1: 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 81: 175}, - {165, 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 268, 77: 165, 82: 406, 405, 106: 410}, - {151, 77: 151}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 412}, - {413}, + {154, 79: 154}, + {169, 77: 274, 79: 169, 84: 415}, + {152, 79: 152}, + {153, 79: 153}, + {1: 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 83: 179}, // 220 - {1: 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 81: 177}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 415}, - {79: 416}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 419, 420, 418, 118: 417}, + {169, 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 274, 79: 169, 84: 414, 413, 108: 418}, + {155, 79: 155}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 420}, {421}, + {1: 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 83: 181}, // 225 - {138}, - {137}, - {136}, - {1: 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 81: 178}, - {75: 268, 82: 423, 84: 165}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 423}, + {81: 424}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 427, 428, 426, 120: 425}, + {429}, + {142}, // 230 - {84: 424}, - {425}, - {1: 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 81: 179}, - {75: 268, 82: 427, 84: 165}, - {84: 428}, + {141}, + {140}, + {1: 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 83: 182}, + {77: 274, 84: 431, 86: 169}, + {86: 432}, // 235 - {429}, - {1: 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 81: 180}, - {165, 62: 165, 165, 165, 165, 75: 268, 82: 431}, - {142, 62: 435, 436, 437, 438, 100: 434, 115: 433, 432}, - {441}, + {433}, + {1: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 83: 183}, + {77: 274, 84: 435, 86: 169}, + {86: 436}, + {437}, // 240 - {141, 61: 439}, - {140, 61: 140}, - {96, 61: 96}, - {95, 61: 95}, - {94, 61: 94}, + {1: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 83: 184}, + {169, 64: 169, 169, 169, 169, 77: 274, 84: 439}, + {146, 64: 443, 444, 445, 446, 102: 442, 117: 441, 440}, + {449}, + {145, 63: 447}, // 245 - {93, 61: 93}, - {62: 435, 436, 437, 438, 100: 440}, - {139, 61: 139}, - {1: 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 81: 181}, - {1: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 444, 91: 443}, + {144, 63: 144}, + {98, 63: 98}, + {97, 63: 97}, + {96, 63: 96}, + {95, 63: 95}, // 250 - {452}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 278, 87: 445}, - {163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 361, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 86: 446}, - {146, 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 449, 110: 448, 447}, - {147}, + {64: 443, 444, 445, 446, 102: 448}, + {143, 63: 143}, + {1: 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 83: 185}, + {1: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 452, 93: 451}, + {460}, // 255 - {145, 61: 450}, - {144, 61: 144}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 451}, - {143, 61: 143}, - {1: 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 81: 182}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 284, 89: 453}, + {167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 369, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 88: 454}, + {150, 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 457, 112: 456, 455}, + {151}, + {149, 63: 458}, // 260 - {1: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 444, 91: 454}, - {455}, - {1: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 81: 183}, - {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 459, 88: 458, 94: 457}, - {460}, + {148, 63: 148}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 459}, + {147, 63: 147}, + {1: 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 83: 186}, + {1: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 452, 93: 462}, // 265 - {157, 61: 367}, - {156, 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 278, 87: 277}, - {1: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 81: 184}, - {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 459, 88: 458, 94: 462}, {463}, + {1: 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 83: 187}, + {169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 467, 90: 466, 96: 465}, + {468}, + {161, 63: 375}, // 270 - {1: 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 81: 185}, - {1: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 276, 88: 465}, - {466, 61: 367}, - {1: 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 81: 186}, - {165, 75: 268, 82: 468}, + {160, 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 284, 89: 283}, + {1: 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 83: 188}, + {169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 467, 90: 466, 96: 470}, + {471}, + {1: 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 83: 189}, // 275 - {469}, - {1: 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 81: 187}, - {1: 258, 226, 219, 221, 248, 254, 234, 235, 236, 246, 262, 238, 230, 228, 233, 198, 216, 217, 218, 237, 259, 205, 210, 229, 255, 256, 239, 220, 222, 265, 223, 241, 260, 224, 240, 242, 250, 244, 232, 206, 209, 214, 261, 215, 208, 249, 264, 207, 227, 243, 225, 263, 257, 231, 211, 252, 245, 247, 253, 251, 90: 212, 95: 199, 213, 98: 472, 204, 101: 203, 201, 471, 202, 200}, - {1: 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 81: 190}, - {1: 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 81: 188}, + {1: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 282, 90: 473}, + {474, 63: 375}, + {1: 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 83: 190}, + {169, 77: 274, 84: 476}, + {477}, + // 280 + {1: 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 83: 191}, + {1: 264, 230, 223, 225, 252, 260, 238, 239, 240, 250, 268, 242, 234, 232, 237, 202, 220, 221, 222, 253, 241, 265, 209, 214, 233, 261, 262, 243, 224, 226, 271, 227, 245, 266, 254, 228, 244, 246, 256, 248, 236, 210, 213, 218, 267, 219, 212, 255, 270, 211, 231, 247, 229, 269, 263, 235, 215, 258, 249, 251, 259, 257, 92: 216, 97: 203, 217, 100: 480, 208, 103: 207, 205, 479, 206, 204}, + {1: 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 83: 194}, + {1: 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 83: 192}, } ) @@ -954,7 +969,7 @@ func yyhintlex1(yylex yyhintLexer, lval *yyhintSymType) (n int) { } func yyhintParse(yylex yyhintLexer, parser *hintParser) int { - const yyError = 121 + const yyError = 123 yyEx, _ := yylex.(yyhintLexerEx) var yyn int diff --git a/parser/hintparser.y b/parser/hintparser.y index a2ccd21d2d401..0d92905bb86f3 100644 --- a/parser/hintparser.y +++ b/parser/hintparser.y @@ -103,6 +103,8 @@ import ( hintSwapJoinInputs "SWAP_JOIN_INPUTS" hintUseIndexMerge "USE_INDEX_MERGE" hintUseIndex "USE_INDEX" + hintKeepOrder "KEEP_ORDER" + hintNoKeepOrder "NO_KEEP_ORDER" hintUsePlanCache "USE_PLAN_CACHE" hintUseToja "USE_TOJA" hintTimeRange "TIME_RANGE" @@ -607,6 +609,8 @@ SupportedIndexLevelOptimizerHintName: | "IGNORE_INDEX" | "USE_INDEX_MERGE" | "FORCE_INDEX" +| "KEEP_ORDER" +| "NO_KEEP_ORDER" SubqueryOptimizerHintName: "SEMIJOIN" @@ -699,6 +703,8 @@ Identifier: | "SWAP_JOIN_INPUTS" | "USE_INDEX_MERGE" | "USE_INDEX" +| "KEEP_ORDER" +| "NO_KEEP_ORDER" | "USE_PLAN_CACHE" | "USE_TOJA" | "TIME_RANGE" diff --git a/parser/misc.go b/parser/misc.go index 3c859c30283b1..c5bda32024409 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -956,6 +956,8 @@ var hintTokenMap = map[string]int{ "SWAP_JOIN_INPUTS": hintSwapJoinInputs, "USE_INDEX_MERGE": hintUseIndexMerge, "USE_INDEX": hintUseIndex, + "KEEP_ORDER": hintKeepOrder, + "NO_KEEP_ORDER": hintNoKeepOrder, "USE_PLAN_CACHE": hintUsePlanCache, "USE_TOJA": hintUseToja, "TIME_RANGE": hintTimeRange, diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 7bb9eaef01e6f..c278ffb56167d 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -316,12 +316,19 @@ type MultiSchemaInfo struct { AddIndexes []CIStr `json:"-"` DropIndexes []CIStr `json:"-"` AlterIndexes []CIStr `json:"-"` - ForeignKeys []CIStr `json:"-"` + + AddForeignKeys []AddForeignKeyInfo `json:"-"` RelativeColumns []CIStr `json:"-"` PositionColumns []CIStr `json:"-"` } +// AddForeignKeyInfo contains foreign key information. +type AddForeignKeyInfo struct { + Name CIStr + Cols []CIStr +} + // NewMultiSchemaInfo new a MultiSchemaInfo. func NewMultiSchemaInfo() *MultiSchemaInfo { return &MultiSchemaInfo{ diff --git a/parser/model/model.go b/parser/model/model.go index 19aabf4a06572..411db3fcf1d20 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -381,8 +381,8 @@ func FindFKInfoByName(fks []*FKInfo, name string) *FKInfo { } // FindIndexByColumns find IndexInfo in indices which is cover the specified columns. -func FindIndexByColumns(tbInfo *TableInfo, cols ...CIStr) *IndexInfo { - for _, index := range tbInfo.Indices { +func FindIndexByColumns(tbInfo *TableInfo, indices []*IndexInfo, cols ...CIStr) *IndexInfo { + for _, index := range indices { if IsIndexPrefixCovered(tbInfo, index, cols...) { return index } diff --git a/parser/mysql/errcode.go b/parser/mysql/errcode.go index 0ea2134f88d75..9861d3e853a00 100644 --- a/parser/mysql/errcode.go +++ b/parser/mysql/errcode.go @@ -944,6 +944,7 @@ const ( ErrFKIncompatibleColumns = 3780 ErrFunctionalIndexRowValueIsNotAllowed = 3800 ErrDependentByFunctionalIndex = 3837 + ErrDependentByPartitionFunctional = 3885 ErrInvalidJsonValueForFuncIndex = 3903 //nolint: revive ErrJsonValueOutOfRangeForFuncIndex = 3904 //nolint: revive ErrFunctionalIndexDataIsTooLong = 3907 diff --git a/parser/mysql/errname.go b/parser/mysql/errname.go index 3066d9f6fb6b0..308de714bb79d 100644 --- a/parser/mysql/errname.go +++ b/parser/mysql/errname.go @@ -956,6 +956,7 @@ var MySQLErrName = map[uint16]*ErrMessage{ ErrJsonValueOutOfRangeForFuncIndex: Message("Out of range JSON value for CAST for functional index '%s'", nil), ErrFunctionalIndexDataIsTooLong: Message("Data too long for functional index '%s'", nil), ErrFunctionalIndexNotApplicable: Message("Cannot use functional index '%s' due to type or collation conversion", nil), + ErrDependentByPartitionFunctional: Message("Column '%s' has a partitioning function dependency and cannot be dropped or renamed", nil), // MariaDB errors. ErrOnlyOneDefaultPartionAllowed: Message("Only one DEFAULT partition allowed", nil), diff --git a/parser/parser_test.go b/parser/parser_test.go index 7b72117f69d16..dae08a80afde3 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -3244,6 +3244,7 @@ func TestDDL(t *testing.T) { {"create table t (a bigint, b bigint as (a+1) not null);", true, "CREATE TABLE `t` (`a` BIGINT,`b` BIGINT GENERATED ALWAYS AS(`a`+1) VIRTUAL NOT NULL)"}, {"create table t (a bigint, b bigint as (a+1) not null);", true, "CREATE TABLE `t` (`a` BIGINT,`b` BIGINT GENERATED ALWAYS AS(`a`+1) VIRTUAL NOT NULL)"}, {"create table t (a bigint, b bigint as (a+1) not null comment 'ttt');", true, "CREATE TABLE `t` (`a` BIGINT,`b` BIGINT GENERATED ALWAYS AS(`a`+1) VIRTUAL NOT NULL COMMENT 'ttt')"}, + {"create table t(a int, index idx((cast(a as binary(1)))));", true, "CREATE TABLE `t` (`a` INT,INDEX `idx`((CAST(`a` AS BINARY(1)))))"}, {"alter table t add column (f timestamp as (a+1) default '2019-01-01 11:11:11');", false, ""}, {"alter table t modify column f int as (a+1) default 55;", false, ""}, @@ -3891,6 +3892,44 @@ func TestOptimizerHints(t *testing.T) { require.Len(t, hints[1].Indexes, 1) require.Equal(t, "t4", hints[1].Indexes[0].L) + // Test KEEP_ORDER + stmt, _, err = p.Parse("select /*+ KEEP_ORDER(T1,T2), keep_order(t3,t4) */ c1, c2 from t1, t2 where t1.c1 = t2.c1", "", "") + require.NoError(t, err) + selectStmt = stmt[0].(*ast.SelectStmt) + + hints = selectStmt.TableHints + require.Len(t, hints, 2) + require.Equal(t, "keep_order", hints[0].HintName.L) + require.Len(t, hints[0].Tables, 1) + require.Equal(t, "t1", hints[0].Tables[0].TableName.L) + require.Len(t, hints[0].Indexes, 1) + require.Equal(t, "t2", hints[0].Indexes[0].L) + + require.Equal(t, "keep_order", hints[1].HintName.L) + require.Len(t, hints[1].Tables, 1) + require.Equal(t, "t3", hints[1].Tables[0].TableName.L) + require.Len(t, hints[1].Indexes, 1) + require.Equal(t, "t4", hints[1].Indexes[0].L) + + // Test NO_KEEP_ORDER + stmt, _, err = p.Parse("select /*+ NO_KEEP_ORDER(T1,T2), no_keep_order(t3,t4) */ c1, c2 from t1, t2 where t1.c1 = t2.c1", "", "") + require.NoError(t, err) + selectStmt = stmt[0].(*ast.SelectStmt) + + hints = selectStmt.TableHints + require.Len(t, hints, 2) + require.Equal(t, "no_keep_order", hints[0].HintName.L) + require.Len(t, hints[0].Tables, 1) + require.Equal(t, "t1", hints[0].Tables[0].TableName.L) + require.Len(t, hints[0].Indexes, 1) + require.Equal(t, "t2", hints[0].Indexes[0].L) + + require.Equal(t, "no_keep_order", hints[1].HintName.L) + require.Len(t, hints[1].Tables, 1) + require.Equal(t, "t3", hints[1].Tables[0].TableName.L) + require.Len(t, hints[1].Indexes, 1) + require.Equal(t, "t4", hints[1].Indexes[0].L) + // Test TIDB_SMJ stmt, _, err = p.Parse("select /*+ TIDB_SMJ(T1,t2), tidb_smj(T3,t4) */ c1, c2 from t1, t2 where t1.c1 = t2.c1", "", "") require.NoError(t, err) diff --git a/parser/types/field_type.go b/parser/types/field_type.go index ff0ac9793cf17..991dc3d087d75 100644 --- a/parser/types/field_type.go +++ b/parser/types/field_type.go @@ -518,7 +518,7 @@ func (ft *FieldType) Restore(ctx *format.RestoreCtx) error { // RestoreAsCastType is used for write AST back to string. func (ft *FieldType) RestoreAsCastType(ctx *format.RestoreCtx, explicitCharset bool) { switch ft.tp { - case mysql.TypeVarString: + case mysql.TypeVarString, mysql.TypeString: skipWriteBinary := false if ft.charset == charset.CharsetBin && ft.collate == charset.CollationBin { ctx.WriteKeyWord("BINARY") diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index 0495f1f50cae0..b7f37923547be 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "handle_cols.go", "hashcode.go", "hints.go", + "indexmerge_path.go", "initialize.go", "logical_plan_builder.go", "logical_plans.go", @@ -170,6 +171,7 @@ go_test( "flat_plan_test.go", "fragment_test.go", "indexmerge_intersection_test.go", + "indexmerge_path_test.go", "indexmerge_test.go", "integration_partition_test.go", "integration_test.go", diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 8feb357745853..53aa657d2f9c0 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -252,6 +252,8 @@ const ( OpSetBindingStatus // OpSQLBindDropByDigest is used to drop SQL binds by digest OpSQLBindDropByDigest + // OpSetBindingStatusByDigest represents the operation to set SQL binding status by sql digest. + OpSetBindingStatusByDigest ) // SQLBindPlan represents a plan for SQL bind. diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 2817f370ffcec..dfd8d9572db81 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1928,7 +1928,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P } }) - if (p.preferJoinType&preferBCJoin) == 0 && p.preferJoinType > 0 { + if (p.preferJoinType&preferBCJoin) == 0 && (p.preferJoinType&preferShuffleJoin) == 0 && p.preferJoinType > 0 { p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because you have used hint to specify a join algorithm which is not supported by mpp now.") if prop.IsFlashProp() { return nil, false, nil @@ -1957,6 +1957,21 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P mppJoins := p.tryToGetMppHashJoin(prop, false) joins = append(joins, mppJoins...) } + } else { + hasMppHints := false + var errMsg string + if (p.preferJoinType & preferShuffleJoin) > 0 { + errMsg = "The join can not push down to the MPP side, the shuffle_join() hint is invalid" + hasMppHints = true + } + if (p.preferJoinType & preferBCJoin) > 0 { + errMsg = "The join can not push down to the MPP side, the broadcast_join() hint is invalid" + hasMppHints = true + } + if hasMppHints { + warning := ErrInternal.GenWithStack(errMsg) + p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + } } if prop.IsFlashProp() { return joins, true, nil @@ -2858,6 +2873,21 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy } if canPushDownToMPP { taskTypes = append(taskTypes, property.MppTaskType) + } else { + hasMppHints := false + var errMsg string + if la.aggHints.preferAggType&preferMPP1PhaseAgg > 0 { + errMsg = "The agg can not push down to the MPP side, the MPP_1PHASE_AGG() hint is invalid" + hasMppHints = true + } + if la.aggHints.preferAggType&preferMPP2PhaseAgg > 0 { + errMsg = "The agg can not push down to the MPP side, the MPP_2PHASE_AGG() hint is invalid" + hasMppHints = true + } + if hasMppHints { + warning := ErrInternal.GenWithStack(errMsg) + la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + } } if prop.IsFlashProp() { taskTypes = []property.TaskType{prop.TaskTp} diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 639bc15dbdc98..f14e343610b5d 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1419,6 +1419,12 @@ func (ds *DataSource) addSelection4PlanCache(task *rootTask, stats *property.Sta // convertToIndexScan converts the DataSource to index scan with idx. func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candidate *candidatePath, _ *physicalOptimizeOp) (task task, err error) { + if candidate.path.Index.MVIndex { + // MVIndex is special since different index rows may return the same _row_id and this can break some assumptions of IndexReader. + // Currently only support using IndexMerge to access MVIndex instead of IndexReader. + // TODO: make IndexReader support accessing MVIndex directly. + return invalidTask, nil + } if !candidate.path.IsSingleScan { // If it's parent requires single read task, return max cost. if prop.TaskTp == property.CopSingleReadTaskType { diff --git a/planner/core/foreign_key.go b/planner/core/foreign_key.go index 00b87e11c01f8..f6cec0fa8f069 100644 --- a/planner/core/foreign_key.go +++ b/planner/core/foreign_key.go @@ -159,6 +159,17 @@ func (p *Insert) buildOnInsertFKTriggers(ctx sessionctx.Context, is infoschema.I if len(referredFKCascades) > 0 { fkCascades = append(fkCascades, referredFKCascades...) } + } else if p.IsReplace { + referredFKChecks, referredFKCascades, err := p.buildOnReplaceReferredFKTriggers(ctx, is, dbName, tblInfo) + if err != nil { + return err + } + if len(referredFKChecks) > 0 { + fkChecks = append(fkChecks, referredFKChecks...) + } + if len(referredFKCascades) > 0 { + fkCascades = append(fkCascades, referredFKCascades...) + } } for _, fk := range tblInfo.ForeignKeys { if fk.Version < 1 { @@ -186,6 +197,25 @@ func (p *Insert) buildOnDuplicateUpdateColumns() map[string]struct{} { return m } +func (p *Insert) buildOnReplaceReferredFKTriggers(ctx sessionctx.Context, is infoschema.InfoSchema, dbName string, tblInfo *model.TableInfo) ([]*FKCheck, []*FKCascade, error) { + referredFKs := is.GetTableReferredForeignKeys(dbName, tblInfo.Name.L) + fkChecks := make([]*FKCheck, 0, len(referredFKs)) + fkCascades := make([]*FKCascade, 0, len(referredFKs)) + for _, referredFK := range referredFKs { + fkCheck, fkCascade, err := buildOnDeleteOrUpdateFKTrigger(ctx, is, referredFK, FKCascadeOnDelete) + if err != nil { + return nil, nil, err + } + if fkCheck != nil { + fkChecks = append(fkChecks, fkCheck) + } + if fkCascade != nil { + fkCascades = append(fkCascades, fkCascade) + } + } + return fkChecks, fkCascades, nil +} + func (updt *Update) buildOnUpdateFKTriggers(ctx sessionctx.Context, is infoschema.InfoSchema, tblID2table map[int64]table.Table) error { if !ctx.GetSessionVars().ForeignKeyChecks { return nil @@ -414,7 +444,7 @@ func buildFKCheck(ctx sessionctx.Context, tbl table.Table, cols []model.CIStr, f } } - referTbIdxInfo := model.FindIndexByColumns(tblInfo, cols...) + referTbIdxInfo := model.FindIndexByColumns(tblInfo, tblInfo.Indices, cols...) if referTbIdxInfo == nil { return nil, failedErr } @@ -460,7 +490,7 @@ func buildFKCascade(ctx sessionctx.Context, tp FKCascadeType, referredFK *model. return fkCascade, nil } } - indexForFK := model.FindIndexByColumns(childTable.Meta(), fk.Cols...) + indexForFK := model.FindIndexByColumns(childTable.Meta(), childTable.Meta().Indices, 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) } diff --git a/planner/core/fragment.go b/planner/core/fragment.go index c6aec17f21e6d..7e86696ccc4d6 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -16,6 +16,7 @@ package core import ( "context" + "sync/atomic" "time" "unsafe" @@ -79,29 +80,45 @@ type tasksAndFrags struct { } type mppTaskGenerator struct { - ctx sessionctx.Context - startTS uint64 - is infoschema.InfoSchema - frags []*Fragment - cache map[int]tasksAndFrags + ctx sessionctx.Context + startTS uint64 + mppQueryID kv.MPPQueryID + is infoschema.InfoSchema + frags []*Fragment + cache map[int]tasksAndFrags } // GenerateRootMPPTasks generate all mpp tasks and return root ones. -func GenerateRootMPPTasks(ctx sessionctx.Context, startTs uint64, sender *PhysicalExchangeSender, is infoschema.InfoSchema) ([]*Fragment, error) { +func GenerateRootMPPTasks(ctx sessionctx.Context, startTs uint64, mppQueryID kv.MPPQueryID, sender *PhysicalExchangeSender, is infoschema.InfoSchema) ([]*Fragment, error) { g := &mppTaskGenerator{ - ctx: ctx, - startTS: startTs, - is: is, - cache: make(map[int]tasksAndFrags), + ctx: ctx, + startTS: startTs, + mppQueryID: mppQueryID, + is: is, + cache: make(map[int]tasksAndFrags), } return g.generateMPPTasks(sender) } +// AllocMPPTaskID allocates task id for mpp tasks. It will reset the task id when the query finished. +func AllocMPPTaskID(ctx sessionctx.Context) int64 { + mppQueryInfo := &ctx.GetSessionVars().StmtCtx.MPPQueryInfo + return mppQueryInfo.AllocatedMPPTaskID.Add(1) +} + +var mppQueryID uint64 = 1 + +// AllocMPPQueryID allocates local query id for mpp queries. +func AllocMPPQueryID() uint64 { + return atomic.AddUint64(&mppQueryID, 1) +} + func (e *mppTaskGenerator) generateMPPTasks(s *PhysicalExchangeSender) ([]*Fragment, error) { logutil.BgLogger().Info("Mpp will generate tasks", zap.String("plan", ToString(s))) tidbTask := &kv.MPPTask{ - StartTs: e.startTS, - ID: -1, + StartTs: e.startTS, + MppQueryID: e.mppQueryID, + ID: -1, } _, frags, err := e.generateMPPTasksForExchangeSender(s) if err != nil { @@ -132,10 +149,11 @@ func (e *mppTaskGenerator) constructMPPTasksByChildrenTasks(tasks []*kv.MPPTask) _, ok := addressMap[addr] if !ok { mppTask := &kv.MPPTask{ - Meta: &mppAddr{addr: addr}, - ID: e.ctx.GetSessionVars().AllocMPPTaskID(e.startTS), - StartTs: e.startTS, - TableID: -1, + Meta: &mppAddr{addr: addr}, + ID: AllocMPPTaskID(e.ctx), + MppQueryID: e.mppQueryID, + StartTs: e.startTS, + TableID: -1, } newTasks = append(newTasks, mppTask) addressMap[addr] = struct{}{} @@ -378,14 +396,19 @@ func (e *mppTaskGenerator) constructMPPTasksImpl(ctx context.Context, ts *Physic logutil.BgLogger().Warn("MPP store fail ttl is invalid", zap.Error(err)) ttl = 30 * time.Second } - metas, err := e.ctx.GetMPPClient().ConstructMPPTasks(ctx, req, e.ctx.GetSessionVars().MPPStoreLastFailTime, ttl) + metas, err := e.ctx.GetMPPClient().ConstructMPPTasks(ctx, req, ttl) if err != nil { return nil, errors.Trace(err) } tasks := make([]*kv.MPPTask, 0, len(metas)) for _, meta := range metas { - task := &kv.MPPTask{Meta: meta, ID: e.ctx.GetSessionVars().AllocMPPTaskID(e.startTS), StartTs: e.startTS, TableID: ts.Table.ID, PartitionTableIDs: allPartitionsIDs} + task := &kv.MPPTask{Meta: meta, + ID: AllocMPPTaskID(e.ctx), + StartTs: e.startTS, + MppQueryID: e.mppQueryID, + TableID: ts.Table.ID, + PartitionTableIDs: allPartitionsIDs} tasks = append(tasks, task) } return tasks, nil diff --git a/planner/core/indexmerge_path.go b/planner/core/indexmerge_path.go new file mode 100644 index 0000000000000..f0ecf02a00231 --- /dev/null +++ b/planner/core/indexmerge_path.go @@ -0,0 +1,633 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "fmt" + "math" + "strings" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/util" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/ranger" + "go.uber.org/zap" +) + +// generateIndexMergePath generates IndexMerge AccessPaths on this DataSource. +func (ds *DataSource) generateIndexMergePath() error { + // Consider the IndexMergePath. Now, we just generate `IndexMergePath` in DNF case. + // Use allConds instread of pushedDownConds, + // because we want to use IndexMerge even if some expr cannot be pushed to TiKV. + // We will create new Selection for exprs that cannot be pushed in convertToIndexMergeScan. + indexMergeConds := make([]expression.Expression, 0, len(ds.allConds)) + for _, expr := range ds.allConds { + indexMergeConds = append(indexMergeConds, expression.PushDownNot(ds.ctx, expr)) + } + + stmtCtx := ds.ctx.GetSessionVars().StmtCtx + isPossibleIdxMerge := len(indexMergeConds) > 0 && len(ds.possibleAccessPaths) > 1 + sessionAndStmtPermission := (ds.ctx.GetSessionVars().GetEnableIndexMerge() || len(ds.indexMergeHints) > 0) && !stmtCtx.NoIndexMergeHint + // We current do not consider `IndexMergePath`: + // 1. If there is an index path. + // 2. TODO: If there exists exprs that cannot be pushed down. This is to avoid wrongly estRow of Selection added by rule_predicate_push_down. + needConsiderIndexMerge := true + if len(ds.indexMergeHints) == 0 { + for i := 1; i < len(ds.possibleAccessPaths); i++ { + if len(ds.possibleAccessPaths[i].AccessConds) != 0 { + needConsiderIndexMerge = false + break + } + } + if needConsiderIndexMerge { + // PushDownExprs() will append extra warnings, which is annoying. So we reset warnings here. + warnings := stmtCtx.GetWarnings() + extraWarnings := stmtCtx.GetExtraWarnings() + _, remaining := expression.PushDownExprs(stmtCtx, indexMergeConds, ds.ctx.GetClient(), kv.UnSpecified) + stmtCtx.SetWarnings(warnings) + stmtCtx.SetExtraWarnings(extraWarnings) + + remainingExpr := 0 + for _, expr := range remaining { + // Handle these 3 functions specially since they can be used to access MVIndex. + if sf, ok := expr.(*expression.ScalarFunction); ok { + if sf.FuncName.L == ast.JSONMemberOf || sf.FuncName.L == ast.JSONOverlaps || + sf.FuncName.L == ast.JSONContains { + continue + } + } + remainingExpr++ + } + if remainingExpr > 0 { + needConsiderIndexMerge = false + } + } + } + + if isPossibleIdxMerge && sessionAndStmtPermission && needConsiderIndexMerge && ds.tableInfo.TempTableType != model.TempTableLocal { + err := ds.generateAndPruneIndexMergePath(indexMergeConds, ds.indexMergeHints != nil) + if err != nil { + return err + } + } else if len(ds.indexMergeHints) > 0 { + ds.indexMergeHints = nil + var msg string + if !isPossibleIdxMerge { + msg = "No available filter or available index." + } else if !sessionAndStmtPermission { + msg = "Got no_index_merge hint or tidb_enable_index_merge is off." + } else if ds.tableInfo.TempTableType == model.TempTableLocal { + msg = "Cannot use IndexMerge on temporary table." + } + msg = fmt.Sprintf("IndexMerge is inapplicable or disabled. %s", msg) + stmtCtx.AppendWarning(errors.Errorf(msg)) + logutil.BgLogger().Debug(msg) + } + + return nil +} + +// getIndexMergeOrPath generates all possible IndexMergeOrPaths. +func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) error { + usedIndexCount := len(ds.possibleAccessPaths) + for i, cond := range filters { + sf, ok := cond.(*expression.ScalarFunction) + if !ok || sf.FuncName.L != ast.LogicOr { + continue + } + var partialPaths = make([]*util.AccessPath, 0, usedIndexCount) + dnfItems := expression.FlattenDNFConditions(sf) + for _, item := range dnfItems { + cnfItems := expression.SplitCNFItems(item) + itemPaths := ds.accessPathsForConds(cnfItems, usedIndexCount) + if len(itemPaths) == 0 { + partialPaths = nil + break + } + partialPath, err := ds.buildIndexMergePartialPath(itemPaths) + if err != nil { + return err + } + if partialPath == nil { + partialPaths = nil + break + } + partialPaths = append(partialPaths, partialPath) + } + // If all of the partialPaths use the same index, we will not use the indexMerge. + singlePath := true + for i := len(partialPaths) - 1; i >= 1; i-- { + if partialPaths[i].Index != partialPaths[i-1].Index { + singlePath = false + break + } + } + if singlePath { + continue + } + if len(partialPaths) > 1 { + possiblePath := ds.buildIndexMergeOrPath(filters, partialPaths, i) + if possiblePath == nil { + return nil + } + + accessConds := make([]expression.Expression, 0, len(partialPaths)) + for _, p := range partialPaths { + indexCondsForP := p.AccessConds[:] + indexCondsForP = append(indexCondsForP, p.IndexFilters...) + if len(indexCondsForP) > 0 { + accessConds = append(accessConds, expression.ComposeCNFCondition(ds.ctx, indexCondsForP...)) + } + } + accessDNF := expression.ComposeDNFCondition(ds.ctx, accessConds...) + sel, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, []expression.Expression{accessDNF}, nil) + if err != nil { + logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) + sel = SelectionFactor + } + possiblePath.CountAfterAccess = sel * ds.tableStats.RowCount + ds.possibleAccessPaths = append(ds.possibleAccessPaths, possiblePath) + } + } + return nil +} + +// isInIndexMergeHints returns true if the input index name is not excluded by the IndexMerge hints, which means either +// (1) there's no IndexMerge hint, (2) there's IndexMerge hint but no specified index names, or (3) the input index +// name is specified in the IndexMerge hints. +func (ds *DataSource) isInIndexMergeHints(name string) bool { + if len(ds.indexMergeHints) == 0 { + return true + } + for _, hint := range ds.indexMergeHints { + if hint.indexHint == nil || len(hint.indexHint.IndexNames) == 0 { + return true + } + for _, hintName := range hint.indexHint.IndexNames { + if strings.EqualFold(strings.ToLower(name), strings.ToLower(hintName.String())) { + return true + } + } + } + return false +} + +// indexMergeHintsHasSpecifiedIdx returns true if there's IndexMerge hint, and it has specified index names. +func (ds *DataSource) indexMergeHintsHasSpecifiedIdx() bool { + for _, hint := range ds.indexMergeHints { + if hint.indexHint == nil || len(hint.indexHint.IndexNames) == 0 { + continue + } + if len(hint.indexHint.IndexNames) > 0 { + return true + } + } + return false +} + +// indexMergeHintsHasSpecifiedIdx return true if the input index name is specified in the IndexMerge hint. +func (ds *DataSource) isSpecifiedInIndexMergeHints(name string) bool { + for _, hint := range ds.indexMergeHints { + if hint.indexHint == nil || len(hint.indexHint.IndexNames) == 0 { + continue + } + for _, hintName := range hint.indexHint.IndexNames { + if strings.EqualFold(strings.ToLower(name), strings.ToLower(hintName.String())) { + return true + } + } + } + return false +} + +// accessPathsForConds generates all possible index paths for conditions. +func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, usedIndexCount int) []*util.AccessPath { + var results = make([]*util.AccessPath, 0, usedIndexCount) + for i := 0; i < usedIndexCount; i++ { + path := &util.AccessPath{} + if ds.possibleAccessPaths[i].IsTablePath() { + if !ds.isInIndexMergeHints("primary") { + continue + } + if ds.tableInfo.IsCommonHandle { + path.IsCommonHandlePath = true + path.Index = ds.possibleAccessPaths[i].Index + } else { + path.IsIntHandlePath = true + } + err := ds.deriveTablePathStats(path, conditions, true) + if err != nil { + logutil.BgLogger().Debug("can not derive statistics of a path", zap.Error(err)) + continue + } + var unsignedIntHandle bool + if path.IsIntHandlePath && ds.tableInfo.PKIsHandle { + if pkColInfo := ds.tableInfo.GetPkColInfo(); pkColInfo != nil { + unsignedIntHandle = mysql.HasUnsignedFlag(pkColInfo.GetFlag()) + } + } + // If the path contains a full range, ignore it. + if ranger.HasFullRange(path.Ranges, unsignedIntHandle) { + continue + } + // If we have point or empty range, just remove other possible paths. + if len(path.Ranges) == 0 || path.OnlyPointRange(ds.SCtx()) { + if len(results) == 0 { + results = append(results, path) + } else { + results[0] = path + results = results[:1] + } + break + } + } else { + path.Index = ds.possibleAccessPaths[i].Index + if !ds.isInIndexMergeHints(path.Index.Name.L) { + continue + } + err := ds.fillIndexPath(path, conditions) + if err != nil { + logutil.BgLogger().Debug("can not derive statistics of a path", zap.Error(err)) + continue + } + ds.deriveIndexPathStats(path, conditions, true) + // If the path contains a full range, ignore it. + if ranger.HasFullRange(path.Ranges, false) { + continue + } + // If we have empty range, or point range on unique index, just remove other possible paths. + if len(path.Ranges) == 0 || (path.OnlyPointRange(ds.SCtx()) && path.Index.Unique) { + if len(results) == 0 { + results = append(results, path) + } else { + results[0] = path + results = results[:1] + } + break + } + } + results = append(results, path) + } + return results +} + +// buildIndexMergePartialPath chooses the best index path from all possible paths. +// Now we choose the index with minimal estimate row count. +func (ds *DataSource) buildIndexMergePartialPath(indexAccessPaths []*util.AccessPath) (*util.AccessPath, error) { + if len(indexAccessPaths) == 1 { + return indexAccessPaths[0], nil + } + + minEstRowIndex := 0 + minEstRow := math.MaxFloat64 + for i := 0; i < len(indexAccessPaths); i++ { + rc := indexAccessPaths[i].CountAfterAccess + if len(indexAccessPaths[i].IndexFilters) > 0 { + rc = indexAccessPaths[i].CountAfterIndex + } + if rc < minEstRow { + minEstRowIndex = i + minEstRow = rc + } + } + return indexAccessPaths[minEstRowIndex], nil +} + +// buildIndexMergeOrPath generates one possible IndexMergePath. +func (ds *DataSource) buildIndexMergeOrPath(filters []expression.Expression, partialPaths []*util.AccessPath, current int) *util.AccessPath { + indexMergePath := &util.AccessPath{PartialIndexPaths: partialPaths} + indexMergePath.TableFilters = append(indexMergePath.TableFilters, filters[:current]...) + indexMergePath.TableFilters = append(indexMergePath.TableFilters, filters[current+1:]...) + var addCurrentFilter bool + for _, path := range partialPaths { + // If any partial path contains table filters, we need to keep the whole DNF filter in the Selection. + if len(path.TableFilters) > 0 { + addCurrentFilter = true + } + // If any partial path's index filter cannot be pushed to TiKV, we should keep the whole DNF filter. + if len(path.IndexFilters) != 0 && !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, path.IndexFilters, ds.ctx.GetClient(), kv.TiKV) { + addCurrentFilter = true + // Clear IndexFilter, the whole filter will be put in indexMergePath.TableFilters. + path.IndexFilters = nil + } + if len(path.TableFilters) != 0 && !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, path.TableFilters, ds.ctx.GetClient(), kv.TiKV) { + addCurrentFilter = true + path.TableFilters = nil + } + } + if addCurrentFilter { + indexMergePath.TableFilters = append(indexMergePath.TableFilters, filters[current]) + } + return indexMergePath +} + +// generateIndexMergeAndPaths generates IndexMerge paths for `AND` (a.k.a. intersection type IndexMerge) +func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int) *util.AccessPath { + // For now, we only consider intersection type IndexMerge when the index names are specified in the hints. + if !ds.indexMergeHintsHasSpecifiedIdx() { + return nil + } + + // 1. Collect partial paths from normal paths. + var partialPaths []*util.AccessPath + for i := 0; i < normalPathCnt; i++ { + originalPath := ds.possibleAccessPaths[i] + // No need to consider table path as a partial path. + if ds.possibleAccessPaths[i].IsTablePath() { + continue + } + if !ds.isSpecifiedInIndexMergeHints(originalPath.Index.Name.L) { + continue + } + // If the path contains a full range, ignore it. + if ranger.HasFullRange(originalPath.Ranges, false) { + continue + } + newPath := originalPath.Clone() + partialPaths = append(partialPaths, newPath) + } + if len(partialPaths) < 2 { + return nil + } + + // 2. Collect filters that can't be covered by the partial paths and deduplicate them. + finalFilters := make([]expression.Expression, 0) + partialFilters := make([]expression.Expression, 0, len(partialPaths)) + hashCodeSet := make(map[string]struct{}) + for _, path := range partialPaths { + // Classify filters into coveredConds and notCoveredConds. + coveredConds := make([]expression.Expression, 0, len(path.AccessConds)+len(path.IndexFilters)) + notCoveredConds := make([]expression.Expression, 0, len(path.IndexFilters)+len(path.TableFilters)) + // AccessConds can be covered by partial path. + coveredConds = append(coveredConds, path.AccessConds...) + for i, cond := range path.IndexFilters { + // IndexFilters can be covered by partial path if it can be pushed down to TiKV. + if !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, []expression.Expression{cond}, ds.ctx.GetClient(), kv.TiKV) { + path.IndexFilters = append(path.IndexFilters[:i], path.IndexFilters[i+1:]...) + notCoveredConds = append(notCoveredConds, cond) + } else { + coveredConds = append(coveredConds, cond) + } + } + // TableFilters can't be covered by partial path. + notCoveredConds = append(notCoveredConds, path.TableFilters...) + + // Record covered filters in hashCodeSet. + // Note that we only record filters that not appear in the notCoveredConds. It's possible that a filter appear + // in both coveredConds and notCoveredConds (e.g. because of prefix index). So we need this extra check to + // avoid wrong deduplication. + notCoveredHashCodeSet := make(map[string]struct{}) + for _, cond := range notCoveredConds { + hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) + notCoveredHashCodeSet[hashCode] = struct{}{} + } + for _, cond := range coveredConds { + hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) + if _, ok := notCoveredHashCodeSet[hashCode]; !ok { + hashCodeSet[hashCode] = struct{}{} + } + } + + finalFilters = append(finalFilters, notCoveredConds...) + partialFilters = append(partialFilters, coveredConds...) + } + + // Remove covered filters from finalFilters and deduplicate finalFilters. + dedupedFinalFilters := make([]expression.Expression, 0, len(finalFilters)) + for _, cond := range finalFilters { + hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) + if _, ok := hashCodeSet[hashCode]; !ok { + dedupedFinalFilters = append(dedupedFinalFilters, cond) + hashCodeSet[hashCode] = struct{}{} + } + } + + // 3. Estimate the row count after partial paths. + sel, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, partialFilters, nil) + if err != nil { + logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) + sel = SelectionFactor + } + + indexMergePath := &util.AccessPath{ + PartialIndexPaths: partialPaths, + IndexMergeIsIntersection: true, + TableFilters: dedupedFinalFilters, + CountAfterAccess: sel * ds.tableStats.RowCount, + } + return indexMergePath +} + +func (ds *DataSource) generateAndPruneIndexMergePath(indexMergeConds []expression.Expression, needPrune bool) error { + regularPathCount := len(ds.possibleAccessPaths) + // 1. Generate possible IndexMerge paths for `OR`. + err := ds.generateIndexMergeOrPaths(indexMergeConds) + if err != nil { + return err + } + // 2. Generate possible IndexMerge paths for `AND`. + indexMergeAndPath := ds.generateIndexMergeAndPaths(regularPathCount) + if indexMergeAndPath != nil { + ds.possibleAccessPaths = append(ds.possibleAccessPaths, indexMergeAndPath) + } + // 3. Generate possible IndexMerge paths for MVIndex. + mvIndexMergePath, err := ds.generateIndexMergeJSONMVIndexPath(regularPathCount, indexMergeConds) + if err != nil { + return err + } + if mvIndexMergePath != nil { + ds.possibleAccessPaths = append(ds.possibleAccessPaths, mvIndexMergePath...) + } + + // 4. If needed, append a warning if no IndexMerge is generated. + + // If without hints, it means that `enableIndexMerge` is true + if len(ds.indexMergeHints) == 0 { + return nil + } + // With hints and without generated IndexMerge paths + if regularPathCount == len(ds.possibleAccessPaths) { + ds.indexMergeHints = nil + ds.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("IndexMerge is inapplicable")) + return nil + } + + // 4. If needPrune is true, prune non-IndexMerge paths. + + // Do not need to consider the regular paths in find_best_task(). + // So we can use index merge's row count as DataSource's row count. + if needPrune { + ds.possibleAccessPaths = ds.possibleAccessPaths[regularPathCount:] + minRowCount := ds.possibleAccessPaths[0].CountAfterAccess + for _, path := range ds.possibleAccessPaths { + if minRowCount < path.CountAfterAccess { + minRowCount = path.CountAfterAccess + } + } + if ds.stats.RowCount > minRowCount { + ds.stats = ds.tableStats.ScaleByExpectCnt(minRowCount) + } + } + return nil +} + +// generateIndexMergeJSONMVIndexPath generates paths for (json_member_of / json_overlaps / json_contains) on multi-valued index. +/* + 1. select * from t where 1 member of (a) + IndexMerge(AND) + IndexRangeScan(a, [1,1]) + TableRowIdScan(t) + 2. select * from t where json_contains(a, '[1, 2, 3]') + IndexMerge(AND) + IndexRangeScan(a, [1,1]) + IndexRangeScan(a, [2,2]) + IndexRangeScan(a, [3,3]) + TableRowIdScan(t) + 3. select * from t where json_overlap(a, '[1, 2, 3]') + IndexMerge(OR) + IndexRangeScan(a, [1,1]) + IndexRangeScan(a, [2,2]) + IndexRangeScan(a, [3,3]) + TableRowIdScan(t) +*/ +func (ds *DataSource) generateIndexMergeJSONMVIndexPath(normalPathCnt int, filters []expression.Expression) (mvIndexPaths []*util.AccessPath, err error) { + for idx := 0; idx < normalPathCnt; idx++ { + if ds.possibleAccessPaths[idx].IsTablePath() || ds.possibleAccessPaths[idx].Index == nil || !ds.possibleAccessPaths[idx].Index.MVIndex { + continue // not a MVIndex path + } + if !ds.isSpecifiedInIndexMergeHints(ds.possibleAccessPaths[idx].Index.Name.L) { + continue // for safety, only consider using MVIndex when there is a `use_index_merge` hint now. + // TODO: remove this limitation + } + + // Step 1. Extract the underlying JSON column from MVIndex Info. + mvIndex := ds.possibleAccessPaths[idx].Index + if len(mvIndex.Columns) != 1 { + // only support single-column MVIndex now: idx((cast(a->'$.zip' as signed array))) + // TODO: support composite MVIndex idx((x, cast(a->'$.zip' as int array), z)) + continue + } + mvVirColOffset := mvIndex.Columns[0].Offset + mvVirColMeta := ds.table.Meta().Cols()[mvVirColOffset] + + var virCol *expression.Column + for _, ce := range ds.TblCols { + if ce.ID == mvVirColMeta.ID { + virCol = ce.Clone().(*expression.Column) + virCol.RetType = ce.GetType().ArrayType() // use the underlying type directly: JSON-ARRAY(INT) --> INT + break + } + } + // unwrap the outside cast: cast(json_extract(test.t.a, $.zip), JSON) --> json_extract(test.t.a, $.zip) + targetJSONPath, ok := unwrapJSONCast(virCol.VirtualExpr) + if !ok { + continue + } + + // Step 2. Iterate all filters and generate corresponding IndexMerge paths. + for filterIdx, filter := range filters { + // Step 2.1. Extract jsonPath and vals from json_member / json_overlaps / json_contains functions. + sf, ok := filter.(*expression.ScalarFunction) + if !ok { + continue + } + + var jsonPath expression.Expression + var vals []expression.Expression + switch sf.FuncName.L { + case ast.JSONMemberOf: // (1 member of a->'$.zip') + jsonPath = sf.GetArgs()[1] + v, ok := unwrapJSONCast(sf.GetArgs()[0]) // cast(1 as json) --> 1 + if !ok { + continue + } + vals = append(vals, v) + case ast.JSONOverlaps: // (json_overlaps(a->'$.zip', '[1, 2, 3]') + continue // TODO: support json_overlaps + case ast.JSONContains: // (json_contains(a->'$.zip', '[1, 2, 3]') + continue // TODO: support json_contains + default: + continue + } + + // Step 2.2. Check some limitations. + if jsonPath == nil || len(vals) == 0 { + continue + } + if !jsonPath.Equal(ds.ctx, targetJSONPath) { + continue // not on the same JSON col + } + // only support INT now + // TODO: support more types + if jsonPath.GetType().EvalType() == types.ETInt { + continue + } + allInt := true + // TODO: support using IndexLookUp to handle single-value cases. + for _, v := range vals { + if v.GetType().EvalType() != types.ETInt { + allInt = false + } + } + if !allInt { + continue + } + + // Step 2.3. Generate a IndexMerge Path of this filter on the current MVIndex. + var partialPaths []*util.AccessPath + for _, v := range vals { + partialPath := &util.AccessPath{Index: mvIndex} + partialPath.Ranges = ranger.FullRange() + // TODO: get the actual column length of this virtual column + partialPath.IdxCols, partialPath.IdxColLens = []*expression.Column{virCol}, []int{types.UnspecifiedLength} + partialPath.FullIdxCols, partialPath.FullIdxColLens = []*expression.Column{virCol}, []int{types.UnspecifiedLength} + + // calculate the path range with the condition `a->'$.zip' = 1`. + eq, err := expression.NewFunction(ds.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), virCol, v) + if err != nil { + return nil, err + } + if err = ds.detachCondAndBuildRangeForPath(partialPath, []expression.Expression{eq}); err != nil { + return nil, err + } + + partialPaths = append(partialPaths, partialPath) + } + indexMergePath := ds.buildIndexMergeOrPath(filters, partialPaths, filterIdx) + mvIndexPaths = append(mvIndexPaths, indexMergePath) + } + } + return +} + +func unwrapJSONCast(expr expression.Expression) (expression.Expression, bool) { + if expr == nil { + return nil, false + } + sf, ok := expr.(*expression.ScalarFunction) + if !ok { + return nil, false + } + if sf == nil || sf.FuncName.L != ast.Cast || sf.GetType().EvalType() != types.ETJson { + return nil, false + } + return sf.GetArgs()[0], true +} diff --git a/planner/core/indexmerge_path_test.go b/planner/core/indexmerge_path_test.go new file mode 100644 index 0000000000000..fcb0d27903c64 --- /dev/null +++ b/planner/core/indexmerge_path_test.go @@ -0,0 +1,53 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core_test + +import ( + "testing" + + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" +) + +func TestIndexMergeJSONMemberOf(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table t( +a int, j0 json, j1 json, +index j0_0((cast(j0->'$.path0' as signed array))), +index j0_1((cast(j0->'$.path1' as signed array))), +index j1((cast(j1 as signed array))))`) + + var input []string + var output []struct { + SQL string + Plan []string + } + planSuiteData := core.GetIndexMergeSuiteData() + planSuiteData.LoadTestCases(t, &input, &output) + + for i, query := range input { + testdata.OnRecord(func() { + output[i].SQL = query + }) + result := tk.MustQuery("explain format = 'brief' " + query) + testdata.OnRecord(func() { + output[i].Plan = testdata.ConvertRowsToStrings(result.Rows()) + }) + result.Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 6bc8a677bc6de..f60c1f617e467 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4673,7 +4673,10 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as if i < len(columns) { if columns[i].IsGenerated() && !columns[i].GeneratedStored { var err error + originVal := b.allowBuildCastArray + b.allowBuildCastArray = true expr, _, err = b.rewrite(ctx, columns[i].GeneratedExpr, ds, nil, true) + b.allowBuildCastArray = originVal if err != nil { return nil, err } diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 7e5b13b588e16..640c0f04630c1 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -813,9 +813,11 @@ func TestMPPHints(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("create table t (a int, b int, c int)") + tk.MustExec("create table t (a int, b int, c int, index idx_a(a), index idx_b(b))") tk.MustExec("alter table t set tiflash replica 1") tk.MustExec("set @@session.tidb_allow_mpp=ON") + tk.MustExec("create definer='root'@'localhost' view v as select a, sum(b) from t group by a, c;") + tk.MustExec("create definer='root'@'localhost' view v1 as select t1.a from t t1, t t2 where t1.a=t2.a;") tb := external.GetTableByName(t, tk, "test", "t") err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) require.NoError(t, err) @@ -824,6 +826,7 @@ func TestMPPHints(t *testing.T) { var output []struct { SQL string Plan []string + Warn []string } planSuiteData := core.GetPlanSuiteData() @@ -833,11 +836,101 @@ func TestMPPHints(t *testing.T) { testdata.OnRecord(func() { output[i].SQL = ts output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) }) tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } +func TestMPPHintsScope(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("create table t (a int, b int, c int, index idx_a(a), index idx_b(b))") + tk.MustExec("select /*+ MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The agg can not push down to the MPP side, the MPP_1PHASE_AGG() hint is invalid")) + tk.MustExec("select /*+ MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The agg can not push down to the MPP side, the MPP_2PHASE_AGG() hint is invalid")) + tk.MustExec("select /*+ shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The join can not push down to the MPP side, the shuffle_join() hint is invalid")) + tk.MustExec("select /*+ broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The join can not push down to the MPP side, the broadcast_join() hint is invalid")) + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_allow_mpp=true") + tk.MustExec("explain select /*+ MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("explain select /*+ MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("explain select /*+ shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("explain select /*+ broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows()) + + tk.MustExec("set @@session.tidb_allow_mpp=false") + tk.MustExec("explain select /*+ MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The agg can not push down to the MPP side, the MPP_1PHASE_AGG() hint is invalid")) + tk.MustExec("explain select /*+ MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The agg can not push down to the MPP side, the MPP_2PHASE_AGG() hint is invalid")) + tk.MustExec("explain select /*+ shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The join can not push down to the MPP side, the shuffle_join() hint is invalid")) + tk.MustExec("explain select /*+ broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The join can not push down to the MPP side, the broadcast_join() hint is invalid")) +} + +func TestMPPHintsWithBinding(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("create table t (a int, b int, c int)") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tk.MustExec("explain select a, sum(b) from t group by a, c") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + tk.MustExec("create global binding for select a, sum(b) from t group by a, c using select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c;") + tk.MustExec("explain select a, sum(b) from t group by a, c") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res := tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select `a` , sum ( `b` ) from `test` . `t` group by `a` , `c`") + require.Equal(t, res[0][1], "SELECT /*+ read_from_storage(tiflash[`t`]) MPP_1PHASE_AGG()*/ `a`,sum(`b`) FROM `test`.`t` GROUP BY `a`,`c`") + tk.MustExec("create global binding for select a, sum(b) from t group by a, c using select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c;") + tk.MustExec("explain select a, sum(b) from t group by a, c") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select `a` , sum ( `b` ) from `test` . `t` group by `a` , `c`") + require.Equal(t, res[0][1], "SELECT /*+ read_from_storage(tiflash[`t`]) MPP_2PHASE_AGG()*/ `a`,sum(`b`) FROM `test`.`t` GROUP BY `a`,`c`") + tk.MustExec("drop global binding for select a, sum(b) from t group by a, c;") + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, len(res), 0) + + tk.MustExec("explain select * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + tk.MustExec("create global binding for select * from t t1, t t2 where t1.a=t2.a using select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustExec("explain select * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select * from ( `test` . `t` as `t1` ) join `test` . `t` as `t2` where `t1` . `a` = `t2` . `a`") + require.Equal(t, res[0][1], "SELECT /*+ read_from_storage(tiflash[`t1`, `t2`]) shuffle_join(`t1`, `t2`)*/ * FROM (`test`.`t` AS `t1`) JOIN `test`.`t` AS `t2` WHERE `t1`.`a` = `t2`.`a`") + tk.MustExec("create global binding for select * from t t1, t t2 where t1.a=t2.a using select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a;") + tk.MustExec("explain select * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select * from ( `test` . `t` as `t1` ) join `test` . `t` as `t2` where `t1` . `a` = `t2` . `a`") + require.Equal(t, res[0][1], "SELECT /*+ read_from_storage(tiflash[`t1`, `t2`]) broadcast_join(`t1`, `t2`)*/ * FROM (`test`.`t` AS `t1`) JOIN `test`.`t` AS `t2` WHERE `t1`.`a` = `t2`.`a`") + tk.MustExec("drop global binding for select * from t t1, t t2 where t1.a=t2.a;") + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, len(res), 0) +} + func TestHintScope(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index 7a4ac860d8593..a480c583d0434 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -221,6 +221,29 @@ func TestIssue38533(t *testing.T) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } +func TestIgnoreInsertStmt(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int)") + + // do not cache native insert-stmt + tk.MustExec("prepare st from 'insert into t values (1)'") + tk.MustExec("execute st") + tk.MustExec("execute st") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + // ignore-hint in insert-stmt can work + tk.MustExec("prepare st from 'insert into t select * from t'") + tk.MustExec("execute st") + tk.MustExec("execute st") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustExec("prepare st from 'insert /*+ ignore_plan_cache() */ into t select * from t'") + tk.MustExec("execute st") + tk.MustExec("execute st") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) +} + func TestIssue38710(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/plan_cache_utils.go b/planner/core/plan_cache_utils.go index 8dc867316207d..6408d269ef799 100644 --- a/planner/core/plan_cache_utils.go +++ b/planner/core/plan_cache_utils.go @@ -405,10 +405,13 @@ func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.Ta // PlanCacheStmt store prepared ast from PrepareExec and other related fields type PlanCacheStmt struct { - PreparedAst *ast.Prepared - StmtDB string // which DB the statement will be processed over - VisitInfos []visitInfo - ColumnInfos interface{} + PreparedAst *ast.Prepared + StmtDB string // which DB the statement will be processed over + VisitInfos []visitInfo + ColumnInfos interface{} + // Executor is only used for point get scene. + // Notice that we should only cache the PointGetExecutor that have a snapshot with MaxTS in it. + // If the current plan is not PointGet or does not use MaxTS optimization, this value should be nil here. Executor interface{} NormalizedSQL string NormalizedPlan string diff --git a/planner/core/plan_cacheable_checker.go b/planner/core/plan_cacheable_checker.go index 3412ace379271..041509d224792 100644 --- a/planner/core/plan_cacheable_checker.go +++ b/planner/core/plan_cacheable_checker.go @@ -94,6 +94,21 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren return in, true } } + case *ast.InsertStmt: + if node.Select == nil { + // do not cache insert-values-stmt like 'insert into t values (...)' since + // no performance benefit and to save memory. + checker.cacheable = false + checker.reason = "ignore insert-values-stmt" + return in, true + } + for _, hints := range node.TableHints { + if hints.HintName.L == HintIgnorePlanCache { + checker.cacheable = false + checker.reason = "ignore plan cache by hint" + return in, true + } + } case *ast.VariableExpr, *ast.ExistsSubqueryExpr, *ast.SubqueryExpr: checker.cacheable = false checker.reason = "query has sub-queries is un-cacheable" diff --git a/planner/core/plan_cacheable_checker_test.go b/planner/core/plan_cacheable_checker_test.go index 658537bde4188..e87a08592eb16 100644 --- a/planner/core/plan_cacheable_checker_test.go +++ b/planner/core/plan_cacheable_checker_test.go @@ -53,7 +53,9 @@ func TestCacheable(t *testing.T) { tableRefsClause := &ast.TableRefsClause{TableRefs: &ast.Join{Left: &ast.TableSource{Source: tbl}}} // test InsertStmt - stmt = &ast.InsertStmt{Table: tableRefsClause} + stmt = &ast.InsertStmt{Table: tableRefsClause} // insert-values-stmt + require.False(t, core.Cacheable(stmt, is)) + stmt = &ast.InsertStmt{Table: tableRefsClause, Select: &ast.SelectStmt{}} // insert-select-stmt require.True(t, core.Cacheable(stmt, is)) // test DeleteStmt diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 26508814523d2..60a3eba28292f 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1014,10 +1014,20 @@ func (b *PlanBuilder) buildDropBindPlan(v *ast.DropBindingStmt) (Plan, error) { } func (b *PlanBuilder) buildSetBindingStatusPlan(v *ast.SetBindingStmt) (Plan, error) { - p := &SQLBindPlan{ - SQLBindOp: OpSetBindingStatus, - NormdOrigSQL: parser.Normalize(utilparser.RestoreWithDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB, v.OriginNode.Text())), - Db: utilparser.GetDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB), + var p *SQLBindPlan + if v.OriginNode != nil { + p = &SQLBindPlan{ + SQLBindOp: OpSetBindingStatus, + NormdOrigSQL: parser.Normalize(utilparser.RestoreWithDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB, v.OriginNode.Text())), + Db: utilparser.GetDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB), + } + } else if v.SQLDigest != "" { + p = &SQLBindPlan{ + SQLBindOp: OpSetBindingStatusByDigest, + SQLDigest: v.SQLDigest, + } + } else { + return nil, errors.New("sql digest is empty") } switch v.BindingStatusType { case ast.BindingStatusTypeEnabled: @@ -1419,10 +1429,6 @@ func getPossibleAccessPaths(ctx sessionctx.Context, tableHints *tableHintInfo, i } available = removeIgnoredPaths(available, ignored, tblInfo) - if staleread.IsStmtStaleness(ctx) { - // skip tiflash if the statement is for stale read until tiflash support stale read - available = removeTiflashDuringStaleRead(available) - } // If we have got "FORCE" or "USE" index hint but got no available index, // we have to use table scan. @@ -3600,7 +3606,10 @@ func (b *PlanBuilder) resolveGeneratedColumns(ctx context.Context, columns []*ta } colExpr := mockPlan.Schema().Columns[idx] + originalVal := b.allowBuildCastArray + b.allowBuildCastArray = true expr, _, err := b.rewrite(ctx, column.GeneratedExpr, mockPlan, nil, true) + b.allowBuildCastArray = originalVal if err != nil { return igc, err } diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 2cb5cca6ab4d6..4f20721bb9e0a 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -1373,7 +1373,6 @@ func TestPlanCacheSwitchDB(t *testing.T) { } func TestPlanCacheHitInfo(t *testing.T) { - t.Skip("unstable, skip it and fix it before 20210705") store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) diff --git a/planner/core/stats.go b/planner/core/stats.go index 71e1037c52c76..96065fccf12b5 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -22,9 +22,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/ast" - "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" @@ -411,109 +408,13 @@ func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, return nil, err } - // Consider the IndexMergePath. Now, we just generate `IndexMergePath` in DNF case. - // Use allConds instread of pushedDownConds, - // because we want to use IndexMerge even if some expr cannot be pushed to TiKV. - // We will create new Selection for exprs that cannot be pushed in convertToIndexMergeScan. - indexMergeConds := make([]expression.Expression, 0, len(ds.allConds)) - for _, expr := range ds.allConds { - indexMergeConds = append(indexMergeConds, expression.PushDownNot(ds.ctx, expr)) - } - - stmtCtx := ds.ctx.GetSessionVars().StmtCtx - isPossibleIdxMerge := len(indexMergeConds) > 0 && len(ds.possibleAccessPaths) > 1 - sessionAndStmtPermission := (ds.ctx.GetSessionVars().GetEnableIndexMerge() || len(ds.indexMergeHints) > 0) && !stmtCtx.NoIndexMergeHint - // We current do not consider `IndexMergePath`: - // 1. If there is an index path. - // 2. TODO: If there exists exprs that cannot be pushed down. This is to avoid wrongly estRow of Selection added by rule_predicate_push_down. - needConsiderIndexMerge := true - if len(ds.indexMergeHints) == 0 { - for i := 1; i < len(ds.possibleAccessPaths); i++ { - if len(ds.possibleAccessPaths[i].AccessConds) != 0 { - needConsiderIndexMerge = false - break - } - } - if needConsiderIndexMerge { - // PushDownExprs() will append extra warnings, which is annoying. So we reset warnings here. - warnings := stmtCtx.GetWarnings() - extraWarnings := stmtCtx.GetExtraWarnings() - _, remaining := expression.PushDownExprs(stmtCtx, indexMergeConds, ds.ctx.GetClient(), kv.UnSpecified) - stmtCtx.SetWarnings(warnings) - stmtCtx.SetExtraWarnings(extraWarnings) - if len(remaining) != 0 { - needConsiderIndexMerge = false - } - } + if err := ds.generateIndexMergePath(); err != nil { + return nil, err } - if isPossibleIdxMerge && sessionAndStmtPermission && needConsiderIndexMerge && ds.tableInfo.TempTableType != model.TempTableLocal { - err := ds.generateAndPruneIndexMergePath(indexMergeConds, ds.indexMergeHints != nil) - if err != nil { - return nil, err - } - } else if len(ds.indexMergeHints) > 0 { - ds.indexMergeHints = nil - var msg string - if !isPossibleIdxMerge { - msg = "No available filter or available index." - } else if !sessionAndStmtPermission { - msg = "Got no_index_merge hint or tidb_enable_index_merge is off." - } else if ds.tableInfo.TempTableType == model.TempTableLocal { - msg = "Cannot use IndexMerge on temporary table." - } - msg = fmt.Sprintf("IndexMerge is inapplicable or disabled. %s", msg) - stmtCtx.AppendWarning(errors.Errorf(msg)) - logutil.BgLogger().Debug(msg) - } return ds.stats, nil } -func (ds *DataSource) generateAndPruneIndexMergePath(indexMergeConds []expression.Expression, needPrune bool) error { - regularPathCount := len(ds.possibleAccessPaths) - // 1. Generate possible IndexMerge paths for `OR`. - err := ds.generateIndexMergeOrPaths(indexMergeConds) - if err != nil { - return err - } - // 2. Generate possible IndexMerge paths for `AND`. - indexMergeAndPath := ds.generateIndexMergeAndPaths(regularPathCount) - if indexMergeAndPath != nil { - ds.possibleAccessPaths = append(ds.possibleAccessPaths, indexMergeAndPath) - } - - // 3. If needed, append a warning if no IndexMerge is generated. - - // If without hints, it means that `enableIndexMerge` is true - if len(ds.indexMergeHints) == 0 { - return nil - } - // With hints and without generated IndexMerge paths - if regularPathCount == len(ds.possibleAccessPaths) { - ds.indexMergeHints = nil - ds.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("IndexMerge is inapplicable")) - return nil - } - - // 4. If needPrune is true, prune non-IndexMerge paths. - - // Do not need to consider the regular paths in find_best_task(). - // So we can use index merge's row count as DataSource's row count. - if needPrune { - ds.possibleAccessPaths = ds.possibleAccessPaths[regularPathCount:] - minRowCount := ds.possibleAccessPaths[0].CountAfterAccess - for _, path := range ds.possibleAccessPaths { - if minRowCount < path.CountAfterAccess { - minRowCount = path.CountAfterAccess - } - } - if ds.stats.RowCount > minRowCount { - ds.stats = ds.tableStats.ScaleByExpectCnt(minRowCount) - } - } - return nil -} - // DeriveStats implements LogicalPlan DeriveStats interface. func (ts *LogicalTableScan) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (_ *property.StatsInfo, err error) { ts.Source.initStats(nil) @@ -566,337 +467,6 @@ func (is *LogicalIndexScan) DeriveStats(_ []*property.StatsInfo, selfSchema *exp return is.stats, nil } -// getIndexMergeOrPath generates all possible IndexMergeOrPaths. -func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) error { - usedIndexCount := len(ds.possibleAccessPaths) - for i, cond := range filters { - sf, ok := cond.(*expression.ScalarFunction) - if !ok || sf.FuncName.L != ast.LogicOr { - continue - } - var partialPaths = make([]*util.AccessPath, 0, usedIndexCount) - dnfItems := expression.FlattenDNFConditions(sf) - for _, item := range dnfItems { - cnfItems := expression.SplitCNFItems(item) - itemPaths := ds.accessPathsForConds(cnfItems, usedIndexCount) - if len(itemPaths) == 0 { - partialPaths = nil - break - } - partialPath, err := ds.buildIndexMergePartialPath(itemPaths) - if err != nil { - return err - } - if partialPath == nil { - partialPaths = nil - break - } - partialPaths = append(partialPaths, partialPath) - } - // If all of the partialPaths use the same index, we will not use the indexMerge. - singlePath := true - for i := len(partialPaths) - 1; i >= 1; i-- { - if partialPaths[i].Index != partialPaths[i-1].Index { - singlePath = false - break - } - } - if singlePath { - continue - } - if len(partialPaths) > 1 { - possiblePath := ds.buildIndexMergeOrPath(filters, partialPaths, i) - if possiblePath == nil { - return nil - } - - accessConds := make([]expression.Expression, 0, len(partialPaths)) - for _, p := range partialPaths { - indexCondsForP := p.AccessConds[:] - indexCondsForP = append(indexCondsForP, p.IndexFilters...) - if len(indexCondsForP) > 0 { - accessConds = append(accessConds, expression.ComposeCNFCondition(ds.ctx, indexCondsForP...)) - } - } - accessDNF := expression.ComposeDNFCondition(ds.ctx, accessConds...) - sel, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, []expression.Expression{accessDNF}, nil) - if err != nil { - logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) - sel = SelectionFactor - } - possiblePath.CountAfterAccess = sel * ds.tableStats.RowCount - ds.possibleAccessPaths = append(ds.possibleAccessPaths, possiblePath) - } - } - return nil -} - -// isInIndexMergeHints returns true if the input index name is not excluded by the IndexMerge hints, which means either -// (1) there's no IndexMerge hint, (2) there's IndexMerge hint but no specified index names, or (3) the input index -// name is specified in the IndexMerge hints. -func (ds *DataSource) isInIndexMergeHints(name string) bool { - if len(ds.indexMergeHints) == 0 { - return true - } - for _, hint := range ds.indexMergeHints { - if hint.indexHint == nil || len(hint.indexHint.IndexNames) == 0 { - return true - } - for _, hintName := range hint.indexHint.IndexNames { - if strings.EqualFold(strings.ToLower(name), strings.ToLower(hintName.String())) { - return true - } - } - } - return false -} - -// indexMergeHintsHasSpecifiedIdx returns true if there's IndexMerge hint, and it has specified index names. -func (ds *DataSource) indexMergeHintsHasSpecifiedIdx() bool { - for _, hint := range ds.indexMergeHints { - if hint.indexHint == nil || len(hint.indexHint.IndexNames) == 0 { - continue - } - if len(hint.indexHint.IndexNames) > 0 { - return true - } - } - return false -} - -// indexMergeHintsHasSpecifiedIdx return true if the input index name is specified in the IndexMerge hint. -func (ds *DataSource) isSpecifiedInIndexMergeHints(name string) bool { - for _, hint := range ds.indexMergeHints { - if hint.indexHint == nil || len(hint.indexHint.IndexNames) == 0 { - continue - } - for _, hintName := range hint.indexHint.IndexNames { - if strings.EqualFold(strings.ToLower(name), strings.ToLower(hintName.String())) { - return true - } - } - } - return false -} - -// accessPathsForConds generates all possible index paths for conditions. -func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, usedIndexCount int) []*util.AccessPath { - var results = make([]*util.AccessPath, 0, usedIndexCount) - for i := 0; i < usedIndexCount; i++ { - path := &util.AccessPath{} - if ds.possibleAccessPaths[i].IsTablePath() { - if !ds.isInIndexMergeHints("primary") { - continue - } - if ds.tableInfo.IsCommonHandle { - path.IsCommonHandlePath = true - path.Index = ds.possibleAccessPaths[i].Index - } else { - path.IsIntHandlePath = true - } - err := ds.deriveTablePathStats(path, conditions, true) - if err != nil { - logutil.BgLogger().Debug("can not derive statistics of a path", zap.Error(err)) - continue - } - var unsignedIntHandle bool - if path.IsIntHandlePath && ds.tableInfo.PKIsHandle { - if pkColInfo := ds.tableInfo.GetPkColInfo(); pkColInfo != nil { - unsignedIntHandle = mysql.HasUnsignedFlag(pkColInfo.GetFlag()) - } - } - // If the path contains a full range, ignore it. - if ranger.HasFullRange(path.Ranges, unsignedIntHandle) { - continue - } - // If we have point or empty range, just remove other possible paths. - if len(path.Ranges) == 0 || path.OnlyPointRange(ds.SCtx()) { - if len(results) == 0 { - results = append(results, path) - } else { - results[0] = path - results = results[:1] - } - break - } - } else { - path.Index = ds.possibleAccessPaths[i].Index - if !ds.isInIndexMergeHints(path.Index.Name.L) { - continue - } - err := ds.fillIndexPath(path, conditions) - if err != nil { - logutil.BgLogger().Debug("can not derive statistics of a path", zap.Error(err)) - continue - } - ds.deriveIndexPathStats(path, conditions, true) - // If the path contains a full range, ignore it. - if ranger.HasFullRange(path.Ranges, false) { - continue - } - // If we have empty range, or point range on unique index, just remove other possible paths. - if len(path.Ranges) == 0 || (path.OnlyPointRange(ds.SCtx()) && path.Index.Unique) { - if len(results) == 0 { - results = append(results, path) - } else { - results[0] = path - results = results[:1] - } - break - } - } - results = append(results, path) - } - return results -} - -// buildIndexMergePartialPath chooses the best index path from all possible paths. -// Now we choose the index with minimal estimate row count. -func (ds *DataSource) buildIndexMergePartialPath(indexAccessPaths []*util.AccessPath) (*util.AccessPath, error) { - if len(indexAccessPaths) == 1 { - return indexAccessPaths[0], nil - } - - minEstRowIndex := 0 - minEstRow := math.MaxFloat64 - for i := 0; i < len(indexAccessPaths); i++ { - rc := indexAccessPaths[i].CountAfterAccess - if len(indexAccessPaths[i].IndexFilters) > 0 { - rc = indexAccessPaths[i].CountAfterIndex - } - if rc < minEstRow { - minEstRowIndex = i - minEstRow = rc - } - } - return indexAccessPaths[minEstRowIndex], nil -} - -// buildIndexMergeOrPath generates one possible IndexMergePath. -func (ds *DataSource) buildIndexMergeOrPath(filters []expression.Expression, partialPaths []*util.AccessPath, current int) *util.AccessPath { - indexMergePath := &util.AccessPath{PartialIndexPaths: partialPaths} - indexMergePath.TableFilters = append(indexMergePath.TableFilters, filters[:current]...) - indexMergePath.TableFilters = append(indexMergePath.TableFilters, filters[current+1:]...) - var addCurrentFilter bool - for _, path := range partialPaths { - // If any partial path contains table filters, we need to keep the whole DNF filter in the Selection. - if len(path.TableFilters) > 0 { - addCurrentFilter = true - } - // If any partial path's index filter cannot be pushed to TiKV, we should keep the whole DNF filter. - if len(path.IndexFilters) != 0 && !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, path.IndexFilters, ds.ctx.GetClient(), kv.TiKV) { - addCurrentFilter = true - // Clear IndexFilter, the whole filter will be put in indexMergePath.TableFilters. - path.IndexFilters = nil - } - if len(path.TableFilters) != 0 && !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, path.TableFilters, ds.ctx.GetClient(), kv.TiKV) { - addCurrentFilter = true - path.TableFilters = nil - } - } - if addCurrentFilter { - indexMergePath.TableFilters = append(indexMergePath.TableFilters, filters[current]) - } - return indexMergePath -} - -// generateIndexMergeAndPaths generates IndexMerge paths for `AND` (a.k.a. intersection type IndexMerge) -func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int) *util.AccessPath { - // For now, we only consider intersection type IndexMerge when the index names are specified in the hints. - if !ds.indexMergeHintsHasSpecifiedIdx() { - return nil - } - - // 1. Collect partial paths from normal paths. - var partialPaths []*util.AccessPath - for i := 0; i < normalPathCnt; i++ { - originalPath := ds.possibleAccessPaths[i] - // No need to consider table path as a partial path. - if ds.possibleAccessPaths[i].IsTablePath() { - continue - } - if !ds.isSpecifiedInIndexMergeHints(originalPath.Index.Name.L) { - continue - } - // If the path contains a full range, ignore it. - if ranger.HasFullRange(originalPath.Ranges, false) { - continue - } - newPath := originalPath.Clone() - partialPaths = append(partialPaths, newPath) - } - if len(partialPaths) < 2 { - return nil - } - - // 2. Collect filters that can't be covered by the partial paths and deduplicate them. - finalFilters := make([]expression.Expression, 0) - partialFilters := make([]expression.Expression, 0, len(partialPaths)) - hashCodeSet := make(map[string]struct{}) - for _, path := range partialPaths { - // Classify filters into coveredConds and notCoveredConds. - coveredConds := make([]expression.Expression, 0, len(path.AccessConds)+len(path.IndexFilters)) - notCoveredConds := make([]expression.Expression, 0, len(path.IndexFilters)+len(path.TableFilters)) - // AccessConds can be covered by partial path. - coveredConds = append(coveredConds, path.AccessConds...) - for i, cond := range path.IndexFilters { - // IndexFilters can be covered by partial path if it can be pushed down to TiKV. - if !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, []expression.Expression{cond}, ds.ctx.GetClient(), kv.TiKV) { - path.IndexFilters = append(path.IndexFilters[:i], path.IndexFilters[i+1:]...) - notCoveredConds = append(notCoveredConds, cond) - } else { - coveredConds = append(coveredConds, cond) - } - } - // TableFilters can't be covered by partial path. - notCoveredConds = append(notCoveredConds, path.TableFilters...) - - // Record covered filters in hashCodeSet. - // Note that we only record filters that not appear in the notCoveredConds. It's possible that a filter appear - // in both coveredConds and notCoveredConds (e.g. because of prefix index). So we need this extra check to - // avoid wrong deduplication. - notCoveredHashCodeSet := make(map[string]struct{}) - for _, cond := range notCoveredConds { - hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) - notCoveredHashCodeSet[hashCode] = struct{}{} - } - for _, cond := range coveredConds { - hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) - if _, ok := notCoveredHashCodeSet[hashCode]; !ok { - hashCodeSet[hashCode] = struct{}{} - } - } - - finalFilters = append(finalFilters, notCoveredConds...) - partialFilters = append(partialFilters, coveredConds...) - } - - // Remove covered filters from finalFilters and deduplicate finalFilters. - dedupedFinalFilters := make([]expression.Expression, 0, len(finalFilters)) - for _, cond := range finalFilters { - hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) - if _, ok := hashCodeSet[hashCode]; !ok { - dedupedFinalFilters = append(dedupedFinalFilters, cond) - hashCodeSet[hashCode] = struct{}{} - } - } - - // 3. Estimate the row count after partial paths. - sel, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, partialFilters, nil) - if err != nil { - logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) - sel = SelectionFactor - } - - indexMergePath := &util.AccessPath{ - PartialIndexPaths: partialPaths, - IndexMergeIsIntersection: true, - TableFilters: dedupedFinalFilters, - CountAfterAccess: sel * ds.tableStats.RowCount, - } - return indexMergePath -} - // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalSelection) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { if p.stats != nil { diff --git a/planner/core/testdata/index_merge_suite_in.json b/planner/core/testdata/index_merge_suite_in.json index db7ebacdb29c7..2841de33bae0c 100644 --- a/planner/core/testdata/index_merge_suite_in.json +++ b/planner/core/testdata/index_merge_suite_in.json @@ -1,4 +1,14 @@ [ + { + "name": "TestIndexMergeJSONMemberOf", + "cases": [ + "select /*+ use_index_merge(t, j0_0) */ * from t where (1 member of (j0->'$.path0'))", + "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and a<10", + "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.XXX')) and a<10", + "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10", + "select /*+ use_index_merge(t, j1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10" + ] + }, { "name": "TestIndexMergePathGeneration", "cases": [ diff --git a/planner/core/testdata/index_merge_suite_out.json b/planner/core/testdata/index_merge_suite_out.json index 3d67e5e372251..31427fbf4c7e0 100644 --- a/planner/core/testdata/index_merge_suite_out.json +++ b/planner/core/testdata/index_merge_suite_out.json @@ -1,4 +1,57 @@ [ + { + "Name": "TestIndexMergeJSONMemberOf", + "Cases": [ + { + "SQL": "select /*+ use_index_merge(t, j0_0) */ * from t where (1 member of (j0->'$.path0'))", + "Plan": [ + "Selection 0.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path0\"))", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_0(cast(json_extract(`j0`, _utf8mb4'$.path0') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and a<10", + "Plan": [ + "Selection 0.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\"))", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.XXX')) and a<10", + "Plan": [ + "Selection 2658.67 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.XXX\"))", + "└─TableReader 3323.33 root data:Selection", + " └─Selection 3323.33 cop[tikv] lt(test.t.a, 10)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j0_1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10", + "Plan": [ + "Selection 0.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\")), json_memberof(cast(2, json BINARY), test.t.j1)", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j0_1(cast(json_extract(`j0`, _utf8mb4'$.path1') as signed array)) range:[1,1], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select /*+ use_index_merge(t, j1) */ * from t where (1 member of (j0->'$.path1')) and (2 member of (j1)) and a<10", + "Plan": [ + "Selection 0.00 root json_memberof(cast(1, json BINARY), json_extract(test.t.j0, \"$.path1\")), json_memberof(cast(2, json BINARY), test.t.j1)", + "└─IndexMerge 0.00 root type: union", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:j1(cast(`j1` as signed array)) range:[2,2], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] lt(test.t.a, 10)", + " └─TableRowIDScan 0.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, { "Name": "TestIndexMergePathGeneration", "Cases": [ diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index d61124de927bd..c1dbae406a922 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -6229,9 +6229,9 @@ { "SQL": "explain format = 'brief' select /*+ qb_name(qb, v12) read_from_storage(tiflash[t1@qb, t@qb]), shuffle_join(t1@qb, t@qb) */ * from v12;", "Plan": [ - "Projection 12500.00 root test.t.a, test.t.b", - "└─TableReader 12500.00 root data:ExchangeSender", - " └─ExchangeSender 12500.00 mpp[tiflash] ExchangeType: PassThrough", + "TableReader 12500.00 root data:ExchangeSender", + "└─ExchangeSender 12500.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12500.00 mpp[tiflash] test.t.a, test.t.b", " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index c38250c802454..d433f5dd88dbe 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -2,10 +2,70 @@ { "name": "TestMPPHints", "cases": [ + "select /*+ MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c", + "select /*+ MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c", + "select /*+ shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + + // READ_FROM_STORAGE hint "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c", "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c", "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", - "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a" + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + + // Join hint + "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + + "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join_build(t1) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join_build(t2) */ * from t t1, t t2 where t1.a=t2.a", + + "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join_probe(t1) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join_probe(t2) */ * from t t1, t t2 where t1.a=t2.a", + + "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), merge_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), merge_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + + "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + + // AGG hint + "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), hash_agg() */ a, sum(b) from t group by a, c", + "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), stream_agg() */ a, sum(b) from t group by a, c", + + // Index hint + "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), use_index(t, idx_a) */ a, sum(b) from t where a > 1 group by a, c", + "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), ignore_index(t, idx_a) */ a, sum(b) from t where a > 1 group by a, c", + "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), force_index(t, idx_b) */ a, sum(b) from t where b < 2 group by a, c", + "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), index_merge(t, idx_b, idx_a) */ a, sum(b) from t where b < 2 or a > 2 group by a, c", + + // Join Order hint + "select /*+ read_from_storage(tiflash[t1, t2, t3]), shuffle_join(t1, t2, t3), straight_join() */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + "select /*+ read_from_storage(tiflash[t1, t2, t3]), shuffle_join(t1, t2, t3), leading(t3, t1) */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + "select /*+ read_from_storage(tiflash[t1, t2, t3]), broadcast_join(t1, t2, t3), straight_join() */ * from t t2, t t1, t t3 where t1.a=t2.a and t2.b=t3.b", + "select /*+ read_from_storage(tiflash[t1, t2, t3]), broadcast_join(t1, t2, t3), leading(t2, t3) */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + + // View Hint + "select /*+ qb_name(qb, v), MPP_1PHASE_AGG(@qb) */ * from v", + "select /*+ qb_name(qb, v), MPP_2PHASE_AGG(@qb) */ * from v", + "select /*+ qb_name(qb, v1), shuffle_join(t1@qb, t2@qb) */ * from v1", + "select /*+ qb_name(qb, v1), broadcast_join(t1@qb, t2@qb) */ * from v1", + + // Subquery hint + "SELECT /*+ shuffle_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", + "SELECT /*+ broadcast_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", + "select * from t t1 where t1.a < (select /*+ MPP_1PHASE_AGG() */ sum(t2.a) from t t2 where t2.b = t1.b);", + "select * from t t1 where t1.a < (select /*+ MPP_2PHASE_AGG() */ sum(t2.a) from t t2 where t2.b = t1.b);", + + // CTE + "WITH CTE AS (SELECT /*+ MPP_1PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ MPP_2PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ shuffle_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ broadcast_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ MERGE(), MPP_1PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ MERGE(), MPP_2PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ MERGE(), shuffle_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ MERGE(), broadcast_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;" ] }, { diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index b3a7664b2b2fd..31964823e95f2 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -2,6 +2,69 @@ { "Name": "TestMPPHints", "Cases": [ + { + "SQL": "select /*+ MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, Column#11, funcs:sum(Column#8)->Column#5, funcs:firstrow(Column#9)->test.t.a", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#8, test.t.a, test.t.a, test.t.c", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, test.t.c, funcs:sum(Column#10)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, Column#14, funcs:sum(Column#12)->Column#10", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#12, test.t.a, test.t.c", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, { "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c", "Plan": [ @@ -14,7 +77,8 @@ " └─ExchangeReceiver 10000.00 mpp[tiflash] ", " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" - ] + ], + "Warn": null }, { "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c", @@ -29,7 +93,8 @@ " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, Column#14, funcs:sum(Column#12)->Column#10", " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#12, test.t.a, test.t.c", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" - ] + ], + "Warn": null }, { "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", @@ -45,7 +110,8 @@ " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" - ] + ], + "Warn": null }, { "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", @@ -59,6 +125,732 @@ " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join_build(t1) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join_build(t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join_probe(t1) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join_probe(t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), merge_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), merge_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), hash_agg() */ a, sum(b) from t group by a, c", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, Column#11, funcs:sum(Column#8)->Column#5, funcs:firstrow(Column#9)->test.t.a", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#8, test.t.a, test.t.a, test.t.c", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), stream_agg() */ a, sum(b) from t group by a, c", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, test.t.c, funcs:sum(Column#6)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, Column#9, funcs:sum(Column#8)->Column#6", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#8, test.t.a, test.t.c", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), use_index(t, idx_a) */ a, sum(b) from t where a > 1 group by a, c", + "Plan": [ + "Projection 2666.67 root test.t.a, Column#5", + "└─HashAgg 2666.67 root group by:test.t.a, test.t.c, funcs:sum(Column#7)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─IndexLookUp 2666.67 root ", + " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t, index:idx_a(a) range:(1,+inf], keep order:false, stats:pseudo", + " └─HashAgg(Probe) 2666.67 cop[tikv] group by:test.t.a, test.t.c, funcs:sum(test.t.b)->Column#7", + " └─TableRowIDScan 3333.33 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]No available path for table test.t with the store type tiflash of the hint /*+ read_from_storage */, please check the status of the table replica and variable value of tidb_isolation_read_engines(map[0:{} 1:{} 2:{}])", + "[planner:1815]The agg can not push down to the MPP side, the MPP_1PHASE_AGG() hint is invalid" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), ignore_index(t, idx_a) */ a, sum(b) from t where a > 1 group by a, c", + "Plan": [ + "TableReader 2666.67 root data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] test.t.a, Column#5", + " └─Projection 2666.67 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 2666.67 mpp[tiflash] group by:Column#10, Column#11, funcs:sum(Column#8)->Column#5, funcs:firstrow(Column#9)->test.t.a", + " └─Projection 3333.33 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#8, test.t.a, test.t.a, test.t.c", + " └─ExchangeReceiver 3333.33 mpp[tiflash] ", + " └─ExchangeSender 3333.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─Selection 3333.33 mpp[tiflash] gt(test.t.a, 1)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), force_index(t, idx_b) */ a, sum(b) from t where b < 2 group by a, c", + "Plan": [ + "Projection 2658.67 root test.t.a, Column#5", + "└─HashAgg 2658.67 root group by:test.t.a, test.t.c, funcs:sum(Column#7)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─IndexLookUp 2658.67 root ", + " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t, index:idx_b(b) range:[-inf,2), keep order:false, stats:pseudo", + " └─HashAgg(Probe) 2658.67 cop[tikv] group by:test.t.a, test.t.c, funcs:sum(test.t.b)->Column#7", + " └─TableRowIDScan 3323.33 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]No available path for table test.t with the store type tiflash of the hint /*+ read_from_storage */, please check the status of the table replica and variable value of tidb_isolation_read_engines(map[0:{} 1:{} 2:{}])", + "[planner:1815]The agg can not push down to the MPP side, the MPP_2PHASE_AGG() hint is invalid" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), index_merge(t, idx_b, idx_a) */ a, sum(b) from t where b < 2 or a > 2 group by a, c", + "Plan": [ + "TableReader 4439.11 root data:ExchangeSender", + "└─ExchangeSender 4439.11 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 4439.11 mpp[tiflash] test.t.a, Column#5", + " └─Projection 4439.11 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 4439.11 mpp[tiflash] group by:test.t.a, test.t.c, funcs:sum(Column#13)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─ExchangeReceiver 4439.11 mpp[tiflash] ", + " └─ExchangeSender 4439.11 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg 4439.11 mpp[tiflash] group by:Column#16, Column#17, funcs:sum(Column#15)->Column#13", + " └─Projection 5548.89 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#15, test.t.a, test.t.c", + " └─Selection 5548.89 mpp[tiflash] or(lt(test.t.b, 2), gt(test.t.a, 2))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "[parser:8061]Optimizer hint index_merge is not supported by TiDB and is ignored" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2, t3]), shuffle_join(t1, t2, t3), straight_join() */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 12475.01 mpp[tiflash] ", + " └─ExchangeSender 12475.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─HashJoin 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2, t3]), shuffle_join(t1, t2, t3), leading(t3, t1) */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 12475.01 mpp[tiflash] ", + " └─ExchangeSender 12475.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─HashJoin 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]leading hint is inapplicable, check the join type or the join algorithm hint", + "[planner:1815]leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2, t3]), broadcast_join(t1, t2, t3), straight_join() */ * from t t2, t t1, t t3 where t1.a=t2.a and t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2, t3]), broadcast_join(t1, t2, t3), leading(t2, t3) */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]leading hint is inapplicable, check the join type or the join algorithm hint", + "[planner:1815]leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "select /*+ qb_name(qb, v), MPP_1PHASE_AGG(@qb) */ * from v", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, Column#11, funcs:sum(Column#8)->Column#5, funcs:firstrow(Column#9)->test.t.a", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#8, test.t.a, test.t.a, test.t.c", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ qb_name(qb, v), MPP_2PHASE_AGG(@qb) */ * from v", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, test.t.c, funcs:sum(Column#10)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, Column#14, funcs:sum(Column#12)->Column#10", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#12, test.t.a, test.t.c", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ qb_name(qb, v1), shuffle_join(t1@qb, t2@qb) */ * from v1", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 mpp[tiflash] test.t.a", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ qb_name(qb, v1), broadcast_join(t1@qb, t2@qb) */ * from v1", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 mpp[tiflash] test.t.a", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "SELECT /*+ shuffle_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", + "Plan": [ + "TableReader 7992.00 root data:ExchangeSender", + "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 mpp[tiflash] semi join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]There are no matching table names for (t) in optimizer hint /*+ SHUFFLE_JOIN(t) */ or /*+ SHUFFLE_JOIN(t) */. Maybe you can use the table alias name", + "[parser:1064]Optimizer hint syntax error at line 1 column 109 near \"\" " + ] + }, + { + "SQL": "SELECT /*+ broadcast_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", + "Plan": [ + "TableReader 7992.00 root data:ExchangeSender", + "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 mpp[tiflash] semi join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]There are no matching table names for (t) in optimizer hint /*+ BROADCAST_JOIN(t) */ or /*+ TIDB_BCJ(t) */. Maybe you can use the table alias name", + "[parser:1064]Optimizer hint syntax error at line 1 column 111 near \"\" " + ] + }, + { + "SQL": "select * from t t1 where t1.a < (select /*+ MPP_1PHASE_AGG() */ sum(t2.a) from t t2 where t2.b = t1.b);", + "Plan": [ + "TableReader 9990.00 root data:ExchangeSender", + "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 9990.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)], other cond:lt(cast(test.t.a, decimal(10,0) BINARY), Column#9)", + " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 mpp[tiflash] Column#9, test.t.b", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:Column#26, funcs:sum(Column#24)->Column#9, funcs:firstrow(Column#25)->test.t.b", + " │ └─Projection 9990.00 mpp[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#24, test.t.b, test.t.b", + " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select * from t t1 where t1.a < (select /*+ MPP_2PHASE_AGG() */ sum(t2.a) from t t2 where t2.b = t1.b);", + "Plan": [ + "TableReader 9990.00 root data:ExchangeSender", + "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 9990.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)], other cond:lt(cast(test.t.a, decimal(10,0) BINARY), Column#9)", + " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 mpp[tiflash] Column#9, test.t.b", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.b, funcs:sum(Column#13)->Column#9, funcs:firstrow(test.t.b)->test.t.b", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:Column#29, funcs:sum(Column#28)->Column#13", + " │ └─Projection 9990.00 mpp[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#28, test.t.b", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MPP_1PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "HashAgg 3403.09 root group by:Column#10, Column#11, funcs:firstrow(Column#10)->Column#10, funcs:firstrow(Column#11)->Column#11", + "└─Union 3403.09 root ", + " ├─Selection 1701.55 root lt(Column#6, 18)", + " │ └─CTEFullScan 2126.93 root CTE:cte data:CTE_0", + " └─Selection 1701.55 root gt(test.t.b, 1)", + " └─CTEFullScan 2126.93 root CTE:cte data:CTE_0", + "CTE_0 2126.93 root Non-Recursive CTE", + "└─TableReader(Seed Part) 2126.93 root data:ExchangeSender", + " └─ExchangeSender 2126.93 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 2126.93 mpp[tiflash] or(lt(Column#5, 18), gt(test.t.b, 1))", + " └─Projection 2658.67 mpp[tiflash] Column#5, test.t.b", + " └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#5, funcs:firstrow(test.t.b)->test.t.b", + " └─ExchangeReceiver 3323.33 mpp[tiflash] ", + " └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 60)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MPP_2PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "HashAgg 3403.09 root group by:Column#10, Column#11, funcs:firstrow(Column#10)->Column#10, funcs:firstrow(Column#11)->Column#11", + "└─Union 3403.09 root ", + " ├─Selection 1701.55 root lt(Column#6, 18)", + " │ └─CTEFullScan 2126.93 root CTE:cte data:CTE_0", + " └─Selection 1701.55 root gt(test.t.b, 1)", + " └─CTEFullScan 2126.93 root CTE:cte data:CTE_0", + "CTE_0 2126.93 root Non-Recursive CTE", + "└─TableReader(Seed Part) 2126.93 root data:ExchangeSender", + " └─ExchangeSender 2126.93 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 2126.93 mpp[tiflash] or(lt(Column#5, 18), gt(test.t.b, 1))", + " └─Projection 2658.67 mpp[tiflash] Column#5, test.t.b", + " └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:sum(Column#22)->Column#5, funcs:firstrow(test.t.b)->test.t.b", + " └─ExchangeReceiver 2658.67 mpp[tiflash] ", + " └─ExchangeSender 2658.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#22", + " └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 60)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ shuffle_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "HashAgg 7095.48 root group by:Column#13, Column#14, funcs:firstrow(Column#13)->Column#13, funcs:firstrow(Column#14)->Column#14", + "└─Union 11086.68 root ", + " ├─Selection 5543.34 root lt(test.t.a, 18)", + " │ └─CTEFullScan 6929.18 root CTE:cte data:CTE_0", + " └─Selection 5543.34 root gt(test.t.b, 1)", + " └─CTEFullScan 6929.18 root CTE:cte data:CTE_0", + "CTE_0 6929.18 root Non-Recursive CTE", + "└─TableReader(Seed Part) 6929.18 root data:ExchangeSender", + " └─ExchangeSender 6929.18 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6929.18 mpp[tiflash] test.t.a, test.t.b", + " └─HashJoin 6929.18 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)], other cond:or(lt(test.t.a, 18), gt(test.t.b, 1))", + " ├─ExchangeReceiver(Build) 5543.34 mpp[tiflash] ", + " │ └─ExchangeSender 5543.34 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 5543.34 mpp[tiflash] not(isnull(test.t.a)), or(lt(test.t.a, 18), gt(test.t.b, 1))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ broadcast_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "HashAgg 7095.48 root group by:Column#13, Column#14, funcs:firstrow(Column#13)->Column#13, funcs:firstrow(Column#14)->Column#14", + "└─Union 11086.68 root ", + " ├─Selection 5543.34 root lt(test.t.a, 18)", + " │ └─CTEFullScan 6929.18 root CTE:cte data:CTE_0", + " └─Selection 5543.34 root gt(test.t.b, 1)", + " └─CTEFullScan 6929.18 root CTE:cte data:CTE_0", + "CTE_0 6929.18 root Non-Recursive CTE", + "└─TableReader(Seed Part) 6929.18 root data:ExchangeSender", + " └─ExchangeSender 6929.18 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6929.18 mpp[tiflash] test.t.a, test.t.b", + " └─HashJoin 6929.18 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)], other cond:or(lt(test.t.a, 18), gt(test.t.b, 1))", + " ├─ExchangeReceiver(Build) 5543.34 mpp[tiflash] ", + " │ └─ExchangeSender 5543.34 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 5543.34 mpp[tiflash] not(isnull(test.t.a)), or(lt(test.t.a, 18), gt(test.t.b, 1))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MERGE(), MPP_1PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "TableReader 3013.16 root data:ExchangeSender", + "└─ExchangeSender 3013.16 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 3013.16 mpp[tiflash] Column#20, Column#21", + " └─HashAgg 3013.16 mpp[tiflash] group by:Column#20, Column#21, funcs:firstrow(Column#20)->Column#20, funcs:firstrow(Column#21)->Column#21", + " └─ExchangeReceiver 3013.16 mpp[tiflash] ", + " └─ExchangeSender 3013.16 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: binary], [name: Column#21, collate: binary]", + " └─Union 3013.16 mpp[tiflash] ", + " ├─Projection 2126.93 mpp[tiflash] cast(Column#12, bigint(21) BINARY)->Column#20, test.t.b", + " │ └─Selection 2126.93 mpp[tiflash] lt(Column#12, 18)", + " │ └─Projection 2658.67 mpp[tiflash] Column#12, test.t.b", + " │ └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#12, funcs:firstrow(test.t.b)->test.t.b", + " │ └─ExchangeReceiver 3323.33 mpp[tiflash] ", + " │ └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 60)", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 886.22 mpp[tiflash] cast(Column#20, bigint(21) BINARY)->Column#20, Column#21", + " └─Projection 886.22 mpp[tiflash] Column#19, test.t.b", + " └─HashAgg 886.22 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#19, funcs:firstrow(test.t.b)->test.t.b", + " └─ExchangeReceiver 1107.78 mpp[tiflash] ", + " └─ExchangeSender 1107.78 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─Selection 1107.78 mpp[tiflash] gt(test.t.b, 1), lt(test.t.a, 60)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MERGE(), MPP_2PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "TableReader 3013.16 root data:ExchangeSender", + "└─ExchangeSender 3013.16 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 3013.16 mpp[tiflash] Column#20, Column#21", + " └─HashAgg 3013.16 mpp[tiflash] group by:Column#20, Column#21, funcs:firstrow(Column#20)->Column#20, funcs:firstrow(Column#21)->Column#21", + " └─ExchangeReceiver 3013.16 mpp[tiflash] ", + " └─ExchangeSender 3013.16 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: binary], [name: Column#21, collate: binary]", + " └─Union 3013.16 mpp[tiflash] ", + " ├─Projection 2126.93 mpp[tiflash] cast(Column#12, bigint(21) BINARY)->Column#20, test.t.b", + " │ └─Selection 2126.93 mpp[tiflash] lt(Column#12, 18)", + " │ └─Projection 2658.67 mpp[tiflash] Column#12, test.t.b", + " │ └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:sum(Column#32)->Column#12, funcs:firstrow(test.t.b)->test.t.b", + " │ └─ExchangeReceiver 2658.67 mpp[tiflash] ", + " │ └─ExchangeSender 2658.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#32", + " │ └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 60)", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 886.22 mpp[tiflash] cast(Column#20, bigint(21) BINARY)->Column#20, Column#21", + " └─Projection 886.22 mpp[tiflash] Column#19, test.t.b", + " └─HashAgg 886.22 mpp[tiflash] group by:test.t.b, funcs:sum(Column#46)->Column#19, funcs:firstrow(test.t.b)->test.t.b", + " └─ExchangeReceiver 886.22 mpp[tiflash] ", + " └─ExchangeSender 886.22 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─HashAgg 886.22 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#46", + " └─Selection 1107.78 mpp[tiflash] gt(test.t.b, 1), lt(test.t.a, 60)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MERGE(), shuffle_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "TableReader 5322.67 root data:ExchangeSender", + "└─ExchangeSender 5322.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 5322.67 mpp[tiflash] Column#29, Column#30", + " └─HashAgg 5322.67 mpp[tiflash] group by:Column#29, Column#30, funcs:firstrow(Column#29)->Column#29, funcs:firstrow(Column#30)->Column#30", + " └─ExchangeReceiver 5322.67 mpp[tiflash] ", + " └─ExchangeSender 5322.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#29, collate: binary], [name: Column#30, collate: binary]", + " └─HashAgg 5322.67 mpp[tiflash] group by:Column#29, Column#30, ", + " └─Union 8316.67 mpp[tiflash] ", + " ├─Projection 4154.17 mpp[tiflash] test.t.a, test.t.b", + " │ └─HashJoin 4154.17 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " │ ├─ExchangeReceiver(Build) 3323.33 mpp[tiflash] ", + " │ │ └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ │ └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 18), not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver(Probe) 3323.33 mpp[tiflash] ", + " │ └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 18), not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Projection 4162.50 mpp[tiflash] test.t.a, test.t.b", + " └─HashJoin 4162.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 3330.00 mpp[tiflash] ", + " │ └─ExchangeSender 3330.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 3330.00 mpp[tiflash] gt(test.t.b, 1), not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]There are no matching table names for (t1, t) in optimizer hint /*+ SHUFFLE_JOIN(t1, t, t1, t) */ or /*+ SHUFFLE_JOIN(t1, t, t1, t) */. Maybe you can use the table alias name", + "[planner:1815]There are no matching table names for (t1, t, t1, t) in optimizer hint /*+ SHUFFLE_JOIN(t1, t, t1, t, t1, t) */ or /*+ SHUFFLE_JOIN(t1, t, t1, t, t1, t) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MERGE(), broadcast_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "TableReader 5322.67 root data:ExchangeSender", + "└─ExchangeSender 5322.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 5322.67 mpp[tiflash] Column#29, Column#30", + " └─HashAgg 5322.67 mpp[tiflash] group by:Column#29, Column#30, funcs:firstrow(Column#29)->Column#29, funcs:firstrow(Column#30)->Column#30", + " └─ExchangeReceiver 5322.67 mpp[tiflash] ", + " └─ExchangeSender 5322.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#29, collate: binary], [name: Column#30, collate: binary]", + " └─HashAgg 5322.67 mpp[tiflash] group by:Column#29, Column#30, ", + " └─Union 8316.67 mpp[tiflash] ", + " ├─Projection 4154.17 mpp[tiflash] test.t.a, test.t.b", + " │ └─HashJoin 4154.17 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " │ ├─ExchangeReceiver(Build) 3323.33 mpp[tiflash] ", + " │ │ └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: Broadcast", + " │ │ └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 18), not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─Selection(Probe) 3323.33 mpp[tiflash] lt(test.t.a, 18), not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Projection 4162.50 mpp[tiflash] test.t.a, test.t.b", + " └─HashJoin 4162.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 3330.00 mpp[tiflash] ", + " │ └─ExchangeSender 3330.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 3330.00 mpp[tiflash] gt(test.t.b, 1), not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]There are no matching table names for (t1, t) in optimizer hint /*+ BROADCAST_JOIN(t1, t, t1, t) */ or /*+ TIDB_BCJ(t1, t, t1, t) */. Maybe you can use the table alias name", + "[planner:1815]There are no matching table names for (t1, t, t1, t) in optimizer hint /*+ BROADCAST_JOIN(t1, t, t1, t, t1, t) */ or /*+ TIDB_BCJ(t1, t, t1, t, t1, t) */. Maybe you can use the table alias name" ] } ] diff --git a/server/BUILD.bazel b/server/BUILD.bazel index 191ae1c1c6f7d..afcb983fad76e 100644 --- a/server/BUILD.bazel +++ b/server/BUILD.bazel @@ -58,6 +58,7 @@ go_library( "//sessionctx/stmtctx", "//sessionctx/variable", "//sessiontxn", + "//statistics/handle", "//store", "//store/driver/error", "//store/gcworker", @@ -91,6 +92,7 @@ go_library( "//util/topsql/stmtstats", "//util/versioninfo", "@com_github_blacktear23_go_proxyprotocol//:go-proxyprotocol", + "@com_github_burntsushi_toml//:toml", "@com_github_gorilla_mux//:mux", "@com_github_opentracing_opentracing_go//:opentracing-go", "@com_github_pingcap_errors//:errors", diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 43a2253fff477..acb12ae660b94 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -158,7 +158,10 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e return mysql.NewErrf(mysql.ErrUnknown, "unsupported flag: CursorTypeScrollable", nil) } - if !useCursor { + if useCursor { + cc.ctx.GetSessionVars().SetStatusFlag(mysql.ServerStatusCursorExists, true) + defer cc.ctx.GetSessionVars().SetStatusFlag(mysql.ServerStatusCursorExists, false) + } else { // not using streaming ,can reuse chunk cc.ctx.GetSessionVars().SetAlloc(cc.chunkAlloc) } @@ -251,7 +254,8 @@ func (cc *clientConn) executePlanCacheStmt(ctx context.Context, stmt interface{} // The first return value indicates whether the call of executePreparedStmtAndWriteResult has no side effect and can be retried. // Currently the first return value is used to fallback to TiKV when TiFlash is down. func (cc *clientConn) executePreparedStmtAndWriteResult(ctx context.Context, stmt PreparedStatement, args []expression.Expression, useCursor bool) (bool, error) { - prepStmt, err := (&cc.ctx).GetSessionVars().GetPreparedStmtByID(uint32(stmt.ID())) + vars := (&cc.ctx).GetSessionVars() + prepStmt, err := vars.GetPreparedStmtByID(uint32(stmt.ID())) if err != nil { return true, errors.Annotate(err, cc.preparedStmt2String(uint32(stmt.ID()))) } @@ -264,6 +268,9 @@ func (cc *clientConn) executePreparedStmtAndWriteResult(ctx context.Context, stm return true, errors.Annotate(err, cc.preparedStmt2String(uint32(stmt.ID()))) } if rs == nil { + if useCursor { + vars.SetStatusFlag(mysql.ServerStatusCursorExists, false) + } return false, cc.writeOK(ctx) } // since there are multiple implementations of ResultSet (the rs might be wrapped), we have to unwrap the rs before @@ -294,7 +301,7 @@ func (cc *clientConn) executePreparedStmtAndWriteResult(ctx context.Context, stm cl.OnFetchReturned() } // explicitly flush columnInfo to client. - err = cc.writeEOF(ctx, cc.ctx.Status()|mysql.ServerStatusCursorExists) + err = cc.writeEOF(ctx, cc.ctx.Status()) if err != nil { return false, err } @@ -316,6 +323,8 @@ const ( func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err error) { cc.ctx.GetSessionVars().StartTime = time.Now() cc.ctx.GetSessionVars().ClearAlloc(nil, false) + cc.ctx.GetSessionVars().SetStatusFlag(mysql.ServerStatusCursorExists, true) + defer cc.ctx.GetSessionVars().SetStatusFlag(mysql.ServerStatusCursorExists, false) stmtID, fetchSize, err := parseStmtFetchCmd(data) if err != nil { @@ -344,7 +353,7 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err strconv.FormatUint(uint64(stmtID), 10), "stmt_fetch_rs"), cc.preparedStmt2String(stmtID)) } - _, err = cc.writeResultset(ctx, rs, true, cc.ctx.Status()|mysql.ServerStatusCursorExists, int(fetchSize)) + _, err = cc.writeResultset(ctx, rs, true, cc.ctx.Status(), int(fetchSize)) if err != nil { return errors.Annotate(err, cc.preparedStmt2String(stmtID)) } diff --git a/server/conn_stmt_test.go b/server/conn_stmt_test.go index 2e60fc1085332..366e5c54ac222 100644 --- a/server/conn_stmt_test.go +++ b/server/conn_stmt_test.go @@ -15,6 +15,7 @@ package server import ( + "bytes" "context" "encoding/binary" "testing" @@ -340,3 +341,59 @@ func TestCursorReadHoldTS(t *testing.T) { require.Zero(t, tk.Session().ShowProcess().GetMinStartTS(0)) require.Zero(t, srv.GetMinStartTS(0)) } + +func TestCursorExistsFlag(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + srv := CreateMockServer(t, store) + srv.SetDomain(dom) + defer srv.Close() + + appendUint32 := binary.LittleEndian.AppendUint32 + ctx := context.Background() + c := CreateMockConn(t, srv).(*mockConn) + out := new(bytes.Buffer) + c.pkt.bufWriter.Reset(out) + c.capability |= mysql.ClientDeprecateEOF | mysql.ClientProtocol41 + tk := testkit.NewTestKitWithSession(t, store, c.Context().Session) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key)") + tk.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8)") + tk.MustQuery("select count(*) from t").Check(testkit.Rows("8")) + + getLastStatus := func() uint16 { + raw := out.Bytes() + return binary.LittleEndian.Uint16(raw[len(raw)-4 : len(raw)-2]) + } + + stmt, _, _, err := c.Context().Prepare("select * from t") + require.NoError(t, err) + + require.NoError(t, c.Dispatch(ctx, append( + appendUint32([]byte{mysql.ComStmtExecute}, uint32(stmt.ID())), + mysql.CursorTypeReadOnly, 0x1, 0x0, 0x0, 0x0, + ))) + require.True(t, mysql.HasCursorExistsFlag(getLastStatus())) + + // fetch first 5 + require.NoError(t, c.Dispatch(ctx, appendUint32(appendUint32([]byte{mysql.ComStmtFetch}, uint32(stmt.ID())), 5))) + require.True(t, mysql.HasCursorExistsFlag(getLastStatus())) + + // COM_QUERY during fetch + require.NoError(t, c.Dispatch(ctx, append([]byte{mysql.ComQuery}, "select * from t"...))) + require.False(t, mysql.HasCursorExistsFlag(getLastStatus())) + + // fetch last 3 + require.NoError(t, c.Dispatch(ctx, appendUint32(appendUint32([]byte{mysql.ComStmtFetch}, uint32(stmt.ID())), 5))) + require.True(t, mysql.HasCursorExistsFlag(getLastStatus())) + + // final fetch with no row retured + // (tidb doesn't unset cursor-exists flag in the previous response like mysql, one more fetch is needed) + require.NoError(t, c.Dispatch(ctx, appendUint32(appendUint32([]byte{mysql.ComStmtFetch}, uint32(stmt.ID())), 5))) + require.False(t, mysql.HasCursorExistsFlag(getLastStatus())) + require.True(t, getLastStatus()&mysql.ServerStatusLastRowSend > 0) + + // COM_QUERY after fetch + require.NoError(t, c.Dispatch(ctx, append([]byte{mysql.ComQuery}, "select * from t"...))) + require.False(t, mysql.HasCursorExistsFlag(getLastStatus())) +} diff --git a/server/plan_replayer.go b/server/plan_replayer.go index 13160d3b6202f..beb202638d1fd 100644 --- a/server/plan_replayer.go +++ b/server/plan_replayer.go @@ -15,15 +15,26 @@ package server import ( + "archive/zip" + "bytes" + "encoding/json" "fmt" "io" "net/http" "os" "path/filepath" + "strconv" + "strings" + "github.com/BurntSushi/toml" "github.com/gorilla/mux" + "github.com/pingcap/errors" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/replayer" @@ -32,9 +43,11 @@ import ( // PlanReplayerHandler is the handler for dumping plan replayer file. type PlanReplayerHandler struct { - infoGetter *infosync.InfoSyncer - address string - statusPort uint + is infoschema.InfoSchema + statsHandle *handle.Handle + infoGetter *infosync.InfoSyncer + address string + statusPort uint } func (s *Server) newPlanReplayerHandler() *PlanReplayerHandler { @@ -46,6 +59,12 @@ func (s *Server) newPlanReplayerHandler() *PlanReplayerHandler { if s.dom != nil && s.dom.InfoSyncer() != nil { prh.infoGetter = s.dom.InfoSyncer() } + if s.dom != nil && s.dom.InfoSchema() != nil { + prh.is = s.dom.InfoSchema() + } + if s.dom != nil && s.dom.StatsHandle() != nil { + prh.statsHandle = s.dom.StatsHandle() + } return prh } @@ -61,6 +80,8 @@ func (prh PlanReplayerHandler) ServeHTTP(w http.ResponseWriter, req *http.Reques urlPath: fmt.Sprintf("plan_replayer/dump/%s", name), downloadedFilename: "plan_replayer", scheme: util.InternalHTTPSchema(), + statsHandle: prh.statsHandle, + is: prh.is, } handleDownloadFile(handler, w, req) } @@ -93,6 +114,13 @@ func handleDownloadFile(handler downloadFileHandler, w http.ResponseWriter, req writeError(w, err) return } + if handler.downloadedFilename == "plan_replayer" { + content, err = handlePlanReplayerContinuesCaptureFile(content, path, handler) + if err != nil { + writeError(w, err) + return + } + } _, err = w.Write(content) if err != nil { writeError(w, err) @@ -175,6 +203,9 @@ type downloadFileHandler struct { statusPort uint urlPath string downloadedFilename string + + statsHandle *handle.Handle + is infoschema.InfoSchema } func isExists(path string) (bool, error) { @@ -187,3 +218,162 @@ func isExists(path string) (bool, error) { } return true, nil } + +func handlePlanReplayerContinuesCaptureFile(content []byte, path string, handler downloadFileHandler) ([]byte, error) { + if !strings.Contains(handler.filePath, "continues_replayer") { + return content, nil + } + b := bytes.NewReader(content) + zr, err := zip.NewReader(b, int64(len(content))) + if err != nil { + return nil, err + } + startTS, err := loadSQLMetaFile(zr) + if err != nil { + return nil, err + } + if startTS == 0 { + return content, nil + } + tbls, err := loadSchemaMeta(zr, handler.is) + if err != nil { + return nil, err + } + for _, tbl := range tbls { + jsonStats, err := handler.statsHandle.DumpHistoricalStatsBySnapshot(tbl.dbName, tbl.info, startTS) + if err != nil { + return nil, err + } + tbl.jsonStats = jsonStats + } + newPath, err := dumpJSONStatsIntoZip(tbls, content, path) + if err != nil { + return nil, err + } + //nolint: gosec + file, err := os.Open(newPath) + if err != nil { + return nil, err + } + content, err = io.ReadAll(file) + if err != nil { + return nil, err + } + err = file.Close() + if err != nil { + return nil, err + } + return content, nil +} + +func loadSQLMetaFile(z *zip.Reader) (uint64, error) { + for _, zipFile := range z.File { + if zipFile.Name == domain.PlanReplayerSQLMetaFile { + varMap := make(map[string]string) + v, err := zipFile.Open() + if err != nil { + return 0, errors.AddStack(err) + } + //nolint: errcheck,all_revive + defer v.Close() + _, err = toml.DecodeReader(v, &varMap) + if err != nil { + return 0, errors.AddStack(err) + } + startTS, err := strconv.ParseUint(varMap[domain.PlanReplayerSQLMetaStartTS], 10, 64) + if err != nil { + return 0, err + } + return startTS, nil + } + } + return 0, nil +} + +func loadSchemaMeta(z *zip.Reader, is infoschema.InfoSchema) (map[int64]*tblInfo, error) { + r := make(map[int64]*tblInfo, 0) + for _, zipFile := range z.File { + if zipFile.Name == fmt.Sprintf("schema/%v", domain.PlanReplayerSchemaMetaFile) { + v, err := zipFile.Open() + if err != nil { + return nil, errors.AddStack(err) + } + //nolint: errcheck,all_revive + defer v.Close() + buf := new(bytes.Buffer) + _, err = buf.ReadFrom(v) + if err != nil { + return nil, errors.AddStack(err) + } + rows := strings.Split(buf.String(), "\n") + for _, row := range rows { + s := strings.Split(row, ";") + databaseName := s[0] + tableName := s[1] + t, err := is.TableByName(model.NewCIStr(databaseName), model.NewCIStr(tableName)) + if err != nil { + return nil, err + } + r[t.Meta().ID] = &tblInfo{ + info: t.Meta(), + dbName: databaseName, + tblName: tableName, + } + } + break + } + } + return r, nil +} + +func dumpJSONStatsIntoZip(tbls map[int64]*tblInfo, content []byte, path string) (string, error) { + zr, err := zip.NewReader(bytes.NewReader(content), int64(len(content))) + if err != nil { + return "", err + } + newPath := fmt.Sprintf("copy_%v.zip", path[0:len(path)-4]) + zf, err := os.Create(newPath) + if err != nil { + return "", err + } + zw := zip.NewWriter(zf) + for _, f := range zr.File { + err = zw.Copy(f) + if err != nil { + logutil.BgLogger().Error("copy plan replayer zip file failed", zap.Error(err)) + return "", err + } + } + for _, tbl := range tbls { + w, err := zw.Create(fmt.Sprintf("stats/%v.%v.json", tbl.dbName, tbl.tblName)) + if err != nil { + return "", err + } + data, err := json.Marshal(tbl.jsonStats) + if err != nil { + return "", err + } + _, err = w.Write(data) + if err != nil { + return "", err + } + } + err = zw.Close() + if err != nil { + logutil.BgLogger().Error("Closing file failed", zap.Error(err)) + return "", err + } + err = zf.Close() + if err != nil { + logutil.BgLogger().Error("Closing file failed", zap.Error(err)) + return "", err + } + return newPath, nil +} + +type tblInfo struct { + info *model.TableInfo + jsonStats *handle.JSONTable + dbName string + tblName string +} diff --git a/session/bootstrap.go b/session/bootstrap.go index 6644da210f4c7..94484eefd9438 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -23,6 +23,7 @@ import ( "encoding/hex" "flag" "fmt" + "io/ioutil" osuser "os/user" "runtime/debug" "strconv" @@ -526,6 +527,7 @@ func bootstrap(s Session) { if dom.DDL().OwnerManager().IsOwner() { doDDLWorks(s) doDMLWorks(s) + runBootstrapSQLFile = true logutil.BgLogger().Info("bootstrap successful", zap.Duration("take time", time.Since(startTime))) return @@ -735,6 +737,8 @@ const ( version108 = 108 // version109 add column source to mysql.stats_meta_history version109 = 109 + // version110 sets tidb_enable_gc_aware_memory_track to off when a cluster upgrades from some version lower than v6.5.0. + version110 = 110 ) // currentBootstrapVersion is defined as a variable, so we can modify its value for testing. @@ -744,6 +748,9 @@ var currentBootstrapVersion int64 = version109 // DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it. var internalSQLTimeout = owner.ManagerSessionTTL + 15 +// whether to run the sql file in bootstrap. +var runBootstrapSQLFile = false + var ( bootstrapVersion = []func(Session, int64){ upgradeToVer2, @@ -853,6 +860,7 @@ var ( upgradeToVer107, upgradeToVer108, upgradeToVer109, + upgradeToVer110, } ) @@ -2201,6 +2209,15 @@ func upgradeToVer109(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.stats_meta_history ADD COLUMN IF NOT EXISTS `source` varchar(40) NOT NULL after `version`;") } +// For users that upgrade TiDB from a 6.2-6.4 version, we want to disable tidb gc_aware_memory_track by default. +func upgradeToVer110(s Session, ver int64) { + if ver >= version110 { + return + } + mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", + mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableGCAwareMemoryTrack, 0) +} + func writeOOMAction(s Session) { comment := "oom-action is `log` by default in v3.0.x, `cancel` by default in v4.0.11+" mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE VARIABLE_VALUE= %?`, @@ -2309,6 +2326,38 @@ func doDDLWorks(s Session) { mustExecute(s, CreateTTLTableStatus) } +// doBootstrapSQLFile executes SQL commands in a file as the last stage of bootstrap. +// It is useful for setting the initial value of GLOBAL variables. +func doBootstrapSQLFile(s Session) { + sqlFile := config.GetGlobalConfig().InitializeSQLFile + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) + if sqlFile == "" { + return + } + logutil.BgLogger().Info("executing -initialize-sql-file", zap.String("file", sqlFile)) + b, err := ioutil.ReadFile(sqlFile) //nolint:gosec + if err != nil { + logutil.BgLogger().Fatal("unable to read InitializeSQLFile", zap.Error(err)) + } + stmts, err := s.Parse(ctx, string(b)) + if err != nil { + logutil.BgLogger().Fatal("unable to parse InitializeSQLFile", zap.Error(err)) + } + for _, stmt := range stmts { + rs, err := s.ExecuteStmt(ctx, stmt) + if err != nil { + logutil.BgLogger().Warn("InitializeSQLFile error", zap.Error(err)) + } + if rs != nil { + // I don't believe we need to drain the result-set in bootstrap mode + // but if required we can do this here in future. + if err := rs.Close(); err != nil { + logutil.BgLogger().Fatal("unable to close result", zap.Error(err)) + } + } + } +} + // inTestSuite checks if we are bootstrapping in the context of tests. // There are some historical differences in behavior between tests and non-tests. func inTestSuite() bool { diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index 4ebb7001461a6..a010daf32b14c 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -17,12 +17,14 @@ package session import ( "context" "fmt" + "os" "strconv" "strings" "testing" "time" "github.com/pingcap/tidb/bindinfo" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/auth" @@ -1043,6 +1045,60 @@ func TestUpgradeToVer85(t *testing.T) { mustExec(t, se, "delete from mysql.bind_info where default_db = 'test'") } +func TestInitializeSQLFile(t *testing.T) { + // We create an initialize-sql-file and then bootstrap the server with it. + // The observed behavior should be that tidb_enable_noop_variables is now + // disabled, and the feature works as expected. + initializeSQLFile, err := os.CreateTemp("", "init.sql") + require.NoError(t, err) + defer func() { + path := initializeSQLFile.Name() + err = initializeSQLFile.Close() + require.NoError(t, err) + err = os.Remove(path) + require.NoError(t, err) + }() + // Implicitly test multi-line init files + _, err = initializeSQLFile.WriteString( + "CREATE DATABASE initsqlfiletest;\n" + + "SET GLOBAL tidb_enable_noop_variables = OFF;\n") + require.NoError(t, err) + + // Create a mock store + // Set the config parameter for initialize sql file + store, err := mockstore.NewMockStore() + require.NoError(t, err) + config.GetGlobalConfig().InitializeSQLFile = initializeSQLFile.Name() + defer func() { + require.NoError(t, store.Close()) + config.GetGlobalConfig().InitializeSQLFile = "" + }() + + // Bootstrap with the InitializeSQLFile config option + dom, err := BootstrapSession(store) + require.NoError(t, err) + defer dom.Close() + se := createSessionAndSetID(t, store) + ctx := context.Background() + r, err := exec(se, `SHOW VARIABLES LIKE 'query_cache_type'`) + require.NoError(t, err) + req := r.NewChunk(nil) + err = r.Next(ctx, req) + require.NoError(t, err) + require.Equal(t, 0, req.NumRows()) // not shown in noopvariables mode + require.NoError(t, r.Close()) + + r, err = exec(se, `SHOW VARIABLES LIKE 'tidb_enable_noop_variables'`) + require.NoError(t, err) + req = r.NewChunk(nil) + err = r.Next(ctx, req) + require.NoError(t, err) + require.Equal(t, 1, req.NumRows()) + row := req.GetRow(0) + require.Equal(t, []byte("OFF"), row.GetBytes(1)) + require.NoError(t, r.Close()) +} + func TestTiDBEnablePagingVariable(t *testing.T) { store, dom := createStoreAndBootstrap(t) se := createSessionAndSetID(t, store) @@ -1273,3 +1329,56 @@ func TestTiDBCostModelUpgradeFrom610To650(t *testing.T) { }() } } + +func TestTiDBGCAwareUpgradeFrom630To650(t *testing.T) { + ctx := context.Background() + store, _ := createStoreAndBootstrap(t) + defer func() { require.NoError(t, store.Close()) }() + + // upgrade from 6.3 to 6.5+. + ver63 := version93 + seV63 := createSessionAndSetID(t, store) + txn, err := store.Begin() + require.NoError(t, err) + m := meta.NewMeta(txn) + err = m.FinishBootstrap(int64(ver63)) + require.NoError(t, err) + err = txn.Commit(context.Background()) + require.NoError(t, err) + mustExec(t, seV63, fmt.Sprintf("update mysql.tidb set variable_value=%d where variable_name='tidb_server_version'", ver63)) + mustExec(t, seV63, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", "1", variable.TiDBEnableGCAwareMemoryTrack)) + mustExec(t, seV63, "commit") + unsetStoreBootstrapped(store.UUID()) + ver, err := getBootstrapVersion(seV63) + require.NoError(t, err) + require.Equal(t, int64(ver63), ver) + + // We are now in 6.3, tidb_enable_gc_aware_memory_track is ON. + res := mustExecToRecodeSet(t, seV63, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableGCAwareMemoryTrack)) + chk := res.NewChunk(nil) + err = res.Next(ctx, chk) + require.NoError(t, err) + require.Equal(t, 1, chk.NumRows()) + row := chk.GetRow(0) + require.Equal(t, 2, row.Len()) + require.Equal(t, "1", row.GetString(1)) + + // Upgrade to 6.5. + domCurVer, err := BootstrapSession(store) + require.NoError(t, err) + defer domCurVer.Close() + seCurVer := createSessionAndSetID(t, store) + ver, err = getBootstrapVersion(seCurVer) + require.NoError(t, err) + require.Equal(t, currentBootstrapVersion, ver) + + // We are now in 6.5. + res = mustExecToRecodeSet(t, seCurVer, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableGCAwareMemoryTrack)) + chk = res.NewChunk(nil) + err = res.Next(ctx, chk) + require.NoError(t, err) + require.Equal(t, 1, chk.NumRows()) + row = chk.GetRow(0) + require.Equal(t, 2, row.Len()) + require.Equal(t, "0", row.GetString(1)) +} diff --git a/session/session.go b/session/session.go index 63bc1c970fe08..528f647978240 100644 --- a/session/session.go +++ b/session/session.go @@ -2091,7 +2091,7 @@ func (s *session) ExecRestrictedSQL(ctx context.Context, opts []sqlexec.OptionFu metrics.SessionRestrictedSQLCounter.Inc() ctx = context.WithValue(ctx, execdetails.StmtExecDetailKey, &execdetails.StmtExecDetails{}) ctx = context.WithValue(ctx, tikvutil.ExecDetailsKey, &tikvutil.ExecDetails{}) - rs, err := se.ExecuteStmt(ctx, stmt) + rs, err := se.ExecuteInternalStmt(ctx, stmt) if err != nil { se.sessionVars.StmtCtx.AppendError(err) } @@ -2113,6 +2113,18 @@ func (s *session) ExecRestrictedSQL(ctx context.Context, opts []sqlexec.OptionFu }) } +// ExecuteInternalStmt execute internal stmt +func (s *session) ExecuteInternalStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlexec.RecordSet, error) { + origin := s.sessionVars.InRestrictedSQL + s.sessionVars.InRestrictedSQL = true + defer func() { + s.sessionVars.InRestrictedSQL = origin + // Restore the goroutine label by using the original ctx after execution is finished. + pprof.SetGoroutineLabels(ctx) + }() + return s.ExecuteStmt(ctx, stmtNode) +} + func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlexec.RecordSet, error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("session.ExecuteStmt", opentracing.ChildOf(span.Context())) @@ -2168,10 +2180,6 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex // Transform abstract syntax tree to a physical plan(stored in executor.ExecStmt). compiler := executor.Compiler{Ctx: s} stmt, err := compiler.Compile(ctx, stmtNode) - if err == nil { - err = sessiontxn.OptimizeWithPlanAndThenWarmUp(s, stmt.Plan) - } - if err != nil { s.rollbackOnError(ctx) @@ -3285,7 +3293,7 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { analyzeConcurrencyQuota := int(config.GetGlobalConfig().Performance.AnalyzePartitionConcurrencyQuota) concurrency := int(config.GetGlobalConfig().Performance.StatsLoadConcurrency) - ses, err := createSessions(store, 9) + ses, err := createSessions(store, 10) if err != nil { return nil, err } @@ -3385,7 +3393,13 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { // setup historical stats worker dom.SetupHistoricalStatsWorker(ses[8]) dom.StartHistoricalStatsWorker() - + if runBootstrapSQLFile { + pm := &privileges.UserPrivileges{ + Handle: dom.PrivilegeHandle(), + } + privilege.BindPrivilegeManager(ses[9], pm) + doBootstrapSQLFile(ses[9]) + } // A sub context for update table stats, and other contexts for concurrent stats loading. cnt := 1 + concurrency syncStatsCtxs, err := createSessions(store, cnt) @@ -4143,23 +4157,26 @@ func (s *session) setRequestSource(ctx context.Context, stmtLabel string, stmtNo } else { s.sessionVars.RequestSourceType = stmtLabel } - } else { - if source := ctx.Value(kv.RequestSourceKey); source != nil { - s.sessionVars.RequestSourceType = source.(kv.RequestSource).RequestSourceType - } else { - // panic in test mode in case there are requests without source in the future. - // log warnings in production mode. - if flag.Lookup("test.v") != nil || flag.Lookup("check.v") != nil { - panic("unexpected no source type context, if you see this error, " + - "the `RequestSourceTypeKey` is missing in your context") - } else { - logutil.Logger(ctx).Warn("unexpected no source type context, if you see this warning, "+ - "the `RequestSourceTypeKey` is missing in the context", - zap.Bool("internal", s.isInternal()), - zap.String("sql", stmtNode.Text())) - } + return + } + if source := ctx.Value(kv.RequestSourceKey); source != nil { + requestSource := source.(kv.RequestSource) + if requestSource.RequestSourceType != "" { + s.sessionVars.RequestSourceType = requestSource.RequestSourceType + return } } + // panic in test mode in case there are requests without source in the future. + // log warnings in production mode. + if flag.Lookup("test.v") != nil || flag.Lookup("check.v") != nil { + panic("unexpected no source type context, if you see this error, " + + "the `RequestSourceTypeKey` is missing in your context") + } else { + logutil.Logger(ctx).Warn("unexpected no source type context, if you see this warning, "+ + "the `RequestSourceTypeKey` is missing in the context", + zap.Bool("internal", s.isInternal()), + zap.String("sql", stmtNode.Text())) + } } // RemoveLockDDLJobs removes the DDL jobs which doesn't get the metadata lock from job2ver. diff --git a/sessionctx/sessionstates/session_states.go b/sessionctx/sessionstates/session_states.go index 36ea0b22455d7..c9e1652a9c1df 100644 --- a/sessionctx/sessionstates/session_states.go +++ b/sessionctx/sessionstates/session_states.go @@ -15,8 +15,6 @@ package sessionstates import ( - "time" - "github.com/pingcap/tidb/errno" ptypes "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -79,7 +77,6 @@ type SessionStates struct { FoundInPlanCache bool `json:"in-plan-cache,omitempty"` FoundInBinding bool `json:"in-binding,omitempty"` SequenceLatestValues map[int64]int64 `json:"seq-values,omitempty"` - MPPStoreLastFailTime map[string]time.Time `json:"store-fail-time,omitempty"` LastAffectedRows int64 `json:"affected-rows,omitempty"` LastInsertID uint64 `json:"last-insert-id,omitempty"` Warnings []stmtctx.SQLWarn `json:"warnings,omitempty"` diff --git a/sessionctx/sessionstates/session_states_test.go b/sessionctx/sessionstates/session_states_test.go index 5910d6b18e071..21de8d53727d6 100644 --- a/sessionctx/sessionstates/session_states_test.go +++ b/sessionctx/sessionstates/session_states_test.go @@ -20,9 +20,7 @@ import ( "fmt" "strconv" "strings" - "sync" "testing" - "time" "github.com/pingcap/errors" "github.com/pingcap/tidb/config" @@ -133,7 +131,7 @@ func TestSystemVars(t *testing.T) { { // sem invisible variable inSessionStates: false, - varName: variable.TiDBAllowRemoveAutoInc, + varName: variable.TiDBConfig, }, { // noop variables @@ -378,23 +376,6 @@ func TestSessionCtx(t *testing.T) { tk.MustQuery("select nextval(test.s)").Check(testkit.Rows("2")) }, }, - { - // check MPPStoreLastFailTime - setFunc: func(tk *testkit.TestKit) any { - m := sync.Map{} - m.Store("store1", time.Now()) - tk.Session().GetSessionVars().MPPStoreLastFailTime = &m - return tk.Session().GetSessionVars().MPPStoreLastFailTime - }, - checkFunc: func(tk *testkit.TestKit, param any) { - failTime := tk.Session().GetSessionVars().MPPStoreLastFailTime - tm, ok := failTime.Load("store1") - require.True(t, ok) - v, ok := (param.(*sync.Map)).Load("store1") - require.True(t, ok) - require.True(t, tm.(time.Time).Equal(v.(time.Time))) - }, - }, { // check FoundInPlanCache setFunc: func(tk *testkit.TestKit) any { diff --git a/sessionctx/stmtctx/BUILD.bazel b/sessionctx/stmtctx/BUILD.bazel index b42939f7ec324..a2e12d1e860e1 100644 --- a/sessionctx/stmtctx/BUILD.bazel +++ b/sessionctx/stmtctx/BUILD.bazel @@ -6,6 +6,7 @@ go_library( importpath = "github.com/pingcap/tidb/sessionctx/stmtctx", visibility = ["//visibility:public"], deps = [ + "//errno", "//parser", "//parser/ast", "//parser/model", diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 47159dc8f8a60..799f2f8bdeea8 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -24,6 +24,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" @@ -379,6 +380,13 @@ type StatementContext struct { HasFKCascades bool } + // MPPQueryInfo stores some id and timestamp of current MPP query statement. + MPPQueryInfo struct { + QueryID atomic2.Uint64 + QueryTS atomic2.Uint64 + AllocatedMPPTaskID atomic2.Int64 + } + // TableStats stores the visited runtime table stats by table id during query TableStats map[int64]interface{} // useChunkAlloc indicates whether statement use chunk alloc @@ -865,6 +873,21 @@ func (sc *StatementContext) HandleTruncate(err error) error { if err == nil { return nil } + + err = errors.Cause(err) + if e, ok := err.(*errors.Error); !ok || + (e.Code() != errno.ErrTruncatedWrongValue && + e.Code() != errno.ErrDataTooLong && + e.Code() != errno.ErrTruncatedWrongValueForField && + e.Code() != errno.ErrWarnDataOutOfRange && + e.Code() != errno.ErrDataOutOfRange && + e.Code() != errno.ErrBadNumber && + e.Code() != errno.ErrWrongValueForType && + e.Code() != errno.ErrDatetimeFunctionOverflow && + e.Code() != errno.WarnDataTruncated) { + return err + } + if sc.IgnoreTruncate { return nil } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 695d0ad48a5a5..977b34b1578a9 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -671,13 +671,6 @@ type SessionVars struct { value string } - // mppTaskIDAllocator is used to allocate mpp task id for a session. - mppTaskIDAllocator struct { - mu sync.Mutex - lastTS uint64 - taskID int64 - } - // Status stands for the session status. e.g. in transaction or not, auto commit is on or off, and so on. Status uint16 @@ -1171,9 +1164,6 @@ type SessionVars struct { // TemporaryTableData stores committed kv values for temporary table for current session. TemporaryTableData TemporaryTableData - // MPPStoreLastFailTime records the lastest fail time that a TiFlash store failed. It maps store address(string) to fail time(time.Time). - MPPStoreLastFailTime *sync.Map - // MPPStoreFailTTL indicates the duration that protect TiDB from sending task to a new recovered TiFlash. MPPStoreFailTTL string @@ -1454,20 +1444,6 @@ func (s *SessionVars) InitStatementContext() *stmtctx.StatementContext { return sc } -// AllocMPPTaskID allocates task id for mpp tasks. It will reset the task id if the query's -// startTs is different. -func (s *SessionVars) AllocMPPTaskID(startTS uint64) int64 { - s.mppTaskIDAllocator.mu.Lock() - defer s.mppTaskIDAllocator.mu.Unlock() - if s.mppTaskIDAllocator.lastTS == startTS { - s.mppTaskIDAllocator.taskID++ - return s.mppTaskIDAllocator.taskID - } - s.mppTaskIDAllocator.lastTS = startTS - s.mppTaskIDAllocator.taskID = 1 - return 1 -} - // IsMPPAllowed returns whether mpp execution is allowed. func (s *SessionVars) IsMPPAllowed() bool { return s.allowMPPExecution @@ -1713,7 +1689,6 @@ func NewSessionVars(hctx HookContext) *SessionVars { AllowFallbackToTiKV: make(map[kv.StoreType]struct{}), CTEMaxRecursionDepth: DefCTEMaxRecursionDepth, TMPTableSize: DefTiDBTmpTableMaxSize, - MPPStoreLastFailTime: new(sync.Map), MPPStoreFailTTL: DefTiDBMPPStoreFailTTL, Rng: mathutil.NewWithTime(), StatsLoadSyncWait: StatsLoadSyncWait.Load(), @@ -2336,12 +2311,6 @@ func (s *SessionVars) EncodeSessionStates(ctx context.Context, sessionStates *se } sessionStates.LastFoundRows = s.LastFoundRows sessionStates.SequenceLatestValues = s.SequenceState.GetAllStates() - sessionStates.MPPStoreLastFailTime = make(map[string]time.Time, 0) - s.MPPStoreLastFailTime.Range( - func(key, value interface{}) bool { - sessionStates.MPPStoreLastFailTime[key.(string)] = value.(time.Time) - return true - }) sessionStates.FoundInPlanCache = s.PrevFoundInPlanCache sessionStates.FoundInBinding = s.PrevFoundInBinding @@ -2377,9 +2346,6 @@ func (s *SessionVars) DecodeSessionStates(ctx context.Context, sessionStates *se } s.LastFoundRows = sessionStates.LastFoundRows s.SequenceState.SetAllStates(sessionStates.SequenceLatestValues) - for k, v := range sessionStates.MPPStoreLastFailTime { - s.MPPStoreLastFailTime.Store(k, v) - } s.FoundInPlanCache = sessionStates.FoundInPlanCache s.FoundInBinding = sessionStates.FoundInBinding diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index b624005b7b512..5df5e187088d0 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -130,16 +130,9 @@ func TestSession(t *testing.T) { func TestAllocMPPID(t *testing.T) { ctx := mock.NewContext() - - seVar := ctx.GetSessionVars() - require.NotNil(t, seVar) - - require.Equal(t, int64(1), seVar.AllocMPPTaskID(1)) - require.Equal(t, int64(2), seVar.AllocMPPTaskID(1)) - require.Equal(t, int64(3), seVar.AllocMPPTaskID(1)) - require.Equal(t, int64(1), seVar.AllocMPPTaskID(2)) - require.Equal(t, int64(2), seVar.AllocMPPTaskID(2)) - require.Equal(t, int64(3), seVar.AllocMPPTaskID(2)) + require.Equal(t, int64(1), plannercore.AllocMPPTaskID(ctx)) + require.Equal(t, int64(2), plannercore.AllocMPPTaskID(ctx)) + require.Equal(t, int64(3), plannercore.AllocMPPTaskID(ctx)) } func TestSlowLogFormat(t *testing.T) { diff --git a/sessiontxn/interface.go b/sessiontxn/interface.go index 85d9217a90c0f..d91f1c291b588 100644 --- a/sessiontxn/interface.go +++ b/sessiontxn/interface.go @@ -160,8 +160,10 @@ type TxnManager interface { // GetReadReplicaScope returns the read replica scope GetReadReplicaScope() string // GetStmtReadTS returns the read timestamp used by select statement (not for select ... for update) + // Calling this method will activate the txn implicitly if current read is not stale/historical read GetStmtReadTS() (uint64, error) // GetStmtForUpdateTS returns the read timestamp used by update/insert/delete or select ... for update + // Calling this method will activate the txn implicitly if current read is not stale/historical read GetStmtForUpdateTS() (uint64, error) // GetContextProvider returns the current TxnContextProvider GetContextProvider() TxnContextProvider diff --git a/sessiontxn/isolation/optimistic.go b/sessiontxn/isolation/optimistic.go index 3c60eba09331b..9a1f8d58aabbd 100644 --- a/sessiontxn/isolation/optimistic.go +++ b/sessiontxn/isolation/optimistic.go @@ -114,6 +114,12 @@ func (p *OptimisticTxnContextProvider) AdviseOptimizeWithPlan(plan interface{}) return nil } + if p.txn != nil { + // `p.txn != nil` means the txn has already been activated, we should not optimize the startTS because the startTS + // has already been used. + return nil + } + realPlan, ok := plan.(plannercore.Plan) if !ok { return nil @@ -141,7 +147,7 @@ func (p *OptimisticTxnContextProvider) AdviseOptimizeWithPlan(plan interface{}) zap.Uint64("conn", sessVars.ConnectionID), zap.String("text", sessVars.StmtCtx.OriginalSQL), ) - return nil + return err } p.optimizeWithMaxTS = true diff --git a/sessiontxn/txn_rc_tso_optimize_test.go b/sessiontxn/txn_rc_tso_optimize_test.go index f321d40340496..3e945778ca107 100644 --- a/sessiontxn/txn_rc_tso_optimize_test.go +++ b/sessiontxn/txn_rc_tso_optimize_test.go @@ -105,7 +105,7 @@ func TestRcTSOCmdCountForPrepareExecuteNormal(t *testing.T) { tk.MustExec("commit") } countTsoRequest, countTsoUseConstant, countWaitTsoOracle := getAllTsoCounter(sctx) - require.Equal(t, uint64(398), countTsoRequest.(uint64)) + require.Equal(t, uint64(496), countTsoRequest.(uint64)) require.Equal(t, uint64(594), countTsoUseConstant.(uint64)) require.Equal(t, uint64(198), countWaitTsoOracle.(uint64)) @@ -137,7 +137,7 @@ func TestRcTSOCmdCountForPrepareExecuteNormal(t *testing.T) { tk.MustExec("commit") } count := sctx.Value(sessiontxn.TsoRequestCount) - require.Equal(t, uint64(594), count) + require.Equal(t, uint64(693), count) } func TestRcTSOCmdCountForPrepareExecuteExtra(t *testing.T) { @@ -234,7 +234,7 @@ func TestRcTSOCmdCountForPrepareExecuteExtra(t *testing.T) { tk.MustExec("commit") } countTsoRequest, countTsoUseConstant, countWaitTsoOracle = getAllTsoCounter(sctx) - require.Equal(t, uint64(16), countTsoRequest.(uint64)) + require.Equal(t, uint64(20), countTsoRequest.(uint64)) require.Equal(t, uint64(5), countTsoUseConstant.(uint64)) require.Equal(t, uint64(5), countWaitTsoOracle.(uint64)) @@ -412,7 +412,7 @@ func TestRcTSOCmdCountForPrepareExecuteExtra(t *testing.T) { require.Nil(t, stmt) tk.MustExec("commit") countTsoRequest, countTsoUseConstant, countWaitTsoOracle = getAllTsoCounter(sctx) - require.Equal(t, uint64(3), countTsoRequest.(uint64)) + require.Equal(t, uint64(4), countTsoRequest.(uint64)) require.Equal(t, uint64(2), countTsoUseConstant.(uint64)) require.Equal(t, 0, countWaitTsoOracle.(int)) tk.MustQuery("SELECT * FROM t1 WHERE id1 = 1").Check(testkit.Rows("1 1 1")) diff --git a/statistics/handle/gc.go b/statistics/handle/gc.go index ab2159bf22bf0..f16e2c9719088 100644 --- a/statistics/handle/gc.go +++ b/statistics/handle/gc.go @@ -22,7 +22,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/sqlexec" @@ -153,14 +155,32 @@ func (h *Handle) ClearOutdatedHistoryStats() error { h.mu.Lock() defer h.mu.Unlock() exec := h.mu.ctx.(sqlexec.SQLExecutor) - sql := "delete from mysql.stats_meta_history where NOW() - create_time >= %?" - _, err := exec.ExecuteInternal(ctx, sql, variable.HistoricalStatsDuration.Load().Seconds()) + sql := "select count(*) from mysql.stats_meta_history where NOW() - create_time >= %?" + rs, err := exec.ExecuteInternal(ctx, sql, variable.HistoricalStatsDuration.Load().Seconds()) if err != nil { return err } - sql = "delete from mysql.stats_history where NOW() - create_time >= %? " - _, err = exec.ExecuteInternal(ctx, sql, variable.HistoricalStatsDuration.Load().Seconds()) - return err + if rs == nil { + return nil + } + var rows []chunk.Row + defer terror.Call(rs.Close) + if rows, err = sqlexec.DrainRecordSet(ctx, rs, 8); err != nil { + return errors.Trace(err) + } + count := rows[0].GetInt64(0) + if count > 0 { + sql = "delete from mysql.stats_meta_history where NOW() - create_time >= %?" + _, err = exec.ExecuteInternal(ctx, sql, variable.HistoricalStatsDuration.Load().Seconds()) + if err != nil { + return err + } + sql = "delete from mysql.stats_history where NOW() - create_time >= %? " + _, err = exec.ExecuteInternal(ctx, sql, variable.HistoricalStatsDuration.Load().Seconds()) + logutil.BgLogger().Info("clear outdated historical stats") + return err + } + return nil } func (h *Handle) gcHistoryStatsFromKV(physicalID int64) error { diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index c85656ff99d4b..f53f075301acb 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -1906,7 +1906,9 @@ func (h *Handle) histogramFromStorage(reader *statsReader, tableID int64, colID lowerBound = rows[i].GetDatum(2, &fields[2].Column.FieldType) upperBound = rows[i].GetDatum(3, &fields[3].Column.FieldType) } else { - sc := &stmtctx.StatementContext{TimeZone: time.UTC} + // Invalid date values may be inserted into table under some relaxed sql mode. Those values may exist in statistics. + // Hence, when reading statistics, we should skip invalid date check. See #39336. + sc := &stmtctx.StatementContext{TimeZone: time.UTC, AllowInvalidDate: true, IgnoreZeroInDate: true} d := rows[i].GetDatum(2, &fields[2].Column.FieldType) // For new collation data, when storing the bounds of the histogram, we store the collate key instead of the // original value. diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 5e41b1c1279a2..9bb80498bc90f 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -3543,3 +3543,35 @@ func TestStatsLockAndUnlockTables(t *testing.T) { tbl2Stats2 := handle.GetTableStats(tbl2.Meta()) require.Equal(t, int64(2), tbl2Stats2.Count) } + +func TestIssue39336(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(` +create table t1 ( + a datetime(3) default null, + b int +) partition by range (b) ( + partition p0 values less than (1000), + partition p1 values less than (maxvalue) +)`) + tk.MustExec("set @@sql_mode=''") + tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk.MustExec(` +insert into t1 values +('1000-00-09 00:00:00.000', 1), +('1000-00-06 00:00:00.000', 1), +('1000-00-06 00:00:00.000', 1), +('2022-11-23 14:24:30.000', 1), +('2022-11-23 14:24:32.000', 1), +('2022-11-23 14:24:33.000', 1), +('2022-11-23 14:24:35.000', 1), +('2022-11-23 14:25:08.000', 1001), +('2022-11-23 14:25:09.000', 1001)`) + tk.MustExec("analyze table t1 with 0 topn") + rows := tk.MustQuery("show analyze status where job_info like 'merge global stats%'").Rows() + require.Len(t, rows, 1) + require.Equal(t, "finished", rows[0][7]) +} diff --git a/store/copr/BUILD.bazel b/store/copr/BUILD.bazel index eb3eb2f016424..f6cbe57efa2d7 100644 --- a/store/copr/BUILD.bazel +++ b/store/copr/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "coprocessor_cache.go", "key_ranges.go", "mpp.go", + "mpp_probe.go", "region_cache.go", "store.go", ], @@ -66,6 +67,7 @@ go_test( "coprocessor_test.go", "key_ranges_test.go", "main_test.go", + "mpp_probe_test.go", ], embed = [":copr"], flaky = True, @@ -75,12 +77,15 @@ go_test( "//store/driver/backoff", "//testkit/testsetup", "//util/paging", + "@com_github_pingcap_errors//:errors", "@com_github_pingcap_kvproto//pkg/coprocessor", + "@com_github_pingcap_kvproto//pkg/mpp", "@com_github_stathat_consistent//:consistent", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//config", "@com_github_tikv_client_go_v2//testutils", "@com_github_tikv_client_go_v2//tikv", + "@com_github_tikv_client_go_v2//tikvrpc", "@org_uber_go_goleak//:goleak", ], ) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 801eebc40de9a..b316d10acaf6e 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/log" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" @@ -295,12 +294,11 @@ func balanceBatchCopTaskWithContinuity(storeTaskMap map[uint64]*batchCopTask, ca // // The second balance strategy: Not only consider the region count between TiFlash stores, but also try to make the regions' range continuous(stored in TiFlash closely). // If balanceWithContinuity is true, the second balance strategy is enable. -func balanceBatchCopTask(ctx context.Context, kvStore *kvStore, originalTasks []*batchCopTask, mppStoreLastFailTime *sync.Map, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64) []*batchCopTask { +func balanceBatchCopTask(ctx context.Context, kvStore *kvStore, originalTasks []*batchCopTask, isMPP bool, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64) []*batchCopTask { if len(originalTasks) == 0 { log.Info("Batch cop task balancer got an empty task set.") return originalTasks } - isMPP := mppStoreLastFailTime != nil // for mpp, we still need to detect the store availability if len(originalTasks) <= 1 && !isMPP { return originalTasks @@ -331,45 +329,21 @@ func balanceBatchCopTask(ctx context.Context, kvStore *kvStore, originalTasks [] var wg sync.WaitGroup var mu sync.Mutex wg.Add(len(stores)) - cur := time.Now() for i := range stores { go func(idx int) { defer wg.Done() s := stores[idx] - var lastAny any - var ok bool - mu.Lock() - if lastAny, ok = mppStoreLastFailTime.Load(s.GetAddr()); ok && cur.Sub(lastAny.(time.Time)) < 100*time.Millisecond { - // The interval time is so short that may happen in a same query, so we needn't to check again. - mu.Unlock() - return - } else if !ok { - lastAny = time.Time{} - } - mu.Unlock() - - resp, err := kvStore.GetTiKVClient().SendRequest(ctx, s.GetAddr(), &tikvrpc.Request{ - Type: tikvrpc.CmdMPPAlive, - StoreTp: tikvrpc.TiFlash, - Req: &mpp.IsAliveRequest{}, - Context: kvrpcpb.Context{}, - }, 2*time.Second) - - if err != nil || !resp.Resp.(*mpp.IsAliveResponse).Available { - errMsg := "store not ready to serve" - if err != nil { - errMsg = err.Error() - } - logutil.BgLogger().Warn("Store is not ready", zap.String("store address", s.GetAddr()), zap.String("err message", errMsg)) - mu.Lock() - mppStoreLastFailTime.Store(s.GetAddr(), time.Now()) - mu.Unlock() + // check if store is failed already. + ok := GlobalMPPFailedStoreProber.IsRecovery(ctx, s.GetAddr(), ttl) + if !ok { return } - if cur.Sub(lastAny.(time.Time)) < ttl { - logutil.BgLogger().Warn("Cannot detect store's availability because the current time has not reached MPPStoreLastFailTime + MPPStoreFailTTL", zap.String("store address", s.GetAddr()), zap.Time("last fail time", lastAny.(time.Time))) + tikvClient := kvStore.GetTiKVClient() + ok = detectMPPStore(ctx, tikvClient, s.GetAddr(), DetectTimeoutLimit) + if !ok { + GlobalMPPFailedStoreProber.Add(ctx, s.GetAddr(), tikvClient) return } @@ -534,29 +508,29 @@ func buildBatchCopTasksForNonPartitionedTable(bo *backoff.Backoffer, store *kvStore, ranges *KeyRanges, storeType kv.StoreType, - mppStoreLastFailTime *sync.Map, + isMPP bool, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64) ([]*batchCopTask, error) { if config.GetGlobalConfig().DisaggregatedTiFlash { - return buildBatchCopTasksConsistentHash(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) + return buildBatchCopTasksConsistentHash(bo, store, []*KeyRanges{ranges}, storeType, isMPP, ttl, balanceWithContinuity, balanceContinuousRegionCount) } - return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) + return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, isMPP, ttl, balanceWithContinuity, balanceContinuousRegionCount) } func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType, - mppStoreLastFailTime *sync.Map, + isMPP bool, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64, partitionIDs []int64) (batchTasks []*batchCopTask, err error) { if config.GetGlobalConfig().DisaggregatedTiFlash { - batchTasks, err = buildBatchCopTasksConsistentHash(bo, store, rangesForEachPhysicalTable, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) + batchTasks, err = buildBatchCopTasksConsistentHash(bo, store, rangesForEachPhysicalTable, storeType, isMPP, ttl, balanceWithContinuity, balanceContinuousRegionCount) } else { - batchTasks, err = buildBatchCopTasksCore(bo, store, rangesForEachPhysicalTable, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) + batchTasks, err = buildBatchCopTasksCore(bo, store, rangesForEachPhysicalTable, storeType, isMPP, ttl, balanceWithContinuity, balanceContinuousRegionCount) } if err != nil { return nil, err @@ -566,8 +540,8 @@ func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, return batchTasks, nil } -func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType, mppStoreLastFailTime *sync.Map, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64) ([]*batchCopTask, error) { - batchTasks, err := buildBatchCopTasksCore(bo, store, rangesForEachPhysicalTable, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) +func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType, isMPP bool, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64) ([]*batchCopTask, error) { + batchTasks, err := buildBatchCopTasksCore(bo, store, rangesForEachPhysicalTable, storeType, isMPP, ttl, balanceWithContinuity, balanceContinuousRegionCount) if err != nil { return nil, err } @@ -614,7 +588,7 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran // When `partitionIDs != nil`, it means that buildBatchCopTasksCore is constructing a batch cop tasks for PartitionTableScan. // At this time, `len(rangesForEachPhysicalTable) == len(partitionIDs)` and `rangesForEachPhysicalTable[i]` is for partition `partitionIDs[i]`. // Otherwise, `rangesForEachPhysicalTable[0]` indicates the range for the single physical table. -func buildBatchCopTasksCore(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType, mppStoreLastFailTime *sync.Map, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64) ([]*batchCopTask, error) { +func buildBatchCopTasksCore(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType, isMPP bool, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64) ([]*batchCopTask, error) { cache := store.GetRegionCache() start := time.Now() const cmdType = tikvrpc.CmdBatchCop @@ -644,7 +618,6 @@ func buildBatchCopTasksCore(bo *backoff.Backoffer, store *kvStore, rangesForEach storeTaskMap := make(map[string]*batchCopTask) needRetry := false - isMPP := mppStoreLastFailTime != nil for _, task := range tasks { rpcCtx, err := cache.GetTiFlashRPCContext(bo.TiKVBackoffer(), task.region, isMPP) if err != nil { @@ -696,7 +669,7 @@ func buildBatchCopTasksCore(bo *backoff.Backoffer, store *kvStore, rangesForEach logutil.BgLogger().Debug(msg) } balanceStart := time.Now() - batchTasks = balanceBatchCopTask(bo.GetCtx(), store, batchTasks, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) + batchTasks = balanceBatchCopTask(bo.GetCtx(), store, batchTasks, isMPP, ttl, balanceWithContinuity, balanceContinuousRegionCount) balanceElapsed := time.Since(balanceStart) if log.GetLevel() <= zap.DebugLevel { msg := "After region balance:" @@ -762,11 +735,11 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V keyRanges = append(keyRanges, NewKeyRanges(pi.KeyRanges)) partitionIDs = append(partitionIDs, pi.ID) } - tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store.kvStore, keyRanges, req.StoreType, nil, 0, false, 0, partitionIDs) + tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store.kvStore, keyRanges, req.StoreType, false, 0, false, 0, partitionIDs) } else { // TODO: merge the if branch. ranges := NewKeyRanges(req.KeyRanges.FirstPartitionRange()) - tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store.kvStore, ranges, req.StoreType, nil, 0, false, 0) + tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store.kvStore, ranges, req.StoreType, false, 0, false, 0) } if err != nil { @@ -913,7 +886,7 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoff.Ba ranges = append(ranges, *ran) }) } - ret, err := buildBatchCopTasksForNonPartitionedTable(bo, b.store, NewKeyRanges(ranges), b.req.StoreType, nil, 0, false, 0) + ret, err := buildBatchCopTasksForNonPartitionedTable(bo, b.store, NewKeyRanges(ranges), b.req.StoreType, false, 0, false, 0) return ret, err } // Retry Partition Table Scan @@ -932,7 +905,7 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoff.Ba } keyRanges = append(keyRanges, NewKeyRanges(ranges)) } - ret, err := buildBatchCopTasksForPartitionedTable(bo, b.store, keyRanges, b.req.StoreType, nil, 0, false, 0, pid) + ret, err := buildBatchCopTasksForPartitionedTable(bo, b.store, keyRanges, b.req.StoreType, false, 0, false, 0, pid) return ret, err } diff --git a/store/copr/batch_coprocessor_test.go b/store/copr/batch_coprocessor_test.go index 5616f61c54365..3e10ce627b1f6 100644 --- a/store/copr/batch_coprocessor_test.go +++ b/store/copr/batch_coprocessor_test.go @@ -120,13 +120,13 @@ func TestBalanceBatchCopTaskWithContinuity(t *testing.T) { func TestBalanceBatchCopTaskWithEmptyTaskSet(t *testing.T) { { var nilTaskSet []*batchCopTask - nilResult := balanceBatchCopTask(nil, nil, nilTaskSet, nil, time.Second, false, 0) + nilResult := balanceBatchCopTask(nil, nil, nilTaskSet, false, time.Second, false, 0) require.True(t, nilResult == nil) } { emptyTaskSet := make([]*batchCopTask, 0) - emptyResult := balanceBatchCopTask(nil, nil, emptyTaskSet, nil, time.Second, false, 0) + emptyResult := balanceBatchCopTask(nil, nil, emptyTaskSet, false, time.Second, false, 0) require.True(t, emptyResult != nil) require.True(t, len(emptyResult) == 0) } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 390c5ffe8e63a..94a2f478290a8 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -213,7 +213,8 @@ func (c *CopClient) BuildCopIterator(ctx context.Context, req *kv.Request, vars // higher concurrency, the data is just cached and not consumed for a while, this increase the memory usage. // Set concurrency to 2 can reduce the memory usage and I've tested that it does not necessarily // decrease the performance. - if it.concurrency > 2 { + // For ReqTypeAnalyze, we keep its concurrency to avoid slow analyze(see https://github.com/pingcap/tidb/issues/40162 for details). + if it.concurrency > 2 && it.req.Tp != kv.ReqTypeAnalyze { oldConcurrency := it.concurrency it.concurrency = 2 diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 37d5629ac6f3d..c3225c40d1455 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -62,7 +62,7 @@ func (c *MPPClient) selectAllTiFlashStore() []kv.MPPTaskMeta { } // ConstructMPPTasks receives ScheduleRequest, which are actually collects of kv ranges. We allocates MPPTaskMeta for them and returns. -func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasksRequest, mppStoreLastFailTime *sync.Map, ttl time.Duration) ([]kv.MPPTaskMeta, error) { +func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasksRequest, ttl time.Duration) ([]kv.MPPTaskMeta, error) { ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTS) bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) var tasks []*batchCopTask @@ -74,13 +74,13 @@ func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasks rangesForEachPartition[i] = NewKeyRanges(p.KeyRanges) partitionIDs[i] = p.ID } - tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store, rangesForEachPartition, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20, partitionIDs) + tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store, rangesForEachPartition, kv.TiFlash, true, ttl, true, 20, partitionIDs) } else { if req.KeyRanges == nil { return c.selectAllTiFlashStore(), nil } ranges := NewKeyRanges(req.KeyRanges) - tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store, ranges, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20) + tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store, ranges, kv.TiFlash, true, ttl, true, 20) } if err != nil { @@ -143,7 +143,8 @@ type mppIterator struct { tasks []*kv.MPPDispatchRequest finishCh chan struct{} - startTs uint64 + startTs uint64 + mppQueryID kv.MPPQueryID respChan chan *mppResponse @@ -220,7 +221,8 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req } // meta for current task. - taskMeta := &mpp.TaskMeta{StartTs: req.StartTs, TaskId: req.ID, Address: req.Meta.GetAddress()} + taskMeta := &mpp.TaskMeta{StartTs: req.StartTs, QueryTs: req.MppQueryID.QueryTs, LocalQueryId: req.MppQueryID.LocalQueryID, TaskId: req.ID, ServerId: req.MppQueryID.ServerID, + Address: req.Meta.GetAddress()} mppReq := &mpp.DispatchTaskRequest{ Meta: taskMeta, @@ -334,7 +336,7 @@ func (m *mppIterator) cancelMppTasks() { m.mu.Lock() defer m.mu.Unlock() killReq := &mpp.CancelTaskRequest{ - Meta: &mpp.TaskMeta{StartTs: m.startTs}, + Meta: &mpp.TaskMeta{StartTs: m.startTs, QueryTs: m.mppQueryID.QueryTs, LocalQueryId: m.mppQueryID.LocalQueryID, ServerId: m.mppQueryID.ServerID}, } disaggregatedTiFlash := config.GetGlobalConfig().DisaggregatedTiFlash @@ -374,8 +376,11 @@ func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchReques connReq := &mpp.EstablishMPPConnectionRequest{ SenderMeta: taskMeta, ReceiverMeta: &mpp.TaskMeta{ - StartTs: req.StartTs, - TaskId: -1, + StartTs: req.StartTs, + QueryTs: m.mppQueryID.QueryTs, + LocalQueryId: m.mppQueryID.LocalQueryID, + ServerId: m.mppQueryID.ServerID, + TaskId: -1, }, } @@ -528,7 +533,7 @@ func (m *mppIterator) Next(ctx context.Context) (kv.ResultSubset, error) { } // DispatchMPPTasks dispatches all the mpp task and waits for the responses. -func (c *MPPClient) DispatchMPPTasks(ctx context.Context, variables interface{}, dispatchReqs []*kv.MPPDispatchRequest, needTriggerFallback bool, startTs uint64) kv.Response { +func (c *MPPClient) DispatchMPPTasks(ctx context.Context, variables interface{}, dispatchReqs []*kv.MPPDispatchRequest, needTriggerFallback bool, startTs uint64, mppQueryID kv.MPPQueryID) kv.Response { vars := variables.(*tikv.Variables) ctxChild, cancelFunc := context.WithCancel(ctx) iter := &mppIterator{ @@ -539,6 +544,7 @@ func (c *MPPClient) DispatchMPPTasks(ctx context.Context, variables interface{}, cancelFunc: cancelFunc, respChan: make(chan *mppResponse), startTs: startTs, + mppQueryID: mppQueryID, vars: vars, needTriggerFallback: needTriggerFallback, enableCollectExecutionInfo: config.GetGlobalConfig().Instance.EnableCollectExecutionInfo.Load(), diff --git a/store/copr/mpp_probe.go b/store/copr/mpp_probe.go new file mode 100644 index 0000000000000..0a0eba286648e --- /dev/null +++ b/store/copr/mpp_probe.go @@ -0,0 +1,270 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package copr + +import ( + "context" + "fmt" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/mpp" + "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/util/logutil" + "github.com/tikv/client-go/v2/tikv" + "github.com/tikv/client-go/v2/tikvrpc" + "go.uber.org/zap" +) + +// GlobalMPPFailedStoreProber mpp failed store probe +var GlobalMPPFailedStoreProber *MPPFailedStoreProber + +const ( + // DetectPeriod detect period + DetectPeriod = 3 * time.Second + // DetectTimeoutLimit detect timeout + DetectTimeoutLimit = 2 * time.Second + // MaxRecoveryTimeLimit wait TiFlash recovery,more than MPPStoreFailTTL + MaxRecoveryTimeLimit = 15 * time.Minute + // MaxObsoletTimeLimit no request for a long time,that might be obsoleted + MaxObsoletTimeLimit = time.Hour +) + +// MPPStoreState the state for MPPStore. +type MPPStoreState struct { + address string // MPPStore TiFlash address + tikvClient tikv.Client + + lock struct { + sync.Mutex + + recoveryTime time.Time + lastLookupTime time.Time + lastDetectTime time.Time + } +} + +// MPPFailedStoreProber use for detecting of failed TiFlash instance +type MPPFailedStoreProber struct { + failedMPPStores *sync.Map + lock *sync.Mutex + isStop *atomic.Bool + wg *sync.WaitGroup + ctx context.Context + cancel context.CancelFunc + + detectPeriod time.Duration + detectTimeoutLimit time.Duration + maxRecoveryTimeLimit time.Duration + maxObsoletTimeLimit time.Duration +} + +func (t *MPPStoreState) detect(ctx context.Context, detectPeriod time.Duration, detectTimeoutLimit time.Duration) { + if time.Since(t.lock.lastDetectTime) < detectPeriod { + return + } + + defer func() { t.lock.lastDetectTime = time.Now() }() + metrics.TiFlashFailedMPPStoreState.WithLabelValues(t.address).Set(0) + ok := detectMPPStore(ctx, t.tikvClient, t.address, detectTimeoutLimit) + if !ok { + metrics.TiFlashFailedMPPStoreState.WithLabelValues(t.address).Set(1) + t.lock.recoveryTime = time.Time{} // if detect failed,reset recovery time to zero. + return + } + + // record the time of the first recovery + if t.lock.recoveryTime.IsZero() { + t.lock.recoveryTime = time.Now() + } +} + +func (t *MPPStoreState) isRecovery(ctx context.Context, recoveryTTL time.Duration) bool { + if !t.lock.TryLock() { + return false + } + defer t.lock.Unlock() + + t.lock.lastLookupTime = time.Now() + if !t.lock.recoveryTime.IsZero() && time.Since(t.lock.recoveryTime) > recoveryTTL { + return true + } + logutil.Logger(ctx).Debug("Cannot detect store's availability "+ + "because the current time has not recovery or wait mppStoreFailTTL", + zap.String("store address", t.address), + zap.Time("recovery time", t.lock.recoveryTime), + zap.Duration("MPPStoreFailTTL", recoveryTTL)) + return false +} + +func (t MPPFailedStoreProber) scan(ctx context.Context) { + defer func() { + if r := recover(); r != nil { + logutil.Logger(ctx).Warn("mpp failed store probe scan error,will restart", zap.Any("recover", r), zap.Stack("stack")) + } + }() + + do := func(k, v any) { + address := fmt.Sprint(k) + state, ok := v.(*MPPStoreState) + if !ok { + logutil.BgLogger().Warn("MPPStoreState struct assert failed,will be clean", + zap.String("address", address)) + t.Delete(address) + return + } + + if !state.lock.TryLock() { + return + } + defer state.lock.Unlock() + + state.detect(ctx, t.detectPeriod, t.detectTimeoutLimit) + + // clean restored store + if !state.lock.recoveryTime.IsZero() && time.Since(state.lock.recoveryTime) > t.maxRecoveryTimeLimit { + t.Delete(address) + // clean store that may be obsolete + } else if state.lock.recoveryTime.IsZero() && time.Since(state.lock.lastLookupTime) > t.maxObsoletTimeLimit { + t.Delete(address) + } + } + + f := func(k, v any) bool { + go do(k, v) + return true + } + + metrics.TiFlashFailedMPPStoreState.WithLabelValues("probe").Set(-1) //probe heartbeat + t.failedMPPStores.Range(f) +} + +// Add add a store when sync probe failed +func (t *MPPFailedStoreProber) Add(ctx context.Context, address string, tikvClient tikv.Client) { + state := MPPStoreState{ + address: address, + tikvClient: tikvClient, + } + state.lock.lastLookupTime = time.Now() + logutil.Logger(ctx).Debug("add mpp store to failed list", zap.String("address", address)) + t.failedMPPStores.Store(address, &state) +} + +// IsRecovery check whether the store is recovery +func (t *MPPFailedStoreProber) IsRecovery(ctx context.Context, address string, recoveryTTL time.Duration) bool { + logutil.Logger(ctx).Debug("check failed store recovery", + zap.String("address", address), zap.Duration("ttl", recoveryTTL)) + v, ok := t.failedMPPStores.Load(address) + if !ok { + // store not in failed map + return true + } + + state, ok := v.(*MPPStoreState) + if !ok { + logutil.BgLogger().Warn("MPPStoreState struct assert failed,will be clean", + zap.String("address", address)) + t.Delete(address) + return false + } + + return state.isRecovery(ctx, recoveryTTL) +} + +// Run a loop of scan +// there can be only one background task +func (t *MPPFailedStoreProber) Run() { + if !t.lock.TryLock() { + return + } + t.wg.Add(1) + t.isStop.Swap(false) + go func() { + defer t.wg.Done() + defer t.lock.Unlock() + ticker := time.NewTicker(time.Second) + defer ticker.Stop() + + for { + select { + case <-t.ctx.Done(): + logutil.BgLogger().Debug("ctx.done") + return + case <-ticker.C: + t.scan(t.ctx) + } + } + }() + logutil.BgLogger().Debug("run a background probe process for mpp") +} + +// Stop stop background goroutine +func (t *MPPFailedStoreProber) Stop() { + if !t.isStop.CompareAndSwap(false, true) { + return + } + t.cancel() + t.wg.Wait() + logutil.BgLogger().Debug("stop background task") +} + +// Delete clean store from failed map +func (t *MPPFailedStoreProber) Delete(address string) { + metrics.TiFlashFailedMPPStoreState.DeleteLabelValues(address) + _, ok := t.failedMPPStores.LoadAndDelete(address) + if !ok { + logutil.BgLogger().Warn("Store is deleted", zap.String("address", address)) + } +} + +// MPPStore detect function +func detectMPPStore(ctx context.Context, client tikv.Client, address string, detectTimeoutLimit time.Duration) bool { + resp, err := client.SendRequest(ctx, address, &tikvrpc.Request{ + Type: tikvrpc.CmdMPPAlive, + StoreTp: tikvrpc.TiFlash, + Req: &mpp.IsAliveRequest{}, + Context: kvrpcpb.Context{}, + }, detectTimeoutLimit) + if err != nil || !resp.Resp.(*mpp.IsAliveResponse).Available { + if err == nil { + err = fmt.Errorf("store not ready to serve") + } + logutil.BgLogger().Warn("Store is not ready", + zap.String("store address", address), + zap.String("err message", err.Error())) + return false + } + return true +} + +func init() { + ctx, cancel := context.WithCancel(context.Background()) + isStop := atomic.Bool{} + isStop.Swap(true) + GlobalMPPFailedStoreProber = &MPPFailedStoreProber{ + failedMPPStores: &sync.Map{}, + lock: &sync.Mutex{}, + isStop: &isStop, + ctx: ctx, + cancel: cancel, + wg: &sync.WaitGroup{}, + detectPeriod: DetectPeriod, + detectTimeoutLimit: DetectTimeoutLimit, + maxRecoveryTimeLimit: MaxRecoveryTimeLimit, + maxObsoletTimeLimit: MaxObsoletTimeLimit, + } +} diff --git a/store/copr/mpp_probe_test.go b/store/copr/mpp_probe_test.go new file mode 100644 index 0000000000000..7826c970d3e1e --- /dev/null +++ b/store/copr/mpp_probe_test.go @@ -0,0 +1,177 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package copr + +import ( + "context" + "testing" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/mpp" + "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/tikvrpc" +) + +const ( + testimeout = "timeout" + Error = "error" + Normal = "normal" +) + +type mockDetectClient struct { + errortestype string +} + +func (t *mockDetectClient) CloseAddr(string) error { + return nil +} + +func (t *mockDetectClient) Close() error { + return nil +} + +func (t *mockDetectClient) SendRequest( + ctx context.Context, + addr string, + req *tikvrpc.Request, + timeout time.Duration, +) (*tikvrpc.Response, error) { + if t.errortestype == Error { + return nil, errors.New("store error") + } else if t.errortestype == testimeout { + return &tikvrpc.Response{Resp: &mpp.IsAliveResponse{}}, nil + } + + return &tikvrpc.Response{Resp: &mpp.IsAliveResponse{Available: true}}, nil +} + +type ProbeTest map[string]*mockDetectClient + +func (t ProbeTest) add(ctx context.Context) { + for k, v := range t { + GlobalMPPFailedStoreProber.Add(ctx, k, v) + } +} + +func (t ProbeTest) reSetErrortestype(to string) { + for k, v := range t { + if to == Normal { + v.errortestype = Normal + } else { + v.errortestype = k + } + } +} + +func (t ProbeTest) judge(ctx context.Context, test *testing.T, recoveryTTL time.Duration, need bool) { + for k := range t { + ok := GlobalMPPFailedStoreProber.IsRecovery(ctx, k, recoveryTTL) + require.Equal(test, need, ok) + } +} + +func failedStoreSizeJudge(ctx context.Context, test *testing.T, need int) { + var l int + GlobalMPPFailedStoreProber.scan(ctx) + time.Sleep(time.Second / 10) + GlobalMPPFailedStoreProber.failedMPPStores.Range(func(k, v interface{}) bool { + l++ + return true + }) + require.Equal(test, need, l) +} + +func testFlow(ctx context.Context, probetestest ProbeTest, test *testing.T, flow []string) { + probetestest.add(ctx) + for _, to := range flow { + probetestest.reSetErrortestype(to) + + GlobalMPPFailedStoreProber.scan(ctx) + time.Sleep(time.Second / 10) //wait detect goroutine finish + + var need bool + if to == Normal { + need = true + } + probetestest.judge(ctx, test, 0, need) + probetestest.judge(ctx, test, time.Minute, false) + } + + lastTo := flow[len(flow)-1] + cleanRecover := func(need int) { + GlobalMPPFailedStoreProber.maxRecoveryTimeLimit = 0 - time.Second + failedStoreSizeJudge(ctx, test, need) + GlobalMPPFailedStoreProber.maxRecoveryTimeLimit = MaxRecoveryTimeLimit + } + + cleanObsolet := func(need int) { + GlobalMPPFailedStoreProber.maxObsoletTimeLimit = 0 - time.Second + failedStoreSizeJudge(ctx, test, need) + GlobalMPPFailedStoreProber.maxObsoletTimeLimit = MaxObsoletTimeLimit + } + + if lastTo == Error { + cleanRecover(2) + cleanObsolet(0) + } else if lastTo == Normal { + cleanObsolet(2) + cleanRecover(0) + } +} + +func TestMPPFailedStoreProbe(t *testing.T) { + ctx := context.Background() + + notExistAddress := "not exist address" + + GlobalMPPFailedStoreProber.detectPeriod = 0 - time.Second + + // check not exist address + ok := GlobalMPPFailedStoreProber.IsRecovery(ctx, notExistAddress, 0) + require.True(t, ok) + + GlobalMPPFailedStoreProber.scan(ctx) + + probetestest := map[string]*mockDetectClient{ + testimeout: {errortestype: testimeout}, + Error: {errortestype: Error}, + } + + testFlowFinallyRecover := []string{Error, Normal, Error, Error, Normal} + testFlow(ctx, probetestest, t, testFlowFinallyRecover) + testFlowFinallyDesert := []string{Error, Normal, Normal, Error, Error} + testFlow(ctx, probetestest, t, testFlowFinallyDesert) +} + +func TestMPPFailedStoreProbeGoroutineTask(t *testing.T) { + // Confirm that multiple tasks are not allowed + GlobalMPPFailedStoreProber.lock.Lock() + GlobalMPPFailedStoreProber.Run() + GlobalMPPFailedStoreProber.lock.Unlock() + + GlobalMPPFailedStoreProber.Run() + GlobalMPPFailedStoreProber.Stop() +} + +func TestMPPFailedStoreAssertFailed(t *testing.T) { + ctx := context.Background() + + GlobalMPPFailedStoreProber.failedMPPStores.Store("errorinfo", nil) + GlobalMPPFailedStoreProber.scan(ctx) + + GlobalMPPFailedStoreProber.failedMPPStores.Store("errorinfo", nil) + GlobalMPPFailedStoreProber.IsRecovery(ctx, "errorinfo", 0) +} diff --git a/store/copr/store.go b/store/copr/store.go index 758109b81d805..32553961acc67 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -84,6 +84,7 @@ func NewStore(s *tikv.KVStore, coprCacheConfig *config.CoprocessorCache) (*Store if err != nil { return nil, errors.Trace(err) } + /* #nosec G404 */ return &Store{ kvStore: &kvStore{store: s}, diff --git a/table/column.go b/table/column.go index 0a80ddae4e8d4..c63febe75b0ba 100644 --- a/table/column.go +++ b/table/column.go @@ -323,6 +323,7 @@ func CastValue(ctx sessionctx.Context, val types.Datum, col *model.ColumnInfo, r } err = sc.HandleTruncate(err) + err = sc.HandleOverflow(err, err) if forceIgnoreTruncate { err = nil diff --git a/tests/realtikvtest/addindextest/add_index_test.go b/tests/realtikvtest/addindextest/add_index_test.go index 7dd4919570594..1c1403f66a922 100644 --- a/tests/realtikvtest/addindextest/add_index_test.go +++ b/tests/realtikvtest/addindextest/add_index_test.go @@ -100,3 +100,29 @@ func TestCreateMultiColsIndex(t *testing.T) { ctx := initTest(t) testTwoColsFrame(ctx, coliIDs, coljIDs, addIndexMultiCols) } + +func TestAddForeignKeyWithAutoCreateIndex(t *testing.T) { + store := realtikvtest.CreateMockStoreAndSetup(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("drop database if exists fk_index;") + tk.MustExec("create database fk_index;") + tk.MustExec("use fk_index;") + tk.MustExec(`set global tidb_ddl_enable_fast_reorg=1;`) + tk.MustExec("create table employee (id bigint auto_increment key, pid bigint)") + tk.MustExec("insert into employee (id) values (1),(2),(3),(4),(5),(6),(7),(8)") + for i := 0; i < 14; i++ { + tk.MustExec("insert into employee (pid) select pid from employee") + } + tk.MustExec("update employee set pid=id-1 where id>1") + tk.MustQuery("select count(*) from employee").Check(testkit.Rows("131072")) + tk.MustExec("alter table employee add foreign key fk_1(pid) references employee(id)") + tk.MustExec("alter table employee drop foreign key fk_1") + tk.MustExec("alter table employee drop index fk_1") + tk.MustExec("update employee set pid=0 where id=1") + tk.MustGetErrMsg("alter table employee add foreign key fk_1(pid) references employee(id)", + "[ddl:1452]Cannot add or update a child row: a foreign key constraint fails (`fk_index`.`employee`, CONSTRAINT `fk_1` FOREIGN KEY (`pid`) REFERENCES `employee` (`id`))") + tk.MustExec("update employee set pid=null where id=1") + tk.MustExec("insert into employee (pid) select pid from employee") + tk.MustExec("update employee set pid=id-1 where id>1 and pid is null") + tk.MustExec("alter table employee add foreign key fk_1(pid) references employee(id)") +} diff --git a/tests/realtikvtest/pessimistictest/BUILD.bazel b/tests/realtikvtest/pessimistictest/BUILD.bazel index 97890c8b8b70b..67a01e83cf386 100644 --- a/tests/realtikvtest/pessimistictest/BUILD.bazel +++ b/tests/realtikvtest/pessimistictest/BUILD.bazel @@ -18,6 +18,7 @@ go_test( "//parser/model", "//parser/mysql", "//parser/terror", + "//planner/core", "//session", "//sessionctx/variable", "//sessiontxn", diff --git a/tests/realtikvtest/pessimistictest/pessimistic_test.go b/tests/realtikvtest/pessimistictest/pessimistic_test.go index ae7545e0e91f6..a70b31f0a87b8 100644 --- a/tests/realtikvtest/pessimistictest/pessimistic_test.go +++ b/tests/realtikvtest/pessimistictest/pessimistic_test.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" @@ -2816,6 +2817,66 @@ func TestAsyncCommitCalTSFail(t *testing.T) { tk2.MustExec("commit") } +func TestAsyncCommitAndForeignKey(t *testing.T) { + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.TiKVClient.AsyncCommit.SafeWindow = time.Second + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + }) + store := realtikvtest.CreateMockStoreAndSetup(t) + tk := createAsyncCommitTestKit(t, store) + tk.MustExec("drop table if exists t_parent, t_child") + tk.MustExec("create table t_parent (id int primary key)") + tk.MustExec("create table t_child (id int primary key, pid int, foreign key (pid) references t_parent(id) on delete cascade on update cascade)") + tk.MustExec("insert into t_parent values (1),(2),(3),(4)") + tk.MustExec("insert into t_child values (1,1),(2,2),(3,3)") + tk.MustExec("set tidb_enable_1pc = true") + tk.MustExec("begin pessimistic") + tk.MustExec("delete from t_parent where id in (1,4)") + tk.MustExec("update t_parent set id=22 where id=2") + tk.MustExec("commit") + tk.MustQuery("select * from t_parent order by id").Check(testkit.Rows("3", "22")) + tk.MustQuery("select * from t_child order by id").Check(testkit.Rows("2 22", "3 3")) +} + +func TestTransactionIsolationAndForeignKey(t *testing.T) { + if !*realtikvtest.WithRealTiKV { + t.Skip("The test only support test with tikv.") + } + store := realtikvtest.CreateMockStoreAndSetup(t) + tk := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk2.MustExec("use test") + tk.MustExec("drop table if exists t1,t2") + tk.MustExec("create table t1 (id int primary key)") + tk.MustExec("create table t2 (id int primary key, pid int, foreign key (pid) references t1(id) on delete cascade on update cascade)") + tk.MustExec("insert into t1 values (1)") + tk.MustExec("set tx_isolation = 'READ-COMMITTED'") + tk.MustExec("begin pessimistic") + tk.MustExec("insert into t2 values (1,1)") + tk.MustGetDBError("insert into t2 values (2,2)", plannercore.ErrNoReferencedRow2) + tk2.MustExec("insert into t1 values (2)") + tk.MustQuery("select * from t1").Check(testkit.Rows("1", "2")) + tk.MustExec("insert into t2 values (2,2)") + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + tk2.MustExec("delete from t1 where id=2") + }() + time.Sleep(time.Millisecond * 10) + tk.MustExec("commit") + wg.Wait() + tk.MustQuery("select * from t1").Check(testkit.Rows("1")) + tk.MustQuery("select * from t2").Check(testkit.Rows("1 1")) + tk2.MustExec("delete from t1 where id=1") + tk.MustQuery("select * from t1").Check(testkit.Rows()) + tk.MustQuery("select * from t2").Check(testkit.Rows()) + tk.MustExec("admin check table t1") + tk.MustExec("admin check table t2") +} + func TestChangeLockToPut(t *testing.T) { store := realtikvtest.CreateMockStoreAndSetup(t) diff --git a/tests/realtikvtest/sessiontest/session_fail_test.go b/tests/realtikvtest/sessiontest/session_fail_test.go index 919932e6f9357..a3df51be821c0 100644 --- a/tests/realtikvtest/sessiontest/session_fail_test.go +++ b/tests/realtikvtest/sessiontest/session_fail_test.go @@ -200,7 +200,6 @@ func TestAutoCommitNeedNotLinearizability(t *testing.T) { func TestKill(t *testing.T) { store := realtikvtest.CreateMockStoreAndSetup(t) - tk := testkit.NewTestKit(t, store) tk.MustExec("kill connection_id();") } diff --git a/tests/realtikvtest/testkit.go b/tests/realtikvtest/testkit.go index b3ae5f3c6a2ac..4b8a749e65c9d 100644 --- a/tests/realtikvtest/testkit.go +++ b/tests/realtikvtest/testkit.go @@ -19,6 +19,7 @@ package realtikvtest import ( "flag" "fmt" + "strings" "sync/atomic" "testing" "time" @@ -110,8 +111,12 @@ func CreateMockStoreAndDomainAndSetup(t *testing.T, opts ...mockstore.MockTiKVSt tk.MustExec(fmt.Sprintf("set global innodb_lock_wait_timeout = %d", variable.DefInnodbLockWaitTimeout)) tk.MustExec("use test") rs := tk.MustQuery("show tables") + tables := []string{} for _, row := range rs.Rows() { - tk.MustExec(fmt.Sprintf("drop table %s", row[0])) + tables = append(tables, fmt.Sprintf("`%v`", row[0])) + } + if len(tables) > 0 { + tk.MustExec(fmt.Sprintf("drop table %s", strings.Join(tables, ","))) } } else { store, err = mockstore.NewMockStore(opts...) diff --git a/tidb-server/BUILD.bazel b/tidb-server/BUILD.bazel index 493eedfcccbfa..361a929351642 100644 --- a/tidb-server/BUILD.bazel +++ b/tidb-server/BUILD.bazel @@ -29,6 +29,7 @@ go_library( "//sessionctx/variable", "//statistics", "//store", + "//store/copr", "//store/driver", "//store/mockstore", "//store/mockstore/unistore/metrics", diff --git a/tidb-server/main.go b/tidb-server/main.go index d58e24a531f4c..97ad1917105ca 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -54,6 +54,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" kvstore "github.com/pingcap/tidb/store" + "github.com/pingcap/tidb/store/copr" "github.com/pingcap/tidb/store/driver" "github.com/pingcap/tidb/store/mockstore" uni_metrics "github.com/pingcap/tidb/store/mockstore/unistore/metrics" @@ -121,6 +122,7 @@ const ( nmInitializeSecure = "initialize-secure" nmInitializeInsecure = "initialize-insecure" + nmInitializeSQLFile = "initialize-sql-file" nmDisconnectOnExpiredPassword = "disconnect-on-expired-password" ) @@ -165,9 +167,10 @@ var ( proxyProtocolNetworks = flag.String(nmProxyProtocolNetworks, "", "proxy protocol networks allowed IP or *, empty mean disable proxy protocol support") proxyProtocolHeaderTimeout = flag.Uint(nmProxyProtocolHeaderTimeout, 5, "proxy protocol header read timeout, unit is second. (Deprecated: as proxy protocol using lazy mode, header read timeout no longer used)") - // Security + // Bootstrap and security initializeSecure = flagBoolean(nmInitializeSecure, false, "bootstrap tidb-server in secure mode") initializeInsecure = flagBoolean(nmInitializeInsecure, true, "bootstrap tidb-server in insecure mode") + initializeSQLFile = flag.String(nmInitializeSQLFile, "", "SQL file to execute on first bootstrap") disconnectOnExpiredPassword = flagBoolean(nmDisconnectOnExpiredPassword, true, "the server disconnects the client when the password is expired") ) @@ -309,6 +312,7 @@ func createStoreAndDomain() (kv.Storage, *domain.Domain) { var err error storage, err := kvstore.New(fullPath) terror.MustNil(err) + copr.GlobalMPPFailedStoreProber.Run() err = infosync.CheckTiKVVersion(storage, *semver.New(versioninfo.TiKVMinVersion)) terror.MustNil(err) // Bootstrap a session to load information schema. @@ -529,7 +533,7 @@ func overrideConfig(cfg *config.Config) { // Sanity check: can't specify both options if actualFlags[nmInitializeSecure] && actualFlags[nmInitializeInsecure] { - err = fmt.Errorf("the options --initialize-insecure and --initialize-secure are mutually exclusive") + err = fmt.Errorf("the options -initialize-insecure and -initialize-secure are mutually exclusive") terror.MustNil(err) } // The option --initialize-secure=true ensures that a secure bootstrap is used. @@ -548,9 +552,19 @@ func overrideConfig(cfg *config.Config) { // which is not supported on windows. Only the insecure bootstrap // method is supported. if runtime.GOOS == "windows" && cfg.Security.SecureBootstrap { - err = fmt.Errorf("the option --initialize-secure is not supported on Windows") + err = fmt.Errorf("the option -initialize-secure is not supported on Windows") terror.MustNil(err) } + // Initialize SQL File is used to run a set of SQL statements after first bootstrap. + // It is important in the use case that you want to set GLOBAL variables, which + // are persisted to the cluster and not read from a config file. + if actualFlags[nmInitializeSQLFile] { + if _, err := os.Stat(*initializeSQLFile); err != nil { + err = fmt.Errorf("can not access -initialize-sql-file %s", *initializeSQLFile) + terror.MustNil(err) + } + cfg.InitializeSQLFile = *initializeSQLFile + } } func setVersions() { @@ -786,16 +800,7 @@ func setupMetrics() { systimeErrHandler := func() { metrics.TimeJumpBackCounter.Inc() } - callBackCount := 0 - successCallBack := func() { - callBackCount++ - // It is callback by monitor per second, we increase metrics.KeepAliveCounter per 5s. - if callBackCount >= 5 { - callBackCount = 0 - metrics.KeepAliveCounter.Inc() - } - } - go systimemon.StartMonitor(time.Now, systimeErrHandler, successCallBack) + go systimemon.StartMonitor(time.Now, systimeErrHandler) pushMetric(cfg.Status.MetricsAddr, time.Duration(cfg.Status.MetricsInterval)*time.Second) } @@ -814,6 +819,7 @@ func setupTracing() { func closeDomainAndStorage(storage kv.Storage, dom *domain.Domain) { tikv.StoreShuttingDown(1) dom.Close() + copr.GlobalMPPFailedStoreProber.Stop() err := storage.Close() terror.Log(errors.Trace(err)) } diff --git a/ttl/cache/ttlstatus.go b/ttl/cache/ttlstatus.go index 1657105e6c3e7..f14c1a559ae95 100644 --- a/ttl/cache/ttlstatus.go +++ b/ttl/cache/ttlstatus.go @@ -16,7 +16,6 @@ package cache import ( "context" - "fmt" "time" "github.com/pingcap/tidb/sessionctx" @@ -43,8 +42,8 @@ const ( const selectFromTTLTableStatus = "SELECT LOW_PRIORITY table_id,parent_table_id,table_statistics,last_job_id,last_job_start_time,last_job_finish_time,last_job_ttl_expire,last_job_summary,current_job_id,current_job_owner_id,current_job_owner_addr,current_job_owner_hb_time,current_job_start_time,current_job_ttl_expire,current_job_state,current_job_status,current_job_status_update_time FROM mysql.tidb_ttl_table_status" // SelectFromTTLTableStatusWithID returns an SQL statement to get the table status from table id -func SelectFromTTLTableStatusWithID(tableID int64) string { - return selectFromTTLTableStatus + fmt.Sprintf(" WHERE table_id = %d", tableID) +func SelectFromTTLTableStatusWithID(tableID int64) (string, []interface{}) { + return selectFromTTLTableStatus + " WHERE table_id = %?", []interface{}{tableID} } // TableStatus contains the corresponding information in the system table `mysql.tidb_ttl_table_status` diff --git a/ttl/ttlworker/BUILD.bazel b/ttl/ttlworker/BUILD.bazel index 8d031e22e6e4a..527c782ed691e 100644 --- a/ttl/ttlworker/BUILD.bazel +++ b/ttl/ttlworker/BUILD.bazel @@ -51,6 +51,7 @@ go_test( flaky = True, deps = [ "//infoschema", + "//kv", "//parser/ast", "//parser/model", "//parser/mysql", diff --git a/ttl/ttlworker/job.go b/ttl/ttlworker/job.go index 1a2f351465027..8628e9a6c9746 100644 --- a/ttl/ttlworker/job.go +++ b/ttl/ttlworker/job.go @@ -17,7 +17,6 @@ package ttlworker import ( "context" "encoding/json" - "fmt" "sync" "time" @@ -29,20 +28,34 @@ import ( "go.uber.org/zap" ) -const updateJobCurrentStatusTemplate = "UPDATE mysql.tidb_ttl_table_status SET current_job_status = '%s' WHERE table_id = %d AND current_job_status = '%s' AND current_job_id = '%s'" -const finishJobTemplate = "UPDATE mysql.tidb_ttl_table_status SET last_job_id = current_job_id, last_job_start_time = current_job_start_time, last_job_finish_time = '%s', last_job_ttl_expire = current_job_ttl_expire, last_job_summary = '%s', current_job_id = NULL, current_job_owner_id = NULL, current_job_owner_hb_time = NULL, current_job_start_time = NULL, current_job_ttl_expire = NULL, current_job_state = NULL, current_job_status = NULL, current_job_status_update_time = NULL WHERE table_id = %d AND current_job_id = '%s'" -const updateJobStateTemplate = "UPDATE mysql.tidb_ttl_table_status SET current_job_state = '%s' WHERE table_id = %d AND current_job_id = '%s' AND current_job_owner_id = '%s'" +const updateJobCurrentStatusTemplate = "UPDATE mysql.tidb_ttl_table_status SET current_job_status = %? WHERE table_id = %? AND current_job_status = %? AND current_job_id = %?" +const finishJobTemplate = `UPDATE mysql.tidb_ttl_table_status + SET last_job_id = current_job_id, + last_job_start_time = current_job_start_time, + last_job_finish_time = %?, + last_job_ttl_expire = current_job_ttl_expire, + last_job_summary = %?, + current_job_id = NULL, + current_job_owner_id = NULL, + current_job_owner_hb_time = NULL, + current_job_start_time = NULL, + current_job_ttl_expire = NULL, + current_job_state = NULL, + current_job_status = NULL, + current_job_status_update_time = NULL + WHERE table_id = %? AND current_job_id = %?` +const updateJobStateTemplate = "UPDATE mysql.tidb_ttl_table_status SET current_job_state = %? WHERE table_id = %? AND current_job_id = %? AND current_job_owner_id = %?" -func updateJobCurrentStatusSQL(tableID int64, oldStatus cache.JobStatus, newStatus cache.JobStatus, jobID string) string { - return fmt.Sprintf(updateJobCurrentStatusTemplate, newStatus, tableID, oldStatus, jobID) +func updateJobCurrentStatusSQL(tableID int64, oldStatus cache.JobStatus, newStatus cache.JobStatus, jobID string) (string, []interface{}) { + return updateJobCurrentStatusTemplate, []interface{}{newStatus, tableID, oldStatus, jobID} } -func finishJobSQL(tableID int64, finishTime time.Time, summary string, jobID string) string { - return fmt.Sprintf(finishJobTemplate, finishTime.Format(timeFormat), summary, tableID, jobID) +func finishJobSQL(tableID int64, finishTime time.Time, summary string, jobID string) (string, []interface{}) { + return finishJobTemplate, []interface{}{finishTime.Format(timeFormat), summary, tableID, jobID} } -func updateJobState(tableID int64, currentJobID string, currentJobState string, currentJobOwnerID string) string { - return fmt.Sprintf(updateJobStateTemplate, currentJobState, tableID, currentJobID, currentJobOwnerID) +func updateJobState(tableID int64, currentJobID string, currentJobState string, currentJobOwnerID string) (string, []interface{}) { + return updateJobStateTemplate, []interface{}{currentJobState, tableID, currentJobID, currentJobOwnerID} } type ttlJob struct { @@ -76,9 +89,10 @@ func (job *ttlJob) changeStatus(ctx context.Context, se session.Session, status job.status = status job.statusMutex.Unlock() - _, err := se.ExecuteSQL(ctx, updateJobCurrentStatusSQL(job.tbl.ID, oldStatus, status, job.id)) + sql, args := updateJobCurrentStatusSQL(job.tbl.ID, oldStatus, status, job.id) + _, err := se.ExecuteSQL(ctx, sql, args...) if err != nil { - return errors.Trace(err) + return errors.Wrapf(err, "execute sql: %s", sql) } return nil @@ -89,9 +103,10 @@ func (job *ttlJob) updateState(ctx context.Context, se session.Session) error { if err != nil { logutil.Logger(job.ctx).Warn("fail to generate summary for ttl job", zap.Error(err)) } - _, err = se.ExecuteSQL(ctx, updateJobState(job.tbl.ID, job.id, summary, job.ownerID)) + sql, args := updateJobState(job.tbl.ID, job.id, summary, job.ownerID) + _, err = se.ExecuteSQL(ctx, sql, args...) if err != nil { - return errors.Trace(err) + return errors.Wrapf(err, "execute sql: %s", sql) } return nil @@ -115,9 +130,10 @@ func (job *ttlJob) finish(se session.Session, now time.Time) { } // at this time, the job.ctx may have been canceled (to cancel this job) // even when it's canceled, we'll need to update the states, so use another context - _, err = se.ExecuteSQL(context.TODO(), finishJobSQL(job.tbl.ID, now, summary, job.id)) + sql, args := finishJobSQL(job.tbl.ID, now, summary, job.id) + _, err = se.ExecuteSQL(context.TODO(), sql, args...) if err != nil { - logutil.Logger(job.ctx).Error("fail to finish a ttl job", zap.Error(err), zap.Int64("tableID", job.tbl.ID), zap.String("jobID", job.id)) + logutil.Logger(job.ctx).Error("fail to finish a ttl job", zap.Error(err), zap.Int64("tableID", job.tbl.ID), zap.String("jobID", job.id), zap.String("sql", sql), zap.Any("arguments", args)) } } diff --git a/ttl/ttlworker/job_manager.go b/ttl/ttlworker/job_manager.go index 58c47d05efe4f..25af41e46ca58 100644 --- a/ttl/ttlworker/job_manager.go +++ b/ttl/ttlworker/job_manager.go @@ -16,7 +16,6 @@ package ttlworker import ( "context" - "fmt" "time" "github.com/pingcap/errors" @@ -31,22 +30,30 @@ import ( "go.uber.org/zap" ) -const insertNewTableIntoStatusTemplate = "INSERT INTO mysql.tidb_ttl_table_status (table_id,parent_table_id) VALUES (%d, %d)" -const setTableStatusOwnerTemplate = "UPDATE mysql.tidb_ttl_table_status SET current_job_id = UUID(), current_job_owner_id = '%s',current_job_start_time = '%s',current_job_status = 'waiting',current_job_status_update_time = '%s',current_job_ttl_expire = '%s',current_job_owner_hb_time = '%s' WHERE table_id = %d" -const updateHeartBeatTemplate = "UPDATE mysql.tidb_ttl_table_status SET current_job_owner_hb_time = '%s' WHERE table_id = %d AND current_job_owner_id = '%s'" +const insertNewTableIntoStatusTemplate = "INSERT INTO mysql.tidb_ttl_table_status (table_id,parent_table_id) VALUES (%?, %?)" +const setTableStatusOwnerTemplate = `UPDATE mysql.tidb_ttl_table_status + SET current_job_id = UUID(), + current_job_owner_id = %?, + current_job_start_time = %?, + current_job_status = 'waiting', + current_job_status_update_time = %?, + current_job_ttl_expire = %?, + current_job_owner_hb_time = %? + WHERE table_id = %?` +const updateHeartBeatTemplate = "UPDATE mysql.tidb_ttl_table_status SET current_job_owner_hb_time = %? WHERE table_id = %? AND current_job_owner_id = %?" const timeFormat = "2006-01-02 15:04:05" -func insertNewTableIntoStatusSQL(tableID int64, parentTableID int64) string { - return fmt.Sprintf(insertNewTableIntoStatusTemplate, tableID, parentTableID) +func insertNewTableIntoStatusSQL(tableID int64, parentTableID int64) (string, []interface{}) { + return insertNewTableIntoStatusTemplate, []interface{}{tableID, parentTableID} } -func setTableStatusOwnerSQL(tableID int64, now time.Time, currentJobTTLExpire time.Time, id string) string { - return fmt.Sprintf(setTableStatusOwnerTemplate, id, now.Format(timeFormat), now.Format(timeFormat), currentJobTTLExpire.Format(timeFormat), now.Format(timeFormat), tableID) +func setTableStatusOwnerSQL(tableID int64, now time.Time, currentJobTTLExpire time.Time, id string) (string, []interface{}) { + return setTableStatusOwnerTemplate, []interface{}{id, now.Format(timeFormat), now.Format(timeFormat), currentJobTTLExpire.Format(timeFormat), now.Format(timeFormat), tableID} } -func updateHeartBeatSQL(tableID int64, now time.Time, id string) string { - return fmt.Sprintf(updateHeartBeatTemplate, now.Format(timeFormat), tableID, id) +func updateHeartBeatSQL(tableID int64, now time.Time, id string) (string, []interface{}) { + return updateHeartBeatTemplate, []interface{}{now.Format(timeFormat), tableID, id} } // JobManager schedules and manages the ttl jobs on this instance @@ -503,19 +510,22 @@ func (m *JobManager) lockNewJob(ctx context.Context, se session.Session, table * var expireTime time.Time err := se.RunInTxn(ctx, func() error { - rows, err := se.ExecuteSQL(ctx, cache.SelectFromTTLTableStatusWithID(table.ID)) + sql, args := cache.SelectFromTTLTableStatusWithID(table.ID) + rows, err := se.ExecuteSQL(ctx, sql, args...) if err != nil { - return err + return errors.Wrapf(err, "execute sql: %s", sql) } if len(rows) == 0 { // cannot find the row, insert the status row - _, err = se.ExecuteSQL(ctx, insertNewTableIntoStatusSQL(table.ID, table.TableInfo.ID)) + sql, args := insertNewTableIntoStatusSQL(table.ID, table.TableInfo.ID) + _, err = se.ExecuteSQL(ctx, sql, args...) if err != nil { - return err + return errors.Wrapf(err, "execute sql: %s", sql) } - rows, err = se.ExecuteSQL(ctx, cache.SelectFromTTLTableStatusWithID(table.ID)) + sql, args = cache.SelectFromTTLTableStatusWithID(table.ID) + rows, err = se.ExecuteSQL(ctx, sql, args...) if err != nil { - return err + return errors.Wrapf(err, "execute sql: %s", sql) } if len(rows) == 0 { return errors.New("table status row still doesn't exist after insertion") @@ -534,9 +544,9 @@ func (m *JobManager) lockNewJob(ctx context.Context, se session.Session, table * return err } - _, err = se.ExecuteSQL(ctx, setTableStatusOwnerSQL(table.ID, now, expireTime, m.id)) - - return err + sql, args = setTableStatusOwnerSQL(table.ID, now, expireTime, m.id) + _, err = se.ExecuteSQL(ctx, sql, args...) + return errors.Wrapf(err, "execute sql: %s", sql) }) if err != nil { return nil, err @@ -599,9 +609,10 @@ func (m *JobManager) createNewJob(expireTime time.Time, now time.Time, table *ca func (m *JobManager) updateHeartBeat(ctx context.Context, se session.Session) error { now := se.Now() for _, job := range m.localJobs() { - _, err := se.ExecuteSQL(ctx, updateHeartBeatSQL(job.tbl.ID, now, m.id)) + sql, args := updateHeartBeatSQL(job.tbl.ID, now, m.id) + _, err := se.ExecuteSQL(ctx, sql, args...) if err != nil { - return errors.Trace(err) + return errors.Wrapf(err, "execute sql: %s", sql) } // also updates some internal state for this job err = job.updateState(ctx, se) diff --git a/ttl/ttlworker/job_manager_integration_test.go b/ttl/ttlworker/job_manager_integration_test.go index 8c299afcd48de..8d33a736d2b9d 100644 --- a/ttl/ttlworker/job_manager_integration_test.go +++ b/ttl/ttlworker/job_manager_integration_test.go @@ -21,6 +21,8 @@ import ( "testing" "time" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" dbsession "github.com/pingcap/tidb/session" @@ -35,10 +37,8 @@ import ( "go.uber.org/zap" ) -func TestParallelLockNewJob(t *testing.T) { - store := testkit.CreateMockStore(t) - - sessionFactory := func() session.Session { +func sessionFactory(t *testing.T, store kv.Storage) func() session.Session { + return func() session.Session { dbSession, err := dbsession.CreateSession4Test(store) require.NoError(t, err) se := session.NewSession(dbSession, dbSession, nil) @@ -50,6 +50,12 @@ func TestParallelLockNewJob(t *testing.T) { return se } +} + +func TestParallelLockNewJob(t *testing.T) { + store := testkit.CreateMockStore(t) + + sessionFactory := sessionFactory(t, store) storedTTLJobRunInterval := variable.TTLJobRunInterval.Load() variable.TTLJobRunInterval.Store(0) @@ -96,3 +102,24 @@ func TestParallelLockNewJob(t *testing.T) { successJob.Finish(se, time.Now()) } } + +func TestFinishJob(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + sessionFactory := sessionFactory(t, store) + + testTable := &cache.PhysicalTable{ID: 2, TableInfo: &model.TableInfo{ID: 1, TTLInfo: &model.TTLInfo{IntervalExprStr: "1", IntervalTimeUnit: int(ast.TimeUnitDay)}}} + + tk.MustExec("insert into mysql.tidb_ttl_table_status(table_id) values (2)") + + // finish with error + m := ttlworker.NewJobManager("test-id", nil, store) + se := sessionFactory() + job, err := m.LockNewJob(context.Background(), se, testTable, time.Now()) + require.NoError(t, err) + job.SetScanErr(errors.New(`"'an error message contains both single and double quote'"`)) + job.Finish(se, time.Now()) + + tk.MustQuery("select table_id, last_job_summary from mysql.tidb_ttl_table_status").Check(testkit.Rows("2 {\"total_rows\":0,\"success_rows\":0,\"error_rows\":0,\"total_scan_task\":1,\"scheduled_scan_task\":0,\"finished_scan_task\":0,\"scan_task_err\":\"\\\"'an error message contains both single and double quote'\\\"\"}")) +} diff --git a/ttl/ttlworker/job_manager_test.go b/ttl/ttlworker/job_manager_test.go index 6718c384543fe..1f649abb065af 100644 --- a/ttl/ttlworker/job_manager_test.go +++ b/ttl/ttlworker/job_manager_test.go @@ -156,6 +156,10 @@ func (j *ttlJob) ID() string { return j.id } +func (j *ttlJob) SetScanErr(err error) { + j.scanTaskErr = err +} + func newMockTTLJob(tbl *cache.PhysicalTable, status cache.JobStatus) *ttlJob { statistics := &ttlStatistics{} return &ttlJob{tbl: tbl, ctx: context.Background(), statistics: statistics, status: status, tasks: []*ttlScanTask{{ctx: context.Background(), tbl: tbl, statistics: statistics}}} @@ -216,8 +220,18 @@ func TestLockNewTable(t *testing.T) { testPhysicalTable := &cache.PhysicalTable{ID: 1, TableInfo: &model.TableInfo{ID: 1, TTLInfo: &model.TTLInfo{ColumnName: model.NewCIStr("test"), IntervalExprStr: "5 Year"}}} + type executeInfo struct { + sql string + args []interface{} + } + getExecuteInfo := func(sql string, args []interface{}) executeInfo { + return executeInfo{ + sql, + args, + } + } type sqlExecute struct { - sql string + executeInfo rows []chunk.Row err error @@ -231,47 +245,47 @@ func TestLockNewTable(t *testing.T) { }{ {"normal lock table", testPhysicalTable, []sqlExecute{ { - cache.SelectFromTTLTableStatusWithID(1), + getExecuteInfo(cache.SelectFromTTLTableStatusWithID(1)), newTTLTableStatusRows(&cache.TableStatus{TableID: 1}), nil, }, { - setTableStatusOwnerSQL(1, now, expireTime, "test-id"), + getExecuteInfo(setTableStatusOwnerSQL(1, now, expireTime, "test-id")), nil, nil, }, { - updateStatusSQL, + getExecuteInfo(updateStatusSQL, nil), newTTLTableStatusRows(&cache.TableStatus{TableID: 1}), nil, }, }, true, false}, {"select nothing", testPhysicalTable, []sqlExecute{ { - cache.SelectFromTTLTableStatusWithID(1), + getExecuteInfo(cache.SelectFromTTLTableStatusWithID(1)), nil, nil, }, { - insertNewTableIntoStatusSQL(1, 1), + getExecuteInfo(insertNewTableIntoStatusSQL(1, 1)), nil, nil, }, { - cache.SelectFromTTLTableStatusWithID(1), + getExecuteInfo(cache.SelectFromTTLTableStatusWithID(1)), newTTLTableStatusRows(&cache.TableStatus{TableID: 1}), nil, }, { - setTableStatusOwnerSQL(1, now, expireTime, "test-id"), + getExecuteInfo(setTableStatusOwnerSQL(1, now, expireTime, "test-id")), nil, nil, }, { - updateStatusSQL, + getExecuteInfo(updateStatusSQL, nil), newTTLTableStatusRows(&cache.TableStatus{TableID: 1}), nil, }, }, true, false}, {"return error", testPhysicalTable, []sqlExecute{ { - cache.SelectFromTTLTableStatusWithID(1), + getExecuteInfo(cache.SelectFromTTLTableStatusWithID(1)), newTTLTableStatusRows(&cache.TableStatus{TableID: 1}), nil, }, { - setTableStatusOwnerSQL(1, now, expireTime, "test-id"), + getExecuteInfo(setTableStatusOwnerSQL(1, now, expireTime, "test-id")), nil, errors.New("test error message"), }, }, false, true}, @@ -287,6 +301,7 @@ func TestLockNewTable(t *testing.T) { se.executeSQL = func(ctx context.Context, sql string, args ...interface{}) (rows []chunk.Row, err error) { assert.Less(t, sqlCounter, len(c.sqls)) assert.Equal(t, sql, c.sqls[sqlCounter].sql) + assert.Equal(t, args, c.sqls[sqlCounter].args) rows = c.sqls[sqlCounter].rows err = c.sqls[sqlCounter].err @@ -509,11 +524,26 @@ func TestCheckFinishedJob(t *testing.T) { m.runningJobs[0].taskIter = 1 m.runningJobs[0].finishedScanTaskCounter = 1 - m.checkFinishedJob(se, se.Now()) + // meetArg records whether the sql statement uses the arg + meetArg := false + now := se.Now() + jobID := m.runningJobs[0].id + se.executeSQL = func(ctx context.Context, sql string, args ...interface{}) ([]chunk.Row, error) { + if len(args) > 0 { + meetArg = true + expectedSQL, expectedArgs := finishJobSQL(tbl.ID, now, "{\"total_rows\":1,\"success_rows\":1,\"error_rows\":0,\"total_scan_task\":1,\"scheduled_scan_task\":1,\"finished_scan_task\":1}", jobID) + assert.Equal(t, expectedSQL, sql) + assert.Equal(t, expectedArgs, args) + } + return nil, nil + } + m.checkFinishedJob(se, now) assert.Len(t, m.runningJobs, 0) + assert.Equal(t, true, meetArg) + se.executeSQL = nil // check timeout job - now := se.Now() + now = se.Now() createTime := now.Add(-20 * time.Hour) m = NewJobManager("test-id", newMockSessionPool(t, tbl), nil) m.runningJobs = []*ttlJob{ diff --git a/ttl/ttlworker/session_test.go b/ttl/ttlworker/session_test.go index 877fd7996eaa7..712c696fa5f35 100644 --- a/ttl/ttlworker/session_test.go +++ b/ttl/ttlworker/session_test.go @@ -179,7 +179,7 @@ func (s *mockSession) ExecuteSQL(ctx context.Context, sql string, args ...interf } if s.executeSQL != nil { - return s.executeSQL(ctx, sql, args) + return s.executeSQL(ctx, sql, args...) } return s.rows, s.execErr } diff --git a/util/cpu/BUILD.bazel b/util/cpu/BUILD.bazel index 08893520caaa0..b831ba544947b 100644 --- a/util/cpu/BUILD.bazel +++ b/util/cpu/BUILD.bazel @@ -21,5 +21,6 @@ go_test( srcs = ["cpu_test.go"], embed = [":cpu"], flaky = True, + race = "on", deps = ["@com_github_stretchr_testify//require"], ) diff --git a/util/cpu/cpu_test.go b/util/cpu/cpu_test.go index cd330a11e5196..a191227b72f78 100644 --- a/util/cpu/cpu_test.go +++ b/util/cpu/cpu_test.go @@ -24,8 +24,7 @@ import ( ) func TestCPUValue(t *testing.T) { - Observer := NewCPUObserver() - Observer.Start() + observer := NewCPUObserver() exit := make(chan struct{}) var wg sync.WaitGroup for i := 0; i < 10; i++ { @@ -42,11 +41,11 @@ func TestCPUValue(t *testing.T) { } }() } - Observer.Start() + observer.Start() time.Sleep(5 * time.Second) require.GreaterOrEqual(t, GetCPUUsage(), 0.0) require.Less(t, GetCPUUsage(), 1.0) - Observer.Stop() + observer.Stop() close(exit) wg.Wait() } diff --git a/util/dbterror/ddl_terror.go b/util/dbterror/ddl_terror.go index 6c80d60ad6ec2..e2042513ba0f1 100644 --- a/util/dbterror/ddl_terror.go +++ b/util/dbterror/ddl_terror.go @@ -369,6 +369,8 @@ var ( ErrDependentByFunctionalIndex = ClassDDL.NewStd(mysql.ErrDependentByFunctionalIndex) // ErrFunctionalIndexOnBlob when the expression of expression index returns blob or text. ErrFunctionalIndexOnBlob = ClassDDL.NewStd(mysql.ErrFunctionalIndexOnBlob) + // ErrDependentByPartitionFunctional returns when the dropped column depends by expression partition. + ErrDependentByPartitionFunctional = ClassDDL.NewStd(mysql.ErrDependentByPartitionFunctional) // ErrUnsupportedAlterTableSpec means we don't support this alter table specification (i.e. unknown) ErrUnsupportedAlterTableSpec = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "Unsupported/unknown ALTER TABLE specification"), nil)) @@ -429,4 +431,7 @@ var ( ErrTempTableNotAllowedWithTTL = ClassDDL.NewStd(mysql.ErrTempTableNotAllowedWithTTL) // ErrUnsupportedTTLReferencedByFK returns when the TTL config is set for a table referenced by foreign key ErrUnsupportedTTLReferencedByFK = ClassDDL.NewStd(mysql.ErrUnsupportedTTLReferencedByFK) + + // ErrNotSupportedYet returns when tidb does not support this feature. + ErrNotSupportedYet = ClassDDL.NewStd(mysql.ErrNotSupportedYet) ) diff --git a/util/replayer/replayer.go b/util/replayer/replayer.go index 9fb0bc4f629fa..f89d26ec97717 100644 --- a/util/replayer/replayer.go +++ b/util/replayer/replayer.go @@ -33,13 +33,13 @@ type PlanReplayerTaskKey struct { } // GeneratePlanReplayerFile generates plan replayer file -func GeneratePlanReplayerFile() (*os.File, string, error) { +func GeneratePlanReplayerFile(isContinues bool) (*os.File, string, error) { path := GetPlanReplayerDirName() err := os.MkdirAll(path, os.ModePerm) if err != nil { return nil, "", errors.AddStack(err) } - fileName, err := generatePlanReplayerFileName() + fileName, err := generatePlanReplayerFileName(isContinues) if err != nil { return nil, "", errors.AddStack(err) } @@ -50,7 +50,7 @@ func GeneratePlanReplayerFile() (*os.File, string, error) { return zf, fileName, err } -func generatePlanReplayerFileName() (string, error) { +func generatePlanReplayerFileName(isContinues bool) (string, error) { // Generate key and create zip file time := time.Now().UnixNano() b := make([]byte, 16) @@ -60,6 +60,9 @@ func generatePlanReplayerFileName() (string, error) { return "", err } key := base64.URLEncoding.EncodeToString(b) + if isContinues { + return fmt.Sprintf("continues_replayer_%v_%v.zip", key, time), nil + } return fmt.Sprintf("replayer_%v_%v.zip", key, time), nil } diff --git a/util/sem/sem.go b/util/sem/sem.go index 161334c880a4a..e2364893b05b2 100644 --- a/util/sem/sem.go +++ b/util/sem/sem.go @@ -136,7 +136,6 @@ func IsInvisibleStatusVar(varName string) bool { func IsInvisibleSysVar(varNameInLower string) bool { switch varNameInLower { case variable.TiDBDDLSlowOprThreshold, // ddl_slow_threshold - variable.TiDBAllowRemoveAutoInc, variable.TiDBCheckMb4ValueInUTF8, variable.TiDBConfig, variable.TiDBEnableSlowLog, diff --git a/util/sem/sem_test.go b/util/sem/sem_test.go index cb0c47ad9225d..d19c2af27adcb 100644 --- a/util/sem/sem_test.go +++ b/util/sem/sem_test.go @@ -81,8 +81,8 @@ func TestIsInvisibleSysVar(t *testing.T) { assert.False(IsInvisibleSysVar(variable.Hostname)) // changes the value to default, but is not invisible assert.False(IsInvisibleSysVar(variable.TiDBEnableEnhancedSecurity)) // should be able to see the mode is on. + assert.False(IsInvisibleSysVar(variable.TiDBAllowRemoveAutoInc)) - assert.True(IsInvisibleSysVar(variable.TiDBAllowRemoveAutoInc)) assert.True(IsInvisibleSysVar(variable.TiDBCheckMb4ValueInUTF8)) assert.True(IsInvisibleSysVar(variable.TiDBConfig)) assert.True(IsInvisibleSysVar(variable.TiDBEnableSlowLog)) diff --git a/util/systimemon/systime_mon.go b/util/systimemon/systime_mon.go index 67b23ed894b5b..26819a4159776 100644 --- a/util/systimemon/systime_mon.go +++ b/util/systimemon/systime_mon.go @@ -22,11 +22,10 @@ import ( ) // StartMonitor calls systimeErrHandler if system time jump backward. -func StartMonitor(now func() time.Time, systimeErrHandler func(), successCallback func()) { +func StartMonitor(now func() time.Time, systimeErrHandler func()) { logutil.BgLogger().Info("start system time monitor") tick := time.NewTicker(100 * time.Millisecond) defer tick.Stop() - tickCount := 0 for { last := now().UnixNano() <-tick.C @@ -34,11 +33,5 @@ func StartMonitor(now func() time.Time, systimeErrHandler func(), successCallbac logutil.BgLogger().Error("system time jump backward", zap.Int64("last", last)) systimeErrHandler() } - // call successCallback per second. - tickCount++ - if tickCount >= 10 { - tickCount = 0 - successCallback() - } } } diff --git a/util/systimemon/systime_mon_test.go b/util/systimemon/systime_mon_test.go index f05098b63db7e..6964a9ac3d70d 100644 --- a/util/systimemon/systime_mon_test.go +++ b/util/systimemon/systime_mon_test.go @@ -35,7 +35,7 @@ func TestSystimeMonitor(t *testing.T) { return time.Now().Add(-2 * time.Second) }, func() { errTriggered.Store(true) - }, func() {}) + }) require.Eventually(t, errTriggered.Load, time.Second, 10*time.Millisecond) }