Skip to content

Commit

Permalink
Revert "store/tikv: move EtcdBackend out (pingcap#22676)"
Browse files Browse the repository at this point in the history
This reverts commit 7a275a7.
  • Loading branch information
ichn-hu authored Mar 10, 2021
1 parent 6f76357 commit d1f970b
Show file tree
Hide file tree
Showing 10 changed files with 19 additions and 21 deletions.
2 changes: 1 addition & 1 deletion domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -700,7 +700,7 @@ const serverIDForStandalone = 1 // serverID for standalone deployment.
// Init initializes a domain.
func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.Resource, error)) error {
perfschema.Init()
if ebd, ok := do.store.(kv.EtcdBackend); ok {
if ebd, ok := do.store.(tikv.EtcdBackend); ok {
var addrs []string
var err error
if addrs, err = ebd.EtcdAddrs(); err != nil {
Expand Down
3 changes: 1 addition & 2 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@ import (
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/domain/infosync"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/privilege"
Expand Down Expand Up @@ -1981,7 +1980,7 @@ type tiflashInstanceInfo struct {

func (e *TiFlashSystemTableRetriever) initialize(sctx sessionctx.Context, tiflashInstances set.StringSet) error {
store := sctx.GetStore()
if etcd, ok := store.(kv.EtcdBackend); ok {
if etcd, ok := store.(tikv.EtcdBackend); ok {
var addrs []string
var err error
if addrs, err = etcd.EtcdAddrs(); err != nil {
Expand Down
2 changes: 1 addition & 1 deletion executor/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -802,7 +802,7 @@ func getRegionMeta(tikvStore helper.Storage, regionMetas []*tikv.Region, uniqueR

func getRegionInfo(store helper.Storage, regions []regionMeta) ([]regionMeta, error) {
// check pd server exists.
etcd, ok := store.(kv.EtcdBackend)
etcd, ok := store.(tikv.EtcdBackend)
if !ok {
return regions, nil
}
Expand Down
2 changes: 1 addition & 1 deletion infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -1460,7 +1460,7 @@ func FormatVersion(TiDBVersion string, isDefaultVersion bool) string {
func GetPDServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) {
// Get PD servers info.
store := ctx.GetStore()
etcd, ok := store.(kv.EtcdBackend)
etcd, ok := store.(tikv.EtcdBackend)
if !ok {
return nil, errors.Errorf("%T not an etcd backend", store)
}
Expand Down
8 changes: 0 additions & 8 deletions kv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ package kv

import (
"context"
"crypto/tls"
"sync"
"time"

Expand Down Expand Up @@ -533,13 +532,6 @@ type Storage interface {
GetMemCache() MemManager
}

// EtcdBackend is used for judging a storage is a real TiKV.
type EtcdBackend interface {
EtcdAddrs() ([]string, error)
TLSConfig() *tls.Config
StartGCWorker() error
}

// FnKeyCmp is the function for iterator the keys
type FnKeyCmp func(key Key) bool

Expand Down
2 changes: 1 addition & 1 deletion server/http_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -1189,7 +1189,7 @@ func (h ddlResignOwnerHandler) ServeHTTP(w http.ResponseWriter, req *http.Reques
}

func (h tableHandler) getPDAddr() ([]string, error) {
etcd, ok := h.Store.(kv.EtcdBackend)
etcd, ok := h.Store.(tikv.EtcdBackend)
if !ok {
return nil, errors.New("not implemented")
}
Expand Down
2 changes: 1 addition & 1 deletion session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -2228,7 +2228,7 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) {
if err != nil {
return nil, err
}
if raw, ok := store.(kv.EtcdBackend); ok {
if raw, ok := store.(tikv.EtcdBackend); ok {
err = raw.StartGCWorker()
if err != nil {
return nil, err
Expand Down
4 changes: 2 additions & 2 deletions session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ func clearStorage(store kv.Storage) error {
return txn.Commit(context.Background())
}

func clearETCD(ebd kv.EtcdBackend) error {
func clearETCD(ebd tikv.EtcdBackend) error {
endpoints, err := ebd.EtcdAddrs()
if err != nil {
return err
Expand Down Expand Up @@ -190,7 +190,7 @@ func (s *testSessionSuiteBase) SetUpSuite(c *C) {
c.Assert(err, IsNil)
err = clearStorage(store)
c.Assert(err, IsNil)
err = clearETCD(store.(kv.EtcdBackend))
err = clearETCD(store.(tikv.EtcdBackend))
c.Assert(err, IsNil)
session.ResetStoreForWithTiKVTest(store)
s.store = store
Expand Down
8 changes: 4 additions & 4 deletions store/helper/helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ func (h *Helper) ScrapeHotInfo(rw string, allSchemas []*model.DBInfo) ([]HotTabl

// FetchHotRegion fetches the hot region information from PD's http api.
func (h *Helper) FetchHotRegion(rw string) (map[uint64]RegionMetric, error) {
etcd, ok := h.Store.(kv.EtcdBackend)
etcd, ok := h.Store.(tikv.EtcdBackend)
if !ok {
return nil, errors.WithStack(errors.New("not implemented"))
}
Expand Down Expand Up @@ -690,7 +690,7 @@ func (h *Helper) GetRegionInfoByID(regionID uint64) (*RegionInfo, error) {

// request PD API, decode the response body into res
func (h *Helper) requestPD(method, uri string, body io.Reader, res interface{}) error {
etcd, ok := h.Store.(kv.EtcdBackend)
etcd, ok := h.Store.(tikv.EtcdBackend)
if !ok {
return errors.WithStack(errors.New("not implemented"))
}
Expand Down Expand Up @@ -776,7 +776,7 @@ type StoreDetailStat struct {

// GetStoresStat gets the TiKV store information by accessing PD's api.
func (h *Helper) GetStoresStat() (*StoresStat, error) {
etcd, ok := h.Store.(kv.EtcdBackend)
etcd, ok := h.Store.(tikv.EtcdBackend)
if !ok {
return nil, errors.WithStack(errors.New("not implemented"))
}
Expand Down Expand Up @@ -811,7 +811,7 @@ func (h *Helper) GetStoresStat() (*StoresStat, error) {

// GetPDAddr return the PD Address.
func (h *Helper) GetPDAddr() ([]string, error) {
etcd, ok := h.Store.(kv.EtcdBackend)
etcd, ok := h.Store.(tikv.EtcdBackend)
if !ok {
return nil, errors.New("not implemented")
}
Expand Down
7 changes: 7 additions & 0 deletions store/tikv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,13 @@ func createEtcdKV(addrs []string, tlsConfig *tls.Config) (*clientv3.Client, erro
return cli, nil
}

// EtcdBackend is used for judging a storage is a real TiKV.
type EtcdBackend interface {
EtcdAddrs() ([]string, error)
TLSConfig() *tls.Config
StartGCWorker() error
}

// update oracle's lastTS every 2000ms.
var oracleUpdateInterval = 2000

Expand Down

0 comments on commit d1f970b

Please sign in to comment.