Skip to content

Commit

Permalink
ddl: do not switch to import mode during adding index (#48083) (#48460)
Browse files Browse the repository at this point in the history
close #48259
  • Loading branch information
ti-chi-bot authored Nov 10, 2023
1 parent 4bddd59 commit edd6080
Show file tree
Hide file tree
Showing 6 changed files with 24 additions and 26 deletions.
2 changes: 1 addition & 1 deletion pkg/ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -810,7 +810,7 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error {
}
defer d.sessPool.Put(ctx)

ingest.InitGlobalLightningEnv(d.ctx, ctx)
ingest.InitGlobalLightningEnv()

return nil
}
Expand Down
1 change: 1 addition & 0 deletions pkg/ddl/ingest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ go_library(
"//pkg/util/dbterror",
"//pkg/util/generic",
"//pkg/util/logutil",
"//pkg/util/memory",
"//pkg/util/size",
"@com_github_google_uuid//:uuid",
"@com_github_pingcap_errors//:errors",
Expand Down
21 changes: 6 additions & 15 deletions pkg/ddl/ingest/backend_mgr.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,6 @@ import (

"github.com/pingcap/tidb/br/pkg/lightning/backend/local"
"github.com/pingcap/tidb/br/pkg/lightning/config"
"github.com/pingcap/tidb/pkg/ddl/util"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/util/generic"
"github.com/pingcap/tidb/pkg/util/logutil"
kvutil "github.com/tikv/client-go/v2/util"
Expand All @@ -42,12 +40,11 @@ type BackendCtxMgr interface {

type litBackendCtxMgr struct {
generic.SyncMap[int64, *litBackendCtx]
memRoot MemRoot
diskRoot DiskRoot
isRaftKV2 bool
memRoot MemRoot
diskRoot DiskRoot
}

func newLitBackendCtxMgr(ctx context.Context, sctx sessionctx.Context, path string, memQuota uint64) BackendCtxMgr {
func newLitBackendCtxMgr(path string, memQuota uint64) BackendCtxMgr {
mgr := &litBackendCtxMgr{
SyncMap: generic.NewSyncMap[int64, *litBackendCtx](10),
memRoot: nil,
Expand All @@ -62,11 +59,6 @@ func newLitBackendCtxMgr(ctx context.Context, sctx sessionctx.Context, path stri
if err != nil {
logutil.BgLogger().Warn("ingest backfill may not be available", zap.String("category", "ddl-ingest"), zap.Error(err))
}
isRaftKV2, err := util.IsRaftKv2(ctx, sctx)
if err != nil {
logutil.BgLogger().Warn("failed to get 'storage.engine'", zap.String("category", "ddl-ingest"), zap.Error(err))
}
mgr.isRaftKV2 = isRaftKV2
return mgr
}

Expand Down Expand Up @@ -95,7 +87,7 @@ func (m *litBackendCtxMgr) Register(ctx context.Context, unique bool, jobID int6
if !ok {
return nil, genBackendAllocMemFailedErr(ctx, m.memRoot, jobID)
}
cfg, err := genConfig(ctx, m.memRoot, jobID, unique, m.isRaftKV2)
cfg, err := genConfig(ctx, m.memRoot, jobID, unique)
if err != nil {
logutil.Logger(ctx).Warn(LitWarnConfigError, zap.Int64("job ID", jobID), zap.Error(err))
return nil, err
Expand Down Expand Up @@ -130,10 +122,9 @@ func createLocalBackend(ctx context.Context, cfg *Config, resourceGroupName stri
regionSizeGetter := &local.TableRegionSizeGetterImpl{
DB: nil,
}
// We disable the switch TiKV mode feature for now,
// because the impact is not fully tested.
var raftKV2SwitchModeDuration time.Duration
if cfg.IsRaftKV2 {
raftKV2SwitchModeDuration = config.DefaultSwitchTiKVModeInterval
}
backendConfig := local.NewBackendConfig(cfg.Lightning, int(LitRLimit), cfg.KeyspaceName, resourceGroupName, kvutil.ExplicitTypeDDL, raftKV2SwitchModeDuration)
return local.NewBackend(ctx, tls, backendConfig, regionSizeGetter)
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ddl/ingest/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ type Config struct {
IsRaftKV2 bool
}

func genConfig(ctx context.Context, memRoot MemRoot, jobID int64, unique bool, isRaftKV2 bool) (*Config, error) {
func genConfig(ctx context.Context, memRoot MemRoot, jobID int64, unique bool) (*Config, error) {
tidbCfg := tidb.GetGlobalConfig()
cfg := lightning.NewConfig()
cfg.TikvImporter.Backend = lightning.BackendLocal
Expand Down Expand Up @@ -75,7 +75,7 @@ func genConfig(ctx context.Context, memRoot MemRoot, jobID int64, unique bool, i
c := &Config{
Lightning: cfg,
KeyspaceName: tidb.GetGlobalKeyspaceName(),
IsRaftKV2: isRaftKV2,
IsRaftKV2: false,
}

return c, err
Expand Down
18 changes: 12 additions & 6 deletions pkg/ddl/ingest/env.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,16 +15,15 @@
package ingest

import (
"context"
"os"
"path/filepath"
"strconv"

"github.com/pingcap/tidb/br/pkg/lightning/log"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/pingcap/tidb/pkg/util/memory"
"github.com/pingcap/tidb/pkg/util/size"
"go.uber.org/zap"
)
Expand All @@ -44,10 +43,10 @@ var (
LitInitialized bool
)

const maxMemoryQuota = 2 * size.GB
const defaultMemoryQuota = 2 * size.GB

// InitGlobalLightningEnv initialize Lightning backfill environment.
func InitGlobalLightningEnv(ctx context.Context, sctx sessionctx.Context) {
func InitGlobalLightningEnv() {
log.SetAppLogger(logutil.BgLogger())
globalCfg := config.GetGlobalConfig()
if globalCfg.Store != "tikv" {
Expand All @@ -66,12 +65,19 @@ func InitGlobalLightningEnv(ctx context.Context, sctx sessionctx.Context) {
return
}
LitSortPath = sPath
LitBackCtxMgr = newLitBackendCtxMgr(ctx, sctx, LitSortPath, maxMemoryQuota)
memTotal, err := memory.MemTotal()
if err != nil {
logutil.BgLogger().Warn("get total memory fail", zap.Error(err))
memTotal = defaultMemoryQuota
} else {
memTotal = memTotal / 2
}
LitBackCtxMgr = newLitBackendCtxMgr(LitSortPath, memTotal)
LitRLimit = util.GenRLimit("ddl-ingest")
LitInitialized = true
logutil.BgLogger().Info(LitInfoEnvInitSucc,
zap.String("category", "ddl-ingest"),
zap.Uint64("memory limitation", maxMemoryQuota),
zap.Uint64("memory limitation", memTotal),
zap.String("disk usage info", LitDiskRoot.UsageInfo()),
zap.Uint64("max open file number", LitRLimit),
zap.Bool("lightning is initialized", LitInitialized))
Expand Down
4 changes: 2 additions & 2 deletions pkg/ddl/util/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -328,7 +328,7 @@ func WrapKey2String(key []byte) string {
}

const (
getRaftKvVersionSQL = "show config where type = 'tikv' and name = 'storage.engine'"
getRaftKvVersionSQL = "select `value` from information_schema.cluster_config where type = 'tikv' and `key` = 'storage.engine'"
raftKv2 = "raft-kv2"
)

Expand Down Expand Up @@ -359,6 +359,6 @@ func IsRaftKv2(ctx context.Context, sctx sessionctx.Context) (bool, error) {
}

// All nodes should have the same type of engine
raftVersion := rows[0].GetString(3)
raftVersion := rows[0].GetString(0)
return raftVersion == raftKv2, nil
}

0 comments on commit edd6080

Please sign in to comment.