test: fix standalone TiDB of next-gen report 'invalid key' (#63043)
ref pingcap/tidb#61702
This commit is contained in:
@ -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()
|
||||
|
||||
@ -34,6 +34,7 @@ go_test(
|
||||
flaky = True,
|
||||
shard_count = 3,
|
||||
deps = [
|
||||
"//pkg/config/kerneltype",
|
||||
"//pkg/parser/ast",
|
||||
"//pkg/store/mockstore",
|
||||
"//pkg/testkit",
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -426,6 +426,7 @@ go_test(
|
||||
"//pkg/expression/exprstatic",
|
||||
"//pkg/extension",
|
||||
"//pkg/infoschema",
|
||||
"//pkg/keyspace",
|
||||
"//pkg/kv",
|
||||
"//pkg/lightning/log",
|
||||
"//pkg/meta",
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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,
|
||||
}
|
||||
|
||||
|
||||
@ -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
21
pkg/kv/unistore.go
Normal 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
|
||||
@ -165,6 +165,7 @@ go_test(
|
||||
deps = [
|
||||
"//pkg/autoid_service",
|
||||
"//pkg/config",
|
||||
"//pkg/config/kerneltype",
|
||||
"//pkg/ddl",
|
||||
"//pkg/domain",
|
||||
"//pkg/executor",
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
|
||||
@ -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,
|
||||
}
|
||||
}
|
||||
|
||||
@ -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",
|
||||
],
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
67
pkg/util/rowcodec/common_test.go
Normal file
67
pkg/util/rowcodec/common_test.go
Normal 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))
|
||||
})
|
||||
}
|
||||
Reference in New Issue
Block a user