Skip to content

Commit ad1cb78

Browse files
authored
*: support paging protocol on unistore (#35244)
ref #35242, close #35243
1 parent 4c443d5 commit ad1cb78

File tree

14 files changed

+217
-20
lines changed

14 files changed

+217
-20
lines changed

executor/builder.go

Lines changed: 30 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3304,6 +3304,9 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E
33043304
if len(partitions) == 0 {
33053305
return &TableDualExec{baseExecutor: *ret.base()}
33063306
}
3307+
3308+
// Sort the partition is necessary to make the final multiple partition key ranges ordered.
3309+
sort.Sort(partitionSlice(partitions))
33073310
ret.kvRangeBuilder = kvRangeBuilderFromRangeAndPartition{
33083311
sctx: b.ctx,
33093312
partitions: partitions,
@@ -3423,6 +3426,9 @@ func (builder *dataReaderBuilder) prunePartitionForInnerExecutor(tbl table.Table
34233426
usedPartition = append(usedPartition, p)
34243427
}
34253428
}
3429+
3430+
// To make the final key ranges involving multiple partitions ordered.
3431+
sort.Sort(partitionSlice(usedPartition))
34263432
return usedPartition, true, contentPos, nil
34273433
}
34283434

@@ -3998,6 +4004,10 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte
39984004
kvRanges = append(tmp, kvRanges...)
39994005
}
40004006
}
4007+
// The key ranges should be ordered.
4008+
sort.Slice(kvRanges, func(i, j int) bool {
4009+
return bytes.Compare(kvRanges[i].StartKey, kvRanges[j].StartKey) < 0
4010+
})
40014011
return builder.buildTableReaderFromKvRanges(ctx, e, kvRanges)
40024012
}
40034013

@@ -4028,6 +4038,11 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte
40284038
kvRanges = append(kvRanges, tmp...)
40294039
}
40304040
}
4041+
4042+
// The key ranges should be ordered.
4043+
sort.Slice(kvRanges, func(i, j int) bool {
4044+
return bytes.Compare(kvRanges[i].StartKey, kvRanges[j].StartKey) < 0
4045+
})
40314046
return builder.buildTableReaderFromKvRanges(ctx, e, kvRanges)
40324047
}
40334048

@@ -4056,6 +4071,21 @@ type kvRangeBuilderFromRangeAndPartition struct {
40564071
partitions []table.PhysicalTable
40574072
}
40584073

