Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 3 additions & 1 deletion pkg/sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -3657,7 +3657,9 @@ func GlobalSystemVariableInitialValue(varName, varVal string) string {
case vardef.TiDBRowFormatVersion:
varVal = strconv.Itoa(vardef.DefTiDBRowFormatV2)
case vardef.TiDBTxnAssertionLevel:
if !kerneltype.IsNextGen() {
if kerneltype.IsNextGen() {
varVal = vardef.GetDefaultTxnAssertionLevel()
} else {
varVal = vardef.AssertionFastStr
}
case vardef.TiDBEnableMutationChecker:
Expand Down
1 change: 1 addition & 0 deletions pkg/sessiontxn/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ go_test(
shard_count = 25,
deps = [
":sessiontxn",
"//pkg/config",
"//pkg/config/kerneltype",
"//pkg/domain",
"//pkg/errno",
Expand Down
1 change: 1 addition & 0 deletions pkg/sessiontxn/isolation/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ go_test(
deps = [
":isolation",
"//pkg/config",
"//pkg/config/kerneltype",
"//pkg/executor",
"//pkg/expression",
"//pkg/infoschema",
Expand Down
13 changes: 13 additions & 0 deletions pkg/sessiontxn/isolation/readcommitted_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/executor"
"github.com/pingcap/tidb/pkg/expression"
"github.com/pingcap/tidb/pkg/infoschema"
Expand Down Expand Up @@ -565,6 +566,12 @@ func initializePessimisticRCProvider(t testing.TB, tk *testkit.TestKit) *isolati
}

func TestFailedDMLConsistency1(t *testing.T) {
if kerneltype.IsNextGen() {
// NextGen hangs when acquiring pessimistic locks after failed DML with fair locking disabled
// root cause: cleanup not triggered properly for non-fair mode
// this 35682 might be related.
t.Skip("skip for next-gen kernel, as this test requires fair-locking")
}
store := testkit.CreateMockStore(t)

tk1 := testkit.NewTestKit(t, store)
Expand Down Expand Up @@ -595,6 +602,12 @@ func TestFailedDMLConsistency1(t *testing.T) {
}

func TestFailedDMLConsistency2(t *testing.T) {
if kerneltype.IsNextGen() {
// NextGen hangs when acquiring pessimistic locks after failed DML with fair locking disabled
// root cause: cleanup not triggered properly for non-fair mode.
// this 35682 might be related.
t.Skip("skip for next-gen kernel, as this test requires fair-locking")
}
store := testkit.CreateMockStore(t)
tk1 := testkit.NewTestKit(t, store)
tk1.MustExec("set @@tidb_txn_assertion_level=strict")
Expand Down
5 changes: 5 additions & 0 deletions pkg/sessiontxn/txn_context_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"time"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/domain"
"github.com/pingcap/tidb/pkg/errno"
Expand Down Expand Up @@ -813,6 +814,10 @@ func TestStillWriteConflictAfterRetry(t *testing.T) {
}

func TestOptimisticTxnRetryInPessimisticMode(t *testing.T) {
defer config.RestoreFunc()()
config.UpdateGlobal(func(conf *config.Config) {
conf.PessimisticTxn.PessimisticAutoCommit.Store(false)
})
store, _ := setupTxnContextTest(t)

queries := []string{
Expand Down
1 change: 1 addition & 0 deletions pkg/store/copr/copr_test/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_test(
shard_count = 5,
deps = [
"//pkg/config",
"//pkg/config/kerneltype",
"//pkg/kv",
"//pkg/resourcegroup/runaway",
"//pkg/store/copr",
Expand Down
47 changes: 44 additions & 3 deletions pkg/store/copr/copr_test/coprocessor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/meta_storagepb"
rmpb "github.com/pingcap/kvproto/pkg/resource_manager"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/resourcegroup/runaway"
"github.com/pingcap/tidb/pkg/store/copr"
Expand All @@ -39,12 +40,33 @@ import (
rmclient "github.com/tikv/pd/client/resource_group/controller"
)

// getKeyspaceAwareKey uses the actual store codec to encode keys properly
// This ensures we use the single source of truth for keyspace encoding
func getKeyspaceAwareKey(store kv.Storage, key []byte) []byte {
if !kerneltype.IsNextGen() {
return key
}

// Use the store's codec to encode the key - this is the single source of truth
codec := store.GetCodec()
return codec.EncodeKey(key)
}

func TestBuildCopIteratorWithRowCountHint(t *testing.T) {
// nil --- 'g' --- 'n' --- 't' --- nil
// <- 0 -> <- 1 -> <- 2 -> <- 3 ->

// Get keyspace-aware region boundaries by creating a temp store to access codec
tempStore, err := mockstore.NewMockStore()
require.NoError(t, err)
g := getKeyspaceAwareKey(tempStore, []byte("g"))
n := getKeyspaceAwareKey(tempStore, []byte("n"))
tKey := getKeyspaceAwareKey(tempStore, []byte("t"))
tempStore.Close()

store, err := mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c testutils.Cluster) {
mockstore.BootstrapWithMultiRegions(c, []byte("g"), []byte("n"), []byte("t"))
mockstore.BootstrapWithMultiRegions(c, g, n, tKey)
}),
)
require.NoError(t, err)
Expand Down Expand Up @@ -116,9 +138,19 @@ func TestBuildCopIteratorWithRowCountHint(t *testing.T) {
func TestBuildCopIteratorWithBatchStoreCopr(t *testing.T) {
// nil --- 'g' --- 'n' --- 't' --- nil
// <- 0 -> <- 1 -> <- 2 -> <- 3 ->
// Note: In NextGen mode, keys are keyspace-prefixed, so we need to adjust region boundaries

// Get keyspace-aware region boundaries by creating a temp store to access codec
tempStore, err := mockstore.NewMockStore()
require.NoError(t, err)
g := getKeyspaceAwareKey(tempStore, []byte("g"))
n := getKeyspaceAwareKey(tempStore, []byte("n"))
tKey := getKeyspaceAwareKey(tempStore, []byte("t"))
tempStore.Close()

store, err := mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c testutils.Cluster) {
mockstore.BootstrapWithMultiRegions(c, []byte("g"), []byte("n"), []byte("t"))
mockstore.BootstrapWithMultiRegions(c, g, n, tKey)
}),
)
require.NoError(t, err)
Expand Down Expand Up @@ -246,9 +278,18 @@ func (p *mockResourceGroupProvider) GetResourceGroup(ctx context.Context, name s
func TestBuildCopIteratorWithRunawayChecker(t *testing.T) {
// nil --- 'g' --- 'n' --- 't' --- nil
// <- 0 -> <- 1 -> <- 2 -> <- 3 ->

// Get keyspace-aware region boundaries by creating a temp store to access codec
tempStore, err := mockstore.NewMockStore()
require.NoError(t, err)
g := getKeyspaceAwareKey(tempStore, []byte("g"))
n := getKeyspaceAwareKey(tempStore, []byte("n"))
tKey := getKeyspaceAwareKey(tempStore, []byte("t"))
tempStore.Close()

store, err := mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c testutils.Cluster) {
mockstore.BootstrapWithMultiRegions(c, []byte("g"), []byte("n"), []byte("t"))
mockstore.BootstrapWithMultiRegions(c, g, n, tKey)
}),
)
require.NoError(t, err)
Expand Down
1 change: 1 addition & 0 deletions pkg/store/gcworker/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ go_test(
shard_count = 26,
deps = [
"//pkg/config",
"//pkg/config/kerneltype",
"//pkg/ddl/placement",
"//pkg/ddl/util",
"//pkg/domain",
Expand Down
16 changes: 16 additions & 0 deletions pkg/store/gcworker/gc_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/ddl/placement"
"github.com/pingcap/tidb/pkg/ddl/util"
"github.com/pingcap/tidb/pkg/domain"
Expand Down Expand Up @@ -314,6 +315,9 @@ func TestGetOracleTime(t *testing.T) {
}

func TestPrepareGC(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("skip TestPrepareGC when kernel type is NextGen - test not yet adjusted to support next-gen")
}
// as we are adjusting the base TS, we need a larger schema lease to avoid
// the info schema outdated error. as we keep adding offset to time oracle,
// so we need set a very large lease.
Expand Down Expand Up @@ -968,6 +972,9 @@ Loop:
}

func TestLeaderTick(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("skip TestLeaderTick when kernel type is NextGen - test not yet adjusted to support next-gen")
}
// as we are adjusting the base TS, we need a larger schema lease to avoid
// the info schema outdated error.
s := createGCWorkerSuiteWithStoreType(t, mockstore.EmbedUnistore, time.Hour)
Expand Down Expand Up @@ -1256,6 +1263,9 @@ func TestResolveLockRangeMeetRegionEnlargeCausedByRegionMerge(t *testing.T) {
}

func TestRunGCJob(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("skip TestRunGCJob when kernel type is NextGen - test not yet adjusted to support next-gen")
}
s := createGCWorkerSuite(t)

txnSafePointSyncWaitTime = 0
Expand Down Expand Up @@ -1502,6 +1512,9 @@ func TestGCLabelRules(t *testing.T) {
}

func TestGCWithPendingTxn(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("skip TestGCWithPendingTxn when kernel type is NextGen - test not yet adjusted to support next-gen")
}
s := createGCWorkerSuiteWithStoreType(t, mockstore.EmbedUnistore, 30*time.Minute)

ctx := gcContext()
Expand Down Expand Up @@ -1628,6 +1641,9 @@ func TestGCWithPendingTxn2(t *testing.T) {
}

func TestSkipGCAndOnlyResolveLock(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("skip TestSkipGCAndOnlyResolveLock when kernel type is NextGen - test not yet adjusted to support next-gen")
}
// as we are adjusting the base TS, we need a larger schema lease to avoid
// the info schema outdated error.
s := createGCWorkerSuiteWithStoreType(t, mockstore.EmbedUnistore, 10*time.Minute)
Expand Down
2 changes: 2 additions & 0 deletions pkg/store/helper/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,9 @@ go_test(
flaky = True,
shard_count = 6,
deps = [
"//pkg/config/kerneltype",
"//pkg/infoschema/context",
"//pkg/kv",
"//pkg/meta/model",
"//pkg/parser/ast",
"//pkg/store/mockstore",
Expand Down
23 changes: 22 additions & 1 deletion pkg/store/helper/helper_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,9 @@ import (

"github.com/gorilla/mux"
"github.com/pingcap/log"
"github.com/pingcap/tidb/pkg/config/kerneltype"
infoschema "github.com/pingcap/tidb/pkg/infoschema/context"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta/model"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/store/helper"
Expand All @@ -43,6 +45,18 @@ import (
"go.uber.org/zap"
)

// getKeyspaceAwareKey uses the store codec to encode keys properly in NextGen mode
// This ensures compatibility with keyspace encoding
func getKeyspaceAwareKey(store kv.Storage, key []byte) []byte {
if !kerneltype.IsNextGen() || store == nil {
return key
}

// Use the store's codec to encode the key - single source of truth
codec := store.GetCodec()
return codec.EncodeKey(key)
}

func TestHotRegion(t *testing.T) {
store := createMockStore(t)

Expand Down Expand Up @@ -151,9 +165,16 @@ func createMockStore(t *testing.T) (store helper.Storage) {
server := mockPDHTTPServer()

pdAddrs := []string{"invalid_pd_address", server.URL[len("http://"):]}

// Get keyspace-aware region boundary by creating a temp store to access codec
tempStore, err := teststore.NewMockStoreWithoutBootstrap()
require.NoError(t, err)
xKey := getKeyspaceAwareKey(tempStore, []byte("x"))
tempStore.Close()

s, err := teststore.NewMockStoreWithoutBootstrap(
mockstore.WithClusterInspector(func(c testutils.Cluster) {
mockstore.BootstrapWithMultiRegions(c, []byte("x"))
mockstore.BootstrapWithMultiRegions(c, xKey)
}),
mockstore.WithTiKVOptions(tikv.WithPDHTTPClient("store-helper-test", pdAddrs)),
mockstore.WithPDAddr(pdAddrs),
Expand Down
19 changes: 12 additions & 7 deletions pkg/telemetry/data_feature_usage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,13 +78,15 @@ func TestTxnUsageInfo(t *testing.T) {
txnUsage = telemetry.GetTxnUsageInfo(tk.Session())
require.True(t, txnUsage.RCWriteCheckTS)

tk.MustExec(fmt.Sprintf("set global %s = 0", vardef.TiDBPessimisticTransactionFairLocking))
txnUsage = telemetry.GetTxnUsageInfo(tk.Session())
require.False(t, txnUsage.FairLocking)

tk.MustExec(fmt.Sprintf("set global %s = 1", vardef.TiDBPessimisticTransactionFairLocking))
txnUsage = telemetry.GetTxnUsageInfo(tk.Session())
require.True(t, txnUsage.FairLocking)
if kerneltype.IsClassic() {
tk.MustExec(fmt.Sprintf("set global %s = 0", vardef.TiDBPessimisticTransactionFairLocking))
txnUsage = telemetry.GetTxnUsageInfo(tk.Session())
require.False(t, txnUsage.FairLocking)

tk.MustExec(fmt.Sprintf("set global %s = 1", vardef.TiDBPessimisticTransactionFairLocking))
txnUsage = telemetry.GetTxnUsageInfo(tk.Session())
require.True(t, txnUsage.FairLocking)
}
})

t.Run("Count", func(t *testing.T) {
Expand Down Expand Up @@ -855,6 +857,9 @@ func TestStoreBatchCopr(t *testing.T) {
}

func TestFairLockingUsage(t *testing.T) {
if kerneltype.IsNextGen() {
t.Skip("fair locking is not supported for next-gen yet")
}
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk2 := testkit.NewTestKit(t, store)
Expand Down