gc_worker: Stablize TestResolveLocksWithKeyspaces (#64169)

ref pingcap/tidb#58720
This commit is contained in:
MyonKeminta
2025-10-30 15:57:33 +08:00
committed by GitHub
parent 4b652ccec3
commit 993eabc3cc
4 changed files with 119 additions and 29 deletions

View File

@ -54,7 +54,8 @@ go_test(
],
embed = [":gcworker"],
flaky = True,
shard_count = 26,
race = "on",
shard_count = 35,
deps = [
"//pkg/config",
"//pkg/config/kerneltype",

View File

@ -1237,9 +1237,26 @@ func (w *GCWorker) resolveLocks(
isNullKeyspace := w.store == nil || w.store.GetCodec().GetKeyspace() == nil
var keyspaceBatch []*keyspacepb.KeyspaceMeta
// Failpoint to override the batch size for faster test
loadKeyspacesBatchSize := uint32(loadAllKeyspacesForUnifiedGCBatchSize)
failpoint.Inject("overrideLoadKeyspacesBatchSize", func(val failpoint.Value) {
v, ok := val.(int)
if !ok {
panic(fmt.Sprintf("invalid argument for failpoint overrideLoadKeyspacesBatchSize: expected integer, got %T: %v", val, val))
}
loadKeyspacesBatchSize = uint32(v)
})
// Counter for tests to check how many batches was done during resolving locks.
loadKeyspacesBatchCount := 0
defer func() {
failpoint.InjectCall("getLoadKeyspacesBatchCount", loadKeyspacesBatchCount)
}()
if isNullKeyspace {
var err error
keyspaceBatch, err = w.pdClient.GetAllKeyspaces(ctx, 0, loadAllKeyspacesForUnifiedGCBatchSize)
keyspaceBatch, err = w.pdClient.GetAllKeyspaces(ctx, 0, loadKeyspacesBatchSize)
loadKeyspacesBatchCount++
if err != nil {
return err
}
@ -1355,7 +1372,8 @@ func (w *GCWorker) resolveLocks(
break
}
var err error
keyspaceBatch, err = w.pdClient.GetAllKeyspaces(ctx, nextKeyspaceID, loadAllKeyspacesForUnifiedGCBatchSize)
keyspaceBatch, err = w.pdClient.GetAllKeyspaces(ctx, nextKeyspaceID, loadKeyspacesBatchSize)
loadKeyspacesBatchCount++
if err != nil {
return err
}

View File

@ -1283,12 +1283,16 @@ func TestResolveLockRangeMeetRegionEnlargeCausedByRegionMerge(t *testing.T) {
}
}
func TestResolveLocksWithKeyspaces(t *testing.T) {
func testResolveLocksWithKeyspacesImpl(t *testing.T, subCaseName string) {
// Note: this test is expected not to respect to whether compiled as NextGen, but tests the logic designed to work
// on both classic and NextGen, and even mixed keyspaced & un-keyspaced usages in the same cluster.
// The unified GC, which is not used and won't be used in next gen, is also covered here.
// However, as the NextGen flag is currently overused, causing some keyspace-specific code unable to run on
// non-next-gen compilation or vice versa, the sub-tests must be filtered by the compilation flag for now.
// Note: this test case consists of several sub-tests, but not managed with .t.Run(), because it can't be split
// into multiple ones when running on CI, and might cause timeout.
type reqRange struct {
StartKey []byte
EndKey []byte
@ -1348,7 +1352,9 @@ func TestResolveLocksWithKeyspaces(t *testing.T) {
return mergedRanges
}
t.Run("NullKeyspaceOnly", func(t *testing.T) {
subCases := make(map[string]func(t *testing.T))
subCases["NullKeyspaceOnly"] = func(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip()
}
@ -1365,9 +1371,9 @@ func TestResolveLocksWithKeyspaces(t *testing.T) {
TxnSafePoint: 100,
}}, ranges)
require.Equal(t, int64(1), counter.Load())
})
}
t.Run("NullKeyspaceOnlyMultiRegion", func(t *testing.T) {
subCases["NullKeyspaceOnlyMultiRegion"] = func(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip()
}
@ -1383,7 +1389,7 @@ func TestResolveLocksWithKeyspaces(t *testing.T) {
TxnSafePoint: 100,
}}, ranges)
require.Equal(t, int64(4), counter.Load())
})
}
makeKeyspace := func(id uint32, name string, enableKeyspaceLevelGC bool) *keyspacepb.KeyspaceMeta {
gcManagementType := pd.KeyspaceConfigGCManagementTypeKeyspaceLevel
@ -1403,7 +1409,7 @@ func TestResolveLocksWithKeyspaces(t *testing.T) {
return string(c.EncodeKey([]byte(key)))
}
t.Run("NullKeyspaceInMultiKeyspaceEnvironment", func(t *testing.T) {
subCases["NullKeyspaceInMultiKeyspaceEnvironment"] = func(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip()
}
@ -1428,9 +1434,9 @@ func TestResolveLocksWithKeyspaces(t *testing.T) {
{StartKey: []byte("y"), EndKey: nil, TxnSafePoint: 100}, // 1 region
}, ranges)
require.Equal(t, int64(6), counter.Load())
})
}
t.Run("NonNullKeyspaceInMultiKeyspaceEnvironment", func(t *testing.T) {
subCases["NonNullKeyspaceInMultiKeyspaceEnvironment"] = func(t *testing.T) {
if !kerneltype.IsNextGen() {
t.Skip()
}
@ -1464,9 +1470,9 @@ func TestResolveLocksWithKeyspaces(t *testing.T) {
{StartKey: nil, EndKey: nil, TxnSafePoint: 100}, // 2 regions split at "a"
}, ranges)
require.Equal(t, int64(2), counter.Load())
})
}
t.Run("UnifiedGCInMixedUsage", func(t *testing.T) {
subCases["UnifiedGCInMixedUsage"] = func(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip()
}
@ -1496,9 +1502,9 @@ func TestResolveLocksWithKeyspaces(t *testing.T) {
{StartKey: []byte("y"), EndKey: nil, TxnSafePoint: 100}, // 1 region
}, ranges)
require.Equal(t, int64(12), counter.Load())
})
}
t.Run("UnifiedGCWithMaxKeyspaceID", func(t *testing.T) {
subCases["UnifiedGCWithMaxKeyspaceID"] = func(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip()
}
@ -1519,13 +1525,28 @@ func TestResolveLocksWithKeyspaces(t *testing.T) {
}, ranges)
// Note: The range ["y", "y\x00\x00\x00") is actually repeatedly handled, but it doesn't matter for now as it's never used and contains only 3 keys.
require.Equal(t, int64(10), counter.Load())
})
}
testUnifiedGCInLargeAmountOfKeyspacesImpl := func(t *testing.T, startID uint32, count uint32, step uint32) {
testUnifiedGCInMultiBatchesOfKeyspacesImpl := func(t *testing.T, startID uint32, count uint32, step uint32, loadBatchSize int, expectedBatchCount int) {
if kerneltype.IsNextGen() {
t.Skip()
}
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/store/gcworker/overrideLoadKeyspacesBatchSize", fmt.Sprintf("return(%d)", loadBatchSize)))
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/store/gcworker/overrideLoadKeyspacesBatchSize"))
}()
// Retrieve the actual batch count of loading all keyspaces, for ensuring that the failpoint
// `overrideLoadKeyspacesBatchSize` actually takes effect.
loadKeyspacesBatchCount := 0
require.NoError(t, failpoint.EnableCall("github.com/pingcap/tidb/pkg/store/gcworker/getLoadKeyspacesBatchCount", func(v int) {
loadKeyspacesBatchCount += v
}))
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/store/gcworker/getLoadKeyspacesBatchCount"))
}()
keyspaces := make([]*keyspacepb.KeyspaceMeta, 0, count)
splitKeys := make([]string, 0, count*2+3)
expectedRanges := make([]reqRange, 0, count+3)
@ -1562,23 +1583,71 @@ func TestResolveLocksWithKeyspaces(t *testing.T) {
ranges := collectAndMergeRanges(t, ch)
require.Equal(t, expectedRanges, ranges)
require.Equal(t, int64(expectedScanLocksCount), counter.Load())
require.Equal(t, expectedBatchCount, loadKeyspacesBatchCount)
}
t.Run("UnifiedGCInLargeAmountOfKeyspaces_1_to_180", func(t *testing.T) {
testUnifiedGCInLargeAmountOfKeyspacesImpl(t, 1, 180, 1)
})
subCases["UnifiedGCInMultiBatchesOfKeyspaces_8"] = func(t *testing.T) {
// Load keyspaces batches: [1,2,3], [4,5,6], [7,8], []
testUnifiedGCInMultiBatchesOfKeyspacesImpl(t, 1, 8, 1, 3, 4)
}
t.Run("UnifiedGCInLargeAmountOfKeyspaces_0xffff00_to_0xffffff_step2", func(t *testing.T) {
testUnifiedGCInLargeAmountOfKeyspacesImpl(t, 0xffff00, 128, 2)
})
subCases["UnifiedGCInMultiBatchesOfKeyspaces_Last8_Step2"] = func(t *testing.T) {
// Load keyspaces batches: [(MaxKeyspaceID)-14,-12,-10], [-8,-6,-4], [-2,0]
testUnifiedGCInMultiBatchesOfKeyspacesImpl(t, constants.MaxKeyspaceID-14, 8, 2, 3, 3)
}
t.Run("UnifiedGCInLargeAmountOfKeyspaces_MultipleOfBatchSize", func(t *testing.T) {
testUnifiedGCInLargeAmountOfKeyspacesImpl(t, 1, loadAllKeyspacesForUnifiedGCBatchSize*2, 1)
})
subCases["UnifiedGCInMultiBatchesOfKeyspaces_MultipleOfBatchSize"] = func(t *testing.T) {
// Load keyspaces batches: [1,2,3], [4,5,6], []
testUnifiedGCInMultiBatchesOfKeyspacesImpl(t, 1, 6, 1, 3, 3)
}
t.Run("UnifiedGCInLargeAmountOfKeyspaces_MultipleOfBatchSizeToEnd", func(t *testing.T) {
testUnifiedGCInLargeAmountOfKeyspacesImpl(t, constants.MaxKeyspaceID-loadAllKeyspacesForUnifiedGCBatchSize*2+1, loadAllKeyspacesForUnifiedGCBatchSize*2, 1)
})
subCases["UnifiedGCInMultiBatchesOfKeyspaces_MultipleOfBatchSizeToEnd"] = func(t *testing.T) {
// Load keyspaces batches: [(MaxKeyspaceID)-5,-4,-3], [-2,-1,0]
testUnifiedGCInMultiBatchesOfKeyspacesImpl(t, constants.MaxKeyspaceID-5, 6, 1, 3, 2)
}
subCases[subCaseName](t)
}
func TestResolveLocksWithKeyspaces_NullKeyspaceOnly(t *testing.T) {
testResolveLocksWithKeyspacesImpl(t, "NullKeyspaceOnly")
}
func TestResolveLocksWithKeyspaces_NullKeyspaceOnlyMultiRegion(t *testing.T) {
testResolveLocksWithKeyspacesImpl(t, "NullKeyspaceOnlyMultiRegion")
}
func TestResolveLocksWithKeyspaces_NullKeyspaceInMultiKeyspaceEnvironment(t *testing.T) {
testResolveLocksWithKeyspacesImpl(t, "NullKeyspaceInMultiKeyspaceEnvironment")
}
func TestResolveLocksWithKeyspaces_NonNullKeyspaceInMultiKeyspaceEnvironment(t *testing.T) {
testResolveLocksWithKeyspacesImpl(t, "NonNullKeyspaceInMultiKeyspaceEnvironment")
}
func TestResolveLocksWithKeyspaces_UnifiedGCInMixedUsage(t *testing.T) {
testResolveLocksWithKeyspacesImpl(t, "UnifiedGCInMixedUsage")
}
func TestResolveLocksWithKeyspaces_UnifiedGCWithMaxKeyspaceID(t *testing.T) {
testResolveLocksWithKeyspacesImpl(t, "UnifiedGCWithMaxKeyspaceID")
}
func TestResolveLocksWithKeyspaces_UnifiedGCInMultiBatchesOfKeyspaces_8(t *testing.T) {
testResolveLocksWithKeyspacesImpl(t, "UnifiedGCInMultiBatchesOfKeyspaces_8")
}
func TestResolveLocksWithKeyspaces_UnifiedGCInMultiBatchesOfKeyspaces_Last8_Step2(t *testing.T) {
testResolveLocksWithKeyspacesImpl(t, "UnifiedGCInMultiBatchesOfKeyspaces_Last8_Step2")
}
func TestResolveLocksWithKeyspaces_UnifiedGCInMultiBatchesOfKeyspaces_MultipleOfBatchSize(t *testing.T) {
testResolveLocksWithKeyspacesImpl(t, "UnifiedGCInMultiBatchesOfKeyspaces_MultipleOfBatchSize")
}
func TestResolveLocksWithKeyspaces_UnifiedGCInMultiBatchesOfKeyspaces_MultipleOfBatchSizeToEnd(t *testing.T) {
testResolveLocksWithKeyspacesImpl(t, "UnifiedGCInMultiBatchesOfKeyspaces_MultipleOfBatchSizeToEnd")
}
func TestRunGCJob(t *testing.T) {

View File

@ -415,6 +415,8 @@ func (rm *MockRegionManager) SplitArbitrary(keys ...[]byte) {
encKey := codec.EncodeBytes(nil, key)
splitKeys = append(splitKeys, encKey)
}
rm.mu.Lock()
defer rm.mu.Unlock()
if _, err := rm.splitKeys(splitKeys); err != nil {
panic(err)
}