diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index b0b06cad85f3b..75dc998d69ffd 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -495,15 +495,10 @@ func (*Domain) fetchSchemasWithTables(ctx context.Context, schemas []*model.DBIn }) for _, di := range schemas { - // if the ctx has been canceled, stop fetch schemas. + // if the ctx has been canceled, stop fetching schemas. if err := ctx.Err(); err != nil { return err } - - if di.State != model.StatePublic { - // schema is not public, can't be used outside. - continue - } var tables []*model.TableInfo var err error if variable.SchemaCacheSize.Load() > 0 && !infoschema.IsSpecialDB(di.Name.L) { @@ -527,10 +522,6 @@ func (*Domain) fetchSchemasWithTables(ctx context.Context, schemas []*model.DBIn } diTables := make([]*model.TableInfo, 0, len(tables)) for _, tbl := range tables { - if tbl.State != model.StatePublic { - // schema is not public, can't be used outside. - continue - } infoschema.ConvertCharsetCollateToLowerCaseIfNeed(tbl) // Check whether the table is in repair mode. if domainutil.RepairInfo.InRepairMode() && domainutil.RepairInfo.CheckAndFetchRepairedTable(di, tbl) { diff --git a/pkg/executor/batch_point_get.go b/pkg/executor/batch_point_get.go index 1a4a8d344a4fc..261c95459b4e9 100644 --- a/pkg/executor/batch_point_get.go +++ b/pkg/executor/batch_point_get.go @@ -168,11 +168,15 @@ func (e *BatchPointGetExec) Close() error { if e.RuntimeStats() != nil && e.snapshot != nil { e.snapshot.SetOption(kv.CollectRuntimeStats, nil) } - if e.indexUsageReporter != nil && e.idxInfo != nil { + if e.indexUsageReporter != nil { kvReqTotal := e.stats.GetCmdRPCCount(tikvrpc.CmdBatchGet) // We cannot distinguish how many rows are coming from each partition. Here, we calculate all index usages // percentage according to the row counts for the whole table. - e.indexUsageReporter.ReportPointGetIndexUsage(e.tblInfo.ID, e.tblInfo.ID, e.idxInfo.ID, e.ID(), kvReqTotal) + if e.idxInfo != nil { + e.indexUsageReporter.ReportPointGetIndexUsage(e.tblInfo.ID, e.tblInfo.ID, e.idxInfo.ID, e.ID(), kvReqTotal) + } else { + e.indexUsageReporter.ReportPointGetIndexUsageForHandle(e.tblInfo, e.tblInfo.ID, e.ID(), kvReqTotal) + } } e.inited = 0 e.index = 0 diff --git a/pkg/executor/builder.go b/pkg/executor/builder.go index 56d5dc9115f5e..9fd6e36a77a22 100644 --- a/pkg/executor/builder.go +++ b/pkg/executor/builder.go @@ -3468,6 +3468,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea e := &TableReaderExecutor{ BaseExecutorV2: exec.NewBaseExecutorV2(b.ctx.GetSessionVars(), v.Schema(), v.ID()), tableReaderExecutorContext: newTableReaderExecutorContext(b.ctx), + indexUsageReporter: b.buildIndexUsageReporter(v), dagPB: dagReq, startTS: startTS, txnScope: b.txnScope, @@ -4168,13 +4169,13 @@ type tableStatsPreloader interface { LoadTableStats(sessionctx.Context) } -func (b *executorBuilder) buildIndexUsageReporter(plan tableStatsPreloader) (indexUsageReporter *exec.IndexUsageReporter) { - sc := b.ctx.GetSessionVars().StmtCtx - if b.ctx.GetSessionVars().StmtCtx.IndexUsageCollector != nil && +func buildIndexUsageReporter(ctx sessionctx.Context, plan tableStatsPreloader) (indexUsageReporter *exec.IndexUsageReporter) { + sc := ctx.GetSessionVars().StmtCtx + if ctx.GetSessionVars().StmtCtx.IndexUsageCollector != nil && sc.RuntimeStatsColl != nil { // Preload the table stats. If the statement is a point-get or execute, the planner may not have loaded the // stats. - plan.LoadTableStats(b.ctx) + plan.LoadTableStats(ctx) statsMap := sc.GetUsedStatsInfo(false) indexUsageReporter = exec.NewIndexUsageReporter( @@ -4185,6 +4186,10 @@ func (b *executorBuilder) buildIndexUsageReporter(plan tableStatsPreloader) (ind return indexUsageReporter } +func (b *executorBuilder) buildIndexUsageReporter(plan tableStatsPreloader) (indexUsageReporter *exec.IndexUsageReporter) { + return buildIndexUsageReporter(b.ctx, plan) +} + func (b *executorBuilder) buildIndexMergeReader(v *plannercore.PhysicalIndexMergeReader) exec.Executor { ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) if err := b.validCanReadTemporaryOrCacheTable(ts.Table); err != nil { diff --git a/pkg/executor/index_merge_reader.go b/pkg/executor/index_merge_reader.go index ce6629f46e47e..78c95001138b3 100644 --- a/pkg/executor/index_merge_reader.go +++ b/pkg/executor/index_merge_reader.go @@ -928,12 +928,12 @@ func (e *IndexMergeReaderExecutor) Close() error { } if e.indexUsageReporter != nil { for _, p := range e.partialPlans { - is, ok := p[0].(*plannercore.PhysicalIndexScan) - if !ok { - continue + switch p := p[0].(type) { + case *plannercore.PhysicalTableScan: + e.indexUsageReporter.ReportCopIndexUsageForHandle(e.table, p.ID()) + case *plannercore.PhysicalIndexScan: + e.indexUsageReporter.ReportCopIndexUsageForTable(e.table, p.Index.ID, p.ID()) } - - e.indexUsageReporter.ReportCopIndexUsageForTable(e.table, is.Index.ID, is.ID()) } } if e.finished == nil { diff --git a/pkg/executor/infoschema_reader.go b/pkg/executor/infoschema_reader.go index 9ee22fc54f4a9..6f99a26070582 100644 --- a/pkg/executor/infoschema_reader.go +++ b/pkg/executor/infoschema_reader.go @@ -750,7 +750,7 @@ func (e *memtableRetriever) setDataFromTables(ctx context.Context, sctx sessionc if !ex.HasTableSchema(t.DBName.L) { return true } - if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, t.DBName.L, t.TableName.L, "", mysql.SelectPriv) { + if checker != nil && !checker.RequestVerification(sctx.GetSessionVars().ActiveRoles, t.DBName.L, t.TableName.L, "", mysql.AllPrivMask) { return true } diff --git a/pkg/executor/internal/exec/BUILD.bazel b/pkg/executor/internal/exec/BUILD.bazel index c7dbd065af020..4f998b4731132 100644 --- a/pkg/executor/internal/exec/BUILD.bazel +++ b/pkg/executor/internal/exec/BUILD.bazel @@ -11,6 +11,7 @@ go_library( deps = [ "//pkg/domain", "//pkg/expression", + "//pkg/meta/model", "//pkg/parser", "//pkg/sessionctx", "//pkg/sessionctx/stmtctx", @@ -36,7 +37,7 @@ go_test( timeout = "short", srcs = ["indexusage_test.go"], flaky = True, - shard_count = 5, + shard_count = 6, deps = [ ":exec", "//pkg/domain", diff --git a/pkg/executor/internal/exec/indexusage.go b/pkg/executor/internal/exec/indexusage.go index 95a0b832a4b62..223149042844d 100644 --- a/pkg/executor/internal/exec/indexusage.go +++ b/pkg/executor/internal/exec/indexusage.go @@ -15,6 +15,7 @@ package exec import ( + "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" "github.com/pingcap/tidb/pkg/statistics" "github.com/pingcap/tidb/pkg/statistics/handle/usage/indexusage" @@ -40,6 +41,17 @@ func NewIndexUsageReporter(reporter *indexusage.StmtIndexUsageCollector, } } +// ReportCopIndexUsageForHandle wraps around `ReportCopIndexUsageForTable` to get the `indexID` automatically +// from the `table.Table` if the table has a clustered index or integer primary key. +func (e *IndexUsageReporter) ReportCopIndexUsageForHandle(tbl table.Table, planID int) { + idxID, ok := getClusterIndexID(tbl.Meta()) + if !ok { + return + } + + e.ReportCopIndexUsageForTable(tbl, idxID, planID) +} + // ReportCopIndexUsageForTable wraps around `ReportCopIndexUsage` to get `tableID` and `physicalTableID` from the // `table.Table`. If it's expected to calculate the percentage according to the size of partition, the `tbl` argument // should be a `table.PhysicalTable`, or the percentage will be calculated using the size of whole table. @@ -75,6 +87,17 @@ func (e *IndexUsageReporter) ReportCopIndexUsage(tableID int64, physicalTableID e.reporter.Update(tableID, indexID, sample) } +// ReportPointGetIndexUsageForHandle wraps around `ReportPointGetIndexUsage` to get the `indexID` automatically +// from the `table.Table` if the table has a clustered index or integer primary key. +func (e *IndexUsageReporter) ReportPointGetIndexUsageForHandle(tblInfo *model.TableInfo, physicalTableID int64, planID int, kvRequestTotal int64) { + idxID, ok := getClusterIndexID(tblInfo) + if !ok { + return + } + + e.ReportPointGetIndexUsage(tblInfo.ID, physicalTableID, idxID, planID, kvRequestTotal) +} + // ReportPointGetIndexUsage reports the index usage of a point get or batch point get func (e *IndexUsageReporter) ReportPointGetIndexUsage(tableID int64, physicalTableID int64, indexID int64, planID int, kvRequestTotal int64) { tableRowCount, ok := e.getTableRowCount(physicalTableID) @@ -104,3 +127,23 @@ func (e *IndexUsageReporter) getTableRowCount(tableID int64) (int64, bool) { } return stats.RealtimeCount, true } + +// getClusterIndexID returns the indexID of the clustered index. If the table doesn't have a clustered index, it returns +// (0, false). +func getClusterIndexID(tblInfo *model.TableInfo) (int64, bool) { + var idxID int64 + if tblInfo.PKIsHandle { + idxID = 0 + } else if tblInfo.IsCommonHandle { + for _, idx := range tblInfo.Indices { + if idx.Primary { + idxID = idx.ID + } + } + } else { + // just ignore, this table is read through rowid. + return 0, false + } + + return idxID, true +} diff --git a/pkg/executor/internal/exec/indexusage_test.go b/pkg/executor/internal/exec/indexusage_test.go index 4377d39f25cb7..0f97bc4dd887b 100644 --- a/pkg/executor/internal/exec/indexusage_test.go +++ b/pkg/executor/internal/exec/indexusage_test.go @@ -447,3 +447,104 @@ func TestDisableIndexUsageReporter(t *testing.T) { time.Sleep(time.Millisecond * 100) } } + +func TestIndexUsageReporterWithClusterIndex(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t0 (id int primary key, a int)") + tk.MustExec("create table t1 (id char(255) primary key, a int)") + tk.MustExec("create table t2 (id char(255) primary key nonclustered, a int)") + tk.MustExec("create table t3 (id int primary key, a int, unique key idx_a(a))") + + type testTableInfo struct { + tableID int64 + pkID int64 + extraIdxID int64 + } + testTableInfos := []testTableInfo{} + for i := 0; i < 4; i++ { + table, err := dom.InfoSchema().TableByName(context.Background(), model.NewCIStr("test"), model.NewCIStr(fmt.Sprintf("t%d", i))) + require.NoError(t, err) + tableID := table.Meta().ID + pkID := int64(0) + extraIdxID := int64(0) + for _, idx := range table.Indices() { + if idx.Meta().Primary { + pkID = idx.Meta().ID + } else { + extraIdxID = idx.Meta().ID + } + } + testTableInfos = append(testTableInfos, testTableInfo{tableID, pkID, extraIdxID}) + } + + for i := 0; i < 4; i++ { + for val := 0; val < 100; val++ { + tk.MustExec(fmt.Sprintf("insert into t%d values (?, ?)", i), val, val) + } + tk.MustExec(fmt.Sprintf("analyze table t%d", i)) + } + tk.RefreshSession() + tk.MustExec("use test") + + cases := []testCase{ + // TableReader on PKAsHandle + { + "select id from t0 where id >= 30", + "TableReader", + []indexStatsExpect{{testTableInfos[0].tableID, testTableInfos[0].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 70, 100)}}}, + }, + // TableReader on CommonHandle + { + "select id from t1 where id >= \"30\"", + "TableReader", + // It'll scan 76 rows according to the string order + []indexStatsExpect{{testTableInfos[1].tableID, testTableInfos[1].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 76, 100)}}}, + }, + // IndexRangeScan on NonClustered PK + { + "select id from t2 where id >= \"30\"", + "IndexRangeScan", + // It'll scan 76 rows according to the string order + []indexStatsExpect{{testTableInfos[2].tableID, testTableInfos[2].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 76, 100)}}}, + }, + // IndexMerge on PK and a normal Unique Key + { + "select /*+ USE_INDEX_MERGE(t3) */ * from t3 where id >= 30 or id < 5 or a >= 50", + "IndexMerge", + // It'll scan 76 rows according to the string order + []indexStatsExpect{ + {testTableInfos[3].tableID, testTableInfos[3].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 70, 100)}}, + {testTableInfos[3].tableID, testTableInfos[3].pkID, []indexusage.Sample{indexusage.NewSample(0, 1, 5, 100)}}, + {testTableInfos[3].tableID, testTableInfos[3].extraIdxID, []indexusage.Sample{indexusage.NewSample(1, 1, 50, 100)}}, + }, + }, + // PointGet on PKAsHandle + { + "select * from t0 where id = 1", + "Point_Get", + []indexStatsExpect{{testTableInfos[0].tableID, testTableInfos[0].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 1, 100)}}}, + }, + // PointGet on CommonHandle + { + "select * from t1 where id = \"1\"", + "Point_Get", + []indexStatsExpect{{testTableInfos[1].tableID, testTableInfos[1].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 1, 100)}}}, + }, + // BatchPointGet on PKAsHandle + { + "select * from t0 where id in (1,3,5,9)", + "Batch_Point_Get", + []indexStatsExpect{{testTableInfos[0].tableID, testTableInfos[0].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 4, 100)}}}, + }, + // BatchPointGet on CommonHandle + { + "select * from t1 where id in (\"1\",\"3\",\"5\",\"9\")", + "Batch_Point_Get", + []indexStatsExpect{{testTableInfos[1].tableID, testTableInfos[1].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 4, 100)}}}, + }, + } + + runIndexUsageTestCases(t, dom, tk, append(cases, wrapTestCaseWithPrepare(cases)...)) +} diff --git a/pkg/executor/join/base_join_probe.go b/pkg/executor/join/base_join_probe.go index db3f3fb8dacc2..7088122d05e73 100644 --- a/pkg/executor/join/base_join_probe.go +++ b/pkg/executor/join/base_join_probe.go @@ -364,10 +364,8 @@ func (j *baseJoinProbe) appendBuildRowToChunkInternal(chk *chunk.Chunk, usedCols var currentColumn *chunk.Column if ok { currentColumn = chk.Column(indexInDstChk) - // Other goroutine will use `atomic.StoreUint32` to write to the first 32 bit in nullmap when it need to set usedFlag - // so read from nullMap may meet concurrent write if meta.colOffsetInNullMap == 1 && (columnIndex + meta.colOffsetInNullMap <= 32) - mayConcurrentWrite := meta.colOffsetInNullMap == 1 && columnIndex <= 31 - if !mayConcurrentWrite { + readNullMapThreadSafe := meta.isReadNullMapThreadSafe(columnIndex) + if readNullMapThreadSafe { for index := range j.cachedBuildRows { currentColumn.AppendNullBitmap(!meta.isColumnNull(*(*unsafe.Pointer)(unsafe.Pointer(&j.cachedBuildRows[index].buildRowStart)), columnIndex)) j.cachedBuildRows[index].buildRowOffset = chunk.AppendCellFromRawData(currentColumn, *(*unsafe.Pointer)(unsafe.Pointer(&j.cachedBuildRows[index].buildRowStart)), j.cachedBuildRows[index].buildRowOffset) diff --git a/pkg/executor/join/join_row_table.go b/pkg/executor/join/join_row_table.go index 73d70bae9a426..2cdd9ff8caa2a 100644 --- a/pkg/executor/join/join_row_table.go +++ b/pkg/executor/join/join_row_table.go @@ -218,6 +218,13 @@ func (meta *TableMeta) getSerializedKeyLength(rowStart unsafe.Pointer) uint64 { return *(*uint64)(unsafe.Add(rowStart, sizeOfNextPtr+meta.nullMapLength)) } +func (meta *TableMeta) isReadNullMapThreadSafe(columnIndex int) bool { + // Other goroutine will use `atomic.StoreUint32` to write to the first 32 bit in nullmap when it need to set usedFlag + // so read from nullMap may meet concurrent write if meta.colOffsetInNullMap == 1 && (columnIndex + meta.colOffsetInNullMap < 32) + mayConcurrentWrite := meta.colOffsetInNullMap == 1 && columnIndex < 31 + return !mayConcurrentWrite +} + // used in tests func (meta *TableMeta) getKeyBytes(rowStart unsafe.Pointer) []byte { switch meta.keyMode { diff --git a/pkg/executor/join/join_row_table_test.go b/pkg/executor/join/join_row_table_test.go index f82e64765fc5b..721f572e939b6 100644 --- a/pkg/executor/join/join_row_table_test.go +++ b/pkg/executor/join/join_row_table_test.go @@ -184,6 +184,20 @@ func TestJoinTableMetaKeyInlinedAndFixed(t *testing.T) { } } +func TestReadNullMapThreadSafe(t *testing.T) { + // meta with usedFlag + tinyTp := types.NewFieldType(mysql.TypeTiny) + metaWithUsedFlag := newTableMeta([]int{0}, []*types.FieldType{tinyTp}, []*types.FieldType{tinyTp}, []*types.FieldType{tinyTp}, nil, []int{}, true) + for columnIndex := 0; columnIndex < 100; columnIndex++ { + require.Equal(t, columnIndex >= 31, metaWithUsedFlag.isReadNullMapThreadSafe(columnIndex)) + } + // meta without usedFlag + metaWithoutUsedFlag := newTableMeta([]int{0}, []*types.FieldType{tinyTp}, []*types.FieldType{tinyTp}, []*types.FieldType{tinyTp}, nil, []int{}, false) + for columnIndex := 0; columnIndex < 100; columnIndex++ { + require.Equal(t, true, metaWithoutUsedFlag.isReadNullMapThreadSafe(columnIndex)) + } +} + func TestJoinTableMetaSerializedMode(t *testing.T) { intTp := types.NewFieldType(mysql.TypeLonglong) uintTp := types.NewFieldType(mysql.TypeLonglong) diff --git a/pkg/executor/point_get.go b/pkg/executor/point_get.go index 4849040ac79e9..9cd754984248f 100644 --- a/pkg/executor/point_get.go +++ b/pkg/executor/point_get.go @@ -217,6 +217,11 @@ func (e *PointGetExecutor) Init(p *plannercore.PointGetPlan) { e.partitionDefIdx = p.PartitionIdx e.columns = p.Columns e.buildVirtualColumnInfo() + + // It's necessary to at least reset the `runtimeStats` of the `BaseExecutor`. + // As the `StmtCtx` may have changed, a new index usage reporter should also be created. + e.BaseExecutor = exec.NewBaseExecutor(e.Ctx(), p.Schema(), p.ID()) + e.indexUsageReporter = buildIndexUsageReporter(e.Ctx(), p) } // buildVirtualColumnInfo saves virtual column indices and sort them in definition order @@ -252,11 +257,15 @@ func (e *PointGetExecutor) Close() error { if e.RuntimeStats() != nil && e.snapshot != nil { e.snapshot.SetOption(kv.CollectRuntimeStats, nil) } - if e.indexUsageReporter != nil && e.idxInfo != nil { + if e.indexUsageReporter != nil { tableID := e.tblInfo.ID physicalTableID := GetPhysID(e.tblInfo, e.partitionDefIdx) kvReqTotal := e.stats.SnapshotRuntimeStats.GetCmdRPCCount(tikvrpc.CmdGet) - e.indexUsageReporter.ReportPointGetIndexUsage(tableID, physicalTableID, e.idxInfo.ID, e.ID(), kvReqTotal) + if e.idxInfo != nil { + e.indexUsageReporter.ReportPointGetIndexUsage(tableID, physicalTableID, e.idxInfo.ID, e.ID(), kvReqTotal) + } else { + e.indexUsageReporter.ReportPointGetIndexUsageForHandle(e.tblInfo, physicalTableID, e.ID(), kvReqTotal) + } } e.done = false return nil diff --git a/pkg/executor/table_reader.go b/pkg/executor/table_reader.go index 0c5e497b9de5a..349efdb6394c6 100644 --- a/pkg/executor/table_reader.go +++ b/pkg/executor/table_reader.go @@ -135,6 +135,7 @@ func newTableReaderExecutorContext(sctx sessionctx.Context) tableReaderExecutorC type TableReaderExecutor struct { tableReaderExecutorContext exec.BaseExecutorV2 + indexUsageReporter *exec.IndexUsageReporter table table.Table @@ -341,6 +342,10 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error // Close implements the Executor Close interface. func (e *TableReaderExecutor) Close() error { + if e.indexUsageReporter != nil { + e.indexUsageReporter.ReportCopIndexUsageForHandle(e.table, e.plans[0].ID()) + } + var err error if e.resultHandler != nil { err = e.resultHandler.Close() diff --git a/pkg/planner/core/physical_plans.go b/pkg/planner/core/physical_plans.go index ab26132ad6676..0476d60df92e1 100644 --- a/pkg/planner/core/physical_plans.go +++ b/pkg/planner/core/physical_plans.go @@ -155,6 +155,12 @@ type PhysicalTableReader struct { TableScanAndPartitionInfos []tableScanAndPartitionInfo `plan-cache-clone:"must-nil"` } +// LoadTableStats loads the stats of the table read by this plan. +func (p *PhysicalTableReader) LoadTableStats(ctx sessionctx.Context) { + ts := p.TablePlans[0].(*PhysicalTableScan) + loadTableStats(ctx, ts.Table, ts.physicalTableID) +} + // PhysPlanPartInfo indicates partition helper info in physical plan. type PhysPlanPartInfo struct { PruningConds []expression.Expression diff --git a/pkg/ttl/cache/split_test.go b/pkg/ttl/cache/split_test.go index 89020947ad9f1..f62a2a19dec27 100644 --- a/pkg/ttl/cache/split_test.go +++ b/pkg/ttl/cache/split_test.go @@ -517,9 +517,10 @@ func TestSplitTTLScanRangesWithBytes(t *testing.T) { createTTLTable(t, tk, "t3", "varchar(32) CHARACTER SET BINARY"), createTTLTable(t, tk, "t4", "bit(32)"), create2PKTTLTable(t, tk, "t5", "binary(32)"), - createTTLTable(t, tk, "t6", "char(32) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin"), - createTTLTable(t, tk, "t7", "char(32) CHARACTER SET utf8 COLLATE utf8_bin"), - create2PKTTLTable(t, tk, "t8", "char(32) CHARACTER SET utf8mb4 COLLATE utf8mb4_0900_bin"), + createTTLTable(t, tk, "t6", "varbinary(32)"), + createTTLTable(t, tk, "t7", "char(32) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin"), + createTTLTable(t, tk, "t8", "char(32) CHARACTER SET utf8 COLLATE utf8_bin"), + create2PKTTLTable(t, tk, "t9", "char(32) CHARACTER SET utf8mb4 COLLATE utf8mb4_0900_bin"), } cases := []struct {