From 4bbd56b6214226f5114e1c73d75f2851d63936c3 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 22 Nov 2023 15:32:41 +0800 Subject: [PATCH 01/36] *: support alter table row_format for the compatibility (#48769) close pingcap/tidb#48754 --- pkg/ddl/ddl_api.go | 1 + tests/integrationtest/r/util/admin.result | 2 +- tests/integrationtest/t/util/admin.test | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/ddl/ddl_api.go b/pkg/ddl/ddl_api.go index 2995daf1bf812..0c7738b6b0a21 100644 --- a/pkg/ddl/ddl_api.go +++ b/pkg/ddl/ddl_api.go @@ -3794,6 +3794,7 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast Name: model.NewCIStr(opt.StrValue), } case ast.TableOptionEngine: + case ast.TableOptionRowFormat: case ast.TableOptionTTL, ast.TableOptionTTLEnable, ast.TableOptionTTLJobInterval: var ttlInfo *model.TTLInfo var ttlEnable *bool diff --git a/tests/integrationtest/r/util/admin.result b/tests/integrationtest/r/util/admin.result index 144578d8d295c..6903d3c1eb77e 100644 --- a/tests/integrationtest/r/util/admin.result +++ b/tests/integrationtest/r/util/admin.result @@ -9,7 +9,7 @@ ALTER TABLE t1 ADD COLUMN cc1 CHAR(36) NULL DEFAULT ''; ALTER TABLE t1 ADD COLUMN cc2 VARCHAR(36) NULL DEFAULT ''; ALTER TABLE t1 ADD INDEX idx1 (cc1); ALTER TABLE t1 ADD INDEX idx2 (cc2); -ALTER TABLE t1 engine=innodb; +ALTER TABLE t1 engine=innodb, ROW_FORMAT=DYNAMIC; admin check table t1; drop table if exists t1; create table t1 ( diff --git a/tests/integrationtest/t/util/admin.test b/tests/integrationtest/t/util/admin.test index 19afa81ef5131..984e65a8abb5c 100644 --- a/tests/integrationtest/t/util/admin.test +++ b/tests/integrationtest/t/util/admin.test @@ -10,7 +10,7 @@ ALTER TABLE t1 ADD COLUMN cc1 CHAR(36) NULL DEFAULT ''; ALTER TABLE t1 ADD COLUMN cc2 VARCHAR(36) NULL DEFAULT ''; ALTER TABLE t1 ADD INDEX idx1 (cc1); ALTER TABLE t1 ADD INDEX idx2 (cc2); -ALTER TABLE t1 engine=innodb; +ALTER TABLE t1 engine=innodb, ROW_FORMAT=DYNAMIC; admin check table t1; drop table if exists t1; create table t1 ( From c4677188c43e0a057d4ea28a7615ebbd9f5fe745 Mon Sep 17 00:00:00 2001 From: Luo Yangzhixin Date: Wed, 22 Nov 2023 16:23:41 +0800 Subject: [PATCH 02/36] lightning: add complex unit tests for lightning post-import conflict detection "replace" mode (#48514) ref pingcap/tidb#45774 --- br/pkg/lightning/errormanager/BUILD.bazel | 13 +- .../errormanager/resolveconflict_test.go | 794 ++++++++++++++++++ 2 files changed, 805 insertions(+), 2 deletions(-) create mode 100644 br/pkg/lightning/errormanager/resolveconflict_test.go diff --git a/br/pkg/lightning/errormanager/BUILD.bazel b/br/pkg/lightning/errormanager/BUILD.bazel index 05d8cb8f6e1f6..291befefb1a8b 100644 --- a/br/pkg/lightning/errormanager/BUILD.bazel +++ b/br/pkg/lightning/errormanager/BUILD.bazel @@ -34,22 +34,31 @@ go_library( go_test( name = "errormanager_test", timeout = "short", - srcs = ["errormanager_test.go"], + srcs = [ + "errormanager_test.go", + "resolveconflict_test.go", + ], embed = [":errormanager"], flaky = True, - shard_count = 6, + shard_count = 10, deps = [ "//br/pkg/lightning/backend/encode", "//br/pkg/lightning/backend/kv", "//br/pkg/lightning/config", "//br/pkg/lightning/log", "//br/pkg/utils", + "//pkg/ddl", + "//pkg/parser", + "//pkg/parser/ast", "//pkg/parser/model", "//pkg/parser/mysql", + "//pkg/sessionctx/variable", "//pkg/table/tables", "//pkg/types", + "//pkg/util/mock", "@com_github_data_dog_go_sqlmock//:go-sqlmock", "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//error", "@org_uber_go_atomic//:atomic", ], ) diff --git a/br/pkg/lightning/errormanager/resolveconflict_test.go b/br/pkg/lightning/errormanager/resolveconflict_test.go new file mode 100644 index 0000000000000..d7e7581deb335 --- /dev/null +++ b/br/pkg/lightning/errormanager/resolveconflict_test.go @@ -0,0 +1,794 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// 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 errormanager_test + +import ( + "bytes" + "context" + "database/sql/driver" + "fmt" + "testing" + + "github.com/DATA-DOG/go-sqlmock" + "github.com/pingcap/tidb/br/pkg/lightning/backend/encode" + tidbkv "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" + "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/errormanager" + "github.com/pingcap/tidb/br/pkg/lightning/log" + "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/pkg/ddl" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/table/tables" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/mock" + "github.com/stretchr/testify/require" + tikverr "github.com/tikv/client-go/v2/error" + "go.uber.org/atomic" +) + +func TestReplaceConflictMultipleKeysNonclusteredPk(t *testing.T) { + p := parser.New() + node, _, err := p.ParseSQL("create table a (a int primary key nonclustered, b int not null, c int not null, d text, key key_b(b), key key_c(c));") + require.NoError(t, err) + mockSctx := mock.NewContext() + mockSctx.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff + info, err := ddl.MockTableInfo(mockSctx, node[0].(*ast.CreateTableStmt), 108) + require.NoError(t, err) + info.State = model.StatePublic + require.False(t, info.PKIsHandle) + tbl, err := tables.TableFromMeta(tidbkv.NewPanickingAllocators(0), info) + require.NoError(t, err) + require.False(t, tbl.Meta().HasClusteredIndex()) + + sessionOpts := encode.SessionOptions{ + SQLMode: mysql.ModeStrictAllTables, + Timestamp: 1234567890, + } + + encoder, err := tidbkv.NewBaseKVEncoder(&encode.EncodingConfig{ + Table: tbl, + SessionOptions: sessionOpts, + Logger: log.L(), + }) + require.NoError(t, err) + encoder.SessionCtx.GetSessionVars().RowEncoder.Enable = true + + data1 := []types.Datum{ + types.NewIntDatum(1), + types.NewIntDatum(1), + types.NewIntDatum(1), + types.NewStringDatum("1.csv"), + types.NewIntDatum(1), + } + data2 := []types.Datum{ + types.NewIntDatum(1), + types.NewIntDatum(1), + types.NewIntDatum(2), + types.NewStringDatum("2.csv"), + types.NewIntDatum(2), + } + data3 := []types.Datum{ + types.NewIntDatum(2), + types.NewIntDatum(2), + types.NewIntDatum(3), + types.NewStringDatum("3.csv"), + types.NewIntDatum(3), + } + data4 := []types.Datum{ + types.NewIntDatum(3), + types.NewIntDatum(3), + types.NewIntDatum(3), + types.NewStringDatum("3.csv"), + types.NewIntDatum(4), + } + data5 := []types.Datum{ + types.NewIntDatum(3), + types.NewIntDatum(3), + types.NewIntDatum(3), + types.NewStringDatum("4.csv"), + types.NewIntDatum(5), + } + data6 := []types.Datum{ + types.NewIntDatum(4), + types.NewIntDatum(4), + types.NewIntDatum(4), + types.NewStringDatum("4.csv"), + types.NewIntDatum(6), + } + data7 := []types.Datum{ + types.NewIntDatum(5), + types.NewIntDatum(4), + types.NewIntDatum(5), + types.NewStringDatum("5.csv"), + types.NewIntDatum(7), + } + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data1) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data2) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data3) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data4) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data5) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data6) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data7) + require.NoError(t, err) + kvPairs := encoder.SessionCtx.TakeKvPairs() + + data2IndexKey := kvPairs.Pairs[5].Key + data2IndexValue := kvPairs.Pairs[5].Val + data6IndexKey := kvPairs.Pairs[17].Key + + data1RowKey := kvPairs.Pairs[0].Key + data2RowKey := kvPairs.Pairs[3].Key + data2RowValue := kvPairs.Pairs[3].Val + data3RowKey := kvPairs.Pairs[6].Key + data3RowValue := kvPairs.Pairs[6].Val + data5RowKey := kvPairs.Pairs[12].Key + data6RowKey := kvPairs.Pairs[15].Key + data6RowValue := kvPairs.Pairs[15].Val + data7RowKey := kvPairs.Pairs[18].Key + data7RowValue := kvPairs.Pairs[18].Val + + data2NonclusteredKey := kvPairs.Pairs[4].Key + data2NonclusteredValue := kvPairs.Pairs[4].Val + data3NonclusteredValue := kvPairs.Pairs[7].Val + data6NonclusteredKey := kvPairs.Pairs[16].Key + data6NonclusteredValue := kvPairs.Pairs[16].Val + data7NonclusteredValue := kvPairs.Pairs[19].Val + + db, mockDB, err := sqlmock.New() + require.NoError(t, err) + defer func() { + _ = db.Close() + }() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + mockDB.ExpectExec("CREATE SCHEMA IF NOT EXISTS `lightning_task_info`"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mockDB.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_task_info`\\.conflict_error_v2.*"). + WillReturnResult(sqlmock.NewResult(2, 1)) + mockDB.ExpectQuery("\\QSELECT raw_key, index_name, raw_value, raw_handle FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? AND is_data_kv = 0 ORDER BY raw_key\\E"). + WillReturnRows(sqlmock.NewRows([]string{"raw_key", "index_name", "raw_value", "raw_handle"}). + AddRow(data2RowKey, "PRIMARY", data2RowValue, data1RowKey). + AddRow(data2RowKey, "PRIMARY", data3NonclusteredValue, data2NonclusteredKey). + AddRow(data6RowKey, "PRIMARY", data6RowValue, data5RowKey). + AddRow(data6RowKey, "PRIMARY", data7NonclusteredValue, data6NonclusteredKey)) + mockDB.ExpectBegin() + mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v2.*"). + WithArgs(0, "a", nil, nil, data2NonclusteredKey, data2NonclusteredValue, 1). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() + mockDB.ExpectBegin() + mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v2.*"). + WithArgs(0, "a", nil, nil, data6NonclusteredKey, data6NonclusteredValue, 1). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() + mockDB.ExpectQuery("\\QSELECT raw_key, raw_value FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? AND is_data_kv = 1 ORDER BY raw_key\\E"). + WillReturnRows(sqlmock.NewRows([]string{"raw_key", "raw_value"}). + AddRow(data2NonclusteredKey, data2NonclusteredValue). + AddRow(data6NonclusteredKey, data6NonclusteredValue)) + + cfg := config.NewConfig() + cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace + cfg.App.TaskInfoSchemaName = "lightning_task_info" + em := errormanager.New(db, cfg, log.L()) + err = em.Init(ctx) + require.NoError(t, err) + + fnGetLatestCount := atomic.NewInt32(0) + fnDeleteKeyCount := atomic.NewInt32(0) + pool := utils.NewWorkerPool(16, "resolve duplicate rows by replace") + err = em.ReplaceConflictKeys( + ctx, tbl, "a", pool, + func(ctx context.Context, key []byte) ([]byte, error) { + fnGetLatestCount.Add(1) + switch { + case bytes.Equal(key, data2RowKey): + return data2RowValue, nil + case bytes.Equal(key, data2NonclusteredKey): + if fnGetLatestCount.String() == "3" { + return data2NonclusteredValue, nil + } else { + return nil, tikverr.ErrNotExist + } + case bytes.Equal(key, data6RowKey): + return data6RowValue, nil + case bytes.Equal(key, data6NonclusteredKey): + if fnGetLatestCount.String() == "6" { + return data6NonclusteredValue, nil + } else { + return nil, tikverr.ErrNotExist + } + case bytes.Equal(key, data2IndexKey): + return data2IndexValue, nil + case bytes.Equal(key, data3RowKey): + return data3RowValue, nil + case bytes.Equal(key, data6IndexKey): + return data3RowValue, nil + case bytes.Equal(key, data7RowKey): + return data7RowValue, nil + default: + return nil, fmt.Errorf("key %v is not expected", key) + } + }, + func(ctx context.Context, key []byte) error { + fnDeleteKeyCount.Add(1) + if !bytes.Equal(key, data2NonclusteredKey) && !bytes.Equal(key, data6NonclusteredKey) && !bytes.Equal(key, data2IndexKey) && !bytes.Equal(key, data3RowKey) && !bytes.Equal(key, data6IndexKey) && !bytes.Equal(key, data7RowKey) { + return fmt.Errorf("key %v is not expected", key) + } + return nil + }, + ) + require.NoError(t, err) + require.Equal(t, int32(16), fnGetLatestCount.Load()) + require.Equal(t, int32(6), fnDeleteKeyCount.Load()) + err = mockDB.ExpectationsWereMet() + require.NoError(t, err) +} + +func TestReplaceConflictOneKeyNonclusteredPk(t *testing.T) { + p := parser.New() + node, _, err := p.ParseSQL("create table a (a int primary key nonclustered, b int not null, c text, key key_b(b));") + require.NoError(t, err) + mockSctx := mock.NewContext() + mockSctx.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff + info, err := ddl.MockTableInfo(mockSctx, node[0].(*ast.CreateTableStmt), 108) + require.NoError(t, err) + info.State = model.StatePublic + require.False(t, info.PKIsHandle) + tbl, err := tables.TableFromMeta(tidbkv.NewPanickingAllocators(0), info) + require.NoError(t, err) + require.False(t, tbl.Meta().HasClusteredIndex()) + + sessionOpts := encode.SessionOptions{ + SQLMode: mysql.ModeStrictAllTables, + Timestamp: 1234567890, + } + + encoder, err := tidbkv.NewBaseKVEncoder(&encode.EncodingConfig{ + Table: tbl, + SessionOptions: sessionOpts, + Logger: log.L(), + }) + require.NoError(t, err) + encoder.SessionCtx.GetSessionVars().RowEncoder.Enable = true + + data1 := []types.Datum{ + types.NewIntDatum(1), + types.NewIntDatum(6), + types.NewStringDatum("1.csv"), + types.NewIntDatum(1), + } + data2 := []types.Datum{ + types.NewIntDatum(2), + types.NewIntDatum(6), + types.NewStringDatum("2.csv"), + types.NewIntDatum(2), + } + data3 := []types.Datum{ + types.NewIntDatum(3), + types.NewIntDatum(3), + types.NewStringDatum("3.csv"), + types.NewIntDatum(3), + } + data4 := []types.Datum{ + types.NewIntDatum(3), + types.NewIntDatum(4), + types.NewStringDatum("4.csv"), + types.NewIntDatum(4), + } + data5 := []types.Datum{ + types.NewIntDatum(5), + types.NewIntDatum(4), + types.NewStringDatum("5.csv"), + types.NewIntDatum(5), + } + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data1) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data2) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data3) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data4) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data5) + require.NoError(t, err) + kvPairs := encoder.SessionCtx.TakeKvPairs() + + data3IndexKey := kvPairs.Pairs[8].Key + data3IndexValue := kvPairs.Pairs[8].Val + data4IndexValue := kvPairs.Pairs[11].Val + data3RowKey := kvPairs.Pairs[6].Key + data4RowKey := kvPairs.Pairs[9].Key + data4RowValue := kvPairs.Pairs[9].Val + data4NonclusteredKey := kvPairs.Pairs[10].Key + data4NonclusteredValue := kvPairs.Pairs[10].Val + + db, mockDB, err := sqlmock.New() + require.NoError(t, err) + defer func() { + _ = db.Close() + }() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + mockDB.ExpectExec("CREATE SCHEMA IF NOT EXISTS `lightning_task_info`"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mockDB.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_task_info`\\.conflict_error_v2.*"). + WillReturnResult(sqlmock.NewResult(2, 1)) + mockDB.ExpectQuery("\\QSELECT raw_key, index_name, raw_value, raw_handle FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? AND is_data_kv = 0 ORDER BY raw_key\\E"). + WillReturnRows(sqlmock.NewRows([]string{"raw_key", "index_name", "raw_value", "raw_handle"}). + AddRow(data3IndexKey, "PRIMARY", data3IndexValue, data3RowKey). + AddRow(data3IndexKey, "PRIMARY", data4IndexValue, data4RowKey)) + mockDB.ExpectBegin() + mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v2.*"). + WithArgs(0, "a", nil, nil, data4RowKey, data4RowValue, 1). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() + mockDB.ExpectQuery("\\QSELECT raw_key, raw_value FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? AND is_data_kv = 1 ORDER BY raw_key\\E"). + WillReturnRows(sqlmock.NewRows([]string{"raw_key", "raw_value"}). + AddRow(data4RowKey, data4RowValue)) + + cfg := config.NewConfig() + cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace + cfg.App.TaskInfoSchemaName = "lightning_task_info" + em := errormanager.New(db, cfg, log.L()) + err = em.Init(ctx) + require.NoError(t, err) + + fnGetLatestCount := atomic.NewInt32(0) + fnDeleteKeyCount := atomic.NewInt32(0) + pool := utils.NewWorkerPool(16, "resolve duplicate rows by replace") + err = em.ReplaceConflictKeys( + ctx, tbl, "a", pool, + func(ctx context.Context, key []byte) ([]byte, error) { + fnGetLatestCount.Add(1) + switch { + case bytes.Equal(key, data3IndexKey): + return data3IndexValue, nil + case bytes.Equal(key, data4RowKey): + if fnGetLatestCount.String() == "3" { + return data4RowValue, nil + } else { + return nil, tikverr.ErrNotExist + } + case bytes.Equal(key, data4NonclusteredKey): + return data4NonclusteredValue, nil + default: + return nil, fmt.Errorf("key %v is not expected", key) + } + }, + func(ctx context.Context, key []byte) error { + fnDeleteKeyCount.Add(1) + if !bytes.Equal(key, data4RowKey) && !bytes.Equal(key, data4NonclusteredKey) { + return fmt.Errorf("key %v is not expected", key) + } + return nil + }, + ) + require.NoError(t, err) + require.Equal(t, int32(7), fnGetLatestCount.Load()) + require.Equal(t, int32(2), fnDeleteKeyCount.Load()) + err = mockDB.ExpectationsWereMet() + require.NoError(t, err) +} + +func TestReplaceConflictOneUniqueKeyNonclusteredPk(t *testing.T) { + p := parser.New() + node, _, err := p.ParseSQL("create table a (a int primary key nonclustered, b int not null, c text, unique key uni_b(b));") + require.NoError(t, err) + mockSctx := mock.NewContext() + mockSctx.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff + info, err := ddl.MockTableInfo(mockSctx, node[0].(*ast.CreateTableStmt), 108) + require.NoError(t, err) + info.State = model.StatePublic + require.False(t, info.PKIsHandle) + tbl, err := tables.TableFromMeta(tidbkv.NewPanickingAllocators(0), info) + require.NoError(t, err) + require.False(t, tbl.Meta().HasClusteredIndex()) + + sessionOpts := encode.SessionOptions{ + SQLMode: mysql.ModeStrictAllTables, + Timestamp: 1234567890, + } + + encoder, err := tidbkv.NewBaseKVEncoder(&encode.EncodingConfig{ + Table: tbl, + SessionOptions: sessionOpts, + Logger: log.L(), + }) + require.NoError(t, err) + encoder.SessionCtx.GetSessionVars().RowEncoder.Enable = true + + data1 := []types.Datum{ + types.NewIntDatum(1), + types.NewIntDatum(6), + types.NewStringDatum("1.csv"), + types.NewIntDatum(1), + } + data2 := []types.Datum{ + types.NewIntDatum(2), + types.NewIntDatum(6), + types.NewStringDatum("2.csv"), + types.NewIntDatum(2), + } + data3 := []types.Datum{ + types.NewIntDatum(3), + types.NewIntDatum(3), + types.NewStringDatum("3.csv"), + types.NewIntDatum(3), + } + data4 := []types.Datum{ + types.NewIntDatum(3), + types.NewIntDatum(4), + types.NewStringDatum("4.csv"), + types.NewIntDatum(4), + } + data5 := []types.Datum{ + types.NewIntDatum(5), + types.NewIntDatum(4), + types.NewStringDatum("5.csv"), + types.NewIntDatum(5), + } + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data1) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data2) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data3) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data4) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data5) + require.NoError(t, err) + kvPairs := encoder.SessionCtx.TakeKvPairs() + + data1RowKey := kvPairs.Pairs[0].Key + data2RowKey := kvPairs.Pairs[3].Key + data2RowValue := kvPairs.Pairs[3].Val + data3RowKey := kvPairs.Pairs[6].Key + data4RowKey := kvPairs.Pairs[9].Key + data4RowValue := kvPairs.Pairs[9].Val + data5RowKey := kvPairs.Pairs[12].Key + data5RowValue := kvPairs.Pairs[12].Val + + data2IndexKey := kvPairs.Pairs[5].Key + data2IndexValue := kvPairs.Pairs[5].Val + data3IndexKey := kvPairs.Pairs[8].Key + data3IndexValue := kvPairs.Pairs[8].Val + data5IndexKey := kvPairs.Pairs[14].Key + data5IndexValue := kvPairs.Pairs[14].Val + + data1NonclusteredKey := kvPairs.Pairs[1].Key + data1NonclusteredValue := kvPairs.Pairs[1].Val + data2NonclusteredValue := kvPairs.Pairs[4].Val + data4NonclusteredKey := kvPairs.Pairs[10].Key + data4NonclusteredValue := kvPairs.Pairs[10].Val + data5NonclusteredValue := kvPairs.Pairs[13].Val + + db, mockDB, err := sqlmock.New() + require.NoError(t, err) + defer func() { + _ = db.Close() + }() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + mockDB.ExpectExec("CREATE SCHEMA IF NOT EXISTS `lightning_task_info`"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mockDB.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_task_info`\\.conflict_error_v2.*"). + WillReturnResult(sqlmock.NewResult(2, 1)) + mockDB.ExpectQuery("\\QSELECT raw_key, index_name, raw_value, raw_handle FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? AND is_data_kv = 0 ORDER BY raw_key\\E"). + WillReturnRows(sqlmock.NewRows([]string{"raw_key", "index_name", "raw_value", "raw_handle"}). + AddRow(data4NonclusteredKey, "uni_b", data4NonclusteredValue, data4RowKey). + AddRow(data4NonclusteredKey, "uni_b", data5NonclusteredValue, data5RowKey). + AddRow(data1NonclusteredKey, "uni_b", data1NonclusteredValue, data1RowKey). + AddRow(data1NonclusteredKey, "uni_b", data2NonclusteredValue, data2RowKey). + AddRow(data3IndexKey, "PRIMARY", data3IndexValue, data3RowKey). + AddRow(data3IndexKey, "PRIMARY", data4NonclusteredValue, data4RowKey)) + mockDB.ExpectBegin() + mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v2.*"). + WithArgs(0, "a", nil, nil, data5RowKey, data5RowValue, 1). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() + mockDB.ExpectBegin() + mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v2.*"). + WithArgs(0, "a", nil, nil, data2RowKey, data2RowValue, 1). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() + mockDB.ExpectBegin() + mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v2.*"). + WithArgs(0, "a", nil, nil, data4RowKey, data4RowValue, 1). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() + mockDB.ExpectQuery("\\QSELECT raw_key, raw_value FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? AND is_data_kv = 1 ORDER BY raw_key\\E"). + WillReturnRows(sqlmock.NewRows([]string{"raw_key", "raw_value"}). + AddRow(data5RowKey, data5RowValue). + AddRow(data2RowKey, data2RowValue). + AddRow(data4RowKey, data4RowValue)) + + cfg := config.NewConfig() + cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace + cfg.App.TaskInfoSchemaName = "lightning_task_info" + em := errormanager.New(db, cfg, log.L()) + err = em.Init(ctx) + require.NoError(t, err) + + fnGetLatestCount := atomic.NewInt32(0) + fnDeleteKeyCount := atomic.NewInt32(0) + pool := utils.NewWorkerPool(16, "resolve duplicate rows by replace") + err = em.ReplaceConflictKeys( + ctx, tbl, "a", pool, + func(ctx context.Context, key []byte) ([]byte, error) { + fnGetLatestCount.Add(1) + switch { + case bytes.Equal(key, data4NonclusteredKey): + if fnGetLatestCount.String() != "20" { + return data4NonclusteredValue, nil + } else { + return nil, tikverr.ErrNotExist + } + case bytes.Equal(key, data5RowKey): + if fnGetLatestCount.String() == "3" { + return data5RowValue, nil + } else { + return nil, tikverr.ErrNotExist + } + case bytes.Equal(key, data1NonclusteredKey): + return data1NonclusteredValue, nil + case bytes.Equal(key, data2RowKey): + if fnGetLatestCount.String() == "6" { + return data2RowValue, nil + } else { + return nil, tikverr.ErrNotExist + } + case bytes.Equal(key, data3IndexKey): + return data3IndexValue, nil + case bytes.Equal(key, data4RowKey): + return data4RowValue, nil + case bytes.Equal(key, data2IndexKey): + return data2IndexValue, nil + case bytes.Equal(key, data5IndexKey): + return data5IndexValue, nil + default: + return nil, fmt.Errorf("key %x is not expected", key) + } + }, + func(ctx context.Context, key []byte) error { + fnDeleteKeyCount.Add(1) + if !bytes.Equal(key, data5RowKey) && !bytes.Equal(key, data2RowKey) && !bytes.Equal(key, data4RowKey) && !bytes.Equal(key, data2IndexKey) && !bytes.Equal(key, data4NonclusteredKey) && !bytes.Equal(key, data5IndexKey) { + return fmt.Errorf("key %v is not expected", key) + } + return nil + }, + ) + require.NoError(t, err) + require.Equal(t, int32(18), fnGetLatestCount.Load()) + require.Equal(t, int32(5), fnDeleteKeyCount.Load()) + err = mockDB.ExpectationsWereMet() + require.NoError(t, err) +} + +func TestReplaceConflictOneUniqueKeyNonclusteredVarcharPk(t *testing.T) { + p := parser.New() + node, _, err := p.ParseSQL("create table a (a varchar(20) primary key nonclustered, b int not null, c text, unique key uni_b(b));") + require.NoError(t, err) + mockSctx := mock.NewContext() + mockSctx.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff + info, err := ddl.MockTableInfo(mockSctx, node[0].(*ast.CreateTableStmt), 108) + require.NoError(t, err) + info.State = model.StatePublic + require.False(t, info.PKIsHandle) + tbl, err := tables.TableFromMeta(tidbkv.NewPanickingAllocators(0), info) + require.NoError(t, err) + require.False(t, tbl.Meta().HasClusteredIndex()) + + sessionOpts := encode.SessionOptions{ + SQLMode: mysql.ModeStrictAllTables, + Timestamp: 1234567890, + } + + encoder, err := tidbkv.NewBaseKVEncoder(&encode.EncodingConfig{ + Table: tbl, + SessionOptions: sessionOpts, + Logger: log.L(), + }) + require.NoError(t, err) + encoder.SessionCtx.GetSessionVars().RowEncoder.Enable = true + + data1 := []types.Datum{ + types.NewStringDatum("x"), + types.NewIntDatum(6), + types.NewStringDatum("1.csv"), + types.NewIntDatum(1), + } + data2 := []types.Datum{ + types.NewStringDatum("y"), + types.NewIntDatum(6), + types.NewStringDatum("2.csv"), + types.NewIntDatum(2), + } + data3 := []types.Datum{ + types.NewStringDatum("z"), + types.NewIntDatum(3), + types.NewStringDatum("3.csv"), + types.NewIntDatum(3), + } + data4 := []types.Datum{ + types.NewStringDatum("z"), + types.NewIntDatum(4), + types.NewStringDatum("4.csv"), + types.NewIntDatum(4), + } + data5 := []types.Datum{ + types.NewStringDatum("t"), + types.NewIntDatum(4), + types.NewStringDatum("5.csv"), + types.NewIntDatum(5), + } + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data1) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data2) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data3) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data4) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data5) + require.NoError(t, err) + kvPairs := encoder.SessionCtx.TakeKvPairs() + + data1RowKey := kvPairs.Pairs[0].Key + data2RowKey := kvPairs.Pairs[3].Key + data2RowValue := kvPairs.Pairs[3].Val + data3RowKey := kvPairs.Pairs[6].Key + data4RowKey := kvPairs.Pairs[9].Key + data4RowValue := kvPairs.Pairs[9].Val + data5RowKey := kvPairs.Pairs[12].Key + data5RowValue := kvPairs.Pairs[12].Val + + data2IndexKey := kvPairs.Pairs[5].Key + data2IndexValue := kvPairs.Pairs[5].Val + data3IndexKey := kvPairs.Pairs[8].Key + data3IndexValue := kvPairs.Pairs[8].Val + data4IndexValue := kvPairs.Pairs[11].Val + data5IndexKey := kvPairs.Pairs[14].Key + data5IndexValue := kvPairs.Pairs[14].Val + + data1NonclusteredKey := kvPairs.Pairs[1].Key + data1NonclusteredValue := kvPairs.Pairs[1].Val + data2NonclusteredValue := kvPairs.Pairs[4].Val + data4NonclusteredKey := kvPairs.Pairs[10].Key + data4NonclusteredValue := kvPairs.Pairs[10].Val + data5NonclusteredValue := kvPairs.Pairs[13].Val + + db, mockDB, err := sqlmock.New() + require.NoError(t, err) + defer func() { + _ = db.Close() + }() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + mockDB.ExpectExec("CREATE SCHEMA IF NOT EXISTS `lightning_task_info`"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mockDB.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_task_info`\\.conflict_error_v2.*"). + WillReturnResult(sqlmock.NewResult(2, 1)) + mockDB.ExpectQuery("\\QSELECT raw_key, index_name, raw_value, raw_handle FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? AND is_data_kv = 0 ORDER BY raw_key\\E"). + WillReturnRows(sqlmock.NewRows([]string{"raw_key", "index_name", "raw_value", "raw_handle"}). + AddRow(data4NonclusteredKey, "uni_b", data4NonclusteredValue, data4RowKey). + AddRow(data4NonclusteredKey, "uni_b", data5NonclusteredValue, data5RowKey). + AddRow(data1NonclusteredKey, "uni_b", data1NonclusteredValue, data1RowKey). + AddRow(data1NonclusteredKey, "uni_b", data2NonclusteredValue, data2RowKey). + AddRow(data3IndexKey, "PRIMARY", data3IndexValue, data3RowKey). + AddRow(data3IndexKey, "PRIMARY", data4IndexValue, data4RowKey)) + mockDB.ExpectBegin() + mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v2.*"). + WithArgs(0, "a", nil, nil, data5RowKey, data5RowValue, 1). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() + mockDB.ExpectBegin() + mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v2.*"). + WithArgs(0, "a", nil, nil, data2RowKey, data2RowValue, 1). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() + mockDB.ExpectBegin() + mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v2.*"). + WithArgs(0, "a", nil, nil, data4RowKey, data4RowValue, 1). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() + mockDB.ExpectQuery("\\QSELECT raw_key, raw_value FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? AND is_data_kv = 1 ORDER BY raw_key\\E"). + WillReturnRows(sqlmock.NewRows([]string{"raw_key", "raw_value"}). + AddRow(data5RowKey, data5RowValue). + AddRow(data2RowKey, data2RowValue). + AddRow(data4RowKey, data4RowValue)) + + cfg := config.NewConfig() + cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace + cfg.App.TaskInfoSchemaName = "lightning_task_info" + em := errormanager.New(db, cfg, log.L()) + err = em.Init(ctx) + require.NoError(t, err) + + fnGetLatestCount := atomic.NewInt32(0) + fnDeleteKeyCount := atomic.NewInt32(0) + pool := utils.NewWorkerPool(16, "resolve duplicate rows by replace") + err = em.ReplaceConflictKeys( + ctx, tbl, "a", pool, + func(ctx context.Context, key []byte) ([]byte, error) { + fnGetLatestCount.Add(1) + switch { + case bytes.Equal(key, data4NonclusteredKey): + if fnGetLatestCount.String() != "20" { + return data4NonclusteredValue, nil + } else { + return nil, tikverr.ErrNotExist + } + case bytes.Equal(key, data5RowKey): + if fnGetLatestCount.String() == "3" { + return data5RowValue, nil + } else { + return nil, tikverr.ErrNotExist + } + case bytes.Equal(key, data1NonclusteredKey): + return data1NonclusteredValue, nil + case bytes.Equal(key, data2RowKey): + if fnGetLatestCount.String() == "6" { + return data2RowValue, nil + } else { + return nil, tikverr.ErrNotExist + } + case bytes.Equal(key, data3IndexKey): + return data3IndexValue, nil + case bytes.Equal(key, data4RowKey): + if fnGetLatestCount.String() == "9" { + return data4RowValue, nil + } else { + return nil, tikverr.ErrNotExist + } + case bytes.Equal(key, data2IndexKey): + return data2IndexValue, nil + case bytes.Equal(key, data5IndexKey): + return data5IndexValue, nil + default: + return nil, fmt.Errorf("key %x is not expected", key) + } + }, + func(ctx context.Context, key []byte) error { + fnDeleteKeyCount.Add(1) + if !bytes.Equal(key, data5RowKey) && !bytes.Equal(key, data2RowKey) && !bytes.Equal(key, data4RowKey) && !bytes.Equal(key, data2IndexKey) && !bytes.Equal(key, data4NonclusteredKey) && !bytes.Equal(key, data5IndexKey) { + return fmt.Errorf("key %v is not expected", key) + } + return nil + }, + ) + require.NoError(t, err) + require.Equal(t, int32(21), fnGetLatestCount.Load()) + require.Equal(t, int32(5), fnDeleteKeyCount.Load()) + err = mockDB.ExpectationsWereMet() + require.NoError(t, err) +} From 7d1000c79383e00850ff24a3fbf7ad671aec42ca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20van=20Eeden?= Date: Wed, 22 Nov 2023 10:41:43 +0100 Subject: [PATCH 03/36] executor,infoschema: Change column type on information_schema.CLUSTER_INFO (#29771) close pingcap/tidb#45221 --- pkg/executor/infoschema_cluster_table_test.go | 5 +++-- pkg/executor/infoschema_reader.go | 6 +++--- pkg/executor/inspection_result_test.go | 15 +++++++++------ pkg/infoschema/tables.go | 2 +- pkg/infoschema/test/clustertablestest/BUILD.bazel | 2 +- .../test/clustertablestest/tables_test.go | 8 ++++++++ 6 files changed, 25 insertions(+), 13 deletions(-) diff --git a/pkg/executor/infoschema_cluster_table_test.go b/pkg/executor/infoschema_cluster_table_test.go index 5ac6d7e147236..332bcfa52221f 100644 --- a/pkg/executor/infoschema_cluster_table_test.go +++ b/pkg/executor/infoschema_cluster_table_test.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/pkg/store/helper" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/tikv" @@ -223,10 +224,10 @@ func TestTiDBClusterInfo(t *testing.T) { row("pd", mockAddr, mockAddr, "4.0.0-alpha", "mock-pd-githash"), row("tikv", "store1", "", "", ""), )) - startTime := s.startTime.Format(time.RFC3339) + startTime := types.NewTime(types.FromGoTime(s.startTime), mysql.TypeDatetime, 0).String() tk.MustQuery("select type, instance, start_time from information_schema.cluster_info where type != 'tidb'").Check(testkit.Rows( row("pd", mockAddr, startTime), - row("tikv", "store1", ""), + row("tikv", "store1", startTime), )) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/infoschema/mockStoreTombstone", `return(true)`)) diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 9fabb6cfbb8b5..5fbff1bc6b4cc 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -1420,11 +1420,11 @@ func (e *memtableRetriever) dataForTiDBClusterInfo(ctx sessionctx.Context) error } rows := make([][]types.Datum, 0, len(servers)) for _, server := range servers { - startTimeStr := "" upTimeStr := "" + startTimeNative := types.NewTime(types.FromGoTime(time.Now()), mysql.TypeDatetime, 0) if server.StartTimestamp > 0 { startTime := time.Unix(server.StartTimestamp, 0) - startTimeStr = startTime.Format(time.RFC3339) + startTimeNative = types.NewTime(types.FromGoTime(startTime), mysql.TypeDatetime, 0) upTimeStr = time.Since(startTime).String() } serverType := server.ServerType @@ -1437,7 +1437,7 @@ func (e *memtableRetriever) dataForTiDBClusterInfo(ctx sessionctx.Context) error server.StatusAddr, server.Version, server.GitHash, - startTimeStr, + startTimeNative, upTimeStr, server.ServerID, ) diff --git a/pkg/executor/inspection_result_test.go b/pkg/executor/inspection_result_test.go index f144ec0e0e632..27f47dcd3dd54 100644 --- a/pkg/executor/inspection_result_test.go +++ b/pkg/executor/inspection_result_test.go @@ -21,6 +21,7 @@ import ( "path/filepath" "strings" "testing" + "time" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/diagnosticspb" @@ -204,14 +205,16 @@ func createInspectionContext(t *testing.T, mockData map[string][][]types.Datum, }, } // mock cluster information + timeNow := types.NewTime(types.FromGoTime(time.Now()), mysql.TypeDatetime, 0) configurations[infoschema.TableClusterInfo] = variable.TableSnapshot{ Rows: [][]types.Datum{ - types.MakeDatums("pd", "pd-0", "pd-0", "4.0", "a234c", "", ""), - types.MakeDatums("tidb", "tidb-0", "tidb-0s", "4.0", "a234c", "", ""), - types.MakeDatums("tidb", "tidb-1", "tidb-1s", "4.0", "a234c", "", ""), - types.MakeDatums("tikv", "tikv-0", "tikv-0s", "4.0", "a234c", "", ""), - types.MakeDatums("tikv", "tikv-1", "tikv-1s", "4.0", "a234c", "", ""), - types.MakeDatums("tikv", "tikv-2", "tikv-2s", "4.0", "a234c", "", ""), + // Columns: TYPE, INSTANCE, STATUS_ADDRESS, VERSION, GIT_HASH, START_TIME, UPTIME + types.MakeDatums("pd", "pd-0", "pd-0", "4.0", "a234c", timeNow, ""), + types.MakeDatums("tidb", "tidb-0", "tidb-0s", "4.0", "a234c", timeNow, ""), + types.MakeDatums("tidb", "tidb-1", "tidb-1s", "4.0", "a234c", timeNow, ""), + types.MakeDatums("tikv", "tikv-0", "tikv-0s", "4.0", "a234c", timeNow, ""), + types.MakeDatums("tikv", "tikv-1", "tikv-1s", "4.0", "a234c", timeNow, ""), + types.MakeDatums("tikv", "tikv-2", "tikv-2s", "4.0", "a234c", timeNow, ""), }, } // mock cluster system information diff --git a/pkg/infoschema/tables.go b/pkg/infoschema/tables.go index d2992118bd2e0..0236f3bf77456 100644 --- a/pkg/infoschema/tables.go +++ b/pkg/infoschema/tables.go @@ -1160,7 +1160,7 @@ var tableClusterInfoCols = []columnInfo{ {name: "STATUS_ADDRESS", tp: mysql.TypeVarchar, size: 64}, {name: "VERSION", tp: mysql.TypeVarchar, size: 64}, {name: "GIT_HASH", tp: mysql.TypeVarchar, size: 64}, - {name: "START_TIME", tp: mysql.TypeVarchar, size: 32}, + {name: "START_TIME", tp: mysql.TypeDatetime, size: 19}, {name: "UPTIME", tp: mysql.TypeVarchar, size: 32}, {name: "SERVER_ID", tp: mysql.TypeLonglong, size: 21, comment: "invalid if the configuration item `enable-global-kill` is set to FALSE"}, } diff --git a/pkg/infoschema/test/clustertablestest/BUILD.bazel b/pkg/infoschema/test/clustertablestest/BUILD.bazel index e0e053c35698a..e992561b17378 100644 --- a/pkg/infoschema/test/clustertablestest/BUILD.bazel +++ b/pkg/infoschema/test/clustertablestest/BUILD.bazel @@ -9,7 +9,7 @@ go_test( "tables_test.go", ], flaky = True, - shard_count = 45, + shard_count = 46, deps = [ "//pkg/config", "//pkg/domain", diff --git a/pkg/infoschema/test/clustertablestest/tables_test.go b/pkg/infoschema/test/clustertablestest/tables_test.go index ddcf6c69a6b88..4f646173e65be 100644 --- a/pkg/infoschema/test/clustertablestest/tables_test.go +++ b/pkg/infoschema/test/clustertablestest/tables_test.go @@ -1417,3 +1417,11 @@ func TestAddFieldsForBinding(t *testing.T) { require.Equal(t, rows[0][7], "use_index(@`sel_1` `test`.`t` ), ignore_index(`t` `a`)") require.Equal(t, rows[0][8], "select * from `t` where `a` = ?") } + +func TestClusterInfoTime(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustQuery("SELECT START_TIME+1 FROM information_schema.CLUSTER_INFO") + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Nil(t, warnings) +} From 297872a7b27d45c3cf550263017f484f66f1ef02 Mon Sep 17 00:00:00 2001 From: Rustin Liu Date: Wed, 22 Nov 2023 17:41:51 +0800 Subject: [PATCH 04/36] statistics: move interfaces out of the util package (#48612) --- pkg/executor/lockstats/BUILD.bazel | 2 +- pkg/executor/lockstats/lock_stats_executor.go | 8 +- .../core/casetest/planstats/BUILD.bazel | 2 +- .../casetest/planstats/plan_stats_test.go | 4 +- pkg/statistics/handle/BUILD.bazel | 3 +- pkg/statistics/handle/autoanalyze/BUILD.bazel | 1 + .../handle/autoanalyze/autoanalyze.go | 47 +++++--- pkg/statistics/handle/bootstrap.go | 25 ++-- pkg/statistics/handle/cache/BUILD.bazel | 3 +- pkg/statistics/handle/cache/bench_test.go | 8 +- pkg/statistics/handle/cache/statscache.go | 9 +- pkg/statistics/handle/ddl/BUILD.bazel | 1 + pkg/statistics/handle/ddl/ddl.go | 19 +-- pkg/statistics/handle/globalstats/BUILD.bazel | 1 + .../handle/globalstats/global_stats.go | 13 +- .../handle/globalstats/global_stats_async.go | 5 +- pkg/statistics/handle/handle.go | 112 ++++++------------ pkg/statistics/handle/history/BUILD.bazel | 1 + .../handle/history/history_stats.go | 7 +- pkg/statistics/handle/lockstats/BUILD.bazel | 2 + pkg/statistics/handle/lockstats/lock_stats.go | 9 +- .../handle/lockstats/lock_stats_test.go | 3 +- .../handle/lockstats/unlock_stats.go | 3 +- .../handle/lockstats/unlock_stats_test.go | 3 +- pkg/statistics/handle/storage/BUILD.bazel | 1 + pkg/statistics/handle/storage/gc.go | 11 +- pkg/statistics/handle/storage/read.go | 7 +- .../handle/storage/stats_read_writer.go | 7 +- pkg/statistics/handle/storage/update.go | 5 +- pkg/statistics/handle/syncload/BUILD.bazel | 1 + .../handle/syncload/stats_syncload.go | 29 ++--- pkg/statistics/handle/types/BUILD.bazel | 20 ++++ .../handle/{util => types}/interfaces.go | 45 +++---- .../handle/updatetest/update_test.go | 6 +- pkg/statistics/handle/usage/BUILD.bazel | 1 + .../handle/usage/predicate_column.go | 17 +-- pkg/statistics/handle/util/BUILD.bazel | 9 +- .../util/auto_analyze_proc_id_generator.go | 40 +++++++ pkg/statistics/handle/util/lease_getter.go | 52 ++++++++ pkg/statistics/handle/util/pool.go | 72 +++++++++++ pkg/statistics/handle/util/util.go | 16 +-- 41 files changed, 401 insertions(+), 229 deletions(-) create mode 100644 pkg/statistics/handle/types/BUILD.bazel rename pkg/statistics/handle/{util => types}/interfaces.go (94%) create mode 100644 pkg/statistics/handle/util/auto_analyze_proc_id_generator.go create mode 100644 pkg/statistics/handle/util/lease_getter.go create mode 100644 pkg/statistics/handle/util/pool.go diff --git a/pkg/executor/lockstats/BUILD.bazel b/pkg/executor/lockstats/BUILD.bazel index 7e9bb0b700c7c..59888276e9c40 100644 --- a/pkg/executor/lockstats/BUILD.bazel +++ b/pkg/executor/lockstats/BUILD.bazel @@ -14,7 +14,7 @@ go_library( "//pkg/infoschema", "//pkg/parser/ast", "//pkg/parser/model", - "//pkg/statistics/handle/util", + "//pkg/statistics/handle/types", "//pkg/table/tables", "//pkg/util/chunk", "@com_github_pingcap_errors//:errors", diff --git a/pkg/executor/lockstats/lock_stats_executor.go b/pkg/executor/lockstats/lock_stats_executor.go index 50852883882ed..2a59ee41fdb80 100644 --- a/pkg/executor/lockstats/lock_stats_executor.go +++ b/pkg/executor/lockstats/lock_stats_executor.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" - "github.com/pingcap/tidb/pkg/statistics/handle/util" + "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/util/chunk" ) @@ -135,11 +135,11 @@ func populatePartitionIDAndNames( func populateTableAndPartitionIDs( tables []*ast.TableName, is infoschema.InfoSchema, -) (map[int64]*util.StatsLockTable, error) { +) (map[int64]*types.StatsLockTable, error) { if len(tables) == 0 { return nil, errors.New("table list should not be empty") } - tableWithPartitions := make(map[int64]*util.StatsLockTable, len(tables)) + tableWithPartitions := make(map[int64]*types.StatsLockTable, len(tables)) for _, table := range tables { tbl, err := is.TableByName(table.Schema, table.Name) @@ -147,7 +147,7 @@ func populateTableAndPartitionIDs( return nil, err } tid := tbl.Meta().ID - tableWithPartitions[tid] = &util.StatsLockTable{ + tableWithPartitions[tid] = &types.StatsLockTable{ FullName: fmt.Sprintf("%s.%s", table.Schema.L, table.Name.L), } diff --git a/pkg/planner/core/casetest/planstats/BUILD.bazel b/pkg/planner/core/casetest/planstats/BUILD.bazel index c789c86065f71..dab3681d43653 100644 --- a/pkg/planner/core/casetest/planstats/BUILD.bazel +++ b/pkg/planner/core/casetest/planstats/BUILD.bazel @@ -21,7 +21,7 @@ go_test( "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", "//pkg/statistics", - "//pkg/statistics/handle/util", + "//pkg/statistics/handle/types", "//pkg/table", "//pkg/testkit", "//pkg/testkit/testdata", diff --git a/pkg/planner/core/casetest/planstats/plan_stats_test.go b/pkg/planner/core/casetest/planstats/plan_stats_test.go index f3ca791622b8e..87e1c488d21b6 100644 --- a/pkg/planner/core/casetest/planstats/plan_stats_test.go +++ b/pkg/planner/core/casetest/planstats/plan_stats_test.go @@ -32,7 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" - utilstats "github.com/pingcap/tidb/pkg/statistics/handle/util" + "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/testkit/testdata" @@ -268,7 +268,7 @@ func TestPlanStatsLoadTimeout(t *testing.T) { neededColumn := model.TableItemID{TableID: tableInfo.ID, ID: tableInfo.Columns[0].ID, IsIndex: false} resultCh := make(chan stmtctx.StatsLoadResult, 1) timeout := time.Duration(1<<63 - 1) - task := &utilstats.NeededItemTask{ + task := &types.NeededItemTask{ TableItemID: neededColumn, ResultCh: resultCh, ToTimeout: time.Now().Local().Add(timeout), diff --git a/pkg/statistics/handle/BUILD.bazel b/pkg/statistics/handle/BUILD.bazel index 0aaf565f80322..8854a85d5db92 100644 --- a/pkg/statistics/handle/BUILD.bazel +++ b/pkg/statistics/handle/BUILD.bazel @@ -27,14 +27,13 @@ go_library( "//pkg/statistics/handle/logutil", "//pkg/statistics/handle/storage", "//pkg/statistics/handle/syncload", + "//pkg/statistics/handle/types", "//pkg/statistics/handle/usage", "//pkg/statistics/handle/util", "//pkg/types", "//pkg/util/chunk", "//pkg/util/logutil", "@com_github_pingcap_errors//:errors", - "@com_github_tiancaiamao_gp//:gp", - "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", ], ) diff --git a/pkg/statistics/handle/autoanalyze/BUILD.bazel b/pkg/statistics/handle/autoanalyze/BUILD.bazel index 310a3dd31bfc5..45fe7126994c1 100644 --- a/pkg/statistics/handle/autoanalyze/BUILD.bazel +++ b/pkg/statistics/handle/autoanalyze/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/handle/logutil", + "//pkg/statistics/handle/types", "//pkg/statistics/handle/util", "//pkg/table", "//pkg/types", diff --git a/pkg/statistics/handle/autoanalyze/autoanalyze.go b/pkg/statistics/handle/autoanalyze/autoanalyze.go index f21c90de4d21a..66f9ed161d564 100644 --- a/pkg/statistics/handle/autoanalyze/autoanalyze.go +++ b/pkg/statistics/handle/autoanalyze/autoanalyze.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" + statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -47,12 +48,17 @@ import ( // statsAnalyze implements util.StatsAnalyze. // statsAnalyze is used to handle auto-analyze and manage analyze jobs. type statsAnalyze struct { - statsHandle statsutil.StatsHandle + statsHandle statstypes.StatsHandle + // sysProcTracker is used to track sys process like analyze + sysProcTracker sessionctx.SysProcTracker } // NewStatsAnalyze creates a new StatsAnalyze. -func NewStatsAnalyze(statsHandle statsutil.StatsHandle) statsutil.StatsAnalyze { - return &statsAnalyze{statsHandle: statsHandle} +func NewStatsAnalyze( + statsHandle statstypes.StatsHandle, + sysProcTracker sessionctx.SysProcTracker, +) statstypes.StatsAnalyze { + return &statsAnalyze{statsHandle: statsHandle, sysProcTracker: sysProcTracker} } // InsertAnalyzeJob inserts the analyze job to the storage. @@ -73,7 +79,7 @@ func (sa *statsAnalyze) DeleteAnalyzeJobs(updateTime time.Time) error { // HandleAutoAnalyze analyzes the newly created table or index. func (sa *statsAnalyze) HandleAutoAnalyze(is infoschema.InfoSchema) (analyzed bool) { _ = statsutil.CallWithSCtx(sa.statsHandle.SPool(), func(sctx sessionctx.Context) error { - analyzed = HandleAutoAnalyze(sctx, sa.statsHandle, is) + analyzed = HandleAutoAnalyze(sctx, sa.statsHandle, sa.sysProcTracker, is) return nil }) return @@ -151,7 +157,8 @@ func getAllTidsAndPids(tbls []table.Table) []int64 { // HandleAutoAnalyze analyzes the newly created table or index. func HandleAutoAnalyze( sctx sessionctx.Context, - statsHandle statsutil.StatsHandle, + statsHandle statstypes.StatsHandle, + sysProcTracker sessionctx.SysProcTracker, is infoschema.InfoSchema, ) (analyzed bool) { defer func() { @@ -233,7 +240,7 @@ func HandleAutoAnalyze( if pi == nil { statsTbl := statsHandle.GetTableStats(tblInfo) sql := "analyze table %n.%n" - analyzed := tryAutoAnalyzeTable(sctx, statsHandle, tblInfo, statsTbl, autoAnalyzeRatio, sql, db, tblInfo.Name.O) + analyzed := tryAutoAnalyzeTable(sctx, statsHandle, sysProcTracker, tblInfo, statsTbl, autoAnalyzeRatio, sql, db, tblInfo.Name.O) if analyzed { // analyze one table at a time to let it get the freshest parameters. // others will be analyzed next round which is just 3s later. @@ -249,7 +256,7 @@ func HandleAutoAnalyze( } } if pruneMode == variable.Dynamic { - analyzed := tryAutoAnalyzePartitionTableInDynamicMode(sctx, statsHandle, tblInfo, partitionDefs, db, autoAnalyzeRatio) + analyzed := tryAutoAnalyzePartitionTableInDynamicMode(sctx, statsHandle, sysProcTracker, tblInfo, partitionDefs, db, autoAnalyzeRatio) if analyzed { return true } @@ -258,7 +265,7 @@ func HandleAutoAnalyze( for _, def := range partitionDefs { sql := "analyze table %n.%n partition %n" statsTbl := statsHandle.GetPartitionStats(tblInfo, def.ID) - analyzed := tryAutoAnalyzeTable(sctx, statsHandle, tblInfo, statsTbl, autoAnalyzeRatio, sql, db, tblInfo.Name.O, def.Name.O) + analyzed := tryAutoAnalyzeTable(sctx, statsHandle, sysProcTracker, tblInfo, statsTbl, autoAnalyzeRatio, sql, db, tblInfo.Name.O, def.Name.O) if analyzed { return true } @@ -276,7 +283,8 @@ var AutoAnalyzeMinCnt int64 = 1000 // Determine whether the table and index require analysis. func tryAutoAnalyzeTable( sctx sessionctx.Context, - statsHandle statsutil.StatsHandle, + statsHandle statstypes.StatsHandle, + sysProcTracker sessionctx.SysProcTracker, tblInfo *model.TableInfo, statsTbl *statistics.Table, ratio float64, @@ -307,7 +315,7 @@ func tryAutoAnalyzeTable( tableStatsVer := sctx.GetSessionVars().AnalyzeVersion statistics.CheckAnalyzeVerOnTable(statsTbl, &tableStatsVer) - execAutoAnalyze(sctx, statsHandle, tableStatsVer, sql, params...) + execAutoAnalyze(sctx, statsHandle, sysProcTracker, tableStatsVer, sql, params...) return true } @@ -328,7 +336,7 @@ func tryAutoAnalyzeTable( ) tableStatsVer := sctx.GetSessionVars().AnalyzeVersion statistics.CheckAnalyzeVerOnTable(statsTbl, &tableStatsVer) - execAutoAnalyze(sctx, statsHandle, tableStatsVer, sqlWithIdx, paramsWithIdx...) + execAutoAnalyze(sctx, statsHandle, sysProcTracker, tableStatsVer, sqlWithIdx, paramsWithIdx...) return true } } @@ -380,7 +388,8 @@ func TableAnalyzed(tbl *statistics.Table) bool { // It is very similar to tryAutoAnalyzeTable, but it commits the analyze job in batch for partitions. func tryAutoAnalyzePartitionTableInDynamicMode( sctx sessionctx.Context, - statsHandle statsutil.StatsHandle, + statsHandle statstypes.StatsHandle, + sysProcTracker sessionctx.SysProcTracker, tblInfo *model.TableInfo, partitionDefs []model.PartitionDefinition, db string, @@ -454,7 +463,7 @@ func tryAutoAnalyzePartitionTableInDynamicMode( zap.String("table", tblInfo.Name.String()), zap.Any("partitions", needAnalyzePartitionNames[start:end]), ) - execAutoAnalyze(sctx, statsHandle, tableStatsVer, sql, params...) + execAutoAnalyze(sctx, statsHandle, sysProcTracker, tableStatsVer, sql, params...) } return true @@ -492,7 +501,7 @@ func tryAutoAnalyzePartitionTableInDynamicMode( zap.String("index", idx.Name.String()), zap.Any("partitions", needAnalyzePartitionNames[start:end]), ) - execAutoAnalyze(sctx, statsHandle, tableStatsVer, sql, params...) + execAutoAnalyze(sctx, statsHandle, sysProcTracker, tableStatsVer, sql, params...) } return true @@ -510,13 +519,14 @@ var execOptionForAnalyze = map[int]sqlexec.OptionFuncAlias{ func execAutoAnalyze( sctx sessionctx.Context, - statsHandle statsutil.StatsHandle, + statsHandle statstypes.StatsHandle, + sysProcTracker sessionctx.SysProcTracker, statsVer int, sql string, params ...interface{}, ) { startTime := time.Now() - _, _, err := execAnalyzeStmt(sctx, statsHandle, statsVer, sql, params...) + _, _, err := execAnalyzeStmt(sctx, statsHandle, sysProcTracker, statsVer, sql, params...) dur := time.Since(startTime) metrics.AutoAnalyzeHistogram.Observe(dur.Seconds()) if err != nil { @@ -538,7 +548,8 @@ func execAutoAnalyze( func execAnalyzeStmt( sctx sessionctx.Context, - statsHandle statsutil.StatsHandle, + statsHandle statstypes.StatsHandle, + sysProcTracker sessionctx.SysProcTracker, statsVer int, sql string, params ...interface{}, @@ -550,7 +561,7 @@ func execAnalyzeStmt( sqlexec.GetAnalyzeSnapshotOption(analyzeSnapshot), sqlexec.GetPartitionPruneModeOption(pruneMode), sqlexec.ExecOptionUseCurSession, - sqlexec.ExecOptionWithSysProcTrack(statsHandle.AutoAnalyzeProcID(), statsHandle.SysProcTracker().Track, statsHandle.SysProcTracker().UnTrack), + sqlexec.ExecOptionWithSysProcTrack(statsHandle.AutoAnalyzeProcID(), sysProcTracker.Track, sysProcTracker.UnTrack), } return statsutil.ExecWithOpts(sctx, optFuncs, sql, params...) } diff --git a/pkg/statistics/handle/bootstrap.go b/pkg/statistics/handle/bootstrap.go index 83dd83d1a209a..0e3adc40d2c45 100644 --- a/pkg/statistics/handle/bootstrap.go +++ b/pkg/statistics/handle/bootstrap.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/cache" + statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -35,7 +36,7 @@ import ( "go.uber.org/zap" ) -func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, cache util.StatsCache, iter *chunk.Iterator4Chunk) { +func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, cache statstypes.StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { physicalID := row.GetInt64(1) // The table is read-only. Please do not modify it. @@ -62,7 +63,7 @@ func (h *Handle) initStatsMeta4Chunk(is infoschema.InfoSchema, cache util.StatsC } } -func (h *Handle) initStatsMeta(is infoschema.InfoSchema) (util.StatsCache, error) { +func (h *Handle) initStatsMeta(is infoschema.InfoSchema) (statstypes.StatsCache, error) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) sql := "select HIGH_PRIORITY version, table_id, modify_count, count from mysql.stats_meta" rc, err := util.Exec(h.initStatsCtx, sql) @@ -89,7 +90,7 @@ func (h *Handle) initStatsMeta(is infoschema.InfoSchema) (util.StatsCache, error return tables, nil } -func (h *Handle) initStatsHistograms4ChunkLite(is infoschema.InfoSchema, cache util.StatsCache, iter *chunk.Iterator4Chunk) { +func (h *Handle) initStatsHistograms4ChunkLite(is infoschema.InfoSchema, cache statstypes.StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { tblID := row.GetInt64(0) table, ok := cache.Get(tblID) @@ -160,7 +161,7 @@ func (h *Handle) initStatsHistograms4ChunkLite(is infoschema.InfoSchema, cache u } } -func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache util.StatsCache, iter *chunk.Iterator4Chunk) { +func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache statstypes.StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { tblID, statsVer := row.GetInt64(0), row.GetInt64(8) table, ok := cache.Get(tblID) @@ -230,7 +231,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache util. } } -func (h *Handle) initStatsHistogramsLite(is infoschema.InfoSchema, cache util.StatsCache) error { +func (h *Handle) initStatsHistogramsLite(is infoschema.InfoSchema, cache statstypes.StatsCache) error { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) sql := "select HIGH_PRIORITY table_id, is_index, hist_id, distinct_count, version, null_count, tot_col_size, stats_ver, correlation, flag, last_analyze_pos from mysql.stats_histograms" rc, err := util.Exec(h.initStatsCtx, sql) @@ -253,7 +254,7 @@ func (h *Handle) initStatsHistogramsLite(is infoschema.InfoSchema, cache util.St return nil } -func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, cache util.StatsCache) error { +func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, cache statstypes.StatsCache) error { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) sql := "select HIGH_PRIORITY table_id, is_index, hist_id, distinct_count, version, null_count, cm_sketch, tot_col_size, stats_ver, correlation, flag, last_analyze_pos from mysql.stats_histograms" rc, err := util.Exec(h.initStatsCtx, sql) @@ -276,7 +277,7 @@ func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, cache util.StatsC return nil } -func (*Handle) initStatsTopN4Chunk(cache util.StatsCache, iter *chunk.Iterator4Chunk) { +func (*Handle) initStatsTopN4Chunk(cache statstypes.StatsCache, iter *chunk.Iterator4Chunk) { affectedIndexes := make(map[*statistics.Index]struct{}) for row := iter.Begin(); row != iter.End(); row = iter.Next() { table, ok := cache.Get(row.GetInt64(0)) @@ -302,7 +303,7 @@ func (*Handle) initStatsTopN4Chunk(cache util.StatsCache, iter *chunk.Iterator4C } } -func (h *Handle) initStatsTopN(cache util.StatsCache) error { +func (h *Handle) initStatsTopN(cache statstypes.StatsCache) error { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) sql := "select HIGH_PRIORITY table_id, hist_id, value, count from mysql.stats_top_n where is_index = 1" rc, err := util.Exec(h.initStatsCtx, sql) @@ -325,7 +326,7 @@ func (h *Handle) initStatsTopN(cache util.StatsCache) error { return nil } -func (*Handle) initStatsFMSketch4Chunk(cache util.StatsCache, iter *chunk.Iterator4Chunk) { +func (*Handle) initStatsFMSketch4Chunk(cache statstypes.StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { table, ok := cache.Get(row.GetInt64(0)) if !ok { @@ -352,7 +353,7 @@ func (*Handle) initStatsFMSketch4Chunk(cache util.StatsCache, iter *chunk.Iterat } } -func (h *Handle) initStatsFMSketch(cache util.StatsCache) error { +func (h *Handle) initStatsFMSketch(cache statstypes.StatsCache) error { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) sql := "select HIGH_PRIORITY table_id, is_index, hist_id, value from mysql.stats_fm_sketch" rc, err := util.Exec(h.initStatsCtx, sql) @@ -375,7 +376,7 @@ func (h *Handle) initStatsFMSketch(cache util.StatsCache) error { return nil } -func (*Handle) initStatsBuckets4Chunk(cache util.StatsCache, iter *chunk.Iterator4Chunk) { +func (*Handle) initStatsBuckets4Chunk(cache statstypes.StatsCache, iter *chunk.Iterator4Chunk) { for row := iter.Begin(); row != iter.End(); row = iter.Next() { tableID, isIndex, histID := row.GetInt64(0), row.GetInt64(1), row.GetInt64(2) table, ok := cache.Get(tableID) @@ -426,7 +427,7 @@ func (*Handle) initStatsBuckets4Chunk(cache util.StatsCache, iter *chunk.Iterato } } -func (h *Handle) initStatsBuckets(cache util.StatsCache) error { +func (h *Handle) initStatsBuckets(cache statstypes.StatsCache) error { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) sql := "select HIGH_PRIORITY table_id, is_index, hist_id, count, repeats, lower_bound, upper_bound, ndv from mysql.stats_buckets order by table_id, is_index, hist_id, bucket_id" rc, err := util.Exec(h.initStatsCtx, sql) diff --git a/pkg/statistics/handle/cache/BUILD.bazel b/pkg/statistics/handle/cache/BUILD.bazel index 587c7eda09d1c..a9d1c1fed8ed8 100644 --- a/pkg/statistics/handle/cache/BUILD.bazel +++ b/pkg/statistics/handle/cache/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "//pkg/statistics/handle/cache/internal/metrics", "//pkg/statistics/handle/logutil", "//pkg/statistics/handle/metrics", + "//pkg/statistics/handle/types", "//pkg/statistics/handle/util", "//pkg/types", "//pkg/util/chunk", @@ -41,7 +42,7 @@ go_test( "//pkg/config", "//pkg/statistics", "//pkg/statistics/handle/cache/internal/testutil", - "//pkg/statistics/handle/util", + "//pkg/statistics/handle/types", "//pkg/util/benchdaily", ], ) diff --git a/pkg/statistics/handle/cache/bench_test.go b/pkg/statistics/handle/cache/bench_test.go index 4c359da6816ac..7990a1f8a47fa 100644 --- a/pkg/statistics/handle/cache/bench_test.go +++ b/pkg/statistics/handle/cache/bench_test.go @@ -22,11 +22,11 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/cache/internal/testutil" - "github.com/pingcap/tidb/pkg/statistics/handle/util" + "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/util/benchdaily" ) -func benchCopyAndUpdate(b *testing.B, c util.StatsCache) { +func benchCopyAndUpdate(b *testing.B, c types.StatsCache) { var wg sync.WaitGroup b.ResetTimer() for i := 0; i < b.N; i++ { @@ -42,7 +42,7 @@ func benchCopyAndUpdate(b *testing.B, c util.StatsCache) { b.StopTimer() } -func benchPutGet(b *testing.B, c util.StatsCache) { +func benchPutGet(b *testing.B, c types.StatsCache) { var wg sync.WaitGroup b.ResetTimer() for i := 0; i < b.N; i++ { @@ -65,7 +65,7 @@ func benchPutGet(b *testing.B, c util.StatsCache) { b.StopTimer() } -func benchGet(b *testing.B, c util.StatsCache) { +func benchGet(b *testing.B, c types.StatsCache) { var w sync.WaitGroup for i := 0; i < b.N; i++ { w.Add(1) diff --git a/pkg/statistics/handle/cache/statscache.go b/pkg/statistics/handle/cache/statscache.go index 028d0d9525fe5..e09ff2df107d8 100644 --- a/pkg/statistics/handle/cache/statscache.go +++ b/pkg/statistics/handle/cache/statscache.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/statistics/handle/cache/internal/metrics" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" handle_metrics "github.com/pingcap/tidb/pkg/statistics/handle/metrics" + "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/logutil" @@ -35,11 +36,11 @@ import ( type StatsCacheImpl struct { atomic.Pointer[StatsCache] - statsHandle util.StatsHandle + statsHandle types.StatsHandle } // NewStatsCacheImpl creates a new StatsCache. -func NewStatsCacheImpl(statsHandle util.StatsHandle) (util.StatsCache, error) { +func NewStatsCacheImpl(statsHandle types.StatsHandle) (types.StatsCache, error) { newCache, err := NewStatsCache() if err != nil { return nil, err @@ -52,7 +53,7 @@ func NewStatsCacheImpl(statsHandle util.StatsHandle) (util.StatsCache, error) { } // NewStatsCacheImplForTest creates a new StatsCache for test. -func NewStatsCacheImplForTest() (util.StatsCache, error) { +func NewStatsCacheImplForTest() (types.StatsCache, error) { return NewStatsCacheImpl(nil) } @@ -119,7 +120,7 @@ func (s *StatsCacheImpl) Update(is infoschema.InfoSchema) error { } // Replace replaces this cache. -func (s *StatsCacheImpl) Replace(cache util.StatsCache) { +func (s *StatsCacheImpl) Replace(cache types.StatsCache) { x := cache.(*StatsCacheImpl) s.replace(x.Load()) } diff --git a/pkg/statistics/handle/ddl/BUILD.bazel b/pkg/statistics/handle/ddl/BUILD.bazel index 72998348ef78d..067d58c44f5ed 100644 --- a/pkg/statistics/handle/ddl/BUILD.bazel +++ b/pkg/statistics/handle/ddl/BUILD.bazel @@ -8,6 +8,7 @@ go_library( deps = [ "//pkg/parser/model", "//pkg/sessionctx/variable", + "//pkg/statistics/handle/types", "//pkg/statistics/handle/util", ], ) diff --git a/pkg/statistics/handle/ddl/ddl.go b/pkg/statistics/handle/ddl/ddl.go index 4ddf7cacce491..e4657614a5219 100644 --- a/pkg/statistics/handle/ddl/ddl.go +++ b/pkg/statistics/handle/ddl/ddl.go @@ -17,22 +17,23 @@ package ddl import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" ) type ddlHandlerImpl struct { ddlEventCh chan *util.DDLEvent - statsWriter util.StatsReadWriter - statsHandler util.StatsHandle - globalStatsHandler util.StatsGlobal + statsWriter types.StatsReadWriter + statsHandler types.StatsHandle + globalStatsHandler types.StatsGlobal } // NewDDLHandler creates a new ddl handler. func NewDDLHandler( - statsWriter util.StatsReadWriter, - statsHandler util.StatsHandle, - globalStatsHandler util.StatsGlobal, -) util.DDL { + statsWriter types.StatsReadWriter, + statsHandler types.StatsHandle, + globalStatsHandler types.StatsGlobal, +) types.DDL { return &ddlHandlerImpl{ ddlEventCh: make(chan *util.DDLEvent, 1000), statsWriter: statsWriter, @@ -115,7 +116,7 @@ func (h *ddlHandlerImpl) HandleDDLEvent(t *util.DDLEvent) error { } } case model.ActionDropTablePartition: - pruneMode, err := h.statsHandler.GetCurrentPruneMode() + pruneMode, err := util.GetCurrentPruneMode(h.statsHandler.SPool()) if err != nil { return err } @@ -175,7 +176,7 @@ func (h *ddlHandlerImpl) getInitStateTableIDs(tblInfo *model.TableInfo) (ids []i for _, def := range pi.Definitions { ids = append(ids, def.ID) } - pruneMode, err := h.statsHandler.GetCurrentPruneMode() + pruneMode, err := util.GetCurrentPruneMode(h.statsHandler.SPool()) if err != nil { return nil, err } diff --git a/pkg/statistics/handle/globalstats/BUILD.bazel b/pkg/statistics/handle/globalstats/BUILD.bazel index 23de43713d579..81f237df5ae2c 100644 --- a/pkg/statistics/handle/globalstats/BUILD.bazel +++ b/pkg/statistics/handle/globalstats/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/statistics", "//pkg/statistics/handle/logutil", "//pkg/statistics/handle/storage", + "//pkg/statistics/handle/types", "//pkg/statistics/handle/util", "//pkg/types", "//pkg/util/hack", diff --git a/pkg/statistics/handle/globalstats/global_stats.go b/pkg/statistics/handle/globalstats/global_stats.go index c769daff4f293..af2eaf52d745f 100644 --- a/pkg/statistics/handle/globalstats/global_stats.go +++ b/pkg/statistics/handle/globalstats/global_stats.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/statistics" + statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/logutil" @@ -38,11 +39,11 @@ const ( // statsGlobalImpl implements util.StatsGlobal type statsGlobalImpl struct { - statsHandler util.StatsHandle + statsHandler statstypes.StatsHandle } // NewStatsGlobal creates a new StatsGlobal. -func NewStatsGlobal(statsHandler util.StatsHandle) util.StatsGlobal { +func NewStatsGlobal(statsHandler statstypes.StatsHandle) statstypes.StatsGlobal { return &statsGlobalImpl{statsHandler: statsHandler} } @@ -95,7 +96,7 @@ func newGlobalStats(histCount int) *GlobalStats { // MergePartitionStats2GlobalStats merge the partition-level stats to global-level stats based on the tableInfo. func MergePartitionStats2GlobalStats( sc sessionctx.Context, - statsHandle util.StatsHandle, + statsHandle statstypes.StatsHandle, opts map[ast.AnalyzeOptionType]uint64, is infoschema.InfoSchema, globalTableInfo *model.TableInfo, @@ -119,7 +120,7 @@ func MergePartitionStats2GlobalStats( // MergePartitionStats2GlobalStatsByTableID merge the partition-level stats to global-level stats based on the tableID. func MergePartitionStats2GlobalStatsByTableID( sc sessionctx.Context, - statsHandle util.StatsHandle, + statsHandle statstypes.StatsHandle, opts map[ast.AnalyzeOptionType]uint64, is infoschema.InfoSchema, tableID int64, @@ -172,7 +173,7 @@ var analyzeOptionDefault = map[ast.AnalyzeOptionType]uint64{ // UpdateGlobalStats update the global-level stats based on the partition-level stats. func UpdateGlobalStats( sctx sessionctx.Context, - statsHandle util.StatsHandle, + statsHandle statstypes.StatsHandle, tblInfo *model.TableInfo) error { tableID := tblInfo.ID is := sessiontxn.GetTxnManager(sctx).GetTxnInfoSchema() @@ -281,7 +282,7 @@ func blockingMergePartitionStats2GlobalStats( isIndex bool, histIDs []int64, allPartitionStats map[int64]*statistics.Table, - statsHandle util.StatsHandle, + statsHandle statstypes.StatsHandle, ) (globalStats *GlobalStats, err error) { externalCache := false if allPartitionStats != nil { diff --git a/pkg/statistics/handle/globalstats/global_stats_async.go b/pkg/statistics/handle/globalstats/global_stats_async.go index 584209e3f5d9f..0ecfd66463866 100644 --- a/pkg/statistics/handle/globalstats/global_stats_async.go +++ b/pkg/statistics/handle/globalstats/global_stats_async.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/statistics" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" "github.com/pingcap/tidb/pkg/statistics/handle/storage" + statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" "go.uber.org/zap" @@ -72,7 +73,7 @@ func toSQLIndex(isIndex bool) int { // └────────────────────────┘ └───────────────────────┘ type AsyncMergePartitionStats2GlobalStats struct { is infoschema.InfoSchema - statsHandle util.StatsHandle + statsHandle statstypes.StatsHandle globalStats *GlobalStats cmsketch chan mergeItem[*statistics.CMSketch] fmsketch chan mergeItem[*statistics.FMSketch] @@ -96,7 +97,7 @@ type AsyncMergePartitionStats2GlobalStats struct { // NewAsyncMergePartitionStats2GlobalStats creates a new AsyncMergePartitionStats2GlobalStats. func NewAsyncMergePartitionStats2GlobalStats( - statsHandle util.StatsHandle, + statsHandle statstypes.StatsHandle, globalTableInfo *model.TableInfo, histIDs []int64, is infoschema.InfoSchema) (*AsyncMergePartitionStats2GlobalStats, error) { diff --git a/pkg/statistics/handle/handle.go b/pkg/statistics/handle/handle.go index 608d4be32ea33..2b7e3044becb6 100644 --- a/pkg/statistics/handle/handle.go +++ b/pkg/statistics/handle/handle.go @@ -15,7 +15,6 @@ package handle import ( - "math" "time" "github.com/pingcap/tidb/pkg/parser/model" @@ -30,10 +29,9 @@ import ( statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" "github.com/pingcap/tidb/pkg/statistics/handle/storage" "github.com/pingcap/tidb/pkg/statistics/handle/syncload" + "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/usage" "github.com/pingcap/tidb/pkg/statistics/handle/util" - "github.com/tiancaiamao/gp" - atomic2 "go.uber.org/atomic" "go.uber.org/zap" ) @@ -46,56 +44,52 @@ const ( // Handle can update stats info periodically. type Handle struct { - pool util.SessionPool + // Pool is used to get a session or a goroutine to execute stats updating. + util.Pool + + // AutoAnalyzeProcIDGenerator is used to generate auto analyze proc ID. + util.AutoAnalyzeProcIDGenerator + + // LeaseGetter is used to get stats lease. + util.LeaseGetter // initStatsCtx is the ctx only used for initStats initStatsCtx sessionctx.Context - // sysProcTracker is used to track sys process like analyze - sysProcTracker sessionctx.SysProcTracker - // TableInfoGetter is used to fetch table meta info. util.TableInfoGetter // StatsGC is used to GC stats. - util.StatsGC + types.StatsGC // StatsUsage is used to track the usage of column / index statistics. - util.StatsUsage + types.StatsUsage // StatsHistory is used to manage historical stats. - util.StatsHistory + types.StatsHistory // StatsAnalyze is used to handle auto-analyze and manage analyze jobs. - util.StatsAnalyze + types.StatsAnalyze // StatsSyncLoad is used to load stats syncly. - util.StatsSyncLoad + types.StatsSyncLoad // StatsReadWriter is used to read/write stats from/to storage. - util.StatsReadWriter + types.StatsReadWriter // StatsLock is used to manage locked stats. - util.StatsLock + types.StatsLock // StatsGlobal is used to manage global stats. - util.StatsGlobal + types.StatsGlobal // DDL is used to handle ddl events. - util.DDL - - // This gpool is used to reuse goroutine in the mergeGlobalStatsTopN. - gpool *gp.Pool - - // autoAnalyzeProcIDGetter is used to generate auto analyze ID. - autoAnalyzeProcIDGetter func() uint64 + types.DDL InitStatsDone chan struct{} // StatsCache ... - util.StatsCache - - lease atomic2.Duration + types.StatsCache } // Clear the statsCache, only for test. @@ -108,45 +102,40 @@ func (h *Handle) Clear() { } // NewHandle creates a Handle for update stats. -func NewHandle(_, initStatsCtx sessionctx.Context, lease time.Duration, pool util.SessionPool, tracker sessionctx.SysProcTracker, autoAnalyzeProcIDGetter func() uint64) (*Handle, error) { +func NewHandle( + _, /* ctx, keep it for feature usage */ + initStatsCtx sessionctx.Context, + lease time.Duration, + pool util.SessionPool, + tracker sessionctx.SysProcTracker, + autoAnalyzeProcIDGetter func() uint64, +) (*Handle, error) { handle := &Handle{ - gpool: gp.New(math.MaxInt16, time.Minute), - pool: pool, - sysProcTracker: tracker, - autoAnalyzeProcIDGetter: autoAnalyzeProcIDGetter, - InitStatsDone: make(chan struct{}), - TableInfoGetter: util.NewTableInfoGetter(), - StatsLock: lockstats.NewStatsLock(pool), + InitStatsDone: make(chan struct{}), + TableInfoGetter: util.NewTableInfoGetter(), + StatsLock: lockstats.NewStatsLock(pool), } handle.StatsGC = storage.NewStatsGC(handle) handle.StatsReadWriter = storage.NewStatsReadWriter(handle) handle.initStatsCtx = initStatsCtx - handle.lease.Store(lease) statsCache, err := cache.NewStatsCacheImpl(handle) if err != nil { return nil, err } + handle.Pool = util.NewPool(pool) + handle.AutoAnalyzeProcIDGenerator = util.NewGenerator(autoAnalyzeProcIDGetter) + handle.LeaseGetter = util.NewLeaseGetter(lease) handle.StatsCache = statsCache handle.StatsHistory = history.NewStatsHistory(handle) handle.StatsUsage = usage.NewStatsUsageImpl(handle) - handle.StatsAnalyze = autoanalyze.NewStatsAnalyze(handle) + handle.StatsAnalyze = autoanalyze.NewStatsAnalyze(handle, tracker) handle.StatsSyncLoad = syncload.NewStatsSyncLoad(handle) handle.StatsGlobal = globalstats.NewStatsGlobal(handle) handle.DDL = ddl.NewDDLHandler(handle.StatsReadWriter, handle, handle.StatsGlobal) return handle, nil } -// Lease returns the stats lease. -func (h *Handle) Lease() time.Duration { - return h.lease.Load() -} - -// SetLease sets the stats lease. -func (h *Handle) SetLease(lease time.Duration) { - h.lease.Store(lease) -} - // GetTableStats retrieves the statistics table from cache, and the cache will be updated by a goroutine. // TODO: remove GetTableStats later on. func (h *Handle) GetTableStats(tblInfo *model.TableInfo) *statistics.Table { @@ -154,7 +143,7 @@ func (h *Handle) GetTableStats(tblInfo *model.TableInfo) *statistics.Table { } // GetPartitionStats retrieves the partition stats from cache. -// TODO: remove GetPartitionStats later on. +// TODO: remove GetTableStats later on. func (h *Handle) GetPartitionStats(tblInfo *model.TableInfo, pid int64) *statistics.Table { var tbl *statistics.Table if h == nil { @@ -189,35 +178,6 @@ func (h *Handle) FlushStats() { // Close stops the background func (h *Handle) Close() { - h.gpool.Close() + h.Pool.Close() h.StatsCache.Close() } - -// GetCurrentPruneMode returns the current latest partitioning table prune mode. -func (h *Handle) GetCurrentPruneMode() (mode string, err error) { - err = util.CallWithSCtx(h.pool, func(sctx sessionctx.Context) error { - mode = sctx.GetSessionVars().PartitionPruneMode.Load() - return nil - }) - return -} - -// GPool returns the goroutine pool of handle. -func (h *Handle) GPool() *gp.Pool { - return h.gpool -} - -// SPool returns the session pool. -func (h *Handle) SPool() util.SessionPool { - return h.pool -} - -// SysProcTracker is used to track sys process like analyze -func (h *Handle) SysProcTracker() sessionctx.SysProcTracker { - return h.sysProcTracker -} - -// AutoAnalyzeProcID generates an analyze ID. -func (h *Handle) AutoAnalyzeProcID() uint64 { - return h.autoAnalyzeProcIDGetter() -} diff --git a/pkg/statistics/handle/history/BUILD.bazel b/pkg/statistics/handle/history/BUILD.bazel index 6b6f2ab12aee9..ab62dd729d37a 100644 --- a/pkg/statistics/handle/history/BUILD.bazel +++ b/pkg/statistics/handle/history/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "//pkg/sessionctx", "//pkg/statistics/handle/cache", "//pkg/statistics/handle/storage", + "//pkg/statistics/handle/types", "//pkg/statistics/handle/util", "//pkg/util/logutil", "@com_github_pingcap_errors//:errors", diff --git a/pkg/statistics/handle/history/history_stats.go b/pkg/statistics/handle/history/history_stats.go index 690a265e05eff..9acebeb010d8e 100644 --- a/pkg/statistics/handle/history/history_stats.go +++ b/pkg/statistics/handle/history/history_stats.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics/handle/cache" "github.com/pingcap/tidb/pkg/statistics/handle/storage" + "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/zap" @@ -29,12 +30,12 @@ import ( // statsHistoryImpl implements util.StatsHistory. type statsHistoryImpl struct { - statsHandle util.StatsHandle + statsHandle types.StatsHandle } // NewStatsHistory creates a new StatsHistory. -func NewStatsHistory(statsHandle util.StatsHandle, -) util.StatsHistory { +func NewStatsHistory(statsHandle types.StatsHandle, +) types.StatsHistory { return &statsHistoryImpl{ statsHandle: statsHandle, } diff --git a/pkg/statistics/handle/lockstats/BUILD.bazel b/pkg/statistics/handle/lockstats/BUILD.bazel index c104b9bb79a5e..986c1c43e31e3 100644 --- a/pkg/statistics/handle/lockstats/BUILD.bazel +++ b/pkg/statistics/handle/lockstats/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "//pkg/sessionctx", "//pkg/statistics/handle/cache", "//pkg/statistics/handle/logutil", + "//pkg/statistics/handle/types", "//pkg/statistics/handle/util", "//pkg/util/sqlexec", "@com_github_pingcap_errors//:errors", @@ -36,6 +37,7 @@ go_test( "//pkg/kv", "//pkg/parser/mysql", "//pkg/sessionctx", + "//pkg/statistics/handle/types", "//pkg/statistics/handle/util", "//pkg/testkit/testsetup", "//pkg/types", diff --git a/pkg/statistics/handle/lockstats/lock_stats.go b/pkg/statistics/handle/lockstats/lock_stats.go index e120417706e2e..20c78784efcba 100644 --- a/pkg/statistics/handle/lockstats/lock_stats.go +++ b/pkg/statistics/handle/lockstats/lock_stats.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics/handle/logutil" + "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/util/sqlexec" "go.uber.org/zap" @@ -41,14 +42,14 @@ type statsLockImpl struct { } // NewStatsLock creates a new StatsLock. -func NewStatsLock(pool util.SessionPool) util.StatsLock { +func NewStatsLock(pool util.SessionPool) types.StatsLock { return &statsLockImpl{pool: pool} } // LockTables add locked tables id to store. // - tables: tables that will be locked. // Return the message of skipped tables and error. -func (sl *statsLockImpl) LockTables(tables map[int64]*util.StatsLockTable) (skipped string, err error) { +func (sl *statsLockImpl) LockTables(tables map[int64]*types.StatsLockTable) (skipped string, err error) { err = util.CallWithSCtx(sl.pool, func(sctx sessionctx.Context) error { skipped, err = AddLockedTables(sctx, tables) return err @@ -78,7 +79,7 @@ func (sl *statsLockImpl) LockPartitions( // RemoveLockedTables remove tables from table locked records. // - tables: tables of which will be unlocked. // Return the message of skipped tables and error. -func (sl *statsLockImpl) RemoveLockedTables(tables map[int64]*util.StatsLockTable) (skipped string, err error) { +func (sl *statsLockImpl) RemoveLockedTables(tables map[int64]*types.StatsLockTable) (skipped string, err error) { err = util.CallWithSCtx(sl.pool, func(sctx sessionctx.Context) error { skipped, err = RemoveLockedTables(sctx, tables) return err @@ -139,7 +140,7 @@ var ( // Return the message of skipped tables and error. func AddLockedTables( sctx sessionctx.Context, - tables map[int64]*util.StatsLockTable, + tables map[int64]*types.StatsLockTable, ) (string, error) { // Load tables to check duplicate before insert. lockedTables, err := QueryLockedTables(sctx) diff --git a/pkg/statistics/handle/lockstats/lock_stats_test.go b/pkg/statistics/handle/lockstats/lock_stats_test.go index 2433ecab9e257..64932cf625c33 100644 --- a/pkg/statistics/handle/lockstats/lock_stats_test.go +++ b/pkg/statistics/handle/lockstats/lock_stats_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/mysql" + statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -211,7 +212,7 @@ func TestAddLockedTables(t *testing.T) { gomock.Eq([]interface{}{int64(4), int64(4)}), ) - tables := map[int64]*util.StatsLockTable{ + tables := map[int64]*statstypes.StatsLockTable{ 1: { FullName: "test.t1", PartitionInfo: map[int64]string{ diff --git a/pkg/statistics/handle/lockstats/unlock_stats.go b/pkg/statistics/handle/lockstats/unlock_stats.go index 18cb49b591785..21d422d66d691 100644 --- a/pkg/statistics/handle/lockstats/unlock_stats.go +++ b/pkg/statistics/handle/lockstats/unlock_stats.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/statistics/handle/cache" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" + "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "go.uber.org/zap" ) @@ -37,7 +38,7 @@ const ( // Return the message of skipped tables and error. func RemoveLockedTables( sctx sessionctx.Context, - tables map[int64]*util.StatsLockTable, + tables map[int64]*types.StatsLockTable, ) (string, error) { // Load tables to check locked before delete. lockedTables, err := QueryLockedTables(sctx) diff --git a/pkg/statistics/handle/lockstats/unlock_stats_test.go b/pkg/statistics/handle/lockstats/unlock_stats_test.go index 750e935d1d429..62dfce7352bcf 100644 --- a/pkg/statistics/handle/lockstats/unlock_stats_test.go +++ b/pkg/statistics/handle/lockstats/unlock_stats_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" + stststypes "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -243,7 +244,7 @@ func TestRemoveLockedTables(t *testing.T) { gomock.Eq([]interface{}{int64(4)}), ).Return(nil, nil, nil) - tables := map[int64]*util.StatsLockTable{ + tables := map[int64]*stststypes.StatsLockTable{ 1: { FullName: "test.t1", PartitionInfo: map[int64]string{ diff --git a/pkg/statistics/handle/storage/BUILD.bazel b/pkg/statistics/handle/storage/BUILD.bazel index dcae7525d1172..def09ccf1bf7d 100644 --- a/pkg/statistics/handle/storage/BUILD.bazel +++ b/pkg/statistics/handle/storage/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "//pkg/statistics/handle/lockstats", "//pkg/statistics/handle/logutil", "//pkg/statistics/handle/metrics", + "//pkg/statistics/handle/types", "//pkg/statistics/handle/util", "//pkg/types", "//pkg/util/chunk", diff --git a/pkg/statistics/handle/storage/gc.go b/pkg/statistics/handle/storage/gc.go index 155830c59af34..51664008fd11a 100644 --- a/pkg/statistics/handle/storage/gc.go +++ b/pkg/statistics/handle/storage/gc.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/cache" "github.com/pingcap/tidb/pkg/statistics/handle/lockstats" + "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/logutil" @@ -39,11 +40,11 @@ import ( // statsGCImpl implements StatsGC interface. type statsGCImpl struct { - statsHandle util.StatsHandle + statsHandle types.StatsHandle } // NewStatsGC creates a new StatsGC. -func NewStatsGC(statsHandle util.StatsHandle) util.StatsGC { +func NewStatsGC(statsHandle types.StatsHandle) types.StatsGC { return &statsGCImpl{ statsHandle: statsHandle, } @@ -76,7 +77,7 @@ func (gc *statsGCImpl) DeleteTableStatsFromKV(statsIDs []int64) (err error) { // For dropped tables, we will first update their version // so that other tidb could know that table is deleted. func GCStats(sctx sessionctx.Context, - statsHandle util.StatsHandle, + statsHandle types.StatsHandle, is infoschema.InfoSchema, ddlLease time.Duration) (err error) { // To make sure that all the deleted tables' schema and stats info have been acknowledged to all tidb, // we only garbage collect version before 10 lease. @@ -263,7 +264,7 @@ func removeDeletedExtendedStats(sctx sessionctx.Context, version uint64) (err er // gcTableStats GC this table's stats. func gcTableStats(sctx sessionctx.Context, - statsHandler util.StatsHandle, + statsHandler types.StatsHandle, is infoschema.InfoSchema, physicalID int64) error { rows, _, err := util.ExecRows(sctx, "select is_index, hist_id from mysql.stats_histograms where table_id = %?", physicalID) if err != nil { @@ -384,7 +385,7 @@ func writeGCTimestampToKV(sctx sessionctx.Context, newTS uint64) error { // MarkExtendedStatsDeleted update the status of mysql.stats_extended to be `deleted` and the version of mysql.stats_meta. func MarkExtendedStatsDeleted(sctx sessionctx.Context, - statsCache util.StatsCache, + statsCache types.StatsCache, statsName string, tableID int64, ifExists bool) (statsVer uint64, err error) { rows, _, err := util.ExecRows(sctx, "SELECT name FROM mysql.stats_extended WHERE name = %? and table_id = %? and status in (%?, %?)", statsName, tableID, statistics.ExtendedStatsInited, statistics.ExtendedStatsAnalyzed) if err != nil { diff --git a/pkg/statistics/handle/storage/read.go b/pkg/statistics/handle/storage/read.go index 0f2e88889509c..531b8872d3c66 100644 --- a/pkg/statistics/handle/storage/read.go +++ b/pkg/statistics/handle/storage/read.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" statslogutil "github.com/pingcap/tidb/pkg/statistics/handle/logutil" + statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" @@ -492,7 +493,7 @@ func LoadHistogram(sctx sessionctx.Context, tableID int64, isIndex int, histID i } // LoadNeededHistograms will load histograms for those needed columns/indices. -func LoadNeededHistograms(sctx sessionctx.Context, statsCache util.StatsCache, loadFMSketch bool) (err error) { +func LoadNeededHistograms(sctx sessionctx.Context, statsCache statstypes.StatsCache, loadFMSketch bool) (err error) { items := statistics.HistogramNeededItems.AllItems() for _, item := range items { if !item.IsIndex { @@ -507,7 +508,7 @@ func LoadNeededHistograms(sctx sessionctx.Context, statsCache util.StatsCache, l return nil } -func loadNeededColumnHistograms(sctx sessionctx.Context, statsCache util.StatsCache, col model.TableItemID, loadFMSketch bool) (err error) { +func loadNeededColumnHistograms(sctx sessionctx.Context, statsCache statstypes.StatsCache, col model.TableItemID, loadFMSketch bool) (err error) { tbl, ok := statsCache.Get(col.TableID) if !ok { return nil @@ -567,7 +568,7 @@ func loadNeededColumnHistograms(sctx sessionctx.Context, statsCache util.StatsCa return nil } -func loadNeededIndexHistograms(sctx sessionctx.Context, statsCache util.StatsCache, idx model.TableItemID, loadFMSketch bool) (err error) { +func loadNeededIndexHistograms(sctx sessionctx.Context, statsCache statstypes.StatsCache, idx model.TableItemID, loadFMSketch bool) (err error) { tbl, ok := statsCache.Get(idx.TableID) if !ok { return nil diff --git a/pkg/statistics/handle/storage/stats_read_writer.go b/pkg/statistics/handle/storage/stats_read_writer.go index b187b7b1e557b..596fc64495379 100644 --- a/pkg/statistics/handle/storage/stats_read_writer.go +++ b/pkg/statistics/handle/storage/stats_read_writer.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" handle_metrics "github.com/pingcap/tidb/pkg/statistics/handle/metrics" + statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/intest" @@ -39,11 +40,11 @@ import ( // statsReadWriter implements the util.StatsReadWriter interface. type statsReadWriter struct { - statsHandler util.StatsHandle + statsHandler statstypes.StatsHandle } // NewStatsReadWriter creates a new StatsReadWriter. -func NewStatsReadWriter(statsHandler util.StatsHandle) util.StatsReadWriter { +func NewStatsReadWriter(statsHandler statstypes.StatsHandle) statstypes.StatsReadWriter { return &statsReadWriter{statsHandler: statsHandler} } @@ -415,7 +416,7 @@ func (s *statsReadWriter) DumpHistoricalStatsBySnapshot( // DumpStatsToJSONBySnapshot dumps statistic to json. func (s *statsReadWriter) DumpStatsToJSONBySnapshot(dbName string, tableInfo *model.TableInfo, snapshot uint64, dumpPartitionStats bool) (*util.JSONTable, error) { - pruneMode, err := s.statsHandler.GetCurrentPruneMode() + pruneMode, err := util.GetCurrentPruneMode(s.statsHandler.SPool()) if err != nil { return nil, err } diff --git a/pkg/statistics/handle/storage/update.go b/pkg/statistics/handle/storage/update.go index bc304ff596e90..7ff82eade236b 100644 --- a/pkg/statistics/handle/storage/update.go +++ b/pkg/statistics/handle/storage/update.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/cache" + "github.com/pingcap/tidb/pkg/statistics/handle/types" statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" ) @@ -102,7 +103,7 @@ func DumpTableStatColSizeToKV(sctx sessionctx.Context, id int64, delta variable. // InsertExtendedStats inserts a record into mysql.stats_extended and update version in mysql.stats_meta. func InsertExtendedStats(sctx sessionctx.Context, - statsCache statsutil.StatsCache, + statsCache types.StatsCache, statsName string, colIDs []int64, tp int, tableID int64, ifNotExists bool) (statsVer uint64, err error) { slices.Sort(colIDs) bytes, err := json.Marshal(colIDs) @@ -194,7 +195,7 @@ func SaveExtendedStatsToStorage(sctx sessionctx.Context, return statsVer, nil } -func removeExtendedStatsItem(statsCache statsutil.StatsCache, +func removeExtendedStatsItem(statsCache types.StatsCache, tableID int64, statsName string) { tbl, ok := statsCache.Get(tableID) if !ok || tbl.ExtendedStats == nil || len(tbl.ExtendedStats.Stats) == 0 { diff --git a/pkg/statistics/handle/syncload/BUILD.bazel b/pkg/statistics/handle/syncload/BUILD.bazel index 910011c8f6f00..901964873fda2 100644 --- a/pkg/statistics/handle/syncload/BUILD.bazel +++ b/pkg/statistics/handle/syncload/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/sessionctx/stmtctx", "//pkg/statistics", "//pkg/statistics/handle/storage", + "//pkg/statistics/handle/types", "//pkg/statistics/handle/util", "//pkg/types", "//pkg/util", diff --git a/pkg/statistics/handle/syncload/stats_syncload.go b/pkg/statistics/handle/syncload/stats_syncload.go index b071cdac5523b..9e98c5f9e5829 100644 --- a/pkg/statistics/handle/syncload/stats_syncload.go +++ b/pkg/statistics/handle/syncload/stats_syncload.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/storage" + statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" utilstats "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" @@ -36,17 +37,17 @@ import ( ) type statsSyncLoad struct { - statsHandle utilstats.StatsHandle - StatsLoad utilstats.StatsLoad + statsHandle statstypes.StatsHandle + StatsLoad statstypes.StatsLoad } // NewStatsSyncLoad creates a new StatsSyncLoad. -func NewStatsSyncLoad(statsHandle utilstats.StatsHandle) utilstats.StatsSyncLoad { +func NewStatsSyncLoad(statsHandle statstypes.StatsHandle) statstypes.StatsSyncLoad { s := &statsSyncLoad{statsHandle: statsHandle} cfg := config.GetGlobalConfig() s.StatsLoad.SubCtxs = make([]sessionctx.Context, cfg.Performance.StatsLoadConcurrency) - s.StatsLoad.NeededItemsCh = make(chan *utilstats.NeededItemTask, cfg.Performance.StatsLoadQueueSize) - s.StatsLoad.TimeoutItemsCh = make(chan *utilstats.NeededItemTask, cfg.Performance.StatsLoadQueueSize) + s.StatsLoad.NeededItemsCh = make(chan *statstypes.NeededItemTask, cfg.Performance.StatsLoadQueueSize) + s.StatsLoad.TimeoutItemsCh = make(chan *statstypes.NeededItemTask, cfg.Performance.StatsLoadQueueSize) s.StatsLoad.WorkingColMap = map[model.TableItemID][]chan stmtctx.StatsLoadResult{} return s } @@ -81,9 +82,9 @@ func (s *statsSyncLoad) SendLoadRequests(sc *stmtctx.StatementContext, neededHis sc.StatsLoad.Timeout = timeout sc.StatsLoad.NeededItems = remainedItems sc.StatsLoad.ResultCh = make(chan stmtctx.StatsLoadResult, len(remainedItems)) - tasks := make([]*utilstats.NeededItemTask, 0) + tasks := make([]*statstypes.NeededItemTask, 0) for _, item := range remainedItems { - task := &utilstats.NeededItemTask{ + task := &statstypes.NeededItemTask{ TableItemID: item, ToTimeout: time.Now().Local().Add(timeout), ResultCh: sc.StatsLoad.ResultCh, @@ -166,7 +167,7 @@ func (s *statsSyncLoad) removeHistLoadedColumns(neededItems []model.TableItemID) } // AppendNeededItem appends needed columns/indices to ch, it is only used for test -func (s *statsSyncLoad) AppendNeededItem(task *utilstats.NeededItemTask, timeout time.Duration) error { +func (s *statsSyncLoad) AppendNeededItem(task *statstypes.NeededItemTask, timeout time.Duration) error { timer := time.NewTimer(timeout) defer timer.Stop() select { @@ -186,7 +187,7 @@ func (s *statsSyncLoad) SubLoadWorker(sctx sessionctx.Context, exit chan struct{ logutil.BgLogger().Info("SubLoadWorker exited.") }() // if the last task is not successfully handled in last round for error or panic, pass it to this round to retry - var lastTask *utilstats.NeededItemTask + var lastTask *statstypes.NeededItemTask for { task, err := s.HandleOneTask(sctx, lastTask, exit) lastTask = task @@ -203,7 +204,7 @@ func (s *statsSyncLoad) SubLoadWorker(sctx sessionctx.Context, exit chan struct{ } // HandleOneTask handles last task if not nil, else handle a new task from chan, and return current task if fail somewhere. -func (s *statsSyncLoad) HandleOneTask(sctx sessionctx.Context, lastTask *utilstats.NeededItemTask, exit chan struct{}) (task *utilstats.NeededItemTask, err error) { +func (s *statsSyncLoad) HandleOneTask(sctx sessionctx.Context, lastTask *statstypes.NeededItemTask, exit chan struct{}) (task *statstypes.NeededItemTask, err error) { defer func() { // recover for each task, worker keeps working if r := recover(); r != nil { @@ -225,7 +226,7 @@ func (s *statsSyncLoad) HandleOneTask(sctx sessionctx.Context, lastTask *utilsta return s.handleOneItemTask(sctx, task) } -func (s *statsSyncLoad) handleOneItemTask(sctx sessionctx.Context, task *utilstats.NeededItemTask) (*utilstats.NeededItemTask, error) { +func (s *statsSyncLoad) handleOneItemTask(sctx sessionctx.Context, task *statstypes.NeededItemTask) (*statstypes.NeededItemTask, error) { result := stmtctx.StatsLoadResult{Item: task.TableItemID} item := result.Item tbl, ok := s.statsHandle.Get(item.TableID) @@ -367,7 +368,7 @@ func (*statsSyncLoad) readStatsForOneItem(sctx sessionctx.Context, item model.Ta } // drainColTask will hang until a column task can return, and either task or error will be returned. -func (s *statsSyncLoad) drainColTask(exit chan struct{}) (*utilstats.NeededItemTask, error) { +func (s *statsSyncLoad) drainColTask(exit chan struct{}) (*statstypes.NeededItemTask, error) { // select NeededColumnsCh firstly, if no task, then select TimeoutColumnsCh for { select { @@ -407,7 +408,7 @@ func (s *statsSyncLoad) drainColTask(exit chan struct{}) (*utilstats.NeededItemT } // writeToTimeoutChan writes in a nonblocking way, and if the channel queue is full, it's ok to drop the task. -func (*statsSyncLoad) writeToTimeoutChan(taskCh chan *utilstats.NeededItemTask, task *utilstats.NeededItemTask) { +func (*statsSyncLoad) writeToTimeoutChan(taskCh chan *statstypes.NeededItemTask, task *statstypes.NeededItemTask) { select { case taskCh <- task: default: @@ -415,7 +416,7 @@ func (*statsSyncLoad) writeToTimeoutChan(taskCh chan *utilstats.NeededItemTask, } // writeToChanWithTimeout writes a task to a channel and blocks until timeout. -func (*statsSyncLoad) writeToChanWithTimeout(taskCh chan *utilstats.NeededItemTask, task *utilstats.NeededItemTask, timeout time.Duration) error { +func (*statsSyncLoad) writeToChanWithTimeout(taskCh chan *statstypes.NeededItemTask, task *statstypes.NeededItemTask, timeout time.Duration) error { timer := time.NewTimer(timeout) defer timer.Stop() select { diff --git a/pkg/statistics/handle/types/BUILD.bazel b/pkg/statistics/handle/types/BUILD.bazel new file mode 100644 index 0000000000000..5c2585c1b1a30 --- /dev/null +++ b/pkg/statistics/handle/types/BUILD.bazel @@ -0,0 +1,20 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "types", + srcs = ["interfaces.go"], + importpath = "github.com/pingcap/tidb/pkg/statistics/handle/types", + visibility = ["//visibility:public"], + deps = [ + "//pkg/infoschema", + "//pkg/parser/ast", + "//pkg/parser/model", + "//pkg/sessionctx", + "//pkg/sessionctx/stmtctx", + "//pkg/statistics", + "//pkg/statistics/handle/util", + "//pkg/types", + "//pkg/util", + "//pkg/util/sqlexec", + ], +) diff --git a/pkg/statistics/handle/util/interfaces.go b/pkg/statistics/handle/types/interfaces.go similarity index 94% rename from pkg/statistics/handle/util/interfaces.go rename to pkg/statistics/handle/types/interfaces.go index cdbfd915adf25..c65de0f68802d 100644 --- a/pkg/statistics/handle/util/interfaces.go +++ b/pkg/statistics/handle/types/interfaces.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package util +package types import ( "context" @@ -25,10 +25,10 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" + statsutil "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/sqlexec" - "github.com/tiancaiamao/gp" ) // StatsGC is used to GC unnecessary stats. @@ -260,11 +260,11 @@ type StatsReadWriter interface { ChangeGlobalStatsID(from, to int64) (err error) // TableStatsToJSON dumps table stats to JSON. - TableStatsToJSON(dbName string, tableInfo *model.TableInfo, physicalID int64, snapshot uint64) (*JSONTable, error) + TableStatsToJSON(dbName string, tableInfo *model.TableInfo, physicalID int64, snapshot uint64) (*statsutil.JSONTable, error) // DumpStatsToJSON dumps statistic to json. DumpStatsToJSON(dbName string, tableInfo *model.TableInfo, - historyStatsExec sqlexec.RestrictedSQLExecutor, dumpPartitionStats bool) (*JSONTable, error) + historyStatsExec sqlexec.RestrictedSQLExecutor, dumpPartitionStats bool) (*statsutil.JSONTable, error) // DumpHistoricalStatsBySnapshot dumped json tables from mysql.stats_meta_history and mysql.stats_history. // As implemented in getTableHistoricalStatsToJSONWithFallback, if historical stats are nonexistent, it will fall back @@ -274,20 +274,20 @@ type StatsReadWriter interface { tableInfo *model.TableInfo, snapshot uint64, ) ( - jt *JSONTable, + jt *statsutil.JSONTable, fallbackTbls []string, err error, ) // DumpStatsToJSONBySnapshot dumps statistic to json. - DumpStatsToJSONBySnapshot(dbName string, tableInfo *model.TableInfo, snapshot uint64, dumpPartitionStats bool) (*JSONTable, error) + DumpStatsToJSONBySnapshot(dbName string, tableInfo *model.TableInfo, snapshot uint64, dumpPartitionStats bool) (*statsutil.JSONTable, error) // LoadStatsFromJSON will load statistic from JSONTable, and save it to the storage. // In final, it will also udpate the stats cache. - LoadStatsFromJSON(ctx context.Context, is infoschema.InfoSchema, jsonTbl *JSONTable, concurrencyForPartition uint8) error + LoadStatsFromJSON(ctx context.Context, is infoschema.InfoSchema, jsonTbl *statsutil.JSONTable, concurrencyForPartition uint8) error // LoadStatsFromJSONNoUpdate will load statistic from JSONTable, and save it to the storage. - LoadStatsFromJSONNoUpdate(ctx context.Context, is infoschema.InfoSchema, jsonTbl *JSONTable, concurrencyForPartition uint8) error + LoadStatsFromJSONNoUpdate(ctx context.Context, is infoschema.InfoSchema, jsonTbl *statsutil.JSONTable, concurrencyForPartition uint8) error // Methods for extended stast. @@ -356,27 +356,24 @@ type StatsGlobal interface { // DDL is used to handle ddl events. type DDL interface { // HandleDDLEvent handles ddl events. - HandleDDLEvent(event *DDLEvent) error + HandleDDLEvent(event *statsutil.DDLEvent) error // DDLEventCh returns ddl events channel in handle. - DDLEventCh() chan *DDLEvent + DDLEventCh() chan *statsutil.DDLEvent } // StatsHandle is used to manage TiDB Statistics. type StatsHandle interface { - // GPool returns the goroutine pool. - GPool() *gp.Pool + // Pool is used to get a session or a goroutine to execute stats updating. + statsutil.Pool - // SPool returns the session pool. - SPool() SessionPool + // AutoAnalyzeProcIDGenerator is used to generate auto analyze proc ID. + statsutil.AutoAnalyzeProcIDGenerator - // Lease returns the stats lease. - Lease() time.Duration + // LeaseGetter is used to get stats lease. + statsutil.LeaseGetter - // SysProcTracker is used to track sys process like analyze - SysProcTracker() sessionctx.SysProcTracker - - // AutoAnalyzeProcID generates an analyze ID. - AutoAnalyzeProcID() uint64 + // TableInfoGetter is used to get table meta info. + statsutil.TableInfoGetter // GetTableStats retrieves the statistics table from cache, and the cache will be updated by a goroutine. GetTableStats(tblInfo *model.TableInfo) *statistics.Table @@ -384,12 +381,6 @@ type StatsHandle interface { // GetPartitionStats retrieves the partition stats from cache. GetPartitionStats(tblInfo *model.TableInfo, pid int64) *statistics.Table - // GetCurrentPruneMode returns the current latest partitioning table prune mode. - GetCurrentPruneMode() (mode string, err error) - - // TableInfoGetter is used to get table meta info. - TableInfoGetter - // StatsGC is used to do the GC job. StatsGC diff --git a/pkg/statistics/handle/updatetest/update_test.go b/pkg/statistics/handle/updatetest/update_test.go index 71573bee20236..ff03126a57ba7 100644 --- a/pkg/statistics/handle/updatetest/update_test.go +++ b/pkg/statistics/handle/updatetest/update_test.go @@ -297,7 +297,7 @@ func TestTxnWithFailure(t *testing.T) { func TestUpdatePartition(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) testKit := testkit.NewTestKit(t, store) - pruneMode, err := dom.StatsHandle().GetCurrentPruneMode() + pruneMode, err := util.GetCurrentPruneMode(dom.StatsHandle().SPool()) require.NoError(t, err) testKit.MustQuery("select @@tidb_partition_prune_mode").Check(testkit.Rows(pruneMode)) testKit.MustExec("use test") @@ -753,7 +753,7 @@ func TestStatsVariables(t *testing.T) { h := dom.StatsHandle() sctx := tk.Session().(sessionctx.Context) - pruneMode, err := h.GetCurrentPruneMode() + pruneMode, err := util.GetCurrentPruneMode(h.SPool()) require.NoError(t, err) require.Equal(t, string(variable.Dynamic), pruneMode) err = util.UpdateSCtxVarsForStats(sctx) @@ -770,7 +770,7 @@ func TestStatsVariables(t *testing.T) { tk.MustExec(`set global tidb_enable_analyze_snapshot=1`) tk.MustExec(`set global tidb_skip_missing_partition_stats=0`) - pruneMode, err = h.GetCurrentPruneMode() + pruneMode, err = util.GetCurrentPruneMode(h.SPool()) require.NoError(t, err) require.Equal(t, string(variable.Static), pruneMode) err = util.UpdateSCtxVarsForStats(sctx) diff --git a/pkg/statistics/handle/usage/BUILD.bazel b/pkg/statistics/handle/usage/BUILD.bazel index 10b9760d723d4..13d4fb48b0b6c 100644 --- a/pkg/statistics/handle/usage/BUILD.bazel +++ b/pkg/statistics/handle/usage/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/statistics/handle/storage", + "//pkg/statistics/handle/types", "//pkg/statistics/handle/util", "//pkg/types", "//pkg/util", diff --git a/pkg/statistics/handle/usage/predicate_column.go b/pkg/statistics/handle/usage/predicate_column.go index cd60c53b3afef..f4b58acb2bb41 100644 --- a/pkg/statistics/handle/usage/predicate_column.go +++ b/pkg/statistics/handle/usage/predicate_column.go @@ -24,15 +24,16 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" + statstypes "github.com/pingcap/tidb/pkg/statistics/handle/types" utilstats "github.com/pingcap/tidb/pkg/statistics/handle/util" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/zap" ) -// statsUsageImpl implements utilstats.StatsUsage. +// statsUsageImpl implements statstypes.StatsUsage. type statsUsageImpl struct { - statsHandle utilstats.StatsHandle + statsHandle statstypes.StatsHandle // idxUsageListHead contains all the index usage collectors required by session. idxUsageListHead *SessionIndexUsageCollector @@ -41,8 +42,8 @@ type statsUsageImpl struct { *SessionStatsList } -// NewStatsUsageImpl creates a utilstats.StatsUsage. -func NewStatsUsageImpl(statsHandle utilstats.StatsHandle) utilstats.StatsUsage { +// NewStatsUsageImpl creates a statstypes.StatsUsage. +func NewStatsUsageImpl(statsHandle statstypes.StatsHandle) statstypes.StatsUsage { return &statsUsageImpl{ statsHandle: statsHandle, idxUsageListHead: newSessionIndexUsageCollector(nil), @@ -50,7 +51,7 @@ func NewStatsUsageImpl(statsHandle utilstats.StatsHandle) utilstats.StatsUsage { } // LoadColumnStatsUsage returns all columns' usage information. -func (u *statsUsageImpl) LoadColumnStatsUsage(loc *time.Location) (colStatsMap map[model.TableItemID]utilstats.ColStatsTimeInfo, err error) { +func (u *statsUsageImpl) LoadColumnStatsUsage(loc *time.Location) (colStatsMap map[model.TableItemID]statstypes.ColStatsTimeInfo, err error) { err = utilstats.CallWithSCtx(u.statsHandle.SPool(), func(sctx sessionctx.Context) error { colStatsMap, err = LoadColumnStatsUsage(sctx, loc) return err @@ -77,7 +78,7 @@ func (u *statsUsageImpl) CollectColumnsInExtendedStats(tableID int64) (columnIDs } // LoadColumnStatsUsage loads column stats usage information from disk. -func LoadColumnStatsUsage(sctx sessionctx.Context, loc *time.Location) (map[model.TableItemID]utilstats.ColStatsTimeInfo, error) { +func LoadColumnStatsUsage(sctx sessionctx.Context, loc *time.Location) (map[model.TableItemID]statstypes.ColStatsTimeInfo, error) { disableTime, err := getDisableColumnTrackingTime(sctx) if err != nil { return nil, errors.Trace(err) @@ -87,13 +88,13 @@ func LoadColumnStatsUsage(sctx sessionctx.Context, loc *time.Location) (map[mode if err != nil { return nil, errors.Trace(err) } - colStatsMap := make(map[model.TableItemID]utilstats.ColStatsTimeInfo, len(rows)) + colStatsMap := make(map[model.TableItemID]statstypes.ColStatsTimeInfo, len(rows)) for _, row := range rows { if row.IsNull(0) || row.IsNull(1) { continue } tblColID := model.TableItemID{TableID: row.GetInt64(0), ID: row.GetInt64(1), IsIndex: false} - var statsUsage utilstats.ColStatsTimeInfo + var statsUsage statstypes.ColStatsTimeInfo if !row.IsNull(2) { gt, err := row.GetTime(2).GoTime(time.UTC) if err != nil { diff --git a/pkg/statistics/handle/util/BUILD.bazel b/pkg/statistics/handle/util/BUILD.bazel index e6c4e3d814737..1f10614908675 100644 --- a/pkg/statistics/handle/util/BUILD.bazel +++ b/pkg/statistics/handle/util/BUILD.bazel @@ -3,8 +3,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "util", srcs = [ + "auto_analyze_proc_id_generator.go", "ddl_event.go", - "interfaces.go", + "lease_getter.go", + "pool.go", "table_info.go", "util.go", ], @@ -17,12 +19,8 @@ go_library( "//pkg/parser/model", "//pkg/parser/terror", "//pkg/sessionctx", - "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", - "//pkg/statistics", "//pkg/table", - "//pkg/types", - "//pkg/util", "//pkg/util/chunk", "//pkg/util/intest", "//pkg/util/sqlexec", @@ -32,6 +30,7 @@ go_library( "@com_github_pingcap_tipb//go-tipb", "@com_github_tiancaiamao_gp//:gp", "@com_github_tikv_client_go_v2//oracle", + "@org_uber_go_atomic//:atomic", ], ) diff --git a/pkg/statistics/handle/util/auto_analyze_proc_id_generator.go b/pkg/statistics/handle/util/auto_analyze_proc_id_generator.go new file mode 100644 index 0000000000000..b65dca74d369a --- /dev/null +++ b/pkg/statistics/handle/util/auto_analyze_proc_id_generator.go @@ -0,0 +1,40 @@ +// Copyright 2023 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 util + +// AutoAnalyzeProcIDGenerator is used to generate auto analyze proc ID. +type AutoAnalyzeProcIDGenerator interface { + // AutoAnalyzeProcID generates an analyze ID. + AutoAnalyzeProcID() uint64 +} + +var _ AutoAnalyzeProcIDGenerator = (*generator)(nil) + +type generator struct { + // autoAnalyzeProcIDGetter is used to generate auto analyze ID. + autoAnalyzeProcIDGetter func() uint64 +} + +// NewGenerator creates a new Generator. +func NewGenerator(autoAnalyzeProcIDGetter func() uint64) AutoAnalyzeProcIDGenerator { + return &generator{ + autoAnalyzeProcIDGetter: autoAnalyzeProcIDGetter, + } +} + +// AutoAnalyzeProcID implements AutoAnalyzeProcIDGenerator. +func (g *generator) AutoAnalyzeProcID() uint64 { + return g.autoAnalyzeProcIDGetter() +} diff --git a/pkg/statistics/handle/util/lease_getter.go b/pkg/statistics/handle/util/lease_getter.go new file mode 100644 index 0000000000000..8113dfcd878d6 --- /dev/null +++ b/pkg/statistics/handle/util/lease_getter.go @@ -0,0 +1,52 @@ +// Copyright 2023 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 util + +import ( + "time" + + "go.uber.org/atomic" +) + +// LeaseGetter is used to get the stats lease. +type LeaseGetter interface { + // Lease returns the stats lease. + Lease() time.Duration + // SetLease sets the stats lease. Only used for test. + SetLease(lease time.Duration) +} + +var _ LeaseGetter = (*leaseGetter)(nil) + +type leaseGetter struct { + lease *atomic.Duration +} + +// NewLeaseGetter creates a new LeaseGetter. +func NewLeaseGetter(lease time.Duration) LeaseGetter { + return &leaseGetter{ + lease: atomic.NewDuration(lease), + } +} + +// Lease implements LeaseGetter. +func (g *leaseGetter) Lease() time.Duration { + return g.lease.Load() +} + +// SetLease sets the stats lease. Only used for test. +func (g *leaseGetter) SetLease(lease time.Duration) { + g.lease.Store(lease) +} diff --git a/pkg/statistics/handle/util/pool.go b/pkg/statistics/handle/util/pool.go new file mode 100644 index 0000000000000..6f6a3633459c9 --- /dev/null +++ b/pkg/statistics/handle/util/pool.go @@ -0,0 +1,72 @@ +// Copyright 2023 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 util + +import ( + "math" + "time" + + "github.com/ngaut/pools" + "github.com/tiancaiamao/gp" +) + +// SessionPool is used to recycle sessionctx. +type SessionPool interface { + Get() (pools.Resource, error) + Put(pools.Resource) +} + +// Pool is used to reuse goroutine and session. +type Pool interface { + // GPool returns the goroutine pool. + GPool() *gp.Pool + + // SPool returns the session pool. + SPool() SessionPool + + // Close closes the goroutine pool. + Close() +} + +var _ Pool = (*pool)(nil) + +type pool struct { + // This gpool is used to reuse goroutine in the mergeGlobalStatsTopN. + gpool *gp.Pool + pool SessionPool +} + +// NewPool creates a new Pool. +func NewPool(p SessionPool) Pool { + return &pool{ + gpool: gp.New(math.MaxInt16, time.Minute), + pool: p, + } +} + +// GPool returns the goroutine pool. +func (p *pool) GPool() *gp.Pool { + return p.gpool +} + +// SPool returns the session pool. +func (p *pool) SPool() SessionPool { + return p.pool +} + +// Close close the goroutine pool. +func (p *pool) Close() { + p.gpool.Close() +} diff --git a/pkg/statistics/handle/util/util.go b/pkg/statistics/handle/util/util.go index 50add779777d0..930173255ee50 100644 --- a/pkg/statistics/handle/util/util.go +++ b/pkg/statistics/handle/util/util.go @@ -19,7 +19,6 @@ import ( "strconv" "time" - "github.com/ngaut/pools" "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" @@ -60,12 +59,6 @@ var ( StatsCtx = kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) ) -// SessionPool is used to recycle sessionctx. -type SessionPool interface { - Get() (pools.Resource, error) - Put(pools.Resource) -} - // finishTransaction will execute `commit` when error is nil, otherwise `rollback`. func finishTransaction(sctx sessionctx.Context, err error) error { if err == nil { @@ -171,6 +164,15 @@ func UpdateSCtxVarsForStats(sctx sessionctx.Context) error { return nil } +// GetCurrentPruneMode returns the current latest partitioning table prune mode. +func GetCurrentPruneMode(pool SessionPool) (mode string, err error) { + err = CallWithSCtx(pool, func(sctx sessionctx.Context) error { + mode = sctx.GetSessionVars().PartitionPruneMode.Load() + return nil + }) + return +} + // WrapTxn uses a transaction here can let different SQLs in this operation have the same data visibility. func WrapTxn(sctx sessionctx.Context, f func(sctx sessionctx.Context) error) (err error) { // TODO: check whether this sctx is already in a txn From 5bc2136a60878a78b13a24f933b22f4fbb8d5e4c Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 22 Nov 2023 17:41:58 +0800 Subject: [PATCH 05/36] membuf: add SliceLocation to reduce GC pressure (#48710) ref pingcap/tidb#48779 --- br/pkg/lightning/backend/external/BUILD.bazel | 4 +- br/pkg/lightning/backend/external/kv_buf.go | 75 -------- .../lightning/backend/external/kv_buf_test.go | 77 --------- br/pkg/lightning/backend/external/writer.go | 51 +++--- .../lightning/backend/external/writer_test.go | 1 - br/pkg/membuf/BUILD.bazel | 1 + br/pkg/membuf/buffer.go | 127 ++++++++++---- br/pkg/membuf/buffer_test.go | 160 ++++++++++++++++++ 8 files changed, 277 insertions(+), 219 deletions(-) delete mode 100644 br/pkg/lightning/backend/external/kv_buf.go delete mode 100644 br/pkg/lightning/backend/external/kv_buf_test.go diff --git a/br/pkg/lightning/backend/external/BUILD.bazel b/br/pkg/lightning/backend/external/BUILD.bazel index a4390fbf03242..93299ec3e0c6b 100644 --- a/br/pkg/lightning/backend/external/BUILD.bazel +++ b/br/pkg/lightning/backend/external/BUILD.bazel @@ -9,7 +9,6 @@ go_library( "engine.go", "file.go", "iter.go", - "kv_buf.go", "kv_reader.go", "merge.go", "split.go", @@ -57,7 +56,6 @@ go_test( "engine_test.go", "file_test.go", "iter_test.go", - "kv_buf_test.go", "merge_test.go", "split_test.go", "util_test.go", @@ -65,7 +63,7 @@ go_test( ], embed = [":external"], flaky = True, - shard_count = 46, + shard_count = 45, deps = [ "//br/pkg/lightning/backend/kv", "//br/pkg/lightning/common", diff --git a/br/pkg/lightning/backend/external/kv_buf.go b/br/pkg/lightning/backend/external/kv_buf.go deleted file mode 100644 index 5079c3f475b41..0000000000000 --- a/br/pkg/lightning/backend/external/kv_buf.go +++ /dev/null @@ -1,75 +0,0 @@ -// Copyright 2023 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 external - -import "github.com/docker/go-units" - -// DefaultBlockSize is the default block size for preAllocKVBuf. -const DefaultBlockSize = 16 * units.MiB - -// preAllocKVBuf pre allocates a large buffer of limit memLimit to reduce memory -// allocation, all space in this buffer will be reused when reset. -type preAllocKVBuf struct { - blocks [][]byte - blockSize int - curBlock []byte - curBlockIdx int - curIdx int -} - -func newPreAllocKVBuf(memLimit uint64, blockSize int) *preAllocKVBuf { - blockCount := (memLimit + uint64(blockSize) - 1) / uint64(blockSize) - b := &preAllocKVBuf{ - blocks: make([][]byte, 0, blockCount), - blockSize: blockSize, - } - for i := 0; i < int(blockCount); i++ { - b.blocks = append(b.blocks, make([]byte, blockSize)) - } - b.reset() - return b -} - -func (b *preAllocKVBuf) Alloc(s int) (blockIdx int32, res []byte, offset int32, allocated bool) { - if s > b.blockSize { - return - } - if b.blockSize-b.curIdx < s { - if b.curBlockIdx+1 >= len(b.blocks) { - return - } - b.curBlockIdx++ - b.curBlock = b.blocks[b.curBlockIdx] - b.curIdx = 0 - } - blockIdx = int32(b.curBlockIdx) - res = b.curBlock[b.curIdx : b.curIdx+s] - offset = int32(b.curIdx) - allocated = true - - b.curIdx += s - return -} - -func (b *preAllocKVBuf) reset() { - b.curBlockIdx = 0 - b.curBlock = b.blocks[0] - b.curIdx = 0 -} - -func (b *preAllocKVBuf) destroy() { - b.blocks = nil - b.curBlock = nil -} diff --git a/br/pkg/lightning/backend/external/kv_buf_test.go b/br/pkg/lightning/backend/external/kv_buf_test.go deleted file mode 100644 index fddb3230b417d..0000000000000 --- a/br/pkg/lightning/backend/external/kv_buf_test.go +++ /dev/null @@ -1,77 +0,0 @@ -// Copyright 2023 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 external - -import ( - "testing" - - "github.com/stretchr/testify/require" -) - -func TestNewPreAllocKVBuf(t *testing.T) { - cases := []struct { - memLimit uint64 - expectBlocks int - }{ - {1, 1}, - {15, 1}, - {16, 1}, - {17, 2}, - {31, 2}, - {32, 2}, - } - for _, c := range cases { - buf := newPreAllocKVBuf(c.memLimit, 16) - require.Equal(t, c.expectBlocks, len(buf.blocks)) - require.Equal(t, 16, buf.blockSize) - require.Equal(t, buf.blocks[0], buf.curBlock) - require.Equal(t, 0, buf.curBlockIdx) - require.Equal(t, 0, buf.curIdx) - } - - buf := newPreAllocKVBuf(16, 8) - // alloc larger than block size. - _, _, _, allocated := buf.Alloc(9) - require.False(t, allocated) - blockIdx, res, offset, allocated := buf.Alloc(8) - require.Equal(t, int32(0), blockIdx) - require.Equal(t, int32(0), offset) - require.True(t, allocated) - copy(res, "12345678") - blockIdx, res, offset, allocated = buf.Alloc(4) - require.Equal(t, int32(1), blockIdx) - require.Equal(t, int32(0), offset) - require.True(t, allocated) - copy(res, "aaaa") - blockIdx, res, offset, allocated = buf.Alloc(4) - require.Equal(t, int32(1), blockIdx) - require.Equal(t, int32(4), offset) - require.True(t, allocated) - copy(res, "bbbb") - _, _, _, allocated = buf.Alloc(4) - require.False(t, allocated) - - require.Equal(t, "12345678", string(buf.blocks[0])) - require.Equal(t, "aaaabbbb", string(buf.blocks[1])) - - buf.reset() - require.Equal(t, buf.blocks[0], buf.curBlock) - require.Equal(t, 0, buf.curBlockIdx) - require.Equal(t, 0, buf.curIdx) - - buf.destroy() - require.Nil(t, buf.blocks) - require.Nil(t, buf.curBlock) -} diff --git a/br/pkg/lightning/backend/external/writer.go b/br/pkg/lightning/backend/external/writer.go index cb3ad887114e1..1122f0a8b554a 100644 --- a/br/pkg/lightning/backend/external/writer.go +++ b/br/pkg/lightning/backend/external/writer.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/backend/encode" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/pingcap/tidb/br/pkg/membuf" "github.com/pingcap/tidb/br/pkg/storage" tidbkv "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/metrics" @@ -51,6 +52,8 @@ var ( const ( // DefaultMemSizeLimit is the default memory size limit for writer. DefaultMemSizeLimit = 256 * size.MB + // DefaultBlockSize is the default block size for writer. + DefaultBlockSize = 16 * units.MiB ) // rangePropertiesCollector collects range properties for each range. The zero @@ -175,6 +178,7 @@ func (b *WriterBuilder) Build( if b.keyDupeEncoding { keyAdapter = common.DupDetectKeyAdapter{} } + p := membuf.NewPool(membuf.WithPoolSize(0), membuf.WithBlockSize(b.blockSize)) ret := &Writer{ rc: &rangePropertiesCollector{ props: make([]*rangeProperty, 0, 1024), @@ -184,7 +188,7 @@ func (b *WriterBuilder) Build( }, memSizeLimit: b.memSizeLimit, store: store, - kvBuffer: newPreAllocKVBuf(b.memSizeLimit, b.blockSize), + kvBuffer: p.NewBuffer(membuf.WithMemoryLimit(b.memSizeLimit)), currentSeq: 0, filenamePrefix: filenamePrefix, keyAdapter: keyAdapter, @@ -249,12 +253,6 @@ func GetMaxOverlappingTotal(stats []MultipleFilesStat) int64 { return GetMaxOverlapping(points) } -type kvLocation struct { - blockIdx int32 - offset int32 - length int32 -} - // Writer is used to write data into external storage. type Writer struct { store storage.ExternalStorage @@ -268,8 +266,8 @@ type Writer struct { memSizeLimit uint64 - kvBuffer *preAllocKVBuf - kvLocations []kvLocation + kvBuffer *membuf.Buffer + kvLocations []membuf.SliceLocation kvSize int64 onClose OnCloseFunc @@ -299,14 +297,14 @@ func (w *Writer) WriteRow(ctx context.Context, idxKey, idxVal []byte, handle tid } encodedKeyLen := keyAdapter.EncodedLen(idxKey, rowID) length := encodedKeyLen + len(idxVal) + lengthBytes*2 - blockIdx, dataBuf, off, allocated := w.kvBuffer.Alloc(length) - if !allocated { + dataBuf, loc := w.kvBuffer.AllocBytesWithSliceLocation(length) + if dataBuf == nil { if err := w.flushKVs(ctx, false); err != nil { return err } - blockIdx, dataBuf, off, allocated = w.kvBuffer.Alloc(length) + dataBuf, loc = w.kvBuffer.AllocBytesWithSliceLocation(length) // we now don't support KV larger than blockSize - if !allocated { + if dataBuf == nil { return errors.Errorf("failed to allocate kv buffer: %d", length) } } @@ -315,11 +313,7 @@ func (w *Writer) WriteRow(ctx context.Context, idxKey, idxVal []byte, handle tid binary.BigEndian.AppendUint64(dataBuf[lengthBytes+encodedKeyLen:lengthBytes+encodedKeyLen], uint64(len(idxVal))) copy(dataBuf[lengthBytes*2+encodedKeyLen:], idxVal) - w.kvLocations = append(w.kvLocations, kvLocation{ - blockIdx: blockIdx, - offset: off, - length: int32(length)}, - ) + w.kvLocations = append(w.kvLocations, loc) w.kvSize += int64(encodedKeyLen + len(idxVal)) w.batchSize += uint64(length) return nil @@ -338,7 +332,7 @@ func (w *Writer) Close(ctx context.Context) error { return errors.Errorf("writer %s has been closed", w.writerID) } w.closed = true - defer w.kvBuffer.destroy() + defer w.kvBuffer.Destroy() err := w.flushKVs(ctx, true) if err != nil { return err @@ -439,7 +433,7 @@ func (w *Writer) flushKVs(ctx context.Context, fromClose bool) (err error) { }() sortStart := time.Now() - slices.SortFunc(w.kvLocations, func(i, j kvLocation) int { + slices.SortFunc(w.kvLocations, func(i, j membuf.SliceLocation) int { return bytes.Compare(w.getKeyByLoc(i), w.getKeyByLoc(j)) }) sortDuration = time.Since(sortStart) @@ -453,7 +447,7 @@ func (w *Writer) flushKVs(ctx context.Context, fromClose bool) (err error) { } for _, pair := range w.kvLocations { - err = w.kvStore.addEncodedData(w.getEncodedKVData(pair)) + err = w.kvStore.addEncodedData(w.kvBuffer.GetSlice(pair)) if err != nil { return err } @@ -489,21 +483,16 @@ func (w *Writer) flushKVs(ctx context.Context, fromClose bool) (err error) { w.kvLocations = w.kvLocations[:0] w.kvSize = 0 - w.kvBuffer.reset() + w.kvBuffer.Reset() w.rc.reset() w.batchSize = 0 return nil } -func (w *Writer) getEncodedKVData(pos kvLocation) []byte { - block := w.kvBuffer.blocks[pos.blockIdx] - return block[pos.offset : pos.offset+pos.length] -} - -func (w *Writer) getKeyByLoc(pos kvLocation) []byte { - block := w.kvBuffer.blocks[pos.blockIdx] - keyLen := binary.BigEndian.Uint64(block[pos.offset : pos.offset+lengthBytes]) - return block[pos.offset+lengthBytes : uint64(pos.offset)+lengthBytes+keyLen] +func (w *Writer) getKeyByLoc(loc membuf.SliceLocation) []byte { + block := w.kvBuffer.GetSlice(loc) + keyLen := binary.BigEndian.Uint64(block[:lengthBytes]) + return block[lengthBytes : lengthBytes+keyLen] } func (w *Writer) createStorageWriter(ctx context.Context) ( diff --git a/br/pkg/lightning/backend/external/writer_test.go b/br/pkg/lightning/backend/external/writer_test.go index f47dfd219e047..7bc853f63dc1c 100644 --- a/br/pkg/lightning/backend/external/writer_test.go +++ b/br/pkg/lightning/backend/external/writer_test.go @@ -115,7 +115,6 @@ func TestWriterFlushMultiFileNames(t *testing.T) { SetBlockSize(3*(lengthBytes*2+20)). Build(memStore, "/test", "0") - require.Equal(t, 3*(lengthBytes*2+20), writer.kvBuffer.blockSize) // 200 bytes key values. kvCnt := 10 kvs := make([]common.KvPair, kvCnt) diff --git a/br/pkg/membuf/BUILD.bazel b/br/pkg/membuf/BUILD.bazel index d971a8408abbc..a6a2024306f54 100644 --- a/br/pkg/membuf/BUILD.bazel +++ b/br/pkg/membuf/BUILD.bazel @@ -13,5 +13,6 @@ go_test( srcs = ["buffer_test.go"], embed = [":membuf"], flaky = True, + shard_count = 3, deps = ["@com_github_stretchr_testify//require"], ) diff --git a/br/pkg/membuf/buffer.go b/br/pkg/membuf/buffer.go index 9c725f647e38b..3a999d75e7404 100644 --- a/br/pkg/membuf/buffer.go +++ b/br/pkg/membuf/buffer.go @@ -110,11 +110,6 @@ func (p *Pool) release(b []byte) { } } -// NewBuffer creates a new buffer in current pool. -func (p *Pool) NewBuffer() *Buffer { - return &Buffer{pool: p, bufs: make([][]byte, 0, 128), curBufIdx: -1} -} - // Destroy frees all buffers. func (p *Pool) Destroy() { close(p.blockCache) @@ -130,51 +125,65 @@ func (p *Pool) TotalSize() int64 { // Buffer represents the reuse buffer. type Buffer struct { - pool *Pool - bufs [][]byte - curBuf []byte - curIdx int - curBufIdx int - curBufLen int + pool *Pool + blocks [][]byte + blockCntLimit int + curBlock []byte + curBlockIdx int + curIdx int } -// addBuf adds buffer to Buffer. -func (b *Buffer) addBuf() { - if b.curBufIdx < len(b.bufs)-1 { - b.curBufIdx++ - b.curBuf = b.bufs[b.curBufIdx] - } else { - buf := b.pool.acquire() - b.bufs = append(b.bufs, buf) - b.curBuf = buf - b.curBufIdx = len(b.bufs) - 1 +// BufferOption configures a buffer. +type BufferOption func(*Buffer) + +// WithMemoryLimit approximately limits the maximum memory size of this Buffer. +// Due to it use blocks to allocate memory, the actual memory size is +// blockSize*ceil(limit/blockSize). +// In order to keep compatibility, it will only restrict AllocBytesWithSliceLocation. +func WithMemoryLimit(limit uint64) BufferOption { + return func(b *Buffer) { + blockCntLimit := int(limit+uint64(b.pool.blockSize)-1) / b.pool.blockSize + b.blockCntLimit = blockCntLimit + b.blocks = make([][]byte, 0, blockCntLimit) } +} - b.curBufLen = len(b.curBuf) - b.curIdx = 0 +// NewBuffer creates a new buffer in current pool. +func (p *Pool) NewBuffer(opts ...BufferOption) *Buffer { + b := &Buffer{ + pool: p, + curBlockIdx: -1, + blockCntLimit: -1, + } + for _, opt := range opts { + opt(b) + } + if b.blocks == nil { + b.blocks = make([][]byte, 0, 128) + } + return b } // Reset resets the buffer. func (b *Buffer) Reset() { - if len(b.bufs) > 0 { - b.curBuf = b.bufs[0] - b.curBufLen = len(b.bufs[0]) - b.curBufIdx = 0 + if len(b.blocks) > 0 { + b.curBlock = b.blocks[0] + b.curBlockIdx = 0 b.curIdx = 0 } } // Destroy frees all buffers. func (b *Buffer) Destroy() { - for _, buf := range b.bufs { + for _, buf := range b.blocks { b.pool.release(buf) } - b.bufs = nil + b.blocks = nil } // TotalSize represents the total memory size of this Buffer. func (b *Buffer) TotalSize() int64 { - return int64(len(b.bufs) * b.pool.blockSize) + return int64(len(b.blocks) * b.pool.blockSize) } // AllocBytes allocates bytes with the given length. @@ -182,12 +191,66 @@ func (b *Buffer) AllocBytes(n int) []byte { if n > b.pool.largeAllocThreshold { return make([]byte, n) } - if b.curIdx+n > b.curBufLen { + if b.curIdx+n > len(b.curBlock) { b.addBuf() } idx := b.curIdx b.curIdx += n - return b.curBuf[idx:b.curIdx:b.curIdx] + return b.curBlock[idx:b.curIdx:b.curIdx] +} + +// addBuf adds buffer to Buffer. +func (b *Buffer) addBuf() { + if b.curBlockIdx < len(b.blocks)-1 { + b.curBlockIdx++ + b.curBlock = b.blocks[b.curBlockIdx] + } else { + buf := b.pool.acquire() + b.blocks = append(b.blocks, buf) + b.curBlock = buf + b.curBlockIdx = len(b.blocks) - 1 + } + + b.curIdx = 0 +} + +// SliceLocation is like a reflect.SliceHeader, but it's associated with a +// Buffer. The advantage is that it's smaller than a slice, and it doesn't +// contain a pointer thus more GC-friendly. +type SliceLocation struct { + bufIdx int32 + offset int32 + length int32 +} + +// AllocBytesWithSliceLocation is like AllocBytes, but it also returns a +// SliceLocation. The expected usage is after writing data into returned slice we +// do not store the slice itself, but only the SliceLocation. Later we can use +// the SliceLocation to get the slice again. When we have a large number of +// slices in memory this can improve performance. +// nil returned slice means allocation failed. +func (b *Buffer) AllocBytesWithSliceLocation(n int) ([]byte, SliceLocation) { + if n > b.pool.blockSize { + return nil, SliceLocation{} + } + + if b.curIdx+n > len(b.curBlock) { + if b.blockCntLimit >= 0 && len(b.blocks) >= b.blockCntLimit { + return nil, SliceLocation{} + } + b.addBuf() + } + blockIdx := int32(b.curBlockIdx) + offset := int32(b.curIdx) + loc := SliceLocation{bufIdx: blockIdx, offset: offset, length: int32(n)} + + idx := b.curIdx + b.curIdx += n + return b.curBlock[idx:b.curIdx:b.curIdx], loc +} + +func (b *Buffer) GetSlice(loc SliceLocation) []byte { + return b.blocks[loc.bufIdx][loc.offset : loc.offset+loc.length] } // AddBytes adds the bytes into this Buffer. diff --git a/br/pkg/membuf/buffer_test.go b/br/pkg/membuf/buffer_test.go index 0d8e3ba06e609..44e8441066985 100644 --- a/br/pkg/membuf/buffer_test.go +++ b/br/pkg/membuf/buffer_test.go @@ -15,7 +15,11 @@ package membuf import ( + "bytes" "crypto/rand" + rand2 "math/rand" + "runtime" + "slices" "testing" "github.com/stretchr/testify/require" @@ -90,3 +94,159 @@ func TestBufferIsolation(t *testing.T) { require.Equal(t, b3, b2) require.NotEqual(t, b2, b1) } + +func TestBufferMemLimit(t *testing.T) { + pool := NewPool(WithBlockSize(10)) + defer pool.Destroy() + // the actual memory limit is 10 bytes. + bytesBuf := pool.NewBuffer(WithMemoryLimit(5)) + + got, _ := bytesBuf.AllocBytesWithSliceLocation(9) + require.NotNil(t, got) + got, _ = bytesBuf.AllocBytesWithSliceLocation(3) + require.Nil(t, got) + + bytesBuf.Destroy() + + // exactly 2 block + bytesBuf = pool.NewBuffer(WithMemoryLimit(20)) + + got, _ = bytesBuf.AllocBytesWithSliceLocation(9) + require.NotNil(t, got) + got, _ = bytesBuf.AllocBytesWithSliceLocation(9) + require.NotNil(t, got) + got, _ = bytesBuf.AllocBytesWithSliceLocation(2) + require.Nil(t, got) +} + +const dataNum = 100 * 1024 * 1024 + +func BenchmarkStoreSlice(b *testing.B) { + data := make([][]byte, dataNum) + for i := 0; i < b.N; i++ { + func() { + pool := NewPool() + defer pool.Destroy() + bytesBuf := pool.NewBuffer() + defer bytesBuf.Destroy() + + for j := range data { + data[j] = bytesBuf.AllocBytes(10) + } + }() + } +} + +func BenchmarkStoreLocation(b *testing.B) { + data := make([]SliceLocation, dataNum) + for i := 0; i < b.N; i++ { + func() { + pool := NewPool() + defer pool.Destroy() + bytesBuf := pool.NewBuffer() + defer bytesBuf.Destroy() + + for j := range data { + _, data[j] = bytesBuf.AllocBytesWithSliceLocation(10) + } + }() + } +} + +const sortDataNum = 1024 * 1024 + +func BenchmarkSortSlice(b *testing.B) { + data := make([][]byte, sortDataNum) + // fixed seed for benchmark + rnd := rand2.New(rand2.NewSource(6716)) + + for i := 0; i < b.N; i++ { + func() { + pool := NewPool() + defer pool.Destroy() + bytesBuf := pool.NewBuffer() + defer bytesBuf.Destroy() + + for j := range data { + data[j] = bytesBuf.AllocBytes(10) + rnd.Read(data[j]) + } + slices.SortFunc(data, func(a, b []byte) int { + return bytes.Compare(a, b) + }) + }() + } +} + +func BenchmarkSortLocation(b *testing.B) { + data := make([]SliceLocation, sortDataNum) + // fixed seed for benchmark + rnd := rand2.New(rand2.NewSource(6716)) + + for i := 0; i < b.N; i++ { + func() { + pool := NewPool() + defer pool.Destroy() + bytesBuf := pool.NewBuffer() + defer bytesBuf.Destroy() + + for j := range data { + var buf []byte + buf, data[j] = bytesBuf.AllocBytesWithSliceLocation(10) + rnd.Read(buf) + } + slices.SortFunc(data, func(a, b SliceLocation) int { + return bytes.Compare(bytesBuf.GetSlice(a), bytesBuf.GetSlice(b)) + }) + }() + } +} + +func BenchmarkSortSliceWithGC(b *testing.B) { + data := make([][]byte, sortDataNum) + // fixed seed for benchmark + rnd := rand2.New(rand2.NewSource(6716)) + + for i := 0; i < b.N; i++ { + func() { + pool := NewPool() + defer pool.Destroy() + bytesBuf := pool.NewBuffer() + defer bytesBuf.Destroy() + + for j := range data { + data[j] = bytesBuf.AllocBytes(10) + rnd.Read(data[j]) + } + runtime.GC() + slices.SortFunc(data, func(a, b []byte) int { + return bytes.Compare(a, b) + }) + }() + } +} + +func BenchmarkSortLocationWithGC(b *testing.B) { + data := make([]SliceLocation, sortDataNum) + // fixed seed for benchmark + rnd := rand2.New(rand2.NewSource(6716)) + + for i := 0; i < b.N; i++ { + func() { + pool := NewPool() + defer pool.Destroy() + bytesBuf := pool.NewBuffer() + defer bytesBuf.Destroy() + + for j := range data { + var buf []byte + buf, data[j] = bytesBuf.AllocBytesWithSliceLocation(10) + rnd.Read(buf) + } + runtime.GC() + slices.SortFunc(data, func(a, b SliceLocation) int { + return bytes.Compare(bytesBuf.GetSlice(a), bytesBuf.GetSlice(b)) + }) + }() + } +} From adad058be192da79645ce41f3357b2d777774f52 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 22 Nov 2023 17:42:05 +0800 Subject: [PATCH 06/36] *: upgrade rules_go and x/tools (#48774) --- DEPS.bzl | 24 ++++++++++++------------ WORKSPACE | 10 +++++----- go.mod | 4 ++-- go.sum | 7 ++++--- 4 files changed, 23 insertions(+), 22 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index f75530ea23ccf..2ca31f47279a9 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2348,13 +2348,13 @@ def go_deps(): name = "com_github_go_ole_go_ole", build_file_proto_mode = "disable_global", importpath = "github.com/go-ole/go-ole", - sha256 = "95b192df81ca16f0fb7d2d98ff6596d70256d73e49e899c55fabd511fd6768ef", - strip_prefix = "github.com/go-ole/go-ole@v1.2.6", + sha256 = "bbf5b3bfa227a5daa06eb16ecdecccc0b20e08749bf103afb523fd72764e727a", + strip_prefix = "github.com/go-ole/go-ole@v1.3.0", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/github.com/go-ole/go-ole/com_github_go_ole_go_ole-v1.2.6.zip", - "http://ats.apps.svc/gomod/github.com/go-ole/go-ole/com_github_go_ole_go_ole-v1.2.6.zip", - "https://cache.hawkingrei.com/gomod/github.com/go-ole/go-ole/com_github_go_ole_go_ole-v1.2.6.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/github.com/go-ole/go-ole/com_github_go_ole_go_ole-v1.2.6.zip", + "http://bazel-cache.pingcap.net:8080/gomod/github.com/go-ole/go-ole/com_github_go_ole_go_ole-v1.3.0.zip", + "http://ats.apps.svc/gomod/github.com/go-ole/go-ole/com_github_go_ole_go_ole-v1.3.0.zip", + "https://cache.hawkingrei.com/gomod/github.com/go-ole/go-ole/com_github_go_ole_go_ole-v1.3.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/go-ole/go-ole/com_github_go_ole_go_ole-v1.3.0.zip", ], ) go_repository( @@ -10347,13 +10347,13 @@ def go_deps(): name = "org_golang_x_tools", build_file_proto_mode = "disable_global", importpath = "golang.org/x/tools", - sha256 = "1e4ec8fbe5395eae1923e29b734224108b933086bb4d46d03bab6d5479c54242", - strip_prefix = "golang.org/x/tools@v0.14.0", + sha256 = "5cdd3e8b6d805e11c63e8a0262050cd6eac9b0c51bd9b35cd82d5f309d290c70", + strip_prefix = "golang.org/x/tools@v0.15.0", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/golang.org/x/tools/org_golang_x_tools-v0.14.0.zip", - "http://ats.apps.svc/gomod/golang.org/x/tools/org_golang_x_tools-v0.14.0.zip", - "https://cache.hawkingrei.com/gomod/golang.org/x/tools/org_golang_x_tools-v0.14.0.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/golang.org/x/tools/org_golang_x_tools-v0.14.0.zip", + "http://bazel-cache.pingcap.net:8080/gomod/golang.org/x/tools/org_golang_x_tools-v0.15.0.zip", + "http://ats.apps.svc/gomod/golang.org/x/tools/org_golang_x_tools-v0.15.0.zip", + "https://cache.hawkingrei.com/gomod/golang.org/x/tools/org_golang_x_tools-v0.15.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/golang.org/x/tools/org_golang_x_tools-v0.15.0.zip", ], ) go_repository( diff --git a/WORKSPACE b/WORKSPACE index fd1ef1fedb502..f0eaf4460c355 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -16,12 +16,12 @@ versions.check(minimum_bazel_version = "6.0.0") http_archive( name = "io_bazel_rules_go", - sha256 = "eff2811014f8b26d91fadda335000cbfce145674368bcd732fbf782adb53a778", - strip_prefix = "rules_go-df20c987afcbbc721518c79f9c6489d87d73582c", + sha256 = "d6ab6b57e48c09523e93050f13698f708428cfd5e619252e369d377af6597707", urls = [ - "http://bazel-cache.pingcap.net:8080/bazelbuild/rules_go/archive/df20c987afcbbc721518c79f9c6489d87d73582c.zip", - "http://ats.apps.svc/bazelbuild/rules_go/archive/df20c987afcbbc721518c79f9c6489d87d73582c.zip", - "https://github.com/bazelbuild/rules_go/archive/df20c987afcbbc721518c79f9c6489d87d73582c.zip", + "http://bazel-cache.pingcap.net:8080/bazelbuild/rules_go/releases/download/v0.43.0/rules_go-v0.43.0.zip", + "http://ats.apps.svc/bazelbuild/rules_go/releases/download/v0.43.0/rules_go-v0.43.0.zip", + "https://github.com/bazelbuild/rules_go/releases/download/v0.43.0/rules_go-v0.43.0.zip", + "https://mirror.bazel.build/github.com/bazelbuild/rules_go/releases/download/v0.43.0/rules_go-v0.43.0.zip", ], ) diff --git a/go.mod b/go.mod index 412516e3c31fc..0c1eee32198b6 100644 --- a/go.mod +++ b/go.mod @@ -131,7 +131,7 @@ require ( golang.org/x/term v0.14.0 golang.org/x/text v0.14.0 golang.org/x/time v0.4.0 - golang.org/x/tools v0.14.0 + golang.org/x/tools v0.15.0 google.golang.org/api v0.128.0 google.golang.org/grpc v1.59.0 gopkg.in/yaml.v2 v2.4.0 @@ -200,7 +200,7 @@ require ( github.com/go-kit/kit v0.9.0 // indirect github.com/go-logfmt/logfmt v0.5.1 // indirect github.com/go-logr/logr v1.2.3 // indirect - github.com/go-ole/go-ole v1.2.6 // indirect + github.com/go-ole/go-ole v1.3.0 // indirect github.com/goccy/go-json v0.10.2 // indirect github.com/golang-jwt/jwt v3.2.1+incompatible // indirect github.com/golang/glog v1.1.2 // indirect diff --git a/go.sum b/go.sum index c8eb62ed5532a..2f9fcf89c5e93 100644 --- a/go.sum +++ b/go.sum @@ -336,8 +336,9 @@ github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= -github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= +github.com/go-ole/go-ole v1.3.0 h1:Dt6ye7+vXGIKZ7Xtk4s6/xVdGDQynvom7xCFEdWr6uE= +github.com/go-ole/go-ole v1.3.0/go.mod h1:5LS6F96DhAwUc7C+1HLexzMXY1xGRSryjyPPKW6zv78= github.com/go-sql-driver/mysql v1.4.1/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= github.com/go-sql-driver/mysql v1.7.1 h1:lUIinVbN1DY0xBg0eMOzmmtGoHwWBbvnWubQUrtU8EI= @@ -1451,8 +1452,8 @@ golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc golang.org/x/tools v0.2.0/go.mod h1:y4OqIKeOV/fWJetJ8bXPU1sEVniLMIyDAZWeHdV+NTA= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.8.0/go.mod h1:JxBZ99ISMI5ViVkT1tr6tdNmXeTrcpVSD3vZ1RsRdN4= -golang.org/x/tools v0.14.0 h1:jvNa2pY0M4r62jkRQ6RwEZZyPcymeL9XZMLBbV7U2nc= -golang.org/x/tools v0.14.0/go.mod h1:uYBEerGOWcJyEORxN+Ek8+TT266gXkNlHdJBwexUsBg= +golang.org/x/tools v0.15.0 h1:zdAyfUGbYmuVokhzVmghFl2ZJh5QhcfebBgmVPFYA+8= +golang.org/x/tools v0.15.0/go.mod h1:hpksKq4dtpQWS1uQ61JkdqWM3LscIS6Slf+VVkm+wQk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= From 66f3a25442f7ce31ed8b0a59462b21b3d72a9228 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 22 Nov 2023 17:42:12 +0800 Subject: [PATCH 07/36] session: move some test into new package (#48784) ref pingcap/tidb#44940 --- pkg/session/BUILD.bazel | 3 -- pkg/session/test/BUILD.bazel | 5 ++- pkg/session/test/tidb_test.go | 75 +++++++++++++++++++++++++++++++++++ pkg/session/tidb_test.go | 52 ------------------------ pkg/session/txn.go | 5 ++- 5 files changed, 82 insertions(+), 58 deletions(-) create mode 100644 pkg/session/test/tidb_test.go diff --git a/pkg/session/BUILD.bazel b/pkg/session/BUILD.bazel index 5a27a9785a923..14b3a33123521 100644 --- a/pkg/session/BUILD.bazel +++ b/pkg/session/BUILD.bazel @@ -127,7 +127,6 @@ go_test( "main_test.go", "tidb_test.go", ], - data = glob(["testdata/**"]), embed = [":session"], flaky = True, race = "on", @@ -143,12 +142,10 @@ go_test( "//pkg/meta", "//pkg/parser/ast", "//pkg/parser/auth", - "//pkg/planner/core", "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/statistics", "//pkg/store/mockstore", - "//pkg/tablecodec", "//pkg/telemetry", "//pkg/testkit/testmain", "//pkg/testkit/testsetup", diff --git a/pkg/session/test/BUILD.bazel b/pkg/session/test/BUILD.bazel index 328aadcb5b3b4..a38ff77583362 100644 --- a/pkg/session/test/BUILD.bazel +++ b/pkg/session/test/BUILD.bazel @@ -6,9 +6,10 @@ go_test( srcs = [ "main_test.go", "session_test.go", + "tidb_test.go", ], flaky = True, - shard_count = 26, + shard_count = 28, deps = [ "//pkg/config", "//pkg/domain", @@ -19,8 +20,10 @@ go_test( "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", + "//pkg/planner/core", "//pkg/session", "//pkg/store/mockstore", + "//pkg/tablecodec", "//pkg/testkit", "//pkg/testkit/testmain", "//pkg/testkit/testsetup", diff --git a/pkg/session/test/tidb_test.go b/pkg/session/test/tidb_test.go new file mode 100644 index 0000000000000..4c185ffeca756 --- /dev/null +++ b/pkg/session/test/tidb_test.go @@ -0,0 +1,75 @@ +// Copyright 2023 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 test + +import ( + "testing" + + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/tablecodec" + "github.com/stretchr/testify/require" +) + +func TestParseErrorWarn(t *testing.T) { + ctx := core.MockContext() + defer func() { + domain.GetDomain(ctx).StatsHandle().Close() + }() + nodes, err := session.Parse(ctx, "select /*+ adf */ 1") + require.NoError(t, err) + require.Len(t, nodes, 1) + require.Len(t, ctx.GetSessionVars().StmtCtx.GetWarnings(), 1) + + _, err = session.Parse(ctx, "select") + require.Error(t, err) +} + +func TestKeysNeedLock(t *testing.T) { + rowKey := tablecodec.EncodeRowKeyWithHandle(1, kv.IntHandle(1)) + uniqueIndexKey := tablecodec.EncodeIndexSeekKey(1, 1, []byte{1}) + nonUniqueIndexKey := tablecodec.EncodeIndexSeekKey(1, 2, []byte{1}) + uniqueValue := make([]byte, 8) + uniqueUntouched := append(uniqueValue, '1') + nonUniqueVal := []byte{'0'} + nonUniqueUntouched := []byte{'1'} + var deleteVal []byte + rowVal := []byte{'a', 'b', 'c'} + tests := []struct { + key []byte + val []byte + need bool + }{ + {rowKey, rowVal, true}, + {rowKey, deleteVal, true}, + {nonUniqueIndexKey, nonUniqueVal, false}, + {nonUniqueIndexKey, nonUniqueUntouched, false}, + {uniqueIndexKey, uniqueValue, true}, + {uniqueIndexKey, uniqueUntouched, false}, + {uniqueIndexKey, deleteVal, false}, + } + + for _, test := range tests { + need := session.KeyNeedToLock(test.key, test.val, 0) + require.Equal(t, test.need, need) + + flag := kv.KeyFlags(1) + need = session.KeyNeedToLock(test.key, test.val, flag) + require.True(t, flag.HasPresumeKeyNotExists()) + require.True(t, need) + } +} diff --git a/pkg/session/tidb_test.go b/pkg/session/tidb_test.go index 527f2cba37c98..ef672898045f8 100644 --- a/pkg/session/tidb_test.go +++ b/pkg/session/tidb_test.go @@ -18,11 +18,8 @@ import ( "context" "testing" - "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" - "github.com/pingcap/tidb/pkg/planner/core" - "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util" "github.com/stretchr/testify/require" ) @@ -63,52 +60,3 @@ func TestSysSessionPoolGoroutineLeak(t *testing.T) { } wg.Wait() } - -func TestParseErrorWarn(t *testing.T) { - ctx := core.MockContext() - defer func() { - domain.GetDomain(ctx).StatsHandle().Close() - }() - nodes, err := Parse(ctx, "select /*+ adf */ 1") - require.NoError(t, err) - require.Len(t, nodes, 1) - require.Len(t, ctx.GetSessionVars().StmtCtx.GetWarnings(), 1) - - _, err = Parse(ctx, "select") - require.Error(t, err) -} - -func TestKeysNeedLock(t *testing.T) { - rowKey := tablecodec.EncodeRowKeyWithHandle(1, kv.IntHandle(1)) - uniqueIndexKey := tablecodec.EncodeIndexSeekKey(1, 1, []byte{1}) - nonUniqueIndexKey := tablecodec.EncodeIndexSeekKey(1, 2, []byte{1}) - uniqueValue := make([]byte, 8) - uniqueUntouched := append(uniqueValue, '1') - nonUniqueVal := []byte{'0'} - nonUniqueUntouched := []byte{'1'} - var deleteVal []byte - rowVal := []byte{'a', 'b', 'c'} - tests := []struct { - key []byte - val []byte - need bool - }{ - {rowKey, rowVal, true}, - {rowKey, deleteVal, true}, - {nonUniqueIndexKey, nonUniqueVal, false}, - {nonUniqueIndexKey, nonUniqueUntouched, false}, - {uniqueIndexKey, uniqueValue, true}, - {uniqueIndexKey, uniqueUntouched, false}, - {uniqueIndexKey, deleteVal, false}, - } - - for _, test := range tests { - need := keyNeedToLock(test.key, test.val, 0) - require.Equal(t, test.need, need) - - flag := kv.KeyFlags(1) - need = keyNeedToLock(test.key, test.val, flag) - require.True(t, flag.HasPresumeKeyNotExists()) - require.True(t, need) - } -} diff --git a/pkg/session/txn.go b/pkg/session/txn.go index 3ab0a6e22c781..17fdbbf1ea278 100644 --- a/pkg/session/txn.go +++ b/pkg/session/txn.go @@ -569,7 +569,7 @@ func (txn *LazyTxn) KeysNeedToLock() ([]kv.Key, error) { keys := make([]kv.Key, 0, txn.countHint()) buf := txn.Transaction.GetMemBuffer() buf.InspectStage(txn.stagingHandle, func(k kv.Key, flags kv.KeyFlags, v []byte) { - if !keyNeedToLock(k, v, flags) { + if !KeyNeedToLock(k, v, flags) { return } keys = append(keys, k) @@ -603,7 +603,8 @@ func (txn *LazyTxn) Wait(ctx context.Context, sctx sessionctx.Context) (kv.Trans return txn, nil } -func keyNeedToLock(k, v []byte, flags kv.KeyFlags) bool { +// KeyNeedToLock returns true if the key need to lock. +func KeyNeedToLock(k, v []byte, flags kv.KeyFlags) bool { isTableKey := bytes.HasPrefix(k, tablecodec.TablePrefix()) if !isTableKey { // meta key always need to lock. From a84ac1cbb15a12a6b4b901aefd4c5e9054092d43 Mon Sep 17 00:00:00 2001 From: Rustin Liu Date: Wed, 22 Nov 2023 17:43:12 +0800 Subject: [PATCH 08/36] github: fix some grammar issues in the pull request template (#48798) --- .github/pull_request_template.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index 6ba37f8ef01d6..251b45f4aa5f3 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -13,7 +13,7 @@ PR Title Format: Please create an issue first to describe the problem. -There MUST be one line starting with "Issue Number: " and +There MUST be one line starting with "Issue Number: " and linking the relevant issues via the "close" or "ref". For more info, check https://pingcap.github.io/tidb-dev-guide/contribute-to-tidb/contribute-code.html#referring-to-an-issue. @@ -24,7 +24,7 @@ Issue Number: close #xxx Problem Summary: -### What is changed and how it works? +### What changed and how does it work? ### Check List @@ -35,7 +35,7 @@ Tests - [ ] Manual test (add detailed scripts or steps below) - [ ] No need to test > - [ ] I checked and no code files have been changed. - > + > Side effects From 3ed7732958bde87c200e0888a8e7268499b6a115 Mon Sep 17 00:00:00 2001 From: Rustin Liu Date: Wed, 22 Nov 2023 18:47:42 +0800 Subject: [PATCH 09/36] planner: fix a panic during column pruning (#47883) close pingcap/tidb#47331 --- pkg/executor/explain_test.go | 51 +++++++++++++++++++++++++++ pkg/planner/core/rule_join_reorder.go | 3 +- 2 files changed, 53 insertions(+), 1 deletion(-) diff --git a/pkg/executor/explain_test.go b/pkg/executor/explain_test.go index cb6048953f701..1feaae20a5dbb 100644 --- a/pkg/executor/explain_test.go +++ b/pkg/executor/explain_test.go @@ -83,6 +83,57 @@ func checkMemoryInfo(t *testing.T, tk *testkit.TestKit, sql string) { } } +func TestIssue47331(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec(`create table t1( + id1 varchar(2) DEFAULT '00', + id2 varchar(30) NOT NULL, + id3 datetime DEFAULT NULL, + id4 varchar(100) NOT NULL DEFAULT 'ecifdata', + id5 datetime NOT NULL DEFAULT CURRENT_TIMESTAMP, + id6 int(11) DEFAULT NULL, + id7 int(11) DEFAULT NULL, + UNIQUE KEY UI_id2 (id2), + KEY ix_id1 (id1) + )`) + tk.MustExec("drop table if exists t2") + tk.MustExec(`create table t2( + id10 varchar(40) NOT NULL, + id2 varchar(30) NOT NULL, + KEY IX_id2 (id2), + PRIMARY KEY (id10) + )`) + tk.MustExec("drop table if exists t3") + tk.MustExec(`create table t3( + id20 varchar(40) DEFAULT NULL, + UNIQUE KEY IX_id20 (id20) + )`) + tk.MustExec(` + explain + UPDATE t1 a + SET a.id1 = '04', + a.id3 = CURRENT_TIMESTAMP, + a.id4 = SUBSTRING_INDEX(USER(), '@', 1), + a.id5 = CURRENT_TIMESTAMP + WHERE a.id1 = '03' + AND a.id6 - IFNULL(a.id7, 0) = + ( + SELECT COUNT(1) + FROM t2 b, t3 c + WHERE b.id10 = c.id20 + AND b.id2 = a.id2 + AND b.id2 in ( + SELECT rn.id2 + FROM t1 rn + WHERE rn.id1 = '03' + ) + ); + `) +} + func TestMemoryAndDiskUsageAfterClose(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/pkg/planner/core/rule_join_reorder.go b/pkg/planner/core/rule_join_reorder.go index 2dabaf7e77e4c..271778c7c8bdc 100644 --- a/pkg/planner/core/rule_join_reorder.go +++ b/pkg/planner/core/rule_join_reorder.go @@ -316,7 +316,8 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP proj := LogicalProjection{ Exprs: expression.Column2Exprs(originalSchema.Columns), }.Init(p.SCtx(), p.SelectBlockOffset()) - proj.SetSchema(originalSchema) + // Clone the schema here, because the schema may be changed by column pruning rules. + proj.SetSchema(originalSchema.Clone()) proj.SetChildren(p) p = proj } From d2cfbdef67f03eceb0225faa6b18c24e13a8d1ac Mon Sep 17 00:00:00 2001 From: tangenta Date: Wed, 22 Nov 2023 20:56:41 +0800 Subject: [PATCH 10/36] ddl,lightning: fix ingest data unexpectedly using zero commit ts (#48797) close pingcap/tidb#48804 --- br/pkg/lightning/backend/local/local.go | 6 +++ br/pkg/lightning/backend/local/region_job.go | 6 ++- pkg/ddl/ingest/backend.go | 40 +++++++++++--- pkg/ddl/ingest/message.go | 1 + tests/realtikvtest/addindextest1/BUILD.bazel | 4 ++ .../addindextest1/disttask_test.go | 54 +++++++++++++++++++ .../realtikvtest/addindextest4/ingest_test.go | 1 + 7 files changed, 104 insertions(+), 8 deletions(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 34ee420c461a3..c526188bf55fd 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -1761,6 +1761,12 @@ func (local *Backend) ResetEngine(ctx context.Context, engineUUID uuid.UUID) err if err = local.allocateTSIfNotExists(ctx, localEngine); err != nil { return errors.Trace(err) } + failpoint.Inject("mockAllocateTSErr", func() { + // mock generate timestamp error when reset engine. + localEngine.TS = 0 + mockGRPCErr, _ := status.FromError(errors.Errorf("mock generate timestamp error")) + failpoint.Return(errors.Trace(mockGRPCErr.Err())) + }) } localEngine.pendingFileSize.Store(0) diff --git a/br/pkg/lightning/backend/local/region_job.go b/br/pkg/lightning/backend/local/region_job.go index 34e9f9ab3ffa9..1c25434974c55 100644 --- a/br/pkg/lightning/backend/local/region_job.go +++ b/br/pkg/lightning/backend/local/region_job.go @@ -318,9 +318,10 @@ func (local *Backend) doWrite(ctx context.Context, j *regionJob) error { clients = append(clients, wstream) allPeers = append(allPeers, peer) } + dataCommitTS := j.ingestData.GetTS() req.Chunk = &sst.WriteRequest_Batch{ Batch: &sst.WriteBatch{ - CommitTs: j.ingestData.GetTS(), + CommitTs: dataCommitTS, }, } @@ -407,7 +408,8 @@ func (local *Backend) doWrite(ctx context.Context, j *regionJob) error { logutil.Key("endKey", j.keyRange.End), logutil.Key("remainStart", remainingStartKey), logutil.Region(region), - logutil.Leader(j.region.Leader)) + logutil.Leader(j.region.Leader), + zap.Uint64("commitTS", dataCommitTS)) } break } diff --git a/pkg/ddl/ingest/backend.go b/pkg/ddl/ingest/backend.go index 3d4bf54a68b33..130faa466156c 100644 --- a/pkg/ddl/ingest/backend.go +++ b/pkg/ddl/ingest/backend.go @@ -20,6 +20,7 @@ import ( "time" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/encode" "github.com/pingcap/tidb/br/pkg/lightning/backend/local" lightning "github.com/pingcap/tidb/br/pkg/lightning/config" @@ -214,18 +215,45 @@ func (bc *litBackendCtx) Flush(indexID int64, mode FlushMode) (flushed, imported } }() } - - logutil.Logger(bc.ctx).Info(LitInfoUnsafeImport, zap.Int64("index ID", indexID), - zap.String("usage info", bc.diskRoot.UsageInfo())) - err = bc.backend.UnsafeImportAndReset(bc.ctx, ei.uuid, int64(lightning.SplitRegionSize)*int64(lightning.MaxSplitRegionSizeRatio), int64(lightning.SplitRegionKeys)) + err = bc.unsafeImportAndReset(ei) if err != nil { - logutil.Logger(bc.ctx).Error(LitErrIngestDataErr, zap.Int64("index ID", indexID), - zap.String("usage info", bc.diskRoot.UsageInfo())) return true, false, err } return true, true, nil } +func (bc *litBackendCtx) unsafeImportAndReset(ei *engineInfo) error { + logutil.Logger(bc.ctx).Info(LitInfoUnsafeImport, zap.Int64("index ID", ei.indexID), + zap.String("usage info", bc.diskRoot.UsageInfo())) + logger := log.FromContext(bc.ctx).With( + zap.Stringer("engineUUID", ei.uuid), + ) + + ei.closedEngine = backend.NewClosedEngine(bc.backend, logger, ei.uuid, 0) + + regionSplitSize := int64(lightning.SplitRegionSize) * int64(lightning.MaxSplitRegionSizeRatio) + regionSplitKeys := int64(lightning.SplitRegionKeys) + if err := ei.closedEngine.Import(bc.ctx, regionSplitSize, regionSplitKeys); err != nil { + logutil.Logger(bc.ctx).Error(LitErrIngestDataErr, zap.Int64("index ID", ei.indexID), + zap.String("usage info", bc.diskRoot.UsageInfo())) + return err + } + + err := bc.backend.ResetEngine(bc.ctx, ei.uuid) + if err != nil { + logutil.Logger(bc.ctx).Error(LitErrResetEngineFail, zap.Int64("index ID", ei.indexID)) + err1 := ei.closedEngine.Cleanup(bc.ctx) + if err1 != nil { + logutil.Logger(ei.ctx).Error(LitErrCleanEngineErr, zap.Error(err1), + zap.Int64("job ID", ei.jobID), zap.Int64("index ID", ei.indexID)) + } + ei.openedEngine = nil + ei.closedEngine = nil + return err + } + return nil +} + // ForceSyncFlagForTest is a flag to force sync only for test. var ForceSyncFlagForTest = false diff --git a/pkg/ddl/ingest/message.go b/pkg/ddl/ingest/message.go index 0f74de19dd0b4..1217244c6f2ac 100644 --- a/pkg/ddl/ingest/message.go +++ b/pkg/ddl/ingest/message.go @@ -42,6 +42,7 @@ const ( LitErrCloseWriterErr string = "close writer error" LitErrReadSortPath string = "cannot read sort path" LitErrCleanSortPath string = "cannot cleanup sort path" + LitErrResetEngineFail string = "reset engine failed" LitWarnEnvInitFail string = "initialize environment failed" LitWarnConfigError string = "build config for backend failed" LitInfoEnvInitSucc string = "init global ingest backend environment finished" diff --git a/tests/realtikvtest/addindextest1/BUILD.bazel b/tests/realtikvtest/addindextest1/BUILD.bazel index 448de5e3da98c..9126f15f9bca9 100644 --- a/tests/realtikvtest/addindextest1/BUILD.bazel +++ b/tests/realtikvtest/addindextest1/BUILD.bazel @@ -15,8 +15,12 @@ go_test( "//pkg/disttask/framework/dispatcher", "//pkg/disttask/framework/proto", "//pkg/errno", + "//pkg/kv", "//pkg/parser/model", + "//pkg/store/helper", + "//pkg/tablecodec", "//pkg/testkit", + "//pkg/types", "//tests/realtikvtest", "@com_github_pingcap_failpoint//:failpoint", "@com_github_stretchr_testify//require", diff --git a/tests/realtikvtest/addindextest1/disttask_test.go b/tests/realtikvtest/addindextest1/disttask_test.go index eca65ea88fc3c..27715a0613ecb 100644 --- a/tests/realtikvtest/addindextest1/disttask_test.go +++ b/tests/realtikvtest/addindextest1/disttask_test.go @@ -24,8 +24,12 @@ import ( "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/proto" "github.com/pingcap/tidb/pkg/errno" + "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/store/helper" + "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/tests/realtikvtest" "github.com/stretchr/testify/require" ) @@ -229,3 +233,53 @@ func TestAddIndexForCurrentTimestampColumn(t *testing.T) { tk.MustExec("alter table t add index idx(a);") tk.MustExec("admin check table t;") } + +func TestAddIndexTSErrorWhenResetImportEngine(t *testing.T) { + store, dom := realtikvtest.CreateMockStoreAndDomainAndSetup(t) + var tblInfo *model.TableInfo + var idxInfo *model.IndexInfo + cb := &callback.TestDDLCallback{} + interceptFn := func(job *model.Job) { + if idxInfo == nil { + tbl, _ := dom.InfoSchema().TableByID(job.TableID) + tblInfo = tbl.Meta() + if len(tblInfo.Indices) == 0 { + return + } + idxInfo = tblInfo.Indices[0] + } + } + cb.OnJobUpdatedExported.Store(&interceptFn) + tk := testkit.NewTestKit(t, store) + tk.MustExec("drop database if exists addindexlit;") + tk.MustExec("create database addindexlit;") + tk.MustExec("use addindexlit;") + t.Cleanup(func() { + tk.MustExec("set global tidb_enable_dist_task = off;") + }) + tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`) + tk.MustExec("set global tidb_enable_dist_task = on;") + + err := failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/mockAllocateTSErr", `1*return`) + require.NoError(t, err) + tk.MustExec("create table t (a int);") + tk.MustExec("insert into t values (1), (2), (3);") + dom.DDL().SetHook(cb) + tk.MustExec("alter table t add index idx(a);") + err = failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/mockAllocateTSErr") + require.NoError(t, err) + + dts := []types.Datum{types.NewIntDatum(1)} + sctx := tk.Session().GetSessionVars().StmtCtx + idxKey, _, err := tablecodec.GenIndexKey(sctx, tblInfo, idxInfo, tblInfo.ID, dts, kv.IntHandle(1), nil) + require.NoError(t, err) + + tikvStore := dom.Store().(helper.Storage) + newHelper := helper.NewHelper(tikvStore) + mvccResp, err := newHelper.GetMvccByEncodedKeyWithTS(idxKey, 0) + require.NoError(t, err) + require.NotNil(t, mvccResp) + require.NotNil(t, mvccResp.Info) + require.Greater(t, len(mvccResp.Info.Writes), 0) + require.Greater(t, mvccResp.Info.Writes[0].CommitTs, uint64(0)) +} diff --git a/tests/realtikvtest/addindextest4/ingest_test.go b/tests/realtikvtest/addindextest4/ingest_test.go index 8b5b9683aa0d6..8d2ab0c8b075b 100644 --- a/tests/realtikvtest/addindextest4/ingest_test.go +++ b/tests/realtikvtest/addindextest4/ingest_test.go @@ -399,6 +399,7 @@ func TestAddIndexRemoteDuplicateCheck(t *testing.T) { tk.MustExec("use addindexlit;") tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`) tk.MustExec("set global tidb_ddl_reorg_worker_cnt=1;") + tk.MustExec("set global tidb_enable_dist_task = 0;") tk.MustExec("create table t(id int primary key, b int, k int);") tk.MustQuery("split table t by (30000);").Check(testkit.Rows("1 1")) From fcda2f8ad2fc1cfc482b5acd8e2f949b934ca9a9 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 22 Nov 2023 22:32:42 +0800 Subject: [PATCH 11/36] *: upgrade otelgrpc to fix security problem (#48800) close pingcap/tidb#48799 --- DEPS.bzl | 1527 ++++++++++++++++++++++++++++-------------------------- go.mod | 32 +- go.sum | 75 ++- 3 files changed, 837 insertions(+), 797 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 2ca31f47279a9..4ae046fa60f2d 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2322,13 +2322,26 @@ def go_deps(): name = "com_github_go_logr_logr", build_file_proto_mode = "disable_global", importpath = "github.com/go-logr/logr", - sha256 = "4b4b79b5863ab1d35c329d34a9cbba2a58a9b83b4a33c426facd2aa73b132f04", - strip_prefix = "github.com/go-logr/logr@v1.2.3", + sha256 = "9f2fe2600670561e7ea60903e736f3e38c304bfd217d0b06194daa1cf04a904f", + strip_prefix = "github.com/go-logr/logr@v1.3.0", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/github.com/go-logr/logr/com_github_go_logr_logr-v1.2.3.zip", - "http://ats.apps.svc/gomod/github.com/go-logr/logr/com_github_go_logr_logr-v1.2.3.zip", - "https://cache.hawkingrei.com/gomod/github.com/go-logr/logr/com_github_go_logr_logr-v1.2.3.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/github.com/go-logr/logr/com_github_go_logr_logr-v1.2.3.zip", + "http://bazel-cache.pingcap.net:8080/gomod/github.com/go-logr/logr/com_github_go_logr_logr-v1.3.0.zip", + "http://ats.apps.svc/gomod/github.com/go-logr/logr/com_github_go_logr_logr-v1.3.0.zip", + "https://cache.hawkingrei.com/gomod/github.com/go-logr/logr/com_github_go_logr_logr-v1.3.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/go-logr/logr/com_github_go_logr_logr-v1.3.0.zip", + ], + ) + go_repository( + name = "com_github_go_logr_stdr", + build_file_proto_mode = "disable_global", + importpath = "github.com/go-logr/stdr", + sha256 = "9dd6893bf700198485ae699640b49bc1efbc6c73b37cb5792a0476e1fd8f7fef", + strip_prefix = "github.com/go-logr/stdr@v1.2.2", + urls = [ + "http://bazel-cache.pingcap.net:8080/gomod/github.com/go-logr/stdr/com_github_go_logr_stdr-v1.2.2.zip", + "http://ats.apps.svc/gomod/github.com/go-logr/stdr/com_github_go_logr_stdr-v1.2.2.zip", + "https://cache.hawkingrei.com/gomod/github.com/go-logr/stdr/com_github_go_logr_stdr-v1.2.2.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/go-logr/stdr/com_github_go_logr_stdr-v1.2.2.zip", ], ) go_repository( @@ -3011,6 +3024,19 @@ def go_deps(): "https://storage.googleapis.com/pingcapmirror/gomod/github.com/google/go-github/v33/com_github_google_go_github_v33-v33.0.0.zip", ], ) + go_repository( + name = "com_github_google_go_pkcs11", + build_file_proto_mode = "disable_global", + importpath = "github.com/google/go-pkcs11", + sha256 = "b9bf12c2450efa77c8b27134d5f206633057fcf0c324883797d7fde5bc3a4887", + strip_prefix = "github.com/google/go-pkcs11@v0.2.1-0.20230907215043-c6f79328ddf9", + urls = [ + "http://bazel-cache.pingcap.net:8080/gomod/github.com/google/go-pkcs11/com_github_google_go_pkcs11-v0.2.1-0.20230907215043-c6f79328ddf9.zip", + "http://ats.apps.svc/gomod/github.com/google/go-pkcs11/com_github_google_go_pkcs11-v0.2.1-0.20230907215043-c6f79328ddf9.zip", + "https://cache.hawkingrei.com/gomod/github.com/google/go-pkcs11/com_github_google_go_pkcs11-v0.2.1-0.20230907215043-c6f79328ddf9.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/google/go-pkcs11/com_github_google_go_pkcs11-v0.2.1-0.20230907215043-c6f79328ddf9.zip", + ], + ) go_repository( name = "com_github_google_go_querystring", build_file_proto_mode = "disable_global", @@ -3119,13 +3145,13 @@ def go_deps(): name = "com_github_google_s2a_go", build_file_proto_mode = "disable_global", importpath = "github.com/google/s2a-go", - sha256 = "b01ff39fc8c27f944da1c3f78106d57e165f234d98115c344f448b603ae24ff3", - strip_prefix = "github.com/google/s2a-go@v0.1.4", + sha256 = "4392e675b6f0ff0b90f970c0280d63e34b32d077e1f8c0abd1006ad0dbeb2f2e", + strip_prefix = "github.com/google/s2a-go@v0.1.7", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/github.com/google/s2a-go/com_github_google_s2a_go-v0.1.4.zip", - "http://ats.apps.svc/gomod/github.com/google/s2a-go/com_github_google_s2a_go-v0.1.4.zip", - "https://cache.hawkingrei.com/gomod/github.com/google/s2a-go/com_github_google_s2a_go-v0.1.4.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/github.com/google/s2a-go/com_github_google_s2a_go-v0.1.4.zip", + "http://bazel-cache.pingcap.net:8080/gomod/github.com/google/s2a-go/com_github_google_s2a_go-v0.1.7.zip", + "http://ats.apps.svc/gomod/github.com/google/s2a-go/com_github_google_s2a_go-v0.1.7.zip", + "https://cache.hawkingrei.com/gomod/github.com/google/s2a-go/com_github_google_s2a_go-v0.1.7.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/google/s2a-go/com_github_google_s2a_go-v0.1.7.zip", ], ) go_repository( @@ -3145,26 +3171,26 @@ def go_deps(): name = "com_github_google_uuid", build_file_proto_mode = "disable_global", importpath = "github.com/google/uuid", - sha256 = "9d9d6cfb28ce6dbe4b518c42c6bccd67bb531a106859808f36e82a5c3fb8c64d", - strip_prefix = "github.com/google/uuid@v1.3.1", + sha256 = "15ff3397648edf4999d37359b3fc041212d49bcd6b457327c0f8dff19f594050", + strip_prefix = "github.com/google/uuid@v1.4.0", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/github.com/google/uuid/com_github_google_uuid-v1.3.1.zip", - "http://ats.apps.svc/gomod/github.com/google/uuid/com_github_google_uuid-v1.3.1.zip", - "https://cache.hawkingrei.com/gomod/github.com/google/uuid/com_github_google_uuid-v1.3.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/github.com/google/uuid/com_github_google_uuid-v1.3.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/github.com/google/uuid/com_github_google_uuid-v1.4.0.zip", + "http://ats.apps.svc/gomod/github.com/google/uuid/com_github_google_uuid-v1.4.0.zip", + "https://cache.hawkingrei.com/gomod/github.com/google/uuid/com_github_google_uuid-v1.4.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/google/uuid/com_github_google_uuid-v1.4.0.zip", ], ) go_repository( name = "com_github_googleapis_enterprise_certificate_proxy", build_file_proto_mode = "disable_global", importpath = "github.com/googleapis/enterprise-certificate-proxy", - sha256 = "f0642434f18b33f21c5b2a908907f4c3ae24223791c1c4b92d13d351bfa7ed7e", - strip_prefix = "github.com/googleapis/enterprise-certificate-proxy@v0.2.4", + sha256 = "56127cb8bea94f438c4e867f9217bdfc55865282953e54c74eee019575c1020e", + strip_prefix = "github.com/googleapis/enterprise-certificate-proxy@v0.3.2", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/github.com/googleapis/enterprise-certificate-proxy/com_github_googleapis_enterprise_certificate_proxy-v0.2.4.zip", - "http://ats.apps.svc/gomod/github.com/googleapis/enterprise-certificate-proxy/com_github_googleapis_enterprise_certificate_proxy-v0.2.4.zip", - "https://cache.hawkingrei.com/gomod/github.com/googleapis/enterprise-certificate-proxy/com_github_googleapis_enterprise_certificate_proxy-v0.2.4.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/github.com/googleapis/enterprise-certificate-proxy/com_github_googleapis_enterprise_certificate_proxy-v0.2.4.zip", + "http://bazel-cache.pingcap.net:8080/gomod/github.com/googleapis/enterprise-certificate-proxy/com_github_googleapis_enterprise_certificate_proxy-v0.3.2.zip", + "http://ats.apps.svc/gomod/github.com/googleapis/enterprise-certificate-proxy/com_github_googleapis_enterprise_certificate_proxy-v0.3.2.zip", + "https://cache.hawkingrei.com/gomod/github.com/googleapis/enterprise-certificate-proxy/com_github_googleapis_enterprise_certificate_proxy-v0.3.2.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/googleapis/enterprise-certificate-proxy/com_github_googleapis_enterprise_certificate_proxy-v0.3.2.zip", ], ) go_repository( @@ -7730,338 +7756,338 @@ def go_deps(): name = "com_google_cloud_go", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go", - sha256 = "3d0ed6092ddd6ffdc4ec4f39e627a706c8d71e09330768c8174428db289d21a4", - strip_prefix = "cloud.google.com/go@v0.110.8", + sha256 = "65e6a32b316f200d6b3cf58f2a160dc6f68a85f073dca24c0513f74f30c78471", + strip_prefix = "cloud.google.com/go@v0.110.10", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/com_google_cloud_go-v0.110.8.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/com_google_cloud_go-v0.110.8.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/com_google_cloud_go-v0.110.8.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/com_google_cloud_go-v0.110.8.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/com_google_cloud_go-v0.110.10.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/com_google_cloud_go-v0.110.10.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/com_google_cloud_go-v0.110.10.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/com_google_cloud_go-v0.110.10.zip", ], ) go_repository( name = "com_google_cloud_go_accessapproval", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/accessapproval", - sha256 = "e81216a40f4ed1779d4fd2a031ba6df523c9dc4cbe459ed8e746b6044c865248", - strip_prefix = "cloud.google.com/go/accessapproval@v1.7.2", + sha256 = "1e41e9d3de183c5a7f18e92e7fb8f8225b0875bf1aea89ab5d4744fddcbe64e3", + strip_prefix = "cloud.google.com/go/accessapproval@v1.7.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/accessapproval/com_google_cloud_go_accessapproval-v1.7.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/accessapproval/com_google_cloud_go_accessapproval-v1.7.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/accessapproval/com_google_cloud_go_accessapproval-v1.7.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/accessapproval/com_google_cloud_go_accessapproval-v1.7.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/accessapproval/com_google_cloud_go_accessapproval-v1.7.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/accessapproval/com_google_cloud_go_accessapproval-v1.7.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/accessapproval/com_google_cloud_go_accessapproval-v1.7.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/accessapproval/com_google_cloud_go_accessapproval-v1.7.4.zip", ], ) go_repository( name = "com_google_cloud_go_accesscontextmanager", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/accesscontextmanager", - sha256 = "4010c95bbdceab4050e4b931bdbe978f50e83de78034c9671a23c414bdb97dc3", - strip_prefix = "cloud.google.com/go/accesscontextmanager@v1.8.2", + sha256 = "19144f658a09ae2e8414ea683e1a40eac484e66177ca80224c77b622d18f7304", + strip_prefix = "cloud.google.com/go/accesscontextmanager@v1.8.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/accesscontextmanager/com_google_cloud_go_accesscontextmanager-v1.8.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/accesscontextmanager/com_google_cloud_go_accesscontextmanager-v1.8.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/accesscontextmanager/com_google_cloud_go_accesscontextmanager-v1.8.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/accesscontextmanager/com_google_cloud_go_accesscontextmanager-v1.8.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/accesscontextmanager/com_google_cloud_go_accesscontextmanager-v1.8.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/accesscontextmanager/com_google_cloud_go_accesscontextmanager-v1.8.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/accesscontextmanager/com_google_cloud_go_accesscontextmanager-v1.8.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/accesscontextmanager/com_google_cloud_go_accesscontextmanager-v1.8.4.zip", ], ) go_repository( name = "com_google_cloud_go_aiplatform", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/aiplatform", - sha256 = "9f49dbeaf81d8bf076b2a23f3d2c822f4d60ab41997ca8e9db081ad2e0945e42", - strip_prefix = "cloud.google.com/go/aiplatform@v1.51.1", + sha256 = "ac9a8ef2af71031070a4601a8c617b2d87f7fc6d1a3372be8c3c43ef66c98fb6", + strip_prefix = "cloud.google.com/go/aiplatform@v1.52.0", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/aiplatform/com_google_cloud_go_aiplatform-v1.51.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/aiplatform/com_google_cloud_go_aiplatform-v1.51.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/aiplatform/com_google_cloud_go_aiplatform-v1.51.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/aiplatform/com_google_cloud_go_aiplatform-v1.51.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/aiplatform/com_google_cloud_go_aiplatform-v1.52.0.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/aiplatform/com_google_cloud_go_aiplatform-v1.52.0.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/aiplatform/com_google_cloud_go_aiplatform-v1.52.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/aiplatform/com_google_cloud_go_aiplatform-v1.52.0.zip", ], ) go_repository( name = "com_google_cloud_go_analytics", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/analytics", - sha256 = "1f0b79e19aa45178c06545ac0eb1ec067583f7742520c933d25722ebdb8d1c2c", - strip_prefix = "cloud.google.com/go/analytics@v0.21.4", + sha256 = "4bc313552680c60feb0171f8e940913a8a61fc3830cb8973833613019528d96b", + strip_prefix = "cloud.google.com/go/analytics@v0.21.6", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/analytics/com_google_cloud_go_analytics-v0.21.4.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/analytics/com_google_cloud_go_analytics-v0.21.4.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/analytics/com_google_cloud_go_analytics-v0.21.4.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/analytics/com_google_cloud_go_analytics-v0.21.4.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/analytics/com_google_cloud_go_analytics-v0.21.6.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/analytics/com_google_cloud_go_analytics-v0.21.6.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/analytics/com_google_cloud_go_analytics-v0.21.6.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/analytics/com_google_cloud_go_analytics-v0.21.6.zip", ], ) go_repository( name = "com_google_cloud_go_apigateway", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/apigateway", - sha256 = "d3a522706734344ff09513c72b05a8e39bbfe093f9cbab07c3c081698306b014", - strip_prefix = "cloud.google.com/go/apigateway@v1.6.2", + sha256 = "32d4450fb4849b0aae790de816ec13d85f58bb53d295ffa685767fa196ec0069", + strip_prefix = "cloud.google.com/go/apigateway@v1.6.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/apigateway/com_google_cloud_go_apigateway-v1.6.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/apigateway/com_google_cloud_go_apigateway-v1.6.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/apigateway/com_google_cloud_go_apigateway-v1.6.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/apigateway/com_google_cloud_go_apigateway-v1.6.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/apigateway/com_google_cloud_go_apigateway-v1.6.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/apigateway/com_google_cloud_go_apigateway-v1.6.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/apigateway/com_google_cloud_go_apigateway-v1.6.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/apigateway/com_google_cloud_go_apigateway-v1.6.4.zip", ], ) go_repository( name = "com_google_cloud_go_apigeeconnect", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/apigeeconnect", - sha256 = "5d6c4ab3a4a0e921c26b073f7c29ad7dcaff23eef07bd510f2d42fc2a4bb9fc9", - strip_prefix = "cloud.google.com/go/apigeeconnect@v1.6.2", + sha256 = "d26fb32c3e2e3b47d16031bad1c28dd7f277860c5f74d67286a02599fcc3a32c", + strip_prefix = "cloud.google.com/go/apigeeconnect@v1.6.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/apigeeconnect/com_google_cloud_go_apigeeconnect-v1.6.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/apigeeconnect/com_google_cloud_go_apigeeconnect-v1.6.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/apigeeconnect/com_google_cloud_go_apigeeconnect-v1.6.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/apigeeconnect/com_google_cloud_go_apigeeconnect-v1.6.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/apigeeconnect/com_google_cloud_go_apigeeconnect-v1.6.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/apigeeconnect/com_google_cloud_go_apigeeconnect-v1.6.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/apigeeconnect/com_google_cloud_go_apigeeconnect-v1.6.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/apigeeconnect/com_google_cloud_go_apigeeconnect-v1.6.4.zip", ], ) go_repository( name = "com_google_cloud_go_apigeeregistry", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/apigeeregistry", - sha256 = "949009434d483756469a40a091e20b979fde2811df9a7f9d5955e1cceecb9b65", - strip_prefix = "cloud.google.com/go/apigeeregistry@v0.7.2", + sha256 = "79691228fa1d9d9ab252bcd702401e20a4cad8965909c9cffdfef7686be57a74", + strip_prefix = "cloud.google.com/go/apigeeregistry@v0.8.2", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/apigeeregistry/com_google_cloud_go_apigeeregistry-v0.7.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/apigeeregistry/com_google_cloud_go_apigeeregistry-v0.7.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/apigeeregistry/com_google_cloud_go_apigeeregistry-v0.7.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/apigeeregistry/com_google_cloud_go_apigeeregistry-v0.7.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/apigeeregistry/com_google_cloud_go_apigeeregistry-v0.8.2.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/apigeeregistry/com_google_cloud_go_apigeeregistry-v0.8.2.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/apigeeregistry/com_google_cloud_go_apigeeregistry-v0.8.2.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/apigeeregistry/com_google_cloud_go_apigeeregistry-v0.8.2.zip", ], ) go_repository( name = "com_google_cloud_go_appengine", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/appengine", - sha256 = "ef8ebfc267080d470ebe4bcebef59e8bae90a1418b3f03b27f334e2058d4517e", - strip_prefix = "cloud.google.com/go/appengine@v1.8.2", + sha256 = "5179d561ca87d19aec04a7b495c1f48093b317491463093a00fc0b2842cbdbfd", + strip_prefix = "cloud.google.com/go/appengine@v1.8.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/appengine/com_google_cloud_go_appengine-v1.8.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/appengine/com_google_cloud_go_appengine-v1.8.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/appengine/com_google_cloud_go_appengine-v1.8.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/appengine/com_google_cloud_go_appengine-v1.8.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/appengine/com_google_cloud_go_appengine-v1.8.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/appengine/com_google_cloud_go_appengine-v1.8.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/appengine/com_google_cloud_go_appengine-v1.8.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/appengine/com_google_cloud_go_appengine-v1.8.4.zip", ], ) go_repository( name = "com_google_cloud_go_area120", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/area120", - sha256 = "04c79c0f28dce15cc4c3ff476995e9691431417b85293b8b202923ea85c2bab5", - strip_prefix = "cloud.google.com/go/area120@v0.8.2", + sha256 = "89798c53eae268b748c706963ca24d6513dcbb2b321ba4294c3c65c7df931db3", + strip_prefix = "cloud.google.com/go/area120@v0.8.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/area120/com_google_cloud_go_area120-v0.8.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/area120/com_google_cloud_go_area120-v0.8.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/area120/com_google_cloud_go_area120-v0.8.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/area120/com_google_cloud_go_area120-v0.8.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/area120/com_google_cloud_go_area120-v0.8.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/area120/com_google_cloud_go_area120-v0.8.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/area120/com_google_cloud_go_area120-v0.8.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/area120/com_google_cloud_go_area120-v0.8.4.zip", ], ) go_repository( name = "com_google_cloud_go_artifactregistry", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/artifactregistry", - sha256 = "121b1aba80e678166214cdcf45093fdface59a86ff7a930f3a44381e9c3c2f43", - strip_prefix = "cloud.google.com/go/artifactregistry@v1.14.3", + sha256 = "5514a73eefd4a0f7a3120d2d932272e982e31c13be07e234f9d6e76e3436bf7c", + strip_prefix = "cloud.google.com/go/artifactregistry@v1.14.6", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/artifactregistry/com_google_cloud_go_artifactregistry-v1.14.3.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/artifactregistry/com_google_cloud_go_artifactregistry-v1.14.3.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/artifactregistry/com_google_cloud_go_artifactregistry-v1.14.3.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/artifactregistry/com_google_cloud_go_artifactregistry-v1.14.3.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/artifactregistry/com_google_cloud_go_artifactregistry-v1.14.6.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/artifactregistry/com_google_cloud_go_artifactregistry-v1.14.6.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/artifactregistry/com_google_cloud_go_artifactregistry-v1.14.6.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/artifactregistry/com_google_cloud_go_artifactregistry-v1.14.6.zip", ], ) go_repository( name = "com_google_cloud_go_asset", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/asset", - sha256 = "8c99032799a39f65d87c1ed91bbaebe5ed2b84675231933106b0a8b48997214a", - strip_prefix = "cloud.google.com/go/asset@v1.15.1", + sha256 = "98a53c57e877c9917eb8ef5427f31127fefa99ec4f2e2695f83479f1673c55a2", + strip_prefix = "cloud.google.com/go/asset@v1.15.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/asset/com_google_cloud_go_asset-v1.15.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/asset/com_google_cloud_go_asset-v1.15.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/asset/com_google_cloud_go_asset-v1.15.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/asset/com_google_cloud_go_asset-v1.15.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/asset/com_google_cloud_go_asset-v1.15.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/asset/com_google_cloud_go_asset-v1.15.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/asset/com_google_cloud_go_asset-v1.15.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/asset/com_google_cloud_go_asset-v1.15.3.zip", ], ) go_repository( name = "com_google_cloud_go_assuredworkloads", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/assuredworkloads", - sha256 = "31788ef9db908faea41ecbd7f8d49246d47e77897029f153df641759f9456e78", - strip_prefix = "cloud.google.com/go/assuredworkloads@v1.11.2", + sha256 = "65cc222f6804acb22126ab3b0233eb397b49b54a6005547041ead1ec333cf173", + strip_prefix = "cloud.google.com/go/assuredworkloads@v1.11.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/assuredworkloads/com_google_cloud_go_assuredworkloads-v1.11.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/assuredworkloads/com_google_cloud_go_assuredworkloads-v1.11.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/assuredworkloads/com_google_cloud_go_assuredworkloads-v1.11.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/assuredworkloads/com_google_cloud_go_assuredworkloads-v1.11.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/assuredworkloads/com_google_cloud_go_assuredworkloads-v1.11.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/assuredworkloads/com_google_cloud_go_assuredworkloads-v1.11.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/assuredworkloads/com_google_cloud_go_assuredworkloads-v1.11.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/assuredworkloads/com_google_cloud_go_assuredworkloads-v1.11.4.zip", ], ) go_repository( name = "com_google_cloud_go_automl", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/automl", - sha256 = "2d4aea70974f6409654bad3125ae1d80b810a1cb1777aee622c8502dd52c6693", - strip_prefix = "cloud.google.com/go/automl@v1.13.2", + sha256 = "9bfb0e21231d251103bb04d5334cb64951c17d8e4c55ff191472a7c3ce758f94", + strip_prefix = "cloud.google.com/go/automl@v1.13.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/automl/com_google_cloud_go_automl-v1.13.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/automl/com_google_cloud_go_automl-v1.13.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/automl/com_google_cloud_go_automl-v1.13.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/automl/com_google_cloud_go_automl-v1.13.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/automl/com_google_cloud_go_automl-v1.13.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/automl/com_google_cloud_go_automl-v1.13.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/automl/com_google_cloud_go_automl-v1.13.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/automl/com_google_cloud_go_automl-v1.13.4.zip", ], ) go_repository( name = "com_google_cloud_go_baremetalsolution", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/baremetalsolution", - sha256 = "79a2af9446dad6522ffaf60e3da8f564813b6a3ec7d71f43080f86e49bf90460", - strip_prefix = "cloud.google.com/go/baremetalsolution@v1.2.1", + sha256 = "44292fd306af8a1997d7fafcbe1b5bb85afdd1ed5d924a6b3030c3fd49c3bc88", + strip_prefix = "cloud.google.com/go/baremetalsolution@v1.2.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/baremetalsolution/com_google_cloud_go_baremetalsolution-v1.2.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/baremetalsolution/com_google_cloud_go_baremetalsolution-v1.2.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/baremetalsolution/com_google_cloud_go_baremetalsolution-v1.2.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/baremetalsolution/com_google_cloud_go_baremetalsolution-v1.2.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/baremetalsolution/com_google_cloud_go_baremetalsolution-v1.2.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/baremetalsolution/com_google_cloud_go_baremetalsolution-v1.2.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/baremetalsolution/com_google_cloud_go_baremetalsolution-v1.2.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/baremetalsolution/com_google_cloud_go_baremetalsolution-v1.2.3.zip", ], ) go_repository( name = "com_google_cloud_go_batch", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/batch", - sha256 = "d06b057177356f5d91140f59c2015a295f98f040f81d54b25f0d0bff230e3b2a", - strip_prefix = "cloud.google.com/go/batch@v1.5.1", + sha256 = "d7f09a5da02b32c3e00759974de19b562b33869c43de1dbfa65a7b3e8c916ac4", + strip_prefix = "cloud.google.com/go/batch@v1.6.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/batch/com_google_cloud_go_batch-v1.5.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/batch/com_google_cloud_go_batch-v1.5.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/batch/com_google_cloud_go_batch-v1.5.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/batch/com_google_cloud_go_batch-v1.5.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/batch/com_google_cloud_go_batch-v1.6.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/batch/com_google_cloud_go_batch-v1.6.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/batch/com_google_cloud_go_batch-v1.6.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/batch/com_google_cloud_go_batch-v1.6.3.zip", ], ) go_repository( name = "com_google_cloud_go_beyondcorp", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/beyondcorp", - sha256 = "e7497be44bc10e4c468a3b100f65ae8e3d351034544c2feb1447f54300659bfd", - strip_prefix = "cloud.google.com/go/beyondcorp@v1.0.1", + sha256 = "bf9fe334b6f6ce1896e0339eaea1a6e39405cb37cee0dddbea4f1f0545af267a", + strip_prefix = "cloud.google.com/go/beyondcorp@v1.0.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/beyondcorp/com_google_cloud_go_beyondcorp-v1.0.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/beyondcorp/com_google_cloud_go_beyondcorp-v1.0.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/beyondcorp/com_google_cloud_go_beyondcorp-v1.0.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/beyondcorp/com_google_cloud_go_beyondcorp-v1.0.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/beyondcorp/com_google_cloud_go_beyondcorp-v1.0.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/beyondcorp/com_google_cloud_go_beyondcorp-v1.0.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/beyondcorp/com_google_cloud_go_beyondcorp-v1.0.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/beyondcorp/com_google_cloud_go_beyondcorp-v1.0.3.zip", ], ) go_repository( name = "com_google_cloud_go_bigquery", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/bigquery", - sha256 = "e876ce8407d288df3075f142c19c429540a7a917b1fdd6dd68b3438ad8349412", - strip_prefix = "cloud.google.com/go/bigquery@v1.56.0", + sha256 = "fdf3bd1f87405dfdd1942182302c0954fdb09f05268ff632cf6ce686fec68444", + strip_prefix = "cloud.google.com/go/bigquery@v1.57.1", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/bigquery/com_google_cloud_go_bigquery-v1.56.0.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/bigquery/com_google_cloud_go_bigquery-v1.56.0.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/bigquery/com_google_cloud_go_bigquery-v1.56.0.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/bigquery/com_google_cloud_go_bigquery-v1.56.0.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/bigquery/com_google_cloud_go_bigquery-v1.57.1.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/bigquery/com_google_cloud_go_bigquery-v1.57.1.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/bigquery/com_google_cloud_go_bigquery-v1.57.1.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/bigquery/com_google_cloud_go_bigquery-v1.57.1.zip", ], ) go_repository( name = "com_google_cloud_go_billing", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/billing", - sha256 = "fecfc00cf9ea70a1109ec8e1a190e328fb1cc2f0159b1d4b10111cd5651b2bae", - strip_prefix = "cloud.google.com/go/billing@v1.17.2", + sha256 = "5df058fed01cf1fb5a2a5e924e8e1aba7c790a63fb8c7c0d6d60237fa1917b44", + strip_prefix = "cloud.google.com/go/billing@v1.17.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/billing/com_google_cloud_go_billing-v1.17.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/billing/com_google_cloud_go_billing-v1.17.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/billing/com_google_cloud_go_billing-v1.17.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/billing/com_google_cloud_go_billing-v1.17.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/billing/com_google_cloud_go_billing-v1.17.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/billing/com_google_cloud_go_billing-v1.17.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/billing/com_google_cloud_go_billing-v1.17.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/billing/com_google_cloud_go_billing-v1.17.4.zip", ], ) go_repository( name = "com_google_cloud_go_binaryauthorization", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/binaryauthorization", - sha256 = "afb102bcbd2836c1371d9e6a179da9109cdaa5c41a286d73ee6c93d3ae775736", - strip_prefix = "cloud.google.com/go/binaryauthorization@v1.7.1", + sha256 = "689b3c95ee0449009ba3bf2b3ebddb750ac56ff6b4524f0786bb8ad85cebca39", + strip_prefix = "cloud.google.com/go/binaryauthorization@v1.7.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/binaryauthorization/com_google_cloud_go_binaryauthorization-v1.7.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/binaryauthorization/com_google_cloud_go_binaryauthorization-v1.7.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/binaryauthorization/com_google_cloud_go_binaryauthorization-v1.7.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/binaryauthorization/com_google_cloud_go_binaryauthorization-v1.7.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/binaryauthorization/com_google_cloud_go_binaryauthorization-v1.7.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/binaryauthorization/com_google_cloud_go_binaryauthorization-v1.7.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/binaryauthorization/com_google_cloud_go_binaryauthorization-v1.7.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/binaryauthorization/com_google_cloud_go_binaryauthorization-v1.7.3.zip", ], ) go_repository( name = "com_google_cloud_go_certificatemanager", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/certificatemanager", - sha256 = "877ddd1a5c8e2efa94f0055b5371306eb07cf4cd52d5a70c15e0c38d6f6d8e32", - strip_prefix = "cloud.google.com/go/certificatemanager@v1.7.2", + sha256 = "afc6bab867cbab53582d921d726378daf46938e4f24fdf9a05373b0664ad78d2", + strip_prefix = "cloud.google.com/go/certificatemanager@v1.7.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/certificatemanager/com_google_cloud_go_certificatemanager-v1.7.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/certificatemanager/com_google_cloud_go_certificatemanager-v1.7.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/certificatemanager/com_google_cloud_go_certificatemanager-v1.7.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/certificatemanager/com_google_cloud_go_certificatemanager-v1.7.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/certificatemanager/com_google_cloud_go_certificatemanager-v1.7.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/certificatemanager/com_google_cloud_go_certificatemanager-v1.7.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/certificatemanager/com_google_cloud_go_certificatemanager-v1.7.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/certificatemanager/com_google_cloud_go_certificatemanager-v1.7.4.zip", ], ) go_repository( name = "com_google_cloud_go_channel", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/channel", - sha256 = "fd6990eda15ff2f698c8f09db37f5ba11d3a39b89fae50b6231c9ae2eae4a768", - strip_prefix = "cloud.google.com/go/channel@v1.17.1", + sha256 = "3d69ec1b2b1fd69def2d60e7798430bfc74408f223db4e0afe7eb07db188a0d6", + strip_prefix = "cloud.google.com/go/channel@v1.17.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/channel/com_google_cloud_go_channel-v1.17.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/channel/com_google_cloud_go_channel-v1.17.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/channel/com_google_cloud_go_channel-v1.17.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/channel/com_google_cloud_go_channel-v1.17.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/channel/com_google_cloud_go_channel-v1.17.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/channel/com_google_cloud_go_channel-v1.17.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/channel/com_google_cloud_go_channel-v1.17.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/channel/com_google_cloud_go_channel-v1.17.3.zip", ], ) go_repository( name = "com_google_cloud_go_cloudbuild", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/cloudbuild", - sha256 = "f3e6b2f036308af4749695e059c274459a89a3329785a7a68492e85ecb5a5a22", - strip_prefix = "cloud.google.com/go/cloudbuild@v1.14.1", + sha256 = "f1ef849f9d6fed0cb77f9dcc7122d857863044850df8007e5155ba8487209965", + strip_prefix = "cloud.google.com/go/cloudbuild@v1.14.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/cloudbuild/com_google_cloud_go_cloudbuild-v1.14.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/cloudbuild/com_google_cloud_go_cloudbuild-v1.14.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/cloudbuild/com_google_cloud_go_cloudbuild-v1.14.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/cloudbuild/com_google_cloud_go_cloudbuild-v1.14.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/cloudbuild/com_google_cloud_go_cloudbuild-v1.14.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/cloudbuild/com_google_cloud_go_cloudbuild-v1.14.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/cloudbuild/com_google_cloud_go_cloudbuild-v1.14.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/cloudbuild/com_google_cloud_go_cloudbuild-v1.14.3.zip", ], ) go_repository( name = "com_google_cloud_go_clouddms", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/clouddms", - sha256 = "bbdc27660a1a25cbe7f5c8d2ef3b87fc97910178f43a2ba037f7735b5939b186", - strip_prefix = "cloud.google.com/go/clouddms@v1.7.1", + sha256 = "49ea23b98c66c920e6326e516937d10a09f171751b53f9d6eab59150394e0550", + strip_prefix = "cloud.google.com/go/clouddms@v1.7.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/clouddms/com_google_cloud_go_clouddms-v1.7.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/clouddms/com_google_cloud_go_clouddms-v1.7.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/clouddms/com_google_cloud_go_clouddms-v1.7.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/clouddms/com_google_cloud_go_clouddms-v1.7.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/clouddms/com_google_cloud_go_clouddms-v1.7.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/clouddms/com_google_cloud_go_clouddms-v1.7.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/clouddms/com_google_cloud_go_clouddms-v1.7.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/clouddms/com_google_cloud_go_clouddms-v1.7.3.zip", ], ) go_repository( name = "com_google_cloud_go_cloudtasks", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/cloudtasks", - sha256 = "4a4c78416add083ce1b7a8e82138f08f5940c132a1a8705c97e4523f5fbd91e9", - strip_prefix = "cloud.google.com/go/cloudtasks@v1.12.2", + sha256 = "149863d6b4f1f5b937be07a8c3253500f6238a7818f3e8446140be09c0b6c304", + strip_prefix = "cloud.google.com/go/cloudtasks@v1.12.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/cloudtasks/com_google_cloud_go_cloudtasks-v1.12.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/cloudtasks/com_google_cloud_go_cloudtasks-v1.12.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/cloudtasks/com_google_cloud_go_cloudtasks-v1.12.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/cloudtasks/com_google_cloud_go_cloudtasks-v1.12.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/cloudtasks/com_google_cloud_go_cloudtasks-v1.12.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/cloudtasks/com_google_cloud_go_cloudtasks-v1.12.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/cloudtasks/com_google_cloud_go_cloudtasks-v1.12.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/cloudtasks/com_google_cloud_go_cloudtasks-v1.12.4.zip", ], ) go_repository( name = "com_google_cloud_go_compute", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/compute", - sha256 = "cde4afb8f4c63ff4b1b17feb0a9ae75abbd0fbfdd9e94ffa0eaaf300e803dce7", - strip_prefix = "cloud.google.com/go/compute@v1.23.1", + sha256 = "9214cf85837ea8b96bd61e56e297f06d3008b9d1fc81841e79aaad2940f3ed56", + strip_prefix = "cloud.google.com/go/compute@v1.23.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/compute/com_google_cloud_go_compute-v1.23.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/compute/com_google_cloud_go_compute-v1.23.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/compute/com_google_cloud_go_compute-v1.23.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/compute/com_google_cloud_go_compute-v1.23.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/compute/com_google_cloud_go_compute-v1.23.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/compute/com_google_cloud_go_compute-v1.23.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/compute/com_google_cloud_go_compute-v1.23.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/compute/com_google_cloud_go_compute-v1.23.3.zip", ], ) go_repository( @@ -8081,143 +8107,143 @@ def go_deps(): name = "com_google_cloud_go_contactcenterinsights", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/contactcenterinsights", - sha256 = "9e08c7acf5ffb2fcff937872cad37d7e1a2dc1b7d0d70aa450beb7cb21c61b1c", - strip_prefix = "cloud.google.com/go/contactcenterinsights@v1.11.1", + sha256 = "d26b376d57b67a99e35f1dbb68b6396dfcc59552eb6346dae9c1fd54fc0ef6a5", + strip_prefix = "cloud.google.com/go/contactcenterinsights@v1.11.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/contactcenterinsights/com_google_cloud_go_contactcenterinsights-v1.11.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/contactcenterinsights/com_google_cloud_go_contactcenterinsights-v1.11.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/contactcenterinsights/com_google_cloud_go_contactcenterinsights-v1.11.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/contactcenterinsights/com_google_cloud_go_contactcenterinsights-v1.11.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/contactcenterinsights/com_google_cloud_go_contactcenterinsights-v1.11.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/contactcenterinsights/com_google_cloud_go_contactcenterinsights-v1.11.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/contactcenterinsights/com_google_cloud_go_contactcenterinsights-v1.11.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/contactcenterinsights/com_google_cloud_go_contactcenterinsights-v1.11.3.zip", ], ) go_repository( name = "com_google_cloud_go_container", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/container", - sha256 = "f93f3636acb226560294d803e2f0b563e5ea5a0383707343ddd40c3287e53f43", - strip_prefix = "cloud.google.com/go/container@v1.26.1", + sha256 = "0185d0ab299a3468d8e96f6d96ab680c96cb9e0718b6e017151ee2d4de681e44", + strip_prefix = "cloud.google.com/go/container@v1.27.1", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/container/com_google_cloud_go_container-v1.26.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/container/com_google_cloud_go_container-v1.26.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/container/com_google_cloud_go_container-v1.26.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/container/com_google_cloud_go_container-v1.26.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/container/com_google_cloud_go_container-v1.27.1.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/container/com_google_cloud_go_container-v1.27.1.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/container/com_google_cloud_go_container-v1.27.1.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/container/com_google_cloud_go_container-v1.27.1.zip", ], ) go_repository( name = "com_google_cloud_go_containeranalysis", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/containeranalysis", - sha256 = "afe6c1616e50df702c99867da30e415b6fcaa6212175b4552c8fba41b171e803", - strip_prefix = "cloud.google.com/go/containeranalysis@v0.11.1", + sha256 = "d5920157486da69e257434e94e072952f6596084f262291a18ea1b0018295ef6", + strip_prefix = "cloud.google.com/go/containeranalysis@v0.11.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/containeranalysis/com_google_cloud_go_containeranalysis-v0.11.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/containeranalysis/com_google_cloud_go_containeranalysis-v0.11.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/containeranalysis/com_google_cloud_go_containeranalysis-v0.11.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/containeranalysis/com_google_cloud_go_containeranalysis-v0.11.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/containeranalysis/com_google_cloud_go_containeranalysis-v0.11.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/containeranalysis/com_google_cloud_go_containeranalysis-v0.11.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/containeranalysis/com_google_cloud_go_containeranalysis-v0.11.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/containeranalysis/com_google_cloud_go_containeranalysis-v0.11.3.zip", ], ) go_repository( name = "com_google_cloud_go_datacatalog", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/datacatalog", - sha256 = "92d8c36abbd4a2224889e077ca5cfdf25ec9eecbbfd08e3c77817bbdfa414947", - strip_prefix = "cloud.google.com/go/datacatalog@v1.18.1", + sha256 = "b553802a86e5c851719265f2ead1e9b5a82e385f71be94d72871cc09d9f64473", + strip_prefix = "cloud.google.com/go/datacatalog@v1.18.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/datacatalog/com_google_cloud_go_datacatalog-v1.18.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/datacatalog/com_google_cloud_go_datacatalog-v1.18.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/datacatalog/com_google_cloud_go_datacatalog-v1.18.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/datacatalog/com_google_cloud_go_datacatalog-v1.18.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/datacatalog/com_google_cloud_go_datacatalog-v1.18.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/datacatalog/com_google_cloud_go_datacatalog-v1.18.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/datacatalog/com_google_cloud_go_datacatalog-v1.18.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/datacatalog/com_google_cloud_go_datacatalog-v1.18.3.zip", ], ) go_repository( name = "com_google_cloud_go_dataflow", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/dataflow", - sha256 = "1b26af290f0f57e70e5ddf886e44c4e6e1d4c209819c3dcc698c199ff51ef00d", - strip_prefix = "cloud.google.com/go/dataflow@v0.9.2", + sha256 = "0f71665b59fcf45966f5aa3ea2c96a21134d8134965ad707a567b8a49e893294", + strip_prefix = "cloud.google.com/go/dataflow@v0.9.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dataflow/com_google_cloud_go_dataflow-v0.9.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/dataflow/com_google_cloud_go_dataflow-v0.9.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dataflow/com_google_cloud_go_dataflow-v0.9.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dataflow/com_google_cloud_go_dataflow-v0.9.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dataflow/com_google_cloud_go_dataflow-v0.9.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/dataflow/com_google_cloud_go_dataflow-v0.9.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dataflow/com_google_cloud_go_dataflow-v0.9.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dataflow/com_google_cloud_go_dataflow-v0.9.4.zip", ], ) go_repository( name = "com_google_cloud_go_dataform", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/dataform", - sha256 = "da816fdd03b9c9240c1a62adeb2aed112e3bf68f775e45944791c8a912c4a69e", - strip_prefix = "cloud.google.com/go/dataform@v0.8.2", + sha256 = "cfc3c35c07e83a83a01af2eb90d2cbdc2ee1c3d0ce2be81fbf74ec7796fbcbbb", + strip_prefix = "cloud.google.com/go/dataform@v0.9.1", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dataform/com_google_cloud_go_dataform-v0.8.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/dataform/com_google_cloud_go_dataform-v0.8.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dataform/com_google_cloud_go_dataform-v0.8.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dataform/com_google_cloud_go_dataform-v0.8.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dataform/com_google_cloud_go_dataform-v0.9.1.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/dataform/com_google_cloud_go_dataform-v0.9.1.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dataform/com_google_cloud_go_dataform-v0.9.1.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dataform/com_google_cloud_go_dataform-v0.9.1.zip", ], ) go_repository( name = "com_google_cloud_go_datafusion", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/datafusion", - sha256 = "ea3e48d218c57cbfb2e2bbdab083d4218f918da3864c723fa36cd36af8dacf7e", - strip_prefix = "cloud.google.com/go/datafusion@v1.7.2", + sha256 = "6128703042c84c3d685b784d76db2ecae1efc3eea722303e7de13d91b6fbbd2f", + strip_prefix = "cloud.google.com/go/datafusion@v1.7.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/datafusion/com_google_cloud_go_datafusion-v1.7.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/datafusion/com_google_cloud_go_datafusion-v1.7.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/datafusion/com_google_cloud_go_datafusion-v1.7.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/datafusion/com_google_cloud_go_datafusion-v1.7.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/datafusion/com_google_cloud_go_datafusion-v1.7.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/datafusion/com_google_cloud_go_datafusion-v1.7.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/datafusion/com_google_cloud_go_datafusion-v1.7.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/datafusion/com_google_cloud_go_datafusion-v1.7.4.zip", ], ) go_repository( name = "com_google_cloud_go_datalabeling", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/datalabeling", - sha256 = "51e5e5eb727485adc627c9ef3031b27747600804a7a2ae42275f2c4475dfab64", - strip_prefix = "cloud.google.com/go/datalabeling@v0.8.2", + sha256 = "667688171b5b6b5996cc1fc1b29f7e622ee1ff9c3da365f5fba67af1f795da94", + strip_prefix = "cloud.google.com/go/datalabeling@v0.8.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/datalabeling/com_google_cloud_go_datalabeling-v0.8.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/datalabeling/com_google_cloud_go_datalabeling-v0.8.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/datalabeling/com_google_cloud_go_datalabeling-v0.8.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/datalabeling/com_google_cloud_go_datalabeling-v0.8.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/datalabeling/com_google_cloud_go_datalabeling-v0.8.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/datalabeling/com_google_cloud_go_datalabeling-v0.8.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/datalabeling/com_google_cloud_go_datalabeling-v0.8.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/datalabeling/com_google_cloud_go_datalabeling-v0.8.4.zip", ], ) go_repository( name = "com_google_cloud_go_dataplex", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/dataplex", - sha256 = "a2bcf17307e944d70a8745586df8e57f8a46feb464607715b2d916d69ea3431d", - strip_prefix = "cloud.google.com/go/dataplex@v1.10.1", + sha256 = "b5b04335e6d5ca695c41e795219c4a140c46e3175b8802bb5faf3a7d0fcbe5b5", + strip_prefix = "cloud.google.com/go/dataplex@v1.11.1", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dataplex/com_google_cloud_go_dataplex-v1.10.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/dataplex/com_google_cloud_go_dataplex-v1.10.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dataplex/com_google_cloud_go_dataplex-v1.10.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dataplex/com_google_cloud_go_dataplex-v1.10.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dataplex/com_google_cloud_go_dataplex-v1.11.1.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/dataplex/com_google_cloud_go_dataplex-v1.11.1.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dataplex/com_google_cloud_go_dataplex-v1.11.1.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dataplex/com_google_cloud_go_dataplex-v1.11.1.zip", ], ) go_repository( name = "com_google_cloud_go_dataproc_v2", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/dataproc/v2", - sha256 = "9f170a095d8a5b7976ce660a31414882a8f38d4443c3cf05d70fe28ceab1e985", - strip_prefix = "cloud.google.com/go/dataproc/v2@v2.2.1", + sha256 = "b521fdfdbc339546d214b318d7f918888a16abff55f4761d22c4bd7002cdd494", + strip_prefix = "cloud.google.com/go/dataproc/v2@v2.2.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dataproc/v2/com_google_cloud_go_dataproc_v2-v2.2.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/dataproc/v2/com_google_cloud_go_dataproc_v2-v2.2.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dataproc/v2/com_google_cloud_go_dataproc_v2-v2.2.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dataproc/v2/com_google_cloud_go_dataproc_v2-v2.2.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dataproc/v2/com_google_cloud_go_dataproc_v2-v2.2.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/dataproc/v2/com_google_cloud_go_dataproc_v2-v2.2.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dataproc/v2/com_google_cloud_go_dataproc_v2-v2.2.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dataproc/v2/com_google_cloud_go_dataproc_v2-v2.2.3.zip", ], ) go_repository( name = "com_google_cloud_go_dataqna", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/dataqna", - sha256 = "69fcb32f4dc8b37e5a1a9e6fd3c33e64953ea06cb91fcfbcf59cafa31dfa8d86", - strip_prefix = "cloud.google.com/go/dataqna@v0.8.2", + sha256 = "59a30948890053cd9753d08670f73dd7caf58baa3f96f40eeb631dbc912d274c", + strip_prefix = "cloud.google.com/go/dataqna@v0.8.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dataqna/com_google_cloud_go_dataqna-v0.8.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/dataqna/com_google_cloud_go_dataqna-v0.8.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dataqna/com_google_cloud_go_dataqna-v0.8.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dataqna/com_google_cloud_go_dataqna-v0.8.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dataqna/com_google_cloud_go_dataqna-v0.8.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/dataqna/com_google_cloud_go_dataqna-v0.8.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dataqna/com_google_cloud_go_dataqna-v0.8.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dataqna/com_google_cloud_go_dataqna-v0.8.4.zip", ], ) go_repository( @@ -8237,91 +8263,91 @@ def go_deps(): name = "com_google_cloud_go_datastream", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/datastream", - sha256 = "29df6dde384fe4c964970ef77462fd939b8c45d49ff7cb82fbc39596f8e34893", - strip_prefix = "cloud.google.com/go/datastream@v1.10.1", + sha256 = "869245a4d245b607df6b5e2735c563523c6a7d305d2ba4a5bb2916dfcb3ee153", + strip_prefix = "cloud.google.com/go/datastream@v1.10.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/datastream/com_google_cloud_go_datastream-v1.10.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/datastream/com_google_cloud_go_datastream-v1.10.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/datastream/com_google_cloud_go_datastream-v1.10.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/datastream/com_google_cloud_go_datastream-v1.10.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/datastream/com_google_cloud_go_datastream-v1.10.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/datastream/com_google_cloud_go_datastream-v1.10.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/datastream/com_google_cloud_go_datastream-v1.10.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/datastream/com_google_cloud_go_datastream-v1.10.3.zip", ], ) go_repository( name = "com_google_cloud_go_deploy", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/deploy", - sha256 = "f7555f1cd13f36ae70982f4729531176cf322a75c6b9dde6c72f8a843d525481", - strip_prefix = "cloud.google.com/go/deploy@v1.13.1", + sha256 = "ebb8d298df020747a043567258f4bd7dbea45392ebe63b0fdf2791992586e539", + strip_prefix = "cloud.google.com/go/deploy@v1.14.2", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/deploy/com_google_cloud_go_deploy-v1.13.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/deploy/com_google_cloud_go_deploy-v1.13.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/deploy/com_google_cloud_go_deploy-v1.13.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/deploy/com_google_cloud_go_deploy-v1.13.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/deploy/com_google_cloud_go_deploy-v1.14.2.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/deploy/com_google_cloud_go_deploy-v1.14.2.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/deploy/com_google_cloud_go_deploy-v1.14.2.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/deploy/com_google_cloud_go_deploy-v1.14.2.zip", ], ) go_repository( name = "com_google_cloud_go_dialogflow", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/dialogflow", - sha256 = "9a17b92cec11b7877f38b7d31bb42928710b87f55b54c703b0858a86bf26421f", - strip_prefix = "cloud.google.com/go/dialogflow@v1.44.1", + sha256 = "085bda58ebc4499d6b4c178d9c1c09ddb574716817ef3f3d3881469d710ab34c", + strip_prefix = "cloud.google.com/go/dialogflow@v1.44.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dialogflow/com_google_cloud_go_dialogflow-v1.44.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/dialogflow/com_google_cloud_go_dialogflow-v1.44.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dialogflow/com_google_cloud_go_dialogflow-v1.44.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dialogflow/com_google_cloud_go_dialogflow-v1.44.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dialogflow/com_google_cloud_go_dialogflow-v1.44.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/dialogflow/com_google_cloud_go_dialogflow-v1.44.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dialogflow/com_google_cloud_go_dialogflow-v1.44.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dialogflow/com_google_cloud_go_dialogflow-v1.44.3.zip", ], ) go_repository( name = "com_google_cloud_go_dlp", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/dlp", - sha256 = "787fb0c860a5a6df47080feb59dc34bb26fc23fed5a98d8fc7b42636bc81b5ac", - strip_prefix = "cloud.google.com/go/dlp@v1.10.2", + sha256 = "76d5b843509ca2acf293bbf1296b3468a4d93ac107bdc87f84dbff7e8ede9f1d", + strip_prefix = "cloud.google.com/go/dlp@v1.11.1", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dlp/com_google_cloud_go_dlp-v1.10.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/dlp/com_google_cloud_go_dlp-v1.10.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dlp/com_google_cloud_go_dlp-v1.10.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dlp/com_google_cloud_go_dlp-v1.10.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/dlp/com_google_cloud_go_dlp-v1.11.1.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/dlp/com_google_cloud_go_dlp-v1.11.1.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/dlp/com_google_cloud_go_dlp-v1.11.1.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/dlp/com_google_cloud_go_dlp-v1.11.1.zip", ], ) go_repository( name = "com_google_cloud_go_documentai", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/documentai", - sha256 = "d306b29ea9ed00003eb4a72de10527a9898b32a67a5d9b1c028845db6ee977e6", - strip_prefix = "cloud.google.com/go/documentai@v1.23.2", + sha256 = "908e696281ff4bddd73363cceabdeff568a3dc4f781b3c8ffebd2c337481755a", + strip_prefix = "cloud.google.com/go/documentai@v1.23.5", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/documentai/com_google_cloud_go_documentai-v1.23.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/documentai/com_google_cloud_go_documentai-v1.23.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/documentai/com_google_cloud_go_documentai-v1.23.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/documentai/com_google_cloud_go_documentai-v1.23.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/documentai/com_google_cloud_go_documentai-v1.23.5.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/documentai/com_google_cloud_go_documentai-v1.23.5.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/documentai/com_google_cloud_go_documentai-v1.23.5.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/documentai/com_google_cloud_go_documentai-v1.23.5.zip", ], ) go_repository( name = "com_google_cloud_go_domains", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/domains", - sha256 = "541b361f96b0f849324e794b2b24f53aa73e30dbb1ba02f12e94b5dd38759db7", - strip_prefix = "cloud.google.com/go/domains@v0.9.2", + sha256 = "c2b2502a12b12fb176edcf6f8d1c6051b9a32879aa172cfcb8e46a0db60d2d28", + strip_prefix = "cloud.google.com/go/domains@v0.9.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/domains/com_google_cloud_go_domains-v0.9.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/domains/com_google_cloud_go_domains-v0.9.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/domains/com_google_cloud_go_domains-v0.9.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/domains/com_google_cloud_go_domains-v0.9.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/domains/com_google_cloud_go_domains-v0.9.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/domains/com_google_cloud_go_domains-v0.9.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/domains/com_google_cloud_go_domains-v0.9.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/domains/com_google_cloud_go_domains-v0.9.4.zip", ], ) go_repository( name = "com_google_cloud_go_edgecontainer", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/edgecontainer", - sha256 = "6671ed73144587cd1f5d20982d395a8628ad130ffea7d064d790e2c92274b3b3", - strip_prefix = "cloud.google.com/go/edgecontainer@v1.1.2", + sha256 = "aff88494c5b7882cc57985afa9f4da589cceb4eb8e8e7948747ffcde24a8c860", + strip_prefix = "cloud.google.com/go/edgecontainer@v1.1.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/edgecontainer/com_google_cloud_go_edgecontainer-v1.1.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/edgecontainer/com_google_cloud_go_edgecontainer-v1.1.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/edgecontainer/com_google_cloud_go_edgecontainer-v1.1.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/edgecontainer/com_google_cloud_go_edgecontainer-v1.1.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/edgecontainer/com_google_cloud_go_edgecontainer-v1.1.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/edgecontainer/com_google_cloud_go_edgecontainer-v1.1.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/edgecontainer/com_google_cloud_go_edgecontainer-v1.1.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/edgecontainer/com_google_cloud_go_edgecontainer-v1.1.4.zip", ], ) go_repository( @@ -8341,221 +8367,221 @@ def go_deps(): name = "com_google_cloud_go_essentialcontacts", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/essentialcontacts", - sha256 = "100839140d920ea39df237c99782dae60aa6827be723a8f17dcd77f29ff71eca", - strip_prefix = "cloud.google.com/go/essentialcontacts@v1.6.3", + sha256 = "b94201d1b97c96abf58f5ad57cce9cc8eb55d591f794c20564bf2a1bf2dd4898", + strip_prefix = "cloud.google.com/go/essentialcontacts@v1.6.5", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/essentialcontacts/com_google_cloud_go_essentialcontacts-v1.6.3.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/essentialcontacts/com_google_cloud_go_essentialcontacts-v1.6.3.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/essentialcontacts/com_google_cloud_go_essentialcontacts-v1.6.3.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/essentialcontacts/com_google_cloud_go_essentialcontacts-v1.6.3.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/essentialcontacts/com_google_cloud_go_essentialcontacts-v1.6.5.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/essentialcontacts/com_google_cloud_go_essentialcontacts-v1.6.5.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/essentialcontacts/com_google_cloud_go_essentialcontacts-v1.6.5.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/essentialcontacts/com_google_cloud_go_essentialcontacts-v1.6.5.zip", ], ) go_repository( name = "com_google_cloud_go_eventarc", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/eventarc", - sha256 = "c1abc76d8cdebcf2fbff6a5f5c289479749713033ba188853f6156b1f3a2c575", - strip_prefix = "cloud.google.com/go/eventarc@v1.13.1", + sha256 = "8a76bfffb3adf0ae07cf0d1b9017272748c0fb5da4e1ccc54bee96bf37510d94", + strip_prefix = "cloud.google.com/go/eventarc@v1.13.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/eventarc/com_google_cloud_go_eventarc-v1.13.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/eventarc/com_google_cloud_go_eventarc-v1.13.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/eventarc/com_google_cloud_go_eventarc-v1.13.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/eventarc/com_google_cloud_go_eventarc-v1.13.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/eventarc/com_google_cloud_go_eventarc-v1.13.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/eventarc/com_google_cloud_go_eventarc-v1.13.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/eventarc/com_google_cloud_go_eventarc-v1.13.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/eventarc/com_google_cloud_go_eventarc-v1.13.3.zip", ], ) go_repository( name = "com_google_cloud_go_filestore", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/filestore", - sha256 = "ddfc413e66b4e18263d250a7bc7d2d723b4007729107f4a33efc90fafb7149ea", - strip_prefix = "cloud.google.com/go/filestore@v1.7.2", + sha256 = "5e264d7e910ad767c88a4e4236ea1b26d52f5b56e18a5d6dcdb989a5020ceefb", + strip_prefix = "cloud.google.com/go/filestore@v1.7.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/filestore/com_google_cloud_go_filestore-v1.7.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/filestore/com_google_cloud_go_filestore-v1.7.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/filestore/com_google_cloud_go_filestore-v1.7.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/filestore/com_google_cloud_go_filestore-v1.7.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/filestore/com_google_cloud_go_filestore-v1.7.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/filestore/com_google_cloud_go_filestore-v1.7.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/filestore/com_google_cloud_go_filestore-v1.7.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/filestore/com_google_cloud_go_filestore-v1.7.4.zip", ], ) go_repository( name = "com_google_cloud_go_firestore", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/firestore", - sha256 = "4e14ba924858cda4925eccd288c8fb4ad377f227252138fe1681c0e9391ad3e0", - strip_prefix = "cloud.google.com/go/firestore@v1.13.0", + sha256 = "426e3589567d5b7bea9f7936863b4fe9fc7172029afc2b03cded5f69bcf3baf2", + strip_prefix = "cloud.google.com/go/firestore@v1.14.0", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/firestore/com_google_cloud_go_firestore-v1.13.0.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/firestore/com_google_cloud_go_firestore-v1.13.0.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/firestore/com_google_cloud_go_firestore-v1.13.0.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/firestore/com_google_cloud_go_firestore-v1.13.0.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/firestore/com_google_cloud_go_firestore-v1.14.0.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/firestore/com_google_cloud_go_firestore-v1.14.0.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/firestore/com_google_cloud_go_firestore-v1.14.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/firestore/com_google_cloud_go_firestore-v1.14.0.zip", ], ) go_repository( name = "com_google_cloud_go_functions", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/functions", - sha256 = "34232aa309d00ffef25ac784e2e1a702c8aaf5b921152c257772f59dbe4234ee", - strip_prefix = "cloud.google.com/go/functions@v1.15.2", + sha256 = "6d40befb2ebbb0e4cdeff7f54a21aa2c9df428e374608e7c6c940d8982f5301c", + strip_prefix = "cloud.google.com/go/functions@v1.15.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/functions/com_google_cloud_go_functions-v1.15.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/functions/com_google_cloud_go_functions-v1.15.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/functions/com_google_cloud_go_functions-v1.15.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/functions/com_google_cloud_go_functions-v1.15.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/functions/com_google_cloud_go_functions-v1.15.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/functions/com_google_cloud_go_functions-v1.15.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/functions/com_google_cloud_go_functions-v1.15.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/functions/com_google_cloud_go_functions-v1.15.4.zip", ], ) go_repository( name = "com_google_cloud_go_gkebackup", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/gkebackup", - sha256 = "f1617ab86d537328e3f3c36790da6d432caf00df1c60d7f7c59e49b3552296bf", - strip_prefix = "cloud.google.com/go/gkebackup@v1.3.2", + sha256 = "d9be77b8e0e5924d48a135848bddd27ea0a08174083de2981cbe7ababd432d13", + strip_prefix = "cloud.google.com/go/gkebackup@v1.3.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/gkebackup/com_google_cloud_go_gkebackup-v1.3.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/gkebackup/com_google_cloud_go_gkebackup-v1.3.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/gkebackup/com_google_cloud_go_gkebackup-v1.3.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/gkebackup/com_google_cloud_go_gkebackup-v1.3.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/gkebackup/com_google_cloud_go_gkebackup-v1.3.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/gkebackup/com_google_cloud_go_gkebackup-v1.3.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/gkebackup/com_google_cloud_go_gkebackup-v1.3.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/gkebackup/com_google_cloud_go_gkebackup-v1.3.4.zip", ], ) go_repository( name = "com_google_cloud_go_gkeconnect", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/gkeconnect", - sha256 = "009e1bb490e0091744f0d5ff7b3b7cbe5085a7795b775204fe45e80535f452ce", - strip_prefix = "cloud.google.com/go/gkeconnect@v0.8.2", + sha256 = "22d47713695fb99181e4dd67f6e705e67a261caab3f2ca75cee39e2d42ca7b7d", + strip_prefix = "cloud.google.com/go/gkeconnect@v0.8.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/gkeconnect/com_google_cloud_go_gkeconnect-v0.8.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/gkeconnect/com_google_cloud_go_gkeconnect-v0.8.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/gkeconnect/com_google_cloud_go_gkeconnect-v0.8.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/gkeconnect/com_google_cloud_go_gkeconnect-v0.8.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/gkeconnect/com_google_cloud_go_gkeconnect-v0.8.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/gkeconnect/com_google_cloud_go_gkeconnect-v0.8.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/gkeconnect/com_google_cloud_go_gkeconnect-v0.8.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/gkeconnect/com_google_cloud_go_gkeconnect-v0.8.4.zip", ], ) go_repository( name = "com_google_cloud_go_gkehub", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/gkehub", - sha256 = "acef02e283a877fae6242895bea73e00c655a239b6a34e6c4f26dde75214e897", - strip_prefix = "cloud.google.com/go/gkehub@v0.14.2", + sha256 = "6bff0481ab77aa7b1cf47d1f14aa9f10713aae9615ad1ed2a6d004b3378bb0b5", + strip_prefix = "cloud.google.com/go/gkehub@v0.14.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/gkehub/com_google_cloud_go_gkehub-v0.14.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/gkehub/com_google_cloud_go_gkehub-v0.14.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/gkehub/com_google_cloud_go_gkehub-v0.14.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/gkehub/com_google_cloud_go_gkehub-v0.14.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/gkehub/com_google_cloud_go_gkehub-v0.14.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/gkehub/com_google_cloud_go_gkehub-v0.14.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/gkehub/com_google_cloud_go_gkehub-v0.14.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/gkehub/com_google_cloud_go_gkehub-v0.14.4.zip", ], ) go_repository( name = "com_google_cloud_go_gkemulticloud", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/gkemulticloud", - sha256 = "ad4f2be30a3e031aaec680b8f4548876b024e02a835a51b4418f04c1a0d45437", - strip_prefix = "cloud.google.com/go/gkemulticloud@v1.0.1", + sha256 = "ec3d60fcb963da75d5bf6108161567acd79f7887e4b9cfb2e4a5e53e3f69857d", + strip_prefix = "cloud.google.com/go/gkemulticloud@v1.0.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/gkemulticloud/com_google_cloud_go_gkemulticloud-v1.0.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/gkemulticloud/com_google_cloud_go_gkemulticloud-v1.0.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/gkemulticloud/com_google_cloud_go_gkemulticloud-v1.0.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/gkemulticloud/com_google_cloud_go_gkemulticloud-v1.0.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/gkemulticloud/com_google_cloud_go_gkemulticloud-v1.0.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/gkemulticloud/com_google_cloud_go_gkemulticloud-v1.0.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/gkemulticloud/com_google_cloud_go_gkemulticloud-v1.0.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/gkemulticloud/com_google_cloud_go_gkemulticloud-v1.0.3.zip", ], ) go_repository( name = "com_google_cloud_go_gsuiteaddons", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/gsuiteaddons", - sha256 = "c31266cc003017a841473f2eaa162d0d4a58302ac6085153c8961b8673af1b6a", - strip_prefix = "cloud.google.com/go/gsuiteaddons@v1.6.2", + sha256 = "9edba07949bf3749625469fecab250d227de45a50766d31c0a32a688b43f8686", + strip_prefix = "cloud.google.com/go/gsuiteaddons@v1.6.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/gsuiteaddons/com_google_cloud_go_gsuiteaddons-v1.6.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/gsuiteaddons/com_google_cloud_go_gsuiteaddons-v1.6.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/gsuiteaddons/com_google_cloud_go_gsuiteaddons-v1.6.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/gsuiteaddons/com_google_cloud_go_gsuiteaddons-v1.6.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/gsuiteaddons/com_google_cloud_go_gsuiteaddons-v1.6.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/gsuiteaddons/com_google_cloud_go_gsuiteaddons-v1.6.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/gsuiteaddons/com_google_cloud_go_gsuiteaddons-v1.6.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/gsuiteaddons/com_google_cloud_go_gsuiteaddons-v1.6.4.zip", ], ) go_repository( name = "com_google_cloud_go_iam", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/iam", - sha256 = "56e6aba936af03c61fc21eb58f562596cadd6bacc30a07a7fb2a2516c28764bb", - strip_prefix = "cloud.google.com/go/iam@v1.1.3", + sha256 = "7d218318eef5255587d6db35f2aa9952125881c640f6e04fe76d521c14d5d7f3", + strip_prefix = "cloud.google.com/go/iam@v1.1.5", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/iam/com_google_cloud_go_iam-v1.1.3.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/iam/com_google_cloud_go_iam-v1.1.3.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/iam/com_google_cloud_go_iam-v1.1.3.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/iam/com_google_cloud_go_iam-v1.1.3.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/iam/com_google_cloud_go_iam-v1.1.5.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/iam/com_google_cloud_go_iam-v1.1.5.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/iam/com_google_cloud_go_iam-v1.1.5.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/iam/com_google_cloud_go_iam-v1.1.5.zip", ], ) go_repository( name = "com_google_cloud_go_iap", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/iap", - sha256 = "70dd5562de160017ea166cbd5a959eda628b025bc6dc93a269fc183d96eec8cf", - strip_prefix = "cloud.google.com/go/iap@v1.9.1", + sha256 = "622b8ebcb2d06aee0397e1362f29b571ed9a40b2810dda7140ce607eda9147dc", + strip_prefix = "cloud.google.com/go/iap@v1.9.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/iap/com_google_cloud_go_iap-v1.9.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/iap/com_google_cloud_go_iap-v1.9.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/iap/com_google_cloud_go_iap-v1.9.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/iap/com_google_cloud_go_iap-v1.9.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/iap/com_google_cloud_go_iap-v1.9.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/iap/com_google_cloud_go_iap-v1.9.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/iap/com_google_cloud_go_iap-v1.9.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/iap/com_google_cloud_go_iap-v1.9.3.zip", ], ) go_repository( name = "com_google_cloud_go_ids", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/ids", - sha256 = "5774a1cf5f3e09af43d38f37163c3ead590bb06119d4256e1a2670d40190094d", - strip_prefix = "cloud.google.com/go/ids@v1.4.2", + sha256 = "31f2675e6254fe0c42fff191d07c85c22ba89def513b6a7a0c67bcb9d0659cb8", + strip_prefix = "cloud.google.com/go/ids@v1.4.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/ids/com_google_cloud_go_ids-v1.4.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/ids/com_google_cloud_go_ids-v1.4.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/ids/com_google_cloud_go_ids-v1.4.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/ids/com_google_cloud_go_ids-v1.4.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/ids/com_google_cloud_go_ids-v1.4.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/ids/com_google_cloud_go_ids-v1.4.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/ids/com_google_cloud_go_ids-v1.4.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/ids/com_google_cloud_go_ids-v1.4.4.zip", ], ) go_repository( name = "com_google_cloud_go_iot", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/iot", - sha256 = "4dbe9fdf18ecd066a3986acd9a98680adac6a1cf50bbac6c04432e3059570fea", - strip_prefix = "cloud.google.com/go/iot@v1.7.2", + sha256 = "2366dffa5bc629cfb7a29c36e03089d8f5edc59017fc9c4911013ccda25e8ff2", + strip_prefix = "cloud.google.com/go/iot@v1.7.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/iot/com_google_cloud_go_iot-v1.7.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/iot/com_google_cloud_go_iot-v1.7.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/iot/com_google_cloud_go_iot-v1.7.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/iot/com_google_cloud_go_iot-v1.7.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/iot/com_google_cloud_go_iot-v1.7.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/iot/com_google_cloud_go_iot-v1.7.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/iot/com_google_cloud_go_iot-v1.7.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/iot/com_google_cloud_go_iot-v1.7.4.zip", ], ) go_repository( name = "com_google_cloud_go_kms", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/kms", - sha256 = "89ef8ac2bde3827d875928c8911c3ac874b7f0f6d1396ba3f6f17b51448738c4", - strip_prefix = "cloud.google.com/go/kms@v1.15.3", + sha256 = "4647fc192826a8ebcfc08ad8381bdde27689754ccab38a55b128fdf91b0dec4f", + strip_prefix = "cloud.google.com/go/kms@v1.15.5", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/kms/com_google_cloud_go_kms-v1.15.3.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/kms/com_google_cloud_go_kms-v1.15.3.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/kms/com_google_cloud_go_kms-v1.15.3.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/kms/com_google_cloud_go_kms-v1.15.3.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/kms/com_google_cloud_go_kms-v1.15.5.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/kms/com_google_cloud_go_kms-v1.15.5.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/kms/com_google_cloud_go_kms-v1.15.5.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/kms/com_google_cloud_go_kms-v1.15.5.zip", ], ) go_repository( name = "com_google_cloud_go_language", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/language", - sha256 = "353e525423b6547a806aea28f63f401759fd090855a1544c1228f48337470201", - strip_prefix = "cloud.google.com/go/language@v1.11.1", + sha256 = "cc8d49db7853a9dee0bc9f28af2992f62a1700a425089de1c6364bed22a8d9a1", + strip_prefix = "cloud.google.com/go/language@v1.12.2", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/language/com_google_cloud_go_language-v1.11.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/language/com_google_cloud_go_language-v1.11.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/language/com_google_cloud_go_language-v1.11.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/language/com_google_cloud_go_language-v1.11.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/language/com_google_cloud_go_language-v1.12.2.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/language/com_google_cloud_go_language-v1.12.2.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/language/com_google_cloud_go_language-v1.12.2.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/language/com_google_cloud_go_language-v1.12.2.zip", ], ) go_repository( name = "com_google_cloud_go_lifesciences", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/lifesciences", - sha256 = "32e38b08fcef1d06b7512261d1116fbac7e5e331942e4512a26d73f62625e5d6", - strip_prefix = "cloud.google.com/go/lifesciences@v0.9.2", + sha256 = "a18a1221135b5c60a38041bc1aea12f2129d847402fe88f4abf47ea109459b3a", + strip_prefix = "cloud.google.com/go/lifesciences@v0.9.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/lifesciences/com_google_cloud_go_lifesciences-v0.9.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/lifesciences/com_google_cloud_go_lifesciences-v0.9.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/lifesciences/com_google_cloud_go_lifesciences-v0.9.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/lifesciences/com_google_cloud_go_lifesciences-v0.9.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/lifesciences/com_google_cloud_go_lifesciences-v0.9.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/lifesciences/com_google_cloud_go_lifesciences-v0.9.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/lifesciences/com_google_cloud_go_lifesciences-v0.9.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/lifesciences/com_google_cloud_go_lifesciences-v0.9.4.zip", ], ) go_repository( @@ -8575,247 +8601,247 @@ def go_deps(): name = "com_google_cloud_go_longrunning", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/longrunning", - sha256 = "ba7fd6475a3e6f6335461d5a707232ccf8336397802e83c5401c2308906ee76b", - strip_prefix = "cloud.google.com/go/longrunning@v0.5.2", + sha256 = "a3eb14ee47bfdbca7df05c0011bcd9a86997ae8ef0a12c3a44a340c7e9ce2c0f", + strip_prefix = "cloud.google.com/go/longrunning@v0.5.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/longrunning/com_google_cloud_go_longrunning-v0.5.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/longrunning/com_google_cloud_go_longrunning-v0.5.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/longrunning/com_google_cloud_go_longrunning-v0.5.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/longrunning/com_google_cloud_go_longrunning-v0.5.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/longrunning/com_google_cloud_go_longrunning-v0.5.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/longrunning/com_google_cloud_go_longrunning-v0.5.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/longrunning/com_google_cloud_go_longrunning-v0.5.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/longrunning/com_google_cloud_go_longrunning-v0.5.4.zip", ], ) go_repository( name = "com_google_cloud_go_managedidentities", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/managedidentities", - sha256 = "d81fe6c82e62b737a430b640c698220c61722b122b6ac9722e7b365eecd4e074", - strip_prefix = "cloud.google.com/go/managedidentities@v1.6.2", + sha256 = "859ac519d11387f148109348509d4239a165a7b8acc0d208b2c53c667dcd7fb0", + strip_prefix = "cloud.google.com/go/managedidentities@v1.6.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/managedidentities/com_google_cloud_go_managedidentities-v1.6.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/managedidentities/com_google_cloud_go_managedidentities-v1.6.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/managedidentities/com_google_cloud_go_managedidentities-v1.6.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/managedidentities/com_google_cloud_go_managedidentities-v1.6.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/managedidentities/com_google_cloud_go_managedidentities-v1.6.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/managedidentities/com_google_cloud_go_managedidentities-v1.6.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/managedidentities/com_google_cloud_go_managedidentities-v1.6.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/managedidentities/com_google_cloud_go_managedidentities-v1.6.4.zip", ], ) go_repository( name = "com_google_cloud_go_maps", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/maps", - sha256 = "c9cb6250a7ff92586fb2e212fc3b36437c9baa74e9b373461a0b33e40d359909", - strip_prefix = "cloud.google.com/go/maps@v1.4.1", + sha256 = "c05c26dd3bcafd9d6f617cc9d2291760cc83d1946b5cb3de00e29e210147404d", + strip_prefix = "cloud.google.com/go/maps@v1.6.1", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/maps/com_google_cloud_go_maps-v1.4.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/maps/com_google_cloud_go_maps-v1.4.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/maps/com_google_cloud_go_maps-v1.4.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/maps/com_google_cloud_go_maps-v1.4.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/maps/com_google_cloud_go_maps-v1.6.1.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/maps/com_google_cloud_go_maps-v1.6.1.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/maps/com_google_cloud_go_maps-v1.6.1.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/maps/com_google_cloud_go_maps-v1.6.1.zip", ], ) go_repository( name = "com_google_cloud_go_mediatranslation", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/mediatranslation", - sha256 = "6805e80cc3a6615eb086c1efdbc57bd378779962d134e22509af1ef77db1eb7f", - strip_prefix = "cloud.google.com/go/mediatranslation@v0.8.2", + sha256 = "808f50d13611936320d1547257f697d1d1f736c01001aad32a1538a8b62cf3fc", + strip_prefix = "cloud.google.com/go/mediatranslation@v0.8.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/mediatranslation/com_google_cloud_go_mediatranslation-v0.8.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/mediatranslation/com_google_cloud_go_mediatranslation-v0.8.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/mediatranslation/com_google_cloud_go_mediatranslation-v0.8.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/mediatranslation/com_google_cloud_go_mediatranslation-v0.8.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/mediatranslation/com_google_cloud_go_mediatranslation-v0.8.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/mediatranslation/com_google_cloud_go_mediatranslation-v0.8.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/mediatranslation/com_google_cloud_go_mediatranslation-v0.8.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/mediatranslation/com_google_cloud_go_mediatranslation-v0.8.4.zip", ], ) go_repository( name = "com_google_cloud_go_memcache", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/memcache", - sha256 = "80d1544a452cdb9a051cdd577a2dc018b56a9250c54ca5df194c65855a6cf7b5", - strip_prefix = "cloud.google.com/go/memcache@v1.10.2", + sha256 = "d75fc4d67a4b5999301fbc9c90d230b943ae26855b703e30015d367aa8261bce", + strip_prefix = "cloud.google.com/go/memcache@v1.10.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/memcache/com_google_cloud_go_memcache-v1.10.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/memcache/com_google_cloud_go_memcache-v1.10.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/memcache/com_google_cloud_go_memcache-v1.10.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/memcache/com_google_cloud_go_memcache-v1.10.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/memcache/com_google_cloud_go_memcache-v1.10.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/memcache/com_google_cloud_go_memcache-v1.10.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/memcache/com_google_cloud_go_memcache-v1.10.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/memcache/com_google_cloud_go_memcache-v1.10.4.zip", ], ) go_repository( name = "com_google_cloud_go_metastore", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/metastore", - sha256 = "fe4aa0aa4abd1bd6c1cb3c6d506c3acfb58e9851cdbe91b017360e4ce6533ff9", - strip_prefix = "cloud.google.com/go/metastore@v1.13.1", + sha256 = "844a54ec039a5e6c531924b7de9007a82a5eb9f4e90f4f4bbf4d4da4460fd54f", + strip_prefix = "cloud.google.com/go/metastore@v1.13.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/metastore/com_google_cloud_go_metastore-v1.13.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/metastore/com_google_cloud_go_metastore-v1.13.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/metastore/com_google_cloud_go_metastore-v1.13.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/metastore/com_google_cloud_go_metastore-v1.13.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/metastore/com_google_cloud_go_metastore-v1.13.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/metastore/com_google_cloud_go_metastore-v1.13.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/metastore/com_google_cloud_go_metastore-v1.13.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/metastore/com_google_cloud_go_metastore-v1.13.3.zip", ], ) go_repository( name = "com_google_cloud_go_monitoring", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/monitoring", - sha256 = "545af97f19cde57c99d37c8741d45f110a472f62e348313cef2054f8623661cd", - strip_prefix = "cloud.google.com/go/monitoring@v1.16.1", + sha256 = "3efde7106653d6791f116cfd445dcfdd51f6b732226688b5ad5feae69fa42924", + strip_prefix = "cloud.google.com/go/monitoring@v1.16.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/monitoring/com_google_cloud_go_monitoring-v1.16.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/monitoring/com_google_cloud_go_monitoring-v1.16.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/monitoring/com_google_cloud_go_monitoring-v1.16.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/monitoring/com_google_cloud_go_monitoring-v1.16.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/monitoring/com_google_cloud_go_monitoring-v1.16.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/monitoring/com_google_cloud_go_monitoring-v1.16.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/monitoring/com_google_cloud_go_monitoring-v1.16.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/monitoring/com_google_cloud_go_monitoring-v1.16.3.zip", ], ) go_repository( name = "com_google_cloud_go_networkconnectivity", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/networkconnectivity", - sha256 = "fa63079ab0dfcd34b074145057487d18d95e6b380b2b19c4a9a2113303333fdb", - strip_prefix = "cloud.google.com/go/networkconnectivity@v1.14.1", + sha256 = "677e203c626f430e1486beca3644a1d0ee8ef4e5323468ae7f587afba5e59b10", + strip_prefix = "cloud.google.com/go/networkconnectivity@v1.14.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/networkconnectivity/com_google_cloud_go_networkconnectivity-v1.14.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/networkconnectivity/com_google_cloud_go_networkconnectivity-v1.14.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/networkconnectivity/com_google_cloud_go_networkconnectivity-v1.14.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/networkconnectivity/com_google_cloud_go_networkconnectivity-v1.14.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/networkconnectivity/com_google_cloud_go_networkconnectivity-v1.14.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/networkconnectivity/com_google_cloud_go_networkconnectivity-v1.14.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/networkconnectivity/com_google_cloud_go_networkconnectivity-v1.14.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/networkconnectivity/com_google_cloud_go_networkconnectivity-v1.14.3.zip", ], ) go_repository( name = "com_google_cloud_go_networkmanagement", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/networkmanagement", - sha256 = "bed1460ce979230c94121f814c62aee524f223568bfcf00962e8683379016c49", - strip_prefix = "cloud.google.com/go/networkmanagement@v1.9.1", + sha256 = "8813a77f3851bc9663b4b6fd575980eec7c662ff2132b69b84256a4fb35171e7", + strip_prefix = "cloud.google.com/go/networkmanagement@v1.9.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/networkmanagement/com_google_cloud_go_networkmanagement-v1.9.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/networkmanagement/com_google_cloud_go_networkmanagement-v1.9.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/networkmanagement/com_google_cloud_go_networkmanagement-v1.9.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/networkmanagement/com_google_cloud_go_networkmanagement-v1.9.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/networkmanagement/com_google_cloud_go_networkmanagement-v1.9.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/networkmanagement/com_google_cloud_go_networkmanagement-v1.9.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/networkmanagement/com_google_cloud_go_networkmanagement-v1.9.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/networkmanagement/com_google_cloud_go_networkmanagement-v1.9.3.zip", ], ) go_repository( name = "com_google_cloud_go_networksecurity", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/networksecurity", - sha256 = "b4e959bd1d9c97e7267c529ee023fa55a7bbcd7b5f2964b6c99f0fb51006dbcb", - strip_prefix = "cloud.google.com/go/networksecurity@v0.9.2", + sha256 = "26a8c4573bc86d8bfcd411c72deb2f26b6778028d88246bdbb8f65fe72e6e541", + strip_prefix = "cloud.google.com/go/networksecurity@v0.9.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/networksecurity/com_google_cloud_go_networksecurity-v0.9.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/networksecurity/com_google_cloud_go_networksecurity-v0.9.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/networksecurity/com_google_cloud_go_networksecurity-v0.9.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/networksecurity/com_google_cloud_go_networksecurity-v0.9.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/networksecurity/com_google_cloud_go_networksecurity-v0.9.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/networksecurity/com_google_cloud_go_networksecurity-v0.9.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/networksecurity/com_google_cloud_go_networksecurity-v0.9.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/networksecurity/com_google_cloud_go_networksecurity-v0.9.4.zip", ], ) go_repository( name = "com_google_cloud_go_notebooks", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/notebooks", - sha256 = "9f33be59f75e363bd6a90a465ada18ad8cc549b82facbb9541b81ae9b3ba7937", - strip_prefix = "cloud.google.com/go/notebooks@v1.10.1", + sha256 = "0f7d6e082b6c5d1baa059ce8021d7fb80d91ff6719ac6f6cdd527cd6268262bd", + strip_prefix = "cloud.google.com/go/notebooks@v1.11.2", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/notebooks/com_google_cloud_go_notebooks-v1.10.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/notebooks/com_google_cloud_go_notebooks-v1.10.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/notebooks/com_google_cloud_go_notebooks-v1.10.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/notebooks/com_google_cloud_go_notebooks-v1.10.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/notebooks/com_google_cloud_go_notebooks-v1.11.2.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/notebooks/com_google_cloud_go_notebooks-v1.11.2.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/notebooks/com_google_cloud_go_notebooks-v1.11.2.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/notebooks/com_google_cloud_go_notebooks-v1.11.2.zip", ], ) go_repository( name = "com_google_cloud_go_optimization", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/optimization", - sha256 = "feb9d564067168d48be5e2bde7f19032a7c27a779beefc09d3aa19f0c2b9eaf2", - strip_prefix = "cloud.google.com/go/optimization@v1.5.1", + sha256 = "554a95c20b6343708d5713ecba3ac5dc0c11ba287e455021b1832cfcf293cbe2", + strip_prefix = "cloud.google.com/go/optimization@v1.6.2", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/optimization/com_google_cloud_go_optimization-v1.5.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/optimization/com_google_cloud_go_optimization-v1.5.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/optimization/com_google_cloud_go_optimization-v1.5.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/optimization/com_google_cloud_go_optimization-v1.5.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/optimization/com_google_cloud_go_optimization-v1.6.2.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/optimization/com_google_cloud_go_optimization-v1.6.2.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/optimization/com_google_cloud_go_optimization-v1.6.2.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/optimization/com_google_cloud_go_optimization-v1.6.2.zip", ], ) go_repository( name = "com_google_cloud_go_orchestration", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/orchestration", - sha256 = "81f752cb325e335254f9c33be9d7bacdf0fbc8ef929828a0496e35b2787dc2df", - strip_prefix = "cloud.google.com/go/orchestration@v1.8.2", + sha256 = "a378359d0f2d657c05e7b474a0a527952a5541d9d9798f5184e96e8c73c25cee", + strip_prefix = "cloud.google.com/go/orchestration@v1.8.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/orchestration/com_google_cloud_go_orchestration-v1.8.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/orchestration/com_google_cloud_go_orchestration-v1.8.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/orchestration/com_google_cloud_go_orchestration-v1.8.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/orchestration/com_google_cloud_go_orchestration-v1.8.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/orchestration/com_google_cloud_go_orchestration-v1.8.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/orchestration/com_google_cloud_go_orchestration-v1.8.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/orchestration/com_google_cloud_go_orchestration-v1.8.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/orchestration/com_google_cloud_go_orchestration-v1.8.4.zip", ], ) go_repository( name = "com_google_cloud_go_orgpolicy", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/orgpolicy", - sha256 = "2a4f68b17cf411c15e384e0bb1a7d5c823e9212129535075588cbd4c3e3da73d", - strip_prefix = "cloud.google.com/go/orgpolicy@v1.11.2", + sha256 = "97839eca96155da1a5d4e9ed13491c1a7afd8e57fe98efa8c590fad77f935dcd", + strip_prefix = "cloud.google.com/go/orgpolicy@v1.11.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/orgpolicy/com_google_cloud_go_orgpolicy-v1.11.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/orgpolicy/com_google_cloud_go_orgpolicy-v1.11.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/orgpolicy/com_google_cloud_go_orgpolicy-v1.11.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/orgpolicy/com_google_cloud_go_orgpolicy-v1.11.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/orgpolicy/com_google_cloud_go_orgpolicy-v1.11.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/orgpolicy/com_google_cloud_go_orgpolicy-v1.11.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/orgpolicy/com_google_cloud_go_orgpolicy-v1.11.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/orgpolicy/com_google_cloud_go_orgpolicy-v1.11.4.zip", ], ) go_repository( name = "com_google_cloud_go_osconfig", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/osconfig", - sha256 = "275f62c21ecf8371b55e778ab87e9ef588ced27cc63a9e985ab5029eddcdb843", - strip_prefix = "cloud.google.com/go/osconfig@v1.12.2", + sha256 = "b84d51954b532b8bc4b0133b4332c8640bf6b49c37beabf272832dfbaff60dab", + strip_prefix = "cloud.google.com/go/osconfig@v1.12.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/osconfig/com_google_cloud_go_osconfig-v1.12.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/osconfig/com_google_cloud_go_osconfig-v1.12.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/osconfig/com_google_cloud_go_osconfig-v1.12.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/osconfig/com_google_cloud_go_osconfig-v1.12.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/osconfig/com_google_cloud_go_osconfig-v1.12.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/osconfig/com_google_cloud_go_osconfig-v1.12.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/osconfig/com_google_cloud_go_osconfig-v1.12.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/osconfig/com_google_cloud_go_osconfig-v1.12.4.zip", ], ) go_repository( name = "com_google_cloud_go_oslogin", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/oslogin", - sha256 = "29ef72254fe0efd3778f6cc8fbc3df9c33f7ce7b6045d0f6d96eb876044b2237", - strip_prefix = "cloud.google.com/go/oslogin@v1.11.1", + sha256 = "bc959a271e1af25265b8e45debbf32170cf1999a5b8e201235d2a6c115f3416d", + strip_prefix = "cloud.google.com/go/oslogin@v1.12.2", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/oslogin/com_google_cloud_go_oslogin-v1.11.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/oslogin/com_google_cloud_go_oslogin-v1.11.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/oslogin/com_google_cloud_go_oslogin-v1.11.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/oslogin/com_google_cloud_go_oslogin-v1.11.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/oslogin/com_google_cloud_go_oslogin-v1.12.2.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/oslogin/com_google_cloud_go_oslogin-v1.12.2.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/oslogin/com_google_cloud_go_oslogin-v1.12.2.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/oslogin/com_google_cloud_go_oslogin-v1.12.2.zip", ], ) go_repository( name = "com_google_cloud_go_phishingprotection", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/phishingprotection", - sha256 = "6244bb1f396e3cb42e513ddc934923f0e14ab55f40b828f16c90be57a4b94a84", - strip_prefix = "cloud.google.com/go/phishingprotection@v0.8.2", + sha256 = "2bce1ef758b382998dfa041a85139a89511969fdc8596bd7f912381904235605", + strip_prefix = "cloud.google.com/go/phishingprotection@v0.8.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/phishingprotection/com_google_cloud_go_phishingprotection-v0.8.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/phishingprotection/com_google_cloud_go_phishingprotection-v0.8.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/phishingprotection/com_google_cloud_go_phishingprotection-v0.8.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/phishingprotection/com_google_cloud_go_phishingprotection-v0.8.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/phishingprotection/com_google_cloud_go_phishingprotection-v0.8.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/phishingprotection/com_google_cloud_go_phishingprotection-v0.8.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/phishingprotection/com_google_cloud_go_phishingprotection-v0.8.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/phishingprotection/com_google_cloud_go_phishingprotection-v0.8.4.zip", ], ) go_repository( name = "com_google_cloud_go_policytroubleshooter", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/policytroubleshooter", - sha256 = "b46e74184e6b8ed9943f474b976fb6bedd6a4d0700ac696012300886922f9d98", - strip_prefix = "cloud.google.com/go/policytroubleshooter@v1.9.1", + sha256 = "a9976677e157f786b43c82e682dcb5e3257e330d4940ddb2aea025fac70bdabc", + strip_prefix = "cloud.google.com/go/policytroubleshooter@v1.10.2", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/policytroubleshooter/com_google_cloud_go_policytroubleshooter-v1.9.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/policytroubleshooter/com_google_cloud_go_policytroubleshooter-v1.9.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/policytroubleshooter/com_google_cloud_go_policytroubleshooter-v1.9.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/policytroubleshooter/com_google_cloud_go_policytroubleshooter-v1.9.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/policytroubleshooter/com_google_cloud_go_policytroubleshooter-v1.10.2.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/policytroubleshooter/com_google_cloud_go_policytroubleshooter-v1.10.2.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/policytroubleshooter/com_google_cloud_go_policytroubleshooter-v1.10.2.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/policytroubleshooter/com_google_cloud_go_policytroubleshooter-v1.10.2.zip", ], ) go_repository( name = "com_google_cloud_go_privatecatalog", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/privatecatalog", - sha256 = "a43190e1dfba2ed7fcb63e5571937bdfc2ed97594fa9b2b7bd119678e977b0f4", - strip_prefix = "cloud.google.com/go/privatecatalog@v0.9.2", + sha256 = "49e1f68b1405978933b1cb43c94928f11873d69ebe9623ceccf3968a78da2183", + strip_prefix = "cloud.google.com/go/privatecatalog@v0.9.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/privatecatalog/com_google_cloud_go_privatecatalog-v0.9.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/privatecatalog/com_google_cloud_go_privatecatalog-v0.9.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/privatecatalog/com_google_cloud_go_privatecatalog-v0.9.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/privatecatalog/com_google_cloud_go_privatecatalog-v0.9.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/privatecatalog/com_google_cloud_go_privatecatalog-v0.9.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/privatecatalog/com_google_cloud_go_privatecatalog-v0.9.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/privatecatalog/com_google_cloud_go_privatecatalog-v0.9.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/privatecatalog/com_google_cloud_go_privatecatalog-v0.9.4.zip", ], ) go_repository( @@ -8848,208 +8874,208 @@ def go_deps(): name = "com_google_cloud_go_recaptchaenterprise_v2", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/recaptchaenterprise/v2", - sha256 = "8151e658e29acc0617fa5bc36f7d6f06a61e8b97558f79fd6137429538ad903f", - strip_prefix = "cloud.google.com/go/recaptchaenterprise/v2@v2.8.1", + sha256 = "7f00c16714a2ae8caad220cc9f91188024a2dd46495fb410f44fc9872d3734c5", + strip_prefix = "cloud.google.com/go/recaptchaenterprise/v2@v2.8.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/recaptchaenterprise/v2/com_google_cloud_go_recaptchaenterprise_v2-v2.8.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/recaptchaenterprise/v2/com_google_cloud_go_recaptchaenterprise_v2-v2.8.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/recaptchaenterprise/v2/com_google_cloud_go_recaptchaenterprise_v2-v2.8.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/recaptchaenterprise/v2/com_google_cloud_go_recaptchaenterprise_v2-v2.8.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/recaptchaenterprise/v2/com_google_cloud_go_recaptchaenterprise_v2-v2.8.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/recaptchaenterprise/v2/com_google_cloud_go_recaptchaenterprise_v2-v2.8.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/recaptchaenterprise/v2/com_google_cloud_go_recaptchaenterprise_v2-v2.8.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/recaptchaenterprise/v2/com_google_cloud_go_recaptchaenterprise_v2-v2.8.3.zip", ], ) go_repository( name = "com_google_cloud_go_recommendationengine", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/recommendationengine", - sha256 = "a2636073ab9bd418361f38332b4e922fcfe5ca6bc10aca96f6fcbdab7a37456d", - strip_prefix = "cloud.google.com/go/recommendationengine@v0.8.2", + sha256 = "2c6528549fb5dbddc6c225376e4b222e08ba2cc3d7c826c4720220cab8ef22cb", + strip_prefix = "cloud.google.com/go/recommendationengine@v0.8.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/recommendationengine/com_google_cloud_go_recommendationengine-v0.8.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/recommendationengine/com_google_cloud_go_recommendationengine-v0.8.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/recommendationengine/com_google_cloud_go_recommendationengine-v0.8.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/recommendationengine/com_google_cloud_go_recommendationengine-v0.8.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/recommendationengine/com_google_cloud_go_recommendationengine-v0.8.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/recommendationengine/com_google_cloud_go_recommendationengine-v0.8.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/recommendationengine/com_google_cloud_go_recommendationengine-v0.8.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/recommendationengine/com_google_cloud_go_recommendationengine-v0.8.4.zip", ], ) go_repository( name = "com_google_cloud_go_recommender", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/recommender", - sha256 = "1f0585da517bd4163b8482c6810cf1c119c0ad5a4d038bdcaa6491a5b3d1417b", - strip_prefix = "cloud.google.com/go/recommender@v1.11.1", + sha256 = "91d47908960d25962b698dfa918db6fe8ce7f7c997b8c70243f2522019cdce92", + strip_prefix = "cloud.google.com/go/recommender@v1.11.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/recommender/com_google_cloud_go_recommender-v1.11.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/recommender/com_google_cloud_go_recommender-v1.11.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/recommender/com_google_cloud_go_recommender-v1.11.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/recommender/com_google_cloud_go_recommender-v1.11.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/recommender/com_google_cloud_go_recommender-v1.11.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/recommender/com_google_cloud_go_recommender-v1.11.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/recommender/com_google_cloud_go_recommender-v1.11.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/recommender/com_google_cloud_go_recommender-v1.11.3.zip", ], ) go_repository( name = "com_google_cloud_go_redis", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/redis", - sha256 = "53e5bd33b17517627ce13404b784a7c8b2b8c65719e8f70977616b061834ee87", - strip_prefix = "cloud.google.com/go/redis@v1.13.2", + sha256 = "eb4875557eb8116a4247a1bcc796f6e3402e930259d7b7874cba4edb040c0002", + strip_prefix = "cloud.google.com/go/redis@v1.14.1", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/redis/com_google_cloud_go_redis-v1.13.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/redis/com_google_cloud_go_redis-v1.13.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/redis/com_google_cloud_go_redis-v1.13.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/redis/com_google_cloud_go_redis-v1.13.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/redis/com_google_cloud_go_redis-v1.14.1.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/redis/com_google_cloud_go_redis-v1.14.1.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/redis/com_google_cloud_go_redis-v1.14.1.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/redis/com_google_cloud_go_redis-v1.14.1.zip", ], ) go_repository( name = "com_google_cloud_go_resourcemanager", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/resourcemanager", - sha256 = "bb51f46e5a6a219191c258f1b395a1129fc96d4ea940eff412191522c0dbd043", - strip_prefix = "cloud.google.com/go/resourcemanager@v1.9.2", + sha256 = "e09c544224dbcde4c3241bb8c3f1246a0a89c00bb4b5419660009cfa3208b03a", + strip_prefix = "cloud.google.com/go/resourcemanager@v1.9.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/resourcemanager/com_google_cloud_go_resourcemanager-v1.9.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/resourcemanager/com_google_cloud_go_resourcemanager-v1.9.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/resourcemanager/com_google_cloud_go_resourcemanager-v1.9.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/resourcemanager/com_google_cloud_go_resourcemanager-v1.9.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/resourcemanager/com_google_cloud_go_resourcemanager-v1.9.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/resourcemanager/com_google_cloud_go_resourcemanager-v1.9.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/resourcemanager/com_google_cloud_go_resourcemanager-v1.9.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/resourcemanager/com_google_cloud_go_resourcemanager-v1.9.4.zip", ], ) go_repository( name = "com_google_cloud_go_resourcesettings", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/resourcesettings", - sha256 = "f2327ef037487c9f183e3a52e6456c087f8dc8325311bc6dcb77e5a8c030e360", - strip_prefix = "cloud.google.com/go/resourcesettings@v1.6.2", + sha256 = "44b8bdbf1e73ceda14c4a09134fd1094e1d8cbd0c1790eb769bd9aff53cf49a0", + strip_prefix = "cloud.google.com/go/resourcesettings@v1.6.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/resourcesettings/com_google_cloud_go_resourcesettings-v1.6.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/resourcesettings/com_google_cloud_go_resourcesettings-v1.6.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/resourcesettings/com_google_cloud_go_resourcesettings-v1.6.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/resourcesettings/com_google_cloud_go_resourcesettings-v1.6.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/resourcesettings/com_google_cloud_go_resourcesettings-v1.6.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/resourcesettings/com_google_cloud_go_resourcesettings-v1.6.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/resourcesettings/com_google_cloud_go_resourcesettings-v1.6.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/resourcesettings/com_google_cloud_go_resourcesettings-v1.6.4.zip", ], ) go_repository( name = "com_google_cloud_go_retail", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/retail", - sha256 = "85c1df965d36c1449655ae20ff44c18eab4177babbc2a851764941073b623862", - strip_prefix = "cloud.google.com/go/retail@v1.14.2", + sha256 = "4085ecd37e6bf1e34cd99f499a69a559bcad3782b124989a84f62db0c43717de", + strip_prefix = "cloud.google.com/go/retail@v1.14.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/retail/com_google_cloud_go_retail-v1.14.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/retail/com_google_cloud_go_retail-v1.14.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/retail/com_google_cloud_go_retail-v1.14.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/retail/com_google_cloud_go_retail-v1.14.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/retail/com_google_cloud_go_retail-v1.14.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/retail/com_google_cloud_go_retail-v1.14.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/retail/com_google_cloud_go_retail-v1.14.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/retail/com_google_cloud_go_retail-v1.14.4.zip", ], ) go_repository( name = "com_google_cloud_go_run", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/run", - sha256 = "5382527d044acc067f77f94001d094b1bd990fe91e68dd6de38d86b8eed9bc04", - strip_prefix = "cloud.google.com/go/run@v1.3.1", + sha256 = "fa7447cad93791127df03fa82f00c4dac3ac11de5a936f1da376930d372be1d3", + strip_prefix = "cloud.google.com/go/run@v1.3.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/run/com_google_cloud_go_run-v1.3.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/run/com_google_cloud_go_run-v1.3.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/run/com_google_cloud_go_run-v1.3.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/run/com_google_cloud_go_run-v1.3.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/run/com_google_cloud_go_run-v1.3.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/run/com_google_cloud_go_run-v1.3.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/run/com_google_cloud_go_run-v1.3.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/run/com_google_cloud_go_run-v1.3.3.zip", ], ) go_repository( name = "com_google_cloud_go_scheduler", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/scheduler", - sha256 = "171f330ba8477da1ab647cc6707f963300fec30750a37e5dd7935d2387c2116d", - strip_prefix = "cloud.google.com/go/scheduler@v1.10.2", + sha256 = "eb4159cd7e874e6382b8e6aa828956bd3e3d01b26baf7c98ce25f80c67d19955", + strip_prefix = "cloud.google.com/go/scheduler@v1.10.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/scheduler/com_google_cloud_go_scheduler-v1.10.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/scheduler/com_google_cloud_go_scheduler-v1.10.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/scheduler/com_google_cloud_go_scheduler-v1.10.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/scheduler/com_google_cloud_go_scheduler-v1.10.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/scheduler/com_google_cloud_go_scheduler-v1.10.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/scheduler/com_google_cloud_go_scheduler-v1.10.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/scheduler/com_google_cloud_go_scheduler-v1.10.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/scheduler/com_google_cloud_go_scheduler-v1.10.4.zip", ], ) go_repository( name = "com_google_cloud_go_secretmanager", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/secretmanager", - sha256 = "4dc8dc1f3dc60a6aa5038ccc7ac988e53af136a544021391a82af303cf811e5a", - strip_prefix = "cloud.google.com/go/secretmanager@v1.11.2", + sha256 = "0639d29ee3e59ca7fe54cd07ece668bf1faaa973179b18c903dfda285e11d005", + strip_prefix = "cloud.google.com/go/secretmanager@v1.11.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/secretmanager/com_google_cloud_go_secretmanager-v1.11.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/secretmanager/com_google_cloud_go_secretmanager-v1.11.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/secretmanager/com_google_cloud_go_secretmanager-v1.11.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/secretmanager/com_google_cloud_go_secretmanager-v1.11.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/secretmanager/com_google_cloud_go_secretmanager-v1.11.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/secretmanager/com_google_cloud_go_secretmanager-v1.11.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/secretmanager/com_google_cloud_go_secretmanager-v1.11.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/secretmanager/com_google_cloud_go_secretmanager-v1.11.4.zip", ], ) go_repository( name = "com_google_cloud_go_security", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/security", - sha256 = "8c32c5308500ac6dad8bf4ab42d84c18e5d19bcbfbdb1879a349a6a62d428a61", - strip_prefix = "cloud.google.com/go/security@v1.15.2", + sha256 = "0b42a54801d9321fab57ab3811befcf071a13426ca49f2c498ffd429ce4057b3", + strip_prefix = "cloud.google.com/go/security@v1.15.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/security/com_google_cloud_go_security-v1.15.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/security/com_google_cloud_go_security-v1.15.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/security/com_google_cloud_go_security-v1.15.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/security/com_google_cloud_go_security-v1.15.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/security/com_google_cloud_go_security-v1.15.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/security/com_google_cloud_go_security-v1.15.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/security/com_google_cloud_go_security-v1.15.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/security/com_google_cloud_go_security-v1.15.4.zip", ], ) go_repository( name = "com_google_cloud_go_securitycenter", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/securitycenter", - sha256 = "ed0594a9ed6c492d125c67f490e3a94c135e870a98342e9216df12162fa6911e", - strip_prefix = "cloud.google.com/go/securitycenter@v1.23.1", + sha256 = "9f8495ac103e255ec42d0b1de9f93ebe7308b6614f705cbce1e13f02d260781d", + strip_prefix = "cloud.google.com/go/securitycenter@v1.24.2", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/securitycenter/com_google_cloud_go_securitycenter-v1.23.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/securitycenter/com_google_cloud_go_securitycenter-v1.23.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/securitycenter/com_google_cloud_go_securitycenter-v1.23.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/securitycenter/com_google_cloud_go_securitycenter-v1.23.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/securitycenter/com_google_cloud_go_securitycenter-v1.24.2.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/securitycenter/com_google_cloud_go_securitycenter-v1.24.2.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/securitycenter/com_google_cloud_go_securitycenter-v1.24.2.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/securitycenter/com_google_cloud_go_securitycenter-v1.24.2.zip", ], ) go_repository( name = "com_google_cloud_go_servicedirectory", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/servicedirectory", - sha256 = "266651c6851b26c8047bec746633d5773d80463352f8e708da3cf2cd7fdfbe40", - strip_prefix = "cloud.google.com/go/servicedirectory@v1.11.1", + sha256 = "b491da281b94e6e585d8062f97f9a845dfeb47ec8333705597295929e21937be", + strip_prefix = "cloud.google.com/go/servicedirectory@v1.11.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/servicedirectory/com_google_cloud_go_servicedirectory-v1.11.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/servicedirectory/com_google_cloud_go_servicedirectory-v1.11.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/servicedirectory/com_google_cloud_go_servicedirectory-v1.11.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/servicedirectory/com_google_cloud_go_servicedirectory-v1.11.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/servicedirectory/com_google_cloud_go_servicedirectory-v1.11.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/servicedirectory/com_google_cloud_go_servicedirectory-v1.11.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/servicedirectory/com_google_cloud_go_servicedirectory-v1.11.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/servicedirectory/com_google_cloud_go_servicedirectory-v1.11.3.zip", ], ) go_repository( name = "com_google_cloud_go_shell", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/shell", - sha256 = "f50cd0726dd7109c75b9775b6750b3316acb1f764d608ff02278e98cff327ecd", - strip_prefix = "cloud.google.com/go/shell@v1.7.2", + sha256 = "2e56de8da5669fc04e19ebc2af5581bea7778d39aaf0b13676fce72f41eb5302", + strip_prefix = "cloud.google.com/go/shell@v1.7.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/shell/com_google_cloud_go_shell-v1.7.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/shell/com_google_cloud_go_shell-v1.7.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/shell/com_google_cloud_go_shell-v1.7.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/shell/com_google_cloud_go_shell-v1.7.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/shell/com_google_cloud_go_shell-v1.7.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/shell/com_google_cloud_go_shell-v1.7.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/shell/com_google_cloud_go_shell-v1.7.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/shell/com_google_cloud_go_shell-v1.7.4.zip", ], ) go_repository( name = "com_google_cloud_go_spanner", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/spanner", - sha256 = "eb0dd423ec976db7be0e6a709cab5d7ad2b9e20ca53cf9cd9663475bf896531a", - strip_prefix = "cloud.google.com/go/spanner@v1.50.0", + sha256 = "4d60c2d98601cd8d11e332bb821ce5ee3c06ca8b4e58989d5efb18e135893f85", + strip_prefix = "cloud.google.com/go/spanner@v1.51.0", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/spanner/com_google_cloud_go_spanner-v1.50.0.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/spanner/com_google_cloud_go_spanner-v1.50.0.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/spanner/com_google_cloud_go_spanner-v1.50.0.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/spanner/com_google_cloud_go_spanner-v1.50.0.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/spanner/com_google_cloud_go_spanner-v1.51.0.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/spanner/com_google_cloud_go_spanner-v1.51.0.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/spanner/com_google_cloud_go_spanner-v1.51.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/spanner/com_google_cloud_go_spanner-v1.51.0.zip", ], ) go_repository( name = "com_google_cloud_go_speech", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/speech", - sha256 = "1c184b4698eea3072656dc4e4a0279befdd6d2fa31989f5006b0e46ee7ea5ceb", - strip_prefix = "cloud.google.com/go/speech@v1.19.1", + sha256 = "814ff4b2392758a14b7c4ec91e028564c984920ccecfaac5b817f086dcf295b6", + strip_prefix = "cloud.google.com/go/speech@v1.20.1", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/speech/com_google_cloud_go_speech-v1.19.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/speech/com_google_cloud_go_speech-v1.19.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/speech/com_google_cloud_go_speech-v1.19.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/speech/com_google_cloud_go_speech-v1.19.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/speech/com_google_cloud_go_speech-v1.20.1.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/speech/com_google_cloud_go_speech-v1.20.1.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/speech/com_google_cloud_go_speech-v1.20.1.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/speech/com_google_cloud_go_speech-v1.20.1.zip", ], ) go_repository( @@ -9069,195 +9095,195 @@ def go_deps(): name = "com_google_cloud_go_storagetransfer", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/storagetransfer", - sha256 = "e5c5f002aa7c05a4702d1bb4568b0d63a3384e095402517afaae4147fd6169e8", - strip_prefix = "cloud.google.com/go/storagetransfer@v1.10.1", + sha256 = "9e17ac17ad4986a5af3f36aec52b5e4befaa745050e3c46b4007a90933de8be0", + strip_prefix = "cloud.google.com/go/storagetransfer@v1.10.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/storagetransfer/com_google_cloud_go_storagetransfer-v1.10.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/storagetransfer/com_google_cloud_go_storagetransfer-v1.10.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/storagetransfer/com_google_cloud_go_storagetransfer-v1.10.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/storagetransfer/com_google_cloud_go_storagetransfer-v1.10.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/storagetransfer/com_google_cloud_go_storagetransfer-v1.10.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/storagetransfer/com_google_cloud_go_storagetransfer-v1.10.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/storagetransfer/com_google_cloud_go_storagetransfer-v1.10.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/storagetransfer/com_google_cloud_go_storagetransfer-v1.10.3.zip", ], ) go_repository( name = "com_google_cloud_go_talent", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/talent", - sha256 = "b797a2106e3aca18898ea51144e2308574f49f840fe51fe06f03d1dea56646e1", - strip_prefix = "cloud.google.com/go/talent@v1.6.3", + sha256 = "bd6065c07fb20189b4faa5ea4c06c0a508348257362c7802345f355441eee44b", + strip_prefix = "cloud.google.com/go/talent@v1.6.5", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/talent/com_google_cloud_go_talent-v1.6.3.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/talent/com_google_cloud_go_talent-v1.6.3.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/talent/com_google_cloud_go_talent-v1.6.3.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/talent/com_google_cloud_go_talent-v1.6.3.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/talent/com_google_cloud_go_talent-v1.6.5.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/talent/com_google_cloud_go_talent-v1.6.5.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/talent/com_google_cloud_go_talent-v1.6.5.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/talent/com_google_cloud_go_talent-v1.6.5.zip", ], ) go_repository( name = "com_google_cloud_go_texttospeech", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/texttospeech", - sha256 = "4a897af8724879bf479f715a57d0894f6fa3b52706e35870c385bcaa799aef2f", - strip_prefix = "cloud.google.com/go/texttospeech@v1.7.2", + sha256 = "c71d33c6ca31ff097331951ea52478e16caa73472e78d14758d6de9f64d10a06", + strip_prefix = "cloud.google.com/go/texttospeech@v1.7.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/texttospeech/com_google_cloud_go_texttospeech-v1.7.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/texttospeech/com_google_cloud_go_texttospeech-v1.7.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/texttospeech/com_google_cloud_go_texttospeech-v1.7.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/texttospeech/com_google_cloud_go_texttospeech-v1.7.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/texttospeech/com_google_cloud_go_texttospeech-v1.7.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/texttospeech/com_google_cloud_go_texttospeech-v1.7.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/texttospeech/com_google_cloud_go_texttospeech-v1.7.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/texttospeech/com_google_cloud_go_texttospeech-v1.7.4.zip", ], ) go_repository( name = "com_google_cloud_go_tpu", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/tpu", - sha256 = "48e359c9edd853357bb8f157a4ead1601d9c926b1c539fde86b5531139f60647", - strip_prefix = "cloud.google.com/go/tpu@v1.6.2", + sha256 = "96f561cae8d5fd1126c67b24eb25d3f601f88d79599cd251819802dc6531e4e7", + strip_prefix = "cloud.google.com/go/tpu@v1.6.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/tpu/com_google_cloud_go_tpu-v1.6.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/tpu/com_google_cloud_go_tpu-v1.6.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/tpu/com_google_cloud_go_tpu-v1.6.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/tpu/com_google_cloud_go_tpu-v1.6.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/tpu/com_google_cloud_go_tpu-v1.6.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/tpu/com_google_cloud_go_tpu-v1.6.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/tpu/com_google_cloud_go_tpu-v1.6.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/tpu/com_google_cloud_go_tpu-v1.6.4.zip", ], ) go_repository( name = "com_google_cloud_go_trace", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/trace", - sha256 = "40dd16a13c80f021b31ee309e80f6ee21323b67b2d3aac8473717ac3b3efce08", - strip_prefix = "cloud.google.com/go/trace@v1.10.2", + sha256 = "77dca06e712fe4fda0d79b8ff57f118ce038bf0cadec22657f5abc385a01a720", + strip_prefix = "cloud.google.com/go/trace@v1.10.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/trace/com_google_cloud_go_trace-v1.10.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/trace/com_google_cloud_go_trace-v1.10.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/trace/com_google_cloud_go_trace-v1.10.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/trace/com_google_cloud_go_trace-v1.10.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/trace/com_google_cloud_go_trace-v1.10.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/trace/com_google_cloud_go_trace-v1.10.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/trace/com_google_cloud_go_trace-v1.10.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/trace/com_google_cloud_go_trace-v1.10.4.zip", ], ) go_repository( name = "com_google_cloud_go_translate", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/translate", - sha256 = "1776353be213f41195b9da35ae6f39cff060f9c163a0213711d7cb11e4f067ff", - strip_prefix = "cloud.google.com/go/translate@v1.9.1", + sha256 = "c7130da1e8941ea8db679b62e0e94d8b01441a13fe780d8c2c18bc2132385197", + strip_prefix = "cloud.google.com/go/translate@v1.9.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/translate/com_google_cloud_go_translate-v1.9.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/translate/com_google_cloud_go_translate-v1.9.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/translate/com_google_cloud_go_translate-v1.9.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/translate/com_google_cloud_go_translate-v1.9.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/translate/com_google_cloud_go_translate-v1.9.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/translate/com_google_cloud_go_translate-v1.9.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/translate/com_google_cloud_go_translate-v1.9.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/translate/com_google_cloud_go_translate-v1.9.3.zip", ], ) go_repository( name = "com_google_cloud_go_video", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/video", - sha256 = "758131934c35db8aa8d39efa5ce411785da78dd55d82edbb7a9fcb8e9518d2a9", - strip_prefix = "cloud.google.com/go/video@v1.20.1", + sha256 = "863e12dae253753daebb1e5e9af15f68a1321adf177f3e7e9732bacf369def63", + strip_prefix = "cloud.google.com/go/video@v1.20.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/video/com_google_cloud_go_video-v1.20.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/video/com_google_cloud_go_video-v1.20.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/video/com_google_cloud_go_video-v1.20.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/video/com_google_cloud_go_video-v1.20.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/video/com_google_cloud_go_video-v1.20.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/video/com_google_cloud_go_video-v1.20.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/video/com_google_cloud_go_video-v1.20.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/video/com_google_cloud_go_video-v1.20.3.zip", ], ) go_repository( name = "com_google_cloud_go_videointelligence", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/videointelligence", - sha256 = "0ca9d8c68825f07c208157bf24503f4a64aec960efe3ea2ff8c1ce2dac92b351", - strip_prefix = "cloud.google.com/go/videointelligence@v1.11.2", + sha256 = "d37362c58b1cc6ff875fc308eabc7d82f88b900432fe84f1984775217778fef8", + strip_prefix = "cloud.google.com/go/videointelligence@v1.11.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/videointelligence/com_google_cloud_go_videointelligence-v1.11.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/videointelligence/com_google_cloud_go_videointelligence-v1.11.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/videointelligence/com_google_cloud_go_videointelligence-v1.11.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/videointelligence/com_google_cloud_go_videointelligence-v1.11.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/videointelligence/com_google_cloud_go_videointelligence-v1.11.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/videointelligence/com_google_cloud_go_videointelligence-v1.11.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/videointelligence/com_google_cloud_go_videointelligence-v1.11.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/videointelligence/com_google_cloud_go_videointelligence-v1.11.4.zip", ], ) go_repository( name = "com_google_cloud_go_vision_v2", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/vision/v2", - sha256 = "89b8e20f0db42816208b1d9c5a6cb1abd276fee95fbdd563e750da588d110464", - strip_prefix = "cloud.google.com/go/vision/v2@v2.7.3", + sha256 = "76b71582199e7a74a0277f7f421f4ecf9d858b5be3e34334479454091a5223d2", + strip_prefix = "cloud.google.com/go/vision/v2@v2.7.5", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/vision/v2/com_google_cloud_go_vision_v2-v2.7.3.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/vision/v2/com_google_cloud_go_vision_v2-v2.7.3.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/vision/v2/com_google_cloud_go_vision_v2-v2.7.3.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/vision/v2/com_google_cloud_go_vision_v2-v2.7.3.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/vision/v2/com_google_cloud_go_vision_v2-v2.7.5.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/vision/v2/com_google_cloud_go_vision_v2-v2.7.5.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/vision/v2/com_google_cloud_go_vision_v2-v2.7.5.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/vision/v2/com_google_cloud_go_vision_v2-v2.7.5.zip", ], ) go_repository( name = "com_google_cloud_go_vmmigration", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/vmmigration", - sha256 = "8ef0ba7a5fa6b436593782de63111e4274cb61267008bff10c9dc90285405dce", - strip_prefix = "cloud.google.com/go/vmmigration@v1.7.2", + sha256 = "bc850873b55bf103ccbe803b13c1406fa2931beed8bec5216c5428ed2b526312", + strip_prefix = "cloud.google.com/go/vmmigration@v1.7.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/vmmigration/com_google_cloud_go_vmmigration-v1.7.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/vmmigration/com_google_cloud_go_vmmigration-v1.7.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/vmmigration/com_google_cloud_go_vmmigration-v1.7.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/vmmigration/com_google_cloud_go_vmmigration-v1.7.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/vmmigration/com_google_cloud_go_vmmigration-v1.7.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/vmmigration/com_google_cloud_go_vmmigration-v1.7.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/vmmigration/com_google_cloud_go_vmmigration-v1.7.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/vmmigration/com_google_cloud_go_vmmigration-v1.7.4.zip", ], ) go_repository( name = "com_google_cloud_go_vmwareengine", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/vmwareengine", - sha256 = "906ad046857c81df8a0e8f30d09f3db9d2c13021a3374587d3acd2a734c60a13", - strip_prefix = "cloud.google.com/go/vmwareengine@v1.0.1", + sha256 = "ada0386900ac32eae7098d2c6a8647017ee476dffa5bb8731ec276086670fdb5", + strip_prefix = "cloud.google.com/go/vmwareengine@v1.0.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/vmwareengine/com_google_cloud_go_vmwareengine-v1.0.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/vmwareengine/com_google_cloud_go_vmwareengine-v1.0.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/vmwareengine/com_google_cloud_go_vmwareengine-v1.0.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/vmwareengine/com_google_cloud_go_vmwareengine-v1.0.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/vmwareengine/com_google_cloud_go_vmwareengine-v1.0.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/vmwareengine/com_google_cloud_go_vmwareengine-v1.0.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/vmwareengine/com_google_cloud_go_vmwareengine-v1.0.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/vmwareengine/com_google_cloud_go_vmwareengine-v1.0.3.zip", ], ) go_repository( name = "com_google_cloud_go_vpcaccess", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/vpcaccess", - sha256 = "80207274d8a780413505c4efdd881c5798d988ade2bc647ac803d18831a42250", - strip_prefix = "cloud.google.com/go/vpcaccess@v1.7.2", + sha256 = "aa17c93acf6059410989fd94086a632705e63844bee25d250488f3c6690da2a1", + strip_prefix = "cloud.google.com/go/vpcaccess@v1.7.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/vpcaccess/com_google_cloud_go_vpcaccess-v1.7.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/vpcaccess/com_google_cloud_go_vpcaccess-v1.7.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/vpcaccess/com_google_cloud_go_vpcaccess-v1.7.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/vpcaccess/com_google_cloud_go_vpcaccess-v1.7.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/vpcaccess/com_google_cloud_go_vpcaccess-v1.7.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/vpcaccess/com_google_cloud_go_vpcaccess-v1.7.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/vpcaccess/com_google_cloud_go_vpcaccess-v1.7.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/vpcaccess/com_google_cloud_go_vpcaccess-v1.7.4.zip", ], ) go_repository( name = "com_google_cloud_go_webrisk", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/webrisk", - sha256 = "f1b45df15670274eae77a62a7ae243b5eafb4e10f8f04c852ca73a026b9d03f7", - strip_prefix = "cloud.google.com/go/webrisk@v1.9.2", + sha256 = "25624aeb1e1984ea65d435ce16d12ec6e1a154d589a2d69d0d8e14a59de693a7", + strip_prefix = "cloud.google.com/go/webrisk@v1.9.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/webrisk/com_google_cloud_go_webrisk-v1.9.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/webrisk/com_google_cloud_go_webrisk-v1.9.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/webrisk/com_google_cloud_go_webrisk-v1.9.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/webrisk/com_google_cloud_go_webrisk-v1.9.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/webrisk/com_google_cloud_go_webrisk-v1.9.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/webrisk/com_google_cloud_go_webrisk-v1.9.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/webrisk/com_google_cloud_go_webrisk-v1.9.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/webrisk/com_google_cloud_go_webrisk-v1.9.4.zip", ], ) go_repository( name = "com_google_cloud_go_websecurityscanner", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/websecurityscanner", - sha256 = "ce37d83c1998f0dde1ca5b8e107a8654466271fda7c9b35614672da9d8a33144", - strip_prefix = "cloud.google.com/go/websecurityscanner@v1.6.2", + sha256 = "4fe6d2490392c4f85aff40f9609565c75531e02ecd48ac06b135dc71cef2dc15", + strip_prefix = "cloud.google.com/go/websecurityscanner@v1.6.4", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/websecurityscanner/com_google_cloud_go_websecurityscanner-v1.6.2.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/websecurityscanner/com_google_cloud_go_websecurityscanner-v1.6.2.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/websecurityscanner/com_google_cloud_go_websecurityscanner-v1.6.2.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/websecurityscanner/com_google_cloud_go_websecurityscanner-v1.6.2.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/websecurityscanner/com_google_cloud_go_websecurityscanner-v1.6.4.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/websecurityscanner/com_google_cloud_go_websecurityscanner-v1.6.4.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/websecurityscanner/com_google_cloud_go_websecurityscanner-v1.6.4.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/websecurityscanner/com_google_cloud_go_websecurityscanner-v1.6.4.zip", ], ) go_repository( name = "com_google_cloud_go_workflows", build_file_proto_mode = "disable_global", importpath = "cloud.google.com/go/workflows", - sha256 = "b8de38a09b26fc4a98a10a8370f4780361c27a13cc84244fcf8840c2ca0f402a", - strip_prefix = "cloud.google.com/go/workflows@v1.12.1", + sha256 = "5b3f1383cfc87ebba495ffb4f0d60aa68869b4ec61d63775095b84974bdcfb2d", + strip_prefix = "cloud.google.com/go/workflows@v1.12.3", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/workflows/com_google_cloud_go_workflows-v1.12.1.zip", - "http://ats.apps.svc/gomod/cloud.google.com/go/workflows/com_google_cloud_go_workflows-v1.12.1.zip", - "https://cache.hawkingrei.com/gomod/cloud.google.com/go/workflows/com_google_cloud_go_workflows-v1.12.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/workflows/com_google_cloud_go_workflows-v1.12.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/cloud.google.com/go/workflows/com_google_cloud_go_workflows-v1.12.3.zip", + "http://ats.apps.svc/gomod/cloud.google.com/go/workflows/com_google_cloud_go_workflows-v1.12.3.zip", + "https://cache.hawkingrei.com/gomod/cloud.google.com/go/workflows/com_google_cloud_go_workflows-v1.12.3.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/cloud.google.com/go/workflows/com_google_cloud_go_workflows-v1.12.3.zip", ], ) go_repository( @@ -9892,26 +9918,26 @@ def go_deps(): name = "io_opentelemetry_go_contrib_instrumentation_google_golang_org_grpc_otelgrpc", build_file_proto_mode = "disable_global", importpath = "go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc", - sha256 = "f7abd5e3fe79b49a398912f67f79d853e329bb4f653b98804a961d5178dadc5e", - strip_prefix = "go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc@v0.25.0", + sha256 = "77a1264ef9fbcbf742ac4df5eebfb6f3b46e5b34c222aa81391b5c24ac1acc5c", + strip_prefix = "go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc@v0.46.1", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc/io_opentelemetry_go_contrib_instrumentation_google_golang_org_grpc_otelgrpc-v0.25.0.zip", - "http://ats.apps.svc/gomod/go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc/io_opentelemetry_go_contrib_instrumentation_google_golang_org_grpc_otelgrpc-v0.25.0.zip", - "https://cache.hawkingrei.com/gomod/go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc/io_opentelemetry_go_contrib_instrumentation_google_golang_org_grpc_otelgrpc-v0.25.0.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc/io_opentelemetry_go_contrib_instrumentation_google_golang_org_grpc_otelgrpc-v0.25.0.zip", + "http://bazel-cache.pingcap.net:8080/gomod/go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc/io_opentelemetry_go_contrib_instrumentation_google_golang_org_grpc_otelgrpc-v0.46.1.zip", + "http://ats.apps.svc/gomod/go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc/io_opentelemetry_go_contrib_instrumentation_google_golang_org_grpc_otelgrpc-v0.46.1.zip", + "https://cache.hawkingrei.com/gomod/go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc/io_opentelemetry_go_contrib_instrumentation_google_golang_org_grpc_otelgrpc-v0.46.1.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc/io_opentelemetry_go_contrib_instrumentation_google_golang_org_grpc_otelgrpc-v0.46.1.zip", ], ) go_repository( name = "io_opentelemetry_go_otel", build_file_proto_mode = "disable_global", importpath = "go.opentelemetry.io/otel", - sha256 = "e8c4d785d6a230d5c954d7afbbb0df5c8a2ffb59aeb07bc4f7c731c6f55e0626", - strip_prefix = "go.opentelemetry.io/otel@v1.0.1", + sha256 = "4c633a67b2c5fd2db8a67ccf95edfea244188ebbefaf114b659679f02d33e9e1", + strip_prefix = "go.opentelemetry.io/otel@v1.21.0", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/go.opentelemetry.io/otel/io_opentelemetry_go_otel-v1.0.1.zip", - "http://ats.apps.svc/gomod/go.opentelemetry.io/otel/io_opentelemetry_go_otel-v1.0.1.zip", - "https://cache.hawkingrei.com/gomod/go.opentelemetry.io/otel/io_opentelemetry_go_otel-v1.0.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/go.opentelemetry.io/otel/io_opentelemetry_go_otel-v1.0.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/go.opentelemetry.io/otel/io_opentelemetry_go_otel-v1.21.0.zip", + "http://ats.apps.svc/gomod/go.opentelemetry.io/otel/io_opentelemetry_go_otel-v1.21.0.zip", + "https://cache.hawkingrei.com/gomod/go.opentelemetry.io/otel/io_opentelemetry_go_otel-v1.21.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/go.opentelemetry.io/otel/io_opentelemetry_go_otel-v1.21.0.zip", ], ) go_repository( @@ -9940,30 +9966,43 @@ def go_deps(): "https://storage.googleapis.com/pingcapmirror/gomod/go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc/io_opentelemetry_go_otel_exporters_otlp_otlptrace_otlptracegrpc-v1.0.1.zip", ], ) + go_repository( + name = "io_opentelemetry_go_otel_metric", + build_file_proto_mode = "disable_global", + importpath = "go.opentelemetry.io/otel/metric", + sha256 = "f1f0370f589bc0daa18b5a6b7a8b56f2c7c1d652da0e2541491b69ec31ac9303", + strip_prefix = "go.opentelemetry.io/otel/metric@v1.21.0", + urls = [ + "http://bazel-cache.pingcap.net:8080/gomod/go.opentelemetry.io/otel/metric/io_opentelemetry_go_otel_metric-v1.21.0.zip", + "http://ats.apps.svc/gomod/go.opentelemetry.io/otel/metric/io_opentelemetry_go_otel_metric-v1.21.0.zip", + "https://cache.hawkingrei.com/gomod/go.opentelemetry.io/otel/metric/io_opentelemetry_go_otel_metric-v1.21.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/go.opentelemetry.io/otel/metric/io_opentelemetry_go_otel_metric-v1.21.0.zip", + ], + ) go_repository( name = "io_opentelemetry_go_otel_sdk", build_file_proto_mode = "disable_global", importpath = "go.opentelemetry.io/otel/sdk", - sha256 = "760e9297c941b22cd3a5a2b217de46f8f2411cc7ef8dc1bab8ed02d75e10217d", - strip_prefix = "go.opentelemetry.io/otel/sdk@v1.0.1", + sha256 = "1c3f3d971627ae2c9c6102fe62c6bd68149feebba06b4a17cf0367c7d97c912d", + strip_prefix = "go.opentelemetry.io/otel/sdk@v1.21.0", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/go.opentelemetry.io/otel/sdk/io_opentelemetry_go_otel_sdk-v1.0.1.zip", - "http://ats.apps.svc/gomod/go.opentelemetry.io/otel/sdk/io_opentelemetry_go_otel_sdk-v1.0.1.zip", - "https://cache.hawkingrei.com/gomod/go.opentelemetry.io/otel/sdk/io_opentelemetry_go_otel_sdk-v1.0.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/go.opentelemetry.io/otel/sdk/io_opentelemetry_go_otel_sdk-v1.0.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/go.opentelemetry.io/otel/sdk/io_opentelemetry_go_otel_sdk-v1.21.0.zip", + "http://ats.apps.svc/gomod/go.opentelemetry.io/otel/sdk/io_opentelemetry_go_otel_sdk-v1.21.0.zip", + "https://cache.hawkingrei.com/gomod/go.opentelemetry.io/otel/sdk/io_opentelemetry_go_otel_sdk-v1.21.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/go.opentelemetry.io/otel/sdk/io_opentelemetry_go_otel_sdk-v1.21.0.zip", ], ) go_repository( name = "io_opentelemetry_go_otel_trace", build_file_proto_mode = "disable_global", importpath = "go.opentelemetry.io/otel/trace", - sha256 = "d7f303c3e1a9d844121309c132fab5f99dc68e9ac6518ef1d9c44f92ae9e97ea", - strip_prefix = "go.opentelemetry.io/otel/trace@v1.0.1", + sha256 = "4fc5d58e527a30baf406a702c1fff18c44739c39f6d63f1c648a93751a788ba5", + strip_prefix = "go.opentelemetry.io/otel/trace@v1.21.0", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/go.opentelemetry.io/otel/trace/io_opentelemetry_go_otel_trace-v1.0.1.zip", - "http://ats.apps.svc/gomod/go.opentelemetry.io/otel/trace/io_opentelemetry_go_otel_trace-v1.0.1.zip", - "https://cache.hawkingrei.com/gomod/go.opentelemetry.io/otel/trace/io_opentelemetry_go_otel_trace-v1.0.1.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/go.opentelemetry.io/otel/trace/io_opentelemetry_go_otel_trace-v1.0.1.zip", + "http://bazel-cache.pingcap.net:8080/gomod/go.opentelemetry.io/otel/trace/io_opentelemetry_go_otel_trace-v1.21.0.zip", + "http://ats.apps.svc/gomod/go.opentelemetry.io/otel/trace/io_opentelemetry_go_otel_trace-v1.21.0.zip", + "https://cache.hawkingrei.com/gomod/go.opentelemetry.io/otel/trace/io_opentelemetry_go_otel_trace-v1.21.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/go.opentelemetry.io/otel/trace/io_opentelemetry_go_otel_trace-v1.21.0.zip", ], ) go_repository( @@ -10061,13 +10100,13 @@ def go_deps(): name = "org_golang_google_api", build_file_proto_mode = "disable_global", importpath = "google.golang.org/api", - sha256 = "1c7547012d828329aa90dc77bfa7d826184b14229cc72c93eeca50cb9882158d", - strip_prefix = "google.golang.org/api@v0.128.0", + sha256 = "72ad7e7194310ac3eaae621dda412787b35382478edcdde4f12df9cad2f393c9", + strip_prefix = "google.golang.org/api@v0.149.0", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/google.golang.org/api/org_golang_google_api-v0.128.0.zip", - "http://ats.apps.svc/gomod/google.golang.org/api/org_golang_google_api-v0.128.0.zip", - "https://cache.hawkingrei.com/gomod/google.golang.org/api/org_golang_google_api-v0.128.0.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/google.golang.org/api/org_golang_google_api-v0.128.0.zip", + "http://bazel-cache.pingcap.net:8080/gomod/google.golang.org/api/org_golang_google_api-v0.149.0.zip", + "http://ats.apps.svc/gomod/google.golang.org/api/org_golang_google_api-v0.149.0.zip", + "https://cache.hawkingrei.com/gomod/google.golang.org/api/org_golang_google_api-v0.149.0.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/google.golang.org/api/org_golang_google_api-v0.149.0.zip", ], ) go_repository( @@ -10087,52 +10126,52 @@ def go_deps(): name = "org_golang_google_genproto", build_file_proto_mode = "disable_global", importpath = "google.golang.org/genproto", - sha256 = "11c4f5d4c24c25c9dec4fb2d4e723dead4c558bea81ab3b2deb3b1f5f98f278a", - strip_prefix = "google.golang.org/genproto@v0.0.0-20231016165738-49dd2c1f3d0b", + sha256 = "2d1407c2ce93329ed8348a11126eeccf88a96353a49540463f55bd26f7a342fb", + strip_prefix = "google.golang.org/genproto@v0.0.0-20231106174013-bbf56f31fb17", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20231016165738-49dd2c1f3d0b.zip", - "http://ats.apps.svc/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20231016165738-49dd2c1f3d0b.zip", - "https://cache.hawkingrei.com/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20231016165738-49dd2c1f3d0b.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20231016165738-49dd2c1f3d0b.zip", + "http://bazel-cache.pingcap.net:8080/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20231106174013-bbf56f31fb17.zip", + "http://ats.apps.svc/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20231106174013-bbf56f31fb17.zip", + "https://cache.hawkingrei.com/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20231106174013-bbf56f31fb17.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/google.golang.org/genproto/org_golang_google_genproto-v0.0.0-20231106174013-bbf56f31fb17.zip", ], ) go_repository( name = "org_golang_google_genproto_googleapis_api", build_file_proto_mode = "disable_global", importpath = "google.golang.org/genproto/googleapis/api", - sha256 = "26f62026bcd267b4fbde3bd16aa9334568af09e623fd64a8e7cd8ec99c0dbf5d", - strip_prefix = "google.golang.org/genproto/googleapis/api@v0.0.0-20231016165738-49dd2c1f3d0b", + sha256 = "521565d08641a65c1425b74db6bfff04ca2ee5cfde35aa784c1d647f418b6108", + strip_prefix = "google.golang.org/genproto/googleapis/api@v0.0.0-20231030173426-d783a09b4405", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/google.golang.org/genproto/googleapis/api/org_golang_google_genproto_googleapis_api-v0.0.0-20231016165738-49dd2c1f3d0b.zip", - "http://ats.apps.svc/gomod/google.golang.org/genproto/googleapis/api/org_golang_google_genproto_googleapis_api-v0.0.0-20231016165738-49dd2c1f3d0b.zip", - "https://cache.hawkingrei.com/gomod/google.golang.org/genproto/googleapis/api/org_golang_google_genproto_googleapis_api-v0.0.0-20231016165738-49dd2c1f3d0b.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/google.golang.org/genproto/googleapis/api/org_golang_google_genproto_googleapis_api-v0.0.0-20231016165738-49dd2c1f3d0b.zip", + "http://bazel-cache.pingcap.net:8080/gomod/google.golang.org/genproto/googleapis/api/org_golang_google_genproto_googleapis_api-v0.0.0-20231030173426-d783a09b4405.zip", + "http://ats.apps.svc/gomod/google.golang.org/genproto/googleapis/api/org_golang_google_genproto_googleapis_api-v0.0.0-20231030173426-d783a09b4405.zip", + "https://cache.hawkingrei.com/gomod/google.golang.org/genproto/googleapis/api/org_golang_google_genproto_googleapis_api-v0.0.0-20231030173426-d783a09b4405.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/google.golang.org/genproto/googleapis/api/org_golang_google_genproto_googleapis_api-v0.0.0-20231030173426-d783a09b4405.zip", ], ) go_repository( name = "org_golang_google_genproto_googleapis_bytestream", build_file_proto_mode = "disable_global", importpath = "google.golang.org/genproto/googleapis/bytestream", - sha256 = "cab1c8c198b4c5a226590e8b5b1b847e505a7aaf10a0145ce8c29951eca86b6a", - strip_prefix = "google.golang.org/genproto/googleapis/bytestream@v0.0.0-20230530153820-e85fd2cbaebc", + sha256 = "dfc6aba2d6c8a069e582efa17f6e00569ce90c2be98edd7a38283deeb69c493a", + strip_prefix = "google.golang.org/genproto/googleapis/bytestream@v0.0.0-20231030173426-d783a09b4405", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/google.golang.org/genproto/googleapis/bytestream/org_golang_google_genproto_googleapis_bytestream-v0.0.0-20230530153820-e85fd2cbaebc.zip", - "http://ats.apps.svc/gomod/google.golang.org/genproto/googleapis/bytestream/org_golang_google_genproto_googleapis_bytestream-v0.0.0-20230530153820-e85fd2cbaebc.zip", - "https://cache.hawkingrei.com/gomod/google.golang.org/genproto/googleapis/bytestream/org_golang_google_genproto_googleapis_bytestream-v0.0.0-20230530153820-e85fd2cbaebc.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/google.golang.org/genproto/googleapis/bytestream/org_golang_google_genproto_googleapis_bytestream-v0.0.0-20230530153820-e85fd2cbaebc.zip", + "http://bazel-cache.pingcap.net:8080/gomod/google.golang.org/genproto/googleapis/bytestream/org_golang_google_genproto_googleapis_bytestream-v0.0.0-20231030173426-d783a09b4405.zip", + "http://ats.apps.svc/gomod/google.golang.org/genproto/googleapis/bytestream/org_golang_google_genproto_googleapis_bytestream-v0.0.0-20231030173426-d783a09b4405.zip", + "https://cache.hawkingrei.com/gomod/google.golang.org/genproto/googleapis/bytestream/org_golang_google_genproto_googleapis_bytestream-v0.0.0-20231030173426-d783a09b4405.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/google.golang.org/genproto/googleapis/bytestream/org_golang_google_genproto_googleapis_bytestream-v0.0.0-20231030173426-d783a09b4405.zip", ], ) go_repository( name = "org_golang_google_genproto_googleapis_rpc", build_file_proto_mode = "disable_global", importpath = "google.golang.org/genproto/googleapis/rpc", - sha256 = "b35528074783811faaaec1d36c8f42d88b30878e509c5f407c53cb83ec02af78", - strip_prefix = "google.golang.org/genproto/googleapis/rpc@v0.0.0-20231016165738-49dd2c1f3d0b", + sha256 = "b351b49112507e61aeca7718b51570799d615ee5e5564f3e25124a9e01a85835", + strip_prefix = "google.golang.org/genproto/googleapis/rpc@v0.0.0-20231120223509-83a465c0220f", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/google.golang.org/genproto/googleapis/rpc/org_golang_google_genproto_googleapis_rpc-v0.0.0-20231016165738-49dd2c1f3d0b.zip", - "http://ats.apps.svc/gomod/google.golang.org/genproto/googleapis/rpc/org_golang_google_genproto_googleapis_rpc-v0.0.0-20231016165738-49dd2c1f3d0b.zip", - "https://cache.hawkingrei.com/gomod/google.golang.org/genproto/googleapis/rpc/org_golang_google_genproto_googleapis_rpc-v0.0.0-20231016165738-49dd2c1f3d0b.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/google.golang.org/genproto/googleapis/rpc/org_golang_google_genproto_googleapis_rpc-v0.0.0-20231016165738-49dd2c1f3d0b.zip", + "http://bazel-cache.pingcap.net:8080/gomod/google.golang.org/genproto/googleapis/rpc/org_golang_google_genproto_googleapis_rpc-v0.0.0-20231120223509-83a465c0220f.zip", + "http://ats.apps.svc/gomod/google.golang.org/genproto/googleapis/rpc/org_golang_google_genproto_googleapis_rpc-v0.0.0-20231120223509-83a465c0220f.zip", + "https://cache.hawkingrei.com/gomod/google.golang.org/genproto/googleapis/rpc/org_golang_google_genproto_googleapis_rpc-v0.0.0-20231120223509-83a465c0220f.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/google.golang.org/genproto/googleapis/rpc/org_golang_google_genproto_googleapis_rpc-v0.0.0-20231120223509-83a465c0220f.zip", ], ) go_repository( diff --git a/go.mod b/go.mod index 0c1eee32198b6..b16a42b492f51 100644 --- a/go.mod +++ b/go.mod @@ -53,7 +53,7 @@ require ( github.com/google/btree v1.1.2 github.com/google/pprof v0.0.0-20211122183932-1daafda22083 github.com/google/skylark v0.0.0-20181101142754-a5f7082aabed - github.com/google/uuid v1.3.1 + github.com/google/uuid v1.4.0 github.com/gordonklaus/ineffassign v0.0.0-20230610083614-0e73809eb601 github.com/gorilla/mux v1.8.0 github.com/gostaticanalysis/forcetypeassert v0.1.0 @@ -132,7 +132,7 @@ require ( golang.org/x/text v0.14.0 golang.org/x/time v0.4.0 golang.org/x/tools v0.15.0 - google.golang.org/api v0.128.0 + google.golang.org/api v0.149.0 google.golang.org/grpc v1.59.0 gopkg.in/yaml.v2 v2.4.0 honnef.co/go/tools v0.4.6 @@ -144,8 +144,9 @@ require ( require ( github.com/cenkalti/backoff/v4 v4.1.1 // indirect github.com/dolthub/maphash v0.1.0 // indirect + github.com/go-logr/stdr v1.2.2 // indirect github.com/golang-jwt/jwt/v4 v4.4.2 // indirect - github.com/google/s2a-go v0.1.4 // indirect + github.com/google/s2a-go v0.1.7 // indirect github.com/jfcg/sixb v1.3.8 // indirect github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 // indirect github.com/pkg/errors v0.9.1 // indirect @@ -154,16 +155,17 @@ require ( github.com/shabbyrobe/gocovmerge v0.0.0-20190829150210-3e036491d500 // indirect go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.0.1 // indirect go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.0.1 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b // indirect + go.opentelemetry.io/otel/metric v1.21.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20231030173426-d783a09b4405 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f // indirect k8s.io/utils v0.0.0-20230209194617-a36077c30491 // indirect ) require ( - cloud.google.com/go v0.110.8 // indirect - cloud.google.com/go/compute v1.23.1 // indirect + cloud.google.com/go v0.110.10 // indirect + cloud.google.com/go/compute v1.23.3 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect - cloud.google.com/go/iam v1.1.3 // indirect + cloud.google.com/go/iam v1.1.5 // indirect cloud.google.com/go/pubsub v1.33.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 // indirect github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 // indirect @@ -199,7 +201,7 @@ require ( github.com/go-asn1-ber/asn1-ber v1.5.4 // indirect github.com/go-kit/kit v0.9.0 // indirect github.com/go-logfmt/logfmt v0.5.1 // indirect - github.com/go-logr/logr v1.2.3 // indirect + github.com/go-logr/logr v1.3.0 // indirect github.com/go-ole/go-ole v1.3.0 // indirect github.com/goccy/go-json v0.10.2 // indirect github.com/golang-jwt/jwt v3.2.1+incompatible // indirect @@ -208,7 +210,7 @@ require ( github.com/google/gofuzz v1.1.0 // indirect github.com/google/licensecheck v0.3.1 // indirect github.com/google/renameio/v2 v2.0.0 // indirect - github.com/googleapis/enterprise-certificate-proxy v0.2.4 // indirect + github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect github.com/googleapis/gax-go/v2 v2.12.0 // indirect github.com/gorilla/handlers v1.5.1 // indirect github.com/gorilla/websocket v1.4.2 // indirect @@ -283,10 +285,10 @@ require ( go.etcd.io/etcd/client/v2 v2.305.10 // indirect go.etcd.io/etcd/pkg/v3 v3.5.10 // indirect go.etcd.io/etcd/raft/v3 v3.5.10 // indirect - go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.25.0 // indirect - go.opentelemetry.io/otel v1.0.1 // indirect - go.opentelemetry.io/otel/sdk v1.0.1 // indirect - go.opentelemetry.io/otel/trace v1.0.1 // indirect + go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1 // indirect + go.opentelemetry.io/otel v1.21.0 // indirect + go.opentelemetry.io/otel/sdk v1.21.0 // indirect + go.opentelemetry.io/otel/trace v1.21.0 // indirect go.opentelemetry.io/proto/otlp v0.9.0 // indirect golang.org/x/crypto v0.15.0 // indirect golang.org/x/exp/typeparams v0.0.0-20230307190834-24139beb5833 // indirect @@ -294,7 +296,7 @@ require ( golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect gonum.org/v1/gonum v0.8.2 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b // indirect + google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17 // indirect google.golang.org/protobuf v1.31.0 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect diff --git a/go.sum b/go.sum index 2f9fcf89c5e93..9e8da840933b6 100644 --- a/go.sum +++ b/go.sum @@ -13,24 +13,24 @@ cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKV cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= -cloud.google.com/go v0.110.8 h1:tyNdfIxjzaWctIiLYOTalaLKZ17SI44SKFW26QbOhME= -cloud.google.com/go v0.110.8/go.mod h1:Iz8AkXJf1qmxC3Oxoep8R1T36w8B92yU29PcBhHO5fk= +cloud.google.com/go v0.110.10 h1:LXy9GEO+timppncPIAZoOj3l58LIU9k+kn48AN7IO3Y= +cloud.google.com/go v0.110.10/go.mod h1:v1OoFqYxiBkUrruItNM3eT4lLByNjxmJSV/xDKJNnic= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= -cloud.google.com/go/compute v1.23.1 h1:V97tBoDaZHb6leicZ1G6DLK2BAaZLJ/7+9BB/En3hR0= -cloud.google.com/go/compute v1.23.1/go.mod h1:CqB3xpmPKKt3OJpW2ndFIXnA9A4xAy/F3Xp1ixncW78= +cloud.google.com/go/compute v1.23.3 h1:6sVlXXBmbd7jNX0Ipq0trII3e4n1/MsADLK6a+aiVlk= +cloud.google.com/go/compute v1.23.3/go.mod h1:VCgBUoMnIVIR0CscqQiPJLAG25E3ZRZMzcFZeQ+h8CI= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= -cloud.google.com/go/iam v1.1.3 h1:18tKG7DzydKWUnLjonWcJO6wjSCAtzh4GcRKlH/Hrzc= -cloud.google.com/go/iam v1.1.3/go.mod h1:3khUlaBXfPKKe7huYgEpDn6FtgRyMEqbkvBxrQyY5SE= -cloud.google.com/go/kms v1.15.3 h1:RYsbxTRmk91ydKCzekI2YjryO4c5Y2M80Zwcs9/D/cI= -cloud.google.com/go/kms v1.15.3/go.mod h1:AJdXqHxS2GlPyduM99s9iGqi2nwbviBbhV/hdmt4iOQ= +cloud.google.com/go/iam v1.1.5 h1:1jTsCu4bcsNsE4iiqNT5SHwrDRCfRmIaaaVFhRveTJI= +cloud.google.com/go/iam v1.1.5/go.mod h1:rB6P/Ic3mykPbFio+vo7403drjlgvoWfYpJhMXEbzv8= +cloud.google.com/go/kms v1.15.5 h1:pj1sRfut2eRbD9pFRjNnPNg/CzJPuQAzUujMIM1vVeM= +cloud.google.com/go/kms v1.15.5/go.mod h1:cU2H5jnp6G2TDpUGZyqTCoy1n16fbubHZjmVXSMtwDI= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -180,11 +180,7 @@ github.com/cloudfoundry/gosigar v1.3.6 h1:gIc08FbB3QPb+nAQhINIK/qhf5REKkY0FTGgRG github.com/cloudfoundry/gosigar v1.3.6/go.mod h1:lNWstu5g5gw59O09Y+wsMNFzBSnU8a0u+Sfx4dq360E= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= -github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI= -github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20210805033703-aa0b78936158/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4 h1:/inchEIKaYC1Akx+H+gqO04wryn5h75LSazbRlnya1k= github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMeY4+DwBQ= @@ -273,7 +269,6 @@ github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.m github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= -github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ= github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go.mod h1:AFq3mo9L8Lqqiid3OhADV3RfLJnjiw63cSpi+fDTRC0= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/envoyproxy/protoc-gen-validate v1.0.2 h1:QkIBuU5k+x7/QXPvPPnWXWlCdaBFApVqftFV6k087DA= @@ -332,8 +327,11 @@ github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V github.com/go-logfmt/logfmt v0.5.1 h1:otpy5pqBCBZ1ng9RQ0dPu4PN7ba75Y/aA+UpowDyNVA= github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v1.2.0/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/logr v1.2.3 h1:2DntVwHkVopvECVRSlL5PSo9eG+cAkDCuckLubN+rq0= -github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.3.0 h1:2y3SDp0ZXuc6/cjLSZ+Q3ir+QB9T/iG5yYRXqsagWSY= +github.com/go-logr/logr v1.3.0/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= +github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= @@ -465,16 +463,16 @@ github.com/google/pprof v0.0.0-20211122183932-1daafda22083/go.mod h1:KgnwoLYCZ8I github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/renameio/v2 v2.0.0 h1:UifI23ZTGY8Tt29JbYFiuyIU3eX+RNFtUwefq9qAhxg= github.com/google/renameio/v2 v2.0.0/go.mod h1:BtmJXm5YlszgC+TD4HOEEUFgkJP3nLxehU6hfe7jRt4= -github.com/google/s2a-go v0.1.4 h1:1kZ/sQM3srePvKs3tXAvQzo66XfcReoqFpIpIccE7Oc= -github.com/google/s2a-go v0.1.4/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= +github.com/google/s2a-go v0.1.7 h1:60BLSyTrOV4/haCDW4zb1guZItoSq8foHCXrAnjBo/o= +github.com/google/s2a-go v0.1.7/go.mod h1:50CgR4k1jNlWBu4UfS4AcfhVe1r6pdZPygJ3R8F0Qdw= github.com/google/skylark v0.0.0-20181101142754-a5f7082aabed h1:rZdD1GeRTHD1aG+VIvhQEYXurx6Wfg4QIT5YVl2tSC8= github.com/google/skylark v0.0.0-20181101142754-a5f7082aabed/go.mod h1:CKSX6SxHW1vp20ZNaeGe3TFFBIwCG6vaYrpAiOzX+NA= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4= -github.com/google/uuid v1.3.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/googleapis/enterprise-certificate-proxy v0.2.4 h1:uGy6JWR/uMIILU8wbf+OkstIrNiMjGpEIyhx8f6W7s4= -github.com/googleapis/enterprise-certificate-proxy v0.2.4/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= +github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4= +github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfFxPRy3Bf7vr3h0cechB90XaQs= +github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gax-go/v2 v2.12.0 h1:A+gCJKdRfqXkr+BIRGtZLibNXf0m1f9E4HG56etFpas= @@ -1079,18 +1077,23 @@ go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.25.0 h1:Wx7nFnvCaissIUZxPkBqDz2963Z+Cl+PkYbDKzTxDqQ= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.25.0/go.mod h1:E5NNboN0UqSAki0Atn9kVwaN7I+l25gGxDqBueo/74E= -go.opentelemetry.io/otel v1.0.1 h1:4XKyXmfqJLOQ7feyV5DB6gsBFZ0ltB8vLtp6pj4JIcc= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1 h1:SpGay3w+nEwMpfVnbqOLH5gY52/foP8RE8UzTZ1pdSE= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.46.1/go.mod h1:4UoMYEZOC0yN/sPGH76KPkkU7zgiEWYWL9vwmbnTJPE= go.opentelemetry.io/otel v1.0.1/go.mod h1:OPEOD4jIT2SlZPMmwT6FqZz2C0ZNdQqiWcoK6M0SNFU= +go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= +go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.0.1 h1:ofMbch7i29qIUf7VtF+r0HRF6ac0SBaPSziSsKp7wkk= go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.0.1/go.mod h1:Kv8liBeVNFkkkbilbgWRpV+wWuu+H5xdOT6HAgd30iw= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.0.1 h1:CFMFNoz+CGprjFAFy+RJFrfEe4GBia3RRm2a4fREvCA= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.0.1/go.mod h1:xOvWoTOrQjxjW61xtOmD/WKGRYb/P4NzRo3bs65U6Rk= -go.opentelemetry.io/otel/sdk v1.0.1 h1:wXxFEWGo7XfXupPwVJvTBOaPBC9FEg0wB8hMNrKk+cA= +go.opentelemetry.io/otel/metric v1.21.0 h1:tlYWfeo+Bocx5kLEloTjbcDwBuELRrIFxwdQ36PlJu4= +go.opentelemetry.io/otel/metric v1.21.0/go.mod h1:o1p3CA8nNHW8j5yuQLdc1eeqEaPfzug24uvsyIEJRWM= go.opentelemetry.io/otel/sdk v1.0.1/go.mod h1:HrdXne+BiwsOHYYkBE5ysIcv2bvdZstxzmCQhxTcZkI= -go.opentelemetry.io/otel/trace v1.0.1 h1:StTeIH6Q3G4r0Fiw34LTokUFESZgIDUr0qIJ7mKmAfw= +go.opentelemetry.io/otel/sdk v1.21.0 h1:FTt8qirL1EysG6sTQRZ5TokkU8d0ugCj8htOgThZXQ8= +go.opentelemetry.io/otel/sdk v1.21.0/go.mod h1:Nna6Yv7PWTdgJHVRD9hIYywQBRx7pbox6nwBnZIxl/E= go.opentelemetry.io/otel/trace v1.0.1/go.mod h1:5g4i4fKLaX2BQpSBsxw8YYcgKpMMSW3x7ZTuYBr3sUk= +go.opentelemetry.io/otel/trace v1.21.0 h1:WD9i5gzvoUPuXIXH24ZNBudiarZDKuekPqi/E8fpfLc= +go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.9.0 h1:C0g6TWmQYvjKRnljRULLWUVJGy8Uvu0NEL/5frY2/t4= go.opentelemetry.io/proto/otlp v0.9.0/go.mod h1:1vKfU9rv61e9EVGthD1zNvUbiwPcimSsOPU9brfSHJg= @@ -1143,7 +1146,6 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20220314234659-1baeb1ce4c0b/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20220518034528-6f7dac969898/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.1.0/go.mod h1:RecgLatLF4+eUMCP1PoPZQb+cVrJcOPbHkTkbkB9sbw= golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= @@ -1373,7 +1375,6 @@ golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= @@ -1486,8 +1487,8 @@ google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0M google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= -google.golang.org/api v0.128.0 h1:RjPESny5CnQRn9V6siglged+DZCgfu9l6mO9dkX9VOg= -google.golang.org/api v0.128.0/go.mod h1:Y611qgqaE92On/7g65MQgxYul3c0rEB894kniWLY750= +google.golang.org/api v0.149.0 h1:b2CqT6kG+zqJIVKRQ3ELJVLN1PwHZ6DJ3dW8yl82rgY= +google.golang.org/api v0.149.0/go.mod h1:Mwn1B7JTXrzXtnvmzQE2BD6bYZQ8DShKZDZbeN9I7qI= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1529,12 +1530,12 @@ google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7Fc google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b h1:+YaDE2r2OG8t/z5qmsh7Y+XXwCbvadxxZ0YY6mTdrVA= -google.golang.org/genproto v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:CgAqfJo+Xmu0GwA0411Ht3OU3OntXwsGmrmjI8ioGXI= -google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b h1:CIC2YMXmIhYw6evmhPxBKJ4fmLbOFtXQN/GV3XOZR8k= -google.golang.org/genproto/googleapis/api v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:IBQ646DjkDkvUIsVq/cc03FUFQ9wbZu7yE396YcL870= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b h1:ZlWIi1wSK56/8hn4QcBp/j9M7Gt3U/3hZw3mC7vDICo= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231016165738-49dd2c1f3d0b/go.mod h1:swOH3j0KzcDDgGUWr+SNpyTen5YrXjS3eyPzFYKc6lc= +google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17 h1:wpZ8pe2x1Q3f2KyT5f8oP/fa9rHAKgFPr/HZdNuS+PQ= +google.golang.org/genproto v0.0.0-20231106174013-bbf56f31fb17/go.mod h1:J7XzRzVy1+IPwWHZUzoD0IccYZIrXILAQpc+Qy9CMhY= +google.golang.org/genproto/googleapis/api v0.0.0-20231030173426-d783a09b4405 h1:HJMDndgxest5n2y77fnErkM62iUsptE/H8p0dC2Huo4= +google.golang.org/genproto/googleapis/api v0.0.0-20231030173426-d783a09b4405/go.mod h1:oT32Z4o8Zv2xPQTg0pbVaPr0MPOH6f14RgXt7zfIpwg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f h1:ultW7fxlIvee4HYrtnaRPon9HpEgFk5zYpmfMgtKB5I= +google.golang.org/genproto/googleapis/rpc v0.0.0-20231120223509-83a465c0220f/go.mod h1:L9KNLi232K1/xB6f7AlSX692koaRnKaWSR0stBki0Yc= google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= @@ -1556,9 +1557,7 @@ google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTp google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= google.golang.org/grpc v1.37.1/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= -google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= google.golang.org/grpc v1.41.0/go.mod h1:U3l9uK9J0sini8mHphKoXyaqDA/8VyGnDee1zzIUK6k= -google.golang.org/grpc v1.45.0/go.mod h1:lN7owxKUQEqMfSyQikvvk5tf/6zMPsrK+ONuO11+0rQ= google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= From a49630b3f571850359a7d84cb4e77222cedd6dd9 Mon Sep 17 00:00:00 2001 From: GMHDBJD <35025882+GMHDBJD@users.noreply.github.com> Date: Thu, 23 Nov 2023 00:08:41 +0800 Subject: [PATCH 12/36] ddl: use memory available as the memory limit for add index (#48268) ref pingcap/tidb#47757 --- pkg/ddl/backfilling_operators.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/pkg/ddl/backfilling_operators.go b/pkg/ddl/backfilling_operators.go index d5d434d6c3dac..a766142977243 100644 --- a/pkg/ddl/backfilling_operators.go +++ b/pkg/ddl/backfilling_operators.go @@ -203,7 +203,13 @@ func NewWriteIndexToExternalStoragePipeline( if err != nil { return nil, err } - memSize := (memTotal / 2) / uint64(writerCnt) / uint64(len(indexes)) + memUsed, err := memory.MemUsed() + if err != nil { + return nil, err + } + memAvailable := memTotal - memUsed + memSize := (memAvailable / 2) / uint64(writerCnt) / uint64(len(indexes)) + logutil.BgLogger().Info("build operators that write index to cloud storage", zap.Uint64("memory total", memTotal), zap.Uint64("memory used", memUsed), zap.Uint64("memory size", memSize)) srcOp := NewTableScanTaskSource(ctx, store, tbl, startKey, endKey) scanOp := NewTableScanOperator(ctx, sessPool, copCtx, srcChkPool, readerCnt) From 1cf28267277626c4ee5bacaf3519231068df6fdf Mon Sep 17 00:00:00 2001 From: wjHuang Date: Thu, 23 Nov 2023 01:11:40 +0800 Subject: [PATCH 13/36] ddl: fix add wrong timezone during adding index (#48805) ref pingcap/tidb#48724 --- pkg/ddl/backfilling_scheduler.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pkg/ddl/backfilling_scheduler.go b/pkg/ddl/backfilling_scheduler.go index 5ceb9624f81b5..958862f342e83 100644 --- a/pkg/ddl/backfilling_scheduler.go +++ b/pkg/ddl/backfilling_scheduler.go @@ -143,12 +143,18 @@ func newSessCtx( return sessCtx, nil } +// initSessCtx initializes the session context. Be careful to the timezone. func initSessCtx( sessCtx sessionctx.Context, sqlMode mysql.SQLMode, tzLocation *model.TimeZoneLocation, resGroupName string, ) error { + // Correct the initial timezone. + tz := *time.UTC + sessCtx.GetSessionVars().TimeZone = &tz + sessCtx.GetSessionVars().StmtCtx.SetTimeZone(&tz) + // Set the row encode format version. rowFormat := variable.GetDDLReorgRowFormat() sessCtx.GetSessionVars().RowEncoder.Enable = rowFormat != variable.DefTiDBRowFormatV1 From 0e8f8cd2940ff2c0c48d730fafd3340df0e503ee Mon Sep 17 00:00:00 2001 From: tangenta Date: Thu, 23 Nov 2023 10:19:10 +0800 Subject: [PATCH 14/36] infoschema: make information_schema.columns copyable (#45552) close pingcap/tidb#42030 --- pkg/ddl/tests/serial/serial_test.go | 6 ++++++ pkg/infoschema/tables.go | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/pkg/ddl/tests/serial/serial_test.go b/pkg/ddl/tests/serial/serial_test.go index c309da045d59e..79818967e8e8c 100644 --- a/pkg/ddl/tests/serial/serial_test.go +++ b/pkg/ddl/tests/serial/serial_test.go @@ -217,6 +217,12 @@ func TestCreateTableWithLike(t *testing.T) { tk.MustExec("drop database ctwl_db") tk.MustExec("drop database ctwl_db1") + + // Test information_schema.columns copiability. + // See https://github.com/pingcap/tidb/issues/42030. + tk.MustExec("use test") + tk.MustExec("create table cc like information_schema.columns;") + tk.MustExec("insert into cc select * from information_schema.columns;") } func TestCreateTableWithLikeAtTemporaryMode(t *testing.T) { diff --git a/pkg/infoschema/tables.go b/pkg/infoschema/tables.go index 0236f3bf77456..5f0e2b86c70cb 100644 --- a/pkg/infoschema/tables.go +++ b/pkg/infoschema/tables.go @@ -462,7 +462,7 @@ var columnsCols = []columnInfo{ {name: "COLLATION_NAME", tp: mysql.TypeVarchar, size: 32}, {name: "COLUMN_TYPE", tp: mysql.TypeBlob, size: 196606}, {name: "COLUMN_KEY", tp: mysql.TypeVarchar, size: 3}, - {name: "EXTRA", tp: mysql.TypeVarchar, size: 30}, + {name: "EXTRA", tp: mysql.TypeVarchar, size: 45}, {name: "PRIVILEGES", tp: mysql.TypeVarchar, size: 80}, {name: "COLUMN_COMMENT", tp: mysql.TypeVarchar, size: 1024}, {name: "GENERATION_EXPRESSION", tp: mysql.TypeBlob, size: 589779, flag: mysql.NotNullFlag}, From eff04125e73b1b7ee87dcadffa919e908043ac2d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Thu, 23 Nov 2023 11:40:11 +0800 Subject: [PATCH 15/36] br, util/ranger: replace `EvalWithInnerCtx` with `Eval` for packages br and util/ranger (#48785) close pingcap/tidb#48783 --- br/pkg/lightning/backend/kv/base.go | 2 +- pkg/executor/importer/kv_encode.go | 2 +- pkg/util/ranger/detacher.go | 4 +-- pkg/util/ranger/points.go | 52 ++++++++++++++++------------- pkg/util/ranger/ranger.go | 4 +-- 5 files changed, 34 insertions(+), 30 deletions(-) diff --git a/br/pkg/lightning/backend/kv/base.go b/br/pkg/lightning/backend/kv/base.go index 22644d1628082..b4cfeb1d318f5 100644 --- a/br/pkg/lightning/backend/kv/base.go +++ b/br/pkg/lightning/backend/kv/base.go @@ -353,7 +353,7 @@ func evalGeneratedColumns(se *Session, record []types.Datum, cols []*table.Colum mutRow := chunk.MutRowFromDatums(record) for _, gc := range genCols { col := cols[gc.Index].ToInfo() - evaluated, err := gc.Expr.EvalWithInnerCtx(mutRow.ToRow()) + evaluated, err := gc.Expr.Eval(se, mutRow.ToRow()) if err != nil { return col, err } diff --git a/pkg/executor/importer/kv_encode.go b/pkg/executor/importer/kv_encode.go index 15b6a1aa47d5a..f39e9c69a179d 100644 --- a/pkg/executor/importer/kv_encode.go +++ b/pkg/executor/importer/kv_encode.go @@ -142,7 +142,7 @@ func (en *tableKVEncoder) parserData2TableData(parserData []types.Datum, rowID i } for i := 0; i < len(en.columnAssignments); i++ { // eval expression of `SET` clause - d, err := en.columnAssignments[i].EvalWithInnerCtx(chunk.Row{}) + d, err := en.columnAssignments[i].Eval(en.SessionCtx, chunk.Row{}) if err != nil { return nil, err } diff --git a/pkg/util/ranger/detacher.go b/pkg/util/ranger/detacher.go index 4db29070fc2e4..b898e2260eeb9 100644 --- a/pkg/util/ranger/detacher.go +++ b/pkg/util/ranger/detacher.go @@ -608,7 +608,7 @@ func extractValueInfo(expr expression.Expression) *valueInfo { func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Expression, cols []*expression.Column, lengths []int) ([]expression.Expression, []expression.Expression, []expression.Expression, []*valueInfo, bool) { var filters []expression.Expression - rb := builder{sc: sctx.GetSessionVars().StmtCtx} + rb := builder{ctx: sctx} accesses := make([]expression.Expression, len(cols)) points := make([][]*point, len(cols)) mergedAccesses := make([]expression.Expression, len(cols)) @@ -715,7 +715,7 @@ func (d *rangeDetacher) detachDNFCondAndBuildRangeForIndex(condition *expression optPrefixIndexSingleScan: d.sctx.GetSessionVars().OptPrefixIndexSingleScan, ctx: d.sctx, } - rb := builder{sc: d.sctx.GetSessionVars().StmtCtx} + rb := builder{ctx: d.sctx} dnfItems := expression.FlattenDNFConditions(condition) newAccessItems := make([]expression.Expression, 0, len(dnfItems)) var totalRanges Ranges diff --git a/pkg/util/ranger/points.go b/pkg/util/ranger/points.go index 2e83a7c4cbd2f..feebebb36441d 100644 --- a/pkg/util/ranger/points.go +++ b/pkg/util/ranger/points.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/collate" @@ -86,7 +86,7 @@ func (rp *point) Clone(value types.Datum) *point { type pointSorter struct { err error collator collate.Collator - sc *stmtctx.StatementContext + tc types.Context points []*point } @@ -97,25 +97,25 @@ func (r *pointSorter) Len() int { func (r *pointSorter) Less(i, j int) bool { a := r.points[i] b := r.points[j] - less, err := rangePointLess(r.sc, a, b, r.collator) + less, err := rangePointLess(r.tc, a, b, r.collator) if err != nil { r.err = err } return less } -func rangePointLess(sc *stmtctx.StatementContext, a, b *point, collator collate.Collator) (bool, error) { +func rangePointLess(tc types.Context, a, b *point, collator collate.Collator) (bool, error) { if a.value.Kind() == types.KindMysqlEnum && b.value.Kind() == types.KindMysqlEnum { - return rangePointEnumLess(sc, a, b) + return rangePointEnumLess(a, b) } - cmp, err := a.value.Compare(sc.TypeCtx(), &b.value, collator) + cmp, err := a.value.Compare(tc, &b.value, collator) if cmp != 0 { return cmp < 0, nil } return rangePointEqualValueLess(a, b), errors.Trace(err) } -func rangePointEnumLess(_ *stmtctx.StatementContext, a, b *point) (bool, error) { +func rangePointEnumLess(a, b *point) (bool, error) { cmp := cmp.Compare(a.value.GetInt64(), b.value.GetInt64()) if cmp != 0 { return cmp < 0, nil @@ -183,7 +183,7 @@ func NullRange() Ranges { // builder is the range builder struct. type builder struct { err error - sc *stmtctx.StatementContext + ctx sessionctx.Context } func (r *builder) build(expr expression.Expression, collator collate.Collator) []*point { @@ -200,7 +200,7 @@ func (r *builder) build(expr expression.Expression, collator collate.Collator) [ } func (r *builder) buildFromConstant(expr *expression.Constant) []*point { - dt, err := expr.EvalWithInnerCtx(chunk.Row{}) + dt, err := expr.Eval(r.ctx, chunk.Row{}) if err != nil { r.err = err return nil @@ -209,7 +209,8 @@ func (r *builder) buildFromConstant(expr *expression.Constant) []*point { return nil } - val, err := dt.ToBool(r.sc.TypeCtx()) + tc := r.ctx.GetSessionVars().StmtCtx.TypeCtx() + val, err := dt.ToBool(tc) if err != nil { r.err = err return nil @@ -242,6 +243,7 @@ func (r *builder) buildFromBinOp(expr *expression.ScalarFunction) []*point { ft *types.FieldType ) + tc := r.ctx.GetSessionVars().StmtCtx.TypeCtx() // refineValueAndOp refines the constant datum and operator: // 1. for string type since we may eval the constant to another collation instead of its own collation. // 2. for year type since 2-digit year value need adjustment, see https://dev.mysql.com/doc/refman/5.6/en/year.html @@ -254,11 +256,11 @@ func (r *builder) buildFromBinOp(expr *expression.ScalarFunction) []*point { // If the original value is adjusted, we need to change the condition. // For example, col < 2156. Since the max year is 2155, 2156 is changed to 2155. // col < 2155 is wrong. It should be col <= 2155. - preValue, err1 := value.ToInt64(r.sc.TypeCtx()) + preValue, err1 := value.ToInt64(tc) if err1 != nil { return err1 } - *value, err = value.ConvertToMysqlYear(r.sc.TypeCtx(), col.RetType) + *value, err = value.ConvertToMysqlYear(tc, col.RetType) if errors.ErrorEqual(err, types.ErrWarnDataOutOfRange) { // Keep err for EQ and NE. switch *op { @@ -283,7 +285,7 @@ func (r *builder) buildFromBinOp(expr *expression.ScalarFunction) []*point { var ok bool if col, ok = expr.GetArgs()[0].(*expression.Column); ok { ft = col.RetType - value, err = expr.GetArgs()[1].EvalWithInnerCtx(chunk.Row{}) + value, err = expr.GetArgs()[1].Eval(r.ctx, chunk.Row{}) if err != nil { return nil } @@ -294,7 +296,7 @@ func (r *builder) buildFromBinOp(expr *expression.ScalarFunction) []*point { return nil } ft = col.RetType - value, err = expr.GetArgs()[0].EvalWithInnerCtx(chunk.Row{}) + value, err = expr.GetArgs()[0].Eval(r.ctx, chunk.Row{}) if err != nil { return nil } @@ -335,7 +337,7 @@ func (r *builder) buildFromBinOp(expr *expression.ScalarFunction) []*point { } if ft.GetType() == mysql.TypeEnum && ft.EvalType() == types.ETString { - return handleEnumFromBinOp(r.sc, ft, value, op) + return handleEnumFromBinOp(tc, ft, value, op) } switch op { @@ -452,7 +454,7 @@ func handleBoundCol(ft *types.FieldType, val types.Datum, op string) (types.Datu return val, op, true } -func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val types.Datum, op string) []*point { +func handleEnumFromBinOp(tc types.Context, ft *types.FieldType, val types.Datum, op string) []*point { res := make([]*point, 0, len(ft.GetElems())*2) appendPointFunc := func(d types.Datum) { res = append(res, &point{value: d, excl: false, start: true}) @@ -473,7 +475,7 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val } d := types.NewCollateMysqlEnumDatum(tmpEnum, ft.GetCollate()) - if v, err := d.Compare(sc.TypeCtx(), &val, collate.GetCollator(ft.GetCollate())); err == nil { + if v, err := d.Compare(tc, &val, collate.GetCollator(ft.GetCollate())); err == nil { switch op { case ast.LT: if v < 0 { @@ -558,13 +560,14 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool) rangePoints := make([]*point, 0, len(list)*2) hasNull := false colCollate := expr.GetArgs()[0].GetType().GetCollate() + tc := r.ctx.GetSessionVars().StmtCtx.TypeCtx() for _, e := range list { v, ok := e.(*expression.Constant) if !ok { r.err = ErrUnsupportedType.GenWithStack("expr:%v is not constant", e) return getFullRange(), hasNull } - dt, err := v.EvalWithInnerCtx(chunk.Row{}) + dt, err := v.Eval(r.ctx, chunk.Row{}) if err != nil { r.err = ErrUnsupportedType.GenWithStack("expr:%v is not evaluated", e) return getFullRange(), hasNull @@ -585,7 +588,7 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool) err = parseErr } default: - dt, err = dt.ConvertTo(r.sc.TypeCtx(), expr.GetArgs()[0].GetType()) + dt, err = dt.ConvertTo(tc, expr.GetArgs()[0].GetType()) } if err != nil { @@ -594,7 +597,7 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool) } } if expr.GetArgs()[0].GetType().GetType() == mysql.TypeYear { - dt, err = dt.ConvertToMysqlYear(r.sc.TypeCtx(), expr.GetArgs()[0].GetType()) + dt, err = dt.ConvertToMysqlYear(tc, expr.GetArgs()[0].GetType()) if err != nil { // in (..., an impossible value (not valid year), ...), the range is empty, so skip it. continue @@ -610,7 +613,7 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool) endPoint := &point{value: endValue} rangePoints = append(rangePoints, startPoint, endPoint) } - sorter := pointSorter{points: rangePoints, sc: r.sc, collator: collate.GetCollator(colCollate)} + sorter := pointSorter{points: rangePoints, tc: tc, collator: collate.GetCollator(colCollate)} sort.Sort(&sorter) if sorter.err != nil { r.err = sorter.err @@ -637,7 +640,7 @@ func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []*po if !collate.CompatibleCollate(expr.GetArgs()[0].GetType().GetCollate(), collation) { return getFullRange() } - pdt, err := expr.GetArgs()[1].(*expression.Constant).EvalWithInnerCtx(chunk.Row{}) + pdt, err := expr.GetArgs()[1].(*expression.Constant).Eval(r.ctx, chunk.Row{}) tpOfPattern := expr.GetArgs()[0].GetType() if err != nil { r.err = errors.Trace(err) @@ -654,7 +657,7 @@ func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []*po return []*point{startPoint, endPoint} } lowValue := make([]byte, 0, len(pattern)) - edt, err := expr.GetArgs()[2].(*expression.Constant).EvalWithInnerCtx(chunk.Row{}) + edt, err := expr.GetArgs()[2].(*expression.Constant).Eval(r.ctx, chunk.Row{}) if err != nil { r.err = errors.Trace(err) return getFullRange() @@ -811,8 +814,9 @@ func (r *builder) union(a, b []*point, collator collate.Collator) []*point { func (r *builder) mergeSorted(a, b []*point, collator collate.Collator) []*point { ret := make([]*point, 0, len(a)+len(b)) i, j := 0, 0 + tc := r.ctx.GetSessionVars().StmtCtx.TypeCtx() for i < len(a) && j < len(b) { - less, err := rangePointLess(r.sc, a[i], b[j], collator) + less, err := rangePointLess(tc, a[i], b[j], collator) if err != nil { r.err = err return nil diff --git a/pkg/util/ranger/ranger.go b/pkg/util/ranger/ranger.go index f77e562a90cce..40f1dcf6ca5b5 100644 --- a/pkg/util/ranger/ranger.go +++ b/pkg/util/ranger/ranger.go @@ -412,7 +412,7 @@ func points2TableRanges(sctx sessionctx.Context, rangePoints []*point, tp *types // The second return value is the conditions used to build ranges and the third return value is the remained conditions. func buildColumnRange(accessConditions []expression.Expression, sctx sessionctx.Context, tp *types.FieldType, tableRange bool, colLen int, rangeMaxSize int64) (Ranges, []expression.Expression, []expression.Expression, error) { - rb := builder{sc: sctx.GetSessionVars().StmtCtx} + rb := builder{ctx: sctx} rangePoints := getFullRange() for _, cond := range accessConditions { collator := collate.GetCollator(tp.GetCollate()) @@ -486,7 +486,7 @@ func BuildColumnRange(conds []expression.Expression, sctx sessionctx.Context, tp func (d *rangeDetacher) buildRangeOnColsByCNFCond(newTp []*types.FieldType, eqAndInCount int, accessConds []expression.Expression) (Ranges, []expression.Expression, []expression.Expression, error) { - rb := builder{sc: d.sctx.GetSessionVars().StmtCtx} + rb := builder{ctx: d.sctx} var ( ranges Ranges rangeFallback bool From f8fae6ecdcfce7830909207ff8480395945c49d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Thu, 23 Nov 2023 11:40:18 +0800 Subject: [PATCH 16/36] expression: remove call of `EvalWithInnerCtx` in method `Constant.HashCode` (#48791) close pingcap/tidb#48790 --- pkg/expression/constant.go | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/pkg/expression/constant.go b/pkg/expression/constant.go index bfac227a1b6b9..33fd425559daa 100644 --- a/pkg/expression/constant.go +++ b/pkg/expression/constant.go @@ -19,7 +19,6 @@ import ( "unsafe" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/types" @@ -465,10 +464,7 @@ func (c *Constant) HashCode(sc *stmtctx.StatementContext) []byte { return c.hashcode } - _, err := c.EvalWithInnerCtx(chunk.Row{}) - if err != nil { - terror.Log(err) - } + intest.Assert(c.DeferredExpr == nil && c.ParamMarker == nil) c.hashcode = append(c.hashcode, constantFlag) c.hashcode = codec.HashCode(c.hashcode, c.Value) return c.hashcode From 204c9accf0af6fe1d4b159e5af2a7fab4c5462fb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Thu, 23 Nov 2023 12:11:42 +0800 Subject: [PATCH 17/36] planner: replace `EvalWithInnerCtx` with `Eval` in `exprToString` in planner (#48788) close pingcap/tidb#48787 --- pkg/planner/cardinality/trace.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/planner/cardinality/trace.go b/pkg/planner/cardinality/trace.go index e0fa5314f7d9d..eb552b36de4c0 100644 --- a/pkg/planner/cardinality/trace.go +++ b/pkg/planner/cardinality/trace.go @@ -37,7 +37,7 @@ import ( // ceTraceExpr appends an expression and related information into CE trace func ceTraceExpr(sctx sessionctx.Context, tableID int64, tp string, expr expression.Expression, rowCount float64) { - exprStr, err := exprToString(expr) + exprStr, err := exprToString(sctx, expr) if err != nil { logutil.BgLogger().Debug("Failed to trace CE of an expression", zap.String("category", "OptimizerTrace"), zap.Any("expression", expr)) @@ -64,7 +64,7 @@ func ceTraceExpr(sctx sessionctx.Context, tableID int64, tp string, expr express // It may be more appropriate to put this in expression package. But currently we only use it for CE trace, // // and it may not be general enough to handle all possible expressions. So we put it here for now. -func exprToString(e expression.Expression) (string, error) { +func exprToString(ctx sessionctx.Context, e expression.Expression) (string, error) { switch expr := e.(type) { case *expression.ScalarFunction: var buffer bytes.Buffer @@ -72,7 +72,7 @@ func exprToString(e expression.Expression) (string, error) { switch expr.FuncName.L { case ast.Cast: for _, arg := range expr.GetArgs() { - argStr, err := exprToString(arg) + argStr, err := exprToString(ctx, arg) if err != nil { return "", err } @@ -82,7 +82,7 @@ func exprToString(e expression.Expression) (string, error) { } default: for i, arg := range expr.GetArgs() { - argStr, err := exprToString(arg) + argStr, err := exprToString(ctx, arg) if err != nil { return "", err } @@ -99,7 +99,7 @@ func exprToString(e expression.Expression) (string, error) { case *expression.CorrelatedColumn: return "", errors.New("tracing for correlated columns not supported now") case *expression.Constant: - value, err := expr.EvalWithInnerCtx(chunk.Row{}) + value, err := expr.Eval(ctx, chunk.Row{}) if err != nil { return "", err } From 077a3e96cb4f5977142aa22c235980d3c2bf75ff Mon Sep 17 00:00:00 2001 From: Hangjie Mo Date: Thu, 23 Nov 2023 12:11:49 +0800 Subject: [PATCH 18/36] tests: Update mysql-tester, move tests related with `RowsAffected` and `LastMessage` (#48789) ref pingcap/tidb#45961 --- pkg/executor/insert_test.go | 6 - pkg/executor/internal/BUILD.bazel | 9 - pkg/executor/internal/testkit.go | 31 - pkg/executor/stale_txn_test.go | 15 - pkg/executor/test/executor/BUILD.bazel | 3 +- pkg/executor/test/executor/executor_test.go | 228 --- pkg/executor/test/partitiontest/BUILD.bazel | 18 - pkg/executor/test/partitiontest/main_test.go | 15 - .../test/partitiontest/partition_test.go | 449 ----- pkg/executor/test/writetest/BUILD.bazel | 4 +- pkg/executor/test/writetest/write_test.go | 1640 ----------------- pkg/executor/update_test.go | 26 - pkg/planner/core/binary_plan_test.go | 17 - pkg/planner/core/plan_cache_test.go | 255 --- pkg/planner/core/plan_test.go | 47 - tests/integrationtest/r/executor/cte.result | 1 + .../integrationtest/r/executor/delete.result | 93 + .../r/executor/executor.result | 144 ++ .../integrationtest/r/executor/insert.result | 909 +++++++++ .../r/executor/partition/write.result | 814 ++++++++ .../r/executor/stale_txn.result | 6 + .../integrationtest/r/executor/update.result | 507 +++++ tests/integrationtest/r/executor/write.result | 172 ++ .../r/planner/core/binary_plan.result | 18 + .../integrationtest/r/planner/core/cbo.result | 12 +- .../r/planner/core/plan.result | 32 + .../r/planner/core/plan_cache.result | 347 ++++ tests/integrationtest/run-tests.sh | 2 +- tests/integrationtest/t/executor/cte.test | 3 + tests/integrationtest/t/executor/delete.test | 85 + .../integrationtest/t/executor/executor.test | 177 ++ tests/integrationtest/t/executor/insert.test | 648 +++++++ .../t/executor/partition/write.test | 604 ++++++ .../integrationtest/t/executor/stale_txn.test | 7 + tests/integrationtest/t/executor/update.test | 430 +++++ tests/integrationtest/t/executor/write.test | 134 ++ .../t/planner/core/binary_plan.test | 10 + tests/integrationtest/t/planner/core/cbo.test | 2 +- .../integrationtest/t/planner/core/plan.test | 36 + .../t/planner/core/plan_cache.test | 224 +++ 40 files changed, 5411 insertions(+), 2769 deletions(-) delete mode 100644 pkg/executor/internal/testkit.go delete mode 100644 pkg/executor/test/partitiontest/BUILD.bazel delete mode 100644 pkg/executor/test/partitiontest/main_test.go delete mode 100644 pkg/executor/test/partitiontest/partition_test.go create mode 100644 tests/integrationtest/r/executor/partition/write.result create mode 100644 tests/integrationtest/r/planner/core/binary_plan.result create mode 100644 tests/integrationtest/t/executor/partition/write.test create mode 100644 tests/integrationtest/t/planner/core/binary_plan.test diff --git a/pkg/executor/insert_test.go b/pkg/executor/insert_test.go index aef0018a08660..6b0733ed5426f 100644 --- a/pkg/executor/insert_test.go +++ b/pkg/executor/insert_test.go @@ -30,12 +30,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestInsertOnDuplicateKey(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - testInsertOnDuplicateKey(t, tk) -} - func TestInsertOnDuplicateKeyWithBinlog(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/pkg/executor/internal/BUILD.bazel b/pkg/executor/internal/BUILD.bazel index 240510f070017..e69de29bb2d1d 100644 --- a/pkg/executor/internal/BUILD.bazel +++ b/pkg/executor/internal/BUILD.bazel @@ -1,9 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "internal", - srcs = ["testkit.go"], - importpath = "github.com/pingcap/tidb/pkg/executor/internal", - visibility = ["//pkg/executor:__subpackages__"], - deps = ["//pkg/testkit"], -) diff --git a/pkg/executor/internal/testkit.go b/pkg/executor/internal/testkit.go deleted file mode 100644 index 96c1ecdbfa8bc..0000000000000 --- a/pkg/executor/internal/testkit.go +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2023 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 internal - -import ( - "fmt" - - "github.com/pingcap/tidb/pkg/testkit" -) - -// FillData fill data into table -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/pkg/executor/stale_txn_test.go b/pkg/executor/stale_txn_test.go index 4b30125453be9..8a880e31febb6 100644 --- a/pkg/executor/stale_txn_test.go +++ b/pkg/executor/stale_txn_test.go @@ -1315,21 +1315,6 @@ func TestPlanCacheWithStaleReadByBinaryProto(t *testing.T) { tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) } -func TestIssue33728(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)") - err := tk.ExecToErr("select * from t1 as of timestamp NULL") - require.Error(t, err) - require.Equal(t, "[planner:8135]invalid as of timestamp: as of timestamp cannot be NULL", err.Error()) - - err = tk.ExecToErr("start transaction read only as of timestamp NULL") - require.Error(t, err) - require.Equal(t, "[planner:8135]invalid as of timestamp: as of timestamp cannot be NULL", err.Error()) -} - func TestStalePrepare(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/pkg/executor/test/executor/BUILD.bazel b/pkg/executor/test/executor/BUILD.bazel index 8046b5d761d57..ba040bee380cf 100644 --- a/pkg/executor/test/executor/BUILD.bazel +++ b/pkg/executor/test/executor/BUILD.bazel @@ -8,7 +8,7 @@ go_test( "main_test.go", ], flaky = True, - shard_count = 50, + shard_count = 46, deps = [ "//pkg/config", "//pkg/ddl", @@ -32,7 +32,6 @@ go_test( "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", "//pkg/sessiontxn", - "//pkg/store/driver/error", "//pkg/store/mockstore", "//pkg/table/tables", "//pkg/tablecodec", diff --git a/pkg/executor/test/executor/executor_test.go b/pkg/executor/test/executor/executor_test.go index 54a3c0a1b79e7..15eec21cdcbf8 100644 --- a/pkg/executor/test/executor/executor_test.go +++ b/pkg/executor/test/executor/executor_test.go @@ -52,7 +52,6 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" - error2 "github.com/pingcap/tidb/pkg/store/driver/error" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" @@ -633,117 +632,6 @@ func TestTiDBLastQueryInfo(t *testing.T) { tk.MustExec("rollback") } -func TestSelectForUpdate(t *testing.T) { - store := testkit.CreateMockStore(t) - - setTxnTk := testkit.NewTestKit(t, store) - setTxnTk.MustExec("set global tidb_txn_mode=''") - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk1 := testkit.NewTestKit(t, store) - tk1.MustExec("use test") - tk2 := testkit.NewTestKit(t, store) - tk2.MustExec("use test") - - tk.MustExec("drop table if exists t, t1") - - txn, err := tk.Session().Txn(true) - require.True(t, kv.ErrInvalidTxn.Equal(err)) - require.False(t, txn.Valid()) - tk.MustExec("create table t (c1 int, c2 int, c3 int)") - tk.MustExec("insert t values (11, 2, 3)") - tk.MustExec("insert t values (12, 2, 3)") - tk.MustExec("insert t values (13, 2, 3)") - - tk.MustExec("create table t1 (c1 int)") - tk.MustExec("insert t1 values (11)") - - // conflict - tk1.MustExec("begin") - tk1.MustQuery("select * from t where c1=11 for update") - - tk2.MustExec("begin") - tk2.MustExec("update t set c2=211 where c1=11") - tk2.MustExec("commit") - - err = tk1.ExecToErr("commit") - require.Error(t, err) - - // no conflict for subquery. - tk1.MustExec("begin") - tk1.MustQuery("select * from t where exists(select null from t1 where t1.c1=t.c1) for update") - - tk2.MustExec("begin") - tk2.MustExec("update t set c2=211 where c1=12") - tk2.MustExec("commit") - - tk1.MustExec("commit") - - // not conflict - tk1.MustExec("begin") - tk1.MustQuery("select * from t where c1=11 for update") - - tk2.MustExec("begin") - tk2.MustExec("update t set c2=22 where c1=12") - tk2.MustExec("commit") - - tk1.MustExec("commit") - - // not conflict, auto commit - tk1.MustExec("set @@autocommit=1;") - tk1.MustQuery("select * from t where c1=11 for update") - - tk2.MustExec("begin") - tk2.MustExec("update t set c2=211 where c1=11") - tk2.MustExec("commit") - - tk1.MustExec("commit") - - // conflict - tk1.MustExec("begin") - tk1.MustQuery("select * from (select * from t for update) t join t1 for update") - - tk2.MustExec("begin") - tk2.MustExec("update t1 set c1 = 13") - tk2.MustExec("commit") - - err = tk1.ExecToErr("commit") - require.Error(t, err) -} - -func TestSelectForUpdateOf(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk1 := testkit.NewTestKit(t, store) - tk1.MustExec("use test") - - tk.MustExec("drop table if exists t, t1") - tk.MustExec("create table t (i int)") - tk.MustExec("create table t1 (i int)") - tk.MustExec("insert t values (1)") - tk.MustExec("insert t1 values (1)") - - tk.MustExec("begin pessimistic") - tk.MustQuery("select * from t, t1 where t.i = t1.i for update of t").Check(testkit.Rows("1 1")) - - tk1.MustExec("begin pessimistic") - - // no lock for t - tk1.MustQuery("select * from t1 for update").Check(testkit.Rows("1")) - - // meet lock for t1 - err := tk1.ExecToErr("select * from t for update nowait") - require.True(t, terror.ErrorEqual(err, error2.ErrLockAcquireFailAndNoWaitSet), fmt.Sprintf("err: %v", err)) - - // t1 rolled back, tk1 acquire the lock - tk.MustExec("rollback") - tk1.MustQuery("select * from t for update nowait").Check(testkit.Rows("1")) - - tk1.MustExec("rollback") -} - func TestPartitionHashCode(t *testing.T) { store := testkit.CreateMockStore(t) @@ -763,37 +651,6 @@ func TestPartitionHashCode(t *testing.T) { wg.Wait() } -func TestIndexLookupRuntimeStats(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t1 (a int, b int, index(a))") - tk.MustExec("insert into t1 values (1,2),(2,3),(3,4)") - rows := tk.MustQuery("explain analyze select * from t1 use index(a) where a > 1").Rows() - require.Len(t, rows, 3) - explain := fmt.Sprintf("%v", rows[0]) - require.Regexp(t, ".*time:.*loops:.*index_task:.*table_task: {total_time.*num.*concurrency.*}.*", explain) - indexExplain := fmt.Sprintf("%v", rows[1]) - tableExplain := fmt.Sprintf("%v", rows[2]) - require.Regexp(t, ".*time:.*loops:.*cop_task:.*", indexExplain) - require.Regexp(t, ".*time:.*loops:.*cop_task:.*", tableExplain) -} - -func TestHashAggRuntimeStats(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t1 (a int, b int)") - tk.MustExec("insert into t1 values (1,2),(2,3),(3,4)") - rows := tk.MustQuery("explain analyze SELECT /*+ HASH_AGG() */ count(*) FROM t1 WHERE a < 10;").Rows() - require.Len(t, rows, 5) - explain := fmt.Sprintf("%v", rows[0]) - pattern := ".*time:.*loops:.*partial_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*final_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*" - require.Regexp(t, pattern, explain) -} - func TestPrevStmtDesensitization(t *testing.T) { store := testkit.CreateMockStore(t) @@ -1681,24 +1538,6 @@ func TestAdminShowDDLJobs(t *testing.T) { require.Equal(t, t2.In(time.UTC), tt.In(time.UTC)) } -func TestAdminShowDDLJobsRowCount(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // Test for issue: https://github.com/pingcap/tidb/issues/25968 - tk.MustExec("use test") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (id bigint key,b int);") - tk.MustExec("split table t by (10),(20),(30);") - tk.MustExec("insert into t values (0,0),(10,10),(20,20),(30,30);") - tk.MustExec("alter table t add index idx1(b);") - require.Equal(t, "4", tk.MustQuery("admin show ddl jobs 1").Rows()[0][7]) - - tk.MustExec("insert into t values (1,0),(2,10),(3,20),(4,30);") - tk.MustExec("alter table t add index idx2(b);") - require.Equal(t, "8", tk.MustQuery("admin show ddl jobs 1").Rows()[0][7]) -} - func TestAdminShowDDLJobsInfo(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2015,16 +1854,6 @@ func TestLowResolutionTSORead(t *testing.T) { tk.MustQuery("select * from low_resolution_tso").Check(testkit.Rows("2")) } -func TestStaleReadAtFutureTime(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk := testkit.NewTestKit(t, store) - // Setting tx_read_ts to a time in the future will fail. (One day before the 2038 problem) - tk.MustGetErrMsg("set @@tx_read_ts = '2038-01-18 03:14:07'", "cannot set read timestamp to a future time") - // TxnReadTS Is not updated if check failed. - require.Zero(t, tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) -} - func TestAdapterStatement(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2692,40 +2521,6 @@ func TestAdmin(t *testing.T) { require.Equal(t, historyJobs2, historyJobs) } -func TestForSelectScopeInUnion(t *testing.T) { - store := testkit.CreateMockStore(t) - setTxnTk := testkit.NewTestKit(t, store) - setTxnTk.MustExec("set global tidb_txn_mode=''") - // A union B for update, the "for update" option belongs to union statement, so - // it should works on both A and B. - tk1 := testkit.NewTestKit(t, store) - tk2 := testkit.NewTestKit(t, store) - tk1.MustExec("use test") - tk1.MustExec("drop table if exists t") - tk1.MustExec("create table t(a int)") - tk1.MustExec("insert into t values (1)") - - tk1.MustExec("begin") - // 'For update' would act on the second select. - tk1.MustQuery("select 1 as a union select a from t for update") - - tk2.MustExec("use test") - tk2.MustExec("update t set a = a + 1") - - // As tk1 use select 'for update', it should detect conflict and fail. - _, err := tk1.Exec("commit") - require.Error(t, err) - - tk1.MustExec("begin") - tk1.MustQuery("select 1 as a union select a from t limit 5 for update") - tk1.MustQuery("select 1 as a union select a from t order by a for update") - - tk2.MustExec("update t set a = a + 1") - - _, err = tk1.Exec("commit") - require.Error(t, err) -} - func TestMaxOneRow(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -2746,29 +2541,6 @@ func TestMaxOneRow(t *testing.T) { require.NoError(t, rs.Close()) } -func TestSummaryFailedUpdate(t *testing.T) { - store, dom := testkit.CreateMockStoreAndDomain(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int as(-a))") - tk.MustExec("insert into t(a) values(1), (3), (7)") - sm := &testkit.MockSessionManager{ - PS: make([]*util.ProcessInfo, 0), - } - tk.Session().SetSessionManager(sm) - dom.ExpensiveQueryHandle().SetSessionManager(sm) - defer tk.MustExec("SET GLOBAL tidb_mem_oom_action = DEFAULT") - tk.MustQuery("select variable_value from mysql.GLOBAL_VARIABLES where variable_name = 'tidb_mem_oom_action'").Check(testkit.Rows("LOG")) - - tk.MustExec("SET GLOBAL tidb_mem_oom_action='CANCEL'") - require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil, nil)) - tk.MustExec("set @@tidb_mem_quota_query=1") - require.True(t, exeerrors.ErrMemoryExceedForQuery.Equal(tk.ExecToErr("update t set t.a = t.a - 1 where t.a in (select a from t where a < 4)"))) - tk.MustExec("set @@tidb_mem_quota_query=1000000000") - tk.MustQuery("select stmt_type from information_schema.statements_summary where digest_text = 'update `t` set `t` . `a` = `t` . `a` - ? where `t` . `a` in ( select `a` from `t` where `a` < ? )'").Check(testkit.Rows("Update")) -} - func TestIsFastPlan(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/pkg/executor/test/partitiontest/BUILD.bazel b/pkg/executor/test/partitiontest/BUILD.bazel deleted file mode 100644 index eba32c340a02d..0000000000000 --- a/pkg/executor/test/partitiontest/BUILD.bazel +++ /dev/null @@ -1,18 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_test") - -go_test( - name = "partitiontest_test", - timeout = "short", - srcs = [ - "main_test.go", - "partition_test.go", - ], - flaky = True, - race = "on", - shard_count = 4, - deps = [ - "//pkg/testkit", - "@com_github_pingcap_failpoint//:failpoint", - "@com_github_stretchr_testify//require", - ], -) diff --git a/pkg/executor/test/partitiontest/main_test.go b/pkg/executor/test/partitiontest/main_test.go deleted file mode 100644 index 139a1418e142b..0000000000000 --- a/pkg/executor/test/partitiontest/main_test.go +++ /dev/null @@ -1,15 +0,0 @@ -// Copyright 2023 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 partitiontest diff --git a/pkg/executor/test/partitiontest/partition_test.go b/pkg/executor/test/partitiontest/partition_test.go deleted file mode 100644 index cbffd97769b9e..0000000000000 --- a/pkg/executor/test/partitiontest/partition_test.go +++ /dev/null @@ -1,449 +0,0 @@ -// Copyright 2023 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 partitiontest - -import ( - "fmt" - "testing" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/testkit" - "github.com/stretchr/testify/require" -) - -func TestPartitionedTableReplace(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - testSQL := `drop table if exists replace_test; - create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) - partition by range (id) ( - PARTITION p0 VALUES LESS THAN (3), - PARTITION p1 VALUES LESS THAN (5), - PARTITION p2 VALUES LESS THAN (7), - PARTITION p3 VALUES LESS THAN (9));` - tk.MustExec(testSQL) - testSQL = `replace replace_test (c1) values (1),(2),(NULL);` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") - - errReplaceSQL := `replace replace_test (c1) values ();` - tk.MustExec("begin") - err := tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (c1, c2) values (1,2),(1);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (xxx) values (3);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test_xxx (c1) values ();` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - replaceSetSQL := `replace replace_test set c1 = 3;` - tk.MustExec(replaceSetSQL) - require.Empty(t, tk.Session().LastMessage()) - - errReplaceSetSQL := `replace replace_test set c1 = 4, c1 = 5;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSetSQL = `replace replace_test set xxx = 6;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - tk.MustExec(`drop table if exists replace_test_1`) - tk.MustExec(`create table replace_test_1 (id int, c1 int) partition by range (id) ( - PARTITION p0 VALUES LESS THAN (4), - PARTITION p1 VALUES LESS THAN (6), - PARTITION p2 VALUES LESS THAN (8), - PARTITION p3 VALUES LESS THAN (10), - PARTITION p4 VALUES LESS THAN (100))`) - tk.MustExec(`replace replace_test_1 select id, c1 from replace_test;`) - require.Equal(t, tk.Session().LastMessage(), "Records: 4 Duplicates: 0 Warnings: 0") - - tk.MustExec(`drop table if exists replace_test_2`) - tk.MustExec(`create table replace_test_2 (id int, c1 int) partition by range (id) ( - PARTITION p0 VALUES LESS THAN (10), - PARTITION p1 VALUES LESS THAN (50), - PARTITION p2 VALUES LESS THAN (100), - PARTITION p3 VALUES LESS THAN (300))`) - tk.MustExec(`replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test;`) - require.Equal(t, tk.Session().LastMessage(), "Records: 8 Duplicates: 0 Warnings: 0") - - errReplaceSelectSQL := `replace replace_test_1 select c1 from replace_test;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - tk.MustExec(`drop table if exists replace_test_3`) - replaceUniqueIndexSQL := `create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by range (c2) ( - PARTITION p0 VALUES LESS THAN (4), - PARTITION p1 VALUES LESS THAN (7), - PARTITION p2 VALUES LESS THAN (11))` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=8;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=8;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - replaceUniqueIndexSQL = `replace into replace_test_3 set c1=8, c2=8;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replaceUniqueIndexSQL = `create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by range (c1) ( - PARTITION p0 VALUES LESS THAN (4), - PARTITION p1 VALUES LESS THAN (7), - PARTITION p2 VALUES LESS THAN (11));` - tk.MustExec(`drop table if exists replace_test_4`) - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - - replacePrimaryKeySQL := `create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by range (c2) ( - PARTITION p0 VALUES LESS THAN (4), - PARTITION p1 VALUES LESS THAN (7), - PARTITION p2 VALUES LESS THAN (11));` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - - issue989SQL := `CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by range (b) ( - PARTITION p1 VALUES LESS THAN (100), - PARTITION p2 VALUES LESS THAN (200))` - tk.MustExec(issue989SQL) - issue989SQL = `insert into tIssue989 (a, b) values (1, 2);` - tk.MustExec(issue989SQL) - issue989SQL = `replace into tIssue989(a, b) values (111, 2);` - tk.MustExec(issue989SQL) - r := tk.MustQuery("select * from tIssue989;") - r.Check(testkit.Rows("111 2")) -} - -func TestHashPartitionedTableReplace(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set @@session.tidb_enable_table_partition = '1';") - tk.MustExec("drop table if exists replace_test;") - testSQL := `create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) - partition by hash(id) partitions 4;` - tk.MustExec(testSQL) - - testSQL = `replace replace_test (c1) values (1),(2),(NULL);` - tk.MustExec(testSQL) - - errReplaceSQL := `replace replace_test (c1) values ();` - tk.MustExec("begin") - err := tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (c1, c2) values (1,2),(1);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (xxx) values (3);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test_xxx (c1) values ();` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSetSQL := `replace replace_test set c1 = 4, c1 = 5;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSetSQL = `replace replace_test set xxx = 6;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - tk.MustExec(`replace replace_test set c1 = 3;`) - tk.MustExec(`replace replace_test set c1 = 4;`) - tk.MustExec(`replace replace_test set c1 = 5;`) - tk.MustExec(`replace replace_test set c1 = 6;`) - tk.MustExec(`replace replace_test set c1 = 7;`) - - tk.MustExec(`drop table if exists replace_test_1`) - tk.MustExec(`create table replace_test_1 (id int, c1 int) partition by hash(id) partitions 5;`) - tk.MustExec(`replace replace_test_1 select id, c1 from replace_test;`) - - tk.MustExec(`drop table if exists replace_test_2`) - tk.MustExec(`create table replace_test_2 (id int, c1 int) partition by hash(id) partitions 6;`) - - tk.MustExec(`replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test;`) - - errReplaceSelectSQL := `replace replace_test_1 select c1 from replace_test;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - tk.MustExec(`drop table if exists replace_test_3`) - replaceUniqueIndexSQL := `create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by hash(c2) partitions 7;` - tk.MustExec(replaceUniqueIndexSQL) - - tk.MustExec(`replace into replace_test_3 set c2=8;`) - tk.MustExec(`replace into replace_test_3 set c2=8;`) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - tk.MustExec(`replace into replace_test_3 set c1=8, c2=8;`) - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - - tk.MustExec(`replace into replace_test_3 set c2=NULL;`) - tk.MustExec(`replace into replace_test_3 set c2=NULL;`) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - - for i := 0; i < 100; i++ { - sql := fmt.Sprintf("replace into replace_test_3 set c2=%d;", i) - tk.MustExec(sql) - } - result := tk.MustQuery("select count(*) from replace_test_3") - result.Check(testkit.Rows("102")) - - replaceUniqueIndexSQL = `create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by hash(c1) partitions 8;` - tk.MustExec(`drop table if exists replace_test_4`) - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - - replacePrimaryKeySQL := `create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by hash (c2) partitions 9;` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - - issue989SQL := `CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by hash (b) partitions 10;` - tk.MustExec(issue989SQL) - issue989SQL = `insert into tIssue989 (a, b) values (1, 2);` - tk.MustExec(issue989SQL) - issue989SQL = `replace into tIssue989(a, b) values (111, 2);` - tk.MustExec(issue989SQL) - r := tk.MustQuery("select * from tIssue989;") - r.Check(testkit.Rows("111 2")) -} - -func TestPartitionedTableUpdate(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec(`create table t (id int not null default 1, name varchar(255)) - PARTITION BY RANGE ( id ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21))`) - - tk.MustExec(`insert INTO t VALUES (1, "hello");`) - tk.CheckExecResult(1, 0) - tk.MustExec(`insert INTO t VALUES (7, "hello");`) - tk.CheckExecResult(1, 0) - - // update non partition column - tk.MustExec(`UPDATE t SET name = "abc" where id > 0;`) - tk.CheckExecResult(2, 0) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - r := tk.MustQuery(`SELECT * from t order by id limit 2;`) - r.Check(testkit.Rows("1 abc", "7 abc")) - - // update partition column - tk.MustExec(`update t set id = id + 1`) - tk.CheckExecResult(2, 0) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - r = tk.MustQuery(`SELECT * from t order by id limit 2;`) - r.Check(testkit.Rows("2 abc", "8 abc")) - - // update partition column, old and new record locates on different partitions - tk.MustExec(`update t set id = 20 where id = 8`) - tk.CheckExecResult(1, 0) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery(`SELECT * from t order by id limit 2;`) - r.Check(testkit.Rows("2 abc", "20 abc")) - - // table option is auto-increment - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t (id int not null auto_increment, name varchar(255), primary key(id)) - PARTITION BY RANGE ( id ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21))`) - - tk.MustExec("insert into t(name) values ('aa')") - tk.MustExec("update t set id = 8 where name = 'aa'") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustExec("insert into t(name) values ('bb')") - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows("8 aa", "9 bb")) - - err := tk.ExecToErr("update t set id = null where name = 'aa'") - require.EqualError(t, err, "[table:1048]Column 'id' cannot be null") - - // Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t (id int, name int unique) - PARTITION BY RANGE ( name ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21))`) - tk.MustExec("insert t values (1, 1), (2, 2);") - err = tk.ExecToErr("update t set name = 1 where id = 2") - require.Error(t, err) - tk.MustQuery("select * from t").Check(testkit.Rows("1 1", "2 2")) - - // test update ignore for pimary key - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t(a bigint, primary key (a)) - PARTITION BY RANGE (a) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11))`) - tk.MustExec("insert into t values (5)") - tk.MustExec("insert into t values (7)") - err = tk.ExecToErr("update ignore t set a = 5 where a = 7;") - require.NoError(t, err) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1062 Duplicate entry '5' for key 't.PRIMARY'")) - tk.MustQuery("select * from t order by a").Check(testkit.Rows("5", "7")) - - // test update ignore for truncate as warning - err = tk.ExecToErr("update ignore t set a = 1 where a = (select '2a')") - require.NoError(t, err) - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'")) - - // test update ignore for unique key - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t(a bigint, unique key I_uniq (a)) - PARTITION BY RANGE (a) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11))`) - tk.MustExec("insert into t values (5)") - tk.MustExec("insert into t values (7)") - err = tk.ExecToErr("update ignore t set a = 5 where a = 7;") - require.NoError(t, err) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1062 Duplicate entry '5' for key 't.I_uniq'")) - tk.MustQuery("select * from t order by a").Check(testkit.Rows("5", "7")) -} - -func TestPartitionedTableDelete(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - createTable := `CREATE TABLE test.t (id int not null default 1, name varchar(255), index(id)) - PARTITION BY RANGE ( id ) ( - PARTITION p0 VALUES LESS THAN (6), - PARTITION p1 VALUES LESS THAN (11), - PARTITION p2 VALUES LESS THAN (16), - PARTITION p3 VALUES LESS THAN (21))` - - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec(createTable) - for i := 1; i < 21; i++ { - tk.MustExec(fmt.Sprintf(`insert into t values (%d, "hello")`, i)) - } - - tk.MustExec(`delete from t where id = 2 limit 1;`) - tk.CheckExecResult(1, 0) - - // Test delete with false condition - tk.MustExec(`delete from t where 0;`) - tk.CheckExecResult(0, 0) - - tk.MustExec("insert into t values (2, 'abc')") - tk.MustExec(`delete from t where t.id = 2 limit 1`) - tk.CheckExecResult(1, 0) - - // Test delete ignore - tk.MustExec("insert into t values (2, 'abc')") - err := tk.ExecToErr("delete from t where id = (select '2a')") - require.Error(t, err) - err = tk.ExecToErr("delete ignore from t where id = (select '2a')") - require.NoError(t, err) - tk.CheckExecResult(1, 0) - r := tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'")) - - // Test delete without using index, involve multiple partitions. - tk.MustExec("delete from t ignore index(id) where id >= 13 and id <= 17") - tk.CheckExecResult(5, 0) - - tk.MustExec("admin check table t") - tk.MustExec(`delete from t;`) - tk.CheckExecResult(14, 0) - - // Fix that partitioned table should not use PointGetPlan. - tk.MustExec(`create table t1 (c1 bigint, c2 bigint, c3 bigint, primary key(c1)) partition by range (c1) (partition p0 values less than (3440))`) - tk.MustExec("insert into t1 values (379, 379, 379)") - tk.MustExec("delete from t1 where c1 = 379") - tk.CheckExecResult(1, 0) - tk.MustExec(`drop table t1;`) -} diff --git a/pkg/executor/test/writetest/BUILD.bazel b/pkg/executor/test/writetest/BUILD.bazel index 1080b392de576..59e3023cb91fe 100644 --- a/pkg/executor/test/writetest/BUILD.bazel +++ b/pkg/executor/test/writetest/BUILD.bazel @@ -8,17 +8,15 @@ go_test( "write_test.go", ], flaky = True, - shard_count = 27, + shard_count = 10, deps = [ "//br/pkg/lightning/mydump", "//pkg/config", "//pkg/executor", - "//pkg/executor/internal", "//pkg/kv", "//pkg/meta/autoid", "//pkg/parser/model", "//pkg/parser/mysql", - "//pkg/planner/core", "//pkg/session", "//pkg/sessionctx", "//pkg/sessiontxn", diff --git a/pkg/executor/test/writetest/write_test.go b/pkg/executor/test/writetest/write_test.go index c8bb54ef2cabb..d279259fc0636 100644 --- a/pkg/executor/test/writetest/write_test.go +++ b/pkg/executor/test/writetest/write_test.go @@ -24,11 +24,9 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/executor" - "github.com/pingcap/tidb/pkg/executor/internal" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -41,422 +39,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestInsert(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - testSQL := `drop table if exists insert_test;create table insert_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1);` - tk.MustExec(testSQL) - testSQL = `insert insert_test (c1) values (1),(2),(NULL);` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") - - errInsertSelectSQL := `insert insert_test (c1) values ();` - tk.MustExec("begin") - err := tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errInsertSelectSQL = `insert insert_test (c1, c2) values (1,2),(1);` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errInsertSelectSQL = `insert insert_test (xxx) values (3);` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errInsertSelectSQL = `insert insert_test_xxx (c1) values ();` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - insertSetSQL := `insert insert_test set c1 = 3;` - tk.MustExec(insertSetSQL) - require.Empty(t, tk.Session().LastMessage()) - - errInsertSelectSQL = `insert insert_test set c1 = 4, c1 = 5;` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errInsertSelectSQL = `insert insert_test set xxx = 6;` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - insertSelectSQL := `create table insert_test_1 (id int, c1 int);` - tk.MustExec(insertSelectSQL) - insertSelectSQL = `insert insert_test_1 select id, c1 from insert_test;` - tk.MustExec(insertSelectSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 4 Duplicates: 0 Warnings: 0") - - insertSelectSQL = `create table insert_test_2 (id int, c1 int);` - tk.MustExec(insertSelectSQL) - insertSelectSQL = `insert insert_test_1 select id, c1 from insert_test union select id * 10, c1 * 10 from insert_test;` - tk.MustExec(insertSelectSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 8 Duplicates: 0 Warnings: 0") - - errInsertSelectSQL = `insert insert_test_1 select c1 from insert_test;` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errInsertSelectSQL = `insert insert_test_1 values(default, default, default, default, default)` - tk.MustExec("begin") - err = tk.ExecToErr(errInsertSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - // Updating column is PK handle. - // Make sure the record is "1, 1, nil, 1". - r := tk.MustQuery("select * from insert_test where id = 1;") - rowStr := fmt.Sprintf("%v %v %v %v", "1", "1", nil, "1") - r.Check(testkit.Rows(rowStr)) - insertSQL := `insert into insert_test (id, c3) values (1, 2) on duplicate key update id=values(id), c2=10;` - tk.MustExec(insertSQL) - require.Empty(t, tk.Session().LastMessage()) - r = tk.MustQuery("select * from insert_test where id = 1;") - rowStr = fmt.Sprintf("%v %v %v %v", "1", "1", "10", "1") - r.Check(testkit.Rows(rowStr)) - - insertSQL = `insert into insert_test (id, c2) values (1, 1) on duplicate key update insert_test.c2=10;` - tk.MustExec(insertSQL) - require.Empty(t, tk.Session().LastMessage()) - - err = tk.ExecToErr(`insert into insert_test (id, c2) values(1, 1) on duplicate key update t.c2 = 10`) - require.Error(t, err) - - // for on duplicate key - insertSQL = `INSERT INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3;` - tk.MustExec(insertSQL) - require.Empty(t, tk.Session().LastMessage()) - r = tk.MustQuery("select * from insert_test where id = 1;") - rowStr = fmt.Sprintf("%v %v %v %v", "1", "1", "10", "6") - r.Check(testkit.Rows(rowStr)) - - // for on duplicate key with ignore - insertSQL = `INSERT IGNORE INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3;` - tk.MustExec(insertSQL) - require.Empty(t, tk.Session().LastMessage()) - r = tk.MustQuery("select * from insert_test where id = 1;") - rowStr = fmt.Sprintf("%v %v %v %v", "1", "1", "10", "11") - r.Check(testkit.Rows(rowStr)) - - tk.MustExec("create table insert_err (id int, c1 varchar(8))") - err = tk.ExecToErr("insert insert_err values (1, 'abcdabcdabcd')") - require.True(t, types.ErrDataTooLong.Equal(err)) - err = tk.ExecToErr("insert insert_err values (1, '你好,世界')") - require.NoError(t, err) - - tk.MustExec("create table TEST1 (ID INT NOT NULL, VALUE INT DEFAULT NULL, PRIMARY KEY (ID))") - err = tk.ExecToErr("INSERT INTO TEST1(id,value) VALUE(3,3) on DUPLICATE KEY UPDATE VALUE=4") - require.NoError(t, err) - require.Empty(t, tk.Session().LastMessage()) - - tk.MustExec("create table t (id int)") - tk.MustExec("insert into t values(1)") - tk.MustExec("update t t1 set id = (select count(*) + 1 from t t2 where t1.id = t2.id)") - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows("2")) - - // issue 3235 - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c decimal(5, 5))") - err = tk.ExecToErr("insert into t value(0)") - require.NoError(t, err) - err = tk.ExecToErr("insert into t value(1)") - require.True(t, types.ErrWarnDataOutOfRange.Equal(err)) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c binary(255))") - err = tk.ExecToErr("insert into t value(1)") - require.NoError(t, err) - r = tk.MustQuery("select length(c) from t;") - r.Check(testkit.Rows("255")) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c varbinary(255))") - err = tk.ExecToErr("insert into t value(1)") - require.NoError(t, err) - r = tk.MustQuery("select length(c) from t;") - r.Check(testkit.Rows("1")) - - // issue 3509 - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c int)") - tk.MustExec("set @origin_time_zone = @@time_zone") - tk.MustExec("set @@time_zone = '+08:00'") - err = tk.ExecToErr("insert into t value(Unix_timestamp('2002-10-27 01:00'))") - require.NoError(t, err) - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows("1035651600")) - tk.MustExec("set @@time_zone = @origin_time_zone") - - // issue 3832 - tk.MustExec("create table t1 (b char(0));") - err = tk.ExecToErr(`insert into t1 values ("");`) - require.NoError(t, err) - - // issue 3895 - tk.MustExec("USE test;") - tk.MustExec("DROP TABLE IF EXISTS t;") - tk.MustExec("CREATE TABLE t(a DECIMAL(4,2));") - tk.MustExec("INSERT INTO t VALUES (1.000001);") - r = tk.MustQuery("SHOW WARNINGS;") - // TODO: MySQL8.0 reports Note 1265 Data truncated for column 'a' at row 1 - r.Check(testkit.Rows("Warning 1366 Incorrect decimal value: '1.000001' for column 'a' at row 1")) - tk.MustExec("INSERT INTO t VALUES (1.000000);") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows()) - - // issue 4653 - tk.MustExec("DROP TABLE IF EXISTS t;") - tk.MustExec("CREATE TABLE t(a datetime);") - err = tk.ExecToErr("INSERT INTO t VALUES('2017-00-00')") - require.Error(t, err) - tk.MustExec("set sql_mode = ''") - tk.MustExec("INSERT INTO t VALUES('2017-00-00')") - r = tk.MustQuery("SELECT * FROM t;") - r.Check(testkit.Rows("2017-00-00 00:00:00")) - tk.MustExec("set sql_mode = 'strict_all_tables';") - r = tk.MustQuery("SELECT * FROM t;") - r.Check(testkit.Rows("2017-00-00 00:00:00")) - - // test auto_increment with unsigned. - tk.MustExec("drop table if exists test") - tk.MustExec("CREATE TABLE test(id int(10) UNSIGNED NOT NULL AUTO_INCREMENT, p int(10) UNSIGNED NOT NULL, PRIMARY KEY(p), KEY(id))") - tk.MustExec("insert into test(p) value(1)") - tk.MustQuery("select * from test").Check(testkit.Rows("1 1")) - tk.MustQuery("select * from test use index (id) where id = 1").Check(testkit.Rows("1 1")) - tk.MustExec("insert into test values(NULL, 2)") - tk.MustQuery("select * from test use index (id) where id = 2").Check(testkit.Rows("2 2")) - tk.MustExec("insert into test values(2, 3)") - tk.MustQuery("select * from test use index (id) where id = 2").Check(testkit.Rows("2 2", "2 3")) - - // issue 6360 - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a bigint unsigned);") - tk.MustExec(" set @orig_sql_mode = @@sql_mode; set @@sql_mode = 'strict_all_tables';") - err = tk.ExecToErr("insert into t value (-1);") - require.True(t, types.ErrWarnDataOutOfRange.Equal(err)) - tk.MustExec("set @@sql_mode = '';") - tk.MustExec("insert into t value (-1);") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1")) - tk.MustExec("insert into t select -1;") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1690 constant -1 overflows bigint")) - tk.MustExec("insert into t select cast(-1 as unsigned);") - tk.MustExec("insert into t value (-1.111);") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1")) - tk.MustExec("insert into t value ('-1.111');") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1")) - tk.MustExec("update t set a = -1 limit 1;") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1690 constant -1 overflows bigint")) - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows("0", "0", "18446744073709551615", "0", "0")) - tk.MustExec("set @@sql_mode = @orig_sql_mode;") - - // issue 6424 & issue 20207 - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a time(6))") - tk.MustExec("insert into t value('20070219173709.055870'), ('20070219173709.055'), ('20070219173709.055870123')") - tk.MustQuery("select * from t").Check(testkit.Rows("17:37:09.055870", "17:37:09.055000", "17:37:09.055870")) - tk.MustExec("truncate table t") - tk.MustExec("insert into t value(20070219173709.055870), (20070219173709.055), (20070219173709.055870123)") - tk.MustQuery("select * from t").Check(testkit.Rows("17:37:09.055870", "17:37:09.055000", "17:37:09.055870")) - err = tk.ExecToErr("insert into t value(-20070219173709.055870)") - require.EqualError(t, err, "[table:1292]Incorrect time value: '-20070219173709.055870' for column 'a' at row 1") - - tk.MustExec("drop table if exists t") - tk.MustExec("set @@sql_mode=''") - tk.MustExec("create table t(a float unsigned, b double unsigned)") - tk.MustExec("insert into t value(-1.1, -1.1), (-2.1, -2.1), (0, 0), (1.1, 1.1)") - tk.MustQuery("show warnings"). - Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1", "Warning 1264 Out of range value for column 'b' at row 1", - "Warning 1264 Out of range value for column 'a' at row 2", "Warning 1264 Out of range value for column 'b' at row 2")) - tk.MustQuery("select * from t").Check(testkit.Rows("0 0", "0 0", "0 0", "1.1 1.1")) - - // issue 7061 - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int default 1, b int default 2)") - tk.MustExec("insert into t values(default, default)") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2")) - tk.MustExec("truncate table t") - tk.MustExec("insert into t values(default(b), default(a))") - tk.MustQuery("select * from t").Check(testkit.Rows("2 1")) - tk.MustExec("truncate table t") - tk.MustExec("insert into t (b) values(default)") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2")) - tk.MustExec("truncate table t") - tk.MustExec("insert into t (b) values(default(a))") - tk.MustQuery("select * from t").Check(testkit.Rows("1 1")) - - tk.MustExec("create view v as select * from t") - err = tk.ExecToErr("insert into v values(1,2)") - require.EqualError(t, err, "insert into view v is not supported now") - err = tk.ExecToErr("replace into v values(1,2)") - require.EqualError(t, err, "replace into view v is not supported now") - tk.MustExec("drop view v") - - tk.MustExec("create sequence seq") - err = tk.ExecToErr("insert into seq values()") - require.EqualError(t, err, "insert into sequence seq is not supported now") - err = tk.ExecToErr("replace into seq values()") - require.EqualError(t, err, "replace into sequence seq is not supported now") - tk.MustExec("drop sequence seq") - - // issue 22851 - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(name varchar(255), b int, c int, primary key(name(2)))") - tk.MustExec("insert into t(name, b) values(\"cha\", 3)") - err = tk.ExecToErr("insert into t(name, b) values(\"chb\", 3)") - require.EqualError(t, err, "[kv:1062]Duplicate entry 'ch' for key 't.PRIMARY'") - tk.MustExec("insert into t(name, b) values(\"测试\", 3)") - err = tk.ExecToErr("insert into t(name, b) values(\"测试\", 3)") - require.EqualError(t, err, "[kv:1062]Duplicate entry '\xe6\xb5' for key 't.PRIMARY'") -} - -func TestInsertAutoInc(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - createSQL := `drop table if exists insert_autoinc_test; create table insert_autoinc_test (id int primary key auto_increment, c1 int);` - tk.MustExec(createSQL) - - insertSQL := `insert into insert_autoinc_test(c1) values (1), (2)` - tk.MustExec(insertSQL) - tk.MustExec("begin") - r := tk.MustQuery("select * from insert_autoinc_test;") - rowStr1 := fmt.Sprintf("%v %v", "1", "1") - rowStr2 := fmt.Sprintf("%v %v", "2", "2") - r.Check(testkit.Rows(rowStr1, rowStr2)) - tk.MustExec("commit") - - tk.MustExec("begin") - insertSQL = `insert into insert_autoinc_test(id, c1) values (5,5)` - tk.MustExec(insertSQL) - insertSQL = `insert into insert_autoinc_test(c1) values (6)` - tk.MustExec(insertSQL) - tk.MustExec("commit") - tk.MustExec("begin") - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr3 := fmt.Sprintf("%v %v", "5", "5") - rowStr4 := fmt.Sprintf("%v %v", "6", "6") - r.Check(testkit.Rows(rowStr1, rowStr2, rowStr3, rowStr4)) - tk.MustExec("commit") - - tk.MustExec("begin") - insertSQL = `insert into insert_autoinc_test(id, c1) values (3,3)` - tk.MustExec(insertSQL) - tk.MustExec("commit") - tk.MustExec("begin") - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr5 := fmt.Sprintf("%v %v", "3", "3") - r.Check(testkit.Rows(rowStr1, rowStr2, rowStr5, rowStr3, rowStr4)) - tk.MustExec("commit") - - tk.MustExec("begin") - insertSQL = `insert into insert_autoinc_test(c1) values (7)` - tk.MustExec(insertSQL) - tk.MustExec("commit") - tk.MustExec("begin") - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr6 := fmt.Sprintf("%v %v", "7", "7") - r.Check(testkit.Rows(rowStr1, rowStr2, rowStr5, rowStr3, rowStr4, rowStr6)) - tk.MustExec("commit") - - // issue-962 - createSQL = `drop table if exists insert_autoinc_test; create table insert_autoinc_test (id int primary key auto_increment, c1 int);` - tk.MustExec(createSQL) - insertSQL = `insert into insert_autoinc_test(id, c1) values (0.3, 1)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr1 = fmt.Sprintf("%v %v", "1", "1") - r.Check(testkit.Rows(rowStr1)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (-0.3, 2)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr2 = fmt.Sprintf("%v %v", "2", "2") - r.Check(testkit.Rows(rowStr1, rowStr2)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (-3.3, 3)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr3 = fmt.Sprintf("%v %v", "-3", "3") - r.Check(testkit.Rows(rowStr3, rowStr1, rowStr2)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (4.3, 4)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr4 = fmt.Sprintf("%v %v", "4", "4") - r.Check(testkit.Rows(rowStr3, rowStr1, rowStr2, rowStr4)) - insertSQL = `insert into insert_autoinc_test(c1) values (5)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr5 = fmt.Sprintf("%v %v", "5", "5") - r.Check(testkit.Rows(rowStr3, rowStr1, rowStr2, rowStr4, rowStr5)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (null, 6)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr6 = fmt.Sprintf("%v %v", "6", "6") - r.Check(testkit.Rows(rowStr3, rowStr1, rowStr2, rowStr4, rowStr5, rowStr6)) - - // SQL_MODE=NO_AUTO_VALUE_ON_ZERO - createSQL = `drop table if exists insert_autoinc_test; create table insert_autoinc_test (id int primary key auto_increment, c1 int);` - tk.MustExec(createSQL) - insertSQL = `insert into insert_autoinc_test(id, c1) values (5, 1)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr1 = fmt.Sprintf("%v %v", "5", "1") - r.Check(testkit.Rows(rowStr1)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (0, 2)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr2 = fmt.Sprintf("%v %v", "6", "2") - r.Check(testkit.Rows(rowStr1, rowStr2)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (0, 3)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr3 = fmt.Sprintf("%v %v", "7", "3") - r.Check(testkit.Rows(rowStr1, rowStr2, rowStr3)) - tk.MustExec("set SQL_MODE=NO_AUTO_VALUE_ON_ZERO") - insertSQL = `insert into insert_autoinc_test(id, c1) values (0, 4)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr4 = fmt.Sprintf("%v %v", "0", "4") - r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (0, 5)` - err := tk.ExecToErr(insertSQL) - // ERROR 1062 (23000): Duplicate entry '0' for key 'PRIMARY' - require.Error(t, err) - insertSQL = `insert into insert_autoinc_test(c1) values (6)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr5 = fmt.Sprintf("%v %v", "8", "6") - r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3, rowStr5)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (null, 7)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr6 = fmt.Sprintf("%v %v", "9", "7") - r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3, rowStr5, rowStr6)) - tk.MustExec("set SQL_MODE='';") - insertSQL = `insert into insert_autoinc_test(id, c1) values (0, 8)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr7 := fmt.Sprintf("%v %v", "10", "8") - r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3, rowStr5, rowStr6, rowStr7)) - insertSQL = `insert into insert_autoinc_test(id, c1) values (null, 9)` - tk.MustExec(insertSQL) - r = tk.MustQuery("select * from insert_autoinc_test;") - rowStr8 := fmt.Sprintf("%v %v", "11", "9") - r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3, rowStr5, rowStr6, rowStr7, rowStr8)) -} - func TestInsertIgnore(t *testing.T) { store := testkit.CreateMockStore(t) var cfg kv.InjectionConfig @@ -577,656 +159,6 @@ commit;` tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1526 Table has no partition for value 3")) } -func TestIssue38950(t *testing.T) { - store := testkit.CreateMockStore(t) - var cfg kv.InjectionConfig - tk := testkit.NewTestKit(t, kv.NewInjectedStore(store, &cfg)) - tk.MustExec("use test;") - tk.MustExec("drop table if exists t; create table t (id smallint auto_increment primary key);") - tk.MustExec("alter table t add column c1 int default 1;") - tk.MustExec("insert ignore into t(id) values (194626268);") - require.Empty(t, tk.Session().LastMessage()) - - tk.MustQuery("select * from t").Check(testkit.Rows("32767 1")) - - tk.MustExec("insert ignore into t(id) values ('*') on duplicate key update c1 = 2;") - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - tk.MustQuery("select * from t").Check(testkit.Rows("32767 2")) -} - -func TestInsertOnDup(t *testing.T) { - store := testkit.CreateMockStore(t) - var cfg kv.InjectionConfig - tk := testkit.NewTestKit(t, kv.NewInjectedStore(store, &cfg)) - tk.MustExec("use test") - testSQL := `drop table if exists t; - create table t (i int unique key);` - tk.MustExec(testSQL) - testSQL = `insert into t values (1),(2);` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") - - r := tk.MustQuery("select * from t;") - rowStr1 := fmt.Sprintf("%v", "1") - rowStr2 := fmt.Sprintf("%v", "2") - r.Check(testkit.Rows(rowStr1, rowStr2)) - - tk.MustExec("insert into t values (1), (2) on duplicate key update i = values(i)") - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows(rowStr1, rowStr2)) - - tk.MustExec("insert into t values (2), (3) on duplicate key update i = 3") - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 1 Warnings: 0") - r = tk.MustQuery("select * from t;") - rowStr3 := fmt.Sprintf("%v", "3") - r.Check(testkit.Rows(rowStr1, rowStr3)) - - testSQL = `drop table if exists t; - create table t (i int primary key, j int unique key);` - tk.MustExec(testSQL) - testSQL = `insert into t values (-1, 1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - - r = tk.MustQuery("select * from t;") - rowStr1 = fmt.Sprintf("%v %v", "-1", "1") - r.Check(testkit.Rows(rowStr1)) - - tk.MustExec("insert into t values (1, 1) on duplicate key update j = values(j)") - require.Empty(t, tk.Session().LastMessage()) - r = tk.MustQuery("select * from t;") - r.Check(testkit.Rows(rowStr1)) - - testSQL = `drop table if exists test; -create table test (i int primary key, j int unique); -begin; -insert into test values (1,1); -insert into test values (2,1) on duplicate key update i = -i, j = -j; -commit;` - tk.MustExec(testSQL) - testSQL = `select * from test;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("-1 -1")) - - testSQL = `delete from test; -insert into test values (1, 1); -begin; -delete from test where i = 1; -insert into test values (2, 1) on duplicate key update i = -i, j = -j; -commit;` - tk.MustExec(testSQL) - testSQL = `select * from test;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("2 1")) - - testSQL = `delete from test; -insert into test values (1, 1); -begin; -update test set i = 2, j = 2 where i = 1; -insert into test values (1, 3) on duplicate key update i = -i, j = -j; -insert into test values (2, 4) on duplicate key update i = -i, j = -j; -commit;` - tk.MustExec(testSQL) - testSQL = `select * from test order by i;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("-2 -2", "1 3")) - - testSQL = `delete from test; -begin; -insert into test values (1, 3), (1, 3) on duplicate key update i = values(i), j = values(j); -commit;` - tk.MustExec(testSQL) - testSQL = `select * from test order by i;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1 3")) - - testSQL = `create table tmp (id int auto_increment, code int, primary key(id, code)); - create table m (id int primary key auto_increment, code int unique); - insert tmp (code) values (1); - insert tmp (code) values (1); - set tidb_init_chunk_size=1; - insert m (code) select code from tmp on duplicate key update code = values(code);` - tk.MustExec(testSQL) - testSQL = `select * from m;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1 1")) - - // The following two cases are used for guaranteeing the last_insert_id - // to be set as the value of on-duplicate-update assigned. - testSQL = `DROP TABLE IF EXISTS t1; - CREATE TABLE t1 (f1 INT AUTO_INCREMENT PRIMARY KEY, - f2 VARCHAR(5) NOT NULL UNIQUE); - INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `SELECT LAST_INSERT_ID();` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1")) - testSQL = `INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `SELECT LAST_INSERT_ID();` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1")) - - testSQL = `DROP TABLE IF EXISTS t1; - CREATE TABLE t1 (f1 INT AUTO_INCREMENT UNIQUE, - f2 VARCHAR(5) NOT NULL UNIQUE); - INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `SELECT LAST_INSERT_ID();` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1")) - testSQL = `INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `SELECT LAST_INSERT_ID();` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1")) - testSQL = `INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = 2;` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `SELECT LAST_INSERT_ID();` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1")) - - testSQL = `DROP TABLE IF EXISTS t1; - CREATE TABLE t1 (f1 INT); - INSERT t1 VALUES (1) ON DUPLICATE KEY UPDATE f1 = 1;` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - tk.MustQuery(`SELECT * FROM t1;`).Check(testkit.Rows("1")) - - testSQL = `DROP TABLE IF EXISTS t1; - CREATE TABLE t1 (f1 INT PRIMARY KEY, f2 INT NOT NULL UNIQUE); - INSERT t1 VALUES (1, 1);` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - tk.MustExec(`INSERT t1 VALUES (1, 1), (1, 1) ON DUPLICATE KEY UPDATE f1 = 2, f2 = 2;`) - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 1 Warnings: 0") - tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 1", "2 2")) - err := tk.ExecToErr(`INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;`) - require.Error(t, err) - tk.MustExec(`INSERT IGNORE t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;`) - require.Empty(t, tk.Session().LastMessage()) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'f2' cannot be null")) - tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 0", "2 2")) - - tk.MustExec(`SET sql_mode='';`) - tk.MustExec(`INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;`) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'f2' cannot be null")) - tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 0", "2 2")) -} - -func TestInsertIgnoreOnDup(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - testSQL := `drop table if exists t; - create table t (i int not null primary key, j int unique key);` - tk.MustExec(testSQL) - testSQL = `insert into t values (1, 1), (2, 2);` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") - testSQL = `insert ignore into t values(1, 1) on duplicate key update i = 2;` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `select * from t;` - r := tk.MustQuery(testSQL) - r.Check(testkit.Rows("1 1", "2 2")) - testSQL = `insert ignore into t values(1, 1) on duplicate key update j = 2;` - tk.MustExec(testSQL) - require.Empty(t, tk.Session().LastMessage()) - testSQL = `select * from t;` - r = tk.MustQuery(testSQL) - r.Check(testkit.Rows("1 1", "2 2")) - - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t2(`col_25` set('Alice','Bob','Charlie','David') NOT NULL,`col_26` date NOT NULL DEFAULT '2016-04-15', PRIMARY KEY (`col_26`) clustered, UNIQUE KEY `idx_9` (`col_25`,`col_26`),UNIQUE KEY `idx_10` (`col_25`))") - tk.MustExec("insert into t2(col_25, col_26) values('Bob', '1989-03-23'),('Alice', '2023-11-24'), ('Charlie', '2023-12-05')") - tk.MustExec("insert ignore into t2 (col_25,col_26) values ( 'Bob','1977-11-23' ) on duplicate key update col_25 = 'Alice', col_26 = '2036-12-13'") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1062 Duplicate entry 'Alice' for key 't2.idx_10'")) - tk.MustQuery("select * from t2").Check(testkit.Rows("Bob 1989-03-23", "Alice 2023-11-24", "Charlie 2023-12-05")) - - tk.MustExec("drop table if exists t4") - tk.MustExec("create table t4(id int primary key clustered, k int, v int, unique key uk1(k))") - tk.MustExec("insert into t4 values (1, 10, 100), (3, 30, 300)") - tk.MustExec("insert ignore into t4 (id, k, v) values(1, 0, 0) on duplicate key update id = 2, k = 30") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1062 Duplicate entry '30' for key 't4.uk1'")) - tk.MustQuery("select * from t4").Check(testkit.Rows("1 10 100", "3 30 300")) - - tk.MustExec("drop table if exists t5") - tk.MustExec("create table t5(k1 varchar(100), k2 varchar(100), uk1 int, v int, primary key(k1, k2) clustered, unique key ukk1(uk1), unique key ukk2(v))") - tk.MustExec("insert into t5(k1, k2, uk1, v) values('1', '1', 1, '100'), ('1', '3', 2, '200')") - tk.MustExec("update ignore t5 set k2 = '2', uk1 = 2 where k1 = '1' and k2 = '1'") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1062 Duplicate entry '2' for key 't5.ukk1'")) - tk.MustQuery("select * from t5").Check(testkit.Rows("1 1 1 100", "1 3 2 200")) - - tk.MustExec("drop table if exists t6") - tk.MustExec("create table t6 (a int, b int, c int, primary key(a, b) clustered, unique key idx_14(b), unique key idx_15(b), unique key idx_16(a, b))") - tk.MustExec("insert into t6 select 10, 10, 20") - tk.MustExec("insert ignore into t6 set a = 20, b = 10 on duplicate key update a = 100") - tk.MustQuery("select * from t6").Check(testkit.Rows("100 10 20")) - tk.MustExec("insert ignore into t6 set a = 200, b= 10 on duplicate key update c = 1000") - tk.MustQuery("select * from t6").Check(testkit.Rows("100 10 1000")) -} - -func TestReplace(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - testSQL := `drop table if exists replace_test; - create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1);` - tk.MustExec(testSQL) - testSQL = `replace replace_test (c1) values (1),(2),(NULL);` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") - - errReplaceSQL := `replace replace_test (c1) values ();` - tk.MustExec("begin") - err := tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (c1, c2) values (1,2),(1);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test (xxx) values (3);` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSQL = `replace replace_test_xxx (c1) values ();` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSQL) - require.Error(t, err) - tk.MustExec("rollback") - - replaceSetSQL := `replace replace_test set c1 = 3;` - tk.MustExec(replaceSetSQL) - require.Empty(t, tk.Session().LastMessage()) - - errReplaceSetSQL := `replace replace_test set c1 = 4, c1 = 5;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - errReplaceSetSQL = `replace replace_test set xxx = 6;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSetSQL) - require.Error(t, err) - tk.MustExec("rollback") - - replaceSelectSQL := `create table replace_test_1 (id int, c1 int);` - tk.MustExec(replaceSelectSQL) - replaceSelectSQL = `replace replace_test_1 select id, c1 from replace_test;` - tk.MustExec(replaceSelectSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 4 Duplicates: 0 Warnings: 0") - - replaceSelectSQL = `create table replace_test_2 (id int, c1 int);` - tk.MustExec(replaceSelectSQL) - replaceSelectSQL = `replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test;` - tk.MustExec(replaceSelectSQL) - require.Equal(t, tk.Session().LastMessage(), "Records: 8 Duplicates: 0 Warnings: 0") - - errReplaceSelectSQL := `replace replace_test_1 select c1 from replace_test;` - tk.MustExec("begin") - err = tk.ExecToErr(errReplaceSelectSQL) - require.Error(t, err) - tk.MustExec("rollback") - - replaceUniqueIndexSQL := `create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2));` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=1;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=1;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replaceUniqueIndexSQL = `replace into replace_test_3 set c1=1, c2=1;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replaceUniqueIndexSQL = `create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2));` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` - tk.MustExec(replaceUniqueIndexSQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - replacePrimaryKeySQL := `create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2));` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` - tk.MustExec(replacePrimaryKeySQL) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - - // For Issue989 - issue989SQL := `CREATE TABLE tIssue989 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b));` - tk.MustExec(issue989SQL) - issue989SQL = `insert into tIssue989 (a, b) values (1, 2);` - tk.MustExec(issue989SQL) - require.Empty(t, tk.Session().LastMessage()) - issue989SQL = `replace into tIssue989(a, b) values (111, 2);` - tk.MustExec(issue989SQL) - require.Empty(t, tk.Session().LastMessage()) - r := tk.MustQuery("select * from tIssue989;") - r.Check(testkit.Rows("111 2")) - - // For Issue1012 - issue1012SQL := `CREATE TABLE tIssue1012 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b));` - tk.MustExec(issue1012SQL) - issue1012SQL = `insert into tIssue1012 (a, b) values (1, 2);` - tk.MustExec(issue1012SQL) - issue1012SQL = `insert into tIssue1012 (a, b) values (2, 1);` - tk.MustExec(issue1012SQL) - issue1012SQL = `replace into tIssue1012(a, b) values (1, 1);` - tk.MustExec(issue1012SQL) - require.Equal(t, int64(3), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - r = tk.MustQuery("select * from tIssue1012;") - r.Check(testkit.Rows("1 1")) - - // Test Replace with info message - tk.MustExec(`drop table if exists t1`) - tk.MustExec(`create table t1(a int primary key, b int);`) - tk.MustExec(`insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5);`) - tk.MustExec(`replace into t1 values(1,1);`) - require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) - require.Empty(t, tk.Session().LastMessage()) - tk.MustExec(`replace into t1 values(1,1),(2,2);`) - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") - tk.MustExec(`replace into t1 values(4,14),(5,15),(6,16),(7,17),(8,18)`) - require.Equal(t, int64(7), int64(tk.Session().AffectedRows())) - require.Equal(t, tk.Session().LastMessage(), "Records: 5 Duplicates: 2 Warnings: 0") - tk.MustExec(`replace into t1 select * from (select 1, 2) as tmp;`) - require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) - require.Equal(t, tk.Session().LastMessage(), "Records: 1 Duplicates: 1 Warnings: 0") - - // Assign `DEFAULT` in `REPLACE` statement - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("create table t1 (a int primary key, b int default 20, c int default 30);") - tk.MustExec("insert into t1 value (1, 2, 3);") - tk.MustExec("replace t1 set a=1, b=default;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 30")) - tk.MustExec("replace t1 set a=2, b=default, c=default") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 30", "2 20 30")) - tk.MustExec("replace t1 set a=2, b=default(c), c=default(b);") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 30", "2 30 20")) - tk.MustExec("replace t1 set a=default(b)+default(c)") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 20 30", "2 30 20", "50 20 30")) - // With generated columns - tk.MustExec("create table t2 (pk int primary key, a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored);") - tk.MustExec("replace t2 set pk=1, b=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 1 -1 -1")) - tk.MustExec("replace t2 set pk=2, a=10, b=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 1 -1 -1", "2 10 -10 -10")) - tk.MustExec("replace t2 set pk=2, c=default, a=20;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 1 -1 -1", "2 20 -20 -20")) - tk.MustExec("replace t2 set pk=2, a=default, b=default, c=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 1 -1 -1", "2 1 -1 -1")) - tk.MustExec("replace t2 set pk=3, a=default(a), b=default, c=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 1 -1 -1", "2 1 -1 -1", "3 1 -1 -1")) - tk.MustGetErrCode("replace t2 set b=default(a);", mysql.ErrBadGeneratedColumn) - tk.MustGetErrCode("replace t2 set a=default(b), b=default(b);", mysql.ErrBadGeneratedColumn) - tk.MustGetErrCode("replace t2 set a=default(a), c=default(c);", mysql.ErrNoDefaultForField) - tk.MustGetErrCode("replace t2 set c=default(a);", mysql.ErrBadGeneratedColumn) - tk.MustExec("drop table t1, t2") -} - -// TestUpdateCastOnlyModifiedValues for issue #4514. -func TestUpdateCastOnlyModifiedValues(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table update_modified (col_1 int, col_2 enum('a', 'b'))") - tk.MustExec("set SQL_MODE=''") - tk.MustExec("insert into update_modified values (0, 3)") - r := tk.MustQuery("SELECT * FROM update_modified") - r.Check(testkit.Rows("0 ")) - tk.MustExec("set SQL_MODE=STRICT_ALL_TABLES") - tk.MustExec("update update_modified set col_1 = 1") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("SELECT * FROM update_modified") - r.Check(testkit.Rows("1 ")) - err := tk.ExecToErr("update update_modified set col_1 = 2, col_2 = 'c'") - require.Error(t, err) - r = tk.MustQuery("SELECT * FROM update_modified") - r.Check(testkit.Rows("1 ")) - tk.MustExec("update update_modified set col_1 = 3, col_2 = 'a'") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("SELECT * FROM update_modified") - r.Check(testkit.Rows("3 a")) - - // Test update a field with different column type. - tk.MustExec(`CREATE TABLE update_with_diff_type (a int, b JSON)`) - tk.MustExec(`INSERT INTO update_with_diff_type VALUES(3, '{"a": "测试"}')`) - tk.MustExec(`UPDATE update_with_diff_type SET a = '300'`) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("SELECT a FROM update_with_diff_type") - r.Check(testkit.Rows("300")) - tk.MustExec(`UPDATE update_with_diff_type SET b = '{"a": "\\u6d4b\\u8bd5"}'`) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 0") - r = tk.MustQuery("SELECT b FROM update_with_diff_type") - r.Check(testkit.Rows(`{"a": "测试"}`)) -} - -func fillMultiTableForUpdate(tk *testkit.TestKit) { - // Create and fill table items - tk.MustExec("CREATE TABLE items (id int, price TEXT);") - tk.MustExec(`insert into items values (11, "items_price_11"), (12, "items_price_12"), (13, "items_price_13");`) - tk.CheckExecResult(3, 0) - // Create and fill table month - tk.MustExec("CREATE TABLE month (mid int, mprice TEXT);") - tk.MustExec(`insert into month values (11, "month_price_11"), (22, "month_price_22"), (13, "month_price_13");`) - tk.CheckExecResult(3, 0) -} - -func TestMultipleTableUpdate(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - fillMultiTableForUpdate(tk) - - tk.MustExec(`UPDATE items, month SET items.price=month.mprice WHERE items.id=month.mid;`) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - tk.MustExec("begin") - r := tk.MustQuery("SELECT * FROM items") - r.Check(testkit.Rows("11 month_price_11", "12 items_price_12", "13 month_price_13")) - tk.MustExec("commit") - - // Single-table syntax but with multiple tables - tk.MustExec(`UPDATE items join month on items.id=month.mid SET items.price=month.mid;`) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - tk.MustExec("begin") - r = tk.MustQuery("SELECT * FROM items") - r.Check(testkit.Rows("11 11", "12 items_price_12", "13 13")) - tk.MustExec("commit") - - // JoinTable with alias table name. - tk.MustExec(`UPDATE items T0 join month T1 on T0.id=T1.mid SET T0.price=T1.mprice;`) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - tk.MustExec("begin") - r = tk.MustQuery("SELECT * FROM items") - r.Check(testkit.Rows("11 month_price_11", "12 items_price_12", "13 month_price_13")) - tk.MustExec("commit") - - // fix https://github.com/pingcap/tidb/issues/369 - testSQL := ` - DROP TABLE IF EXISTS t1, t2; - create table t1 (c int); - create table t2 (c varchar(256)); - insert into t1 values (1), (2); - insert into t2 values ("a"), ("b"); - update t1, t2 set t1.c = 10, t2.c = "abc";` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 4 Changed: 4 Warnings: 0") - - // fix https://github.com/pingcap/tidb/issues/376 - testSQL = `DROP TABLE IF EXISTS t1, t2; - create table t1 (c1 int); - create table t2 (c2 int); - insert into t1 values (1), (2); - insert into t2 values (1), (2); - update t1, t2 set t1.c1 = 10, t2.c2 = 2 where t2.c2 = 1;` - tk.MustExec(testSQL) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 3 Changed: 3 Warnings: 0") - - r = tk.MustQuery("select * from t1") - r.Check(testkit.Rows("10", "10")) - - // test https://github.com/pingcap/tidb/issues/3604 - tk.MustExec("drop table if exists t, t") - tk.MustExec("create table t (a int, b int)") - tk.MustExec("insert into t values(1, 1), (2, 2), (3, 3)") - require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") - tk.MustExec("update t m, t n set m.a = m.a + 1") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 3 Changed: 3 Warnings: 0") - tk.MustQuery("select * from t").Check(testkit.Rows("2 1", "3 2", "4 3")) - tk.MustExec("update t m, t n set n.a = n.a - 1, n.b = n.b + 1") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 3 Changed: 3 Warnings: 0") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2", "2 3", "3 4")) -} - -func TestDelete(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - internal.FillData(tk, "delete_test") - - tk.MustExec(`update delete_test set name = "abc" where id = 2;`) - tk.CheckExecResult(1, 0) - - tk.MustExec(`delete from delete_test where id = 2 limit 1;`) - tk.CheckExecResult(1, 0) - - // Test delete with false condition - tk.MustExec(`delete from delete_test where 0;`) - tk.CheckExecResult(0, 0) - - tk.MustExec("insert into delete_test values (2, 'abc')") - tk.MustExec(`delete from delete_test where delete_test.id = 2 limit 1`) - tk.CheckExecResult(1, 0) - - // Select data - tk.MustExec("begin") - rows := tk.MustQuery(`SELECT * from delete_test limit 2;`) - rows.Check(testkit.Rows("1 hello")) - tk.MustExec("commit") - - // Test delete ignore - tk.MustExec("insert into delete_test values (2, 'abc')") - err := tk.ExecToErr("delete from delete_test where id = (select '2a')") - require.Error(t, err) - err = tk.ExecToErr("delete ignore from delete_test where id = (select '2a')") - require.NoError(t, err) - tk.CheckExecResult(1, 0) - r := tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'")) - - tk.MustExec(`delete from delete_test ;`) - tk.CheckExecResult(1, 0) - - tk.MustExec("create view v as select * from delete_test") - err = tk.ExecToErr("delete from v where name = 'aaa'") - require.EqualError(t, err, core.ErrViewInvalid.GenWithStackByArgs("test", "v").Error()) - tk.MustExec("drop view v") - - tk.MustExec("create sequence seq") - err = tk.ExecToErr("delete from seq") - require.EqualError(t, err, "delete sequence seq is not supported now") - tk.MustExec("drop sequence seq") -} - -func fillDataMultiTable(tk *testkit.TestKit) { - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2, t3") - // Create and fill table t1 - tk.MustExec("create table t1 (id int, data int);") - tk.MustExec("insert into t1 values (11, 121), (12, 122), (13, 123);") - tk.CheckExecResult(3, 0) - // Create and fill table t2 - tk.MustExec("create table t2 (id int, data int);") - tk.MustExec("insert into t2 values (11, 221), (22, 222), (23, 223);") - tk.CheckExecResult(3, 0) - // Create and fill table t3 - tk.MustExec("create table t3 (id int, data int);") - tk.MustExec("insert into t3 values (11, 321), (22, 322), (23, 323);") - tk.CheckExecResult(3, 0) -} - -func TestMultiTableDelete(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - fillDataMultiTable(tk) - - tk.MustExec(`delete t1, t2 from t1 inner join t2 inner join t3 where t1.id=t2.id and t2.id=t3.id;`) - tk.CheckExecResult(2, 0) - - // Select data - r := tk.MustQuery("select * from t3") - require.Len(t, r.Rows(), 3) -} - -func TestQualifiedDelete(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - tk.MustExec("drop table if exists t2") - tk.MustExec("create table t1 (c1 int, c2 int, index (c1))") - tk.MustExec("create table t2 (c1 int, c2 int)") - tk.MustExec("insert into t1 values (1, 1), (2, 2)") - - // delete with index - tk.MustExec("delete from t1 where t1.c1 = 1") - tk.CheckExecResult(1, 0) - - // delete with no index - tk.MustExec("delete from t1 where t1.c2 = 2") - tk.CheckExecResult(1, 0) - - r := tk.MustQuery("select * from t1") - require.Len(t, r.Rows(), 0) - tk.MustExec("insert into t1 values (1, 3)") - tk.MustExec("delete from t1 as a where a.c1 = 1") - tk.CheckExecResult(1, 0) - - tk.MustExec("insert into t1 values (1, 1), (2, 2)") - tk.MustExec("insert into t2 values (2, 1), (3,1)") - tk.MustExec("delete t1, t2 from t1 join t2 where t1.c1 = t2.c2") - tk.CheckExecResult(3, 0) - - tk.MustExec("insert into t2 values (2, 1), (3,1)") - tk.MustExec("delete a, b from t1 as a join t2 as b where a.c2 = b.c1") - tk.CheckExecResult(2, 0) - - err := tk.ExecToErr("delete t1, t2 from t1 as a join t2 as b where a.c2 = b.c1") - require.Error(t, err) -} - type testCase struct { data []byte expected []string @@ -1407,60 +339,6 @@ func TestLatch(t *testing.T) { tk1.MustExec("commit") } -func TestUpdateSelect(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table msg (id varchar(8), b int, status int, primary key (id, b))") - tk.MustExec("insert msg values ('abc', 1, 1)") - tk.MustExec("create table detail (id varchar(8), start varchar(8), status int, index idx_start(start))") - tk.MustExec("insert detail values ('abc', '123', 2)") - tk.MustExec("UPDATE msg SET msg.status = (SELECT detail.status FROM detail WHERE msg.id = detail.id)") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustExec("admin check table msg") -} - -func TestUpdateDelete(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("CREATE TABLE ttt (id bigint(20) NOT NULL, host varchar(30) NOT NULL, PRIMARY KEY (id), UNIQUE KEY i_host (host));") - tk.MustExec("insert into ttt values (8,8),(9,9);") - - tk.MustExec("begin") - tk.MustExec("update ttt set id = 0, host='9' where id = 9 limit 1;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustExec("delete from ttt where id = 0 limit 1;") - tk.MustQuery("select * from ttt use index (i_host) order by host;").Check(testkit.Rows("8 8")) - tk.MustExec("update ttt set id = 0, host='8' where id = 8 limit 1;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustExec("delete from ttt where id = 0 limit 1;") - tk.MustQuery("select * from ttt use index (i_host) order by host;").Check(testkit.Rows()) - tk.MustExec("commit") - tk.MustExec("admin check table ttt;") - tk.MustExec("drop table ttt") -} - -func TestUpdateAffectRowCnt(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table a(id int auto_increment, a int default null, primary key(id))") - tk.MustExec("insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1)") - tk.MustExec("update a set id = id*10 where a = 1001") - ctx := tk.Session().(sessionctx.Context) - require.Equal(t, uint64(2), ctx.GetSessionVars().StmtCtx.AffectedRows()) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - - tk.MustExec("drop table a") - tk.MustExec("create table a ( a bigint, b bigint)") - tk.MustExec("insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1)") - tk.MustExec("update a set a = a*10 where b = 1001") - ctx = tk.Session().(sessionctx.Context) - require.Equal(t, uint64(2), ctx.GetSessionVars().StmtCtx.AffectedRows()) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") -} - func TestReplaceLog(t *testing.T) { store, domain := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -1683,524 +561,6 @@ func TestPessimisticDeleteYourWrites(t *testing.T) { session2.MustQuery("select * from x").Check(testkit.Rows("1 2")) } -// TestWriteListPartitionTable2 test for write list partition when the partition expression is complicated and contain generated column. -func TestWriteListPartitionTable2(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - tk.MustExec("drop table if exists t") - tk.MustExec(`create table t (id int, name varchar(10),b int generated always as (length(name)+1) virtual) - partition by list (id*2 + b*b + b*b - b*b*2 - abs(id)) ( - partition p0 values in (3,5,6,9,17), - partition p1 values in (1,2,10,11,19,20), - partition p2 values in (4,12,13,14,18), - partition p3 values in (7,8,15,16,null) - );`) - - // Test add unique index failed. - tk.MustExec("insert into t (id,name) values (1, 'a'),(1,'b')") - err := tk.ExecToErr("alter table t add unique index idx (id,b)") - require.EqualError(t, err, "[kv:1062]Duplicate entry '1-2' for key 't.idx'") - // Test add unique index success. - tk.MustExec("delete from t where name='b'") - tk.MustExec("alter table t add unique index idx (id,b)") - - // --------------------------Test insert--------------------------- - // Test insert 1 partition. - tk.MustExec("delete from t") - tk.MustExec("insert into t (id,name) values (1, 'a'),(2,'b'),(10,'c')") - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows("1 a", "2 b", "10 c")) - // Test insert multi-partitions. - tk.MustExec("delete from t") - tk.MustExec("insert into t (id,name) values (1, 'a'),(3,'c'),(4,'e')") - tk.MustQuery("select id,name from t partition(p0) order by id").Check(testkit.Rows("3 c")) - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows("1 a")) - tk.MustQuery("select id,name from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows()) - // Test insert on duplicate. - tk.MustExec("insert into t (id,name) values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'") - tk.MustQuery("select id,name from t partition(p0) order by id").Check(testkit.Rows("3 x", "5 g")) - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows("1 x")) - tk.MustQuery("select id,name from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows()) - // Test insert on duplicate error - err = tk.ExecToErr("insert into t (id,name) values (3, 'a'), (11,'x') on duplicate key update id=id+1") - require.EqualError(t, err, "[kv:1062]Duplicate entry '4-2' for key 't.idx'") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 x", "3 x", "4 e", "5 g")) - // Test insert ignore with duplicate - tk.MustExec("insert ignore into t (id,name) values (1, 'b'), (5,'a'),(null,'y')") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1062 Duplicate entry '1-2' for key 't.idx'", "Warning 1062 Duplicate entry '5-2' for key 't.idx'")) - tk.MustQuery("select id,name from t partition(p0) order by id").Check(testkit.Rows("3 x", "5 g")) - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows("1 x")) - tk.MustQuery("select id,name from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows(" y")) - // Test insert ignore without duplicate - tk.MustExec("insert ignore into t (id,name) values (15, 'a'),(17,'a')") - tk.MustQuery("select id,name from t partition(p0,p1,p2) order by id").Check(testkit.Rows("1 x", "3 x", "4 e", "5 g", "17 a")) - tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows(" y", "15 a")) - // Test insert meet no partition error. - err = tk.ExecToErr("insert into t (id,name) values (100, 'd')") - require.EqualError(t, err, "[table:1526]Table has no partition for value 100") - - // --------------------------Test update--------------------------- - // Test update 1 partition. - tk.MustExec("delete from t") - tk.MustExec("insert into t (id,name) values (1, 'a'),(2,'b'),(3,'c')") - tk.MustExec("update t set name='b' where id=2;") - tk.MustQuery("select id,name from t partition(p1)").Check(testkit.Rows("1 a", "2 b")) - tk.MustExec("update t set name='x' where id in (1,2)") - tk.MustQuery("select id,name from t partition(p1)").Check(testkit.Rows("1 x", "2 x")) - tk.MustExec("update t set name='y' where id < 3") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) - // Test update meet duplicate error. - tk.MustGetErrMsg("update t set id=2 where id = 1", "[kv:1062]Duplicate entry '2-2' for key 't.idx'") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) - - // Test update multi-partitions - tk.MustExec("update t set name='z' where id in (1,2,3);") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 z", "2 z", "3 z")) - tk.MustExec("update t set name='a' limit 3") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 a", "2 a", "3 a")) - tk.MustExec("update t set id=id*10 where id in (1,2)") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - // Test update meet duplicate error. - tk.MustGetErrMsg("update t set id=id+17 where id in (3,10)", "[kv:1062]Duplicate entry '20-2' for key 't.idx'") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - // Test update meet no partition error. - tk.MustGetErrMsg("update t set id=id*2 where id in (3,20)", "[table:1526]Table has no partition for value 40") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - - // --------------------------Test replace--------------------------- - // Test replace 1 partition. - tk.MustExec("delete from t") - tk.MustExec("replace into t (id,name) values (1, 'a'),(2,'b')") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 a", "2 b")) - // Test replace multi-partitions. - tk.MustExec("replace into t (id,name) values (3, 'c'),(4,'d'),(7,'f')") - tk.MustQuery("select id,name from t partition(p0) order by id").Check(testkit.Rows("3 c")) - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows("1 a", "2 b")) - tk.MustQuery("select id,name from t partition(p2) order by id").Check(testkit.Rows("4 d")) - tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows("7 f")) - // Test replace on duplicate. - tk.MustExec("replace into t (id,name) values (1, 'x'),(7,'x')") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) - // Test replace meet no partition error. - tk.MustGetErrMsg("replace into t (id,name) values (10,'x'),(50,'x')", "[table:1526]Table has no partition for value 50") - tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) - - // --------------------------Test delete--------------------------- - // Test delete 1 partition. - tk.MustExec("delete from t where id = 3") - tk.MustQuery("select id,name from t partition(p0) order by id").Check(testkit.Rows()) - tk.MustExec("delete from t where id in (1,2)") - tk.MustQuery("select id,name from t partition(p1) order by id").Check(testkit.Rows()) - // Test delete multi-partitions. - tk.MustExec("delete from t where id in (4,7,10,11)") - tk.MustQuery("select id,name from t").Check(testkit.Rows()) - tk.MustExec("insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f')") - tk.MustExec("delete from t where id < 10") - tk.MustQuery("select id,name from t").Check(testkit.Rows()) - tk.MustExec("insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f')") - tk.MustExec("delete from t limit 3") - tk.MustQuery("select id,name from t").Check(testkit.Rows()) -} - -func TestWriteListColumnsPartitionTable1(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - - tk.MustExec("drop table if exists t") - tk.MustExec(`create table t (id int, name varchar(10)) partition by list columns (id) ( - partition p0 values in (3,5,6,9,17), - partition p1 values in (1,2,10,11,19,20), - partition p2 values in (4,12,13,14,18), - partition p3 values in (7,8,15,16,null) - );`) - - // Test add unique index failed. - tk.MustExec("insert into t values (1, 'a'),(1,'b')") - tk.MustGetErrMsg("alter table t add unique index idx (id)", "[kv:1062]Duplicate entry '1' for key 't.idx'") - // Test add unique index success. - tk.MustExec("delete from t where name='b'") - tk.MustExec("alter table t add unique index idx (id)") - - // --------------------------Test insert--------------------------- - // Test insert 1 partition. - tk.MustExec("delete from t") - tk.MustExec("insert into t values (1, 'a'),(2,'b'),(10,'c')") - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows("1 a", "2 b", "10 c")) - // Test insert multi-partitions. - tk.MustExec("delete from t") - tk.MustExec("insert into t values (1, 'a'),(3,'c'),(4,'e')") - tk.MustQuery("select * from t partition(p0) order by id").Check(testkit.Rows("3 c")) - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows("1 a")) - tk.MustQuery("select * from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows()) - // Test insert on duplicate. - tk.MustExec("insert into t values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'") - tk.MustQuery("select * from t partition(p0) order by id").Check(testkit.Rows("3 x", "5 g")) - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows("1 x")) - tk.MustQuery("select * from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows()) - // Test insert on duplicate error - tk.MustGetErrMsg("insert into t values (3, 'a'), (11,'x') on duplicate key update id=id+1", "[kv:1062]Duplicate entry '4' for key 't.idx'") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "3 x", "4 e", "5 g")) - // Test insert ignore with duplicate - tk.MustExec("insert ignore into t values (1, 'b'), (5,'a'),(null,'y')") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 't.idx'", "Warning 1062 Duplicate entry '5' for key 't.idx'")) - tk.MustQuery("select * from t partition(p0) order by id").Check(testkit.Rows("3 x", "5 g")) - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows("1 x")) - tk.MustQuery("select * from t partition(p2) order by id").Check(testkit.Rows("4 e")) - tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows(" y")) - // Test insert ignore without duplicate - tk.MustExec("insert ignore into t values (15, 'a'),(17,'a')") - tk.MustQuery("select * from t partition(p0,p1,p2) order by id").Check(testkit.Rows("1 x", "3 x", "4 e", "5 g", "17 a")) - tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows(" y", "15 a")) - // Test insert meet no partition error. - tk.MustGetErrMsg("insert into t values (100, 'd')", "[table:1526]Table has no partition for value from column_list") - - // --------------------------Test update--------------------------- - // Test update 1 partition. - tk.MustExec("delete from t") - tk.MustExec("insert into t values (1, 'a'),(2,'b'),(3,'c')") - tk.MustExec("update t set name='b' where id=2;") - tk.MustQuery("select * from t partition(p1)").Check(testkit.Rows("1 a", "2 b")) - tk.MustExec("update t set name='x' where id in (1,2)") - tk.MustQuery("select * from t partition(p1)").Check(testkit.Rows("1 x", "2 x")) - tk.MustExec("update t set name='y' where id < 3") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) - // Test update meet duplicate error. - tk.MustGetErrMsg("update t set id=2 where id = 1", "[kv:1062]Duplicate entry '2' for key 't.idx'") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) - - // Test update multi-partitions - tk.MustExec("update t set name='z' where id in (1,2,3);") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 z", "2 z", "3 z")) - tk.MustExec("update t set name='a' limit 3") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 a", "2 a", "3 a")) - tk.MustExec("update t set id=id*10 where id in (1,2)") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - // Test update meet duplicate error. - tk.MustGetErrMsg("update t set id=id+17 where id in (3,10)", "[kv:1062]Duplicate entry '20' for key 't.idx'") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - // Test update meet no partition error. - tk.MustGetErrMsg("update t set id=id*2 where id in (3,20)", "[table:1526]Table has no partition for value from column_list") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) - - // --------------------------Test replace--------------------------- - // Test replace 1 partition. - tk.MustExec("delete from t") - tk.MustExec("replace into t values (1, 'a'),(2,'b')") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 a", "2 b")) - // Test replace multi-partitions. - tk.MustExec("replace into t values (3, 'c'),(4,'d'),(7,'f')") - tk.MustQuery("select * from t partition(p0) order by id").Check(testkit.Rows("3 c")) - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows("1 a", "2 b")) - tk.MustQuery("select * from t partition(p2) order by id").Check(testkit.Rows("4 d")) - tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows("7 f")) - // Test replace on duplicate. - tk.MustExec("replace into t values (1, 'x'),(7,'x')") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) - // Test replace meet no partition error. - tk.MustGetErrMsg("replace into t values (10,'x'),(100,'x')", "[table:1526]Table has no partition for value from column_list") - tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) - - // --------------------------Test delete--------------------------- - // Test delete 1 partition. - tk.MustExec("delete from t where id = 3") - tk.MustQuery("select * from t partition(p0) order by id").Check(testkit.Rows()) - tk.MustExec("delete from t where id in (1,2)") - tk.MustQuery("select * from t partition(p1) order by id").Check(testkit.Rows()) - // Test delete multi-partitions. - tk.MustExec("delete from t where id in (4,7,10,11)") - tk.MustQuery("select * from t").Check(testkit.Rows()) - tk.MustExec("insert into t values (3, 'c'),(4,'d'),(7,'f')") - tk.MustExec("delete from t where id < 10") - tk.MustQuery("select * from t").Check(testkit.Rows()) - tk.MustExec("insert into t values (3, 'c'),(4,'d'),(7,'f')") - tk.MustExec("delete from t limit 3") - tk.MustQuery("select * from t").Check(testkit.Rows()) -} - -func TestUpdate(t *testing.T) { - failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - internal.FillData(tk, "update_test") - - updateStr := `UPDATE update_test SET name = "abc" where id > 0;` - tk.MustExec(updateStr) - tk.CheckExecResult(2, 0) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - - // select data - tk.MustExec("begin") - r := tk.MustQuery(`SELECT * from update_test limit 2;`) - r.Check(testkit.Rows("1 abc", "2 abc")) - tk.MustExec("commit") - - tk.MustExec(`UPDATE update_test SET name = "foo"`) - tk.CheckExecResult(2, 0) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") - - // table option is auto-increment - tk.MustExec("begin") - tk.MustExec("drop table if exists update_test;") - tk.MustExec("commit") - tk.MustExec("begin") - tk.MustExec("create table update_test(id int not null auto_increment, name varchar(255), primary key(id))") - tk.MustExec("insert into update_test(name) values ('aa')") - tk.MustExec("update update_test set id = 8 where name = 'aa'") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustExec("insert into update_test(name) values ('bb')") - tk.MustExec("commit") - tk.MustExec("begin") - r = tk.MustQuery("select * from update_test;") - r.Check(testkit.Rows("8 aa", "9 bb")) - tk.MustExec("commit") - - tk.MustExec("begin") - tk.MustExec("drop table if exists update_test;") - tk.MustExec("commit") - tk.MustExec("begin") - tk.MustExec("create table update_test(id int not null auto_increment, name varchar(255), index(id))") - tk.MustExec("insert into update_test(name) values ('aa')") - err := tk.ExecToErr("update update_test set id = null where name = 'aa'") - require.EqualError(t, err, "[table:1048]Column 'id' cannot be null") - - tk.MustExec("drop table update_test") - tk.MustExec("create table update_test(id int)") - tk.MustExec("begin") - tk.MustExec("insert into update_test(id) values (1)") - tk.MustExec("update update_test set id = 2 where id = 1 limit 1") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("select * from update_test;") - r.Check(testkit.Rows("2")) - tk.MustExec("commit") - - // Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. - tk.MustExec("create table update_unique (id int primary key, name int unique)") - tk.MustExec("insert update_unique values (1, 1), (2, 2);") - tk.MustExec("begin") - err = tk.ExecToErr("update update_unique set name = 1 where id = 2") - require.Error(t, err) - tk.MustExec("commit") - tk.MustQuery("select * from update_unique").Check(testkit.Rows("1 1", "2 2")) - - // test update ignore for pimary key - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a bigint, primary key (a));") - tk.MustExec("insert into t values (1)") - tk.MustExec("insert into t values (2)") - err = tk.ExecToErr("update ignore t set a = 1 where a = 2;") - require.NoError(t, err) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 't.PRIMARY'")) - tk.MustQuery("select * from t").Check(testkit.Rows("1", "2")) - - // test update ignore for truncate as warning - err = tk.ExecToErr("update ignore t set a = 1 where a = (select '2a')") - require.NoError(t, err) - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1062 Duplicate entry '1' for key 't.PRIMARY'")) - - tk.MustExec("update ignore t set a = 42 where a = 2;") - tk.MustQuery("select * from t").Check(testkit.Rows("1", "42")) - - // test update ignore for unique key - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a bigint, unique key I_uniq (a));") - tk.MustExec("insert into t values (1)") - tk.MustExec("insert into t values (2)") - err = tk.ExecToErr("update ignore t set a = 1 where a = 2;") - require.NoError(t, err) - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 't.I_uniq'")) - tk.MustQuery("select * from t").Check(testkit.Rows("1", "2")) - - // test issue21965 - tk.MustExec("drop table if exists t;") - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - tk.MustExec("create table t (a int) partition by list (a) (partition p0 values in (0,1));") - tk.MustExec("insert ignore into t values (1);") - tk.MustExec("update ignore t set a=2 where a=1;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 0") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (a int key) partition by list (a) (partition p0 values in (0,1));") - tk.MustExec("insert ignore into t values (1);") - tk.MustExec("update ignore t set a=2 where a=1;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 0") - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(id integer auto_increment, t1 datetime, t2 datetime, primary key (id))") - tk.MustExec("insert into t(t1, t2) values('2000-10-01 01:01:01', '2017-01-01 10:10:10')") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2000-10-01 01:01:01 2017-01-01 10:10:10")) - tk.MustExec("update t set t1 = '2017-10-01 10:10:11', t2 = date_add(t1, INTERVAL 10 MINUTE) where id = 1") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2017-10-01 10:10:11 2000-10-01 01:11:01")) - - // for issue #5132 - tk.MustExec("CREATE TABLE `tt1` (" + - "`a` int(11) NOT NULL," + - "`b` varchar(32) DEFAULT NULL," + - "`c` varchar(32) DEFAULT NULL," + - "PRIMARY KEY (`a`)," + - "UNIQUE KEY `b_idx` (`b`)" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;") - tk.MustExec("insert into tt1 values(1, 'a', 'a');") - tk.MustExec("insert into tt1 values(2, 'd', 'b');") - r = tk.MustQuery("select * from tt1;") - r.Check(testkit.Rows("1 a a", "2 d b")) - tk.MustExec("update tt1 set a=5 where c='b';") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("select * from tt1;") - r.Check(testkit.Rows("1 a a", "5 d b")) - - // Automatic Updating for TIMESTAMP - tk.MustExec("CREATE TABLE `tsup` (" + - "`a` int," + - "`ts` TIMESTAMP DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP," + - "KEY `idx` (`ts`)" + - ");") - tk.MustExec("set @orig_sql_mode=@@sql_mode; set @@sql_mode='';") - tk.MustExec("insert into tsup values(1, '0000-00-00 00:00:00');") - tk.MustExec("update tsup set a=5;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - r1 := tk.MustQuery("select ts from tsup use index (idx);") - r2 := tk.MustQuery("select ts from tsup;") - r1.Check(r2.Rows()) - tk.MustExec("update tsup set ts='2019-01-01';") - tk.MustQuery("select ts from tsup;").Check(testkit.Rows("2019-01-01 00:00:00")) - tk.MustExec("set @@sql_mode=@orig_sql_mode;") - - // issue 5532 - tk.MustExec("create table decimals (a decimal(20, 0) not null)") - tk.MustExec("insert into decimals values (201)") - // A warning rather than data truncated error. - tk.MustExec("update decimals set a = a + 1.23;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 1") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect DECIMAL value: '202.23'")) - r = tk.MustQuery("select * from decimals") - r.Check(testkit.Rows("202")) - - tk.MustExec("drop table t") - tk.MustExec("CREATE TABLE `t` ( `c1` year DEFAULT NULL, `c2` year DEFAULT NULL, `c3` date DEFAULT NULL, `c4` datetime DEFAULT NULL, KEY `idx` (`c1`,`c2`))") - err = tk.ExecToErr("UPDATE t SET c2=16777215 WHERE c1>= -8388608 AND c1 < -9 ORDER BY c1 LIMIT 2") - require.NoError(t, err) - - tk.MustGetErrCode("update (select * from t) t set c1 = 1111111", mysql.ErrNonUpdatableTable) - - // test update ignore for bad null error - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t (i int not null default 10)`) - tk.MustExec("insert into t values (1)") - tk.MustExec("update ignore t set i = null;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1048 Column 'i' cannot be null")) - tk.MustQuery("select * from t").Check(testkit.Rows("0")) - - // issue 7237, update subquery table should be forbidden - tk.MustExec("drop table t") - tk.MustExec("create table t (k int, v int)") - err = tk.ExecToErr("update t, (select * from t) as b set b.k = t.k") - require.EqualError(t, err, "[planner:1288]The target table b of the UPDATE is not updatable") - tk.MustExec("update t, (select * from t) as b set t.k = b.k") - - // issue 8045 - tk.MustExec("drop table if exists t1") - tk.MustExec(`CREATE TABLE t1 (c1 float)`) - tk.MustExec("INSERT INTO t1 SET c1 = 1") - tk.MustExec("UPDATE t1 SET c1 = 1.2 WHERE c1=1;") - require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") - - // issue 8119 - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (c1 float(1,1));") - tk.MustExec("insert into t values (0.0);") - err = tk.ExecToErr("update t set c1 = 2.0;") - require.True(t, types.ErrWarnDataOutOfRange.Equal(err)) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a datetime not null, b datetime)") - tk.MustExec("insert into t value('1999-12-12', '1999-12-13')") - tk.MustExec("set @orig_sql_mode=@@sql_mode; set @@sql_mode='';") - tk.MustQuery("select * from t").Check(testkit.Rows("1999-12-12 00:00:00 1999-12-13 00:00:00")) - tk.MustExec("update t set a = ''") - tk.MustQuery("select * from t").Check(testkit.Rows("0000-00-00 00:00:00 1999-12-13 00:00:00")) - tk.MustExec("update t set b = ''") - tk.MustQuery("select * from t").Check(testkit.Rows("0000-00-00 00:00:00 0000-00-00 00:00:00")) - tk.MustExec("set @@sql_mode=@orig_sql_mode;") - - tk.MustExec("create view v as select * from t") - err = tk.ExecToErr("update v set a = '2000-11-11'") - require.EqualError(t, err, core.ErrViewInvalid.GenWithStackByArgs("test", "v").Error()) - tk.MustExec("drop view v") - - tk.MustExec("create sequence seq") - tk.MustGetErrCode("update seq set minvalue=1", mysql.ErrBadField) - tk.MustExec("drop sequence seq") - - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int, b int, c int, d int, e int, index idx(a))") - tk.MustExec("create table t2(a int, b int, c int)") - tk.MustExec("update t1 join t2 on t1.a=t2.a set t1.a=1 where t2.b=1 and t2.c=2") - - // Assign `DEFAULT` in `UPDATE` statement - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("create table t1 (a int default 1, b int default 2);") - tk.MustExec("insert into t1 values (10, 10), (20, 20);") - tk.MustExec("update t1 set a=default where b=10;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "20 20")) - tk.MustExec("update t1 set a=30, b=default where a=20;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "30 2")) - tk.MustExec("update t1 set a=default, b=default where a=30;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "1 2")) - tk.MustExec("insert into t1 values (40, 40)") - tk.MustExec("update t1 set a=default, b=default") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 2", "1 2", "1 2")) - tk.MustExec("update t1 set a=default(b), b=default(a)") - tk.MustQuery("select * from t1;").Check(testkit.Rows("2 1", "2 1", "2 1")) - // With generated columns - tk.MustExec("create table t2 (a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored);") - tk.MustExec("insert into t2 values (10, default, default), (20, default, default)") - tk.MustExec("update t2 set b=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("10 -10 -10", "20 -20 -20")) - tk.MustExec("update t2 set a=30, b=default where a=10;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("30 -30 -30", "20 -20 -20")) - tk.MustExec("update t2 set c=default, a=40 where c=-20;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("30 -30 -30", "40 -40 -40")) - tk.MustExec("update t2 set a=default, b=default, c=default where b=-30;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "40 -40 -40")) - tk.MustExec("update t2 set a=default(a), b=default, c=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "1 -1 -1")) - // Same as in MySQL 8.0.27, but still weird behavior: a=default(b) => NULL - tk.MustExec("update t2 set a=default(b), b=default, c=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows(" ", " ")) - tk.MustGetErrCode("update t2 set b=default(a);", mysql.ErrBadGeneratedColumn) - tk.MustExec("update t2 set a=default(a), c=default(c)") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "1 -1 -1")) - // Same as in MySQL 8.0.27, but still weird behavior: a=default(b) => NULL - tk.MustExec("update t2 set a=default(b), b=default(b)") - tk.MustQuery("select * from t2;").Check(testkit.Rows(" ", " ")) - tk.MustExec("update t2 set a=default(a), c=default(c)") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "1 -1 -1")) - // Allowed in MySQL, but should probably not be allowed. - tk.MustGetErrCode("update t2 set a=default(a), c=default(a)", mysql.ErrBadGeneratedColumn) - tk.MustExec("drop table t1, t2") -} - func TestListColumnsPartitionWithGlobalIndex(t *testing.T) { failpoint.Enable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune", `return(true)`) defer failpoint.Disable("github.com/pingcap/tidb/pkg/planner/core/forceDynamicPrune") diff --git a/pkg/executor/update_test.go b/pkg/executor/update_test.go index 03586f8225c84..e6052ca91ea61 100644 --- a/pkg/executor/update_test.go +++ b/pkg/executor/update_test.go @@ -69,32 +69,6 @@ func getPresumeExistsCount(t *testing.T, se session.Session) int { return presumeNotExistsCnt } -func TestIssue21447(t *testing.T) { - store := testkit.CreateMockStore(t) - - tk1, tk2 := testkit.NewTestKit(t, store), testkit.NewTestKit(t, store) - tk1.MustExec("use test") - tk2.MustExec("use test") - - tk1.MustExec("drop table if exists t1") - tk1.MustExec("create table t1(id int primary key, name varchar(40))") - tk1.MustExec("insert into t1 values(1, 'abc')") - - tk1.MustExec("begin pessimistic") - tk2.MustExec("begin pessimistic") - tk2.MustExec("update t1 set name='xyz' where id=1") - tk2.CheckExecResult(1, 0) - tk2.MustQuery("select * from t1 where id = 1").Check(testkit.Rows("1 xyz")) - tk2.MustExec("commit") - tk1.MustExec("update t1 set name='xyz' where id=1") - tk1.CheckExecResult(0, 0) - tk1.MustQuery("select * from t1 where id = 1").Check(testkit.Rows("1 abc")) - tk1.MustQuery("select * from t1 where id = 1 for update").Check(testkit.Rows("1 xyz")) - tk1.MustQuery("select * from t1 where id in (1, 2)").Check(testkit.Rows("1 abc")) - tk1.MustQuery("select * from t1 where id in (1, 2) for update").Check(testkit.Rows("1 xyz")) - tk1.MustExec("commit") -} - func TestLockUnchangedUniqueKeys(t *testing.T) { store := testkit.CreateMockStore(t) diff --git a/pkg/planner/core/binary_plan_test.go b/pkg/planner/core/binary_plan_test.go index 55959a7d0f647..268510d156d6c 100644 --- a/pkg/planner/core/binary_plan_test.go +++ b/pkg/planner/core/binary_plan_test.go @@ -377,23 +377,6 @@ func TestDecodeBinaryPlan(t *testing.T) { } } -func TestInvalidDecodeBinaryPlan(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - - str1 := "some random bytes" - str2 := base64.StdEncoding.EncodeToString([]byte(str1)) - str3 := base64.StdEncoding.EncodeToString(snappy.Encode(nil, []byte(str1))) - - tk.MustQuery(`select tidb_decode_binary_plan('` + str1 + `')`).Check(testkit.Rows("")) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 illegal base64 data at input byte 4")) - tk.MustQuery(`select tidb_decode_binary_plan('` + str2 + `')`).Check(testkit.Rows("")) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 snappy: corrupt input")) - tk.MustQuery(`select tidb_decode_binary_plan('` + str3 + `')`).Check(testkit.Rows("")) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 proto: illegal wireType 7")) -} - func TestUnnecessaryBinaryPlanInSlowLog(t *testing.T) { originCfg := config.GetGlobalConfig() newCfg := *originCfg diff --git a/pkg/planner/core/plan_cache_test.go b/pkg/planner/core/plan_cache_test.go index f7d10edaf5ea2..a8be4a0413b04 100644 --- a/pkg/planner/core/plan_cache_test.go +++ b/pkg/planner/core/plan_cache_test.go @@ -51,45 +51,6 @@ func TestInitLRUWithSystemVar(t *testing.T) { require.NotNil(t, lru) } -func TestIssue45086(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec(`use test`) - - tk.MustExec(`CREATE TABLE t (a int(11) DEFAULT NULL, b date DEFAULT NULL)`) - tk.MustExec(`INSERT INTO t VALUES (1, current_date())`) - - tk.MustExec(`PREPARE stmt FROM 'SELECT * FROM t WHERE b=current_date()'`) - require.Equal(t, len(tk.MustQuery(`EXECUTE stmt`).Rows()), 1) -} - -func TestPlanCacheSizeSwitch(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - // default value = 100 - tk.MustQuery(`select @@tidb_prepared_plan_cache_size`).Check(testkit.Rows("100")) - tk.MustQuery(`select @@tidb_session_plan_cache_size`).Check(testkit.Rows("100")) - - // keep the same value when updating any one of them - tk.MustExec(`set @@tidb_prepared_plan_cache_size = 200`) - tk.MustQuery(`select @@tidb_prepared_plan_cache_size`).Check(testkit.Rows("200")) - tk.MustQuery(`select @@tidb_session_plan_cache_size`).Check(testkit.Rows("200")) - tk.MustExec(`set @@tidb_session_plan_cache_size = 300`) - tk.MustQuery(`select @@tidb_prepared_plan_cache_size`).Check(testkit.Rows("300")) - tk.MustQuery(`select @@tidb_session_plan_cache_size`).Check(testkit.Rows("300")) - - tk.MustExec(`set global tidb_prepared_plan_cache_size = 400`) - tk1 := testkit.NewTestKit(t, store) - tk1.MustQuery(`select @@tidb_prepared_plan_cache_size`).Check(testkit.Rows("400")) - tk1.MustQuery(`select @@tidb_session_plan_cache_size`).Check(testkit.Rows("400")) - - tk.MustExec(`set global tidb_session_plan_cache_size = 500`) - tk2 := testkit.NewTestKit(t, store) - tk2.MustQuery(`select @@tidb_prepared_plan_cache_size`).Check(testkit.Rows("500")) - tk2.MustQuery(`select @@tidb_session_plan_cache_size`).Check(testkit.Rows("500")) -} - func TestNonPreparedPlanCachePlanString(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -119,34 +80,6 @@ func TestNonPreparedPlanCachePlanString(t *testing.T) { tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) } -func TestNonPreparedPlanCacheWithExplain(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec(`use test`) - tk.MustExec("create table t(a int)") - tk.MustExec("set tidb_enable_non_prepared_plan_cache=1") - tk.MustExec("select * from t where a=1") // cache this plan - - tk.MustQuery("explain select * from t where a=2").Check(testkit.Rows( - `TableReader_7 10.00 root data:Selection_6`, - `└─Selection_6 10.00 cop[tikv] eq(test.t.a, 2)`, - ` └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) - - tk.MustQuery("explain format=verbose select * from t where a=2").Check(testkit.Rows( - `TableReader_7 10.00 168975.57 root data:Selection_6`, - `└─Selection_6 10.00 2534000.00 cop[tikv] eq(test.t.a, 2)`, - ` └─TableFullScan_5 10000.00 2035000.00 cop[tikv] table:t keep order:false, stats:pseudo`)) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) - - tk.MustQuery("explain analyze select * from t where a=2").CheckAt([]int{0, 1, 2, 3}, [][]interface{}{ - {"TableReader_7", "10.00", "0", "root"}, - {"└─Selection_6", "10.00", "0", "cop[tikv]"}, - {" └─TableFullScan_5", "10000.00", "0", "cop[tikv]"}, - }) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) -} - func TestNonPreparedPlanCacheInformationSchema(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -339,95 +272,6 @@ func TestIssue38533(t *testing.T) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } -func TestPlanCacheGeneratedCols(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`set @@tidb_opt_fix_control = "45798:on"`) - tk.MustExec(`create table t1 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))))`) - tk.MustExec(`create table t2 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) virtual)`) - tk.MustExec(`create table t3 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) stored)`) - tk.MustExec(`create table t4 (a int, info json, index zips( (CAST(info->'$.zipcode' AS UNSIGNED ARRAY))))`) - - tk.MustExec(`set @a=1`) - tk.MustExec(`set @b=2`) - - tk.MustExec(`prepare s1 from 'select * from t1 where a=?'`) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustQuery(`execute s1 using @a`).Check(testkit.Rows()) - tk.MustQuery(`execute s1 using @b`).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(`1`)) // hit cache - - tk.MustExec(`prepare s1 from 'select * from t2 where a=?'`) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustQuery(`execute s1 using @a`).Check(testkit.Rows()) - tk.MustQuery(`execute s1 using @b`).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(`1`)) // hit cache - - tk.MustExec(`prepare s1 from 'select * from t3 where a=?'`) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustQuery(`execute s1 using @a`).Check(testkit.Rows()) - tk.MustQuery(`execute s1 using @b`).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(`1`)) // hit cache - - tk.MustExec(`prepare s1 from 'select * from t4 where a=?'`) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustQuery(`execute s1 using @a`).Check(testkit.Rows()) - tk.MustQuery(`execute s1 using @b`).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(`1`)) // hit cache -} - -func TestPlanCacheGeneratedCols2(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`set @@tidb_opt_fix_control = "45798:on"`) - tk.MustExec(`CREATE TABLE t1 ( - ipk varbinary(255) NOT NULL, - i_id varchar(45) DEFAULT NULL, - i_set_id varchar(45) DEFAULT NULL, - p_id varchar(45) DEFAULT NULL, - p_set_id varchar(45) DEFAULT NULL, - m_id bigint(20) DEFAULT NULL, - m_i_id varchar(127) DEFAULT NULL, - m_i_set_id varchar(127) DEFAULT NULL, - d json DEFAULT NULL, - p_sources json DEFAULT NULL, - nslc json DEFAULT NULL, - cl json DEFAULT NULL, - fii json DEFAULT NULL, - fpi json DEFAULT NULL, - PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */, - UNIQUE KEY i_id (i_id), - KEY d ((cast(d as char(253) array))), - KEY m_i_id (m_i_id), - KEY m_i_set_id (m_i_set_id), - KEY fpi ((cast(fpi as unsigned array))), - KEY nslc ((cast(nslc as char(1000) array))), - KEY cl ((cast(cl as char(3000) array))), - KEY fii ((cast(fii as unsigned array))), - KEY m_id (m_id), - KEY i_set_id (i_set_id), - KEY m_i_and_m_id (m_i_id,m_id))`) - - tk.MustExec(`CREATE TABLE t2 ( - ipk varbinary(255) NOT NULL, - created_time bigint(20) DEFAULT NULL, - arrival_time bigint(20) DEFAULT NULL, - updated_time bigint(20) DEFAULT NULL, - timestamp_data json DEFAULT NULL, - PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */)`) - - tk.MustExec(`prepare stmt from 'select * - from ( t1 left outer join t2 on ( t1 . ipk = t2 . ipk ) ) - where ( t1 . i_id = ? )'`) - tk.MustQuery(`show warnings`).Check(testkit.Rows()) // no warning - tk.MustExec(`set @a='a', @b='b'`) - tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows()) - tk.MustQuery(`execute stmt using @b`).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(`1`)) // hit cache -} - func TestInvalidRange(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -511,25 +355,6 @@ func TestIssue38205(t *testing.T) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } -func TestPlanCacheExprBlacklistCompatibility(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("create table t (a int)") - - tk.MustExec("prepare st from 'select * from t where mod(a, 2)=1'") - tk.MustExec("execute st") - tk.MustExec("execute st") - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) - - tk.MustExec("insert into mysql.expr_pushdown_blacklist(name) values('mod')") - tk.MustExec(`admin reload expr_pushdown_blacklist`) - tk.MustExec("execute st") // no `mod can not be pushed-down` error - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) // expr blacklist is updated - tk.MustExec("execute st") - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) -} - func TestIssue40224(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -1050,41 +875,6 @@ func TestPlanCacheSubquerySPMEffective(t *testing.T) { } } -func TestNonPreparedPlanCacheFieldNames(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec(`use test`) - tk.MustExec("create table t(a int, index(a))") - tk.MustExec("create table tt(a varchar(10))") - tk.MustExec("set tidb_enable_non_prepared_plan_cache=1") - - checkFieldName := func(sql, hit string, fields ...string) { - rs, err := tk.Exec(sql) - require.NoError(t, err) - for i, f := range rs.Fields() { - require.Equal(t, f.Column.Name.L, fields[i]) - } - require.NoError(t, rs.Close()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows(hit)) - } - - checkFieldName(`select a+1 from t where a<10`, `0`, `a+1`) - checkFieldName(`select a+1 from t where a<20`, `1`, `a+1`) - checkFieldName(`select a+2 from t where a<30`, `0`, `a+2`) // can not hit since field names changed - checkFieldName(`select a+2 from t where a<40`, `1`, `a+2`) - checkFieldName(`select a,a+1 from t where a<30`, `0`, `a`, `a+1`) // can not hit since field names changed - checkFieldName(`select a,a+1 from t where a<40`, `1`, `a`, `a+1`) - checkFieldName(`select a+'123' from tt where a='1'`, `0`, `a+'123'`) - checkFieldName(`select a+'123' from tt where a='2'`, `1`, `a+'123'`) - - checkFieldName(`select 1 from t where a<10`, `0`, `1`) - checkFieldName(`select 1 from t where a<20`, `1`, `1`) - checkFieldName(`select 2 from t where a<10`, `0`, `2`) - checkFieldName(`select 2 from t where a<20`, `1`, `2`) - checkFieldName(`select 1,2 from t where a<10`, `0`, `1`, `2`) - checkFieldName(`select 1,2 from t where a<20`, `1`, `1`, `2`) -} - func TestIssue42125(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -1402,51 +1192,6 @@ func TestBuiltinFuncFlen(t *testing.T) { } } -func TestNonPreparedPlanCacheBuiltinFuncs(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) - tk.MustExec(`create table t (a int, b varchar(32), c datetime, key(a))`) - - // normal builtin functions can be supported - supportedCases := []string{ - `select * from t where mod(a, 5) < 2`, - `select * from t where c < now()`, - `select date_format(c, '%Y-%m-%d') from t where a < 10`, - `select str_to_date(b, '%Y-%m-%d') from t where a < 10`, - `select * from t where a-2 < 20`, - `select * from t where a+b > 100`, - } - for _, sql := range supportedCases { - tk.MustExec(sql) - tk.MustExec(sql) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) - } - - // unsupported cases - unsupportedCases := []string{ - `select * from t where -a > 10`, // '-' cannot support - `select * from t where a < 1 and b like '%abc%'`, // LIKE - `select database() from t`, - } - for _, sql := range unsupportedCases { - tk.MustExec(sql) - tk.MustExec(sql) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) - } -} - -func TestIssue48165(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec(`create table t(a int)`) - tk.MustExec(`insert into t values(1)`) - tk.MustExec(`prepare s from "select * from t where tidb_parse_tso(a) > unix_timestamp()"`) - tk.MustQuery(`execute s`).Check(testkit.Rows("1")) -} - func BenchmarkPlanCacheInsert(b *testing.B) { store := testkit.CreateMockStore(b) tk := testkit.NewTestKit(b, store) diff --git a/pkg/planner/core/plan_test.go b/pkg/planner/core/plan_test.go index 894fc966a9f35..1b858bad5c877 100644 --- a/pkg/planner/core/plan_test.go +++ b/pkg/planner/core/plan_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/expression/aggregation" - "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" @@ -702,17 +701,6 @@ func TestBuildFinalModeAggregation(t *testing.T) { checkResult(ctx, mixedAggFuncs, groupByItems) } -func TestIssue40857(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test;") - tk.MustExec("drop table if exists t;") - tk.MustExec("CREATE TABLE t (c1 mediumint(9) DEFAULT '-4747160',c2 year(4) NOT NULL DEFAULT '2075',c3 double DEFAULT '1.1559030660251948',c4 enum('wbv4','eli','d8ym','m3gsx','lz7td','o','d1k7l','y1x','xcxq','bj','n7') DEFAULT 'xcxq',c5 int(11) DEFAULT '255080866',c6 tinyint(1) DEFAULT '1',PRIMARY KEY (c2),KEY `c4d86d54-091c-4307-957b-b164c9652b7f` (c6,c4) );") - tk.MustExec("insert into t values (-4747160, 2075, 722.5719203870632, 'xcxq', 1576824797, 1);") - tk.MustExec("select /*+ stream_agg() */ bit_or(t.c5) as r0 from t where t.c3 in (select c6 from t where not(t.c6 <> 1) and not(t.c3 in(9263.749352636818))) group by t.c1;") - require.Empty(t, tk.Session().LastMessage()) -} - func TestCloneFineGrainedShuffleStreamCount(t *testing.T) { window := &core.PhysicalWindow{} newPlan, err := window.Clone() @@ -742,38 +730,3 @@ func TestCloneFineGrainedShuffleStreamCount(t *testing.T) { require.Equal(t, ok, true) require.Equal(t, sort.TiFlashFineGrainedShuffleStreamCount, newSort.TiFlashFineGrainedShuffleStreamCount) } - -func TestIssue40535(t *testing.T) { - store := testkit.CreateMockStore(t) - var cfg kv.InjectionConfig - tk := testkit.NewTestKit(t, kv.NewInjectedStore(store, &cfg)) - tk.MustExec("use test;") - tk.MustExec("drop table if exists t1; drop table if exists t2;") - tk.MustExec("CREATE TABLE `t1`(`c1` bigint(20) NOT NULL DEFAULT '-2312745469307452950', `c2` datetime DEFAULT '5316-02-03 06:54:49', `c3` tinyblob DEFAULT NULL, PRIMARY KEY (`c1`) /*T![clustered_index] CLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;") - tk.MustExec("CREATE TABLE `t2`(`c1` set('kn8pu','7et','vekx6','v3','liwrh','q14','1met','nnd5i','5o0','8cz','l') DEFAULT '7et,vekx6,liwrh,q14,1met', `c2` float DEFAULT '1.683167', KEY `k1` (`c2`,`c1`), KEY `k2` (`c2`)) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_chinese_ci;") - tk.MustExec("(select /*+ agg_to_cop()*/ locate(t1.c3, t1.c3) as r0, t1.c3 as r1 from t1 where not( IsNull(t1.c1)) order by r0,r1) union all (select concat_ws(',', t2.c2, t2.c1) as r0, t2.c1 as r1 from t2 order by r0, r1) order by 1 limit 273;") - require.Empty(t, tk.Session().LastMessage()) -} - -func TestIssue47445(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test;") - tk.MustExec("CREATE TABLE golang1 ( `fcbpdt` CHAR (8) COLLATE utf8_general_ci NOT NULL, `fcbpsq` VARCHAR (20) COLLATE utf8_general_ci NOT NULL, `procst` char (4) COLLATE utf8_general_ci DEFAULT NULL,`cipstx` VARCHAR (105) COLLATE utf8_general_ci DEFAULT NULL, `cipsst` CHAR (4) COLLATE utf8_general_ci DEFAULT NULL, `dyngtg` VARCHAR(4) COLLATE utf8_general_ci DEFAULT NULL, `blncdt` VARCHAR (8) COLLATE utf8_general_ci DEFAULT NULL, PRIMARY KEY ( fcbpdt, fcbpsq ))") - tk.MustExec("insert into golang1 values('20230925','12023092502158016','abc','','','','')") - tk.MustExec("create table golang2 (`sysgrp` varchar(20) NOT NULL,`procst` varchar(8) NOT NULL,`levlid` int(11) NOT NULL,PRIMARY key (procst));") - tk.MustExec("insert into golang2 VALUES('COMMON','ACSC',90)") - tk.MustExec("insert into golang2 VALUES('COMMON','abc',8)") - tk.MustExec("insert into golang2 VALUES('COMMON','CH02',6)") - tk.MustExec("UPDATE golang1 a SET procst =(CASE WHEN ( SELECT levlid FROM golang2 b WHERE b.sysgrp = 'COMMON' AND b.procst = 'ACSC' ) > ( SELECT levlid FROM golang2 c WHERE c.sysgrp = 'COMMON' AND c.procst = a.procst ) THEN 'ACSC' ELSE a.procst END ), cipstx = 'CI010000', cipsst = 'ACSC', dyngtg = 'EAYT', blncdt= '20230925' WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'") - tk.MustQuery("select * from golang1").Check(testkit.Rows("20230925 12023092502158016 ACSC CI010000 ACSC EAYT 20230925")) - tk.MustExec("UPDATE golang1 a SET procst= (SELECT 1 FROM golang2 c WHERE c.procst = a.procst) WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'") - tk.MustQuery("select * from golang1").Check(testkit.Rows("20230925 12023092502158016 1 CI010000 ACSC EAYT 20230925")) -} - -func TestExplainValuesStatement(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - - tk.MustMatchErrMsg("EXPLAIN FORMAT = TRADITIONAL ((VALUES ROW ()) ORDER BY 1)", ".*Unknown table ''.*") -} diff --git a/tests/integrationtest/r/executor/cte.result b/tests/integrationtest/r/executor/cte.result index 2721c2f4b9c10..60f4967359e55 100644 --- a/tests/integrationtest/r/executor/cte.result +++ b/tests/integrationtest/r/executor/cte.result @@ -1,3 +1,4 @@ +set tidb_max_chunk_size=default; with recursive cte1 as (select 1 c1 union all select c1 + 1 c1 from cte1 where c1 < 5) select * from cte1; c1 1 diff --git a/tests/integrationtest/r/executor/delete.result b/tests/integrationtest/r/executor/delete.result index f92a7a503effd..d452dd98a9e21 100644 --- a/tests/integrationtest/r/executor/delete.result +++ b/tests/integrationtest/r/executor/delete.result @@ -14,3 +14,96 @@ execute stmt using @a; select * from t; a 2 +drop table if exists delete_test; +drop view if exists v; +drop sequence if exists seq; +create table delete_test(id int not null default 1, name varchar(255), PRIMARY KEY(id)); +insert INTO delete_test VALUES (1, "hello"); +insert into delete_test values (2, "hello"); +update delete_test set name = "abc" where id = 2; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +delete from delete_test where id = 2 limit 1; +affected rows: 1 +info: +delete from delete_test where 0; +affected rows: 0 +info: +insert into delete_test values (2, 'abc'); +delete from delete_test where delete_test.id = 2 limit 1; +affected rows: 1 +info: +begin; +SELECT * from delete_test limit 2; +id name +1 hello +commit; +insert into delete_test values (2, 'abc'); +delete from delete_test where id = (select '2a'); +Error 1292 (22007): Truncated incorrect DOUBLE value: '2a' +delete ignore from delete_test where id = (select '2a'); +affected rows: 1 +info: +SHOW WARNINGS; +Level Code Message +Warning 1292 Truncated incorrect DOUBLE value: '2a' +Warning 1292 Truncated incorrect DOUBLE value: '2a' +delete from delete_test; +affected rows: 1 +info: +create view v as select * from delete_test; +delete from v where name = 'aaa'; +Error 1105 (HY000): delete view v is not supported now +drop view v; +create sequence seq; +delete from seq; +Error 1105 (HY000): delete sequence seq is not supported now +drop sequence seq; +drop table if exists t1, t2; +create table t1 (c1 int, c2 int, index (c1)); +create table t2 (c1 int, c2 int); +insert into t1 values (1, 1), (2, 2); +delete from t1 where t1.c1 = 1; +affected rows: 1 +info: +delete from t1 where t1.c2 = 2; +affected rows: 1 +info: +select * from t1; +c1 c2 +insert into t1 values (1, 3); +delete from t1 as a where a.c1 = 1; +affected rows: 1 +info: +insert into t1 values (1, 1), (2, 2); +insert into t2 values (2, 1), (3,1); +delete t1, t2 from t1 join t2 where t1.c1 = t2.c2; +affected rows: 3 +info: +insert into t2 values (2, 1), (3,1); +delete a, b from t1 as a join t2 as b where a.c2 = b.c1; +affected rows: 2 +info: +delete t1, t2 from t1 as a join t2 as b where a.c2 = b.c1; +Error 1109 (42S02): Unknown table 't1' in MULTI DELETE +drop table if exists t1, t2, t3; +create table t1 (id int, data int); +insert into t1 values (11, 121), (12, 122), (13, 123); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +create table t2 (id int, data int); +insert into t2 values (11, 221), (22, 222), (23, 223); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +create table t3 (id int, data int); +insert into t3 values (11, 321), (22, 322), (23, 323); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +delete t1, t2 from t1 inner join t2 inner join t3 where t1.id=t2.id and t2.id=t3.id; +affected rows: 2 +info: +select * from t3; +id data +11 321 +22 322 +23 323 diff --git a/tests/integrationtest/r/executor/executor.result b/tests/integrationtest/r/executor/executor.result index c7f2c7bde73e0..e84cda08a7f47 100644 --- a/tests/integrationtest/r/executor/executor.result +++ b/tests/integrationtest/r/executor/executor.result @@ -4405,3 +4405,147 @@ id a b c d 1 1 1 1 1 5 5 5 5 5 set @@tidb_enable_collect_execution_info=default; +drop table if exists t1; +create table t1 (a int, b int, index(a)); +insert into t1 values (1,2),(2,3),(3,4); +explain analyze select * from t1 use index(a) where a > 1; +id estRows actRows task access object execution info operator info memory disk +IndexLookUp_7 3333.33 2 root NULL .*time:.*loops:.*index_task:.*table_task: {total_time.*num.*concurrency.*}.* NULL KB N/A +├─IndexRangeScan_5(Build) 3333.33 2 cop[tikv] table:t1, index:a(a) .*time:.*loops:.*cop_task:.* range:(1,+inf], keep order:false, stats:pseudo N/A N/A +└─TableRowIDScan_6(Probe) 3333.33 2 cop[tikv] table:t1 .*time:.*loops:.*cop_task:.* keep order:false, stats:pseudo N/A N/A +drop table if exists t1; +create table t1 (a int, b int); +insert into t1 values (1,2),(2,3),(3,4); +explain analyze SELECT /*+ HASH_AGG() */ count(*) FROM t1 WHERE a < 10; +id estRows actRows task access object execution info operator info memory disk +HashAgg_11 1.00 1 root NULL .*time:.*loops:.*partial_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*final_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.* funcs:count(Column#5)->Column#4 KB N/A +└─TableReader_12 1.00 1 root NULL time.*loops.*cop_task.* data:HashAgg_6 Bytes N/A + └─HashAgg_6 1.00 1 cop[tikv] NULL tikv_task:.* funcs:count(1)->Column#5 N/A N/A + └─Selection_10 3323.33 3 cop[tikv] NULL tikv_task:.* lt(executor__executor.t1.a, 10) N/A N/A + └─TableFullScan_9 10000.00 3 cop[tikv] table:t1 tikv_task:.* keep order:false, stats:pseudo N/A N/A +set global tidb_txn_mode=''; +drop table if exists t, t1; +create table t (c1 int, c2 int, c3 int); +insert t values (11, 2, 3); +insert t values (12, 2, 3); +insert t values (13, 2, 3); +create table t1 (c1 int); +insert t1 values (11); +begin; +select * from t where c1=11 for update; +c1 c2 c3 +11 2 3 +begin; +update t set c2=211 where c1=11; +commit; +commit; +Error 9007 (HY000): Write conflict, reason=Optimistic [try again later] +begin; +select * from t where exists(select null from t1 where t1.c1=t.c1) for update; +c1 c2 c3 +11 211 3 +begin; +update t set c2=211 where c1=12; +commit; +commit; +begin; +select * from t where c1=11 for update; +c1 c2 c3 +11 211 3 +begin; +update t set c2=22 where c1=12; +commit; +commit; +set @@autocommit=1; +select * from t where c1=11 for update; +c1 c2 c3 +11 211 3 +begin; +update t set c2=211 where c1=11; +commit; +commit; +begin; +select * from (select * from t for update) t join t1 for update; +c1 c2 c3 c1 +11 211 3 11 +12 22 3 11 +13 2 3 11 +begin; +update t1 set c1 = 13; +commit; +commit; +Error 9007 (HY000): Write conflict, reason=Optimistic [try again later] +set global tidb_txn_mode=pessimistic; +drop table if exists t, t1; +create table t (i int); +create table t1 (i int); +insert t values (1); +insert t1 values (1); +begin pessimistic; +select * from t, t1 where t.i = t1.i for update of t; +i i +1 1 +begin pessimistic; +select * from t1 for update; +i +1 +select * from t for update nowait; +Error 3572 (HY000): Statement aborted because lock(s) could not be acquired immediately and NOWAIT is set. +rollback; +select * from t for update nowait; +i +1 +rollback; +set session tidb_txn_mode=''; +drop table if exists t; +create table t(a int); +insert into t values (1); +begin; +select 1 as a union select a from t for update; +a +1 +set session tidb_txn_mode=''; +update t set a = a + 1; +commit; +Error 9007 (HY000): Write conflict, reason=Optimistic [try again later] +begin; +select 1 as a union select a from t limit 5 for update; +a +1 +2 +select 1 as a union select a from t order by a for update; +a +1 +2 +update t set a = a + 1; +commit; +Error 9007 (HY000): Write conflict, reason=Optimistic [try again later] +set session tidb_txn_mode=pessimistic; +drop table if exists t; +create table t (id bigint key,b int); +split table t by (10),(20),(30); +TOTAL_SPLIT_REGION SCATTER_FINISH_RATIO +3 1 +insert into t values (0,0),(10,10),(20,20),(30,30); +alter table t add index idx1(b); +admin show ddl jobs 1; +JOB_ID DB_NAME TABLE_NAME JOB_TYPE SCHEMA_STATE SCHEMA_ID TABLE_ID ROW_COUNT CREATE_TIME START_TIME END_TIME STATE + executor__executor t public 4 synced +insert into t values (1,0),(2,10),(3,20),(4,30); +alter table t add index idx2(b); +admin show ddl jobs 1; +JOB_ID DB_NAME TABLE_NAME JOB_TYPE SCHEMA_STATE SCHEMA_ID TABLE_ID ROW_COUNT CREATE_TIME START_TIME END_TIME STATE + executor__executor t public 8 synced +drop table if exists t; +create table t(a int, b int as(-a)); +insert into t(a) values(1), (3), (7); +SET GLOBAL tidb_mem_oom_action='CANCEL'; +set @@tidb_mem_quota_query=1; +update t set t.a = t.a - 1 where t.a in (select a from t where a < 4); +Error 8175 (HY000): Your query has been cancelled due to exceeding the allowed memory limit for a single SQL query. Please try narrowing your query scope or increase the tidb_mem_quota_query limit and try again.[conn=] +set @@tidb_mem_quota_query=1000000000; +select stmt_type from information_schema.statements_summary where digest_text = 'update `t` set `t` . `a` = `t` . `a` - ? where `t` . `a` in ( select `a` from `t` where `a` < ? )'; +stmt_type +Update +set @@tidb_mem_quota_query=default; +set global tidb_mem_oom_action=default; diff --git a/tests/integrationtest/r/executor/insert.result b/tests/integrationtest/r/executor/insert.result index 1c9bd502d91bf..977ff1e9ac41e 100644 --- a/tests/integrationtest/r/executor/insert.result +++ b/tests/integrationtest/r/executor/insert.result @@ -1222,3 +1222,912 @@ commit; commit; Error 1062 (23000): Duplicate entry '146576795' for key 't.PRIMARY' set global tidb_disable_txn_auto_retry=default; +drop table if exists t; +create table t (id smallint auto_increment primary key); +alter table t add column c1 int default 1; +insert ignore into t(id) values (194626268); +affected rows: 1 +info: +select * from t; +id c1 +32767 1 +insert ignore into t(id) values ('*') on duplicate key update c1 = 2; +affected rows: 2 +info: +select * from t; +id c1 +32767 2 +drop table if exists t; +create table t (i int not null primary key, j int unique key); +insert into t values (1, 1), (2, 2); +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +insert ignore into t values(1, 1) on duplicate key update i = 2; +affected rows: 0 +info: +select * from t; +i j +1 1 +2 2 +insert ignore into t values(1, 1) on duplicate key update j = 2; +affected rows: 0 +info: +select * from t; +i j +1 1 +2 2 +drop table if exists t2; +create table t2(`col_25` set('Alice','Bob','Charlie','David') NOT NULL,`col_26` date NOT NULL DEFAULT '2016-04-15', PRIMARY KEY (`col_26`) clustered, UNIQUE KEY `idx_9` (`col_25`,`col_26`),UNIQUE KEY `idx_10` (`col_25`)); +insert into t2(col_25, col_26) values('Bob', '1989-03-23'),('Alice', '2023-11-24'), ('Charlie', '2023-12-05'); +insert ignore into t2 (col_25,col_26) values ( 'Bob','1977-11-23' ) on duplicate key update col_25 = 'Alice', col_26 = '2036-12-13'; +show warnings; +Level Code Message +Warning 1062 Duplicate entry 'Alice' for key 't2.idx_10' +select * from t2; +col_25 col_26 +Bob 1989-03-23 +Alice 2023-11-24 +Charlie 2023-12-05 +drop table if exists t4; +create table t4(id int primary key clustered, k int, v int, unique key uk1(k)); +insert into t4 values (1, 10, 100), (3, 30, 300); +insert ignore into t4 (id, k, v) values(1, 0, 0) on duplicate key update id = 2, k = 30; +show warnings; +Level Code Message +Warning 1062 Duplicate entry '30' for key 't4.uk1' +select * from t4; +id k v +1 10 100 +3 30 300 +drop table if exists t5; +create table t5(k1 varchar(100), k2 varchar(100), uk1 int, v int, primary key(k1, k2) clustered, unique key ukk1(uk1), unique key ukk2(v)); +insert into t5(k1, k2, uk1, v) values('1', '1', 1, '100'), ('1', '3', 2, '200'); +update ignore t5 set k2 = '2', uk1 = 2 where k1 = '1' and k2 = '1'; +show warnings; +Level Code Message +Warning 1062 Duplicate entry '2' for key 't5.ukk1' +select * from t5; +k1 k2 uk1 v +1 1 1 100 +1 3 2 200 +drop table if exists t6; +create table t6 (a int, b int, c int, primary key(a, b) clustered, unique key idx_14(b), unique key idx_15(b), unique key idx_16(a, b)); +insert into t6 select 10, 10, 20; +insert ignore into t6 set a = 20, b = 10 on duplicate key update a = 100; +select * from t6; +a b c +100 10 20 +insert ignore into t6 set a = 200, b= 10 on duplicate key update c = 1000; +select * from t6; +a b c +100 10 1000 +drop table if exists insert_autoinc_test; +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(c1) values (1), (2); +begin; +select * from insert_autoinc_test; +id c1 +1 1 +2 2 +commit; +begin; +insert into insert_autoinc_test(id, c1) values (5,5); +insert into insert_autoinc_test(c1) values (6); +commit; +begin; +select * from insert_autoinc_test; +id c1 +1 1 +2 2 +5 5 +6 6 +commit; +begin; +insert into insert_autoinc_test(id, c1) values (3,3); +commit; +begin; +select * from insert_autoinc_test; +id c1 +1 1 +2 2 +3 3 +5 5 +6 6 +commit; +begin; +insert into insert_autoinc_test(c1) values (7); +commit; +begin; +select * from insert_autoinc_test; +id c1 +1 1 +2 2 +3 3 +5 5 +6 6 +7 7 +commit; +drop table if exists insert_autoinc_test; +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(id, c1) values (0.3, 1); +select * from insert_autoinc_test; +id c1 +1 1 +insert into insert_autoinc_test(id, c1) values (-0.3, 2); +select * from insert_autoinc_test; +id c1 +1 1 +2 2 +insert into insert_autoinc_test(id, c1) values (-3.3, 3); +select * from insert_autoinc_test; +id c1 +-3 3 +1 1 +2 2 +insert into insert_autoinc_test(id, c1) values (4.3, 4); +select * from insert_autoinc_test; +id c1 +-3 3 +1 1 +2 2 +4 4 +insert into insert_autoinc_test(c1) values (5); +select * from insert_autoinc_test; +id c1 +-3 3 +1 1 +2 2 +4 4 +5 5 +insert into insert_autoinc_test(id, c1) values (null, 6); +select * from insert_autoinc_test; +id c1 +-3 3 +1 1 +2 2 +4 4 +5 5 +6 6 +drop table if exists insert_autoinc_test; +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(id, c1) values (5, 1); +select * from insert_autoinc_test; +id c1 +5 1 +insert into insert_autoinc_test(id, c1) values (0, 2); +select * from insert_autoinc_test; +id c1 +5 1 +6 2 +insert into insert_autoinc_test(id, c1) values (0, 3); +select * from insert_autoinc_test; +id c1 +5 1 +6 2 +7 3 +set SQL_MODE=NO_AUTO_VALUE_ON_ZERO; +insert into insert_autoinc_test(id, c1) values (0, 4); +select * from insert_autoinc_test; +id c1 +0 4 +5 1 +6 2 +7 3 +insert into insert_autoinc_test(id, c1) values (0, 5); +Error 1062 (23000): Duplicate entry '0' for key 'insert_autoinc_test.PRIMARY' +insert into insert_autoinc_test(c1) values (6); +select * from insert_autoinc_test; +id c1 +0 4 +5 1 +6 2 +7 3 +8 6 +insert into insert_autoinc_test(id, c1) values (null, 7); +select * from insert_autoinc_test; +id c1 +0 4 +5 1 +6 2 +7 3 +8 6 +9 7 +set SQL_MODE=''; +insert into insert_autoinc_test(id, c1) values (0, 8); +select * from insert_autoinc_test; +id c1 +0 4 +5 1 +6 2 +7 3 +8 6 +9 7 +10 8 +insert into insert_autoinc_test(id, c1) values (null, 9); +select * from insert_autoinc_test; +id c1 +0 4 +5 1 +6 2 +7 3 +8 6 +9 7 +10 8 +11 9 +set sql_mode = default; +drop table if exists insert_test; +create table insert_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1); +insert insert_test (c1) values (1),(2),(NULL); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +begin; +insert insert_test (c1) values (); +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +begin; +insert insert_test (c1, c2) values (1,2),(1); +Error 1136 (21S01): Column count doesn't match value count at row 2 +rollback; +begin; +insert insert_test (xxx) values (3); +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +begin; +insert insert_test_xxx (c1) values (); +Error 1146 (42S02): Table 'executor__insert.insert_test_xxx' doesn't exist +rollback; +insert insert_test set c1 = 3; +affected rows: 1 +info: +begin; +insert insert_test set c1 = 4, c1 = 5; +Error 1110 (42000): Column 'c1' specified twice +rollback; +begin; +insert insert_test set xxx = 6; +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +drop table if exists insert_test_1, insert_test_2; +create table insert_test_1 (id int, c1 int); +insert insert_test_1 select id, c1 from insert_test; +affected rows: 4 +info: Records: 4 Duplicates: 0 Warnings: 0 +create table insert_test_2 (id int, c1 int); +insert insert_test_1 select id, c1 from insert_test union select id * 10, c1 * 10 from insert_test; +affected rows: 8 +info: Records: 8 Duplicates: 0 Warnings: 0 +begin; +insert insert_test_1 select c1 from insert_test; +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +begin; +insert insert_test_1 values(default, default, default, default, default); +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +select * from insert_test where id = 1; +id c1 c2 c3 +1 1 NULL 1 +insert into insert_test (id, c3) values (1, 2) on duplicate key update id=values(id), c2=10; +affected rows: 2 +info: +select * from insert_test where id = 1; +id c1 c2 c3 +1 1 10 1 +insert into insert_test (id, c2) values (1, 1) on duplicate key update insert_test.c2=10; +affected rows: 0 +info: +insert into insert_test (id, c2) values(1, 1) on duplicate key update t.c2 = 10; +Error 1054 (42S22): Unknown column 't.c2' in 'field list' +INSERT INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3; +affected rows: 2 +info: +select * from insert_test where id = 1; +id c1 c2 c3 +1 1 10 6 +INSERT IGNORE INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3; +affected rows: 2 +info: +select * from insert_test where id = 1; +id c1 c2 c3 +1 1 10 11 +drop table if exists insert_err; +create table insert_err (id int, c1 varchar(8)); +insert insert_err values (1, 'abcdabcdabcd'); +Error 1406 (22001): Data too long for column 'c1' at row 1 +insert insert_err values (1, '你好,世界'); +create table TEST1 (ID INT NOT NULL, VALUE INT DEFAULT NULL, PRIMARY KEY (ID)); +INSERT INTO TEST1(id,value) VALUE(3,3) on DUPLICATE KEY UPDATE VALUE=4; +affected rows: 1 +info: +drop table if exists t; +create table t (id int); +insert into t values(1); +update t t1 set id = (select count(*) + 1 from t t2 where t1.id = t2.id); +select * from t; +id +2 +drop table if exists t; +create table t(c decimal(5, 5)); +insert into t value(0); +insert into t value(1); +Error 1264 (22003): Out of range value for column 'c' at row 1 +drop table if exists t; +create table t(c binary(255)); +insert into t value(1); +select length(c) from t; +length(c) +255 +drop table if exists t; +create table t(c varbinary(255)); +insert into t value(1); +select length(c) from t; +length(c) +1 +drop table if exists t; +create table t(c int); +set @@time_zone = '+08:00'; +insert into t value(Unix_timestamp('2002-10-27 01:00')); +select * from t; +c +1035651600 +set @@time_zone = default; +drop table if exists t1; +create table t1 (b char(0)); +insert into t1 values (""); +DROP TABLE IF EXISTS t; +CREATE TABLE t(a DECIMAL(4,2)); +INSERT INTO t VALUES (1.000001); +SHOW WARNINGS; +Level Code Message +Warning 1366 Incorrect decimal value: '1.000001' for column 'a' at row 1 +INSERT INTO t VALUES (1.000000); +SHOW WARNINGS; +Level Code Message +DROP TABLE IF EXISTS t; +CREATE TABLE t(a datetime); +INSERT INTO t VALUES('2017-00-00'); +Error 1292 (22007): Incorrect datetime value: '2017-00-00' for column 'a' at row 1 +set sql_mode = ''; +INSERT INTO t VALUES('2017-00-00'); +SELECT * FROM t; +a +2017-00-00 00:00:00 +set sql_mode = 'strict_all_tables'; +SELECT * FROM t; +a +2017-00-00 00:00:00 +set sql_mode = default; +drop table if exists test; +CREATE TABLE test(id int(10) UNSIGNED NOT NULL AUTO_INCREMENT, p int(10) UNSIGNED NOT NULL, PRIMARY KEY(p), KEY(id)); +insert into test(p) value(1); +select * from test; +id p +1 1 +select * from test use index (id) where id = 1; +id p +1 1 +insert into test values(NULL, 2); +select * from test use index (id) where id = 2; +id p +2 2 +insert into test values(2, 3); +select * from test use index (id) where id = 2; +id p +2 2 +2 3 +drop table if exists t; +create table t(a bigint unsigned); +set @@sql_mode = 'strict_all_tables'; +insert into t value (-1); +Error 1264 (22003): Out of range value for column 'a' at row 1 +set @@sql_mode = ''; +insert into t value (-1); +show warnings; +Level Code Message +Warning 1264 Out of range value for column 'a' at row 1 +insert into t select -1; +show warnings; +Level Code Message +Warning 1690 constant -1 overflows bigint +insert into t select cast(-1 as unsigned); +insert into t value (-1.111); +show warnings; +Level Code Message +Warning 1264 Out of range value for column 'a' at row 1 +insert into t value ('-1.111'); +show warnings; +Level Code Message +Warning 1264 Out of range value for column 'a' at row 1 +update t set a = -1 limit 1; +show warnings; +Level Code Message +Warning 1690 constant -1 overflows bigint +select * from t; +a +0 +0 +18446744073709551615 +0 +0 +set @@sql_mode = default; +drop table if exists t; +create table t(a time(6)); +insert into t value('20070219173709.055870'), ('20070219173709.055'), ('20070219173709.055870123'); +select * from t; +a +17:37:09.055870 +17:37:09.055000 +17:37:09.055870 +truncate table t; +insert into t value(20070219173709.055870), (20070219173709.055), (20070219173709.055870123); +select * from t; +a +17:37:09.055870 +17:37:09.055000 +17:37:09.055870 +insert into t value(-20070219173709.055870); +Error 1292 (22007): Incorrect time value: '-20070219173709.055870' for column 'a' at row 1 +drop table if exists t; +set @@sql_mode=''; +create table t(a float unsigned, b double unsigned); +insert into t value(-1.1, -1.1), (-2.1, -2.1), (0, 0), (1.1, 1.1); +show warnings; +Level Code Message +Warning 1264 Out of range value for column 'a' at row 1 +Warning 1264 Out of range value for column 'b' at row 1 +Warning 1264 Out of range value for column 'a' at row 2 +Warning 1264 Out of range value for column 'b' at row 2 +select * from t; +a b +0 0 +0 0 +0 0 +1.1 1.1 +set @@sql_mode=default; +drop table if exists t; +create table t(a int default 1, b int default 2); +insert into t values(default, default); +select * from t; +a b +1 2 +truncate table t; +insert into t values(default(b), default(a)); +select * from t; +a b +2 1 +truncate table t; +insert into t (b) values(default); +select * from t; +a b +1 2 +truncate table t; +insert into t (b) values(default(a)); +select * from t; +a b +1 1 +drop view if exists v; +create view v as select * from t; +insert into v values(1,2); +Error 1105 (HY000): insert into view v is not supported now +replace into v values(1,2); +Error 1105 (HY000): replace into view v is not supported now +drop view v; +drop sequence if exists seq; +create sequence seq; +insert into seq values(); +Error 1105 (HY000): insert into sequence seq is not supported now +replace into seq values(); +Error 1105 (HY000): replace into sequence seq is not supported now +drop sequence seq; +drop table if exists t; +create table t(name varchar(255), b int, c int, primary key(name(2))); +insert into t(name, b) values("cha", 3); +insert into t(name, b) values("chb", 3); +Error 1062 (23000): Duplicate entry 'ch' for key 't.PRIMARY' +insert into t(name, b) values("测试", 3); +insert into t(name, b) values("测试", 3); +Error 1062 (23000): Duplicate entry 'æµ' for key 't.PRIMARY' +drop table if exists t; +create table t (i int unique key); +insert into t values (1),(2); +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +select * from t; +i +1 +2 +insert into t values (1), (2) on duplicate key update i = values(i); +affected rows: 0 +info: Records: 2 Duplicates: 0 Warnings: 0 +select * from t; +i +1 +2 +insert into t values (2), (3) on duplicate key update i = 3; +affected rows: 2 +info: Records: 2 Duplicates: 1 Warnings: 0 +select * from t; +i +1 +3 +drop table if exists t; +create table t (i int primary key, j int unique key); +insert into t values (-1, 1); +affected rows: 1 +info: +select * from t; +i j +-1 1 +insert into t values (1, 1) on duplicate key update j = values(j); +affected rows: 0 +info: +select * from t; +i j +-1 1 +drop table if exists test; +create table test (i int primary key, j int unique); +begin; +insert into test values (1,1); +insert into test values (2,1) on duplicate key update i = -i, j = -j; +commit; +select * from test; +i j +-1 -1 +delete from test; +insert into test values (1, 1); +begin; +delete from test where i = 1; +insert into test values (2, 1) on duplicate key update i = -i, j = -j; +commit; +select * from test; +i j +2 1 +delete from test; +insert into test values (1, 1); +begin; +update test set i = 2, j = 2 where i = 1; +insert into test values (1, 3) on duplicate key update i = -i, j = -j; +insert into test values (2, 4) on duplicate key update i = -i, j = -j; +commit; +select * from test order by i; +i j +-2 -2 +1 3 +delete from test; +begin; +insert into test values (1, 3), (1, 3) on duplicate key update i = values(i), j = values(j); +commit; +select * from test order by i; +i j +1 3 +create table tmp (id int auto_increment, code int, primary key(id, code)); +create table m (id int primary key auto_increment, code int unique); +insert tmp (code) values (1); +insert tmp (code) values (1); +set tidb_init_chunk_size=1; +insert m (code) select code from tmp on duplicate key update code = values(code); +select * from m; +id code +1 1 +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT AUTO_INCREMENT PRIMARY KEY, +f2 VARCHAR(5) NOT NULL UNIQUE); +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +affected rows: 1 +info: +SELECT LAST_INSERT_ID(); +LAST_INSERT_ID() +1 +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +affected rows: 0 +info: +SELECT LAST_INSERT_ID(); +LAST_INSERT_ID() +1 +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT AUTO_INCREMENT UNIQUE, +f2 VARCHAR(5) NOT NULL UNIQUE); +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +affected rows: 1 +info: +SELECT LAST_INSERT_ID(); +LAST_INSERT_ID() +1 +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +affected rows: 0 +info: +SELECT LAST_INSERT_ID(); +LAST_INSERT_ID() +1 +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = 2; +affected rows: 2 +info: +SELECT LAST_INSERT_ID(); +LAST_INSERT_ID() +1 +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT); +INSERT t1 VALUES (1) ON DUPLICATE KEY UPDATE f1 = 1; +affected rows: 1 +info: +SELECT * FROM t1; +f1 +1 +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT PRIMARY KEY, f2 INT NOT NULL UNIQUE); +INSERT t1 VALUES (1, 1); +affected rows: 1 +info: +INSERT t1 VALUES (1, 1), (1, 1) ON DUPLICATE KEY UPDATE f1 = 2, f2 = 2; +affected rows: 3 +info: Records: 2 Duplicates: 1 Warnings: 0 +SELECT * FROM t1 order by f1; +f1 f2 +1 1 +2 2 +INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +Error 1048 (23000): Column 'f2' cannot be null +INSERT IGNORE t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +affected rows: 2 +info: +show warnings; +Level Code Message +Warning 1048 Column 'f2' cannot be null +SELECT * FROM t1 order by f1; +f1 f2 +1 0 +2 2 +SET sql_mode=''; +INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +show warnings; +Level Code Message +Warning 1048 Column 'f2' cannot be null +SELECT * FROM t1 order by f1; +f1 f2 +1 0 +2 2 +set sql_mode=default; +set tidb_init_chunk_size=default; +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +insert into t1 values(1, 100); +affected rows: 1 +info: +insert into t2 values(1, 200); +affected rows: 1 +info: +insert into t1 select a2, b2 from t2 on duplicate key update b1 = a2; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from t1; +a1 b1 +1 1 +insert into t1 select a2, b2 from t2 on duplicate key update b1 = b2; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from t1; +a1 b1 +1 200 +insert into t1 select a2, b2 from t2 on duplicate key update a1 = a2; +affected rows: 0 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from t1; +a1 b1 +1 200 +insert into t1 select a2, b2 from t2 on duplicate key update b1 = 300; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from t1; +a1 b1 +1 300 +insert into t1 values(1, 1) on duplicate key update b1 = 400; +affected rows: 2 +info: +select * from t1; +a1 b1 +1 400 +insert into t1 select 1, 500 from t2 on duplicate key update b1 = 400; +affected rows: 0 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from t1; +a1 b1 +1 400 +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +insert into t1 select * from t2 on duplicate key update c = t2.b; +Error 1054 (42S22): Unknown column 'c' in 'field list' +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +insert into t1 select * from t2 on duplicate key update a = b; +Error 1052 (23000): Column 'b' in field list is ambiguous +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +insert into t1 select * from t2 on duplicate key update c = b; +Error 1054 (42S22): Unknown column 'c' in 'field list' +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +insert into t1 select * from t2 on duplicate key update a1 = values(b2); +Error 1054 (42S22): Unknown column 'b2' in 'field list' +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +insert into t1 values(1, 100); +affected rows: 1 +info: +insert into t2 values(1, 200); +affected rows: 1 +info: +insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from t1; +a1 b1 +1 400 +insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2; +affected rows: 0 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from t1; +a1 b1 +1 400 +drop table if exists t; +create table t(k1 bigint, k2 bigint, val bigint, primary key(k1, k2)); +insert into t (val, k1, k2) values (3, 1, 2); +affected rows: 1 +info: +select * from t; +k1 k2 val +1 2 3 +insert into t (val, k1, k2) select c, a, b from (select 1 as a, 2 as b, 4 as c) tmp on duplicate key update val = tmp.c; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from t; +k1 k2 val +1 2 4 +drop table if exists t; +create table t(k1 double, k2 double, v double, primary key(k1, k2)); +insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c; +affected rows: 1 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from t; +k1 k2 v +1 2 3 +insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c; +affected rows: 0 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from t; +k1 k2 v +1 2 3 +drop table if exists t1, t2; +create table t1(id int, a int, b int); +insert into t1 values (1, 1, 1); +affected rows: 1 +info: +insert into t1 values (2, 2, 1); +affected rows: 1 +info: +insert into t1 values (3, 3, 1); +affected rows: 1 +info: +create table t2(a int primary key, b int, unique(b)); +insert into t2 select a, b from t1 order by id on duplicate key update a=t1.a, b=t1.b; +affected rows: 5 +info: Records: 3 Duplicates: 2 Warnings: 0 +select * from t2 order by a; +a b +3 1 +drop table if exists t1, t2; +create table t1(id int, a int, b int); +insert into t1 values (1, 1, 1); +affected rows: 1 +info: +insert into t1 values (2, 1, 2); +affected rows: 1 +info: +insert into t1 values (3, 3, 1); +affected rows: 1 +info: +create table t2(a int primary key, b int, unique(b)); +insert into t2 select a, b from t1 order by id on duplicate key update a=t1.a, b=t1.b; +affected rows: 4 +info: Records: 3 Duplicates: 1 Warnings: 0 +select * from t2 order by a; +a b +1 2 +3 1 +drop table if exists t1, t2; +create table t1(id int, a int, b int, c int); +insert into t1 values (1, 1, 1, 1); +affected rows: 1 +info: +insert into t1 values (2, 2, 1, 2); +affected rows: 1 +info: +insert into t1 values (3, 3, 2, 2); +affected rows: 1 +info: +insert into t1 values (4, 4, 2, 2); +affected rows: 1 +info: +create table t2(a int primary key, b int, c int, unique(b), unique(c)); +insert into t2 select a, b, c from t1 order by id on duplicate key update b=t2.b, c=t2.c; +affected rows: 2 +info: Records: 4 Duplicates: 0 Warnings: 0 +select * from t2 order by a; +a b c +1 1 1 +3 2 2 +drop table if exists t1; +create table t1(a int primary key, b int); +insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5); +affected rows: 5 +info: Records: 5 Duplicates: 0 Warnings: 0 +insert into t1 values(4,14),(5,15),(6,16),(7,17),(8,18) on duplicate key update b=b+10; +affected rows: 7 +info: Records: 5 Duplicates: 2 Warnings: 0 +drop table if exists a, b; +create table a(x int primary key); +create table b(x int, y int); +insert into a values(1); +affected rows: 1 +info: +insert into b values(1, 2); +affected rows: 1 +info: +insert into a select x from b ON DUPLICATE KEY UPDATE a.x=b.y; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +select * from a; +x +2 +## Test issue 28078. +## Use different types of columns so that there's likely to be error if the types mismatches. +drop table if exists a, b; +create table a(id int, a1 timestamp, a2 varchar(10), a3 float, unique(id)); +create table b(id int, b1 time, b2 varchar(10), b3 int); +insert into a values (1, '2022-01-04 07:02:04', 'a', 1.1), (2, '2022-01-04 07:02:05', 'b', 2.2); +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +insert into b values (2, '12:34:56', 'c', 10), (3, '01:23:45', 'd', 20); +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +insert into a (id) select id from b on duplicate key update a.a2 = b.b2, a.a3 = 3.3; +affected rows: 3 +info: Records: 2 Duplicates: 1 Warnings: 0 +select * from a; +id a1 a2 a3 +1 2022-01-04 07:02:04 a 1.1 +2 2022-01-04 07:02:05 c 3.3 +3 NULL NULL NULL +insert into a (id) select 4 from b where b3 = 20 on duplicate key update a.a3 = b.b3; +affected rows: 1 +info: Records: 1 Duplicates: 0 Warnings: 0 +select * from a; +id a1 a2 a3 +1 2022-01-04 07:02:04 a 1.1 +2 2022-01-04 07:02:05 c 3.3 +3 NULL NULL NULL +4 NULL NULL NULL +insert into a (a2, a3) select 'x', 1.2 from b on duplicate key update a.a2 = b.b3; +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +select * from a; +id a1 a2 a3 +1 2022-01-04 07:02:04 a 1.1 +2 2022-01-04 07:02:05 c 3.3 +3 NULL NULL NULL +4 NULL NULL NULL +NULL NULL x 1.2 +NULL NULL x 1.2 +## reproduce insert on duplicate key update bug under new row format. +drop table if exists t1; +create table t1(c1 decimal(6,4), primary key(c1)); +insert into t1 set c1 = 0.1; +insert into t1 set c1 = 0.1 on duplicate key update c1 = 1; +select * from t1 use index(primary); +c1 +1.0000 diff --git a/tests/integrationtest/r/executor/partition/write.result b/tests/integrationtest/r/executor/partition/write.result new file mode 100644 index 0000000000000..06c6839eef19c --- /dev/null +++ b/tests/integrationtest/r/executor/partition/write.result @@ -0,0 +1,814 @@ +# TestWriteListPartitionTable2 +# test for write list partition when the partition expression is complicated and contain generated column. +set @@session.tidb_enable_list_partition = ON; +drop table if exists t; +create table t (id int, name varchar(10),b int generated always as (length(name)+1) virtual) +partition by list (id*2 + b*b + b*b - b*b*2 - abs(id)) ( +partition p0 values in (3,5,6,9,17), +partition p1 values in (1,2,10,11,19,20), +partition p2 values in (4,12,13,14,18), +partition p3 values in (7,8,15,16,null) +); +analyze table t; +## Test add unique index failed. +insert into t (id,name) values (1, 'a'),(1,'b'); +alter table t add unique index idx (id,b); +Error 1062 (23000): Duplicate entry '1-2' for key 't.idx' +## Test add unique index success. +delete from t where name='b'; +alter table t add unique index idx (id,b); +## --------------------------Test insert--------------------------- +## Test insert 1 partition. +delete from t; +insert into t (id,name) values (1, 'a'),(2,'b'),(10,'c'); +select id,name from t partition(p1) order by id; +id name +1 a +2 b +10 c +## Test insert multi-partitions. +delete from t; +insert into t (id,name) values (1, 'a'),(3,'c'),(4,'e'); +select id,name from t partition(p0) order by id; +id name +3 c +select id,name from t partition(p1) order by id; +id name +1 a +select id,name from t partition(p2) order by id; +id name +4 e +select id,name from t partition(p3) order by id; +id name +## Test insert on duplicate. +insert into t (id,name) values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'; +select id,name from t partition(p0) order by id; +id name +3 x +5 g +select id,name from t partition(p1) order by id; +id name +1 x +select id,name from t partition(p2) order by id; +id name +4 e +select id,name from t partition(p3) order by id; +id name +## Test insert on duplicate error +insert into t (id,name) values (3, 'a'), (11,'x') on duplicate key update id=id+1; +Error 1062 (23000): Duplicate entry '4-2' for key 't.idx' +select id,name from t order by id; +id name +1 x +3 x +4 e +5 g +## Test insert ignore with duplicate +insert ignore into t (id,name) values (1, 'b'), (5,'a'),(null,'y'); +show warnings; +Level Code Message +Warning 1062 Duplicate entry '1-2' for key 't.idx' +Warning 1062 Duplicate entry '5-2' for key 't.idx' +select id,name from t partition(p0) order by id; +id name +3 x +5 g +select id,name from t partition(p1) order by id; +id name +1 x +select id,name from t partition(p2) order by id; +id name +4 e +select id,name from t partition(p3) order by id; +id name +NULL y +## Test insert ignore without duplicate +insert ignore into t (id,name) values (15, 'a'),(17,'a'); +select id,name from t partition(p0,p1,p2) order by id; +id name +1 x +3 x +4 e +5 g +17 a +select id,name from t partition(p3) order by id; +id name +NULL y +15 a +## Test insert meet no partition error. +insert into t (id,name) values (100, 'd'); +Error 1526 (HY000): Table has no partition for value 100 +## --------------------------Test update--------------------------- +## Test update 1 partition. +delete from t; +insert into t (id,name) values (1, 'a'),(2,'b'),(3,'c'); +update t set name='b' where id=2;; +select id,name from t partition(p1); +id name +1 a +2 b +update t set name='x' where id in (1,2); +select id,name from t partition(p1); +id name +1 x +2 x +update t set name='y' where id < 3; +select id,name from t order by id; +id name +1 y +2 y +3 c +## Test update meet duplicate error. +update t set id=2 where id = 1; +Error 1062 (23000): Duplicate entry '2-2' for key 't.idx' +select id,name from t order by id; +id name +1 y +2 y +3 c +## Test update multi-partitions +update t set name='z' where id in (1,2,3);; +select id,name from t order by id; +id name +1 z +2 z +3 z +update t set name='a' limit 3; +select id,name from t order by id; +id name +1 a +2 a +3 a +update t set id=id*10 where id in (1,2); +select id,name from t order by id; +id name +3 a +10 a +20 a +## Test update meet duplicate error. +update t set id=id+17 where id in (3,10); +Error 1062 (23000): Duplicate entry '20-2' for key 't.idx' +select id,name from t order by id; +id name +3 a +10 a +20 a +## Test update meet no partition error. +update t set id=id*2 where id in (3,20); +Error 1526 (HY000): Table has no partition for value 40 +select id,name from t order by id; +id name +3 a +10 a +20 a +## --------------------------Test replace--------------------------- +## Test replace 1 partition. +delete from t; +replace into t (id,name) values (1, 'a'),(2,'b'); +select id,name from t order by id; +id name +1 a +2 b +## Test replace multi-partitions. +replace into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +select id,name from t partition(p0) order by id; +id name +3 c +select id,name from t partition(p1) order by id; +id name +1 a +2 b +select id,name from t partition(p2) order by id; +id name +4 d +select id,name from t partition(p3) order by id; +id name +7 f +## Test replace on duplicate. +replace into t (id,name) values (1, 'x'),(7,'x'); +select id,name from t order by id; +id name +1 x +2 b +3 c +4 d +7 x +## Test replace meet no partition error. +replace into t (id,name) values (10,'x'),(50,'x'); +Error 1526 (HY000): Table has no partition for value 50 +select id,name from t order by id; +id name +1 x +2 b +3 c +4 d +7 x +## --------------------------Test delete--------------------------- +## Test delete 1 partition. +delete from t where id = 3; +select id,name from t partition(p0) order by id; +id name +delete from t where id in (1,2); +select id,name from t partition(p1) order by id; +id name +## Test delete multi-partitions. +delete from t where id in (4,7,10,11); +select id,name from t; +id name +insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +delete from t where id < 10; +select id,name from t; +id name +insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +delete from t limit 3; +select id,name from t; +id name +set @@session.tidb_enable_list_partition = default; +# TestWriteListColumnsPartitionTable1 +set @@session.tidb_enable_list_partition = ON; +drop table if exists t; +create table t (id int, name varchar(10)) partition by list columns (id) ( +partition p0 values in (3,5,6,9,17), +partition p1 values in (1,2,10,11,19,20), +partition p2 values in (4,12,13,14,18), +partition p3 values in (7,8,15,16,null) +); +analyze table t; +## Test add unique index failed. +insert into t values (1, 'a'),(1,'b'); +alter table t add unique index idx (id); +Error 1062 (23000): Duplicate entry '1' for key 't.idx' +## Test add unique index success. +delete from t where name='b'; +alter table t add unique index idx (id); +## --------------------------Test insert--------------------------- +## Test insert 1 partition. +delete from t; +insert into t values (1, 'a'),(2,'b'),(10,'c'); +select * from t partition(p1) order by id; +id name +1 a +2 b +10 c +## Test insert multi-partitions. +delete from t; +insert into t values (1, 'a'),(3,'c'),(4,'e'); +select * from t partition(p0) order by id; +id name +3 c +select * from t partition(p1) order by id; +id name +1 a +select * from t partition(p2) order by id; +id name +4 e +select * from t partition(p3) order by id; +id name +## Test insert on duplicate. +insert into t values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'; +select * from t partition(p0) order by id; +id name +3 x +5 g +select * from t partition(p1) order by id; +id name +1 x +select * from t partition(p2) order by id; +id name +4 e +select * from t partition(p3) order by id; +id name +## Test insert on duplicate error +insert into t values (3, 'a'), (11,'x') on duplicate key update id=id+1; +Error 1062 (23000): Duplicate entry '4' for key 't.idx' +select * from t order by id; +id name +1 x +3 x +4 e +5 g +## Test insert ignore with duplicate +insert ignore into t values (1, 'b'), (5,'a'),(null,'y'); +show warnings; +Level Code Message +Warning 1062 Duplicate entry '1' for key 't.idx' +Warning 1062 Duplicate entry '5' for key 't.idx' +select * from t partition(p0) order by id; +id name +3 x +5 g +select * from t partition(p1) order by id; +id name +1 x +select * from t partition(p2) order by id; +id name +4 e +select * from t partition(p3) order by id; +id name +NULL y +## Test insert ignore without duplicate +insert ignore into t values (15, 'a'),(17,'a'); +select * from t partition(p0,p1,p2) order by id; +id name +1 x +3 x +4 e +5 g +17 a +select * from t partition(p3) order by id; +id name +NULL y +15 a +## Test insert meet no partition error. +insert into t values (100, 'd'); +Error 1526 (HY000): Table has no partition for value from column_list +## --------------------------Test update--------------------------- +## Test update 1 partition. +delete from t; +insert into t values (1, 'a'),(2,'b'),(3,'c'); +update t set name='b' where id=2;; +select * from t partition(p1); +id name +1 a +2 b +update t set name='x' where id in (1,2); +select * from t partition(p1); +id name +1 x +2 x +update t set name='y' where id < 3; +select * from t order by id; +id name +1 y +2 y +3 c +## Test update meet duplicate error. +update t set id=2 where id = 1; +Error 1062 (23000): Duplicate entry '2' for key 't.idx' +select * from t order by id; +id name +1 y +2 y +3 c +## Test update multi-partitions +update t set name='z' where id in (1,2,3);; +select * from t order by id; +id name +1 z +2 z +3 z +update t set name='a' limit 3; +select * from t order by id; +id name +1 a +2 a +3 a +update t set id=id*10 where id in (1,2); +select * from t order by id; +id name +3 a +10 a +20 a +## Test update meet duplicate error. +update t set id=id+17 where id in (3,10); +Error 1062 (23000): Duplicate entry '20' for key 't.idx' +select * from t order by id; +id name +3 a +10 a +20 a +## Test update meet no partition error. +update t set id=id*2 where id in (3,20); +Error 1526 (HY000): Table has no partition for value from column_list +select * from t order by id; +id name +3 a +10 a +20 a +## --------------------------Test replace--------------------------- +## Test replace 1 partition. +delete from t; +replace into t values (1, 'a'),(2,'b'); +select * from t order by id; +id name +1 a +2 b +## Test replace multi-partitions. +replace into t values (3, 'c'),(4,'d'),(7,'f'); +select * from t partition(p0) order by id; +id name +3 c +select * from t partition(p1) order by id; +id name +1 a +2 b +select * from t partition(p2) order by id; +id name +4 d +select * from t partition(p3) order by id; +id name +7 f +## Test replace on duplicate. +replace into t values (1, 'x'),(7,'x'); +select * from t order by id; +id name +1 x +2 b +3 c +4 d +7 x +## Test replace meet no partition error. +replace into t values (10,'x'),(100,'x'); +Error 1526 (HY000): Table has no partition for value from column_list +select * from t order by id; +id name +1 x +2 b +3 c +4 d +7 x +## --------------------------Test delete--------------------------- +## Test delete 1 partition. +delete from t where id = 3; +select * from t partition(p0) order by id; +id name +delete from t where id in (1,2); +select * from t partition(p1) order by id; +id name +## Test delete multi-partitions. +delete from t where id in (4,7,10,11); +select * from t; +id name +insert into t values (3, 'c'),(4,'d'),(7,'f'); +delete from t where id < 10; +select * from t; +id name +insert into t values (3, 'c'),(4,'d'),(7,'f'); +delete from t limit 3; +select * from t; +id name +set @@session.tidb_enable_list_partition = default; +set tidb_opt_fix_control = "44262:ON"; +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) +partition by range (id) ( +PARTITION p0 VALUES LESS THAN (3), +PARTITION p1 VALUES LESS THAN (5), +PARTITION p2 VALUES LESS THAN (7), +PARTITION p3 VALUES LESS THAN (9)); +replace replace_test (c1) values (1),(2),(NULL); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +begin; +replace replace_test (c1) values (); +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +begin; +replace replace_test (c1, c2) values (1,2),(1); +Error 1136 (21S01): Column count doesn't match value count at row 2 +rollback; +begin; +replace replace_test (xxx) values (3); +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +begin; +replace replace_test_xxx (c1) values (); +Error 1146 (42S02): Table 'executor__partition__write.replace_test_xxx' doesn't exist +rollback; +replace replace_test set c1 = 3; +affected rows: 1 +info: +begin; +replace replace_test set c1 = 4, c1 = 5; +Error 1110 (42000): Column 'c1' specified twice +rollback; +begin; +replace replace_test set xxx = 6; +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int) partition by range (id) ( +PARTITION p0 VALUES LESS THAN (4), +PARTITION p1 VALUES LESS THAN (6), +PARTITION p2 VALUES LESS THAN (8), +PARTITION p3 VALUES LESS THAN (10), +PARTITION p4 VALUES LESS THAN (100)); +replace replace_test_1 select id, c1 from replace_test; +affected rows: 4 +info: Records: 4 Duplicates: 0 Warnings: 0 +drop table if exists replace_test_2; +create table replace_test_2 (id int, c1 int) partition by range (id) ( +PARTITION p0 VALUES LESS THAN (10), +PARTITION p1 VALUES LESS THAN (50), +PARTITION p2 VALUES LESS THAN (100), +PARTITION p3 VALUES LESS THAN (300)); +replace replace_test_2 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +affected rows: 8 +info: Records: 8 Duplicates: 0 Warnings: 0 +begin; +replace replace_test_2 select c1 from replace_test; +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by range (c2) ( +PARTITION p0 VALUES LESS THAN (4), +PARTITION p1 VALUES LESS THAN (7), +PARTITION p2 VALUES LESS THAN (11)); +replace into replace_test_3 set c2=8; +affected rows: 1 +info: +replace into replace_test_3 set c2=8; +affected rows: 1 +info: +replace into replace_test_3 set c1=8, c2=8; +affected rows: 2 +info: +replace into replace_test_3 set c2=NULL; +affected rows: 1 +info: +replace into replace_test_3 set c2=NULL; +affected rows: 1 +info: +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by range (c1) ( +PARTITION p0 VALUES LESS THAN (4), +PARTITION p1 VALUES LESS THAN (7), +PARTITION p2 VALUES LESS THAN (11)); +replace into replace_test_4 set c2=NULL; +affected rows: 1 +info: +replace into replace_test_4 set c2=NULL; +affected rows: 1 +info: +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by range (c2) ( +PARTITION p0 VALUES LESS THAN (4), +PARTITION p1 VALUES LESS THAN (7), +PARTITION p2 VALUES LESS THAN (11)); +replace into replace_test_5 set c1=1, c2=2; +affected rows: 1 +info: +replace into replace_test_5 set c1=1, c2=2; +affected rows: 1 +info: +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by range (b) ( +PARTITION p1 VALUES LESS THAN (100), +PARTITION p2 VALUES LESS THAN (200)); +insert into tIssue989 (a, b) values (1, 2); +affected rows: 1 +info: +replace into tIssue989(a, b) values (111, 2); +affected rows: 2 +info: +select * from tIssue989; +a b +111 2 +set tidb_opt_fix_control = default; +set tidb_opt_fix_control = "44262:ON"; +drop table if exists t; +create table t (id int not null default 1, name varchar(255)) +PARTITION BY RANGE ( id ) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11), +PARTITION p2 VALUES LESS THAN (16), +PARTITION p3 VALUES LESS THAN (21)); +insert INTO t VALUES (1, "hello"); +insert INTO t VALUES (7, "hello"); +## update non partition column +UPDATE t SET name = "abc" where id > 0; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +SELECT * from t order by id limit 2; +id name +1 abc +7 abc +## update partition column +update t set id = id + 1; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +SELECT * from t order by id limit 2; +id name +2 abc +8 abc +## update partition column, old and new record locates on different partitions +update t set id = 20 where id = 8; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +SELECT * from t order by id limit 2; +id name +2 abc +20 abc +## table option is auto-increment +drop table if exists t; +create table t (id int not null auto_increment, name varchar(255), primary key(id)) +PARTITION BY RANGE ( id ) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11), +PARTITION p2 VALUES LESS THAN (16), +PARTITION p3 VALUES LESS THAN (21)); +insert into t(name) values ('aa'); +update t set id = 8 where name = 'aa'; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +insert into t(name) values ('bb'); +select * from t; +id name +8 aa +9 bb +update t set id = null where name = 'aa'; +Error 1048 (23000): Column 'id' cannot be null +## Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. +drop table if exists t; +create table t (id int, name int unique) +PARTITION BY RANGE ( name ) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11), +PARTITION p2 VALUES LESS THAN (16), +PARTITION p3 VALUES LESS THAN (21)); +insert t values (1, 1), (2, 2); +update t set name = 1 where id = 2; +Error 1062 (23000): Duplicate entry '1' for key 't.name' +select * from t; +id name +1 1 +2 2 +## test update ignore for pimary key +drop table if exists t; +create table t(a bigint, primary key (a)) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11)); +insert into t values (5); +insert into t values (7); +update ignore t set a = 5 where a = 7; +SHOW WARNINGS; +Level Code Message +Warning 1062 Duplicate entry '5' for key 't.PRIMARY' +select * from t order by a; +a +5 +7 +## test update ignore for truncate as warning +update ignore t set a = 1 where a = (select '2a'); +SHOW WARNINGS; +Level Code Message +Warning 1292 Truncated incorrect DOUBLE value: '2a' +Warning 1292 Truncated incorrect DOUBLE value: '2a' +## test update ignore for unique key +drop table if exists t; +create table t(a bigint, unique key I_uniq (a)) +PARTITION BY RANGE (a) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11)); +insert into t values (5); +insert into t values (7); +update ignore t set a = 5 where a = 7; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 1 +SHOW WARNINGS; +Level Code Message +Warning 1062 Duplicate entry '5' for key 't.I_uniq' +select * from t order by a; +a +5 +7 +set tidb_opt_fix_control = default; +drop table if exists t; +set tidb_opt_fix_control = "44262:ON"; +CREATE TABLE t (id int not null default 1, name varchar(255), index(id)) +PARTITION BY RANGE ( id ) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11), +PARTITION p2 VALUES LESS THAN (16), +PARTITION p3 VALUES LESS THAN (21)); +insert into t values (1, "hello"),(2, "hello"),(3, "hello"),(4, "hello"),(5, "hello"),(6, "hello"),(7, "hello"),(8, "hello"),(9, "hello"),(10, "hello"),(11, "hello"),(12, "hello"),(13, "hello"),(14, "hello"),(15, "hello"),(16, "hello"),(17, "hello"),(18, "hello"),(19, "hello"),(20, "hello"); +delete from t where id = 2 limit 1; +affected rows: 1 +info: +## Test delete with false condition +delete from t where 0; +affected rows: 0 +info: +insert into t values (2, 'abc'); +delete from t where t.id = 2 limit 1; +affected rows: 1 +info: +## Test delete ignore +insert into t values (2, 'abc'); +delete from t where id = (select '2a'); +Error 1292 (22007): Truncated incorrect DOUBLE value: '2a' +delete ignore from t where id = (select '2a'); +affected rows: 1 +info: +SHOW WARNINGS; +Level Code Message +Warning 1292 Truncated incorrect DOUBLE value: '2a' +Warning 1292 Truncated incorrect DOUBLE value: '2a' +## Test delete without using index, involve multiple partitions. +delete from t ignore index(id) where id >= 13 and id <= 17; +affected rows: 5 +info: +admin check table t; +delete from t; +affected rows: 14 +info: +## Fix that partitioned table should not use PointGetPlan. +drop table if exists t1; +create table t1 (c1 bigint, c2 bigint, c3 bigint, primary key(c1)) partition by range (c1) (partition p0 values less than (3440)); +insert into t1 values (379, 379, 379); +delete from t1 where c1 = 379; +affected rows: 1 +info: +drop table t1; +set tidb_opt_fix_control=default; +set @@session.tidb_enable_table_partition = '1'; +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) +partition by hash(id) partitions 4; +replace replace_test (c1) values (1),(2),(NULL); +begin; +replace replace_test (c1) values (); +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +begin; +replace replace_test (c1, c2) values (1,2),(1); +Error 1136 (21S01): Column count doesn't match value count at row 2 +rollback; +begin; +replace replace_test (xxx) values (3); +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +begin; +replace replace_test_xxx (c1) values (); +Error 1146 (42S02): Table 'executor__partition__write.replace_test_xxx' doesn't exist +rollback; +begin; +replace replace_test set c1 = 4, c1 = 5; +Error 1110 (42000): Column 'c1' specified twice +rollback; +begin; +replace replace_test set xxx = 6; +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +replace replace_test set c1 = 3; +replace replace_test set c1 = 4; +replace replace_test set c1 = 5; +replace replace_test set c1 = 6; +replace replace_test set c1 = 7; +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int) partition by hash(id) partitions 5; +replace replace_test_1 select id, c1 from replace_test; +drop table if exists replace_test_2; +create table replace_test_2 (id int, c1 int) partition by hash(id) partitions 6; +replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +begin; +replace replace_test_1 select c1 from replace_test; +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by hash(c2) partitions 7; +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c2=8; +affected rows: 1 +info: +replace into replace_test_3 set c1=8, c2=8; +affected rows: 2 +info: +replace into replace_test_3 set c2=NULL; +replace into replace_test_3 set c2=NULL; +affected rows: 1 +info: +replace into replace_test_3 set c2=0; +replace into replace_test_3 set c2=1; +replace into replace_test_3 set c2=2; +replace into replace_test_3 set c2=3; +replace into replace_test_3 set c2=4; +replace into replace_test_3 set c2=5; +replace into replace_test_3 set c2=6; +replace into replace_test_3 set c2=7; +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c2=9; +select count(*) from replace_test_3; +count(*) +12 +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by hash(c1) partitions 8; +replace into replace_test_4 set c2=NULL; +replace into replace_test_4 set c2=NULL; +affected rows: 1 +info: +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by hash (c2) partitions 9; +replace into replace_test_5 set c1=1, c2=2; +replace into replace_test_5 set c1=1, c2=2; +affected rows: 1 +info: +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by hash (b) partitions 10; +insert into tIssue989 (a, b) values (1, 2); +replace into tIssue989(a, b) values (111, 2); +select * from tIssue989; +a b +111 2 +set @@session.tidb_enable_table_partition = default; diff --git a/tests/integrationtest/r/executor/stale_txn.result b/tests/integrationtest/r/executor/stale_txn.result index d1e7c85a2829b..5b4498b73834c 100644 --- a/tests/integrationtest/r/executor/stale_txn.result +++ b/tests/integrationtest/r/executor/stale_txn.result @@ -36,3 +36,9 @@ id v set tidb_txn_mode = default; set tx_isolation = default; set autocommit = default; +drop table if exists t1; +create table t1 (id int primary key, v int); +select * from t1 as of timestamp NULL; +Error 8135 (HY000): invalid as of timestamp: as of timestamp cannot be NULL +start transaction read only as of timestamp NULL; +Error 8135 (HY000): invalid as of timestamp: as of timestamp cannot be NULL diff --git a/tests/integrationtest/r/executor/update.result b/tests/integrationtest/r/executor/update.result index 5accbf8f0f70e..5b11c91279039 100644 --- a/tests/integrationtest/r/executor/update.result +++ b/tests/integrationtest/r/executor/update.result @@ -384,3 +384,510 @@ id a 1 0 2 4294967295 set sql_mode=default; +drop table if exists t1; +create table t1(id int primary key, name varchar(40)); +insert into t1 values(1, 'abc'); +begin pessimistic; +begin pessimistic; +update t1 set name='xyz' where id=1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +select * from t1 where id = 1; +id name +1 xyz +commit; +update t1 set name='xyz' where id=1; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 0 +select * from t1 where id = 1; +id name +1 abc +select * from t1 where id = 1 for update; +id name +1 xyz +select * from t1 where id in (1, 2); +id name +1 abc +select * from t1 where id in (1, 2) for update; +id name +1 xyz +commit; +drop table if exists update_test; +create table update_test(id int not null default 1, name varchar(255), PRIMARY KEY(id)); +insert INTO update_test VALUES (1, "hello"); +insert into update_test values (2, "hello"); +UPDATE update_test SET name = "abc" where id > 0; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +begin; +SELECT * from update_test limit 2; +id name +1 abc +2 abc +commit; +UPDATE update_test SET name = "foo"; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +begin; +drop table if exists update_test; +commit; +begin; +create table update_test(id int not null auto_increment, name varchar(255), primary key(id)); +insert into update_test(name) values ('aa'); +update update_test set id = 8 where name = 'aa'; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +insert into update_test(name) values ('bb'); +commit; +begin; +select * from update_test; +id name +8 aa +9 bb +commit; +begin; +drop table if exists update_test; +commit; +begin; +create table update_test(id int not null auto_increment, name varchar(255), index(id)); +insert into update_test(name) values ('aa'); +update update_test set id = null where name = 'aa'; +Error 1048 (23000): Column 'id' cannot be null +drop table update_test; +create table update_test(id int); +begin; +insert into update_test(id) values (1); +update update_test set id = 2 where id = 1 limit 1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +select * from update_test; +id +2 +commit; +drop table if exists update_unique; +create table update_unique (id int primary key, name int unique); +insert update_unique values (1, 1), (2, 2); +begin; +update update_unique set name = 1 where id = 2; +Error 1062 (23000): Duplicate entry '1' for key 'update_unique.name' +commit; +select * from update_unique; +id name +1 1 +2 2 +drop table if exists t; +create table t(a bigint, primary key (a)); +insert into t values (1); +insert into t values (2); +update ignore t set a = 1 where a = 2; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 1 +SHOW WARNINGS; +Level Code Message +Warning 1062 Duplicate entry '1' for key 't.PRIMARY' +select * from t; +a +1 +2 +update ignore t set a = 1 where a = (select '2a'); +SHOW WARNINGS; +Level Code Message +Warning 1292 Truncated incorrect DOUBLE value: '2a' +Warning 1292 Truncated incorrect DOUBLE value: '2a' +Warning 1062 Duplicate entry '1' for key 't.PRIMARY' +update ignore t set a = 42 where a = 2; +select * from t; +a +1 +42 +drop table if exists t; +create table t(a bigint, unique key I_uniq (a)); +insert into t values (1); +insert into t values (2); +update ignore t set a = 1 where a = 2; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 1 +SHOW WARNINGS; +Level Code Message +Warning 1062 Duplicate entry '1' for key 't.I_uniq' +select * from t; +a +1 +2 +drop table if exists t; +set @@session.tidb_enable_list_partition = ON; +create table t (a int) partition by list (a) (partition p0 values in (0,1)); +analyze table t; +insert ignore into t values (1); +update ignore t set a=2 where a=1; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 0 +drop table if exists t; +create table t (a int key) partition by list (a) (partition p0 values in (0,1)); +insert ignore into t values (1); +update ignore t set a=2 where a=1; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 0 +set @@session.tidb_enable_list_partition = default; +drop table if exists t; +create table t(id integer auto_increment, t1 datetime, t2 datetime, primary key (id)); +insert into t(t1, t2) values('2000-10-01 01:01:01', '2017-01-01 10:10:10'); +select * from t; +id t1 t2 +1 2000-10-01 01:01:01 2017-01-01 10:10:10 +update t set t1 = '2017-10-01 10:10:11', t2 = date_add(t1, INTERVAL 10 MINUTE) where id = 1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +select * from t; +id t1 t2 +1 2017-10-01 10:10:11 2000-10-01 01:11:01 +drop table if exists tt1; +CREATE TABLE `tt1` (`a` int(11) NOT NULL,`b` varchar(32) DEFAULT NULL,`c` varchar(32) DEFAULT NULL,PRIMARY KEY (`a`),UNIQUE KEY `b_idx` (`b`)) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; +insert into tt1 values(1, 'a', 'a'); +insert into tt1 values(2, 'd', 'b'); +select * from tt1; +a b c +1 a a +2 d b +update tt1 set a=5 where c='b'; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +select * from tt1; +a b c +1 a a +5 d b +drop table if exists tsup; +CREATE TABLE `tsup` (`a` int,`ts` TIMESTAMP DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,KEY `idx` (`ts`)); +set @@sql_mode=''; +insert into tsup values(1, '0000-00-00 00:00:00'); +update tsup set a=5; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +select t1.ts = t2.ts from (select ts from tsup use index (idx)) as t1, (select ts from tsup use index ()) as t2; +t1.ts = t2.ts +1 +update tsup set ts='2019-01-01'; +select ts from tsup; +ts +2019-01-01 00:00:00 +set @@sql_mode=default; +drop table if exists decimals; +create table decimals (a decimal(20, 0) not null); +insert into decimals values (201); +update decimals set a = a + 1.23; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 1 +show warnings; +Level Code Message +Warning 1292 Truncated incorrect DECIMAL value: '202.23' +select * from decimals; +a +202 +drop table t; +CREATE TABLE `t` ( `c1` year DEFAULT NULL, `c2` year DEFAULT NULL, `c3` date DEFAULT NULL, `c4` datetime DEFAULT NULL, KEY `idx` (`c1`,`c2`)); +UPDATE t SET c2=16777215 WHERE c1>= -8388608 AND c1 < -9 ORDER BY c1 LIMIT 2; +update (select * from t) t set c1 = 1111111; +Error 1288 (HY000): The target table t of the UPDATE is not updatable +drop table if exists t; +create table t (i int not null default 10); +insert into t values (1); +update ignore t set i = null; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 1 +SHOW WARNINGS; +Level Code Message +Warning 1048 Column 'i' cannot be null +select * from t; +i +0 +drop table t; +create table t (k int, v int); +update t, (select * from t) as b set b.k = t.k; +Error 1288 (HY000): The target table b of the UPDATE is not updatable +update t, (select * from t) as b set t.k = b.k; +drop table if exists t1; +CREATE TABLE t1 (c1 float); +INSERT INTO t1 SET c1 = 1; +UPDATE t1 SET c1 = 1.2 WHERE c1=1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +drop table if exists t; +create table t (c1 float(1,1)); +insert into t values (0.0); +update t set c1 = 2.0; +Error 1264 (22003): Out of range value for column 'c1' at row 1 +drop table if exists t; +create table t(a datetime not null, b datetime); +insert into t value('1999-12-12', '1999-12-13'); +set @@sql_mode=''; +select * from t; +a b +1999-12-12 00:00:00 1999-12-13 00:00:00 +update t set a = ''; +select * from t; +a b +0000-00-00 00:00:00 1999-12-13 00:00:00 +update t set b = ''; +select * from t; +a b +0000-00-00 00:00:00 0000-00-00 00:00:00 +set @@sql_mode=default; +drop view if exists v; +create view v as select * from t; +update v set a = '2000-11-11'; +Error 1288 (HY000): The target table v of the UPDATE is not updatable +drop view v; +drop sequence if exists seq; +create sequence seq; +update seq set minvalue=1; +Error 1054 (42S22): Unknown column 'minvalue' in 'field list' +drop sequence seq; +drop table if exists t1, t2; +create table t1(a int, b int, c int, d int, e int, index idx(a)); +create table t2(a int, b int, c int); +update t1 join t2 on t1.a=t2.a set t1.a=1 where t2.b=1 and t2.c=2; +drop table if exists t1, t2; +create table t1 (a int default 1, b int default 2); +insert into t1 values (10, 10), (20, 20); +update t1 set a=default where b=10; +select * from t1; +a b +1 10 +20 20 +update t1 set a=30, b=default where a=20; +select * from t1; +a b +1 10 +30 2 +update t1 set a=default, b=default where a=30; +select * from t1; +a b +1 10 +1 2 +insert into t1 values (40, 40); +update t1 set a=default, b=default; +select * from t1; +a b +1 2 +1 2 +1 2 +update t1 set a=default(b), b=default(a); +select * from t1; +a b +2 1 +2 1 +2 1 +create table t2 (a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored); +insert into t2 values (10, default, default), (20, default, default); +update t2 set b=default; +select * from t2; +a b c +10 -10 -10 +20 -20 -20 +update t2 set a=30, b=default where a=10; +select * from t2; +a b c +30 -30 -30 +20 -20 -20 +update t2 set c=default, a=40 where c=-20; +select * from t2; +a b c +30 -30 -30 +40 -40 -40 +update t2 set a=default, b=default, c=default where b=-30; +select * from t2; +a b c +1 -1 -1 +40 -40 -40 +update t2 set a=default(a), b=default, c=default; +select * from t2; +a b c +1 -1 -1 +1 -1 -1 +update t2 set a=default(b), b=default, c=default; +select * from t2; +a b c +NULL NULL NULL +NULL NULL NULL +update t2 set b=default(a); +Error 3105 (HY000): The value specified for generated column 'b' in table 't2' is not allowed. +update t2 set a=default(a), c=default(c); +select * from t2; +a b c +1 -1 -1 +1 -1 -1 +update t2 set a=default(b), b=default(b); +select * from t2; +a b c +NULL NULL NULL +NULL NULL NULL +update t2 set a=default(a), c=default(c); +select * from t2; +a b c +1 -1 -1 +1 -1 -1 +update t2 set a=default(a), c=default(a); +Error 3105 (HY000): The value specified for generated column 'c' in table 't2' is not allowed. +drop table t1, t2; +drop table if exists msg, detail; +create table msg (id varchar(8), b int, status int, primary key (id, b)); +insert msg values ('abc', 1, 1); +create table detail (id varchar(8), start varchar(8), status int, index idx_start(start)); +insert detail values ('abc', '123', 2); +UPDATE msg SET msg.status = (SELECT detail.status FROM detail WHERE msg.id = detail.id); +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +admin check table msg; +drop table if exists ttt; +CREATE TABLE ttt (id bigint(20) NOT NULL, host varchar(30) NOT NULL, PRIMARY KEY (id), UNIQUE KEY i_host (host)); +insert into ttt values (8,8),(9,9); +begin; +update ttt set id = 0, host='9' where id = 9 limit 1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +delete from ttt where id = 0 limit 1; +select * from ttt use index (i_host) order by host; +id host +8 8 +update ttt set id = 0, host='8' where id = 8 limit 1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +delete from ttt where id = 0 limit 1; +select * from ttt use index (i_host) order by host; +id host +commit; +admin check table ttt; +drop table ttt; +drop table if exists a; +create table a(id int auto_increment, a int default null, primary key(id)); +insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1); +update a set id = id*10 where a = 1001; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +drop table a; +create table a ( a bigint, b bigint); +insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1); +update a set a = a*10 where b = 1001; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +drop table if exists items, month; +CREATE TABLE items (id int, price TEXT); +insert into items values (11, "items_price_11"), (12, "items_price_12"), (13, "items_price_13"); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +CREATE TABLE month (mid int, mprice TEXT); +insert into month values (11, "month_price_11"), (22, "month_price_22"), (13, "month_price_13"); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +UPDATE items, month SET items.price=month.mprice WHERE items.id=month.mid; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +begin; +SELECT * FROM items; +id price +11 month_price_11 +12 items_price_12 +13 month_price_13 +commit; +UPDATE items join month on items.id=month.mid SET items.price=month.mid; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +begin; +SELECT * FROM items; +id price +11 11 +12 items_price_12 +13 13 +commit; +UPDATE items T0 join month T1 on T0.id=T1.mid SET T0.price=T1.mprice; +affected rows: 2 +info: Rows matched: 2 Changed: 2 Warnings: 0 +begin; +SELECT * FROM items; +id price +11 month_price_11 +12 items_price_12 +13 month_price_13 +commit; +DROP TABLE IF EXISTS t1, t2; +create table t1 (c int); +create table t2 (c varchar(256)); +insert into t1 values (1), (2); +insert into t2 values ("a"), ("b"); +update t1, t2 set t1.c = 10, t2.c = "abc"; +affected rows: 4 +info: Rows matched: 4 Changed: 4 Warnings: 0 +DROP TABLE IF EXISTS t1, t2; +create table t1 (c1 int); +create table t2 (c2 int); +insert into t1 values (1), (2); +insert into t2 values (1), (2); +update t1, t2 set t1.c1 = 10, t2.c2 = 2 where t2.c2 = 1; +affected rows: 3 +info: Rows matched: 3 Changed: 3 Warnings: 0 +select * from t1; +c1 +10 +10 +drop table if exists t; +create table t (a int, b int); +insert into t values(1, 1), (2, 2), (3, 3); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +update t m, t n set m.a = m.a + 1; +affected rows: 3 +info: Rows matched: 3 Changed: 3 Warnings: 0 +select * from t; +a b +2 1 +3 2 +4 3 +update t m, t n set n.a = n.a - 1, n.b = n.b + 1; +affected rows: 3 +info: Rows matched: 3 Changed: 3 Warnings: 0 +select * from t; +a b +1 2 +2 3 +3 4 +drop table if exists update_modified; +create table update_modified (col_1 int, col_2 enum('a', 'b')); +set SQL_MODE=''; +insert into update_modified values (0, 3); +SELECT * FROM update_modified; +col_1 col_2 +0 +set SQL_MODE=STRICT_ALL_TABLES; +update update_modified set col_1 = 1; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +SELECT * FROM update_modified; +col_1 col_2 +1 +update update_modified set col_1 = 2, col_2 = 'c'; +Error 1265 (01000): Data truncated for column '%s' at row %d +SELECT * FROM update_modified; +col_1 col_2 +1 +update update_modified set col_1 = 3, col_2 = 'a'; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +SELECT * FROM update_modified; +col_1 col_2 +3 a +drop table if exists update_with_diff_type; +CREATE TABLE update_with_diff_type (a int, b JSON); +INSERT INTO update_with_diff_type VALUES(3, '{"a": "测试"}'); +UPDATE update_with_diff_type SET a = '300'; +affected rows: 1 +info: Rows matched: 1 Changed: 1 Warnings: 0 +SELECT a FROM update_with_diff_type; +a +300 +UPDATE update_with_diff_type SET b = '{"a": "\\u6d4b\\u8bd5"}'; +affected rows: 0 +info: Rows matched: 1 Changed: 0 Warnings: 0 +SELECT b FROM update_with_diff_type; +b +{"a": "测试"} +set SQL_MODE=default; diff --git a/tests/integrationtest/r/executor/write.result b/tests/integrationtest/r/executor/write.result index 1217d60a5b38c..70bfb62329b75 100644 --- a/tests/integrationtest/r/executor/write.result +++ b/tests/integrationtest/r/executor/write.result @@ -1974,3 +1974,175 @@ b 4 set @@session.tidb_enable_list_partition = default; set @@allow_auto_random_explicit_insert = default; +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1); +replace replace_test (c1) values (1),(2),(NULL); +affected rows: 3 +info: Records: 3 Duplicates: 0 Warnings: 0 +begin; +replace replace_test (c1) values (); +Error 1136 (21S01): Column count doesn't match value count at row 1 +rollback; +begin; +replace replace_test (c1, c2) values (1,2),(1); +Error 1136 (21S01): Column count doesn't match value count at row 2 +rollback; +begin; +replace replace_test (xxx) values (3); +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +begin; +replace replace_test_xxx (c1) values (); +Error 1146 (42S02): Table 'executor__write.replace_test_xxx' doesn't exist +rollback; +replace replace_test set c1 = 3; +affected rows: 1 +info: +begin; +replace replace_test set c1 = 4, c1 = 5; +Error 1110 (42000): Column 'c1' specified twice +rollback; +begin; +replace replace_test set xxx = 6; +Error 1054 (42S22): Unknown column 'xxx' in 'field list' +rollback; +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int); +replace replace_test_1 select id, c1 from replace_test; +affected rows: 4 +info: Records: 4 Duplicates: 0 Warnings: 0 +begin; +replace replace_test_0 select c1 from replace_test; +Error 1146 (42S02): Table 'executor__write.replace_test_0' doesn't exist +rollback; +create table replace_test_2 (id int, c1 int); +replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +affected rows: 8 +info: Records: 8 Duplicates: 0 Warnings: 0 +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)); +replace into replace_test_3 set c2=1; +affected rows: 1 +info: +replace into replace_test_3 set c2=1; +affected rows: 1 +info: +replace into replace_test_3 set c1=1, c2=1; +affected rows: 2 +info: +replace into replace_test_3 set c2=NULL; +affected rows: 1 +info: +replace into replace_test_3 set c2=NULL; +affected rows: 1 +info: +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)); +replace into replace_test_4 set c2=NULL; +affected rows: 1 +info: +replace into replace_test_4 set c2=NULL; +affected rows: 1 +info: +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)); +replace into replace_test_5 set c1=1, c2=2; +affected rows: 1 +info: +replace into replace_test_5 set c1=1, c2=2; +affected rows: 1 +info: +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b)); +insert into tIssue989 (a, b) values (1, 2); +affected rows: 1 +info: +replace into tIssue989(a, b) values (111, 2); +affected rows: 2 +info: +select * from tIssue989; +a b +111 2 +drop table if exists tIssue1012; +CREATE TABLE tIssue1012 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b)); +insert into tIssue1012 (a, b) values (1, 2); +insert into tIssue1012 (a, b) values (2, 1); +replace into tIssue1012(a, b) values (1, 1); +affected rows: 3 +info: +select * from tIssue1012; +a b +1 1 +drop table if exists t1; +create table t1(a int primary key, b int); +insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5); +replace into t1 values(1,1); +affected rows: 1 +info: +replace into t1 values(1,1),(2,2); +affected rows: 2 +info: Records: 2 Duplicates: 0 Warnings: 0 +replace into t1 values(4,14),(5,15),(6,16),(7,17),(8,18); +affected rows: 7 +info: Records: 5 Duplicates: 2 Warnings: 0 +replace into t1 select * from (select 1, 2) as tmp; +affected rows: 2 +info: Records: 1 Duplicates: 1 Warnings: 0 +drop table if exists t1, t2; +create table t1 (a int primary key, b int default 20, c int default 30); +insert into t1 value (1, 2, 3); +replace t1 set a=1, b=default; +select * from t1; +a b c +1 20 30 +replace t1 set a=2, b=default, c=default; +select * from t1; +a b c +1 20 30 +2 20 30 +replace t1 set a=2, b=default(c), c=default(b); +select * from t1; +a b c +1 20 30 +2 30 20 +replace t1 set a=default(b)+default(c); +select * from t1; +a b c +1 20 30 +2 30 20 +50 20 30 +create table t2 (pk int primary key, a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored); +replace t2 set pk=1, b=default; +select * from t2; +pk a b c +1 1 -1 -1 +replace t2 set pk=2, a=10, b=default; +select * from t2; +pk a b c +1 1 -1 -1 +2 10 -10 -10 +replace t2 set pk=2, c=default, a=20; +select * from t2; +pk a b c +1 1 -1 -1 +2 20 -20 -20 +replace t2 set pk=2, a=default, b=default, c=default; +select * from t2; +pk a b c +1 1 -1 -1 +2 1 -1 -1 +replace t2 set pk=3, a=default(a), b=default, c=default; +select * from t2; +pk a b c +1 1 -1 -1 +2 1 -1 -1 +3 1 -1 -1 +replace t2 set b=default(a); +Error 3105 (HY000): The value specified for generated column 'b' in table 't2' is not allowed. +replace t2 set a=default(b), b=default(b); +Error 3105 (HY000): The value specified for generated column 'a' in table 't2' is not allowed. +replace t2 set a=default(a), c=default(c); +Error 1364 (HY000): Field 'pk' doesn't have a default value +replace t2 set c=default(a); +Error 3105 (HY000): The value specified for generated column 'c' in table 't2' is not allowed. +drop table t1, t2; diff --git a/tests/integrationtest/r/planner/core/binary_plan.result b/tests/integrationtest/r/planner/core/binary_plan.result new file mode 100644 index 0000000000000..776df6e6ec382 --- /dev/null +++ b/tests/integrationtest/r/planner/core/binary_plan.result @@ -0,0 +1,18 @@ +select tidb_decode_binary_plan('some random bytes'); +tidb_decode_binary_plan('some random bytes') + +show warnings; +Level Code Message +Warning 1105 illegal base64 data at input byte 4 +select tidb_decode_binary_plan('c29tZSByYW5kb20gYnl0ZXM='); +tidb_decode_binary_plan('c29tZSByYW5kb20gYnl0ZXM=') + +show warnings; +Level Code Message +Warning 1105 snappy: corrupt input +select tidb_decode_binary_plan('EUBzb21lIHJhbmRvbSBieXRlcw=='); +tidb_decode_binary_plan('EUBzb21lIHJhbmRvbSBieXRlcw==') + +show warnings; +Level Code Message +Warning 1105 proto: illegal wireType 7 diff --git a/tests/integrationtest/r/planner/core/cbo.result b/tests/integrationtest/r/planner/core/cbo.result index f9d2ff5a8a726..230e53fe949ca 100644 --- a/tests/integrationtest/r/planner/core/cbo.result +++ b/tests/integrationtest/r/planner/core/cbo.result @@ -69,14 +69,14 @@ insert into t2 values (2, 22), (3, 33), (5, 55), (233, 2), (333, 3), (3434, 5); analyze table t1, t2; explain analyze select t1.a, t1.b, sum(t1.c) from t1 join t2 on t1.a = t2.b where t1.a > 1; id estRows actRows task access object execution info operator info memory disk -Projection_9 1.00 1 root NULL time:, loops:, RU:, Concurrency:OFF planner__core__cbo.t1.a, planner__core__cbo.t1.b, Column#8 KB N/A -└─StreamAgg_11 1.00 1 root NULL time:, loops: funcs:sum(Column#16)->Column#8, funcs:firstrow(Column#17)->planner__core__cbo.t1.a, funcs:firstrow(Column#18)->planner__core__cbo.t1.b KB N/A - └─Projection_53 4.00 3 root NULL time:, loops:, Concurrency:OFF cast(planner__core__cbo.t1.c, decimal(10,0) BINARY)->Column#16, planner__core__cbo.t1.a->Column#17, planner__core__cbo.t1.b->Column#18 KB N/A - └─HashJoin_51 4.00 3 root NULL time:, loops:, build_hash_table:{total:, fetch:, build:}, probe:{concurrency:, total:, max:, probe:, fetch:} inner join, equal:[eq(planner__core__cbo.t1.a, planner__core__cbo.t2.b)] KB Bytes - ├─TableReader_30(Build) 6.00 6 root NULL time:, loops:, cop_task: {num:, max:, proc_keys:, rpc_num:, rpc_time:, copr_cache_hit_ratio:, build_task_duration:, max_distsql_concurrency:} data:Selection_29 Bytes N/A +Projection_9 1.00 1 root NULL time:, loops:, RU:, Concurrency:OFF planner__core__cbo.t1.a, planner__core__cbo.t1.b, Column#8 N/A +└─StreamAgg_11 1.00 1 root NULL time:, loops: funcs:sum(Column#16)->Column#8, funcs:firstrow(Column#17)->planner__core__cbo.t1.a, funcs:firstrow(Column#18)->planner__core__cbo.t1.b N/A + └─Projection_53 4.00 3 root NULL time:, loops:, Concurrency:OFF cast(planner__core__cbo.t1.c, decimal(10,0) BINARY)->Column#16, planner__core__cbo.t1.a->Column#17, planner__core__cbo.t1.b->Column#18 N/A + └─HashJoin_51 4.00 3 root NULL time:, loops:, build_hash_table:{total:, fetch:, build:}, probe:{concurrency:, total:, max:, probe:, fetch:} inner join, equal:[eq(planner__core__cbo.t1.a, planner__core__cbo.t2.b)] + ├─TableReader_30(Build) 6.00 6 root NULL time:, loops:, cop_task: {num:, max:, proc_keys:, rpc_num:, rpc_time:, copr_cache_hit_ratio:, build_task_duration:, max_distsql_concurrency:} data:Selection_29 N/A │ └─Selection_29 6.00 6 cop[tikv] NULL tikv_task:{time:, loops:} gt(planner__core__cbo.t2.b, 1), not(isnull(planner__core__cbo.t2.b)) N/A N/A │ └─TableFullScan_28 6.00 6 cop[tikv] table:t2 tikv_task:{time:, loops:} keep order:false N/A N/A - └─TableReader_33(Probe) 4.00 4 root NULL time:, loops:, cop_task: {num:, max:, proc_keys:, rpc_num:, rpc_time:, copr_cache_hit_ratio:, build_task_duration:, max_distsql_concurrency:} data:Selection_32 Bytes N/A + └─TableReader_33(Probe) 4.00 4 root NULL time:, loops:, cop_task: {num:, max:, proc_keys:, rpc_num:, rpc_time:, copr_cache_hit_ratio:, build_task_duration:, max_distsql_concurrency:} data:Selection_32 N/A └─Selection_32 4.00 4 cop[tikv] NULL tikv_task:{time:, loops:} gt(planner__core__cbo.t1.a, 1), not(isnull(planner__core__cbo.t1.a)) N/A N/A └─TableFullScan_31 5.00 5 cop[tikv] table:t1 tikv_task:{time:, loops:} keep order:false N/A N/A set sql_mode=default; diff --git a/tests/integrationtest/r/planner/core/plan.result b/tests/integrationtest/r/planner/core/plan.result index 7a49cd0486ea4..76bee6627810d 100644 --- a/tests/integrationtest/r/planner/core/plan.result +++ b/tests/integrationtest/r/planner/core/plan.result @@ -392,3 +392,35 @@ explain select a from t; id estRows task access object operator info IndexReader_7 10000.00 root index:IndexFullScan_6 └─IndexFullScan_6 10000.00 cop[tikv] table:t, index:hypo_a(a) keep order:false, stats:pseudo +drop table if exists t; +CREATE TABLE t (c1 mediumint(9) DEFAULT '-4747160',c2 year(4) NOT NULL DEFAULT '2075',c3 double DEFAULT '1.1559030660251948',c4 enum('wbv4','eli','d8ym','m3gsx','lz7td','o','d1k7l','y1x','xcxq','bj','n7') DEFAULT 'xcxq',c5 int(11) DEFAULT '255080866',c6 tinyint(1) DEFAULT '1',PRIMARY KEY (c2),KEY `c4d86d54-091c-4307-957b-b164c9652b7f` (c6,c4) ); +insert into t values (-4747160, 2075, 722.5719203870632, 'xcxq', 1576824797, 1); +select /*+ stream_agg() */ bit_or(t.c5) as r0 from t where t.c3 in (select c6 from t where not(t.c6 <> 1) and not(t.c3 in(9263.749352636818))) group by t.c1; +r0 +affected rows: 0 +info: +drop table if exists t1; +drop table if exists t2; +CREATE TABLE `t1`(`c1` bigint(20) NOT NULL DEFAULT '-2312745469307452950', `c2` datetime DEFAULT '5316-02-03 06:54:49', `c3` tinyblob DEFAULT NULL, PRIMARY KEY (`c1`) /*T![clustered_index] CLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; +CREATE TABLE `t2`(`c1` set('kn8pu','7et','vekx6','v3','liwrh','q14','1met','nnd5i','5o0','8cz','l') DEFAULT '7et,vekx6,liwrh,q14,1met', `c2` float DEFAULT '1.683167', KEY `k1` (`c2`,`c1`), KEY `k2` (`c2`)) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_chinese_ci; +(select /*+ agg_to_cop()*/ locate(t1.c3, t1.c3) as r0, t1.c3 as r1 from t1 where not( IsNull(t1.c1)) order by r0,r1) union all (select concat_ws(',', t2.c2, t2.c1) as r0, t2.c1 as r1 from t2 order by r0, r1) order by 1 limit 273; +r0 r1 +affected rows: 0 +info: +drop table if exists golang1, golang2; +CREATE TABLE golang1 ( `fcbpdt` CHAR (8) COLLATE utf8_general_ci NOT NULL, `fcbpsq` VARCHAR (20) COLLATE utf8_general_ci NOT NULL, `procst` char (4) COLLATE utf8_general_ci DEFAULT NULL,`cipstx` VARCHAR (105) COLLATE utf8_general_ci DEFAULT NULL, `cipsst` CHAR (4) COLLATE utf8_general_ci DEFAULT NULL, `dyngtg` VARCHAR(4) COLLATE utf8_general_ci DEFAULT NULL, `blncdt` VARCHAR (8) COLLATE utf8_general_ci DEFAULT NULL, PRIMARY KEY ( fcbpdt, fcbpsq )); +insert into golang1 values('20230925','12023092502158016','abc','','','',''); +create table golang2 (`sysgrp` varchar(20) NOT NULL,`procst` varchar(8) NOT NULL,`levlid` int(11) NOT NULL,PRIMARY key (procst));; +insert into golang2 VALUES('COMMON','ACSC',90); +insert into golang2 VALUES('COMMON','abc',8); +insert into golang2 VALUES('COMMON','CH02',6); +UPDATE golang1 a SET procst =(CASE WHEN ( SELECT levlid FROM golang2 b WHERE b.sysgrp = 'COMMON' AND b.procst = 'ACSC' ) > ( SELECT levlid FROM golang2 c WHERE c.sysgrp = 'COMMON' AND c.procst = a.procst ) THEN 'ACSC' ELSE a.procst END ), cipstx = 'CI010000', cipsst = 'ACSC', dyngtg = 'EAYT', blncdt= '20230925' WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'; +select * from golang1; +fcbpdt fcbpsq procst cipstx cipsst dyngtg blncdt +20230925 12023092502158016 ACSC CI010000 ACSC EAYT 20230925 +UPDATE golang1 a SET procst= (SELECT 1 FROM golang2 c WHERE c.procst = a.procst) WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'; +select * from golang1; +fcbpdt fcbpsq procst cipstx cipsst dyngtg blncdt +20230925 12023092502158016 1 CI010000 ACSC EAYT 20230925 +EXPLAIN FORMAT = TRADITIONAL ((VALUES ROW ()) ORDER BY 1); +Error 1051 (42S02): Unknown table '' diff --git a/tests/integrationtest/r/planner/core/plan_cache.result b/tests/integrationtest/r/planner/core/plan_cache.result index 69db6972f45d8..cfcc58922905a 100644 --- a/tests/integrationtest/r/planner/core/plan_cache.result +++ b/tests/integrationtest/r/planner/core/plan_cache.result @@ -2249,3 +2249,350 @@ select @@last_plan_from_cache; @@last_plan_from_cache 0 set tidb_enable_non_prepared_plan_cache=DEFAULT; +select @@tidb_prepared_plan_cache_size; +@@tidb_prepared_plan_cache_size +100 +select @@tidb_session_plan_cache_size; +@@tidb_session_plan_cache_size +100 +set @@tidb_prepared_plan_cache_size = 200; +select @@tidb_prepared_plan_cache_size; +@@tidb_prepared_plan_cache_size +200 +select @@tidb_session_plan_cache_size; +@@tidb_session_plan_cache_size +200 +set @@tidb_session_plan_cache_size = 300; +select @@tidb_prepared_plan_cache_size; +@@tidb_prepared_plan_cache_size +300 +select @@tidb_session_plan_cache_size; +@@tidb_session_plan_cache_size +300 +set global tidb_prepared_plan_cache_size = 400; +select @@tidb_prepared_plan_cache_size; +@@tidb_prepared_plan_cache_size +400 +select @@tidb_session_plan_cache_size; +@@tidb_session_plan_cache_size +400 +set global tidb_session_plan_cache_size = 500; +select @@tidb_prepared_plan_cache_size; +@@tidb_prepared_plan_cache_size +500 +select @@tidb_session_plan_cache_size; +@@tidb_session_plan_cache_size +500 +set global tidb_prepared_plan_cache_size = default; +set global tidb_session_plan_cache_size = default; +set @@tidb_session_plan_cache_size = default; +set @@tidb_prepared_plan_cache_size = default; +drop table if exists t; +create table t(a int); +set tidb_enable_non_prepared_plan_cache=1; +select * from t where a=1; +a +explain select * from t where a=2; +id estRows task access object operator info +TableReader_7 10.00 root data:Selection_6 +└─Selection_6 10.00 cop[tikv] eq(planner__core__plan_cache.t.a, 2) + └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +explain format=verbose select * from t where a=2; +id estRows estCost task access object operator info +TableReader_7 10.00 168975.57 root data:Selection_6 +└─Selection_6 10.00 2534000.00 cop[tikv] eq(planner__core__plan_cache.t.a, 2) + └─TableFullScan_5 10000.00 2035000.00 cop[tikv] table:t keep order:false, stats:pseudo +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +explain analyze select * from t where a=2; +id estRows actRows task access object execution info operator info memory disk +TableReader_7 10.00 0 root +└─Selection_6 10.00 0 cop[tikv] + └─TableFullScan_5 10000.00 0 cop[tikv] +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +set tidb_enable_non_prepared_plan_cache=default; +drop table if exists t1, t2, t3, t4; +set @@tidb_opt_fix_control = "45798:on"; +create table t1 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city')))); +create table t2 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) virtual); +create table t3 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) stored); +create table t4 (a int, info json, index zips( (CAST(info->'$.zipcode' AS UNSIGNED ARRAY)))); +set @a=1; +set @b=2; +prepare s1 from 'select * from t1 where a=?'; +show warnings; +Level Code Message +execute s1 using @a; +a info city +execute s1 using @b; +a info city +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +prepare s1 from 'select * from t2 where a=?'; +show warnings; +Level Code Message +execute s1 using @a; +a info city +execute s1 using @b; +a info city +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +prepare s1 from 'select * from t3 where a=?'; +show warnings; +Level Code Message +execute s1 using @a; +a info city +execute s1 using @b; +a info city +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +prepare s1 from 'select * from t4 where a=?'; +show warnings; +Level Code Message +execute s1 using @a; +a info +execute s1 using @b; +a info +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +set @@tidb_opt_fix_control = default; +drop table if exists t1, t2; +set @@tidb_opt_fix_control = "45798:on"; +CREATE TABLE t1 ( +ipk varbinary(255) NOT NULL, +i_id varchar(45) DEFAULT NULL, +i_set_id varchar(45) DEFAULT NULL, +p_id varchar(45) DEFAULT NULL, +p_set_id varchar(45) DEFAULT NULL, +m_id bigint(20) DEFAULT NULL, +m_i_id varchar(127) DEFAULT NULL, +m_i_set_id varchar(127) DEFAULT NULL, +d json DEFAULT NULL, +p_sources json DEFAULT NULL, +nslc json DEFAULT NULL, +cl json DEFAULT NULL, +fii json DEFAULT NULL, +fpi json DEFAULT NULL, +PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */, +UNIQUE KEY i_id (i_id), +KEY d ((cast(d as char(253) array))), +KEY m_i_id (m_i_id), +KEY m_i_set_id (m_i_set_id), +KEY fpi ((cast(fpi as unsigned array))), +KEY nslc ((cast(nslc as char(1000) array))), +KEY cl ((cast(cl as char(3000) array))), +KEY fii ((cast(fii as unsigned array))), +KEY m_id (m_id), +KEY i_set_id (i_set_id), +KEY m_i_and_m_id (m_i_id,m_id)); +CREATE TABLE t2 ( +ipk varbinary(255) NOT NULL, +created_time bigint(20) DEFAULT NULL, +arrival_time bigint(20) DEFAULT NULL, +updated_time bigint(20) DEFAULT NULL, +timestamp_data json DEFAULT NULL, +PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */); +prepare stmt from 'select * +from ( t1 left outer join t2 on ( t1 . ipk = t2 . ipk ) ) +where ( t1 . i_id = ? )'; +show warnings; +Level Code Message +set @a='a', @b='b'; +execute stmt using @a; +ipk i_id i_set_id p_id p_set_id m_id m_i_id m_i_set_id d p_sources nslc cl fii fpi ipk created_time arrival_time updated_time timestamp_data +execute stmt using @b; +ipk i_id i_set_id p_id p_set_id m_id m_i_id m_i_set_id d p_sources nslc cl fii fpi ipk created_time arrival_time updated_time timestamp_data +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +set @@tidb_opt_fix_control = default; +drop table if exists t; +create table t (a int); +prepare st from 'select * from t where mod(a, 2)=1'; +execute st; +a +execute st; +a +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +insert into mysql.expr_pushdown_blacklist(name) values('mod'); +admin reload expr_pushdown_blacklist; +execute st; +a +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +execute st; +a +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +delete from mysql.expr_pushdown_blacklist; +admin reload expr_pushdown_blacklist; +drop table if exists t, tt; +create table t(a int, index(a)); +create table tt(a varchar(10)); +set tidb_enable_non_prepared_plan_cache=1; +select a+1 from t where a<10; +a+1 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select a+1 from t where a<20; +a+1 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select a+2 from t where a<30; +a+2 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select a+2 from t where a<40; +a+2 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select a,a+1 from t where a<30; +a a+1 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select a,a+1 from t where a<40; +a a+1 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select a+'123' from tt where a='1'; +a+'123' +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select a+'123' from tt where a='2'; +a+'123' +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select 1 from t where a<10; +1 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select 1 from t where a<20; +1 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select 2 from t where a<10; +2 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select 2 from t where a<20; +2 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select 1,2 from t where a<10; +1 2 +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select 1,2 from t where a<20; +1 2 +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +set tidb_enable_non_prepared_plan_cache=default; +drop table if exists t; +create table t(a int); +insert into t values(1); +prepare s from "select * from t where tidb_parse_tso(a) > unix_timestamp()"; +execute s; +a +1 +drop table if exists t; +set tidb_enable_non_prepared_plan_cache=1; +create table t (a int, b varchar(32), c datetime, key(a)); +select * from t where mod(a, 5) < 2; +a b c +select * from t where mod(a, 5) < 2; +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select * from t where c < now(); +a b c +select * from t where c < now(); +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select date_format(c, '%Y-%m-%d') from t where a < 10; +date_format(c, '%Y-%m-%d') +select date_format(c, '%Y-%m-%d') from t where a < 10; +date_format(c, '%Y-%m-%d') +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select str_to_date(b, '%Y-%m-%d') from t where a < 10; +str_to_date(b, '%Y-%m-%d') +select str_to_date(b, '%Y-%m-%d') from t where a < 10; +str_to_date(b, '%Y-%m-%d') +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select * from t where a-2 < 20; +a b c +select * from t where a-2 < 20; +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select * from t where a+b > 100; +a b c +select * from t where a+b > 100; +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +1 +select * from t where -a > 10; +a b c +select * from t where -a > 10; +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select * from t where a < 1 and b like '%abc%'; +a b c +select * from t where a < 1 and b like '%abc%'; +a b c +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +select database() from t; +database() +select database() from t; +database() +select @@last_plan_from_cache; +@@last_plan_from_cache +0 +set tidb_enable_non_prepared_plan_cache=default; +drop table if exists t; +CREATE TABLE t (a int(11) DEFAULT NULL, b date DEFAULT NULL); +INSERT INTO t VALUES (1, current_date()); +PREPARE stmt FROM 'SELECT a FROM t WHERE b=current_date()'; +EXECUTE stmt; +a +1 diff --git a/tests/integrationtest/run-tests.sh b/tests/integrationtest/run-tests.sh index 8aaf08847f048..e0142e77d0692 100755 --- a/tests/integrationtest/run-tests.sh +++ b/tests/integrationtest/run-tests.sh @@ -88,7 +88,7 @@ function build_mysql_tester() { echo "building mysql-tester binary: $mysql_tester" rm -rf $mysql_tester - GOBIN=$PWD go install github.com/pingcap/mysql-tester/src@fc3c1e6c6233a10e1a77b168d182f2906ad0b4ee + GOBIN=$PWD go install github.com/pingcap/mysql-tester/src@77628a8d2fae0c2f4cbc059d45785ae9615c817a mv src mysql_tester } diff --git a/tests/integrationtest/t/executor/cte.test b/tests/integrationtest/t/executor/cte.test index 25bdab72cdee4..1176a509c1456 100644 --- a/tests/integrationtest/t/executor/cte.test +++ b/tests/integrationtest/t/executor/cte.test @@ -1,3 +1,6 @@ +## Delete it when https://github.com/pingcap/tidb/issues/48808 merged +set tidb_max_chunk_size=default; + # TestBasicCTE with recursive cte1 as (select 1 c1 union all select c1 + 1 c1 from cte1 where c1 < 5) select * from cte1; with recursive cte1 as (select 1 c1 union all select 2 c1 union all select c1 + 1 c1 from cte1 where c1 < 10) select * from cte1 order by c1; diff --git a/tests/integrationtest/t/executor/delete.test b/tests/integrationtest/t/executor/delete.test index ed853f680a438..45b5f3ec4df8e 100644 --- a/tests/integrationtest/t/executor/delete.test +++ b/tests/integrationtest/t/executor/delete.test @@ -13,3 +13,88 @@ set @a=1; execute stmt using @a; select * from t; +# TestDelete +drop table if exists delete_test; +drop view if exists v; +drop sequence if exists seq; +create table delete_test(id int not null default 1, name varchar(255), PRIMARY KEY(id)); +insert INTO delete_test VALUES (1, "hello"); +insert into delete_test values (2, "hello"); +--enable_info +update delete_test set name = "abc" where id = 2; +delete from delete_test where id = 2 limit 1; +delete from delete_test where 0; +--disable_info +insert into delete_test values (2, 'abc'); +--enable_info +delete from delete_test where delete_test.id = 2 limit 1; +--disable_info +begin; +SELECT * from delete_test limit 2; +commit; +insert into delete_test values (2, 'abc'); +## TODO: https://github.com/pingcap/tidb/issues/48120 +--replace_regex /INTEGER/DOUBLE/ +-- error 1292 +delete from delete_test where id = (select '2a'); +--enable_info +delete ignore from delete_test where id = (select '2a'); +--disable_info +SHOW WARNINGS; +--enable_info +delete from delete_test; +--disable_info +create view v as select * from delete_test; +-- error 1356 +delete from v where name = 'aaa'; +drop view v; +create sequence seq; +-- error 1105 +delete from seq; +drop sequence seq; + +# TestQualifiedDelete +drop table if exists t1, t2; +create table t1 (c1 int, c2 int, index (c1)); +create table t2 (c1 int, c2 int); +insert into t1 values (1, 1), (2, 2); +--enable_info +delete from t1 where t1.c1 = 1; +delete from t1 where t1.c2 = 2; +--disable_info +select * from t1; +insert into t1 values (1, 3); +--enable_info +delete from t1 as a where a.c1 = 1; +--disable_info +insert into t1 values (1, 1), (2, 2); +insert into t2 values (2, 1), (3,1); +--enable_info +delete t1, t2 from t1 join t2 where t1.c1 = t2.c2; +--disable_info +insert into t2 values (2, 1), (3,1); +--enable_info +delete a, b from t1 as a join t2 as b where a.c2 = b.c1; +--disable_info +-- error 1109 +delete t1, t2 from t1 as a join t2 as b where a.c2 = b.c1; + +# TestMultiTableDelete +drop table if exists t1, t2, t3; +create table t1 (id int, data int); +--enable_info +insert into t1 values (11, 121), (12, 122), (13, 123); +--disable_info +create table t2 (id int, data int); +--enable_info +insert into t2 values (11, 221), (22, 222), (23, 223); +--disable_info +create table t3 (id int, data int); +--enable_info +insert into t3 values (11, 321), (22, 322), (23, 323); +delete t1, t2 from t1 inner join t2 inner join t3 where t1.id=t2.id and t2.id=t3.id; +--disable_info +--sorted_result +select * from t3; + + diff --git a/tests/integrationtest/t/executor/executor.test b/tests/integrationtest/t/executor/executor.test index dde049d75536f..bf5ac6da7dda2 100644 --- a/tests/integrationtest/t/executor/executor.test +++ b/tests/integrationtest/t/executor/executor.test @@ -2671,3 +2671,180 @@ set @@tidb_enable_collect_execution_info=0; select /*+ use_index_merge(t1, primary, t1a) */ * from t1 where id < 2 or a > 4 order by a; set @@tidb_enable_collect_execution_info=default; +# TestIndexLookupRuntimeStats +drop table if exists t1; +create table t1 (a int, b int, index(a)); +insert into t1 values (1,2),(2,3),(3,4); +--replace_regex /.*time:.*loops:.*index_task:.*table_task: {total_time.*num.*concurrency.*}.*/.*time:.*loops:.*index_task:.*table_task: {total_time.*num.*concurrency.*}.*/ /.*time:.*loops:.*cop_task:.*/.*time:.*loops:.*cop_task:.*/ /[.0-9]+ KB/ KB/ /[0-9]+ Bytes/ Bytes/ +explain analyze select * from t1 use index(a) where a > 1; + +# TestHashAggRuntimeStats +drop table if exists t1; +create table t1 (a int, b int); +insert into t1 values (1,2),(2,3),(3,4); +--replace_regex /.*time:.*loops:.*partial_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*final_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*/.*time:.*loops:.*partial_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*final_worker:{wall_time:.*concurrency:.*task_num:.*tot_wait:.*tot_exec:.*tot_time:.*max:.*p95:.*}.*/ /time:.*loops:.*cop_task.*/time.*loops.*cop_task.*/ /tikv_task:.*/tikv_task:.*/ /[.0-9]+ KB/ KB/ /[.0-9]+ Bytes/ Bytes/ +explain analyze SELECT /*+ HASH_AGG() */ count(*) FROM t1 WHERE a < 10; + +# TestSelectForUpdate +set global tidb_txn_mode=''; +drop table if exists t, t1; +create table t (c1 int, c2 int, c3 int); +insert t values (11, 2, 3); +insert t values (12, 2, 3); +insert t values (13, 2, 3); +create table t1 (c1 int); +insert t1 values (11); + +connect (conn1, localhost, root,, executor__executor); +begin; +select * from t where c1=11 for update; + +connect (conn2, localhost, root,, executor__executor); +begin; +update t set c2=211 where c1=11; +commit; + +connection conn1; +--replace_regex /txnStartTS.*reason/ reason/ +--error 9007 +commit; + +begin; +select * from t where exists(select null from t1 where t1.c1=t.c1) for update; + +connection conn2; +begin; +update t set c2=211 where c1=12; +commit; + +connection conn1; +commit; + +begin; +select * from t where c1=11 for update; + +connection conn2; +begin; +update t set c2=22 where c1=12; +commit; + +connection conn1; +commit; + +set @@autocommit=1; +select * from t where c1=11 for update; + +connection conn2; +begin; +update t set c2=211 where c1=11; +commit; + +connection conn1; +commit; + +begin; +--sorted_result +select * from (select * from t for update) t join t1 for update; + +connection conn2; +begin; +update t1 set c1 = 13; +commit; + +connection conn1; +--replace_regex /txnStartTS.*reason/ reason/ +--error 9007 +commit; + +disconnect conn1; +disconnect conn2; +set global tidb_txn_mode=pessimistic; + +# TestSelectForUpdateOf +drop table if exists t, t1; +create table t (i int); +create table t1 (i int); +insert t values (1); +insert t1 values (1); +begin pessimistic; +select * from t, t1 where t.i = t1.i for update of t; + +connect (conn1, localhost, root,, executor__executor); +begin pessimistic; +select * from t1 for update; +--error 3572 +select * from t for update nowait; + +connection default; +rollback; + +connection conn1; +select * from t for update nowait; +rollback; +disconnect conn1; + +# TestForSelectScopeInUnion +set session tidb_txn_mode=''; +# A union B for update, the "for update" option belongs to union statement, so +# it should works on both A and B. +drop table if exists t; +create table t(a int); +insert into t values (1); +begin; +select 1 as a union select a from t for update; + +connect (conn1, localhost, root,, executor__executor); +set session tidb_txn_mode=''; +update t set a = a + 1; + +connection default; +## As tk1 use select 'for update', it should detect conflict and fail. +--replace_regex /txnStartTS.*reason/ reason/ +--error 9007 +commit; + +begin; +--sorted_result +select 1 as a union select a from t limit 5 for update; +select 1 as a union select a from t order by a for update; + +connection conn1; +update t set a = a + 1; + +connection default; +--replace_regex /txnStartTS.*reason/ reason/ +--error 9007 +commit; + +disconnect conn1; +set session tidb_txn_mode=pessimistic; + +# TestAdminShowDDLJobsRowCount +# https://github.com/pingcap/tidb/issues/25968 +drop table if exists t; +create table t (id bigint key,b int); +split table t by (10),(20),(30); +insert into t values (0,0),(10,10),(20,20),(30,30); +alter table t add index idx1(b); +--replace_column 1 4 6 7 9 10 11 +admin show ddl jobs 1; + +insert into t values (1,0),(2,10),(3,20),(4,30); +alter table t add index idx2(b); +--replace_column 1 4 6 7 9 10 11 +admin show ddl jobs 1; + +# TestSummaryFailedUpdate +drop table if exists t; +create table t(a int, b int as(-a)); +insert into t(a) values(1), (3), (7); +SET GLOBAL tidb_mem_oom_action='CANCEL'; +set @@tidb_mem_quota_query=1; +--replace_regex /conn=[0-9]+/conn=/ +--error 8175 +update t set t.a = t.a - 1 where t.a in (select a from t where a < 4); +set @@tidb_mem_quota_query=1000000000; +select stmt_type from information_schema.statements_summary where digest_text = 'update `t` set `t` . `a` = `t` . `a` - ? where `t` . `a` in ( select `a` from `t` where `a` < ? )'; + +set @@tidb_mem_quota_query=default; +set global tidb_mem_oom_action=default; diff --git a/tests/integrationtest/t/executor/insert.test b/tests/integrationtest/t/executor/insert.test index 15bae116746ab..8d241c2a5d7e6 100644 --- a/tests/integrationtest/t/executor/insert.test +++ b/tests/integrationtest/t/executor/insert.test @@ -967,3 +967,651 @@ commit; disconnect conn1; set global tidb_disable_txn_auto_retry=default; + +# TestIssue38950 +drop table if exists t; +create table t (id smallint auto_increment primary key); +alter table t add column c1 int default 1; +--enable_info +insert ignore into t(id) values (194626268); +--disable_info +select * from t; +--enable_info +insert ignore into t(id) values ('*') on duplicate key update c1 = 2; +--disable_info +select * from t; + +# TestInsertIgnoreOnDup +drop table if exists t; +create table t (i int not null primary key, j int unique key); +--enable_info +insert into t values (1, 1), (2, 2); +insert ignore into t values(1, 1) on duplicate key update i = 2; +--disable_info +select * from t; +--enable_info +insert ignore into t values(1, 1) on duplicate key update j = 2; +--disable_info +select * from t; + +drop table if exists t2; +create table t2(`col_25` set('Alice','Bob','Charlie','David') NOT NULL,`col_26` date NOT NULL DEFAULT '2016-04-15', PRIMARY KEY (`col_26`) clustered, UNIQUE KEY `idx_9` (`col_25`,`col_26`),UNIQUE KEY `idx_10` (`col_25`)); +insert into t2(col_25, col_26) values('Bob', '1989-03-23'),('Alice', '2023-11-24'), ('Charlie', '2023-12-05'); +insert ignore into t2 (col_25,col_26) values ( 'Bob','1977-11-23' ) on duplicate key update col_25 = 'Alice', col_26 = '2036-12-13'; +show warnings; +select * from t2; + +drop table if exists t4; +create table t4(id int primary key clustered, k int, v int, unique key uk1(k)); +insert into t4 values (1, 10, 100), (3, 30, 300); +insert ignore into t4 (id, k, v) values(1, 0, 0) on duplicate key update id = 2, k = 30; +show warnings; +select * from t4; + +drop table if exists t5; +create table t5(k1 varchar(100), k2 varchar(100), uk1 int, v int, primary key(k1, k2) clustered, unique key ukk1(uk1), unique key ukk2(v)); +insert into t5(k1, k2, uk1, v) values('1', '1', 1, '100'), ('1', '3', 2, '200'); +update ignore t5 set k2 = '2', uk1 = 2 where k1 = '1' and k2 = '1'; +show warnings; +select * from t5; + +drop table if exists t6; +create table t6 (a int, b int, c int, primary key(a, b) clustered, unique key idx_14(b), unique key idx_15(b), unique key idx_16(a, b)); +insert into t6 select 10, 10, 20; +insert ignore into t6 set a = 20, b = 10 on duplicate key update a = 100; +select * from t6; +insert ignore into t6 set a = 200, b= 10 on duplicate key update c = 1000; +select * from t6; + +# TestInsertAutoInc +drop table if exists insert_autoinc_test; +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(c1) values (1), (2); +begin; +select * from insert_autoinc_test; +commit; +begin; +insert into insert_autoinc_test(id, c1) values (5,5); +insert into insert_autoinc_test(c1) values (6); +commit; +begin; +select * from insert_autoinc_test; +commit; +begin; +insert into insert_autoinc_test(id, c1) values (3,3); +commit; +begin; +select * from insert_autoinc_test; +commit; +begin; +insert into insert_autoinc_test(c1) values (7); +commit; +begin; +select * from insert_autoinc_test; +commit; +drop table if exists insert_autoinc_test; + +## issue-962 +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(id, c1) values (0.3, 1); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (-0.3, 2); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (-3.3, 3); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (4.3, 4); +select * from insert_autoinc_test; +insert into insert_autoinc_test(c1) values (5); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (null, 6); +select * from insert_autoinc_test; +drop table if exists insert_autoinc_test; + +## SQL_MODE=NO_AUTO_VALUE_ON_ZERO +create table insert_autoinc_test (id int primary key auto_increment, c1 int); +insert into insert_autoinc_test(id, c1) values (5, 1); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (0, 2); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (0, 3); +select * from insert_autoinc_test; +set SQL_MODE=NO_AUTO_VALUE_ON_ZERO; +insert into insert_autoinc_test(id, c1) values (0, 4); +select * from insert_autoinc_test; +-- error 1062 +insert into insert_autoinc_test(id, c1) values (0, 5); +insert into insert_autoinc_test(c1) values (6); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (null, 7); +select * from insert_autoinc_test; +set SQL_MODE=''; +insert into insert_autoinc_test(id, c1) values (0, 8); +select * from insert_autoinc_test; +insert into insert_autoinc_test(id, c1) values (null, 9); +select * from insert_autoinc_test; +set sql_mode = default; + +# TestInsert +drop table if exists insert_test; +create table insert_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1); +--enable_info +insert insert_test (c1) values (1),(2),(NULL); +--disable_info +begin; +-- error 1136 +insert insert_test (c1) values (); +rollback; +begin; +-- error 1136 +insert insert_test (c1, c2) values (1,2),(1); +rollback; +begin; +-- error 1054 +insert insert_test (xxx) values (3); +rollback; +begin; +-- error 1146 +insert insert_test_xxx (c1) values (); +rollback; +--enable_info +insert insert_test set c1 = 3; +--disable_info +begin; +-- error 1110 +insert insert_test set c1 = 4, c1 = 5; +rollback; +begin; +-- error 1054 +insert insert_test set xxx = 6; +rollback; + +drop table if exists insert_test_1, insert_test_2; +create table insert_test_1 (id int, c1 int); +--enable_info +insert insert_test_1 select id, c1 from insert_test; +--disable_info +create table insert_test_2 (id int, c1 int); +--enable_info +insert insert_test_1 select id, c1 from insert_test union select id * 10, c1 * 10 from insert_test; +--disable_info +begin; +-- error 1136 +insert insert_test_1 select c1 from insert_test; +rollback; +begin; +-- error 1136 +insert insert_test_1 values(default, default, default, default, default); +rollback; +select * from insert_test where id = 1; +--enable_info +insert into insert_test (id, c3) values (1, 2) on duplicate key update id=values(id), c2=10; +--disable_info +select * from insert_test where id = 1; +--enable_info +insert into insert_test (id, c2) values (1, 1) on duplicate key update insert_test.c2=10; +--disable_info +-- error 1054 +insert into insert_test (id, c2) values(1, 1) on duplicate key update t.c2 = 10; +--enable_info +INSERT INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3; +--disable_info +select * from insert_test where id = 1; +--enable_info +INSERT IGNORE INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3; +--disable_info +select * from insert_test where id = 1; + +drop table if exists insert_err; +create table insert_err (id int, c1 varchar(8)); +-- error 1406 +insert insert_err values (1, 'abcdabcdabcd'); +insert insert_err values (1, '你好,世界'); +create table TEST1 (ID INT NOT NULL, VALUE INT DEFAULT NULL, PRIMARY KEY (ID)); +--enable_info +INSERT INTO TEST1(id,value) VALUE(3,3) on DUPLICATE KEY UPDATE VALUE=4; +--disable_info + +drop table if exists t; +create table t (id int); +insert into t values(1); +update t t1 set id = (select count(*) + 1 from t t2 where t1.id = t2.id); +select * from t; + +## issue 3235 +drop table if exists t; +create table t(c decimal(5, 5)); +insert into t value(0); +-- error 1264 +insert into t value(1); + +drop table if exists t; +create table t(c binary(255)); +insert into t value(1); +select length(c) from t; + +drop table if exists t; +create table t(c varbinary(255)); +insert into t value(1); +select length(c) from t; + +## issue 3509 +drop table if exists t; +create table t(c int); +set @@time_zone = '+08:00'; +insert into t value(Unix_timestamp('2002-10-27 01:00')); +select * from t; +set @@time_zone = default; + +## issue 3832 +drop table if exists t1; +create table t1 (b char(0)); +insert into t1 values (""); + +## issue 3895 +DROP TABLE IF EXISTS t; +CREATE TABLE t(a DECIMAL(4,2)); +INSERT INTO t VALUES (1.000001); +SHOW WARNINGS; +INSERT INTO t VALUES (1.000000); +SHOW WARNINGS; + +## issue 4653 +DROP TABLE IF EXISTS t; +CREATE TABLE t(a datetime); +-- error 1292 +INSERT INTO t VALUES('2017-00-00'); +set sql_mode = ''; +INSERT INTO t VALUES('2017-00-00'); +SELECT * FROM t; +set sql_mode = 'strict_all_tables'; +SELECT * FROM t; +set sql_mode = default; + +drop table if exists test; +CREATE TABLE test(id int(10) UNSIGNED NOT NULL AUTO_INCREMENT, p int(10) UNSIGNED NOT NULL, PRIMARY KEY(p), KEY(id)); +insert into test(p) value(1); +select * from test; +select * from test use index (id) where id = 1; +insert into test values(NULL, 2); +select * from test use index (id) where id = 2; +insert into test values(2, 3); +select * from test use index (id) where id = 2; + +## issue 6360 +drop table if exists t; +create table t(a bigint unsigned); +set @@sql_mode = 'strict_all_tables'; +-- error 1264 +insert into t value (-1); +set @@sql_mode = ''; +insert into t value (-1); +show warnings; +insert into t select -1; +show warnings; +insert into t select cast(-1 as unsigned); +insert into t value (-1.111); +show warnings; +insert into t value ('-1.111'); +show warnings; +update t set a = -1 limit 1; +show warnings; +select * from t; +set @@sql_mode = default; + +# issue 6424 & issue 20207 +drop table if exists t; +create table t(a time(6)); +insert into t value('20070219173709.055870'), ('20070219173709.055'), ('20070219173709.055870123'); +select * from t; +truncate table t; +insert into t value(20070219173709.055870), (20070219173709.055), (20070219173709.055870123); +select * from t; +-- error 1292 +insert into t value(-20070219173709.055870); + +drop table if exists t; +set @@sql_mode=''; +create table t(a float unsigned, b double unsigned); +insert into t value(-1.1, -1.1), (-2.1, -2.1), (0, 0), (1.1, 1.1); +show warnings; +select * from t; +set @@sql_mode=default; + +## issue 7061 +drop table if exists t; +create table t(a int default 1, b int default 2); +insert into t values(default, default); +select * from t; +truncate table t; +insert into t values(default(b), default(a)); +select * from t; +truncate table t; +insert into t (b) values(default); +select * from t; +truncate table t; +insert into t (b) values(default(a)); +select * from t; + +drop view if exists v; +create view v as select * from t; +-- error 1105 +insert into v values(1,2); +-- error 1105 +replace into v values(1,2); +drop view v; + +drop sequence if exists seq; +create sequence seq; +-- error 1105 +insert into seq values(); +-- error 1105 +replace into seq values(); +drop sequence seq; + +## issue 22851 +drop table if exists t; +create table t(name varchar(255), b int, c int, primary key(name(2))); +insert into t(name, b) values("cha", 3); +-- error 1062 +insert into t(name, b) values("chb", 3); +insert into t(name, b) values("测试", 3); +-- error 1062 +insert into t(name, b) values("测试", 3); + +# TestInsertOnDup +drop table if exists t; +create table t (i int unique key); +--enable_info +insert into t values (1),(2); +--disable_info +select * from t; +--enable_info +insert into t values (1), (2) on duplicate key update i = values(i); +--disable_info +select * from t; +--enable_info +insert into t values (2), (3) on duplicate key update i = 3; +--disable_info +select * from t; + +drop table if exists t; +create table t (i int primary key, j int unique key); +--enable_info +insert into t values (-1, 1); +--disable_info +select * from t; +--enable_info +insert into t values (1, 1) on duplicate key update j = values(j); +--disable_info +select * from t; + +drop table if exists test; +create table test (i int primary key, j int unique); +begin; +insert into test values (1,1); +insert into test values (2,1) on duplicate key update i = -i, j = -j; +commit; +select * from test; +delete from test; +insert into test values (1, 1); +begin; +delete from test where i = 1; +insert into test values (2, 1) on duplicate key update i = -i, j = -j; +commit; +select * from test; +delete from test; +insert into test values (1, 1); +begin; +update test set i = 2, j = 2 where i = 1; +insert into test values (1, 3) on duplicate key update i = -i, j = -j; +insert into test values (2, 4) on duplicate key update i = -i, j = -j; +commit; +select * from test order by i; +delete from test; +begin; +insert into test values (1, 3), (1, 3) on duplicate key update i = values(i), j = values(j); +commit; +select * from test order by i; +create table tmp (id int auto_increment, code int, primary key(id, code)); +create table m (id int primary key auto_increment, code int unique); +insert tmp (code) values (1); +insert tmp (code) values (1); +set tidb_init_chunk_size=1; +insert m (code) select code from tmp on duplicate key update code = values(code); +select * from m; + +## The following two cases are used for guaranteeing the last_insert_id +## to be set as the value of on-duplicate-update assigned. +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT AUTO_INCREMENT PRIMARY KEY, +f2 VARCHAR(5) NOT NULL UNIQUE); +--enable_info +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +--disable_info +SELECT LAST_INSERT_ID(); +--enable_info +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +--disable_info +SELECT LAST_INSERT_ID(); + +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT AUTO_INCREMENT UNIQUE, +f2 VARCHAR(5) NOT NULL UNIQUE); +--enable_info +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +--disable_info +SELECT LAST_INSERT_ID(); +--enable_info +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1); +--disable_info +SELECT LAST_INSERT_ID(); +--enable_info +INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = 2; +--disable_info +SELECT LAST_INSERT_ID(); + +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT); +--enable_info +INSERT t1 VALUES (1) ON DUPLICATE KEY UPDATE f1 = 1; +--disable_info +SELECT * FROM t1; + +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 (f1 INT PRIMARY KEY, f2 INT NOT NULL UNIQUE); +--enable_info +INSERT t1 VALUES (1, 1); +INSERT t1 VALUES (1, 1), (1, 1) ON DUPLICATE KEY UPDATE f1 = 2, f2 = 2; +--disable_info +SELECT * FROM t1 order by f1; +-- error 1048 +INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +--enable_info +INSERT IGNORE t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +--disable_info +show warnings; +SELECT * FROM t1 order by f1; + +SET sql_mode=''; +INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null; +show warnings; +SELECT * FROM t1 order by f1; +set sql_mode=default; + +set tidb_init_chunk_size=default; + + +# TestInsertOnDuplicateKey +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +--enable_info +insert into t1 values(1, 100); +insert into t2 values(1, 200); +insert into t1 select a2, b2 from t2 on duplicate key update b1 = a2; +--disable_info +select * from t1; +--enable_info +insert into t1 select a2, b2 from t2 on duplicate key update b1 = b2; +--disable_info +select * from t1; +--enable_info +insert into t1 select a2, b2 from t2 on duplicate key update a1 = a2; +--disable_info +select * from t1; +--enable_info +insert into t1 select a2, b2 from t2 on duplicate key update b1 = 300; +--disable_info +select * from t1; +--enable_info +insert into t1 values(1, 1) on duplicate key update b1 = 400; +--disable_info +select * from t1; +--enable_info +insert into t1 select 1, 500 from t2 on duplicate key update b1 = 400; +--disable_info +select * from t1; + +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +-- error 1054 +insert into t1 select * from t2 on duplicate key update c = t2.b; + +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +-- error 1052 +insert into t1 select * from t2 on duplicate key update a = b; + +drop table if exists t1, t2; +create table t1(a bigint primary key, b bigint); +create table t2(a bigint primary key, b bigint); +-- error 1054 +insert into t1 select * from t2 on duplicate key update c = b; + +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +-- error 1054 +insert into t1 select * from t2 on duplicate key update a1 = values(b2); + +drop table if exists t1, t2; +create table t1(a1 bigint primary key, b1 bigint); +create table t2(a2 bigint primary key, b2 bigint); +--enable_info +insert into t1 values(1, 100); +insert into t2 values(1, 200); +insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2; +--disable_info +select * from t1; +--enable_info +insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2; +--disable_info +select * from t1; + +drop table if exists t; +create table t(k1 bigint, k2 bigint, val bigint, primary key(k1, k2)); +--enable_info +insert into t (val, k1, k2) values (3, 1, 2); +--disable_info +select * from t; +--enable_info +insert into t (val, k1, k2) select c, a, b from (select 1 as a, 2 as b, 4 as c) tmp on duplicate key update val = tmp.c; +--disable_info +select * from t; + +drop table if exists t; +create table t(k1 double, k2 double, v double, primary key(k1, k2)); +--enable_info +insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c; +--disable_info +select * from t; +--enable_info +insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c; +--disable_info +select * from t; + +drop table if exists t1, t2; +create table t1(id int, a int, b int); +--enable_info +insert into t1 values (1, 1, 1); +insert into t1 values (2, 2, 1); +insert into t1 values (3, 3, 1); +--disable_info +create table t2(a int primary key, b int, unique(b)); +--enable_info +insert into t2 select a, b from t1 order by id on duplicate key update a=t1.a, b=t1.b; +--disable_info +select * from t2 order by a; + +drop table if exists t1, t2; +create table t1(id int, a int, b int); +--enable_info +insert into t1 values (1, 1, 1); +insert into t1 values (2, 1, 2); +insert into t1 values (3, 3, 1); +--disable_info +create table t2(a int primary key, b int, unique(b)); +--enable_info +insert into t2 select a, b from t1 order by id on duplicate key update a=t1.a, b=t1.b; +--disable_info +select * from t2 order by a; + +drop table if exists t1, t2; +create table t1(id int, a int, b int, c int); +--enable_info +insert into t1 values (1, 1, 1, 1); +insert into t1 values (2, 2, 1, 2); +insert into t1 values (3, 3, 2, 2); +insert into t1 values (4, 4, 2, 2); +--disable_info +create table t2(a int primary key, b int, c int, unique(b), unique(c)); +--enable_info +insert into t2 select a, b, c from t1 order by id on duplicate key update b=t2.b, c=t2.c; +--disable_info +select * from t2 order by a; + +drop table if exists t1; +create table t1(a int primary key, b int); +--enable_info +insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5); +insert into t1 values(4,14),(5,15),(6,16),(7,17),(8,18) on duplicate key update b=b+10; +--disable_info + +drop table if exists a, b; +create table a(x int primary key); +create table b(x int, y int); +--enable_info +insert into a values(1); +insert into b values(1, 2); +insert into a select x from b ON DUPLICATE KEY UPDATE a.x=b.y; +--disable_info +select * from a; + +--echo ## Test issue 28078. +--echo ## Use different types of columns so that there's likely to be error if the types mismatches. +drop table if exists a, b; +create table a(id int, a1 timestamp, a2 varchar(10), a3 float, unique(id)); +create table b(id int, b1 time, b2 varchar(10), b3 int); +--enable_info +insert into a values (1, '2022-01-04 07:02:04', 'a', 1.1), (2, '2022-01-04 07:02:05', 'b', 2.2); +insert into b values (2, '12:34:56', 'c', 10), (3, '01:23:45', 'd', 20); +insert into a (id) select id from b on duplicate key update a.a2 = b.b2, a.a3 = 3.3; +--disable_info +select * from a; +--enable_info +insert into a (id) select 4 from b where b3 = 20 on duplicate key update a.a3 = b.b3; +--disable_info +select * from a; +--enable_info +insert into a (a2, a3) select 'x', 1.2 from b on duplicate key update a.a2 = b.b3; +--disable_info +select * from a; + +--echo ## reproduce insert on duplicate key update bug under new row format. +drop table if exists t1; +create table t1(c1 decimal(6,4), primary key(c1)); +insert into t1 set c1 = 0.1; +insert into t1 set c1 = 0.1 on duplicate key update c1 = 1; +select * from t1 use index(primary); diff --git a/tests/integrationtest/t/executor/partition/write.test b/tests/integrationtest/t/executor/partition/write.test new file mode 100644 index 0000000000000..6eb3724786d05 --- /dev/null +++ b/tests/integrationtest/t/executor/partition/write.test @@ -0,0 +1,604 @@ +--echo # TestWriteListPartitionTable2 +--echo # test for write list partition when the partition expression is complicated and contain generated column. +set @@session.tidb_enable_list_partition = ON; +drop table if exists t; +create table t (id int, name varchar(10),b int generated always as (length(name)+1) virtual) + partition by list (id*2 + b*b + b*b - b*b*2 - abs(id)) ( + partition p0 values in (3,5,6,9,17), + partition p1 values in (1,2,10,11,19,20), + partition p2 values in (4,12,13,14,18), + partition p3 values in (7,8,15,16,null) +); + +analyze table t; + +--echo ## Test add unique index failed. +insert into t (id,name) values (1, 'a'),(1,'b'); +--error 1062 +alter table t add unique index idx (id,b); +--echo ## Test add unique index success. +delete from t where name='b'; +alter table t add unique index idx (id,b); + +--echo ## --------------------------Test insert--------------------------- +--echo ## Test insert 1 partition. +delete from t; +insert into t (id,name) values (1, 'a'),(2,'b'),(10,'c'); +select id,name from t partition(p1) order by id; +--echo ## Test insert multi-partitions. +delete from t; +insert into t (id,name) values (1, 'a'),(3,'c'),(4,'e'); +select id,name from t partition(p0) order by id; +select id,name from t partition(p1) order by id; +select id,name from t partition(p2) order by id; +select id,name from t partition(p3) order by id; +--echo ## Test insert on duplicate. +insert into t (id,name) values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'; +select id,name from t partition(p0) order by id; +select id,name from t partition(p1) order by id; +select id,name from t partition(p2) order by id; +select id,name from t partition(p3) order by id; +--echo ## Test insert on duplicate error +--error 1062 +insert into t (id,name) values (3, 'a'), (11,'x') on duplicate key update id=id+1; +select id,name from t order by id; +--echo ## Test insert ignore with duplicate +insert ignore into t (id,name) values (1, 'b'), (5,'a'),(null,'y'); +show warnings; +select id,name from t partition(p0) order by id; +select id,name from t partition(p1) order by id; +select id,name from t partition(p2) order by id; +select id,name from t partition(p3) order by id; +--echo ## Test insert ignore without duplicate +insert ignore into t (id,name) values (15, 'a'),(17,'a'); +select id,name from t partition(p0,p1,p2) order by id; +select id,name from t partition(p3) order by id; +--echo ## Test insert meet no partition error. +--error 1526 +insert into t (id,name) values (100, 'd'); + +--echo ## --------------------------Test update--------------------------- +--echo ## Test update 1 partition. +delete from t; +insert into t (id,name) values (1, 'a'),(2,'b'),(3,'c'); +update t set name='b' where id=2;; +select id,name from t partition(p1); +update t set name='x' where id in (1,2); +select id,name from t partition(p1); +update t set name='y' where id < 3; +select id,name from t order by id; +--echo ## Test update meet duplicate error. +--error 1062 +update t set id=2 where id = 1; +select id,name from t order by id; + +--echo ## Test update multi-partitions +update t set name='z' where id in (1,2,3);; +select id,name from t order by id; +update t set name='a' limit 3; +select id,name from t order by id; +update t set id=id*10 where id in (1,2); +select id,name from t order by id; +--echo ## Test update meet duplicate error. +--error 1062 +update t set id=id+17 where id in (3,10); +select id,name from t order by id; +--echo ## Test update meet no partition error. +--error 1526 +update t set id=id*2 where id in (3,20); +select id,name from t order by id; + +--echo ## --------------------------Test replace--------------------------- +--echo ## Test replace 1 partition. +delete from t; +replace into t (id,name) values (1, 'a'),(2,'b'); +select id,name from t order by id; +--echo ## Test replace multi-partitions. +replace into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +select id,name from t partition(p0) order by id; +select id,name from t partition(p1) order by id; +select id,name from t partition(p2) order by id; +select id,name from t partition(p3) order by id; +--echo ## Test replace on duplicate. +replace into t (id,name) values (1, 'x'),(7,'x'); +select id,name from t order by id; +--echo ## Test replace meet no partition error. +--error 1526 +replace into t (id,name) values (10,'x'),(50,'x'); +select id,name from t order by id; + +--echo ## --------------------------Test delete--------------------------- +--echo ## Test delete 1 partition. +delete from t where id = 3; +select id,name from t partition(p0) order by id; +delete from t where id in (1,2); +select id,name from t partition(p1) order by id; +--echo ## Test delete multi-partitions. +delete from t where id in (4,7,10,11); +select id,name from t; +insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +delete from t where id < 10; +select id,name from t; +insert into t (id,name) values (3, 'c'),(4,'d'),(7,'f'); +delete from t limit 3; +select id,name from t; +set @@session.tidb_enable_list_partition = default; + +--echo # TestWriteListColumnsPartitionTable1 +set @@session.tidb_enable_list_partition = ON; + +drop table if exists t; +create table t (id int, name varchar(10)) partition by list columns (id) ( + partition p0 values in (3,5,6,9,17), + partition p1 values in (1,2,10,11,19,20), + partition p2 values in (4,12,13,14,18), + partition p3 values in (7,8,15,16,null) +); + +analyze table t; + +--echo ## Test add unique index failed. +insert into t values (1, 'a'),(1,'b'); +--error 1062 +alter table t add unique index idx (id); +--echo ## Test add unique index success. +delete from t where name='b'; +alter table t add unique index idx (id); + +--echo ## --------------------------Test insert--------------------------- +--echo ## Test insert 1 partition. +delete from t; +insert into t values (1, 'a'),(2,'b'),(10,'c'); +select * from t partition(p1) order by id; +--echo ## Test insert multi-partitions. +delete from t; +insert into t values (1, 'a'),(3,'c'),(4,'e'); +select * from t partition(p0) order by id; +select * from t partition(p1) order by id; +select * from t partition(p2) order by id; +select * from t partition(p3) order by id; +--echo ## Test insert on duplicate. +insert into t values (1, 'd'), (3,'f'),(5,'g') on duplicate key update name='x'; +select * from t partition(p0) order by id; +select * from t partition(p1) order by id; +select * from t partition(p2) order by id; +select * from t partition(p3) order by id; +--echo ## Test insert on duplicate error +--error 1062 +insert into t values (3, 'a'), (11,'x') on duplicate key update id=id+1; +select * from t order by id; +--echo ## Test insert ignore with duplicate +insert ignore into t values (1, 'b'), (5,'a'),(null,'y'); +show warnings; +select * from t partition(p0) order by id; +select * from t partition(p1) order by id; +select * from t partition(p2) order by id; +select * from t partition(p3) order by id; +--echo ## Test insert ignore without duplicate +insert ignore into t values (15, 'a'),(17,'a'); +select * from t partition(p0,p1,p2) order by id; +select * from t partition(p3) order by id; +--echo ## Test insert meet no partition error. +--error 1526 +insert into t values (100, 'd'); + +--echo ## --------------------------Test update--------------------------- +--echo ## Test update 1 partition. +delete from t; +insert into t values (1, 'a'),(2,'b'),(3,'c'); +update t set name='b' where id=2;; +select * from t partition(p1); +update t set name='x' where id in (1,2); +select * from t partition(p1); +update t set name='y' where id < 3; +select * from t order by id; +--echo ## Test update meet duplicate error. +--error 1062 +update t set id=2 where id = 1; +select * from t order by id; + +--echo ## Test update multi-partitions +update t set name='z' where id in (1,2,3);; +select * from t order by id; +update t set name='a' limit 3; +select * from t order by id; +update t set id=id*10 where id in (1,2); +select * from t order by id; +--echo ## Test update meet duplicate error. +--error 1062 +update t set id=id+17 where id in (3,10); +select * from t order by id; +--echo ## Test update meet no partition error. +--error 1526 +update t set id=id*2 where id in (3,20); +select * from t order by id; + +--echo ## --------------------------Test replace--------------------------- +--echo ## Test replace 1 partition. +delete from t; +replace into t values (1, 'a'),(2,'b'); +select * from t order by id; +--echo ## Test replace multi-partitions. +replace into t values (3, 'c'),(4,'d'),(7,'f'); +select * from t partition(p0) order by id; +select * from t partition(p1) order by id; +select * from t partition(p2) order by id; +select * from t partition(p3) order by id; +--echo ## Test replace on duplicate. +replace into t values (1, 'x'),(7,'x'); +select * from t order by id; +--echo ## Test replace meet no partition error. +--error 1526 +replace into t values (10,'x'),(100,'x'); +select * from t order by id; + +--echo ## --------------------------Test delete--------------------------- +--echo ## Test delete 1 partition. +delete from t where id = 3; +select * from t partition(p0) order by id; +delete from t where id in (1,2); +select * from t partition(p1) order by id; +--echo ## Test delete multi-partitions. +delete from t where id in (4,7,10,11); +select * from t; +insert into t values (3, 'c'),(4,'d'),(7,'f'); +delete from t where id < 10; +select * from t; +insert into t values (3, 'c'),(4,'d'),(7,'f'); +delete from t limit 3; +select * from t; + +set @@session.tidb_enable_list_partition = default; + + +# TestPartitionedTableReplace +set tidb_opt_fix_control = "44262:ON"; +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) + partition by range (id) ( + PARTITION p0 VALUES LESS THAN (3), + PARTITION p1 VALUES LESS THAN (5), + PARTITION p2 VALUES LESS THAN (7), + PARTITION p3 VALUES LESS THAN (9)); +--enable_info +replace replace_test (c1) values (1),(2),(NULL); +--disable_info +begin; +-- error 1136 +replace replace_test (c1) values (); +rollback; +begin; +-- error 1136 +replace replace_test (c1, c2) values (1,2),(1); +rollback; +begin; +-- error 1054 +replace replace_test (xxx) values (3); +rollback; +begin; +-- error 1146 +replace replace_test_xxx (c1) values (); +rollback; +--enable_info +replace replace_test set c1 = 3; +--disable_info +begin; +-- error 1110 +replace replace_test set c1 = 4, c1 = 5; +rollback; +begin; +-- error 1054 +replace replace_test set xxx = 6; +rollback; + +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int) partition by range (id) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (6), + PARTITION p2 VALUES LESS THAN (8), + PARTITION p3 VALUES LESS THAN (10), + PARTITION p4 VALUES LESS THAN (100)); +--enable_info +replace replace_test_1 select id, c1 from replace_test; +--disable_info + +drop table if exists replace_test_2; +create table replace_test_2 (id int, c1 int) partition by range (id) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (50), + PARTITION p2 VALUES LESS THAN (100), + PARTITION p3 VALUES LESS THAN (300)); +--enable_info +replace replace_test_2 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +--disable_info +begin; +-- error 1136 +replace replace_test_2 select c1 from replace_test; +rollback; + +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by range (c2) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (7), + PARTITION p2 VALUES LESS THAN (11)); +--enable_info +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c1=8, c2=8; +replace into replace_test_3 set c2=NULL; +replace into replace_test_3 set c2=NULL; +--disable_info + +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by range (c1) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (7), + PARTITION p2 VALUES LESS THAN (11)); +--enable_info +replace into replace_test_4 set c2=NULL; +replace into replace_test_4 set c2=NULL; +--disable_info + +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by range (c2) ( + PARTITION p0 VALUES LESS THAN (4), + PARTITION p1 VALUES LESS THAN (7), + PARTITION p2 VALUES LESS THAN (11)); +--enable_info +replace into replace_test_5 set c1=1, c2=2; +replace into replace_test_5 set c1=1, c2=2; +--disable_info + +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by range (b) ( + PARTITION p1 VALUES LESS THAN (100), + PARTITION p2 VALUES LESS THAN (200)); +--enable_info +insert into tIssue989 (a, b) values (1, 2); +replace into tIssue989(a, b) values (111, 2); +--disable_info +select * from tIssue989; + +set tidb_opt_fix_control = default; + + +# TestPartitionedTableUpdate +set tidb_opt_fix_control = "44262:ON"; +drop table if exists t; +create table t (id int not null default 1, name varchar(255)) + PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21)); +insert INTO t VALUES (1, "hello"); +insert INTO t VALUES (7, "hello"); + +--echo ## update non partition column +--enable_info +UPDATE t SET name = "abc" where id > 0; +--disable_info +SELECT * from t order by id limit 2; + +--echo ## update partition column +--enable_info +update t set id = id + 1; +--disable_info +SELECT * from t order by id limit 2; + +--echo ## update partition column, old and new record locates on different partitions +--enable_info +update t set id = 20 where id = 8; +--disable_info +SELECT * from t order by id limit 2; + +--echo ## table option is auto-increment +drop table if exists t; +create table t (id int not null auto_increment, name varchar(255), primary key(id)) + PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21)); +insert into t(name) values ('aa'); +--enable_info +update t set id = 8 where name = 'aa'; +--disable_info +insert into t(name) values ('bb'); +select * from t; +-- error 1048 +update t set id = null where name = 'aa'; + +--echo ## Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. +drop table if exists t; +create table t (id int, name int unique) + PARTITION BY RANGE ( name ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21)); +insert t values (1, 1), (2, 2); +-- error 1062 +update t set name = 1 where id = 2; +select * from t; + +--echo ## test update ignore for pimary key +drop table if exists t; +create table t(a bigint, primary key (a)) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11)); +insert into t values (5); +insert into t values (7); +update ignore t set a = 5 where a = 7; +SHOW WARNINGS; +select * from t order by a; + +--echo ## test update ignore for truncate as warning +update ignore t set a = 1 where a = (select '2a'); +SHOW WARNINGS; + +--echo ## test update ignore for unique key +drop table if exists t; +create table t(a bigint, unique key I_uniq (a)) + PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11)); +insert into t values (5); +insert into t values (7); +--enable_info +update ignore t set a = 5 where a = 7; +--disable_info +SHOW WARNINGS; +select * from t order by a; +set tidb_opt_fix_control = default; + + +# TestPartitionedTableDelete +drop table if exists t; +set tidb_opt_fix_control = "44262:ON"; +CREATE TABLE t (id int not null default 1, name varchar(255), index(id)) + PARTITION BY RANGE ( id ) ( + PARTITION p0 VALUES LESS THAN (6), + PARTITION p1 VALUES LESS THAN (11), + PARTITION p2 VALUES LESS THAN (16), + PARTITION p3 VALUES LESS THAN (21)); +insert into t values (1, "hello"),(2, "hello"),(3, "hello"),(4, "hello"),(5, "hello"),(6, "hello"),(7, "hello"),(8, "hello"),(9, "hello"),(10, "hello"),(11, "hello"),(12, "hello"),(13, "hello"),(14, "hello"),(15, "hello"),(16, "hello"),(17, "hello"),(18, "hello"),(19, "hello"),(20, "hello"); +--enable_info +delete from t where id = 2 limit 1; + +--echo ## Test delete with false condition +delete from t where 0; +--disable_info + +insert into t values (2, 'abc'); +--enable_info +delete from t where t.id = 2 limit 1; +--disable_info + +--echo ## Test delete ignore +insert into t values (2, 'abc'); +## TODO: https://github.com/pingcap/tidb/issues/48120 +--replace_regex /INTEGER/DOUBLE/ +-- error 1292 +delete from t where id = (select '2a'); +--enable_info +delete ignore from t where id = (select '2a'); +--disable_info +SHOW WARNINGS; + +--echo ## Test delete without using index, involve multiple partitions. +--enable_info +delete from t ignore index(id) where id >= 13 and id <= 17; +--disable_info + +admin check table t; +--enable_info +delete from t; +--disable_info + +--echo ## Fix that partitioned table should not use PointGetPlan. +drop table if exists t1; +create table t1 (c1 bigint, c2 bigint, c3 bigint, primary key(c1)) partition by range (c1) (partition p0 values less than (3440)); +insert into t1 values (379, 379, 379); +--enable_info +delete from t1 where c1 = 379; +--disable_info +drop table t1; + +set tidb_opt_fix_control=default; + + +# TestHashPartitionedTableReplace +set @@session.tidb_enable_table_partition = '1'; +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) + partition by hash(id) partitions 4; +replace replace_test (c1) values (1),(2),(NULL); +begin; +-- error 1136 +replace replace_test (c1) values (); +rollback; +begin; +-- error 1136 +replace replace_test (c1, c2) values (1,2),(1); +rollback; +begin; +-- error 1054 +replace replace_test (xxx) values (3); +rollback; +begin; +-- error 1146 +replace replace_test_xxx (c1) values (); +rollback; +begin; +-- error 1110 +replace replace_test set c1 = 4, c1 = 5; +rollback; +begin; +-- error 1054 +replace replace_test set xxx = 6; +rollback; +replace replace_test set c1 = 3; +replace replace_test set c1 = 4; +replace replace_test set c1 = 5; +replace replace_test set c1 = 6; +replace replace_test set c1 = 7; + +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int) partition by hash(id) partitions 5; +replace replace_test_1 select id, c1 from replace_test; + +drop table if exists replace_test_2; +create table replace_test_2 (id int, c1 int) partition by hash(id) partitions 6; +replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +begin; +-- error 1136 +replace replace_test_1 select c1 from replace_test; +rollback; + +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)) partition by hash(c2) partitions 7; +replace into replace_test_3 set c2=8; +--enable_info +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c1=8, c2=8; +--disable_info +replace into replace_test_3 set c2=NULL; +--enable_info +replace into replace_test_3 set c2=NULL; +--disable_info + +replace into replace_test_3 set c2=0; +replace into replace_test_3 set c2=1; +replace into replace_test_3 set c2=2; +replace into replace_test_3 set c2=3; +replace into replace_test_3 set c2=4; +replace into replace_test_3 set c2=5; +replace into replace_test_3 set c2=6; +replace into replace_test_3 set c2=7; +replace into replace_test_3 set c2=8; +replace into replace_test_3 set c2=9; +select count(*) from replace_test_3; + +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by hash(c1) partitions 8; +replace into replace_test_4 set c2=NULL; +--enable_info +replace into replace_test_4 set c2=NULL; +--disable_info + +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by hash (c2) partitions 9; +replace into replace_test_5 set c1=1, c2=2; +--enable_info +replace into replace_test_5 set c1=1, c2=2; +--disable_info + +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by hash (b) partitions 10; +insert into tIssue989 (a, b) values (1, 2); +replace into tIssue989(a, b) values (111, 2); +select * from tIssue989; + +set @@session.tidb_enable_table_partition = default; diff --git a/tests/integrationtest/t/executor/stale_txn.test b/tests/integrationtest/t/executor/stale_txn.test index 0d11911ead762..77c0998c0aceb 100644 --- a/tests/integrationtest/t/executor/stale_txn.test +++ b/tests/integrationtest/t/executor/stale_txn.test @@ -32,3 +32,10 @@ set tidb_txn_mode = default; set tx_isolation = default; set autocommit = default; +# TestIssue33728 +drop table if exists t1; +create table t1 (id int primary key, v int); +--error 8135 +select * from t1 as of timestamp NULL; +--error 8135 +start transaction read only as of timestamp NULL; diff --git a/tests/integrationtest/t/executor/update.test b/tests/integrationtest/t/executor/update.test index 2eed96a2cd228..26989f69b92ce 100644 --- a/tests/integrationtest/t/executor/update.test +++ b/tests/integrationtest/t/executor/update.test @@ -266,3 +266,433 @@ update t1 set a='1000000000000000000' where id=2; select id, a from t1 order by id asc; set sql_mode=default; +# TestIssue21447 +drop table if exists t1; +create table t1(id int primary key, name varchar(40)); +insert into t1 values(1, 'abc'); + +begin pessimistic; + +connect (conn1, localhost, root,, executor__update); +begin pessimistic; +--enable_info +update t1 set name='xyz' where id=1; +--disable_info +select * from t1 where id = 1; +commit; +disconnect conn1; + +--enable_info +update t1 set name='xyz' where id=1; +--disable_info +select * from t1 where id = 1; +select * from t1 where id = 1 for update; +select * from t1 where id in (1, 2); +select * from t1 where id in (1, 2) for update; +commit; + +# TestUpdate +drop table if exists update_test; +create table update_test(id int not null default 1, name varchar(255), PRIMARY KEY(id)); +insert INTO update_test VALUES (1, "hello"); +insert into update_test values (2, "hello"); +--enable_info +UPDATE update_test SET name = "abc" where id > 0; +--disable_info + +## select data +begin; +SELECT * from update_test limit 2; +commit; + +--enable_info +UPDATE update_test SET name = "foo"; +--disable_info + +## table option is auto-increment +begin; +drop table if exists update_test; +commit; +begin; +create table update_test(id int not null auto_increment, name varchar(255), primary key(id)); +insert into update_test(name) values ('aa'); +--enable_info +update update_test set id = 8 where name = 'aa'; +--disable_info +insert into update_test(name) values ('bb'); +commit; +begin; +select * from update_test; +commit; +begin; +drop table if exists update_test; +commit; +begin; +create table update_test(id int not null auto_increment, name varchar(255), index(id)); +insert into update_test(name) values ('aa'); +-- error 1048 +update update_test set id = null where name = 'aa'; + +drop table update_test; +create table update_test(id int); +begin; +insert into update_test(id) values (1); +--enable_info +update update_test set id = 2 where id = 1 limit 1; +--disable_info +select * from update_test; +commit; + +## Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. +drop table if exists update_unique; +create table update_unique (id int primary key, name int unique); +insert update_unique values (1, 1), (2, 2); +begin; +-- error 1062 +update update_unique set name = 1 where id = 2; +commit; +select * from update_unique; + +## test update ignore for pimary key +drop table if exists t; +create table t(a bigint, primary key (a)); +insert into t values (1); +insert into t values (2); +--enable_info +update ignore t set a = 1 where a = 2; +--disable_info +SHOW WARNINGS; +select * from t; + +## test update ignore for truncate as warning +update ignore t set a = 1 where a = (select '2a'); +SHOW WARNINGS; + +update ignore t set a = 42 where a = 2; +select * from t; + +## test update ignore for unique key +drop table if exists t; +create table t(a bigint, unique key I_uniq (a)); +insert into t values (1); +insert into t values (2); +--enable_info +update ignore t set a = 1 where a = 2; +--disable_info +SHOW WARNINGS; +select * from t; + +## test issue21965 +drop table if exists t; +set @@session.tidb_enable_list_partition = ON; +create table t (a int) partition by list (a) (partition p0 values in (0,1)); +analyze table t; +insert ignore into t values (1); +--enable_info +update ignore t set a=2 where a=1; +--disable_info +drop table if exists t; +create table t (a int key) partition by list (a) (partition p0 values in (0,1)); +insert ignore into t values (1); +--enable_info +update ignore t set a=2 where a=1; +--disable_info +set @@session.tidb_enable_list_partition = default; + +drop table if exists t; +create table t(id integer auto_increment, t1 datetime, t2 datetime, primary key (id)); +insert into t(t1, t2) values('2000-10-01 01:01:01', '2017-01-01 10:10:10'); +select * from t; +--enable_info +update t set t1 = '2017-10-01 10:10:11', t2 = date_add(t1, INTERVAL 10 MINUTE) where id = 1; +--disable_info +select * from t; + +## for issue #5132 +drop table if exists tt1; +CREATE TABLE `tt1` (`a` int(11) NOT NULL,`b` varchar(32) DEFAULT NULL,`c` varchar(32) DEFAULT NULL,PRIMARY KEY (`a`),UNIQUE KEY `b_idx` (`b`)) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; +insert into tt1 values(1, 'a', 'a'); +insert into tt1 values(2, 'd', 'b'); +select * from tt1; +--enable_info +update tt1 set a=5 where c='b'; +--disable_info +select * from tt1; + +## Automatic Updating for TIMESTAMP +drop table if exists tsup; +CREATE TABLE `tsup` (`a` int,`ts` TIMESTAMP DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,KEY `idx` (`ts`)); +set @@sql_mode=''; +insert into tsup values(1, '0000-00-00 00:00:00'); +--enable_info +update tsup set a=5; +--disable_info +select t1.ts = t2.ts from (select ts from tsup use index (idx)) as t1, (select ts from tsup use index ()) as t2; +update tsup set ts='2019-01-01'; +select ts from tsup; +set @@sql_mode=default; + +## issue 5532 +drop table if exists decimals; +create table decimals (a decimal(20, 0) not null); +insert into decimals values (201); +## A warning rather than data truncated error. +--enable_info +update decimals set a = a + 1.23; +--disable_info +show warnings; +select * from decimals; + +drop table t; +CREATE TABLE `t` ( `c1` year DEFAULT NULL, `c2` year DEFAULT NULL, `c3` date DEFAULT NULL, `c4` datetime DEFAULT NULL, KEY `idx` (`c1`,`c2`)); +UPDATE t SET c2=16777215 WHERE c1>= -8388608 AND c1 < -9 ORDER BY c1 LIMIT 2; +-- error 1288 +update (select * from t) t set c1 = 1111111; + +## test update ignore for bad null error +drop table if exists t; +create table t (i int not null default 10); +insert into t values (1); +--enable_info +update ignore t set i = null; +--disable_info +SHOW WARNINGS; +select * from t; + +## issue 7237, update subquery table should be forbidden +drop table t; +create table t (k int, v int); +-- error 1288 +update t, (select * from t) as b set b.k = t.k; +update t, (select * from t) as b set t.k = b.k; + +## issue 8045 +drop table if exists t1; +CREATE TABLE t1 (c1 float); +INSERT INTO t1 SET c1 = 1; +--enable_info +UPDATE t1 SET c1 = 1.2 WHERE c1=1; +--disable_info + +## issue 8119 +drop table if exists t; +create table t (c1 float(1,1)); +insert into t values (0.0); +-- error 1264 +update t set c1 = 2.0; + +drop table if exists t; +create table t(a datetime not null, b datetime); +insert into t value('1999-12-12', '1999-12-13'); +set @@sql_mode=''; +select * from t; +update t set a = ''; +select * from t; +update t set b = ''; +select * from t; +set @@sql_mode=default; + +drop view if exists v; +create view v as select * from t; +-- error 1356 +update v set a = '2000-11-11'; +drop view v; + +drop sequence if exists seq; +create sequence seq; +-- error 1054 +update seq set minvalue=1; +drop sequence seq; + +drop table if exists t1, t2; +create table t1(a int, b int, c int, d int, e int, index idx(a)); +create table t2(a int, b int, c int); +update t1 join t2 on t1.a=t2.a set t1.a=1 where t2.b=1 and t2.c=2; + +## Assign `DEFAULT` in `UPDATE` statement +drop table if exists t1, t2; +create table t1 (a int default 1, b int default 2); +insert into t1 values (10, 10), (20, 20); +update t1 set a=default where b=10; +select * from t1; +update t1 set a=30, b=default where a=20; +select * from t1; +update t1 set a=default, b=default where a=30; +select * from t1; +insert into t1 values (40, 40); +update t1 set a=default, b=default; +select * from t1; +update t1 set a=default(b), b=default(a); +select * from t1; +## With generated columns +create table t2 (a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored); +insert into t2 values (10, default, default), (20, default, default); +update t2 set b=default; +select * from t2; +update t2 set a=30, b=default where a=10; +select * from t2; +update t2 set c=default, a=40 where c=-20; +select * from t2; +update t2 set a=default, b=default, c=default where b=-30; +select * from t2; +update t2 set a=default(a), b=default, c=default; +select * from t2; +## Same as in MySQL 8.0.27, but still weird behavior: a=default(b) => NULL +update t2 set a=default(b), b=default, c=default; +select * from t2; +-- error 3105 +update t2 set b=default(a); +update t2 set a=default(a), c=default(c); +select * from t2; +## Same as in MySQL 8.0.27, but still weird behavior: a=default(b) => NULL +update t2 set a=default(b), b=default(b); +select * from t2; +update t2 set a=default(a), c=default(c); +select * from t2; +## Allowed in MySQL, but should probably not be allowed. +-- error 3105 +update t2 set a=default(a), c=default(a); +drop table t1, t2; + +# TestUpdateSelect +drop table if exists msg, detail; +create table msg (id varchar(8), b int, status int, primary key (id, b)); +insert msg values ('abc', 1, 1); +create table detail (id varchar(8), start varchar(8), status int, index idx_start(start)); +insert detail values ('abc', '123', 2); +--enable_info +UPDATE msg SET msg.status = (SELECT detail.status FROM detail WHERE msg.id = detail.id); +--disable_info +admin check table msg; + +# TestUpdateDelete +drop table if exists ttt; +CREATE TABLE ttt (id bigint(20) NOT NULL, host varchar(30) NOT NULL, PRIMARY KEY (id), UNIQUE KEY i_host (host)); +insert into ttt values (8,8),(9,9); +begin; +--enable_info +update ttt set id = 0, host='9' where id = 9 limit 1; +--disable_info +delete from ttt where id = 0 limit 1; +select * from ttt use index (i_host) order by host; +--enable_info +update ttt set id = 0, host='8' where id = 8 limit 1; +--disable_info +delete from ttt where id = 0 limit 1; +select * from ttt use index (i_host) order by host; +commit; +admin check table ttt; +drop table ttt; + +# TestUpdateAffectRowCnt +drop table if exists a; +create table a(id int auto_increment, a int default null, primary key(id)); +insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1); +--enable_info +update a set id = id*10 where a = 1001; +--disable_info +drop table a; +create table a ( a bigint, b bigint); +insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1); +--enable_info +update a set a = a*10 where b = 1001; +--disable_info + +# TestMultipleTableUpdate +drop table if exists items, month; +CREATE TABLE items (id int, price TEXT); +--enable_info +insert into items values (11, "items_price_11"), (12, "items_price_12"), (13, "items_price_13"); +--disable_info +CREATE TABLE month (mid int, mprice TEXT); +--enable_info +insert into month values (11, "month_price_11"), (22, "month_price_22"), (13, "month_price_13"); +UPDATE items, month SET items.price=month.mprice WHERE items.id=month.mid; +--disable_info +begin; +SELECT * FROM items; +commit; + +## Single-table syntax but with multiple tables +--enable_info +UPDATE items join month on items.id=month.mid SET items.price=month.mid; +--disable_info +begin; +SELECT * FROM items; +commit; + +## JoinTable with alias table name. +--enable_info +UPDATE items T0 join month T1 on T0.id=T1.mid SET T0.price=T1.mprice; +--disable_info +begin; +SELECT * FROM items; +commit; + +## fix https://github.com/pingcap/tidb/issues/369 +DROP TABLE IF EXISTS t1, t2; +create table t1 (c int); +create table t2 (c varchar(256)); +insert into t1 values (1), (2); +insert into t2 values ("a"), ("b"); +--enable_info +update t1, t2 set t1.c = 10, t2.c = "abc"; +--disable_info + +## fix https://github.com/pingcap/tidb/issues/376 +DROP TABLE IF EXISTS t1, t2; +create table t1 (c1 int); +create table t2 (c2 int); +insert into t1 values (1), (2); +insert into t2 values (1), (2); +--enable_info +update t1, t2 set t1.c1 = 10, t2.c2 = 2 where t2.c2 = 1; +--disable_info +select * from t1; + +## test https://github.com/pingcap/tidb/issues/3604 +drop table if exists t; +create table t (a int, b int); +--enable_info +insert into t values(1, 1), (2, 2), (3, 3); +update t m, t n set m.a = m.a + 1; +--disable_info +select * from t; +--enable_info +update t m, t n set n.a = n.a - 1, n.b = n.b + 1; +--disable_info +select * from t; + +# TestUpdateCastOnlyModifiedValues for issue #4514. +drop table if exists update_modified; +create table update_modified (col_1 int, col_2 enum('a', 'b')); +set SQL_MODE=''; +insert into update_modified values (0, 3); +SELECT * FROM update_modified; +set SQL_MODE=STRICT_ALL_TABLES; +--enable_info +update update_modified set col_1 = 1; +--disable_info +SELECT * FROM update_modified; +-- error 1265 +update update_modified set col_1 = 2, col_2 = 'c'; +SELECT * FROM update_modified; +--enable_info +update update_modified set col_1 = 3, col_2 = 'a'; +--disable_info +SELECT * FROM update_modified; + +## Test update a field with different column type. +drop table if exists update_with_diff_type; +CREATE TABLE update_with_diff_type (a int, b JSON); +INSERT INTO update_with_diff_type VALUES(3, '{"a": "测试"}'); +--enable_info +UPDATE update_with_diff_type SET a = '300'; +--disable_info +SELECT a FROM update_with_diff_type; +--enable_info +UPDATE update_with_diff_type SET b = '{"a": "\\u6d4b\\u8bd5"}'; +--disable_info +SELECT b FROM update_with_diff_type; +set SQL_MODE=default; + diff --git a/tests/integrationtest/t/executor/write.test b/tests/integrationtest/t/executor/write.test index 6ff53470eb9ff..a76250b25ab29 100644 --- a/tests/integrationtest/t/executor/write.test +++ b/tests/integrationtest/t/executor/write.test @@ -1216,3 +1216,137 @@ select b from t where a=1; set @@session.tidb_enable_list_partition = default; set @@allow_auto_random_explicit_insert = default; + + +# TestReplace +drop table if exists replace_test; +create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1); +--enable_info +replace replace_test (c1) values (1),(2),(NULL); +--disable_info +begin; +-- error 1136 +replace replace_test (c1) values (); +rollback; +begin; +-- error 1136 +replace replace_test (c1, c2) values (1,2),(1); +rollback; +begin; +-- error 1054 +replace replace_test (xxx) values (3); +rollback; +begin; +-- error 1146 +replace replace_test_xxx (c1) values (); +rollback; +--enable_info +replace replace_test set c1 = 3; +--disable_info +begin; +-- error 1110 +replace replace_test set c1 = 4, c1 = 5; +rollback; +begin; +-- error 1054 +replace replace_test set xxx = 6; +rollback; + +drop table if exists replace_test_1; +create table replace_test_1 (id int, c1 int); +--enable_info +replace replace_test_1 select id, c1 from replace_test; +--disable_info +begin; +-- error 1135 +replace replace_test_0 select c1 from replace_test; +rollback; + +create table replace_test_2 (id int, c1 int); +--enable_info +replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test; +--disable_info + +drop table if exists replace_test_3; +create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2)); +--enable_info +replace into replace_test_3 set c2=1; +replace into replace_test_3 set c2=1; +replace into replace_test_3 set c1=1, c2=1; +replace into replace_test_3 set c2=NULL; +replace into replace_test_3 set c2=NULL; +--disable_info + +drop table if exists replace_test_4; +create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)); +--enable_info +replace into replace_test_4 set c2=NULL; +replace into replace_test_4 set c2=NULL; +--disable_info + +drop table if exists replace_test_5; +create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)); +--enable_info +replace into replace_test_5 set c1=1, c2=2; +replace into replace_test_5 set c1=1, c2=2; +--disable_info + +drop table if exists tIssue989; +CREATE TABLE tIssue989 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b)); +--enable_info +insert into tIssue989 (a, b) values (1, 2); +replace into tIssue989(a, b) values (111, 2); +--disable_info +select * from tIssue989; + +drop table if exists tIssue1012; +CREATE TABLE tIssue1012 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b)); +insert into tIssue1012 (a, b) values (1, 2); +insert into tIssue1012 (a, b) values (2, 1); +--enable_info +replace into tIssue1012(a, b) values (1, 1); +--disable_info +select * from tIssue1012; + +drop table if exists t1; +create table t1(a int primary key, b int); +insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5); +--enable_info +replace into t1 values(1,1); +replace into t1 values(1,1),(2,2); +replace into t1 values(4,14),(5,15),(6,16),(7,17),(8,18); +replace into t1 select * from (select 1, 2) as tmp; +--disable_info + +drop table if exists t1, t2; +create table t1 (a int primary key, b int default 20, c int default 30); +insert into t1 value (1, 2, 3); +replace t1 set a=1, b=default; +select * from t1; +replace t1 set a=2, b=default, c=default; +select * from t1; +replace t1 set a=2, b=default(c), c=default(b); +select * from t1; +replace t1 set a=default(b)+default(c); +select * from t1; +create table t2 (pk int primary key, a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored); +replace t2 set pk=1, b=default; +select * from t2; +replace t2 set pk=2, a=10, b=default; +select * from t2; +replace t2 set pk=2, c=default, a=20; +select * from t2; +replace t2 set pk=2, a=default, b=default, c=default; +select * from t2; +replace t2 set pk=3, a=default(a), b=default, c=default; +select * from t2; +-- error 3105 +replace t2 set b=default(a); +-- error 3105 +replace t2 set a=default(b), b=default(b); +-- error 1364 +replace t2 set a=default(a), c=default(c); +-- error 3105 +replace t2 set c=default(a); +drop table t1, t2; + diff --git a/tests/integrationtest/t/planner/core/binary_plan.test b/tests/integrationtest/t/planner/core/binary_plan.test new file mode 100644 index 0000000000000..5aae531269fd8 --- /dev/null +++ b/tests/integrationtest/t/planner/core/binary_plan.test @@ -0,0 +1,10 @@ +# TestInvalidDecodeBinaryPlan +select tidb_decode_binary_plan('some random bytes'); +show warnings; +## base64.StdEncoding.EncodeToString("some random bytes") +select tidb_decode_binary_plan('c29tZSByYW5kb20gYnl0ZXM='); +show warnings; +## base64.StdEncoding.EncodeToString(snappy.Encode(nil, "some random bytes")) +select tidb_decode_binary_plan('EUBzb21lIHJhbmRvbSBieXRlcw=='); +show warnings; + diff --git a/tests/integrationtest/t/planner/core/cbo.test b/tests/integrationtest/t/planner/core/cbo.test index 09bb8aa09203e..7eab31bc66b5e 100644 --- a/tests/integrationtest/t/planner/core/cbo.test +++ b/tests/integrationtest/t/planner/core/cbo.test @@ -59,6 +59,6 @@ create table t2(a int, b int); insert into t1 values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5); insert into t2 values (2, 22), (3, 33), (5, 55), (233, 2), (333, 3), (3434, 5); analyze table t1, t2; ---replace_regex /:[ ]?[.0-9]+[nµms]*/:/ /, scan_detail: {.*}// / tot_proc:.*?, tot_wait:.*?,// /[0-9]+ Bytes/ Bytes/ /[.0-9]+ KB/ KB/ +--replace_regex /:[ ]?[.0-9]+[nµms]*/:/ /, scan_detail: {.*}// / tot_proc:.*?, tot_wait:.*?,// /[.0-9]+ ((KB)|(Bytes))// explain analyze select t1.a, t1.b, sum(t1.c) from t1 join t2 on t1.a = t2.b where t1.a > 1; set sql_mode=default; diff --git a/tests/integrationtest/t/planner/core/plan.test b/tests/integrationtest/t/planner/core/plan.test index fcdd997eef335..896fd62e082f7 100644 --- a/tests/integrationtest/t/planner/core/plan.test +++ b/tests/integrationtest/t/planner/core/plan.test @@ -162,3 +162,39 @@ alter table t set hypo tiflash replica 0; explain select a from t; +# TestIssue40857 +drop table if exists t; +CREATE TABLE t (c1 mediumint(9) DEFAULT '-4747160',c2 year(4) NOT NULL DEFAULT '2075',c3 double DEFAULT '1.1559030660251948',c4 enum('wbv4','eli','d8ym','m3gsx','lz7td','o','d1k7l','y1x','xcxq','bj','n7') DEFAULT 'xcxq',c5 int(11) DEFAULT '255080866',c6 tinyint(1) DEFAULT '1',PRIMARY KEY (c2),KEY `c4d86d54-091c-4307-957b-b164c9652b7f` (c6,c4) ); +insert into t values (-4747160, 2075, 722.5719203870632, 'xcxq', 1576824797, 1); +--enable_info +select /*+ stream_agg() */ bit_or(t.c5) as r0 from t where t.c3 in (select c6 from t where not(t.c6 <> 1) and not(t.c3 in(9263.749352636818))) group by t.c1; +--disable_info + + +# TestIssue40535 +drop table if exists t1; +drop table if exists t2; +CREATE TABLE `t1`(`c1` bigint(20) NOT NULL DEFAULT '-2312745469307452950', `c2` datetime DEFAULT '5316-02-03 06:54:49', `c3` tinyblob DEFAULT NULL, PRIMARY KEY (`c1`) /*T![clustered_index] CLUSTERED */) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; +CREATE TABLE `t2`(`c1` set('kn8pu','7et','vekx6','v3','liwrh','q14','1met','nnd5i','5o0','8cz','l') DEFAULT '7et,vekx6,liwrh,q14,1met', `c2` float DEFAULT '1.683167', KEY `k1` (`c2`,`c1`), KEY `k2` (`c2`)) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_chinese_ci; +--enable_info +(select /*+ agg_to_cop()*/ locate(t1.c3, t1.c3) as r0, t1.c3 as r1 from t1 where not( IsNull(t1.c1)) order by r0,r1) union all (select concat_ws(',', t2.c2, t2.c1) as r0, t2.c1 as r1 from t2 order by r0, r1) order by 1 limit 273; +--disable_info + + +# TestIssue47445 +drop table if exists golang1, golang2; +CREATE TABLE golang1 ( `fcbpdt` CHAR (8) COLLATE utf8_general_ci NOT NULL, `fcbpsq` VARCHAR (20) COLLATE utf8_general_ci NOT NULL, `procst` char (4) COLLATE utf8_general_ci DEFAULT NULL,`cipstx` VARCHAR (105) COLLATE utf8_general_ci DEFAULT NULL, `cipsst` CHAR (4) COLLATE utf8_general_ci DEFAULT NULL, `dyngtg` VARCHAR(4) COLLATE utf8_general_ci DEFAULT NULL, `blncdt` VARCHAR (8) COLLATE utf8_general_ci DEFAULT NULL, PRIMARY KEY ( fcbpdt, fcbpsq )); +insert into golang1 values('20230925','12023092502158016','abc','','','',''); +create table golang2 (`sysgrp` varchar(20) NOT NULL,`procst` varchar(8) NOT NULL,`levlid` int(11) NOT NULL,PRIMARY key (procst));; +insert into golang2 VALUES('COMMON','ACSC',90); +insert into golang2 VALUES('COMMON','abc',8); +insert into golang2 VALUES('COMMON','CH02',6); +UPDATE golang1 a SET procst =(CASE WHEN ( SELECT levlid FROM golang2 b WHERE b.sysgrp = 'COMMON' AND b.procst = 'ACSC' ) > ( SELECT levlid FROM golang2 c WHERE c.sysgrp = 'COMMON' AND c.procst = a.procst ) THEN 'ACSC' ELSE a.procst END ), cipstx = 'CI010000', cipsst = 'ACSC', dyngtg = 'EAYT', blncdt= '20230925' WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'; +select * from golang1; +UPDATE golang1 a SET procst= (SELECT 1 FROM golang2 c WHERE c.procst = a.procst) WHERE fcbpdt = '20230925' AND fcbpsq = '12023092502158016'; +select * from golang1; + + +# TestExplainValuesStatement +--error 1051 +EXPLAIN FORMAT = TRADITIONAL ((VALUES ROW ()) ORDER BY 1); diff --git a/tests/integrationtest/t/planner/core/plan_cache.test b/tests/integrationtest/t/planner/core/plan_cache.test index 1be4a42c71b6a..23d20b496bb1d 100644 --- a/tests/integrationtest/t/planner/core/plan_cache.test +++ b/tests/integrationtest/t/planner/core/plan_cache.test @@ -1431,3 +1431,227 @@ select database() from t; select @@last_plan_from_cache; set tidb_enable_non_prepared_plan_cache=DEFAULT; +# TestPlanCacheSizeSwitch +select @@tidb_prepared_plan_cache_size; +select @@tidb_session_plan_cache_size; +set @@tidb_prepared_plan_cache_size = 200; +select @@tidb_prepared_plan_cache_size; +select @@tidb_session_plan_cache_size; +set @@tidb_session_plan_cache_size = 300; +select @@tidb_prepared_plan_cache_size; +select @@tidb_session_plan_cache_size; + +set global tidb_prepared_plan_cache_size = 400; +connect (conn1, localhost, root,,); +select @@tidb_prepared_plan_cache_size; +select @@tidb_session_plan_cache_size; +disconnect conn1; + +set global tidb_session_plan_cache_size = 500; +connect (conn1, localhost, root,,); +select @@tidb_prepared_plan_cache_size; +select @@tidb_session_plan_cache_size; +disconnect conn1; + +set global tidb_prepared_plan_cache_size = default; +set global tidb_session_plan_cache_size = default; +set @@tidb_session_plan_cache_size = default; +set @@tidb_prepared_plan_cache_size = default; + +# TestNonPreparedPlanCacheWithExplain +drop table if exists t; +create table t(a int); +set tidb_enable_non_prepared_plan_cache=1; +select * from t where a=1; +explain select * from t where a=2; +select @@last_plan_from_cache; +explain format=verbose select * from t where a=2; +select @@last_plan_from_cache; +--replace_column 5 6 7 8 9 +explain analyze select * from t where a=2; +select @@last_plan_from_cache; +set tidb_enable_non_prepared_plan_cache=default; + +# TestPlanCacheGeneratedCols +drop table if exists t1, t2, t3, t4; +set @@tidb_opt_fix_control = "45798:on"; +create table t1 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city')))); +create table t2 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) virtual); +create table t3 (a int, info json, city varchar(64) as (JSON_UNQUOTE(JSON_EXTRACT(info, '$.city'))) stored); +create table t4 (a int, info json, index zips( (CAST(info->'$.zipcode' AS UNSIGNED ARRAY)))); +set @a=1; +set @b=2; +prepare s1 from 'select * from t1 where a=?'; +show warnings; +execute s1 using @a; +execute s1 using @b; +select @@last_plan_from_cache; +prepare s1 from 'select * from t2 where a=?'; +show warnings; +execute s1 using @a; +execute s1 using @b; +select @@last_plan_from_cache; +prepare s1 from 'select * from t3 where a=?'; +show warnings; +execute s1 using @a; +execute s1 using @b; +select @@last_plan_from_cache; +prepare s1 from 'select * from t4 where a=?'; +show warnings; +execute s1 using @a; +execute s1 using @b; +select @@last_plan_from_cache; + +set @@tidb_opt_fix_control = default; + +# TestPlanCacheGeneratedCols2 +drop table if exists t1, t2; +set @@tidb_opt_fix_control = "45798:on"; +CREATE TABLE t1 ( + ipk varbinary(255) NOT NULL, + i_id varchar(45) DEFAULT NULL, + i_set_id varchar(45) DEFAULT NULL, + p_id varchar(45) DEFAULT NULL, + p_set_id varchar(45) DEFAULT NULL, + m_id bigint(20) DEFAULT NULL, + m_i_id varchar(127) DEFAULT NULL, + m_i_set_id varchar(127) DEFAULT NULL, + d json DEFAULT NULL, + p_sources json DEFAULT NULL, + nslc json DEFAULT NULL, + cl json DEFAULT NULL, + fii json DEFAULT NULL, + fpi json DEFAULT NULL, + PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */, + UNIQUE KEY i_id (i_id), + KEY d ((cast(d as char(253) array))), + KEY m_i_id (m_i_id), + KEY m_i_set_id (m_i_set_id), + KEY fpi ((cast(fpi as unsigned array))), + KEY nslc ((cast(nslc as char(1000) array))), + KEY cl ((cast(cl as char(3000) array))), + KEY fii ((cast(fii as unsigned array))), + KEY m_id (m_id), + KEY i_set_id (i_set_id), + KEY m_i_and_m_id (m_i_id,m_id)); +CREATE TABLE t2 ( + ipk varbinary(255) NOT NULL, + created_time bigint(20) DEFAULT NULL, + arrival_time bigint(20) DEFAULT NULL, + updated_time bigint(20) DEFAULT NULL, + timestamp_data json DEFAULT NULL, + PRIMARY KEY (ipk) /*T![clustered_index] CLUSTERED */); +prepare stmt from 'select * + from ( t1 left outer join t2 on ( t1 . ipk = t2 . ipk ) ) + where ( t1 . i_id = ? )'; +show warnings; +set @a='a', @b='b'; +execute stmt using @a; +execute stmt using @b; +select @@last_plan_from_cache; + +set @@tidb_opt_fix_control = default; + +# TestPlanCacheExprBlacklistCompatibility +drop table if exists t; +create table t (a int); +prepare st from 'select * from t where mod(a, 2)=1'; +execute st; +execute st; +select @@last_plan_from_cache; +insert into mysql.expr_pushdown_blacklist(name) values('mod'); +admin reload expr_pushdown_blacklist; +execute st; +select @@last_plan_from_cache; +execute st; +select @@last_plan_from_cache; + +delete from mysql.expr_pushdown_blacklist; +admin reload expr_pushdown_blacklist; + +# TestNonPreparedPlanCacheFieldNames +drop table if exists t, tt; +create table t(a int, index(a)); +create table tt(a varchar(10)); +set tidb_enable_non_prepared_plan_cache=1; +select a+1 from t where a<10; +select @@last_plan_from_cache; +select a+1 from t where a<20; +select @@last_plan_from_cache; +select a+2 from t where a<30; +select @@last_plan_from_cache; +select a+2 from t where a<40; +select @@last_plan_from_cache; +select a,a+1 from t where a<30; +select @@last_plan_from_cache; +select a,a+1 from t where a<40; +select @@last_plan_from_cache; +select a+'123' from tt where a='1'; +select @@last_plan_from_cache; +select a+'123' from tt where a='2'; +select @@last_plan_from_cache; +select 1 from t where a<10; +select @@last_plan_from_cache; +select 1 from t where a<20; +select @@last_plan_from_cache; +select 2 from t where a<10; +select @@last_plan_from_cache; +select 2 from t where a<20; +select @@last_plan_from_cache; +select 1,2 from t where a<10; +select @@last_plan_from_cache; +select 1,2 from t where a<20; +select @@last_plan_from_cache; +set tidb_enable_non_prepared_plan_cache=default; + +# TestIssue48165 +drop table if exists t; +create table t(a int); +insert into t values(1); +prepare s from "select * from t where tidb_parse_tso(a) > unix_timestamp()"; +execute s; + +# TestNonPreparedPlanCacheBuiltinFuncs +drop table if exists t; +set tidb_enable_non_prepared_plan_cache=1; +create table t (a int, b varchar(32), c datetime, key(a)); + +## normal builtin functions can be supported +select * from t where mod(a, 5) < 2; +select * from t where mod(a, 5) < 2; +select @@last_plan_from_cache; +select * from t where c < now(); +select * from t where c < now(); +select @@last_plan_from_cache; +select date_format(c, '%Y-%m-%d') from t where a < 10; +select date_format(c, '%Y-%m-%d') from t where a < 10; +select @@last_plan_from_cache; +select str_to_date(b, '%Y-%m-%d') from t where a < 10; +select str_to_date(b, '%Y-%m-%d') from t where a < 10; +select @@last_plan_from_cache; +select * from t where a-2 < 20; +select * from t where a-2 < 20; +select @@last_plan_from_cache; +select * from t where a+b > 100; +select * from t where a+b > 100; +select @@last_plan_from_cache; + +## '-' cannot support +select * from t where -a > 10; +select * from t where -a > 10; +select @@last_plan_from_cache; +## LIKE +select * from t where a < 1 and b like '%abc%'; +select * from t where a < 1 and b like '%abc%'; +select @@last_plan_from_cache; +select database() from t; +select database() from t; +select @@last_plan_from_cache; +set tidb_enable_non_prepared_plan_cache=default; + +# TestIssue45086 +drop table if exists t; +CREATE TABLE t (a int(11) DEFAULT NULL, b date DEFAULT NULL); +INSERT INTO t VALUES (1, current_date()); +PREPARE stmt FROM 'SELECT a FROM t WHERE b=current_date()'; +EXECUTE stmt; From 7e8de2f1c7bd763822110103169448afbaea3124 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Thu, 23 Nov 2023 12:53:41 +0800 Subject: [PATCH 19/36] infosync: integrate PD HTTP client into the label manager (#48738) ref pingcap/tidb#35319 --- DEPS.bzl | 12 ++-- go.mod | 2 +- go.sum | 4 +- pkg/ddl/attributes_sql_test.go | 8 +-- pkg/ddl/label/BUILD.bazel | 2 + pkg/ddl/label/attributes.go | 41 ++++++-------- pkg/ddl/label/attributes_test.go | 31 +++++----- pkg/ddl/label/rule.go | 38 ++++++------- pkg/ddl/label/rule_test.go | 11 ++-- pkg/ddl/main_test.go | 2 +- pkg/ddl/partition.go | 4 +- pkg/ddl/tests/serial/main_test.go | 2 +- pkg/domain/BUILD.bazel | 1 + pkg/domain/db_test.go | 4 +- pkg/domain/domain.go | 15 ++++- pkg/domain/infosync/info.go | 49 +++++++++++++--- pkg/domain/infosync/info_test.go | 6 +- pkg/domain/infosync/label_manager.go | 56 ++++++------------- pkg/executor/infoschema_reader.go | 15 ++--- pkg/kv/BUILD.bazel | 1 + pkg/kv/kv.go | 2 + pkg/server/handler/tests/BUILD.bazel | 1 + pkg/server/handler/tests/http_handler_test.go | 15 ++--- pkg/server/stat_test.go | 2 +- pkg/store/gcworker/gc_worker.go | 2 +- 25 files changed, 170 insertions(+), 156 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 4ae046fa60f2d..d0adcb9d90879 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -7158,13 +7158,13 @@ def go_deps(): name = "com_github_tikv_pd_client", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/pd/client", - sha256 = "014bb8796797b8b5cecc22866a1aab8491e3718c540168ac91257cf7f220cc84", - strip_prefix = "github.com/tikv/pd/client@v0.0.0-20231117041718-dda748abe55d", + sha256 = "440821579da980d0405695b463da892608a59252a296cd7e52b4f97881c5fdb7", + strip_prefix = "github.com/tikv/pd/client@v0.0.0-20231121080541-8919bc11f770", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20231117041718-dda748abe55d.zip", - "http://ats.apps.svc/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20231117041718-dda748abe55d.zip", - "https://cache.hawkingrei.com/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20231117041718-dda748abe55d.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20231117041718-dda748abe55d.zip", + "http://bazel-cache.pingcap.net:8080/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20231121080541-8919bc11f770.zip", + "http://ats.apps.svc/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20231121080541-8919bc11f770.zip", + "https://cache.hawkingrei.com/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20231121080541-8919bc11f770.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/tikv/pd/client/com_github_tikv_pd_client-v0.0.0-20231121080541-8919bc11f770.zip", ], ) go_repository( diff --git a/go.mod b/go.mod index b16a42b492f51..c9942e6ef235e 100644 --- a/go.mod +++ b/go.mod @@ -103,7 +103,7 @@ require ( github.com/tdakkota/asciicheck v0.2.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 github.com/tikv/client-go/v2 v2.0.8-0.20231116051730-1c2351c28173 - github.com/tikv/pd/client v0.0.0-20231117041718-dda748abe55d + github.com/tikv/pd/client v0.0.0-20231121080541-8919bc11f770 github.com/timakin/bodyclose v0.0.0-20230421092635-574207250966 github.com/twmb/murmur3 v1.1.6 github.com/uber/jaeger-client-go v2.22.1+incompatible diff --git a/go.sum b/go.sum index 9e8da840933b6..211a12881ffa8 100644 --- a/go.sum +++ b/go.sum @@ -992,8 +992,8 @@ github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= github.com/tikv/client-go/v2 v2.0.8-0.20231116051730-1c2351c28173 h1:lmJzX0kqrV7kO21wrZPbtjkidzwbDCfXeQrhDWEi5dE= github.com/tikv/client-go/v2 v2.0.8-0.20231116051730-1c2351c28173/go.mod h1:BOGTSZtbMHEnGC4HOpbONdnTQF+E9nb2Io7c3P9sb7g= -github.com/tikv/pd/client v0.0.0-20231117041718-dda748abe55d h1:6isljjnUH8zzkJx2X8MUGh+5AlMv+pCEhCy5MSyuhSM= -github.com/tikv/pd/client v0.0.0-20231117041718-dda748abe55d/go.mod h1:cd6zBqRM9aogxf26K8NnFRPVtq9BnRE59tKEpX8IaWQ= +github.com/tikv/pd/client v0.0.0-20231121080541-8919bc11f770 h1:YSXDKT9+KngRSAShoSQVKD/CK1kR4X/9hutKkSK9gn0= +github.com/tikv/pd/client v0.0.0-20231121080541-8919bc11f770/go.mod h1:cd6zBqRM9aogxf26K8NnFRPVtq9BnRE59tKEpX8IaWQ= github.com/timakin/bodyclose v0.0.0-20230421092635-574207250966 h1:quvGphlmUVU+nhpFa4gg4yJyTRJ13reZMDHrKwYw53M= github.com/timakin/bodyclose v0.0.0-20230421092635-574207250966/go.mod h1:27bSVNWSBOHm+qRp1T9qzaIpsWEP6TbUnei/43HK+PQ= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= diff --git a/pkg/ddl/attributes_sql_test.go b/pkg/ddl/attributes_sql_test.go index c7c240d80e2bd..b5215b9b6ef28 100644 --- a/pkg/ddl/attributes_sql_test.go +++ b/pkg/ddl/attributes_sql_test.go @@ -252,7 +252,7 @@ PARTITION BY RANGE (c) ( func TestFlashbackTable(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), dom.GetPDHTTPClient(), keyspace.CodecV1, true) require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -310,7 +310,7 @@ PARTITION BY RANGE (c) ( func TestDropTable(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), dom.GetPDHTTPClient(), keyspace.CodecV1, true) require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -363,7 +363,7 @@ PARTITION BY RANGE (c) ( func TestCreateWithSameName(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), dom.GetPDHTTPClient(), keyspace.CodecV1, true) require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -427,7 +427,7 @@ PARTITION BY RANGE (c) ( func TestPartition(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), dom.GetPDHTTPClient(), keyspace.CodecV1, true) require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") diff --git a/pkg/ddl/label/BUILD.bazel b/pkg/ddl/label/BUILD.bazel index 3e8f40d4c88ca..75a89a0767e97 100644 --- a/pkg/ddl/label/BUILD.bazel +++ b/pkg/ddl/label/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "//pkg/parser/ast", "//pkg/tablecodec", "//pkg/util/codec", + "@com_github_tikv_pd_client//http", "@in_gopkg_yaml_v2//:yaml_v2", ], ) @@ -32,6 +33,7 @@ go_test( "//pkg/parser/ast", "//pkg/testkit/testsetup", "@com_github_stretchr_testify//require", + "@com_github_tikv_pd_client//http", "@org_uber_go_goleak//:goleak", ], ) diff --git a/pkg/ddl/label/attributes.go b/pkg/ddl/label/attributes.go index b797f66e73168..b51642373cca1 100644 --- a/pkg/ddl/label/attributes.go +++ b/pkg/ddl/label/attributes.go @@ -17,6 +17,8 @@ package label import ( "fmt" "strings" + + pd "github.com/tikv/pd/client/http" ) const ( @@ -37,15 +39,9 @@ const ( AttributesDuplicated ) -// Label is used to describe attributes -type Label struct { - Key string `json:"key,omitempty"` - Value string `json:"value,omitempty"` -} - // NewLabel creates a new label for a given string. -func NewLabel(attr string) (Label, error) { - l := Label{} +func NewLabel(attr string) (pd.RegionLabel, error) { + l := pd.RegionLabel{} kv := strings.Split(attr, "=") if len(kv) != 2 { return l, fmt.Errorf("%w: %s", ErrInvalidAttributesFormat, attr) @@ -66,14 +62,14 @@ func NewLabel(attr string) (Label, error) { return l, nil } -// Restore converts a Attribute to a string. -func (l *Label) Restore() string { +// RestoreRegionLabel converts a Attribute to a string. +func RestoreRegionLabel(l *pd.RegionLabel) string { return l.Key + "=" + l.Value } // CompatibleWith will check if two constraints are compatible. // Return (compatible, duplicated). -func (l *Label) CompatibleWith(o *Label) AttributesCompatibility { +func CompatibleWith(l *pd.RegionLabel, o *pd.RegionLabel) AttributesCompatibility { if l.Key != o.Key { return AttributesCompatible } @@ -85,26 +81,23 @@ func (l *Label) CompatibleWith(o *Label) AttributesCompatibility { return AttributesIncompatible } -// Labels is a slice of Label. -type Labels []Label - // NewLabels creates a slice of Label for given attributes. -func NewLabels(attrs []string) (Labels, error) { - labels := make(Labels, 0, len(attrs)) +func NewLabels(attrs []string) ([]pd.RegionLabel, error) { + labels := make([]pd.RegionLabel, 0, len(attrs)) for _, attr := range attrs { label, err := NewLabel(attr) if err != nil { return nil, err } - if err := labels.Add(label); err != nil { + if err := Add(&labels, label); err != nil { return nil, err } } return labels, nil } -// Restore converts Attributes to a string. -func (labels *Labels) Restore() string { +// RestoreRegionLabels converts Attributes to a string. +func RestoreRegionLabels(labels *[]pd.RegionLabel) string { var sb strings.Builder for i, label := range *labels { switch label.Key { @@ -117,25 +110,25 @@ func (labels *Labels) Restore() string { sb.WriteByte(',') } sb.WriteByte('"') - sb.WriteString(label.Restore()) + sb.WriteString(RestoreRegionLabel(&label)) sb.WriteByte('"') } return sb.String() } // Add will add a new attribute, with validation of all attributes. -func (labels *Labels) Add(label Label) error { +func Add(labels *[]pd.RegionLabel, label pd.RegionLabel) error { for i := range *labels { l := (*labels)[i] - res := label.CompatibleWith(&l) + res := CompatibleWith(&label, &l) if res == AttributesCompatible { continue } if res == AttributesDuplicated { return nil } - s1 := label.Restore() - s2 := l.Restore() + s1 := RestoreRegionLabel(&label) + s2 := RestoreRegionLabel(&l) return fmt.Errorf("'%s' and '%s' are conflicted", s1, s2) } diff --git a/pkg/ddl/label/attributes_test.go b/pkg/ddl/label/attributes_test.go index fcf53db3706b8..a80794a72014d 100644 --- a/pkg/ddl/label/attributes_test.go +++ b/pkg/ddl/label/attributes_test.go @@ -18,20 +18,21 @@ import ( "testing" "github.com/stretchr/testify/require" + pd "github.com/tikv/pd/client/http" ) func TestNewLabel(t *testing.T) { type TestCase struct { name string input string - label Label + label pd.RegionLabel } tests := []TestCase{ { name: "normal", input: "merge_option=allow", - label: Label{ + label: pd.RegionLabel{ Key: "merge_option", Value: "allow", }, @@ -39,7 +40,7 @@ func TestNewLabel(t *testing.T) { { name: "normal with space", input: " merge_option=allow ", - label: Label{ + label: pd.RegionLabel{ Key: "merge_option", Value: "allow", }, @@ -58,7 +59,7 @@ func TestNewLabel(t *testing.T) { func TestRestoreLabel(t *testing.T) { type TestCase struct { name string - input Label + input pd.RegionLabel output string } @@ -83,7 +84,7 @@ func TestRestoreLabel(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - output := test.input.Restore() + output := RestoreRegionLabel(&test.input) require.Equal(t, test.output, output) }) } @@ -124,8 +125,8 @@ func TestNewLabels(t *testing.T) { func TestAddLabels(t *testing.T) { type TestCase struct { name string - labels Labels - label Label + labels []pd.RegionLabel + label pd.RegionLabel err bool } @@ -154,7 +155,7 @@ func TestAddLabels(t *testing.T) { }, { "duplicated attributes, skip", - append(labels, Label{ + append(labels, pd.RegionLabel{ Key: "merge_option", Value: "allow", }), @@ -171,7 +172,7 @@ func TestAddLabels(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - err = test.labels.Add(test.label) + err = Add(&test.labels, test.label) if test.err { require.Error(t, err) } else { @@ -185,7 +186,7 @@ func TestAddLabels(t *testing.T) { func TestRestoreLabels(t *testing.T) { type TestCase struct { name string - input Labels + input []pd.RegionLabel output string } @@ -203,29 +204,29 @@ func TestRestoreLabels(t *testing.T) { tests := []TestCase{ { "normal1", - Labels{}, + []pd.RegionLabel{}, "", }, { "normal2", - Labels{input1, input2}, + []pd.RegionLabel{input1, input2}, `"merge_option=allow","key=value"`, }, { "normal3", - Labels{input3, input4, input5}, + []pd.RegionLabel{input3, input4, input5}, "", }, { "normal4", - Labels{input1, input2, input3}, + []pd.RegionLabel{input1, input2, input3}, `"merge_option=allow","key=value"`, }, } for _, test := range tests { t.Run(test.name, func(t *testing.T) { - output := test.input.Restore() + output := RestoreRegionLabels(&test.input) require.Equal(t, test.output, output) }) } diff --git a/pkg/ddl/label/rule.go b/pkg/ddl/label/rule.go index e3d08999ee8ed..2d5316a7dfe89 100644 --- a/pkg/ddl/label/rule.go +++ b/pkg/ddl/label/rule.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util/codec" + pd "github.com/tikv/pd/client/http" "gopkg.in/yaml.v2" ) @@ -53,13 +54,7 @@ var ( ) // Rule is used to establish the relationship between labels and a key range. -type Rule struct { - ID string `json:"id"` - Index int `json:"index"` - Labels Labels `json:"labels"` - RuleType string `json:"rule_type"` - Data []interface{} `json:"data"` -} +type Rule pd.LabelRule // NewRule creates a rule. func NewRule() *Rule { @@ -69,7 +64,7 @@ func NewRule() *Rule { // ApplyAttributesSpec will transfer attributes defined in AttributesSpec to the labels. func (r *Rule) ApplyAttributesSpec(spec *ast.AttributesSpec) error { if spec.Default { - r.Labels = []Label{} + r.Labels = []pd.RegionLabel{} return nil } // construct a string list @@ -129,26 +124,27 @@ func (r *Rule) Reset(dbName, tableName, partName string, ids ...int64) *Rule { } if !hasDBKey { - r.Labels = append(r.Labels, Label{Key: dbKey, Value: dbName}) + r.Labels = append(r.Labels, pd.RegionLabel{Key: dbKey, Value: dbName}) } if !hasTableKey { - r.Labels = append(r.Labels, Label{Key: tableKey, Value: tableName}) + r.Labels = append(r.Labels, pd.RegionLabel{Key: tableKey, Value: tableName}) } if isPartition && !hasPartitionKey { - r.Labels = append(r.Labels, Label{Key: partitionKey, Value: partName}) + r.Labels = append(r.Labels, pd.RegionLabel{Key: partitionKey, Value: partName}) } r.RuleType = ruleType - r.Data = []interface{}{} + dataSlice := make([]interface{}, 0, len(ids)) slices.Sort(ids) for i := 0; i < len(ids); i++ { data := map[string]string{ "start_key": hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(ids[i]))), "end_key": hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(ids[i]+1))), } - r.Data = append(r.Data, data) + dataSlice = append(dataSlice, data) } + r.Data = dataSlice // We may support more types later. r.Index = RuleIndexTable if isPartition { @@ -157,16 +153,14 @@ func (r *Rule) Reset(dbName, tableName, partName string, ids ...int64) *Rule { return r } -// RulePatch is the patch to update the label rules. -type RulePatch struct { - SetRules []*Rule `json:"sets"` - DeleteRules []string `json:"deletes"` -} - // NewRulePatch returns a patch of rules which need to be set or deleted. -func NewRulePatch(setRules []*Rule, deleteRules []string) *RulePatch { - return &RulePatch{ - SetRules: setRules, +func NewRulePatch(setRules []*Rule, deleteRules []string) *pd.LabelRulePatch { + labelRules := make([]*pd.LabelRule, 0, len(setRules)) + for _, rule := range setRules { + labelRules = append(labelRules, (*pd.LabelRule)(rule)) + } + return &pd.LabelRulePatch{ + SetRules: labelRules, DeleteRules: deleteRules, } } diff --git a/pkg/ddl/label/rule_test.go b/pkg/ddl/label/rule_test.go index f392e3dc58c9e..69928b3614e89 100644 --- a/pkg/ddl/label/rule_test.go +++ b/pkg/ddl/label/rule_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/stretchr/testify/require" + pd "github.com/tikv/pd/client/http" ) func TestApplyAttributesSpec(t *testing.T) { @@ -74,13 +75,13 @@ func TestReset(t *testing.T) { require.Equal(t, "t1", rule.Labels[2].Value) require.Equal(t, rule.Index, 2) - r := rule.Data[0].(map[string]string) + r := rule.Data.([]interface{})[0].(map[string]string) require.Equal(t, "7480000000000000ff0100000000000000f8", r["start_key"]) require.Equal(t, "7480000000000000ff0200000000000000f8", r["end_key"]) - r = rule.Data[1].(map[string]string) + r = rule.Data.([]interface{})[1].(map[string]string) require.Equal(t, "7480000000000000ff0200000000000000f8", r["start_key"]) require.Equal(t, "7480000000000000ff0300000000000000f8", r["end_key"]) - r = rule.Data[2].(map[string]string) + r = rule.Data.([]interface{})[2].(map[string]string) require.Equal(t, "7480000000000000ff0300000000000000f8", r["start_key"]) require.Equal(t, "7480000000000000ff0400000000000000f8", r["end_key"]) @@ -96,14 +97,14 @@ func TestReset(t *testing.T) { require.Equal(t, "p2", rule.Labels[3].Value) require.Equal(t, rule.Index, 3) - r = r2.Data[0].(map[string]string) + r = r2.Data.([]interface{})[0].(map[string]string) require.Equal(t, "7480000000000000ff0200000000000000f8", r["start_key"]) require.Equal(t, "7480000000000000ff0300000000000000f8", r["end_key"]) // default case spec = &ast.AttributesSpec{Default: true} rule, expected := NewRule(), NewRule() - expected.ID, expected.Labels = "schema/db3/t3/p3", []Label{} + expected.ID, expected.Labels = "schema/db3/t3/p3", []pd.RegionLabel{} require.NoError(t, rule.ApplyAttributesSpec(spec)) r3 := rule.Reset("db3", "t3", "p3", 3) require.Equal(t, r3, expected) diff --git a/pkg/ddl/main_test.go b/pkg/ddl/main_test.go index a763a5dff4ff1..405ea5594706a 100644 --- a/pkg/ddl/main_test.go +++ b/pkg/ddl/main_test.go @@ -54,7 +54,7 @@ func TestMain(m *testing.M) { conf.Experimental.AllowsExpressionIndex = true }) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, keyspace.CodecV1, true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, nil, keyspace.CodecV1, true) if err != nil { _, _ = fmt.Fprintf(os.Stderr, "ddl: infosync.GlobalInfoSyncerInit: %v\n", err) os.Exit(1) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 1e6316e6f77a5..7ebba13273101 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -1887,14 +1887,14 @@ func getTableInfoWithOriginalPartitions(t *model.TableInfo, oldIDs []int64, newI return nt } -func dropLabelRules(_ *ddlCtx, schemaName, tableName string, partNames []string) error { +func dropLabelRules(d *ddlCtx, schemaName, tableName string, partNames []string) error { deleteRules := make([]string, 0, len(partNames)) for _, partName := range partNames { deleteRules = append(deleteRules, fmt.Sprintf(label.PartitionIDFormat, label.IDPrefix, schemaName, tableName, partName)) } // delete batch rules patch := label.NewRulePatch([]*label.Rule{}, deleteRules) - return infosync.UpdateLabelRules(context.TODO(), patch) + return infosync.UpdateLabelRules(d.ctx, patch) } // onDropTablePartition deletes old partition meta. diff --git a/pkg/ddl/tests/serial/main_test.go b/pkg/ddl/tests/serial/main_test.go index bb582f02785d0..73100516be9f6 100644 --- a/pkg/ddl/tests/serial/main_test.go +++ b/pkg/ddl/tests/serial/main_test.go @@ -58,7 +58,7 @@ func TestMain(m *testing.M) { conf.Experimental.AllowsExpressionIndex = true }) - _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, keyspace.CodecV1, true) + _, err := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, nil, keyspace.CodecV1, true) if err != nil { _, _ = fmt.Fprintf(os.Stderr, "ddl: infosync.GlobalInfoSyncerInit: %v\n", err) os.Exit(1) diff --git a/pkg/domain/BUILD.bazel b/pkg/domain/BUILD.bazel index 0641165203a8a..7f70278cf5dd6 100644 --- a/pkg/domain/BUILD.bazel +++ b/pkg/domain/BUILD.bazel @@ -96,6 +96,7 @@ go_library( "@com_github_tikv_client_go_v2//tikv", "@com_github_tikv_client_go_v2//txnkv/transaction", "@com_github_tikv_pd_client//:client", + "@com_github_tikv_pd_client//http", "@com_github_tikv_pd_client//resource_group/controller", "@io_etcd_go_etcd_client_v3//:client", "@io_etcd_go_etcd_client_v3//concurrency", diff --git a/pkg/domain/db_test.go b/pkg/domain/db_test.go index bcb0d3f53c3dc..983e526745c93 100644 --- a/pkg/domain/db_test.go +++ b/pkg/domain/db_test.go @@ -74,7 +74,7 @@ func TestNormalSessionPool(t *testing.T) { domain, err := session.BootstrapSession(store) require.NoError(t, err) defer domain.Close() - info, err1 := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, keyspace.CodecV1, true) + info, err1 := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, nil, keyspace.CodecV1, true) require.NoError(t, err1) conf := config.GetGlobalConfig() conf.Socket = "" @@ -107,7 +107,7 @@ func TestAbnormalSessionPool(t *testing.T) { domain, err := session.BootstrapSession(store) require.NoError(t, err) defer domain.Close() - info, err1 := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, keyspace.CodecV1, true) + info, err1 := infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, nil, nil, nil, keyspace.CodecV1, true) require.NoError(t, err1) conf := config.GetGlobalConfig() conf.Socket = "" diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index 60e5cc444ac16..d9bd664fd036f 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -90,6 +90,7 @@ import ( "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/txnkv/transaction" pd "github.com/tikv/pd/client" + pdhttp "github.com/tikv/pd/client/http" rmclient "github.com/tikv/pd/client/resource_group/controller" clientv3 "go.etcd.io/etcd/client/v3" "go.etcd.io/etcd/client/v3/concurrency" @@ -1186,11 +1187,11 @@ func (do *Domain) Init( } // step 1: prepare the info/schema syncer which domain reload needed. - pdCli := do.GetPDClient() + pdCli, pdHTTPCli := do.GetPDClient(), do.GetPDHTTPClient() skipRegisterToDashboard := config.GetGlobalConfig().SkipRegisterToDashboard do.info, err = infosync.GlobalInfoSyncerInit(ctx, do.ddl.GetID(), do.ServerID, - do.etcdClient, do.unprefixedEtcdCli, pdCli, do.Store().GetCodec(), - skipRegisterToDashboard) + do.etcdClient, do.unprefixedEtcdCli, pdCli, pdHTTPCli, + do.Store().GetCodec(), skipRegisterToDashboard) if err != nil { return err } @@ -1629,6 +1630,14 @@ func (do *Domain) GetPDClient() pd.Client { return nil } +// GetPDHTTPClient returns the PD HTTP client. +func (do *Domain) GetPDHTTPClient() pdhttp.Client { + if store, ok := do.store.(kv.StorageWithPD); ok { + return store.GetPDHTTPClient() + } + return nil +} + // LoadPrivilegeLoop create a goroutine loads privilege tables in a loop, it // should be called only once in BootstrapSession. func (do *Domain) LoadPrivilegeLoop(sctx sessionctx.Context) error { diff --git a/pkg/domain/infosync/info.go b/pkg/domain/infosync/info.go index 1278fd2469da7..d60806c0f941b 100644 --- a/pkg/domain/infosync/info.go +++ b/pkg/domain/infosync/info.go @@ -194,7 +194,7 @@ func GlobalInfoSyncerInit( id string, serverIDGetter func() uint64, etcdCli, unprefixedEtcdCli *clientv3.Client, - pdCli pd.Client, + pdCli pd.Client, pdHTTPCli pdhttp.Client, codec tikv.Codec, skipRegisterToDashBoard bool, ) (*InfoSyncer, error) { @@ -209,7 +209,10 @@ func GlobalInfoSyncerInit( if err != nil { return nil, err } - is.labelRuleManager = initLabelRuleManager(etcdCli) + if pdHTTPCli != nil { + pdHTTPCli = pdHTTPCli.WithRespHandler(pdResponseHandler) + } + is.labelRuleManager = initLabelRuleManager(pdHTTPCli) is.placementManager = initPlacementManager(etcdCli) is.scheduleManager = initScheduleManager(etcdCli) is.tiflashReplicaManager = initTiFlashReplicaManager(etcdCli, codec) @@ -244,11 +247,11 @@ func (is *InfoSyncer) GetSessionManager() util2.SessionManager { return is.managerMu.SessionManager } -func initLabelRuleManager(etcdCli *clientv3.Client) LabelRuleManager { - if etcdCli == nil { +func initLabelRuleManager(pdHTTPCli pdhttp.Client) LabelRuleManager { + if pdHTTPCli == nil { return &mockLabelManager{labelRules: map[string][]byte{}} } - return &PDLabelManager{etcdCli: etcdCli} + return &PDLabelManager{pdHTTPCli} } func initPlacementManager(etcdCli *clientv3.Client) PlacementManager { @@ -446,11 +449,39 @@ func MustGetTiFlashProgress(tableID int64, replicaCount uint64, tiFlashStores *m return progress, nil } +// pdResponseHandler will be injected into the PD HTTP client to handle the response, +// this is to maintain consistency with the logic in the `doRequest`. +func pdResponseHandler(resp *http.Response, res interface{}) error { + defer func() { terror.Log(resp.Body.Close()) }() + bodyBytes, err := io.ReadAll(resp.Body) + if err != nil { + return err + } + if resp.StatusCode == http.StatusOK { + if res != nil && bodyBytes != nil { + return json.Unmarshal(bodyBytes, res) + } + return nil + } + logutil.BgLogger().Warn("response not 200", + zap.String("method", resp.Request.Method), + zap.String("host", resp.Request.URL.Host), + zap.String("url", resp.Request.URL.RequestURI()), + zap.Int("http status", resp.StatusCode), + ) + if resp.StatusCode != http.StatusNotFound && resp.StatusCode != http.StatusPreconditionFailed { + return ErrHTTPServiceError.FastGen("%s", bodyBytes) + } + return nil +} + // TODO: replace with the unified PD HTTP client. func doRequest(ctx context.Context, apiName string, addrs []string, route, method string, body io.Reader) ([]byte, error) { - var err error - var req *http.Request - var res *http.Response + var ( + err error + req *http.Request + res *http.Response + ) for idx, addr := range addrs { url := util2.ComposeURL(addr, route) req, err = http.NewRequestWithContext(ctx, method, url, body) @@ -1052,7 +1083,7 @@ func PutLabelRule(ctx context.Context, rule *label.Rule) error { } // UpdateLabelRules synchronizes the label rule to PD. -func UpdateLabelRules(ctx context.Context, patch *label.RulePatch) error { +func UpdateLabelRules(ctx context.Context, patch *pdhttp.LabelRulePatch) error { if patch == nil || (len(patch.DeleteRules) == 0 && len(patch.SetRules) == 0) { return nil } diff --git a/pkg/domain/infosync/info_test.go b/pkg/domain/infosync/info_test.go index e56aebb054d7a..79c8ebf37e0a6 100644 --- a/pkg/domain/infosync/info_test.go +++ b/pkg/domain/infosync/info_test.go @@ -69,7 +69,7 @@ func TestTopology(t *testing.T) { require.NoError(t, err) }() - info, err := GlobalInfoSyncerInit(ctx, currentID, func() uint64 { return 1 }, client, client, nil, keyspace.CodecV1, false) + info, err := GlobalInfoSyncerInit(ctx, currentID, func() uint64 { return 1 }, client, client, nil, nil, keyspace.CodecV1, false) require.NoError(t, err) err = info.newTopologySessionAndStoreServerInfo(ctx, util2.NewSessionDefaultRetryCnt) @@ -154,7 +154,7 @@ func (is *InfoSyncer) ttlKeyExists(ctx context.Context) (bool, error) { } func TestPutBundlesRetry(t *testing.T) { - _, err := GlobalInfoSyncerInit(context.TODO(), "test", func() uint64 { return 1 }, nil, nil, nil, keyspace.CodecV1, false) + _, err := GlobalInfoSyncerInit(context.TODO(), "test", func() uint64 { return 1 }, nil, nil, nil, nil, keyspace.CodecV1, false) require.NoError(t, err) bundle, err := placement.NewBundleFromOptions(&model.PlacementSettings{PrimaryRegion: "r1", Regions: "r1,r2"}) @@ -218,7 +218,7 @@ func TestPutBundlesRetry(t *testing.T) { func TestTiFlashManager(t *testing.T) { ctx := context.Background() - _, err := GlobalInfoSyncerInit(ctx, "test", func() uint64 { return 1 }, nil, nil, nil, keyspace.CodecV1, false) + _, err := GlobalInfoSyncerInit(ctx, "test", func() uint64 { return 1 }, nil, nil, nil, nil, keyspace.CodecV1, false) tiflash := NewMockTiFlash() SetMockTiFlash(tiflash) diff --git a/pkg/domain/infosync/label_manager.go b/pkg/domain/infosync/label_manager.go index a6b95ce964991..0c5d0296136a4 100644 --- a/pkg/domain/infosync/label_manager.go +++ b/pkg/domain/infosync/label_manager.go @@ -15,79 +15,59 @@ package infosync import ( - "bytes" "context" "encoding/json" - "path" "sync" "github.com/pingcap/tidb/pkg/ddl/label" pd "github.com/tikv/pd/client/http" - clientv3 "go.etcd.io/etcd/client/v3" ) // LabelRuleManager manages label rules type LabelRuleManager interface { PutLabelRule(ctx context.Context, rule *label.Rule) error - UpdateLabelRules(ctx context.Context, patch *label.RulePatch) error + UpdateLabelRules(ctx context.Context, patch *pd.LabelRulePatch) error GetAllLabelRules(ctx context.Context) ([]*label.Rule, error) GetLabelRules(ctx context.Context, ruleIDs []string) (map[string]*label.Rule, error) } // PDLabelManager manages rules with pd type PDLabelManager struct { - etcdCli *clientv3.Client + pdHTTPCli pd.Client } // PutLabelRule implements PutLabelRule func (lm *PDLabelManager) PutLabelRule(ctx context.Context, rule *label.Rule) error { - r, err := json.Marshal(rule) - if err != nil { - return err - } - _, err = doRequest(ctx, "PutLabelRule", lm.etcdCli.Endpoints(), path.Join(pd.Config, "region-label", "rule"), "POST", bytes.NewReader(r)) - return err + return lm.pdHTTPCli.SetRegionLabelRule(ctx, (*pd.LabelRule)(rule)) } // UpdateLabelRules implements UpdateLabelRules -func (lm *PDLabelManager) UpdateLabelRules(ctx context.Context, patch *label.RulePatch) error { - r, err := json.Marshal(patch) - if err != nil { - return err - } - - _, err = doRequest(ctx, "UpdateLabelRules", lm.etcdCli.Endpoints(), path.Join(pd.Config, "region-label", "rules"), "PATCH", bytes.NewReader(r)) - return err +func (lm *PDLabelManager) UpdateLabelRules(ctx context.Context, patch *pd.LabelRulePatch) error { + return lm.pdHTTPCli.PatchRegionLabelRules(ctx, patch) } // GetAllLabelRules implements GetAllLabelRules func (lm *PDLabelManager) GetAllLabelRules(ctx context.Context) ([]*label.Rule, error) { - var rules []*label.Rule - res, err := doRequest(ctx, "GetAllLabelRules", lm.etcdCli.Endpoints(), path.Join(pd.Config, "region-label", "rules"), "GET", nil) - - if err == nil && res != nil { - err = json.Unmarshal(res, &rules) + labelRules, err := lm.pdHTTPCli.GetAllRegionLabelRules(ctx) + if err != nil { + return nil, err } - return rules, err + r := make([]*label.Rule, 0, len(labelRules)) + for _, labelRule := range labelRules { + r = append(r, (*label.Rule)(labelRule)) + } + return r, nil } // GetLabelRules implements GetLabelRules func (lm *PDLabelManager) GetLabelRules(ctx context.Context, ruleIDs []string) (map[string]*label.Rule, error) { - ids, err := json.Marshal(ruleIDs) + labelRules, err := lm.pdHTTPCli.GetAllRegionLabelRules(ctx) if err != nil { return nil, err } - - rules := []*label.Rule{} - res, err := doRequest(ctx, "GetLabelRules", lm.etcdCli.Endpoints(), path.Join(pd.Config, "region-label", "rules", "ids"), "GET", bytes.NewReader(ids)) - - if err == nil && res != nil { - err = json.Unmarshal(res, &rules) - } - - ruleMap := make(map[string]*label.Rule, len((rules))) - for _, r := range rules { - ruleMap[r.ID] = r + ruleMap := make(map[string]*label.Rule, len((labelRules))) + for _, r := range labelRules { + ruleMap[r.ID] = (*label.Rule)(r) } return ruleMap, err } @@ -113,7 +93,7 @@ func (mm *mockLabelManager) PutLabelRule(ctx context.Context, rule *label.Rule) } // UpdateLabelRules implements UpdateLabelRules -func (mm *mockLabelManager) UpdateLabelRules(ctx context.Context, patch *label.RulePatch) error { +func (mm *mockLabelManager) UpdateLabelRules(ctx context.Context, patch *pd.LabelRulePatch) error { mm.Lock() defer mm.Unlock() if patch == nil { diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 5fbff1bc6b4cc..af79439a5eef7 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -3200,7 +3200,7 @@ func (e *memtableRetriever) setDataForAttributes(ctx sessionctx.Context, is info rules = []*label.Rule{ { ID: "schema/test/test_label", - Labels: []label.Label{{Key: "merge_option", Value: "allow"}, {Key: "db", Value: "test"}, {Key: "table", Value: "test_label"}}, + Labels: []pd.RegionLabel{{Key: "merge_option", Value: "allow"}, {Key: "db", Value: "test"}, {Key: "table", Value: "test_label"}}, RuleType: "key-range", Data: convert(map[string]interface{}{ "start_key": "7480000000000000ff395f720000000000fa", @@ -3209,7 +3209,7 @@ func (e *memtableRetriever) setDataForAttributes(ctx sessionctx.Context, is info }, { ID: "invalidIDtest", - Labels: []label.Label{{Key: "merge_option", Value: "allow"}, {Key: "db", Value: "test"}, {Key: "table", Value: "test_label"}}, + Labels: []pd.RegionLabel{{Key: "merge_option", Value: "allow"}, {Key: "db", Value: "test"}, {Key: "table", Value: "test_label"}}, RuleType: "key-range", Data: convert(map[string]interface{}{ "start_key": "7480000000000000ff395f720000000000fa", @@ -3218,7 +3218,7 @@ func (e *memtableRetriever) setDataForAttributes(ctx sessionctx.Context, is info }, { ID: "schema/test/test_label", - Labels: []label.Label{{Key: "merge_option", Value: "allow"}, {Key: "db", Value: "test"}, {Key: "table", Value: "test_label"}}, + Labels: []pd.RegionLabel{{Key: "merge_option", Value: "allow"}, {Key: "db", Value: "test"}, {Key: "table", Value: "test_label"}}, RuleType: "key-range", Data: convert(map[string]interface{}{ "start_key": "aaaaa", @@ -3259,9 +3259,9 @@ func (e *memtableRetriever) setDataForAttributes(ctx sessionctx.Context, is info continue } - labels := rule.Labels.Restore() + labels := label.RestoreRegionLabels(&rule.Labels) var ranges []string - for _, data := range rule.Data { + for _, data := range rule.Data.([]interface{}) { if kv, ok := data.(map[string]interface{}); ok { startKey := kv["start_key"] endKey := kv["end_key"] @@ -3462,11 +3462,12 @@ func checkRule(rule *label.Rule) (dbName, tableName string, partitionName string } func decodeTableIDFromRule(rule *label.Rule) (tableID int64, err error) { - if len(rule.Data) == 0 { + datas := rule.Data.([]interface{}) + if len(datas) == 0 { err = fmt.Errorf("there is no data in rule %s", rule.ID) return } - data := rule.Data[0] + data := datas[0] dataMap, ok := data.(map[string]interface{}) if !ok { err = fmt.Errorf("get the label rules %s failed", rule.ID) diff --git a/pkg/kv/BUILD.bazel b/pkg/kv/BUILD.bazel index 99ba85562acdd..61433c6824390 100644 --- a/pkg/kv/BUILD.bazel +++ b/pkg/kv/BUILD.bazel @@ -53,6 +53,7 @@ go_library( "@com_github_tikv_client_go_v2//tikvrpc/interceptor", "@com_github_tikv_client_go_v2//util", "@com_github_tikv_pd_client//:client", + "@com_github_tikv_pd_client//http", "@org_uber_go_atomic//:atomic", "@org_uber_go_zap//:zap", ], diff --git a/pkg/kv/kv.go b/pkg/kv/kv.go index 6a2d547f284ae..f82c7919e5180 100644 --- a/pkg/kv/kv.go +++ b/pkg/kv/kv.go @@ -37,6 +37,7 @@ import ( "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/util" pd "github.com/tikv/pd/client" + pdhttp "github.com/tikv/pd/client/http" "go.uber.org/atomic" ) @@ -715,6 +716,7 @@ type EtcdBackend interface { // StorageWithPD is used to get pd client. type StorageWithPD interface { GetPDClient() pd.Client + GetPDHTTPClient() pdhttp.Client } // FnKeyCmp is the function for iterator the keys diff --git a/pkg/server/handler/tests/BUILD.bazel b/pkg/server/handler/tests/BUILD.bazel index 7acaf6cb34af5..f992b3170f853 100644 --- a/pkg/server/handler/tests/BUILD.bazel +++ b/pkg/server/handler/tests/BUILD.bazel @@ -28,6 +28,7 @@ go_test( "//pkg/server/handler/optimizor", "//pkg/server/handler/tikvhandler", "//pkg/server/internal/testserverclient", + "//pkg/server/internal/testutil", "//pkg/server/internal/util", "//pkg/session", "//pkg/sessionctx", diff --git a/pkg/server/handler/tests/http_handler_test.go b/pkg/server/handler/tests/http_handler_test.go index 4113a3ef42675..7610e5d3d9bac 100644 --- a/pkg/server/handler/tests/http_handler_test.go +++ b/pkg/server/handler/tests/http_handler_test.go @@ -50,6 +50,7 @@ import ( "github.com/pingcap/tidb/pkg/server/handler/optimizor" "github.com/pingcap/tidb/pkg/server/handler/tikvhandler" "github.com/pingcap/tidb/pkg/server/internal/testserverclient" + "github.com/pingcap/tidb/pkg/server/internal/testutil" "github.com/pingcap/tidb/pkg/server/internal/util" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/sessionctx" @@ -449,13 +450,7 @@ func TestBinlogRecover(t *testing.T) { func (ts *basicHTTPHandlerTestSuite) startServer(t *testing.T) { var err error - ts.Port = uint(rand.Int31n(50000)) + 10000 - ts.StatusPort = ts.Port + 1 - ts.store, err = mockstore.NewMockStore( - mockstore.WithTiKVOptions( - tikv.WithPDHTTPClient([]string{ts.Addr()}), - ), - ) + ts.store, err = mockstore.NewMockStore() require.NoError(t, err) ts.domain, err = session.BootstrapSession(ts.store) require.NoError(t, err) @@ -463,12 +458,14 @@ func (ts *basicHTTPHandlerTestSuite) startServer(t *testing.T) { cfg := util.NewTestConfig() cfg.Store = "tikv" - cfg.Port = ts.Port - cfg.Status.StatusPort = ts.StatusPort + cfg.Port = 0 + cfg.Status.StatusPort = 0 cfg.Status.ReportStatus = true server, err := server2.NewServer(cfg, ts.tidbdrv) require.NoError(t, err) + ts.Port = testutil.GetPortFromTCPAddr(server.ListenAddr()) + ts.StatusPort = testutil.GetPortFromTCPAddr(server.StatusListenerAddr()) ts.server = server ts.server.SetDomain(ts.domain) go func() { diff --git a/pkg/server/stat_test.go b/pkg/server/stat_test.go index 31f7983cd000d..5b0fca478780d 100644 --- a/pkg/server/stat_test.go +++ b/pkg/server/stat_test.go @@ -48,7 +48,7 @@ func TestUptime(t *testing.T) { }() require.NoError(t, err) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), keyspace.CodecV1, true) + _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), dom.GetEtcdClient(), dom.GetPDClient(), dom.GetPDHTTPClient(), keyspace.CodecV1, true) require.NoError(t, err) tidbdrv := NewTiDBDriver(store) diff --git a/pkg/store/gcworker/gc_worker.go b/pkg/store/gcworker/gc_worker.go index caedf9e9ec33e..b090f082e6582 100644 --- a/pkg/store/gcworker/gc_worker.go +++ b/pkg/store/gcworker/gc_worker.go @@ -2017,7 +2017,7 @@ func getGCRules(ids []int64, rules map[string]*label.Rule) []string { var gcRules []string for _, rule := range rules { find := false - for _, d := range rule.Data { + for _, d := range rule.Data.([]interface{}) { if r, ok := d.(map[string]interface{}); ok { nowRange := fmt.Sprintf("%s%s", r["start_key"], r["end_key"]) if _, ok := oldRange[nowRange]; ok { From b8515ae354a537ae1cce9ef32c4ca1afe84eea9e Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 23 Nov 2023 13:28:11 +0800 Subject: [PATCH 20/36] session: move session interface into new package (#48792) ref pingcap/tidb#44940 --- br/pkg/gluetidb/BUILD.bazel | 1 + br/pkg/gluetidb/glue.go | 9 +- cmd/benchdb/BUILD.bazel | 1 + cmd/benchdb/main.go | 3 +- cmd/ddltest/BUILD.bazel | 1 + cmd/ddltest/ddl_test.go | 3 +- pkg/ddl/BUILD.bazel | 1 + pkg/ddl/db_change_test.go | 3 +- pkg/ddl/ddl_api_test.go | 4 +- pkg/ddl/stat_test.go | 4 +- pkg/ddl/table_modify_test.go | 4 +- pkg/ddl/testutil/BUILD.bazel | 1 + pkg/ddl/testutil/testutil.go | 3 +- pkg/executor/BUILD.bazel | 1 + pkg/executor/inspection_result_test.go | 4 +- pkg/executor/test/issuetest/BUILD.bazel | 2 +- .../test/issuetest/executor_issue_test.go | 4 +- pkg/executor/update_test.go | 4 +- pkg/planner/core/BUILD.bazel | 1 + .../core/memtable_predicate_extractor_test.go | 3 +- pkg/planner/core/tests/prepare/BUILD.bazel | 1 + .../core/tests/prepare/prepare_test.go | 5 +- pkg/server/BUILD.bazel | 1 + pkg/server/driver_tidb.go | 3 +- pkg/server/handler/tikvhandler/BUILD.bazel | 1 + .../handler/tikvhandler/tikv_handler.go | 3 +- pkg/server/rpc_server.go | 3 +- pkg/session/BUILD.bazel | 2 + pkg/session/bench_test.go | 13 +- pkg/session/bootstrap.go | 307 +++++++++--------- pkg/session/bootstrap_test.go | 7 +- pkg/session/bootstraptest/BUILD.bazel | 1 + .../bootstraptest/bootstrap_upgrade_test.go | 11 +- pkg/session/mock_bootstrap.go | 31 +- pkg/session/nontransactional.go | 19 +- pkg/session/session.go | 79 +---- pkg/session/sync_upgrade.go | 3 +- pkg/session/testutil.go | 9 +- pkg/session/tidb.go | 3 +- pkg/session/types/BUILD.bazel | 21 ++ pkg/session/types/sesson_interface.go | 91 ++++++ pkg/store/gcworker/BUILD.bazel | 1 + pkg/store/gcworker/gc_worker.go | 5 +- pkg/table/tables/BUILD.bazel | 1 + pkg/table/tables/state_remote_test.go | 4 +- pkg/telemetry/cte_test/BUILD.bazel | 1 + pkg/telemetry/cte_test/cte_test.go | 3 +- pkg/testkit/BUILD.bazel | 1 + pkg/testkit/asynctestkit.go | 5 +- pkg/testkit/mocksessionmanager.go | 3 +- pkg/testkit/testkit.go | 13 +- pkg/util/ddl-checker/BUILD.bazel | 1 + pkg/util/ddl-checker/executable_checker.go | 3 +- 53 files changed, 405 insertions(+), 307 deletions(-) create mode 100644 pkg/session/types/BUILD.bazel create mode 100644 pkg/session/types/sesson_interface.go diff --git a/br/pkg/gluetidb/BUILD.bazel b/br/pkg/gluetidb/BUILD.bazel index 979f05b4e5b48..6ff71aa916e86 100644 --- a/br/pkg/gluetidb/BUILD.bazel +++ b/br/pkg/gluetidb/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/session", + "//pkg/session/types", "//pkg/sessionctx", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_log//:log", diff --git a/br/pkg/gluetidb/glue.go b/br/pkg/gluetidb/glue.go index da42b1b1d9b71..c618281b4b2b3 100644 --- a/br/pkg/gluetidb/glue.go +++ b/br/pkg/gluetidb/glue.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" pd "github.com/tikv/pd/client" "go.uber.org/zap" @@ -58,7 +59,7 @@ type Glue struct { } type tidbSession struct { - se session.Session + se sessiontypes.Session } // GetDomain implements glue.Glue. @@ -358,7 +359,7 @@ func (gs *tidbSession) showCreatePlacementPolicy(policy *model.PolicyInfo) strin // mockSession is used for test. type mockSession struct { - se session.Session + se sessiontypes.Session globalVars map[string]string } @@ -434,11 +435,11 @@ func (s *mockSession) GetGlobalVariable(name string) (string, error) { // MockGlue only used for test type MockGlue struct { - se session.Session + se sessiontypes.Session GlobalVars map[string]string } -func (m *MockGlue) SetSession(se session.Session) { +func (m *MockGlue) SetSession(se sessiontypes.Session) { m.se = se } diff --git a/cmd/benchdb/BUILD.bazel b/cmd/benchdb/BUILD.bazel index cb9959c92f175..6e8acbd0bd3ff 100644 --- a/cmd/benchdb/BUILD.bazel +++ b/cmd/benchdb/BUILD.bazel @@ -8,6 +8,7 @@ go_library( deps = [ "//pkg/parser/terror", "//pkg/session", + "//pkg/session/types", "//pkg/store", "//pkg/store/driver", "//pkg/util/logutil", diff --git a/cmd/benchdb/main.go b/cmd/benchdb/main.go index 3af0e74fe7cc5..abe57173805d3 100644 --- a/cmd/benchdb/main.go +++ b/cmd/benchdb/main.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/store" "github.com/pingcap/tidb/pkg/store/driver" "github.com/pingcap/tidb/pkg/util/logutil" @@ -88,7 +89,7 @@ func main() { type benchDB struct { store tikv.Storage - session session.Session + session sessiontypes.Session } func newBenchDB() *benchDB { diff --git a/cmd/ddltest/BUILD.bazel b/cmd/ddltest/BUILD.bazel index a036bcd5bba6f..bf3fce97cf57a 100644 --- a/cmd/ddltest/BUILD.bazel +++ b/cmd/ddltest/BUILD.bazel @@ -20,6 +20,7 @@ go_test( "//pkg/parser/model", "//pkg/parser/terror", "//pkg/session", + "//pkg/session/types", "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/sessiontxn", diff --git a/cmd/ddltest/ddl_test.go b/cmd/ddltest/ddl_test.go index 6711cb73fd25a..6c90ebf3a2fcc 100644 --- a/cmd/ddltest/ddl_test.go +++ b/cmd/ddltest/ddl_test.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -76,7 +77,7 @@ type server struct { type ddlSuite struct { store kv.Storage dom *domain.Domain - s session.Session + s sessiontypes.Session ctx sessionctx.Context m sync.Mutex diff --git a/pkg/ddl/BUILD.bazel b/pkg/ddl/BUILD.bazel index c244607ec0351..744be405f4dc5 100644 --- a/pkg/ddl/BUILD.bazel +++ b/pkg/ddl/BUILD.bazel @@ -276,6 +276,7 @@ go_test( "//pkg/planner/core", "//pkg/server", "//pkg/session", + "//pkg/session/types", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", diff --git a/pkg/ddl/db_change_test.go b/pkg/ddl/db_change_test.go index d2a02d1960fe2..8984002bd21af 100644 --- a/pkg/ddl/db_change_test.go +++ b/pkg/ddl/db_change_test.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/testkit" @@ -289,7 +290,7 @@ func TestTwoStates(t *testing.T) { } type stateCase struct { - session session.Session + session sessiontypes.Session rawStmt ast.StmtNode stmt sqlexec.Statement expectedExecErr string diff --git a/pkg/ddl/ddl_api_test.go b/pkg/ddl/ddl_api_test.go index e6dec271e9fac..9426844e43ae4 100644 --- a/pkg/ddl/ddl_api_test.go +++ b/pkg/ddl/ddl_api_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" - "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" ) @@ -134,7 +134,7 @@ func TestIsJobRollbackable(t *testing.T) { } } -func enQueueDDLJobs(t *testing.T, sess session.Session, txn kv.Transaction, jobType model.ActionType, start, end int) { +func enQueueDDLJobs(t *testing.T, sess sessiontypes.Session, txn kv.Transaction, jobType model.ActionType, start, end int) { for i := start; i < end; i++ { job := &model.Job{ ID: int64(i), diff --git a/pkg/ddl/stat_test.go b/pkg/ddl/stat_test.go index 291b01e79f13e..8aaac9d9d913b 100644 --- a/pkg/ddl/stat_test.go +++ b/pkg/ddl/stat_test.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/testkit" @@ -148,7 +148,7 @@ func TestGetDDLInfo(t *testing.T) { tk.MustExec("rollback") } -func addDDLJobs(sess session.Session, txn kv.Transaction, job *model.Job) error { +func addDDLJobs(sess sessiontypes.Session, txn kv.Transaction, job *model.Job) error { b, err := job.Encode(true) if err != nil { return err diff --git a/pkg/ddl/table_modify_test.go b/pkg/ddl/table_modify_test.go index 975c09f518a55..1b69aed2d66e1 100644 --- a/pkg/ddl/table_modify_test.go +++ b/pkg/ddl/table_modify_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util" @@ -127,7 +127,7 @@ func TestConcurrentLockTables(t *testing.T) { tk2.MustExec("unlock tables") } -func testParallelExecSQL(t *testing.T, store kv.Storage, dom *domain.Domain, sql1, sql2 string, se1, se2 session.Session, f func(t *testing.T, err1, err2 error)) { +func testParallelExecSQL(t *testing.T, store kv.Storage, dom *domain.Domain, sql1, sql2 string, se1, se2 sessiontypes.Session, f func(t *testing.T, err1, err2 error)) { callback := &callback.TestDDLCallback{} times := 0 callback.OnJobRunBeforeExported = func(job *model.Job) { diff --git a/pkg/ddl/testutil/BUILD.bazel b/pkg/ddl/testutil/BUILD.bazel index 12628e9c102f2..54d11c7cc4084 100644 --- a/pkg/ddl/testutil/BUILD.bazel +++ b/pkg/ddl/testutil/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "//pkg/kv", "//pkg/parser/model", "//pkg/session", + "//pkg/session/types", "//pkg/sessiontxn", "//pkg/table", "//pkg/table/tables", diff --git a/pkg/ddl/testutil/testutil.go b/pkg/ddl/testutil/testutil.go index 0fe58f7c34762..1e02e6a28d7a1 100644 --- a/pkg/ddl/testutil/testutil.go +++ b/pkg/ddl/testutil/testutil.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessiontxn" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" @@ -68,7 +69,7 @@ func ExecMultiSQLInGoroutine(s kv.Storage, dbName string, multiSQL []string, don } // ExtractAllTableHandles extracts all handles of a given table. -func ExtractAllTableHandles(se session.Session, dbName, tbName string) ([]int64, error) { +func ExtractAllTableHandles(se sessiontypes.Session, dbName, tbName string) ([]int64, error) { dom := domain.GetDomain(se) tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(dbName), model.NewCIStr(tbName)) if err != nil { diff --git a/pkg/executor/BUILD.bazel b/pkg/executor/BUILD.bazel index 76277ce2336e0..5cf987f160f5b 100644 --- a/pkg/executor/BUILD.bazel +++ b/pkg/executor/BUILD.bazel @@ -401,6 +401,7 @@ go_test( "//pkg/planner/util", "//pkg/server", "//pkg/session", + "//pkg/session/types", "//pkg/sessionctx", "//pkg/sessionctx/binloginfo", "//pkg/sessionctx/stmtctx", diff --git a/pkg/executor/inspection_result_test.go b/pkg/executor/inspection_result_test.go index 27f47dcd3dd54..29638352b75e3 100644 --- a/pkg/executor/inspection_result_test.go +++ b/pkg/executor/inspection_result_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/sysutil" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" @@ -179,7 +179,7 @@ func TestInspectionResult(t *testing.T) { } } -func parseTime(t *testing.T, se session.Session, str string) types.Time { +func parseTime(t *testing.T, se sessiontypes.Session, str string) types.Time { time, err := types.ParseTime(se.GetSessionVars().StmtCtx.TypeCtx(), str, mysql.TypeDatetime, types.MaxFsp) require.NoError(t, err) return time diff --git a/pkg/executor/test/issuetest/BUILD.bazel b/pkg/executor/test/issuetest/BUILD.bazel index f03a2147ddc4d..bf67149870188 100644 --- a/pkg/executor/test/issuetest/BUILD.bazel +++ b/pkg/executor/test/issuetest/BUILD.bazel @@ -17,7 +17,7 @@ go_test( "//pkg/parser/auth", "//pkg/parser/charset", "//pkg/parser/mysql", - "//pkg/session", + "//pkg/session/types", "//pkg/testkit", "//pkg/util", "//pkg/util/dbterror/exeerrors", diff --git a/pkg/executor/test/issuetest/executor_issue_test.go b/pkg/executor/test/issuetest/executor_issue_test.go index 78eeee0f0e019..74cb0556517b9 100644 --- a/pkg/executor/test/issuetest/executor_issue_test.go +++ b/pkg/executor/test/issuetest/executor_issue_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" @@ -576,7 +576,7 @@ func TestIssue42662(t *testing.T) { sm := &testkit.MockSessionManager{ PS: []*util.ProcessInfo{tk.Session().ShowProcess()}, } - sm.Conn = make(map[uint64]session.Session) + sm.Conn = make(map[uint64]sessiontypes.Session) sm.Conn[tk.Session().GetSessionVars().ConnectionID] = tk.Session() dom.ServerMemoryLimitHandle().SetSessionManager(sm) go dom.ServerMemoryLimitHandle().Run() diff --git a/pkg/executor/update_test.go b/pkg/executor/update_test.go index e6052ca91ea61..8c17c045808be 100644 --- a/pkg/executor/update_test.go +++ b/pkg/executor/update_test.go @@ -21,7 +21,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -50,7 +50,7 @@ func testUpdatePKLazyCheck(t *testing.T, tk *testkit.TestKit, clusteredIndex var tk.MustExec("commit") } -func getPresumeExistsCount(t *testing.T, se session.Session) int { +func getPresumeExistsCount(t *testing.T, se sessiontypes.Session) int { txn, err := se.Txn(false) require.NoError(t, err) buf := txn.GetMemBuffer() diff --git a/pkg/planner/core/BUILD.bazel b/pkg/planner/core/BUILD.bazel index d1723b8a7b38e..0f3516a348b7b 100644 --- a/pkg/planner/core/BUILD.bazel +++ b/pkg/planner/core/BUILD.bazel @@ -250,6 +250,7 @@ go_test( "//pkg/planner/property", "//pkg/planner/util", "//pkg/session", + "//pkg/session/types", "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/sessiontxn", diff --git a/pkg/planner/core/memtable_predicate_extractor_test.go b/pkg/planner/core/memtable_predicate_extractor_test.go index 2c53fa8f2120b..77f1585807d2b 100644 --- a/pkg/planner/core/memtable_predicate_extractor_test.go +++ b/pkg/planner/core/memtable_predicate_extractor_test.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/planner" plannercore "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/hint" @@ -39,7 +40,7 @@ import ( "github.com/stretchr/testify/require" ) -func getLogicalMemTable(t *testing.T, dom *domain.Domain, se session.Session, parser *parser.Parser, sql string) *plannercore.LogicalMemTable { +func getLogicalMemTable(t *testing.T, dom *domain.Domain, se sessiontypes.Session, parser *parser.Parser, sql string) *plannercore.LogicalMemTable { stmt, err := parser.ParseOneStmt(sql, "", "") require.NoError(t, err) diff --git a/pkg/planner/core/tests/prepare/BUILD.bazel b/pkg/planner/core/tests/prepare/BUILD.bazel index 106222cc3069d..970c173c54bf6 100644 --- a/pkg/planner/core/tests/prepare/BUILD.bazel +++ b/pkg/planner/core/tests/prepare/BUILD.bazel @@ -20,6 +20,7 @@ go_test( "//pkg/parser/auth", "//pkg/planner/core", "//pkg/session", + "//pkg/session/types", "//pkg/sessionctx/variable", "//pkg/testkit", "//pkg/testkit/testsetup", diff --git a/pkg/planner/core/tests/prepare/prepare_test.go b/pkg/planner/core/tests/prepare/prepare_test.go index 3da9d010e8974..99ace9c81672e 100644 --- a/pkg/planner/core/tests/prepare/prepare_test.go +++ b/pkg/planner/core/tests/prepare/prepare_test.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/auth" "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" "github.com/pingcap/tidb/pkg/util/hint" @@ -635,7 +636,7 @@ func TestPrepareCacheForPartition(t *testing.T) { } } -func newSession(t *testing.T, store kv.Storage, dbName string) session.Session { +func newSession(t *testing.T, store kv.Storage, dbName string) sessiontypes.Session { se, err := session.CreateSession4Test(store) require.NoError(t, err) mustExec(t, se, "create database if not exists "+dbName) @@ -643,7 +644,7 @@ func newSession(t *testing.T, store kv.Storage, dbName string) session.Session { return se } -func mustExec(t *testing.T, se session.Session, sql string) { +func mustExec(t *testing.T, se sessiontypes.Session, sql string) { _, err := se.Execute(context.Background(), sql) require.NoError(t, err) } diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index bdf0715b60e44..86901c1b84b5e 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -63,6 +63,7 @@ go_library( "//pkg/server/metrics", "//pkg/session", "//pkg/session/txninfo", + "//pkg/session/types", "//pkg/sessionctx", "//pkg/sessionctx/sessionstates", "//pkg/sessionctx/stmtctx", diff --git a/pkg/server/driver_tidb.go b/pkg/server/driver_tidb.go index 9e70f5ad18d30..c1d51b354e01a 100644 --- a/pkg/server/driver_tidb.go +++ b/pkg/server/driver_tidb.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/server/internal/column" "github.com/pingcap/tidb/pkg/server/internal/resultset" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" @@ -55,7 +56,7 @@ func NewTiDBDriver(store kv.Storage) *TiDBDriver { // TiDBContext implements QueryCtx. type TiDBContext struct { - session.Session + sessiontypes.Session stmts map[int]*TiDBStatement } diff --git a/pkg/server/handler/tikvhandler/BUILD.bazel b/pkg/server/handler/tikvhandler/BUILD.bazel index e4ba4a2ae109a..824f6a4053f4e 100644 --- a/pkg/server/handler/tikvhandler/BUILD.bazel +++ b/pkg/server/handler/tikvhandler/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/server/handler", "//pkg/session", "//pkg/session/txninfo", + "//pkg/session/types", "//pkg/sessionctx", "//pkg/sessionctx/binloginfo", "//pkg/sessionctx/variable", diff --git a/pkg/server/handler/tikvhandler/tikv_handler.go b/pkg/server/handler/tikvhandler/tikv_handler.go index 4f5863a62d817..394212e331ea9 100644 --- a/pkg/server/handler/tikvhandler/tikv_handler.go +++ b/pkg/server/handler/tikvhandler/tikv_handler.go @@ -46,6 +46,7 @@ import ( "github.com/pingcap/tidb/pkg/server/handler" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/session/txninfo" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/binloginfo" "github.com/pingcap/tidb/pkg/sessionctx/variable" @@ -825,7 +826,7 @@ type SchemaTableStorage struct { } func getSchemaTablesStorageInfo(h *SchemaStorageHandler, schema *model.CIStr, table *model.CIStr) (messages []*SchemaTableStorage, err error) { - var s session.Session + var s sessiontypes.Session if s, err = session.CreateSession(h.Store); err != nil { return } diff --git a/pkg/server/rpc_server.go b/pkg/server/rpc_server.go index 64225ccaca210..231a8ecc0fcef 100644 --- a/pkg/server/rpc_server.go +++ b/pkg/server/rpc_server.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/privilege/privileges" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/util" "github.com/pingcap/tidb/pkg/util/logutil" @@ -214,7 +215,7 @@ func (s *rpcServer) handleCopRequest(ctx context.Context, req *coprocessor.Reque return h.HandleRequest(ctx, req) } -func (s *rpcServer) createSession() (session.Session, error) { +func (s *rpcServer) createSession() (sessiontypes.Session, error) { se, err := session.CreateSessionWithDomain(s.dom.Store(), s.dom) if err != nil { return nil, err diff --git a/pkg/session/BUILD.bazel b/pkg/session/BUILD.bazel index 14b3a33123521..b4c484de4123f 100644 --- a/pkg/session/BUILD.bazel +++ b/pkg/session/BUILD.bazel @@ -53,6 +53,7 @@ go_library( "//pkg/privilege/privileges", "//pkg/session/metrics", "//pkg/session/txninfo", + "//pkg/session/types", "//pkg/sessionctx", "//pkg/sessionctx/binloginfo", "//pkg/sessionctx/sessionstates", @@ -142,6 +143,7 @@ go_test( "//pkg/meta", "//pkg/parser/ast", "//pkg/parser/auth", + "//pkg/session/types", "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/statistics", diff --git a/pkg/session/bench_test.go b/pkg/session/bench_test.go index 0064336dd5d3c..ec8a226c6e009 100644 --- a/pkg/session/bench_test.go +++ b/pkg/session/bench_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/ast" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/util/benchdaily" "github.com/pingcap/tidb/pkg/util/chunk" @@ -43,7 +44,7 @@ import ( var smallCount = 100 var bigCount = 10000 -func prepareBenchSession() (Session, *domain.Domain, kv.Storage) { +func prepareBenchSession() (sessiontypes.Session, *domain.Domain, kv.Storage) { config.UpdateGlobal(func(cfg *config.Config) { cfg.Instance.EnableSlowLog.Store(false) }) @@ -65,7 +66,7 @@ func prepareBenchSession() (Session, *domain.Domain, kv.Storage) { return se, domain, store } -func prepareBenchData(se Session, colType string, valueFormat string, valueCount int) { +func prepareBenchData(se sessiontypes.Session, colType string, valueFormat string, valueCount int) { mustExecute(se, "drop table if exists t") mustExecute(se, fmt.Sprintf("create table t (pk int primary key auto_increment, col %s, index idx (col))", colType)) mustExecute(se, "begin") @@ -75,7 +76,7 @@ func prepareBenchData(se Session, colType string, valueFormat string, valueCount mustExecute(se, "commit") } -func prepareNonclusteredBenchData(se Session, colType string, valueFormat string, valueCount int) { +func prepareNonclusteredBenchData(se sessiontypes.Session, colType string, valueFormat string, valueCount int) { mustExecute(se, "drop table if exists t") mustExecute(se, fmt.Sprintf("create table t (pk int primary key /*T![clustered_index] NONCLUSTERED */ auto_increment, col %s, index idx (col))", colType)) mustExecute(se, "begin") @@ -85,7 +86,7 @@ func prepareNonclusteredBenchData(se Session, colType string, valueFormat string mustExecute(se, "commit") } -func prepareSortBenchData(se Session, colType string, valueFormat string, valueCount int) { +func prepareSortBenchData(se sessiontypes.Session, colType string, valueFormat string, valueCount int) { mustExecute(se, "drop table if exists t") mustExecute(se, fmt.Sprintf("create table t (pk int primary key auto_increment, col %s)", colType)) mustExecute(se, "begin") @@ -100,7 +101,7 @@ func prepareSortBenchData(se Session, colType string, valueFormat string, valueC mustExecute(se, "commit") } -func prepareJoinBenchData(se Session, colType string, valueFormat string, valueCount int) { +func prepareJoinBenchData(se sessiontypes.Session, colType string, valueFormat string, valueCount int) { mustExecute(se, "drop table if exists t") mustExecute(se, fmt.Sprintf("create table t (pk int primary key auto_increment, col %s)", colType)) mustExecute(se, "begin") @@ -125,7 +126,7 @@ func readResult(ctx context.Context, rs sqlexec.RecordSet, count int) { rs.Close() } -func hasPlan(ctx context.Context, b *testing.B, se Session, plan string) { +func hasPlan(ctx context.Context, b *testing.B, se sessiontypes.Session, plan string) { find := false rs, err := se.Execute(ctx, "explain select * from t where col = 'hello 64'") if err != nil { diff --git a/pkg/session/bootstrap.go b/pkg/session/bootstrap.go index bc3b231aa5787..ba14a7828b1b5 100644 --- a/pkg/session/bootstrap.go +++ b/pkg/session/bootstrap.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/planner/core" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table/tables" timertable "github.com/pingcap/tidb/pkg/timer/tablestore" @@ -694,7 +695,7 @@ const ( var CreateTimers = timertable.CreateTimerTableSQL("mysql", "tidb_timers") // bootstrap initiates system DB for a store. -func bootstrap(s Session) { +func bootstrap(s sessiontypes.Session) { startTime := time.Now() err := InitMDLVariableForBootstrap(s.GetStore()) if err != nil { @@ -1045,7 +1046,7 @@ func DisableRunBootstrapSQLFileInTest() { } var ( - bootstrapVersion = []func(Session, int64){ + bootstrapVersion = []func(sessiontypes.Session, int64){ upgradeToVer2, upgradeToVer3, upgradeToVer4, @@ -1187,7 +1188,7 @@ var ( } ) -func checkBootstrapped(s Session) (bool, error) { +func checkBootstrapped(s sessiontypes.Session) (bool, error) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) // Check if system db exists. _, err := s.ExecuteInternal(ctx, "USE %n", mysql.SystemDB) @@ -1215,7 +1216,7 @@ func checkBootstrapped(s Session) (bool, error) { // getTiDBVar gets variable value from mysql.tidb table. // Those variables are used by TiDB server. -func getTiDBVar(s Session, name string) (sVal string, isNull bool, e error) { +func getTiDBVar(s sessiontypes.Session, name string) (sVal string, isNull bool, e error) { ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) rs, err := s.ExecuteInternal(ctx, `SELECT HIGH_PRIORITY VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME= %?`, mysql.SystemDB, @@ -1249,7 +1250,7 @@ var ( // upgrade function will do some upgrade works, when the system is bootstrapped by low version TiDB server // For example, add new system variables into mysql.global_variables table. -func upgrade(s Session) { +func upgrade(s sessiontypes.Session) { ver, err := getBootstrapVersion(s) terror.MustNil(err) if ver >= currentBootstrapVersion { @@ -1342,7 +1343,7 @@ func checkOwnerVersion(ctx context.Context, dom *domain.Domain) (bool, error) { } // upgradeToVer2 updates to version 2. -func upgradeToVer2(s Session, ver int64) { +func upgradeToVer2(s sessiontypes.Session, ver int64) { if ver >= version2 { return } @@ -1360,7 +1361,7 @@ func upgradeToVer2(s Session, ver int64) { } // upgradeToVer3 updates to version 3. -func upgradeToVer3(s Session, ver int64) { +func upgradeToVer3(s sessiontypes.Session, ver int64) { if ver >= version3 { return } @@ -1369,14 +1370,14 @@ func upgradeToVer3(s Session, ver int64) { } // upgradeToVer4 updates to version 4. -func upgradeToVer4(s Session, ver int64) { +func upgradeToVer4(s sessiontypes.Session, ver int64) { if ver >= version4 { return } mustExecute(s, CreateStatsMetaTable) } -func upgradeToVer5(s Session, ver int64) { +func upgradeToVer5(s sessiontypes.Session, ver int64) { if ver >= version5 { return } @@ -1384,7 +1385,7 @@ func upgradeToVer5(s Session, ver int64) { mustExecute(s, CreateStatsBucketsTable) } -func upgradeToVer6(s Session, ver int64) { +func upgradeToVer6(s sessiontypes.Session, ver int64) { if ver >= version6 { return } @@ -1393,7 +1394,7 @@ func upgradeToVer6(s Session, ver int64) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Super_priv='Y'") } -func upgradeToVer7(s Session, ver int64) { +func upgradeToVer7(s sessiontypes.Session, ver int64) { if ver >= version7 { return } @@ -1402,7 +1403,7 @@ func upgradeToVer7(s Session, ver int64) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Process_priv='Y'") } -func upgradeToVer8(s Session, ver int64) { +func upgradeToVer8(s sessiontypes.Session, ver int64) { if ver >= version8 { return } @@ -1414,7 +1415,7 @@ func upgradeToVer8(s Session, ver int64) { upgradeToVer7(s, ver) } -func upgradeToVer9(s Session, ver int64) { +func upgradeToVer9(s sessiontypes.Session, ver int64) { if ver >= version9 { return } @@ -1423,7 +1424,7 @@ func upgradeToVer9(s Session, ver int64) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Trigger_priv='Y'") } -func doReentrantDDL(s Session, sql string, ignorableErrs ...error) { +func doReentrantDDL(s sessiontypes.Session, sql string, ignorableErrs ...error) { ctx, cancel := context.WithTimeout(context.Background(), time.Duration(internalSQLTimeout)*time.Second) ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnBootstrap) _, err := s.ExecuteInternal(ctx, sql) @@ -1438,7 +1439,7 @@ func doReentrantDDL(s Session, sql string, ignorableErrs ...error) { } } -func upgradeToVer10(s Session, ver int64) { +func upgradeToVer10(s sessiontypes.Session, ver int64) { if ver >= version10 { return } @@ -1449,7 +1450,7 @@ func upgradeToVer10(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms DROP COLUMN use_count_to_estimate", dbterror.ErrCantDropFieldOrKey) } -func upgradeToVer11(s Session, ver int64) { +func upgradeToVer11(s sessiontypes.Session, ver int64) { if ver >= version11 { return } @@ -1457,7 +1458,7 @@ func upgradeToVer11(s Session, ver int64) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET References_priv='Y'") } -func upgradeToVer12(s Session, ver int64) { +func upgradeToVer12(s sessiontypes.Session, ver int64) { if ver >= version12 { return } @@ -1502,7 +1503,7 @@ func upgradeToVer12(s Session, ver int64) { mustExecute(s, "COMMIT") } -func upgradeToVer13(s Session, ver int64) { +func upgradeToVer13(s sessiontypes.Session, ver int64) { if ver >= version13 { return } @@ -1522,7 +1523,7 @@ func upgradeToVer13(s Session, ver int64) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Create_view_priv='Y',Show_view_priv='Y' WHERE Create_priv='Y'") } -func upgradeToVer14(s Session, ver int64) { +func upgradeToVer14(s sessiontypes.Session, ver int64) { if ver >= version14 { return } @@ -1542,35 +1543,35 @@ func upgradeToVer14(s Session, ver int64) { } } -func upgradeToVer15(s Session, ver int64) { +func upgradeToVer15(s sessiontypes.Session, ver int64) { if ver >= version15 { return } doReentrantDDL(s, CreateGCDeleteRangeTable) } -func upgradeToVer16(s Session, ver int64) { +func upgradeToVer16(s sessiontypes.Session, ver int64) { if ver >= version16 { return } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `cm_sketch` BLOB", infoschema.ErrColumnExists) } -func upgradeToVer17(s Session, ver int64) { +func upgradeToVer17(s sessiontypes.Session, ver int64) { if ver >= version17 { return } doReentrantDDL(s, "ALTER TABLE mysql.user MODIFY User CHAR(32)") } -func upgradeToVer18(s Session, ver int64) { +func upgradeToVer18(s sessiontypes.Session, ver int64) { if ver >= version18 { return } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `tot_col_size` BIGINT(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } -func upgradeToVer19(s Session, ver int64) { +func upgradeToVer19(s sessiontypes.Session, ver int64) { if ver >= version19 { return } @@ -1579,7 +1580,7 @@ func upgradeToVer19(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.columns_priv MODIFY User CHAR(32)") } -func upgradeToVer20(s Session, ver int64) { +func upgradeToVer20(s sessiontypes.Session, ver int64) { if ver >= version20 { return } @@ -1587,7 +1588,7 @@ func upgradeToVer20(s Session, ver int64) { doReentrantDDL(s, CreateStatsFeedbackTable) } -func upgradeToVer21(s Session, ver int64) { +func upgradeToVer21(s sessiontypes.Session, ver int64) { if ver >= version21 { return } @@ -1598,14 +1599,14 @@ func upgradeToVer21(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.gc_delete_range DROP INDEX element_id", dbterror.ErrCantDropFieldOrKey) } -func upgradeToVer22(s Session, ver int64) { +func upgradeToVer22(s sessiontypes.Session, ver int64) { if ver >= version22 { return } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `stats_ver` BIGINT(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } -func upgradeToVer23(s Session, ver int64) { +func upgradeToVer23(s sessiontypes.Session, ver int64) { if ver >= version23 { return } @@ -1613,7 +1614,7 @@ func upgradeToVer23(s Session, ver int64) { } // writeSystemTZ writes system timezone info into mysql.tidb -func writeSystemTZ(s Session) { +func writeSystemTZ(s sessiontypes.Session) { mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, "TiDB Global System Timezone.") ON DUPLICATE KEY UPDATE VARIABLE_VALUE= %?`, mysql.SystemDB, mysql.TiDBTable, @@ -1624,7 +1625,7 @@ func writeSystemTZ(s Session) { } // upgradeToVer24 initializes `System` timezone according to docs/design/2018-09-10-adding-tz-env.md -func upgradeToVer24(s Session, ver int64) { +func upgradeToVer24(s sessiontypes.Session, ver int64) { if ver >= version24 { return } @@ -1632,7 +1633,7 @@ func upgradeToVer24(s Session, ver int64) { } // upgradeToVer25 updates tidb_max_chunk_size to new low bound value 32 if previous value is small than 32. -func upgradeToVer25(s Session, ver int64) { +func upgradeToVer25(s sessiontypes.Session, ver int64) { if ver >= version25 { return } @@ -1641,7 +1642,7 @@ func upgradeToVer25(s Session, ver int64) { mustExecute(s, sql) } -func upgradeToVer26(s Session, ver int64) { +func upgradeToVer26(s sessiontypes.Session, ver int64) { if ver >= version26 { return } @@ -1656,21 +1657,21 @@ func upgradeToVer26(s Session, ver int64) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Create_view_priv='Y',Show_view_priv='Y' WHERE Create_priv='Y'") } -func upgradeToVer27(s Session, ver int64) { +func upgradeToVer27(s sessiontypes.Session, ver int64) { if ver >= version27 { return } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `correlation` DOUBLE NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } -func upgradeToVer28(s Session, ver int64) { +func upgradeToVer28(s sessiontypes.Session, ver int64) { if ver >= version28 { return } doReentrantDDL(s, CreateBindInfoTable) } -func upgradeToVer29(s Session, ver int64) { +func upgradeToVer29(s sessiontypes.Session, ver int64) { // upgradeToVer29 only need to be run when the current version is 28. if ver != version28 { return @@ -1680,42 +1681,42 @@ func upgradeToVer29(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.bind_info ADD INDEX sql_index (original_sql(1024),default_db(1024))", dbterror.ErrDupKeyName) } -func upgradeToVer30(s Session, ver int64) { +func upgradeToVer30(s sessiontypes.Session, ver int64) { if ver >= version30 { return } mustExecute(s, CreateStatsTopNTable) } -func upgradeToVer31(s Session, ver int64) { +func upgradeToVer31(s sessiontypes.Session, ver int64) { if ver >= version31 { return } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `last_analyze_pos` BLOB DEFAULT NULL", infoschema.ErrColumnExists) } -func upgradeToVer32(s Session, ver int64) { +func upgradeToVer32(s sessiontypes.Session, ver int64) { if ver >= version32 { return } doReentrantDDL(s, "ALTER TABLE mysql.tables_priv MODIFY table_priv SET('Select','Insert','Update','Delete','Create','Drop','Grant', 'Index', 'Alter', 'Create View', 'Show View', 'Trigger', 'References')") } -func upgradeToVer33(s Session, ver int64) { +func upgradeToVer33(s sessiontypes.Session, ver int64) { if ver >= version33 { return } doReentrantDDL(s, CreateExprPushdownBlacklist) } -func upgradeToVer34(s Session, ver int64) { +func upgradeToVer34(s sessiontypes.Session, ver int64) { if ver >= version34 { return } doReentrantDDL(s, CreateOptRuleBlacklist) } -func upgradeToVer35(s Session, ver int64) { +func upgradeToVer35(s sessiontypes.Session, ver int64) { if ver >= version35 { return } @@ -1724,7 +1725,7 @@ func upgradeToVer35(s Session, ver int64) { mustExecute(s, sql) } -func upgradeToVer36(s Session, ver int64) { +func upgradeToVer36(s sessiontypes.Session, ver int64) { if ver >= version36 { return } @@ -1734,7 +1735,7 @@ func upgradeToVer36(s Session, ver int64) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Create_tmp_table_priv='Y',Lock_tables_priv='Y',Create_routine_priv='Y',Alter_routine_priv='Y',Event_priv='Y' WHERE Super_priv='Y'") } -func upgradeToVer37(s Session, ver int64) { +func upgradeToVer37(s sessiontypes.Session, ver int64) { if ver >= version37 { return } @@ -1744,14 +1745,14 @@ func upgradeToVer37(s Session, ver int64) { mustExecute(s, sql) } -func upgradeToVer38(s Session, ver int64) { +func upgradeToVer38(s sessiontypes.Session, ver int64) { if ver >= version38 { return } doReentrantDDL(s, CreateGlobalPrivTable) } -func writeNewCollationParameter(s Session, flag bool) { +func writeNewCollationParameter(s sessiontypes.Session, flag bool) { comment := "If the new collations are enabled. Do not edit it." b := varFalse if flag { @@ -1762,7 +1763,7 @@ func writeNewCollationParameter(s Session, flag bool) { ) } -func upgradeToVer40(s Session, ver int64) { +func upgradeToVer40(s sessiontypes.Session, ver int64) { if ver >= version40 { return } @@ -1770,7 +1771,7 @@ func upgradeToVer40(s Session, ver int64) { writeNewCollationParameter(s, false) } -func upgradeToVer41(s Session, ver int64) { +func upgradeToVer41(s sessiontypes.Session, ver int64) { if ver >= version41 { return } @@ -1779,12 +1780,12 @@ func upgradeToVer41(s Session, ver int64) { } // writeDefaultExprPushDownBlacklist writes default expr pushdown blacklist into mysql.expr_pushdown_blacklist -func writeDefaultExprPushDownBlacklist(s Session) { +func writeDefaultExprPushDownBlacklist(s sessiontypes.Session) { mustExecute(s, "INSERT HIGH_PRIORITY INTO mysql.expr_pushdown_blacklist VALUES"+ "('date_add','tiflash', 'DST(daylight saving time) does not take effect in TiFlash date_add')") } -func upgradeToVer42(s Session, ver int64) { +func upgradeToVer42(s sessiontypes.Session, ver int64) { if ver >= version42 { return } @@ -1794,7 +1795,7 @@ func upgradeToVer42(s Session, ver int64) { } // Convert statement summary global variables to non-empty values. -func writeStmtSummaryVars(s Session) { +func writeStmtSummaryVars(s sessiontypes.Session) { sql := "UPDATE %n.%n SET variable_value= %? WHERE variable_name= %? AND variable_value=''" mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, variable.BoolToOnOff(variable.DefTiDBEnableStmtSummary), variable.TiDBEnableStmtSummary) mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, variable.BoolToOnOff(variable.DefTiDBStmtSummaryInternalQuery), variable.TiDBStmtSummaryInternalQuery) @@ -1804,21 +1805,21 @@ func writeStmtSummaryVars(s Session) { mustExecute(s, sql, mysql.SystemDB, mysql.GlobalVariablesTable, strconv.FormatUint(uint64(variable.DefTiDBStmtSummaryMaxSQLLength), 10), variable.TiDBStmtSummaryMaxSQLLength) } -func upgradeToVer43(s Session, ver int64) { +func upgradeToVer43(s sessiontypes.Session, ver int64) { if ver >= version43 { return } writeStmtSummaryVars(s) } -func upgradeToVer44(s Session, ver int64) { +func upgradeToVer44(s sessiontypes.Session, ver int64) { if ver >= version44 { return } mustExecute(s, "DELETE FROM mysql.global_variables where variable_name = \"tidb_isolation_read_engines\"") } -func upgradeToVer45(s Session, ver int64) { +func upgradeToVer45(s sessiontypes.Session, ver int64) { if ver >= version45 { return } @@ -1828,7 +1829,7 @@ func upgradeToVer45(s Session, ver int64) { // In v3.1.1, we wrongly replace the context of upgradeToVer39 with upgradeToVer44. If we upgrade from v3.1.1 to a newer version, // upgradeToVer39 will be missed. So we redo upgradeToVer39 here to make sure the upgrading from v3.1.1 succeed. -func upgradeToVer46(s Session, ver int64) { +func upgradeToVer46(s sessiontypes.Session, ver int64) { if ver >= version46 { return } @@ -1838,28 +1839,28 @@ func upgradeToVer46(s Session, ver int64) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET File_priv='Y' WHERE Super_priv='Y'") } -func upgradeToVer47(s Session, ver int64) { +func upgradeToVer47(s sessiontypes.Session, ver int64) { if ver >= version47 { return } doReentrantDDL(s, "ALTER TABLE mysql.bind_info ADD COLUMN `source` varchar(10) NOT NULL default 'unknown'", infoschema.ErrColumnExists) } -func upgradeToVer50(s Session, ver int64) { +func upgradeToVer50(s sessiontypes.Session, ver int64) { if ver >= version50 { return } doReentrantDDL(s, CreateSchemaIndexUsageTable) } -func upgradeToVer52(s Session, ver int64) { +func upgradeToVer52(s sessiontypes.Session, ver int64) { if ver >= version52 { return } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms MODIFY cm_sketch BLOB(6291456)") } -func upgradeToVer53(s Session, ver int64) { +func upgradeToVer53(s sessiontypes.Session, ver int64) { if ver >= version53 { return } @@ -1869,7 +1870,7 @@ func upgradeToVer53(s Session, ver int64) { mustExecute(s, sql) } -func upgradeToVer54(s Session, ver int64) { +func upgradeToVer54(s sessiontypes.Session, ver int64) { if ver >= version54 { return } @@ -1894,7 +1895,7 @@ func upgradeToVer54(s Session, ver int64) { // When cherry-pick upgradeToVer52 to v4.0, we wrongly name it upgradeToVer48. // If we upgrade from v4.0 to a newer version, the real upgradeToVer48 will be missed. // So we redo upgradeToVer48 here to make sure the upgrading from v4.0 succeeds. -func upgradeToVer55(s Session, ver int64) { +func upgradeToVer55(s sessiontypes.Session, ver int64) { if ver >= version55 { return } @@ -1944,33 +1945,33 @@ func upgradeToVer55(s Session, ver int64) { // When cherry-pick upgradeToVer54 to v4.0, we wrongly name it upgradeToVer49. // If we upgrade from v4.0 to a newer version, the real upgradeToVer49 will be missed. // So we redo upgradeToVer49 here to make sure the upgrading from v4.0 succeeds. -func upgradeToVer56(s Session, ver int64) { +func upgradeToVer56(s sessiontypes.Session, ver int64) { if ver >= version56 { return } doReentrantDDL(s, CreateStatsExtended) } -func upgradeToVer57(s Session, ver int64) { +func upgradeToVer57(s sessiontypes.Session, ver int64) { if ver >= version57 { return } insertBuiltinBindInfoRow(s) } -func initBindInfoTable(s Session) { +func initBindInfoTable(s sessiontypes.Session) { mustExecute(s, CreateBindInfoTable) insertBuiltinBindInfoRow(s) } -func insertBuiltinBindInfoRow(s Session) { +func insertBuiltinBindInfoRow(s sessiontypes.Session) { mustExecute(s, `INSERT HIGH_PRIORITY INTO mysql.bind_info(original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source) VALUES (%?, %?, "mysql", %?, "0000-00-00 00:00:00", "0000-00-00 00:00:00", "", "", %?)`, bindinfo.BuiltinPseudoSQL4BindLock, bindinfo.BuiltinPseudoSQL4BindLock, bindinfo.Builtin, bindinfo.Builtin, ) } -func upgradeToVer59(s Session, ver int64) { +func upgradeToVer59(s sessiontypes.Session, ver int64) { if ver >= version59 { return } @@ -1985,7 +1986,7 @@ func upgradeToVer59(s Session, ver int64) { writeOOMAction(s) } -func upgradeToVer60(s Session, ver int64) { +func upgradeToVer60(s sessiontypes.Session, ver int64) { if ver >= version60 { return } @@ -2002,7 +2003,7 @@ type bindInfo struct { source string } -func upgradeToVer67(s Session, ver int64) { +func upgradeToVer67(s sessiontypes.Session, ver int64) { if ver >= version67 { return } @@ -2098,21 +2099,21 @@ func updateBindInfo(iter *chunk.Iterator4Chunk, p *parser.Parser, bindMap map[st } } -func writeMemoryQuotaQuery(s Session) { +func writeMemoryQuotaQuery(s sessiontypes.Session) { comment := "memory_quota_query is 32GB by default in v3.0.x, 1GB by default in v4.0.x+" mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE VARIABLE_VALUE=%?`, mysql.SystemDB, mysql.TiDBTable, tidbDefMemoryQuotaQuery, 32<<30, comment, 32<<30, ) } -func upgradeToVer62(s Session, ver int64) { +func upgradeToVer62(s sessiontypes.Session, ver int64) { if ver >= version62 { return } doReentrantDDL(s, "ALTER TABLE mysql.stats_buckets ADD COLUMN `ndv` bigint not null default 0", infoschema.ErrColumnExists) } -func upgradeToVer63(s Session, ver int64) { +func upgradeToVer63(s sessiontypes.Session, ver int64) { if ver >= version63 { return } @@ -2120,7 +2121,7 @@ func upgradeToVer63(s Session, ver int64) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Create_tablespace_priv='Y' where Super_priv='Y'") } -func upgradeToVer64(s Session, ver int64) { +func upgradeToVer64(s sessiontypes.Session, ver int64) { if ver >= version64 { return } @@ -2129,35 +2130,35 @@ func upgradeToVer64(s Session, ver int64) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Repl_slave_priv='Y',Repl_client_priv='Y' where Super_priv='Y'") } -func upgradeToVer65(s Session, ver int64) { +func upgradeToVer65(s sessiontypes.Session, ver int64) { if ver >= version65 { return } doReentrantDDL(s, CreateStatsFMSketchTable) } -func upgradeToVer66(s Session, ver int64) { +func upgradeToVer66(s sessiontypes.Session, ver int64) { if ver >= version66 { return } mustExecute(s, "set @@global.tidb_track_aggregate_memory_usage = 1") } -func upgradeToVer68(s Session, ver int64) { +func upgradeToVer68(s sessiontypes.Session, ver int64) { if ver >= version68 { return } mustExecute(s, "DELETE FROM mysql.global_variables where VARIABLE_NAME = 'tidb_enable_clustered_index' and VARIABLE_VALUE = 'OFF'") } -func upgradeToVer69(s Session, ver int64) { +func upgradeToVer69(s sessiontypes.Session, ver int64) { if ver >= version69 { return } doReentrantDDL(s, CreateGlobalGrantsTable) } -func upgradeToVer70(s Session, ver int64) { +func upgradeToVer70(s sessiontypes.Session, ver int64) { if ver >= version70 { return } @@ -2165,28 +2166,28 @@ func upgradeToVer70(s Session, ver int64) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET plugin='mysql_native_password'") } -func upgradeToVer71(s Session, ver int64) { +func upgradeToVer71(s sessiontypes.Session, ver int64) { if ver >= version71 { return } mustExecute(s, "UPDATE mysql.global_variables SET VARIABLE_VALUE='OFF' WHERE VARIABLE_NAME = 'tidb_multi_statement_mode' AND VARIABLE_VALUE = 'WARN'") } -func upgradeToVer72(s Session, ver int64) { +func upgradeToVer72(s sessiontypes.Session, ver int64) { if ver >= version72 { return } doReentrantDDL(s, "ALTER TABLE mysql.stats_meta ADD COLUMN snapshot BIGINT(64) UNSIGNED NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } -func upgradeToVer73(s Session, ver int64) { +func upgradeToVer73(s sessiontypes.Session, ver int64) { if ver >= version73 { return } doReentrantDDL(s, CreateCapturePlanBaselinesBlacklist) } -func upgradeToVer74(s Session, ver int64) { +func upgradeToVer74(s sessiontypes.Session, ver int64) { if ver >= version74 { return } @@ -2194,7 +2195,7 @@ func upgradeToVer74(s Session, ver int64) { mustExecute(s, fmt.Sprintf("UPDATE mysql.global_variables SET VARIABLE_VALUE='%[1]v' WHERE VARIABLE_NAME = 'tidb_stmt_summary_max_stmt_count' AND CAST(VARIABLE_VALUE AS SIGNED) = 200", variable.DefTiDBStmtSummaryMaxStmtCount)) } -func upgradeToVer75(s Session, ver int64) { +func upgradeToVer75(s sessiontypes.Session, ver int64) { if ver >= version75 { return } @@ -2205,21 +2206,21 @@ func upgradeToVer75(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.columns_priv MODIFY COLUMN Host CHAR(255)") } -func upgradeToVer76(s Session, ver int64) { +func upgradeToVer76(s sessiontypes.Session, ver int64) { if ver >= version76 { return } doReentrantDDL(s, "ALTER TABLE mysql.columns_priv MODIFY COLUMN Column_priv SET('Select','Insert','Update','References')") } -func upgradeToVer77(s Session, ver int64) { +func upgradeToVer77(s sessiontypes.Session, ver int64) { if ver >= version77 { return } doReentrantDDL(s, CreateColumnStatsUsageTable) } -func upgradeToVer78(s Session, ver int64) { +func upgradeToVer78(s sessiontypes.Session, ver int64) { if ver >= version78 { return } @@ -2228,14 +2229,14 @@ func upgradeToVer78(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms MODIFY last_analyze_pos LONGBLOB DEFAULT NULL") } -func upgradeToVer79(s Session, ver int64) { +func upgradeToVer79(s sessiontypes.Session, ver int64) { if ver >= version79 { return } doReentrantDDL(s, CreateTableCacheMetaTable) } -func upgradeToVer80(s Session, ver int64) { +func upgradeToVer80(s sessiontypes.Session, ver int64) { if ver >= version80 { return } @@ -2258,7 +2259,7 @@ func upgradeToVer80(s Session, ver int64) { // For users that upgrade TiDB from a pre-4.0 version, we want to disable index merge by default. // This helps minimize query plan regressions. -func upgradeToVer81(s Session, ver int64) { +func upgradeToVer81(s sessiontypes.Session, ver int64) { if ver >= version81 { return } @@ -2279,49 +2280,49 @@ func upgradeToVer81(s Session, ver int64) { mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableIndexMerge, variable.Off) } -func upgradeToVer82(s Session, ver int64) { +func upgradeToVer82(s sessiontypes.Session, ver int64) { if ver >= version82 { return } doReentrantDDL(s, CreateAnalyzeOptionsTable) } -func upgradeToVer83(s Session, ver int64) { +func upgradeToVer83(s sessiontypes.Session, ver int64) { if ver >= version83 { return } doReentrantDDL(s, CreateStatsHistory) } -func upgradeToVer84(s Session, ver int64) { +func upgradeToVer84(s sessiontypes.Session, ver int64) { if ver >= version84 { return } doReentrantDDL(s, CreateStatsMetaHistory) } -func upgradeToVer85(s Session, ver int64) { +func upgradeToVer85(s sessiontypes.Session, ver int64) { if ver >= version85 { return } mustExecute(s, fmt.Sprintf("UPDATE HIGH_PRIORITY mysql.bind_info SET status= '%s' WHERE status = '%s'", bindinfo.Enabled, bindinfo.Using)) } -func upgradeToVer86(s Session, ver int64) { +func upgradeToVer86(s sessiontypes.Session, ver int64) { if ver >= version86 { return } doReentrantDDL(s, "ALTER TABLE mysql.tables_priv MODIFY COLUMN Column_priv SET('Select','Insert','Update','References')") } -func upgradeToVer87(s Session, ver int64) { +func upgradeToVer87(s sessiontypes.Session, ver int64) { if ver >= version87 { return } doReentrantDDL(s, CreateAnalyzeJobs) } -func upgradeToVer88(s Session, ver int64) { +func upgradeToVer88(s sessiontypes.Session, ver int64) { if ver >= version88 { return } @@ -2329,7 +2330,7 @@ func upgradeToVer88(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.user CHANGE `Repl_client_priv` `Repl_client_priv` ENUM('N','Y') NOT NULL DEFAULT 'N' AFTER `Repl_slave_priv`") } -func upgradeToVer89(s Session, ver int64) { +func upgradeToVer89(s sessiontypes.Session, ver int64) { if ver >= version89 { return } @@ -2342,7 +2343,7 @@ func upgradeToVer89(s Session, ver int64) { // (not guaranteed to be the same on all servers), and writes a message // to the error log. The message is important since the behavior is weird // (changes to the config file will no longer take effect past this point). -func importConfigOption(s Session, configName, svName, valStr string) { +func importConfigOption(s sessiontypes.Session, configName, svName, valStr string) { message := fmt.Sprintf("%s is now configured by the system variable %s. One-time importing the value specified in tidb.toml file", configName, svName) logutil.BgLogger().Warn(message, zap.String("value", valStr)) // We use insert ignore, since if its a duplicate we don't want to overwrite any user-set values. @@ -2351,7 +2352,7 @@ func importConfigOption(s Session, configName, svName, valStr string) { mustExecute(s, sql) } -func upgradeToVer90(s Session, ver int64) { +func upgradeToVer90(s sessiontypes.Session, ver int64) { if ver >= version90 { return } @@ -2369,7 +2370,7 @@ func upgradeToVer90(s Session, ver int64) { importConfigOption(s, "oom-action", variable.TiDBMemOOMAction, valStr) } -func upgradeToVer91(s Session, ver int64) { +func upgradeToVer91(s sessiontypes.Session, ver int64) { if ver >= version91 { return } @@ -2383,7 +2384,7 @@ func upgradeToVer91(s Session, ver int64) { importConfigOption(s, "prepared-plan-cache.memory-guard-ratio", variable.TiDBPrepPlanCacheMemoryGuardRatio, valStr) } -func upgradeToVer93(s Session, ver int64) { +func upgradeToVer93(s sessiontypes.Session, ver int64) { if ver >= version93 { return } @@ -2391,21 +2392,21 @@ func upgradeToVer93(s Session, ver int64) { importConfigOption(s, "oom-use-tmp-storage", variable.TiDBEnableTmpStorageOnOOM, valStr) } -func upgradeToVer94(s Session, ver int64) { +func upgradeToVer94(s sessiontypes.Session, ver int64) { if ver >= version94 { return } mustExecute(s, CreateMDLView) } -func upgradeToVer95(s Session, ver int64) { +func upgradeToVer95(s sessiontypes.Session, ver int64) { if ver >= version95 { return } doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN IF NOT EXISTS `User_attributes` JSON") } -func upgradeToVer97(s Session, ver int64) { +func upgradeToVer97(s sessiontypes.Session, ver int64) { if ver >= version97 { return } @@ -2426,19 +2427,19 @@ func upgradeToVer97(s Session, ver int64) { mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBOptRangeMaxSize, 0) } -func upgradeToVer98(s Session, ver int64) { +func upgradeToVer98(s sessiontypes.Session, ver int64) { if ver >= version98 { return } doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN IF NOT EXISTS `Token_issuer` varchar(255)") } -func upgradeToVer99Before(s Session) { +func upgradeToVer99Before(s sessiontypes.Session) { mustExecute(s, "INSERT HIGH_PRIORITY IGNORE INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableMDL, 0) } -func upgradeToVer99After(s Session) { +func upgradeToVer99After(s sessiontypes.Session) { sql := fmt.Sprintf("UPDATE HIGH_PRIORITY %[1]s.%[2]s SET VARIABLE_VALUE = %[4]d WHERE VARIABLE_NAME = '%[3]s'", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableMDL, 1) mustExecute(s, sql) @@ -2449,7 +2450,7 @@ func upgradeToVer99After(s Session) { terror.MustNil(err) } -func upgradeToVer100(s Session, ver int64) { +func upgradeToVer100(s sessiontypes.Session, ver int64) { if ver >= version100 { return } @@ -2457,28 +2458,28 @@ func upgradeToVer100(s Session, ver int64) { importConfigOption(s, "performance.server-memory-quota", variable.TiDBServerMemoryLimit, valStr) } -func upgradeToVer101(s Session, ver int64) { +func upgradeToVer101(s sessiontypes.Session, ver int64) { if ver >= version101 { return } doReentrantDDL(s, CreatePlanReplayerStatusTable) } -func upgradeToVer102(s Session, ver int64) { +func upgradeToVer102(s sessiontypes.Session, ver int64) { if ver >= version102 { return } doReentrantDDL(s, CreatePlanReplayerTaskTable) } -func upgradeToVer103(s Session, ver int64) { +func upgradeToVer103(s sessiontypes.Session, ver int64) { if ver >= version103 { return } doReentrantDDL(s, CreateStatsTableLocked) } -func upgradeToVer104(s Session, ver int64) { +func upgradeToVer104(s sessiontypes.Session, ver int64) { if ver >= version104 { return } @@ -2488,7 +2489,7 @@ func upgradeToVer104(s Session, ver int64) { } // For users that upgrade TiDB from a pre-6.0 version, we want to disable tidb cost model2 by default to keep plans unchanged. -func upgradeToVer105(s Session, ver int64) { +func upgradeToVer105(s sessiontypes.Session, ver int64) { if ver >= version105 { return } @@ -2507,7 +2508,7 @@ func upgradeToVer105(s Session, ver int64) { mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBCostModelVersion, "1") } -func upgradeToVer106(s Session, ver int64) { +func upgradeToVer106(s sessiontypes.Session, ver int64) { if ver >= version106 { return } @@ -2516,7 +2517,7 @@ func upgradeToVer106(s Session, ver int64) { doReentrantDDL(s, "Alter table mysql.user add COLUMN IF NOT EXISTS `Password_reuse_time` smallint unsigned DEFAULT NULL AFTER `Password_reuse_history`") } -func upgradeToVer107(s Session, ver int64) { +func upgradeToVer107(s sessiontypes.Session, ver int64) { if ver >= version107 { return } @@ -2525,7 +2526,7 @@ func upgradeToVer107(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN IF NOT EXISTS `Password_lifetime` SMALLINT UNSIGNED DEFAULT NULL") } -func upgradeToVer108(s Session, ver int64) { +func upgradeToVer108(s sessiontypes.Session, ver int64) { if ver >= version108 { return } @@ -2533,7 +2534,7 @@ func upgradeToVer108(s Session, ver int64) { } // For users that upgrade TiDB from a 6.2-6.4 version, we want to disable tidb gc_aware_memory_track by default. -func upgradeToVer109(s Session, ver int64) { +func upgradeToVer109(s sessiontypes.Session, ver int64) { if ver >= version109 { return } @@ -2542,7 +2543,7 @@ func upgradeToVer109(s Session, ver int64) { } // For users that upgrade TiDB from a 5.4-6.4 version, we want to enable tidb tidb_stats_load_pseudo_timeout by default. -func upgradeToVer110(s Session, ver int64) { +func upgradeToVer110(s sessiontypes.Session, ver int64) { if ver >= version110 { return } @@ -2550,14 +2551,14 @@ func upgradeToVer110(s Session, ver int64) { mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBStatsLoadPseudoTimeout, 1) } -func upgradeToVer130(s Session, ver int64) { +func upgradeToVer130(s sessiontypes.Session, ver int64) { if ver >= version130 { return } doReentrantDDL(s, "ALTER TABLE mysql.stats_meta_history ADD COLUMN IF NOT EXISTS `source` varchar(40) NOT NULL after `version`;") } -func upgradeToVer131(s Session, ver int64) { +func upgradeToVer131(s sessiontypes.Session, ver int64) { if ver >= version131 { return } @@ -2565,14 +2566,14 @@ func upgradeToVer131(s Session, ver int64) { doReentrantDDL(s, CreateTTLJobHistory) } -func upgradeToVer132(s Session, ver int64) { +func upgradeToVer132(s sessiontypes.Session, ver int64) { if ver >= version132 { return } doReentrantDDL(s, CreateMDLView) } -func upgradeToVer133(s Session, ver int64) { +func upgradeToVer133(s sessiontypes.Session, ver int64) { if ver >= version133 { return } @@ -2580,7 +2581,7 @@ func upgradeToVer133(s Session, ver int64) { mysql.SystemDB, mysql.GlobalVariablesTable, variable.DefTiDBServerMemoryLimit, variable.TiDBServerMemoryLimit, "0") } -func upgradeToVer134(s Session, ver int64) { +func upgradeToVer134(s sessiontypes.Session, ver int64) { if ver >= version134 { return } @@ -2592,7 +2593,7 @@ func upgradeToVer134(s Session, ver int64) { } // For users that upgrade TiDB from a pre-7.0 version, we want to set tidb_opt_advanced_join_hint to off by default to keep plans unchanged. -func upgradeToVer135(s Session, ver int64) { +func upgradeToVer135(s sessiontypes.Session, ver int64) { if ver >= version135 { return } @@ -2612,7 +2613,7 @@ func upgradeToVer135(s Session, ver int64) { mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBOptAdvancedJoinHint, false) } -func upgradeToVer136(s Session, ver int64) { +func upgradeToVer136(s sessiontypes.Session, ver int64) { if ver >= version136 { return } @@ -2621,26 +2622,26 @@ func upgradeToVer136(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask ADD INDEX idx_task_key(task_key)", dbterror.ErrDupKeyName) } -func upgradeToVer137(_ Session, _ int64) { +func upgradeToVer137(_ sessiontypes.Session, _ int64) { // NOOP, we don't depend on ddl to init the default group due to backward compatible issue. } // For users that upgrade TiDB from a version below 7.0, we want to enable tidb tidb_enable_null_aware_anti_join by default. -func upgradeToVer138(s Session, ver int64) { +func upgradeToVer138(s sessiontypes.Session, ver int64) { if ver >= version138 { return } mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBOptimizerEnableNAAJ, variable.On) } -func upgradeToVer139(s Session, ver int64) { +func upgradeToVer139(s sessiontypes.Session, ver int64) { if ver >= version139 { return } mustExecute(s, CreateLoadDataJobs) } -func upgradeToVer140(s Session, ver int64) { +func upgradeToVer140(s sessiontypes.Session, ver int64) { if ver >= version140 { return } @@ -2650,7 +2651,7 @@ func upgradeToVer140(s Session, ver int64) { // upgradeToVer141 sets the value of `tidb_session_plan_cache_size` as `tidb_prepared_plan_cache_size` for compatibility, // and update tidb_load_based_replica_read_threshold from 0 to 4. -func upgradeToVer141(s Session, ver int64) { +func upgradeToVer141(s sessiontypes.Session, ver int64) { if ver >= version141 { return } @@ -2674,7 +2675,7 @@ func upgradeToVer141(s Session, ver int64) { mustExecute(s, "REPLACE HIGH_PRIORITY INTO %n.%n VALUES (%?, %?);", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBLoadBasedReplicaReadThreshold, variable.DefTiDBLoadBasedReplicaReadThreshold.String()) } -func upgradeToVer142(s Session, ver int64) { +func upgradeToVer142(s sessiontypes.Session, ver int64) { if ver >= version142 { return } @@ -2693,7 +2694,7 @@ func upgradeToVer142(s Session, ver int64) { mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableNonPreparedPlanCache, variable.Off) } -func upgradeToVer143(s Session, ver int64) { +func upgradeToVer143(s sessiontypes.Session, ver int64) { if ver >= version143 { return } @@ -2701,7 +2702,7 @@ func upgradeToVer143(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask ADD COLUMN `error` BLOB", infoschema.ErrColumnExists) } -func upgradeToVer144(s Session, ver int64) { +func upgradeToVer144(s sessiontypes.Session, ver int64) { if ver >= version144 { return } @@ -2720,7 +2721,7 @@ func upgradeToVer144(s Session, ver int64) { mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBPlanCacheInvalidationOnFreshStats, variable.Off) } -func upgradeToVer146(s Session, ver int64) { +func upgradeToVer146(s sessiontypes.Session, ver int64) { if ver >= version146 { return } @@ -2728,42 +2729,42 @@ func upgradeToVer146(s Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.stats_history ADD INDEX idx_create_time (create_time)", dbterror.ErrDupKeyName) } -func upgradeToVer167(s Session, ver int64) { +func upgradeToVer167(s sessiontypes.Session, ver int64) { if ver >= version167 { return } doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask ADD COLUMN `step` INT AFTER `id`", infoschema.ErrColumnExists) } -func upgradeToVer168(s Session, ver int64) { +func upgradeToVer168(s sessiontypes.Session, ver int64) { if ver >= version168 { return } mustExecute(s, CreateImportJobs) } -func upgradeToVer169(s Session, ver int64) { +func upgradeToVer169(s sessiontypes.Session, ver int64) { if ver >= version169 { return } mustExecute(s, CreateRunawayTable) } -func upgradeToVer170(s Session, ver int64) { +func upgradeToVer170(s sessiontypes.Session, ver int64) { if ver >= version170 { return } mustExecute(s, CreateTimers) } -func upgradeToVer171(s Session, ver int64) { +func upgradeToVer171(s sessiontypes.Session, ver int64) { if ver >= version171 { return } mustExecute(s, "ALTER TABLE mysql.tidb_runaway_queries CHANGE COLUMN `tidb_server` `tidb_server` varchar(512)") } -func upgradeToVer172(s Session, ver int64) { +func upgradeToVer172(s sessiontypes.Session, ver int64) { if ver >= version172 { return } @@ -2772,14 +2773,14 @@ func upgradeToVer172(s Session, ver int64) { mustExecute(s, CreateDoneRunawayWatchTable) } -func upgradeToVer173(s Session, ver int64) { +func upgradeToVer173(s sessiontypes.Session, ver int64) { if ver >= version173 { return } doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask ADD COLUMN `summary` JSON", infoschema.ErrColumnExists) } -func upgradeToVer174(s Session, ver int64) { +func upgradeToVer174(s sessiontypes.Session, ver int64) { if ver >= version174 { return } @@ -2794,7 +2795,7 @@ func upgradeToVer174(s Session, ver int64) { // the issue #44298 that bindings for `in (?)` can't work for `in (?, ?, ?)`. // After this update, multiple bindings may have the same `original_sql`, but it's OK, and // for safety, don't remove duplicated bindings when upgrading. -func upgradeToVer175(s Session, ver int64) { +func upgradeToVer175(s sessiontypes.Session, ver int64) { if ver >= version175 { return } @@ -2840,14 +2841,14 @@ func upgradeToVer175(s Session, ver int64) { } } -func upgradeToVer176(s Session, ver int64) { +func upgradeToVer176(s sessiontypes.Session, ver int64) { if ver >= version176 { return } mustExecute(s, CreateGlobalTaskHistory) } -func upgradeToVer177(s Session, ver int64) { +func upgradeToVer177(s sessiontypes.Session, ver int64) { if ver >= version177 { return } @@ -2860,7 +2861,7 @@ func upgradeToVer177(s Session, ver int64) { } // writeDDLTableVersion writes mDDLTableVersion into mysql.tidb -func writeDDLTableVersion(s Session) { +func writeDDLTableVersion(s sessiontypes.Session) { var err error var ddlTableVersion meta.DDLTableVersion err = kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap), s.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error { @@ -2878,21 +2879,21 @@ func writeDDLTableVersion(s Session) { ) } -func upgradeToVer178(s Session, ver int64) { +func upgradeToVer178(s sessiontypes.Session, ver int64) { if ver >= version178 { return } writeDDLTableVersion(s) } -func upgradeToVer179(s Session, ver int64) { +func upgradeToVer179(s sessiontypes.Session, ver int64) { if ver >= version179 { return } doReentrantDDL(s, "ALTER TABLE mysql.global_variables MODIFY COLUMN `VARIABLE_VALUE` varchar(16383)") } -func writeOOMAction(s Session) { +func writeOOMAction(s sessiontypes.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= %?`, mysql.SystemDB, mysql.TiDBTable, tidbDefOOMAction, variable.OOMActionLog, comment, variable.OOMActionLog, @@ -2900,7 +2901,7 @@ func writeOOMAction(s Session) { } // updateBootstrapVer updates bootstrap version variable in mysql.TiDB table. -func updateBootstrapVer(s Session) { +func updateBootstrapVer(s sessiontypes.Session) { // Update bootstrap version. mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, "TiDB bootstrap version.") ON DUPLICATE KEY UPDATE VARIABLE_VALUE=%?`, mysql.SystemDB, mysql.TiDBTable, tidbServerVersionVar, currentBootstrapVersion, currentBootstrapVersion, @@ -2908,7 +2909,7 @@ func updateBootstrapVer(s Session) { } // getBootstrapVersion gets bootstrap version from mysql.tidb table; -func getBootstrapVersion(s Session) (int64, error) { +func getBootstrapVersion(s sessiontypes.Session) (int64, error) { sVal, isNull, err := getTiDBVar(s, tidbServerVersionVar) if err != nil { return 0, errors.Trace(err) @@ -2920,7 +2921,7 @@ func getBootstrapVersion(s Session) (int64, error) { } // doDDLWorks executes DDL statements in bootstrap stage. -func doDDLWorks(s Session) { +func doDDLWorks(s sessiontypes.Session) { // Create a test database. mustExecute(s, "CREATE DATABASE IF NOT EXISTS test") // Create system db. @@ -3025,7 +3026,7 @@ func doDDLWorks(s Session) { // 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) error { +func doBootstrapSQLFile(s sessiontypes.Session) error { sqlFile := config.GetGlobalConfig().InitializeSQLFile ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBootstrap) if sqlFile == "" { @@ -3064,7 +3065,7 @@ func doBootstrapSQLFile(s Session) error { // doDMLWorks executes DML statements in bootstrap stage. // All the statements run in a single transaction. -func doDMLWorks(s Session) { +func doDMLWorks(s sessiontypes.Session) { mustExecute(s, "BEGIN") if config.GetGlobalConfig().Security.SecureBootstrap { // If secure bootstrap is enabled, we create a root@localhost account which can login with auth_socket. @@ -3168,7 +3169,7 @@ func doDMLWorks(s Session) { } } -func mustExecute(s Session, sql string, args ...interface{}) { +func mustExecute(s sessiontypes.Session, sql string, args ...interface{}) { ctx, cancel := context.WithTimeout(context.Background(), time.Duration(internalSQLTimeout)*time.Second) ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnBootstrap) _, err := s.ExecuteInternal(ctx, sql, args...) diff --git a/pkg/session/bootstrap_test.go b/pkg/session/bootstrap_test.go index e23b15fca0326..61e87361f9c31 100644 --- a/pkg/session/bootstrap_test.go +++ b/pkg/session/bootstrap_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta" "github.com/pingcap/tidb/pkg/parser/auth" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/statistics" @@ -1590,7 +1591,7 @@ func TestTiDBUpgradeToVer140(t *testing.T) { }() ver139 := version139 - resetTo139 := func(s Session) { + resetTo139 := func(s sessiontypes.Session) { txn, err := store.Begin() require.NoError(t, err) m := meta.NewMeta(txn) @@ -2004,7 +2005,7 @@ func TestTiDBBindingInListToVer175(t *testing.T) { MustExec(t, seV174, "insert into mysql.bind_info values ('select * from `test` . `t` where `a` in ( ? )', 'SELECT /*+ use_index(`t` `c`)*/ * FROM `test`.`t` WHERE `a` IN (1)', 'test', 'enabled', '2023-09-13 14:41:38.319', '2023-09-13 14:41:36.319', 'utf8', 'utf8_general_ci', 'manual', '', '')") MustExec(t, seV174, "insert into mysql.bind_info values ('select * from `test` . `t` where `a` in ( ? ) and `b` in ( ... )', 'SELECT /*+ use_index(`t` `c`)*/ * FROM `test`.`t` WHERE `a` IN (1) AND `b` IN (1,2,3)', 'test', 'enabled', '2023-09-13 14:41:37.319', '2023-09-13 14:41:38.319', 'utf8', 'utf8_general_ci', 'manual', '', '')") - showBindings := func(s Session) (records []string) { + showBindings := func(s sessiontypes.Session) (records []string) { MustExec(t, s, "admin reload bindings") res := MustExecToRecodeSet(t, s, "show global bindings") chk := res.NewChunk(nil) @@ -2043,7 +2044,7 @@ func TestTiDBBindingInListToVer175(t *testing.T) { require.Equal(t, []string{"SELECT /*+ use_index(`t` `c`)*/ * FROM `test`.`t` WHERE `a` IN (1) AND `b` IN (1,2,3):select * from `test` . `t` where `a` in ( ... ) and `b` in ( ... )", "SELECT /*+ use_index(`t` `c`)*/ * FROM `test`.`t` WHERE `a` IN (1):select * from `test` . `t` where `a` in ( ... )"}, bindings) - planFromBinding := func(s Session, q string) { + planFromBinding := func(s sessiontypes.Session, q string) { MustExec(t, s, q) res := MustExecToRecodeSet(t, s, "select @@last_plan_from_binding") chk := res.NewChunk(nil) diff --git a/pkg/session/bootstraptest/BUILD.bazel b/pkg/session/bootstraptest/BUILD.bazel index 67477675738cf..4b25d2e72d946 100644 --- a/pkg/session/bootstraptest/BUILD.bazel +++ b/pkg/session/bootstraptest/BUILD.bazel @@ -19,6 +19,7 @@ go_test( "//pkg/parser/terror", "//pkg/server/handler", "//pkg/session", #keep + "//pkg/session/types", "//pkg/sessionctx", "//pkg/testkit", #keep "//pkg/testkit/testmain", diff --git a/pkg/session/bootstraptest/bootstrap_upgrade_test.go b/pkg/session/bootstraptest/bootstrap_upgrade_test.go index adde5a6437ec9..fbe12509b94c1 100644 --- a/pkg/session/bootstraptest/bootstrap_upgrade_test.go +++ b/pkg/session/bootstraptest/bootstrap_upgrade_test.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/server/handler" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/testkit" tidb_util "github.com/pingcap/tidb/pkg/util" @@ -443,7 +444,7 @@ func TestUpgradeVersionForPausedJob(t *testing.T) { } // checkDDLJobExecSucc is used to make sure the DDL operation is successful. -func checkDDLJobExecSucc(t *testing.T, se session.Session, jobID int64) { +func checkDDLJobExecSucc(t *testing.T, se sessiontypes.Session, jobID int64) { sql := fmt.Sprintf(" admin show ddl jobs where job_id=%d", jobID) suc := false for i := 0; i < 20; i++ { @@ -686,7 +687,7 @@ func TestUpgradeWithPauseDDL(t *testing.T) { tc := session.TestCallback{Cnt: atomicutil.NewInt32(0)} sql := "select job_meta, processing from mysql.tidb_ddl_job where job_id in (select min(job_id) from mysql.tidb_ddl_job group by schema_ids, table_ids, processing) order by processing desc, job_id" - tc.OnBootstrapBeforeExported = func(s session.Session) { + tc.OnBootstrapBeforeExported = func(s sessiontypes.Session) { rows, err := execute(context.Background(), s, sql) require.NoError(t, err) require.Len(t, rows, 0) @@ -709,7 +710,7 @@ func TestUpgradeWithPauseDDL(t *testing.T) { }() <-ch } - checkDDLJobState := func(s session.Session) { + checkDDLJobState := func(s sessiontypes.Session) { rows, err := execute(context.Background(), s, sql) require.NoError(t, err) for _, row := range rows { @@ -727,7 +728,7 @@ func TestUpgradeWithPauseDDL(t *testing.T) { } } // Before every test bootstrap(DDL operation), we add a user and a system DB's DDL operations. - tc.OnBootstrapExported = func(s session.Session) { + tc.OnBootstrapExported = func(s sessiontypes.Session) { var query1, query2 string switch tc.Cnt.Load() % 2 { case 0: @@ -745,7 +746,7 @@ func TestUpgradeWithPauseDDL(t *testing.T) { checkDDLJobState(s) } - tc.OnBootstrapAfterExported = func(s session.Session) { + tc.OnBootstrapAfterExported = func(s sessiontypes.Session) { checkDDLJobState(s) } session.TestHook = tc diff --git a/pkg/session/mock_bootstrap.go b/pkg/session/mock_bootstrap.go index 94d633c13c0b5..9a200b6518a29 100644 --- a/pkg/session/mock_bootstrap.go +++ b/pkg/session/mock_bootstrap.go @@ -22,6 +22,7 @@ import ( "flag" "time" + "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/util/logutil" atomicutil "go.uber.org/atomic" "go.uber.org/zap" @@ -75,7 +76,7 @@ var allDDLs = []string{ var mockLatestVer = currentBootstrapVersion + 1 -func mockUpgradeToVerLatest(s Session, ver int64) { +func mockUpgradeToVerLatest(s types.Session, ver int64) { logutil.BgLogger().Info("mock upgrade to ver latest", zap.Int64("old ver", ver), zap.Int64("mock latest ver", mockLatestVer)) if ver >= mockLatestVer { return @@ -114,7 +115,7 @@ func mockUpgradeToVerLatest(s Session, ver int64) { } // mockSimpleUpgradeToVerLatest mocks a simple bootstrapVersion(make the test faster). -func mockSimpleUpgradeToVerLatest(s Session, ver int64) { +func mockSimpleUpgradeToVerLatest(s types.Session, ver int64) { logutil.BgLogger().Info("mock upgrade to ver latest", zap.Int64("old ver", ver), zap.Int64("mock latest ver", mockLatestVer)) if ver >= mockLatestVer { return @@ -151,7 +152,7 @@ const ( // MockUpgradeToVerLatestKind is used to indicate the use of different mock bootstrapVersion. var MockUpgradeToVerLatestKind = defaultMockUpgradeToVerLatest -func addMockBootstrapVersionForTest(s Session) { +func addMockBootstrapVersionForTest(s types.Session) { if WithMockUpgrade == nil || !*WithMockUpgrade { return } @@ -168,51 +169,51 @@ func addMockBootstrapVersionForTest(s Session) { // Callback is used for Test. type Callback interface { // OnBootstrapBefore is called before doing bootstrap. - OnBootstrapBefore(s Session) + OnBootstrapBefore(s types.Session) // OnBootstrap is called doing bootstrap. - OnBootstrap(s Session) + OnBootstrap(s types.Session) // OnBootstrapAfter is called after doing bootstrap. - OnBootstrapAfter(s Session) + OnBootstrapAfter(s types.Session) } // BaseCallback implements Callback interfaces. type BaseCallback struct{} // OnBootstrapBefore implements Callback interface. -func (*BaseCallback) OnBootstrapBefore(Session) {} +func (*BaseCallback) OnBootstrapBefore(types.Session) {} // OnBootstrap implements Callback interface. -func (*BaseCallback) OnBootstrap(Session) {} +func (*BaseCallback) OnBootstrap(types.Session) {} // OnBootstrapAfter implements Callback interface. -func (*BaseCallback) OnBootstrapAfter(Session) {} +func (*BaseCallback) OnBootstrapAfter(types.Session) {} // TestCallback is used to customize user callback themselves. type TestCallback struct { *BaseCallback Cnt *atomicutil.Int32 - OnBootstrapBeforeExported func(s Session) - OnBootstrapExported func(s Session) - OnBootstrapAfterExported func(s Session) + OnBootstrapBeforeExported func(s types.Session) + OnBootstrapExported func(s types.Session) + OnBootstrapAfterExported func(s types.Session) } // OnBootstrapBefore mocks the same behavior with the main bootstrap hook. -func (tc *TestCallback) OnBootstrapBefore(s Session) { +func (tc *TestCallback) OnBootstrapBefore(s types.Session) { if tc.OnBootstrapBeforeExported != nil { tc.OnBootstrapBeforeExported(s) } } // OnBootstrap mocks the same behavior with the main bootstrap hook. -func (tc *TestCallback) OnBootstrap(s Session) { +func (tc *TestCallback) OnBootstrap(s types.Session) { if tc.OnBootstrapExported != nil { tc.OnBootstrapExported(s) } } // OnBootstrapAfter mocks the same behavior with the main bootstrap hook. -func (tc *TestCallback) OnBootstrapAfter(s Session) { +func (tc *TestCallback) OnBootstrapAfter(s types.Session) { if tc.OnBootstrapAfterExported != nil { tc.OnBootstrapAfterExported(s) } diff --git a/pkg/session/nontransactional.go b/pkg/session/nontransactional.go index 197643c48ebb3..5738a3d545dbf 100644 --- a/pkg/session/nontransactional.go +++ b/pkg/session/nontransactional.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/opcode" "github.com/pingcap/tidb/pkg/planner/core" session_metrics "github.com/pingcap/tidb/pkg/session/metrics" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/types" @@ -74,7 +75,7 @@ func (j job) String(redacted bool) string { } // HandleNonTransactionalDML is the entry point for a non-transactional DML statement -func HandleNonTransactionalDML(ctx context.Context, stmt *ast.NonTransactionalDMLStmt, se Session) (sqlexec.RecordSet, error) { +func HandleNonTransactionalDML(ctx context.Context, stmt *ast.NonTransactionalDMLStmt, se sessiontypes.Session) (sqlexec.RecordSet, error) { sessVars := se.GetSessionVars() originalReadStaleness := se.GetSessionVars().ReadStaleness // NT-DML is a write operation, and should not be affected by read_staleness that is supposed to affect only SELECT. @@ -129,7 +130,7 @@ func HandleNonTransactionalDML(ctx context.Context, stmt *ast.NonTransactionalDM // // Note: this is not a comprehensive check. // We do this to help user prevent some easy mistakes, at an acceptable maintenance cost. -func checkConstraintWithShardColumn(se Session, stmt *ast.NonTransactionalDMLStmt, +func checkConstraintWithShardColumn(se sessiontypes.Session, stmt *ast.NonTransactionalDMLStmt, tableName *ast.TableName, shardColumnInfo *model.ColumnInfo, tableSources []*ast.TableSource) error { switch s := stmt.DMLStmt.(type) { case *ast.UpdateStmt: @@ -148,7 +149,7 @@ func checkConstraintWithShardColumn(se Session, stmt *ast.NonTransactionalDMLStm } // shard column should not be updated. -func checkUpdateShardColumn(se Session, assignments []*ast.Assignment, shardColumnInfo *model.ColumnInfo, +func checkUpdateShardColumn(se sessiontypes.Session, assignments []*ast.Assignment, shardColumnInfo *model.ColumnInfo, tableName *ast.TableName, tableSources []*ast.TableSource, isUpdate bool) error { // if the table has alias, the alias is used in assignments, and we should use aliased name to compare aliasedShardColumnTableName := tableName.Name.L @@ -178,7 +179,7 @@ func checkUpdateShardColumn(se Session, assignments []*ast.Assignment, shardColu return nil } -func checkConstraint(stmt *ast.NonTransactionalDMLStmt, se Session) error { +func checkConstraint(stmt *ast.NonTransactionalDMLStmt, se sessiontypes.Session) error { sessVars := se.GetSessionVars() if !(sessVars.IsAutocommit() && !sessVars.InTxn()) { return errors.Errorf("non-transactional DML can only run in auto-commit mode. auto-commit:%v, inTxn:%v", @@ -256,7 +257,7 @@ func checkReadClauses(limit *ast.Limit, order *ast.OrderByClause) error { // single-threaded worker. work on the key range [start, end] func runJobs(ctx context.Context, jobs []job, stmt *ast.NonTransactionalDMLStmt, - tableName *ast.TableName, se Session, originalCondition ast.ExprNode) ([]string, error) { + tableName *ast.TableName, se sessiontypes.Session, originalCondition ast.ExprNode) ([]string, error) { // prepare for the construction of statement var shardColumnRefer *ast.ResultField var shardColumnType types.FieldType @@ -331,7 +332,7 @@ func runJobs(ctx context.Context, jobs []job, stmt *ast.NonTransactionalDMLStmt, return splitStmts, nil } -func doOneJob(ctx context.Context, job *job, totalJobCount int, options statementBuildInfo, se Session, dryRun bool) string { +func doOneJob(ctx context.Context, job *job, totalJobCount int, options statementBuildInfo, se sessiontypes.Session, dryRun bool) string { var whereCondition ast.ExprNode if job.start.IsNull() { @@ -441,7 +442,7 @@ func doOneJob(ctx context.Context, job *job, totalJobCount int, options statemen return "" } -func buildShardJobs(ctx context.Context, stmt *ast.NonTransactionalDMLStmt, se Session, +func buildShardJobs(ctx context.Context, stmt *ast.NonTransactionalDMLStmt, se sessiontypes.Session, selectSQL string, shardColumnInfo *model.ColumnInfo, memTracker *memory.Tracker) ([]job, error) { var shardColumnCollate string if shardColumnInfo != nil { @@ -536,7 +537,7 @@ func appendNewJob(jobs []job, id int, start types.Datum, end types.Datum, size i return jobs } -func buildSelectSQL(stmt *ast.NonTransactionalDMLStmt, se Session) ( +func buildSelectSQL(stmt *ast.NonTransactionalDMLStmt, se sessiontypes.Session) ( *ast.TableName, string, *model.ColumnInfo, []*ast.TableSource, error) { // only use the first table join, ok := stmt.DMLStmt.TableRefsJoin() @@ -582,7 +583,7 @@ func buildSelectSQL(stmt *ast.NonTransactionalDMLStmt, se Session) ( return tableName, selectSQL, shardColumnInfo, tableSources, nil } -func selectShardColumn(stmt *ast.NonTransactionalDMLStmt, se Session, tableSources []*ast.TableSource, +func selectShardColumn(stmt *ast.NonTransactionalDMLStmt, se sessiontypes.Session, tableSources []*ast.TableSource, leftMostTableName *ast.TableName, leftMostTableSource *ast.TableSource) ( *model.ColumnInfo, *ast.TableName, error) { var indexed bool diff --git a/pkg/session/session.go b/pkg/session/session.go index 152a6706b76da..5fcac90030187 100644 --- a/pkg/session/session.go +++ b/pkg/session/session.go @@ -71,6 +71,7 @@ import ( "github.com/pingcap/tidb/pkg/privilege/privileges" session_metrics "github.com/pingcap/tidb/pkg/session/metrics" "github.com/pingcap/tidb/pkg/session/txninfo" + "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/binloginfo" "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" @@ -115,76 +116,18 @@ import ( "go.uber.org/zap" ) -// Session context, it is consistent with the lifecycle of a client connection. -type Session interface { - sessionctx.Context - Status() uint16 // Flag of current status, such as autocommit. - LastInsertID() uint64 // LastInsertID is the last inserted auto_increment ID. - LastMessage() string // LastMessage is the info message that may be generated by last command - AffectedRows() uint64 // Affected rows by latest executed stmt. - // Execute is deprecated, and only used by plugins. Use ExecuteStmt() instead. - Execute(context.Context, string) ([]sqlexec.RecordSet, error) // Execute a sql statement. - // ExecuteStmt executes a parsed statement. - ExecuteStmt(context.Context, ast.StmtNode) (sqlexec.RecordSet, error) - // Parse is deprecated, use ParseWithParams() instead. - Parse(ctx context.Context, sql string) ([]ast.StmtNode, error) - // ExecuteInternal is a helper around ParseWithParams() and ExecuteStmt(). It is not allowed to execute multiple statements. - ExecuteInternal(context.Context, string, ...interface{}) (sqlexec.RecordSet, error) - String() string // String is used to debug. - CommitTxn(context.Context) error - RollbackTxn(context.Context) - // PrepareStmt executes prepare statement in binary protocol. - PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*ast.ResultField, err error) - // ExecutePreparedStmt executes a prepared statement. - // Deprecated: please use ExecuteStmt, this function is left for testing only. - // TODO: remove ExecutePreparedStmt. - ExecutePreparedStmt(ctx context.Context, stmtID uint32, param []expression.Expression) (sqlexec.RecordSet, error) - DropPreparedStmt(stmtID uint32) error - // SetSessionStatesHandler sets SessionStatesHandler for type stateType. - SetSessionStatesHandler(stateType sessionstates.SessionStateType, handler sessionctx.SessionStatesHandler) - SetClientCapability(uint32) // Set client capability flags. - SetConnectionID(uint64) - SetCommandValue(byte) - SetCompressionAlgorithm(int) - SetCompressionLevel(int) - SetProcessInfo(string, time.Time, byte, uint64) - SetTLSState(*tls.ConnectionState) - SetCollation(coID int) error - SetSessionManager(util.SessionManager) - Close() - Auth(user *auth.UserIdentity, auth, salt []byte, authConn conn.AuthConn) error - AuthWithoutVerification(user *auth.UserIdentity) bool - AuthPluginForUser(user *auth.UserIdentity) (string, error) - MatchIdentity(username, remoteHost string) (*auth.UserIdentity, error) - // Return the information of the txn current running - TxnInfo() *txninfo.TxnInfo - // PrepareTxnCtx is exported for test. - PrepareTxnCtx(context.Context) error - // FieldList returns fields list of a table. - FieldList(tableName string) (fields []*ast.ResultField, err error) - SetPort(port string) - - // set cur session operations allowed when tikv disk full happens. - SetDiskFullOpt(level kvrpcpb.DiskFullOpt) - GetDiskFullOpt() kvrpcpb.DiskFullOpt - ClearDiskFullOpt() - - // SetExtensions sets the `*extension.SessionExtensions` object - SetExtensions(extensions *extension.SessionExtensions) -} - func init() { executor.CreateSession = func(ctx sessionctx.Context) (sessionctx.Context, error) { return CreateSession(ctx.GetStore()) } executor.CloseSession = func(ctx sessionctx.Context) { - if se, ok := ctx.(Session); ok { + if se, ok := ctx.(types.Session); ok { se.Close() } } } -var _ Session = (*session)(nil) +var _ types.Session = (*session)(nil) type stmtRecord struct { st sqlexec.Statement @@ -1889,7 +1832,7 @@ func (s *session) DisableSandBoxMode() { } // ParseWithParams4Test wrapper (s *session) ParseWithParams for test -func ParseWithParams4Test(ctx context.Context, s Session, +func ParseWithParams4Test(ctx context.Context, s types.Session, sql string, args ...interface{}) (ast.StmtNode, error) { return s.(*session).ParseWithParams(ctx, sql, args) } @@ -1945,7 +1888,7 @@ func (s *session) ExecRestrictedStmt(ctx context.Context, stmtNode ast.StmtNode, } // ExecRestrictedStmt4Test wrapper `(s *session) ExecRestrictedStmt` for test. -func ExecRestrictedStmt4Test(ctx context.Context, s Session, +func ExecRestrictedStmt4Test(ctx context.Context, s types.Session, stmtNode ast.StmtNode, opts ...sqlexec.OptionFuncAlias) ( []chunk.Row, []*ast.ResultField, error) { ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnOthers) @@ -3044,7 +2987,7 @@ func (s *session) SetSessionStatesHandler(stateType sessionstates.SessionStateTy } // CreateSession4Test creates a new session environment for test. -func CreateSession4Test(store kv.Storage) (Session, error) { +func CreateSession4Test(store kv.Storage) (types.Session, error) { se, err := CreateSession4TestWithOpt(store, nil) if err == nil { // Cover both chunk rpc encoding and default encoding. @@ -3064,7 +3007,7 @@ type Opt struct { } // CreateSession4TestWithOpt creates a new session environment for test. -func CreateSession4TestWithOpt(store kv.Storage, opt *Opt) (Session, error) { +func CreateSession4TestWithOpt(store kv.Storage, opt *Opt) (types.Session, error) { s, err := CreateSessionWithOpt(store, opt) if err == nil { // initialize session variables for test. @@ -3078,13 +3021,13 @@ func CreateSession4TestWithOpt(store kv.Storage, opt *Opt) (Session, error) { } // CreateSession creates a new session environment. -func CreateSession(store kv.Storage) (Session, error) { +func CreateSession(store kv.Storage) (types.Session, error) { return CreateSessionWithOpt(store, nil) } // CreateSessionWithOpt creates a new session environment with option. // Use default option if opt is nil. -func CreateSessionWithOpt(store kv.Storage, opt *Opt) (Session, error) { +func CreateSessionWithOpt(store kv.Storage, opt *Opt) (types.Session, error) { s, err := createSessionWithOpt(store, opt) if err != nil { return nil, err @@ -3560,7 +3503,7 @@ func GetDomain(store kv.Storage) (*domain.Domain, error) { // If no bootstrap and storage is remote, we must use a little lease time to // bootstrap quickly, after bootstrapped, we will reset the lease time. // TODO: Using a bootstrap tool for doing this may be better later. -func runInBootstrapSession(store kv.Storage, bootstrap func(Session)) { +func runInBootstrapSession(store kv.Storage, bootstrap func(types.Session)) { s, err := createSession(store) if err != nil { // Bootstrap fail will cause program exit. @@ -4417,7 +4360,7 @@ func (s *session) setRequestSource(ctx context.Context, stmtLabel string, stmtNo } // RemoveLockDDLJobs removes the DDL jobs which doesn't get the metadata lock from job2ver. -func RemoveLockDDLJobs(s Session, job2ver map[int64]int64, job2ids map[int64]string, printLog bool) { +func RemoveLockDDLJobs(s types.Session, job2ver map[int64]int64, job2ids map[int64]string, printLog bool) { sv := s.GetSessionVars() if sv.InRestrictedSQL { return diff --git a/pkg/session/sync_upgrade.go b/pkg/session/sync_upgrade.go index 384a505fae871..be20dfc3764c9 100644 --- a/pkg/session/sync_upgrade.go +++ b/pkg/session/sync_upgrade.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/syncer" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/owner" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/zap" @@ -119,7 +120,7 @@ func IsUpgradingClusterState(s sessionctx.Context) (bool, error) { return stateInfo.State == syncer.StateUpgrading, nil } -func printClusterState(s Session, ver int64) { +func printClusterState(s sessiontypes.Session, ver int64) { // After SupportUpgradeHTTPOpVer version, the upgrade by paused user DDL can be notified through the HTTP API. // We check the global state see if we are upgrading by paused the user DDL. if ver >= SupportUpgradeHTTPOpVer { diff --git a/pkg/session/testutil.go b/pkg/session/testutil.go index f73533a508454..78eca23d1a63e 100644 --- a/pkg/session/testutil.go +++ b/pkg/session/testutil.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit/testenv" "github.com/pingcap/tidb/pkg/util/sqlexec" @@ -50,7 +51,7 @@ func CreateStoreAndBootstrap(t *testing.T) (kv.Storage, *domain.Domain) { var sessionKitIDGenerator atomicutil.Uint64 // CreateSessionAndSetID creates a session and set connection ID. -func CreateSessionAndSetID(t *testing.T, store kv.Storage) Session { +func CreateSessionAndSetID(t *testing.T, store kv.Storage) sessiontypes.Session { se, err := CreateSession4Test(store) se.SetConnectionID(sessionKitIDGenerator.Inc()) require.NoError(t, err) @@ -58,7 +59,7 @@ func CreateSessionAndSetID(t *testing.T, store kv.Storage) Session { } // MustExec executes a sql statement and asserts no error occurs. -func MustExec(t *testing.T, se Session, sql string, args ...interface{}) { +func MustExec(t *testing.T, se sessiontypes.Session, sql string, args ...interface{}) { rs, err := exec(se, sql, args...) require.NoError(t, err) if rs != nil { @@ -67,13 +68,13 @@ func MustExec(t *testing.T, se Session, sql string, args ...interface{}) { } // MustExecToRecodeSet executes a sql statement and asserts no error occurs. -func MustExecToRecodeSet(t *testing.T, se Session, sql string, args ...interface{}) sqlexec.RecordSet { +func MustExecToRecodeSet(t *testing.T, se sessiontypes.Session, sql string, args ...interface{}) sqlexec.RecordSet { rs, err := exec(se, sql, args...) require.NoError(t, err) return rs } -func exec(se Session, sql string, args ...interface{}) (sqlexec.RecordSet, error) { +func exec(se sessiontypes.Session, sql string, args ...interface{}) (sqlexec.RecordSet, error) { ctx := context.Background() if len(args) == 0 { rs, err := se.Execute(ctx, sql) diff --git a/pkg/session/tidb.go b/pkg/session/tidb.go index 16e3a2f423678..d4beca68160c5 100644 --- a/pkg/session/tidb.go +++ b/pkg/session/tidb.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" session_metrics "github.com/pingcap/tidb/pkg/session/metrics" + "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/sessiontxn" @@ -376,7 +377,7 @@ func GetRows4Test(ctx context.Context, _ sessionctx.Context, rs sqlexec.RecordSe } // ResultSetToStringSlice changes the RecordSet to [][]string. -func ResultSetToStringSlice(ctx context.Context, s Session, rs sqlexec.RecordSet) ([][]string, error) { +func ResultSetToStringSlice(ctx context.Context, s types.Session, rs sqlexec.RecordSet) ([][]string, error) { rows, err := GetRows4Test(ctx, s, rs) if err != nil { return nil, err diff --git a/pkg/session/types/BUILD.bazel b/pkg/session/types/BUILD.bazel new file mode 100644 index 0000000000000..c7a5e83dcb3c4 --- /dev/null +++ b/pkg/session/types/BUILD.bazel @@ -0,0 +1,21 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "types", + srcs = ["sesson_interface.go"], + importpath = "github.com/pingcap/tidb/pkg/session/types", + visibility = ["//visibility:public"], + deps = [ + "//pkg/expression", + "//pkg/extension", + "//pkg/parser/ast", + "//pkg/parser/auth", + "//pkg/privilege/conn", + "//pkg/session/txninfo", + "//pkg/sessionctx", + "//pkg/sessionctx/sessionstates", + "//pkg/util", + "//pkg/util/sqlexec", + "@com_github_pingcap_kvproto//pkg/kvrpcpb", + ], +) diff --git a/pkg/session/types/sesson_interface.go b/pkg/session/types/sesson_interface.go new file mode 100644 index 0000000000000..62b31b80709d0 --- /dev/null +++ b/pkg/session/types/sesson_interface.go @@ -0,0 +1,91 @@ +// Copyright 2023 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 types + +import ( + "context" + "crypto/tls" + "time" + + "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/extension" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/auth" + "github.com/pingcap/tidb/pkg/privilege/conn" + "github.com/pingcap/tidb/pkg/session/txninfo" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" + "github.com/pingcap/tidb/pkg/util" + "github.com/pingcap/tidb/pkg/util/sqlexec" +) + +// Session context, it is consistent with the lifecycle of a client connection. +type Session interface { + sessionctx.Context + Status() uint16 // Flag of current status, such as autocommit. + LastInsertID() uint64 // LastInsertID is the last inserted auto_increment ID. + LastMessage() string // LastMessage is the info message that may be generated by last command + AffectedRows() uint64 // Affected rows by latest executed stmt. + // Execute is deprecated, and only used by plugins. Use ExecuteStmt() instead. + Execute(context.Context, string) ([]sqlexec.RecordSet, error) // Execute a sql statement. + // ExecuteStmt executes a parsed statement. + ExecuteStmt(context.Context, ast.StmtNode) (sqlexec.RecordSet, error) + // Parse is deprecated, use ParseWithParams() instead. + Parse(ctx context.Context, sql string) ([]ast.StmtNode, error) + // ExecuteInternal is a helper around ParseWithParams() and ExecuteStmt(). It is not allowed to execute multiple statements. + ExecuteInternal(context.Context, string, ...interface{}) (sqlexec.RecordSet, error) + String() string // String is used to debug. + CommitTxn(context.Context) error + RollbackTxn(context.Context) + // PrepareStmt executes prepare statement in binary protocol. + PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*ast.ResultField, err error) + // ExecutePreparedStmt executes a prepared statement. + // Deprecated: please use ExecuteStmt, this function is left for testing only. + // TODO: remove ExecutePreparedStmt. + ExecutePreparedStmt(ctx context.Context, stmtID uint32, param []expression.Expression) (sqlexec.RecordSet, error) + DropPreparedStmt(stmtID uint32) error + // SetSessionStatesHandler sets SessionStatesHandler for type stateType. + SetSessionStatesHandler(stateType sessionstates.SessionStateType, handler sessionctx.SessionStatesHandler) + SetClientCapability(uint32) // Set client capability flags. + SetConnectionID(uint64) + SetCommandValue(byte) + SetCompressionAlgorithm(int) + SetCompressionLevel(int) + SetProcessInfo(string, time.Time, byte, uint64) + SetTLSState(*tls.ConnectionState) + SetCollation(coID int) error + SetSessionManager(util.SessionManager) + Close() + Auth(user *auth.UserIdentity, auth, salt []byte, authConn conn.AuthConn) error + AuthWithoutVerification(user *auth.UserIdentity) bool + AuthPluginForUser(user *auth.UserIdentity) (string, error) + MatchIdentity(username, remoteHost string) (*auth.UserIdentity, error) + // Return the information of the txn current running + TxnInfo() *txninfo.TxnInfo + // PrepareTxnCtx is exported for test. + PrepareTxnCtx(context.Context) error + // FieldList returns fields list of a table. + FieldList(tableName string) (fields []*ast.ResultField, err error) + SetPort(port string) + + // set cur session operations allowed when tikv disk full happens. + SetDiskFullOpt(level kvrpcpb.DiskFullOpt) + GetDiskFullOpt() kvrpcpb.DiskFullOpt + ClearDiskFullOpt() + + // SetExtensions sets the `*extension.SessionExtensions` object + SetExtensions(extensions *extension.SessionExtensions) +} diff --git a/pkg/store/gcworker/BUILD.bazel b/pkg/store/gcworker/BUILD.bazel index 5745182657b9f..25f5b2d2e7134 100644 --- a/pkg/store/gcworker/BUILD.bazel +++ b/pkg/store/gcworker/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/parser/terror", "//pkg/privilege", "//pkg/session", + "//pkg/session/types", "//pkg/sessionctx/variable", "//pkg/tablecodec", "//pkg/util/codec", diff --git a/pkg/store/gcworker/gc_worker.go b/pkg/store/gcworker/gc_worker.go index b090f082e6582..97ccd85bb1a92 100644 --- a/pkg/store/gcworker/gc_worker.go +++ b/pkg/store/gcworker/gc_worker.go @@ -45,6 +45,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/privilege" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/util/codec" @@ -232,7 +233,7 @@ func (w *GCWorker) start(ctx context.Context, wg *sync.WaitGroup) { } } -func createSession(store kv.Storage) session.Session { +func createSession(store kv.Storage) sessiontypes.Session { for { se, err := session.CreateSession(store) if err != nil { @@ -1874,7 +1875,7 @@ func (w *GCWorker) saveValueToSysTable(key, value string) error { // GC placement rules when the partitions are removed by the GC worker. // Placement rules cannot be removed immediately after drop table / truncate table, // because the tables can be flashed back or recovered. -func (w *GCWorker) doGCPlacementRules(se session.Session, safePoint uint64, dr util.DelRangeTask, gcPlacementRuleCache map[int64]interface{}) (err error) { +func (w *GCWorker) doGCPlacementRules(se sessiontypes.Session, safePoint uint64, dr util.DelRangeTask, gcPlacementRuleCache map[int64]interface{}) (err error) { // Get the job from the job history var historyJob *model.Job failpoint.Inject("mockHistoryJobForGC", func(v failpoint.Value) { diff --git a/pkg/table/tables/BUILD.bazel b/pkg/table/tables/BUILD.bazel index b10e03f4b7d4b..2b4d0f6524dd7 100644 --- a/pkg/table/tables/BUILD.bazel +++ b/pkg/table/tables/BUILD.bazel @@ -88,6 +88,7 @@ go_test( "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/session", + "//pkg/session/types", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", "//pkg/sessionctx/variable", diff --git a/pkg/table/tables/state_remote_test.go b/pkg/table/tables/state_remote_test.go index d8b6cde16ee26..898c5aba4392b 100644 --- a/pkg/table/tables/state_remote_test.go +++ b/pkg/table/tables/state_remote_test.go @@ -20,7 +20,7 @@ import ( "time" "github.com/pingcap/tidb/pkg/kv" - "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -28,7 +28,7 @@ import ( ) // initRow add a new record into the cached table meta lock table. -func initRow(ctx context.Context, exec session.Session, tid int) error { +func initRow(ctx context.Context, exec sessiontypes.Session, tid int) error { _, err := exec.ExecuteInternal(ctx, "insert ignore into mysql.table_cache_meta values (%?, 'NONE', 0, 0)", tid) return err } diff --git a/pkg/telemetry/cte_test/BUILD.bazel b/pkg/telemetry/cte_test/BUILD.bazel index 39590ecbd9dd8..0918caac0e410 100644 --- a/pkg/telemetry/cte_test/BUILD.bazel +++ b/pkg/telemetry/cte_test/BUILD.bazel @@ -10,6 +10,7 @@ go_test( "//pkg/domain", "//pkg/kv", "//pkg/session", + "//pkg/session/types", "//pkg/store/mockstore", "//pkg/testkit/testsetup", "@com_github_stretchr_testify//require", diff --git a/pkg/telemetry/cte_test/cte_test.go b/pkg/telemetry/cte_test/cte_test.go index 0f61a3485ac97..445223978b562 100644 --- a/pkg/telemetry/cte_test/cte_test.go +++ b/pkg/telemetry/cte_test/cte_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/testkit/testsetup" "github.com/stretchr/testify/require" @@ -95,7 +96,7 @@ type testSuite struct { store kv.Storage dom *domain.Domain etcdCluster *integration.ClusterV3 - se session.Session + se sessiontypes.Session close func() } diff --git a/pkg/testkit/BUILD.bazel b/pkg/testkit/BUILD.bazel index b9922d4065b45..13890b8f84a8c 100644 --- a/pkg/testkit/BUILD.bazel +++ b/pkg/testkit/BUILD.bazel @@ -26,6 +26,7 @@ go_library( "//pkg/resourcemanager", "//pkg/session", "//pkg/session/txninfo", + "//pkg/session/types", "//pkg/sessionctx", "//pkg/sessionctx/variable", "//pkg/store/driver", diff --git a/pkg/testkit/asynctestkit.go b/pkg/testkit/asynctestkit.go index 2867e4f256eb1..ad6cf72b91ef4 100644 --- a/pkg/testkit/asynctestkit.go +++ b/pkg/testkit/asynctestkit.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/util/sqlexec" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -240,10 +241,10 @@ type sessionCtxKeyType struct{} var sessionKey = sessionCtxKeyType{} // TryRetrieveSession tries retrieve session from context. -func TryRetrieveSession(ctx context.Context) session.Session { +func TryRetrieveSession(ctx context.Context) sessiontypes.Session { s := ctx.Value(sessionKey) if s == nil { return nil } - return s.(session.Session) + return s.(sessiontypes.Session) } diff --git a/pkg/testkit/mocksessionmanager.go b/pkg/testkit/mocksessionmanager.go index af1aa1022660a..88223f21e51d7 100644 --- a/pkg/testkit/mocksessionmanager.go +++ b/pkg/testkit/mocksessionmanager.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/planner/core" "github.com/pingcap/tidb/pkg/session" "github.com/pingcap/tidb/pkg/session/txninfo" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/util" ) @@ -35,7 +36,7 @@ type MockSessionManager struct { SerID uint64 TxnInfo []*txninfo.TxnInfo Dom *domain.Domain - Conn map[uint64]session.Session + Conn map[uint64]sessiontypes.Session mu sync.Mutex ConAttrs map[uint64]map[string]string diff --git a/pkg/testkit/testkit.go b/pkg/testkit/testkit.go index 07949e3b87b04..dca6e0de84b43 100644 --- a/pkg/testkit/testkit.go +++ b/pkg/testkit/testkit.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/terror" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit/testenv" "github.com/pingcap/tidb/pkg/types" @@ -53,7 +54,7 @@ type TestKit struct { assert *assert.Assertions t testing.TB store kv.Storage - session session.Session + session sessiontypes.Session alloc chunk.Allocator } @@ -77,7 +78,7 @@ func NewTestKit(t testing.TB, store kv.Storage) *TestKit { if ok { mockSm.mu.Lock() if mockSm.Conn == nil { - mockSm.Conn = make(map[uint64]session.Session) + mockSm.Conn = make(map[uint64]sessiontypes.Session) } mockSm.Conn[tk.session.GetSessionVars().ConnectionID] = tk.session mockSm.mu.Unlock() @@ -89,7 +90,7 @@ func NewTestKit(t testing.TB, store kv.Storage) *TestKit { } // NewTestKitWithSession returns a new *TestKit. -func NewTestKitWithSession(t testing.TB, store kv.Storage, se session.Session) *TestKit { +func NewTestKitWithSession(t testing.TB, store kv.Storage, se sessiontypes.Session) *TestKit { return &TestKit{ require: require.New(t), assert: assert.New(t), @@ -108,14 +109,14 @@ func (tk *TestKit) RefreshSession() { } // SetSession set the session of testkit -func (tk *TestKit) SetSession(session session.Session) { +func (tk *TestKit) SetSession(session sessiontypes.Session) { tk.session = session // enforce sysvar cache loading, ref loadCommonGlobalVariableIfNeeded tk.MustExec("select 3") } // Session return the session associated with the testkit -func (tk *TestKit) Session() session.Session { +func (tk *TestKit) Session() sessiontypes.Session { return tk.session } @@ -422,7 +423,7 @@ func (tk *TestKit) MustExecToErr(sql string, args ...interface{}) { tk.require.Error(err) } -func newSession(t testing.TB, store kv.Storage) session.Session { +func newSession(t testing.TB, store kv.Storage) sessiontypes.Session { se, err := session.CreateSession4Test(store) require.NoError(t, err) se.SetConnectionID(testKitIDGenerator.Inc()) diff --git a/pkg/util/ddl-checker/BUILD.bazel b/pkg/util/ddl-checker/BUILD.bazel index c8ee1ac53d0b2..96d5884489418 100644 --- a/pkg/util/ddl-checker/BUILD.bazel +++ b/pkg/util/ddl-checker/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//pkg/parser", "//pkg/parser/ast", "//pkg/session", + "//pkg/session/types", "//pkg/store/mockstore", "//pkg/util/dbutil", "//pkg/util/logutil", diff --git a/pkg/util/ddl-checker/executable_checker.go b/pkg/util/ddl-checker/executable_checker.go index a448c6b8fa787..15230d9bcefd7 100644 --- a/pkg/util/ddl-checker/executable_checker.go +++ b/pkg/util/ddl-checker/executable_checker.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/parser" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/session" + sessiontypes "github.com/pingcap/tidb/pkg/session/types" "github.com/pingcap/tidb/pkg/store/mockstore" "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/atomic" @@ -30,7 +31,7 @@ import ( // ExecutableChecker is a part of TiDB to check the sql's executability type ExecutableChecker struct { - session session.Session + session sessiontypes.Session parser *parser.Parser isClosed *atomic.Bool } From 67b70e52072e8001773ac389379718bb9c60624e Mon Sep 17 00:00:00 2001 From: Jianjun Liao <36503113+Leavrth@users.noreply.github.com> Date: Thu, 23 Nov 2023 14:11:41 +0800 Subject: [PATCH 21/36] br: record size in backupmeta (#48772) close pingcap/tidb#48745 --- DEPS.bzl | 12 ++++++------ br/pkg/metautil/metafile.go | 13 +++++++++++++ go.mod | 2 +- go.sum | 4 ++-- 4 files changed, 22 insertions(+), 9 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index d0adcb9d90879..560e71190dcc2 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -5923,13 +5923,13 @@ def go_deps(): name = "com_github_pingcap_kvproto", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/kvproto", - sha256 = "89c00318dfdd036a7121fe0510cf78d3c8a288e34f23e70418b5c0caa7f3b6df", - strip_prefix = "github.com/pingcap/kvproto@v0.0.0-20231116020303-ecf635d1a67b", + sha256 = "457e1273f6c608e8276f6904d7db6c25d8480621e00814a4c911edaffb9041fe", + strip_prefix = "github.com/pingcap/kvproto@v0.0.0-20231122054644-fb0f5c2a0a10", urls = [ - "http://bazel-cache.pingcap.net:8080/gomod/github.com/pingcap/kvproto/com_github_pingcap_kvproto-v0.0.0-20231116020303-ecf635d1a67b.zip", - "http://ats.apps.svc/gomod/github.com/pingcap/kvproto/com_github_pingcap_kvproto-v0.0.0-20231116020303-ecf635d1a67b.zip", - "https://cache.hawkingrei.com/gomod/github.com/pingcap/kvproto/com_github_pingcap_kvproto-v0.0.0-20231116020303-ecf635d1a67b.zip", - "https://storage.googleapis.com/pingcapmirror/gomod/github.com/pingcap/kvproto/com_github_pingcap_kvproto-v0.0.0-20231116020303-ecf635d1a67b.zip", + "http://bazel-cache.pingcap.net:8080/gomod/github.com/pingcap/kvproto/com_github_pingcap_kvproto-v0.0.0-20231122054644-fb0f5c2a0a10.zip", + "http://ats.apps.svc/gomod/github.com/pingcap/kvproto/com_github_pingcap_kvproto-v0.0.0-20231122054644-fb0f5c2a0a10.zip", + "https://cache.hawkingrei.com/gomod/github.com/pingcap/kvproto/com_github_pingcap_kvproto-v0.0.0-20231122054644-fb0f5c2a0a10.zip", + "https://storage.googleapis.com/pingcapmirror/gomod/github.com/pingcap/kvproto/com_github_pingcap_kvproto-v0.0.0-20231122054644-fb0f5c2a0a10.zip", ], ) go_repository( diff --git a/br/pkg/metautil/metafile.go b/br/pkg/metautil/metafile.go index 2521677167d26..b915da8fa10d6 100644 --- a/br/pkg/metautil/metafile.go +++ b/br/pkg/metautil/metafile.go @@ -545,6 +545,9 @@ type MetaWriter struct { // records the total datafile size totalDataFileSize int + + // records the total metafile size for backupmeta v2 + totalMetaFileSize int } // NewMetaWriter creates MetaWriter. @@ -682,6 +685,9 @@ func (writer *MetaWriter) FlushBackupMeta(ctx context.Context) error { writer.backupMeta.Version = MetaV1 } + // update the total size of backup files (include data files and meta files) + writer.backupMeta.BackupSize = writer.MetaFilesSize() + writer.ArchiveSize() + uint64(writer.backupMeta.Size()) + // Flush the writer.backupMeta to storage backupMetaData, err := proto.Marshal(writer.backupMeta) if err != nil { @@ -762,6 +768,7 @@ func (writer *MetaWriter) flushMetasV2(ctx context.Context, op AppendOp) error { return errors.Trace(err) } + writer.totalMetaFileSize += len(encyptedContent) if err = writer.storage.WriteFile(ctx, fname, encyptedContent); err != nil { return errors.Trace(err) } @@ -789,6 +796,12 @@ func (writer *MetaWriter) ArchiveSize() uint64 { return total } +// MetaFilesSize represents the size of meta files from backupmeta v2, +// must be called after everything finishes by `FinishWriteMetas`. +func (writer *MetaWriter) MetaFilesSize() uint64 { + return uint64(writer.totalMetaFileSize) +} + // Backupmeta clones a backupmeta. func (writer *MetaWriter) Backupmeta() *backuppb.BackupMeta { clone := proto.Clone(writer.backupMeta) diff --git a/go.mod b/go.mod index c9942e6ef235e..b5ec3649aa102 100644 --- a/go.mod +++ b/go.mod @@ -81,7 +81,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c github.com/pingcap/fn v1.0.0 - github.com/pingcap/kvproto v0.0.0-20231116020303-ecf635d1a67b + github.com/pingcap/kvproto v0.0.0-20231122054644-fb0f5c2a0a10 github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 github.com/pingcap/tidb/pkg/parser v0.0.0-20211011031125-9b13dc409c5e diff --git a/go.sum b/go.sum index 211a12881ffa8..c34867ec842bd 100644 --- a/go.sum +++ b/go.sum @@ -813,8 +813,8 @@ github.com/pingcap/fn v1.0.0/go.mod h1:u9WZ1ZiOD1RpNhcI42RucFh/lBuzTu6rw88a+oF2Z github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= 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-20231116020303-ecf635d1a67b h1:f2ZC4PxZ+fbZKPH1xaGBiCZmyblEQGojyoe3rBgJe90= -github.com/pingcap/kvproto v0.0.0-20231116020303-ecf635d1a67b/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= +github.com/pingcap/kvproto v0.0.0-20231122054644-fb0f5c2a0a10 h1:qnhfzwdWOy8oOSZYX7/aK9XKDs4hJ6P/Gg+s7Sr9VKY= +github.com/pingcap/kvproto v0.0.0-20231122054644-fb0f5c2a0a10/go.mod h1:rXxWk2UnwfUhLXha1jxRWPADw9eMZGWEWCg92Tgmb/8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/log v1.1.1-0.20230317032135-a0d097d16e22 h1:2SOzvGvE8beiC1Y4g9Onkvu6UmuBBOeWRGQEjJaT/JY= From d41ee80730c13513be248f018ddc99352b40180e Mon Sep 17 00:00:00 2001 From: wuhuizuo Date: Thu, 23 Nov 2023 14:20:42 +0800 Subject: [PATCH 22/36] parser: update `OWNERS` file for `pkg/parser` folder (#48749) close pingcap/tidb#48748 --- OWNERS_ALIASES | 6 ++++++ pkg/parser/OWNERS | 5 ++++- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/OWNERS_ALIASES b/OWNERS_ALIASES index dce8bff5562d3..c679c70f9c91d 100644 --- a/OWNERS_ALIASES +++ b/OWNERS_ALIASES @@ -1,3 +1,4 @@ +# Sort the member alphabetically. aliases: sig-critical-approvers-tidb-server: - yudongusa @@ -89,6 +90,11 @@ aliases: - wjhuang2016 - ywqzzy - zimulala + sig-approvers-parser: # approvers for `parser` module. + - bb7133 + - BornChanger + - D3Hunter + - tangenta sig-approvers-resourcemanager: # approvers for resourcemanager pkg - Benjamin2037 - D3Hunter diff --git a/pkg/parser/OWNERS b/pkg/parser/OWNERS index ce80dfee9729d..c3abf5ba6cc7c 100644 --- a/pkg/parser/OWNERS +++ b/pkg/parser/OWNERS @@ -2,6 +2,9 @@ options: no_parent_owners: true filters: - "(OWNERS|parser\\.y)$": + "(parser\\.y)$": approvers: - sig-critical-approvers-parser + ".*": + approvers: + - sig-approvers-parser From 3159e1d5c3c95a0d90e1345768a6256644c81506 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20van=20Eeden?= Date: Thu, 23 Nov 2023 08:26:42 +0100 Subject: [PATCH 23/36] enterprise: Update submodule (#48732) --- pkg/extension/enterprise | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/extension/enterprise b/pkg/extension/enterprise index eac31cedd37f7..227c9935e1a76 160000 --- a/pkg/extension/enterprise +++ b/pkg/extension/enterprise @@ -1 +1 @@ -Subproject commit eac31cedd37f7143483f4b387c38fc2e8638b379 +Subproject commit 227c9935e1a76a4574a5321a58d19c2bff5f38f2 From 5b8d3deef690ca64e6313cfd926a8ceb7bb4cf78 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 23 Nov 2023 15:26:49 +0800 Subject: [PATCH 24/36] statistics: add daily bench and improve bench (#48824) close pingcap/tidb#48699 --- Makefile | 1 + pkg/statistics/BUILD.bazel | 4 +++- pkg/statistics/bench_daily_test.go | 27 +++++++++++++++++++++++++++ pkg/statistics/builder_test.go | 5 +++-- 4 files changed, 34 insertions(+), 3 deletions(-) create mode 100644 pkg/statistics/bench_daily_test.go diff --git a/Makefile b/Makefile index bdf498162b17c..2d0a08f8b6ecf 100644 --- a/Makefile +++ b/Makefile @@ -301,6 +301,7 @@ bench-daily: go test -tags intest github.com/pingcap/tidb/pkg/util/rowcodec -run TestBenchDaily -bench Ignore --outfile bench_daily.json go test -tags intest github.com/pingcap/tidb/pkg/util/codec -run TestBenchDaily -bench Ignore --outfile bench_daily.json go test -tags intest github.com/pingcap/tidb/pkg/distsql -run TestBenchDaily -bench Ignore --outfile bench_daily.json + go test -tags intest github.com/pingcap/tidb/pkg/statistics -run TestBenchDaily -bench Ignore --outfile bench_daily.json go test -tags intest github.com/pingcap/tidb/pkg/util/benchdaily -run TestBenchDaily -bench Ignore \ -date `git log -n1 --date=unix --pretty=format:%cd` \ -commit `git log -n1 --pretty=format:%h` \ diff --git a/pkg/statistics/BUILD.bazel b/pkg/statistics/BUILD.bazel index 43903c74807da..d6a2cb58b6c10 100644 --- a/pkg/statistics/BUILD.bazel +++ b/pkg/statistics/BUILD.bazel @@ -62,6 +62,7 @@ go_test( name = "statistics_test", timeout = "short", srcs = [ + "bench_daily_test.go", "builder_test.go", "cmsketch_test.go", "fmsketch_test.go", @@ -76,7 +77,7 @@ go_test( data = glob(["testdata/**"]), embed = [":statistics"], flaky = True, - shard_count = 33, + shard_count = 34, deps = [ "//pkg/config", "//pkg/parser/ast", @@ -90,6 +91,7 @@ go_test( "//pkg/testkit/testmain", "//pkg/testkit/testsetup", "//pkg/types", + "//pkg/util/benchdaily", "//pkg/util/chunk", "//pkg/util/codec", "//pkg/util/collate", diff --git a/pkg/statistics/bench_daily_test.go b/pkg/statistics/bench_daily_test.go new file mode 100644 index 0000000000000..a7e8474b1549b --- /dev/null +++ b/pkg/statistics/bench_daily_test.go @@ -0,0 +1,27 @@ +// Copyright 2023 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 statistics + +import ( + "testing" + + "github.com/pingcap/tidb/pkg/util/benchdaily" +) + +func TestBenchDaily(*testing.T) { + benchdaily.Run( + BenchmarkBuildHistAndTopN, + ) +} diff --git a/pkg/statistics/builder_test.go b/pkg/statistics/builder_test.go index 8e3256f5ffd33..ed39d8feffd64 100644 --- a/pkg/statistics/builder_test.go +++ b/pkg/statistics/builder_test.go @@ -28,9 +28,10 @@ import ( // go test -benchmem -run=^$ -bench ^BenchmarkBuildHistAndTopN$ github.com/pingcap/tidb/pkg/statistics func BenchmarkBuildHistAndTopN(b *testing.B) { ctx := mock.NewContext() - sketch := NewFMSketch(1000) + const cnt = 1000_000 + sketch := NewFMSketch(cnt) data := make([]*SampleItem, 0, 8) - for i := 1; i <= 1000; i++ { + for i := 1; i <= cnt; i++ { d := types.NewIntDatum(int64(i)) err := sketch.InsertValue(ctx.GetSessionVars().StmtCtx, d) require.NoError(b, err) From a85ba6fac979194044258f4759797f00238a2aaf Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 23 Nov 2023 15:53:42 +0800 Subject: [PATCH 25/36] bootstrap: split index merge bootstrap case (#48813) close pingcap/tidb#46834 --- pkg/session/bootstrap_test.go | 128 ++++++++++++++++++---------------- 1 file changed, 67 insertions(+), 61 deletions(-) diff --git a/pkg/session/bootstrap_test.go b/pkg/session/bootstrap_test.go index 61e87361f9c31..a5fb48a4e823f 100644 --- a/pkg/session/bootstrap_test.go +++ b/pkg/session/bootstrap_test.go @@ -852,71 +852,77 @@ func TestIndexMergeUpgradeFrom300To540(t *testing.T) { require.Equal(t, int64(0), row.GetInt64(0)) } -func TestIndexMergeUpgradeFrom400To540(t *testing.T) { - for i := 0; i < 2; i++ { - func() { - ctx := context.Background() - store, dom := CreateStoreAndBootstrap(t) - defer func() { require.NoError(t, store.Close()) }() +// We set tidb_enable_index_merge as on. +// And after upgrade to 5.x, tidb_enable_index_merge should remains to be on. +func TestIndexMergeUpgradeFrom400To540Enable(t *testing.T) { + testIndexMergeUpgradeFrom400To540(t, true) +} - // upgrade from 4.0.0 to 5.4+. - ver400 := 46 - seV4 := CreateSessionAndSetID(t, store) - txn, err := store.Begin() - require.NoError(t, err) - m := meta.NewMeta(txn) - err = m.FinishBootstrap(int64(ver400)) - require.NoError(t, err) - err = txn.Commit(context.Background()) - require.NoError(t, err) - MustExec(t, seV4, fmt.Sprintf("update mysql.tidb set variable_value=%d where variable_name='tidb_server_version'", ver400)) - MustExec(t, seV4, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", variable.Off, variable.TiDBEnableIndexMerge)) - MustExec(t, seV4, "commit") - unsetStoreBootstrapped(store.UUID()) - ver, err := getBootstrapVersion(seV4) - require.NoError(t, err) - require.Equal(t, int64(ver400), ver) +func TestIndexMergeUpgradeFrom400To540Disable(t *testing.T) { + testIndexMergeUpgradeFrom400To540(t, false) +} - // We are now in 4.0.0, tidb_enable_index_merge is off. - res := MustExecToRecodeSet(t, seV4, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableIndexMerge)) - 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, variable.Off, row.GetString(1)) +func testIndexMergeUpgradeFrom400To540(t *testing.T, enable bool) { + ctx := context.Background() + store, dom := CreateStoreAndBootstrap(t) + defer func() { require.NoError(t, store.Close()) }() - if i == 0 { - // For the first time, We set tidb_enable_index_merge as on. - // And after upgrade to 5.x, tidb_enable_index_merge should remains to be on. - // For the second it should be off. - MustExec(t, seV4, "set global tidb_enable_index_merge = on") - } - dom.Close() - // Upgrade to 5.x. - 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) + // upgrade from 4.0.0 to 5.4+. + ver400 := 46 + seV4 := CreateSessionAndSetID(t, store) + txn, err := store.Begin() + require.NoError(t, err) + m := meta.NewMeta(txn) + err = m.FinishBootstrap(int64(ver400)) + require.NoError(t, err) + err = txn.Commit(context.Background()) + require.NoError(t, err) + MustExec(t, seV4, fmt.Sprintf("update mysql.tidb set variable_value=%d where variable_name='tidb_server_version'", ver400)) + MustExec(t, seV4, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", variable.Off, variable.TiDBEnableIndexMerge)) + MustExec(t, seV4, "commit") + unsetStoreBootstrapped(store.UUID()) + ver, err := getBootstrapVersion(seV4) + require.NoError(t, err) + require.Equal(t, int64(ver400), ver) - // We are now in 5.x, tidb_enable_index_merge should be on because we enable it in 4.0.0. - res = MustExecToRecodeSet(t, seCurVer, "select @@tidb_enable_index_merge") - 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, 1, row.Len()) - if i == 0 { - require.Equal(t, int64(1), row.GetInt64(0)) - } else { - require.Equal(t, int64(0), row.GetInt64(0)) - } - }() + // We are now in 4.0.0, tidb_enable_index_merge is off. + res := MustExecToRecodeSet(t, seV4, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableIndexMerge)) + 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, variable.Off, row.GetString(1)) + + if enable { + // For the first time, We set tidb_enable_index_merge as on. + // And after upgrade to 5.x, tidb_enable_index_merge should remains to be on. + // For the second it should be off. + MustExec(t, seV4, "set global tidb_enable_index_merge = on") + } + dom.Close() + // Upgrade to 5.x. + 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 5.x, tidb_enable_index_merge should be on because we enable it in 4.0.0. + res = MustExecToRecodeSet(t, seCurVer, "select @@tidb_enable_index_merge") + 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, 1, row.Len()) + if enable { + require.Equal(t, int64(1), row.GetInt64(0)) + } else { + require.Equal(t, int64(0), row.GetInt64(0)) } } From 9cf638fb083461238a26392a559fb239718cc2bc Mon Sep 17 00:00:00 2001 From: EasonBall <592838129@qq.com> Date: Thu, 23 Nov 2023 16:46:12 +0800 Subject: [PATCH 26/36] global sort: merge to one file (#48142) ref pingcap/tidb#45719 --- br/pkg/lightning/backend/external/BUILD.bazel | 4 +- .../lightning/backend/external/bench_test.go | 37 +- br/pkg/lightning/backend/external/engine.go | 3 +- br/pkg/lightning/backend/external/iter.go | 2 - br/pkg/lightning/backend/external/merge.go | 113 +++++- .../lightning/backend/external/merge_test.go | 1 + .../backend/external/onefile_writer.go | 158 ++++++++ .../backend/external/onefile_writer_test.go | 359 ++++++++++++++++++ br/pkg/lightning/backend/external/util.go | 15 +- br/pkg/lightning/backend/external/writer.go | 43 ++- br/pkg/storage/s3.go | 1 + br/pkg/storage/storage.go | 1 + pkg/ddl/backfilling_dispatcher.go | 8 +- pkg/ddl/backfilling_dispatcher_test.go | 2 + pkg/ddl/backfilling_dist_scheduler.go | 2 +- pkg/ddl/backfilling_merge_sort.go | 27 +- pkg/ddl/backfilling_operators.go | 37 +- pkg/ddl/backfilling_read_index.go | 17 +- pkg/disttask/importinto/scheduler.go | 7 +- 19 files changed, 791 insertions(+), 46 deletions(-) create mode 100644 br/pkg/lightning/backend/external/onefile_writer.go create mode 100644 br/pkg/lightning/backend/external/onefile_writer_test.go diff --git a/br/pkg/lightning/backend/external/BUILD.bazel b/br/pkg/lightning/backend/external/BUILD.bazel index 93299ec3e0c6b..0280abc26c41a 100644 --- a/br/pkg/lightning/backend/external/BUILD.bazel +++ b/br/pkg/lightning/backend/external/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "iter.go", "kv_reader.go", "merge.go", + "onefile_writer.go", "split.go", "stat_reader.go", "util.go", @@ -57,13 +58,14 @@ go_test( "file_test.go", "iter_test.go", "merge_test.go", + "onefile_writer_test.go", "split_test.go", "util_test.go", "writer_test.go", ], embed = [":external"], flaky = True, - shard_count = 45, + shard_count = 49, deps = [ "//br/pkg/lightning/backend/kv", "//br/pkg/lightning/common", diff --git a/br/pkg/lightning/backend/external/bench_test.go b/br/pkg/lightning/backend/external/bench_test.go index 798a81eae8025..12378cd8707ee 100644 --- a/br/pkg/lightning/backend/external/bench_test.go +++ b/br/pkg/lightning/backend/external/bench_test.go @@ -166,9 +166,37 @@ func writeExternalFile(s *writeTestSuite) { } } +func writeExternalOneFile(s *writeTestSuite) { + ctx := context.Background() + builder := NewWriterBuilder(). + SetMemorySizeLimit(uint64(s.memoryLimit)) + + if s.beforeCreateWriter != nil { + s.beforeCreateWriter() + } + writer := builder.BuildOneFile( + s.store, "test/external", "writerID") + _ = writer.Init(ctx, 20*1024*1024) + key, val, _ := s.source.next() + for key != nil { + err := writer.WriteRow(ctx, key, val) + intest.AssertNoError(err) + key, val, _ = s.source.next() + } + if s.beforeWriterClose != nil { + s.beforeWriterClose() + } + err := writer.Close(ctx) + intest.AssertNoError(err) + if s.afterWriterClose != nil { + s.afterWriterClose() + } +} + func TestCompareWriter(t *testing.T) { store := openTestingStorage(t) - source := newAscendingKeySource(20, 100, 10000000) + sourceKVNum := 10000000 + source := newAscendingKeySource(20, 100, sourceKVNum) memoryLimit := 64 * 1024 * 1024 fileIdx := 0 var ( @@ -210,10 +238,15 @@ func TestCompareWriter(t *testing.T) { baseSpeed := float64(source.outputSize()) / elapsed.Seconds() / 1024 / 1024 t.Logf("base speed for %d bytes: %.2f MB/s", source.outputSize(), baseSpeed) - suite.source = newAscendingKeySource(20, 100, 10000000) + suite.source = newAscendingKeySource(20, 100, sourceKVNum) writeExternalFile(suite) writerSpeed := float64(source.outputSize()) / elapsed.Seconds() / 1024 / 1024 t.Logf("writer speed for %d bytes: %.2f MB/s", source.outputSize(), writerSpeed) + + suite.source = newAscendingKeySource(20, 100, sourceKVNum) + writeExternalOneFile(suite) + writerSpeed = float64(source.outputSize()) / elapsed.Seconds() / 1024 / 1024 + t.Logf("one file writer speed for %d bytes: %.2f MB/s", source.outputSize(), writerSpeed) } type readTestSuite struct { diff --git a/br/pkg/lightning/backend/external/engine.go b/br/pkg/lightning/backend/external/engine.go index 900bea084fd2a..4465534053917 100644 --- a/br/pkg/lightning/backend/external/engine.go +++ b/br/pkg/lightning/backend/external/engine.go @@ -168,7 +168,8 @@ func (e *Engine) LoadIngestData( zap.Int("concurrency", concurrency), zap.Int("ranges", len(regionRanges)), zap.Int("range-groups", len(rangeGroups)), - zap.Int("data-files", len(e.dataFiles)), + zap.Int("num-data-files", len(e.dataFiles)), + zap.Int("num-stat-files", len(e.statsFiles)), zap.Bool("check-hotspot", e.checkHotspot), ) eg, egCtx := util.NewErrorGroupWithRecoverWithCtx(ctx) diff --git a/br/pkg/lightning/backend/external/iter.go b/br/pkg/lightning/backend/external/iter.go index 768f7bd979a92..547d147132f1d 100644 --- a/br/pkg/lightning/backend/external/iter.go +++ b/br/pkg/lightning/backend/external/iter.go @@ -230,8 +230,6 @@ func (i *mergeIter[T, R]) currElem() T { // next forwards the iterator to the next element. It returns false if there is // no available element. func (i *mergeIter[T, R]) next() bool { - var zeroT T - i.curr = zeroT if i.lastReaderIdx >= 0 { if i.checkHotspot { i.hotspotMap[i.lastReaderIdx] = i.hotspotMap[i.lastReaderIdx] + 1 diff --git a/br/pkg/lightning/backend/external/merge.go b/br/pkg/lightning/backend/external/merge.go index dce4b4aa2959d..a2eb92c924733 100644 --- a/br/pkg/lightning/backend/external/merge.go +++ b/br/pkg/lightning/backend/external/merge.go @@ -2,20 +2,22 @@ package external import ( "context" + "errors" "github.com/google/uuid" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/storage" + tidbkv "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/pingcap/tidb/pkg/util/size" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) // MergeOverlappingFiles reads from given files whose key range may overlap // and writes to new sorted, nonoverlapping files. -func MergeOverlappingFiles(ctx context.Context, paths []string, store storage.ExternalStorage, readBufferSize int, +func MergeOverlappingFiles(ctx context.Context, paths []string, store storage.ExternalStorage, partSize int64, readBufferSize int, newFilePrefix string, blockSize int, writeBatchCount uint64, propSizeDist uint64, propKeysDist uint64, onClose OnCloseFunc, concurrency int, checkHotspot bool) error { var dataFilesSlice [][]string @@ -37,13 +39,15 @@ func MergeOverlappingFiles(ctx context.Context, paths []string, store storage.Ex zap.Int("concurrency", concurrency)) eg, egCtx := errgroup.WithContext(ctx) eg.SetLimit(concurrency) + partSize = max(int64(5*size.MB), partSize+int64(1*size.MB)) for _, files := range dataFilesSlice { files := files eg.Go(func() error { - return mergeOverlappingFilesImpl( + return MergeOverlappingFilesV2( egCtx, files, store, + partSize, readBufferSize, newFilePrefix, uuid.New().String(), @@ -60,6 +64,7 @@ func MergeOverlappingFiles(ctx context.Context, paths []string, store storage.Ex return eg.Wait() } +// unused for now. func mergeOverlappingFilesImpl(ctx context.Context, paths []string, store storage.ExternalStorage, @@ -81,6 +86,69 @@ func mergeOverlappingFilesImpl(ctx context.Context, defer func() { task.End(zap.ErrorLevel, err) }() + + zeroOffsets := make([]uint64, len(paths)) + iter, err := NewMergeKVIter(ctx, paths, zeroOffsets, store, readBufferSize, checkHotspot) + if err != nil { + return err + } + defer func() { + err := iter.Close() + if err != nil { + logutil.Logger(ctx).Warn("close iterator failed", zap.Error(err)) + } + }() + + writer := NewWriterBuilder(). + SetMemorySizeLimit(memSizeLimit). + SetBlockSize(blockSize). + SetOnCloseFunc(onClose). + SetWriterBatchCount(writeBatchCount). + SetPropSizeDistance(propSizeDist). + SetPropKeysDistance(propKeysDist). + Build(store, newFilePrefix, writerID) + + // currently use same goroutine to do read and write. The main advantage is + // there's no KV copy and iter can reuse the buffer. + for iter.Next() { + err = writer.WriteRow(ctx, iter.Key(), iter.Value(), nil) + if err != nil { + return err + } + } + err = iter.Error() + if err != nil { + return err + } + return writer.Close(ctx) +} + +// MergeOverlappingFilesV2 reads from given files whose key range may overlap +// and writes to one new sorted, nonoverlapping files. +func MergeOverlappingFilesV2( + ctx context.Context, + paths []string, + store storage.ExternalStorage, + partSize int64, + readBufferSize int, + newFilePrefix string, + writerID string, + memSizeLimit uint64, + blockSize int, + writeBatchCount uint64, + propSizeDist uint64, + propKeysDist uint64, + onClose OnCloseFunc, + checkHotspot bool, +) (err error) { + task := log.BeginTask(logutil.Logger(ctx).With( + zap.String("writer-id", writerID), + zap.Int("file-count", len(paths)), + ), "merge overlapping files") + defer func() { + task.End(zap.ErrorLevel, err) + }() + failpoint.Inject("mergeOverlappingFilesImpl", func(val failpoint.Value) { if val.(string) == paths[0] { failpoint.Return(errors.New("injected error")) @@ -107,16 +175,24 @@ func mergeOverlappingFilesImpl(ctx context.Context, writer := NewWriterBuilder(). SetMemorySizeLimit(memSizeLimit). SetBlockSize(blockSize). - SetOnCloseFunc(onClose). SetWriterBatchCount(writeBatchCount). - SetPropSizeDistance(propSizeDist). SetPropKeysDistance(propKeysDist). - Build(store, newFilePrefix, writerID) + SetPropSizeDistance(propSizeDist). + SetOnCloseFunc(onClose). + BuildOneFile(store, newFilePrefix, writerID) + err = writer.Init(ctx, partSize) + if err != nil { + return nil + } + var minKey, maxKey tidbkv.Key // currently use same goroutine to do read and write. The main advantage is // there's no KV copy and iter can reuse the buffer. for iter.Next() { - err = writer.WriteRow(ctx, iter.Key(), iter.Value(), nil) + if len(minKey) == 0 { + minKey = tidbkv.Key(iter.Key()).Clone() + } + err = writer.WriteRow(ctx, iter.Key(), iter.Value()) if err != nil { return err } @@ -125,5 +201,26 @@ func mergeOverlappingFilesImpl(ctx context.Context, if err != nil { return err } - return writer.Close(ctx) + maxKey = tidbkv.Key(iter.Key()).Clone() + + var stat MultipleFilesStat + stat.Filenames = append(stat.Filenames, + [2]string{writer.dataFile, writer.statFile}) + stat.build([]tidbkv.Key{minKey}, []tidbkv.Key{maxKey}) + if onClose != nil { + onClose(&WriterSummary{ + WriterID: writer.writerID, + Seq: 0, + Min: minKey, + Max: maxKey, + TotalSize: writer.totalSize, + MultipleFilesStats: []MultipleFilesStat{stat}, + }) + } + + err = writer.Close(ctx) + if err != nil { + return err + } + return nil } diff --git a/br/pkg/lightning/backend/external/merge_test.go b/br/pkg/lightning/backend/external/merge_test.go index 42078e75ce97b..799121c4fc7fb 100644 --- a/br/pkg/lightning/backend/external/merge_test.go +++ b/br/pkg/lightning/backend/external/merge_test.go @@ -32,6 +32,7 @@ func TestMergeOverlappingFiles(t *testing.T) { []string{"a", "b", "c", "d", "e"}, nil, 1, + 1, "", 1, 1, diff --git a/br/pkg/lightning/backend/external/onefile_writer.go b/br/pkg/lightning/backend/external/onefile_writer.go new file mode 100644 index 0000000000000..b0dea86661744 --- /dev/null +++ b/br/pkg/lightning/backend/external/onefile_writer.go @@ -0,0 +1,158 @@ +// Copyright 2023 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 external + +import ( + "context" + "encoding/binary" + "path/filepath" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/br/pkg/membuf" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/pingcap/tidb/pkg/util/size" + "go.uber.org/zap" +) + +// OneFileWriter is used to write data into external storage +// with only one file for data and stat. +type OneFileWriter struct { + // storage related. + store storage.ExternalStorage + kvStore *KeyValueStore + kvBuffer *membuf.Buffer + + // Statistic information per writer. + totalSize uint64 + rc *rangePropertiesCollector + + // file information. + writerID string + filenamePrefix string + dataFile string + statFile string + dataWriter storage.ExternalFileWriter + statWriter storage.ExternalFileWriter + + onClose OnCloseFunc + closed bool + + logger *zap.Logger +} + +// initWriter inits the underlying dataFile/statFile path, dataWriter/statWriter for OneFileWriter. +func (w *OneFileWriter) initWriter(ctx context.Context, partSize int64) ( + err error, +) { + w.dataFile = filepath.Join(w.filenamePrefix, "one-file") + w.dataWriter, err = w.store.Create(ctx, w.dataFile, &storage.WriterOption{Concurrency: 20, PartSize: partSize}) + if err != nil { + return err + } + w.statFile = filepath.Join(w.filenamePrefix+statSuffix, "one-file") + w.statWriter, err = w.store.Create(ctx, w.statFile, &storage.WriterOption{Concurrency: 20, PartSize: int64(5 * size.MB)}) + if err != nil { + _ = w.dataWriter.Close(ctx) + return err + } + w.logger.Info("one file writer", zap.String("data-file", w.dataFile), zap.String("stat-file", w.statFile)) + return nil +} + +// Init inits the OneFileWriter and its underlying KeyValueStore. +func (w *OneFileWriter) Init(ctx context.Context, partSize int64) (err error) { + w.logger = logutil.Logger(ctx) + err = w.initWriter(ctx, partSize) + if err != nil { + return err + } + w.kvStore, err = NewKeyValueStore(ctx, w.dataWriter, w.rc) + return err +} + +// WriteRow implements ingest.Writer. +func (w *OneFileWriter) WriteRow(ctx context.Context, idxKey, idxVal []byte) error { + // 1. encode data and write to kvStore. + keyLen := len(idxKey) + length := len(idxKey) + len(idxVal) + lengthBytes*2 + buf, _ := w.kvBuffer.AllocBytesWithSliceLocation(length) + if buf == nil { + w.kvBuffer.Reset() + buf, _ = w.kvBuffer.AllocBytesWithSliceLocation(length) + // we now don't support KV larger than blockSize + if buf == nil { + return errors.Errorf("failed to allocate kv buffer: %d", length) + } + // 2. write statistics if one kvBuffer is used. + w.kvStore.Close() + encodedStat := w.rc.encode() + _, err := w.statWriter.Write(ctx, encodedStat) + if err != nil { + return err + } + w.rc.reset() + } + binary.BigEndian.AppendUint64(buf[:0], uint64(keyLen)) + copy(buf[lengthBytes:], idxKey) + binary.BigEndian.AppendUint64(buf[lengthBytes+keyLen:lengthBytes+keyLen], uint64(len(idxVal))) + copy(buf[lengthBytes*2+keyLen:], idxVal) + w.kvStore.addEncodedData(buf[:length]) + w.totalSize += uint64(keyLen + len(idxVal)) + return nil +} + +// Close closes the writer. +func (w *OneFileWriter) Close(ctx context.Context) error { + if w.closed { + return errors.Errorf("writer %s has been closed", w.writerID) + } + err := w.closeImpl(ctx) + if err != nil { + return err + } + w.logger.Info("close one file writer", + zap.String("writerID", w.writerID)) + + w.totalSize = 0 + w.closed = true + return nil +} + +func (w *OneFileWriter) closeImpl(ctx context.Context) (err error) { + // 1. write remaining statistic. + w.kvStore.Close() + encodedStat := w.rc.encode() + _, err = w.statWriter.Write(ctx, encodedStat) + if err != nil { + return err + } + w.rc.reset() + // 2. close data writer. + err1 := w.dataWriter.Close(ctx) + if err1 != nil { + w.logger.Error("Close data writer failed", zap.Error(err)) + err = err1 + return + } + // 4. close stat writer. + err2 := w.statWriter.Close(ctx) + if err2 != nil { + w.logger.Error("Close stat writer failed", zap.Error(err)) + err = err2 + return + } + return nil +} diff --git a/br/pkg/lightning/backend/external/onefile_writer_test.go b/br/pkg/lightning/backend/external/onefile_writer_test.go new file mode 100644 index 0000000000000..be1bb8a717ad6 --- /dev/null +++ b/br/pkg/lightning/backend/external/onefile_writer_test.go @@ -0,0 +1,359 @@ +// Copyright 2023 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 external + +import ( + "bytes" + "context" + "fmt" + "io" + "path" + "slices" + "strconv" + "testing" + "time" + + "github.com/cockroachdb/pebble" + "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/pingcap/tidb/br/pkg/membuf" + "github.com/pingcap/tidb/br/pkg/storage" + dbkv "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/util/size" + "github.com/stretchr/testify/require" + "golang.org/x/exp/rand" +) + +func TestOnefileWriterBasic(t *testing.T) { + seed := time.Now().Unix() + rand.Seed(uint64(seed)) + t.Logf("seed: %d", seed) + ctx := context.Background() + memStore := storage.NewMemStorage() + + // 1. write into one file. + // 2. read kv file and check result. + // 3. read stat file and check result. + writer := NewWriterBuilder(). + SetPropSizeDistance(100). + SetPropKeysDistance(2). + BuildOneFile(memStore, "/test", "0") + + err := writer.Init(ctx, 5*1024*1024) + require.NoError(t, err) + + kvCnt := 100 + kvs := make([]common.KvPair, kvCnt) + for i := 0; i < kvCnt; i++ { + randLen := rand.Intn(10) + 1 + kvs[i].Key = make([]byte, randLen) + _, err := rand.Read(kvs[i].Key) + require.NoError(t, err) + randLen = rand.Intn(10) + 1 + kvs[i].Val = make([]byte, randLen) + _, err = rand.Read(kvs[i].Val) + require.NoError(t, err) + } + + for _, item := range kvs { + err := writer.WriteRow(ctx, item.Key, item.Val) + require.NoError(t, err) + } + + err = writer.Close(ctx) + require.NoError(t, err) + + bufSize := rand.Intn(100) + 1 + kvReader, err := newKVReader(ctx, "/test/0/one-file", memStore, 0, bufSize) + require.NoError(t, err) + for i := 0; i < kvCnt; i++ { + key, value, err := kvReader.nextKV() + require.NoError(t, err) + require.Equal(t, kvs[i].Key, key) + require.Equal(t, kvs[i].Val, value) + } + _, _, err = kvReader.nextKV() + require.Equal(t, io.EOF, err) + require.NoError(t, kvReader.Close()) + + statReader, err := newStatsReader(ctx, memStore, "/test/0_stat/one-file", bufSize) + require.NoError(t, err) + + var keyCnt uint64 = 0 + for { + p, err := statReader.nextProp() + if err == io.EOF { + break + } + require.NoError(t, err) + keyCnt += p.keys + } + require.Equal(t, uint64(kvCnt), keyCnt) + require.NoError(t, statReader.Close()) +} + +func TestOnefileWriterStat(t *testing.T) { + distanceCntArr := []uint64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10} + kvCntArr := []int{10, 100, 200, 1000} // won't large than DefaultMemSizeLimit. + // 1. write into one file. + // 2. read kv file and check result. + // 3. read stat file and check result. + for _, kvCnt := range kvCntArr { + for _, distance := range distanceCntArr { + checkOneFileWriterStatWithDistance(t, kvCnt, distance, DefaultMemSizeLimit, "test"+strconv.Itoa(int(distance))) + } + } +} + +func checkOneFileWriterStatWithDistance(t *testing.T, kvCnt int, keysDistance uint64, memSizeLimit uint64, prefix string) { + ctx := context.Background() + memStore := storage.NewMemStorage() + writer := NewWriterBuilder(). + SetPropSizeDistance(100). + SetPropKeysDistance(keysDistance). + BuildOneFile(memStore, "/"+prefix, "0") + + err := writer.Init(ctx, 5*1024*1024) + require.NoError(t, err) + kvs := make([]common.KvPair, 0, kvCnt) + for i := 0; i < kvCnt; i++ { + kvs = append(kvs, common.KvPair{ + Key: []byte(fmt.Sprintf("key%02d", i)), + Val: []byte("56789"), + }) + } + for _, item := range kvs { + err := writer.WriteRow(ctx, item.Key, item.Val) + require.NoError(t, err) + } + err = writer.Close(ctx) + require.NoError(t, err) + + bufSize := rand.Intn(100) + 1 + kvReader, err := newKVReader(ctx, "/"+prefix+"/0/one-file", memStore, 0, bufSize) + require.NoError(t, err) + for i := 0; i < kvCnt; i++ { + key, value, err := kvReader.nextKV() + require.NoError(t, err) + require.Equal(t, kvs[i].Key, key) + require.Equal(t, kvs[i].Val, value) + } + _, _, err = kvReader.nextKV() + require.Equal(t, io.EOF, err) + require.NoError(t, kvReader.Close()) + + statReader, err := newStatsReader(ctx, memStore, "/"+prefix+"/0_stat/one-file", bufSize) + require.NoError(t, err) + + var keyCnt uint64 = 0 + idx := 0 + for { + p, err := statReader.nextProp() + if err == io.EOF { + break + } + require.NoError(t, err) + keyCnt += p.keys + require.Equal(t, kvs[idx].Key, p.firstKey) + lastIdx := idx + int(keysDistance) - 1 + if lastIdx >= len(kvs) { + lastIdx = len(kvs) - 1 + } + require.Equal(t, kvs[lastIdx].Key, p.lastKey) + idx += int(keysDistance) + } + require.Equal(t, uint64(kvCnt), keyCnt) + require.NoError(t, statReader.Close()) +} + +func TestMergeOverlappingFilesV2(t *testing.T) { + // 1. Write to 5 files. + // 2. merge 5 files into one file. + // 3. read one file and check result. + // 4. check duplicate key. + ctx := context.Background() + memStore := storage.NewMemStorage() + writer := NewWriterBuilder(). + SetPropKeysDistance(2). + SetMemorySizeLimit(1000). + SetKeyDuplicationEncoding(true). + Build(memStore, "/test", "0") + + kvCount := 2000000 + for i := 0; i < kvCount; i++ { + v := i + if v == kvCount/2 { + v-- // insert a duplicate key. + } + key, val := []byte{byte(v)}, []byte{byte(v)} + err := writer.WriteRow(ctx, key, val, dbkv.IntHandle(i)) + require.NoError(t, err) + } + err := writer.Close(ctx) + require.NoError(t, err) + + err = MergeOverlappingFilesV2( + ctx, + []string{"/test/0/0", "/test/0/1", "/test/0/2", "/test/0/3", "/test/0/4"}, + memStore, + int64(5*size.MB), + 100, + "/test2", + "mergeID", + 1000, + 1000, + 8*1024, + 1*size.MB, + 2, + nil, + true, + ) + require.NoError(t, err) + + keys := make([][]byte, 0, kvCount) + values := make([][]byte, 0, kvCount) + + kvReader, err := newKVReader(ctx, "/test2/mergeID/one-file", memStore, 0, 100) + require.NoError(t, err) + for i := 0; i < kvCount; i++ { + key, value, err := kvReader.nextKV() + require.NoError(t, err) + clonedKey := make([]byte, len(key)) + copy(clonedKey, key) + clonedVal := make([]byte, len(value)) + copy(clonedVal, value) + keys = append(keys, clonedKey) + values = append(values, clonedVal) + } + _, _, err = kvReader.nextKV() + require.Equal(t, io.EOF, err) + require.NoError(t, kvReader.Close()) + + dir := t.TempDir() + db, err := pebble.Open(path.Join(dir, "duplicate"), nil) + require.NoError(t, err) + keyAdapter := common.DupDetectKeyAdapter{} + data := &MemoryIngestData{ + keyAdapter: keyAdapter, + duplicateDetection: true, + duplicateDB: db, + dupDetectOpt: common.DupDetectOpt{ReportErrOnDup: true}, + keys: keys, + values: values, + ts: 123, + } + pool := membuf.NewPool() + defer pool.Destroy() + iter := data.NewIter(ctx, nil, nil, pool) + + for iter.First(); iter.Valid(); iter.Next() { + } + err = iter.Error() + require.Error(t, err) + require.Contains(t, err.Error(), "found duplicate key") +} + +func TestOnefileWriterManyRows(t *testing.T) { + // 1. write into one file with sorted order. + // 2. merge one file. + // 3. read kv file and check the result. + // 4. check the writeSummary. + ctx := context.Background() + memStore := storage.NewMemStorage() + writer := NewWriterBuilder(). + SetPropKeysDistance(2). + SetMemorySizeLimit(1000). + BuildOneFile(memStore, "/test", "0") + + err := writer.Init(ctx, 5*1024*1024) + require.NoError(t, err) + + kvCnt := 100000 + expectedTotalSize := 0 + kvs := make([]common.KvPair, kvCnt) + for i := 0; i < kvCnt; i++ { + randLen := rand.Intn(10) + 1 + kvs[i].Key = make([]byte, randLen) + _, err := rand.Read(kvs[i].Key) + expectedTotalSize += randLen + + require.NoError(t, err) + randLen = rand.Intn(10) + 1 + kvs[i].Val = make([]byte, randLen) + _, err = rand.Read(kvs[i].Val) + require.NoError(t, err) + expectedTotalSize += randLen + } + + slices.SortFunc(kvs, func(i, j common.KvPair) int { + return bytes.Compare(i.Key, j.Key) + }) + + for _, item := range kvs { + err := writer.WriteRow(ctx, item.Key, item.Val) + require.NoError(t, err) + } + err = writer.Close(ctx) + require.NoError(t, err) + + var resSummary *WriterSummary + onClose := func(summary *WriterSummary) { + resSummary = summary + } + err = MergeOverlappingFilesV2( + ctx, + []string{"/test/0/one-file"}, + memStore, + int64(5*size.MB), + 100, + "/test2", + "mergeID", + 1000, + 1000, + 8*1024, + 1*size.MB, + 2, + onClose, + true, + ) + require.NoError(t, err) + + bufSize := rand.Intn(100) + 1 + kvReader, err := newKVReader(ctx, "/test2/mergeID/one-file", memStore, 0, bufSize) + require.NoError(t, err) + for i := 0; i < kvCnt; i++ { + key, value, err := kvReader.nextKV() + require.NoError(t, err) + require.Equal(t, kvs[i].Key, key) + require.Equal(t, kvs[i].Val, value) + } + _, _, err = kvReader.nextKV() + require.Equal(t, io.EOF, err) + require.NoError(t, kvReader.Close()) + + // check writerSummary. + expected := MultipleFilesStat{ + MinKey: kvs[0].Key, + MaxKey: kvs[len(kvs)-1].Key, + Filenames: [][2]string{ + {"/test2/mergeID/one-file", "/test2/mergeID_stat/one-file"}, + }, + MaxOverlappingNum: 1, + } + require.EqualValues(t, expected.MinKey, resSummary.Min) + require.EqualValues(t, expected.MaxKey, resSummary.Max) + require.Equal(t, expected.Filenames, resSummary.MultipleFilesStats[0].Filenames) + require.Equal(t, expected.MaxOverlappingNum, resSummary.MultipleFilesStats[0].MaxOverlappingNum) + require.EqualValues(t, expectedTotalSize, resSummary.TotalSize) +} diff --git a/br/pkg/lightning/backend/external/util.go b/br/pkg/lightning/backend/external/util.go index e1c4494b89f1b..a9597e42b9883 100644 --- a/br/pkg/lightning/backend/external/util.go +++ b/br/pkg/lightning/backend/external/util.go @@ -20,8 +20,10 @@ import ( "fmt" "slices" "sort" + "strconv" "strings" + "github.com/docker/go-units" "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/pkg/kv" @@ -62,9 +64,10 @@ func seekPropsOffsets( propKey := kv.Key(p.firstKey) if propKey.Cmp(start) > 0 { if !moved { - return nil, fmt.Errorf("start key %s is too small for stat files %v", + return nil, fmt.Errorf("start key %s is too small for stat files %v, propKey %s", start.String(), paths, + propKey.String(), ) } return offsets, nil @@ -292,3 +295,13 @@ func BytesMax(a, b []byte) []byte { } return b } + +func getSpeed(n uint64, dur float64, isBytes bool) string { + if dur == 0 { + return "-" + } + if isBytes { + return units.BytesSize(float64(n) / dur) + } + return strconv.FormatFloat(float64(n)/dur, 'f', 4, 64) +} diff --git a/br/pkg/lightning/backend/external/writer.go b/br/pkg/lightning/backend/external/writer.go index 1122f0a8b554a..6556da432b784 100644 --- a/br/pkg/lightning/backend/external/writer.go +++ b/br/pkg/lightning/backend/external/writer.go @@ -122,6 +122,7 @@ func NewWriterBuilder() *WriterBuilder { // SetMemorySizeLimit sets the memory size limit of the writer. When accumulated // data size exceeds this limit, the writer will flush data as a file to external // storage. +// When the writer is OneFileWriter SetMemorySizeLimit sets the preAllocated memory buffer size. func (b *WriterBuilder) SetMemorySizeLimit(size uint64) *WriterBuilder { b.memSizeLimit = size return b @@ -201,6 +202,35 @@ func (b *WriterBuilder) Build( fileMaxKeys: make([]tidbkv.Key, 0, multiFileStatNum), } ret.multiFileStats[0].Filenames = make([][2]string, 0, multiFileStatNum) + + return ret +} + +// BuildOneFile builds a new one file Writer. The writer will create only one file under the prefix +// of "{prefix}/{writerID}". +func (b *WriterBuilder) BuildOneFile( + store storage.ExternalStorage, + prefix string, + writerID string, +) *OneFileWriter { + filenamePrefix := filepath.Join(prefix, writerID) + p := membuf.NewPool(membuf.WithPoolSize(0), membuf.WithBlockSize(b.blockSize)) + + ret := &OneFileWriter{ + rc: &rangePropertiesCollector{ + props: make([]*rangeProperty, 0, 1024), + currProp: &rangeProperty{}, + propSizeDist: b.propSizeDist, + propKeysDist: b.propKeysDist, + }, + kvBuffer: p.NewBuffer(membuf.WithMemoryLimit(b.memSizeLimit)), + store: store, + filenamePrefix: filenamePrefix, + writerID: writerID, + kvStore: nil, + onClose: b.onClose, + closed: false, + } return ret } @@ -389,15 +419,6 @@ func (w *Writer) flushKVs(ctx context.Context, fromClose bool) (err error) { savedBytes = w.batchSize startTs := time.Now() - getSpeed := func(n uint64, dur float64, isBytes bool) string { - if dur == 0 { - return "-" - } - if isBytes { - return units.BytesSize(float64(n) / dur) - } - return units.HumanSize(float64(n) / dur) - } kvCnt := len(w.kvLocations) defer func() { w.currentSeq++ @@ -501,12 +522,12 @@ func (w *Writer) createStorageWriter(ctx context.Context) ( err error, ) { dataPath := filepath.Join(w.filenamePrefix, strconv.Itoa(w.currentSeq)) - dataWriter, err := w.store.Create(ctx, dataPath, &storage.WriterOption{Concurrency: 20}) + dataWriter, err := w.store.Create(ctx, dataPath, &storage.WriterOption{Concurrency: 20, PartSize: (int64)(5 * size.MB)}) if err != nil { return "", "", nil, nil, err } statPath := filepath.Join(w.filenamePrefix+statSuffix, strconv.Itoa(w.currentSeq)) - statsWriter, err := w.store.Create(ctx, statPath, &storage.WriterOption{Concurrency: 20}) + statsWriter, err := w.store.Create(ctx, statPath, &storage.WriterOption{Concurrency: 20, PartSize: (int64)(5 * size.MB)}) if err != nil { _ = dataWriter.Close(ctx) return "", "", nil, nil, err diff --git a/br/pkg/storage/s3.go b/br/pkg/storage/s3.go index 97f58ec5b89fd..cfb3e8b5bba1a 100644 --- a/br/pkg/storage/s3.go +++ b/br/pkg/storage/s3.go @@ -1031,6 +1031,7 @@ func (rs *S3Storage) Create(ctx context.Context, name string, option *WriterOpti } } else { up := s3manager.NewUploaderWithClient(rs.svc, func(u *s3manager.Uploader) { + u.PartSize = option.PartSize u.Concurrency = option.Concurrency u.BufferProvider = s3manager.NewBufferedReadSeekerWriteToPool(option.Concurrency * hardcodedS3ChunkSize) }) diff --git a/br/pkg/storage/storage.go b/br/pkg/storage/storage.go index 0abecc827414c..348bbd32a5a26 100644 --- a/br/pkg/storage/storage.go +++ b/br/pkg/storage/storage.go @@ -90,6 +90,7 @@ type Writer interface { type WriterOption struct { Concurrency int + PartSize int64 } type ReaderOption struct { diff --git a/pkg/ddl/backfilling_dispatcher.go b/pkg/ddl/backfilling_dispatcher.go index 622ef9deab136..d8f9ff561eed0 100644 --- a/pkg/ddl/backfilling_dispatcher.go +++ b/pkg/ddl/backfilling_dispatcher.go @@ -272,7 +272,11 @@ func generatePartitionPlan(tblInfo *model.TableInfo) (metas [][]byte, err error) } func generateNonPartitionPlan( - d *ddl, tblInfo *model.TableInfo, job *model.Job, useCloud bool, instanceCnt int) (metas [][]byte, err error) { + d *ddl, + tblInfo *model.TableInfo, + job *model.Job, + useCloud bool, + instanceCnt int) (metas [][]byte, err error) { tbl, err := getTable((*asAutoIDRequirement)(d.ddlCtx), job.SchemaID, tblInfo) if err != nil { return nil, err @@ -281,6 +285,7 @@ func generateNonPartitionPlan( if err != nil { return nil, errors.Trace(err) } + startKey, endKey, err := getTableRange(d.jobContext(job.ID, job.ReorgMeta), d.ddlCtx, tbl.(table.PhysicalTable), ver.Ver, job.Priority) if startKey == nil && endKey == nil { // Empty table. @@ -387,6 +392,7 @@ func generateGlobalSortIngestPlan( logger.Info("split subtask range", zap.String("startKey", hex.EncodeToString(startKey)), zap.String("endKey", hex.EncodeToString(endKey))) + if startKey.Cmp(endKey) >= 0 { return nil, errors.Errorf("invalid range, startKey: %s, endKey: %s", hex.EncodeToString(startKey), hex.EncodeToString(endKey)) diff --git a/pkg/ddl/backfilling_dispatcher_test.go b/pkg/ddl/backfilling_dispatcher_test.go index a7a02231b7fcb..5da673b6625e5 100644 --- a/pkg/ddl/backfilling_dispatcher_test.go +++ b/pkg/ddl/backfilling_dispatcher_test.go @@ -54,6 +54,8 @@ func TestBackfillingDispatcherLocalMode(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/mockWriterMemSize", "return()")) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/mockWriterMemSize") /// 1. test partition table. tk.MustExec("create table tp1(id int primary key, v int) PARTITION BY RANGE (id) (\n " + "PARTITION p0 VALUES LESS THAN (10),\n" + diff --git a/pkg/ddl/backfilling_dist_scheduler.go b/pkg/ddl/backfilling_dist_scheduler.go index a7025ddcf3946..d887bb0a5cf91 100644 --- a/pkg/ddl/backfilling_dist_scheduler.go +++ b/pkg/ddl/backfilling_dist_scheduler.go @@ -89,7 +89,7 @@ func NewBackfillSubtaskExecutor(_ context.Context, taskMeta []byte, d *ddl, return newReadIndexExecutor( d, &bgm.Job, indexInfos, tbl.(table.PhysicalTable), jc, bc, summary, bgm.CloudStorageURI), nil case proto.StepTwo: - return newMergeSortExecutor(jobMeta.ID, indexInfos[0], tbl.(table.PhysicalTable), bc, bgm.CloudStorageURI) + return newMergeSortExecutor(jobMeta.ID, len(indexInfos), tbl.(table.PhysicalTable), bc, bgm.CloudStorageURI) case proto.StepThree: if len(bgm.CloudStorageURI) > 0 { return newCloudImportExecutor(&bgm.Job, jobMeta.ID, indexInfos[0], tbl.(table.PhysicalTable), bc, bgm.CloudStorageURI) diff --git a/pkg/ddl/backfilling_merge_sort.go b/pkg/ddl/backfilling_merge_sort.go index 32d27d7146a06..6de8ee0d7b756 100644 --- a/pkg/ddl/backfilling_merge_sort.go +++ b/pkg/ddl/backfilling_merge_sort.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/pkg/ddl/ingest" "github.com/pingcap/tidb/pkg/disttask/framework/proto" - "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util/intest" @@ -37,7 +36,7 @@ import ( type mergeSortExecutor struct { jobID int64 - index *model.IndexInfo + idxNum int ptbl table.PhysicalTable bc ingest.BackendCtx cloudStoreURI string @@ -47,14 +46,14 @@ type mergeSortExecutor struct { func newMergeSortExecutor( jobID int64, - index *model.IndexInfo, + idxNum int, ptbl table.PhysicalTable, bc ingest.BackendCtx, cloudStoreURI string, ) (*mergeSortExecutor, error) { return &mergeSortExecutor{ jobID: jobID, - index: index, + idxNum: idxNum, ptbl: ptbl, bc: bc, cloudStoreURI: cloudStoreURI, @@ -100,9 +99,23 @@ func (m *mergeSortExecutor) RunSubtask(ctx context.Context, subtask *proto.Subta prefix := path.Join(strconv.Itoa(int(m.jobID)), strconv.Itoa(int(subtask.ID))) - // TODO: config generated by plan. - return external.MergeOverlappingFiles(ctx, sm.DataFiles, store, 64*1024, prefix, - external.DefaultBlockSize, 8*1024, 1*size.MB, 8*1024, onClose, + partSize, err := getMergeSortPartSize(int(variable.GetDDLReorgWorkerCounter()), m.idxNum) + if err != nil { + return err + } + + return external.MergeOverlappingFiles( + ctx, + sm.DataFiles, + store, + int64(partSize), + 64*1024, + prefix, + external.DefaultBlockSize, + 8*1024, + 1*size.MB, + 8*1024, + onClose, int(variable.GetDDLReorgWorkerCounter()), true) } diff --git a/pkg/ddl/backfilling_operators.go b/pkg/ddl/backfilling_operators.go index a766142977243..e5b53c0a369eb 100644 --- a/pkg/ddl/backfilling_operators.go +++ b/pkg/ddl/backfilling_operators.go @@ -48,6 +48,7 @@ import ( "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/logutil" "github.com/pingcap/tidb/pkg/util/memory" + "github.com/pingcap/tidb/pkg/util/size" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -111,6 +112,33 @@ func (ctx *OperatorCtx) OperatorErr() error { return *err } +func getWriterMemSize(idxNum int) (uint64, error) { + failpoint.Inject("mockWriterMemSize", func() { + failpoint.Return(1*size.GB, nil) + }) + _, writerCnt := expectedIngestWorkerCnt() + memTotal, err := memory.MemTotal() + if err != nil { + return 0, err + } + memUsed, err := memory.MemUsed() + if err != nil { + return 0, err + } + memAvailable := memTotal - memUsed + memSize := (memAvailable / 2) / uint64(writerCnt) / uint64(idxNum) + logutil.BgLogger().Info("build operators that write index to cloud storage", zap.Uint64("memory total", memTotal), zap.Uint64("memory used", memUsed), zap.Uint64("memory size", memSize)) + return memSize, nil +} + +func getMergeSortPartSize(concurrency int, idxNum int) (uint64, error) { + writerMemSize, err := getWriterMemSize(idxNum) + if err != nil { + return 0, nil + } + return writerMemSize / uint64(concurrency) / 10, nil +} + // NewAddIndexIngestPipeline creates a pipeline for adding index in ingest mode. func NewAddIndexIngestPipeline( ctx *OperatorCtx, @@ -199,17 +227,10 @@ func NewWriteIndexToExternalStoragePipeline( return nil, err } - memTotal, err := memory.MemTotal() + memSize, err := getWriterMemSize(len(indexes)) if err != nil { return nil, err } - memUsed, err := memory.MemUsed() - if err != nil { - return nil, err - } - memAvailable := memTotal - memUsed - memSize := (memAvailable / 2) / uint64(writerCnt) / uint64(len(indexes)) - logutil.BgLogger().Info("build operators that write index to cloud storage", zap.Uint64("memory total", memTotal), zap.Uint64("memory used", memUsed), zap.Uint64("memory size", memSize)) srcOp := NewTableScanTaskSource(ctx, store, tbl, startKey, endKey) scanOp := NewTableScanOperator(ctx, sessPool, copCtx, srcChkPool, readerCnt) diff --git a/pkg/ddl/backfilling_read_index.go b/pkg/ddl/backfilling_read_index.go index 01aac70dac029..f213e56b36443 100644 --- a/pkg/ddl/backfilling_read_index.go +++ b/pkg/ddl/backfilling_read_index.go @@ -273,6 +273,19 @@ func (r *readIndexExecutor) buildExternalStorePipeline( counter := metrics.BackfillTotalCounter.WithLabelValues( metrics.GenerateReorgLabel("add_idx_rate", r.job.SchemaName, tbl.Meta().Name.O)) return NewWriteIndexToExternalStoragePipeline( - opCtx, d.store, r.cloudStorageURI, r.d.sessPool, sessCtx, r.job.ID, subtaskID, - tbl, r.indexes, start, end, totalRowCount, counter, onClose, r.job.ReorgMeta) + opCtx, + d.store, + r.cloudStorageURI, + r.d.sessPool, + sessCtx, + r.job.ID, + subtaskID, + tbl, + r.indexes, + start, + end, + totalRowCount, + counter, + onClose, + r.job.ReorgMeta) } diff --git a/pkg/disttask/importinto/scheduler.go b/pkg/disttask/importinto/scheduler.go index 6645e597d50a4..2afa6ea412960 100644 --- a/pkg/disttask/importinto/scheduler.go +++ b/pkg/disttask/importinto/scheduler.go @@ -266,6 +266,7 @@ type mergeSortStepExecutor struct { // subtask of a task is run in serial now, so we don't need lock here. // change to SyncMap when we support parallel subtask in the future. subtaskSortedKVMeta *external.SortedKVMeta + partSize int64 } var _ execute.SubtaskExecutor = &mergeSortStepExecutor{} @@ -279,6 +280,8 @@ func (m *mergeSortStepExecutor) Init(ctx context.Context) error { return err } m.controller = controller + // 10000 = max part num + m.partSize = int64(getWriterMemorySizeLimit(&m.taskMeta.Plan) / 10000 * uint64(external.MergeSortOverlapThreshold)) return nil } @@ -305,7 +308,9 @@ func (m *mergeSortStepExecutor) RunSubtask(ctx context.Context, subtask *proto.S prefix := subtaskPrefix(m.taskID, subtask.ID) - return external.MergeOverlappingFiles(ctx, sm.DataFiles, m.controller.GlobalSortStore, 64*1024, + logger.Info("merge sort partSize", zap.String("size", units.BytesSize(float64(m.partSize)))) + + return external.MergeOverlappingFiles(ctx, sm.DataFiles, m.controller.GlobalSortStore, m.partSize, 64*1024, prefix, getKVGroupBlockSize(sm.KVGroup), 8*1024, 1*size.MB, 8*1024, onClose, int(m.taskMeta.Plan.ThreadCnt), false) } From 3543275dcf4b6454eb874c1362c87d31a963da6d Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Thu, 23 Nov 2023 17:32:42 +0800 Subject: [PATCH 27/36] statistics: fix the memory tracker in the benchmark (#48831) close pingcap/tidb#48699 --- pkg/statistics/builder_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/statistics/builder_test.go b/pkg/statistics/builder_test.go index ed39d8feffd64..54f2875c399cd 100644 --- a/pkg/statistics/builder_test.go +++ b/pkg/statistics/builder_test.go @@ -69,9 +69,9 @@ func BenchmarkBuildHistAndTopN(b *testing.B) { TotalSize: int64(len(data)) * 8, } filedType := types.NewFieldType(mysql.TypeLong) - memoryTracker := memory.NewTracker(10, 10) + memoryTracker := memory.NewTracker(10, 1024*1024*1024) b.ResetTimer() for i := 0; i < b.N; i++ { - _, _, _ = BuildHistAndTopN(ctx, 0, 0, 0, collector, filedType, true, memoryTracker) + _, _, _ = BuildHistAndTopN(ctx, 256, 500, 0, collector, filedType, true, memoryTracker) } } From 707f8605278c56aa2657203cb460766bb695191f Mon Sep 17 00:00:00 2001 From: YangKeao Date: Thu, 23 Nov 2023 18:19:13 +0800 Subject: [PATCH 28/36] util/rowcodec,tablecodec: remove `stmtctx` dependency from `rowcodec` and `tablecodec` (#48816) close pingcap/tidb#48751 --- pkg/ddl/column.go | 3 +- pkg/errctx/BUILD.bazel | 1 + pkg/errctx/context.go | 20 ++++++ pkg/errctx/context_test.go | 10 +++ pkg/executor/mem_reader.go | 7 +- pkg/executor/test/executor/executor_test.go | 2 +- pkg/executor/write.go | 3 +- pkg/server/handler/tests/http_handler_test.go | 2 +- pkg/sessionctx/stmtctx/stmtctx.go | 4 ++ pkg/statistics/cmsketch.go | 8 ++- pkg/statistics/row_sampler.go | 3 +- pkg/statistics/sample.go | 6 +- pkg/store/mockstore/cluster_test.go | 2 +- .../mockstore/unistore/cophandler/analyze.go | 3 +- .../unistore/cophandler/cop_handler_test.go | 2 +- pkg/store/mockstore/unistore/tikv/BUILD.bazel | 1 - pkg/store/mockstore/unistore/tikv/mvcc.go | 3 +- pkg/table/tables/index.go | 11 +++- pkg/table/tables/mutation_checker_test.go | 8 +-- pkg/table/tables/tables.go | 21 ++++-- pkg/tablecodec/BUILD.bazel | 2 - pkg/tablecodec/tablecodec.go | 58 ++++++++-------- pkg/tablecodec/tablecodec_test.go | 24 +++---- pkg/util/rowDecoder/decoder_test.go | 4 +- pkg/util/rowcodec/BUILD.bazel | 3 +- pkg/util/rowcodec/bench_test.go | 3 +- pkg/util/rowcodec/encoder.go | 38 +++++------ pkg/util/rowcodec/main_test.go | 6 +- pkg/util/rowcodec/rowcodec_test.go | 66 ++++++++----------- .../addindextest1/disttask_test.go | 2 +- 30 files changed, 180 insertions(+), 146 deletions(-) diff --git a/pkg/ddl/column.go b/pkg/ddl/column.go index 2141112d814d5..57e6d1cc4170a 100644 --- a/pkg/ddl/column.go +++ b/pkg/ddl/column.go @@ -1384,7 +1384,8 @@ func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, ra } checksums := w.calcChecksums() sctx, rd := w.sessCtx.GetSessionVars().StmtCtx, &w.sessCtx.GetSessionVars().RowEncoder - newRowVal, err := tablecodec.EncodeRow(sctx, newRow, newColumnIDs, nil, nil, rd, checksums...) + newRowVal, err := tablecodec.EncodeRow(sctx.TimeZone(), newRow, newColumnIDs, nil, nil, rd, checksums...) + err = sctx.HandleError(err) if err != nil { return errors.Trace(err) } diff --git a/pkg/errctx/BUILD.bazel b/pkg/errctx/BUILD.bazel index c2f2edc0efdeb..ef0f7368ccd79 100644 --- a/pkg/errctx/BUILD.bazel +++ b/pkg/errctx/BUILD.bazel @@ -22,5 +22,6 @@ go_test( "//pkg/types", "@com_github_pingcap_errors//:errors", "@com_github_stretchr_testify//require", + "@org_uber_go_multierr//:multierr", ], ) diff --git a/pkg/errctx/context.go b/pkg/errctx/context.go index df47173a58940..18dd3e5b22c1a 100644 --- a/pkg/errctx/context.go +++ b/pkg/errctx/context.go @@ -68,7 +68,27 @@ func (ctx *Context) appendWarning(err error) { } // HandleError handles the error according to the context. See the comment of `HandleErrorWithAlias` for detailed logic. +// +// It also allows using `errors.ErrorGroup`, in this case, it'll handle each error in order, and return the first error +// it founds. func (ctx *Context) HandleError(err error) error { + // The function of handling `errors.ErrorGroup` is placed in `HandleError` but not in `HandleErrorWithAlias`, because + // it's hard to give a proper error and warn alias for an error group. + if errs, ok := err.(errors.ErrorGroup); ok { + for _, singleErr := range errs.Errors() { + singleErr = ctx.HandleError(singleErr) + // If the one error is found, just return it. + // TODO: consider whether it's more appropriate to continue to handle other errors. For example, other errors + // may need to append warnings. The current behavior is same with TiDB original behavior before using + // `errctx` to handle multiple errors. + if singleErr != nil { + return singleErr + } + } + + return nil + } + return ctx.HandleErrorWithAlias(err, err, err) } diff --git a/pkg/errctx/context_test.go b/pkg/errctx/context_test.go index 9cf60797380e7..e78cbb09c7640 100644 --- a/pkg/errctx/context_test.go +++ b/pkg/errctx/context_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/pkg/errctx" "github.com/pingcap/tidb/pkg/types" "github.com/stretchr/testify/require" + "go.uber.org/multierr" ) func TestContext(t *testing.T) { @@ -50,4 +51,13 @@ func TestContext(t *testing.T) { require.Equal(t, warn, testWarn) // newCtx2 will return all errors require.Equal(t, newCtx2.HandleErrorWithAlias(testInternalErr, testErr, testWarn), testErr) + + // test `multierr` + testErrs := multierr.Append(testInternalErr, testErr) + require.Equal(t, ctx.HandleError(testErrs), testInternalErr) + require.Equal(t, newCtx.HandleError(testErrs), testErr) + require.Equal(t, warn, testInternalErr) + + // test nil + require.Nil(t, ctx.HandleError(nil)) } diff --git a/pkg/executor/mem_reader.go b/pkg/executor/mem_reader.go index 2142037adf9af..38fab12f728a2 100644 --- a/pkg/executor/mem_reader.go +++ b/pkg/executor/mem_reader.go @@ -252,7 +252,9 @@ func buildMemTableReader(ctx context.Context, us *UnionScanExec, kvRanges []kv.K if err != nil { return nil, err } - return tablecodec.EncodeValue(us.Ctx().GetSessionVars().StmtCtx, nil, d) + sctx := us.Ctx().GetSessionVars().StmtCtx + buf, err := tablecodec.EncodeValue(sctx.TimeZone(), nil, d) + return buf, sctx.HandleError(err) } cd := NewRowDecoder(us.Ctx(), us.Schema(), us.table.Meta()) rd := rowcodec.NewByteDecoder(colInfo, pkColIDs, defVal, us.Ctx().GetSessionVars().Location()) @@ -1164,7 +1166,8 @@ func getColIDAndPkColIDs(ctx sessionctx.Context, tbl table.Table, columns []*mod if err != nil { return nil, err } - return tablecodec.EncodeValue(ctx.GetSessionVars().StmtCtx, nil, d) + buf, err := tablecodec.EncodeValue(ctx.GetSessionVars().StmtCtx.TimeZone(), nil, d) + return buf, ctx.GetSessionVars().StmtCtx.HandleError(err) } rd := rowcodec.NewByteDecoder(colInfos, pkColIDs, defVal, ctx.GetSessionVars().Location()) return colIDs, pkColIDs, rd diff --git a/pkg/executor/test/executor/executor_test.go b/pkg/executor/test/executor/executor_test.go index 15eec21cdcbf8..b3ebaa566874e 100644 --- a/pkg/executor/test/executor/executor_test.go +++ b/pkg/executor/test/executor/executor_test.go @@ -387,7 +387,7 @@ func setColValue(t *testing.T, txn kv.Transaction, key kv.Key, v types.Datum) { colIDs := []int64{2, 3} sc := stmtctx.NewStmtCtxWithTimeZone(time.Local) rd := rowcodec.Encoder{Enable: true} - value, err := tablecodec.EncodeRow(sc, row, colIDs, nil, nil, &rd) + value, err := tablecodec.EncodeRow(sc.TimeZone(), row, colIDs, nil, nil, &rd) require.NoError(t, err) err = txn.Set(key, value) require.NoError(t, err) diff --git a/pkg/executor/write.go b/pkg/executor/write.go index 28733152208ea..f609110d56b29 100644 --- a/pkg/executor/write.go +++ b/pkg/executor/write.go @@ -268,7 +268,7 @@ func addUnchangedKeysForLockByRow( return count, err } unchangedUniqueKey, _, err := tablecodec.GenIndexKey( - stmtCtx, + stmtCtx.TimeZone(), idx.TableMeta(), meta, physicalID, @@ -276,6 +276,7 @@ func addUnchangedKeysForLockByRow( h, nil, ) + err = stmtCtx.HandleError(err) if err != nil { return count, err } diff --git a/pkg/server/handler/tests/http_handler_test.go b/pkg/server/handler/tests/http_handler_test.go index 7610e5d3d9bac..6435781a957fc 100644 --- a/pkg/server/handler/tests/http_handler_test.go +++ b/pkg/server/handler/tests/http_handler_test.go @@ -702,7 +702,7 @@ func TestDecodeColumnValue(t *testing.T) { } rd := rowcodec.Encoder{Enable: true} sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) - bs, err := tablecodec.EncodeRow(sc, row, colIDs, nil, nil, &rd) + bs, err := tablecodec.EncodeRow(sc.TimeZone(), row, colIDs, nil, nil, &rd) require.NoError(t, err) require.NotNil(t, bs) bin := base64.StdEncoding.EncodeToString(bs) diff --git a/pkg/sessionctx/stmtctx/stmtctx.go b/pkg/sessionctx/stmtctx/stmtctx.go index e65a55ebf5a99..9764ca5fcc1a2 100644 --- a/pkg/sessionctx/stmtctx/stmtctx.go +++ b/pkg/sessionctx/stmtctx/stmtctx.go @@ -504,6 +504,10 @@ func (sc *StatementContext) HandleTruncate(err error) error { // HandleError handles the error based on `ErrCtx()` func (sc *StatementContext) HandleError(err error) error { + intest.AssertNotNil(sc) + if sc == nil { + return err + } errCtx := sc.ErrCtx() return errCtx.HandleError(err) } diff --git a/pkg/statistics/cmsketch.go b/pkg/statistics/cmsketch.go index 9643e6b7188b2..5851749238052 100644 --- a/pkg/statistics/cmsketch.go +++ b/pkg/statistics/cmsketch.go @@ -23,6 +23,7 @@ import ( "slices" "sort" "strings" + "time" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -259,10 +260,15 @@ func (c *CMSketch) SubValue(h1, h2 uint64, count uint64) { // QueryValue is used to query the count of specified value. func QueryValue(sctx sessionctx.Context, c *CMSketch, t *TopN, val types.Datum) (uint64, error) { var sc *stmtctx.StatementContext + tz := time.UTC if sctx != nil { sc = sctx.GetSessionVars().StmtCtx + tz = sc.TimeZone() + } + rawData, err := tablecodec.EncodeValue(tz, nil, val) + if sc != nil { + err = sc.HandleError(err) } - rawData, err := tablecodec.EncodeValue(sc, nil, val) if err != nil { return 0, errors.Trace(err) } diff --git a/pkg/statistics/row_sampler.go b/pkg/statistics/row_sampler.go index 3c26c366ad2b7..97e9235e74412 100644 --- a/pkg/statistics/row_sampler.go +++ b/pkg/statistics/row_sampler.go @@ -195,7 +195,8 @@ func (s *RowSampleBuilder) Collect() (RowSampleCollector, error) { return nil, err } decodedVal.SetBytesAsString(s.Collators[i].Key(decodedVal.GetString()), decodedVal.Collation(), uint32(decodedVal.Length())) - encodedKey, err := tablecodec.EncodeValue(s.Sc, nil, decodedVal) + encodedKey, err := tablecodec.EncodeValue(s.Sc.TimeZone(), nil, decodedVal) + err = s.Sc.HandleError(err) if err != nil { return nil, err } diff --git a/pkg/statistics/sample.go b/pkg/statistics/sample.go index 54dc0a4892d13..5d056abd115d9 100644 --- a/pkg/statistics/sample.go +++ b/pkg/statistics/sample.go @@ -259,7 +259,8 @@ func (s SampleBuilder) CollectColumnStats() ([]*SampleCollector, *SortedBuilder, return nil, nil, err } decodedVal.SetBytesAsString(s.Collators[i].Key(decodedVal.GetString()), decodedVal.Collation(), uint32(decodedVal.Length())) - encodedKey, err := tablecodec.EncodeValue(s.Sc, nil, decodedVal) + encodedKey, err := tablecodec.EncodeValue(s.Sc.TimeZone(), nil, decodedVal) + err = s.Sc.HandleError(err) if err != nil { return nil, nil, err } @@ -306,7 +307,8 @@ func (c *SampleCollector) ExtractTopN(numTop uint32, sc *stmtctx.StatementContex if err != nil { return err } - data, err := tablecodec.EncodeValue(sc, nil, d) + data, err := tablecodec.EncodeValue(sc.TimeZone(), nil, d) + err = sc.HandleError(err) if err != nil { return err } diff --git a/pkg/store/mockstore/cluster_test.go b/pkg/store/mockstore/cluster_test.go index caeaee56722d1..b3958b1ce9475 100644 --- a/pkg/store/mockstore/cluster_test.go +++ b/pkg/store/mockstore/cluster_test.go @@ -58,7 +58,7 @@ func TestClusterSplit(t *testing.T) { colValue := types.NewStringDatum(strconv.Itoa(int(handle))) // TODO: Should use session's TimeZone instead of UTC. rd := rowcodec.Encoder{Enable: true} - rowValue, err1 := tablecodec.EncodeRow(sc, []types.Datum{colValue}, []int64{colID}, nil, nil, &rd) + rowValue, err1 := tablecodec.EncodeRow(sc.TimeZone(), []types.Datum{colValue}, []int64{colID}, nil, nil, &rd) require.NoError(t, err1) txn.Set(rowKey, rowValue) diff --git a/pkg/store/mockstore/unistore/cophandler/analyze.go b/pkg/store/mockstore/unistore/cophandler/analyze.go index 2b58c57d4ecb2..e77a41942a2a1 100644 --- a/pkg/store/mockstore/unistore/cophandler/analyze.go +++ b/pkg/store/mockstore/unistore/cophandler/analyze.go @@ -495,7 +495,8 @@ func (e *analyzeColumnsExec) Process(key, value []byte) error { continue } - value, err := tablecodec.EncodeValue(e.evalCtx.sc, nil, d) + value, err := tablecodec.EncodeValue(e.evalCtx.sc.TimeZone(), nil, d) + err = e.evalCtx.sc.HandleError(err) if err != nil { return err } diff --git a/pkg/store/mockstore/unistore/cophandler/cop_handler_test.go b/pkg/store/mockstore/unistore/cophandler/cop_handler_test.go index 87aea1bd48dd6..b3155d288ee64 100644 --- a/pkg/store/mockstore/unistore/cophandler/cop_handler_test.go +++ b/pkg/store/mockstore/unistore/cophandler/cop_handler_test.go @@ -118,7 +118,7 @@ func prepareTestTableData(keyNumber int, tableID int64) (*data, error) { for i := 0; i < keyNumber; i++ { datum := types.MakeDatums(i, "abc", 10.0) rows[int64(i)] = datum - rowEncodedData, err := tablecodec.EncodeRow(stmtCtx, datum, colIds, nil, nil, encoder) + rowEncodedData, err := tablecodec.EncodeRow(stmtCtx.TimeZone(), datum, colIds, nil, nil, encoder) if err != nil { return nil, err } diff --git a/pkg/store/mockstore/unistore/tikv/BUILD.bazel b/pkg/store/mockstore/unistore/tikv/BUILD.bazel index 978b52db23886..396ae089245c8 100644 --- a/pkg/store/mockstore/unistore/tikv/BUILD.bazel +++ b/pkg/store/mockstore/unistore/tikv/BUILD.bazel @@ -18,7 +18,6 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv", - "//pkg/sessionctx/stmtctx", "//pkg/store/mockstore/unistore/client", "//pkg/store/mockstore/unistore/config", "//pkg/store/mockstore/unistore/cophandler", diff --git a/pkg/store/mockstore/unistore/tikv/mvcc.go b/pkg/store/mockstore/unistore/tikv/mvcc.go index 7f8f8eb9104e3..3607abfd0ae31 100644 --- a/pkg/store/mockstore/unistore/tikv/mvcc.go +++ b/pkg/store/mockstore/unistore/tikv/mvcc.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/log" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/store/mockstore/unistore/config" "github.com/pingcap/tidb/pkg/store/mockstore/unistore/lockstore" "github.com/pingcap/tidb/pkg/store/mockstore/unistore/pd" @@ -1001,7 +1000,7 @@ func encodeFromOldRow(oldRow, buf []byte) ([]byte, error) { } var encoder rowcodec.Encoder buf = buf[:0] - return encoder.Encode(stmtctx.NewStmtCtx(), colIDs, datums, buf) + return encoder.Encode(time.UTC, colIDs, datums, buf) } func (store *MVCCStore) buildPrewriteLock(reqCtx *requestCtx, m *kvrpcpb.Mutation, item *badger.Item, diff --git a/pkg/table/tables/index.go b/pkg/table/tables/index.go index f0fb1b45878e0..8fb48acb87db8 100644 --- a/pkg/table/tables/index.go +++ b/pkg/table/tables/index.go @@ -93,7 +93,9 @@ func (c *index) GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types. if c.idxInfo.Global { idxTblID = c.tblInfo.ID } - return tablecodec.GenIndexKey(sc, c.tblInfo, c.idxInfo, idxTblID, indexedValues, h, buf) + key, distinct, err = tablecodec.GenIndexKey(sc.TimeZone(), c.tblInfo, c.idxInfo, idxTblID, indexedValues, h, buf) + err = sc.HandleError(err) + return } // GenIndexValue generates the index value. @@ -102,7 +104,9 @@ func (c *index) GenIndexValue(sc *stmtctx.StatementContext, distinct bool, index c.initNeedRestoreData.Do(func() { c.needRestoredData = NeedRestoredData(c.idxInfo.Columns, c.tblInfo.Columns) }) - return tablecodec.GenIndexValuePortal(sc, c.tblInfo, c.idxInfo, c.needRestoredData, distinct, false, indexedValues, h, c.phyTblID, restoredData, buf) + idx, err := tablecodec.GenIndexValuePortal(sc.TimeZone(), c.tblInfo, c.idxInfo, c.needRestoredData, distinct, false, indexedValues, h, c.phyTblID, restoredData, buf) + err = sc.HandleError(err) + return idx, err } // getIndexedValue will produce the result like: @@ -233,8 +237,9 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue c.initNeedRestoreData.Do(func() { c.needRestoredData = NeedRestoredData(c.idxInfo.Columns, c.tblInfo.Columns) }) - idxVal, err := tablecodec.GenIndexValuePortal(sctx.GetSessionVars().StmtCtx, c.tblInfo, c.idxInfo, + idxVal, err := tablecodec.GenIndexValuePortal(sctx.GetSessionVars().StmtCtx.TimeZone(), c.tblInfo, c.idxInfo, c.needRestoredData, distinct, opt.Untouched, value, h, c.phyTblID, handleRestoreData, nil) + err = sctx.GetSessionVars().StmtCtx.HandleError(err) if err != nil { return nil, err } diff --git a/pkg/table/tables/mutation_checker_test.go b/pkg/table/tables/mutation_checker_test.go index 759800780f273..ee65710595cee 100644 --- a/pkg/table/tables/mutation_checker_test.go +++ b/pkg/table/tables/mutation_checker_test.go @@ -93,7 +93,7 @@ func TestCheckRowInsertionConsistency(t *testing.T) { // mocked data mockRowKey233 := tablecodec.EncodeRowKeyWithHandle(1, kv.IntHandle(233)) mockValue233, err := tablecodec.EncodeRow( - sessVars.StmtCtx, []types.Datum{types.NewIntDatum(233)}, []int64{101}, nil, nil, &rd, + sessVars.StmtCtx.TimeZone(), []types.Datum{types.NewIntDatum(233)}, []int64{101}, nil, nil, &rd, ) require.Nil(t, err) fakeRowInsertion := mutation{key: []byte{1, 1}, value: []byte{1, 1, 1}} @@ -306,7 +306,7 @@ func TestCheckIndexKeysAndCheckHandleConsistency(t *testing.T) { // test checkHandleConsistency rowKey := tablecodec.EncodeRowKeyWithHandle(table.tableID, handle) corruptedRowKey := tablecodec.EncodeRowKeyWithHandle(table.tableID, corruptedHandle) - rowValue, err := tablecodec.EncodeRow(sessVars.StmtCtx, rowToInsert, []int64{1, 2}, nil, nil, &rd) + rowValue, err := tablecodec.EncodeRow(sessVars.StmtCtx.TimeZone(), rowToInsert, []int64{1, 2}, nil, nil, &rd) require.Nil(t, err) rowMutation := mutation{key: rowKey, value: rowValue} corruptedRowMutation := mutation{key: corruptedRowKey, value: rowValue} @@ -327,14 +327,14 @@ func buildIndexKeyValue(index table.Index, rowToInsert []types.Datum, sessVars * return nil, nil, err } key, distinct, err := tablecodec.GenIndexKey( - sessVars.StmtCtx, &tableInfo, indexInfo, 1, indexedValues, handle, nil, + sessVars.StmtCtx.TimeZone(), &tableInfo, indexInfo, 1, indexedValues, handle, nil, ) if err != nil { return nil, nil, err } rsData := TryGetHandleRestoredDataWrapper(table.meta, rowToInsert, nil, indexInfo) value, err := tablecodec.GenIndexValuePortal( - sessVars.StmtCtx, &tableInfo, indexInfo, NeedRestoredData(indexInfo.Columns, tableInfo.Columns), + sessVars.StmtCtx.TimeZone(), &tableInfo, indexInfo, NeedRestoredData(indexInfo.Columns, tableInfo.Columns), distinct, false, indexedValues, handle, 0, rsData, nil, ) if err != nil { diff --git a/pkg/table/tables/tables.go b/pkg/table/tables/tables.go index ac9878783b261..640f6e0244768 100644 --- a/pkg/table/tables/tables.go +++ b/pkg/table/tables/tables.go @@ -549,7 +549,8 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx sessionctx.Context, key := t.RecordKey(h) sc, rd := sessVars.StmtCtx, &sessVars.RowEncoder checksums, writeBufs.RowValBuf = t.calcChecksums(sctx, h, checksumData, writeBufs.RowValBuf) - writeBufs.RowValBuf, err = tablecodec.EncodeRow(sc, row, colIDs, writeBufs.RowValBuf, writeBufs.AddRowValues, rd, checksums...) + writeBufs.RowValBuf, err = tablecodec.EncodeRow(sc.TimeZone(), row, colIDs, writeBufs.RowValBuf, writeBufs.AddRowValues, rd, checksums...) + err = sc.HandleError(err) if err != nil { return err } @@ -988,7 +989,8 @@ func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts . zap.Stringer("key", key)) sc, rd := sessVars.StmtCtx, &sessVars.RowEncoder checksums, writeBufs.RowValBuf = t.calcChecksums(sctx, recordID, checksumData, writeBufs.RowValBuf) - writeBufs.RowValBuf, err = tablecodec.EncodeRow(sc, row, colIDs, writeBufs.RowValBuf, writeBufs.AddRowValues, rd, checksums...) + writeBufs.RowValBuf, err = tablecodec.EncodeRow(sc.TimeZone(), row, colIDs, writeBufs.RowValBuf, writeBufs.AddRowValues, rd, checksums...) + err = sc.HandleError(err) if err != nil { return nil, err } @@ -1395,7 +1397,8 @@ func (t *TableCommon) addInsertBinlog(ctx sessionctx.Context, h kv.Handle, row [ if err != nil { return err } - value, err := tablecodec.EncodeOldRow(ctx.GetSessionVars().StmtCtx, row, colIDs, nil, nil) + value, err := tablecodec.EncodeOldRow(ctx.GetSessionVars().StmtCtx.TimeZone(), row, colIDs, nil, nil) + err = ctx.GetSessionVars().StmtCtx.HandleError(err) if err != nil { return err } @@ -1406,11 +1409,13 @@ func (t *TableCommon) addInsertBinlog(ctx sessionctx.Context, h kv.Handle, row [ } func (t *TableCommon) addUpdateBinlog(ctx sessionctx.Context, oldRow, newRow []types.Datum, colIDs []int64) error { - old, err := tablecodec.EncodeOldRow(ctx.GetSessionVars().StmtCtx, oldRow, colIDs, nil, nil) + old, err := tablecodec.EncodeOldRow(ctx.GetSessionVars().StmtCtx.TimeZone(), oldRow, colIDs, nil, nil) + err = ctx.GetSessionVars().StmtCtx.HandleError(err) if err != nil { return err } - newVal, err := tablecodec.EncodeOldRow(ctx.GetSessionVars().StmtCtx, newRow, colIDs, nil, nil) + newVal, err := tablecodec.EncodeOldRow(ctx.GetSessionVars().StmtCtx.TimeZone(), newRow, colIDs, nil, nil) + err = ctx.GetSessionVars().StmtCtx.HandleError(err) if err != nil { return err } @@ -1422,7 +1427,8 @@ func (t *TableCommon) addUpdateBinlog(ctx sessionctx.Context, oldRow, newRow []t } func (t *TableCommon) addDeleteBinlog(ctx sessionctx.Context, r []types.Datum, colIDs []int64) error { - data, err := tablecodec.EncodeOldRow(ctx.GetSessionVars().StmtCtx, r, colIDs, nil, nil) + data, err := tablecodec.EncodeOldRow(ctx.GetSessionVars().StmtCtx.TimeZone(), r, colIDs, nil, nil) + err = ctx.GetSessionVars().StmtCtx.HandleError(err) if err != nil { return err } @@ -2316,7 +2322,8 @@ func SetPBColumnsDefaultValue(ctx sessionctx.Context, pbColumns []*tipb.ColumnIn return err } - pbColumns[i].DefaultVal, err = tablecodec.EncodeValue(sessVars.StmtCtx, nil, d) + pbColumns[i].DefaultVal, err = tablecodec.EncodeValue(sessVars.StmtCtx.TimeZone(), nil, d) + err = sessVars.StmtCtx.HandleError(err) if err != nil { return err } diff --git a/pkg/tablecodec/BUILD.bazel b/pkg/tablecodec/BUILD.bazel index 14474ee75b26a..8c891275c29f0 100644 --- a/pkg/tablecodec/BUILD.bazel +++ b/pkg/tablecodec/BUILD.bazel @@ -6,14 +6,12 @@ go_library( importpath = "github.com/pingcap/tidb/pkg/tablecodec", visibility = ["//visibility:public"], deps = [ - "//pkg/errctx", "//pkg/errno", "//pkg/kv", "//pkg/parser/charset", "//pkg/parser/model", "//pkg/parser/mysql", "//pkg/parser/terror", - "//pkg/sessionctx/stmtctx", "//pkg/structure", "//pkg/types", "//pkg/util/codec", diff --git a/pkg/tablecodec/tablecodec.go b/pkg/tablecodec/tablecodec.go index 2220d29761c1e..347d71221cbe0 100644 --- a/pkg/tablecodec/tablecodec.go +++ b/pkg/tablecodec/tablecodec.go @@ -24,14 +24,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/tidb/pkg/errctx" "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/structure" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/codec" @@ -324,22 +322,16 @@ func DecodeRowKey(key kv.Key) (kv.Handle, error) { } // EncodeValue encodes a go value to bytes. -func EncodeValue(sc *stmtctx.StatementContext, b []byte, raw types.Datum) ([]byte, error) { +// This function may return both a valid encoded bytes and an error (actually `"pingcap/errors".ErrorGroup`). If the caller +// expects to handle these errors according to `SQL_MODE` or other configuration, please refer to `pkg/errctx`. +func EncodeValue(loc *time.Location, b []byte, raw types.Datum) ([]byte, error) { var v types.Datum - err := flatten(sc, raw, &v) + err := flatten(loc, raw, &v) if err != nil { return nil, err } - // `sc` is possible to be `nil` here. - tz := time.UTC - errCtx := errctx.StrictNoWarningContext - if sc != nil { - tz = sc.TimeZone() - errCtx = sc.ErrCtx() - } - val, err := codec.EncodeValue(tz, b, v) - err = errCtx.HandleError(err) + val, err := codec.EncodeValue(loc, b, v) return val, err } @@ -347,22 +339,24 @@ func EncodeValue(sc *stmtctx.StatementContext, b []byte, raw types.Datum) ([]byt // EncodeRow encode row data and column ids into a slice of byte. // valBuf and values pass by caller, for reducing EncodeRow allocates temporary bufs. If you pass valBuf and values as nil, // EncodeRow will allocate it. -func EncodeRow(sc *stmtctx.StatementContext, row []types.Datum, colIDs []int64, valBuf []byte, values []types.Datum, e *rowcodec.Encoder, checksums ...uint32) ([]byte, error) { +// This function may return both a valid encoded bytes and an error (actually `"pingcap/errors".ErrorGroup`). If the caller +// expects to handle these errors according to `SQL_MODE` or other configuration, please refer to `pkg/errctx`. +func EncodeRow(loc *time.Location, row []types.Datum, colIDs []int64, valBuf []byte, values []types.Datum, e *rowcodec.Encoder, checksums ...uint32) ([]byte, error) { if len(row) != len(colIDs) { return nil, errors.Errorf("EncodeRow error: data and columnID count not match %d vs %d", len(row), len(colIDs)) } if e.Enable { valBuf = valBuf[:0] - return e.Encode(sc, colIDs, row, valBuf, checksums...) + return e.Encode(loc, colIDs, row, valBuf, checksums...) } - return EncodeOldRow(sc, row, colIDs, valBuf, values) + return EncodeOldRow(loc, row, colIDs, valBuf, values) } // EncodeOldRow encode row data and column ids into a slice of byte. // Row layout: colID1, value1, colID2, value2, ..... // valBuf and values pass by caller, for reducing EncodeOldRow allocates temporary bufs. If you pass valBuf and values as nil, // EncodeOldRow will allocate it. -func EncodeOldRow(sc *stmtctx.StatementContext, row []types.Datum, colIDs []int64, valBuf []byte, values []types.Datum) ([]byte, error) { +func EncodeOldRow(loc *time.Location, row []types.Datum, colIDs []int64, valBuf []byte, values []types.Datum) ([]byte, error) { if len(row) != len(colIDs) { return nil, errors.Errorf("EncodeRow error: data and columnID count not match %d vs %d", len(row), len(colIDs)) } @@ -373,7 +367,7 @@ func EncodeOldRow(sc *stmtctx.StatementContext, row []types.Datum, colIDs []int6 for i, c := range row { id := colIDs[i] values[2*i].SetInt64(id) - err := flatten(sc, c, &values[2*i+1]) + err := flatten(loc, c, &values[2*i+1]) if err != nil { return valBuf, errors.Trace(err) } @@ -382,16 +376,16 @@ func EncodeOldRow(sc *stmtctx.StatementContext, row []types.Datum, colIDs []int6 // We could not set nil value into kv. return append(valBuf, codec.NilFlag), nil } - return codec.EncodeValue(sc.TimeZone(), valBuf, values...) + return codec.EncodeValue(loc, valBuf, values...) } -func flatten(sc *stmtctx.StatementContext, data types.Datum, ret *types.Datum) error { +func flatten(loc *time.Location, data types.Datum, ret *types.Datum) error { switch data.Kind() { case types.KindMysqlTime: // for mysql datetime, timestamp and date type t := data.GetMysqlTime() - if t.Type() == mysql.TypeTimestamp && sc.TimeZone() != time.UTC { - err := t.ConvertTimeZone(sc.TimeZone(), time.UTC) + if t.Type() == mysql.TypeTimestamp && loc != nil && loc != time.UTC { + err := t.ConvertTimeZone(loc, time.UTC) if err != nil { return errors.Trace(err) } @@ -411,7 +405,7 @@ func flatten(sc *stmtctx.StatementContext, data types.Datum, ret *types.Datum) e return nil case types.KindBinaryLiteral, types.KindMysqlBit: // We don't need to handle errors here since the literal is ensured to be able to store in uint64 in convertToMysqlBit. - val, err := data.GetBinaryLiteral().ToInt(sc.TypeCtx()) + val, err := data.GetBinaryLiteral().ToInt(types.StrictContext) if err != nil { return errors.Trace(err) } @@ -1123,7 +1117,7 @@ func GetIndexKeyBuf(buf []byte, defaultCap int) []byte { } // GenIndexKey generates index key using input physical table id -func GenIndexKey(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, +func GenIndexKey(loc *time.Location, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, phyTblID int64, indexedValues []types.Datum, h kv.Handle, buf []byte) (key []byte, distinct bool, err error) { if idxInfo.Unique { // See https://dev.mysql.com/doc/refman/5.7/en/create-index.html @@ -1144,7 +1138,7 @@ func GenIndexKey(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo key = GetIndexKeyBuf(buf, RecordRowKeyLen+len(indexedValues)*9+9) key = appendTableIndexPrefix(key, phyTblID) key = codec.EncodeInt(key, idxInfo.ID) - key, err = codec.EncodeKey(sc.TimeZone(), key, indexedValues...) + key, err = codec.EncodeKey(loc, key, indexedValues...) if err != nil { return nil, false, err } @@ -1440,13 +1434,13 @@ func TempIndexValueIsUntouched(b []byte) bool { // | In v5.0, restored data contains only non-binary data(except for char and _bin). In the above example, the restored data contains only the value of b. // | Besides, if the collation of b is _bin, then restored data is an integer indicate the spaces are truncated. Then we use sortKey // | and the restored data together to restore original data. -func GenIndexValuePortal(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, +func GenIndexValuePortal(loc *time.Location, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, needRestoredData bool, distinct bool, untouched bool, indexedValues []types.Datum, h kv.Handle, partitionID int64, restoredData []types.Datum, buf []byte) ([]byte, error) { if tblInfo.IsCommonHandle && tblInfo.CommonHandleVersion == 1 { - return GenIndexValueForClusteredIndexVersion1(sc, tblInfo, idxInfo, needRestoredData, distinct, untouched, indexedValues, h, partitionID, restoredData, buf) + return GenIndexValueForClusteredIndexVersion1(loc, tblInfo, idxInfo, needRestoredData, distinct, untouched, indexedValues, h, partitionID, restoredData, buf) } - return genIndexValueVersion0(sc, tblInfo, idxInfo, needRestoredData, distinct, untouched, indexedValues, h, partitionID, buf) + return genIndexValueVersion0(loc, tblInfo, idxInfo, needRestoredData, distinct, untouched, indexedValues, h, partitionID, buf) } // TryGetCommonPkColumnRestoredIds get the IDs of primary key columns which need restored data if the table has common handle. @@ -1472,7 +1466,7 @@ func TryGetCommonPkColumnRestoredIds(tbl *model.TableInfo) []int64 { } // GenIndexValueForClusteredIndexVersion1 generates the index value for the clustered index with version 1(New in v5.0.0). -func GenIndexValueForClusteredIndexVersion1(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, +func GenIndexValueForClusteredIndexVersion1(loc *time.Location, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, idxValNeedRestoredData bool, distinct bool, untouched bool, indexedValues []types.Datum, h kv.Handle, partitionID int64, handleRestoredData []types.Datum, buf []byte) ([]byte, error) { var idxVal []byte @@ -1521,7 +1515,7 @@ func GenIndexValueForClusteredIndexVersion1(sc *stmtctx.StatementContext, tblInf rd := rowcodec.Encoder{Enable: true} var err error - idxVal, err = rd.Encode(sc, colIds, allRestoredData, idxVal) + idxVal, err = rd.Encode(loc, colIds, allRestoredData, idxVal) if err != nil { return nil, err } @@ -1537,7 +1531,7 @@ func GenIndexValueForClusteredIndexVersion1(sc *stmtctx.StatementContext, tblInf } // genIndexValueVersion0 create index value for both local and global index. -func genIndexValueVersion0(sc *stmtctx.StatementContext, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, +func genIndexValueVersion0(loc *time.Location, tblInfo *model.TableInfo, idxInfo *model.IndexInfo, idxValNeedRestoredData bool, distinct bool, untouched bool, indexedValues []types.Datum, h kv.Handle, partitionID int64, buf []byte) ([]byte, error) { var idxVal []byte @@ -1565,7 +1559,7 @@ func genIndexValueVersion0(sc *stmtctx.StatementContext, tblInfo *model.TableInf rd := rowcodec.Encoder{Enable: true} // Encode row restored value. var err error - idxVal, err = rd.Encode(sc, colIds, indexedValues, idxVal) + idxVal, err = rd.Encode(loc, colIds, indexedValues, idxVal) if err != nil { return nil, err } diff --git a/pkg/tablecodec/tablecodec_test.go b/pkg/tablecodec/tablecodec_test.go index ffb889c5f2f9d..89d9659619acd 100644 --- a/pkg/tablecodec/tablecodec_test.go +++ b/pkg/tablecodec/tablecodec_test.go @@ -100,7 +100,7 @@ func TestRowCodec(t *testing.T) { } rd := rowcodec.Encoder{Enable: true} sc := stmtctx.NewStmtCtxWithTimeZone(time.Local) - bs, err := EncodeRow(sc, row, colIDs, nil, nil, &rd) + bs, err := EncodeRow(sc.TimeZone(), row, colIDs, nil, nil, &rd) require.NoError(t, err) require.NotNil(t, bs) @@ -155,7 +155,7 @@ func TestRowCodec(t *testing.T) { } // Make sure empty row return not nil value. - bs, err = EncodeOldRow(sc, []types.Datum{}, []int64{}, nil, nil) + bs, err = EncodeOldRow(sc.TimeZone(), []types.Datum{}, []int64{}, nil, nil) require.NoError(t, err) require.Len(t, bs, 1) @@ -169,7 +169,7 @@ func TestDecodeColumnValue(t *testing.T) { // test timestamp d := types.NewTimeDatum(types.NewTime(types.FromGoTime(time.Now()), mysql.TypeTimestamp, types.DefaultFsp)) - bs, err := EncodeOldRow(sc, []types.Datum{d}, []int64{1}, nil, nil) + bs, err := EncodeOldRow(sc.TimeZone(), []types.Datum{d}, []int64{1}, nil, nil) require.NoError(t, err) require.NotNil(t, bs) _, bs, err = codec.CutOne(bs) // ignore colID @@ -185,7 +185,7 @@ func TestDecodeColumnValue(t *testing.T) { elems := []string{"a", "b", "c", "d", "e"} e, _ := types.ParseSetValue(elems, uint64(1)) d = types.NewMysqlSetDatum(e, "") - bs, err = EncodeOldRow(sc, []types.Datum{d}, []int64{1}, nil, nil) + bs, err = EncodeOldRow(sc.TimeZone(), []types.Datum{d}, []int64{1}, nil, nil) require.NoError(t, err) require.NotNil(t, bs) _, bs, err = codec.CutOne(bs) // ignore colID @@ -200,7 +200,7 @@ func TestDecodeColumnValue(t *testing.T) { // test bit d = types.NewMysqlBitDatum(types.NewBinaryLiteralFromUint(3223600, 3)) - bs, err = EncodeOldRow(sc, []types.Datum{d}, []int64{1}, nil, nil) + bs, err = EncodeOldRow(sc.TimeZone(), []types.Datum{d}, []int64{1}, nil, nil) require.NoError(t, err) require.NotNil(t, bs) _, bs, err = codec.CutOne(bs) // ignore colID @@ -215,7 +215,7 @@ func TestDecodeColumnValue(t *testing.T) { // test empty enum d = types.NewMysqlEnumDatum(types.Enum{}) - bs, err = EncodeOldRow(sc, []types.Datum{d}, []int64{1}, nil, nil) + bs, err = EncodeOldRow(sc.TimeZone(), []types.Datum{d}, []int64{1}, nil, nil) require.NoError(t, err) require.NotNil(t, bs) _, bs, err = codec.CutOne(bs) // ignore colID @@ -275,7 +275,7 @@ func TestTimeCodec(t *testing.T) { } rd := rowcodec.Encoder{Enable: true} sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) - bs, err := EncodeRow(sc, row, colIDs, nil, nil, &rd) + bs, err := EncodeRow(sc.TimeZone(), row, colIDs, nil, nil, &rd) require.NoError(t, err) require.NotNil(t, bs) @@ -312,18 +312,18 @@ func TestCutRow(t *testing.T) { sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) data := make([][]byte, 3) - data[0], err = EncodeValue(sc, nil, row[0]) + data[0], err = EncodeValue(sc.TimeZone(), nil, row[0]) require.NoError(t, err) - data[1], err = EncodeValue(sc, nil, row[1]) + data[1], err = EncodeValue(sc.TimeZone(), nil, row[1]) require.NoError(t, err) - data[2], err = EncodeValue(sc, nil, row[2]) + data[2], err = EncodeValue(sc.TimeZone(), nil, row[2]) require.NoError(t, err) // Encode colIDs := make([]int64, 0, 3) for _, col := range cols { colIDs = append(colIDs, col.id) } - bs, err := EncodeOldRow(sc, row, colIDs, nil, nil) + bs, err := EncodeOldRow(sc.TimeZone(), row, colIDs, nil, nil) require.NoError(t, err) require.NotNil(t, bs) @@ -567,7 +567,7 @@ func BenchmarkEncodeValue(b *testing.B) { for i := 0; i < b.N; i++ { for _, d := range row { encodedCol = encodedCol[:0] - _, err := EncodeValue(sc, encodedCol, d) + _, err := EncodeValue(sc.TimeZone(), encodedCol, d) if err != nil { b.Fatal(err) } diff --git a/pkg/util/rowDecoder/decoder_test.go b/pkg/util/rowDecoder/decoder_test.go index 78eb540bedbf5..3f8cadb2fac3b 100644 --- a/pkg/util/rowDecoder/decoder_test.go +++ b/pkg/util/rowDecoder/decoder_test.go @@ -112,7 +112,7 @@ func TestRowDecoder(t *testing.T) { if i > 0 { c7.AddFlag(mysql.UnsignedFlag) } - bs, err := tablecodec.EncodeRow(sc, row.input, row.cols, nil, nil, &rd) + bs, err := tablecodec.EncodeRow(sc.TimeZone(), row.input, row.cols, nil, nil, &rd) require.NoError(t, err) require.NotNil(t, bs) @@ -187,7 +187,7 @@ func TestClusterIndexRowDecoder(t *testing.T) { } rd := rowcodec.Encoder{Enable: true} for _, row := range testRows { - bs, err := tablecodec.EncodeRow(sc, row.input, row.cols, nil, nil, &rd) + bs, err := tablecodec.EncodeRow(sc.TimeZone(), row.input, row.cols, nil, nil, &rd) require.NoError(t, err) require.NotNil(t, bs) diff --git a/pkg/util/rowcodec/BUILD.bazel b/pkg/util/rowcodec/BUILD.bazel index 42eb49b739d7d..22f366b5ac02e 100644 --- a/pkg/util/rowcodec/BUILD.bazel +++ b/pkg/util/rowcodec/BUILD.bazel @@ -14,13 +14,12 @@ go_library( "//pkg/kv", "//pkg/parser/model", "//pkg/parser/mysql", - "//pkg/parser/terror", "//pkg/parser/types", - "//pkg/sessionctx/stmtctx", "//pkg/types", "//pkg/util/chunk", "//pkg/util/codec", "@com_github_pingcap_errors//:errors", + "@org_uber_go_multierr//:multierr", ], ) diff --git a/pkg/util/rowcodec/bench_test.go b/pkg/util/rowcodec/bench_test.go index ef5698f27f9a7..1735b0c96fea6 100644 --- a/pkg/util/rowcodec/bench_test.go +++ b/pkg/util/rowcodec/bench_test.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/benchdaily" @@ -68,7 +67,7 @@ func BenchmarkEncode(b *testing.B) { func BenchmarkEncodeFromOldRow(b *testing.B) { b.ReportAllocs() oldRow := types.MakeDatums(1, "abc", 1.1) - oldRowData, err := tablecodec.EncodeOldRow(stmtctx.NewStmtCtx(), oldRow, []int64{1, 2, 3}, nil, nil) + oldRowData, err := tablecodec.EncodeOldRow(nil, oldRow, []int64{1, 2, 3}, nil, nil) if err != nil { b.Fatal(err) } diff --git a/pkg/util/rowcodec/encoder.go b/pkg/util/rowcodec/encoder.go index 0f72745045f20..a725537a96028 100644 --- a/pkg/util/rowcodec/encoder.go +++ b/pkg/util/rowcodec/encoder.go @@ -21,10 +21,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/parser/terror" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/codec" + "go.uber.org/multierr" ) // Encoder is used to encode a row. @@ -38,16 +37,15 @@ type Encoder struct { // Encode encodes a row from a datums slice. // `buf` is not truncated before encoding. -func (encoder *Encoder) Encode(sc *stmtctx.StatementContext, colIDs []int64, values []types.Datum, buf []byte, checksums ...uint32) ([]byte, error) { +// This function may return both a valid encoded bytes and an error (actually `"pingcap/errors".ErrorGroup`). If the caller +// expects to handle these errors according to `SQL_MODE` or other configuration, please refer to `pkg/errctx`. +func (encoder *Encoder) Encode(loc *time.Location, colIDs []int64, values []types.Datum, buf []byte, checksums ...uint32) ([]byte, error) { encoder.reset() encoder.appendColVals(colIDs, values) numCols, notNullIdx := encoder.reformatCols() - err := encoder.encodeRowCols(sc, numCols, notNullIdx) - if err != nil { - return nil, err - } + err := encoder.encodeRowCols(loc, numCols, notNullIdx) encoder.setChecksums(checksums...) - return encoder.row.toBytes(buf), nil + return encoder.row.toBytes(buf), err } func (encoder *Encoder) reset() { @@ -131,14 +129,15 @@ func (encoder *Encoder) reformatCols() (numCols, notNullIdx int) { return } -func (encoder *Encoder) encodeRowCols(sc *stmtctx.StatementContext, numCols, notNullIdx int) error { +func (encoder *Encoder) encodeRowCols(loc *time.Location, numCols, notNullIdx int) error { r := &encoder.row + var errs error for i := 0; i < notNullIdx; i++ { d := encoder.values[i] var err error - r.data, err = encodeValueDatum(sc, d, r.data) + r.data, err = encodeValueDatum(loc, d, r.data) if err != nil { - return err + errs = multierr.Append(errs, err) } // handle convert to large if len(r.data) > math.MaxUint16 && !r.large() { @@ -158,12 +157,12 @@ func (encoder *Encoder) encodeRowCols(sc *stmtctx.StatementContext, numCols, not r.offsets[i] = uint16(len(r.data)) } } - return nil + return errs } // encodeValueDatum encodes one row datum entry into bytes. // due to encode as value, this method will flatten value type like tablecodec.flatten -func encodeValueDatum(sc *stmtctx.StatementContext, d *types.Datum, buffer []byte) (nBuffer []byte, err error) { +func encodeValueDatum(loc *time.Location, d *types.Datum, buffer []byte) (nBuffer []byte, err error) { switch d.Kind() { case types.KindInt64: buffer = encodeInt(buffer, d.GetInt64()) @@ -174,8 +173,8 @@ func encodeValueDatum(sc *stmtctx.StatementContext, d *types.Datum, buffer []byt case types.KindMysqlTime: // for mysql datetime, timestamp and date type t := d.GetMysqlTime() - if t.Type() == mysql.TypeTimestamp && sc != nil && sc.TimeZone() != time.UTC { - err = t.ConvertTimeZone(sc.TimeZone(), time.UTC) + if t.Type() == mysql.TypeTimestamp && loc != nil && loc != time.UTC { + err = t.ConvertTimeZone(loc, time.UTC) if err != nil { return } @@ -195,7 +194,7 @@ func encodeValueDatum(sc *stmtctx.StatementContext, d *types.Datum, buffer []byt case types.KindBinaryLiteral, types.KindMysqlBit: // We don't need to handle errors here since the literal is ensured to be able to store in uint64 in convertToMysqlBit. var val uint64 - val, err = d.GetBinaryLiteral().ToInt(sc.TypeCtxOrDefault()) + val, err = d.GetBinaryLiteral().ToInt(types.StrictContext) if err != nil { return } @@ -204,13 +203,6 @@ func encodeValueDatum(sc *stmtctx.StatementContext, d *types.Datum, buffer []byt buffer = codec.EncodeFloat(buffer, d.GetFloat64()) case types.KindMysqlDecimal: buffer, err = codec.EncodeDecimal(buffer, d.GetMysqlDecimal(), d.Length(), d.Frac()) - if err != nil && sc != nil { - if terror.ErrorEqual(err, types.ErrTruncated) { - err = sc.HandleTruncate(err) - } else if terror.ErrorEqual(err, types.ErrOverflow) { - err = sc.HandleOverflow(err, err) - } - } case types.KindMysqlJSON: j := d.GetMysqlJSON() buffer = append(buffer, j.TypeCode) diff --git a/pkg/util/rowcodec/main_test.go b/pkg/util/rowcodec/main_test.go index d6c88fe255ad7..a59f3a2efe399 100644 --- a/pkg/util/rowcodec/main_test.go +++ b/pkg/util/rowcodec/main_test.go @@ -16,8 +16,8 @@ package rowcodec import ( "testing" + "time" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/testkit/testsetup" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/codec" @@ -36,7 +36,7 @@ func TestMain(m *testing.M) { // EncodeFromOldRow encodes a row from an old-format row. // this method will be used in test. -func EncodeFromOldRow(encoder *Encoder, sc *stmtctx.StatementContext, oldRow, buf []byte) ([]byte, error) { +func EncodeFromOldRow(encoder *Encoder, loc *time.Location, oldRow, buf []byte) ([]byte, error) { if len(oldRow) > 0 && oldRow[0] == CodecVer { return oldRow, nil } @@ -56,7 +56,7 @@ func EncodeFromOldRow(encoder *Encoder, sc *stmtctx.StatementContext, oldRow, bu encoder.appendColVal(colID, &d) } numCols, notNullIdx := encoder.reformatCols() - err := encoder.encodeRowCols(sc, numCols, notNullIdx) + err := encoder.encodeRowCols(loc, numCols, notNullIdx) if err != nil { return nil, err } diff --git a/pkg/util/rowcodec/rowcodec_test.go b/pkg/util/rowcodec/rowcodec_test.go index 94f49bf3c3abd..700bb9276d73e 100644 --- a/pkg/util/rowcodec/rowcodec_test.go +++ b/pkg/util/rowcodec/rowcodec_test.go @@ -51,7 +51,7 @@ func TestEncodeLargeSmallReuseBug(t *testing.T) { colFt := types.NewFieldType(mysql.TypeString) largeColID := int64(300) - b, err := encoder.Encode(stmtctx.NewStmtCtx(), []int64{largeColID}, []types.Datum{types.NewBytesDatum([]byte(""))}, nil) + b, err := encoder.Encode(nil, []int64{largeColID}, []types.Datum{types.NewBytesDatum([]byte(""))}, nil) require.NoError(t, err) bDecoder := rowcodec.NewDatumMapDecoder([]rowcodec.ColInfo{ @@ -66,7 +66,7 @@ func TestEncodeLargeSmallReuseBug(t *testing.T) { colFt = types.NewFieldType(mysql.TypeLonglong) smallColID := int64(1) - b, err = encoder.Encode(stmtctx.NewStmtCtx(), []int64{smallColID}, []types.Datum{types.NewIntDatum(2)}, nil) + b, err = encoder.Encode(nil, []int64{smallColID}, []types.Datum{types.NewIntDatum(2)}, nil) require.NoError(t, err) bDecoder = rowcodec.NewDatumMapDecoder([]rowcodec.ColInfo{ @@ -162,16 +162,15 @@ func TestDecodeRowWithHandle(t *testing.T) { // test encode input. var encoder rowcodec.Encoder - sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) - newRow, err := encoder.Encode(sc, colIDs, dts, nil) + newRow, err := encoder.Encode(time.UTC, colIDs, dts, nil) require.NoError(t, err) // decode to datum map. - mDecoder := rowcodec.NewDatumMapDecoder(cols, sc.TimeZone()) + mDecoder := rowcodec.NewDatumMapDecoder(cols, time.UTC) dm, err := mDecoder.DecodeToDatumMap(newRow, nil) require.NoError(t, err) - dm, err = tablecodec.DecodeHandleToDatumMap(kv.IntHandle(handleValue), []int64{handleID}, handleColFtMap, sc.TimeZone(), dm) + dm, err = tablecodec.DecodeHandleToDatumMap(kv.IntHandle(handleValue), []int64{handleID}, handleColFtMap, time.UTC, dm) require.NoError(t, err) for _, d := range td { @@ -181,7 +180,7 @@ func TestDecodeRowWithHandle(t *testing.T) { } // decode to chunk. - cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone()) + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, time.UTC) chk := chunk.New(fts, 1, 1) err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(handleValue), chk) require.NoError(t, err) @@ -222,7 +221,6 @@ func TestDecodeRowWithHandle(t *testing.T) { func TestEncodeKindNullDatum(t *testing.T) { var encoder rowcodec.Encoder - sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) colIDs := []int64{1, 2} var nilDt types.Datum @@ -230,11 +228,11 @@ func TestEncodeKindNullDatum(t *testing.T) { dts := []types.Datum{nilDt, types.NewIntDatum(2)} ft := types.NewFieldType(mysql.TypeLonglong) fts := []*types.FieldType{ft, ft} - newRow, err := encoder.Encode(sc, colIDs, dts, nil) + newRow, err := encoder.Encode(time.UTC, colIDs, dts, nil) require.NoError(t, err) cols := []rowcodec.ColInfo{{ID: 1, Ft: ft}, {ID: 2, Ft: ft}} - cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone()) + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, time.UTC) chk := chunk.New(fts, 1, 1) err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) require.NoError(t, err) @@ -247,7 +245,6 @@ func TestEncodeKindNullDatum(t *testing.T) { func TestDecodeDecimalFspNotMatch(t *testing.T) { var encoder rowcodec.Encoder - sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) colIDs := []int64{ 1, } @@ -256,7 +253,7 @@ func TestDecodeDecimalFspNotMatch(t *testing.T) { ft := types.NewFieldType(mysql.TypeNewDecimal) ft.SetDecimal(4) fts := []*types.FieldType{ft} - newRow, err := encoder.Encode(sc, colIDs, dts, nil) + newRow, err := encoder.Encode(time.UTC, colIDs, dts, nil) require.NoError(t, err) // decode to chunk. @@ -267,7 +264,7 @@ func TestDecodeDecimalFspNotMatch(t *testing.T) { ID: 1, Ft: ft, }) - cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone()) + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, time.UTC) chk := chunk.New(fts, 1, 1) err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) require.NoError(t, err) @@ -512,12 +509,11 @@ func TestTypesNewRowCodec(t *testing.T) { } // test encode input. - sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) - newRow, err := encoder.Encode(sc, colIDs, dts, nil) + newRow, err := encoder.Encode(time.UTC, colIDs, dts, nil) require.NoError(t, err) // decode to datum map. - mDecoder := rowcodec.NewDatumMapDecoder(cols, sc.TimeZone()) + mDecoder := rowcodec.NewDatumMapDecoder(cols, time.UTC) dm, err := mDecoder.DecodeToDatumMap(newRow, nil) require.NoError(t, err) @@ -528,7 +524,7 @@ func TestTypesNewRowCodec(t *testing.T) { } // decode to chunk. - cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone()) + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, time.UTC) chk := chunk.New(fts, 1, 1) err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) require.NoError(t, err) @@ -626,12 +622,11 @@ func TestNilAndDefault(t *testing.T) { // test encode input. var encoder rowcodec.Encoder - sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) - newRow, err := encoder.Encode(sc, colIDs, dts, nil) + newRow, err := encoder.Encode(time.UTC, colIDs, dts, nil) require.NoError(t, err) // decode to datum map. - mDecoder := rowcodec.NewDatumMapDecoder(cols, sc.TimeZone()) + mDecoder := rowcodec.NewDatumMapDecoder(cols, time.UTC) dm, err := mDecoder.DecodeToDatumMap(newRow, nil) require.NoError(t, err) @@ -648,7 +643,7 @@ func TestNilAndDefault(t *testing.T) { // decode to chunk. chk := chunk.New(fts, 1, 1) - cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, ddf, sc.TimeZone()) + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, ddf, time.UTC) err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) require.NoError(t, err) @@ -664,7 +659,7 @@ func TestNilAndDefault(t *testing.T) { } chk = chunk.New(fts, 1, 1) - cDecoder = rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone()) + cDecoder = rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, time.UTC) err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) require.NoError(t, err) @@ -682,7 +677,7 @@ func TestNilAndDefault(t *testing.T) { for i, t := range td { colOffset[t.id] = i } - bDecoder := rowcodec.NewByteDecoder(cols, []int64{-1}, bdf, sc.TimeZone()) + bDecoder := rowcodec.NewByteDecoder(cols, []int64{-1}, bdf, time.UTC) oldRow, err := bDecoder.DecodeToBytes(colOffset, kv.IntHandle(-1), newRow, nil) require.NoError(t, err) @@ -735,11 +730,10 @@ func TestVarintCompatibility(t *testing.T) { // test encode input. var encoder rowcodec.Encoder - sc := stmtctx.NewStmtCtxWithTimeZone(time.UTC) - newRow, err := encoder.Encode(sc, colIDs, dts, nil) + newRow, err := encoder.Encode(time.UTC, colIDs, dts, nil) require.NoError(t, err) - decoder := rowcodec.NewByteDecoder(cols, []int64{-1}, nil, sc.TimeZone()) + decoder := rowcodec.NewByteDecoder(cols, []int64{-1}, nil, time.UTC) // decode to old row bytes. colOffset := make(map[int64]int) for i, t := range td { @@ -763,7 +757,7 @@ func TestCodecUtil(t *testing.T) { } tps[3] = types.NewFieldType(mysql.TypeNull) sc := stmtctx.NewStmtCtx() - oldRow, err := tablecodec.EncodeOldRow(sc, types.MakeDatums(1, 2, 3, nil), colIDs, nil, nil) + oldRow, err := tablecodec.EncodeOldRow(sc.TimeZone(), types.MakeDatums(1, 2, 3, nil), colIDs, nil, nil) require.NoError(t, err) var ( @@ -813,7 +807,7 @@ func TestOldRowCodec(t *testing.T) { } tps[3] = types.NewFieldType(mysql.TypeNull) sc := stmtctx.NewStmtCtx() - oldRow, err := tablecodec.EncodeOldRow(sc, types.MakeDatums(1, 2, 3, nil), colIDs, nil, nil) + oldRow, err := tablecodec.EncodeOldRow(sc.TimeZone(), types.MakeDatums(1, 2, 3, nil), colIDs, nil, nil) require.NoError(t, err) var ( @@ -844,10 +838,9 @@ func Test65535Bug(t *testing.T) { colIds := []int64{1} tps := make([]*types.FieldType, 1) tps[0] = types.NewFieldType(mysql.TypeString) - sc := stmtctx.NewStmtCtx() text65535 := strings.Repeat("a", 65535) encode := rowcodec.Encoder{} - bd, err := encode.Encode(sc, colIds, []types.Datum{types.NewStringDatum(text65535)}, nil) + bd, err := encode.Encode(time.UTC, colIds, []types.Datum{types.NewStringDatum(text65535)}, nil) require.NoError(t, err) cols := make([]rowcodec.ColInfo, 1) @@ -1195,7 +1188,6 @@ func TestRowChecksum(t *testing.T) { } func TestEncodeDecodeRowWithChecksum(t *testing.T) { - sc := stmtctx.NewStmtCtx() enc := rowcodec.Encoder{} for _, tt := range []struct { @@ -1208,9 +1200,9 @@ func TestEncodeDecodeRowWithChecksum(t *testing.T) { {"ThreeChecksum", []uint32{1, 2, 3}}, } { t.Run(tt.name, func(t *testing.T) { - raw, err := enc.Encode(sc, nil, nil, nil, tt.checksums...) + raw, err := enc.Encode(time.UTC, nil, nil, nil, tt.checksums...) require.NoError(t, err) - dec := rowcodec.NewDatumMapDecoder([]rowcodec.ColInfo{}, sc.TimeZone()) + dec := rowcodec.NewDatumMapDecoder([]rowcodec.ColInfo{}, time.UTC) _, err = dec.DecodeToDatumMap(raw, nil) require.NoError(t, err) v1, ok1 := enc.GetChecksum() @@ -1245,9 +1237,9 @@ func TestEncodeDecodeRowWithChecksum(t *testing.T) { } t.Run("ReuseDecoder", func(t *testing.T) { - dec := rowcodec.NewDatumMapDecoder([]rowcodec.ColInfo{}, sc.TimeZone()) + dec := rowcodec.NewDatumMapDecoder([]rowcodec.ColInfo{}, time.UTC) - raw1, err := enc.Encode(sc, nil, nil, nil) + raw1, err := enc.Encode(time.UTC, nil, nil, nil) require.NoError(t, err) _, err = dec.DecodeToDatumMap(raw1, nil) require.NoError(t, err) @@ -1258,7 +1250,7 @@ func TestEncodeDecodeRowWithChecksum(t *testing.T) { require.Zero(t, v1) require.Zero(t, v2) - raw2, err := enc.Encode(sc, nil, nil, nil, 1, 2) + raw2, err := enc.Encode(time.UTC, nil, nil, nil, 1, 2) require.NoError(t, err) _, err = dec.DecodeToDatumMap(raw2, nil) require.NoError(t, err) @@ -1269,7 +1261,7 @@ func TestEncodeDecodeRowWithChecksum(t *testing.T) { require.Equal(t, uint32(1), v1) require.Equal(t, uint32(2), v2) - raw3, err := enc.Encode(sc, nil, nil, nil, 1) + raw3, err := enc.Encode(time.UTC, nil, nil, nil, 1) require.NoError(t, err) _, err = dec.DecodeToDatumMap(raw3, nil) require.NoError(t, err) diff --git a/tests/realtikvtest/addindextest1/disttask_test.go b/tests/realtikvtest/addindextest1/disttask_test.go index 27715a0613ecb..54a3ee5d32e6c 100644 --- a/tests/realtikvtest/addindextest1/disttask_test.go +++ b/tests/realtikvtest/addindextest1/disttask_test.go @@ -271,7 +271,7 @@ func TestAddIndexTSErrorWhenResetImportEngine(t *testing.T) { dts := []types.Datum{types.NewIntDatum(1)} sctx := tk.Session().GetSessionVars().StmtCtx - idxKey, _, err := tablecodec.GenIndexKey(sctx, tblInfo, idxInfo, tblInfo.ID, dts, kv.IntHandle(1), nil) + idxKey, _, err := tablecodec.GenIndexKey(sctx.TimeZone(), tblInfo, idxInfo, tblInfo.ID, dts, kv.IntHandle(1), nil) require.NoError(t, err) tikvStore := dom.Store().(helper.Storage) From 7985dc4b4647b2ad65a1400fe8a80d653b3d02df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Thu, 23 Nov 2023 19:02:43 +0800 Subject: [PATCH 29/36] expression: add method `CanonicalHashCode` to `Expression` to decouple it with `stmtctx` (#48666) close pingcap/tidb#48665 --- pkg/expression/column.go | 7 +++- pkg/expression/column_test.go | 4 +- pkg/expression/constant.go | 17 +++++++- pkg/expression/constant_propagation.go | 2 +- pkg/expression/expression.go | 7 +++- pkg/expression/expression_test.go | 2 +- pkg/expression/grouping_sets.go | 12 ++---- pkg/expression/grouping_sets_test.go | 6 +-- .../integration_test/integration_test.go | 3 +- pkg/expression/scalar_function.go | 39 +++++++++---------- pkg/expression/scalar_function_test.go | 28 ++++++------- pkg/expression/util.go | 10 ++--- pkg/expression/util_test.go | 3 +- pkg/planner/cascades/transformation_rules.go | 6 +-- pkg/planner/core/hashcode.go | 4 +- pkg/planner/core/indexmerge_path.go | 16 ++++---- pkg/planner/core/logical_plan_builder.go | 10 ++--- pkg/planner/core/logical_plans.go | 35 ++++++----------- pkg/planner/core/rule_column_pruning.go | 2 +- pkg/planner/core/rule_constant_propagation.go | 4 +- pkg/planner/core/rule_eliminate_projection.go | 6 +-- .../core/rule_generate_column_substitute.go | 3 +- pkg/planner/core/rule_partition_processor.go | 2 +- pkg/planner/core/rule_predicate_push_down.go | 4 +- pkg/planner/core/scalar_subq_expression.go | 7 +++- pkg/planner/property/physical_property.go | 4 +- pkg/sessionctx/stmtctx/stmtctx.go | 2 - pkg/table/tables/partition.go | 2 +- 28 files changed, 122 insertions(+), 125 deletions(-) diff --git a/pkg/expression/column.go b/pkg/expression/column.go index ddbfcd052be92..70915747d1cd5 100644 --- a/pkg/expression/column.go +++ b/pkg/expression/column.go @@ -541,7 +541,7 @@ func (col *Column) Decorrelate(_ *Schema) Expression { } // HashCode implements Expression interface. -func (col *Column) HashCode(_ *stmtctx.StatementContext) []byte { +func (col *Column) HashCode() []byte { if len(col.hashcode) != 0 { return col.hashcode } @@ -551,6 +551,11 @@ func (col *Column) HashCode(_ *stmtctx.StatementContext) []byte { return col.hashcode } +// CanonicalHashCode implements Expression interface. +func (col *Column) CanonicalHashCode() []byte { + return col.HashCode() +} + // CleanHashCode will clean the hashcode you may be cached before. It's used especially in schema-cloned & reallocated-uniqueID's cases. func (col *Column) CleanHashCode() { col.hashcode = make([]byte, 0, 9) diff --git a/pkg/expression/column_test.go b/pkg/expression/column_test.go index db9870f2bb6af..8924ce7f2120f 100644 --- a/pkg/expression/column_test.go +++ b/pkg/expression/column_test.go @@ -102,12 +102,12 @@ func TestColumnHashCode(t *testing.T) { col1 := &Column{ UniqueID: 12, } - require.EqualValues(t, []byte{0x1, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc}, col1.HashCode(nil)) + require.EqualValues(t, []byte{0x1, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xc}, col1.HashCode()) col2 := &Column{ UniqueID: 2, } - require.EqualValues(t, []byte{0x1, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x2}, col2.HashCode(nil)) + require.EqualValues(t, []byte{0x1, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x2}, col2.HashCode()) } func TestColumn2Expr(t *testing.T) { diff --git a/pkg/expression/constant.go b/pkg/expression/constant.go index 33fd425559daa..b260db16a920d 100644 --- a/pkg/expression/constant.go +++ b/pkg/expression/constant.go @@ -448,13 +448,26 @@ func (c *Constant) Decorrelate(_ *Schema) Expression { } // HashCode implements Expression interface. -func (c *Constant) HashCode(sc *stmtctx.StatementContext) []byte { +func (c *Constant) HashCode() []byte { + return c.getHashCode(false) +} + +// CanonicalHashCode implements Expression interface. +func (c *Constant) CanonicalHashCode() []byte { + return c.getHashCode(true) +} + +func (c *Constant) getHashCode(canonical bool) []byte { if len(c.hashcode) > 0 { return c.hashcode } if c.DeferredExpr != nil { - c.hashcode = c.DeferredExpr.HashCode(sc) + if canonical { + c.hashcode = c.DeferredExpr.CanonicalHashCode() + } else { + c.hashcode = c.DeferredExpr.HashCode() + } return c.hashcode } diff --git a/pkg/expression/constant_propagation.go b/pkg/expression/constant_propagation.go index 10bcbba17b02f..c69b2d3867717 100644 --- a/pkg/expression/constant_propagation.go +++ b/pkg/expression/constant_propagation.go @@ -353,7 +353,7 @@ func (s *propConstSolver) solve(conditions []Expression) []Expression { s.propagateConstantEQ() s.propagateColumnEQ() s.conditions = propagateConstantDNF(s.ctx, s.conditions) - s.conditions = RemoveDupExprs(s.ctx, s.conditions) + s.conditions = RemoveDupExprs(s.conditions) return s.conditions } diff --git a/pkg/expression/expression.go b/pkg/expression/expression.go index 2b594dcc869ff..ff48cae555dff 100644 --- a/pkg/expression/expression.go +++ b/pkg/expression/expression.go @@ -196,7 +196,12 @@ type Expression interface { // Constant: ConstantFlag+encoded value // Column: ColumnFlag+encoded value // ScalarFunction: SFFlag+encoded function name + encoded arg_1 + encoded arg_2 + ... - HashCode(sc *stmtctx.StatementContext) []byte + HashCode() []byte + + // CanonicalHashCode creates the canonical hashcode for expression. + // Different with `HashCode`, this method will produce the same hashcode for expressions with the same semantic. + // For example, `a + b` and `b + a` have the same return value of this method. + CanonicalHashCode() []byte // MemoryUsage return the memory usage of Expression MemoryUsage() int64 diff --git a/pkg/expression/expression_test.go b/pkg/expression/expression_test.go index 3d2f2e860dfd8..11c12539a1b13 100644 --- a/pkg/expression/expression_test.go +++ b/pkg/expression/expression_test.go @@ -110,7 +110,7 @@ func TestConstant(t *testing.T) { require.False(t, NewZero().IsCorrelated()) require.True(t, NewZero().ConstItem(sc)) require.True(t, NewZero().Decorrelate(nil).Equal(ctx, NewZero())) - require.Equal(t, []byte{0x0, 0x8, 0x0}, NewZero().HashCode(sc)) + require.Equal(t, []byte{0x0, 0x8, 0x0}, NewZero().HashCode()) require.False(t, NewZero().Equal(ctx, NewOne())) res, err := NewZero().MarshalJSON() require.NoError(t, err) diff --git a/pkg/expression/grouping_sets.go b/pkg/expression/grouping_sets.go index ecd81d38019ce..8fb07a0a4afcc 100644 --- a/pkg/expression/grouping_sets.go +++ b/pkg/expression/grouping_sets.go @@ -19,7 +19,6 @@ import ( "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/util/intset" "github.com/pingcap/tidb/pkg/util/size" @@ -515,23 +514,18 @@ func AdjustNullabilityFromGroupingSets(gss GroupingSets, schema *Schema) { // eg: group by a+b, b+a, b with rollup. // the 1st and 2nd expression is semantically equivalent, so we only need to keep the distinct expression: [a+b, b] // down, and output another position slice out, the [0, 0, 1] for the case above. -func DeduplicateGbyExpression(ctx sessionctx.Context, exprs []Expression) ([]Expression, []int) { +func DeduplicateGbyExpression(exprs []Expression) ([]Expression, []int) { distinctExprs := make([]Expression, 0, len(exprs)) - sc := ctx.GetSessionVars().StmtCtx - sc.CanonicalHashCode = true - defer func() { - sc.CanonicalHashCode = false - }() distinctMap := make(map[string]int, len(exprs)) for _, expr := range exprs { // -1 means pos is not assigned yet. - distinctMap[string(expr.HashCode(sc))] = -1 + distinctMap[string(expr.CanonicalHashCode())] = -1 } // pos is from 0 to len(distinctMap)-1 pos := 0 posSlice := make([]int, 0, len(exprs)) for _, one := range exprs { - key := string(one.HashCode(sc)) + key := string(one.CanonicalHashCode()) if val, ok := distinctMap[key]; ok { if val == -1 { // means a new distinct expr. diff --git a/pkg/expression/grouping_sets_test.go b/pkg/expression/grouping_sets_test.go index 4f270cded4b97..68bf202860220 100644 --- a/pkg/expression/grouping_sets_test.go +++ b/pkg/expression/grouping_sets_test.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/types" - "github.com/pingcap/tidb/pkg/util/mock" "github.com/stretchr/testify/require" "go.opencensus.io/stats/view" ) @@ -327,8 +326,7 @@ func TestDistinctGroupingSets(t *testing.T) { // case1: non-duplicated case. // raw rollup expressions: [a,b,c,d] rawRollupExprs := []Expression{a, b, c, d} - mockCtx := mock.NewContext() - deduplicateExprs, pos := DeduplicateGbyExpression(mockCtx, rawRollupExprs) + deduplicateExprs, pos := DeduplicateGbyExpression(rawRollupExprs) // nothing to deduplicate. require.Equal(t, len(rawRollupExprs), len(deduplicateExprs)) @@ -352,7 +350,7 @@ func TestDistinctGroupingSets(t *testing.T) { // case2: duplicated case. rawRollupExprs = []Expression{a, b, b, c} - deduplicateExprs, pos = DeduplicateGbyExpression(mockCtx, rawRollupExprs) + deduplicateExprs, pos = DeduplicateGbyExpression(rawRollupExprs) require.Equal(t, len(deduplicateExprs), 3) require.Equal(t, deduplicateExprs[0].String(), "Column#1") require.Equal(t, deduplicateExprs[1].String(), "Column#2") diff --git a/pkg/expression/integration_test/integration_test.go b/pkg/expression/integration_test/integration_test.go index 07370262272f2..54dbcb557ad05 100644 --- a/pkg/expression/integration_test/integration_test.go +++ b/pkg/expression/integration_test/integration_test.go @@ -399,7 +399,6 @@ func TestFilterExtractFromDNF(t *testing.T) { for _, tt := range tests { sql := "select * from t where " + tt.exprStr sctx := tk.Session() - sc := sctx.GetSessionVars().StmtCtx stmts, err := session.Parse(sctx, sql) require.NoError(t, err, "error %v, for expr %s", err, tt.exprStr) require.Len(t, stmts, 1) @@ -415,7 +414,7 @@ func TestFilterExtractFromDNF(t *testing.T) { } afterFunc := expression.ExtractFiltersFromDNFs(sctx, conds) sort.Slice(afterFunc, func(i, j int) bool { - return bytes.Compare(afterFunc[i].HashCode(sc), afterFunc[j].HashCode(sc)) < 0 + return bytes.Compare(afterFunc[i].HashCode(), afterFunc[j].HashCode()) < 0 }) require.Equal(t, fmt.Sprintf("%s", afterFunc), tt.result, "wrong result for expr: %s", tt.exprStr) } diff --git a/pkg/expression/scalar_function.go b/pkg/expression/scalar_function.go index a797d21224181..27281648cb4a6 100644 --- a/pkg/expression/scalar_function.go +++ b/pkg/expression/scalar_function.go @@ -499,33 +499,30 @@ func (sf *ScalarFunction) EvalJSON(ctx sessionctx.Context, row chunk.Row) (types } // HashCode implements Expression interface. -func (sf *ScalarFunction) HashCode(sc *stmtctx.StatementContext) []byte { - if sc != nil && sc.CanonicalHashCode { - if len(sf.canonicalhashcode) > 0 { - return sf.canonicalhashcode - } - simpleCanonicalizedHashCode(sf, sc) - return sf.canonicalhashcode - } +func (sf *ScalarFunction) HashCode() []byte { if len(sf.hashcode) > 0 { return sf.hashcode } - ReHashCode(sf, sc) + ReHashCode(sf) return sf.hashcode } +// CanonicalHashCode implements Expression interface. +func (sf *ScalarFunction) CanonicalHashCode() []byte { + if len(sf.canonicalhashcode) > 0 { + return sf.canonicalhashcode + } + simpleCanonicalizedHashCode(sf) + return sf.canonicalhashcode +} + // ExpressionsSemanticEqual is used to judge whether two expression tree is semantic equivalent. -func ExpressionsSemanticEqual(ctx sessionctx.Context, expr1, expr2 Expression) bool { - sc := ctx.GetSessionVars().StmtCtx - sc.CanonicalHashCode = true - defer func() { - sc.CanonicalHashCode = false - }() - return bytes.Equal(expr1.HashCode(sc), expr2.HashCode(sc)) +func ExpressionsSemanticEqual(expr1, expr2 Expression) bool { + return bytes.Equal(expr1.CanonicalHashCode(), expr2.CanonicalHashCode()) } // simpleCanonicalizedHashCode is used to judge whether two expression is semantically equal. -func simpleCanonicalizedHashCode(sf *ScalarFunction, sc *stmtctx.StatementContext) { +func simpleCanonicalizedHashCode(sf *ScalarFunction) { if sf.canonicalhashcode != nil { sf.canonicalhashcode = sf.canonicalhashcode[:0] } @@ -533,7 +530,7 @@ func simpleCanonicalizedHashCode(sf *ScalarFunction, sc *stmtctx.StatementContex argsHashCode := make([][]byte, 0, len(sf.GetArgs())) for _, arg := range sf.GetArgs() { - argsHashCode = append(argsHashCode, arg.HashCode(sc)) + argsHashCode = append(argsHashCode, arg.CanonicalHashCode()) } switch sf.FuncName.L { case ast.Plus, ast.Mul, ast.EQ, ast.In, ast.LogicOr, ast.LogicAnd: @@ -583,7 +580,7 @@ func simpleCanonicalizedHashCode(sf *ScalarFunction, sc *stmtctx.StatementContex } else { childArgsHashCode := make([][]byte, 0, len(child.GetArgs())) for _, arg := range child.GetArgs() { - childArgsHashCode = append(childArgsHashCode, arg.HashCode(sc)) + childArgsHashCode = append(childArgsHashCode, arg.CanonicalHashCode()) } switch child.FuncName.L { case ast.GT: // not GT ==> LE ==> use GE and switch args @@ -624,12 +621,12 @@ func simpleCanonicalizedHashCode(sf *ScalarFunction, sc *stmtctx.StatementContex } // ReHashCode is used after we change the argument in place. -func ReHashCode(sf *ScalarFunction, sc *stmtctx.StatementContext) { +func ReHashCode(sf *ScalarFunction) { sf.hashcode = sf.hashcode[:0] sf.hashcode = append(sf.hashcode, scalarFunctionFlag) sf.hashcode = codec.EncodeCompactBytes(sf.hashcode, hack.Slice(sf.FuncName.L)) for _, arg := range sf.GetArgs() { - sf.hashcode = append(sf.hashcode, arg.HashCode(sc)...) + sf.hashcode = append(sf.hashcode, arg.HashCode()...) } // Cast is a special case. The RetType should also be considered as an argument. // Please see `newFunctionImpl()` for detail. diff --git a/pkg/expression/scalar_function_test.go b/pkg/expression/scalar_function_test.go index 2ba9831452102..97fc0897479d5 100644 --- a/pkg/expression/scalar_function_test.go +++ b/pkg/expression/scalar_function_test.go @@ -16,11 +16,9 @@ package expression import ( "testing" - "time" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/mock" @@ -28,7 +26,6 @@ import ( ) func TestExpressionSemanticEqual(t *testing.T) { - ctx := mock.NewContext() a := &Column{ UniqueID: 1, RetType: types.NewFieldType(mysql.TypeDouble), @@ -41,56 +38,56 @@ func TestExpressionSemanticEqual(t *testing.T) { // a < b; b > a sf1 := newFunction(ast.LT, a, b) sf2 := newFunction(ast.GT, b, a) - require.True(t, ExpressionsSemanticEqual(ctx, sf1, sf2)) + require.True(t, ExpressionsSemanticEqual(sf1, sf2)) // a > b; b < a sf3 := newFunction(ast.GT, a, b) sf4 := newFunction(ast.LT, b, a) - require.True(t, ExpressionsSemanticEqual(ctx, sf3, sf4)) + require.True(t, ExpressionsSemanticEqual(sf3, sf4)) // a<=b; b>=a sf5 := newFunction(ast.LE, a, b) sf6 := newFunction(ast.GE, b, a) - require.True(t, ExpressionsSemanticEqual(ctx, sf5, sf6)) + require.True(t, ExpressionsSemanticEqual(sf5, sf6)) // a>=b; b<=a sf7 := newFunction(ast.GE, a, b) sf8 := newFunction(ast.LE, b, a) - require.True(t, ExpressionsSemanticEqual(ctx, sf7, sf8)) + require.True(t, ExpressionsSemanticEqual(sf7, sf8)) // not(a= b sf9 := newFunction(ast.UnaryNot, sf1) - require.True(t, ExpressionsSemanticEqual(ctx, sf9, sf7)) + require.True(t, ExpressionsSemanticEqual(sf9, sf7)) // a < b; not(a>=b) sf10 := newFunction(ast.UnaryNot, sf7) - require.True(t, ExpressionsSemanticEqual(ctx, sf1, sf10)) + require.True(t, ExpressionsSemanticEqual(sf1, sf10)) // order insensitive cases // a + b; b + a p1 := newFunction(ast.Plus, a, b) p2 := newFunction(ast.Plus, b, a) - require.True(t, ExpressionsSemanticEqual(ctx, p1, p2)) + require.True(t, ExpressionsSemanticEqual(p1, p2)) // a * b; b * a m1 := newFunction(ast.Mul, a, b) m2 := newFunction(ast.Mul, b, a) - require.True(t, ExpressionsSemanticEqual(ctx, m1, m2)) + require.True(t, ExpressionsSemanticEqual(m1, m2)) // a = b; b = a e1 := newFunction(ast.EQ, a, b) e2 := newFunction(ast.EQ, b, a) - require.True(t, ExpressionsSemanticEqual(ctx, e1, e2)) + require.True(t, ExpressionsSemanticEqual(e1, e2)) // a = b AND b + a; a + b AND b = a a1 := newFunction(ast.LogicAnd, e1, p2) a2 := newFunction(ast.LogicAnd, p1, e2) - require.True(t, ExpressionsSemanticEqual(ctx, a1, a2)) + require.True(t, ExpressionsSemanticEqual(a1, a2)) // a * b OR a + b; b + a OR b * a o1 := newFunction(ast.LogicOr, m1, p1) o2 := newFunction(ast.LogicOr, p2, m2) - require.True(t, ExpressionsSemanticEqual(ctx, o1, o2)) + require.True(t, ExpressionsSemanticEqual(o1, o2)) } func TestScalarFunction(t *testing.T) { @@ -99,7 +96,6 @@ func TestScalarFunction(t *testing.T) { UniqueID: 1, RetType: types.NewFieldType(mysql.TypeDouble), } - sc := stmtctx.NewStmtCtxWithTimeZone(time.Local) sf := newFunction(ast.LT, a, NewOne()) res, err := sf.MarshalJSON() require.NoError(t, err) @@ -107,7 +103,7 @@ func TestScalarFunction(t *testing.T) { require.False(t, sf.IsCorrelated()) require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx)) require.True(t, sf.Decorrelate(nil).Equal(ctx, sf)) - require.EqualValues(t, []byte{0x3, 0x4, 0x6c, 0x74, 0x1, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x5, 0xbf, 0xf0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0}, sf.HashCode(sc)) + require.EqualValues(t, []byte{0x3, 0x4, 0x6c, 0x74, 0x1, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x5, 0xbf, 0xf0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0}, sf.HashCode()) sf = NewValuesFunc(ctx, 0, types.NewFieldType(mysql.TypeLonglong)) newSf, ok := sf.Clone().(*ScalarFunction) diff --git a/pkg/expression/util.go b/pkg/expression/util.go index 5429628dc693f..d8e47d5ef1f9d 100644 --- a/pkg/expression/util.go +++ b/pkg/expression/util.go @@ -1016,14 +1016,13 @@ func ExtractFiltersFromDNFs(ctx sessionctx.Context, conditions []Expression) []E // extractFiltersFromDNF extracts the same condition that occurs in every DNF item and remove them from dnf leaves. func extractFiltersFromDNF(ctx sessionctx.Context, dnfFunc *ScalarFunction) ([]Expression, Expression) { dnfItems := FlattenDNFConditions(dnfFunc) - sc := ctx.GetSessionVars().StmtCtx codeMap := make(map[string]int) hashcode2Expr := make(map[string]Expression) for i, dnfItem := range dnfItems { innerMap := make(map[string]struct{}) cnfItems := SplitCNFItems(dnfItem) for _, cnfItem := range cnfItems { - code := cnfItem.HashCode(sc) + code := cnfItem.HashCode() if i == 0 { codeMap[string(code)] = 1 hashcode2Expr[string(code)] = cnfItem @@ -1053,7 +1052,7 @@ func extractFiltersFromDNF(ctx sessionctx.Context, dnfFunc *ScalarFunction) ([]E cnfItems := SplitCNFItems(dnfItem) newCNFItems := make([]Expression, 0, len(cnfItems)) for _, cnfItem := range cnfItems { - code := cnfItem.HashCode(sc) + code := cnfItem.HashCode() _, ok := hashcode2Expr[string(code)] if !ok { newCNFItems = append(newCNFItems, cnfItem) @@ -1379,12 +1378,11 @@ func IsInmutableExpr(expr Expression) bool { // RemoveDupExprs removes identical exprs. Not that if expr contains functions which // are mutable or have side effects, we cannot remove it even if it has duplicates; // if the plan is going to be cached, we cannot remove expressions containing `?` neither. -func RemoveDupExprs(ctx sessionctx.Context, exprs []Expression) []Expression { +func RemoveDupExprs(exprs []Expression) []Expression { res := make([]Expression, 0, len(exprs)) exists := make(map[string]struct{}, len(exprs)) - sc := ctx.GetSessionVars().StmtCtx for _, expr := range exprs { - key := string(expr.HashCode(sc)) + key := string(expr.HashCode()) if _, ok := exists[key]; !ok || IsMutableEffectsExpr(expr) { res = append(res, expr) exists[key] = struct{}{} diff --git a/pkg/expression/util_test.go b/pkg/expression/util_test.go index 105f49686a477..4a7198caf4a96 100644 --- a/pkg/expression/util_test.go +++ b/pkg/expression/util_test.go @@ -592,7 +592,8 @@ func (m *MockExpr) RemapColumn(_ map[int64]*Column) (Expression, error) { return func (m *MockExpr) ExplainInfo() string { return "" } func (m *MockExpr) ExplainNormalizedInfo() string { return "" } func (m *MockExpr) ExplainNormalizedInfo4InList() string { return "" } -func (m *MockExpr) HashCode(sc *stmtctx.StatementContext) []byte { return nil } +func (m *MockExpr) HashCode() []byte { return nil } +func (m *MockExpr) CanonicalHashCode() []byte { return nil } func (m *MockExpr) Vectorized() bool { return false } func (m *MockExpr) SupportReverseEval() bool { return false } func (m *MockExpr) HasCoercibility() bool { return false } diff --git a/pkg/planner/cascades/transformation_rules.go b/pkg/planner/cascades/transformation_rules.go index c2bf354ac42d6..c3d6360cee8a9 100644 --- a/pkg/planner/cascades/transformation_rules.go +++ b/pkg/planner/cascades/transformation_rules.go @@ -938,8 +938,8 @@ func (*pushDownJoin) predicatePushDown( default: // TODO: Enhance this rule to deal with Semi/SmiAnti Joins. } - leftCond = expression.RemoveDupExprs(sctx, leftCond) - rightCond = expression.RemoveDupExprs(sctx, rightCond) + leftCond = expression.RemoveDupExprs(leftCond) + rightCond = expression.RemoveDupExprs(rightCond) return } @@ -1168,7 +1168,7 @@ func (*MergeAdjacentProjection) OnTransform(old *memo.ExprIter) (newExprs []*mem replace := make(map[string]*expression.Column) for i, col := range childGroup.Prop.Schema.Columns { if colOrigin, ok := child.Exprs[i].(*expression.Column); ok { - replace[string(col.HashCode(nil))] = colOrigin + replace[string(col.HashCode())] = colOrigin } } diff --git a/pkg/planner/core/hashcode.go b/pkg/planner/core/hashcode.go index c7854fe5cb7d1..073e8afa1c9c2 100644 --- a/pkg/planner/core/hashcode.go +++ b/pkg/planner/core/hashcode.go @@ -47,7 +47,7 @@ func (p *LogicalProjection) HashCode() []byte { result = encodeIntAsUint32(result, p.SelectBlockOffset()) result = encodeIntAsUint32(result, len(p.Exprs)) for _, expr := range p.Exprs { - exprHashCode := expr.HashCode(p.SCtx().GetSessionVars().StmtCtx) + exprHashCode := expr.HashCode() result = encodeIntAsUint32(result, len(exprHashCode)) result = append(result, exprHashCode...) } @@ -76,7 +76,7 @@ func (p *LogicalSelection) HashCode() []byte { condHashCodes := make([][]byte, len(p.Conditions)) for i, expr := range p.Conditions { - condHashCodes[i] = expr.HashCode(p.SCtx().GetSessionVars().StmtCtx) + condHashCodes[i] = expr.HashCode() } // Sort the conditions, so `a > 1 and a < 100` can equal to `a < 100 and a > 1`. slices.SortFunc(condHashCodes, func(i, j []byte) int { return bytes.Compare(i, j) }) diff --git a/pkg/planner/core/indexmerge_path.go b/pkg/planner/core/indexmerge_path.go index d4eb85f3004e3..5518566d1d2e9 100644 --- a/pkg/planner/core/indexmerge_path.go +++ b/pkg/planner/core/indexmerge_path.go @@ -531,7 +531,7 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int, usedAccessMa // since idx2's access cond has already been covered by idx1. containRelation := true for _, access := range originalPath.AccessConds { - if _, ok := usedAccessMap[string(access.HashCode(ds.SCtx().GetSessionVars().StmtCtx))]; !ok { + if _, ok := usedAccessMap[string(access.HashCode())]; !ok { // some condition is not covered in previous mv index partial path, use it! containRelation = false break @@ -542,8 +542,8 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int, usedAccessMa } // for this picked normal index, mark its access conds. for _, access := range originalPath.AccessConds { - if _, ok := usedAccessMap[string(access.HashCode(ds.SCtx().GetSessionVars().StmtCtx))]; !ok { - usedAccessMap[string(access.HashCode(ds.SCtx().GetSessionVars().StmtCtx))] = access + if _, ok := usedAccessMap[string(access.HashCode())]; !ok { + usedAccessMap[string(access.HashCode())] = access } } } @@ -586,11 +586,11 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int, usedAccessMa // avoid wrong deduplication. notCoveredHashCodeSet := make(map[string]struct{}) for _, cond := range notCoveredConds { - hashCode := string(cond.HashCode(ds.SCtx().GetSessionVars().StmtCtx)) + hashCode := string(cond.HashCode()) notCoveredHashCodeSet[hashCode] = struct{}{} } for _, cond := range coveredConds { - hashCode := string(cond.HashCode(ds.SCtx().GetSessionVars().StmtCtx)) + hashCode := string(cond.HashCode()) if _, ok := notCoveredHashCodeSet[hashCode]; !ok { hashCodeSet[hashCode] = struct{}{} } @@ -603,7 +603,7 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int, usedAccessMa // Remove covered filters from finalFilters and deduplicate finalFilters. dedupedFinalFilters := make([]expression.Expression, 0, len(finalFilters)) for _, cond := range finalFilters { - hashCode := string(cond.HashCode(ds.SCtx().GetSessionVars().StmtCtx)) + hashCode := string(cond.HashCode()) if _, ok := hashCodeSet[hashCode]; !ok { dedupedFinalFilters = append(dedupedFinalFilters, cond) hashCodeSet[hashCode] = struct{}{} @@ -778,7 +778,7 @@ func (ds *DataSource) generateMVIndexMergePartialPaths4And(normalPathCnt int, in // And(path1, path2, And(path3, path4)) => And(path1, path2, path3, path4, merge(table-action like filter) if len(partialPaths) == 1 || isIntersection { for _, accessF := range accessFilters { - usedAccessCondsMap[string(accessF.HashCode(ds.SCtx().GetSessionVars().StmtCtx))] = accessF + usedAccessCondsMap[string(accessF.HashCode())] = accessF } mvAndPartialPath = append(mvAndPartialPath, partialPaths...) } @@ -1065,7 +1065,7 @@ func (ds *DataSource) generateIndexMerge4ComposedIndex(normalPathCnt int, indexM // collect the remained CNF conditions var remainedCNFs []expression.Expression for _, CNFItem := range indexMergeConds { - if _, ok := usedAccessMap[string(CNFItem.HashCode(ds.SCtx().GetSessionVars().StmtCtx))]; !ok { + if _, ok := usedAccessMap[string(CNFItem.HashCode())]; !ok { remainedCNFs = append(remainedCNFs, CNFItem) } } diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index a7e16a51cad48..edc04269de2cf 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -221,7 +221,7 @@ func (b *PlanBuilder) buildExpand(p LogicalPlan, gbyItems []expression.Expressio b.optFlag |= flagResolveExpand // Rollup syntax require expand OP to do the data expansion, different data replica supply the different grouping layout. - distinctGbyExprs, gbyExprsRefPos := expression.DeduplicateGbyExpression(b.ctx, gbyItems) + distinctGbyExprs, gbyExprsRefPos := expression.DeduplicateGbyExpression(gbyItems) // build another projection below. proj := LogicalProjection{Exprs: make([]expression.Expression, 0, p.Schema().Len()+len(distinctGbyExprs))}.Init(b.ctx, b.getSelectOffset()) // project: child's output and distinct GbyExprs in advance. (make every group-by item to be a column) @@ -1562,7 +1562,7 @@ func (b *PlanBuilder) buildProjectionField(ctx context.Context, p LogicalPlan, f if b.ctx.GetSessionVars().MapHashCode2UniqueID4ExtendedCol == nil { b.ctx.GetSessionVars().MapHashCode2UniqueID4ExtendedCol = make(map[string]int, 1) } - b.ctx.GetSessionVars().MapHashCode2UniqueID4ExtendedCol[string(expr.HashCode(b.ctx.GetSessionVars().StmtCtx))] = int(newCol.UniqueID) + b.ctx.GetSessionVars().MapHashCode2UniqueID4ExtendedCol[string(expr.HashCode())] = int(newCol.UniqueID) } newCol.SetCoercibility(expr.Coercibility()) return newCol, name, nil @@ -1829,7 +1829,7 @@ func (b *PlanBuilder) buildProjection(ctx context.Context, p LogicalPlan, fields if expression.CheckFuncInExpr(x, ast.AnyValue) { continue } - scalarUniqueID, ok := fds.IsHashCodeRegistered(string(hack.String(x.HashCode(p.SCtx().GetSessionVars().StmtCtx)))) + scalarUniqueID, ok := fds.IsHashCodeRegistered(string(hack.String(x.HashCode()))) if !ok { logutil.BgLogger().Warn("Error occurred while maintaining the functional dependency") continue @@ -1894,7 +1894,7 @@ func (b *PlanBuilder) buildProjection(ctx context.Context, p LogicalPlan, fields case *expression.Column: projectionUniqueIDs.Insert(int(x.UniqueID)) case *expression.ScalarFunction: - scalarUniqueID, ok := fds.IsHashCodeRegistered(string(hack.String(x.HashCode(p.SCtx().GetSessionVars().StmtCtx)))) + scalarUniqueID, ok := fds.IsHashCodeRegistered(string(hack.String(x.HashCode()))) if !ok { logutil.BgLogger().Warn("Error occurred while maintaining the functional dependency") continue @@ -4909,7 +4909,7 @@ func (b *PlanBuilder) tryBuildCTE(ctx context.Context, tn *ast.TableName, asName } for i, col := range lp.schema.Columns { - lp.cte.ColumnMap[string(col.HashCode(nil))] = prevSchema.Columns[i] + lp.cte.ColumnMap[string(col.HashCode())] = prevSchema.Columns[i] } p = lp p.SetOutputNames(cte.seedLP.OutputNames()) diff --git a/pkg/planner/core/logical_plans.go b/pkg/planner/core/logical_plans.go index 7438f6b811ce9..da3ad0f3bf44f 100644 --- a/pkg/planner/core/logical_plans.go +++ b/pkg/planner/core/logical_plans.go @@ -721,16 +721,10 @@ func (p *LogicalExpand) GenerateGroupingMarks(sourceCols []*expression.Column) [ } func (p *LogicalExpand) trySubstituteExprWithGroupingSetCol(expr expression.Expression) (expression.Expression, bool) { - sc := p.SCtx().GetSessionVars().StmtCtx - sc.CanonicalHashCode = true - defer func() { - sc.CanonicalHashCode = false - }() - // since all the original group items has been projected even single col, // let's check the origin gby expression here, and map it to new gby col. for i, oneExpr := range p.distinctGbyExprs { - if bytes.Equal(expr.HashCode(sc), oneExpr.HashCode(sc)) { + if bytes.Equal(expr.CanonicalHashCode(), oneExpr.CanonicalHashCode()) { // found return p.distinctGroupByCol[i], true } @@ -741,11 +735,6 @@ func (p *LogicalExpand) trySubstituteExprWithGroupingSetCol(expr expression.Expr // CheckGroupingFuncArgsInGroupBy checks whether grouping function args is in grouping items. func (p *LogicalExpand) resolveGroupingFuncArgsInGroupBy(groupingFuncArgs []expression.Expression) ([]*expression.Column, error) { - sc := p.SCtx().GetSessionVars().StmtCtx - sc.CanonicalHashCode = true - defer func() { - sc.CanonicalHashCode = false - }() // build GBYColMap distinctGBYColMap := make(map[int64]struct{}, len(p.distinctGroupByCol)) for _, oneDistinctGBYCol := range p.distinctGroupByCol { @@ -758,7 +747,7 @@ func (p *LogicalExpand) resolveGroupingFuncArgsInGroupBy(groupingFuncArgs []expr // since all the original group items has been projected even single col, // let's check the origin gby expression here, and map it to new gby col. for i, oneExpr := range p.distinctGbyExprs { - if bytes.Equal(oneArg.HashCode(sc), oneExpr.HashCode(sc)) { + if bytes.Equal(oneArg.CanonicalHashCode(), oneExpr.CanonicalHashCode()) { refPos = i break } @@ -877,21 +866,21 @@ func (p *LogicalProjection) ExtractFD() *fd.FDSet { // take c as constant column here. continue case *expression.Constant: - hashCode := string(x.HashCode(p.SCtx().GetSessionVars().StmtCtx)) + hashCode := string(x.HashCode()) var ( ok bool constantUniqueID int ) if constantUniqueID, ok = fds.IsHashCodeRegistered(hashCode); !ok { constantUniqueID = outputColsUniqueIDsArray[idx] - fds.RegisterUniqueID(string(x.HashCode(p.SCtx().GetSessionVars().StmtCtx)), constantUniqueID) + fds.RegisterUniqueID(string(x.HashCode()), constantUniqueID) } fds.AddConstants(intset.NewFastIntSet(constantUniqueID)) case *expression.ScalarFunction: // t1(a,b,c), t2(m,n) // select a, (select c+n from t2 where m=b) from t1; // expr(c+n) contains correlated column , but we can treat it as constant here. - hashCode := string(x.HashCode(p.SCtx().GetSessionVars().StmtCtx)) + hashCode := string(x.HashCode()) var ( ok bool scalarUniqueID int @@ -1037,7 +1026,7 @@ func (la *LogicalAggregation) ExtractFD() *fd.FDSet { // shouldn't be here, interpreted as pos param by plan builder. continue case *expression.ScalarFunction: - hashCode := string(x.HashCode(la.SCtx().GetSessionVars().StmtCtx)) + hashCode := string(x.HashCode()) var ( ok bool scalarUniqueID int @@ -1252,12 +1241,12 @@ func extractConstantCols(conditions []expression.Expression, sctx sessionctx.Con case *expression.Column: constUniqueIDs.Insert(int(x.UniqueID)) case *expression.ScalarFunction: - hashCode := string(x.HashCode(sctx.GetSessionVars().StmtCtx)) + hashCode := string(x.HashCode()) if uniqueID, ok := fds.IsHashCodeRegistered(hashCode); ok { constUniqueIDs.Insert(uniqueID) } else { scalarUniqueID := int(sctx.GetSessionVars().AllocPlanColumnID()) - fds.RegisterUniqueID(string(x.HashCode(sctx.GetSessionVars().StmtCtx)), scalarUniqueID) + fds.RegisterUniqueID(string(x.HashCode()), scalarUniqueID) constUniqueIDs.Insert(scalarUniqueID) } } @@ -1279,12 +1268,12 @@ func extractEquivalenceCols(conditions []expression.Expression, sctx sessionctx. case *expression.Column: lhsUniqueID = int(x.UniqueID) case *expression.ScalarFunction: - hashCode := string(x.HashCode(sctx.GetSessionVars().StmtCtx)) + hashCode := string(x.HashCode()) if uniqueID, ok := fds.IsHashCodeRegistered(hashCode); ok { lhsUniqueID = uniqueID } else { scalarUniqueID := int(sctx.GetSessionVars().AllocPlanColumnID()) - fds.RegisterUniqueID(string(x.HashCode(sctx.GetSessionVars().StmtCtx)), scalarUniqueID) + fds.RegisterUniqueID(string(x.HashCode()), scalarUniqueID) lhsUniqueID = scalarUniqueID } } @@ -1293,12 +1282,12 @@ func extractEquivalenceCols(conditions []expression.Expression, sctx sessionctx. case *expression.Column: rhsUniqueID = int(x.UniqueID) case *expression.ScalarFunction: - hashCode := string(x.HashCode(sctx.GetSessionVars().StmtCtx)) + hashCode := string(x.HashCode()) if uniqueID, ok := fds.IsHashCodeRegistered(hashCode); ok { rhsUniqueID = uniqueID } else { scalarUniqueID := int(sctx.GetSessionVars().AllocPlanColumnID()) - fds.RegisterUniqueID(string(x.HashCode(sctx.GetSessionVars().StmtCtx)), scalarUniqueID) + fds.RegisterUniqueID(string(x.HashCode()), scalarUniqueID) rhsUniqueID = scalarUniqueID } } diff --git a/pkg/planner/core/rule_column_pruning.go b/pkg/planner/core/rule_column_pruning.go index 8832c429518e8..dce44e08ac1c0 100644 --- a/pkg/planner/core/rule_column_pruning.go +++ b/pkg/planner/core/rule_column_pruning.go @@ -216,7 +216,7 @@ func pruneByItems(p LogicalPlan, old []*util.ByItems, opt *logicalOptimizeOp) (b seen := make(map[string]struct{}, len(old)) for _, byItem := range old { pruned := true - hash := string(byItem.Expr.HashCode(nil)) + hash := string(byItem.Expr.HashCode()) _, hashMatch := seen[hash] seen[hash] = struct{}{} cols := expression.ExtractColumns(byItem.Expr) diff --git a/pkg/planner/core/rule_constant_propagation.go b/pkg/planner/core/rule_constant_propagation.go index 0e2d26e5d7379..4c2071fc8610b 100644 --- a/pkg/planner/core/rule_constant_propagation.go +++ b/pkg/planner/core/rule_constant_propagation.go @@ -202,7 +202,7 @@ func (projection *LogicalProjection) pullUpConstantPredicates() []expression.Exp // result predicate : a'=1 replace := make(map[string]*expression.Column) for i, expr := range projection.Exprs { - replace[string(expr.HashCode(nil))] = projection.Schema().Columns[i] + replace[string(expr.HashCode())] = projection.Schema().Columns[i] } result := make([]expression.Expression, 0, len(candidateConstantPredicates)) for _, predicate := range candidateConstantPredicates { @@ -212,7 +212,7 @@ func (projection *LogicalProjection) pullUpConstantPredicates() []expression.Exp if len(columns) != 1 { continue } - if replace[string(columns[0].HashCode(nil))] == nil { + if replace[string(columns[0].HashCode())] == nil { // The column of predicate will not appear on the upper level // This means that this predicate does not apply to the constant propagation optimization rule // For example: select * from t, (select b from s where s.a=1) tmp where t.b=s.b diff --git a/pkg/planner/core/rule_eliminate_projection.go b/pkg/planner/core/rule_eliminate_projection.go index ff60cf4cf1469..46b2d761e6095 100644 --- a/pkg/planner/core/rule_eliminate_projection.go +++ b/pkg/planner/core/rule_eliminate_projection.go @@ -97,7 +97,7 @@ func canProjectionBeEliminatedStrict(p *PhysicalProjection) bool { } func resolveColumnAndReplace(origin *expression.Column, replace map[string]*expression.Column) { - dst := replace[string(origin.HashCode(nil))] + dst := replace[string(origin.HashCode())] if dst != nil { retType, inOperand := origin.RetType, origin.InOperand *origin = *dst @@ -226,7 +226,7 @@ func (pe *projectionEliminator) eliminate(p LogicalPlan, replace map[string]*exp } exprs := proj.Exprs for i, col := range proj.Schema().Columns { - replace[string(col.HashCode(nil))] = exprs[i].(*expression.Column) + replace[string(col.HashCode())] = exprs[i].(*expression.Column) } appendProjEliminateTraceStep(proj, opt) return p.Children()[0] @@ -297,7 +297,7 @@ func (p *LogicalSelection) ReplaceExprColumns(replace map[string]*expression.Col func (la *LogicalApply) ReplaceExprColumns(replace map[string]*expression.Column) { la.LogicalJoin.ReplaceExprColumns(replace) for _, coCol := range la.CorCols { - dst := replace[string(coCol.Column.HashCode(nil))] + dst := replace[string(coCol.Column.HashCode())] if dst != nil { coCol.Column = *dst } diff --git a/pkg/planner/core/rule_generate_column_substitute.go b/pkg/planner/core/rule_generate_column_substitute.go index f39c5aa5d0d55..5e5cc2b28f12c 100644 --- a/pkg/planner/core/rule_generate_column_substitute.go +++ b/pkg/planner/core/rule_generate_column_substitute.go @@ -117,7 +117,6 @@ func substituteExpression(cond expression.Expression, lp LogicalPlan, exprToColu if !ok { return false } - sctx := lp.SCtx().GetSessionVars().StmtCtx changed := false collectChanged := func(partial bool) { if partial && !changed { @@ -127,7 +126,7 @@ func substituteExpression(cond expression.Expression, lp LogicalPlan, exprToColu defer func() { // If the argument is not changed, hash code doesn't need to recount again. if changed { - expression.ReHashCode(sf, sctx) + expression.ReHashCode(sf) } }() var expr *expression.Expression diff --git a/pkg/planner/core/rule_partition_processor.go b/pkg/planner/core/rule_partition_processor.go index f3bfe3eaf3007..d0dda5e8fbf6b 100644 --- a/pkg/planner/core/rule_partition_processor.go +++ b/pkg/planner/core/rule_partition_processor.go @@ -123,7 +123,7 @@ func generateHashPartitionExpr(ctx sessionctx.Context, pi *model.PartitionInfo, if err != nil { return nil, err } - exprs[0].HashCode(ctx.GetSessionVars().StmtCtx) + exprs[0].HashCode() return exprs[0], nil } diff --git a/pkg/planner/core/rule_predicate_push_down.go b/pkg/planner/core/rule_predicate_push_down.go index 9277d3574b2fc..2597727ac6d4a 100644 --- a/pkg/planner/core/rule_predicate_push_down.go +++ b/pkg/planner/core/rule_predicate_push_down.go @@ -234,8 +234,8 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt rightCond = append(p.RightConditions, rightPushCond...) p.RightConditions = nil } - leftCond = expression.RemoveDupExprs(p.SCtx(), leftCond) - rightCond = expression.RemoveDupExprs(p.SCtx(), rightCond) + leftCond = expression.RemoveDupExprs(leftCond) + rightCond = expression.RemoveDupExprs(rightCond) leftRet, lCh := p.children[0].PredicatePushDown(leftCond, opt) rightRet, rCh := p.children[1].PredicatePushDown(rightCond, opt) addSelection(p, lCh, leftRet, 0, opt) diff --git a/pkg/planner/core/scalar_subq_expression.go b/pkg/planner/core/scalar_subq_expression.go index 5e2b6250aef55..c3d0b91c0e147 100644 --- a/pkg/planner/core/scalar_subq_expression.go +++ b/pkg/planner/core/scalar_subq_expression.go @@ -233,7 +233,7 @@ func (s *ScalarSubQueryExpr) ExplainNormalizedInfo() string { } // HashCode implements the Expression interface. -func (s *ScalarSubQueryExpr) HashCode(_ *stmtctx.StatementContext) []byte { +func (s *ScalarSubQueryExpr) HashCode() []byte { if len(s.hashcode) != 0 { return s.hashcode } @@ -243,6 +243,11 @@ func (s *ScalarSubQueryExpr) HashCode(_ *stmtctx.StatementContext) []byte { return s.hashcode } +// CanonicalHashCode implements the Expression interface. +func (s *ScalarSubQueryExpr) CanonicalHashCode() []byte { + return s.HashCode() +} + // MemoryUsage implements the Expression interface. func (s *ScalarSubQueryExpr) MemoryUsage() int64 { ret := int64(0) diff --git a/pkg/planner/property/physical_property.go b/pkg/planner/property/physical_property.go index 3199c6e8d2868..172ecbc2d379c 100644 --- a/pkg/planner/property/physical_property.go +++ b/pkg/planner/property/physical_property.go @@ -95,7 +95,7 @@ type MPPPartitionColumn struct { } func (partitionCol *MPPPartitionColumn) hashCode(ctx *stmtctx.StatementContext) []byte { - hashcode := partitionCol.Col.HashCode(ctx) + hashcode := partitionCol.Col.HashCode() if partitionCol.CollateID < 0 { // collateId < 0 means new collation is not enabled hashcode = codec.EncodeInt(hashcode, int64(partitionCol.CollateID)) @@ -330,7 +330,7 @@ func (p *PhysicalProperty) HashCode() []byte { p.hashcode = codec.EncodeInt(p.hashcode, int64(p.TaskTp)) p.hashcode = codec.EncodeFloat(p.hashcode, p.ExpectedCnt) for _, item := range p.SortItems { - p.hashcode = append(p.hashcode, item.Col.HashCode(nil)...) + p.hashcode = append(p.hashcode, item.Col.HashCode()...) if item.Desc { p.hashcode = codec.EncodeInt(p.hashcode, 1) } else { diff --git a/pkg/sessionctx/stmtctx/stmtctx.go b/pkg/sessionctx/stmtctx/stmtctx.go index 9764ca5fcc1a2..5e3a929c051b5 100644 --- a/pkg/sessionctx/stmtctx/stmtctx.go +++ b/pkg/sessionctx/stmtctx/stmtctx.go @@ -416,8 +416,6 @@ type StatementContext struct { useChunkAlloc bool // Check if TiFlash read engine is removed due to strict sql mode. TiFlashEngineRemovedDueToStrictSQLMode bool - // CanonicalHashCode try to get the canonical hash code from expression. - CanonicalHashCode bool // StaleTSOProvider is used to provide stale timestamp oracle for read-only transactions. StaleTSOProvider struct { sync.Mutex diff --git a/pkg/table/tables/partition.go b/pkg/table/tables/partition.go index 30d7b1950f4d3..dc036ba94f1a3 100644 --- a/pkg/table/tables/partition.go +++ b/pkg/table/tables/partition.go @@ -1207,7 +1207,7 @@ func generateHashPartitionExpr(ctx sessionctx.Context, exprStr string, } } } - exprs.HashCode(ctx.GetSessionVars().StmtCtx) + exprs.HashCode() return &PartitionExpr{ Expr: exprs, OrigExpr: origExpr, From 522cd038678bc993bc4616dac1d16256be480409 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 23 Nov 2023 19:02:50 +0800 Subject: [PATCH 30/36] ddl: fix issue of alter last partition failed when partition column is datetime (#48815) close pingcap/tidb#48814 --- pkg/ddl/partition.go | 39 ++++- pkg/ddl/tests/partition/BUILD.bazel | 2 +- pkg/ddl/tests/partition/db_partition_test.go | 148 +++++++++++++++++++ 3 files changed, 180 insertions(+), 9 deletions(-) diff --git a/pkg/ddl/partition.go b/pkg/ddl/partition.go index 7ebba13273101..bb3c4d932a6a3 100644 --- a/pkg/ddl/partition.go +++ b/pkg/ddl/partition.go @@ -772,12 +772,20 @@ func getPartitionIntervalFromTable(ctx sessionctx.Context, tbInfo *model.TableIn } // comparePartitionAstAndModel compares a generated *ast.PartitionOptions and a *model.PartitionInfo -func comparePartitionAstAndModel(ctx sessionctx.Context, pAst *ast.PartitionOptions, pModel *model.PartitionInfo) error { +func comparePartitionAstAndModel(ctx sessionctx.Context, pAst *ast.PartitionOptions, pModel *model.PartitionInfo, partCol *model.ColumnInfo) error { a := pAst.Definitions m := pModel.Definitions if len(pAst.Definitions) != len(pModel.Definitions) { return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs("INTERVAL partitioning: number of partitions generated != partition defined (%d != %d)", len(a), len(m)) } + + evalFn := func(expr ast.ExprNode) (types.Datum, error) { + val, err := expression.EvalAstExpr(ctx, ast.NewValueExpr(expr, "", "")) + if err != nil || partCol == nil { + return val, err + } + return val.ConvertTo(ctx.GetSessionVars().StmtCtx.TypeCtx(), &partCol.FieldType) + } for i := range pAst.Definitions { // Allow options to differ! (like Placement Rules) // Allow names to differ! @@ -800,16 +808,19 @@ func comparePartitionAstAndModel(ctx sessionctx.Context, pAst *ast.PartitionOpti if len(lessThan) > 1 && lessThan[:1] == "'" && lessThan[len(lessThan)-1:] == "'" { lessThan = driver.UnwrapFromSingleQuotes(lessThan) } - cmpExpr := &ast.BinaryOperationExpr{ - Op: opcode.EQ, - L: ast.NewValueExpr(lessThan, "", ""), - R: generatedExpr, + lessThanVal, err := evalFn(ast.NewValueExpr(lessThan, "", "")) + if err != nil { + return err } - cmp, err := expression.EvalAstExpr(ctx, cmpExpr) + generatedExprVal, err := evalFn(generatedExpr) if err != nil { return err } - if cmp.GetInt64() != 1 { + cmp, err := lessThanVal.Compare(ctx.GetSessionVars().StmtCtx.TypeCtx(), &generatedExprVal, collate.GetBinaryCollator()) + if err != nil { + return err + } + if cmp != 0 { return dbterror.ErrGeneralUnsupportedDDL.GenWithStackByArgs(fmt.Sprintf("INTERVAL partitioning: LESS THAN for partition %s differs between generated and defined", m[i].Name.O)) } } @@ -984,7 +995,7 @@ func generatePartitionDefinitionsFromInterval(ctx sessionctx.Context, partOption // Seems valid, so keep the defined so that the user defined names are kept etc. partOptions.Definitions = definedPartDefs } else if len(tbInfo.Partition.Definitions) > 0 { - err := comparePartitionAstAndModel(ctx, partOptions, tbInfo.Partition) + err := comparePartitionAstAndModel(ctx, partOptions, tbInfo.Partition, partCol) if err != nil { return err } @@ -1058,6 +1069,12 @@ func GeneratePartDefsFromInterval(ctx sessionctx.Context, tp ast.AlterTableType, if err != nil { return err } + if partCol != nil { + lastVal, err = lastVal.ConvertTo(ctx.GetSessionVars().StmtCtx.TypeCtx(), &partCol.FieldType) + if err != nil { + return err + } + } var partDefs []*ast.PartitionDefinition if len(partitionOptions.Definitions) != 0 { partDefs = partitionOptions.Definitions @@ -1101,6 +1118,12 @@ func GeneratePartDefsFromInterval(ctx sessionctx.Context, tp ast.AlterTableType, if err != nil { return err } + if partCol != nil { + currVal, err = currVal.ConvertTo(ctx.GetSessionVars().StmtCtx.TypeCtx(), &partCol.FieldType) + if err != nil { + return err + } + } cmp, err := currVal.Compare(ctx.GetSessionVars().StmtCtx.TypeCtx(), &lastVal, collate.GetBinaryCollator()) if err != nil { return err diff --git a/pkg/ddl/tests/partition/BUILD.bazel b/pkg/ddl/tests/partition/BUILD.bazel index 6f9ba066a2d5c..c3db6c4abdd25 100644 --- a/pkg/ddl/tests/partition/BUILD.bazel +++ b/pkg/ddl/tests/partition/BUILD.bazel @@ -8,7 +8,7 @@ go_test( "main_test.go", ], flaky = True, - shard_count = 47, + shard_count = 48, deps = [ "//pkg/config", "//pkg/ddl", diff --git a/pkg/ddl/tests/partition/db_partition_test.go b/pkg/ddl/tests/partition/db_partition_test.go index 83acd8251234f..f315d3c5d25cd 100644 --- a/pkg/ddl/tests/partition/db_partition_test.go +++ b/pkg/ddl/tests/partition/db_partition_test.go @@ -3645,4 +3645,152 @@ func TestRemovePartitioningAutoIDs(t *testing.T) { "32 31 10", "35 34 21", "38 37 22", "41 40 23")) } +func TestAlterLastIntervalPartition(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t (id int, create_time datetime) + partition by range columns (create_time) + interval (1 day) + first partition less than ('2023-01-01') + last partition less than ('2023-01-03');`) + ctx := tk.Session() + tbl, err := domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + pd := tbl.Meta().Partition.Definitions + require.Equal(t, 3, len(pd)) + require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0]) + require.Equal(t, "'2023-01-02 00:00:00'", pd[1].LessThan[0]) + require.Equal(t, "'2023-01-03 00:00:00'", pd[2].LessThan[0]) + tk.MustExec("alter table t last partition less than ('2024-01-04')") + tk.MustExec("alter table t last partition less than ('2025-01-01 00:00:00')") + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + pd = tbl.Meta().Partition.Definitions + require.Equal(t, 732, len(pd)) + require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0]) + require.Equal(t, "'2023-01-02 00:00:00'", pd[1].LessThan[0]) + require.Equal(t, "'2023-01-03 00:00:00'", pd[2].LessThan[0]) + require.Equal(t, "'2024-12-31 00:00:00'", pd[730].LessThan[0]) + require.Equal(t, "'2025-01-01 00:00:00'", pd[731].LessThan[0]) + + // Test for interval 2 days. + tk.MustExec(`create table t2 (id int, create_time datetime) + partition by range columns (create_time) + interval (2 day) + first partition less than ('2023-01-01') + last partition less than ('2023-01-05');`) + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + require.NoError(t, err) + pd = tbl.Meta().Partition.Definitions + require.Equal(t, 3, len(pd)) + require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0]) + require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0]) + require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0]) + tk.MustExec("alter table t2 last partition less than ('2023-01-09')") + tk.MustExec("alter table t2 last partition less than ('2023-01-11 00:00:00')") + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + require.NoError(t, err) + pd = tbl.Meta().Partition.Definitions + require.Equal(t, 6, len(pd)) + require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0]) + require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0]) + require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0]) + require.Equal(t, "'2023-01-07 00:00:00'", pd[3].LessThan[0]) + require.Equal(t, "'2023-01-09 00:00:00'", pd[4].LessThan[0]) + require.Equal(t, "'2023-01-11 00:00:00'", pd[5].LessThan[0]) + + // Test for day with time. + tk.MustExec(`create table t3 (id int, create_time datetime) + partition by range columns (create_time) + interval (2 day) + first partition less than ('2023-01-01 12:01:02') + last partition less than ('2023-01-05 12:01:02');`) + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t3")) + require.NoError(t, err) + pd = tbl.Meta().Partition.Definitions + require.Equal(t, 3, len(pd)) + require.Equal(t, "'2023-01-01 12:01:02'", pd[0].LessThan[0]) + require.Equal(t, "'2023-01-03 12:01:02'", pd[1].LessThan[0]) + require.Equal(t, "'2023-01-05 12:01:02'", pd[2].LessThan[0]) + tk.MustExec("alter table t3 last partition less than ('2023-01-09 12:01:02')") + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t3")) + require.NoError(t, err) + pd = tbl.Meta().Partition.Definitions + require.Equal(t, 5, len(pd)) + require.Equal(t, "'2023-01-01 12:01:02'", pd[0].LessThan[0]) + require.Equal(t, "'2023-01-03 12:01:02'", pd[1].LessThan[0]) + require.Equal(t, "'2023-01-05 12:01:02'", pd[2].LessThan[0]) + require.Equal(t, "'2023-01-07 12:01:02'", pd[3].LessThan[0]) + require.Equal(t, "'2023-01-09 12:01:02'", pd[4].LessThan[0]) + + // Some other test. + tk.MustExec(`create table t4 (id int, create_time datetime) + partition by range columns (create_time) + interval (48 hour) + first partition less than ('2023-01-01') + last partition less than ('2023-01-05');`) + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t4")) + require.NoError(t, err) + pd = tbl.Meta().Partition.Definitions + require.Equal(t, 3, len(pd)) + require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0]) + require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0]) + require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0]) + tk.MustExec("alter table t4 last partition less than ('2023-01-09 00:00:00')") + tbl, err = domain.GetDomain(ctx).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t4")) + require.NoError(t, err) + pd = tbl.Meta().Partition.Definitions + require.Equal(t, 5, len(pd)) + require.Equal(t, "'2023-01-01 00:00:00'", pd[0].LessThan[0]) + require.Equal(t, "'2023-01-03 00:00:00'", pd[1].LessThan[0]) + require.Equal(t, "'2023-01-05 00:00:00'", pd[2].LessThan[0]) + require.Equal(t, "'2023-01-07 00:00:00'", pd[3].LessThan[0]) + require.Equal(t, "'2023-01-09 00:00:00'", pd[4].LessThan[0]) + tk.MustQuery("show create table t4").Check(testkit.Rows("t4 CREATE TABLE `t4` (\n" + + " `id` int(11) DEFAULT NULL,\n" + + " `create_time` datetime DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE COLUMNS(`create_time`)\n" + + "(PARTITION `P_LT_2023-01-01 00:00:00` VALUES LESS THAN ('2023-01-01 00:00:00'),\n" + + " PARTITION `P_LT_2023-01-03 00:00:00` VALUES LESS THAN ('2023-01-03 00:00:00'),\n" + + " PARTITION `P_LT_2023-01-05 00:00:00` VALUES LESS THAN ('2023-01-05 00:00:00'),\n" + + " PARTITION `P_LT_2023-01-07 00:00:00` VALUES LESS THAN ('2023-01-07 00:00:00'),\n" + + " PARTITION `P_LT_2023-01-09 00:00:00` VALUES LESS THAN ('2023-01-09 00:00:00'))")) + + tk.MustExec(`create table t5 (id int, create_time datetime) + partition by range columns (create_time) + interval (1 month) + first partition less than ('2023-01-01') + last partition less than ('2023-05-01');`) + tk.MustQuery("show create table t5").Check(testkit.Rows("t5 CREATE TABLE `t5` (\n" + + " `id` int(11) DEFAULT NULL,\n" + + " `create_time` datetime DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE COLUMNS(`create_time`)\n" + + "(PARTITION `P_LT_2023-01-01 00:00:00` VALUES LESS THAN ('2023-01-01 00:00:00'),\n" + + " PARTITION `P_LT_2023-02-01 00:00:00` VALUES LESS THAN ('2023-02-01 00:00:00'),\n" + + " PARTITION `P_LT_2023-03-01 00:00:00` VALUES LESS THAN ('2023-03-01 00:00:00'),\n" + + " PARTITION `P_LT_2023-04-01 00:00:00` VALUES LESS THAN ('2023-04-01 00:00:00'),\n" + + " PARTITION `P_LT_2023-05-01 00:00:00` VALUES LESS THAN ('2023-05-01 00:00:00'))")) + + tk.MustExec("CREATE TABLE `t6` (\n" + + " `id` int(11) DEFAULT NULL,\n" + + " `create_time` datetime DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE COLUMNS(`create_time`)\n" + + "(PARTITION `P_LT_2023-01-01` VALUES LESS THAN ('2023-01-01'),\n" + + " PARTITION `P_LT_2023-01-02` VALUES LESS THAN ('2023-01-02'))") + tk.MustExec("alter table t6 last partition less than ('2023-01-04')") + tk.MustQuery("show create table t6").Check(testkit.Rows("t6 CREATE TABLE `t6` (\n" + + " `id` int(11) DEFAULT NULL,\n" + + " `create_time` datetime DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY RANGE COLUMNS(`create_time`)\n" + + "(PARTITION `P_LT_2023-01-01` VALUES LESS THAN ('2023-01-01'),\n" + + " PARTITION `P_LT_2023-01-02` VALUES LESS THAN ('2023-01-02'),\n" + + " PARTITION `P_LT_2023-01-03 00:00:00` VALUES LESS THAN ('2023-01-03 00:00:00'),\n" + + " PARTITION `P_LT_2023-01-04 00:00:00` VALUES LESS THAN ('2023-01-04 00:00:00'))")) +} + // TODO: check EXCHANGE how it handles null (for all types of partitioning!!!) From 6ca9813e5da4754075b6fa841d0f91f470156f8b Mon Sep 17 00:00:00 2001 From: YangKeao Date: Thu, 23 Nov 2023 21:49:14 +0800 Subject: [PATCH 31/36] server: use failpoint rather than real error in test `TestCursorFetchErrorInFetch` (#48832) close pingcap/tidb#47029 --- pkg/server/conn_stmt_test.go | 30 +++++--------------------- pkg/util/chunk/row_container_reader.go | 7 ++++++ 2 files changed, 12 insertions(+), 25 deletions(-) diff --git a/pkg/server/conn_stmt_test.go b/pkg/server/conn_stmt_test.go index 0cdf29d6d5147..cdfec0f6d0cd7 100644 --- a/pkg/server/conn_stmt_test.go +++ b/pkg/server/conn_stmt_test.go @@ -21,12 +21,6 @@ import ( "crypto/rand" "encoding/binary" "fmt" - "io/fs" - "os" - "path/filepath" - "strconv" - "strings" - "syscall" "testing" "github.com/pingcap/failpoint" @@ -334,25 +328,11 @@ func TestCursorFetchErrorInFetch(t *testing.T) { mysql.CursorTypeReadOnly, 0x1, 0x0, 0x0, 0x0, ))) - // close these disk files to produce error - filepath.Walk("/proc/self/fd", func(path string, info fs.FileInfo, err error) error { - if err != nil { - return nil - } - target, err := os.Readlink(path) - if err != nil { - return nil - } - if strings.HasPrefix(target, tmpStoragePath) { - fd, err := strconv.Atoi(filepath.Base(path)) - require.NoError(t, err) - require.NoError(t, syscall.Close(fd)) - } - return nil - }) - - // it'll get "bad file descriptor", as it has been closed in the test. - require.Error(t, c.Dispatch(ctx, appendUint32(appendUint32([]byte{mysql.ComStmtFetch}, uint32(stmt.ID())), 1024))) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/util/chunk/get-chunk-error", "return(true)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/util/chunk/get-chunk-error")) + }() + require.ErrorContains(t, c.Dispatch(ctx, appendUint32(appendUint32([]byte{mysql.ComStmtFetch}, uint32(stmt.ID())), 1024)), "fail to get chunk for test") // after getting a failed FETCH, the cursor should have been reseted require.False(t, stmt.GetCursorActive()) require.Len(t, tk.Session().GetSessionVars().MemTracker.GetChildrenForTest(), 0) diff --git a/pkg/util/chunk/row_container_reader.go b/pkg/util/chunk/row_container_reader.go index e0f3b0e314060..ca124083079c5 100644 --- a/pkg/util/chunk/row_container_reader.go +++ b/pkg/util/chunk/row_container_reader.go @@ -19,6 +19,8 @@ import ( "runtime" "sync" + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/pkg/util/logutil" ) @@ -122,6 +124,11 @@ func (reader *rowContainerReader) startWorker() { for chkIdx := 0; chkIdx < reader.rc.NumChunks(); chkIdx++ { chk, err := reader.rc.GetChunk(chkIdx) + failpoint.Inject("get-chunk-error", func(val failpoint.Value) { + if val.(bool) { + err = errors.New("fail to get chunk for test") + } + }) if err != nil { reader.err = err return From 26db5909628fee0605d9e53d949b7b4d9e2b64e3 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 24 Nov 2023 11:53:42 +0800 Subject: [PATCH 32/36] *: fix wrong result when to concurrency merge global stats (#48852) close pingcap/tidb#48713 --- .../handle/globalstats/merge_worker.go | 59 ++++++++----------- pkg/statistics/handle/globalstats/topn.go | 34 ++++------- 2 files changed, 39 insertions(+), 54 deletions(-) diff --git a/pkg/statistics/handle/globalstats/merge_worker.go b/pkg/statistics/handle/globalstats/merge_worker.go index 7a63adee65fb2..b702acaa797f7 100644 --- a/pkg/statistics/handle/globalstats/merge_worker.go +++ b/pkg/statistics/handle/globalstats/merge_worker.go @@ -43,8 +43,11 @@ type topnStatsMergeWorker struct { respCh chan<- *TopnStatsMergeResponse // the stats in the wrapper should only be read during the worker statsWrapper *StatsWrapper + // Different TopN structures may hold the same value, we have to merge them. + counter map[hack.MutableString]float64 // shardMutex is used to protect `statsWrapper.AllHg` shardMutex []sync.Mutex + mu sync.Mutex } // NewTopnStatsMergeWorker returns topn merge worker @@ -54,8 +57,9 @@ func NewTopnStatsMergeWorker( wrapper *StatsWrapper, killer *sqlkiller.SQLKiller) *topnStatsMergeWorker { worker := &topnStatsMergeWorker{ - taskCh: taskCh, - respCh: respCh, + taskCh: taskCh, + respCh: respCh, + counter: make(map[hack.MutableString]float64), } worker.statsWrapper = wrapper worker.shardMutex = make([]sync.Mutex, len(wrapper.AllHg)) @@ -79,15 +83,11 @@ func NewTopnStatsMergeTask(start, end int) *TopnStatsMergeTask { // TopnStatsMergeResponse indicates topn merge worker response type TopnStatsMergeResponse struct { - Err error - TopN *statistics.TopN - PopedTopn []statistics.TopNMeta + Err error } // Run runs topn merge like statistics.MergePartTopN2GlobalTopN -func (worker *topnStatsMergeWorker) Run(timeZone *time.Location, isIndex bool, - n uint32, - version int) { +func (worker *topnStatsMergeWorker) Run(timeZone *time.Location, isIndex bool, version int) { for task := range worker.taskCh { start := task.start end := task.end @@ -95,17 +95,12 @@ func (worker *topnStatsMergeWorker) Run(timeZone *time.Location, isIndex bool, allTopNs := worker.statsWrapper.AllTopN allHists := worker.statsWrapper.AllHg resp := &TopnStatsMergeResponse{} - if statistics.CheckEmptyTopNs(checkTopNs) { - worker.respCh <- resp - return - } + partNum := len(allTopNs) - // Different TopN structures may hold the same value, we have to merge them. - counter := make(map[hack.MutableString]float64) + // datumMap is used to store the mapping from the string type to datum type. // The datum is used to find the value in the histogram. datumMap := statistics.NewDatumMapCache() - for i, topN := range checkTopNs { i = i + start if err := worker.killer.HandleSignal(); err != nil { @@ -118,12 +113,15 @@ func (worker *topnStatsMergeWorker) Run(timeZone *time.Location, isIndex bool, } for _, val := range topN.TopN { encodedVal := hack.String(val.Encoded) - _, exists := counter[encodedVal] - counter[encodedVal] += float64(val.Count) + worker.mu.Lock() + _, exists := worker.counter[encodedVal] + worker.counter[encodedVal] += float64(val.Count) if exists { + worker.mu.Unlock() // We have already calculated the encodedVal from the histogram, so just continue to next topN value. continue } + worker.mu.Unlock() // We need to check whether the value corresponding to encodedVal is contained in other partition-level stats. // 1. Check the topN first. // 2. If the topN doesn't contain the value corresponding to encodedVal. We should check the histogram. @@ -147,31 +145,26 @@ func (worker *topnStatsMergeWorker) Run(timeZone *time.Location, isIndex bool, } datum = d } + worker.shardMutex[j].Lock() // Get the row count which the value is equal to the encodedVal from histogram. count, _ := allHists[j].EqualRowCount(nil, datum, isIndex) if count != 0 { - counter[encodedVal] += count // Remove the value corresponding to encodedVal from the histogram. - worker.shardMutex[j].Lock() worker.statsWrapper.AllHg[j].BinarySearchRemoveVal(statistics.TopNMeta{Encoded: datum.GetBytes(), Count: uint64(count)}) - worker.shardMutex[j].Unlock() + } + worker.shardMutex[j].Unlock() + if count != 0 { + worker.mu.Lock() + worker.counter[encodedVal] += count + worker.mu.Unlock() } } } } - numTop := len(counter) - if numTop == 0 { - worker.respCh <- resp - continue - } - sorted := make([]statistics.TopNMeta, 0, numTop) - for value, cnt := range counter { - data := hack.Slice(string(value)) - sorted = append(sorted, statistics.TopNMeta{Encoded: data, Count: uint64(cnt)}) - } - globalTopN, leftTopN := statistics.GetMergedTopNFromSortedSlice(sorted, n) - resp.TopN = globalTopN - resp.PopedTopn = leftTopN worker.respCh <- resp } } + +func (worker *topnStatsMergeWorker) Result() map[hack.MutableString]float64 { + return worker.counter +} diff --git a/pkg/statistics/handle/globalstats/topn.go b/pkg/statistics/handle/globalstats/topn.go index 9e9f14a068a54..171756b82357b 100644 --- a/pkg/statistics/handle/globalstats/topn.go +++ b/pkg/statistics/handle/globalstats/topn.go @@ -30,8 +30,12 @@ import ( func mergeGlobalStatsTopN(gp *gp.Pool, sc sessionctx.Context, wrapper *StatsWrapper, timeZone *time.Location, version int, n uint32, isIndex bool) (*statistics.TopN, []statistics.TopNMeta, []*statistics.Histogram, error) { + if statistics.CheckEmptyTopNs(wrapper.AllTopN) { + return nil, nil, wrapper.AllHg, nil + } mergeConcurrency := sc.GetSessionVars().AnalyzePartitionMergeConcurrency killer := &sc.GetSessionVars().SQLKiller + // use original method if concurrency equals 1 or for version1 if mergeConcurrency < 2 { return MergePartTopN2GlobalTopN(timeZone, version, wrapper.AllTopN, n, wrapper.AllHg, isIndex, killer) @@ -78,12 +82,12 @@ func MergeGlobalStatsTopNByConcurrency( taskNum := len(tasks) taskCh := make(chan *TopnStatsMergeTask, taskNum) respCh := make(chan *TopnStatsMergeResponse, taskNum) + worker := NewTopnStatsMergeWorker(taskCh, respCh, wrapper, killer) for i := 0; i < mergeConcurrency; i++ { - worker := NewTopnStatsMergeWorker(taskCh, respCh, wrapper, killer) wg.Add(1) gp.Go(func() { defer wg.Done() - worker.Run(timeZone, isIndex, n, version) + worker.Run(timeZone, isIndex, version) }) } for _, task := range tasks { @@ -92,8 +96,6 @@ func MergeGlobalStatsTopNByConcurrency( close(taskCh) wg.Wait() close(respCh) - resps := make([]*TopnStatsMergeResponse, 0) - // handle Error hasErr := false errMsg := make([]string, 0) @@ -102,27 +104,21 @@ func MergeGlobalStatsTopNByConcurrency( hasErr = true errMsg = append(errMsg, resp.Err.Error()) } - resps = append(resps, resp) } if hasErr { return nil, nil, nil, errors.New(strings.Join(errMsg, ",")) } // fetch the response from each worker and merge them into global topn stats - sorted := make([]statistics.TopNMeta, 0, mergeConcurrency) - leftTopn := make([]statistics.TopNMeta, 0) - for _, resp := range resps { - if resp.TopN != nil { - sorted = append(sorted, resp.TopN.TopN...) - } - leftTopn = append(leftTopn, resp.PopedTopn...) + counter := worker.Result() + numTop := len(counter) + sorted := make([]statistics.TopNMeta, 0, numTop) + for value, cnt := range counter { + data := hack.Slice(string(value)) + sorted = append(sorted, statistics.TopNMeta{Encoded: data, Count: uint64(cnt)}) } - globalTopN, popedTopn := statistics.GetMergedTopNFromSortedSlice(sorted, n) - - result := append(leftTopn, popedTopn...) - statistics.SortTopnMeta(result) - return globalTopN, result, wrapper.AllHg, nil + return globalTopN, popedTopn, wrapper.AllHg, nil } // MergePartTopN2GlobalTopN is used to merge the partition-level topN to global-level topN. @@ -149,10 +145,6 @@ func MergePartTopN2GlobalTopN( isIndex bool, killer *sqlkiller.SQLKiller, ) (*statistics.TopN, []statistics.TopNMeta, []*statistics.Histogram, error) { - if statistics.CheckEmptyTopNs(topNs) { - return nil, nil, hists, nil - } - partNum := len(topNs) // Different TopN structures may hold the same value, we have to merge them. counter := make(map[hack.MutableString]float64) From c771e8b2f94c4cc5c1edab0ce6f170492b36d80e Mon Sep 17 00:00:00 2001 From: lance6716 Date: Fri, 24 Nov 2023 12:44:42 +0800 Subject: [PATCH 33/36] br/storage: enable async prefetch data (#48587) close pingcap/tidb#48781 --- .../lightning/backend/external/bench_test.go | 85 +++++++------ .../lightning/backend/external/byte_reader.go | 3 +- .../lightning/backend/external/kv_reader.go | 5 +- .../lightning/backend/external/stat_reader.go | 2 +- br/pkg/storage/BUILD.bazel | 1 + br/pkg/storage/s3.go | 7 ++ br/pkg/storage/storage.go | 21 ++++ br/pkg/storage/storage_test.go | 8 ++ pkg/util/prefetch/BUILD.bazel | 17 +++ pkg/util/prefetch/reader.go | 113 ++++++++++++++++++ pkg/util/prefetch/reader_test.go | 82 +++++++++++++ 11 files changed, 297 insertions(+), 47 deletions(-) create mode 100644 pkg/util/prefetch/BUILD.bazel create mode 100644 pkg/util/prefetch/reader.go create mode 100644 pkg/util/prefetch/reader_test.go diff --git a/br/pkg/lightning/backend/external/bench_test.go b/br/pkg/lightning/backend/external/bench_test.go index 12378cd8707ee..548c8a7974d22 100644 --- a/br/pkg/lightning/backend/external/bench_test.go +++ b/br/pkg/lightning/backend/external/bench_test.go @@ -37,10 +37,8 @@ func openTestingStorage(t *testing.T) storage.ExternalStorage { if *testingStorageURI == "" { t.Skip("testingStorageURI is not set") } - b, err := storage.ParseBackend(*testingStorageURI, nil) - intest.Assert(err == nil) - s, err := storage.New(context.Background(), b, nil) - intest.Assert(err == nil) + s, err := storage.NewFromURL(context.Background(), *testingStorageURI, nil) + intest.AssertNoError(err) return s } @@ -111,7 +109,7 @@ func writePlainFile(s *writeTestSuite) { offset := 0 flush := func(w storage.ExternalFileWriter) { n, err := w.Write(ctx, buf[:offset]) - intest.Assert(err == nil) + intest.AssertNoError(err) intest.Assert(offset == n) offset = 0 } @@ -120,7 +118,7 @@ func writePlainFile(s *writeTestSuite) { s.beforeCreateWriter() } writer, err := s.store.Create(ctx, "test/plain_file", nil) - intest.Assert(err == nil) + intest.AssertNoError(err) key, val, _ := s.source.next() for key != nil { if offset+len(key)+len(val) > len(buf) { @@ -135,7 +133,7 @@ func writePlainFile(s *writeTestSuite) { s.beforeWriterClose() } err = writer.Close(ctx) - intest.Assert(err == nil) + intest.AssertNoError(err) if s.afterWriterClose != nil { s.afterWriterClose() } @@ -153,14 +151,14 @@ func writeExternalFile(s *writeTestSuite) { key, val, h := s.source.next() for key != nil { err := writer.WriteRow(ctx, key, val, h) - intest.Assert(err == nil) + intest.AssertNoError(err) key, val, h = s.source.next() } if s.beforeWriterClose != nil { s.beforeWriterClose() } err := writer.Close(ctx) - intest.Assert(err == nil) + intest.AssertNoError(err) if s.afterWriterClose != nil { s.afterWriterClose() } @@ -208,17 +206,17 @@ func TestCompareWriter(t *testing.T) { beforeTest := func() { fileIdx++ file, err = os.Create(fmt.Sprintf("cpu-profile-%d.prof", fileIdx)) - intest.Assert(err == nil) + intest.AssertNoError(err) err = pprof.StartCPUProfile(file) - intest.Assert(err == nil) + intest.AssertNoError(err) now = time.Now() } beforeClose := func() { file, err = os.Create(fmt.Sprintf("heap-profile-%d.prof", fileIdx)) - intest.Assert(err == nil) + intest.AssertNoError(err) // check heap profile to see the memory usage is expected err = pprof.WriteHeapProfile(file) - intest.Assert(err == nil) + intest.AssertNoError(err) } afterClose := func() { elapsed = time.Since(now) @@ -261,8 +259,8 @@ type readTestSuite struct { func readFileSequential(s *readTestSuite) { ctx := context.Background() - files, _, err := GetAllFileNames(ctx, s.store, "evenly_distributed") - intest.Assert(err == nil) + files, _, err := GetAllFileNames(ctx, s.store, "/evenly_distributed") + intest.AssertNoError(err) buf := make([]byte, s.memoryLimit) if s.beforeCreateReader != nil { @@ -270,7 +268,7 @@ func readFileSequential(s *readTestSuite) { } for i, file := range files { reader, err := s.store.Open(ctx, file, nil) - intest.Assert(err == nil) + intest.AssertNoError(err) _, err = reader.Read(buf) for err == nil { _, err = reader.Read(buf) @@ -282,7 +280,7 @@ func readFileSequential(s *readTestSuite) { } } err = reader.Close() - intest.Assert(err == nil) + intest.AssertNoError(err) } if s.afterReaderClose != nil { s.afterReaderClose() @@ -291,8 +289,8 @@ func readFileSequential(s *readTestSuite) { func readFileConcurrently(s *readTestSuite) { ctx := context.Background() - files, _, err := GetAllFileNames(ctx, s.store, "evenly_distributed") - intest.Assert(err == nil) + files, _, err := GetAllFileNames(ctx, s.store, "/evenly_distributed") + intest.AssertNoError(err) conc := min(s.concurrency, len(files)) var eg errgroup.Group @@ -306,7 +304,7 @@ func readFileConcurrently(s *readTestSuite) { eg.Go(func() error { buf := make([]byte, s.memoryLimit/conc) reader, err := s.store.Open(ctx, file, nil) - intest.Assert(err == nil) + intest.AssertNoError(err) _, err = reader.Read(buf) for err == nil { _, err = reader.Read(buf) @@ -318,12 +316,12 @@ func readFileConcurrently(s *readTestSuite) { } }) err = reader.Close() - intest.Assert(err == nil) + intest.AssertNoError(err) return nil }) } err = eg.Wait() - intest.Assert(err == nil) + intest.AssertNoError(err) if s.afterReaderClose != nil { s.afterReaderClose() } @@ -336,12 +334,12 @@ func createEvenlyDistributedFiles( store := openTestingStorage(t) ctx := context.Background() - files, statFiles, err := GetAllFileNames(ctx, store, "evenly_distributed") - intest.Assert(err == nil) + files, statFiles, err := GetAllFileNames(ctx, store, "/evenly_distributed") + intest.AssertNoError(err) err = store.DeleteFiles(ctx, files) - intest.Assert(err == nil) + intest.AssertNoError(err) err = store.DeleteFiles(ctx, statFiles) - intest.Assert(err == nil) + intest.AssertNoError(err) value := make([]byte, 100) kvCnt := 0 @@ -350,7 +348,7 @@ func createEvenlyDistributedFiles( SetMemorySizeLimit(uint64(float64(fileSize) * 1.1)) writer := builder.Build( store, - "evenly_distributed", + "/evenly_distributed", fmt.Sprintf("%d", i), ) @@ -359,21 +357,21 @@ func createEvenlyDistributedFiles( for totalSize < fileSize { key := fmt.Sprintf("key_%09d", keyIdx) err := writer.WriteRow(ctx, []byte(key), value, nil) - intest.Assert(err == nil) + intest.AssertNoError(err) keyIdx += fileCount totalSize += len(key) + len(value) kvCnt++ } err := writer.Close(ctx) - intest.Assert(err == nil) + intest.AssertNoError(err) } return store, kvCnt } func readMergeIter(s *readTestSuite) { ctx := context.Background() - files, _, err := GetAllFileNames(ctx, s.store, "evenly_distributed") - intest.Assert(err == nil) + files, _, err := GetAllFileNames(ctx, s.store, "/evenly_distributed") + intest.AssertNoError(err) if s.beforeCreateReader != nil { s.beforeCreateReader() @@ -382,7 +380,7 @@ func readMergeIter(s *readTestSuite) { readBufSize := s.memoryLimit / len(files) zeroOffsets := make([]uint64, len(files)) iter, err := NewMergeKVIter(ctx, files, zeroOffsets, s.store, readBufSize, false) - intest.Assert(err == nil) + intest.AssertNoError(err) kvCnt := 0 for iter.Next() { @@ -395,7 +393,7 @@ func readMergeIter(s *readTestSuite) { } intest.Assert(kvCnt == s.totalKVCnt) err = iter.Close() - intest.Assert(err == nil) + intest.AssertNoError(err) if s.afterReaderClose != nil { s.afterReaderClose() } @@ -416,17 +414,17 @@ func TestCompareReader(t *testing.T) { beforeTest := func() { fileIdx++ file, err = os.Create(fmt.Sprintf("cpu-profile-%d.prof", fileIdx)) - intest.Assert(err == nil) + intest.AssertNoError(err) err = pprof.StartCPUProfile(file) - intest.Assert(err == nil) + intest.AssertNoError(err) now = time.Now() } beforeClose := func() { file, err = os.Create(fmt.Sprintf("heap-profile-%d.prof", fileIdx)) - intest.Assert(err == nil) + intest.AssertNoError(err) // check heap profile to see the memory usage is expected err = pprof.WriteHeapProfile(file) - intest.Assert(err == nil) + intest.AssertNoError(err) } afterClose := func() { elapsed = time.Since(now) @@ -442,23 +440,24 @@ func TestCompareReader(t *testing.T) { beforeReaderClose: beforeClose, afterReaderClose: afterClose, } - readFileSequential(suite) + + readMergeIter(suite) t.Logf( - "sequential read speed for %d bytes: %.2f MB/s", + "merge iter read speed for %d bytes: %.2f MB/s", fileSize*fileCnt, float64(fileSize*fileCnt)/elapsed.Seconds()/1024/1024, ) - readFileConcurrently(suite) + readFileSequential(suite) t.Logf( - "concurrent read speed for %d bytes: %.2f MB/s", + "sequential read speed for %d bytes: %.2f MB/s", fileSize*fileCnt, float64(fileSize*fileCnt)/elapsed.Seconds()/1024/1024, ) - readMergeIter(suite) + readFileConcurrently(suite) t.Logf( - "merge iter read speed for %d bytes: %.2f MB/s", + "concurrent read speed for %d bytes: %.2f MB/s", fileSize*fileCnt, float64(fileSize*fileCnt)/elapsed.Seconds()/1024/1024, ) diff --git a/br/pkg/lightning/backend/external/byte_reader.go b/br/pkg/lightning/backend/external/byte_reader.go index bed2661f50764..1293f6e0336f5 100644 --- a/br/pkg/lightning/backend/external/byte_reader.go +++ b/br/pkg/lightning/backend/external/byte_reader.go @@ -68,8 +68,9 @@ func openStoreReaderAndSeek( store storage.ExternalStorage, name string, initFileOffset uint64, + prefetchSize int, ) (storage.ExternalFileReader, error) { - storageReader, err := store.Open(ctx, name, nil) + storageReader, err := store.Open(ctx, name, &storage.ReaderOption{PrefetchSize: prefetchSize}) if err != nil { return nil, err } diff --git a/br/pkg/lightning/backend/external/kv_reader.go b/br/pkg/lightning/backend/external/kv_reader.go index 7659ecd6bb4f6..3f295f3f4252c 100644 --- a/br/pkg/lightning/backend/external/kv_reader.go +++ b/br/pkg/lightning/backend/external/kv_reader.go @@ -36,11 +36,12 @@ func newKVReader( initFileOffset uint64, bufSize int, ) (*kvReader, error) { - sr, err := openStoreReaderAndSeek(ctx, store, name, initFileOffset) + oneThird := bufSize / 3 + sr, err := openStoreReaderAndSeek(ctx, store, name, initFileOffset, oneThird*2) if err != nil { return nil, err } - br, err := newByteReader(ctx, sr, bufSize) + br, err := newByteReader(ctx, sr, oneThird) if err != nil { return nil, err } diff --git a/br/pkg/lightning/backend/external/stat_reader.go b/br/pkg/lightning/backend/external/stat_reader.go index 512dd8df5f37c..a921a7f7a098b 100644 --- a/br/pkg/lightning/backend/external/stat_reader.go +++ b/br/pkg/lightning/backend/external/stat_reader.go @@ -26,7 +26,7 @@ type statsReader struct { } func newStatsReader(ctx context.Context, store storage.ExternalStorage, name string, bufSize int) (*statsReader, error) { - sr, err := openStoreReaderAndSeek(ctx, store, name, 0) + sr, err := openStoreReaderAndSeek(ctx, store, name, 0, 0) if err != nil { return nil, err } diff --git a/br/pkg/storage/BUILD.bazel b/br/pkg/storage/BUILD.bazel index ad58e18200e58..98ee1882691dd 100644 --- a/br/pkg/storage/BUILD.bazel +++ b/br/pkg/storage/BUILD.bazel @@ -28,6 +28,7 @@ go_library( "//br/pkg/logutil", "//pkg/sessionctx/variable", "//pkg/util/intest", + "//pkg/util/prefetch", "@com_github_aliyun_alibaba_cloud_sdk_go//sdk/auth/credentials", "@com_github_aliyun_alibaba_cloud_sdk_go//sdk/auth/credentials/providers", "@com_github_aws_aws_sdk_go//aws", diff --git a/br/pkg/storage/s3.go b/br/pkg/storage/s3.go index cfb3e8b5bba1a..e65b85cfcc034 100644 --- a/br/pkg/storage/s3.go +++ b/br/pkg/storage/s3.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/logutil" + "github.com/pingcap/tidb/pkg/util/prefetch" "github.com/spf13/pflag" "go.uber.org/zap" ) @@ -735,6 +736,9 @@ func (rs *S3Storage) Open(ctx context.Context, path string, o *ReaderOption) (Ex if err != nil { return nil, errors.Trace(err) } + if o != nil && o.PrefetchSize > 0 { + reader = prefetch.NewReader(reader, o.PrefetchSize) + } return &s3ObjectReader{ storage: rs, name: path, @@ -874,6 +878,9 @@ func (r *s3ObjectReader) Read(p []byte) (n int, err error) { if maxCnt > int64(len(p)) { maxCnt = int64(len(p)) } + if maxCnt == 0 { + return 0, io.EOF + } n, err = r.reader.Read(p[:maxCnt]) // TODO: maybe we should use !errors.Is(err, io.EOF) here to avoid error lint, but currently, pingcap/errors // doesn't implement this method yet. diff --git a/br/pkg/storage/storage.go b/br/pkg/storage/storage.go index 348bbd32a5a26..78d42bf170cfd 100644 --- a/br/pkg/storage/storage.go +++ b/br/pkg/storage/storage.go @@ -98,6 +98,8 @@ type ReaderOption struct { StartOffset *int64 // EndOffset is exclusive. And it's incompatible with Seek. EndOffset *int64 + // PrefetchSize will switch to NewPrefetchReader if value is positive. + PrefetchSize int } // ExternalStorage represents a kind of file system storage. @@ -196,6 +198,25 @@ func NewWithDefaultOpt(ctx context.Context, backend *backuppb.StorageBackend) (E return New(ctx, backend, &opts) } +// NewFromURL creates an ExternalStorage from URL. +func NewFromURL(ctx context.Context, uri string, opts *ExternalStorageOptions) (ExternalStorage, error) { + if len(uri) == 0 { + return nil, errors.Annotate(berrors.ErrStorageInvalidConfig, "empty store is not allowed") + } + u, err := ParseRawURL(uri) + if err != nil { + return nil, errors.Trace(err) + } + if u.Scheme == "memstore" { + return NewMemStorage(), nil + } + b, err := parseBackend(u, uri, nil) + if err != nil { + return nil, errors.Trace(err) + } + return New(ctx, b, opts) +} + // New creates an ExternalStorage with options. func New(ctx context.Context, backend *backuppb.StorageBackend, opts *ExternalStorageOptions) (ExternalStorage, error) { if opts == nil { diff --git a/br/pkg/storage/storage_test.go b/br/pkg/storage/storage_test.go index c6ca5c39b6a02..6183e98c19de2 100644 --- a/br/pkg/storage/storage_test.go +++ b/br/pkg/storage/storage_test.go @@ -3,6 +3,7 @@ package storage_test import ( + "context" "net/http" "testing" @@ -24,3 +25,10 @@ func TestDefaultHttpClient(t *testing.T) { require.Equal(t, int(concurrency), transport.MaxIdleConnsPerHost) require.Equal(t, int(concurrency), transport.MaxIdleConns) } + +func TestNewMemStorage(t *testing.T) { + url := "memstore://" + s, err := storage.NewFromURL(context.Background(), url, nil) + require.NoError(t, err) + require.IsType(t, (*storage.MemStorage)(nil), s) +} diff --git a/pkg/util/prefetch/BUILD.bazel b/pkg/util/prefetch/BUILD.bazel new file mode 100644 index 0000000000000..29b6ff8d31096 --- /dev/null +++ b/pkg/util/prefetch/BUILD.bazel @@ -0,0 +1,17 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "prefetch", + srcs = ["reader.go"], + importpath = "github.com/pingcap/tidb/pkg/util/prefetch", + visibility = ["//visibility:public"], +) + +go_test( + name = "prefetch_test", + timeout = "short", + srcs = ["reader_test.go"], + embed = [":prefetch"], + flaky = True, + deps = ["@com_github_stretchr_testify//require"], +) diff --git a/pkg/util/prefetch/reader.go b/pkg/util/prefetch/reader.go new file mode 100644 index 0000000000000..f61d5bf016764 --- /dev/null +++ b/pkg/util/prefetch/reader.go @@ -0,0 +1,113 @@ +// Copyright 2023 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 prefetch + +import ( + "bytes" + "io" + "sync" +) + +// Reader is a reader that prefetches data from the underlying reader. +type Reader struct { + r io.ReadCloser + curBufReader *bytes.Reader + buf [2][]byte + bufIdx int + bufCh chan []byte + err error // after bufCh is closed + wg sync.WaitGroup + + closed bool + closedCh chan struct{} +} + +// NewReader creates a new Reader. +func NewReader(r io.ReadCloser, prefetchSize int) io.ReadCloser { + ret := &Reader{ + r: r, + bufCh: make(chan []byte), + err: nil, + closedCh: make(chan struct{}), + } + ret.buf[0] = make([]byte, prefetchSize/2) + ret.buf[1] = make([]byte, prefetchSize/2) + ret.wg.Add(1) + go ret.run() + return ret +} + +func (r *Reader) run() { + defer r.wg.Done() + for { + r.bufIdx = (r.bufIdx + 1) % 2 + buf := r.buf[r.bufIdx] + n, err := r.r.Read(buf) + buf = buf[:n] + select { + case <-r.closedCh: + return + case r.bufCh <- buf: + } + if err != nil { + r.err = err + close(r.bufCh) + return + } + } +} + +// Read implements io.Reader. Read should not be called concurrently with Close. +func (r *Reader) Read(data []byte) (int, error) { + total := 0 + for { + if r.curBufReader == nil { + b, ok := <-r.bufCh + if !ok { + if total > 0 { + return total, nil + } + return 0, r.err + } + + r.curBufReader = bytes.NewReader(b) + } + + expected := len(data) + n, err := r.curBufReader.Read(data) + total += n + if n == expected { + return total, nil + } + + data = data[n:] + if err == io.EOF || r.curBufReader.Len() == 0 { + r.curBufReader = nil + continue + } + } +} + +// Close implements io.Closer. Close should not be called concurrently with Read. +func (r *Reader) Close() error { + if r.closed { + return nil + } + ret := r.r.Close() + close(r.closedCh) + r.wg.Wait() + r.closed = true + return ret +} diff --git a/pkg/util/prefetch/reader_test.go b/pkg/util/prefetch/reader_test.go new file mode 100644 index 0000000000000..0d2972ed80289 --- /dev/null +++ b/pkg/util/prefetch/reader_test.go @@ -0,0 +1,82 @@ +// Copyright 2023 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 prefetch + +import ( + "bytes" + "io" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestBasic(t *testing.T) { + source := bytes.NewReader([]byte("01234567890")) + r := NewReader(io.NopCloser(source), 3) + buf := make([]byte, 1) + n, err := r.Read(buf) + require.NoError(t, err) + require.EqualValues(t, 1, n) + require.EqualValues(t, "0", buf[:n]) + + buf = make([]byte, 2) + n, err = r.Read(buf) + require.NoError(t, err) + require.EqualValues(t, 2, n) + require.EqualValues(t, "12", buf[:n]) + + buf = make([]byte, 3) + n, err = r.Read(buf) + require.NoError(t, err) + require.EqualValues(t, 3, n) + require.EqualValues(t, "345", buf[:n]) + + buf = make([]byte, 4) + n, err = r.Read(buf) + require.NoError(t, err) + require.EqualValues(t, 4, n) + require.EqualValues(t, "6789", buf[:n]) + n, err = r.Read(buf) + require.NoError(t, err) + require.EqualValues(t, 1, n) + require.EqualValues(t, "0", buf[:n]) + _, err = r.Read(buf) + require.ErrorIs(t, err, io.EOF) + + source = bytes.NewReader([]byte("01234567890")) + r = NewReader(io.NopCloser(source), 3) + buf = make([]byte, 11) + n, err = r.Read(buf) + require.NoError(t, err) + require.EqualValues(t, 11, n) + _, err = r.Read(buf) + require.ErrorIs(t, err, io.EOF) + + source = bytes.NewReader([]byte("01234")) + r = NewReader(io.NopCloser(source), 100) + buf = make([]byte, 11) + n, err = r.Read(buf) + require.NoError(t, err) + require.EqualValues(t, 5, n) + _, err = r.Read(buf) + require.ErrorIs(t, err, io.EOF) +} + +func TestCloseBeforeDrainRead(t *testing.T) { + data := make([]byte, 1024) + r := NewReader(io.NopCloser(bytes.NewReader(data)), 2) + err := r.Close() + require.NoError(t, err) +} From 8243680d8ac6bdadc0511df9943247a3f376eb08 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Fri, 24 Nov 2023 13:47:44 +0800 Subject: [PATCH 34/36] planner: avoid to use `ScalarFunction.GetCtx` in some planner codes (#48794) close pingcap/tidb#48793 --- pkg/executor/aggfuncs/builder.go | 14 +- pkg/executor/distsql.go | 2 +- pkg/expression/aggregation/window_func.go | 6 +- pkg/expression/builtin_cast.go | 2 +- pkg/expression/builtin_convert_charset.go | 4 +- pkg/expression/constant_fold.go | 55 +++--- pkg/expression/constant_propagation.go | 4 +- pkg/expression/constant_test.go | 156 ++++++++++-------- pkg/expression/expression.go | 6 +- pkg/expression/expression_test.go | 20 +-- pkg/expression/grouping_sets_test.go | 8 +- pkg/expression/scalar_function.go | 4 +- pkg/expression/scalar_function_test.go | 49 +++--- pkg/expression/util.go | 57 ++++--- pkg/expression/util_test.go | 106 ++++++------ pkg/planner/cascades/transformation_rules.go | 15 +- pkg/planner/core/logical_plan_builder.go | 6 +- pkg/planner/core/logical_plans.go | 9 +- pkg/planner/core/optimizer_test.go | 1 + pkg/planner/core/physical_plans.go | 7 +- pkg/planner/core/resolve_indices.go | 5 +- .../core/rule_aggregation_push_down.go | 11 +- pkg/planner/core/rule_eliminate_projection.go | 3 +- pkg/planner/core/rule_predicate_push_down.go | 19 ++- .../core/rule_predicate_simplification.go | 10 +- pkg/planner/core/rule_topn_push_down.go | 3 +- 26 files changed, 309 insertions(+), 273 deletions(-) diff --git a/pkg/executor/aggfuncs/builder.go b/pkg/executor/aggfuncs/builder.go index 72ac5fadc08a5..9965976eda5b2 100644 --- a/pkg/executor/aggfuncs/builder.go +++ b/pkg/executor/aggfuncs/builder.go @@ -92,9 +92,9 @@ func BuildWindowFunctions(ctx sessionctx.Context, windowFuncDesc *aggregation.Ag case ast.WindowFuncCumeDist: return buildCumeDist(ordinal, orderByCols) case ast.WindowFuncNthValue: - return buildNthValue(windowFuncDesc, ordinal) + return buildNthValue(ctx, windowFuncDesc, ordinal) case ast.WindowFuncNtile: - return buildNtile(windowFuncDesc, ordinal) + return buildNtile(ctx, windowFuncDesc, ordinal) case ast.WindowFuncPercentRank: return buildPercentRank(ordinal, orderByCols) case ast.WindowFuncLead: @@ -668,22 +668,22 @@ func buildCumeDist(ordinal int, orderByCols []*expression.Column) AggFunc { return r } -func buildNthValue(aggFuncDesc *aggregation.AggFuncDesc, ordinal int) AggFunc { +func buildNthValue(ctx sessionctx.Context, aggFuncDesc *aggregation.AggFuncDesc, ordinal int) AggFunc { base := baseAggFunc{ args: aggFuncDesc.Args, ordinal: ordinal, } // Already checked when building the function description. - nth, _, _ := expression.GetUint64FromConstant(aggFuncDesc.Args[1]) + nth, _, _ := expression.GetUint64FromConstant(ctx, aggFuncDesc.Args[1]) return &nthValue{baseAggFunc: base, tp: aggFuncDesc.RetTp, nth: nth} } -func buildNtile(aggFuncDes *aggregation.AggFuncDesc, ordinal int) AggFunc { +func buildNtile(ctx sessionctx.Context, aggFuncDes *aggregation.AggFuncDesc, ordinal int) AggFunc { base := baseAggFunc{ args: aggFuncDes.Args, ordinal: ordinal, } - n, _, _ := expression.GetUint64FromConstant(aggFuncDes.Args[0]) + n, _, _ := expression.GetUint64FromConstant(ctx, aggFuncDes.Args[0]) return &ntile{baseAggFunc: base, n: n} } @@ -697,7 +697,7 @@ func buildPercentRank(ordinal int, orderByCols []*expression.Column) AggFunc { func buildLeadLag(ctx sessionctx.Context, aggFuncDesc *aggregation.AggFuncDesc, ordinal int) baseLeadLag { offset := uint64(1) if len(aggFuncDesc.Args) >= 2 { - offset, _, _ = expression.GetUint64FromConstant(aggFuncDesc.Args[1]) + offset, _, _ = expression.GetUint64FromConstant(ctx, aggFuncDesc.Args[1]) } var defaultExpr expression.Expression defaultExpr = expression.NewNull() diff --git a/pkg/executor/distsql.go b/pkg/executor/distsql.go index 925049d40d61d..12555d386cc37 100644 --- a/pkg/executor/distsql.go +++ b/pkg/executor/distsql.go @@ -151,7 +151,7 @@ func closeAll(objs ...Closeable) error { func rebuildIndexRanges(ctx sessionctx.Context, is *plannercore.PhysicalIndexScan, idxCols []*expression.Column, colLens []int) (ranges []*ranger.Range, err error) { access := make([]expression.Expression, 0, len(is.AccessCondition)) for _, cond := range is.AccessCondition { - newCond, err1 := expression.SubstituteCorCol2Constant(cond) + newCond, err1 := expression.SubstituteCorCol2Constant(ctx, cond) if err1 != nil { return nil, err1 } diff --git a/pkg/expression/aggregation/window_func.go b/pkg/expression/aggregation/window_func.go index 1ce06b72d6d2a..ac6ff1f5dfdbc 100644 --- a/pkg/expression/aggregation/window_func.go +++ b/pkg/expression/aggregation/window_func.go @@ -36,13 +36,13 @@ func NewWindowFuncDesc(ctx sessionctx.Context, name string, args []expression.Ex if !skipCheckArgs { switch strings.ToLower(name) { case ast.WindowFuncNthValue: - val, isNull, ok := expression.GetUint64FromConstant(args[1]) + val, isNull, ok := expression.GetUint64FromConstant(ctx, args[1]) // nth_value does not allow `0`, but allows `null`. if !ok || (val == 0 && !isNull) { return nil, nil } case ast.WindowFuncNtile: - val, isNull, ok := expression.GetUint64FromConstant(args[0]) + val, isNull, ok := expression.GetUint64FromConstant(ctx, args[0]) // ntile does not allow `0`, but allows `null`. if !ok || (val == 0 && !isNull) { return nil, nil @@ -51,7 +51,7 @@ func NewWindowFuncDesc(ctx sessionctx.Context, name string, args []expression.Ex if len(args) < 2 { break } - _, isNull, ok := expression.GetUint64FromConstant(args[1]) + _, isNull, ok := expression.GetUint64FromConstant(ctx, args[1]) if !ok || isNull { return nil, nil } diff --git a/pkg/expression/builtin_cast.go b/pkg/expression/builtin_cast.go index e9b81745b2aee..d5ec84df6a735 100644 --- a/pkg/expression/builtin_cast.go +++ b/pkg/expression/builtin_cast.go @@ -2153,7 +2153,7 @@ func BuildCastFunctionWithCheck(ctx sessionctx.Context, expr Expression, tp *typ // since we may reset the flag of the field type of CastAsJson later which // would affect the evaluation of it. if tp.EvalType() != types.ETJson && err == nil { - res = FoldConstant(res) + res = FoldConstant(ctx, res) } return res, err } diff --git a/pkg/expression/builtin_convert_charset.go b/pkg/expression/builtin_convert_charset.go index c515c19d20807..7ecfd8d0fa5e3 100644 --- a/pkg/expression/builtin_convert_charset.go +++ b/pkg/expression/builtin_convert_charset.go @@ -230,7 +230,7 @@ func BuildToBinaryFunction(ctx sessionctx.Context, expr Expression) (res Express Function: f, ctx: ctx, } - return FoldConstant(res) + return FoldConstant(ctx, res) } // BuildFromBinaryFunction builds from_binary function. @@ -246,7 +246,7 @@ func BuildFromBinaryFunction(ctx sessionctx.Context, expr Expression, tp *types. Function: f, ctx: ctx, } - return FoldConstant(res) + return FoldConstant(ctx, res) } type funcProp int8 diff --git a/pkg/expression/constant_fold.go b/pkg/expression/constant_fold.go index 70bd26577d37e..aae42545248f5 100644 --- a/pkg/expression/constant_fold.go +++ b/pkg/expression/constant_fold.go @@ -17,6 +17,7 @@ package expression import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/logutil" @@ -24,10 +25,10 @@ import ( ) // specialFoldHandler stores functions for special UDF to constant fold -var specialFoldHandler = map[string]func(*ScalarFunction) (Expression, bool){} +var specialFoldHandler = map[string]func(sessionctx.Context, *ScalarFunction) (Expression, bool){} func init() { - specialFoldHandler = map[string]func(*ScalarFunction) (Expression, bool){ + specialFoldHandler = map[string]func(sessionctx.Context, *ScalarFunction) (Expression, bool){ ast.If: ifFoldHandler, ast.Ifnull: ifNullFoldHandler, ast.Case: caseWhenHandler, @@ -36,8 +37,8 @@ func init() { } // FoldConstant does constant folding optimization on an expression excluding deferred ones. -func FoldConstant(expr Expression) Expression { - e, _ := foldConstant(expr) +func FoldConstant(ctx sessionctx.Context, expr Expression) Expression { + e, _ := foldConstant(ctx, expr) // keep the original coercibility, charset, collation and repertoire values after folding e.SetCoercibility(expr.Coercibility()) @@ -48,11 +49,11 @@ func FoldConstant(expr Expression) Expression { return e } -func isNullHandler(expr *ScalarFunction) (Expression, bool) { +func isNullHandler(ctx sessionctx.Context, expr *ScalarFunction) (Expression, bool) { arg0 := expr.GetArgs()[0] if constArg, isConst := arg0.(*Constant); isConst { isDeferredConst := constArg.DeferredExpr != nil || constArg.ParamMarker != nil - value, err := expr.EvalWithInnerCtx(chunk.Row{}) + value, err := expr.Eval(ctx, chunk.Row{}) if err != nil { // Failed to fold this expr to a constant, print the DEBUG log and // return the original expression to let the error to be evaluated @@ -71,11 +72,11 @@ func isNullHandler(expr *ScalarFunction) (Expression, bool) { return expr, false } -func ifFoldHandler(expr *ScalarFunction) (Expression, bool) { +func ifFoldHandler(ctx sessionctx.Context, expr *ScalarFunction) (Expression, bool) { args := expr.GetArgs() - foldedArg0, _ := foldConstant(args[0]) + foldedArg0, _ := foldConstant(ctx, args[0]) if constArg, isConst := foldedArg0.(*Constant); isConst { - arg0, isNull0, err := constArg.EvalInt(expr.GetCtx(), chunk.Row{}) + arg0, isNull0, err := constArg.EvalInt(ctx, chunk.Row{}) if err != nil { // Failed to fold this expr to a constant, print the DEBUG log and // return the original expression to let the error to be evaluated @@ -84,23 +85,23 @@ func ifFoldHandler(expr *ScalarFunction) (Expression, bool) { return expr, false } if !isNull0 && arg0 != 0 { - return foldConstant(args[1]) + return foldConstant(ctx, args[1]) } - return foldConstant(args[2]) + return foldConstant(ctx, args[2]) } // if the condition is not const, which branch is unknown to run, so directly return. return expr, false } -func ifNullFoldHandler(expr *ScalarFunction) (Expression, bool) { +func ifNullFoldHandler(ctx sessionctx.Context, expr *ScalarFunction) (Expression, bool) { args := expr.GetArgs() - foldedArg0, isDeferred := foldConstant(args[0]) + foldedArg0, isDeferred := foldConstant(ctx, args[0]) if constArg, isConst := foldedArg0.(*Constant); isConst { // Only check constArg.Value here. Because deferred expression is // evaluated to constArg.Value after foldConstant(args[0]), it's not // needed to be checked. if constArg.Value.IsNull() { - return foldConstant(args[1]) + return foldConstant(ctx, args[1]) } return constArg, isDeferred } @@ -108,11 +109,11 @@ func ifNullFoldHandler(expr *ScalarFunction) (Expression, bool) { return expr, false } -func caseWhenHandler(expr *ScalarFunction) (Expression, bool) { +func caseWhenHandler(ctx sessionctx.Context, expr *ScalarFunction) (Expression, bool) { args, l := expr.GetArgs(), len(expr.GetArgs()) var isDeferred, isDeferredConst bool for i := 0; i < l-1; i += 2 { - expr.GetArgs()[i], isDeferred = foldConstant(args[i]) + expr.GetArgs()[i], isDeferred = foldConstant(ctx, args[i]) isDeferredConst = isDeferredConst || isDeferred if _, isConst := expr.GetArgs()[i].(*Constant); !isConst { // for no-const, here should return directly, because the following branches are unknown to be run or not @@ -121,12 +122,12 @@ func caseWhenHandler(expr *ScalarFunction) (Expression, bool) { // If the condition is const and true, and the previous conditions // has no expr, then the folded execution body is returned, otherwise // the arguments of the casewhen are folded and replaced. - val, isNull, err := args[i].EvalInt(expr.GetCtx(), chunk.Row{}) + val, isNull, err := args[i].EvalInt(ctx, chunk.Row{}) if err != nil { return expr, false } if val != 0 && !isNull { - foldedExpr, isDeferred := foldConstant(args[i+1]) + foldedExpr, isDeferred := foldConstant(ctx, args[i+1]) isDeferredConst = isDeferredConst || isDeferred if _, isConst := foldedExpr.(*Constant); isConst { foldedExpr.GetType().SetDecimal(expr.GetType().GetDecimal()) @@ -139,7 +140,7 @@ func caseWhenHandler(expr *ScalarFunction) (Expression, bool) { // is false, then the folded else execution body is returned. otherwise // the execution body of the else are folded and replaced. if l%2 == 1 { - foldedExpr, isDeferred := foldConstant(args[l-1]) + foldedExpr, isDeferred := foldConstant(ctx, args[l-1]) isDeferredConst = isDeferredConst || isDeferred if _, isConst := foldedExpr.(*Constant); isConst { foldedExpr.GetType().SetDecimal(expr.GetType().GetDecimal()) @@ -150,18 +151,18 @@ func caseWhenHandler(expr *ScalarFunction) (Expression, bool) { return expr, isDeferredConst } -func foldConstant(expr Expression) (Expression, bool) { +func foldConstant(ctx sessionctx.Context, expr Expression) (Expression, bool) { switch x := expr.(type) { case *ScalarFunction: if _, ok := unFoldableFunctions[x.FuncName.L]; ok { return expr, false } - if function := specialFoldHandler[x.FuncName.L]; function != nil && !MaybeOverOptimized4PlanCache(x.GetCtx(), []Expression{expr}) { - return function(x) + if function := specialFoldHandler[x.FuncName.L]; function != nil && !MaybeOverOptimized4PlanCache(ctx, []Expression{expr}) { + return function(ctx, x) } args := x.GetArgs() - sc := x.GetCtx().GetSessionVars().StmtCtx + sc := ctx.GetSessionVars().StmtCtx argIsConst := make([]bool, len(args)) hasNullArg := false allConstArg := true @@ -193,11 +194,11 @@ func foldConstant(expr Expression) (Expression, bool) { constArgs[i] = NewOne() } } - dummyScalarFunc, err := NewFunctionBase(x.GetCtx(), x.FuncName.L, x.GetType(), constArgs...) + dummyScalarFunc, err := NewFunctionBase(ctx, x.FuncName.L, x.GetType(), constArgs...) if err != nil { return expr, isDeferredConst } - value, err := dummyScalarFunc.EvalWithInnerCtx(chunk.Row{}) + value, err := dummyScalarFunc.Eval(ctx, chunk.Row{}) if err != nil { return expr, isDeferredConst } @@ -217,7 +218,7 @@ func foldConstant(expr Expression) (Expression, bool) { } return expr, isDeferredConst } - value, err := x.EvalWithInnerCtx(chunk.Row{}) + value, err := x.Eval(ctx, chunk.Row{}) retType := x.RetType.Clone() if !hasNullArg { // set right not null flag for constant value @@ -245,7 +246,7 @@ func foldConstant(expr Expression) (Expression, bool) { ParamMarker: x.ParamMarker, }, true } else if x.DeferredExpr != nil { - value, err := x.DeferredExpr.EvalWithInnerCtx(chunk.Row{}) + value, err := x.DeferredExpr.Eval(ctx, chunk.Row{}) if err != nil { logutil.BgLogger().Debug("fold expression to constant", zap.String("expression", x.ExplainInfo()), zap.Error(err)) return expr, true diff --git a/pkg/expression/constant_propagation.go b/pkg/expression/constant_propagation.go index c69b2d3867717..0e6b843df0d7c 100644 --- a/pkg/expression/constant_propagation.go +++ b/pkg/expression/constant_propagation.go @@ -212,7 +212,7 @@ func (s *propConstSolver) propagateConstantEQ() { } for i, cond := range s.conditions { if !visited[i] { - s.conditions[i] = ColumnSubstitute(cond, NewSchema(cols...), cons) + s.conditions[i] = ColumnSubstitute(s.ctx, cond, NewSchema(cols...), cons) } } } @@ -470,7 +470,7 @@ func (s *propOuterJoinConstSolver) propagateConstantEQ() { } for i, cond := range s.joinConds { if !visited[i+lenFilters] { - s.joinConds[i] = ColumnSubstitute(cond, NewSchema(cols...), cons) + s.joinConds[i] = ColumnSubstitute(s.ctx, cond, NewSchema(cols...), cons) } } } diff --git a/pkg/expression/constant_test.go b/pkg/expression/constant_test.go index e3cee448792c3..9dc8f2f9aede9 100644 --- a/pkg/expression/constant_test.go +++ b/pkg/expression/constant_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/mock" @@ -54,12 +55,16 @@ func newString(value string, collation string) *Constant { } } -func newFunction(funcName string, args ...Expression) Expression { - return newFunctionWithType(funcName, types.NewFieldType(mysql.TypeLonglong), args...) +func newFunctionWithMockCtx(funcName string, args ...Expression) Expression { + return newFunction(mock.NewContext(), funcName, args...) } -func newFunctionWithType(funcName string, tp *types.FieldType, args ...Expression) Expression { - return NewFunctionInternal(mock.NewContext(), funcName, tp, args...) +func newFunction(ctx sessionctx.Context, funcName string, args ...Expression) Expression { + return newFunctionWithType(ctx, funcName, types.NewFieldType(mysql.TypeLonglong), args...) +} + +func newFunctionWithType(ctx sessionctx.Context, funcName string, tp *types.FieldType, args ...Expression) Expression { + return NewFunctionInternal(ctx, funcName, tp, args...) } func TestConstantPropagation(t *testing.T) { @@ -71,59 +76,59 @@ func TestConstantPropagation(t *testing.T) { { solver: []PropagateConstantSolver{newPropConstSolver()}, conditions: []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.EQ, newColumn(1), newColumn(2)), - newFunction(ast.EQ, newColumn(2), newColumn(3)), - newFunction(ast.EQ, newColumn(3), newLonglong(1)), - newFunction(ast.LogicOr, newLonglong(1), newColumn(0)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.EQ, newColumn(1), newColumn(2)), + newFunctionWithMockCtx(ast.EQ, newColumn(2), newColumn(3)), + newFunctionWithMockCtx(ast.EQ, newColumn(3), newLonglong(1)), + newFunctionWithMockCtx(ast.LogicOr, newLonglong(1), newColumn(0)), }, result: "1, eq(Column#0, 1), eq(Column#1, 1), eq(Column#2, 1), eq(Column#3, 1)", }, { solver: []PropagateConstantSolver{newPropConstSolver()}, conditions: []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.EQ, newColumn(1), newLonglong(1)), - newFunction(ast.NE, newColumn(2), newLonglong(2)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.EQ, newColumn(1), newLonglong(1)), + newFunctionWithMockCtx(ast.NE, newColumn(2), newLonglong(2)), }, result: "eq(Column#0, 1), eq(Column#1, 1), ne(Column#2, 2)", }, { solver: []PropagateConstantSolver{newPropConstSolver()}, conditions: []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.EQ, newColumn(1), newLonglong(1)), - newFunction(ast.EQ, newColumn(2), newColumn(3)), - newFunction(ast.GE, newColumn(2), newLonglong(2)), - newFunction(ast.NE, newColumn(2), newLonglong(4)), - newFunction(ast.NE, newColumn(3), newLonglong(5)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.EQ, newColumn(1), newLonglong(1)), + newFunctionWithMockCtx(ast.EQ, newColumn(2), newColumn(3)), + newFunctionWithMockCtx(ast.GE, newColumn(2), newLonglong(2)), + newFunctionWithMockCtx(ast.NE, newColumn(2), newLonglong(4)), + newFunctionWithMockCtx(ast.NE, newColumn(3), newLonglong(5)), }, result: "eq(Column#0, 1), eq(Column#1, 1), eq(Column#2, Column#3), ge(Column#2, 2), ge(Column#3, 2), ne(Column#2, 4), ne(Column#2, 5), ne(Column#3, 4), ne(Column#3, 5)", }, { solver: []PropagateConstantSolver{newPropConstSolver()}, conditions: []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.EQ, newColumn(0), newColumn(2)), - newFunction(ast.GE, newColumn(1), newLonglong(0)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(2)), + newFunctionWithMockCtx(ast.GE, newColumn(1), newLonglong(0)), }, result: "eq(Column#0, Column#1), eq(Column#0, Column#2), ge(Column#0, 0), ge(Column#1, 0), ge(Column#2, 0)", }, { solver: []PropagateConstantSolver{newPropConstSolver()}, conditions: []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.GT, newColumn(0), newLonglong(2)), - newFunction(ast.GT, newColumn(1), newLonglong(3)), - newFunction(ast.LT, newColumn(0), newLonglong(1)), - newFunction(ast.GT, newLonglong(2), newColumn(1)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.GT, newColumn(0), newLonglong(2)), + newFunctionWithMockCtx(ast.GT, newColumn(1), newLonglong(3)), + newFunctionWithMockCtx(ast.LT, newColumn(0), newLonglong(1)), + newFunctionWithMockCtx(ast.GT, newLonglong(2), newColumn(1)), }, result: "eq(Column#0, Column#1), gt(2, Column#0), gt(2, Column#1), gt(Column#0, 2), gt(Column#0, 3), gt(Column#1, 2), gt(Column#1, 3), lt(Column#0, 1), lt(Column#1, 1)", }, { solver: []PropagateConstantSolver{newPropConstSolver()}, conditions: []Expression{ - newFunction(ast.EQ, newLonglong(1), newColumn(0)), + newFunctionWithMockCtx(ast.EQ, newLonglong(1), newColumn(0)), newLonglong(0), }, result: "0", @@ -131,41 +136,41 @@ func TestConstantPropagation(t *testing.T) { { solver: []PropagateConstantSolver{newPropConstSolver()}, conditions: []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.In, newColumn(0), newLonglong(1), newLonglong(2)), - newFunction(ast.In, newColumn(1), newLonglong(3), newLonglong(4)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.In, newColumn(0), newLonglong(1), newLonglong(2)), + newFunctionWithMockCtx(ast.In, newColumn(1), newLonglong(3), newLonglong(4)), }, result: "eq(Column#0, Column#1), in(Column#0, 1, 2), in(Column#0, 3, 4), in(Column#1, 1, 2), in(Column#1, 3, 4)", }, { solver: []PropagateConstantSolver{newPropConstSolver()}, conditions: []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.EQ, newColumn(0), newFunction(ast.BitLength, newColumn(2))), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newFunctionWithMockCtx(ast.BitLength, newColumn(2))), }, result: "eq(Column#0, Column#1), eq(Column#0, bit_length(cast(Column#2, var_string(20)))), eq(Column#1, bit_length(cast(Column#2, var_string(20))))", }, { solver: []PropagateConstantSolver{newPropConstSolver()}, conditions: []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.LE, newFunction(ast.Mul, newColumn(0), newColumn(0)), newLonglong(50)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.LE, newFunctionWithMockCtx(ast.Mul, newColumn(0), newColumn(0)), newLonglong(50)), }, result: "eq(Column#0, Column#1), le(mul(Column#0, Column#0), 50), le(mul(Column#1, Column#1), 50)", }, { solver: []PropagateConstantSolver{newPropConstSolver()}, conditions: []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.LE, newColumn(0), newFunction(ast.Plus, newColumn(1), newLonglong(1))), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.LE, newColumn(0), newFunctionWithMockCtx(ast.Plus, newColumn(1), newLonglong(1))), }, result: "eq(Column#0, Column#1), le(Column#0, plus(Column#0, 1)), le(Column#0, plus(Column#1, 1)), le(Column#1, plus(Column#1, 1))", }, { solver: []PropagateConstantSolver{newPropConstSolver()}, conditions: []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.LE, newColumn(0), newFunction(ast.Rand)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.LE, newColumn(0), newFunctionWithMockCtx(ast.Rand)), }, result: "eq(Column#0, Column#1), le(cast(Column#0, double BINARY), rand())", }, @@ -175,7 +180,7 @@ func TestConstantPropagation(t *testing.T) { ctx := mock.NewContext() conds := make([]Expression, 0, len(tt.conditions)) for _, cd := range tt.conditions { - conds = append(conds, FoldConstant(cd)) + conds = append(conds, FoldConstant(ctx, cd)) } newConds := solver.PropagateConstant(ctx, conds) var result []string @@ -190,75 +195,93 @@ func TestConstantPropagation(t *testing.T) { func TestConstantFolding(t *testing.T) { tests := []struct { - condition Expression + condition func(ctx sessionctx.Context) Expression result string }{ { - condition: newFunction(ast.LT, newColumn(0), newFunction(ast.Plus, newLonglong(1), newLonglong(2))), - result: "lt(Column#0, 3)", + condition: func(ctx sessionctx.Context) Expression { + return newFunction(ctx, ast.LT, newColumn(0), newFunction(ctx, ast.Plus, newLonglong(1), newLonglong(2))) + }, + result: "lt(Column#0, 3)", }, { - condition: newFunction(ast.LT, newColumn(0), newFunction(ast.Greatest, newLonglong(1), newLonglong(2))), - result: "lt(Column#0, 2)", + condition: func(ctx sessionctx.Context) Expression { + return newFunction(ctx, ast.LT, newColumn(0), newFunction(ctx, ast.Greatest, newLonglong(1), newLonglong(2))) + }, + result: "lt(Column#0, 2)", }, { - condition: newFunction(ast.EQ, newColumn(0), newFunction(ast.Rand)), - result: "eq(cast(Column#0, double BINARY), rand())", + condition: func(ctx sessionctx.Context) Expression { + return newFunction(ctx, ast.EQ, newColumn(0), newFunction(ctx, ast.Rand)) + }, + result: "eq(cast(Column#0, double BINARY), rand())", }, { - condition: newFunction(ast.IsNull, newLonglong(1)), - result: "0", + condition: func(ctx sessionctx.Context) Expression { + return newFunction(ctx, ast.IsNull, newLonglong(1)) + }, + result: "0", }, { - condition: newFunction(ast.EQ, newColumn(0), newFunction(ast.UnaryNot, newFunction(ast.Plus, newLonglong(1), newLonglong(1)))), - result: "eq(Column#0, 0)", + condition: func(ctx sessionctx.Context) Expression { + return newFunction(ctx, ast.EQ, newColumn(0), newFunction(ctx, ast.UnaryNot, newFunctionWithMockCtx(ast.Plus, newLonglong(1), newLonglong(1)))) + }, + result: "eq(Column#0, 0)", }, { - condition: newFunction(ast.LT, newColumn(0), newFunction(ast.Plus, newColumn(1), newFunction(ast.Plus, newLonglong(2), newLonglong(1)))), - result: "lt(Column#0, plus(Column#1, 3))", + condition: func(ctx sessionctx.Context) Expression { + return newFunction(ctx, ast.LT, newColumn(0), newFunction(ctx, ast.Plus, newColumn(1), newFunctionWithMockCtx(ast.Plus, newLonglong(2), newLonglong(1)))) + }, + result: "lt(Column#0, plus(Column#1, 3))", }, { - condition: func() Expression { - expr := newFunction(ast.ConcatWS, newColumn(0), NewNull()) - function := expr.(*ScalarFunction) - function.GetCtx().GetSessionVars().StmtCtx.InNullRejectCheck = true - return function - }(), + condition: func(ctx sessionctx.Context) Expression { + expr := newFunction(ctx, ast.ConcatWS, newColumn(0), NewNull()) + ctx.GetSessionVars().StmtCtx.InNullRejectCheck = true + return expr + }, result: "concat_ws(cast(Column#0, var_string(20)), )", }, } for _, tt := range tests { - newConds := FoldConstant(tt.condition) + ctx := mock.NewContext() + expr := tt.condition(ctx) + newConds := FoldConstant(ctx, expr) require.Equalf(t, tt.result, newConds.String(), "different for expr %s", tt.condition) } } func TestConstantFoldingCharsetConvert(t *testing.T) { + ctx := mock.NewContext() tests := []struct { condition Expression result string }{ { - condition: newFunction(ast.Length, newFunctionWithType( + condition: newFunction(ctx, ast.Length, newFunctionWithType( + ctx, InternalFuncToBinary, types.NewFieldType(mysql.TypeVarchar), newString("中文", "gbk_bin"))), result: "4", }, { - condition: newFunction(ast.Length, newFunctionWithType( + condition: newFunction(ctx, ast.Length, newFunctionWithType( + ctx, InternalFuncToBinary, types.NewFieldType(mysql.TypeVarchar), newString("中文", "utf8mb4_bin"))), result: "6", }, { - condition: newFunction(ast.Concat, newFunctionWithType( + condition: newFunction(ctx, ast.Concat, newFunctionWithType( + ctx, InternalFuncFromBinary, types.NewFieldType(mysql.TypeVarchar), newString("中文", "binary"))), result: "中文", }, { - condition: newFunction(ast.Concat, + condition: newFunction(ctx, ast.Concat, newFunctionWithType( + ctx, InternalFuncFromBinary, types.NewFieldTypeWithCollation(mysql.TypeVarchar, "gbk_bin", -1), newString("\xd2\xbb", "binary")), newString("中文", "gbk_bin"), @@ -266,9 +289,10 @@ func TestConstantFoldingCharsetConvert(t *testing.T) { result: "一中文", }, { - condition: newFunction(ast.Concat, + condition: newFunction(ctx, ast.Concat, newString("中文", "gbk_bin"), newFunctionWithType( + ctx, InternalFuncFromBinary, types.NewFieldTypeWithCollation(mysql.TypeVarchar, "gbk_bin", -1), newString("\xd2\xbb", "binary")), ), @@ -276,7 +300,7 @@ func TestConstantFoldingCharsetConvert(t *testing.T) { }, // The result is binary charset, so gbk constant will convert to binary which is \xd6\xd0\xce\xc4. { - condition: newFunction(ast.Concat, + condition: newFunction(ctx, ast.Concat, newString("中文", "gbk_bin"), newString("\xd2\xbb", "binary"), ), @@ -284,7 +308,7 @@ func TestConstantFoldingCharsetConvert(t *testing.T) { }, } for _, tt := range tests { - newConds := FoldConstant(tt.condition) + newConds := FoldConstant(ctx, tt.condition) require.Equalf(t, tt.result, newConds.String(), "different for expr %s", tt.condition) } } diff --git a/pkg/expression/expression.go b/pkg/expression/expression.go index ff48cae555dff..7e844660abc91 100644 --- a/pkg/expression/expression.go +++ b/pkg/expression/expression.go @@ -857,7 +857,7 @@ func evaluateExprWithNull(ctx sessionctx.Context, schema *Schema, expr Expressio return &Constant{Value: types.Datum{}, RetType: types.NewFieldType(mysql.TypeNull)} case *Constant: if x.DeferredExpr != nil { - return FoldConstant(x) + return FoldConstant(ctx, x) } } return expr @@ -922,7 +922,7 @@ func evaluateExprWithNullInNullRejectCheck(ctx sessionctx.Context, schema *Schem return &Constant{Value: types.Datum{}, RetType: types.NewFieldType(mysql.TypeNull)}, true case *Constant: if x.DeferredExpr != nil { - return FoldConstant(x), false + return FoldConstant(ctx, x), false } } return expr, false @@ -1526,7 +1526,7 @@ func wrapWithIsTrue(ctx sessionctx.Context, keepNull bool, arg Expression, wrapF if keepNull { sf.FuncName = model.NewCIStr(ast.IsTruthWithNull) } - return FoldConstant(sf), nil + return FoldConstant(ctx, sf), nil } // PropagateType propagates the type information to the `expr`. diff --git a/pkg/expression/expression_test.go b/pkg/expression/expression_test.go index 11c12539a1b13..28c1974dfaea2 100644 --- a/pkg/expression/expression_test.go +++ b/pkg/expression/expression_test.go @@ -135,19 +135,19 @@ func TestIsBinaryLiteral(t *testing.T) { func TestConstItem(t *testing.T) { ctx := createContext(t) - sf := newFunction(ast.Rand) + sf := newFunctionWithMockCtx(ast.Rand) require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx)) - sf = newFunction(ast.UUID) + sf = newFunctionWithMockCtx(ast.UUID) require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx)) - sf = newFunction(ast.GetParam, NewOne()) + sf = newFunctionWithMockCtx(ast.GetParam, NewOne()) require.False(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx)) - sf = newFunction(ast.Abs, NewOne()) + sf = newFunctionWithMockCtx(ast.Abs, NewOne()) require.True(t, sf.ConstItem(ctx.GetSessionVars().StmtCtx)) } func TestVectorizable(t *testing.T) { exprs := make([]Expression, 0, 4) - sf := newFunction(ast.Rand) + sf := newFunctionWithMockCtx(ast.Rand) column := &Column{ UniqueID: 0, RetType: types.NewFieldType(mysql.TypeLonglong), @@ -171,21 +171,21 @@ func TestVectorizable(t *testing.T) { RetType: types.NewFieldType(mysql.TypeLonglong), } exprs = exprs[:0] - sf = newFunction(ast.SetVar, column0, column1) + sf = newFunctionWithMockCtx(ast.SetVar, column0, column1) exprs = append(exprs, sf) require.False(t, Vectorizable(exprs)) exprs = exprs[:0] - sf = newFunction(ast.GetVar, column0) + sf = newFunctionWithMockCtx(ast.GetVar, column0) exprs = append(exprs, sf) require.False(t, Vectorizable(exprs)) exprs = exprs[:0] - sf = newFunction(ast.NextVal, column0) + sf = newFunctionWithMockCtx(ast.NextVal, column0) exprs = append(exprs, sf) - sf = newFunction(ast.LastVal, column0) + sf = newFunctionWithMockCtx(ast.LastVal, column0) exprs = append(exprs, sf) - sf = newFunction(ast.SetVal, column1, column2) + sf = newFunctionWithMockCtx(ast.SetVal, column1, column2) exprs = append(exprs, sf) require.False(t, Vectorizable(exprs)) } diff --git a/pkg/expression/grouping_sets_test.go b/pkg/expression/grouping_sets_test.go index 68bf202860220..f61f00ccd032c 100644 --- a/pkg/expression/grouping_sets_test.go +++ b/pkg/expression/grouping_sets_test.go @@ -114,25 +114,25 @@ func TestGroupSetsTargetOneCompoundArgs(t *testing.T) { require.Equal(t, offset, 0) // default // mock normal agg count(d+1) - normalAggArgs = newFunction(ast.Plus, d, newLonglong(1)) + normalAggArgs = newFunctionWithMockCtx(ast.Plus, d, newLonglong(1)) offset = newGroupingSets.TargetOne([]Expression{normalAggArgs}) require.NotEqual(t, offset, -1) require.Equal(t, offset, 0) // default // mock normal agg count(d+c) - normalAggArgs = newFunction(ast.Plus, d, c) + normalAggArgs = newFunctionWithMockCtx(ast.Plus, d, c) offset = newGroupingSets.TargetOne([]Expression{normalAggArgs}) require.NotEqual(t, offset, -1) require.Equal(t, offset, 1) // only {c} can supply d and c // mock normal agg count(d+a) - normalAggArgs = newFunction(ast.Plus, d, a) + normalAggArgs = newFunctionWithMockCtx(ast.Plus, d, a) offset = newGroupingSets.TargetOne([]Expression{normalAggArgs}) require.NotEqual(t, offset, -1) require.Equal(t, offset, 0) // only {a,b} can supply d and a // mock normal agg count(d+a+c) - normalAggArgs = newFunction(ast.Plus, d, newFunction(ast.Plus, a, c)) + normalAggArgs = newFunctionWithMockCtx(ast.Plus, d, newFunctionWithMockCtx(ast.Plus, a, c)) offset = newGroupingSets.TargetOne([]Expression{normalAggArgs}) require.Equal(t, offset, -1) // couldn't find a group that supply d, a and c simultaneously. } diff --git a/pkg/expression/scalar_function.go b/pkg/expression/scalar_function.go index 27281648cb4a6..d155654f30823 100644 --- a/pkg/expression/scalar_function.go +++ b/pkg/expression/scalar_function.go @@ -262,12 +262,12 @@ func newFunctionImpl(ctx sessionctx.Context, fold int, funcName string, retType ctx: ctx, } if fold == 1 { - return FoldConstant(sf), nil + return FoldConstant(ctx, sf), nil } else if fold == -1 { // try to fold constants, and return the original function if errors/warnings occur sc := ctx.GetSessionVars().StmtCtx beforeWarns := sc.WarningCount() - newSf := FoldConstant(sf) + newSf := FoldConstant(ctx, sf) afterWarns := sc.WarningCount() if afterWarns > beforeWarns { sc.TruncateWarnings(int(beforeWarns)) diff --git a/pkg/expression/scalar_function_test.go b/pkg/expression/scalar_function_test.go index 97fc0897479d5..10f48eadd6063 100644 --- a/pkg/expression/scalar_function_test.go +++ b/pkg/expression/scalar_function_test.go @@ -36,57 +36,57 @@ func TestExpressionSemanticEqual(t *testing.T) { } // order sensitive cases // a < b; b > a - sf1 := newFunction(ast.LT, a, b) - sf2 := newFunction(ast.GT, b, a) + sf1 := newFunctionWithMockCtx(ast.LT, a, b) + sf2 := newFunctionWithMockCtx(ast.GT, b, a) require.True(t, ExpressionsSemanticEqual(sf1, sf2)) // a > b; b < a - sf3 := newFunction(ast.GT, a, b) - sf4 := newFunction(ast.LT, b, a) + sf3 := newFunctionWithMockCtx(ast.GT, a, b) + sf4 := newFunctionWithMockCtx(ast.LT, b, a) require.True(t, ExpressionsSemanticEqual(sf3, sf4)) // a<=b; b>=a - sf5 := newFunction(ast.LE, a, b) - sf6 := newFunction(ast.GE, b, a) + sf5 := newFunctionWithMockCtx(ast.LE, a, b) + sf6 := newFunctionWithMockCtx(ast.GE, b, a) require.True(t, ExpressionsSemanticEqual(sf5, sf6)) // a>=b; b<=a - sf7 := newFunction(ast.GE, a, b) - sf8 := newFunction(ast.LE, b, a) + sf7 := newFunctionWithMockCtx(ast.GE, a, b) + sf8 := newFunctionWithMockCtx(ast.LE, b, a) require.True(t, ExpressionsSemanticEqual(sf7, sf8)) // not(a= b - sf9 := newFunction(ast.UnaryNot, sf1) + sf9 := newFunctionWithMockCtx(ast.UnaryNot, sf1) require.True(t, ExpressionsSemanticEqual(sf9, sf7)) // a < b; not(a>=b) - sf10 := newFunction(ast.UnaryNot, sf7) + sf10 := newFunctionWithMockCtx(ast.UnaryNot, sf7) require.True(t, ExpressionsSemanticEqual(sf1, sf10)) // order insensitive cases // a + b; b + a - p1 := newFunction(ast.Plus, a, b) - p2 := newFunction(ast.Plus, b, a) + p1 := newFunctionWithMockCtx(ast.Plus, a, b) + p2 := newFunctionWithMockCtx(ast.Plus, b, a) require.True(t, ExpressionsSemanticEqual(p1, p2)) // a * b; b * a - m1 := newFunction(ast.Mul, a, b) - m2 := newFunction(ast.Mul, b, a) + m1 := newFunctionWithMockCtx(ast.Mul, a, b) + m2 := newFunctionWithMockCtx(ast.Mul, b, a) require.True(t, ExpressionsSemanticEqual(m1, m2)) // a = b; b = a - e1 := newFunction(ast.EQ, a, b) - e2 := newFunction(ast.EQ, b, a) + e1 := newFunctionWithMockCtx(ast.EQ, a, b) + e2 := newFunctionWithMockCtx(ast.EQ, b, a) require.True(t, ExpressionsSemanticEqual(e1, e2)) // a = b AND b + a; a + b AND b = a - a1 := newFunction(ast.LogicAnd, e1, p2) - a2 := newFunction(ast.LogicAnd, p1, e2) + a1 := newFunctionWithMockCtx(ast.LogicAnd, e1, p2) + a2 := newFunctionWithMockCtx(ast.LogicAnd, p1, e2) require.True(t, ExpressionsSemanticEqual(a1, a2)) // a * b OR a + b; b + a OR b * a - o1 := newFunction(ast.LogicOr, m1, p1) - o2 := newFunction(ast.LogicOr, p2, m2) + o1 := newFunctionWithMockCtx(ast.LogicOr, m1, p1) + o2 := newFunctionWithMockCtx(ast.LogicOr, p2, m2) require.True(t, ExpressionsSemanticEqual(o1, o2)) } @@ -96,7 +96,8 @@ func TestScalarFunction(t *testing.T) { UniqueID: 1, RetType: types.NewFieldType(mysql.TypeDouble), } - sf := newFunction(ast.LT, a, NewOne()) + + sf := newFunctionWithMockCtx(ast.LT, a, NewOne()) res, err := sf.MarshalJSON() require.NoError(t, err) require.EqualValues(t, []byte{0x22, 0x6c, 0x74, 0x28, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x23, 0x31, 0x2c, 0x20, 0x31, 0x29, 0x22}, res) @@ -125,7 +126,7 @@ func TestIssue23309(t *testing.T) { a.RetType.SetFlag(a.RetType.GetFlag() | mysql.NotNullFlag) null := NewNull() null.RetType = types.NewFieldType(mysql.TypeNull) - sf, _ := newFunction(ast.NE, a, null).(*ScalarFunction) + sf, _ := newFunctionWithMockCtx(ast.NE, a, null).(*ScalarFunction) v, err := sf.GetArgs()[1].Eval(mock.NewContext(), chunk.Row{}) require.NoError(t, err) require.True(t, v.IsNull()) @@ -138,8 +139,8 @@ func TestScalarFuncs2Exprs(t *testing.T) { UniqueID: 1, RetType: types.NewFieldType(mysql.TypeDouble), } - sf0, _ := newFunction(ast.LT, a, NewZero()).(*ScalarFunction) - sf1, _ := newFunction(ast.LT, a, NewOne()).(*ScalarFunction) + sf0, _ := newFunctionWithMockCtx(ast.LT, a, NewZero()).(*ScalarFunction) + sf1, _ := newFunctionWithMockCtx(ast.LT, a, NewOne()).(*ScalarFunction) funcs := []*ScalarFunction{sf0, sf1} exprs := ScalarFuncs2Exprs(funcs) diff --git a/pkg/expression/util.go b/pkg/expression/util.go index d8e47d5ef1f9d..b8d08b1d84f5b 100644 --- a/pkg/expression/util.go +++ b/pkg/expression/util.go @@ -409,8 +409,8 @@ func SetExprColumnInOperand(expr Expression) Expression { // ColumnSubstitute substitutes the columns in filter to expressions in select fields. // e.g. select * from (select b as a from t) k where a < 10 => select * from (select b as a from t where b < 10) k. -func ColumnSubstitute(expr Expression, schema *Schema, newExprs []Expression) Expression { - _, _, resExpr := ColumnSubstituteImpl(expr, schema, newExprs, false) +func ColumnSubstitute(ctx sessionctx.Context, expr Expression, schema *Schema, newExprs []Expression) Expression { + _, _, resExpr := ColumnSubstituteImpl(ctx, expr, schema, newExprs, false) return resExpr } @@ -419,8 +419,8 @@ func ColumnSubstitute(expr Expression, schema *Schema, newExprs []Expression) Ex // // 1: substitute them all once find col in schema. // 2: nothing in expr can be substituted. -func ColumnSubstituteAll(expr Expression, schema *Schema, newExprs []Expression) (bool, Expression) { - _, hasFail, resExpr := ColumnSubstituteImpl(expr, schema, newExprs, true) +func ColumnSubstituteAll(ctx sessionctx.Context, expr Expression, schema *Schema, newExprs []Expression) (bool, Expression) { + _, hasFail, resExpr := ColumnSubstituteImpl(ctx, expr, schema, newExprs, true) return hasFail, resExpr } @@ -429,7 +429,7 @@ func ColumnSubstituteAll(expr Expression, schema *Schema, newExprs []Expression) // @return bool means whether the expr has changed. // @return bool means whether the expr should change (has the dependency in schema, while the corresponding expr has some compatibility), but finally fallback. // @return Expression, the original expr or the changed expr, it depends on the first @return bool. -func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression, fail1Return bool) (bool, bool, Expression) { +func ColumnSubstituteImpl(ctx sessionctx.Context, expr Expression, schema *Schema, newExprs []Expression, fail1Return bool) (bool, bool, Expression) { switch v := expr.(type) { case *Column: id := schema.ColumnIndex(v) @@ -446,7 +446,7 @@ func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression hasFail := false if v.FuncName.L == ast.Cast || v.FuncName.L == ast.Grouping { var newArg Expression - substituted, hasFail, newArg = ColumnSubstituteImpl(v.GetArgs()[0], schema, newExprs, fail1Return) + substituted, hasFail, newArg = ColumnSubstituteImpl(ctx, v.GetArgs()[0], schema, newExprs, fail1Return) if fail1Return && hasFail { return substituted, hasFail, v } @@ -454,7 +454,7 @@ func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression flag := v.RetType.GetFlag() var e Expression if v.FuncName.L == ast.Cast { - e = BuildCastFunction(v.GetCtx(), newArg, v.RetType) + e = BuildCastFunction(ctx, newArg, v.RetType) } else { // for grouping function recreation, use clone (meta included) instead of newFunction e = v.Clone() @@ -469,7 +469,7 @@ func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression // cowExprRef is a copy-on-write util, args array allocation happens only // when expr in args is changed refExprArr := cowExprRef{v.GetArgs(), nil} - oldCollEt, err := CheckAndDeriveCollationFromExprs(v.GetCtx(), v.FuncName.L, v.RetType.EvalType(), v.GetArgs()...) + oldCollEt, err := CheckAndDeriveCollationFromExprs(ctx, v.FuncName.L, v.RetType.EvalType(), v.GetArgs()...) if err != nil { logutil.BgLogger().Error("Unexpected error happened during ColumnSubstitution", zap.Stack("stack")) return false, false, v @@ -479,7 +479,7 @@ func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression tmpArgForCollCheck = make([]Expression, len(v.GetArgs())) } for idx, arg := range v.GetArgs() { - changed, failed, newFuncExpr := ColumnSubstituteImpl(arg, schema, newExprs, fail1Return) + changed, failed, newFuncExpr := ColumnSubstituteImpl(ctx, arg, schema, newExprs, fail1Return) if fail1Return && failed { return changed, failed, v } @@ -489,7 +489,7 @@ func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression changed = false copy(tmpArgForCollCheck, refExprArr.Result()) tmpArgForCollCheck[idx] = newFuncExpr - newCollEt, err := CheckAndDeriveCollationFromExprs(v.GetCtx(), v.FuncName.L, v.RetType.EvalType(), tmpArgForCollCheck...) + newCollEt, err := CheckAndDeriveCollationFromExprs(ctx, v.FuncName.L, v.RetType.EvalType(), tmpArgForCollCheck...) if err != nil { logutil.BgLogger().Error("Unexpected error happened during ColumnSubstitution", zap.Stack("stack")) return false, failed, v @@ -518,7 +518,7 @@ func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression } } if substituted { - return true, hasFail, NewFunctionInternal(v.GetCtx(), v.FuncName.L, v.RetType, refExprArr.Result()...) + return true, hasFail, NewFunctionInternal(ctx, v.FuncName.L, v.RetType, refExprArr.Result()...) } } return false, false, expr @@ -585,13 +585,13 @@ Loop: // SubstituteCorCol2Constant will substitute correlated column to constant value which it contains. // If the args of one scalar function are all constant, we will substitute it to constant. -func SubstituteCorCol2Constant(expr Expression) (Expression, error) { +func SubstituteCorCol2Constant(ctx sessionctx.Context, expr Expression) (Expression, error) { switch x := expr.(type) { case *ScalarFunction: allConstant := true newArgs := make([]Expression, 0, len(x.GetArgs())) for _, arg := range x.GetArgs() { - newArg, err := SubstituteCorCol2Constant(arg) + newArg, err := SubstituteCorCol2Constant(ctx, arg) if err != nil { return nil, err } @@ -600,7 +600,7 @@ func SubstituteCorCol2Constant(expr Expression) (Expression, error) { allConstant = allConstant && ok } if allConstant { - val, err := x.EvalWithInnerCtx(chunk.Row{}) + val, err := x.Eval(ctx, chunk.Row{}) if err != nil { return nil, err } @@ -611,19 +611,19 @@ func SubstituteCorCol2Constant(expr Expression) (Expression, error) { newSf Expression ) if x.FuncName.L == ast.Cast { - newSf = BuildCastFunction(x.GetCtx(), newArgs[0], x.RetType) + newSf = BuildCastFunction(ctx, newArgs[0], x.RetType) } else if x.FuncName.L == ast.Grouping { newSf = x.Clone() newSf.(*ScalarFunction).GetArgs()[0] = newArgs[0] } else { - newSf, err = NewFunction(x.GetCtx(), x.FuncName.L, x.GetType(), newArgs...) + newSf, err = NewFunction(ctx, x.FuncName.L, x.GetType(), newArgs...) } return newSf, err case *CorrelatedColumn: return &Constant{Value: *x.Data, RetType: x.GetType()}, nil case *Constant: if x.DeferredExpr != nil { - newExpr := FoldConstant(x) + newExpr := FoldConstant(ctx, x) return &Constant{Value: newExpr.(*Constant).Value, RetType: x.GetType()}, nil } } @@ -879,20 +879,20 @@ func pushNotAcrossExpr(ctx sessionctx.Context, expr Expression, not bool) (_ Exp return expr, false } var childExpr Expression - childExpr, changed = pushNotAcrossExpr(f.GetCtx(), child, !not) + childExpr, changed = pushNotAcrossExpr(ctx, child, !not) if !changed && !not { return expr, false } return childExpr, true case ast.LT, ast.GE, ast.GT, ast.LE, ast.EQ, ast.NE: if not { - return NewFunctionInternal(f.GetCtx(), oppositeOp[f.FuncName.L], f.GetType(), f.GetArgs()...), true + return NewFunctionInternal(ctx, oppositeOp[f.FuncName.L], f.GetType(), f.GetArgs()...), true } - newArgs, changed := pushNotAcrossArgs(f.GetCtx(), f.GetArgs(), false) + newArgs, changed := pushNotAcrossArgs(ctx, f.GetArgs(), false) if !changed { return f, false } - return NewFunctionInternal(f.GetCtx(), f.FuncName.L, f.GetType(), newArgs...), true + return NewFunctionInternal(ctx, f.FuncName.L, f.GetType(), newArgs...), true case ast.LogicAnd, ast.LogicOr: var ( newArgs []Expression @@ -900,16 +900,16 @@ func pushNotAcrossExpr(ctx sessionctx.Context, expr Expression, not bool) (_ Exp ) funcName := f.FuncName.L if not { - newArgs, _ = pushNotAcrossArgs(f.GetCtx(), f.GetArgs(), true) + newArgs, _ = pushNotAcrossArgs(ctx, f.GetArgs(), true) funcName = oppositeOp[f.FuncName.L] changed = true } else { - newArgs, changed = pushNotAcrossArgs(f.GetCtx(), f.GetArgs(), false) + newArgs, changed = pushNotAcrossArgs(ctx, f.GetArgs(), false) } if !changed { return f, false } - return NewFunctionInternal(f.GetCtx(), funcName, f.GetType(), newArgs...), true + return NewFunctionInternal(ctx, funcName, f.GetType(), newArgs...), true } } if not { @@ -1081,12 +1081,11 @@ func extractFiltersFromDNF(ctx sessionctx.Context, dnfFunc *ScalarFunction) ([]E // the original expression must satisfy the derived expression. Return nil when the derived expression is universal set. // A running example is: for schema of t1, `(t1.a=1 and t2.a=1) or (t1.a=2 and t2.a=2)` would be derived as // `t1.a=1 or t1.a=2`, while `t1.a=1 or t2.a=1` would get nil. -func DeriveRelaxedFiltersFromDNF(expr Expression, schema *Schema) Expression { +func DeriveRelaxedFiltersFromDNF(ctx sessionctx.Context, expr Expression, schema *Schema) Expression { sf, ok := expr.(*ScalarFunction) if !ok || sf.FuncName.L != ast.LogicOr { return nil } - ctx := sf.GetCtx() dnfItems := FlattenDNFConditions(sf) newDNFItems := make([]Expression, 0, len(dnfItems)) for _, dnfItem := range dnfItems { @@ -1094,7 +1093,7 @@ func DeriveRelaxedFiltersFromDNF(expr Expression, schema *Schema) Expression { newCNFItems := make([]Expression, 0, len(cnfItems)) for _, cnfItem := range cnfItems { if itemSF, ok := cnfItem.(*ScalarFunction); ok && itemSF.FuncName.L == ast.LogicOr { - relaxedCNFItem := DeriveRelaxedFiltersFromDNF(cnfItem, schema) + relaxedCNFItem := DeriveRelaxedFiltersFromDNF(ctx, cnfItem, schema) if relaxedCNFItem != nil { newCNFItems = append(newCNFItems, relaxedCNFItem) } @@ -1392,7 +1391,7 @@ func RemoveDupExprs(exprs []Expression) []Expression { } // GetUint64FromConstant gets a uint64 from constant expression. -func GetUint64FromConstant(expr Expression) (uint64, bool, bool) { +func GetUint64FromConstant(ctx sessionctx.Context, expr Expression) (uint64, bool, bool) { con, ok := expr.(*Constant) if !ok { logutil.BgLogger().Warn("not a constant expression", zap.String("expression", expr.ExplainInfo())) @@ -1403,7 +1402,7 @@ func GetUint64FromConstant(expr Expression) (uint64, bool, bool) { dt = con.ParamMarker.GetUserVar() } else if con.DeferredExpr != nil { var err error - dt, err = con.DeferredExpr.EvalWithInnerCtx(chunk.Row{}) + dt, err = con.DeferredExpr.Eval(ctx, chunk.Row{}) if err != nil { logutil.BgLogger().Warn("eval deferred expr failed", zap.Error(err)) return 0, false, false diff --git a/pkg/expression/util_test.go b/pkg/expression/util_test.go index 4a7198caf4a96..8de7c7b1d56b8 100644 --- a/pkg/expression/util_test.go +++ b/pkg/expression/util_test.go @@ -154,37 +154,37 @@ func TestClone(t *testing.T) { } func TestGetUint64FromConstant(t *testing.T) { + ctx := mock.NewContext() con := &Constant{ Value: types.NewDatum(nil), } - _, isNull, ok := GetUint64FromConstant(con) + _, isNull, ok := GetUint64FromConstant(ctx, con) require.True(t, ok) require.True(t, isNull) con = &Constant{ Value: types.NewIntDatum(-1), } - _, _, ok = GetUint64FromConstant(con) + _, _, ok = GetUint64FromConstant(ctx, con) require.False(t, ok) con.Value = types.NewIntDatum(1) - num, isNull, ok := GetUint64FromConstant(con) + num, isNull, ok := GetUint64FromConstant(ctx, con) require.True(t, ok) require.False(t, isNull) require.Equal(t, uint64(1), num) con.Value = types.NewUintDatum(1) - num, _, _ = GetUint64FromConstant(con) + num, _, _ = GetUint64FromConstant(ctx, con) require.Equal(t, uint64(1), num) con.DeferredExpr = &Constant{Value: types.NewIntDatum(1)} - num, _, _ = GetUint64FromConstant(con) + num, _, _ = GetUint64FromConstant(ctx, con) require.Equal(t, uint64(1), num) - ctx := mock.NewContext() ctx.GetSessionVars().PlanCacheParams.Append(types.NewUintDatum(100)) con.ParamMarker = &ParamMarker{order: 0, ctx: ctx} - num, _, _ = GetUint64FromConstant(con) + num, _, _ = GetUint64FromConstant(ctx, con) require.Equal(t, uint64(100), num) } @@ -245,21 +245,21 @@ func TestSubstituteCorCol2Constant(t *testing.T) { corCol2 := &CorrelatedColumn{Data: &NewOne().Value} corCol2.RetType = types.NewFieldType(mysql.TypeLonglong) cast := BuildCastFunction(ctx, corCol1, types.NewFieldType(mysql.TypeLonglong)) - plus := newFunction(ast.Plus, cast, corCol2) - plus2 := newFunction(ast.Plus, plus, NewOne()) + plus := newFunctionWithMockCtx(ast.Plus, cast, corCol2) + plus2 := newFunctionWithMockCtx(ast.Plus, plus, NewOne()) ans1 := &Constant{Value: types.NewIntDatum(3), RetType: types.NewFieldType(mysql.TypeLonglong)} - ret, err := SubstituteCorCol2Constant(plus2) + ret, err := SubstituteCorCol2Constant(ctx, plus2) require.NoError(t, err) require.True(t, ret.Equal(ctx, ans1)) col1 := &Column{Index: 1, RetType: types.NewFieldType(mysql.TypeLonglong)} - ret, err = SubstituteCorCol2Constant(col1) + ret, err = SubstituteCorCol2Constant(ctx, col1) require.NoError(t, err) ans2 := col1 require.True(t, ret.Equal(ctx, ans2)) - plus3 := newFunction(ast.Plus, plus2, col1) - ret, err = SubstituteCorCol2Constant(plus3) + plus3 := newFunctionWithMockCtx(ast.Plus, plus2, col1) + ret, err = SubstituteCorCol2Constant(ctx, plus3) require.NoError(t, err) - ans3 := newFunction(ast.Plus, ans1, col1) + ans3 := newFunctionWithMockCtx(ast.Plus, ans1, col1) require.True(t, ret.Equal(ctx, ans3)) } @@ -267,14 +267,14 @@ func TestPushDownNot(t *testing.T) { ctx := mock.NewContext() col := &Column{Index: 1, RetType: types.NewFieldType(mysql.TypeLonglong)} // !((a=1||a=1)&&a=1) - eqFunc := newFunction(ast.EQ, col, NewOne()) - orFunc := newFunction(ast.LogicOr, eqFunc, eqFunc) - andFunc := newFunction(ast.LogicAnd, orFunc, eqFunc) - notFunc := newFunction(ast.UnaryNot, andFunc) + eqFunc := newFunctionWithMockCtx(ast.EQ, col, NewOne()) + orFunc := newFunctionWithMockCtx(ast.LogicOr, eqFunc, eqFunc) + andFunc := newFunctionWithMockCtx(ast.LogicAnd, orFunc, eqFunc) + notFunc := newFunctionWithMockCtx(ast.UnaryNot, andFunc) // (a!=1&&a!=1)||a=1 - neFunc := newFunction(ast.NE, col, NewOne()) - andFunc2 := newFunction(ast.LogicAnd, neFunc, neFunc) - orFunc2 := newFunction(ast.LogicOr, andFunc2, neFunc) + neFunc := newFunctionWithMockCtx(ast.NE, col, NewOne()) + andFunc2 := newFunctionWithMockCtx(ast.LogicAnd, neFunc, neFunc) + orFunc2 := newFunctionWithMockCtx(ast.LogicOr, andFunc2, neFunc) notFuncCopy := notFunc.Clone() ret := PushDownNot(ctx, notFunc) require.True(t, ret.Equal(ctx, orFunc2)) @@ -282,37 +282,37 @@ func TestPushDownNot(t *testing.T) { // issue 15725 // (not not a) should be optimized to (a is true) - notFunc = newFunction(ast.UnaryNot, col) - notFunc = newFunction(ast.UnaryNot, notFunc) + notFunc = newFunctionWithMockCtx(ast.UnaryNot, col) + notFunc = newFunctionWithMockCtx(ast.UnaryNot, notFunc) ret = PushDownNot(ctx, notFunc) - require.True(t, ret.Equal(ctx, newFunction(ast.IsTruthWithNull, col))) + require.True(t, ret.Equal(ctx, newFunctionWithMockCtx(ast.IsTruthWithNull, col))) // (not not (a+1)) should be optimized to (a+1 is true) - plusFunc := newFunction(ast.Plus, col, NewOne()) - notFunc = newFunction(ast.UnaryNot, plusFunc) - notFunc = newFunction(ast.UnaryNot, notFunc) + plusFunc := newFunctionWithMockCtx(ast.Plus, col, NewOne()) + notFunc = newFunctionWithMockCtx(ast.UnaryNot, plusFunc) + notFunc = newFunctionWithMockCtx(ast.UnaryNot, notFunc) ret = PushDownNot(ctx, notFunc) - require.True(t, ret.Equal(ctx, newFunction(ast.IsTruthWithNull, plusFunc))) + require.True(t, ret.Equal(ctx, newFunctionWithMockCtx(ast.IsTruthWithNull, plusFunc))) // (not not not a) should be optimized to (not (a is true)) - notFunc = newFunction(ast.UnaryNot, col) - notFunc = newFunction(ast.UnaryNot, notFunc) - notFunc = newFunction(ast.UnaryNot, notFunc) + notFunc = newFunctionWithMockCtx(ast.UnaryNot, col) + notFunc = newFunctionWithMockCtx(ast.UnaryNot, notFunc) + notFunc = newFunctionWithMockCtx(ast.UnaryNot, notFunc) ret = PushDownNot(ctx, notFunc) - require.True(t, ret.Equal(ctx, newFunction(ast.UnaryNot, newFunction(ast.IsTruthWithNull, col)))) + require.True(t, ret.Equal(ctx, newFunctionWithMockCtx(ast.UnaryNot, newFunctionWithMockCtx(ast.IsTruthWithNull, col)))) // (not not not not a) should be optimized to (a is true) - notFunc = newFunction(ast.UnaryNot, col) - notFunc = newFunction(ast.UnaryNot, notFunc) - notFunc = newFunction(ast.UnaryNot, notFunc) - notFunc = newFunction(ast.UnaryNot, notFunc) + notFunc = newFunctionWithMockCtx(ast.UnaryNot, col) + notFunc = newFunctionWithMockCtx(ast.UnaryNot, notFunc) + notFunc = newFunctionWithMockCtx(ast.UnaryNot, notFunc) + notFunc = newFunctionWithMockCtx(ast.UnaryNot, notFunc) ret = PushDownNot(ctx, notFunc) - require.True(t, ret.Equal(ctx, newFunction(ast.IsTruthWithNull, col))) + require.True(t, ret.Equal(ctx, newFunctionWithMockCtx(ast.IsTruthWithNull, col))) } func TestFilter(t *testing.T) { conditions := []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.EQ, newColumn(1), newColumn(2)), - newFunction(ast.LogicOr, newLonglong(1), newColumn(0)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.EQ, newColumn(1), newColumn(2)), + newFunctionWithMockCtx(ast.LogicOr, newLonglong(1), newColumn(0)), } result := make([]Expression, 0, 5) result = Filter(result, conditions, isLogicOrFunction) @@ -321,9 +321,9 @@ func TestFilter(t *testing.T) { func TestFilterOutInPlace(t *testing.T) { conditions := []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.EQ, newColumn(1), newColumn(2)), - newFunction(ast.LogicOr, newLonglong(1), newColumn(0)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.EQ, newColumn(1), newColumn(2)), + newFunctionWithMockCtx(ast.LogicOr, newLonglong(1), newColumn(0)), } remained, filtered := FilterOutInPlace(conditions, isLogicOrFunction) require.Equal(t, 2, len(remained)) @@ -459,11 +459,11 @@ func TestSQLDigestTextRetriever(t *testing.T) { func BenchmarkExtractColumns(b *testing.B) { conditions := []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.EQ, newColumn(1), newColumn(2)), - newFunction(ast.EQ, newColumn(2), newColumn(3)), - newFunction(ast.EQ, newColumn(3), newLonglong(1)), - newFunction(ast.LogicOr, newLonglong(1), newColumn(0)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.EQ, newColumn(1), newColumn(2)), + newFunctionWithMockCtx(ast.EQ, newColumn(2), newColumn(3)), + newFunctionWithMockCtx(ast.EQ, newColumn(3), newLonglong(1)), + newFunctionWithMockCtx(ast.LogicOr, newLonglong(1), newColumn(0)), } expr := ComposeCNFCondition(mock.NewContext(), conditions...) @@ -476,11 +476,11 @@ func BenchmarkExtractColumns(b *testing.B) { func BenchmarkExprFromSchema(b *testing.B) { conditions := []Expression{ - newFunction(ast.EQ, newColumn(0), newColumn(1)), - newFunction(ast.EQ, newColumn(1), newColumn(2)), - newFunction(ast.EQ, newColumn(2), newColumn(3)), - newFunction(ast.EQ, newColumn(3), newLonglong(1)), - newFunction(ast.LogicOr, newLonglong(1), newColumn(0)), + newFunctionWithMockCtx(ast.EQ, newColumn(0), newColumn(1)), + newFunctionWithMockCtx(ast.EQ, newColumn(1), newColumn(2)), + newFunctionWithMockCtx(ast.EQ, newColumn(2), newColumn(3)), + newFunctionWithMockCtx(ast.EQ, newColumn(3), newLonglong(1)), + newFunctionWithMockCtx(ast.LogicOr, newLonglong(1), newColumn(0)), } expr := ComposeCNFCondition(mock.NewContext(), conditions...) schema := &Schema{Columns: ExtractColumns(expr)} diff --git a/pkg/planner/cascades/transformation_rules.go b/pkg/planner/cascades/transformation_rules.go index c3d6360cee8a9..c2c0d9de0149b 100644 --- a/pkg/planner/cascades/transformation_rules.go +++ b/pkg/planner/cascades/transformation_rules.go @@ -549,8 +549,9 @@ func (*PushSelDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo. } canBePushed := make([]expression.Expression, 0, len(sel.Conditions)) canNotBePushed := make([]expression.Expression, 0, len(sel.Conditions)) + ctx := sel.SCtx() for _, cond := range sel.Conditions { - substituted, hasFailed, newFilter := expression.ColumnSubstituteImpl(cond, projSchema, proj.Exprs, true) + substituted, hasFailed, newFilter := expression.ColumnSubstituteImpl(ctx, cond, projSchema, proj.Exprs, true) if substituted && !hasFailed && !expression.HasGetSetVarFunc(newFilter) { canBePushed = append(canBePushed, newFilter) } else { @@ -1303,15 +1304,16 @@ func (*PushTopNDownProjection) OnTransform(old *memo.ExprIter) (newExprs []*memo proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) childGroup := old.Children[0].GetExpr().Children[0] + ctx := topN.SCtx() newTopN := plannercore.LogicalTopN{ Offset: topN.Offset, Count: topN.Count, - }.Init(topN.SCtx(), topN.SelectBlockOffset()) + }.Init(ctx, topN.SelectBlockOffset()) newTopN.ByItems = make([]*util.ByItems, 0, len(topN.ByItems)) for _, by := range topN.ByItems { newTopN.ByItems = append(newTopN.ByItems, &util.ByItems{ - Expr: expression.ColumnSubstitute(by.Expr, old.Children[0].Group.Prop.Schema, proj.Exprs), + Expr: expression.ColumnSubstitute(ctx, by.Expr, old.Children[0].Group.Prop.Schema, proj.Exprs), Desc: by.Desc, }) } @@ -1522,9 +1524,10 @@ func (*MergeAggregationProjection) OnTransform(old *memo.ExprIter) (newExprs []* proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) projSchema := old.Children[0].GetExpr().Schema() + ctx := oldAgg.SCtx() groupByItems := make([]expression.Expression, len(oldAgg.GroupByItems)) for i, item := range oldAgg.GroupByItems { - groupByItems[i] = expression.ColumnSubstitute(item, projSchema, proj.Exprs) + groupByItems[i] = expression.ColumnSubstitute(ctx, item, projSchema, proj.Exprs) } aggFuncs := make([]*aggregation.AggFuncDesc, len(oldAgg.AggFuncs)) @@ -1532,7 +1535,7 @@ func (*MergeAggregationProjection) OnTransform(old *memo.ExprIter) (newExprs []* aggFuncs[i] = aggFunc.Clone() newArgs := make([]expression.Expression, len(aggFunc.Args)) for j, arg := range aggFunc.Args { - newArgs[j] = expression.ColumnSubstitute(arg, projSchema, proj.Exprs) + newArgs[j] = expression.ColumnSubstitute(ctx, arg, projSchema, proj.Exprs) } aggFuncs[i].Args = newArgs } @@ -1540,7 +1543,7 @@ func (*MergeAggregationProjection) OnTransform(old *memo.ExprIter) (newExprs []* newAgg := plannercore.LogicalAggregation{ GroupByItems: groupByItems, AggFuncs: aggFuncs, - }.Init(oldAgg.SCtx(), oldAgg.SelectBlockOffset()) + }.Init(ctx, oldAgg.SelectBlockOffset()) newAggExpr := memo.NewGroupExpr(newAgg) newAggExpr.SetChildren(old.Children[0].GetExpr().Children...) diff --git a/pkg/planner/core/logical_plan_builder.go b/pkg/planner/core/logical_plan_builder.go index edc04269de2cf..36e0178a4bad5 100644 --- a/pkg/planner/core/logical_plan_builder.go +++ b/pkg/planner/core/logical_plan_builder.go @@ -626,6 +626,7 @@ func (p *LogicalJoin) ExtractOnCondition( deriveLeft bool, deriveRight bool) (eqCond []*expression.ScalarFunction, leftCond []expression.Expression, rightCond []expression.Expression, otherCond []expression.Expression) { + ctx := p.SCtx() for _, expr := range conditions { // For queries like `select a in (select a from s where s.b = t.b) from t`, // if subquery is empty caused by `s.b = t.b`, the result should always be @@ -638,7 +639,6 @@ func (p *LogicalJoin) ExtractOnCondition( } binop, ok := expr.(*expression.ScalarFunction) if ok && len(binop.GetArgs()) == 2 { - ctx := binop.GetCtx() arg0, lOK := binop.GetArgs()[0].(*expression.Column) arg1, rOK := binop.GetArgs()[1].(*expression.Column) if lOK && rOK { @@ -695,13 +695,13 @@ func (p *LogicalJoin) ExtractOnCondition( // `expr AND leftRelaxedCond AND rightRelaxedCond`. Motivation is to push filters down to // children as much as possible. if deriveLeft { - leftRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(expr, leftSchema) + leftRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(ctx, expr, leftSchema) if leftRelaxedCond != nil { leftCond = append(leftCond, leftRelaxedCond) } } if deriveRight { - rightRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(expr, rightSchema) + rightRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(ctx, expr, rightSchema) if rightRelaxedCond != nil { rightCond = append(rightCond, rightRelaxedCond) } diff --git a/pkg/planner/core/logical_plans.go b/pkg/planner/core/logical_plans.go index da3ad0f3bf44f..30d028cab5739 100644 --- a/pkg/planner/core/logical_plans.go +++ b/pkg/planner/core/logical_plans.go @@ -452,28 +452,29 @@ func (p *LogicalJoin) columnSubstituteAll(schema *expression.Schema, exprs []exp copy(cpOtherConditions, p.OtherConditions) copy(cpEqualConditions, p.EqualConditions) + ctx := p.SCtx() // try to substitute columns in these condition. for i, cond := range cpLeftConditions { - if hasFail, cpLeftConditions[i] = expression.ColumnSubstituteAll(cond, schema, exprs); hasFail { + if hasFail, cpLeftConditions[i] = expression.ColumnSubstituteAll(ctx, cond, schema, exprs); hasFail { return } } for i, cond := range cpRightConditions { - if hasFail, cpRightConditions[i] = expression.ColumnSubstituteAll(cond, schema, exprs); hasFail { + if hasFail, cpRightConditions[i] = expression.ColumnSubstituteAll(ctx, cond, schema, exprs); hasFail { return } } for i, cond := range cpOtherConditions { - if hasFail, cpOtherConditions[i] = expression.ColumnSubstituteAll(cond, schema, exprs); hasFail { + if hasFail, cpOtherConditions[i] = expression.ColumnSubstituteAll(ctx, cond, schema, exprs); hasFail { return } } for i, cond := range cpEqualConditions { var tmp expression.Expression - if hasFail, tmp = expression.ColumnSubstituteAll(cond, schema, exprs); hasFail { + if hasFail, tmp = expression.ColumnSubstituteAll(ctx, cond, schema, exprs); hasFail { return } cpEqualConditions[i] = tmp.(*expression.ScalarFunction) diff --git a/pkg/planner/core/optimizer_test.go b/pkg/planner/core/optimizer_test.go index 5e876d29caecc..2b1088ded1ef6 100644 --- a/pkg/planner/core/optimizer_test.go +++ b/pkg/planner/core/optimizer_test.go @@ -431,6 +431,7 @@ func TestPrunePhysicalColumns(t *testing.T) { ExchangeType: tipb.ExchangeType_PassThrough, } hashJoin := &PhysicalHashJoin{} + hashJoin = hashJoin.Init(sctx, nil, 0) recv := &PhysicalExchangeReceiver{} recv1 := &PhysicalExchangeReceiver{} hashSender := &PhysicalExchangeSender{ diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index c40ce39b04524..8626ef7e94b3c 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -937,18 +937,19 @@ func (ts *PhysicalTableScan) IsPartition() (bool, int64) { // mem usage when rebuilding ranges during the execution phase. func (ts *PhysicalTableScan) ResolveCorrelatedColumns() ([]*ranger.Range, error) { access := ts.AccessCondition + ctx := ts.SCtx() if ts.Table.IsCommonHandle { pkIdx := tables.FindPrimaryIndex(ts.Table) idxCols, idxColLens := expression.IndexInfo2PrefixCols(ts.Columns, ts.Schema().Columns, pkIdx) for _, cond := range access { - newCond, err := expression.SubstituteCorCol2Constant(cond) + newCond, err := expression.SubstituteCorCol2Constant(ctx, cond) if err != nil { return nil, err } access = append(access, newCond) } // All of access conditions must be used to build ranges, so we don't limit range memory usage. - res, err := ranger.DetachCondAndBuildRangeForIndex(ts.SCtx(), access, idxCols, idxColLens, 0) + res, err := ranger.DetachCondAndBuildRangeForIndex(ctx, access, idxCols, idxColLens, 0) if err != nil { return nil, err } @@ -957,7 +958,7 @@ func (ts *PhysicalTableScan) ResolveCorrelatedColumns() ([]*ranger.Range, error) var err error pkTP := ts.Table.GetPkColInfo().FieldType // All of access conditions must be used to build ranges, so we don't limit range memory usage. - ts.Ranges, _, _, err = ranger.BuildTableRange(access, ts.SCtx(), &pkTP, 0) + ts.Ranges, _, _, err = ranger.BuildTableRange(access, ctx, &pkTP, 0) if err != nil { return nil, err } diff --git a/pkg/planner/core/resolve_indices.go b/pkg/planner/core/resolve_indices.go index 77e4e8d163171..e40a107a88612 100644 --- a/pkg/planner/core/resolve_indices.go +++ b/pkg/planner/core/resolve_indices.go @@ -83,6 +83,7 @@ func refine4NeighbourProj(p, childProj *PhysicalProjection) { func (p *PhysicalHashJoin) ResolveIndicesItself() (err error) { lSchema := p.children[0].Schema() rSchema := p.children[1].Schema() + ctx := p.SCtx() for i, fun := range p.EqualConditions { lArg, err := fun.GetArgs()[0].ResolveIndices(lSchema) if err != nil { @@ -94,7 +95,7 @@ func (p *PhysicalHashJoin) ResolveIndicesItself() (err error) { return err } p.RightJoinKeys[i] = rArg.(*expression.Column) - p.EqualConditions[i] = expression.NewFunctionInternal(fun.GetCtx(), fun.FuncName.L, fun.GetType(), lArg, rArg).(*expression.ScalarFunction) + p.EqualConditions[i] = expression.NewFunctionInternal(ctx, fun.FuncName.L, fun.GetType(), lArg, rArg).(*expression.ScalarFunction) } for i, fun := range p.NAEqualConditions { lArg, err := fun.GetArgs()[0].ResolveIndices(lSchema) @@ -107,7 +108,7 @@ func (p *PhysicalHashJoin) ResolveIndicesItself() (err error) { return err } p.RightNAJoinKeys[i] = rArg.(*expression.Column) - p.NAEqualConditions[i] = expression.NewFunctionInternal(fun.GetCtx(), fun.FuncName.L, fun.GetType(), lArg, rArg).(*expression.ScalarFunction) + p.NAEqualConditions[i] = expression.NewFunctionInternal(ctx, fun.FuncName.L, fun.GetType(), lArg, rArg).(*expression.ScalarFunction) } for i, expr := range p.LeftConditions { p.LeftConditions[i], err = expr.ResolveIndices(lSchema) diff --git a/pkg/planner/core/rule_aggregation_push_down.go b/pkg/planner/core/rule_aggregation_push_down.go index a1296d41f1dfd..3b7fca21fe3ce 100644 --- a/pkg/planner/core/rule_aggregation_push_down.go +++ b/pkg/planner/core/rule_aggregation_push_down.go @@ -393,13 +393,13 @@ func (*aggregationPushDownSolver) pushAggCrossUnion(agg *LogicalAggregation, uni newAggFunc := aggFunc.Clone() newArgs := make([]expression.Expression, 0, len(newAggFunc.Args)) for _, arg := range newAggFunc.Args { - newArgs = append(newArgs, expression.ColumnSubstitute(arg, unionSchema, expression.Column2Exprs(unionChild.Schema().Columns))) + newArgs = append(newArgs, expression.ColumnSubstitute(ctx, arg, unionSchema, expression.Column2Exprs(unionChild.Schema().Columns))) } newAggFunc.Args = newArgs newAgg.AggFuncs = append(newAgg.AggFuncs, newAggFunc) } for _, gbyExpr := range agg.GroupByItems { - newExpr := expression.ColumnSubstitute(gbyExpr, unionSchema, expression.Column2Exprs(unionChild.Schema().Columns)) + newExpr := expression.ColumnSubstitute(ctx, gbyExpr, unionSchema, expression.Column2Exprs(unionChild.Schema().Columns)) newAgg.GroupByItems = append(newAgg.GroupByItems, newExpr) // TODO: if there is a duplicated first_row function, we can delete it. firstRow, err := aggregation.NewAggFuncDesc(agg.SCtx(), ast.AggFuncFirstRow, []expression.Expression{gbyExpr}, false) @@ -551,10 +551,11 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan, opt *logicalOptim // push aggregation across projection // TODO: This optimization is not always reasonable. We have not supported pushing projection to kv layer yet, // so we must do this optimization. + ctx := p.SCtx() noSideEffects := true newGbyItems := make([]expression.Expression, 0, len(agg.GroupByItems)) for _, gbyItem := range agg.GroupByItems { - newGbyItems = append(newGbyItems, expression.ColumnSubstitute(gbyItem, proj.schema, proj.Exprs)) + newGbyItems = append(newGbyItems, expression.ColumnSubstitute(ctx, gbyItem, proj.schema, proj.Exprs)) if ExprsHasSideEffects(newGbyItems) { noSideEffects = false break @@ -569,7 +570,7 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan, opt *logicalOptim oldAggFuncsArgs = append(oldAggFuncsArgs, aggFunc.Args) newArgs := make([]expression.Expression, 0, len(aggFunc.Args)) for _, arg := range aggFunc.Args { - newArgs = append(newArgs, expression.ColumnSubstitute(arg, proj.schema, proj.Exprs)) + newArgs = append(newArgs, expression.ColumnSubstitute(ctx, arg, proj.schema, proj.Exprs)) } if ExprsHasSideEffects(newArgs) { noSideEffects = false @@ -581,7 +582,7 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan, opt *logicalOptim oldAggOrderItems = append(oldAggOrderItems, aggFunc.OrderByItems) newOrderByItems := make([]expression.Expression, 0, len(aggFunc.OrderByItems)) for _, oby := range aggFunc.OrderByItems { - newOrderByItems = append(newOrderByItems, expression.ColumnSubstitute(oby.Expr, proj.schema, proj.Exprs)) + newOrderByItems = append(newOrderByItems, expression.ColumnSubstitute(ctx, oby.Expr, proj.schema, proj.Exprs)) } if ExprsHasSideEffects(newOrderByItems) { noSideEffects = false diff --git a/pkg/planner/core/rule_eliminate_projection.go b/pkg/planner/core/rule_eliminate_projection.go index 46b2d761e6095..36d7dbc82fd8d 100644 --- a/pkg/planner/core/rule_eliminate_projection.go +++ b/pkg/planner/core/rule_eliminate_projection.go @@ -209,9 +209,10 @@ func (pe *projectionEliminator) eliminate(p LogicalPlan, replace map[string]*exp // eliminate duplicate projection: projection with child projection if isProj { if child, ok := p.Children()[0].(*LogicalProjection); ok && !ExprsHasSideEffects(child.Exprs) { + ctx := p.SCtx() for i := range proj.Exprs { proj.Exprs[i] = ReplaceColumnOfExpr(proj.Exprs[i], child, child.Schema()) - foldedExpr := expression.FoldConstant(proj.Exprs[i]) + foldedExpr := expression.FoldConstant(ctx, proj.Exprs[i]) // the folded expr should have the same null flag with the original expr, especially for the projection under union, so forcing it here. foldedExpr.GetType().SetFlag((foldedExpr.GetType().GetFlag() & ^mysql.NotNullFlag) | (proj.Exprs[i].GetType().GetFlag() & mysql.NotNullFlag)) proj.Exprs[i] = foldedExpr diff --git a/pkg/planner/core/rule_predicate_push_down.go b/pkg/planner/core/rule_predicate_push_down.go index 2597727ac6d4a..1a64fe1418766 100644 --- a/pkg/planner/core/rule_predicate_push_down.go +++ b/pkg/planner/core/rule_predicate_push_down.go @@ -350,7 +350,7 @@ func (p *LogicalProjection) appendExpr(expr expression.Expression) *expression.C if col, ok := expr.(*expression.Column); ok { return col } - expr = expression.ColumnSubstitute(expr, p.schema, p.Exprs) + expr = expression.ColumnSubstitute(p.SCtx(), expr, p.schema, p.Exprs) p.Exprs = append(p.Exprs, expr) col := &expression.Column{ @@ -481,8 +481,9 @@ func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression return predicates, p } } + ctx := p.SCtx() for _, cond := range predicates { - substituted, hasFailed, newFilter := expression.ColumnSubstituteImpl(cond, p.Schema(), p.Exprs, true) + substituted, hasFailed, newFilter := expression.ColumnSubstituteImpl(ctx, cond, p.Schema(), p.Exprs, true) if substituted && !hasFailed && !expression.HasGetSetVarFunc(newFilter) { canBePushed = append(canBePushed, newFilter) } else { @@ -525,7 +526,7 @@ func (la *LogicalAggregation) pushDownPredicatesForAggregation(cond expression.E } } if ok { - newFunc := expression.ColumnSubstitute(cond, la.Schema(), exprsOriginal) + newFunc := expression.ColumnSubstitute(la.SCtx(), cond, la.Schema(), exprsOriginal) condsToPush = append(condsToPush, newFunc) } else { ret = append(ret, cond) @@ -635,19 +636,20 @@ func DeriveOtherConditions( deriveLeft bool, deriveRight bool) ( leftCond []expression.Expression, rightCond []expression.Expression) { isOuterSemi := (p.JoinType == LeftOuterSemiJoin) || (p.JoinType == AntiLeftOuterSemiJoin) + ctx := p.SCtx() for _, expr := range p.OtherConditions { if deriveLeft { - leftRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(expr, leftSchema) + leftRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(ctx, expr, leftSchema) if leftRelaxedCond != nil { leftCond = append(leftCond, leftRelaxedCond) } - notNullExpr := deriveNotNullExpr(expr, leftSchema) + notNullExpr := deriveNotNullExpr(ctx, expr, leftSchema) if notNullExpr != nil { leftCond = append(leftCond, notNullExpr) } } if deriveRight { - rightRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(expr, rightSchema) + rightRelaxedCond := expression.DeriveRelaxedFiltersFromDNF(ctx, expr, rightSchema) if rightRelaxedCond != nil { rightCond = append(rightCond, rightRelaxedCond) } @@ -661,7 +663,7 @@ func DeriveOtherConditions( if isOuterSemi { continue } - notNullExpr := deriveNotNullExpr(expr, rightSchema) + notNullExpr := deriveNotNullExpr(ctx, expr, rightSchema) if notNullExpr != nil { rightCond = append(rightCond, notNullExpr) } @@ -673,12 +675,11 @@ func DeriveOtherConditions( // deriveNotNullExpr generates a new expression `not(isnull(col))` given `col1 op col2`, // in which `col` is in specified schema. Caller guarantees that only one of `col1` or // `col2` is in schema. -func deriveNotNullExpr(expr expression.Expression, schema *expression.Schema) expression.Expression { +func deriveNotNullExpr(ctx sessionctx.Context, expr expression.Expression, schema *expression.Schema) expression.Expression { binop, ok := expr.(*expression.ScalarFunction) if !ok || len(binop.GetArgs()) != 2 { return nil } - ctx := binop.GetCtx() arg0, lOK := binop.GetArgs()[0].(*expression.Column) arg1, rOK := binop.GetArgs()[1].(*expression.Column) if !lOK || !rOK { diff --git a/pkg/planner/core/rule_predicate_simplification.go b/pkg/planner/core/rule_predicate_simplification.go index 00f65638423d2..501c78767012e 100644 --- a/pkg/planner/core/rule_predicate_simplification.go +++ b/pkg/planner/core/rule_predicate_simplification.go @@ -81,7 +81,7 @@ func (s *baseLogicalPlan) predicateSimplification(opt *logicalOptimizeOp) Logica // updateInPredicate applies intersection of an in list with <> value. It returns updated In list and a flag for // a special case if an element in the inlist is not removed to keep the list not empty. -func updateInPredicate(inPredicate expression.Expression, notEQPredicate expression.Expression) (expression.Expression, bool) { +func updateInPredicate(ctx sessionctx.Context, inPredicate expression.Expression, notEQPredicate expression.Expression) (expression.Expression, bool) { _, inPredicateType := findPredicateType(inPredicate) _, notEQPredicateType := findPredicateType(notEQPredicate) if inPredicateType != inListPredicate || notEQPredicateType != notEqualPredicate { @@ -97,7 +97,7 @@ func updateInPredicate(inPredicate expression.Expression, notEQPredicate express var lastValue *expression.Constant for _, element := range v.GetArgs() { value, valueOK := element.(*expression.Constant) - redundantValue := valueOK && value.Equal(v.GetCtx(), notEQValue) + redundantValue := valueOK && value.Equal(ctx, notEQValue) if !redundantValue { newValues = append(newValues, element) } @@ -113,7 +113,7 @@ func updateInPredicate(inPredicate expression.Expression, notEQPredicate express newValues = append(newValues, lastValue) specialCase = true } - newPred := expression.NewFunctionInternal(v.GetCtx(), v.FuncName.L, v.RetType, newValues...) + newPred := expression.NewFunctionInternal(ctx, v.FuncName.L, v.RetType, newValues...) return newPred, specialCase } @@ -131,13 +131,13 @@ func applyPredicateSimplification(sctx sessionctx.Context, predicates []expressi jCol, jType := findPredicateType(jthPredicate) if iCol == jCol { if iType == notEqualPredicate && jType == inListPredicate { - predicates[j], specialCase = updateInPredicate(jthPredicate, ithPredicate) + predicates[j], specialCase = updateInPredicate(sctx, jthPredicate, ithPredicate) sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("NE/INList simplification is triggered")) if !specialCase { removeValues = append(removeValues, i) } } else if iType == inListPredicate && jType == notEqualPredicate { - predicates[i], specialCase = updateInPredicate(ithPredicate, jthPredicate) + predicates[i], specialCase = updateInPredicate(sctx, ithPredicate, jthPredicate) sctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("NE/INList simplification is triggered")) if !specialCase { removeValues = append(removeValues, j) diff --git a/pkg/planner/core/rule_topn_push_down.go b/pkg/planner/core/rule_topn_push_down.go index 7f6716cc17948..7574e741ee6ec 100644 --- a/pkg/planner/core/rule_topn_push_down.go +++ b/pkg/planner/core/rule_topn_push_down.go @@ -133,8 +133,9 @@ func (p *LogicalProjection) pushDownTopN(topN *LogicalTopN, opt *logicalOptimize } } if topN != nil { + ctx := p.SCtx() for _, by := range topN.ByItems { - by.Expr = expression.FoldConstant(expression.ColumnSubstitute(by.Expr, p.schema, p.Exprs)) + by.Expr = expression.FoldConstant(ctx, expression.ColumnSubstitute(ctx, by.Expr, p.schema, p.Exprs)) } // remove meaningless constant sort items. From a0f53768d67fe551674e1b722c393342e4553a2d Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 24 Nov 2023 15:04:14 +0800 Subject: [PATCH 35/36] planner: remove duplicated pseudo bindings after loading data with BR (#48806) ref pingcap/tidb#46527 --- pkg/bindinfo/BUILD.bazel | 2 +- pkg/bindinfo/handle.go | 8 ++++++++ pkg/bindinfo/handle_test.go | 34 ++++++++++++++++++++++++++++++++++ 3 files changed, 43 insertions(+), 1 deletion(-) diff --git a/pkg/bindinfo/BUILD.bazel b/pkg/bindinfo/BUILD.bazel index 671c7924bb0b9..fc3ba7b3b04bc 100644 --- a/pkg/bindinfo/BUILD.bazel +++ b/pkg/bindinfo/BUILD.bazel @@ -57,7 +57,7 @@ go_test( embed = [":bindinfo"], flaky = True, race = "on", - shard_count = 40, + shard_count = 41, deps = [ "//pkg/bindinfo/internal", "//pkg/config", diff --git a/pkg/bindinfo/handle.go b/pkg/bindinfo/handle.go index 4c7ad623c1fda..333874da35f30 100644 --- a/pkg/bindinfo/handle.go +++ b/pkg/bindinfo/handle.go @@ -99,6 +99,14 @@ const ( Prompt = "bindinfo" // BuiltinPseudoSQL4BindLock is used to simulate LOCK TABLE for mysql.bind_info. BuiltinPseudoSQL4BindLock = "builtin_pseudo_sql_for_bind_lock" + + // StmtRemoveDuplicatedPseudoBinding is used to remove duplicated pseudo binding. + // After using BR to sync bind_info between two clusters, the pseudo binding may be duplicated, and + // BR use this statement to remove duplicated rows, and this SQL should only be executed by BR. + StmtRemoveDuplicatedPseudoBinding = `DELETE FROM mysql.bind_info + WHERE original_sql='builtin_pseudo_sql_for_bind_lock' AND + _tidb_rowid NOT IN ( -- keep one arbitrary pseudo binding + SELECT _tidb_rowid FROM mysql.bind_info WHERE original_sql='builtin_pseudo_sql_for_bind_lock' limit 1)` ) type bindRecordUpdate struct { diff --git a/pkg/bindinfo/handle_test.go b/pkg/bindinfo/handle_test.go index 984fdb0189be7..a75e667cfdc8d 100644 --- a/pkg/bindinfo/handle_test.go +++ b/pkg/bindinfo/handle_test.go @@ -607,3 +607,37 @@ func TestReloadBindings(t *testing.T) { rows = tk.MustQuery("show global bindings").Rows() require.Equal(t, 0, len(rows)) } + +func TestRemoveDuplicatedPseudoBinding(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + checkPseudoBinding := func(num int) { + tk.MustQuery(fmt.Sprintf("select count(1) from mysql.bind_info where original_sql='%s'", + bindinfo.BuiltinPseudoSQL4BindLock)).Check(testkit.Rows(fmt.Sprintf("%d", num))) + } + insertPseudoBinding := func() { + tk.MustExec(fmt.Sprintf(`INSERT INTO mysql.bind_info(original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source) + VALUES ('%v', '%v', "mysql", '%v', "2000-01-01 00:00:00", "2000-01-01 00:00:00", "", "", '%v')`, + bindinfo.BuiltinPseudoSQL4BindLock, bindinfo.BuiltinPseudoSQL4BindLock, bindinfo.Builtin, bindinfo.Builtin)) + } + removeDuplicated := func() { + tk.MustExec(bindinfo.StmtRemoveDuplicatedPseudoBinding) + } + + checkPseudoBinding(1) + insertPseudoBinding() + checkPseudoBinding(2) + removeDuplicated() + checkPseudoBinding(1) + + insertPseudoBinding() + insertPseudoBinding() + insertPseudoBinding() + checkPseudoBinding(4) + removeDuplicated() + checkPseudoBinding(1) + removeDuplicated() + checkPseudoBinding(1) +} From 27d2ba5fdfb54850457a78736b700a639c86e503 Mon Sep 17 00:00:00 2001 From: Zhou Kunqin <25057648+time-and-fate@users.noreply.github.com> Date: Fri, 24 Nov 2023 16:23:44 +0800 Subject: [PATCH 36/36] util/ranger: add missing `Selection` for range scan from `like` on PAD SPACE column (#48845) ref pingcap/tidb#48181, close pingcap/tidb#48821 --- .../testdata/plan_suite_out.json | 18 +++---- pkg/util/ranger/checker.go | 16 +++++- pkg/util/ranger/ranger_test.go | 12 ++--- .../explain_generate_column_substitute.result | 3 +- .../physicalplantest/physical_plan.result | 44 ++++++++-------- .../core/issuetest/planner_issue.result | 45 +++++++++++++++++ .../r/planner/core/plan.result | 50 +++++++++++-------- .../planner/core/issuetest/planner_issue.test | 15 ++++++ 8 files changed, 144 insertions(+), 59 deletions(-) diff --git a/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json b/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json index 85743605cc0b7..234706233bd4e 100644 --- a/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json +++ b/pkg/planner/core/casetest/physicalplantest/testdata/plan_suite_out.json @@ -2171,11 +2171,11 @@ }, { "SQL": "select a from t where c_str like ''", - "Best": "IndexReader(Index(t.c_d_e_str)[[\"\",\"\"]])->Projection" + "Best": "IndexReader(Index(t.c_d_e_str)[[\"\",\"\"]]->Sel([like(test.t.c_str, , 92)]))->Projection" }, { "SQL": "select a from t where c_str like 'abc'", - "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc\",\"abc\"]])->Projection" + "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc\",\"abc\"]]->Sel([like(test.t.c_str, abc, 92)]))->Projection" }, { "SQL": "select a from t where c_str not like 'abc'", @@ -2191,7 +2191,7 @@ }, { "SQL": "select a from t where c_str like 'abc%'", - "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc\",\"abd\")])->Projection" + "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc\",\"abd\")]->Sel([like(test.t.c_str, abc%, 92)]))->Projection" }, { "SQL": "select a from t where c_str like 'abc_'", @@ -2203,23 +2203,23 @@ }, { "SQL": "select a from t where c_str like 'abc\\_' escape ''", - "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc_\",\"abc_\"]])->Projection" + "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc_\",\"abc_\"]]->Sel([like(test.t.c_str, abc\\_, 92)]))->Projection" }, { "SQL": "select a from t where c_str like 'abc\\_'", - "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc_\",\"abc_\"]])->Projection" + "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc_\",\"abc_\"]]->Sel([like(test.t.c_str, abc\\_, 92)]))->Projection" }, { "SQL": "select a from t where c_str like 'abc\\\\_'", - "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc_\",\"abc_\"]])->Projection" + "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc_\",\"abc_\"]]->Sel([like(test.t.c_str, abc\\_, 92)]))->Projection" }, { "SQL": "select a from t where c_str like 'abc\\_%'", - "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc_\",\"abc`\")])->Projection" + "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc_\",\"abc`\")]->Sel([like(test.t.c_str, abc\\_%, 92)]))->Projection" }, { "SQL": "select a from t where c_str like 'abc=_%' escape '='", - "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc_\",\"abc`\")])->Projection" + "Best": "IndexReader(Index(t.c_d_e_str)[[\"abc_\",\"abc`\")]->Sel([like(test.t.c_str, abc=_%, 61)]))->Projection" }, { "SQL": "select a from t where c_str like 'abc\\__'", @@ -2227,7 +2227,7 @@ }, { "SQL": "select a from t where c_str like 123", - "Best": "IndexReader(Index(t.c_d_e_str)[[\"123\",\"123\"]])->Projection" + "Best": "IndexReader(Index(t.c_d_e_str)[[\"123\",\"123\"]]->Sel([like(test.t.c_str, 123, 92)]))->Projection" }, { "SQL": "select a from t where c = 1.9 and d > 3", diff --git a/pkg/util/ranger/checker.go b/pkg/util/ranger/checker.go index b279678459460..2c7812b6cdb3f 100644 --- a/pkg/util/ranger/checker.go +++ b/pkg/util/ranger/checker.go @@ -17,6 +17,7 @@ package ranger import ( "github.com/pingcap/tidb/pkg/expression" "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/charset" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/types" @@ -168,11 +169,22 @@ func (c *conditionChecker) checkLikeFunc(scalar *expression.ScalarFunction) (isA if err != nil { return false, true } + likeFuncReserve := !c.isFullLengthColumn() + + // Different from `=`, trailing spaces are always significant, and can't be ignored in `like`. + // In tidb's implementation, for PAD SPACE collations, the trailing spaces are removed in the index key. So we are + // unable to distinguish 'xxx' from 'xxx ' by a single index range scan, and we may read more data than needed by + // the `like` function. Therefore, a Selection is needed to filter the data. + // Since all collations, except for binary, implemented in tidb are PAD SPACE collations for now, we use a simple + // collation != binary check here. + if collation != charset.CollationBin { + likeFuncReserve = true + } + if len(patternStr) == 0 { - return true, !c.isFullLengthColumn() + return true, likeFuncReserve } escape := byte(scalar.GetArgs()[2].(*expression.Constant).Value.GetInt64()) - likeFuncReserve := !c.isFullLengthColumn() for i := 0; i < len(patternStr); i++ { if patternStr[i] == escape { i++ diff --git a/pkg/util/ranger/ranger_test.go b/pkg/util/ranger/ranger_test.go index ce519eb053ce9..aeb2b8a1dc31b 100644 --- a/pkg/util/ranger/ranger_test.go +++ b/pkg/util/ranger/ranger_test.go @@ -1106,7 +1106,7 @@ create table t( indexPos: 0, exprStr: `a LIKE 'abc%'`, accessConds: `[like(test.t.a, abc%, 92)]`, - filterConds: "[]", + filterConds: "[like(test.t.a, abc%, 92)]", resultStr: "[[\"abc\",\"abd\")]", }, { @@ -1120,14 +1120,14 @@ create table t( indexPos: 0, exprStr: "a LIKE 'abc'", accessConds: "[like(test.t.a, abc, 92)]", - filterConds: "[]", + filterConds: "[like(test.t.a, abc, 92)]", resultStr: "[[\"abc\",\"abc\"]]", }, { indexPos: 0, exprStr: `a LIKE "ab\_c"`, accessConds: "[like(test.t.a, ab\\_c, 92)]", - filterConds: "[]", + filterConds: "[like(test.t.a, ab\\_c, 92)]", resultStr: "[[\"ab_c\",\"ab_c\"]]", }, { @@ -1141,21 +1141,21 @@ create table t( indexPos: 0, exprStr: `a LIKE '\%a'`, accessConds: "[like(test.t.a, \\%a, 92)]", - filterConds: "[]", + filterConds: "[like(test.t.a, \\%a, 92)]", resultStr: `[["%a","%a"]]`, }, { indexPos: 0, exprStr: `a LIKE "\\"`, accessConds: "[like(test.t.a, \\, 92)]", - filterConds: "[]", + filterConds: "[like(test.t.a, \\, 92)]", resultStr: "[[\"\\\\\",\"\\\\\"]]", }, { indexPos: 0, exprStr: `a LIKE "\\\\a%"`, accessConds: `[like(test.t.a, \\a%, 92)]`, - filterConds: "[]", + filterConds: "[like(test.t.a, \\\\a%, 92)]", resultStr: "[[\"\\\\a\",\"\\\\b\")]", }, { diff --git a/tests/integrationtest/r/explain_generate_column_substitute.result b/tests/integrationtest/r/explain_generate_column_substitute.result index 0a1b4a035342c..b4f9fe097c3c6 100644 --- a/tests/integrationtest/r/explain_generate_column_substitute.result +++ b/tests/integrationtest/r/explain_generate_column_substitute.result @@ -396,7 +396,8 @@ id estRows task access object operator info StreamAgg 1.00 root funcs:count(Column#6)->Column#4 └─IndexReader 1.00 root index:StreamAgg └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#6 - └─IndexRangeScan 250.00 cop[tikv] table:tbl1, index:expression_index(md5(`s`)) range:["02e74f10e0327ad868d138f2b4fdd6f","02e74f10e0327ad868d138f2b4fdd6g"), keep order:false, stats:pseudo + └─Selection 250.00 cop[tikv] like(md5(cast(explain_generate_column_substitute.tbl1.s, var_string(20))), "02e74f10e0327ad868d138f2b4fdd6f%", 92) + └─IndexRangeScan 250.00 cop[tikv] table:tbl1, index:expression_index(md5(`s`)) range:["02e74f10e0327ad868d138f2b4fdd6f","02e74f10e0327ad868d138f2b4fdd6g"), keep order:false, stats:pseudo select count(*) from tbl1 use index() where md5(s) like '02e74f10e0327ad868d138f2b4fdd6f%'; count(*) 64 diff --git a/tests/integrationtest/r/planner/core/casetest/physicalplantest/physical_plan.result b/tests/integrationtest/r/planner/core/casetest/physicalplantest/physical_plan.result index 5db9c5101fc29..541e196db8f36 100644 --- a/tests/integrationtest/r/planner/core/casetest/physicalplantest/physical_plan.result +++ b/tests/integrationtest/r/planner/core/casetest/physicalplantest/physical_plan.result @@ -3308,21 +3308,21 @@ Projection 249.75 root planner__core__casetest__physicalplantest__physical_plan │ └─StreamAgg 249.75 root funcs:max(planner__core__casetest__physicalplantest__physical_plan.tc.id)->Column#14 │ └─TopN 62.38 root planner__core__casetest__physicalplantest__physical_plan.tc.id:desc, offset:0, count:1 │ └─IndexLookUp 62.38 root - │ ├─Selection(Build) 62.44 cop[tikv] eq(planner__core__casetest__physicalplantest__physical_plan.ta.name, planner__core__casetest__physicalplantest__physical_plan.tc.name) + │ ├─Selection(Build) 62.38 cop[tikv] eq(planner__core__casetest__physicalplantest__physical_plan.ta.name, planner__core__casetest__physicalplantest__physical_plan.tc.name), like(planner__core__casetest__physicalplantest__physical_plan.tc.name, "chad99%", 92) │ │ └─IndexRangeScan 62437.50 cop[tikv] table:tc, index:idx_tc_name(name) range:["chad99","chad9:"), keep order:false, stats:pseudo │ └─TopN(Probe) 62.38 cop[tikv] planner__core__casetest__physicalplantest__physical_plan.tc.id:desc, offset:0, count:1 │ └─Selection 62.38 cop[tikv] not(isnull(planner__core__casetest__physicalplantest__physical_plan.tc.id)) - │ └─TableRowIDScan 62.44 cop[tikv] table:tc keep order:false, stats:pseudo + │ └─TableRowIDScan 62.38 cop[tikv] table:tc keep order:false, stats:pseudo └─Selection(Probe) 199.80 root gt(Column#19, 100) └─MaxOneRow 249.75 root └─StreamAgg 249.75 root funcs:max(planner__core__casetest__physicalplantest__physical_plan.td.id)->Column#19 - └─Limit 62.38 root offset:0, count:1 - └─Projection 62.38 root planner__core__casetest__physicalplantest__physical_plan.td.id, planner__core__casetest__physicalplantest__physical_plan.td.name - └─IndexLookUp 62.38 root - ├─Selection(Build) 2495.00 cop[tikv] eq(planner__core__casetest__physicalplantest__physical_plan.ta.id, planner__core__casetest__physicalplantest__physical_plan.td.id) - │ └─IndexFullScan 2495002.50 cop[tikv] table:td, index:idx_tc_id(id) keep order:true, desc, stats:pseudo - └─Selection(Probe) 62.38 cop[tikv] like(planner__core__casetest__physicalplantest__physical_plan.td.name, "chad999%", 92) - └─TableRowIDScan 2495.00 cop[tikv] table:td keep order:false, stats:pseudo + └─TopN 62.38 root planner__core__casetest__physicalplantest__physical_plan.td.id:desc, offset:0, count:1 + └─IndexLookUp 62.38 root + ├─Selection(Build) 1560.94 cop[tikv] like(planner__core__casetest__physicalplantest__physical_plan.td.name, "chad999%", 92) + │ └─IndexRangeScan 62437.50 cop[tikv] table:td, index:idx_tc_name(name) range:["chad999","chad99:"), keep order:false, stats:pseudo + └─TopN(Probe) 62.38 cop[tikv] planner__core__casetest__physicalplantest__physical_plan.td.id:desc, offset:0, count:1 + └─Selection 62.38 cop[tikv] eq(planner__core__casetest__physicalplantest__physical_plan.ta.id, planner__core__casetest__physicalplantest__physical_plan.td.id), not(isnull(planner__core__casetest__physicalplantest__physical_plan.td.id)) + └─TableRowIDScan 1560.94 cop[tikv] table:td keep order:false, stats:pseudo SELECT ta.NAME FROM ta WHERE EXISTS (select /*+ semi_join_rewrite() */ 1 from tb where ta.code = tb.code and tb.NAME LIKE 'chad9%') AND (select /*+ no_decorrelate() */ max(id) from tc where ta.name=tc.name and tc.name like 'chad99%') > 100 and (select /*+ no_decorrelate() */ max(id) from td where ta.id=td.id and td.name like 'chad999%') > 100; NAME show warnings; @@ -3335,29 +3335,31 @@ Projection 10000.00 root planner__core__casetest__physicalplantest__physical_pl │ ├─Apply(Build) 10000.00 root CARTESIAN semi join │ │ ├─TableReader(Build) 10000.00 root data:TableFullScan │ │ │ └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo - │ │ └─TableReader(Probe) 2500.00 root data:Selection - │ │ └─Selection 2500.00 cop[tikv] eq(planner__core__casetest__physicalplantest__physical_plan.ta.code, planner__core__casetest__physicalplantest__physical_plan.tb.code), like(planner__core__casetest__physicalplantest__physical_plan.tb.name, "chad9%", 92) - │ │ └─TableFullScan 100000000.00 cop[tikv] table:tb keep order:false, stats:pseudo + │ │ └─IndexLookUp(Probe) 2500.00 root + │ │ ├─Selection(Build) 62500.00 cop[tikv] like(planner__core__casetest__physicalplantest__physical_plan.tb.name, "chad9%", 92) + │ │ │ └─IndexRangeScan 2500000.00 cop[tikv] table:tb, index:idx_tb_name(name) range:["chad9","chad:"), keep order:false, stats:pseudo + │ │ └─Selection(Probe) 2500.00 cop[tikv] eq(planner__core__casetest__physicalplantest__physical_plan.ta.code, planner__core__casetest__physicalplantest__physical_plan.tb.code) + │ │ └─TableRowIDScan 62500.00 cop[tikv] table:tb keep order:false, stats:pseudo │ └─Selection(Probe) 8000.00 root gt(Column#14, 100) │ └─MaxOneRow 10000.00 root │ └─StreamAgg 10000.00 root funcs:max(planner__core__casetest__physicalplantest__physical_plan.tc.id)->Column#14 │ └─TopN 2497.50 root planner__core__casetest__physicalplantest__physical_plan.tc.id:desc, offset:0, count:1 │ └─IndexLookUp 2497.50 root - │ ├─Selection(Build) 2500.00 cop[tikv] eq(planner__core__casetest__physicalplantest__physical_plan.ta.name, planner__core__casetest__physicalplantest__physical_plan.tc.name) + │ ├─Selection(Build) 2497.50 cop[tikv] eq(planner__core__casetest__physicalplantest__physical_plan.ta.name, planner__core__casetest__physicalplantest__physical_plan.tc.name), like(planner__core__casetest__physicalplantest__physical_plan.tc.name, "chad99%", 92) │ │ └─IndexRangeScan 2500000.00 cop[tikv] table:tc, index:idx_tc_name(name) range:["chad99","chad9:"), keep order:false, stats:pseudo │ └─TopN(Probe) 2497.50 cop[tikv] planner__core__casetest__physicalplantest__physical_plan.tc.id:desc, offset:0, count:1 │ └─Selection 2497.50 cop[tikv] not(isnull(planner__core__casetest__physicalplantest__physical_plan.tc.id)) - │ └─TableRowIDScan 2500.00 cop[tikv] table:tc keep order:false, stats:pseudo + │ └─TableRowIDScan 2497.50 cop[tikv] table:tc keep order:false, stats:pseudo └─Selection(Probe) 8000.00 root gt(Column#19, 100) └─MaxOneRow 10000.00 root └─StreamAgg 10000.00 root funcs:max(planner__core__casetest__physicalplantest__physical_plan.td.id)->Column#19 - └─Limit 2497.50 root offset:0, count:1 - └─Projection 2497.50 root planner__core__casetest__physicalplantest__physical_plan.td.id, planner__core__casetest__physicalplantest__physical_plan.td.name - └─IndexLookUp 2497.50 root - ├─Selection(Build) 99900.00 cop[tikv] eq(planner__core__casetest__physicalplantest__physical_plan.ta.id, planner__core__casetest__physicalplantest__physical_plan.td.id) - │ └─IndexFullScan 99900000.00 cop[tikv] table:td, index:idx_tc_id(id) keep order:true, desc, stats:pseudo - └─Selection(Probe) 2497.50 cop[tikv] like(planner__core__casetest__physicalplantest__physical_plan.td.name, "chad999%", 92) - └─TableRowIDScan 99900.00 cop[tikv] table:td keep order:false, stats:pseudo + └─TopN 2497.50 root planner__core__casetest__physicalplantest__physical_plan.td.id:desc, offset:0, count:1 + └─IndexLookUp 2497.50 root + ├─Selection(Build) 62500.00 cop[tikv] like(planner__core__casetest__physicalplantest__physical_plan.td.name, "chad999%", 92) + │ └─IndexRangeScan 2500000.00 cop[tikv] table:td, index:idx_tc_name(name) range:["chad999","chad99:"), keep order:false, stats:pseudo + └─TopN(Probe) 2497.50 cop[tikv] planner__core__casetest__physicalplantest__physical_plan.td.id:desc, offset:0, count:1 + └─Selection 2497.50 cop[tikv] eq(planner__core__casetest__physicalplantest__physical_plan.ta.id, planner__core__casetest__physicalplantest__physical_plan.td.id), not(isnull(planner__core__casetest__physicalplantest__physical_plan.td.id)) + └─TableRowIDScan 62500.00 cop[tikv] table:td keep order:false, stats:pseudo SELECT ta.NAME FROM ta WHERE EXISTS (select /*+ no_decorrelate() */ 1 from tb where ta.code = tb.code and tb.NAME LIKE 'chad9%') AND (select /*+ no_decorrelate() */ max(id) from tc where ta.name=tc.name and tc.name like 'chad99%') > 100 and (select /*+ no_decorrelate() */ max(id) from td where ta.id=td.id and td.name like 'chad999%') > 100; NAME show warnings; diff --git a/tests/integrationtest/r/planner/core/issuetest/planner_issue.result b/tests/integrationtest/r/planner/core/issuetest/planner_issue.result index 5d95672abacb7..9a5474ce1b188 100644 --- a/tests/integrationtest/r/planner/core/issuetest/planner_issue.result +++ b/tests/integrationtest/r/planner/core/issuetest/planner_issue.result @@ -180,3 +180,48 @@ LEFT JOIN tmp3 c3 ON c3.id = '1'; id id 1 1 1 1 +drop table if exists t1, t2; +create table t1(a varchar(20) collate utf8mb4_bin, index ia(a)); +insert into t1 value('测试'),('测试 '); +explain format = brief select *,length(a) from t1 where a like '测试 %'; +id estRows task access object operator info +Projection 250.00 root planner__core__issuetest__planner_issue.t1.a, length(planner__core__issuetest__planner_issue.t1.a)->Column#3 +└─UnionScan 250.00 root like(planner__core__issuetest__planner_issue.t1.a, "测试 %", 92) + └─IndexReader 250.00 root index:Selection + └─Selection 250.00 cop[tikv] like(planner__core__issuetest__planner_issue.t1.a, "测试 %", 92) + └─IndexRangeScan 250.00 cop[tikv] table:t1, index:ia(a) range:["测试 ","测试!"), keep order:false, stats:pseudo +explain format = brief select *,length(a) from t1 where a like '测试'; +id estRows task access object operator info +Projection 10.00 root planner__core__issuetest__planner_issue.t1.a, length(planner__core__issuetest__planner_issue.t1.a)->Column#3 +└─UnionScan 10.00 root like(planner__core__issuetest__planner_issue.t1.a, "测试", 92) + └─IndexReader 10.00 root index:Selection + └─Selection 10.00 cop[tikv] like(planner__core__issuetest__planner_issue.t1.a, "测试", 92) + └─IndexRangeScan 10.00 cop[tikv] table:t1, index:ia(a) range:["测试","测试"], keep order:false, stats:pseudo +select *,length(a) from t1 where a like '测试 %'; +a length(a) +测试 8 +select *,length(a) from t1 where a like '测试'; +a length(a) +测试 6 +create table t2(a varchar(20) collate gbk_chinese_ci, index ia(a)); +insert into t2 value('测试'),('测试 '); +explain format = brief select *,length(a) from t2 where a like '测试 %'; +id estRows task access object operator info +Projection 8000.00 root planner__core__issuetest__planner_issue.t2.a, length(to_binary(planner__core__issuetest__planner_issue.t2.a))->Column#3 +└─UnionScan 8000.00 root like(planner__core__issuetest__planner_issue.t2.a, "测试 %", 92) + └─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] like(planner__core__issuetest__planner_issue.t2.a, "测试 %", 92) + └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +explain format = brief select *,length(a) from t2 where a like '测试'; +id estRows task access object operator info +Projection 8000.00 root planner__core__issuetest__planner_issue.t2.a, length(to_binary(planner__core__issuetest__planner_issue.t2.a))->Column#3 +└─UnionScan 8000.00 root like(planner__core__issuetest__planner_issue.t2.a, "测试", 92) + └─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] like(planner__core__issuetest__planner_issue.t2.a, "测试", 92) + └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +select *,length(a) from t2 where a like '测试 %'; +a length(a) +测试 6 +select *,length(a) from t2 where a like '测试'; +a length(a) +测试 4 diff --git a/tests/integrationtest/r/planner/core/plan.result b/tests/integrationtest/r/planner/core/plan.result index 76bee6627810d..344e3c2796215 100644 --- a/tests/integrationtest/r/planner/core/plan.result +++ b/tests/integrationtest/r/planner/core/plan.result @@ -137,8 +137,9 @@ CREATE TABLE `t1` ( `a` varchar(10) DEFAULT NULL, `b` varchar(10) DEFAULT NULL explain format='brief' select * from t1 where concat(a, b) like "aadwa" and a = "a"; id estRows task access object operator info Projection 0.10 root planner__core__plan.t1.a, planner__core__plan.t1.b -└─IndexReader 0.10 root index:IndexRangeScan - └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo +└─IndexReader 0.10 root index:Selection + └─Selection 0.10 cop[tikv] like(concat(planner__core__plan.t1.a, planner__core__plan.t1.b), "aadwa", 92) + └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo explain format='brief' select b from t1 where concat(a, b) >= "aa" and a = "b"; id estRows task access object operator info Projection 33.33 root planner__core__plan.t1.b @@ -147,8 +148,9 @@ Projection 33.33 root planner__core__plan.t1.b explain format='brief' select * from t1 where concat(a, b) like "aadwa" and a = "a"; id estRows task access object operator info Projection 0.10 root planner__core__plan.t1.a, planner__core__plan.t1.b -└─IndexReader 0.10 root index:IndexRangeScan - └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo +└─IndexReader 0.10 root index:Selection + └─Selection 0.10 cop[tikv] like(concat(planner__core__plan.t1.a, planner__core__plan.t1.b), "aadwa", 92) + └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo explain format='brief' select b from t1 where concat(a, b) >= "aa" and a = "b"; id estRows task access object operator info Projection 33.33 root planner__core__plan.t1.b @@ -157,8 +159,9 @@ Projection 33.33 root planner__core__plan.t1.b explain format='brief' select * from t1 where concat(a, b) like "aadwa" and a = "a"; id estRows task access object operator info Projection 0.10 root planner__core__plan.t1.a, planner__core__plan.t1.b -└─IndexReader 0.10 root index:IndexRangeScan - └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo +└─IndexReader 0.10 root index:Selection + └─Selection 0.10 cop[tikv] like(concat(planner__core__plan.t1.a, planner__core__plan.t1.b), "aadwa", 92) + └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo explain format='brief' select b from t1 where concat(a, b) >= "aa" and a = "b"; id estRows task access object operator info Projection 33.33 root planner__core__plan.t1.b @@ -167,8 +170,9 @@ Projection 33.33 root planner__core__plan.t1.b explain format='brief' select * from t1 where concat(a, b) like "aadwa" and a = "a"; id estRows task access object operator info Projection 0.10 root planner__core__plan.t1.a, planner__core__plan.t1.b -└─IndexReader 0.10 root index:IndexRangeScan - └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo +└─IndexReader 0.10 root index:Selection + └─Selection 0.10 cop[tikv] like(concat(planner__core__plan.t1.a, planner__core__plan.t1.b), "aadwa", 92) + └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo explain format='brief' select b from t1 where concat(a, b) >= "aa" and a = "b"; id estRows task access object operator info Projection 33.33 root planner__core__plan.t1.b @@ -177,8 +181,9 @@ Projection 33.33 root planner__core__plan.t1.b explain format='brief' select * from t1 where concat(a, b) like "aadwa" and a = "a"; id estRows task access object operator info Projection 0.10 root planner__core__plan.t1.a, planner__core__plan.t1.b -└─IndexReader 0.10 root index:IndexRangeScan - └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo +└─IndexReader 0.10 root index:Selection + └─Selection 0.10 cop[tikv] like(concat(planner__core__plan.t1.a, planner__core__plan.t1.b), "aadwa", 92) + └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo explain format='brief' select b from t1 where concat(a, b) >= "aa" and a = "b"; id estRows task access object operator info Projection 33.33 root planner__core__plan.t1.b @@ -187,8 +192,9 @@ Projection 33.33 root planner__core__plan.t1.b explain format='brief' select * from t1 where concat(a, b) like "aadwa" and a = "a"; id estRows task access object operator info Projection 0.10 root planner__core__plan.t1.a, planner__core__plan.t1.b -└─IndexReader 0.10 root index:IndexRangeScan - └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo +└─IndexReader 0.10 root index:Selection + └─Selection 0.10 cop[tikv] like(concat(planner__core__plan.t1.a, planner__core__plan.t1.b), "aadwa", 92) + └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo explain format='brief' select b from t1 where concat(a, b) >= "aa" and a = "b"; id estRows task access object operator info Projection 33.33 root planner__core__plan.t1.b @@ -197,8 +203,9 @@ Projection 33.33 root planner__core__plan.t1.b explain format='brief' select * from t1 where concat(a, b) like "aadwa" and a = "a"; id estRows task access object operator info Projection 0.10 root planner__core__plan.t1.a, planner__core__plan.t1.b -└─IndexReader 0.10 root index:IndexRangeScan - └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo +└─IndexReader 0.10 root index:Selection + └─Selection 0.10 cop[tikv] like(concat(planner__core__plan.t1.a, planner__core__plan.t1.b), "aadwa", 92) + └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo explain format='brief' select b from t1 where concat(a, b) >= "aa" and a = "b"; id estRows task access object operator info Projection 33.33 root planner__core__plan.t1.b @@ -207,8 +214,9 @@ Projection 33.33 root planner__core__plan.t1.b explain format='brief' select * from t1 where concat(a, b) like "aadwa" and a = "a"; id estRows task access object operator info Projection 0.10 root planner__core__plan.t1.a, planner__core__plan.t1.b -└─IndexReader 0.10 root index:IndexRangeScan - └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo +└─IndexReader 0.10 root index:Selection + └─Selection 0.10 cop[tikv] like(concat(planner__core__plan.t1.a, planner__core__plan.t1.b), "aadwa", 92) + └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo explain format='brief' select b from t1 where concat(a, b) >= "aa" and a = "b"; id estRows task access object operator info Projection 33.33 root planner__core__plan.t1.b @@ -217,8 +225,9 @@ Projection 33.33 root planner__core__plan.t1.b explain format='brief' select * from t1 where concat(a, b) like "aadwa" and a = "a"; id estRows task access object operator info Projection 0.10 root planner__core__plan.t1.a, planner__core__plan.t1.b -└─IndexReader 0.10 root index:IndexRangeScan - └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo +└─IndexReader 0.10 root index:Selection + └─Selection 0.10 cop[tikv] like(concat(planner__core__plan.t1.a, planner__core__plan.t1.b), "aadwa", 92) + └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo explain format='brief' select b from t1 where concat(a, b) >= "aa" and a = "b"; id estRows task access object operator info Projection 33.33 root planner__core__plan.t1.b @@ -227,8 +236,9 @@ Projection 33.33 root planner__core__plan.t1.b explain format='brief' select * from t1 where concat(a, b) like "aadwa" and a = "a"; id estRows task access object operator info Projection 0.10 root planner__core__plan.t1.a, planner__core__plan.t1.b -└─IndexReader 0.10 root index:IndexRangeScan - └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo +└─IndexReader 0.10 root index:Selection + └─Selection 0.10 cop[tikv] like(concat(planner__core__plan.t1.a, planner__core__plan.t1.b), "aadwa", 92) + └─IndexRangeScan 0.10 cop[tikv] table:t1, index:idx2(a, concat(`a`, `b`), b) range:["a" "aadwa","a" "aadwa"], keep order:false, stats:pseudo explain format='brief' select b from t1 where concat(a, b) >= "aa" and a = "b"; id estRows task access object operator info Projection 33.33 root planner__core__plan.t1.b diff --git a/tests/integrationtest/t/planner/core/issuetest/planner_issue.test b/tests/integrationtest/t/planner/core/issuetest/planner_issue.test index 1b58c7c5046c7..372e9eb8a67a6 100644 --- a/tests/integrationtest/t/planner/core/issuetest/planner_issue.test +++ b/tests/integrationtest/t/planner/core/issuetest/planner_issue.test @@ -136,3 +136,18 @@ FROM t2 db LEFT JOIN tmp3 c2 ON c2.id = '1' LEFT JOIN tmp3 c3 ON c3.id = '1'; + +# https://github.com/pingcap/tidb/issues/48821 +drop table if exists t1, t2; +create table t1(a varchar(20) collate utf8mb4_bin, index ia(a)); +insert into t1 value('测试'),('测试 '); +explain format = brief select *,length(a) from t1 where a like '测试 %'; +explain format = brief select *,length(a) from t1 where a like '测试'; +select *,length(a) from t1 where a like '测试 %'; +select *,length(a) from t1 where a like '测试'; +create table t2(a varchar(20) collate gbk_chinese_ci, index ia(a)); +insert into t2 value('测试'),('测试 '); +explain format = brief select *,length(a) from t2 where a like '测试 %'; +explain format = brief select *,length(a) from t2 where a like '测试'; +select *,length(a) from t2 where a like '测试 %'; +select *,length(a) from t2 where a like '测试';