Skip to content

Commit 6c2b98b

Browse files
committed
store/copr: add unit tests for RC paging eligibility and bytes propagation
Test rcPagingEligible conditions (RC enabled, resource group, TiKV, DAG) and verify pagingSizeBytes is correctly propagated to copTasks and cleared when small limit disables paging. Signed-off-by: JmPotato <github@ipotato.me>
1 parent b83dabb commit 6c2b98b

1 file changed

Lines changed: 77 additions & 0 deletions

File tree

pkg/store/copr/coprocessor_test.go

Lines changed: 77 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import (
2121

2222
"github.com/pingcap/kvproto/pkg/coprocessor"
2323
"github.com/pingcap/tidb/pkg/kv"
24+
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
2425
"github.com/pingcap/tidb/pkg/store/driver/backoff"
2526
"github.com/pingcap/tidb/pkg/util/paging"
2627
"github.com/pingcap/tidb/pkg/util/trxevents"
@@ -700,6 +701,82 @@ func TestBuildPagingTasksDisablePagingForSmallLimit(t *testing.T) {
700701
require.Equal(t, tasks[0].pagingSize, uint64(0))
701702
}
702703

704+
func TestRCPagingEligible(t *testing.T) {
705+
// Eligible: RC enabled + resource group + TiKV + DAG
706+
vardef.EnableResourceControl.Store(true)
707+
defer vardef.EnableResourceControl.Store(false)
708+
req := &kv.Request{
709+
Tp: kv.ReqTypeDAG,
710+
StoreType: kv.TiKV,
711+
ResourceGroupName: "rg1",
712+
}
713+
require.True(t, rcPagingEligible(req))
714+
715+
// Not eligible: RC disabled
716+
vardef.EnableResourceControl.Store(false)
717+
require.False(t, rcPagingEligible(req))
718+
719+
// Not eligible: empty resource group
720+
vardef.EnableResourceControl.Store(true)
721+
req2 := &kv.Request{Tp: kv.ReqTypeDAG, StoreType: kv.TiKV}
722+
require.False(t, rcPagingEligible(req2))
723+
724+
// Not eligible: TiFlash
725+
req3 := &kv.Request{Tp: kv.ReqTypeDAG, StoreType: kv.TiFlash, ResourceGroupName: "rg1"}
726+
require.False(t, rcPagingEligible(req3))
727+
728+
// Not eligible: non-DAG
729+
req4 := &kv.Request{Tp: kv.ReqTypeAnalyze, StoreType: kv.TiKV, ResourceGroupName: "rg1"}
730+
require.False(t, rcPagingEligible(req4))
731+
}
732+
733+
func TestBuildCopTasksWithRCPagingSizeBytes(t *testing.T) {
734+
mockClient, cluster, pdClient, err := testutils.NewMockTiKV("", nil)
735+
require.NoError(t, err)
736+
defer func() {
737+
pdClient.Close()
738+
err = mockClient.Close()
739+
require.NoError(t, err)
740+
}()
741+
_, regionIDs, _ := testutils.BootstrapWithMultiRegions(cluster, []byte("g"), []byte("n"), []byte("t"))
742+
743+
pdCli := tikv.NewCodecPDClient(tikv.ModeTxn, pdClient)
744+
defer pdCli.Close()
745+
cache := NewRegionCache(tikv.NewRegionCache(pdCli))
746+
defer cache.Close()
747+
bo := backoff.NewBackofferWithVars(context.Background(), 3000, nil)
748+
749+
req := &kv.Request{}
750+
req.Paging.Enable = true
751+
req.Paging.MinPagingSize = paging.MinPagingSize
752+
753+
// With rcPagingSizeBytes set, tasks should carry the byte budget.
754+
tasks, err := buildCopTasks(bo, buildCopRanges("a", "c"), &buildCopTaskOpt{
755+
req: req,
756+
cache: cache,
757+
respChan: true,
758+
rcPagingSizeBytes: paging.MaxPagingSizeBytes,
759+
})
760+
require.NoError(t, err)
761+
require.Len(t, tasks, 1)
762+
taskEqual(t, tasks[0], regionIDs[0], 0, "a", "c")
763+
require.True(t, tasks[0].paging)
764+
require.Equal(t, paging.MaxPagingSizeBytes, tasks[0].pagingSizeBytes)
765+
766+
// When small limit disables paging, pagingSizeBytes should also be cleared.
767+
req.LimitSize = 1
768+
tasks, err = buildCopTasks(bo, buildCopRanges("a", "c"), &buildCopTaskOpt{
769+
req: req,
770+
cache: cache,
771+
respChan: true,
772+
rcPagingSizeBytes: paging.MaxPagingSizeBytes,
773+
})
774+
require.NoError(t, err)
775+
require.Len(t, tasks, 1)
776+
require.False(t, tasks[0].paging)
777+
require.Equal(t, uint64(0), tasks[0].pagingSizeBytes)
778+
}
779+
703780
func toCopRange(r kv.KeyRange) *coprocessor.KeyRange {
704781
coprRange := coprocessor.KeyRange{}
705782
coprRange.Start = r.StartKey

0 commit comments

Comments
 (0)