diff --git a/DEPS.bzl b/DEPS.bzl index 4c3e94cca5a14..1d23df5e1e756 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2898,8 +2898,8 @@ def go_deps(): name = "com_github_pingcap_log", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/log", - sum = "h1:T7e5Low0BU2ZazI2dz2mh3W1qv+w8wtvq1YR8DneA0c=", - version = "v1.1.1-0.20221110065318-21a4942860b3", + sum = "h1:crhkw6DD+07Bg1wYhW5Piw+kYNKZqFQqfC2puUf6gMI=", + version = "v1.1.1-0.20221116035753-734d527bc87c", ) go_repository( name = "com_github_pingcap_sysutil", diff --git a/Dockerfile b/Dockerfile index 8416ef542a3d3..f3dae2519f53a 100644 --- a/Dockerfile +++ b/Dockerfile @@ -13,16 +13,24 @@ # limitations under the License. # Builder image -FROM alpine:edge as builder +FROM rockylinux:9 as builder -ADD . https://raw.githubusercontent.com/njhallett/apk-fastest-mirror/c4ca44caef3385d830fea34df2dbc2ba4a17e021/apk-fastest-mirror.sh ./proxy -RUN sh ./proxy/apk-fastest-mirror.sh -t 50 && apk add --no-cache git build-base go +ENV GOLANG_VERSION 1.19.3 +ENV ARCH amd64 +ENV GOLANG_DOWNLOAD_URL https://dl.google.com/go/go$GOLANG_VERSION.linux-$ARCH.tar.gz +ENV GOPATH /go +ENV GOROOT /usr/local/go +ENV PATH $GOPATH/bin:$GOROOT/bin:$PATH +RUN yum update -y && yum groupinstall 'Development Tools' -y \ + && curl -fsSL "$GOLANG_DOWNLOAD_URL" -o golang.tar.gz \ + && tar -C /usr/local -xzf golang.tar.gz \ + && rm golang.tar.gz COPY . /tidb ARG GOPROXY RUN export GOPROXY=${GOPROXY} && cd /tidb && make server -FROM alpine:latest +FROM rockylinux:9-minimal COPY --from=builder /tidb/bin/tidb-server /tidb-server diff --git a/autoid_service/autoid.go b/autoid_service/autoid.go index d20c78fd06098..f2836fc80fe85 100644 --- a/autoid_service/autoid.go +++ b/autoid_service/autoid.go @@ -405,12 +405,30 @@ func (s *Service) allocAutoID(ctx context.Context, req *autoid.AutoIDRequest) (* val := s.getAlloc(req.DbID, req.TblID, req.IsUnsigned) - if req.N == 0 && val.base != 0 { - base := val.base + if req.N == 0 { + if val.base != 0 { + return &autoid.AutoIDResponse{ + Min: val.base, + Max: val.base, + }, nil + } + // This item is not initialized, get the data from remote. + var currentEnd int64 + ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) + err := kv.RunInNewTxn(ctx, s.store, true, func(ctx context.Context, txn kv.Transaction) error { + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(req.DbID, req.TblID).RowID() + var err1 error + currentEnd, err1 = idAcc.Get() + if err1 != nil { + return err1 + } + val.end = currentEnd + return nil + }) return &autoid.AutoIDResponse{ - Min: base, - Max: base, - }, nil + Min: currentEnd, + Max: currentEnd, + }, err } val.Lock() diff --git a/ddl/column_test.go b/ddl/column_test.go index cae9a27318dec..d11d33e213d6a 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -959,3 +959,29 @@ func TestGetDefaultValueOfColumn(t *testing.T) { tk.MustQuery("select * from t1").Check(testkit.RowsWithSep("|", ""+ "1962-03-03 1962-03-03 00:00:00 12:23:23 2020-10-13 2020-03-27")) } + +func TestIssue39080(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("CREATE TABLE t1(id INTEGER PRIMARY KEY, authorId INTEGER AUTO_INCREMENT UNIQUE)") + + tk.MustQuery("show create table t1").Check(testkit.RowsWithSep("|", ""+ + "t1 CREATE TABLE `t1` (\n"+ + " `id` int(11) NOT NULL,\n"+ + " `authorId` int(11) NOT NULL AUTO_INCREMENT,\n"+ + " PRIMARY KEY (`id`) /*T![clustered_index] CLUSTERED */,\n"+ + " UNIQUE KEY `authorId` (`authorId`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + + //Do not affect the specified name + tk.MustExec("CREATE TABLE `t2`( `id` INTEGER PRIMARY KEY, `authorId` int(11) AUTO_INCREMENT, UNIQUE KEY `authorIdx` (`authorId`))") + + tk.MustQuery("show create table t2").Check(testkit.RowsWithSep("|", ""+ + "t2 CREATE TABLE `t2` (\n"+ + " `id` int(11) NOT NULL,\n"+ + " `authorId` int(11) NOT NULL AUTO_INCREMENT,\n"+ + " PRIMARY KEY (`id`) /*T![clustered_index] CLUSTERED */,\n"+ + " UNIQUE KEY `authorIdx` (`authorId`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) +} diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index b7cb35a9f8727..b25aa87f2fce8 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2894,17 +2894,23 @@ func TestAutoIncrementTableOption(t *testing.T) { tk.MustExec("create database test_auto_inc_table_opt;") tk.MustExec("use test_auto_inc_table_opt;") - // Empty auto_inc allocator should not cause error. - tk.MustExec("create table t (a bigint primary key clustered) auto_increment = 10;") - tk.MustExec("alter table t auto_increment = 10;") - tk.MustExec("alter table t auto_increment = 12345678901234567890;") - - // Rebase the auto_inc allocator to a large integer should work. - tk.MustExec("drop table t;") - tk.MustExec("create table t (a bigint unsigned auto_increment, unique key idx(a));") - tk.MustExec("alter table t auto_increment = 12345678901234567890;") - tk.MustExec("insert into t values ();") - tk.MustQuery("select * from t;").Check(testkit.Rows("12345678901234567890")) + for _, str := range []string{"", " AUTO_ID_CACHE 1"} { + // Empty auto_inc allocator should not cause error. + tk.MustExec("create table t (a bigint primary key clustered) auto_increment = 10" + str) + tk.MustExec("alter table t auto_increment = 10;") + tk.MustExec("alter table t auto_increment = 12345678901234567890;") + tk.MustExec("drop table t;") + + // Rebase the auto_inc allocator to a large integer should work. + tk.MustExec("create table t (a bigint unsigned auto_increment, unique key idx(a))" + str) + // Set auto_inc to negative is not supported + err := tk.ExecToErr("alter table t auto_increment = -1;") + require.Error(t, err) + tk.MustExec("alter table t auto_increment = 12345678901234567890;") + tk.MustExec("insert into t values ();") + tk.MustQuery("select * from t;").Check(testkit.Rows("12345678901234567890")) + tk.MustExec("drop table t;") + } } func TestAutoIncrementForce(t *testing.T) { @@ -2919,83 +2925,95 @@ func TestAutoIncrementForce(t *testing.T) { require.NoError(t, err) return gid } - // Rebase _tidb_row_id. - tk.MustExec("create table t (a int);") - tk.MustExec("alter table t force auto_increment = 2;") - tk.MustExec("insert into t values (1),(2);") - tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 2", "2 3")) - // Cannot set next global ID to 0. - tk.MustGetErrCode("alter table t force auto_increment = 0;", errno.ErrAutoincReadFailed) - tk.MustExec("alter table t force auto_increment = 1;") - require.Equal(t, uint64(1), getNextGlobalID()) - // inserting new rows can overwrite the existing data. - tk.MustExec("insert into t values (3);") - require.Equal(t, "[kv:1062]Duplicate entry '2' for key 't.PRIMARY'", tk.ExecToErr("insert into t values (3);").Error()) - tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("3 1", "1 2", "2 3")) - - // Rebase auto_increment. - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (a int primary key auto_increment, b int);") - tk.MustExec("insert into t values (1, 1);") - tk.MustExec("insert into t values (100000000, 1);") - tk.MustExec("delete from t where a = 100000000;") - require.Greater(t, getNextGlobalID(), uint64(100000000)) - // Cannot set next global ID to 0. - tk.MustGetErrCode("alter table t /*T![force_inc] force */ auto_increment = 0;", errno.ErrAutoincReadFailed) - tk.MustExec("alter table t /*T![force_inc] force */ auto_increment = 2;") - require.Equal(t, uint64(2), getNextGlobalID()) - tk.MustExec("insert into t(b) values (2);") - tk.MustQuery("select a, b from t;").Check(testkit.Rows("1 1", "2 2")) - - // Rebase auto_random. - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (a bigint primary key auto_random(5));") - tk.MustExec("insert into t values ();") - tk.MustExec("set @@allow_auto_random_explicit_insert = true") - tk.MustExec("insert into t values (100000000);") - tk.MustExec("delete from t where a = 100000000;") - require.Greater(t, getNextGlobalID(), uint64(100000000)) - // Cannot set next global ID to 0. - tk.MustGetErrCode("alter table t force auto_random_base = 0;", errno.ErrAutoincReadFailed) - tk.MustExec("alter table t force auto_random_base = 2;") - require.Equal(t, uint64(2), getNextGlobalID()) - tk.MustExec("insert into t values ();") - tk.MustQuery("select (a & 3) from t order by 1;").Check(testkit.Rows("1", "2")) - - // Change next global ID. - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (a bigint primary key auto_increment);") - tk.MustExec("insert into t values (1);") - bases := []uint64{1, 65535, 10, math.MaxUint64, math.MaxInt64 + 1, 1, math.MaxUint64, math.MaxInt64, 2} - lastBase := fmt.Sprintf("%d", bases[len(bases)-1]) - for _, b := range bases { - tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) - require.Equal(t, b, getNextGlobalID()) + + for _, str := range []string{"", " AUTO_ID_CACHE 1"} { + // Rebase _tidb_row_id. + tk.MustExec("create table t (a int)" + str) + tk.MustExec("alter table t force auto_increment = 2;") + tk.MustExec("insert into t values (1),(2);") + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 2", "2 3")) + // Cannot set next global ID to 0. + tk.MustGetErrCode("alter table t force auto_increment = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t force auto_increment = 1;") + require.Equal(t, uint64(1), getNextGlobalID()) + // inserting new rows can overwrite the existing data. + tk.MustExec("insert into t values (3);") + require.Equal(t, "[kv:1062]Duplicate entry '2' for key 't.PRIMARY'", tk.ExecToErr("insert into t values (3);").Error()) + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("3 1", "1 2", "2 3")) + tk.MustExec("drop table if exists t;") } - tk.MustExec("insert into t values ();") - tk.MustQuery("select a from t;").Check(testkit.Rows("1", lastBase)) - // Force alter unsigned int auto_increment column. - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (a bigint unsigned primary key auto_increment);") - for _, b := range bases { - tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) - require.Equal(t, b, getNextGlobalID()) + + for _, str := range []string{"", " AUTO_ID_CACHE 1"} { + // Rebase auto_increment. + tk.MustExec("create table t (a int primary key auto_increment, b int)" + str) + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (100000000, 1);") + tk.MustExec("delete from t where a = 100000000;") + require.Greater(t, getNextGlobalID(), uint64(100000000)) + // Cannot set next global ID to 0. + tk.MustGetErrCode("alter table t /*T![force_inc] force */ auto_increment = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t /*T![force_inc] force */ auto_increment = 2;") + require.Equal(t, uint64(2), getNextGlobalID()) + tk.MustExec("insert into t(b) values (2);") + tk.MustQuery("select a, b from t;").Check(testkit.Rows("1 1", "2 2")) + tk.MustExec("drop table if exists t;") + } + + for _, str := range []string{"", " AUTO_ID_CACHE 1"} { + // Rebase auto_random. + tk.MustExec("create table t (a bigint primary key auto_random(5))" + str) + tk.MustExec("insert into t values ();") + tk.MustExec("set @@allow_auto_random_explicit_insert = true") + tk.MustExec("insert into t values (100000000);") + tk.MustExec("delete from t where a = 100000000;") + require.Greater(t, getNextGlobalID(), uint64(100000000)) + // Cannot set next global ID to 0. + tk.MustGetErrCode("alter table t force auto_random_base = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t force auto_random_base = 2;") + require.Equal(t, uint64(2), getNextGlobalID()) tk.MustExec("insert into t values ();") - tk.MustQuery("select a from t;").Check(testkit.Rows(fmt.Sprintf("%d", b))) - tk.MustExec("delete from t;") + tk.MustQuery("select (a & 3) from t order by 1;").Check(testkit.Rows("1", "2")) + tk.MustExec("drop table if exists t;") } - // Force alter with @@auto_increment_increment and @@auto_increment_offset. - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a int key auto_increment);") - tk.MustExec("set @@auto_increment_offset=2;") - tk.MustExec("set @@auto_increment_increment = 11;") - tk.MustExec("insert into t values (500);") - tk.MustExec("alter table t force auto_increment=100;") - tk.MustExec("insert into t values (), ();") - tk.MustQuery("select * from t;").Check(testkit.Rows("101", "112", "500")) - tk.MustQuery("select * from t order by a;").Check(testkit.Rows("101", "112", "500")) - tk.MustExec("drop table if exists t;") + for _, str := range []string{"", " AUTO_ID_CACHE 1"} { + // Change next global ID. + tk.MustExec("create table t (a bigint primary key auto_increment)" + str) + tk.MustExec("insert into t values (1);") + bases := []uint64{1, 65535, 10, math.MaxUint64, math.MaxInt64 + 1, 1, math.MaxUint64, math.MaxInt64, 2} + lastBase := fmt.Sprintf("%d", bases[len(bases)-1]) + for _, b := range bases { + fmt.Println("execute alter table force increment to ==", b) + tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) + require.Equal(t, b, getNextGlobalID()) + } + tk.MustExec("insert into t values ();") + tk.MustQuery("select a from t;").Check(testkit.Rows("1", lastBase)) + // Force alter unsigned int auto_increment column. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint unsigned primary key auto_increment)" + str) + for _, b := range bases { + tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) + require.Equal(t, b, getNextGlobalID()) + tk.MustExec("insert into t values ();") + tk.MustQuery("select a from t;").Check(testkit.Rows(fmt.Sprintf("%d", b))) + tk.MustExec("delete from t;") + } + tk.MustExec("drop table if exists t;") + } + + for _, str := range []string{"", " AUTO_ID_CACHE 1"} { + // Force alter with @@auto_increment_increment and @@auto_increment_offset. + tk.MustExec("create table t(a int key auto_increment)" + str) + tk.MustExec("set @@auto_increment_offset=2;") + tk.MustExec("set @@auto_increment_increment = 11;") + tk.MustExec("insert into t values (500);") + tk.MustExec("alter table t force auto_increment=100;") + tk.MustExec("insert into t values (), ();") + tk.MustQuery("select * from t;").Check(testkit.Rows("101", "112", "500")) + tk.MustQuery("select * from t order by a;").Check(testkit.Rows("101", "112", "500")) + tk.MustExec("drop table if exists t;") + } // Check for warning in case we can't set the auto_increment to the desired value tk.MustExec("create table t(a int primary key auto_increment)") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index bbf3d4fc59856..6111dd3fb4fdb 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1693,7 +1693,7 @@ func setEmptyConstraintName(namesMap map[string]bool, constr *ast.Constraint) { } } if colName == "" { - colName = constr.Keys[0].Column.Name.L + colName = constr.Keys[0].Column.Name.O } constrName := colName i := 2 diff --git a/ddl/ddl_tiflash_api.go b/ddl/ddl_tiflash_api.go index 7e456815acd8c..4b8fca2a91c0f 100644 --- a/ddl/ddl_tiflash_api.go +++ b/ddl/ddl_tiflash_api.go @@ -29,20 +29,14 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/ddl/placement" ddlutil "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" atomicutil "go.uber.org/atomic" "go.uber.org/zap" @@ -50,12 +44,13 @@ import ( // TiFlashReplicaStatus records status for each TiFlash replica. type TiFlashReplicaStatus struct { - ID int64 - Count uint64 - LocationLabels []string - Available bool - HighPriority bool - IsPartition bool + ID int64 + Count uint64 + LocationLabels []string + Available bool + LogicalTableAvailable bool + HighPriority bool + IsPartition bool } // TiFlashTick is type for backoff threshold. @@ -283,16 +278,16 @@ func LoadTiFlashReplicaInfo(tblInfo *model.TableInfo, tableList *[]TiFlashReplic for _, p := range pi.Definitions { logutil.BgLogger().Debug(fmt.Sprintf("Table %v has partition %v\n", tblInfo.ID, p.ID)) *tableList = append(*tableList, TiFlashReplicaStatus{p.ID, - tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.IsPartitionAvailable(p.ID), false, true}) + tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.IsPartitionAvailable(p.ID), tblInfo.TiFlashReplica.Available, false, true}) } // partitions that in adding mid-state for _, p := range pi.AddingDefinitions { logutil.BgLogger().Debug(fmt.Sprintf("Table %v has partition adding %v\n", tblInfo.ID, p.ID)) - *tableList = append(*tableList, TiFlashReplicaStatus{p.ID, tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.IsPartitionAvailable(p.ID), true, true}) + *tableList = append(*tableList, TiFlashReplicaStatus{p.ID, tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.IsPartitionAvailable(p.ID), tblInfo.TiFlashReplica.Available, true, true}) } } else { logutil.BgLogger().Debug(fmt.Sprintf("Table %v has no partition\n", tblInfo.ID)) - *tableList = append(*tableList, TiFlashReplicaStatus{tblInfo.ID, tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.Available, false, false}) + *tableList = append(*tableList, TiFlashReplicaStatus{tblInfo.ID, tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.Available, tblInfo.TiFlashReplica.Available, false, false}) } } @@ -355,22 +350,6 @@ func updateTiFlashStores(pollTiFlashContext *TiFlashManagementContext) error { return nil } -func getTiFlashPeerWithoutLagCount(pollTiFlashContext *TiFlashManagementContext, tableID int64) (int, error) { - // storeIDs -> regionID, PD will not create two peer on the same store - var flashPeerCount int - for _, store := range pollTiFlashContext.TiFlashStores { - regionReplica := make(map[int64]int) - err := helper.CollectTiFlashStatus(store.Store.StatusAddress, tableID, ®ionReplica) - if err != nil { - logutil.BgLogger().Error("Fail to get peer status from TiFlash.", - zap.Int64("tableID", tableID)) - return 0, err - } - flashPeerCount += len(regionReplica) - } - return flashPeerCount, nil -} - func pollAvailableTableProgress(schemas infoschema.InfoSchema, ctx sessionctx.Context, pollTiFlashContext *TiFlashManagementContext) { pollMaxCount := RefreshProgressMaxTableCount failpoint.Inject("PollAvailableTableProgressMaxCount", func(val failpoint.Value) { @@ -466,6 +445,21 @@ func (d *ddl) refreshTiFlashTicker(ctx sessionctx.Context, pollTiFlashContext *T } } + failpoint.Inject("waitForAddPartition", func(val failpoint.Value) { + for _, phyTable := range tableList { + is := d.infoCache.GetLatest() + _, ok := is.TableByID(phyTable.ID) + if !ok { + tb, _, _ := is.FindTableByPartitionID(phyTable.ID) + if tb == nil { + logutil.BgLogger().Info("[ddl] waitForAddPartition") + sleepSecond := val.(int) + time.Sleep(time.Duration(sleepSecond) * time.Second) + } + } + } + }) + needPushPending := false if pollTiFlashContext.UpdatingProgressTables.Len() == 0 { needPushPending = true @@ -479,7 +473,7 @@ func (d *ddl) refreshTiFlashTicker(ctx sessionctx.Context, pollTiFlashContext *T available = val.(bool) }) // We only check unavailable tables here, so doesn't include blocked add partition case. - if !available { + if !available && !tb.LogicalTableAvailable { enabled, inqueue, _ := pollTiFlashContext.Backoff.Tick(tb.ID) if inqueue && !enabled { logutil.BgLogger().Info("Escape checking available status due to backoff", zap.Int64("tableId", tb.ID)) @@ -540,110 +534,6 @@ func (d *ddl) refreshTiFlashTicker(ctx sessionctx.Context, pollTiFlashContext *T return nil } -func getDropOrTruncateTableTiflash(ctx sessionctx.Context, currentSchema infoschema.InfoSchema, tikvHelper *helper.Helper, replicaInfos *[]TiFlashReplicaStatus) error { - store := tikvHelper.Store.(kv.Storage) - - txn, err := store.Begin() - if err != nil { - return errors.Trace(err) - } - gcSafePoint, err := gcutil.GetGCSafePoint(ctx) - if err != nil { - return err - } - uniqueIDMap := make(map[int64]struct{}) - handleJobAndTableInfo := func(job *model.Job, tblInfo *model.TableInfo) (bool, error) { - // Avoid duplicate table ID info. - if _, ok := currentSchema.TableByID(tblInfo.ID); ok { - return false, nil - } - if _, ok := uniqueIDMap[tblInfo.ID]; ok { - return false, nil - } - uniqueIDMap[tblInfo.ID] = struct{}{} - LoadTiFlashReplicaInfo(tblInfo, replicaInfos) - return false, nil - } - fn := func(jobs []*model.Job) (bool, error) { - getTable := func(StartTS uint64, SchemaID int64, TableID int64) (*model.TableInfo, error) { - snapMeta := meta.NewSnapshotMeta(store.GetSnapshot(kv.NewVersion(StartTS))) - if err != nil { - return nil, err - } - tbl, err := snapMeta.GetTable(SchemaID, TableID) - return tbl, err - } - return GetDropOrTruncateTableInfoFromJobsByStore(jobs, gcSafePoint, getTable, handleJobAndTableInfo) - } - - err = IterAllDDLJobs(ctx, txn, fn) - if err != nil { - if terror.ErrorEqual(variable.ErrSnapshotTooOld, err) { - // The err indicate that current ddl job and remain DDL jobs was been deleted by GC, - // just ignore the error and return directly. - return nil - } - return err - } - return nil -} - -// HandlePlacementRuleRoutine fetch all rules from pd, remove all obsolete rules. -// It handles rare situation, when we fail to alter pd rules. -func HandlePlacementRuleRoutine(ctx sessionctx.Context, d *ddl, tableList []TiFlashReplicaStatus) error { - c := context.Background() - tikvStore, ok := ctx.GetStore().(helper.Storage) - if !ok { - return errors.New("Can not get Helper") - } - tikvHelper := &helper.Helper{ - Store: tikvStore, - RegionCache: tikvStore.GetRegionCache(), - } - - allRulesArr, err := infosync.GetTiFlashGroupRules(c, "tiflash") - if err != nil { - return errors.Trace(err) - } - allRules := make(map[string]placement.TiFlashRule) - for _, r := range allRulesArr { - allRules[r.ID] = r - } - - start := time.Now() - originLen := len(tableList) - currentSchema := d.GetInfoSchemaWithInterceptor(ctx) - if err := getDropOrTruncateTableTiflash(ctx, currentSchema, tikvHelper, &tableList); err != nil { - // may fail when no `tikv_gc_safe_point` available, should return in order to remove valid pd rules. - logutil.BgLogger().Error("getDropOrTruncateTableTiflash returns error", zap.Error(err)) - return errors.Trace(err) - } - elapsed := time.Since(start) - logutil.BgLogger().Info("getDropOrTruncateTableTiflash cost", zap.Duration("time", elapsed), zap.Int("updated", len(tableList)-originLen)) - for _, tb := range tableList { - // For every region in each table, if it has one replica, we reckon it ready. - ruleID := fmt.Sprintf("table-%v-r", tb.ID) - if _, ok := allRules[ruleID]; !ok { - // Mostly because of a previous failure of setting pd rule. - logutil.BgLogger().Warn(fmt.Sprintf("Table %v exists, but there are no rule for it", tb.ID)) - newRule := infosync.MakeNewRule(tb.ID, tb.Count, tb.LocationLabels) - _ = infosync.SetTiFlashPlacementRule(context.Background(), *newRule) - } - // For every existing table, we do not remove their rules. - delete(allRules, ruleID) - } - - // Remove rules of non-existing table - for _, v := range allRules { - logutil.BgLogger().Info("Remove TiFlash rule", zap.String("id", v.ID)) - if err := infosync.DeleteTiFlashPlacementRule(c, "tiflash", v.ID); err != nil { - logutil.BgLogger().Warn("delete TiFlash pd rule failed", zap.Error(err), zap.String("ruleID", v.ID)) - } - } - - return nil -} - func (d *ddl) PollTiFlashRoutine() { pollTiflashContext, err := NewTiFlashManagementContext() if err != nil { diff --git a/ddl/ddl_tiflash_test.go b/ddl/ddl_tiflash_test.go index 27d05112df483..accf7cc038ebd 100644 --- a/ddl/ddl_tiflash_test.go +++ b/ddl/ddl_tiflash_test.go @@ -1264,3 +1264,38 @@ func TestTiFlashPartitionNotAvailable(t *testing.T) { require.NotNil(t, replica) require.True(t, replica.Available) } + +func TestTiFlashAvailableAfterAddPartition(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists ddltiflash") + tk.MustExec("create table ddltiflash(z int) PARTITION BY RANGE(z) (PARTITION p0 VALUES LESS THAN (10))") + tk.MustExec("alter table ddltiflash set tiflash replica 1") + time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) + CheckTableAvailable(s.dom, t, 1, []string{}) + + tb, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) + require.NoError(t, err) + require.NotNil(t, tb) + + // still available after adding partition. + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/sleepBeforeReplicaOnly", `return(2)`)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/waitForAddPartition", `return(3)`)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/PollTiFlashReplicaStatusReplaceCurAvailableValue", `return(false)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/sleepBeforeReplicaOnly")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/waitForAddPartition")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/PollTiFlashReplicaStatusReplaceCurAvailableValue")) + }() + tk.MustExec("ALTER TABLE ddltiflash ADD PARTITION (PARTITION pn VALUES LESS THAN (20))") + time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) + CheckTableAvailable(s.dom, t, 1, []string{}) + tb, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) + require.NoError(t, err) + pi := tb.Meta().GetPartitionInfo() + require.NotNil(t, pi) + require.Equal(t, len(pi.Definitions), 2) +} diff --git a/ddl/job_table.go b/ddl/job_table.go index d23f083539e87..62bba65cf6678 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -67,7 +67,7 @@ func (dc *ddlCtx) excludeJobIDs() string { } const ( - getJobSQL = "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) and %s reorg %s order by processing desc, job_id" + getJobSQL = "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 limit 1) and %s reorg %s order by processing desc, job_id" ) type jobType int diff --git a/ddl/job_table_test.go b/ddl/job_table_test.go index 8f9e59ae31084..25d8d4300eacd 100644 --- a/ddl/job_table_test.go +++ b/ddl/job_table_test.go @@ -177,3 +177,68 @@ func check(t *testing.T, record []int64, ids ...int64) { } } } + +func TestAlwaysChoiceProcessingJob(t *testing.T) { + if !variable.EnableConcurrentDDL.Load() { + t.Skipf("test requires concurrent ddl") + } + store, dom := testkit.CreateMockStoreAndDomain(t) + + d := dom.DDL() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int)") + + ddlJobs := []string{ + "alter table t add index idx(a)", + "alter table t add index idx(b)", + } + + hook := &ddl.TestDDLCallback{} + var wg util.WaitGroupWrapper + wg.Add(1) + var once sync.Once + var idxa, idxb int64 + hook.OnGetJobBeforeExported = func(jobType string) { + once.Do(func() { + var jobs []*model.Job + for i, job := range ddlJobs { + wg.Run(func() { + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + recordSet, _ := tk.Exec(job) + if recordSet != nil { + require.NoError(t, recordSet.Close()) + } + }) + for { + time.Sleep(time.Millisecond * 100) + var err error + jobs, err = ddl.GetAllDDLJobs(testkit.NewTestKit(t, store).Session(), nil) + require.NoError(t, err) + if len(jobs) == i+1 { + break + } + } + } + idxa = jobs[0].ID + idxb = jobs[1].ID + require.Greater(t, idxb, idxa) + tk := testkit.NewTestKit(t, store) + tk.MustExec("update mysql.tidb_ddl_job set processing = 1 where job_id = ?", idxb) + wg.Done() + }) + } + + record := make([]int64, 0, 16) + hook.OnGetJobAfterExported = func(jobType string, job *model.Job) { + // record the job schedule order + record = append(record, job.ID) + } + + d.SetHook(hook) + wg.Wait() + + check(t, record, idxb, idxa) +} diff --git a/ddl/metadatalocktest/mdl_test.go b/ddl/metadatalocktest/mdl_test.go index d91bee7010013..6f4a0012cdfff 100644 --- a/ddl/metadatalocktest/mdl_test.go +++ b/ddl/metadatalocktest/mdl_test.go @@ -292,7 +292,7 @@ func TestMDLRRUpdateSchema(t *testing.T) { // Modify column(reorg). tk.MustExec("begin") tkDDL.MustExec("alter table test.t modify column a char(10);") - tk.MustGetErrCode("select * from t", mysql.ErrSchemaChanged) + tk.MustGetErrCode("select * from t", mysql.ErrInfoSchemaChanged) tk.MustExec("commit") tk.MustQuery("select * from t").Check(testkit.Rows("1 ")) diff --git a/ddl/partition.go b/ddl/partition.go index 7bba0b1006332..a8947d091bfc5 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -170,6 +170,10 @@ func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (v job.SchemaState = model.StateReplicaOnly case model.StateReplicaOnly: // replica only -> public + failpoint.Inject("sleepBeforeReplicaOnly", func(val failpoint.Value) { + sleepSecond := val.(int) + time.Sleep(time.Duration(sleepSecond) * time.Second) + }) // Here need do some tiflash replica complement check. // TODO: If a table is with no TiFlashReplica or it is not available, the replica-only state can be eliminated. if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Available { @@ -193,6 +197,15 @@ func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (v if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Available { for _, d := range partInfo.Definitions { tblInfo.TiFlashReplica.AvailablePartitionIDs = append(tblInfo.TiFlashReplica.AvailablePartitionIDs, d.ID) + err = infosync.UpdateTiFlashProgressCache(d.ID, 1) + if err != nil { + // just print log, progress will be updated in `refreshTiFlashTicker` + logutil.BgLogger().Error("update tiflash sync progress cache failed", + zap.Error(err), + zap.Int64("tableID", tblInfo.ID), + zap.Int64("partitionID", d.ID), + ) + } } } // For normal and replica finished table, move the `addingDefinitions` into `Definitions`. diff --git a/domain/BUILD.bazel b/domain/BUILD.bazel index f7ef9baba6907..4b575f4dc63e0 100644 --- a/domain/BUILD.bazel +++ b/domain/BUILD.bazel @@ -42,7 +42,9 @@ go_library( "//privilege/privileges", "//sessionctx", "//sessionctx/sessionstates", + "//sessionctx/stmtctx", "//sessionctx/variable", + "//statistics", "//statistics/handle", "//telemetry", "//types", diff --git a/domain/domain.go b/domain/domain.go index 1016d5ba9b5cb..b900cf3eb8d3a 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1532,11 +1532,17 @@ func (do *Domain) TelemetryRotateSubWindowLoop(ctx sessionctx.Context) { } // SetupPlanReplayerHandle setup plan replayer handle -func (do *Domain) SetupPlanReplayerHandle(ctx sessionctx.Context) { - do.planReplayerHandle = &planReplayerHandle{ - planReplayerTaskCollectorHandle: &planReplayerTaskCollectorHandle{ - sctx: ctx, - }, +func (do *Domain) SetupPlanReplayerHandle(collectorSctx, dumperSctx sessionctx.Context) { + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) + do.planReplayerHandle = &planReplayerHandle{} + do.planReplayerHandle.planReplayerTaskCollectorHandle = &planReplayerTaskCollectorHandle{ + ctx: ctx, + sctx: collectorSctx, + } + do.planReplayerHandle.planReplayerTaskDumpHandle = &planReplayerTaskDumpHandle{ + ctx: ctx, + sctx: dumperSctx, + taskCH: make(chan *PlanReplayerDumpTask, 16), } } @@ -1557,27 +1563,42 @@ func (do *Domain) StartPlanReplayerHandle() { if planReplayerHandleLease < 1 { return } - do.wg.Add(1) + do.wg.Add(2) go func() { tikcer := time.NewTicker(planReplayerHandleLease) defer func() { tikcer.Stop() do.wg.Done() - logutil.BgLogger().Info("PlanReplayerHandle exited.") - util.Recover(metrics.LabelDomain, "PlanReplayerHandle", nil, false) + logutil.BgLogger().Info("PlanReplayerTaskCollectHandle exited.") + util.Recover(metrics.LabelDomain, "PlanReplayerTaskCollectHandle", nil, false) }() for { select { case <-do.exit: return case <-tikcer.C: - err := do.planReplayerHandle.CollectPlanReplayerTask(context.Background()) + err := do.planReplayerHandle.CollectPlanReplayerTask() if err != nil { logutil.BgLogger().Warn("plan replayer handle collect tasks failed", zap.Error(err)) } } } }() + go func() { + defer func() { + do.wg.Done() + logutil.BgLogger().Info("PlanReplayerTaskDumpHandle exited.") + util.Recover(metrics.LabelDomain, "PlanReplayerTaskDumpHandle", nil, false) + }() + for { + select { + case <-do.exit: + return + case task := <-do.planReplayerHandle.planReplayerTaskDumpHandle.taskCH: + do.planReplayerHandle.HandlePlanReplayerDumpTask(task) + } + } + }() } // GetPlanReplayerHandle returns plan replayer handle diff --git a/domain/plan_replayer.go b/domain/plan_replayer.go index efc2e8ad21429..faab592950c64 100644 --- a/domain/plan_replayer.go +++ b/domain/plan_replayer.go @@ -16,8 +16,10 @@ package domain import ( "context" + "encoding/base64" "fmt" "io/ioutil" + "math/rand" "os" "path/filepath" "strconv" @@ -33,7 +35,9 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" @@ -122,6 +126,16 @@ func (p *dumpFileGcChecker) gcDumpFilesByPath(path string, t time.Duration) { type planReplayerHandle struct { *planReplayerTaskCollectorHandle + *planReplayerTaskDumpHandle +} + +// HandlePlanReplayerDumpTask handle dump task +func (h *planReplayerHandle) HandlePlanReplayerDumpTask(task *PlanReplayerDumpTask) bool { + success := h.dumpPlanReplayerDumpTask(task) + if success { + h.removeTask(task.PlanReplayerTaskKey) + } + return success } type planReplayerTaskCollectorHandle struct { @@ -129,6 +143,7 @@ type planReplayerTaskCollectorHandle struct { sync.RWMutex tasks map[PlanReplayerTaskKey]struct{} } + ctx context.Context sctx sessionctx.Context } @@ -153,32 +168,30 @@ func insertPlanReplayerStatus(ctx context.Context, sctx sessionctx.Context, reco instance = fmt.Sprintf("%s:%d", serverInfo.IP, serverInfo.Port) } for _, record := range records { - if !record.Internal { - if len(record.FailedReason) > 0 { - insertExternalPlanReplayerErrorStatusRecord(ctx1, sctx, instance, record) - } else { - insertExternalPlanReplayerSuccessStatusRecord(ctx1, sctx, instance, record) - } + if len(record.FailedReason) > 0 { + insertPlanReplayerErrorStatusRecord(ctx1, sctx, instance, record) + } else { + insertPlanReplayerSuccessStatusRecord(ctx1, sctx, instance, record) } } } -func insertExternalPlanReplayerErrorStatusRecord(ctx context.Context, sctx sessionctx.Context, instance string, record PlanReplayerStatusRecord) { +func insertPlanReplayerErrorStatusRecord(ctx context.Context, sctx sessionctx.Context, instance string, record PlanReplayerStatusRecord) { exec := sctx.(sqlexec.SQLExecutor) _, err := exec.ExecuteInternal(ctx, fmt.Sprintf( - "insert into mysql.plan_replayer_status (origin_sql, fail_reason, instance) values ('%s','%s','%s')", - record.OriginSQL, record.FailedReason, instance)) + "insert into mysql.plan_replayer_status (sql_digest, plan_digest, origin_sql, fail_reason, instance) values ('%s','%s','%s','%s','%s')", + record.SQLDigest, record.PlanDigest, record.OriginSQL, record.FailedReason, instance)) if err != nil { logutil.BgLogger().Warn("insert mysql.plan_replayer_status record failed", zap.Error(err)) } } -func insertExternalPlanReplayerSuccessStatusRecord(ctx context.Context, sctx sessionctx.Context, instance string, record PlanReplayerStatusRecord) { +func insertPlanReplayerSuccessStatusRecord(ctx context.Context, sctx sessionctx.Context, instance string, record PlanReplayerStatusRecord) { exec := sctx.(sqlexec.SQLExecutor) _, err := exec.ExecuteInternal(ctx, fmt.Sprintf( - "insert into mysql.plan_replayer_status (origin_sql, token, instance) values ('%s','%s','%s')", - record.OriginSQL, record.Token, instance)) + "insert into mysql.plan_replayer_status (sql_digest, plan_digest, origin_sql, token, instance) values ('%s','%s','%s','%s','%s')", + record.SQLDigest, record.PlanDigest, record.OriginSQL, record.Token, instance)) if err != nil { logutil.BgLogger().Warn("insert mysql.plan_replayer_status record failed", zap.Error(err)) @@ -186,15 +199,14 @@ func insertExternalPlanReplayerSuccessStatusRecord(ctx context.Context, sctx ses } // CollectPlanReplayerTask collects all unhandled plan replayer task -func (h *planReplayerTaskCollectorHandle) CollectPlanReplayerTask(ctx context.Context) error { - ctx1 := kv.WithInternalSourceType(ctx, kv.InternalTxnStats) - allKeys, err := h.collectAllPlanReplayerTask(ctx1) +func (h *planReplayerTaskCollectorHandle) CollectPlanReplayerTask() error { + allKeys, err := h.collectAllPlanReplayerTask(h.ctx) if err != nil { return err } tasks := make([]PlanReplayerTaskKey, 0) for _, key := range allKeys { - unhandled, err := checkUnHandledReplayerTask(ctx1, h.sctx, key) + unhandled, err := checkUnHandledReplayerTask(h.ctx, h.sctx, key) if err != nil { return err } @@ -227,6 +239,12 @@ func (h *planReplayerTaskCollectorHandle) setupTasks(tasks []PlanReplayerTaskKey h.taskMu.tasks = r } +func (h *planReplayerTaskCollectorHandle) removeTask(taskKey PlanReplayerTaskKey) { + h.taskMu.Lock() + defer h.taskMu.Unlock() + delete(h.taskMu.tasks, taskKey) +} + func (h *planReplayerTaskCollectorHandle) collectAllPlanReplayerTask(ctx context.Context) ([]PlanReplayerTaskKey, error) { exec := h.sctx.(sqlexec.SQLExecutor) rs, err := exec.ExecuteInternal(ctx, "select sql_digest, plan_digest from mysql.plan_replayer_task") @@ -245,16 +263,96 @@ func (h *planReplayerTaskCollectorHandle) collectAllPlanReplayerTask(ctx context for _, row := range rows { sqlDigest, planDigest := row.GetString(0), row.GetString(1) allKeys = append(allKeys, PlanReplayerTaskKey{ - sqlDigest: sqlDigest, - planDigest: planDigest, + SQLDigest: sqlDigest, + PlanDigest: planDigest, }) } return allKeys, nil } +type planReplayerTaskDumpHandle struct { + ctx context.Context + sctx sessionctx.Context + taskCH chan *PlanReplayerDumpTask +} + +// DrainTask drain a task for unit test +func (h *planReplayerTaskDumpHandle) DrainTask() *PlanReplayerDumpTask { + return <-h.taskCH +} + +// HandlePlanReplayerDumpTask handled the task +func (h *planReplayerTaskDumpHandle) dumpPlanReplayerDumpTask(task *PlanReplayerDumpTask) (success bool) { + taskKey := task.PlanReplayerTaskKey + unhandled, err := checkUnHandledReplayerTask(h.ctx, h.sctx, taskKey) + if err != nil { + logutil.BgLogger().Warn("check plan replayer capture task failed", + zap.String("sqlDigest", taskKey.SQLDigest), + zap.String("planDigest", taskKey.PlanDigest), + zap.Error(err)) + return false + } + // the task is processed, thus we directly skip it. + if !unhandled { + return true + } + + file, fileName, err := GeneratePlanReplayerFile() + if err != nil { + logutil.BgLogger().Warn("generate plan replayer capture task file failed", + zap.String("sqlDigest", taskKey.SQLDigest), + zap.String("planDigest", taskKey.PlanDigest), + zap.Error(err)) + return + } + task.Zf = file + task.FileName = fileName + task.EncodedPlan, _ = task.EncodePlan(task.SessionVars.StmtCtx, false) + jsStats := make(map[int64]*handle.JSONTable) + is := GetDomain(h.sctx).InfoSchema() + for tblID, stat := range task.TblStats { + tbl, ok := is.TableByID(tblID) + if !ok { + return false + } + schema, ok := is.SchemaByTable(tbl.Meta()) + if !ok { + return false + } + r, err := handle.GenJSONTableFromStats(schema.Name.String(), tbl.Meta(), stat.(*statistics.Table)) + if err != nil { + logutil.BgLogger().Warn("generate plan replayer capture task json stats failed", + zap.String("sqlDigest", taskKey.SQLDigest), + zap.String("planDigest", taskKey.PlanDigest), + zap.Error(err)) + return false + } + jsStats[tblID] = r + } + err = DumpPlanReplayerInfo(h.ctx, h.sctx, task) + if err != nil { + logutil.BgLogger().Warn("dump plan replayer capture task result failed", + zap.String("sqlDigest", taskKey.SQLDigest), + zap.String("planDigest", taskKey.PlanDigest), + zap.Error(err)) + return false + } + return true +} + +// SendTask send dumpTask in background task handler +func (h *planReplayerTaskDumpHandle) SendTask(task *PlanReplayerDumpTask) { + select { + case h.taskCH <- task: + default: + // TODO: add metrics here + // directly discard the task if the task channel is full in order not to block the query process + } +} + func checkUnHandledReplayerTask(ctx context.Context, sctx sessionctx.Context, task PlanReplayerTaskKey) (bool, error) { exec := sctx.(sqlexec.SQLExecutor) - rs, err := exec.ExecuteInternal(ctx, fmt.Sprintf("select * from mysql.plan_replayer_status where sql_digest = '%v' and plan_digest = '%v' and fail_reason is null", task.sqlDigest, task.planDigest)) + rs, err := exec.ExecuteInternal(ctx, fmt.Sprintf("select * from mysql.plan_replayer_status where sql_digest = '%v' and plan_digest = '%v' and fail_reason is null", task.SQLDigest, task.PlanDigest)) if err != nil { return false, err } @@ -274,7 +372,8 @@ func checkUnHandledReplayerTask(ctx context.Context, sctx sessionctx.Context, ta // PlanReplayerStatusRecord indicates record in mysql.plan_replayer_status type PlanReplayerStatusRecord struct { - Internal bool + SQLDigest string + PlanDigest string OriginSQL string Token string FailedReason string @@ -282,18 +381,57 @@ type PlanReplayerStatusRecord struct { // PlanReplayerTaskKey indicates key of a plan replayer task type PlanReplayerTaskKey struct { - sqlDigest string - planDigest string + SQLDigest string + PlanDigest string } // PlanReplayerDumpTask wrap the params for plan replayer dump type PlanReplayerDumpTask struct { + PlanReplayerTaskKey + + // tmp variables stored during the query + EncodePlan func(*stmtctx.StatementContext, bool) (string, string) + TblStats map[int64]interface{} + + // variables used to dump the plan SessionBindings []*bindinfo.BindRecord EncodedPlan string - FileName string - Zf *os.File SessionVars *variable.SessionVars - TblStats map[int64]*handle.JSONTable + JSONTblStats map[int64]*handle.JSONTable ExecStmts []ast.StmtNode Analyze bool + + FileName string + Zf *os.File +} + +// GeneratePlanReplayerFile generates plan replayer file +func GeneratePlanReplayerFile() (*os.File, string, error) { + path := GetPlanReplayerDirName() + err := os.MkdirAll(path, os.ModePerm) + if err != nil { + return nil, "", errors.AddStack(err) + } + fileName, err := generatePlanReplayerFileName() + if err != nil { + return nil, "", errors.AddStack(err) + } + zf, err := os.Create(filepath.Join(path, fileName)) + if err != nil { + return nil, "", errors.AddStack(err) + } + return zf, fileName, err +} + +func generatePlanReplayerFileName() (string, error) { + // Generate key and create zip file + time := time.Now().UnixNano() + b := make([]byte, 16) + //nolint: gosec + _, err := rand.Read(b) + if err != nil { + return "", err + } + key := base64.URLEncoding.EncodeToString(b) + return fmt.Sprintf("replayer_%v_%v.zip", key, time), nil } diff --git a/domain/plan_replayer_dump.go b/domain/plan_replayer_dump.go index 93d0278a4ba3d..195dae7b4a0b1 100644 --- a/domain/plan_replayer_dump.go +++ b/domain/plan_replayer_dump.go @@ -210,7 +210,7 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, } // Dump stats - if err = dumpStats(zw, pairs, task.TblStats, do); err != nil { + if err = dumpStats(zw, pairs, task.JSONTblStats, do); err != nil { return err } @@ -252,9 +252,10 @@ func generateRecords(task *PlanReplayerDumpTask) []PlanReplayerStatusRecord { if len(task.ExecStmts) > 0 { for _, execStmt := range task.ExecStmts { records = append(records, PlanReplayerStatusRecord{ - OriginSQL: execStmt.Text(), - Token: task.FileName, - Internal: false, + SQLDigest: task.SQLDigest, + PlanDigest: task.PlanDigest, + OriginSQL: execStmt.Text(), + Token: task.FileName, }) } } diff --git a/domain/plan_replayer_handle_test.go b/domain/plan_replayer_handle_test.go index 2c25f56e15045..5a824ef4eeeb6 100644 --- a/domain/plan_replayer_handle_test.go +++ b/domain/plan_replayer_handle_test.go @@ -15,7 +15,7 @@ package domain_test import ( - "context" + "fmt" "testing" "github.com/pingcap/tidb/testkit" @@ -31,14 +31,14 @@ func TestPlanReplayerHandleCollectTask(t *testing.T) { tk.MustExec("delete from mysql.plan_replayer_task") tk.MustExec("delete from mysql.plan_replayer_status") tk.MustExec("insert into mysql.plan_replayer_task (sql_digest, plan_digest) values ('123','123');") - err := prHandle.CollectPlanReplayerTask(context.Background()) + err := prHandle.CollectPlanReplayerTask() require.NoError(t, err) require.Len(t, prHandle.GetTasks(), 1) // assert no task tk.MustExec("delete from mysql.plan_replayer_task") tk.MustExec("delete from mysql.plan_replayer_status") - err = prHandle.CollectPlanReplayerTask(context.Background()) + err = prHandle.CollectPlanReplayerTask() require.NoError(t, err) require.Len(t, prHandle.GetTasks(), 0) @@ -48,7 +48,7 @@ func TestPlanReplayerHandleCollectTask(t *testing.T) { tk.MustExec("insert into mysql.plan_replayer_task (sql_digest, plan_digest) values ('123','123');") tk.MustExec("insert into mysql.plan_replayer_task (sql_digest, plan_digest) values ('345','345');") tk.MustExec("insert into mysql.plan_replayer_status(sql_digest, plan_digest, token, instance) values ('123','123','123','123')") - err = prHandle.CollectPlanReplayerTask(context.Background()) + err = prHandle.CollectPlanReplayerTask() require.NoError(t, err) require.Len(t, prHandle.GetTasks(), 1) @@ -58,7 +58,44 @@ func TestPlanReplayerHandleCollectTask(t *testing.T) { tk.MustExec("insert into mysql.plan_replayer_task (sql_digest, plan_digest) values ('123','123');") tk.MustExec("insert into mysql.plan_replayer_task (sql_digest, plan_digest) values ('345','345');") tk.MustExec("insert into mysql.plan_replayer_status(sql_digest, plan_digest, fail_reason, instance) values ('123','123','123','123')") - err = prHandle.CollectPlanReplayerTask(context.Background()) + err = prHandle.CollectPlanReplayerTask() require.NoError(t, err) require.Len(t, prHandle.GetTasks(), 2) } + +func TestPlanReplayerHandleDumpTask(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + prHandle := dom.GetPlanReplayerHandle() + tk.MustExec("use test") + tk.MustExec("create table t(a int)") + tk.MustQuery("select * from t;") + _, d := tk.Session().GetSessionVars().StmtCtx.SQLDigest() + _, pd := tk.Session().GetSessionVars().StmtCtx.GetPlanDigest() + sqlDigest := d.String() + planDigest := pd.String() + + // register task + tk.MustExec("delete from mysql.plan_replayer_task") + tk.MustExec("delete from mysql.plan_replayer_status") + tk.MustExec(fmt.Sprintf("insert into mysql.plan_replayer_task (sql_digest, plan_digest) values ('%v','%v');", sqlDigest, planDigest)) + err := prHandle.CollectPlanReplayerTask() + require.NoError(t, err) + require.Len(t, prHandle.GetTasks(), 1) + + tk.MustExec("SET @@tidb_enable_plan_replayer_capture = ON;") + + // capture task and dump + tk.MustQuery("select * from t;") + task := prHandle.DrainTask() + require.NotNil(t, task) + success := prHandle.HandlePlanReplayerDumpTask(task) + require.True(t, success) + // assert memory task consumed + require.Len(t, prHandle.GetTasks(), 0) + + // assert collect task again and no more memory task + err = prHandle.CollectPlanReplayerTask() + require.NoError(t, err) + require.Len(t, prHandle.GetTasks(), 0) +} diff --git a/executor/adapter.go b/executor/adapter.go index db9fbbaa929e0..3dd0e0ce0877e 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -1629,6 +1629,11 @@ func getPlanDigest(stmtCtx *stmtctx.StatementContext) (string, *parser.Digest) { return normalized, planDigest } +// GetEncodedPlan returned same as getEncodedPlan +func GetEncodedPlan(stmtCtx *stmtctx.StatementContext, genHint bool) (encodedPlan, hintStr string) { + return getEncodedPlan(stmtCtx, genHint) +} + // getEncodedPlan gets the encoded plan, and generates the hint string if indicated. func getEncodedPlan(stmtCtx *stmtctx.StatementContext, genHint bool) (encodedPlan, hintStr string) { var hintSet bool diff --git a/executor/analyze_test.go b/executor/analyze_test.go index bbe3f5b8d1b1e..a6cdea833df50 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -417,3 +417,24 @@ func TestAnalyzePartitionTableByConcurrencyInDynamic(t *testing.T) { tk.MustQuery("show stats_topn where partition_name = 'global' and table_name = 't'").CheckAt([]int{5, 6}, expected) } } + +func TestMergeGlobalStatsWithUnAnalyzedPartition(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_partition_prune_mode=dynamic;") + tk.MustExec("CREATE TABLE `t` ( `id` int(11) DEFAULT NULL, `a` int(11) DEFAULT NULL, `b` int(11) DEFAULT NULL, `c` int(11) DEFAULT NULL ) PARTITION BY RANGE (`id`) (PARTITION `p0` VALUES LESS THAN (3), PARTITION `p1` VALUES LESS THAN (7), PARTITION `p2` VALUES LESS THAN (11));") + tk.MustExec("insert into t values (1,1,1,1),(2,2,2,2),(4,4,4,4),(5,5,5,5),(6,6,6,6),(8,8,8,8),(9,9,9,9);") + tk.MustExec("create index idxa on t (a);") + tk.MustExec("create index idxb on t (b);") + tk.MustExec("create index idxc on t (c);") + tk.MustExec("analyze table t partition p0 index idxa;") + tk.MustExec("analyze table t partition p1 index idxb;") + tk.MustExec("analyze table t partition p2 index idxc;") + tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1105 The version 2 would collect all statistics not only the selected indexes", + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p2")) + tk.MustExec("analyze table t partition p0;") + tk.MustQuery("show warnings").Check(testkit.Rows( + "Note 1105 Analyze use auto adjusted sample rate 1.000000 for table test.t's partition p0")) +} diff --git a/executor/autoidtest/BUILD.bazel b/executor/autoidtest/BUILD.bazel new file mode 100644 index 0000000000000..dd467855dc8d9 --- /dev/null +++ b/executor/autoidtest/BUILD.bazel @@ -0,0 +1,24 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "autoidtest_test", + srcs = [ + "autoid_test.go", + "main_test.go", + ], + flaky = True, + race = "on", + deps = [ + "//config", + "//ddl/testutil", + "//meta/autoid", + "//parser/mysql", + "//session", + "//sessionctx/variable", + "//testkit", + "//testkit/testutil", + "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//tikv", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/executor/autoidtest/autoid_test.go b/executor/autoidtest/autoid_test.go new file mode 100644 index 0000000000000..269f3966f8a01 --- /dev/null +++ b/executor/autoidtest/autoid_test.go @@ -0,0 +1,737 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package autoid_test + +import ( + "context" + "fmt" + "strconv" + "strings" + "testing" + + ddltestutil "github.com/pingcap/tidb/ddl/testutil" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testutil" + "github.com/stretchr/testify/require" +) + +// Test filter different kind of allocators. +// In special ddl type, for example: +// 1: ActionRenameTable : it will abandon all the old allocators. +// 2: ActionRebaseAutoID : it will drop row-id-type allocator. +// 3: ActionModifyTableAutoIdCache : it will drop row-id-type allocator. +// 3: ActionRebaseAutoRandomBase : it will drop auto-rand-type allocator. +func TestFilterDifferentAllocators(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("drop table if exists t1") + + for _, str := range []string{"", " AUTO_ID_CACHE 1"} { + tk.MustExec("create table t(a bigint auto_random(5) key, b int auto_increment unique)" + str) + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("1")) + allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) + orderedHandles := testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + require.Equal(t, int64(1), orderedHandles[0]) + tk.MustExec("delete from t") + + // Test rebase auto_increment. + tk.MustExec("alter table t auto_increment 3000000") + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("3000000")) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) + orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + require.Equal(t, int64(2), orderedHandles[0]) + tk.MustExec("delete from t") + + // Test rebase auto_random. + tk.MustExec("alter table t auto_random_base 3000000") + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("3000001")) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) + orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + require.Equal(t, int64(3000000), orderedHandles[0]) + tk.MustExec("delete from t") + + // Test rename table. + tk.MustExec("rename table t to t1") + tk.MustExec("insert into t1 values()") + res := tk.MustQuery("select b from t1") + strInt64, err := strconv.ParseInt(res.Rows()[0][0].(string), 10, 64) + require.NoError(t, err) + require.GreaterOrEqual(t, strInt64, int64(3000002)) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t1") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) + orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + require.Greater(t, orderedHandles[0], int64(3000001)) + + tk.MustExec("drop table t1") + } +} + +func TestAutoIncrementInsertMinMax(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + cases := []struct { + t string + s string + vals []int64 + expect [][]interface{} + }{ + {"tinyint", "signed", []int64{-128, 0, 127}, testkit.Rows("-128", "1", "2", "3", "127")}, + {"tinyint", "unsigned", []int64{0, 127, 255}, testkit.Rows("1", "2", "127", "128", "255")}, + {"smallint", "signed", []int64{-32768, 0, 32767}, testkit.Rows("-32768", "1", "2", "3", "32767")}, + {"smallint", "unsigned", []int64{0, 32767, 65535}, testkit.Rows("1", "2", "32767", "32768", "65535")}, + {"mediumint", "signed", []int64{-8388608, 0, 8388607}, testkit.Rows("-8388608", "1", "2", "3", "8388607")}, + {"mediumint", "unsigned", []int64{0, 8388607, 16777215}, testkit.Rows("1", "2", "8388607", "8388608", "16777215")}, + {"integer", "signed", []int64{-2147483648, 0, 2147483647}, testkit.Rows("-2147483648", "1", "2", "3", "2147483647")}, + {"integer", "unsigned", []int64{0, 2147483647, 4294967295}, testkit.Rows("1", "2", "2147483647", "2147483648", "4294967295")}, + {"bigint", "signed", []int64{-9223372036854775808, 0, 9223372036854775807}, testkit.Rows("-9223372036854775808", "1", "2", "3", "9223372036854775807")}, + {"bigint", "unsigned", []int64{0, 9223372036854775807}, testkit.Rows("1", "2", "9223372036854775807", "9223372036854775808")}, + } + + for _, option := range []string{"", "auto_id_cache 1", "auto_id_cache 100"} { + for idx, c := range cases { + sql := fmt.Sprintf("create table t%d (a %s %s key auto_increment) %s", idx, c.t, c.s, option) + tk.MustExec(sql) + + for _, val := range c.vals { + tk.MustExec(fmt.Sprintf("insert into t%d values (%d)", idx, val)) + tk.Exec(fmt.Sprintf("insert into t%d values ()", idx)) // ignore error + } + + tk.MustQuery(fmt.Sprintf("select * from t%d order by a", idx)).Check(c.expect) + + tk.MustExec(fmt.Sprintf("drop table t%d", idx)) + } + } + + tk.MustExec("create table t10 (a integer key auto_increment) auto_id_cache 1") + err := tk.ExecToErr("insert into t10 values (2147483648)") + require.Error(t, err) + err = tk.ExecToErr("insert into t10 values (-2147483649)") + require.Error(t, err) +} + +func TestInsertWithAutoidSchema(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 auto_increment, n int);`) + tk.MustExec(`create table t2(id int unsigned primary key auto_increment, n int);`) + tk.MustExec(`create table t3(id tinyint primary key auto_increment, n int);`) + tk.MustExec(`create table t4(id int primary key, n float auto_increment, key I_n(n));`) + tk.MustExec(`create table t5(id int primary key, n float unsigned auto_increment, key I_n(n));`) + tk.MustExec(`create table t6(id int primary key, n double auto_increment, key I_n(n));`) + tk.MustExec(`create table t7(id int primary key, n double unsigned auto_increment, key I_n(n));`) + // test for inserting multiple values + tk.MustExec(`create table t8(id int primary key auto_increment, n int);`) + + testInsertWithAutoidSchema(t, tk) +} + +func TestInsertWithAutoidSchemaCache(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 auto_increment, n int) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t2(id int unsigned primary key auto_increment, n int) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t3(id tinyint primary key auto_increment, n int) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t4(id int primary key, n float auto_increment, key I_n(n)) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t5(id int primary key, n float unsigned auto_increment, key I_n(n)) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t6(id int primary key, n double auto_increment, key I_n(n)) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t7(id int primary key, n double unsigned auto_increment, key I_n(n)) AUTO_ID_CACHE 1;`) + // test for inserting multiple values + tk.MustExec(`create table t8(id int primary key auto_increment, n int);`) + + testInsertWithAutoidSchema(t, tk) +} + +func testInsertWithAutoidSchema(t *testing.T, tk *testkit.TestKit) { + tests := []struct { + insert string + query string + result [][]interface{} + }{ + { + `insert into t1(id, n) values(1, 1)`, + `select * from t1 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t1(n) values(2)`, + `select * from t1 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t1(n) values(3)`, + `select * from t1 where id = 3`, + testkit.Rows(`3 3`), + }, + { + `insert into t1(id, n) values(-1, 4)`, + `select * from t1 where id = -1`, + testkit.Rows(`-1 4`), + }, + { + `insert into t1(n) values(5)`, + `select * from t1 where id = 4`, + testkit.Rows(`4 5`), + }, + { + `insert into t1(id, n) values('5', 6)`, + `select * from t1 where id = 5`, + testkit.Rows(`5 6`), + }, + { + `insert into t1(n) values(7)`, + `select * from t1 where id = 6`, + testkit.Rows(`6 7`), + }, + { + `insert into t1(id, n) values(7.4, 8)`, + `select * from t1 where id = 7`, + testkit.Rows(`7 8`), + }, + { + `insert into t1(id, n) values(7.5, 9)`, + `select * from t1 where id = 8`, + testkit.Rows(`8 9`), + }, + { + `insert into t1(n) values(9)`, + `select * from t1 where id = 9`, + testkit.Rows(`9 9`), + }, + // test last insert id + { + `insert into t1 values(3000, -1), (null, -2)`, + `select * from t1 where id = 3000`, + testkit.Rows(`3000 -1`), + }, + { + `;`, + `select * from t1 where id = 3001`, + testkit.Rows(`3001 -2`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`3001`), + }, + { + `insert into t2(id, n) values(1, 1)`, + `select * from t2 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t2(n) values(2)`, + `select * from t2 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t2(n) values(3)`, + `select * from t2 where id = 3`, + testkit.Rows(`3 3`), + }, + { + `insert into t3(id, n) values(1, 1)`, + `select * from t3 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t3(n) values(2)`, + `select * from t3 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t3(n) values(3)`, + `select * from t3 where id = 3`, + testkit.Rows(`3 3`), + }, + { + `insert into t3(id, n) values(-1, 4)`, + `select * from t3 where id = -1`, + testkit.Rows(`-1 4`), + }, + { + `insert into t3(n) values(5)`, + `select * from t3 where id = 4`, + testkit.Rows(`4 5`), + }, + { + `insert into t4(id, n) values(1, 1)`, + `select * from t4 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t4(id) values(2)`, + `select * from t4 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t4(id, n) values(3, -1)`, + `select * from t4 where id = 3`, + testkit.Rows(`3 -1`), + }, + { + `insert into t4(id) values(4)`, + `select * from t4 where id = 4`, + testkit.Rows(`4 3`), + }, + { + `insert into t4(id, n) values(5, 5.5)`, + `select * from t4 where id = 5`, + testkit.Rows(`5 5.5`), + }, + { + `insert into t4(id) values(6)`, + `select * from t4 where id = 6`, + testkit.Rows(`6 7`), + }, + { + `insert into t4(id, n) values(7, '7.7')`, + `select * from t4 where id = 7`, + testkit.Rows(`7 7.7`), + }, + { + `insert into t4(id) values(8)`, + `select * from t4 where id = 8`, + testkit.Rows(`8 9`), + }, + { + `insert into t4(id, n) values(9, 10.4)`, + `select * from t4 where id = 9`, + testkit.Rows(`9 10.4`), + }, + { + `insert into t4(id) values(10)`, + `select * from t4 where id = 10`, + testkit.Rows(`10 11`), + }, + { + `insert into t5(id, n) values(1, 1)`, + `select * from t5 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t5(id) values(2)`, + `select * from t5 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t5(id) values(3)`, + `select * from t5 where id = 3`, + testkit.Rows(`3 3`), + }, + { + `insert into t6(id, n) values(1, 1)`, + `select * from t6 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t6(id) values(2)`, + `select * from t6 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t6(id, n) values(3, -1)`, + `select * from t6 where id = 3`, + testkit.Rows(`3 -1`), + }, + { + `insert into t6(id) values(4)`, + `select * from t6 where id = 4`, + testkit.Rows(`4 3`), + }, + { + `insert into t6(id, n) values(5, 5.5)`, + `select * from t6 where id = 5`, + testkit.Rows(`5 5.5`), + }, + { + `insert into t6(id) values(6)`, + `select * from t6 where id = 6`, + testkit.Rows(`6 7`), + }, + { + `insert into t6(id, n) values(7, '7.7')`, + `select * from t4 where id = 7`, + testkit.Rows(`7 7.7`), + }, + { + `insert into t6(id) values(8)`, + `select * from t4 where id = 8`, + testkit.Rows(`8 9`), + }, + { + `insert into t6(id, n) values(9, 10.4)`, + `select * from t6 where id = 9`, + testkit.Rows(`9 10.4`), + }, + { + `insert into t6(id) values(10)`, + `select * from t6 where id = 10`, + testkit.Rows(`10 11`), + }, + { + `insert into t7(id, n) values(1, 1)`, + `select * from t7 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t7(id) values(2)`, + `select * from t7 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t7(id) values(3)`, + `select * from t7 where id = 3`, + testkit.Rows(`3 3`), + }, + + // the following is test for insert multiple values. + { + `insert into t8(n) values(1),(2)`, + `select * from t8 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `;`, + `select * from t8 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `;`, + `select last_insert_id();`, + testkit.Rows(`1`), + }, + // test user rebase and auto alloc mixture. + { + `insert into t8 values(null, 3),(-1, -1),(null,4),(null, 5)`, + `select * from t8 where id = 3`, + testkit.Rows(`3 3`), + }, + // -1 won't rebase allocator here cause -1 < base. + { + `;`, + `select * from t8 where id = -1`, + testkit.Rows(`-1 -1`), + }, + { + `;`, + `select * from t8 where id = 4`, + testkit.Rows(`4 4`), + }, + { + `;`, + `select * from t8 where id = 5`, + testkit.Rows(`5 5`), + }, + { + `;`, + `select last_insert_id();`, + testkit.Rows(`3`), + }, + { + `insert into t8 values(null, 6),(10, 7),(null, 8)`, + `select * from t8 where id = 6`, + testkit.Rows(`6 6`), + }, + // 10 will rebase allocator here. + { + `;`, + `select * from t8 where id = 10`, + testkit.Rows(`10 7`), + }, + { + `;`, + `select * from t8 where id = 11`, + testkit.Rows(`11 8`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`6`), + }, + // fix bug for last_insert_id should be first allocated id in insert rows (skip the rebase id). + { + `insert into t8 values(100, 9),(null,10),(null,11)`, + `select * from t8 where id = 100`, + testkit.Rows(`100 9`), + }, + { + `;`, + `select * from t8 where id = 101`, + testkit.Rows(`101 10`), + }, + { + `;`, + `select * from t8 where id = 102`, + testkit.Rows(`102 11`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`101`), + }, + // test with sql_mode: NO_AUTO_VALUE_ON_ZERO. + { + `;`, + `select @@sql_mode`, + testkit.Rows(`ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`), + }, + { + `;`, + "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION,NO_AUTO_VALUE_ON_ZERO`", + nil, + }, + { + `insert into t8 values (0, 12), (null, 13)`, + `select * from t8 where id = 0`, + testkit.Rows(`0 12`), + }, + { + `;`, + `select * from t8 where id = 103`, + testkit.Rows(`103 13`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`103`), + }, + // test without sql_mode: NO_AUTO_VALUE_ON_ZERO. + { + `;`, + "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`", + nil, + }, + // value 0 will be substitute by autoid. + { + `insert into t8 values (0, 14), (null, 15)`, + `select * from t8 where id = 104`, + testkit.Rows(`104 14`), + }, + { + `;`, + `select * from t8 where id = 105`, + testkit.Rows(`105 15`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`104`), + }, + // last test : auto increment allocation can find in retryInfo. + { + `retry : insert into t8 values (null, 16), (null, 17)`, + `select * from t8 where id = 1000`, + testkit.Rows(`1000 16`), + }, + { + `;`, + `select * from t8 where id = 1001`, + testkit.Rows(`1001 17`), + }, + { + `;`, + `select last_insert_id()`, + // this insert doesn't has the last_insert_id, should be same as the last insert case. + testkit.Rows(`104`), + }, + } + + for _, tt := range tests { + if strings.HasPrefix(tt.insert, "retry : ") { + // it's the last retry insert case, change the sessionVars. + retryInfo := &variable.RetryInfo{Retrying: true} + retryInfo.AddAutoIncrementID(1000) + retryInfo.AddAutoIncrementID(1001) + tk.Session().GetSessionVars().RetryInfo = retryInfo + tk.MustExec(tt.insert[8:]) + tk.Session().GetSessionVars().RetryInfo = &variable.RetryInfo{} + } else { + tk.MustExec(tt.insert) + } + if tt.query == "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION,NO_AUTO_VALUE_ON_ZERO`" || + tt.query == "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`" { + tk.MustExec(tt.query) + } else { + tk.MustQuery(tt.query).Check(tt.result) + } + } +} + +// TestAutoIDIncrementAndOffset There is a potential issue in MySQL: when the value of auto_increment_offset is greater +// than that of auto_increment_increment, the value of auto_increment_offset is ignored +// (https://dev.mysql.com/doc/refman/8.0/en/replication-options-master.html#sysvar_auto_increment_increment), +// This issue is a flaw of the implementation of MySQL and it doesn't exist in TiDB. +func TestAutoIDIncrementAndOffset(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + // Test for offset is larger than increment. + tk.Session().GetSessionVars().AutoIncrementIncrement = 5 + tk.Session().GetSessionVars().AutoIncrementOffset = 10 + + for _, str := range []string{"", " AUTO_ID_CACHE 1"} { + tk.MustExec(`create table io (a int key auto_increment)` + str) + tk.MustExec(`insert into io values (null),(null),(null)`) + tk.MustQuery(`select * from io`).Check(testkit.Rows("10", "15", "20")) + tk.MustExec(`drop table io`) + } + + // Test handle is PK. + for _, str := range []string{"", " AUTO_ID_CACHE 1"} { + tk.MustExec(`create table io (a int key auto_increment)` + str) + tk.Session().GetSessionVars().AutoIncrementOffset = 10 + tk.Session().GetSessionVars().AutoIncrementIncrement = 2 + tk.MustExec(`insert into io values (),(),()`) + tk.MustQuery(`select * from io`).Check(testkit.Rows("10", "12", "14")) + tk.MustExec(`delete from io`) + + // Test reset the increment. + tk.Session().GetSessionVars().AutoIncrementIncrement = 5 + tk.MustExec(`insert into io values (),(),()`) + tk.MustQuery(`select * from io`).Check(testkit.Rows("15", "20", "25")) + tk.MustExec(`delete from io`) + + tk.Session().GetSessionVars().AutoIncrementIncrement = 10 + tk.MustExec(`insert into io values (),(),()`) + tk.MustQuery(`select * from io`).Check(testkit.Rows("30", "40", "50")) + tk.MustExec(`delete from io`) + + tk.Session().GetSessionVars().AutoIncrementIncrement = 5 + tk.MustExec(`insert into io values (),(),()`) + tk.MustQuery(`select * from io`).Check(testkit.Rows("55", "60", "65")) + tk.MustExec(`drop table io`) + } + + // Test handle is not PK. + for _, str := range []string{"", " AUTO_ID_CACHE 1"} { + tk.Session().GetSessionVars().AutoIncrementIncrement = 2 + tk.Session().GetSessionVars().AutoIncrementOffset = 10 + tk.MustExec(`create table io (a int, b int auto_increment, key(b))` + str) + tk.MustExec(`insert into io(b) values (null),(null),(null)`) + // AutoID allocation will take increment and offset into consideration. + tk.MustQuery(`select b from io`).Check(testkit.Rows("10", "12", "14")) + // HandleID allocation will ignore the increment and offset. + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("15", "16", "17")) + tk.MustExec(`delete from io`) + + tk.Session().GetSessionVars().AutoIncrementIncrement = 10 + tk.MustExec(`insert into io(b) values (null),(null),(null)`) + tk.MustQuery(`select b from io`).Check(testkit.Rows("20", "30", "40")) + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("41", "42", "43")) + + // Test invalid value. + tk.Session().GetSessionVars().AutoIncrementIncrement = -1 + tk.Session().GetSessionVars().AutoIncrementOffset = -2 + tk.MustGetErrMsg(`insert into io(b) values (null),(null),(null)`, + "[autoid:8060]Invalid auto_increment settings: auto_increment_increment: -1, auto_increment_offset: -2, both of them must be in range [1..65535]") + tk.MustExec(`delete from io`) + + tk.Session().GetSessionVars().AutoIncrementIncrement = 65536 + tk.Session().GetSessionVars().AutoIncrementOffset = 65536 + tk.MustGetErrMsg(`insert into io(b) values (null),(null),(null)`, + "[autoid:8060]Invalid auto_increment settings: auto_increment_increment: 65536, auto_increment_offset: 65536, both of them must be in range [1..65535]") + + tk.MustExec(`drop table io`) + } +} + +func TestRenameTableForAutoIncrement(t *testing.T) { + store, _ := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("USE test;") + tk.MustExec("drop table if exists t1, t2, t3;") + tk.MustExec("create table t1 (id int key auto_increment);") + tk.MustExec("insert into t1 values ()") + tk.MustExec("rename table t1 to t11") + tk.MustExec("insert into t11 values ()") + // TODO(tiancaiamao): fix bug and uncomment here, rename table should not discard the cached AUTO_ID. + // tk.MustQuery("select * from t11").Check(testkit.Rows("1", "2")) + + // auto_id_cache 1 use another implementation and do not have such bug. + tk.MustExec("create table t2 (id int key auto_increment) auto_id_cache 1;") + tk.MustExec("insert into t2 values ()") + tk.MustExec("rename table t2 to t22") + tk.MustExec("insert into t22 values ()") + tk.MustQuery("select * from t22").Check(testkit.Rows("1", "2")) + + tk.MustExec("create table t3 (id int key auto_increment) auto_id_cache 100;") + tk.MustExec("insert into t3 values ()") + tk.MustExec("rename table t3 to t33") + tk.MustExec("insert into t33 values ()") + // TODO(tiancaiamao): fix bug and uncomment here, rename table should not discard the cached AUTO_ID. + // tk.MustQuery("select * from t33").Check(testkit.Rows("1", "2")) +} + +func TestAlterTableAutoIDCache(t *testing.T) { + store, _ := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("USE test;") + tk.MustExec("drop table if exists t_473;") + tk.MustExec("create table t_473 (id int key auto_increment)") + tk.MustExec("insert into t_473 values ()") + tk.MustQuery("select * from t_473").Check(testkit.Rows("1")) + rs, err := tk.Exec("show table t_473 next_row_id") + require.NoError(t, err) + rows, err1 := session.ResultSetToStringSlice(context.Background(), tk.Session(), rs) + require.NoError(t, err1) + // "test t_473 id 1013608 AUTO_INCREMENT" + val, err2 := strconv.ParseUint(rows[0][3], 10, 64) + require.NoError(t, err2) + + tk.MustExec("alter table t_473 auto_id_cache = 100") + tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val))) + tk.MustExec("insert into t_473 values ()") + tk.MustQuery("select * from t_473").Check(testkit.Rows("1", fmt.Sprintf("%d", val))) + tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+100))) + + // Note that auto_id_cache=1 use a different implementation. + tk.MustExec("alter table t_473 auto_id_cache = 1") + tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+100))) + tk.MustExec("insert into t_473 values ()") + tk.MustQuery("select * from t_473").Check(testkit.Rows("1", fmt.Sprintf("%d", val), fmt.Sprintf("%d", val+100))) + tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+101))) + + // alter table from auto_id_cache=1 to default will discard the IDs cached by the autoid service. + // This is because they are two component and TiDB can't tell the autoid service to "save position and exit". + tk.MustExec("alter table t_473 auto_id_cache = 20000") + tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+4100))) + + tk.MustExec("insert into t_473 values ()") + tk.MustQuery("select * from t_473").Check(testkit.Rows("1", + fmt.Sprintf("%d", val), + fmt.Sprintf("%d", val+100), + fmt.Sprintf("%d", val+4100))) + tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+24100))) +} diff --git a/executor/autoidtest/main_test.go b/executor/autoidtest/main_test.go new file mode 100644 index 0000000000000..f87db4afe1371 --- /dev/null +++ b/executor/autoidtest/main_test.go @@ -0,0 +1,44 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package autoid_test + +import ( + "testing" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/meta/autoid" + "github.com/tikv/client-go/v2/tikv" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + autoid.SetStep(5000) + config.UpdateGlobal(func(conf *config.Config) { + conf.Log.SlowThreshold = 30000 // 30s + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + conf.Experimental.AllowsExpressionIndex = true + }) + tikv.EnableFailpoints() + + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), + goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"), + } + goleak.VerifyTestMain(m, opts...) +} diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 99bb6ceec9103..7ff162239ebb9 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -916,18 +916,20 @@ func prepare4HashJoin(testCase *hashJoinTestCase, innerExec, outerExec Executor) } e := &HashJoinExec{ baseExecutor: newBaseExecutor(testCase.ctx, joinSchema, 5, innerExec, outerExec), - probeSideTupleFetcher: probeSideTupleFetcher{ + hashJoinCtx: &hashJoinCtx{ + joinType: testCase.joinType, // 0 for InnerJoin, 1 for LeftOutersJoin, 2 for RightOuterJoin + isOuterJoin: false, + useOuterToBuild: testCase.useOuterToBuild, + }, + probeSideTupleFetcher: &probeSideTupleFetcher{ probeSideExec: outerExec, }, probeWorkers: make([]probeWorker, testCase.concurrency), concurrency: uint(testCase.concurrency), - joinType: testCase.joinType, // 0 for InnerJoin, 1 for LeftOutersJoin, 2 for RightOuterJoin - isOuterJoin: false, buildKeys: joinKeys, probeKeys: probeKeys, buildSideExec: innerExec, buildSideEstCount: float64(testCase.rows), - useOuterToBuild: testCase.useOuterToBuild, } childrenUsedSchema := markChildrenUsedCols(e.Schema(), e.children[0].Schema(), e.children[1].Schema()) diff --git a/executor/builder.go b/executor/builder.go index dfd44d549eff1..ea718aa9df6dc 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1412,11 +1412,14 @@ func (b *executorBuilder) buildHashJoin(v *plannercore.PhysicalHashJoin) Executo } e := &HashJoinExec{ - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID(), leftExec, rightExec), - concurrency: v.Concurrency, - joinType: v.JoinType, - isOuterJoin: v.JoinType.IsOuterJoin(), - useOuterToBuild: v.UseOuterToBuild, + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID(), leftExec, rightExec), + probeSideTupleFetcher: &probeSideTupleFetcher{}, + hashJoinCtx: &hashJoinCtx{ + isOuterJoin: v.JoinType.IsOuterJoin(), + useOuterToBuild: v.UseOuterToBuild, + joinType: v.JoinType, + }, + concurrency: v.Concurrency, } defaultValues := v.DefaultValues lhsTypes, rhsTypes := retTypes(leftExec), retTypes(rightExec) diff --git a/executor/compiler.go b/executor/compiler.go index 241b15874e1e2..4ddb6208a445c 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -21,7 +21,9 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" @@ -154,9 +156,42 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (_ *ExecS } } } + if c.Ctx.GetSessionVars().EnablePlanReplayerCapture { + checkPlanReplayerCaptureTask(c.Ctx, stmtNode) + } + return stmt, nil } +func checkPlanReplayerCaptureTask(sctx sessionctx.Context, stmtNode ast.StmtNode) { + tasks := domain.GetDomain(sctx).GetPlanReplayerHandle().GetTasks() + _, sqlDigest := sctx.GetSessionVars().StmtCtx.SQLDigest() + _, planDigest := getPlanDigest(sctx.GetSessionVars().StmtCtx) + for _, task := range tasks { + if task.SQLDigest == sqlDigest.String() && task.PlanDigest == planDigest.String() { + sendPlanReplayerDumpTask(sqlDigest.String(), planDigest.String(), sctx, stmtNode) + } + } +} + +func sendPlanReplayerDumpTask(sqlDigest, planDigest string, sctx sessionctx.Context, stmtNode ast.StmtNode) { + stmtCtx := sctx.GetSessionVars().StmtCtx + handle := sctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) + dumpTask := &domain.PlanReplayerDumpTask{ + PlanReplayerTaskKey: domain.PlanReplayerTaskKey{ + SQLDigest: sqlDigest, + PlanDigest: planDigest, + }, + EncodePlan: GetEncodedPlan, + TblStats: stmtCtx.TableStats, + SessionBindings: handle.GetAllBindRecord(), + SessionVars: sctx.GetSessionVars(), + ExecStmts: []ast.StmtNode{stmtNode}, + Analyze: false, + } + domain.GetDomain(sctx).GetPlanReplayerHandle().SendTask(dumpTask) +} + // needLowerPriority checks whether it's needed to lower the execution priority // of a query. // If the estimated output row count of any operator in the physical plan tree diff --git a/executor/ddl_test.go b/executor/ddl_test.go index d4d0d59d4ab5b..54a67cbbf9cb7 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -1131,65 +1131,6 @@ func TestAutoRandomClusteredPrimaryKey(t *testing.T) { tk.MustQuery("select a from t;").Check(testkit.Rows("1")) } -// Test filter different kind of allocators. -// In special ddl type, for example: -// 1: ActionRenameTable : it will abandon all the old allocators. -// 2: ActionRebaseAutoID : it will drop row-id-type allocator. -// 3: ActionModifyTableAutoIdCache : it will drop row-id-type allocator. -// 3: ActionRebaseAutoRandomBase : it will drop auto-rand-type allocator. -func TestFilterDifferentAllocators(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("drop table if exists t1") - - tk.MustExec("create table t(a bigint auto_random(5) key, b int auto_increment unique)") - tk.MustExec("insert into t values()") - tk.MustQuery("select b from t").Check(testkit.Rows("1")) - allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") - require.NoError(t, err) - require.Equal(t, 1, len(allHandles)) - orderedHandles := testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) - require.Equal(t, int64(1), orderedHandles[0]) - tk.MustExec("delete from t") - - // Test rebase auto_increment. - tk.MustExec("alter table t auto_increment 3000000") - tk.MustExec("insert into t values()") - tk.MustQuery("select b from t").Check(testkit.Rows("3000000")) - allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") - require.NoError(t, err) - require.Equal(t, 1, len(allHandles)) - orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) - require.Equal(t, int64(2), orderedHandles[0]) - tk.MustExec("delete from t") - - // Test rebase auto_random. - tk.MustExec("alter table t auto_random_base 3000000") - tk.MustExec("insert into t values()") - tk.MustQuery("select b from t").Check(testkit.Rows("3000001")) - allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") - require.NoError(t, err) - require.Equal(t, 1, len(allHandles)) - orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) - require.Equal(t, int64(3000000), orderedHandles[0]) - tk.MustExec("delete from t") - - // Test rename table. - tk.MustExec("rename table t to t1") - tk.MustExec("insert into t1 values()") - res := tk.MustQuery("select b from t1") - strInt64, err := strconv.ParseInt(res.Rows()[0][0].(string), 10, 64) - require.NoError(t, err) - require.Greater(t, strInt64, int64(3000002)) - allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t1") - require.NoError(t, err) - require.Equal(t, 1, len(allHandles)) - orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) - require.Greater(t, orderedHandles[0], int64(3000001)) -} - func TestMaxHandleAddIndex(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/executor/executor_test.go b/executor/executor_test.go index 641f56817cd60..ed4551f809ee4 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -5947,6 +5947,8 @@ func TestSummaryFailedUpdate(t *testing.T) { 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)) tk.MustExec("set @@tidb_mem_quota_query=1") diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index c55c51031fa24..ed6eed4fb4607 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -151,7 +151,7 @@ func TestColumnsTables(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (bit bit(10) DEFAULT b'100')") tk.MustQuery("SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_NAME = 't'").Check(testkit.Rows( - "def test t bit 1 b'100' YES bit 10 0 bit(10) unsigned select,insert,update,references ")) + "def test t bit 1 b'100' YES bit 10 0 bit(10) select,insert,update,references ")) tk.MustExec("drop table if exists t") tk.MustExec("set time_zone='+08:00'") @@ -165,6 +165,11 @@ func TestColumnsTables(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (a bit DEFAULT (rand()))") tk.MustQuery("select column_default from information_schema.columns where TABLE_NAME='t' and TABLE_SCHEMA='test';").Check(testkit.Rows("rand()")) + + tk.MustExec("drop table if exists t") + tk.MustExec("CREATE TABLE t (`COL3` bit(1) NOT NULL,b year) ;") + tk.MustQuery("select column_type from information_schema.columns where TABLE_SCHEMA = 'test' and TABLE_NAME = 't';"). + Check(testkit.Rows("bit(1)", "year(4)")) } func TestEngines(t *testing.T) { diff --git a/executor/insert_test.go b/executor/insert_test.go index 53d0c62cc9776..f3db8c6aca40f 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -589,435 +589,6 @@ func TestAllowInvalidDates(t *testing.T) { runWithMode("ALLOW_INVALID_DATES") } -func TestInsertWithAutoidSchema(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 auto_increment, n int);`) - tk.MustExec(`create table t2(id int unsigned primary key auto_increment, n int);`) - tk.MustExec(`create table t3(id tinyint primary key auto_increment, n int);`) - tk.MustExec(`create table t4(id int primary key, n float auto_increment, key I_n(n));`) - tk.MustExec(`create table t5(id int primary key, n float unsigned auto_increment, key I_n(n));`) - tk.MustExec(`create table t6(id int primary key, n double auto_increment, key I_n(n));`) - tk.MustExec(`create table t7(id int primary key, n double unsigned auto_increment, key I_n(n));`) - // test for inserting multiple values - tk.MustExec(`create table t8(id int primary key auto_increment, n int);`) - - tests := []struct { - insert string - query string - result [][]interface{} - }{ - { - `insert into t1(id, n) values(1, 1)`, - `select * from t1 where id = 1`, - testkit.Rows(`1 1`), - }, - { - `insert into t1(n) values(2)`, - `select * from t1 where id = 2`, - testkit.Rows(`2 2`), - }, - { - `insert into t1(n) values(3)`, - `select * from t1 where id = 3`, - testkit.Rows(`3 3`), - }, - { - `insert into t1(id, n) values(-1, 4)`, - `select * from t1 where id = -1`, - testkit.Rows(`-1 4`), - }, - { - `insert into t1(n) values(5)`, - `select * from t1 where id = 4`, - testkit.Rows(`4 5`), - }, - { - `insert into t1(id, n) values('5', 6)`, - `select * from t1 where id = 5`, - testkit.Rows(`5 6`), - }, - { - `insert into t1(n) values(7)`, - `select * from t1 where id = 6`, - testkit.Rows(`6 7`), - }, - { - `insert into t1(id, n) values(7.4, 8)`, - `select * from t1 where id = 7`, - testkit.Rows(`7 8`), - }, - { - `insert into t1(id, n) values(7.5, 9)`, - `select * from t1 where id = 8`, - testkit.Rows(`8 9`), - }, - { - `insert into t1(n) values(9)`, - `select * from t1 where id = 9`, - testkit.Rows(`9 9`), - }, - // test last insert id - { - `insert into t1 values(3000, -1), (null, -2)`, - `select * from t1 where id = 3000`, - testkit.Rows(`3000 -1`), - }, - { - `;`, - `select * from t1 where id = 3001`, - testkit.Rows(`3001 -2`), - }, - { - `;`, - `select last_insert_id()`, - testkit.Rows(`3001`), - }, - { - `insert into t2(id, n) values(1, 1)`, - `select * from t2 where id = 1`, - testkit.Rows(`1 1`), - }, - { - `insert into t2(n) values(2)`, - `select * from t2 where id = 2`, - testkit.Rows(`2 2`), - }, - { - `insert into t2(n) values(3)`, - `select * from t2 where id = 3`, - testkit.Rows(`3 3`), - }, - { - `insert into t3(id, n) values(1, 1)`, - `select * from t3 where id = 1`, - testkit.Rows(`1 1`), - }, - { - `insert into t3(n) values(2)`, - `select * from t3 where id = 2`, - testkit.Rows(`2 2`), - }, - { - `insert into t3(n) values(3)`, - `select * from t3 where id = 3`, - testkit.Rows(`3 3`), - }, - { - `insert into t3(id, n) values(-1, 4)`, - `select * from t3 where id = -1`, - testkit.Rows(`-1 4`), - }, - { - `insert into t3(n) values(5)`, - `select * from t3 where id = 4`, - testkit.Rows(`4 5`), - }, - { - `insert into t4(id, n) values(1, 1)`, - `select * from t4 where id = 1`, - testkit.Rows(`1 1`), - }, - { - `insert into t4(id) values(2)`, - `select * from t4 where id = 2`, - testkit.Rows(`2 2`), - }, - { - `insert into t4(id, n) values(3, -1)`, - `select * from t4 where id = 3`, - testkit.Rows(`3 -1`), - }, - { - `insert into t4(id) values(4)`, - `select * from t4 where id = 4`, - testkit.Rows(`4 3`), - }, - { - `insert into t4(id, n) values(5, 5.5)`, - `select * from t4 where id = 5`, - testkit.Rows(`5 5.5`), - }, - { - `insert into t4(id) values(6)`, - `select * from t4 where id = 6`, - testkit.Rows(`6 7`), - }, - { - `insert into t4(id, n) values(7, '7.7')`, - `select * from t4 where id = 7`, - testkit.Rows(`7 7.7`), - }, - { - `insert into t4(id) values(8)`, - `select * from t4 where id = 8`, - testkit.Rows(`8 9`), - }, - { - `insert into t4(id, n) values(9, 10.4)`, - `select * from t4 where id = 9`, - testkit.Rows(`9 10.4`), - }, - { - `insert into t4(id) values(10)`, - `select * from t4 where id = 10`, - testkit.Rows(`10 11`), - }, - { - `insert into t5(id, n) values(1, 1)`, - `select * from t5 where id = 1`, - testkit.Rows(`1 1`), - }, - { - `insert into t5(id) values(2)`, - `select * from t5 where id = 2`, - testkit.Rows(`2 2`), - }, - { - `insert into t5(id) values(3)`, - `select * from t5 where id = 3`, - testkit.Rows(`3 3`), - }, - { - `insert into t6(id, n) values(1, 1)`, - `select * from t6 where id = 1`, - testkit.Rows(`1 1`), - }, - { - `insert into t6(id) values(2)`, - `select * from t6 where id = 2`, - testkit.Rows(`2 2`), - }, - { - `insert into t6(id, n) values(3, -1)`, - `select * from t6 where id = 3`, - testkit.Rows(`3 -1`), - }, - { - `insert into t6(id) values(4)`, - `select * from t6 where id = 4`, - testkit.Rows(`4 3`), - }, - { - `insert into t6(id, n) values(5, 5.5)`, - `select * from t6 where id = 5`, - testkit.Rows(`5 5.5`), - }, - { - `insert into t6(id) values(6)`, - `select * from t6 where id = 6`, - testkit.Rows(`6 7`), - }, - { - `insert into t6(id, n) values(7, '7.7')`, - `select * from t4 where id = 7`, - testkit.Rows(`7 7.7`), - }, - { - `insert into t6(id) values(8)`, - `select * from t4 where id = 8`, - testkit.Rows(`8 9`), - }, - { - `insert into t6(id, n) values(9, 10.4)`, - `select * from t6 where id = 9`, - testkit.Rows(`9 10.4`), - }, - { - `insert into t6(id) values(10)`, - `select * from t6 where id = 10`, - testkit.Rows(`10 11`), - }, - { - `insert into t7(id, n) values(1, 1)`, - `select * from t7 where id = 1`, - testkit.Rows(`1 1`), - }, - { - `insert into t7(id) values(2)`, - `select * from t7 where id = 2`, - testkit.Rows(`2 2`), - }, - { - `insert into t7(id) values(3)`, - `select * from t7 where id = 3`, - testkit.Rows(`3 3`), - }, - - // the following is test for insert multiple values. - { - `insert into t8(n) values(1),(2)`, - `select * from t8 where id = 1`, - testkit.Rows(`1 1`), - }, - { - `;`, - `select * from t8 where id = 2`, - testkit.Rows(`2 2`), - }, - { - `;`, - `select last_insert_id();`, - testkit.Rows(`1`), - }, - // test user rebase and auto alloc mixture. - { - `insert into t8 values(null, 3),(-1, -1),(null,4),(null, 5)`, - `select * from t8 where id = 3`, - testkit.Rows(`3 3`), - }, - // -1 won't rebase allocator here cause -1 < base. - { - `;`, - `select * from t8 where id = -1`, - testkit.Rows(`-1 -1`), - }, - { - `;`, - `select * from t8 where id = 4`, - testkit.Rows(`4 4`), - }, - { - `;`, - `select * from t8 where id = 5`, - testkit.Rows(`5 5`), - }, - { - `;`, - `select last_insert_id();`, - testkit.Rows(`3`), - }, - { - `insert into t8 values(null, 6),(10, 7),(null, 8)`, - `select * from t8 where id = 6`, - testkit.Rows(`6 6`), - }, - // 10 will rebase allocator here. - { - `;`, - `select * from t8 where id = 10`, - testkit.Rows(`10 7`), - }, - { - `;`, - `select * from t8 where id = 11`, - testkit.Rows(`11 8`), - }, - { - `;`, - `select last_insert_id()`, - testkit.Rows(`6`), - }, - // fix bug for last_insert_id should be first allocated id in insert rows (skip the rebase id). - { - `insert into t8 values(100, 9),(null,10),(null,11)`, - `select * from t8 where id = 100`, - testkit.Rows(`100 9`), - }, - { - `;`, - `select * from t8 where id = 101`, - testkit.Rows(`101 10`), - }, - { - `;`, - `select * from t8 where id = 102`, - testkit.Rows(`102 11`), - }, - { - `;`, - `select last_insert_id()`, - testkit.Rows(`101`), - }, - // test with sql_mode: NO_AUTO_VALUE_ON_ZERO. - { - `;`, - `select @@sql_mode`, - testkit.Rows(`ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`), - }, - { - `;`, - "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION,NO_AUTO_VALUE_ON_ZERO`", - nil, - }, - { - `insert into t8 values (0, 12), (null, 13)`, - `select * from t8 where id = 0`, - testkit.Rows(`0 12`), - }, - { - `;`, - `select * from t8 where id = 103`, - testkit.Rows(`103 13`), - }, - { - `;`, - `select last_insert_id()`, - testkit.Rows(`103`), - }, - // test without sql_mode: NO_AUTO_VALUE_ON_ZERO. - { - `;`, - "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`", - nil, - }, - // value 0 will be substitute by autoid. - { - `insert into t8 values (0, 14), (null, 15)`, - `select * from t8 where id = 104`, - testkit.Rows(`104 14`), - }, - { - `;`, - `select * from t8 where id = 105`, - testkit.Rows(`105 15`), - }, - { - `;`, - `select last_insert_id()`, - testkit.Rows(`104`), - }, - // last test : auto increment allocation can find in retryInfo. - { - `retry : insert into t8 values (null, 16), (null, 17)`, - `select * from t8 where id = 1000`, - testkit.Rows(`1000 16`), - }, - { - `;`, - `select * from t8 where id = 1001`, - testkit.Rows(`1001 17`), - }, - { - `;`, - `select last_insert_id()`, - // this insert doesn't has the last_insert_id, should be same as the last insert case. - testkit.Rows(`104`), - }, - } - - for _, tt := range tests { - if strings.HasPrefix(tt.insert, "retry : ") { - // it's the last retry insert case, change the sessionVars. - retryInfo := &variable.RetryInfo{Retrying: true} - retryInfo.AddAutoIncrementID(1000) - retryInfo.AddAutoIncrementID(1001) - tk.Session().GetSessionVars().RetryInfo = retryInfo - tk.MustExec(tt.insert[8:]) - tk.Session().GetSessionVars().RetryInfo = &variable.RetryInfo{} - } else { - tk.MustExec(tt.insert) - } - if tt.query == "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION,NO_AUTO_VALUE_ON_ZERO`" || - tt.query == "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`" { - tk.MustExec(tt.query) - } else { - tk.MustQuery(tt.query).Check(tt.result) - } - } -} - func TestPartitionInsertOnDuplicate(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -1139,75 +710,6 @@ func TestInsertFloatOverflow(t *testing.T) { tk.MustExec("drop table if exists t,t1") } -// TestAutoIDIncrementAndOffset There is a potential issue in MySQL: when the value of auto_increment_offset is greater -// than that of auto_increment_increment, the value of auto_increment_offset is ignored -// (https://dev.mysql.com/doc/refman/8.0/en/replication-options-master.html#sysvar_auto_increment_increment), -// This issue is a flaw of the implementation of MySQL and it doesn't exist in TiDB. -func TestAutoIDIncrementAndOffset(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec(`use test`) - // Test for offset is larger than increment. - tk.Session().GetSessionVars().AutoIncrementIncrement = 5 - tk.Session().GetSessionVars().AutoIncrementOffset = 10 - tk.MustExec(`create table io (a int key auto_increment)`) - tk.MustExec(`insert into io values (null),(null),(null)`) - tk.MustQuery(`select * from io`).Check(testkit.Rows("10", "15", "20")) - tk.MustExec(`drop table io`) - - // Test handle is PK. - tk.MustExec(`create table io (a int key auto_increment)`) - tk.Session().GetSessionVars().AutoIncrementOffset = 10 - tk.Session().GetSessionVars().AutoIncrementIncrement = 2 - tk.MustExec(`insert into io values (),(),()`) - tk.MustQuery(`select * from io`).Check(testkit.Rows("10", "12", "14")) - tk.MustExec(`delete from io`) - - // Test reset the increment. - tk.Session().GetSessionVars().AutoIncrementIncrement = 5 - tk.MustExec(`insert into io values (),(),()`) - tk.MustQuery(`select * from io`).Check(testkit.Rows("15", "20", "25")) - tk.MustExec(`delete from io`) - - tk.Session().GetSessionVars().AutoIncrementIncrement = 10 - tk.MustExec(`insert into io values (),(),()`) - tk.MustQuery(`select * from io`).Check(testkit.Rows("30", "40", "50")) - tk.MustExec(`delete from io`) - - tk.Session().GetSessionVars().AutoIncrementIncrement = 5 - tk.MustExec(`insert into io values (),(),()`) - tk.MustQuery(`select * from io`).Check(testkit.Rows("55", "60", "65")) - tk.MustExec(`drop table io`) - - // Test handle is not PK. - tk.Session().GetSessionVars().AutoIncrementIncrement = 2 - tk.Session().GetSessionVars().AutoIncrementOffset = 10 - tk.MustExec(`create table io (a int, b int auto_increment, key(b))`) - tk.MustExec(`insert into io(b) values (null),(null),(null)`) - // AutoID allocation will take increment and offset into consideration. - tk.MustQuery(`select b from io`).Check(testkit.Rows("10", "12", "14")) - // HandleID allocation will ignore the increment and offset. - tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("15", "16", "17")) - tk.MustExec(`delete from io`) - - tk.Session().GetSessionVars().AutoIncrementIncrement = 10 - tk.MustExec(`insert into io(b) values (null),(null),(null)`) - tk.MustQuery(`select b from io`).Check(testkit.Rows("20", "30", "40")) - tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("41", "42", "43")) - - // Test invalid value. - tk.Session().GetSessionVars().AutoIncrementIncrement = -1 - tk.Session().GetSessionVars().AutoIncrementOffset = -2 - tk.MustGetErrMsg(`insert into io(b) values (null),(null),(null)`, - "[autoid:8060]Invalid auto_increment settings: auto_increment_increment: -1, auto_increment_offset: -2, both of them must be in range [1..65535]") - tk.MustExec(`delete from io`) - - tk.Session().GetSessionVars().AutoIncrementIncrement = 65536 - tk.Session().GetSessionVars().AutoIncrementOffset = 65536 - tk.MustGetErrMsg(`insert into io(b) values (null),(null),(null)`, - "[autoid:8060]Invalid auto_increment settings: auto_increment_increment: 65536, auto_increment_offset: 65536, both of them must be in range [1..65535]") -} - // Fix https://github.com/pingcap/tidb/issues/32601. func TestTextTooLongError(t *testing.T) { store := testkit.CreateMockStore(t) diff --git a/executor/join.go b/executor/join.go index 95ecee42c02d4..072f0106d50ef 100644 --- a/executor/join.go +++ b/executor/join.go @@ -46,8 +46,22 @@ var ( _ Executor = &NestedLoopApplyExec{} ) +type hashJoinCtx struct { + joinResultCh chan *hashjoinWorkerResult + // closeCh add a lock for closing executor. + closeCh chan struct{} + finished atomic.Bool + useOuterToBuild bool + isOuterJoin bool + buildFinished chan error + rowContainer *hashRowContainer + joinType plannercore.JoinType +} + // probeSideTupleFetcher reads tuples from probeSideExec and send them to probeWorkers. type probeSideTupleFetcher struct { + *hashJoinCtx + probeSideExec Executor probeChkResourceCh chan *probeChkResource probeResultChs []chan *chunk.Chunk @@ -73,7 +87,8 @@ type probeWorker struct { type HashJoinExec struct { baseExecutor - probeSideTupleFetcher + probeSideTupleFetcher *probeSideTupleFetcher + *hashJoinCtx probeWorkers []probeWorker buildSideExec Executor buildSideEstCount float64 @@ -87,29 +102,19 @@ type HashJoinExec struct { buildTypes []*types.FieldType // concurrency is the number of partition, build and join workers. - concurrency uint - rowContainer *hashRowContainer - buildFinished chan error + concurrency uint - // closeCh add a lock for closing executor. - closeCh chan struct{} - worker util.WaitGroupWrapper - waiter util.WaitGroupWrapper - joinType plannercore.JoinType + worker util.WaitGroupWrapper + waiter util.WaitGroupWrapper joinChkResourceCh []chan *chunk.Chunk - joinResultCh chan *hashjoinWorkerResult memTracker *memory.Tracker // track memory usage. diskTracker *disk.Tracker // track disk usage. outerMatchedStatus []*bitmap.ConcurrentBitmap - useOuterToBuild bool - prepared bool - isOuterJoin bool - - finished atomic.Bool + prepared bool stats *hashJoinRuntimeStats } @@ -212,32 +217,32 @@ func (e *HashJoinExec) Open(ctx context.Context) error { // fetchProbeSideChunks get chunks from fetches chunks from the big table in a background goroutine // and sends the chunks to multiple channels which will be read by multiple join workers. -func (e *HashJoinExec) fetchProbeSideChunks(ctx context.Context) { +func (fetcher *probeSideTupleFetcher) fetchProbeSideChunks(ctx context.Context, maxChunkSize int) { hasWaitedForBuild := false for { - if e.finished.Load() { + if fetcher.finished.Load() { return } var probeSideResource *probeChkResource var ok bool select { - case <-e.closeCh: + case <-fetcher.closeCh: return - case probeSideResource, ok = <-e.probeSideTupleFetcher.probeChkResourceCh: + case probeSideResource, ok = <-fetcher.probeChkResourceCh: if !ok { return } } probeSideResult := probeSideResource.chk - if e.isOuterJoin { - required := int(atomic.LoadInt64(&e.probeSideTupleFetcher.requiredRows)) - probeSideResult.SetRequiredRows(required, e.maxChunkSize) + if fetcher.isOuterJoin { + required := int(atomic.LoadInt64(&fetcher.requiredRows)) + probeSideResult.SetRequiredRows(required, maxChunkSize) } - err := Next(ctx, e.probeSideTupleFetcher.probeSideExec, probeSideResult) + err := Next(ctx, fetcher.probeSideExec, probeSideResult) failpoint.Inject("ConsumeRandomPanic", nil) if err != nil { - e.joinResultCh <- &hashjoinWorkerResult{ + fetcher.joinResultCh <- &hashjoinWorkerResult{ err: err, } return @@ -248,23 +253,18 @@ func (e *HashJoinExec) fetchProbeSideChunks(ctx context.Context) { probeSideResult.Reset() } }) - if probeSideResult.NumRows() == 0 && !e.useOuterToBuild { - e.finished.Store(true) + if probeSideResult.NumRows() == 0 && !fetcher.useOuterToBuild { + fetcher.finished.Store(true) } - emptyBuild, buildErr := e.wait4BuildSide() + emptyBuild, buildErr := fetcher.wait4BuildSide() if buildErr != nil { - e.joinResultCh <- &hashjoinWorkerResult{ + fetcher.joinResultCh <- &hashjoinWorkerResult{ err: buildErr, } return } else if emptyBuild { return } - // after building is finished. the hash null bucket slice is allocated and determined. - // copy it for multi probe worker. - for _, w := range e.probeWorkers { - w.rowContainerForProbe.hashNANullBucket = e.rowContainer.hashNANullBucket - } hasWaitedForBuild = true } @@ -276,16 +276,16 @@ func (e *HashJoinExec) fetchProbeSideChunks(ctx context.Context) { } } -func (e *HashJoinExec) wait4BuildSide() (emptyBuild bool, err error) { +func (fetcher *probeSideTupleFetcher) wait4BuildSide() (emptyBuild bool, err error) { select { - case <-e.closeCh: + case <-fetcher.closeCh: return true, nil - case err := <-e.buildFinished: + case err := <-fetcher.buildFinished: if err != nil { return false, err } } - if e.rowContainer.Len() == uint64(0) && (e.joinType == plannercore.InnerJoin || e.joinType == plannercore.SemiJoin) { + if fetcher.rowContainer.Len() == uint64(0) && (fetcher.joinType == plannercore.InnerJoin || fetcher.joinType == plannercore.SemiJoin) { return true, nil } return false, nil @@ -329,6 +329,11 @@ func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chu } func (e *HashJoinExec) initializeForProbe() { + // e.joinResultCh is for transmitting the join result chunks to the main + // thread. + e.joinResultCh = make(chan *hashjoinWorkerResult, e.concurrency+1) + + e.probeSideTupleFetcher.hashJoinCtx = e.hashJoinCtx // e.probeSideTupleFetcher.probeResultChs is for transmitting the chunks which store the data of // probeSideExec, it'll be written by probe side worker goroutine, and read by join // workers. @@ -354,18 +359,14 @@ func (e *HashJoinExec) initializeForProbe() { e.joinChkResourceCh[i] = make(chan *chunk.Chunk, 1) e.joinChkResourceCh[i] <- newFirstChunk(e) } - - // e.joinResultCh is for transmitting the join result chunks to the main - // thread. - e.joinResultCh = make(chan *hashjoinWorkerResult, e.concurrency+1) } func (e *HashJoinExec) fetchAndProbeHashTable(ctx context.Context) { e.initializeForProbe() e.worker.RunWithRecover(func() { defer trace.StartRegion(ctx, "HashJoinProbeSideFetcher").End() - e.fetchProbeSideChunks(ctx) - }, e.handleProbeSideFetcherPanic) + e.probeSideTupleFetcher.fetchProbeSideChunks(ctx, e.maxChunkSize) + }, e.probeSideTupleFetcher.handleProbeSideFetcherPanic) probeKeyColIdx := make([]int, len(e.probeKeys)) probeNAKeColIdx := make([]int, len(e.probeNAKeys)) @@ -375,7 +376,6 @@ func (e *HashJoinExec) fetchAndProbeHashTable(ctx context.Context) { for i := range e.probeNAKeys { probeNAKeColIdx[i] = e.probeNAKeys[i].Index } - for i := uint(0); i < e.concurrency; i++ { workID := i e.worker.RunWithRecover(func() { @@ -386,12 +386,12 @@ func (e *HashJoinExec) fetchAndProbeHashTable(ctx context.Context) { e.waiter.RunWithRecover(e.waitJoinWorkersAndCloseResultChan, nil) } -func (e *HashJoinExec) handleProbeSideFetcherPanic(r interface{}) { - for i := range e.probeSideTupleFetcher.probeResultChs { - close(e.probeSideTupleFetcher.probeResultChs[i]) +func (fetcher *probeSideTupleFetcher) handleProbeSideFetcherPanic(r interface{}) { + for i := range fetcher.probeResultChs { + close(fetcher.probeResultChs[i]) } if r != nil { - e.joinResultCh <- &hashjoinWorkerResult{err: errors.Errorf("%v", r)} + fetcher.joinResultCh <- &hashjoinWorkerResult{err: errors.Errorf("%v", r)} } } diff --git a/executor/plan_replayer.go b/executor/plan_replayer.go index fec3de1867933..ea6ff6155bf44 100644 --- a/executor/plan_replayer.go +++ b/executor/plan_replayer.go @@ -18,14 +18,10 @@ import ( "archive/zip" "bytes" "context" - "crypto/rand" - "encoding/base64" "encoding/json" "fmt" "os" - "path/filepath" "strings" - "time" "github.com/BurntSushi/toml" "github.com/pingcap/errors" @@ -95,44 +91,13 @@ func (e *PlanReplayerExec) Next(ctx context.Context, req *chunk.Chunk) error { func (e *PlanReplayerExec) createFile() error { var err error - e.DumpInfo.File, e.DumpInfo.FileName, err = GeneratePlanReplayerFile() + e.DumpInfo.File, e.DumpInfo.FileName, err = domain.GeneratePlanReplayerFile() if err != nil { return err } return nil } -// GeneratePlanReplayerFile generates plan replayer file -func GeneratePlanReplayerFile() (*os.File, string, error) { - path := domain.GetPlanReplayerDirName() - err := os.MkdirAll(path, os.ModePerm) - if err != nil { - return nil, "", errors.AddStack(err) - } - fileName, err := generatePlanReplayerFileName() - if err != nil { - return nil, "", errors.AddStack(err) - } - zf, err := os.Create(filepath.Join(path, fileName)) - if err != nil { - return nil, "", errors.AddStack(err) - } - return zf, fileName, err -} - -func generatePlanReplayerFileName() (string, error) { - // Generate key and create zip file - time := time.Now().UnixNano() - b := make([]byte, 16) - //nolint: gosec - _, err := rand.Read(b) - if err != nil { - return "", err - } - key := base64.URLEncoding.EncodeToString(b) - return fmt.Sprintf("replayer_%v_%v.zip", key, time), nil -} - func (e *PlanReplayerDumpInfo) dump(ctx context.Context) (err error) { fileName := e.FileName zf := e.File diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 13ef8fda55c12..8ee6a0aa867c4 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -851,6 +851,7 @@ func TestIssue28782(t *testing.T) { func TestIssue29101(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) + tk.MustExec("set tidb_cost_model_version=1") tk.MustExec(`set tidb_enable_prepared_plan_cache=1`) tk.MustExec(`use test`) tk.MustExec("set @@tidb_enable_collect_execution_info=0;") diff --git a/executor/set.go b/executor/set.go index b3b8c00c1d316..2396356c1245b 100644 --- a/executor/set.go +++ b/executor/set.go @@ -24,7 +24,9 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/plugin" + "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table/temptable" @@ -32,6 +34,7 @@ import ( "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sem" "go.uber.org/zap" ) @@ -109,6 +112,22 @@ func (e *SetExecutor) setSysVariable(ctx context.Context, name string, v *expres } return variable.ErrUnknownSystemVar.GenWithStackByArgs(name) } + + if sysVar.RequireDynamicPrivileges != nil { + semEnabled := sem.IsEnabled() + pm := privilege.GetPrivilegeManager(e.ctx) + privs := sysVar.RequireDynamicPrivileges(v.IsGlobal, semEnabled) + for _, priv := range privs { + if !pm.RequestDynamicVerification(sessionVars.ActiveRoles, priv, false) { + msg := priv + if !semEnabled { + msg = "SUPER or " + msg + } + return core.ErrSpecificAccessDenied.GenWithStackByArgs(msg) + } + } + } + if sysVar.IsNoop && !variable.EnableNoopVariables.Load() { // The variable is a noop. For compatibility we allow it to still // be changed, but we append a warning since users might be expecting diff --git a/executor/set_test.go b/executor/set_test.go index a8d4a0b059246..697209d64836a 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -744,7 +744,7 @@ func TestSetVar(t *testing.T) { tk.MustQuery("select @@tidb_max_auto_analyze_time").Check(testkit.Rows("0")) // test variables for cost model ver2 - tk.MustQuery("select @@tidb_cost_model_version").Check(testkit.Rows("1")) + tk.MustQuery("select @@tidb_cost_model_version").Check(testkit.Rows(fmt.Sprintf("%v", variable.DefTiDBCostModelVer))) tk.MustExec("set tidb_cost_model_version=3") tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect tidb_cost_model_version value: '3'")) tk.MustExec("set tidb_cost_model_version=0") diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index 802d559eb3c80..7047c540fa577 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -558,6 +558,14 @@ func TestExprPushDownToFlash(t *testing.T) { require.NoError(t, err) exprs = append(exprs, function) + // ExtractDuration + extractDurationUnitCol := new(Constant) + extractDurationUnitCol.Value = types.NewStringDatum("microsecond") + extractDurationUnitCol.RetType = types.NewFieldType(mysql.TypeString) + function, err = NewFunction(mock.NewContext(), ast.Extract, types.NewFieldType(mysql.TypeLonglong), extractDurationUnitCol, durationColumn) + require.NoError(t, err) + exprs = append(exprs, function) + // CastIntAsInt function, err = NewFunction(mock.NewContext(), ast.Cast, types.NewFieldType(mysql.TypeLonglong), intColumn) require.NoError(t, err) diff --git a/expression/expression.go b/expression/expression.go index fefa1c403c959..cbfa4bfb0ee24 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1227,7 +1227,7 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { } case ast.Extract: switch function.Function.PbCode() { - case tipb.ScalarFuncSig_ExtractDatetime: + case tipb.ScalarFuncSig_ExtractDatetime, tipb.ScalarFuncSig_ExtractDuration: return true } case ast.Replace: diff --git a/expression/extension.go b/expression/extension.go index 2fca7fdfab6d9..9ab506213d5f0 100644 --- a/expression/extension.go +++ b/expression/extension.go @@ -114,11 +114,13 @@ func (c *extensionFuncClass) getFunction(ctx sessionctx.Context, args []Expressi } func (c *extensionFuncClass) checkPrivileges(ctx sessionctx.Context) error { - privs := c.funcDef.RequireDynamicPrivileges - if semPrivs := c.funcDef.SemRequireDynamicPrivileges; len(semPrivs) > 0 && sem.IsEnabled() { - privs = semPrivs + fn := c.funcDef.RequireDynamicPrivileges + if fn == nil { + return nil } + semEnabled := sem.IsEnabled() + privs := fn(semEnabled) if len(privs) == 0 { return nil } @@ -129,7 +131,7 @@ func (c *extensionFuncClass) checkPrivileges(ctx sessionctx.Context) error { for _, priv := range privs { if !manager.RequestDynamicVerification(activeRoles, priv, false) { msg := priv - if !sem.IsEnabled() { + if !semEnabled { msg = "SUPER or " + msg } return errSpecificAccessDenied.GenWithStackByArgs(msg) diff --git a/extension/BUILD.bazel b/extension/BUILD.bazel index 67b3351e012a4..12ca672b3c86f 100644 --- a/extension/BUILD.bazel +++ b/extension/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "//parser", "//parser/ast", "//parser/auth", + "//parser/mysql", "//sessionctx/stmtctx", "//sessionctx/variable", "//types", diff --git a/extension/extensions.go b/extension/extensions.go index 68bcffd585a89..55f9c0bb83606 100644 --- a/extension/extensions.go +++ b/extension/extensions.go @@ -45,6 +45,21 @@ func (es *Extensions) Bootstrap(ctx BootstrapContext) error { return nil } +// GetAccessCheckFuncs returns spec functions of the custom access check +func (es *Extensions) GetAccessCheckFuncs() (funcs []AccessCheckFunc) { + if es == nil { + return nil + } + + for _, m := range es.manifests { + if m.accessCheckFunc != nil { + funcs = append(funcs, m.accessCheckFunc) + } + } + + return funcs +} + // NewSessionExtensions creates a new ConnExtensions object func (es *Extensions) NewSessionExtensions() *SessionExtensions { if es == nil { diff --git a/extension/function.go b/extension/function.go index d01519e9b751d..cb7c19de0a507 100644 --- a/extension/function.go +++ b/extension/function.go @@ -48,12 +48,8 @@ type FunctionDef struct { EvalStringFunc func(ctx FunctionContext, row chunk.Row) (string, bool, error) // EvalIntFunc is the eval function when `EvalTp` is `types.ETInt` EvalIntFunc func(ctx FunctionContext, row chunk.Row) (int64, bool, error) - // RequireDynamicPrivileges is the dynamic privileges needed to invoke the function - // If `RequireDynamicPrivileges` is empty, it means every one can invoke this function - RequireDynamicPrivileges []string - // SemRequireDynamicPrivileges is the dynamic privileges needed to invoke the function in sem mode - // If `SemRequireDynamicPrivileges` is empty, `DynamicPrivileges` will be used in sem mode - SemRequireDynamicPrivileges []string + // RequireDynamicPrivileges is a function to return a list of dynamic privileges to check. + RequireDynamicPrivileges func(sem bool) []string } // Validate validates the function definition diff --git a/extension/function_test.go b/extension/function_test.go index c9521e91edadd..86fcd0be845c3 100644 --- a/extension/function_test.go +++ b/extension/function_test.go @@ -283,26 +283,37 @@ func TestExtensionFuncPrivilege(t *testing.T) { }, }, { - Name: "custom_only_dyn_priv_func", - EvalTp: types.ETString, - RequireDynamicPrivileges: []string{"CUSTOM_DYN_PRIV_1"}, + Name: "custom_only_dyn_priv_func", + EvalTp: types.ETString, + RequireDynamicPrivileges: func(sem bool) []string { + return []string{"CUSTOM_DYN_PRIV_1"} + }, EvalStringFunc: func(ctx extension.FunctionContext, row chunk.Row) (string, bool, error) { return "abc", false, nil }, }, { - Name: "custom_only_sem_dyn_priv_func", - EvalTp: types.ETString, - SemRequireDynamicPrivileges: []string{"RESTRICTED_CUSTOM_DYN_PRIV_2"}, + Name: "custom_only_sem_dyn_priv_func", + EvalTp: types.ETString, + RequireDynamicPrivileges: func(sem bool) []string { + if sem { + return []string{"RESTRICTED_CUSTOM_DYN_PRIV_2"} + } + return nil + }, EvalStringFunc: func(ctx extension.FunctionContext, row chunk.Row) (string, bool, error) { return "def", false, nil }, }, { - Name: "custom_both_dyn_priv_func", - EvalTp: types.ETString, - RequireDynamicPrivileges: []string{"CUSTOM_DYN_PRIV_1"}, - SemRequireDynamicPrivileges: []string{"RESTRICTED_CUSTOM_DYN_PRIV_2"}, + Name: "custom_both_dyn_priv_func", + EvalTp: types.ETString, + RequireDynamicPrivileges: func(sem bool) []string { + if sem { + return []string{"RESTRICTED_CUSTOM_DYN_PRIV_2"} + } + return []string{"CUSTOM_DYN_PRIV_1"} + }, EvalStringFunc: func(ctx extension.FunctionContext, row chunk.Row) (string, bool, error) { return "ghi", false, nil }, diff --git a/extension/manifest.go b/extension/manifest.go index 4a11b93beb4b8..436067c11d704 100644 --- a/extension/manifest.go +++ b/extension/manifest.go @@ -19,6 +19,7 @@ import ( "github.com/ngaut/pools" "github.com/pingcap/errors" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/chunk" clientv3 "go.etcd.io/etcd/client/v3" @@ -54,6 +55,16 @@ func WithCustomFunctions(funcs []*FunctionDef) Option { } } +// AccessCheckFunc is a function that returns a dynamic privilege list for db/tbl/column access +type AccessCheckFunc func(db, tbl, column string, priv mysql.PrivilegeType, sem bool) []string + +// WithCustomAccessCheck specifies the custom db/tbl/column dynamic privilege check +func WithCustomAccessCheck(fn AccessCheckFunc) Option { + return func(m *Manifest) { + m.accessCheckFunc = fn + } +} + // WithSessionHandlerFactory specifies a factory function to handle session func WithSessionHandlerFactory(factory func() *SessionHandler) Option { return func(m *Manifest) { @@ -106,6 +117,7 @@ type Manifest struct { dynPrivs []string bootstrap func(BootstrapContext) error funcs []*FunctionDef + accessCheckFunc AccessCheckFunc sessionHandlerFactory func() *SessionHandler close func() } diff --git a/extension/registry_test.go b/extension/registry_test.go index 6137d3b0ff013..ba1130d815fbf 100644 --- a/extension/registry_test.go +++ b/extension/registry_test.go @@ -19,8 +19,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/extension" + "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/sem" "github.com/stretchr/testify/require" ) @@ -47,9 +51,7 @@ func TestSetupExtensions(t *testing.T) { } func TestExtensionRegisterName(t *testing.T) { - defer func() { - extension.Reset() - }() + defer extension.Reset() // test empty name extension.Reset() @@ -62,9 +64,7 @@ func TestExtensionRegisterName(t *testing.T) { } func TestRegisterExtensionWithClose(t *testing.T) { - defer func() { - extension.Reset() - }() + defer extension.Reset() // normal register extension.Reset() @@ -97,9 +97,7 @@ func TestRegisterExtensionWithClose(t *testing.T) { } func TestRegisterExtensionWithDyncPrivs(t *testing.T) { - defer func() { - extension.Reset() - }() + defer extension.Reset() origDynPrivs := privileges.GetDynamicPrivileges() origDynPrivs = append([]string{}, origDynPrivs...) @@ -132,9 +130,7 @@ func TestRegisterExtensionWithDyncPrivs(t *testing.T) { } func TestRegisterExtensionWithSysVars(t *testing.T) { - defer func() { - extension.Reset() - }() + defer extension.Reset() sysVar1 := &variable.SysVar{ Scope: variable.ScopeGlobal | variable.ScopeSession, @@ -184,3 +180,170 @@ func TestRegisterExtensionWithSysVars(t *testing.T) { require.Nil(t, variable.GetSysVar("var1")) require.Nil(t, variable.GetSysVar("var2")) } + +func TestSetVariablePrivilege(t *testing.T) { + defer extension.Reset() + + sysVar1 := &variable.SysVar{ + Scope: variable.ScopeGlobal | variable.ScopeSession, + Name: "var1", + Value: "1", + MinValue: 0, + MaxValue: 100, + Type: variable.TypeInt, + RequireDynamicPrivileges: func(isGlobal bool, sem bool) []string { + privs := []string{"priv1"} + if isGlobal { + privs = append(privs, "priv2") + } + + if sem { + privs = append(privs, "restricted_priv3") + } + + return privs + }, + } + + extension.Reset() + require.NoError(t, extension.Register( + "test", + extension.WithCustomSysVariables([]*variable.SysVar{sysVar1}), + extension.WithCustomDynPrivs([]string{"priv1", "priv2", "restricted_priv3"}), + )) + require.NoError(t, extension.Setup()) + + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("create user u2@localhost") + + tk1 := testkit.NewTestKit(t, store) + require.NoError(t, tk1.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil)) + + tk2 := testkit.NewTestKit(t, store) + require.NoError(t, tk2.Session().Auth(&auth.UserIdentity{Username: "u2", Hostname: "localhost"}, nil, nil)) + + sem.Disable() + tk1.MustExec("set @@var1=7") + tk1.MustQuery("select @@var1").Check(testkit.Rows("7")) + + require.EqualError(t, tk2.ExecToErr("set @@var1=10"), "[planner:1227]Access denied; you need (at least one of) the SUPER or priv1 privilege(s) for this operation") + tk2.MustQuery("select @@var1").Check(testkit.Rows("1")) + + tk.MustExec("GRANT priv1 on *.* TO u2@localhost") + tk2.MustExec("set @@var1=8") + tk2.MustQuery("select @@var1").Check(testkit.Rows("8")) + + tk1.MustExec("set @@global.var1=17") + tk1.MustQuery("select @@global.var1").Check(testkit.Rows("17")) + + tk.MustExec("GRANT SYSTEM_VARIABLES_ADMIN on *.* TO u2@localhost") + require.EqualError(t, tk2.ExecToErr("set @@global.var1=18"), "[planner:1227]Access denied; you need (at least one of) the SUPER or priv2 privilege(s) for this operation") + tk2.MustQuery("select @@global.var1").Check(testkit.Rows("17")) + + tk.MustExec("GRANT priv2 on *.* TO u2@localhost") + tk2.MustExec("set @@global.var1=18") + tk2.MustQuery("select @@global.var1").Check(testkit.Rows("18")) + + sem.Enable() + defer sem.Disable() + + require.EqualError(t, tk1.ExecToErr("set @@global.var1=27"), "[planner:1227]Access denied; you need (at least one of) the restricted_priv3 privilege(s) for this operation") + tk1.MustQuery("select @@global.var1").Check(testkit.Rows("18")) + + require.EqualError(t, tk2.ExecToErr("set @@global.var1=27"), "[planner:1227]Access denied; you need (at least one of) the restricted_priv3 privilege(s) for this operation") + tk2.MustQuery("select @@global.var1").Check(testkit.Rows("18")) + + tk.MustExec("GRANT restricted_priv3 on *.* TO u2@localhost") + tk2.MustExec("set @@global.var1=28") + tk2.MustQuery("select @@global.var1").Check(testkit.Rows("28")) +} + +func TestCustomAccessCheck(t *testing.T) { + defer extension.Reset() + extension.Reset() + + require.NoError(t, extension.Register( + "test", + extension.WithCustomDynPrivs([]string{"priv1", "priv2", "restricted_priv3"}), + extension.WithCustomAccessCheck(func(db, tbl, column string, priv mysql.PrivilegeType, sem bool) []string { + if db != "test" || tbl != "t1" { + return nil + } + + var privs []string + if priv == mysql.SelectPriv { + privs = append(privs, "priv1") + } else if priv == mysql.UpdatePriv { + privs = append(privs, "priv2") + if sem { + privs = append(privs, "restricted_priv3") + } + } else { + return nil + } + + return privs + }), + )) + require.NoError(t, extension.Setup()) + + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("create user u2@localhost") + + tk1 := testkit.NewTestKit(t, store) + require.NoError(t, tk1.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil)) + tk1.MustExec("use test") + + tk2 := testkit.NewTestKit(t, store) + require.NoError(t, tk2.Session().Auth(&auth.UserIdentity{Username: "u2", Hostname: "localhost"}, nil, nil)) + tk.MustExec("GRANT all on test.t1 TO u2@localhost") + tk2.MustExec("use test") + + tk1.MustExec("create table t1(id int primary key, v int)") + tk1.MustExec("insert into t1 values (1, 10), (2, 20)") + + tk1.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + tk1.MustQuery("select * from t1").Check(testkit.Rows("1 10", "2 20")) + + require.EqualError(t, tk2.ExecToErr("select * from t1 where id=1"), "[planner:1142]SELECT command denied to user 'u2'@'localhost' for table 't1'") + require.EqualError(t, tk2.ExecToErr("select * from t1"), "[planner:1142]SELECT command denied to user 'u2'@'localhost' for table 't1'") + + tk.MustExec("GRANT priv1 on *.* TO u2@localhost") + tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + tk2.MustQuery("select * from t1").Check(testkit.Rows("1 10", "2 20")) + + require.EqualError(t, tk2.ExecToErr("update t1 set v=11 where id=1"), "[planner:8121]privilege check for 'Update' fail") + require.EqualError(t, tk2.ExecToErr("update t1 set v=11 where id<2"), "[planner:8121]privilege check for 'Update' fail") + tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 10")) + + tk.MustExec("GRANT priv2 on *.* TO u2@localhost") + tk2.MustExec("update t1 set v=11 where id=1") + tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 11")) + + tk2.MustExec("update t1 set v=12 where id<2") + tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 12")) + + sem.Enable() + defer sem.Disable() + + require.EqualError(t, tk1.ExecToErr("update t1 set v=21 where id=1"), "[planner:8121]privilege check for 'Update' fail") + require.EqualError(t, tk1.ExecToErr("update t1 set v=21 where id<2"), "[planner:8121]privilege check for 'Update' fail") + tk1.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 12")) + + require.EqualError(t, tk2.ExecToErr("update t1 set v=21 where id=1"), "[planner:8121]privilege check for 'Update' fail") + require.EqualError(t, tk2.ExecToErr("update t1 set v=21 where id<2"), "[planner:8121]privilege check for 'Update' fail") + tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 12")) + + tk.MustExec("GRANT restricted_priv3 on *.* TO u2@localhost") + tk2.MustExec("update t1 set v=31 where id=1") + tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 31")) + + tk2.MustExec("update t1 set v=32 where id<2") + tk2.MustQuery("select * from t1 where id=1").Check(testkit.Rows("1 32")) +} diff --git a/go.mod b/go.mod index 01400b701b85b..c147af6b8506e 100644 --- a/go.mod +++ b/go.mod @@ -69,7 +69,7 @@ require ( github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3 github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 github.com/pingcap/kvproto v0.0.0-20221103025916-e7e21f0e9cd9 - github.com/pingcap/log v1.1.1-0.20221110065318-21a4942860b3 + github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e github.com/pingcap/tipb v0.0.0-20221020071514-cd933387bcb5 diff --git a/go.sum b/go.sum index 0730b5193bd55..48306f2d8d576 100644 --- a/go.sum +++ b/go.sum @@ -783,8 +783,8 @@ github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= -github.com/pingcap/log v1.1.1-0.20221110065318-21a4942860b3 h1:T7e5Low0BU2ZazI2dz2mh3W1qv+w8wtvq1YR8DneA0c= -github.com/pingcap/log v1.1.1-0.20221110065318-21a4942860b3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= +github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c h1:crhkw6DD+07Bg1wYhW5Piw+kYNKZqFQqfC2puUf6gMI= +github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops= github.com/pingcap/tipb v0.0.0-20221020071514-cd933387bcb5 h1:Yoo8j5xQGxjlsC3yt0ndsiAz0WZXED9rzsKmEN0U0DY= diff --git a/infoschema/tables.go b/infoschema/tables.go index 656a99f52d838..8df6cab9cb84c 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1729,8 +1729,8 @@ func FormatTiDBVersion(TiDBVersion string, isDefaultVersion bool) string { // The user hasn't set the config 'ServerVersion'. if isDefaultVersion { - nodeVersion = TiDBVersion[strings.LastIndex(TiDBVersion, "TiDB-")+len("TiDB-"):] - if nodeVersion[0] == 'v' { + nodeVersion = TiDBVersion[strings.Index(TiDBVersion, "TiDB-")+len("TiDB-"):] + if len(nodeVersion) > 0 && nodeVersion[0] == 'v' { nodeVersion = nodeVersion[1:] } nodeVersions := strings.Split(nodeVersion, "-") diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 3650f2833a1a6..aee92b0afed70 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -676,8 +676,14 @@ func TestSelectHiddenColumn(t *testing.T) { func TestFormatVersion(t *testing.T) { // Test for defaultVersions. - defaultVersions := []string{"5.7.25-TiDB-None", "5.7.25-TiDB-8.0.18", "5.7.25-TiDB-8.0.18-beta.1", "5.7.25-TiDB-v4.0.0-beta-446-g5268094af"} - defaultRes := []string{"None", "8.0.18", "8.0.18-beta.1", "4.0.0-beta"} + defaultVersions := []string{ + "5.7.25-TiDB-None", + "5.7.25-TiDB-8.0.18", + "5.7.25-TiDB-8.0.18-beta.1", + "5.7.25-TiDB-v4.0.0-beta-446-g5268094af", + "5.7.25-TiDB-", + "5.7.25-TiDB-v4.0.0-TiDB-446"} + defaultRes := []string{"None", "8.0.18", "8.0.18-beta.1", "4.0.0-beta", "", "4.0.0-TiDB"} for i, v := range defaultVersions { version := infoschema.FormatTiDBVersion(v, true) require.Equal(t, defaultRes[i], version) diff --git a/parser/ast/misc.go b/parser/ast/misc.go index b0668c7a09a7e..928c26489bee0 100644 --- a/parser/ast/misc.go +++ b/parser/ast/misc.go @@ -3648,9 +3648,13 @@ func (n *TableOptimizerHint) Restore(ctx *format.RestoreCtx) error { } ctx.WriteName(n.QBName.String()) } + if n.HintName.L == "qb_name" && len(n.Tables) == 0 { + ctx.WritePlain(")") + return nil + } // Hints without args except query block. switch n.HintName.L { - case "mpp_1phase_agg", "mpp_2phase_agg", "hash_agg", "stream_agg", "agg_to_cop", "read_consistent_replica", "no_index_merge", "qb_name", "ignore_plan_cache", "limit_to_cop", "straight_join", "merge", "no_decorrelate": + case "mpp_1phase_agg", "mpp_2phase_agg", "hash_agg", "stream_agg", "agg_to_cop", "read_consistent_replica", "no_index_merge", "ignore_plan_cache", "limit_to_cop", "straight_join", "merge", "no_decorrelate": ctx.WritePlain(")") return nil } @@ -3679,6 +3683,16 @@ func (n *TableOptimizerHint) Restore(ctx *format.RestoreCtx) error { } ctx.WriteName(index.String()) } + case "qb_name": + if len(n.Tables) > 0 { + ctx.WritePlain(", ") + for i, table := range n.Tables { + if i != 0 { + ctx.WritePlain(". ") + } + table.Restore(ctx) + } + } case "use_toja", "use_cascades": if n.HintData.(bool) { ctx.WritePlain("TRUE") diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 8a8acaa292666..88e8d8f43bffa 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -98,6 +98,7 @@ const ( ActionMultiSchemaChange ActionType = 61 ActionFlashbackCluster ActionType = 62 ActionRecoverSchema ActionType = 63 + ActionReorganizePartition ActionType = 64 ) var actionMap = map[ActionType]string{ @@ -160,6 +161,7 @@ var actionMap = map[ActionType]string{ ActionMultiSchemaChange: "alter table multi-schema change", ActionFlashbackCluster: "flashback cluster", ActionRecoverSchema: "flashback schema", + ActionReorganizePartition: "alter table reorganize partition", // `ActionAlterTableAlterPartition` is removed and will never be used. // Just left a tombstone here for compatibility. diff --git a/parser/types/field_type.go b/parser/types/field_type.go index 6afc445b30c6e..efc1c20d0c18f 100644 --- a/parser/types/field_type.go +++ b/parser/types/field_type.go @@ -392,7 +392,9 @@ func (ft *FieldType) CompactStr() string { // returns a string. func (ft *FieldType) InfoSchemaStr() string { suffix := "" - if mysql.HasUnsignedFlag(ft.flag) { + if mysql.HasUnsignedFlag(ft.flag) && + ft.tp != mysql.TypeBit && + ft.tp != mysql.TypeYear { suffix = " unsigned" } return ft.CompactStr() + suffix diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index 49fffb149b85d..4a351e60a9018 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -17,7 +17,6 @@ package core import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/sessionctx/variable" ) const ( @@ -51,7 +50,7 @@ type columnStatsUsageCollector struct { visitedtbls map[int64]struct{} } -func newColumnStatsUsageCollector(collectMode uint64) *columnStatsUsageCollector { +func newColumnStatsUsageCollector(collectMode uint64, enabledPlanCapture bool) *columnStatsUsageCollector { collector := &columnStatsUsageCollector{ collectMode: collectMode, // Pre-allocate a slice to reduce allocation, 8 doesn't have special meaning. @@ -64,7 +63,7 @@ func newColumnStatsUsageCollector(collectMode uint64) *columnStatsUsageCollector if collectMode&collectHistNeededColumns != 0 { collector.histNeededCols = make(map[model.TableItemID]struct{}) } - if variable.EnablePlanReplayerCapture.Load() { + if enabledPlanCapture { collector.collectVisitedTable = true collector.visitedtbls = map[int64]struct{}{} } @@ -300,7 +299,7 @@ func CollectColumnStatsUsage(lp LogicalPlan, predicate, histNeeded bool) ([]mode if histNeeded { mode |= collectHistNeededColumns } - collector := newColumnStatsUsageCollector(mode) + collector := newColumnStatsUsageCollector(mode, lp.SCtx().GetSessionVars().EnablePlanReplayerCapture) collector.collectFromPlan(lp) if collector.collectVisitedTable { recordTableRuntimeStats(lp.SCtx(), collector.visitedtbls) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 24b19bad5c3e9..a2aa3d643f609 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -1329,7 +1329,8 @@ func TestViewHint(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.MustExec("drop view if exists v, v1") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop view if exists v, v1, v2") tk.MustExec("drop table if exists t, t1, t2") tk.MustExec("create table t(a int, b int);") tk.MustExec("create table t1(a int, b int);") @@ -1363,6 +1364,44 @@ func TestViewHintScope(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("drop view if exists v, v1, v2, v3") + tk.MustExec("drop table if exists t, t1, t2") + tk.MustExec("create table t(a int, b int);") + tk.MustExec("create table t1(a int, b int);") + tk.MustExec("create table t2(a int, b int);") + tk.MustExec("create table t3(a int, b int)") + tk.MustExec("create definer='root'@'localhost' view v as select t.a, t.b from t join (select count(*) as a from t1 join t2 join t3 where t1.b=t2.b and t2.a = t3.a group by t2.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v1 as select t.a, t.b from t join (select count(*) as a from t1 join v on t1.b=v.b group by v.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v3 as select /*+ merge_join(t) */ t.a, t.b from t join (select /*+ stream_agg() */ count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.LoadTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} + +func TestViewHintWithBinding(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop view if exists v, v1") tk.MustExec("drop table if exists t, t1, t2") tk.MustExec("create table t(a int, b int);") @@ -1373,6 +1412,67 @@ func TestViewHintScope(t *testing.T) { tk.MustExec("create definer='root'@'localhost' view v1 as select t.a, t.b from t join (select count(*) as a from t1 join v on t1.b=v.b group by v.a) tt on t.a = tt.a;") tk.MustExec("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") + tk.MustExec("select * from v2") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + tk.MustExec("create global binding for select * from v2 using select /*+ qb_name(qb_v_2, v2.v1@sel_2 .v@sel_2 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v2. v1@sel_2 .v@sel_2 .@sel_1), merge_join(t@qb_v_1) */ * from v2;") + tk.MustExec("select * from v2") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res := tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select * from `test` . `v2`") + require.Equal(t, res[0][1], "SELECT /*+ qb_name(`qb_v_2` , `v2`. `v1`@`sel_2`. `v`@`sel_2`. ``@`sel_2`) merge_join(`t1`@`qb_v_2`) stream_agg(@`qb_v_2`) qb_name(`qb_v_1` , `v2`. `v1`@`sel_2`. `v`@`sel_2`. ``@`sel_1`) merge_join(`t`@`qb_v_1`)*/ * FROM `test`.`v2`") + + tk.MustExec("drop global binding for select * from v2") + tk.MustExec("select * from v2") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, len(res), 0) +} + +func TestAllViewHintType(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash, tikv'") + tk.MustExec("drop view if exists v, v1, v2, v3, v4, v5, v6, v7, v8, v9, v10, v11, v12") + tk.MustExec("drop table if exists t, t1, t2, t4, t3, t5") + tk.MustExec("create table t(a int not null, b int, index idx_a(a));") + tk.MustExec("create table t1(a int not null, b int, index idx_a(a));") + tk.MustExec("create table t2(a int, b int, index idx_a(a));") + tk.MustExec("create table t3(a int, b int, index idx_a(a));") + tk.MustExec("create table t4(a int, b int, index idx_a(a));") + tk.MustExec("create table t5(a int, b int, index idx_a(a), index idx_b(b));") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("create definer='root'@'localhost' view v as select t.a, t.b from t join t1 on t.a = t1.a;") + tk.MustExec("create definer='root'@'localhost' view v1 as select t2.a, t2.b from t2 join t3 join v where t2.b = t3.b and t3.a = v.a;") + tk.MustExec("create definer='root'@'localhost' view v2 as select t.a, t.b from t join (select count(*) as a from t1 join v1 on t1.b=v1.b group by v1.a) tt on t.a = tt.a;") + tk.MustExec("create definer='root'@'localhost' view v3 as select * from t5 where a > 1 and b < 2;") + tk.MustExec("create definer='root'@'localhost' view v4 as select * from t5 where a > 1 or b < 2;") + tk.MustExec("create definer='root'@'localhost' view v5 as SELECT * FROM t WHERE EXISTS (SELECT 1 FROM t1 WHERE t1.b = t.b);") + tk.MustExec("create definer='root'@'localhost' view v6 as select * from t1 where t1.a < (select sum(t2.a) from t2 where t2.b = t1.b);") + tk.MustExec("create definer='root'@'localhost' view v7 as WITH CTE AS (SELECT * FROM t WHERE t.a < 60) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;") + tk.MustExec("create definer='root'@'localhost' view v8 as WITH CTE1 AS (SELECT b FROM t1), CTE2 AS (WITH CTE3 AS (SELECT a FROM t2), CTE4 AS (SELECT a FROM t3) SELECT CTE3.a FROM CTE3, CTE4) SELECT b FROM CTE1, CTE2 union select * from CTE1;") + tk.MustExec("create definer='root'@'localhost' view v9 as select sum(a) from t;") + tk.MustExec("create definer='root'@'localhost' view v10 as SELECT * FROM t WHERE a > 10 ORDER BY b LIMIT 1;") + tk.MustExec("create definer='root'@'localhost' view v11 as select a, sum(b) from t group by a") + tk.MustExec("create definer='root'@'localhost' view v12 as select t.a, t.b from t join t t1 on t.a = t1.a;") + var input []string var output []struct { SQL string @@ -2620,6 +2720,7 @@ func TestSelectLimit(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") tk.MustExec("insert into t values(1),(1),(2)") @@ -2644,9 +2745,9 @@ func TestSelectLimit(t *testing.T) { result = tk.MustQuery("select (select * from t limit 1) s") // limit write in subquery, has no effect. result.Check(testkit.Rows("1")) result = tk.MustQuery("select * from t where t.a in (select * from t) limit 3") // select_limit will not effect subquery - result.Check(testkit.Rows("1", "1", "2")) + result.Sort().Check(testkit.Rows("1", "1", "2")) result = tk.MustQuery("select * from (select * from t) s limit 3") // select_limit will not effect subquery - result.Check(testkit.Rows("1", "1", "2")) + result.Sort().Check(testkit.Rows("1", "1", "2")) // test for union result = tk.MustQuery("select * from t union all select * from t limit 2") // limit outside subquery @@ -2674,14 +2775,14 @@ func TestSelectLimit(t *testing.T) { result.Check(testkit.Rows("1")) tk.MustExec("set @@session.sql_select_limit=default") result = tk.MustQuery("select * from s") - result.Check(testkit.Rows("1", "1", "2")) + result.Sort().Check(testkit.Rows("1", "1", "2")) // test for DML tk.MustExec("set @@session.sql_select_limit=1") tk.MustExec("create table b (a int)") tk.MustExec("insert into b select * from t") // all values are inserted result = tk.MustQuery("select * from b limit 3") - result.Check(testkit.Rows("1", "1", "2")) + result.Sort().Check(testkit.Rows("1", "1", "2")) tk.MustExec("update b set a = 2 where a = 1") // all values are updated result = tk.MustQuery("select * from b limit 3") result.Check(testkit.Rows("2", "2", "2")) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 18edbd52be994..8ec9b8f48fc06 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -5019,7 +5019,6 @@ func (b *PlanBuilder) BuildDataSourceFromView(ctx context.Context, dbName model. for qbName, viewQbNameHint := range qbNameMap4View { // Check whether the view hint belong the current view or its nested views. selectOffset := -1 - qbNameMap4View[qbName] = viewQbNameHint if len(viewQbNameHint) == 0 { selectOffset = 1 } else if len(viewQbNameHint) == 1 && viewQbNameHint[0].TableName.L == "" { diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index 803b2adb647ba..5f07340f626e1 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -345,6 +345,7 @@ func TestListColumnsPartitionPruner(t *testing.T) { // tk1 use to test partition table with index. tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("set tidb_cost_model_version=2") tk1.MustExec("drop database if exists test_partition_1;") tk1.MustExec(`set @@session.tidb_regard_null_as_point=false`) tk1.MustExec("create database test_partition_1") @@ -357,6 +358,7 @@ func TestListColumnsPartitionPruner(t *testing.T) { // tk2 use to compare the result with normal table. tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("set tidb_cost_model_version=2") tk2.MustExec("drop database if exists test_partition_2;") tk2.MustExec(`set @@session.tidb_regard_null_as_point=false`) tk2.MustExec("create database test_partition_2") diff --git a/planner/core/physical_plan_trace_test.go b/planner/core/physical_plan_trace_test.go index ae93043101098..ff82d3392bcdd 100644 --- a/planner/core/physical_plan_trace_test.go +++ b/planner/core/physical_plan_trace_test.go @@ -62,7 +62,7 @@ func TestPhysicalOptimizeWithTraceEnabled(t *testing.T) { "Limit_20", "IndexReader_21", "Limit_14", - "StreamAgg_10", + "HashAgg_9", "Projection_8", }, }, @@ -95,6 +95,7 @@ func TestPhysicalOptimizeWithTraceEnabled(t *testing.T) { require.NoError(t, err) sctx := core.MockContext() sctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true + sctx.GetSessionVars().CostModelVersion = 2 builder, _ := core.NewPlanBuilder().Init(sctx, dom.InfoSchema(), &hint.BlockHintProcessor{}) domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(dom.InfoSchema()) plan, err := builder.Build(context.TODO(), stmt) diff --git a/planner/core/plan_cost_detail_test.go b/planner/core/plan_cost_detail_test.go index 0cad1210ec2f4..0b2df302b3cdd 100644 --- a/planner/core/plan_cost_detail_test.go +++ b/planner/core/plan_cost_detail_test.go @@ -139,6 +139,7 @@ func optimize(t *testing.T, sql string, p *parser.Parser, ctx sessionctx.Context sctx := plannercore.MockContext() sctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true sctx.GetSessionVars().EnableNewCostInterface = true + sctx.GetSessionVars().CostModelVersion = 1 builder, _ := plannercore.NewPlanBuilder().Init(sctx, dom.InfoSchema(), &hint.BlockHintProcessor{}) domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(dom.InfoSchema()) plan, err := builder.Build(context.TODO(), stmt) diff --git a/planner/core/plan_replayer_capture_test.go b/planner/core/plan_replayer_capture_test.go index 2e88f090bd784..6778cdba20bbf 100644 --- a/planner/core/plan_replayer_capture_test.go +++ b/planner/core/plan_replayer_capture_test.go @@ -35,7 +35,6 @@ func TestPlanReplayerCaptureRecordJsonStats(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t1(a int)") tk.MustExec("create table t2(a int)") - tk.MustExec("SET global tidb_enable_plan_replayer_capture = ON;") tk.MustExec("analyze table t1") tk.MustExec("analyze table t2") testcases := []struct { @@ -68,6 +67,7 @@ func getTableStats(sql string, t *testing.T, ctx sessionctx.Context, dom *domain err = core.Preprocess(context.Background(), ctx, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: dom.InfoSchema()})) require.NoError(t, err) sctx := core.MockContext() + sctx.GetSessionVars().EnablePlanReplayerCapture = true builder, _ := core.NewPlanBuilder().Init(sctx, dom.InfoSchema(), &hint.BlockHintProcessor{}) domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(dom.InfoSchema()) plan, err := builder.Build(context.TODO(), stmt) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 51fa3b365af5d..d059139bd7159 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -1875,7 +1875,7 @@ func tryLockMDLAndUpdateSchemaIfNecessary(sctx sessionctx.Context, dbName model. } } if found { - return nil, ErrSchemaChanged.GenWithStack("public column %s has changed", col.Name) + return nil, domain.ErrInfoSchemaChanged.GenWithStack("public column %s has changed", col.Name) } } } diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index db677ac97d23c..bcce484092644 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -658,7 +658,7 @@ "cases": [ // Hint for view v "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2), merge_join(t1@qb_v_2) */ * from v;", - "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v@sel_1 .@sel1), merge_join(t@qb_v_1) */ * from v;", + "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v@sel_1 .@sel_1), merge_join(t@qb_v_1) */ * from v;", "explain format = 'brief' select /*+ qb_name(qb_v_2, v1@sel_1 . v@sel_2 .@sel_2), merge_join(t1@qb_v_2) */ * from v1;", "explain format = 'brief' select /*+ qb_name(qb_v_2, v1@sel_1 . v@sel_2 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v1@sel_1 . v@sel_2 .@sel_1), merge_join(t@qb_v_1) */ * from v1;", "explain format = 'brief' select /*+ qb_name(qb_v_2, v2@sel_1 . v1@sel_2 . v@sel_2 .@sel_2), merge_join(t1@qb_v_2) */ * from v2;", @@ -672,7 +672,7 @@ // Hint for view v2 "explain format = 'brief' select /*+ qb_name(qb_v2_2, v2@sel_1 .@sel_2), merge_join(t1@qb_v2_2) */ * from v2;", - "explain format = 'brief' select /*+ qb_name(qb_v2_2, v2@sel_1 .@sel_2), merge_join(t1@qb_v2_2), stream_agg(@qb_v2_2), qb_name(qb_v2_1, v1@sel_1 .@sel_1), merge_join(t@qb_v2_1) */ * from v2;" + "explain format = 'brief' select /*+ qb_name(qb_v2_2, v2@sel_1 .@sel_2), merge_join(t1@qb_v2_2), stream_agg(@qb_v2_2), qb_name(qb_v2_1, v2), merge_join(t@qb_v2_1) */ * from v2;" ] }, { @@ -717,7 +717,80 @@ // Use the query block before define it "explain format = 'brief' select /*+ merge_join(t1@qb_v_2), qb_name(qb_v_2, v@sel_1 .@sel_2) */ * from v;", - "explain format = 'brief' select /*+ merge_join(t@qb_v_1), stream_agg(@qb_v_2), qb_name(qb_v_2, v@sel_1 .@sel_2), qb_name(qb_v_1, v@sel_1 .@sel1) */ * from v;" + "explain format = 'brief' select /*+ merge_join(t@qb_v_1), stream_agg(@qb_v_2), qb_name(qb_v_2, v@sel_1 .@sel_2), qb_name(qb_v_1, v@sel_1 .@sel_1) */ * from v;", + + // The view contains the hint when creation + "explain format = 'brief' select /*+ qb_name(qb_v3_2, v3@sel_1 .@sel_2), merge_join(t1@qb_v3_2) */ * from v3;", + "explain format = 'brief' select /*+ qb_name(qb_v3_2, v3@sel_1 .@sel_2), merge_join(t1@qb_v3_2), hash_agg(@qb_v3_2), qb_name(qb_v3_1, v3@sel_1 .@sel_1), hash_join(t@qb_v3_1) */ * from v3;" + ] + }, + { + "name": "TestAllViewHintType", + "cases": [ + // leading hint + // join nodes in the same view + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), leading(@qb_v1 v, t2) */ * from v1;", + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), leading(v@qb_v1, t2@qb_v1) */ * from v1;", + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), leading(@qb_v1 t3, t2) */ * from v1;", + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), leading(t3@qb_v1, t2@qb_v1) */ * from v1;", + + // join node across view + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), qb_name(qb_v, v1.v), leading(t2@qb_v1, t@qb_v) */ * from v1;", + + // hash_join hint + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join(@qb_v1 v, t2) */ * from v1;", + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join(t2@qb_v1, t3@qb_v1) */ * from v1;", + + // hash join build hint + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join_build(@qb_v1 v) */ * from v1;", + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join_build(t2@qb_v1) */ * from v1;", + + // hash join probe hint + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join_build(@qb_v1 v) */ * from v1;", + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join_build(t2@qb_v1) */ * from v1;", + + // merge join hint + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), merge_join(@qb_v1 v) */ * from v1;", + "explain format = 'brief' select /*+ qb_name(qb_v1, v1), merge_join(t2@qb_v1) */ * from v1;", + + // index join hint + "explain format = 'brief' select /*+ qb_name(qb_v, v), INL_JOIN(@qb_v t) */ * from v;", + "explain format = 'brief' select /*+ qb_name(qb_v, v), INL_JOIN(t@qb_v) */ * from v;", + + // agg hint + "explain format = 'brief' select /*+ qb_name(qb_v2, v2.@sel_2), hash_agg(@qb_v2) */ * from v2;", + "explain format = 'brief' select /*+ qb_name(qb_v2, v2.@sel_2), stream_agg(@qb_v2) */ * from v2;", + + // index hint + "explain format = 'brief' select /*+ qb_name(qb_v3, v3), use_index(t5@qb_v3, idx_a) */ * from v3;", + "explain format = 'brief' select /*+ qb_name(qb_v3, v3), use_index(@qb_v3 t5, idx_b) */ * from v3;", + "explain format = 'brief' select /*+ qb_name(qb_v3, v3), force_index(t5@qb_v3, idx_a) */ * from v3;", + "explain format = 'brief' select /*+ qb_name(qb_v3, v3), force_index(@qb_v3 t5, idx_b) */ * from v3;", + "explain format = 'brief' select /*+ qb_name(qb_v3, v3), ignore_index(t5@qb_v3, idx_a) */ * from v3;", + "explain format = 'brief' select /*+ qb_name(qb_v3, v3), ignore_index(@qb_v3 t5, idx_b) */ * from v3;", + "explain format = 'brief' select /*+ qb_name(qb_v4, v4), use_index_merge(t5@qb_v4, idx_a, idx_b) */ * from v4;", + "explain format = 'brief' select /*+ qb_name(qb_v4, v4), use_index_merge(@qb_v4 t5, idx_b, idx_a) */ * from v4;", + + // read from storage + "explain format = 'brief' select /*+ qb_name(qb_v, v), READ_FROM_STORAGE(TIFLASH[t@qb_v], TIKV[t1@qb_v]) */ * from v;", + + // subquery hint + "explain format = 'brief' select /*+ qb_name(qb_v5, v5.@sel_2), SEMI_JOIN_REWRITE(@qb_v5) */ * from v5;", + "explain format = 'brief' select /*+ qb_name(qb_v6, v6.@sel_2), NO_DECORRELATE(@qb_v6) */ * from v6;", + + // cte hint + "explain format = 'brief' select /*+ qb_name(qb_v7, v7), merge(@qb_v7) */ * from v7;", + "explain format = 'brief' select /*+ qb_name(qb_v8, v8), merge(@qb_v8) */ * from v8;", + + // agg to cop hint + "explain format = 'brief' select /*+ qb_name(qb_v9, v9), AGG_TO_COP(@qb_v9) */ * from v9;", + "explain format = 'brief' select /*+ qb_name(qb_v10, v10), LIMIT_TO_COP(@qb_v10) */ * from v10;", + + // MPP hint + "explain format = 'brief' select /*+ qb_name(qb, v11) read_from_storage(tiflash[t@qb]), MPP_1PHASE_AGG(@qb) */ * from v11;", + "explain format = 'brief' select /*+ qb_name(qb, v11) read_from_storage(tiflash[t@qb]), MPP_2PHASE_AGG(@qb) */ * from v11;", + "explain format = 'brief' select /*+ qb_name(qb, v12) read_from_storage(tiflash[t1@qb, t@qb]), shuffle_join(t1@qb, t@qb) */ * from v12;", + "explain format = 'brief' select /*+ qb_name(qb, v12) read_from_storage(tiflash[t1@qb, t@qb]), broadcast_join(t1@qb, t@qb) */ * from v12;" ] }, { diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index c64c004266ffd..b7f0c9ed9cb95 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -4379,23 +4379,25 @@ "Warn": null }, { - "SQL": "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v@sel_1 .@sel1), merge_join(t@qb_v_1) */ * from v;", - "Plan": [ - "HashJoin 9990.00 root inner join, equal:[eq(test.t.a, Column#10)]", - "├─StreamAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#10", - "│ └─Sort 12487.50 root test.t2.a", - "│ └─MergeJoin 12487.50 root inner join, left key:test.t1.b, right key:test.t2.b", - "│ ├─Sort(Build) 9990.00 root test.t2.b", - "│ │ └─TableReader 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ └─Sort(Probe) 9990.00 root test.t1.b", - "│ └─TableReader 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "└─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v_2, v@sel_1 .@sel_2), merge_join(t1@qb_v_2), stream_agg(@qb_v_2), qb_name(qb_v_1, v@sel_1 .@sel_1), merge_join(t@qb_v_1) */ * from v;", + "Plan": [ + "MergeJoin 9990.00 root inner join, left key:test.t.a, right key:Column#10", + "├─Sort(Build) 7992.00 root Column#10", + "│ └─StreamAgg 7992.00 root group by:test.t2.a, funcs:count(1)->Column#10", + "│ └─Sort 12487.50 root test.t2.a", + "│ └─MergeJoin 12487.50 root inner join, left key:test.t1.b, right key:test.t2.b", + "│ ├─Sort(Build) 9990.00 root test.t2.b", + "│ │ └─TableReader 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─Sort(Probe) 9990.00 root test.t1.b", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─Sort(Probe) 9990.00 root test.t.a", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -4416,12 +4418,12 @@ "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -4479,19 +4481,19 @@ "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -4527,12 +4529,12 @@ "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -4550,11 +4552,11 @@ "│ │ ├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#16", "│ │ │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ │ └─TableReader(Probe) 9980.01 root data:Selection", "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", @@ -4581,11 +4583,11 @@ "│ │ ├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#16", "│ │ │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ │ └─TableReader(Probe) 9980.01 root data:Selection", "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", @@ -4611,25 +4613,25 @@ "│ │ ├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#22", "│ │ │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -4647,25 +4649,25 @@ "│ │ ├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#22", "│ │ │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -4685,18 +4687,18 @@ "│ │ │ ├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#22", "│ │ │ │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", "│ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ │ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "│ │ └─TableReader(Probe) 9980.01 root data:Selection", "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", @@ -4711,41 +4713,43 @@ "Warn": null }, { - "SQL": "explain format = 'brief' select /*+ qb_name(qb_v2_2, v2@sel_1 .@sel_2), merge_join(t1@qb_v2_2), stream_agg(@qb_v2_2), qb_name(qb_v2_1, v1@sel_1 .@sel_1), merge_join(t@qb_v2_1) */ * from v2;", + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v2_2, v2@sel_1 .@sel_2), merge_join(t1@qb_v2_2), stream_agg(@qb_v2_2), qb_name(qb_v2_1, v2), merge_join(t@qb_v2_1) */ * from v2;", "Plan": [ - "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#24)]", - "├─StreamAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#24", - "│ └─Sort 12475.01 root test.t.a", - "│ └─MergeJoin 12475.01 root inner join, left key:test.t1.b, right key:test.t.b", - "│ ├─Sort(Build) 9980.01 root test.t.b", - "│ │ └─HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#23)]", - "│ │ ├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#23", - "│ │ │ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#22)]", - "│ │ │ ├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#22", - "│ │ │ │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", - "│ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─Sort(Probe) 9990.00 root test.t1.b", - "│ └─TableReader 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "└─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "MergeJoin 9980.01 root inner join, left key:test.t.a, right key:Column#24", + "├─Sort(Build) 7984.01 root Column#24", + "│ └─StreamAgg 7984.01 root group by:test.t.a, funcs:count(1)->Column#24", + "│ └─Sort 12475.01 root test.t.a", + "│ └─MergeJoin 12475.01 root inner join, left key:test.t1.b, right key:test.t.b", + "│ ├─Sort(Build) 9980.01 root test.t.b", + "│ │ └─HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#23)]", + "│ │ ├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#23", + "│ │ │ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#22)]", + "│ │ │ ├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#22", + "│ │ │ │ └─HashJoin 12487.50 root inner join, equal:[eq(test.t1.b, test.t2.b)]", + "│ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ │ └─TableReader(Probe) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", + "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─Sort(Probe) 9990.00 root test.t1.b", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─Sort(Probe) 9990.00 root test.t.a", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warn": null } @@ -4801,12 +4805,12 @@ "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -4838,19 +4842,19 @@ "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -4871,19 +4875,19 @@ "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -4904,26 +4908,26 @@ "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -4946,19 +4950,19 @@ "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ │ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ │ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "│ │ └─TableReader(Probe) 9980.01 root data:Selection", "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", @@ -4986,26 +4990,26 @@ "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5022,12 +5026,12 @@ "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5047,12 +5051,12 @@ "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5077,19 +5081,19 @@ "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5111,19 +5115,19 @@ "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5146,12 +5150,12 @@ "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5170,12 +5174,12 @@ "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5198,19 +5202,19 @@ "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5231,19 +5235,19 @@ "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5268,32 +5272,32 @@ "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warn": [ - "Hint merge_join(`t1`@`qb_v1_2`, `t`@`qb_v1_1`) is ignored due to unknown query block name" + "Only one query block name is allowed in a view hint, otherwise the hint will be invalid" ] }, { @@ -5321,19 +5325,19 @@ "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5365,19 +5369,19 @@ "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5419,12 +5423,12 @@ "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "└─Sort(Probe) 9990.00 root test.t.a", " └─TableReader 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", @@ -5453,12 +5457,12 @@ "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -5479,12 +5483,12 @@ "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "│ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ │ │ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ │ │ └─TableReader(Probe) 9990.00 root data:Selection", - "│ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", "│ │ └─Sort(Probe) 9980.01 root test.t.a", "│ │ └─TableReader 9980.01 root data:Selection", "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", @@ -5523,26 +5527,710 @@ "Warn": null }, { - "SQL": "explain format = 'brief' select /*+ merge_join(t@qb_v_1), stream_agg(@qb_v_2), qb_name(qb_v_2, v@sel_1 .@sel_2), qb_name(qb_v_1, v@sel_1 .@sel1) */ * from v;", + "SQL": "explain format = 'brief' select /*+ merge_join(t@qb_v_1), stream_agg(@qb_v_2), qb_name(qb_v_2, v@sel_1 .@sel_2), qb_name(qb_v_1, v@sel_1 .@sel_1) */ * from v;", "Plan": [ - "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#13)]", - "├─StreamAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#13", - "│ └─Sort 15593.77 root test.t2.a", - "│ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", - "│ ├─TableReader(Build) 9990.00 root data:Selection", - "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", - "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "MergeJoin 9980.01 root inner join, left key:test.t.a, right key:Column#13", + "├─Sort(Build) 7984.01 root Column#13", + "│ └─StreamAgg 7984.01 root group by:test.t2.a, funcs:count(1)->Column#13", + "│ └─Sort 15593.77 root test.t2.a", + "│ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─Sort(Probe) 9990.00 root test.t.a", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v3_2, v3@sel_1 .@sel_2), merge_join(t1@qb_v3_2) */ * from v3;", + "Plan": [ + "MergeJoin 9980.01 root inner join, left key:test.t.a, right key:Column#27", + "├─Sort(Build) 7984.01 root Column#27", + "│ └─StreamAgg 7984.01 root group by:test.t.a, funcs:count(1)->Column#27", + "│ └─Sort 12475.01 root test.t.a", + "│ └─MergeJoin 12475.01 root inner join, left key:test.t1.b, right key:test.t.b", + "│ ├─Sort(Build) 9980.01 root test.t.b", + "│ │ └─HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#26)]", + "│ │ ├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#26", + "│ │ │ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#25)]", + "│ │ │ ├─HashAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#25", + "│ │ │ │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + "│ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ │ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + "│ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", + "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─Sort(Probe) 9990.00 root test.t1.b", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─Sort(Probe) 9990.00 root test.t.a", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v3_2, v3@sel_1 .@sel_2), merge_join(t1@qb_v3_2), hash_agg(@qb_v3_2), qb_name(qb_v3_1, v3@sel_1 .@sel_1), hash_join(t@qb_v3_1) */ * from v3;", + "Plan": [ + "HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#27)]", + "├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#27", + "│ └─MergeJoin 12475.01 root inner join, left key:test.t1.b, right key:test.t.b", + "│ ├─Sort(Build) 9980.01 root test.t.b", + "│ │ └─HashJoin 9980.01 root inner join, equal:[eq(test.t.a, Column#26)]", + "│ │ ├─HashAgg(Build) 7984.01 root group by:test.t.a, funcs:count(1)->Column#26", + "│ │ │ └─HashJoin 12475.01 root inner join, equal:[eq(test.t.a, Column#25)]", + "│ │ │ ├─HashAgg(Build) 7984.01 root group by:test.t2.a, funcs:count(1)->Column#25", + "│ │ │ │ └─HashJoin 15593.77 root inner join, equal:[eq(test.t2.a, test.t3.a)]", + "│ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.a))", + "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ │ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + "│ │ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ │ │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.b, test.t1.b)]", + "│ │ │ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ │ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ │ └─TableReader(Probe) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─Sort(Probe) 9990.00 root test.t1.b", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join", + "[planner:1815]Optimizer aggregation hints are conflicted" + ] + } + ] + }, + { + "Name": "TestAllViewHintType", + "Cases": [ + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), leading(@qb_v1 v, t2) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t.a, test.t1.a)]", + "├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t3.a, test.t.a)]", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), leading(v@qb_v1, t2@qb_v1) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t.a, test.t1.a)]", + "├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t3.a, test.t.a)]", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), leading(@qb_v1 t3, t2) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t.a, test.t1.a)]", + "├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t3.a, test.t.a)]", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), leading(t3@qb_v1, t2@qb_v1) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t.a, test.t1.a)]", + "├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t3.a, test.t.a)]", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), qb_name(qb_v, v1.v), leading(t2@qb_v1, t@qb_v) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t.a, test.t1.a)]", + "├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:false, stats:pseudo", + "└─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t3.a, test.t.a)]", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "Only one query block name is allowed in a view hint, otherwise the hint will be invalid" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join(@qb_v1 v, t2) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t3.a, test.t.a)]", + "├─MergeJoin(Build) 12500.00 root inner join, left key:test.t.a, right key:test.t1.a", + "│ ├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo", + "│ └─IndexReader(Probe) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx_a(a) keep order:true, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join(t2@qb_v1, t3@qb_v1) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t.a, test.t3.a)]", + "├─MergeJoin(Build) 12500.00 root inner join, left key:test.t.a, right key:test.t1.a", + "│ ├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo", + "│ └─IndexReader(Probe) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx_a(a) keep order:true, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join_build(@qb_v1 v) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t3.a, test.t.a)]", + "├─MergeJoin(Build) 12500.00 root inner join, left key:test.t.a, right key:test.t1.a", + "│ ├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo", + "│ └─IndexReader(Probe) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx_a(a) keep order:true, stats:pseudo", + "└─Projection(Probe) 12475.01 root test.t2.a, test.t2.b, test.t3.a", + " └─HashJoin 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join_build(t2@qb_v1) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t.a, test.t3.a)]", + "├─MergeJoin(Build) 12500.00 root inner join, left key:test.t.a, right key:test.t1.a", + "│ ├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo", + "│ └─IndexReader(Probe) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx_a(a) keep order:true, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join_build(@qb_v1 v) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t3.a, test.t.a)]", + "├─MergeJoin(Build) 12500.00 root inner join, left key:test.t.a, right key:test.t1.a", + "│ ├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo", + "│ └─IndexReader(Probe) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx_a(a) keep order:true, stats:pseudo", + "└─Projection(Probe) 12475.01 root test.t2.a, test.t2.b, test.t3.a", + " └─HashJoin 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), hash_join_build(t2@qb_v1) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t.a, test.t3.a)]", + "├─MergeJoin(Build) 12500.00 root inner join, left key:test.t.a, right key:test.t1.a", + "│ ├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo", + "│ └─IndexReader(Probe) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx_a(a) keep order:true, stats:pseudo", + "└─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t3.b)]", + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), merge_join(@qb_v1 v) */ * from v1;", + "Plan": [ + "MergeJoin 19492.21 root inner join, left key:test.t3.a, right key:test.t.a", + "├─MergeJoin(Build) 12500.00 root inner join, left key:test.t.a, right key:test.t1.a", + "│ ├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo", + "│ └─IndexReader(Probe) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx_a(a) keep order:true, stats:pseudo", + "└─Sort(Probe) 12475.01 root test.t3.a", + " └─Projection 12475.01 root test.t2.a, test.t2.b, test.t3.a", + " └─HashJoin 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v1, v1), merge_join(t2@qb_v1) */ * from v1;", + "Plan": [ + "HashJoin 19492.21 root inner join, equal:[eq(test.t.a, test.t3.a)]", + "├─MergeJoin(Build) 12500.00 root inner join, left key:test.t.a, right key:test.t1.a", + "│ ├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo", + "│ └─IndexReader(Probe) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx_a(a) keep order:true, stats:pseudo", + "└─MergeJoin(Probe) 12475.01 root inner join, left key:test.t2.b, right key:test.t3.b", + " ├─Sort(Build) 9980.01 root test.t3.b", + " │ └─TableReader 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + " └─Sort(Probe) 9990.00 root test.t2.b", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v, v), INL_JOIN(@qb_v t) */ * from v;", + "Plan": [ + "IndexJoin 12500.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t.a, equal cond:eq(test.t1.a, test.t.a)", + "├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 12500.00 root ", + " ├─IndexRangeScan(Build) 12500.00 cop[tikv] table:t, index:idx_a(a) range: decided by [eq(test.t.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 12500.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v, v), INL_JOIN(t@qb_v) */ * from v;", + "Plan": [ + "IndexJoin 12500.00 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t.a, equal cond:eq(test.t1.a, test.t.a)", + "├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 12500.00 root ", + " ├─IndexRangeScan(Build) 12500.00 cop[tikv] table:t, index:idx_a(a) range: decided by [eq(test.t.a, test.t1.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 12500.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v2, v2.@sel_2), hash_agg(@qb_v2) */ * from v2;", + "Plan": [ + "HashJoin 9990.00 root inner join, equal:[eq(test.t.a, Column#19)]", + "├─HashAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#19", + "│ └─HashJoin 24365.26 root inner join, equal:[eq(test.t.a, test.t1.a)]", + "│ ├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t3.a, test.t.a)]", + "│ ├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v2, v2.@sel_2), stream_agg(@qb_v2) */ * from v2;", + "Plan": [ + "HashJoin 9990.00 root inner join, equal:[eq(test.t.a, Column#19)]", + "├─StreamAgg(Build) 7992.00 root group by:test.t2.a, funcs:count(1)->Column#19", + "│ └─Sort 24365.26 root test.t2.a", + "│ └─HashJoin 24365.26 root inner join, equal:[eq(test.t.a, test.t1.a)]", + "│ ├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 19492.21 root inner join, equal:[eq(test.t3.a, test.t.a)]", + "│ ├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t2.b, test.t1.b)]", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t3.b, test.t2.b)]", + "│ ├─TableReader(Build) 9980.01 root data:Selection", + "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v3, v3), use_index(t5@qb_v3, idx_a) */ * from v3;", + "Plan": [ + "IndexLookUp 1107.78 root ", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t5, index:idx_a(a) range:(1,+inf], keep order:false, stats:pseudo", + "└─Selection(Probe) 1107.78 cop[tikv] lt(test.t5.b, 2)", + " └─TableRowIDScan 3333.33 cop[tikv] table:t5 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v3, v3), use_index(@qb_v3 t5, idx_b) */ * from v3;", + "Plan": [ + "IndexLookUp 1107.78 root ", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t5, index:idx_b(b) range:[-inf,2), keep order:false, stats:pseudo", + "└─Selection(Probe) 1107.78 cop[tikv] gt(test.t5.a, 1)", + " └─TableRowIDScan 3323.33 cop[tikv] table:t5 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v3, v3), force_index(t5@qb_v3, idx_a) */ * from v3;", + "Plan": [ + "IndexLookUp 1107.78 root ", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t5, index:idx_a(a) range:(1,+inf], keep order:false, stats:pseudo", + "└─Selection(Probe) 1107.78 cop[tikv] lt(test.t5.b, 2)", + " └─TableRowIDScan 3333.33 cop[tikv] table:t5 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v3, v3), force_index(@qb_v3 t5, idx_b) */ * from v3;", + "Plan": [ + "IndexLookUp 1107.78 root ", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t5, index:idx_b(b) range:[-inf,2), keep order:false, stats:pseudo", + "└─Selection(Probe) 1107.78 cop[tikv] gt(test.t5.a, 1)", + " └─TableRowIDScan 3323.33 cop[tikv] table:t5 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v3, v3), ignore_index(t5@qb_v3, idx_a) */ * from v3;", + "Plan": [ + "TableReader 1107.78 root data:Selection", + "└─Selection 1107.78 cop[tikv] gt(test.t5.a, 1), lt(test.t5.b, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v3, v3), ignore_index(@qb_v3 t5, idx_b) */ * from v3;", + "Plan": [ + "TableReader 1107.78 root data:Selection", + "└─Selection 1107.78 cop[tikv] gt(test.t5.a, 1), lt(test.t5.b, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:t5 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v4, v4), use_index_merge(t5@qb_v4, idx_a, idx_b) */ * from v4;", + "Plan": [ + "IndexMerge 5548.89 root ", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t5, index:idx_a(a) range:(1,+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t5, index:idx_b(b) range:[-inf,2), keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 5548.89 cop[tikv] table:t5 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v4, v4), use_index_merge(@qb_v4 t5, idx_b, idx_a) */ * from v4;", + "Plan": [ + "IndexMerge 5548.89 root ", + "├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t5, index:idx_a(a) range:(1,+inf], keep order:false, stats:pseudo", + "├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t5, index:idx_b(b) range:[-inf,2), keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 5548.89 cop[tikv] table:t5 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v, v), READ_FROM_STORAGE(TIFLASH[t@qb_v], TIKV[t1@qb_v]) */ * from v;", + "Plan": [ + "HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.t1.a)]", + "├─IndexReader(Build) 10000.00 root index:IndexFullScan", + "│ └─IndexFullScan 10000.00 cop[tikv] table:t1, index:idx_a(a) keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v5, v5.@sel_2), SEMI_JOIN_REWRITE(@qb_v5) */ * from v5;", + "Plan": [ + "HashJoin 9990.00 root inner join, equal:[eq(test.t.b, test.t1.b)]", + "├─HashAgg(Build) 7992.00 root group by:test.t1.b, funcs:firstrow(test.t1.b)->test.t1.b", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t1.b, ", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v6, v6.@sel_2), NO_DECORRELATE(@qb_v6) */ * from v6;", + "Plan": [ + "Projection 10000.00 root test.t1.a, test.t1.b", + "└─Apply 10000.00 root CARTESIAN inner join, other cond:lt(cast(test.t1.a, decimal(10,0) BINARY), Column#7)", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─MaxOneRow(Probe) 10000.00 root ", + " └─HashAgg 10000.00 root funcs:sum(Column#8)->Column#7", + " └─TableReader 10000.00 root data:HashAgg", + " └─HashAgg 10000.00 cop[tikv] funcs:sum(test.t2.a)->Column#8", + " └─Selection 100000.00 cop[tikv] eq(test.t2.b, test.t1.b)", + " └─TableFullScan 100000000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v7, v7), merge(@qb_v7) */ * from v7;", + "Plan": [ + "TableReader 3544.89 root data:ExchangeSender", + "└─ExchangeSender 3544.89 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 3544.89 mpp[tiflash] Column#14, Column#15", + " └─HashAgg 3544.89 mpp[tiflash] group by:Column#14, Column#15, funcs:firstrow(Column#14)->Column#14, funcs:firstrow(Column#15)->Column#15", + " └─ExchangeReceiver 3544.89 mpp[tiflash] ", + " └─ExchangeSender 3544.89 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: binary], [name: Column#15, collate: binary]", + " └─HashAgg 3544.89 mpp[tiflash] group by:Column#14, Column#15, ", + " └─Union 4431.11 mpp[tiflash] ", + " ├─Projection 3323.33 mpp[tiflash] cast(test.t.a, int(11) BINARY)->Column#14, test.t.b", + " │ └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 18), lt(test.t.a, 60)", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 1107.78 mpp[tiflash] cast(test.t.a, int(11) BINARY)->Column#14, test.t.b", + " └─Selection 1107.78 mpp[tiflash] gt(test.t.b, 1), lt(test.t.a, 60)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v8, v8), merge(@qb_v8) */ * from v8;", + "Plan": [ + "HashAgg 16000.00 root group by:Column#21, funcs:firstrow(Column#21)->Column#21", + "└─Union 1000000010000.00 root ", + " ├─HashJoin 1000000000000.00 root CARTESIAN inner join", + " │ ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " │ └─CTEFullScan(Probe) 100000000.00 root CTE:cte2 data:CTE_1", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "CTE_1 100000000.00 root Non-Recursive CTE", + "└─HashJoin(Seed Part) 100000000.00 root CARTESIAN inner join", + " ├─CTEFullScan(Build) 10000.00 root CTE:cte4 data:CTE_3", + " └─CTEFullScan(Probe) 10000.00 root CTE:cte3 data:CTE_2", + "CTE_3 10000.00 root Non-Recursive CTE", + "└─IndexReader(Seed Part) 10000.00 root index:IndexFullScan", + " └─IndexFullScan 10000.00 cop[tikv] table:t3, index:idx_a(a) keep order:false, stats:pseudo", + "CTE_2 10000.00 root Non-Recursive CTE", + "└─IndexReader(Seed Part) 10000.00 root index:IndexFullScan", + " └─IndexFullScan 10000.00 cop[tikv] table:t2, index:idx_a(a) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v9, v9), AGG_TO_COP(@qb_v9) */ * from v9;", + "Plan": [ + "HashAgg 1.00 root funcs:sum(Column#6)->Column#4", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:sum(Column#9)->Column#6", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#9", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb_v10, v10), LIMIT_TO_COP(@qb_v10) */ * from v10;", + "Plan": [ + "TopN 1.00 root test.t.b, offset:0, count:1", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─TopN 1.00 mpp[tiflash] test.t.b, offset:0, count:1", + " └─Selection 3333.33 mpp[tiflash] gt(test.t.a, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb, v11) read_from_storage(tiflash[t@qb]), MPP_1PHASE_AGG(@qb) */ * from v11;", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#4", + " └─Projection 8000.00 mpp[tiflash] Column#4, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#9, funcs:sum(Column#7)->Column#4, funcs:firstrow(Column#8)->test.t.a", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#7, test.t.a, test.t.a", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb, v11) read_from_storage(tiflash[t@qb]), MPP_2PHASE_AGG(@qb) */ * from v11;", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#4", + " └─Projection 8000.00 mpp[tiflash] Column#4, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, funcs:sum(Column#9)->Column#4, funcs:firstrow(test.t.a)->test.t.a", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#12, funcs:sum(Column#11)->Column#9", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#11, test.t.a", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb, v12) read_from_storage(tiflash[t1@qb, t@qb]), shuffle_join(t1@qb, t@qb) */ * from v12;", + "Plan": [ + "Projection 12500.00 root test.t.a, test.t.b", + "└─TableReader 12500.00 root data:ExchangeSender", + " └─ExchangeSender 12500.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ qb_name(qb, v12) read_from_storage(tiflash[t1@qb, t@qb]), broadcast_join(t1@qb, t@qb) */ * from v12;", + "Plan": [ + "TableReader 12500.00 root data:ExchangeSender", + "└─ExchangeSender 12500.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12500.00 mpp[tiflash] test.t.a, test.t.b", + " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", + " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], "Warn": null } ] diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/testdata/partition_pruner_out.json index c16109a17bbc2..7b454564bece9 100644 --- a/planner/core/testdata/partition_pruner_out.json +++ b/planner/core/testdata/partition_pruner_out.json @@ -905,9 +905,9 @@ "IndexPlan": [ "Projection 1.00 root Column#5", "└─Sort 1.00 root test_partition_1.t1.id, test_partition_1.t1.a", - " └─StreamAgg 1.00 root funcs:count(Column#13)->Column#5, funcs:firstrow(Column#14)->test_partition_1.t1.id, funcs:firstrow(Column#15)->test_partition_1.t1.a", - " └─IndexReader 1.00 root partition:all index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#13, funcs:firstrow(test_partition_1.t1.id)->Column#14, funcs:firstrow(test_partition_1.t1.a)->Column#15", + " └─HashAgg 1.00 root funcs:count(Column#8)->Column#5, funcs:firstrow(Column#9)->test_partition_1.t1.id, funcs:firstrow(Column#10)->test_partition_1.t1.a", + " └─IndexReader 1.00 root partition:all index:HashAgg", + " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#8, funcs:firstrow(test_partition_1.t1.id)->Column#9, funcs:firstrow(test_partition_1.t1.a)->Column#10", " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a, b, id) keep order:false, stats:pseudo" ] }, @@ -941,9 +941,9 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "StreamAgg 1.00 root funcs:count(Column#10)->Column#5", - "└─IndexReader 1.00 root partition:p0 index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#10", + "HashAgg 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader 1.00 root partition:p0 index:HashAgg", + " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#7", " └─Selection 19.99 cop[tikv] or(eq(test_partition_1.t1.a, 1), eq(test_partition_1.t1.b, 2))", " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a, b, id) keep order:false, stats:pseudo" ] @@ -1646,7 +1646,7 @@ ], "IndexPlan": [ "Sort 199.80 root test_partition_1.t1.a", - "└─IndexHashJoin 199.80 root inner join, inner:IndexReader, outer key:test_partition_1.t2.b, inner key:test_partition_1.t1.a, equal cond:eq(test_partition_1.t2.b, test_partition_1.t1.a)", + "└─IndexJoin 199.80 root inner join, inner:IndexReader, outer key:test_partition_1.t2.b, inner key:test_partition_1.t1.a, equal cond:eq(test_partition_1.t2.b, test_partition_1.t1.a)", " ├─HashAgg(Build) 159.84 root group by:test_partition_1.t2.b, funcs:firstrow(test_partition_1.t2.b)->test_partition_1.t2.b", " │ └─IndexReader 159.84 root partition:p0 index:HashAgg", " │ └─HashAgg 159.84 cop[tikv] group by:test_partition_1.t2.b, ", @@ -1676,7 +1676,7 @@ ], "IndexPlan": [ "Sort 199.80 root test_partition_1.t1.a", - "└─IndexHashJoin 199.80 root inner join, inner:IndexReader, outer key:test_partition_1.t1.b, inner key:test_partition_1.t1.a, equal cond:eq(test_partition_1.t1.b, test_partition_1.t1.a)", + "└─IndexJoin 199.80 root inner join, inner:IndexReader, outer key:test_partition_1.t1.b, inner key:test_partition_1.t1.a, equal cond:eq(test_partition_1.t1.b, test_partition_1.t1.a)", " ├─HashAgg(Build) 159.84 root group by:test_partition_1.t1.b, funcs:firstrow(test_partition_1.t1.b)->test_partition_1.t1.b", " │ └─IndexReader 159.84 root partition:p0 index:HashAgg", " │ └─HashAgg 159.84 cop[tikv] group by:test_partition_1.t1.b, ", @@ -1877,7 +1877,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "IndexPlan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#9", + "HashAgg 1.00 root funcs:count(1)->Column#9", "└─IndexHashJoin 0.41 root inner join, inner:IndexReader, outer key:test_partition_1.t2.b, inner key:test_partition_1.t1.b, equal cond:eq(test_partition_1.t2.b, test_partition_1.t1.b)", " ├─IndexReader(Build) 0.80 root partition:p0 index:Selection", " │ └─Selection 0.80 cop[tikv] not(isnull(test_partition_1.t2.b))", @@ -1948,9 +1948,9 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "StreamAgg 1.00 root funcs:count(Column#10)->Column#5", - "└─IndexReader 1.00 root partition:p0 index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#10", + "HashAgg 1.00 root funcs:count(Column#7)->Column#5", + "└─IndexReader 1.00 root partition:p0 index:HashAgg", + " └─HashAgg 1.00 cop[tikv] funcs:count(1)->Column#7", " └─Selection 5542.21 cop[tikv] or(lt(test_partition_1.t1.a, 1), lt(test_partition_1.t1.b, 2))", " └─IndexFullScan 10000.00 cop[tikv] table:t1, index:a(a, b, id) keep order:false, stats:pseudo" ] @@ -2799,7 +2799,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#9", + "HashAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 7.81 root inner join, equal:[eq(test_partition_1.t1.b, test_partition_1.t2.b)]", " ├─IndexReader(Build) 6.25 root partition:p0,p1 index:Selection", " │ └─Selection 6.25 cop[tikv] ge(test_partition_1.t2.b, 1), le(test_partition_1.t2.b, 6), not(isnull(test_partition_1.t2.b))", @@ -2825,7 +2825,7 @@ " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "IndexPlan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#9", + "HashAgg 1.00 root funcs:count(1)->Column#9", "└─HashJoin 7.81 root inner join, equal:[eq(test_partition_1.t2.b, test_partition_1.t1.b)]", " ├─IndexReader(Build) 6.25 root partition:p0,p1 index:Selection", " │ └─Selection 6.25 cop[tikv] ge(test_partition_1.t1.b, 1), le(test_partition_1.t1.b, 6), not(isnull(test_partition_1.t1.b))", diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 58be6097c2bc0..e983b81f0a976 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -68,6 +68,15 @@ type UserPrivileges struct { user string host string *Handle + extensionAccessCheckFuncs []extension.AccessCheckFunc +} + +// NewUserPrivileges creates a new UserPrivileges +func NewUserPrivileges(handle *Handle, extension *extension.Extensions) *UserPrivileges { + return &UserPrivileges{ + Handle: handle, + extensionAccessCheckFuncs: extension.GetAccessCheckFuncs(), + } } // RequestDynamicVerificationWithUser implements the Manager interface. @@ -128,7 +137,8 @@ func (p *UserPrivileges) RequestVerification(activeRoles []*auth.RoleIdentity, d tblLowerName := strings.ToLower(table) // If SEM is enabled and the user does not have the RESTRICTED_TABLES_ADMIN privilege // There are some hard rules which overwrite system tables and schemas as read-only at most. - if sem.IsEnabled() && !p.RequestDynamicVerification(activeRoles, "RESTRICTED_TABLES_ADMIN", false) { + semEnabled := sem.IsEnabled() + if semEnabled && !p.RequestDynamicVerification(activeRoles, "RESTRICTED_TABLES_ADMIN", false) { if sem.IsInvisibleTable(dbLowerName, tblLowerName) { return false } @@ -158,6 +168,14 @@ func (p *UserPrivileges) RequestVerification(activeRoles []*auth.RoleIdentity, d } } + for _, fn := range p.extensionAccessCheckFuncs { + for _, dynPriv := range fn(db, table, column, priv, semEnabled) { + if !p.RequestDynamicVerification(activeRoles, dynPriv, false) { + return false + } + } + } + mysqlPriv := p.Handle.Get() return mysqlPriv.RequestVerification(activeRoles, p.user, p.host, db, table, column, priv) } diff --git a/server/rpc_server.go b/server/rpc_server.go index 4dcad639ad4a7..f92deaf802d64 100644 --- a/server/rpc_server.go +++ b/server/rpc_server.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/extension" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" @@ -216,11 +217,13 @@ func (s *rpcServer) createSession() (session.Session, error) { if err != nil { return nil, err } + extensions, err := extension.GetExtensions() + if err != nil { + return nil, err + } do := domain.GetDomain(se) is := do.InfoSchema() - pm := &privileges.UserPrivileges{ - Handle: do.PrivilegeHandle(), - } + pm := privileges.NewUserPrivileges(do.PrivilegeHandle(), extensions) privilege.BindPrivilegeManager(se, pm) vars := se.GetSessionVars() vars.TxnCtx.InfoSchema = is diff --git a/session/bootstrap.go b/session/bootstrap.go index 08bf0293db72c..57660af7fa808 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -447,6 +447,7 @@ const ( plan_digest VARCHAR(128) NOT NULL, update_time TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, PRIMARY KEY (sql_digest,plan_digest));` + // CreateStatsTableLocked stores the locked tables CreateStatsTableLocked = `CREATE TABLE IF NOT EXISTS mysql.stats_table_locked( table_id bigint(64) NOT NULL, diff --git a/session/session.go b/session/session.go index 680ddda98ddcd..1b254c879ec40 100644 --- a/session/session.go +++ b/session/session.go @@ -2711,9 +2711,11 @@ func CreateSessionWithOpt(store kv.Storage, opt *Opt) (Session, error) { if err != nil { return nil, err } - pm := &privileges.UserPrivileges{ - Handle: do.PrivilegeHandle(), + extensions, err := extension.GetExtensions() + if err != nil { + return nil, err } + pm := privileges.NewUserPrivileges(do.PrivilegeHandle(), extensions) privilege.BindPrivilegeManager(s, pm) // Add stats collector, and it will be freed by background stats worker @@ -2876,7 +2878,7 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { analyzeConcurrencyQuota := int(config.GetGlobalConfig().Performance.AnalyzePartitionConcurrencyQuota) concurrency := int(config.GetGlobalConfig().Performance.StatsLoadConcurrency) - ses, err := createSessions(store, 8) + ses, err := createSessions(store, 9) if err != nil { return nil, err } @@ -2951,10 +2953,10 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { } // setup plan replayer handle - dom.SetupPlanReplayerHandle(ses[6]) + dom.SetupPlanReplayerHandle(ses[6], ses[7]) dom.StartPlanReplayerHandle() // setup dumpFileGcChecker - dom.SetupDumpFileGCChecker(ses[7]) + dom.SetupDumpFileGCChecker(ses[8]) dom.DumpFileGcCheckerLoop() // A sub context for update table stats, and other contexts for concurrent stats loading. diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index b11b727079630..b8fbcf54848e1 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1305,6 +1305,9 @@ type SessionVars struct { // preuseChunkAlloc indicates whether pre statement use chunk alloc // like select @@last_sql_use_alloc preUseChunkAlloc bool + + // EnablePlanReplayerCapture indicates whether enabled plan replayer capture + EnablePlanReplayerCapture bool } // GetNewChunkWithCapacity Attempt to request memory from the chunk pool diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index a592e7a7e8831..d73885663d957 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1039,15 +1039,17 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return fmt.Sprintf("%d", MemoryUsageAlarmKeepRecordNum.Load()), nil }}, - {Scope: ScopeGlobal, Name: TiDBEnablePlanReplayerCapture, Value: BoolToOnOff(false), Type: TypeBool, - SetGlobal: func(ctx context.Context, s *SessionVars, val string) error { - EnablePlanReplayerCapture.Store(TiDBOptOn(val)) - return nil - }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { - return strconv.FormatBool(EnablePlanReplayerCapture.Load()), nil - }}, /* The system variables below have GLOBAL and SESSION scope */ + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePlanReplayerCapture, Value: BoolToOnOff(false), Type: TypeBool, + SetSession: func(s *SessionVars, val string) error { + s.EnablePlanReplayerCapture = TiDBOptOn(val) + return nil + }, + GetSession: func(vars *SessionVars) (string, error) { + return strconv.FormatBool(vars.EnablePlanReplayerCapture), nil + }, + }, {Scope: ScopeGlobal | ScopeSession, Name: TiDBRowFormatVersion, Value: strconv.Itoa(DefTiDBRowFormatV1), Type: TypeUnsigned, MinValue: 1, MaxValue: 2, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { SetDDLReorgRowFormat(TidbOptInt64(val, DefTiDBRowFormatV2)) return nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 80c9b41f4cc6e..3511775de08f1 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -1148,9 +1148,8 @@ var ( // DefTiDBServerMemoryLimit indicates the default value of TiDBServerMemoryLimit(TotalMem * 80%). // It should be a const and shouldn't be modified after tidb is started. - DefTiDBServerMemoryLimit = serverMemoryLimitDefaultValue() - GOGCTunerThreshold = atomic.NewFloat64(DefTiDBGOGCTunerThreshold) - EnablePlanReplayerCapture = atomic.NewBool(DefTiDBEnablePlanReplayerCapture) + DefTiDBServerMemoryLimit = serverMemoryLimitDefaultValue() + GOGCTunerThreshold = atomic.NewFloat64(DefTiDBGOGCTunerThreshold) ) var ( diff --git a/sessionctx/variable/variable.go b/sessionctx/variable/variable.go index 3acecb6b83483..4b7faa09481c8 100644 --- a/sessionctx/variable/variable.go +++ b/sessionctx/variable/variable.go @@ -164,6 +164,8 @@ type SysVar struct { // If the global variable has the global config name, // it should store the global config into PD(etcd) too when set global variable. GlobalConfigName string + // RequireDynamicPrivileges is a function to return a dynamic privilege list to check the set sysvar privilege + RequireDynamicPrivileges func(isGlobal bool, sem bool) []string } // GetGlobalFromHook calls the GetSession func if it exists. diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 7dfc3f47d2d62..344ac0f72c755 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -775,7 +775,17 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context, allPartitionStats[partitionID] = partitionStats } for i := 0; i < globalStats.Num; i++ { - _, hg, cms, topN, fms := partitionStats.GetStatsInfo(histIDs[i], isIndex == 1) + _, hg, cms, topN, fms, analyzed := partitionStats.GetStatsInfo(histIDs[i], isIndex == 1) + if !analyzed { + var errMsg string + if isIndex == 0 { + errMsg = fmt.Sprintf("`%s`", tableInfo.Name.L) + } else { + errMsg = fmt.Sprintf("`%s` index: `%s`", tableInfo.Name.L, tableInfo.FindIndexNameByID(histIDs[0])) + } + err = types.ErrPartitionStatsMissing.GenWithStackByArgs(errMsg) + return + } // partition stats is not empty but column stats(hist, topn) is missing if partitionStats.Count > 0 && (hg == nil || hg.TotalRowCount() <= 0) && (topN == nil || topN.TotalCount() <= 0) { var errMsg string diff --git a/statistics/table.go b/statistics/table.go index 2e66e39ab8152..20bc5d38b0a5e 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -377,19 +377,19 @@ func (t *Table) ColumnByName(colName string) *Column { } // GetStatsInfo returns their statistics according to the ID of the column or index, including histogram, CMSketch, TopN and FMSketch. -func (t *Table) GetStatsInfo(ID int64, isIndex bool) (int64, *Histogram, *CMSketch, *TopN, *FMSketch) { +func (t *Table) GetStatsInfo(ID int64, isIndex bool) (int64, *Histogram, *CMSketch, *TopN, *FMSketch, bool) { if isIndex { if idxStatsInfo, ok := t.Indices[ID]; ok { - return int64(idxStatsInfo.TotalRowCount()), idxStatsInfo.Histogram.Copy(), idxStatsInfo.CMSketch.Copy(), idxStatsInfo.TopN.Copy(), idxStatsInfo.FMSketch.Copy() + return int64(idxStatsInfo.TotalRowCount()), idxStatsInfo.Histogram.Copy(), idxStatsInfo.CMSketch.Copy(), idxStatsInfo.TopN.Copy(), idxStatsInfo.FMSketch.Copy(), true } // newly added index which is not analyzed yet - return 0, nil, nil, nil, nil + return 0, nil, nil, nil, nil, false } if colStatsInfo, ok := t.Columns[ID]; ok { - return int64(colStatsInfo.TotalRowCount()), colStatsInfo.Histogram.Copy(), colStatsInfo.CMSketch.Copy(), colStatsInfo.TopN.Copy(), colStatsInfo.FMSketch.Copy() + return int64(colStatsInfo.TotalRowCount()), colStatsInfo.Histogram.Copy(), colStatsInfo.CMSketch.Copy(), colStatsInfo.TopN.Copy(), colStatsInfo.FMSketch.Copy(), true } // newly added column which is not analyzed yet - return 0, nil, nil, nil, nil + return 0, nil, nil, nil, nil, false } // GetColRowCount tries to get the row count of the a column if possible. diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index 92c3b535ba5d3..054ec83ee7e8a 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -2028,7 +2028,6 @@ func (w *GCWorker) doGCPlacementRules(se session.Session, safePoint uint64, dr u zap.Int64("tableID", id), zap.String("endKey", string(dr.EndKey)), zap.Uint64("safePoint", safePoint)) ruleID := fmt.Sprintf("table-%v-r", id) if err := infosync.DeleteTiFlashPlacementRule(context.Background(), "tiflash", ruleID); err != nil { - // If DeletePlacementRule fails here, the rule will be deleted in `HandlePlacementRuleRoutine`. logutil.BgLogger().Error("delete TiFlash pd rule failed when gc", zap.Error(err), zap.String("ruleID", ruleID), zap.Uint64("safePoint", safePoint)) } else { diff --git a/telemetry/data_feature_usage_test.go b/telemetry/data_feature_usage_test.go index 12b682b571b39..f2d212f39b41b 100644 --- a/telemetry/data_feature_usage_test.go +++ b/telemetry/data_feature_usage_test.go @@ -422,7 +422,7 @@ func TestCostModelVer2UsageInfo(t *testing.T) { tk := testkit.NewTestKit(t, store) usage, err := telemetry.GetFeatureUsage(tk.Session()) require.NoError(t, err) - require.False(t, usage.EnableCostModelVer2) + require.Equal(t, usage.EnableCostModelVer2, variable.DefTiDBCostModelVer == 2) tk.Session().GetSessionVars().CostModelVersion = 2 usage, err = telemetry.GetFeatureUsage(tk.Session()) diff --git a/util/hint/hint_processor.go b/util/hint/hint_processor.go index 47613a429d99a..b087981abd409 100644 --- a/util/hint/hint_processor.go +++ b/util/hint/hint_processor.go @@ -273,6 +273,13 @@ func ParseHintsSet(p *parser.Parser, sql, charset, collation, db string) (*Hints } for _, tblHint := range tblHints { if tblHint.HintName.L == hintQBName { + if len(tblHint.Tables) > 0 { + newHints = append(newHints, tblHint) + } + continue + } + if processor.isHint4View(tblHint) { + newHints = append(newHints, tblHint) continue } offset := processor.GetHintOffset(tblHint.QBName, curOffset) @@ -444,6 +451,10 @@ func (p *BlockHintProcessor) handleViewHints(hints []*ast.TableOptimizerHint) (l break } } + if !ok { + p.Ctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("Only one query block name is allowed in a view hint, otherwise the hint will be invalid")) + usedHints[i] = true + } } } @@ -539,6 +550,25 @@ func (p *BlockHintProcessor) checkTableQBName(tables []ast.HintTable) bool { return true } +func (p *BlockHintProcessor) isHint4View(hint *ast.TableOptimizerHint) bool { + if hint.QBName.L != "" { + if p.QbNameMap4View != nil { + _, ok := p.QbNameMap4View[hint.QBName.L] + return ok + } + return false + } + allViewHints := true + for _, table := range hint.Tables { + qbName := table.QBName.L + if _, ok := p.QbNameMap4View[qbName]; !ok { + allViewHints = false + break + } + } + return allViewHints +} + // GetCurrentStmtHints extracts all hints that take effects at current stmt. func (p *BlockHintProcessor) GetCurrentStmtHints(hints []*ast.TableOptimizerHint, currentOffset int) []*ast.TableOptimizerHint { if p.QbHints == nil { diff --git a/util/logutil/log.go b/util/logutil/log.go index 83eb44eb9d2cc..a7d59ae0c2d0f 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -111,37 +111,30 @@ func InitLogger(cfg *LogConfig) error { return errors.Trace(err) } - _, _, err = initGRPCLogger(cfg) - if err != nil { - return errors.Trace(err) - } - + initGRPCLogger(gl) return nil } -func initGRPCLogger(cfg *LogConfig) (*zap.Logger, *log.ZapProperties, error) { - // Copy Config struct by assignment. - config := cfg.Config - var l *zap.Logger - var err error - var prop *log.ZapProperties +func initGRPCLogger(gl *zap.Logger) { + level := zapcore.ErrorLevel + verbosity := 0 if len(os.Getenv("GRPC_DEBUG")) > 0 { - config.Level = "debug" - l, prop, err = log.InitLogger(&config, zap.AddStacktrace(zapcore.FatalLevel)) - if err != nil { - return nil, nil, errors.Trace(err) - } - gzap.ReplaceGrpcLoggerV2WithVerbosity(l, 999) - } else { - config.Level = "error" - l, prop, err = log.InitLogger(&config, zap.AddStacktrace(zapcore.FatalLevel)) - if err != nil { - return nil, nil, errors.Trace(err) - } - gzap.ReplaceGrpcLoggerV2(l) + verbosity = 999 + level = zapcore.DebugLevel } - return l, prop, nil + newgl := gl.WithOptions(zap.WrapCore(func(core zapcore.Core) zapcore.Core { + oldcore, ok := core.(*log.TextIOCore) + if !ok { + return oldcore + } + newcore := oldcore.Clone() + leveler := zap.NewAtomicLevel() + leveler.SetLevel(level) + newcore.LevelEnabler = leveler + return newcore + })) + gzap.ReplaceGrpcLoggerV2WithVerbosity(newgl, verbosity) } // ReplaceLogger replace global logger instance with given log config. diff --git a/util/logutil/log_test.go b/util/logutil/log_test.go index d059204973678..57a7786c0e530 100644 --- a/util/logutil/log_test.go +++ b/util/logutil/log_test.go @@ -100,21 +100,6 @@ func TestSetLevel(t *testing.T) { require.Equal(t, zap.DebugLevel, log.GetLevel()) } -func TestGrpcLoggerCreation(t *testing.T) { - level := "info" - conf := NewLogConfig(level, DefaultLogFormat, "", EmptyFileLogConfig, false) - _, p, err := initGRPCLogger(conf) - // assert after init grpc logger, the original conf is not changed - require.Equal(t, conf.Level, level) - require.NoError(t, err) - require.Equal(t, p.Level.Level(), zap.ErrorLevel) - os.Setenv("GRPC_DEBUG", "1") - defer os.Unsetenv("GRPC_DEBUG") - _, newP, err := initGRPCLogger(conf) - require.NoError(t, err) - require.Equal(t, newP.Level.Level(), zap.DebugLevel) -} - func TestSlowQueryLoggerCreation(t *testing.T) { level := "Error" conf := NewLogConfig(level, DefaultLogFormat, "", EmptyFileLogConfig, false)