Skip to content

Commit

Permalink
update
Browse files Browse the repository at this point in the history
  • Loading branch information
CbcWestwolf committed Nov 14, 2024
1 parent a276364 commit 48335a6
Show file tree
Hide file tree
Showing 6 changed files with 53 additions and 0 deletions.
2 changes: 2 additions & 0 deletions ddl/ingest/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/pingcap/tidb/br/pkg/lightning/checkpoints"
lightning "github.com/pingcap/tidb/br/pkg/lightning/config"
tidb "github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/size"
"go.uber.org/zap"
Expand All @@ -42,6 +43,7 @@ func genConfig(memRoot MemRoot, jobID int64, unique bool) (*Config, error) {
cfg.TikvImporter.Backend = lightning.BackendLocal
// Each backend will build a single dir in lightning dir.
cfg.TikvImporter.SortedKVDir = filepath.Join(LitSortPath, EncodeBackendTag(jobID))
cfg.TikvImporter.StoreWriteBWLimit = lightning.ByteSize(variable.DDLReorgMaxWriteSpeed.Load())

Check warning on line 46 in ddl/ingest/config.go

View check run for this annotation

Codecov / codecov/patch

ddl/ingest/config.go#L46

Added line #L46 was not covered by tests
if ImporterRangeConcurrencyForTest != nil {
cfg.TikvImporter.RangeConcurrency = int(ImporterRangeConcurrencyForTest.Load())
}
Expand Down
1 change: 1 addition & 0 deletions executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -446,6 +446,7 @@ go_test(
"//util/tableutil",
"//util/timeutil",
"//util/topsql/state",
"@com_github_docker_go_units//:go-units",
"@com_github_golang_protobuf//proto",
"@com_github_gorilla_mux//:mux",
"@com_github_pingcap_errors//:errors",
Expand Down
26 changes: 26 additions & 0 deletions executor/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"testing"
"time"

"github.com/docker/go-units"
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/ddl/schematracker"
ddltestutil "github.com/pingcap/tidb/ddl/testutil"
Expand Down Expand Up @@ -1318,6 +1319,31 @@ func TestSetDDLErrorCountLimit(t *testing.T) {
res.Check(testkit.Rows("100"))
}

func TestSetDDLReorgMaxWriteSpeed(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
require.Equal(t, int64(variable.DefTiDBDDLReorgMaxWriteSpeed), variable.DDLReorgMaxWriteSpeed.Load())

// valid values
for _, val := range []int64{1, 0, 100, 1024 * 1024, 2147483647, units.PiB} {
tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_max_write_speed = %d", val))
require.Equal(t, val, variable.DDLReorgMaxWriteSpeed.Load())
tk.MustQuery("select @@global.tidb_ddl_reorg_max_write_speed").Check(testkit.Rows(strconv.FormatInt(val, 10)))
}
for _, val := range []string{"1", "0", "100", "2KB", "3MiB", "4 gb", "2147483647", "1125899906842624" /* 1PiB */} {
tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_max_write_speed = '%s'", val))
expected, err := units.RAMInBytes(val)
require.NoError(t, err)
require.Equal(t, expected, variable.DDLReorgMaxWriteSpeed.Load())
tk.MustQuery("select @@global.tidb_ddl_reorg_max_write_speed").Check(testkit.Rows(strconv.FormatInt(expected, 10)))
}

// invalid values
tk.MustExecToErr("set @@global.tidb_ddl_reorg_max_write_speed = -1")
tk.MustExecToErr("set @@global.tidb_ddl_reorg_max_write_speed = invalid_val")
tk.MustExecToErr("set @@global.tidb_ddl_reorg_max_write_speed = %d", units.PiB+1)
}

func TestLoadDDLDistributeVars(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
Expand Down
1 change: 1 addition & 0 deletions sessionctx/variable/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ go_library(
"//util/tls",
"//util/topsql/state",
"//util/versioninfo",
"@com_github_docker_go_units//:go-units",
"@com_github_pingcap_errors//:errors",
"@com_github_tikv_client_go_v2//config",
"@com_github_tikv_client_go_v2//kv",
Expand Down
18 changes: 18 additions & 0 deletions sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"sync/atomic"
"time"

"github.com/docker/go-units"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/keyspace"
Expand Down Expand Up @@ -688,6 +689,23 @@ var defaultSysVars = []*SysVar{
SetDDLReorgBatchSize(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgBatchSize)))
return nil
}},
{Scope: ScopeGlobal, Name: TiDBDDLReorgMaxWriteSpeed, Value: strconv.Itoa(DefTiDBDDLReorgMaxWriteSpeed), Type: TypeStr,
SetGlobal: func(_ context.Context, s *SessionVars, val string) error {
i64, err := units.RAMInBytes(val)
if err != nil {
return errors.Trace(err)
}
if i64 < 0 || i64 > units.PiB {
// Here we limit the max value to 1 PiB instead of math.MaxInt64, since:
// 1. it is large enough
// 2. units.RAMInBytes would first cast the size to a float, and may lose precision when the size is too large
return fmt.Errorf("invalid value for '%d', it should be within [%d, %d]", i64, 0, units.PiB)
}

Check warning on line 703 in sessionctx/variable/sysvar.go

View check run for this annotation

Codecov / codecov/patch

sessionctx/variable/sysvar.go#L699-L703

Added lines #L699 - L703 were not covered by tests
DDLReorgMaxWriteSpeed.Store(i64)
return nil
}, GetGlobal: func(_ context.Context, sv *SessionVars) (string, error) {
return strconv.FormatInt(DDLReorgMaxWriteSpeed.Load(), 10), nil
}},
{Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt64, SetGlobal: func(_ context.Context, s *SessionVars, val string) error {
SetDDLErrorCountLimit(TidbOptInt64(val, DefTiDBDDLErrorCountLimit))
return nil
Expand Down
5 changes: 5 additions & 0 deletions sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -485,6 +485,9 @@ const (
// It can be: PRIORITY_LOW, PRIORITY_NORMAL, PRIORITY_HIGH
TiDBDDLReorgPriority = "tidb_ddl_reorg_priority"

// TiDBDDLReorgMaxWriteSpeed defines the max write limitation for the lightning local backend
TiDBDDLReorgMaxWriteSpeed = "tidb_ddl_reorg_max_write_speed"

// TiDBEnableAutoIncrementInGenerated disables the mysql compatibility check on using auto-incremented columns in
// expression indexes and generated columns described here https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html for details.
TiDBEnableAutoIncrementInGenerated = "tidb_enable_auto_increment_in_generated"
Expand Down Expand Up @@ -1148,6 +1151,7 @@ const (
DefTiDBDDLReorgBatchSize = 256
DefTiDBDDLFlashbackConcurrency = 64
DefTiDBDDLErrorCountLimit = 512
DefTiDBDDLReorgMaxWriteSpeed = 0
DefTiDBMaxDeltaSchemaCount = 1024
DefTiDBPlacementMode = PlacementModeStrict
DefTiDBEnableAutoIncrementInGenerated = false
Expand Down Expand Up @@ -1367,6 +1371,7 @@ var (
ddlFlashbackConcurrency int32 = DefTiDBDDLFlashbackConcurrency
ddlErrorCountLimit int64 = DefTiDBDDLErrorCountLimit
ddlReorgRowFormat int64 = DefTiDBRowFormatV2
DDLReorgMaxWriteSpeed = atomic.NewInt64(DefTiDBDDLReorgMaxWriteSpeed)
maxDeltaSchemaCount int64 = DefTiDBMaxDeltaSchemaCount
// DDLSlowOprThreshold is the threshold for ddl slow operations, uint is millisecond.
DDLSlowOprThreshold = config.GetGlobalConfig().Instance.DDLSlowOprThreshold
Expand Down

0 comments on commit 48335a6

Please sign in to comment.