test: fix standalone TiDB of next-gen report 'invalid key' (#63043)

ref pingcap/tidb#61702
This commit is contained in:
D3Hunter
2025-08-19 16:38:11 +08:00
committed by GitHub
parent 7dfc503c25
commit ec45f18b5d
15 changed files with 138 additions and 15 deletions

View File

@ -476,6 +476,9 @@ func registerStores() error {
}
func createStoreDDLOwnerMgrAndDomain(keyspaceName string) (kv.Storage, *domain.Domain, error) {
if config.GetGlobalConfig().Store == config.StoreTypeUniStore {
kv.StandAloneTiDB = true
}
storage := kvstore.MustInitStorage(keyspaceName)
if tikvStore, ok := storage.(kv.StorageWithPD); ok {
pdhttpCli := tikvStore.GetPDHTTPClient()

View File

@ -34,6 +34,7 @@ go_test(
flaky = True,
shard_count = 3,
deps = [
"//pkg/config/kerneltype",
"//pkg/parser/ast",
"//pkg/store/mockstore",
"//pkg/testkit",

View File

@ -24,6 +24,7 @@ import (
"github.com/pingcap/kvproto/pkg/autoid"
"github.com/pingcap/kvproto/pkg/keyspacepb"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/store/mockstore"
"github.com/pingcap/tidb/pkg/testkit"
@ -142,11 +143,13 @@ func TestAPI(t *testing.T) {
// Testing scenarios without keyspace.
testAPIWithKeyspace(t, nil)
// Testing scenarios with keyspace.
keyspaceMeta := keyspacepb.KeyspaceMeta{}
keyspaceMeta.Id = 2
keyspaceMeta.Name = "test_ks_name2"
testAPIWithKeyspace(t, &keyspaceMeta)
if kerneltype.IsNextGen() {
// Testing scenarios with keyspace.
keyspaceMeta := keyspacepb.KeyspaceMeta{}
keyspaceMeta.Id = 2
keyspaceMeta.Name = "test_ks_name2"
testAPIWithKeyspace(t, &keyspaceMeta)
}
}
func testAPIWithKeyspace(t *testing.T, keyspaceMeta *keyspacepb.KeyspaceMeta) {

View File

@ -426,6 +426,7 @@ go_test(
"//pkg/expression/exprstatic",
"//pkg/extension",
"//pkg/infoschema",
"//pkg/keyspace",
"//pkg/kv",
"//pkg/lightning/log",
"//pkg/meta",

View File

@ -20,7 +20,9 @@ import (
"testing"
"github.com/pingcap/kvproto/pkg/keyspacepb"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/keyspace"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
"github.com/pingcap/tidb/pkg/store/mockstore"
@ -132,10 +134,13 @@ func TestMaxChunkSize(t *testing.T) {
}
func TestKeyspaceSample(t *testing.T) {
if kerneltype.IsClassic() {
t.Skip("Keyspace is not supported in classic mode")
}
// Build an exist keyspace.
keyspaceMeta := keyspacepb.KeyspaceMeta{}
keyspaceMeta.Id = 2
keyspaceMeta.Name = "test_ks_name2"
keyspaceMeta.Name = keyspace.System
opts := mockstore.WithKeyspaceMeta(&keyspaceMeta)
store := testkit.CreateMockStore(t, opts)
@ -147,7 +152,7 @@ func TestKeyspaceSample(t *testing.T) {
// Build another exist keyspace.
keyspaceMeta02 := keyspacepb.KeyspaceMeta{}
keyspaceMeta02.Id = 3
keyspaceMeta02.Name = "test_ks_name3"
keyspaceMeta02.Name = keyspace.System
opts02 := mockstore.WithKeyspaceMeta(&keyspaceMeta02)
store02 := testkit.CreateMockStore(t, opts02)

View File

@ -11,7 +11,9 @@ go_test(
shard_count = 19,
deps = [
"//pkg/config",
"//pkg/config/kerneltype",
"//pkg/domain/infosync",
"//pkg/keyspace",
"//pkg/meta/autoid",
"//pkg/meta/model",
"//pkg/parser/auth",

View File

@ -28,7 +28,9 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/keyspacepb"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/domain/infosync"
"github.com/pingcap/tidb/pkg/keyspace"
"github.com/pingcap/tidb/pkg/meta/model"
"github.com/pingcap/tidb/pkg/parser/auth"
"github.com/pingcap/tidb/pkg/parser/mysql"
@ -1138,8 +1140,10 @@ func TestIndexUsageWithData(t *testing.T) {
}
func TestKeyspaceMeta(t *testing.T) {
if kerneltype.IsClassic() {
t.Skip("Keyspace is not supported in classic mode")
}
keyspaceID := rand.Uint32() >> 8
keyspaceName := fmt.Sprintf("keyspace-%d", keyspaceID)
cfg := map[string]string{
"key_a": "a",
"key_b": "b",
@ -1147,7 +1151,7 @@ func TestKeyspaceMeta(t *testing.T) {
keyspaceMeta := &keyspacepb.KeyspaceMeta{
Id: keyspaceID,
Name: keyspaceName,
Name: keyspace.System,
Config: cfg,
}

View File

@ -15,6 +15,7 @@ go_library(
"option.go",
"txn.go",
"txn_scope_var.go",
"unistore.go",
"utils.go",
"variables.go",
"version.go",

21
pkg/kv/unistore.go Normal file
View File

@ -0,0 +1,21 @@
// Copyright 2025 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package kv
// StandAloneTiDB indicates whether the current TiDB instance is running in
// standalone mode. In this mode, TiDB uses uni-store as the storage engine.
// we add this flag only for nextgen, as uni-store is not removing the keyspace
// prefix from the keys.
var StandAloneTiDB bool

View File

@ -165,6 +165,7 @@ go_test(
deps = [
"//pkg/autoid_service",
"//pkg/config",
"//pkg/config/kerneltype",
"//pkg/ddl",
"//pkg/domain",
"//pkg/executor",

View File

@ -27,6 +27,7 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/keyspacepb"
"github.com/pingcap/log"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/domain"
"github.com/pingcap/tidb/pkg/expression/sessionexpr"
@ -2411,9 +2412,12 @@ func TestIndexJoinMultiPatternByUpgrade650To840(t *testing.T) {
}
func TestKeyspaceEtcdNamespace(t *testing.T) {
if kerneltype.IsClassic() {
t.Skip("keyspace is not supported in classic kernel")
}
keyspaceMeta := keyspacepb.KeyspaceMeta{}
keyspaceMeta.Id = 2
keyspaceMeta.Name = "test_ks_name2"
keyspaceMeta.Name = keyspace.System
makeStore(t, &keyspaceMeta, true)
}

View File

@ -231,7 +231,7 @@ func NewMockStore(options ...MockTiKVStoreOption) (kv.Storage, error) {
// test, we set the default keyspace meta to system keyspace.
if opt.keyspaceMeta == nil {
opt.keyspaceMeta = &keyspacepb.KeyspaceMeta{
Id: 1,
Id: uint32(0xFFFFFF) - 1,
Name: keyspace.System,
}
}

View File

@ -11,6 +11,7 @@ go_library(
importpath = "github.com/pingcap/tidb/pkg/util/rowcodec",
visibility = ["//visibility:public"],
deps = [
"//pkg/config/kerneltype",
"//pkg/kv",
"//pkg/meta/model",
"//pkg/parser/mysql",
@ -29,12 +30,14 @@ go_test(
timeout = "short",
srcs = [
"bench_test.go",
"common_test.go",
"main_test.go",
"rowcodec_test.go",
],
embed = [":rowcodec"],
flaky = True,
deps = [
"//pkg/config/kerneltype",
"//pkg/kv",
"//pkg/meta/model",
"//pkg/parser/mysql",
@ -46,6 +49,7 @@ go_test(
"//pkg/util/chunk",
"//pkg/util/codec",
"//pkg/util/collate",
"//pkg/util/intest",
"@com_github_stretchr_testify//require",
"@org_uber_go_goleak//:goleak",
],

View File

@ -22,6 +22,8 @@ import (
"unsafe"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/parser/types"
@ -349,11 +351,15 @@ func appendLengthValue(buf []byte, val []byte) []byte {
return append(buf, val...)
}
// RemoveKeyspacePrefix is used to remove keyspace prefix from the key.
// RemoveKeyspacePrefix is used to remove keyspace prefix from the key if it's
// nextgen kernel.
func RemoveKeyspacePrefix(key []byte) []byte {
// If it is not a UT scenario, the operation to remove the keyspace prefix is performed in client-go,
// so there is no need to remove it again.
if !intest.InTest {
if kerneltype.IsClassic() {
return key
}
// If it is not in UT and not run in standalone TiDB, the removing of the
// keyspace prefix from the keys is performed in client-go.
if !intest.InTest && !kv.StandAloneTiDB {
return key
}

View File

@ -0,0 +1,67 @@
// Copyright 2025 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package rowcodec
import (
"encoding/hex"
"testing"
"github.com/pingcap/tidb/pkg/config/kerneltype"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/util/intest"
"github.com/stretchr/testify/require"
)
func TestRemoveKeyspacePrefix(t *testing.T) {
// this is the default and expected value for UT, else some UT might change
// them and forget to revert them back.
require.True(t, intest.InTest)
require.False(t, kv.StandAloneTiDB)
nextGenKey, err := hex.DecodeString("78000001748000fffffffffffe5F728000000000000002")
require.NoError(t, err)
classicKey := nextGenKey[4:]
if kerneltype.IsClassic() {
require.EqualValues(t, nextGenKey, RemoveKeyspacePrefix(nextGenKey))
require.EqualValues(t, classicKey, RemoveKeyspacePrefix(classicKey))
return
}
t.Run("when intest enabled, keyspace prefix should be removed", func(t *testing.T) {
require.EqualValues(t, nextGenKey[4:], RemoveKeyspacePrefix(nextGenKey))
require.EqualValues(t, classicKey, RemoveKeyspacePrefix(classicKey))
kv.StandAloneTiDB = true
t.Cleanup(func() {
kv.StandAloneTiDB = false
})
require.EqualValues(t, nextGenKey[4:], RemoveKeyspacePrefix(nextGenKey))
require.EqualValues(t, classicKey, RemoveKeyspacePrefix(classicKey))
})
t.Run("when intest disabled, keyspace prefix should be removed only when run in standalone mode", func(t *testing.T) {
intest.InTest = false
t.Cleanup(func() {
intest.InTest = true
})
require.EqualValues(t, nextGenKey, RemoveKeyspacePrefix(nextGenKey))
require.EqualValues(t, classicKey, RemoveKeyspacePrefix(classicKey))
kv.StandAloneTiDB = true
t.Cleanup(func() {
kv.StandAloneTiDB = false
})
require.EqualValues(t, nextGenKey[4:], RemoveKeyspacePrefix(nextGenKey))
require.EqualValues(t, classicKey, RemoveKeyspacePrefix(classicKey))
})
}