Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

*: Enable plan cache for partitioned tables | tidb-test=pr/2273 #49161

Merged
merged 145 commits into from Mar 12, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
145 commits
Select commit Hold shift + click to select a range
698c4eb
Just remove the limitations, to check what may break...
mjonss Dec 4, 2023
e6186a9
make bazel_prepare, added test..
mjonss Dec 4, 2023
cfe5b52
Added test
mjonss Dec 4, 2023
1f36506
First step to support single column PK+partitioned table (no expression)
mjonss Dec 4, 2023
71dc3d4
Linting
mjonss Dec 5, 2023
63abee1
Updated tests
mjonss Dec 5, 2023
01bc1e6
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Dec 5, 2023
d2ec635
WIP, some progress with point get and re-prune partitions
mjonss Dec 27, 2023
7f1a7b5
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Dec 28, 2023
6774483
Linting
mjonss Dec 28, 2023
342bb1b
Fixed some tests and blocking prepared plan cache if static mode was …
mjonss Jan 2, 2024
b012bdb
Removed renaming of variables due to #49989
mjonss Jan 2, 2024
3340e87
Reverted name change of partDef/Info due to #49989.
mjonss Jan 3, 2024
ce40f2c
Removed bad vim insert...
mjonss Jan 3, 2024
8bb7cb7
Removed some debug panics
mjonss Jan 3, 2024
ddb6a64
Added analyze table, to enable dynamic prune mode in tests.
mjonss Jan 3, 2024
ce22e4d
Added handling of batch_point_get for IndexValues.
mjonss Jan 3, 2024
7a4d6fe
Added rebuild range for table scans and partitioned tables.
mjonss Jan 4, 2024
b40153a
Handle issue with no matching partitions.
mjonss Jan 4, 2024
13353f8
Adjusted TestPartitionWithVariedDataSources test
mjonss Jan 4, 2024
e399b22
Linting / bazel_prepare
mjonss Jan 4, 2024
8a6c6a3
Consider a single partition full range still as a single partition
mjonss Jan 4, 2024
5a320c1
BuildRangeForIndexScan can skip pruning, since it is done later
mjonss Jan 4, 2024
cd74bc2
Re-evaluating ParamMarker in-place and updated some tests
mjonss Jan 5, 2024
b956eca
Updated test, global statistics needs to exists for plan cache of par…
mjonss Jan 5, 2024
9d15c48
Some cleanups
mjonss Jan 5, 2024
07c5229
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Jan 5, 2024
fe4f256
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Jan 7, 2024
0af4b92
Cleaning up
mjonss Jan 8, 2024
2196da4
small cleanups of TODOs
mjonss Jan 25, 2024
0986615
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Jan 25, 2024
f360070
Cleanup and test updates.
mjonss Jan 25, 2024
8e9f7c0
Added test, and fixed issue with hash partition col not first
mjonss Jan 27, 2024
29e8812
Linting
mjonss Jan 28, 2024
43beddd
bazel_prepare
mjonss Jan 28, 2024
021d27c
Linting
mjonss Jan 29, 2024
877a9ad
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Jan 29, 2024
26fa208
Manual merge fix, changed interface{} to any
mjonss Jan 29, 2024
d7d3d61
Linting
mjonss Jan 29, 2024
0bd604c
Cleanup and more tests
mjonss Jan 29, 2024
b1b170a
Test cleanup
mjonss Feb 2, 2024
c9f6b01
Added Fix-control and cleanup
mjonss Feb 5, 2024
69e4017
Minor test updates
mjonss Jan 29, 2024
be49dd2
Check if partitioned table instead of previous pruning result
mjonss Jan 30, 2024
1eeac56
Linting and cleanups
mjonss Feb 5, 2024
3de79c8
WIP 1
mjonss Feb 6, 2024
5774626
wip2
mjonss Feb 6, 2024
ae4e24e
wip
mjonss Feb 12, 2024
5a26bdf
wip, one step further, but messy code
mjonss Feb 12, 2024
e74b6f1
Using LocatePartition instead of Pruning for PointGet
mjonss Feb 12, 2024
08b802a
Updated tests
mjonss Feb 13, 2024
73c1a2d
Updated test
mjonss Feb 13, 2024
c42cf7f
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Feb 13, 2024
e93aeb8
Moved Batch Point Get partition pruning to build phase
mjonss Feb 13, 2024
2d0329e
Cleanup
mjonss Feb 13, 2024
b4360b2
Cleanups
mjonss Feb 13, 2024
d81a894
More cleanups
mjonss Feb 13, 2024
44865b9
Linting
mjonss Feb 13, 2024
30c150b
Fixed BatchPointGet and some cleanups
mjonss Feb 14, 2024
f6babd5
Fixed secondary key lookups for batch get and cleanups
mjonss Feb 14, 2024
062cf39
minor fixes and cleanups
mjonss Feb 14, 2024
8d1d553
Missesd a case for PointGet on int handle
mjonss Feb 14, 2024
7e5f851
Cleanups
mjonss Feb 14, 2024
6115428
Improved BatchPointGet and partition pruning + explain
mjonss Feb 14, 2024
aa0cc95
Linting
mjonss Feb 15, 2024
2dc15fc
Test result update
mjonss Feb 15, 2024
e0d3d2d
Treat any pruning error in PointGet as impossible partition
mjonss Feb 15, 2024
df5377c
wip
mjonss Feb 15, 2024
00138f3
PointGet fix
mjonss Feb 15, 2024
9c9b8fa
Added explicit partition selection to PointGet
mjonss Feb 15, 2024
04a4003
bazel prepare
mjonss Feb 15, 2024
38c8933
Test restult update
mjonss Feb 15, 2024
0ea1a66
Updated test
mjonss Feb 15, 2024
e0db338
Updated test results
mjonss Feb 15, 2024
6308b24
Fixed Point Get for global index
mjonss Feb 15, 2024
c667cb7
Updated test results
mjonss Feb 15, 2024
2df095a
updated test results
mjonss Feb 15, 2024
eeb4a15
Updated test results
mjonss Feb 15, 2024
a0ebde1
fix for global index
mjonss Feb 15, 2024
5f44fa0
Updated test, explain now starts a transaction
mjonss Feb 15, 2024
d1fe102
Updated test
mjonss Feb 15, 2024
cc201b9
Updated point get to fetch the correct index statistics
mjonss Feb 15, 2024
4b08ffb
Updated tests
mjonss Feb 15, 2024
ac8bc07
Updated test results
mjonss Feb 15, 2024
6d86e0b
Fixed test
mjonss Feb 15, 2024
7e45260
Updated test
mjonss Feb 15, 2024
3f2fd78
updated test results
mjonss Feb 15, 2024
5a57548
Fixed assert
mjonss Feb 15, 2024
167ca13
Test updated
mjonss Feb 16, 2024
2b4da02
Added support for explicit partition selection with global index
mjonss Feb 16, 2024
b2c4daa
Delayed enabling more partitioning types for BatchPointGet
mjonss Feb 16, 2024
a273abc
Test updates
mjonss Feb 16, 2024
a291d7a
Updated test
mjonss Feb 16, 2024
bb2bb03
Updated test results
mjonss Feb 16, 2024
000161e
Updated tests
mjonss Feb 16, 2024
d404082
Update test results
mjonss Feb 16, 2024
99f1c47
Updated test results
mjonss Feb 16, 2024
126a26e
Test update
mjonss Feb 17, 2024
65fecf9
Updated test
mjonss Feb 17, 2024
adaa834
Updated test results
mjonss Feb 17, 2024
e03c721
Test update
mjonss Feb 18, 2024
419beb1
Updated comments
mjonss Feb 18, 2024
ac46e40
WIP, will not compile, temporary commit for moving to other computer
mjonss Feb 18, 2024
e394473
Code reshuffle and fixed prefetch for partitioned tables.
mjonss Feb 19, 2024
fde70e0
setting planPhysIDs in buildBatchPointGetExec
mjonss Feb 19, 2024
f4384cf
Propagating more fields from plan to BatchPointGetExec
mjonss Feb 19, 2024
2ec2d36
Linting
mjonss Feb 19, 2024
b3226b1
Missed not...
mjonss Feb 19, 2024
e1095b7
Updated capacity (missed if only idxVals).
mjonss Feb 19, 2024
8158fb8
Single partition bug
mjonss Feb 20, 2024
e32ef0d
Test result update
mjonss Feb 20, 2024
0e19256
Test result updated
mjonss Feb 20, 2024
e062ef2
Test result updated
mjonss Feb 20, 2024
9fda2d9
Test result updated
mjonss Feb 20, 2024
4160625
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Mar 1, 2024
bb1f2af
Fixed case where plan cache had less handles than HandleParams
mjonss Mar 1, 2024
b1305dd
Fixed test case
mjonss Mar 1, 2024
10f87ed
Added temporary tests
mjonss Mar 1, 2024
e9d74d6
Resetting lenght of IndexValues if prev run removed values.
mjonss Mar 4, 2024
0bd8a28
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Mar 4, 2024
02a6c79
Minor code refactoring / renameing
mjonss Mar 4, 2024
dfaeedc
Updated test result
mjonss Mar 4, 2024
99ff082
Added back some limitations for BatchPointGet
mjonss Mar 5, 2024
9c0eb80
Test update
mjonss Mar 5, 2024
4392493
Updated test
mjonss Mar 5, 2024
dc98212
Reverted removal of failpoint in test, to remove noise in PR
mjonss Mar 5, 2024
fbbdba4
Cleanup and test refinement
mjonss Mar 6, 2024
bbb2a0f
Refined test
mjonss Mar 6, 2024
0114d32
Refined test
mjonss Mar 6, 2024
bd4f7a2
Test refinement
mjonss Mar 6, 2024
fcb7950
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Mar 6, 2024
7218194
Removed reEvaluateParamMarker which is no longer needed.
mjonss Mar 7, 2024
84edeb3
Fixed review comments
mjonss Mar 7, 2024
52d6713
Fixed point_get during truncate partition and global index
mjonss Mar 8, 2024
410329c
Addressed review comments
mjonss Mar 8, 2024
e73e022
Review comments fixed
mjonss Mar 8, 2024
72390c1
Refined a loop
mjonss Mar 8, 2024
13b81af
Cleaned up a loop
mjonss Mar 8, 2024
60219a0
Review comments
mjonss Mar 8, 2024
924a51d
Fixed review comments
mjonss Mar 8, 2024
b1a5480
Updated according to review comments
mjonss Mar 8, 2024
71790ee
Accepting duplicate warnings in agreement with reviewer
mjonss Mar 8, 2024
6ccbe69
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Mar 8, 2024
8c573c9
Merge remote-tracking branch 'pingcap/master' into part-enable-prep-p…
mjonss Mar 11, 2024
f3c6807
Linting after manual merge conflict
mjonss Mar 11, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
11 changes: 9 additions & 2 deletions pkg/ddl/tests/partition/db_partition_test.go
Expand Up @@ -1455,12 +1455,15 @@ func TestTruncatePartitionWithGlobalIndex(t *testing.T) {
tk3 := testkit.NewTestKit(t, store)
tk3.MustExec(`begin`)
tk3.MustExec(`use test`)
tk3.MustQuery(`explain format='brief' select b from test_global use index(idx_b) where b = 15`).CheckContain("IndexRangeScan")
tk3.MustQuery(`explain format='brief' select c from test_global use index(idx_c) where c = 15`).CheckContain("IndexRangeScan")
tk3.MustQuery(`select b from test_global use index(idx_b) where b = 15`).Check(testkit.Rows())
tk3.MustQuery(`select c from test_global use index(idx_c) where c = 15`).Check(testkit.Rows())
// Here it will fail with
// the partition is not in public.
err := tk3.ExecToErr(`insert into test_global values (15,15,15)`)
assert.NotNil(t, err)
require.Error(t, err)
require.ErrorContains(t, err, "the partition is in not in public")
tk2.MustExec(`commit`)
tk3.MustExec(`commit`)
<-syncChan
Expand All @@ -1477,6 +1480,10 @@ func TestTruncatePartitionWithGlobalIndex(t *testing.T) {
require.NotNil(t, idxInfo)
cnt = checkGlobalIndexCleanUpDone(t, tk.Session(), tt.Meta(), idxInfo, pid)
require.Equal(t, 3, cnt)
tk.MustQuery(`select b from test_global use index(idx_b) where b = 15`).Check(testkit.Rows())
tk.MustQuery(`select c from test_global use index(idx_c) where c = 15`).Check(testkit.Rows())
tk3.MustQuery(`explain format='brief' select b from test_global use index(idx_b) where b = 15`).CheckContain("Point_Get")
tk3.MustQuery(`explain format='brief' select c from test_global use index(idx_c) where c = 15`).CheckContain("Point_Get")
}

func TestGlobalIndexUpdateInTruncatePartition(t *testing.T) {
Expand Down Expand Up @@ -3438,7 +3445,7 @@ func TestExchangeValidateHandleNullValue(t *testing.T) {
tk.MustExec(`alter table t3 EXCHANGE PARTITION p0 WITH TABLE t4`)

tk.MustExec(`CREATE TABLE t5 (id int, c varchar(128)) partition by range (id)(
partition p0 values less than (10),
partition p0 values less than (10),
partition p1 values less than (20),
partition p2 values less than (maxvalue))`)
tk.MustExec(`CREATE TABLE t6 (id int, c varchar(128))`)
Expand Down
18 changes: 10 additions & 8 deletions pkg/executor/adapter.go
Expand Up @@ -278,7 +278,7 @@ func (a *ExecStmt) PointGet(ctx context.Context) (*recordSet, error) {
}
a.Ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityHigh

var pointExecutor *PointGetExecutor
var executor exec.Executor
useMaxTS := startTs == math.MaxUint64

// try to reuse point get executor
Expand All @@ -293,24 +293,26 @@ func (a *ExecStmt) PointGet(ctx context.Context) (*recordSet, error) {
pointGetPlan := a.PsStmt.PreparedAst.CachedPlan.(*plannercore.PointGetPlan)
exec.Init(pointGetPlan)
a.PsStmt.Executor = exec
pointExecutor = exec
executor = exec
}
}

if pointExecutor == nil {
if executor == nil {
b := newExecutorBuilder(a.Ctx, a.InfoSchema)
pointExecutor = b.build(a.Plan).(*PointGetExecutor)
executor = b.build(a.Plan)
if b.err != nil {
return nil, b.err
}
pointExecutor, ok := executor.(*PointGetExecutor)

if useMaxTS {
// Don't cache the executor for non point-get (table dual) or partitioned tables
if ok && useMaxTS && pointExecutor.partitionDefIdx == nil {
a.PsStmt.Executor = pointExecutor
}
}

if err = exec.Open(ctx, pointExecutor); err != nil {
terror.Log(exec.Close(pointExecutor))
if err = exec.Open(ctx, executor); err != nil {
terror.Log(exec.Close(executor))
return nil, err
}

Expand All @@ -330,7 +332,7 @@ func (a *ExecStmt) PointGet(ctx context.Context) (*recordSet, error) {
}

return &recordSet{
executor: pointExecutor,
executor: executor,
stmt: a,
txnStartTS: startTs,
}, nil
Expand Down
136 changes: 54 additions & 82 deletions pkg/executor/batch_point_get.go
Expand Up @@ -25,17 +25,17 @@ import (
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/planner/core"
plannercore "github.com/pingcap/tidb/pkg/planner/core"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
driver "github.com/pingcap/tidb/pkg/store/driver/txn"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/table/tables"
"github.com/pingcap/tidb/pkg/tablecodec"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/codec"
"github.com/pingcap/tidb/pkg/util/hack"
"github.com/pingcap/tidb/pkg/util/intest"
"github.com/pingcap/tidb/pkg/util/logutil/consistency"
"github.com/pingcap/tidb/pkg/util/rowcodec"
"github.com/tikv/client-go/v2/tikvrpc"
Expand All @@ -46,26 +46,27 @@ type BatchPointGetExec struct {
exec.BaseExecutor
indexUsageReporter *exec.IndexUsageReporter

tblInfo *model.TableInfo
idxInfo *model.IndexInfo
handles []kv.Handle
physIDs []int64
partExpr *tables.PartitionExpr
partPos int
tblInfo *model.TableInfo
idxInfo *model.IndexInfo
handles []kv.Handle
// table/partition IDs for handle or index read
// (can be secondary unique key,
// and need lookup through handle)
planPhysIDs []int64
singlePart bool
partTblID int64
idxVals [][]types.Datum
txn kv.Transaction
lock bool
waitTime int64
inited uint32
values [][]byte
index int
rowDecoder *rowcodec.ChunkDecoder
keepOrder bool
desc bool
batchGetter kv.BatchGetter
// If != 0 then it is a single partition under Static Prune mode.
singlePartID int64
partitionNames []model.CIStr
idxVals [][]types.Datum
txn kv.Transaction
lock bool
waitTime int64
inited uint32
values [][]byte
index int
rowDecoder *rowcodec.ChunkDecoder
keepOrder bool
desc bool
batchGetter kv.BatchGetter

columns []*model.ColumnInfo
// virtualColumnIndex records all the indices of virtual columns and sort them in definition
Expand Down Expand Up @@ -208,15 +209,6 @@ func (e *BatchPointGetExec) Next(ctx context.Context, req *chunk.Chunk) error {
return nil
}

func datumsContainNull(vals []types.Datum) bool {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

FYI: Moved to types/datum.go

for _, val := range vals {
if val.IsNull() {
return true
}
}
return false
}

func (e *BatchPointGetExec) initialize(ctx context.Context) error {
var handleVals map[string][]byte
var indexKeys []kv.Key
Expand All @@ -228,30 +220,13 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
dedup := make(map[hack.MutableString]struct{})
toFetchIndexKeys := make([]kv.Key, 0, len(e.idxVals))
for i, idxVals := range e.idxVals {
// For all x, 'x IN (null)' evaluate to null, so the query get no result.
if datumsContainNull(idxVals) {
continue
}

var physID int64
if e.partPos == core.GlobalWithoutColumnPos {
physID = e.tblInfo.ID
} else {
if len(e.planPhysIDs) > 0 {
physID = e.planPhysIDs[i]
} else {
physID, err = core.GetPhysID(e.tblInfo, e.partExpr, idxVals[e.partPos])
if err != nil {
continue
}
}
physID := e.tblInfo.ID
if e.singlePartID != 0 {
physID = e.singlePartID
} else if len(e.planPhysIDs) > i {
physID = e.planPhysIDs[i]
}

// If this BatchPointGetExec is built only for the specific table partition, skip those filters not matching this partition.
if e.singlePart && e.partTblID != physID {
continue
}
idxKey, err1 := EncodeUniqueIndexKey(e.Ctx(), e.tblInfo, e.idxInfo, idxVals, physID)
idxKey, err1 := plannercore.EncodeUniqueIndexKey(e.Ctx(), e.tblInfo, e.idxInfo, idxVals, physID)
if err1 != nil && !kv.ErrNotExist.Equal(err1) {
return err1
}
Expand All @@ -266,6 +241,10 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
toFetchIndexKeys = append(toFetchIndexKeys, idxKey)
}
if e.keepOrder {
// TODO: if multiple partitions, then the IDs needs to be
// in the same order as the index keys
// and should skip table id part when comparing
intest.Assert(e.singlePartID != 0 || len(e.planPhysIDs) <= 1 || e.idxInfo.Global)
slices.SortFunc(toFetchIndexKeys, func(i, j kv.Key) int {
if e.desc {
return j.Cmp(i)
Expand Down Expand Up @@ -296,7 +275,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {

e.handles = make([]kv.Handle, 0, len(toFetchIndexKeys))
if e.tblInfo.Partition != nil {
e.physIDs = make([]int64, 0, len(toFetchIndexKeys))
e.planPhysIDs = e.planPhysIDs[:0]
}
for _, key := range toFetchIndexKeys {
handleVal := handleVals[string(key)]
Expand All @@ -307,10 +286,6 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
if err1 != nil {
return err1
}
e.handles = append(e.handles, handle)
if rc {
indexKeys = append(indexKeys, key)
}
if e.tblInfo.Partition != nil {
var pid int64
if e.idxInfo.Global {
Expand All @@ -319,15 +294,25 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
if err != nil {
return err
}
e.physIDs = append(e.physIDs, pid)
if e.singlePartID != 0 && e.singlePartID != pid {
continue
}
if !matchPartitionNames(pid, e.partitionNames, e.tblInfo.GetPartitionInfo()) {
continue
}
e.planPhysIDs = append(e.planPhysIDs, pid)
} else {
pid = tablecodec.DecodeTableID(key)
e.physIDs = append(e.physIDs, pid)
e.planPhysIDs = append(e.planPhysIDs, pid)
}
if e.lock {
e.UpdateDeltaForTableID(pid)
}
}
e.handles = append(e.handles, handle)
if rc {
indexKeys = append(indexKeys, key)
}
}

// The injection is used to simulate following scenario:
Expand Down Expand Up @@ -373,36 +358,23 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
}
}
slices.SortFunc(e.handles, less)
// TODO: if partitioned table, sorting the handles would also
// need to have the physIDs rearranged in the same order!
intest.Assert(e.singlePartID != 0 || len(e.planPhysIDs) <= 1)
}

keys := make([]kv.Key, 0, len(e.handles))
newHandles := make([]kv.Handle, 0, len(e.handles))
for i, handle := range e.handles {
var tID int64
if len(e.physIDs) > 0 {
tID = e.physIDs[i]
tID := e.tblInfo.ID
if e.singlePartID != 0 {
tID = e.singlePartID
} else if len(e.planPhysIDs) > 0 {
// Direct handle read
tID = e.planPhysIDs[i]
} else {
if handle.IsInt() {
d := types.NewIntDatum(handle.IntValue())
tID, err = core.GetPhysID(e.tblInfo, e.partExpr, d)
if err != nil {
continue
}
} else {
_, d, err1 := codec.DecodeOne(handle.EncodedCol(e.partPos))
if err1 != nil {
return err1
}
tID, err = core.GetPhysID(e.tblInfo, e.partExpr, d)
if err != nil {
continue
}
}
}
// If this BatchPointGetExec is built only for the specific table partition, skip those handles not matching this partition.
if e.singlePart && e.partTblID != tID {
if tID <= 0 {
// not matching any partition
continue
}
key := tablecodec.EncodeRowKeyWithHandle(tID, handle)
Expand Down