4074+
// partitionSlice implement the sort interface.
4075+
type partitionSlice []table.PhysicalTable
4076+
4077+
func (s partitionSlice) Len() int {
4078+
return len(s)
4079+
}
4080+
4081+
func (s partitionSlice) Less(i, j int) bool {
4082+
return s[i].GetPhysicalID() < s[j].GetPhysicalID()
4083+
}
4084+
4085+
func (s partitionSlice) Swap(i, j int) {
4086+
s[i], s[j] = s[j], s[i]
4087+
}
4088+
40594089
func (h kvRangeBuilderFromRangeAndPartition) buildKeyRangeSeparately(ranges []*ranger.Range) ([]int64, [][]kv.KeyRange, error) {
40604090
ret := make([][]kv.KeyRange, 0, len(h.partitions))
40614091
pids := make([]int64, 0, len(h.partitions))

executor/copr_cache_test.go

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -53,6 +53,9 @@ func TestIntegrationCopCache(t *testing.T) {
5353
tk.MustExec("use test")
5454
tk.MustExec("create table t (a int primary key)")
5555

56+
// TODO(tiancaiamao) update the test and support cop cache for paging.
57+
tk.MustExec("set @@tidb_enable_paging = off")
58+
5659
tblInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t"))
5760
require.NoError(t, err)
5861
tid := tblInfo.Meta().ID

planner/core/cbo_test.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -822,6 +822,10 @@ func TestLimitIndexEstimation(t *testing.T) {
822822
SQL string
823823
Plan []string
824824
}
825+
// When paging is used, there is a 'paging:true' makes the explain output differ.
826+
// IndexLookUp 0.00 root paging:true
827+
tk.MustExec("set @@tidb_enable_paging = off")
828+
825829
analyzeSuiteData := core.GetAnalyzeSuiteData()
826830
analyzeSuiteData.GetTestCases(t, &input, &output)
827831
for i, tt := range input {

planner/core/integration_test.go

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -177,6 +177,9 @@ func TestPushLimitDownIndexLookUpReader(t *testing.T) {
177177
tk.MustExec("insert into tbl values(1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5)")
178178
tk.MustExec("analyze table tbl")
179179

180+
// When paging is enabled, there would be a 'paging: true' in the explain result.
181+
tk.MustExec("set @@tidb_enable_paging = off")
182+
180183
var input []string
181184
var output []struct {
182185
SQL string
@@ -3681,6 +3684,10 @@ func TestExtendedStatsSwitch(t *testing.T) {
36813684
tk.MustQuery("select stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows(
36823685
"1.000000 1",
36833686
))
3687+
3688+
// When paging is enabled, there would be a 'paging: true' in the explain result.
3689+
tk.MustExec("set @@tidb_enable_paging = off")
3690+
36843691
// Estimated index scan count is 4 using extended stats.
36853692
tk.MustQuery("explain format = 'brief' select * from t use index(b) where a > 3 order by b limit 1").Check(testkit.Rows(
36863693
"Limit 1.00 root offset:0, count:1",
@@ -4550,6 +4557,9 @@ func TestLimitIndexLookUpKeepOrder(t *testing.T) {
45504557
tk.MustExec("drop table if exists t;")
45514558
tk.MustExec("create table t(a int, b int, c int, d int, index idx(a,b,c));")
45524559

4560+
// When paging is enabled, there would be a 'paging: true' in the explain result.
4561+
tk.MustExec("set @@tidb_enable_paging = off")
4562+
45534563
var input []string
45544564
var output []struct {
45554565
SQL string
@@ -4748,6 +4758,9 @@ func TestMultiColMaxOneRow(t *testing.T) {
47484758
tk.MustExec("create table t1(a int)")
47494759
tk.MustExec("create table t2(a int, b int, c int, primary key(a,b))")
47504760

4761+
// When paging is enabled, there would be a 'paging: true' in the explain result.
4762+
tk.MustExec("set @@tidb_enable_paging = off")
4763+
47514764
var input []string
47524765
var output []struct {
47534766
SQL string
@@ -5520,6 +5533,8 @@ func TestPreferRangeScanForUnsignedIntHandle(t *testing.T) {
55205533

55215534
// Default RPC encoding may cause statistics explain result differ and then the test unstable.
55225535
tk.MustExec("set @@tidb_enable_chunk_rpc = on")
5536+
// When paging is enabled, there would be a 'paging: true' in the explain result.
5537+
tk.MustExec("set @@tidb_enable_paging = off")
55235538

55245539
var input []string
55255540
var output []struct {
@@ -5559,6 +5574,9 @@ func TestIssue27083(t *testing.T) {
55595574
require.Nil(t, do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll))
55605575
tk.MustExec("analyze table t")
55615576

5577+
// When paging is enabled, there would be a 'paging: true' in the explain result.
5578+
tk.MustExec("set @@tidb_enable_paging = off")
5579+
55625580
var input []string
55635581
var output []struct {
55645582
SQL string

planner/core/physical_plan_test.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -538,6 +538,10 @@ func TestEliminateMaxOneRow(t *testing.T) {
538538
tk.MustExec("create table t2(a int(11) DEFAULT NULL, b int(11) DEFAULT NULL)")
539539
tk.MustExec("create table t3(a int(11) DEFAULT NULL, b int(11) DEFAULT NULL, c int(11) DEFAULT NULL, UNIQUE KEY idx_abc (a, b, c))")
540540

541+
// When paging is used, there is a 'paging:true' makes the explain output differ.
542+
// IndexLookUp 0.00 root paging:true
543+
tk.MustExec("set @@tidb_enable_paging = off")
544+
541545
for i, ts := range input {
542546
testdata.OnRecord(func() {
543547
output[i].SQL = ts

statistics/handle/update_test.go

Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -733,6 +733,10 @@ func TestUpdateErrorRate(t *testing.T) {
733733

734734
testKit := testkit.NewTestKit(t, store)
735735
testKit.MustExec("use test")
736+
737+
// TODO(tiancaiamao): query feedback is broken when paging is on.
738+
testKit.MustExec("set @@tidb_enable_paging = off")
739+
736740
testKit.MustExec("set @@session.tidb_analyze_version = 0")
737741
testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))")
738742
err := h.HandleDDLEvent(<-h.DDLEventCh())
@@ -916,6 +920,10 @@ func TestQueryFeedback(t *testing.T) {
916920
defer clean()
917921
testKit := testkit.NewTestKit(t, store)
918922
testKit.MustExec("use test")
923+
924+
// TODO(tiancaiamao): query feedback is broken when paging is on.
925+
testKit.MustExec("set @@tidb_enable_paging = off")
926+
919927
testKit.MustExec("set @@session.tidb_analyze_version = 0")
920928
testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))")
921929
testKit.MustExec("insert into t values (1,2),(2,2),(4,5)")
@@ -1174,6 +1182,10 @@ func TestUpdateStatsByLocalFeedback(t *testing.T) {
11741182
defer clean()
11751183
testKit := testkit.NewTestKit(t, store)
11761184
testKit.MustExec("use test")
1185+
1186+
// TODO(tiancaiamao): query feedback is broken when paging is on.
1187+
testKit.MustExec("set @@tidb_enable_paging = off")
1188+
11771189
testKit.MustExec("set @@session.tidb_analyze_version = 0")
11781190
testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`)
11791191
testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))")
@@ -1613,6 +1625,10 @@ func TestIndexQueryFeedback4TopN(t *testing.T) {
16131625
handle.MinLogErrorRate.Store(0)
16141626

16151627
testKit.MustExec("use test")
1628+
1629+
// TODO(tiancaiamao): query feedback is broken when paging is on.
1630+
testKit.MustExec("set @@tidb_enable_paging = off")
1631+
16161632
testKit.MustExec("set @@session.tidb_analyze_version = 0")
16171633
testKit.MustExec("create table t (a bigint(64), index idx(a))")
16181634
for i := 0; i < 20; i++ {
@@ -1664,6 +1680,10 @@ func TestAbnormalIndexFeedback(t *testing.T) {
16641680
handle.MinLogErrorRate.Store(0)
16651681

16661682
testKit.MustExec("use test")
1683+
1684+
// TODO(tiancaiamao): query feedback is broken when paging is on.
1685+
testKit.MustExec("set @@tidb_enable_paging = off")
1686+
16671687
testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx_ab(a,b))")
16681688
for i := 0; i < 20; i++ {
16691689
testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i/5, i))
@@ -1741,6 +1761,10 @@ func TestFeedbackRanges(t *testing.T) {
17411761
handle.MinLogErrorRate.Store(0)
17421762

17431763
testKit.MustExec("use test")
1764+
1765+
// TODO(tiancaiamao): query feedback is broken when paging is on.
1766+
testKit.MustExec("set @@tidb_enable_paging = off")
1767+
17441768
testKit.MustExec("create table t (a tinyint, b tinyint, primary key(a), index idx(a, b))")
17451769
for i := 0; i < 20; i++ {
17461770
testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i))
@@ -1820,6 +1844,10 @@ func TestUnsignedFeedbackRanges(t *testing.T) {
18201844
handle.MinLogErrorRate.Store(0)
18211845

18221846
testKit.MustExec("use test")
1847+
1848+
// TODO(tiancaiamao): query feedback is broken when paging is on.
1849+
testKit.MustExec("set @@tidb_enable_paging = off")
1850+
18231851
testKit.MustExec("set @@session.tidb_analyze_version = 0")
18241852
testKit.MustExec("create table t (a tinyint unsigned, primary key(a))")
18251853
testKit.MustExec("create table t1 (a bigint unsigned, primary key(a))")
@@ -2013,6 +2041,10 @@ func TestFeedbackCounter(t *testing.T) {
20132041
err := metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum)
20142042
require.NoError(t, err)
20152043
testKit.MustExec("use test")
2044+
2045+
// TODO(tiancaiamao): query feedback is broken when paging is on.
2046+
testKit.MustExec("set @@tidb_enable_paging = off")
2047+
20162048
testKit.MustExec("create table t (a int, b int, index idx_a(a))")
20172049
testKit.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (5, 5)")
20182050
testKit.MustExec("analyze table t with 0 topn")

store/copr/coprocessor.go

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -81,6 +81,17 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa
8181
logutil.BgLogger().Debug("send batch requests")
8282
return c.sendBatch(ctx, req, vars, option)
8383
}
84+
failpoint.Inject("DisablePaging", func(_ failpoint.Value) {
85+
req.Paging = false
86+
})
87+
if req.StoreType == kv.TiDB {
88+
// coprocessor on TiDB doesn't support paging
89+
req.Paging = false
90+
}
91+
if req.Tp != kv.ReqTypeDAG {
92+
// coprocessor request but type is not DAG
93+
req.Paging = false
94+
}
8495
ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs)
8596
bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars)
8697
ranges := NewKeyRanges(req.KeyRanges)

store/gcworker/gc_worker_test.go

Lines changed: 13 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -97,16 +97,18 @@ type mockGCWorkerSuite struct {
9797
}
9898

9999
func createGCWorkerSuite(t *testing.T) (s *mockGCWorkerSuite, clean func()) {
100-
s = new(mockGCWorkerSuite)
100+
return createGCWorkerSuiteWithStoreType(t, mockstore.EmbedUnistore)
101+
}
101102

103+
func createGCWorkerSuiteWithStoreType(t *testing.T, storeType mockstore.StoreType) (s *mockGCWorkerSuite, clean func()) {
104+
s = new(mockGCWorkerSuite)
102105
hijackClient := func(client tikv.Client) tikv.Client {
103106
s.client = &mockGCWorkerClient{Client: client}
104107
client = s.client
105108
return client
106109
}
107-
108110
opts := []mockstore.MockTiKVStoreOption{
109-
mockstore.WithStoreType(mockstore.MockTiKV),
111+
mockstore.WithStoreType(storeType),
110112
mockstore.WithClusterInspector(func(c testutils.Cluster) {
111113
s.initRegion.storeIDs, s.initRegion.peerIDs, s.initRegion.regionID, _ = mockstore.BootstrapWithMultiStores(c, 3)
112114
s.cluster = c
@@ -943,7 +945,14 @@ func TestResolveLockRangeMeetRegionCacheMiss(t *testing.T) {
943945
}
944946

945947
func TestResolveLockRangeMeetRegionEnlargeCausedByRegionMerge(t *testing.T) {
946-
s, clean := createGCWorkerSuite(t)
948+
// TODO: Update the test code.
949+
// This test rely on the obsolete mock tikv, but mock tikv does not implement paging.
950+
// So use this failpoint to force non-paging protocol.
951+
failpoint.Enable("github.com/pingcap/tidb/store/copr/DisablePaging", `return`)
952+
defer func() {
953+
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/copr/DisablePaging"))
954+
}()
955+
s, clean := createGCWorkerSuiteWithStoreType(t, mockstore.MockTiKV)
947956
defer clean()
948957

949958
var (

store/mockstore/mockcopr/executor_test.go

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ import (
1919
"testing"
2020

2121
"github.com/pingcap/errors"
22+
"github.com/pingcap/failpoint"
2223
"github.com/pingcap/kvproto/pkg/kvrpcpb"
2324
"github.com/pingcap/tidb/domain"
2425
"github.com/pingcap/tidb/kv"
@@ -37,6 +38,12 @@ import (
3738
// This test checks the resolve lock functionality. When a txn meets the lock of a large transaction,
3839
// it should not block by the lock.
3940
func TestResolvedLargeTxnLocks(t *testing.T) {
41+
// This is required since mock tikv does not support paging.
42+
failpoint.Enable("github.com/pingcap/tidb/store/copr/DisablePaging", `return`)
43+
defer func() {
44+
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/copr/DisablePaging"))
45+
}()
46+
4047
rpcClient, cluster, pdClient, err := testutils.NewMockTiKV("", mockcopr.NewCoprRPCHandler())
4148
require.NoError(t, err)
4249

0 commit comments

Comments
 (0)