Skip to content

Commit

Permalink
address some comments
Browse files Browse the repository at this point in the history
  • Loading branch information
3pointer committed Feb 25, 2024
1 parent 21b478d commit c04e0ad
Show file tree
Hide file tree
Showing 6 changed files with 26 additions and 26 deletions.
12 changes: 6 additions & 6 deletions br/pkg/config/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,15 +7,15 @@ import (
"github.com/docker/go-units"
)

type ConfigSet[T uint | uint64] struct {
Value T
HasSet bool
type ConfigTerm[T uint | uint64] struct {
Value T
Modified bool
}

type KVConfig struct {
ImportGoroutines ConfigSet[uint]
MergeRegionSize ConfigSet[uint64]
MergeRegionKeyCount ConfigSet[uint64]
ImportGoroutines ConfigTerm[uint]
MergeRegionSize ConfigTerm[uint64]
MergeRegionKeyCount ConfigTerm[uint64]
}

func ParseImportThreadsFromConfig(resp []byte) (uint, error) {
Expand Down
6 changes: 3 additions & 3 deletions br/pkg/conn/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -301,7 +301,7 @@ func (mgr *Mgr) ProcessTiKVConfigs(ctx context.Context, cfg *kvconfig.KVConfig,
mergeRegionKeyCount := cfg.MergeRegionKeyCount
importGoroutines := cfg.ImportGoroutines

if mergeRegionSize.HasSet && mergeRegionKeyCount.HasSet && importGoroutines.HasSet {
if mergeRegionSize.Modified && mergeRegionKeyCount.Modified && importGoroutines.Modified {
log.Info("no need to retrieve the config from tikv if user has set the config")
return
}
Expand All @@ -311,7 +311,7 @@ func (mgr *Mgr) ProcessTiKVConfigs(ctx context.Context, cfg *kvconfig.KVConfig,
if err != nil {
return err
}
if !mergeRegionSize.HasSet || !mergeRegionKeyCount.HasSet {
if !mergeRegionSize.Modified || !mergeRegionKeyCount.Modified {
size, keys, e := kvconfig.ParseMergeRegionSizeFromConfig(respBytes)
if e != nil {
log.Warn("Failed to parse region split size and keys from config", logutil.ShortError(e))
Expand All @@ -322,7 +322,7 @@ func (mgr *Mgr) ProcessTiKVConfigs(ctx context.Context, cfg *kvconfig.KVConfig,
mergeRegionKeyCount.Value = keys
}
}
if !importGoroutines.HasSet {
if !importGoroutines.Modified {
threads, e := kvconfig.ParseImportThreadsFromConfig(respBytes)
if e != nil {
log.Warn("Failed to parse import num-threads from config", logutil.ShortError(e))
Expand Down
6 changes: 3 additions & 3 deletions br/pkg/conn/conn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -429,9 +429,9 @@ func TestGetMergeRegionSizeAndCount(t *testing.T) {
mgr := &Mgr{PdController: &pdutil.PdController{}}
mgr.PdController.SetPDClient(pdCli)
kvConfigs := &kvconfig.KVConfig{
ImportGoroutines: kvconfig.ConfigSet[uint]{Value: DefaultImportNumGoroutines, HasSet: false},
MergeRegionSize: kvconfig.ConfigSet[uint64]{Value: DefaultMergeRegionSizeBytes, HasSet: false},
MergeRegionKeyCount: kvconfig.ConfigSet[uint64]{Value: DefaultMergeRegionKeyCount, HasSet: false},
ImportGoroutines: kvconfig.ConfigTerm[uint]{Value: DefaultImportNumGoroutines, Modified: false},
MergeRegionSize: kvconfig.ConfigTerm[uint64]{Value: DefaultMergeRegionSizeBytes, Modified: false},
MergeRegionKeyCount: kvconfig.ConfigTerm[uint64]{Value: DefaultMergeRegionKeyCount, Modified: false},
}
mgr.ProcessTiKVConfigs(ctx, kvConfigs, httpCli)
require.EqualValues(t, ca.regionSplitSize, kvConfigs.MergeRegionSize.Value)
Expand Down
4 changes: 2 additions & 2 deletions br/pkg/task/common_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -209,8 +209,8 @@ func expectedDefaultRestoreConfig() RestoreConfig {
Config: defaultConfig,
RestoreCommonConfig: RestoreCommonConfig{Online: false,
Granularity: "fine-grained",
MergeSmallRegionSizeBytes: kvconfig.ConfigSet[uint64]{Value: 0x6000000},
MergeSmallRegionKeyCount: kvconfig.ConfigSet[uint64]{Value: 0xea600},
MergeSmallRegionSizeBytes: kvconfig.ConfigTerm[uint64]{Value: 0x6000000},
MergeSmallRegionKeyCount: kvconfig.ConfigTerm[uint64]{Value: 0xea600},
WithSysTable: true,
ResetSysUsers: []string{"cloud_admin", "root"}},
NoSchema: false,
Expand Down
22 changes: 11 additions & 11 deletions br/pkg/task/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,15 +102,15 @@ const (

// RestoreCommonConfig is the common configuration for all BR restore tasks.
type RestoreCommonConfig struct {
Online bool `json:"online" toml:"online"`
Granularity string `json:"granularity" toml:"granularity"`
ConcurrencyPerStore pconfig.ConfigSet[uint] `json:"tikv-max-restore-concurrency" toml:"tikv-max-restore-concurrency"`
Online bool `json:"online" toml:"online"`
Granularity string `json:"granularity" toml:"granularity"`
ConcurrencyPerStore pconfig.ConfigTerm[uint] `json:"tikv-max-restore-concurrency" toml:"tikv-max-restore-concurrency"`

// MergeSmallRegionSizeBytes is the threshold of merging small regions (Default 96MB, region split size).
// MergeSmallRegionKeyCount is the threshold of merging smalle regions (Default 960_000, region split key count).
// See https://github.com/tikv/tikv/blob/v4.0.8/components/raftstore/src/coprocessor/config.rs#L35-L38
MergeSmallRegionSizeBytes pconfig.ConfigSet[uint64] `json:"merge-region-size-bytes" toml:"merge-region-size-bytes"`
MergeSmallRegionKeyCount pconfig.ConfigSet[uint64] `json:"merge-region-key-count" toml:"merge-region-key-count"`
MergeSmallRegionSizeBytes pconfig.ConfigTerm[uint64] `json:"merge-region-size-bytes" toml:"merge-region-size-bytes"`
MergeSmallRegionKeyCount pconfig.ConfigTerm[uint64] `json:"merge-region-key-count" toml:"merge-region-key-count"`

// determines whether enable restore sys table on default, see fullClusterRestore in restore/client.go
WithSysTable bool `json:"with-sys-table" toml:"with-sys-table"`
Expand All @@ -121,16 +121,16 @@ type RestoreCommonConfig struct {
// adjust adjusts the abnormal config value in the current config.
// useful when not starting BR from CLI (e.g. from BRIE in SQL).
func (cfg *RestoreCommonConfig) adjust() {
if !cfg.MergeSmallRegionKeyCount.HasSet {
if !cfg.MergeSmallRegionKeyCount.Modified {
cfg.MergeSmallRegionKeyCount.Value = conn.DefaultMergeRegionKeyCount
}
if !cfg.MergeSmallRegionSizeBytes.HasSet {
if !cfg.MergeSmallRegionSizeBytes.Modified {
cfg.MergeSmallRegionSizeBytes.Value = conn.DefaultMergeRegionSizeBytes
}
if len(cfg.Granularity) == 0 {
cfg.Granularity = string(restore.FineGrained)
}
if cfg.ConcurrencyPerStore.HasSet {
if cfg.ConcurrencyPerStore.Modified {
cfg.ConcurrencyPerStore.Value = conn.DefaultImportNumGoroutines
}
}
Expand Down Expand Up @@ -182,19 +182,19 @@ func (cfg *RestoreCommonConfig) ParseFromFlags(flags *pflag.FlagSet) error {
if err != nil {
return errors.Trace(err)
}
cfg.ConcurrencyPerStore.HasSet = flags.Changed(flagConcurrencyPerStore)
cfg.ConcurrencyPerStore.Modified = flags.Changed(flagConcurrencyPerStore)

cfg.MergeSmallRegionKeyCount.Value, err = flags.GetUint64(FlagMergeRegionKeyCount)
if err != nil {
return errors.Trace(err)
}
cfg.MergeSmallRegionKeyCount.HasSet = flags.Changed(FlagMergeRegionKeyCount)
cfg.MergeSmallRegionKeyCount.Modified = flags.Changed(FlagMergeRegionKeyCount)

cfg.MergeSmallRegionSizeBytes.Value, err = flags.GetUint64(FlagMergeRegionSizeBytes)
if err != nil {
return errors.Trace(err)
}
cfg.MergeSmallRegionSizeBytes.HasSet = flags.Changed(FlagMergeRegionSizeBytes)
cfg.MergeSmallRegionSizeBytes.Modified = flags.Changed(FlagMergeRegionSizeBytes)

if flags.Lookup(flagWithSysTable) != nil {
cfg.WithSysTable, err = flags.GetBool(flagWithSysTable)
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/task/restore_raw.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ func RunRestoreRaw(c context.Context, g glue.Glue, cmdName string, cfg *RestoreR
MergeRegionKeyCount: cfg.MergeSmallRegionKeyCount,
}

if !kvConfigs.MergeRegionSize.HasSet || !kvConfigs.MergeRegionKeyCount.HasSet {
if !kvConfigs.MergeRegionSize.Modified || !kvConfigs.MergeRegionKeyCount.Modified {
// according to https://github.com/pingcap/tidb/issues/34167.
// we should get the real config from tikv to adapt the dynamic region.
httpCli := httputil.NewClient(mgr.GetTLSConfig())
Expand Down

0 comments on commit c04e0ad

Please sign in to comment.