Skip to content

Commit

Permalink
store: make storage type a enum (#57276)
Browse files Browse the repository at this point in the history
ref #57275
  • Loading branch information
D3Hunter authored Nov 12, 2024
1 parent a5832b4 commit a4faee2
Show file tree
Hide file tree
Showing 38 changed files with 172 additions and 104 deletions.
4 changes: 2 additions & 2 deletions br/pkg/conn/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,8 +195,8 @@ func NewMgr(
return nil, errors.Trace(err)
}

if config.GetGlobalConfig().Store != "tikv" {
config.GetGlobalConfig().Store = "tikv"
if config.GetGlobalConfig().Store != config.StoreTypeTiKV {
config.GetGlobalConfig().Store = config.StoreTypeTiKV
}
// Disable GC because TiDB enables GC already.
path := fmt.Sprintf(
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/mock/mock_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ func (mock *Cluster) Start() error {
cfg := config.NewConfig()
// let tidb random select a port
cfg.Port = 0
cfg.Store = "tikv"
cfg.Store = config.StoreTypeTiKV
cfg.Status.StatusPort = 0
cfg.Status.ReportStatus = true
cfg.Socket = fmt.Sprintf("/tmp/tidb-mock-%d.sock", time.Now().UnixNano())
Expand Down
4 changes: 2 additions & 2 deletions cmd/benchdb/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ func main() {
flag.PrintDefaults()
err := logutil.InitLogger(logutil.NewLogConfig(*logLevel, logutil.DefaultLogFormat, "", "", logutil.EmptyFileLogConfig, false))
terror.MustNil(err)
err = store.Register("tikv", driver.TiKVDriver{})
err = store.Register(config.StoreTypeTiKV, driver.TiKVDriver{})
terror.MustNil(err)
ut := newBenchDB()
works := strings.Split(*runJobs, "|")
Expand Down Expand Up @@ -100,7 +100,7 @@ func newBenchDB() *benchDB {
terror.MustNil(err)
// maybe close below components, but it's for test anyway.
ctx := context.Background()
config.GetGlobalConfig().Store = "tikv"
config.GetGlobalConfig().Store = config.StoreTypeTiKV
err = ddl.StartOwnerManager(ctx, store)
terror.MustNil(err)
_, err = session.BootstrapSession(store)
Expand Down
4 changes: 2 additions & 2 deletions cmd/ddltest/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ func createDDLSuite(t *testing.T) (s *ddlSuite) {

s.quit = make(chan struct{})

config.GetGlobalConfig().Store = "tikv"
config.GetGlobalConfig().Store = config.StoreTypeTiKV
s.store, err = store.New(fmt.Sprintf("tikv://%s%s", *etcd, *tikvPath))
require.NoError(t, err)

Expand Down Expand Up @@ -1161,5 +1161,5 @@ func addEnvPath(newPath string) {
}

func init() {
_ = store.Register("tikv", tidbdriver.TiKVDriver{})
_ = store.Register(config.StoreTypeTiKV, tidbdriver.TiKVDriver{})
}
10 changes: 5 additions & 5 deletions cmd/tidb-server/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -198,7 +198,7 @@ func initFlagSet() *flag.FlagSet {
configStrict = flagBoolean(fset, nmConfigStrict, false, "enforce config file validity")

// Base
store = fset.String(nmStore, "unistore", "registered store name, [tikv, mocktikv, unistore]")
store = fset.String(nmStore, string(config.StoreTypeUniStore), fmt.Sprintf("registered store name, %v", config.StoreTypeList()))
storePath = fset.String(nmStorePath, "/tmp/tidb", "tidb storage path")
host = fset.String(nmHost, "0.0.0.0", "tidb server host")
advertiseAddress = fset.String(nmAdvertiseAddress, "", "tidb server advertise IP")
Expand Down Expand Up @@ -389,11 +389,11 @@ func setCPUAffinity() {
}

func registerStores() {
err := kvstore.Register("tikv", driver.TiKVDriver{})
err := kvstore.Register(config.StoreTypeTiKV, driver.TiKVDriver{})
terror.MustNil(err)
err = kvstore.Register("mocktikv", mockstore.MockTiKVDriver{})
err = kvstore.Register(config.StoreTypeMockTiKV, mockstore.MockTiKVDriver{})
terror.MustNil(err)
err = kvstore.Register("unistore", mockstore.EmbedUnistoreDriver{})
err = kvstore.Register(config.StoreTypeUniStore, mockstore.EmbedUnistoreDriver{})
terror.MustNil(err)
}

Expand Down Expand Up @@ -513,7 +513,7 @@ func overrideConfig(cfg *config.Config, fset *flag.FlagSet) {
cfg.Cors = *cors
}
if actualFlags[nmStore] {
cfg.Store = *store
cfg.Store = config.StoreType(*store)
}
if actualFlags[nmStorePath] {
cfg.Path = *storePath
Expand Down
2 changes: 1 addition & 1 deletion dumpling/export/sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -907,7 +907,7 @@ func CheckTiDBWithTiKV(db *sql.DB) (bool, error) {
}()
tidbConfig, err := getTiDBConfig(conn)
if err == nil {
return tidbConfig.Store == "tikv", nil
return tidbConfig.Store == dbconfig.StoreTypeTiKV, nil
}
}
var count int
Expand Down
5 changes: 2 additions & 3 deletions dumpling/export/sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1618,16 +1618,15 @@ func TestCheckTiDBWithTiKV(t *testing.T) {
}()

tidbConf := dbconfig.NewConfig()
stores := []string{"unistore", "mocktikv", "tikv"}
for _, store := range stores {
for _, store := range dbconfig.StoreTypeList() {
tidbConf.Store = store
tidbConfBytes, err := json.Marshal(tidbConf)
require.NoError(t, err)
mock.ExpectQuery("SELECT @@tidb_config").WillReturnRows(
sqlmock.NewRows([]string{"@@tidb_config"}).AddRow(string(tidbConfBytes)))
hasTiKV, err := CheckTiDBWithTiKV(db)
require.NoError(t, err)
if store == "tikv" {
if store == dbconfig.StoreTypeTiKV {
require.True(t, hasTiKV)
} else {
require.False(t, hasTiKV)
Expand Down
4 changes: 3 additions & 1 deletion pkg/config/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ go_library(
"config.go",
"config_util.go",
"const.go",
"store.go",
],
importpath = "github.com/pingcap/tidb/pkg/config",
visibility = ["//visibility:public"],
Expand Down Expand Up @@ -33,11 +34,12 @@ go_test(
"config_test.go",
"config_util_test.go",
"main_test.go",
"store_test.go",
],
data = glob(["**"]),
embed = [":config"],
flaky = True,
shard_count = 25,
shard_count = 26,
deps = [
"//pkg/testkit/testsetup",
"//pkg/util/logutil",
Expand Down
43 changes: 16 additions & 27 deletions pkg/config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,11 +105,6 @@ const (

// Valid config maps
var (
ValidStorage = map[string]bool{
"mocktikv": true,
"tikv": true,
"unistore": true,
}
// CheckTableBeforeDrop enable to execute `admin check table` before `drop table`.
CheckTableBeforeDrop = false
// checkBeforeDropLDFlag is a go build flag.
Expand Down Expand Up @@ -175,18 +170,18 @@ var (

// Config contains configuration options.
type Config struct {
Host string `toml:"host" json:"host"`
AdvertiseAddress string `toml:"advertise-address" json:"advertise-address"`
Port uint `toml:"port" json:"port"`
Cors string `toml:"cors" json:"cors"`
Store string `toml:"store" json:"store"`
Path string `toml:"path" json:"path"`
Socket string `toml:"socket" json:"socket"`
Lease string `toml:"lease" json:"lease"`
SplitTable bool `toml:"split-table" json:"split-table"`
TokenLimit uint `toml:"token-limit" json:"token-limit"`
TempDir string `toml:"temp-dir" json:"temp-dir"`
TempStoragePath string `toml:"tmp-storage-path" json:"tmp-storage-path"`
Host string `toml:"host" json:"host"`
AdvertiseAddress string `toml:"advertise-address" json:"advertise-address"`
Port uint `toml:"port" json:"port"`
Cors string `toml:"cors" json:"cors"`
Store StoreType `toml:"store" json:"store"`
Path string `toml:"path" json:"path"`
Socket string `toml:"socket" json:"socket"`
Lease string `toml:"lease" json:"lease"`
SplitTable bool `toml:"split-table" json:"split-table"`
TokenLimit uint `toml:"token-limit" json:"token-limit"`
TempDir string `toml:"temp-dir" json:"temp-dir"`
TempStoragePath string `toml:"tmp-storage-path" json:"tmp-storage-path"`
// TempStorageQuota describe the temporary storage Quota during query exector when TiDBEnableTmpStorageOnOOM is enabled
// If the quota exceed the capacity of the TempStoragePath, the tidb-server would exit with fatal error
TempStorageQuota int64 `toml:"tmp-storage-quota" json:"tmp-storage-quota"` // Bytes
Expand Down Expand Up @@ -898,7 +893,7 @@ var defaultConf = Config{
Port: DefPort,
Socket: "/tmp/tidb-{Port}.sock",
Cors: "",
Store: "unistore",
Store: StoreTypeUniStore,
Path: "/tmp/tidb",
RunDDL: true,
SplitTable: true,
Expand Down Expand Up @@ -1322,16 +1317,10 @@ func (c *Config) Valid() error {
if c.Security.SkipGrantTable && !hasRootPrivilege() {
return fmt.Errorf("TiDB run with skip-grant-table need root privilege")
}
if !ValidStorage[c.Store] {
nameList := make([]string, 0, len(ValidStorage))
for k, v := range ValidStorage {
if v {
nameList = append(nameList, k)
}
}
return fmt.Errorf("invalid store=%s, valid storages=%v", c.Store, nameList)
if !c.Store.Valid() {
return fmt.Errorf("invalid store=%s, valid storages=%v", c.Store, StoreTypeList())
}
if c.Store == "mocktikv" && !c.Instance.TiDBEnableDDL.Load() {
if c.Store == StoreTypeMockTiKV && !c.Instance.TiDBEnableDDL.Load() {
return fmt.Errorf("can't disable DDL on mocktikv")
}
if c.MaxIndexLength < DefMaxIndexLength || c.MaxIndexLength > DefMaxOfMaxIndexLength {
Expand Down
48 changes: 48 additions & 0 deletions pkg/config/store.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
// Copyright 2024 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 config

// StoreType is the type of storage.
// TODO maybe put it inside pkg/store, but it introduces a cycle import.
type StoreType string

const (
// StoreTypeTiKV is TiKV type. the underlying storage engines might be one or
// multiple of TiKV/TiFlash/TiDB, see kv.StoreType for more details.
StoreTypeTiKV StoreType = "tikv"
// StoreTypeUniStore is UniStore type which we implemented using badger, for test only.
StoreTypeUniStore StoreType = "unistore"
// StoreTypeMockTiKV is MockTiKV type which we implemented using goleveldb, for test only.
StoreTypeMockTiKV StoreType = "mocktikv"
)

// String implements fmt.Stringer interface.
func (t StoreType) String() string {
return string(t)
}

// Valid returns true if the storage type is valid.
func (t StoreType) Valid() bool {
switch t {
case StoreTypeTiKV, StoreTypeUniStore, StoreTypeMockTiKV:
return true
}
return false
}

// StoreTypeList returns all valid storage types.
func StoreTypeList() []StoreType {
return []StoreType{StoreTypeTiKV, StoreTypeUniStore, StoreTypeMockTiKV}
}
28 changes: 28 additions & 0 deletions pkg/config/store_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
// Copyright 2024 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 config

import (
"testing"

"github.com/stretchr/testify/require"
)

func TestStoreType(t *testing.T) {
require.Len(t, StoreTypeList(), 3)
for _, tp := range StoreTypeList() {
require.True(t, tp.Valid())
}
}
4 changes: 2 additions & 2 deletions pkg/ddl/ingest/env.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,10 +56,10 @@ const defaultMemoryQuota = 2 * size.GB
func InitGlobalLightningEnv(path string) (ok bool) {
log.SetAppLogger(logutil.DDLIngestLogger())
globalCfg := config.GetGlobalConfig()
if globalCfg.Store != "tikv" {
if globalCfg.Store != config.StoreTypeTiKV {
logutil.DDLIngestLogger().Warn(LitWarnEnvInitFail,
zap.String("storage limitation", "only support TiKV storage"),
zap.String("current storage", globalCfg.Store),
zap.Stringer("current storage", globalCfg.Store),
zap.Bool("lightning is initialized", LitInitialized))
return false
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ddl/owner_mgr.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func (om *ownerManager) Start(ctx context.Context, store kv.Storage) error {
if om.started {
return nil
}
if config.GetGlobalConfig().Store != "tikv" {
if config.GetGlobalConfig().Store != config.StoreTypeTiKV {
return nil
}
cli, err := storepkg.NewEtcdCli(store)
Expand Down
4 changes: 2 additions & 2 deletions pkg/ddl/owner_mgr_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ func TestOwnerManager(t *testing.T) {
config.GetGlobalConfig().Store = bak
globalOwnerManager = &ownerManager{}
})
config.GetGlobalConfig().Store = "unistore"
config.GetGlobalConfig().Store = config.StoreTypeUniStore
globalOwnerManager = &ownerManager{}
ctx := context.Background()
require.NoError(t, StartOwnerManager(ctx, nil))
Expand All @@ -47,7 +47,7 @@ func TestOwnerManager(t *testing.T) {
testfailpoint.EnableCall(t, "github.com/pingcap/tidb/pkg/ddl/injectEtcdClient", func(cliP **clientv3.Client) {
*cliP = cli
})
config.GetGlobalConfig().Store = "tikv"
config.GetGlobalConfig().Store = config.StoreTypeTiKV
require.NoError(t, StartOwnerManager(ctx, nil))
require.Same(t, cli, globalOwnerManager.etcdCli)
require.NotEmpty(t, globalOwnerManager.id)
Expand Down
2 changes: 1 addition & 1 deletion pkg/disttask/framework/integrationtests/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ func prepareForBenchTest(b *testing.B) {
var err error
store, err := d.Open("tikv://" + *testkit.WithTiKV)
require.NoError(b, err)
config.GetGlobalConfig().Store = "tikv"
config.GetGlobalConfig().Store = config.StoreTypeTiKV
require.NoError(b, ddl.StartOwnerManager(context.Background(), store))
var dom *domain.Domain
dom, err = session.BootstrapSession(store)
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/brie.go
Original file line number Diff line number Diff line change
Expand Up @@ -315,9 +315,9 @@ func (b *executorBuilder) buildBRIE(s *ast.BRIEStmt, schema *expression.Schema)

store := tidbCfg.Store
failpoint.Inject("modifyStore", func(v failpoint.Value) {
store = v.(string)
store = config.StoreType(v.(string))
})
if store != "tikv" {
if store != config.StoreTypeTiKV {
b.err = errors.Errorf("%s requires tikv store, not %s", s.Kind, store)
return nil
}
Expand Down
12 changes: 7 additions & 5 deletions pkg/kv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -346,17 +346,19 @@ const (
ReqSubTypeAnalyzeCol = 10005
)

// StoreType represents the type of a store.
// StoreType represents the type of storage engine.
type StoreType uint8

const (
// TiKV means the type of a store is TiKV.
// TiKV means the type of store engine is TiKV.
TiKV StoreType = iota
// TiFlash means the type of a store is TiFlash.
// TiFlash means the type of store engine is TiFlash.
TiFlash
// TiDB means the type of a store is TiDB.
// TiDB means the type of store engine is TiDB.
// used to read memory data from other instances to have a global view of the
// data, such as for information_schema.cluster_slow_query.
TiDB
// UnSpecified means the store type is unknown
// UnSpecified means the store engine type is unknown
UnSpecified = 255
)

Expand Down
2 changes: 1 addition & 1 deletion pkg/server/handler/tests/http_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -396,7 +396,7 @@ func (ts *basicHTTPHandlerTestSuite) startServer(t *testing.T) {
ts.tidbdrv = server2.NewTiDBDriver(ts.store)

cfg := util.NewTestConfig()
cfg.Store = "tikv"
cfg.Store = config.StoreTypeTiKV
cfg.Port = 0
cfg.Status.StatusPort = 0
cfg.Status.ReportStatus = true
Expand Down
4 changes: 2 additions & 2 deletions pkg/server/http_status.go
Original file line number Diff line number Diff line change
Expand Up @@ -255,7 +255,7 @@ func (s *Server) startHTTPServer() {
// HTTP path for upgrade operations.
router.Handle("/upgrade/{op}", handler.NewClusterUpgradeHandler(tikvHandlerTool.Store.(kv.Storage))).Name("upgrade operations")

if s.cfg.Store == "tikv" {
if s.cfg.Store == config.StoreTypeTiKV {
// HTTP path for tikv.
router.Handle("/tables/{db}/{table}/regions", tikvhandler.NewTableHandler(tikvHandlerTool, tikvhandler.OpTableRegions))
router.Handle("/tables/{db}/{table}/ranges", tikvhandler.NewTableHandler(tikvHandlerTool, tikvhandler.OpTableRanges))
Expand Down Expand Up @@ -481,7 +481,7 @@ func (s *Server) startStatusServerAndRPCServer(serverMux *http.ServeMux) {
statusServer := &http.Server{Addr: s.statusAddr, Handler: util2.NewCorsHandler(serverMux, s.cfg)}
grpcServer := NewRPCServer(s.cfg, s.dom, s)
service.RegisterChannelzServiceToServer(grpcServer)
if s.cfg.Store == "tikv" {
if s.cfg.Store == config.StoreTypeTiKV {
keyspaceName := config.GetGlobalKeyspaceName()
for {
var fullPath string
Expand Down
Loading

0 comments on commit a4faee2

Please sign in to comment.