Skip to content

Commit

Permalink
*: split definition out of variable pkg and refactor some reorg metho…
Browse files Browse the repository at this point in the history
…ds (#59146)

ref #57229, ref #57497
  • Loading branch information
D3Hunter authored Jan 23, 2025
1 parent dfc0035 commit cf4d252
Show file tree
Hide file tree
Showing 435 changed files with 6,803 additions and 6,544 deletions.
2 changes: 1 addition & 1 deletion br/pkg/backup/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ go_test(
"//br/pkg/storage",
"//br/pkg/utils",
"//pkg/meta/model",
"//pkg/sessionctx/variable",
"//pkg/sessionctx/vardef",
"//pkg/testkit",
"//pkg/testkit/testsetup",
"//pkg/util/table-filter",
Expand Down
12 changes: 6 additions & 6 deletions br/pkg/backup/schema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ import (
"github.com/pingcap/tidb/br/pkg/mock"
"github.com/pingcap/tidb/br/pkg/storage"
"github.com/pingcap/tidb/br/pkg/utils"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
"github.com/pingcap/tidb/pkg/testkit"
filter "github.com/pingcap/tidb/pkg/util/table-filter"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -151,7 +151,7 @@ func TestBuildBackupRangeAndSchema(t *testing.T) {
metaWriter := metautil.NewMetaWriter(es, metautil.MetaFileSize, false, "", &cipher)
ctx := context.Background()
err = backupSchemas.BackupSchemas(
ctx, metaWriter, nil, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh)
ctx, metaWriter, nil, m.Storage, nil, math.MaxUint64, 1, vardef.DefChecksumTableConcurrency, skipChecksum, updateCh)
require.Equal(t, int64(1), updateCh.get())
require.NoError(t, err)
err = metaWriter.FlushBackupMeta(ctx)
Expand Down Expand Up @@ -180,7 +180,7 @@ func TestBuildBackupRangeAndSchema(t *testing.T) {
es2 := GetRandomStorage(t)
metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, "", &cipher)
err = backupSchemas.BackupSchemas(
ctx, metaWriter2, nil, m.Storage, nil, math.MaxUint64, 2, variable.DefChecksumTableConcurrency, skipChecksum, updateCh)
ctx, metaWriter2, nil, m.Storage, nil, math.MaxUint64, 2, vardef.DefChecksumTableConcurrency, skipChecksum, updateCh)
require.Equal(t, int64(2), updateCh.get())
require.NoError(t, err)
err = metaWriter2.FlushBackupMeta(ctx)
Expand Down Expand Up @@ -234,7 +234,7 @@ func TestBuildBackupRangeAndSchemaWithBrokenStats(t *testing.T) {
metaWriter := metautil.NewMetaWriter(es, metautil.MetaFileSize, false, "", &cipher)
ctx := context.Background()
err = backupSchemas.BackupSchemas(
ctx, metaWriter, nil, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh)
ctx, metaWriter, nil, m.Storage, nil, math.MaxUint64, 1, vardef.DefChecksumTableConcurrency, skipChecksum, updateCh)
require.NoError(t, err)
err = metaWriter.FlushBackupMeta(ctx)
require.NoError(t, err)
Expand Down Expand Up @@ -262,7 +262,7 @@ func TestBuildBackupRangeAndSchemaWithBrokenStats(t *testing.T) {
es2 := GetRandomStorage(t)
metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, "", &cipher)
err = backupSchemas.BackupSchemas(
ctx, metaWriter2, nil, m.Storage, statsHandle, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh)
ctx, metaWriter2, nil, m.Storage, statsHandle, math.MaxUint64, 1, vardef.DefChecksumTableConcurrency, skipChecksum, updateCh)
require.NoError(t, err)
err = metaWriter2.FlushBackupMeta(ctx)
require.NoError(t, err)
Expand Down Expand Up @@ -306,7 +306,7 @@ func TestBackupSchemasForSystemTable(t *testing.T) {

metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, "", &cipher)
err = backupSchemas.BackupSchemas(ctx, metaWriter2, nil, m.Storage, nil,
math.MaxUint64, 1, variable.DefChecksumTableConcurrency, true, updateCh)
math.MaxUint64, 1, vardef.DefChecksumTableConcurrency, true, updateCh)
require.NoError(t, err)
err = metaWriter2.FlushBackupMeta(ctx)
require.NoError(t, err)
Expand Down
4 changes: 2 additions & 2 deletions br/pkg/checksum/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ go_library(
"//pkg/distsql",
"//pkg/kv",
"//pkg/meta/model",
"//pkg/sessionctx/variable",
"//pkg/sessionctx/vardef",
"//pkg/tablecodec",
"//pkg/util/ranger",
"@com_github_gogo_protobuf//proto",
Expand Down Expand Up @@ -46,7 +46,7 @@ go_test(
"//pkg/kv",
"//pkg/meta/model",
"//pkg/parser/ast",
"//pkg/sessionctx/variable",
"//pkg/sessionctx/vardef",
"//pkg/testkit",
"//pkg/testkit/testsetup",
"@com_github_pingcap_failpoint//:failpoint",
Expand Down
4 changes: 2 additions & 2 deletions br/pkg/checksum/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ import (
"github.com/pingcap/tidb/pkg/distsql"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta/model"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
"github.com/pingcap/tidb/pkg/tablecodec"
"github.com/pingcap/tidb/pkg/util/ranger"
"github.com/pingcap/tipb/go-tipb"
Expand Down Expand Up @@ -44,7 +44,7 @@ func NewExecutorBuilder(table *model.TableInfo, ts uint64) *ExecutorBuilder {
table: table,
ts: ts,

concurrency: variable.DefDistSQLScanConcurrency,
concurrency: vardef.DefDistSQLScanConcurrency,
}
}

Expand Down
8 changes: 4 additions & 4 deletions br/pkg/checksum/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ import (
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta/model"
"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -44,7 +44,7 @@ func TestChecksumContextDone(t *testing.T) {
tk.MustExec("insert into t1 values (10, 10);")
tableInfo1 := getTableInfo(t, mock, "test", "t1")
exe, err := checksum.NewExecutorBuilder(tableInfo1, math.MaxUint64).
SetConcurrency(variable.DefChecksumTableConcurrency).
SetConcurrency(vardef.DefChecksumTableConcurrency).
Build()
require.NoError(t, err)

Expand Down Expand Up @@ -73,12 +73,12 @@ func TestChecksum(t *testing.T) {
tk.MustExec("insert into t1 values (10);")
tableInfo1 := getTableInfo(t, mock, "test", "t1")
exe1, err := checksum.NewExecutorBuilder(tableInfo1, math.MaxUint64).
SetConcurrency(variable.DefChecksumTableConcurrency).
SetConcurrency(vardef.DefChecksumTableConcurrency).
Build()
require.NoError(t, err)
require.NoError(t, exe1.Each(func(r *kv.Request) error {
require.True(t, r.NotFillCache)
require.Equal(t, variable.DefChecksumTableConcurrency, r.Concurrency)
require.Equal(t, vardef.DefChecksumTableConcurrency, r.Concurrency)
return nil
}))
require.Equal(t, 1, exe1.Len())
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/task/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ go_library(
"//pkg/parser/ast",
"//pkg/parser/mysql",
"//pkg/sessionctx/stmtctx",
"//pkg/sessionctx/variable",
"//pkg/sessionctx/vardef",
"//pkg/statistics/handle",
"//pkg/types",
"//pkg/util",
Expand Down
6 changes: 3 additions & 3 deletions br/pkg/task/common.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import (
"github.com/pingcap/tidb/br/pkg/metautil"
"github.com/pingcap/tidb/br/pkg/storage"
"github.com/pingcap/tidb/br/pkg/utils"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
filter "github.com/pingcap/tidb/pkg/util/table-filter"
"github.com/spf13/cobra"
"github.com/spf13/pflag"
Expand Down Expand Up @@ -294,7 +294,7 @@ func DefineCommonFlags(flags *pflag.FlagSet) {
flags.String(flagCA, "", "CA certificate path for TLS connection")
flags.String(flagCert, "", "Certificate path for TLS connection")
flags.String(flagKey, "", "Private key path for TLS connection")
flags.Uint(flagChecksumConcurrency, variable.DefChecksumTableConcurrency, "The concurrency of checksumming in one table")
flags.Uint(flagChecksumConcurrency, vardef.DefChecksumTableConcurrency, "The concurrency of checksumming in one table")

flags.Uint64(flagRateLimit, unlimited, "The rate limit of the task, MB/s per node")
flags.Bool(FlagChecksum, true, "Run checksum at end of task")
Expand Down Expand Up @@ -947,7 +947,7 @@ func (cfg *Config) adjust() {
cfg.GRPCKeepaliveTimeout = defaultGRPCKeepaliveTimeout
}
if cfg.ChecksumConcurrency == 0 {
cfg.ChecksumConcurrency = variable.DefChecksumTableConcurrency
cfg.ChecksumConcurrency = vardef.DefChecksumTableConcurrency
}
if cfg.MetadataDownloadBatchSize == 0 {
cfg.MetadataDownloadBatchSize = defaultMetadataDownloadBatchSize
Expand Down
2 changes: 1 addition & 1 deletion cmd/ddltest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ go_test(
"//pkg/session",
"//pkg/session/types",
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/sessionctx/vardef",
"//pkg/sessiontxn",
"//pkg/store",
"//pkg/store/driver",
Expand Down
6 changes: 3 additions & 3 deletions cmd/ddltest/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ import (
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
"github.com/pingcap/tidb/pkg/sessiontxn"
"github.com/pingcap/tidb/pkg/store"
tidbdriver "github.com/pingcap/tidb/pkg/store/driver"
Expand Down Expand Up @@ -565,7 +565,7 @@ func (s *ddlSuite) Bootstrap(t *testing.T) {
tk.MustExec("create table test_mixed (c1 int, c2 int, primary key(c1))")
tk.MustExec("create table test_inc (c1 int, c2 int, primary key(c1))")

tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn
tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeOn
tk.MustExec("drop table if exists test_insert_common, test_conflict_insert_common, " +
"test_update_common, test_conflict_update_common, test_delete_common, test_conflict_delete_common, " +
"test_mixed_common, test_inc_common")
Expand All @@ -577,7 +577,7 @@ func (s *ddlSuite) Bootstrap(t *testing.T) {
tk.MustExec("create table test_conflict_delete_common (c1 int, c2 int, primary key(c1, c2))")
tk.MustExec("create table test_mixed_common (c1 int, c2 int, primary key(c1, c2))")
tk.MustExec("create table test_inc_common (c1 int, c2 int, primary key(c1, c2))")
tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly
tk.Session().GetSessionVars().EnableClusteredIndex = vardef.ClusteredIndexDefModeIntOnly
}

func TestSimple(t *testing.T) {
Expand Down
2 changes: 2 additions & 0 deletions cmd/tidb-server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ go_library(
"//pkg/server",
"//pkg/session",
"//pkg/session/txninfo",
"//pkg/sessionctx/vardef",
"//pkg/sessionctx/variable",
"//pkg/statistics",
"//pkg/store",
Expand Down Expand Up @@ -104,6 +105,7 @@ go_test(
deps = [
"//pkg/config",
"//pkg/parser/mysql",
"//pkg/sessionctx/vardef",
"//pkg/sessionctx/variable",
"//pkg/testkit/testsetup",
"@com_github_stretchr_testify//require",
Expand Down
55 changes: 28 additions & 27 deletions cmd/tidb-server/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ import (
"github.com/pingcap/tidb/pkg/server"
"github.com/pingcap/tidb/pkg/session"
"github.com/pingcap/tidb/pkg/session/txninfo"
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/statistics"
kvstore "github.com/pingcap/tidb/pkg/store"
Expand Down Expand Up @@ -279,7 +280,7 @@ func main() {
err := metricsutil.RegisterMetrics()
terror.MustNil(err)

if variable.EnableTmpStorageOnOOM.Load() {
if vardef.EnableTmpStorageOnOOM.Load() {
config.GetGlobalConfig().UpdateTempStoragePath()
err := disk.InitializeTempDir()
terror.MustNil(err)
Expand Down Expand Up @@ -741,56 +742,56 @@ func setGlobalVars() {
kv.TxnEntrySizeLimit.Store(cfg.Performance.TxnEntrySizeLimit)

priority := mysql.Str2Priority(cfg.Instance.ForcePriority)
variable.ForcePriority = int32(priority)
vardef.ForcePriority = int32(priority)

variable.ProcessGeneralLog.Store(cfg.Instance.TiDBGeneralLog)
variable.EnablePProfSQLCPU.Store(cfg.Instance.EnablePProfSQLCPU)
variable.EnableRCReadCheckTS.Store(cfg.Instance.TiDBRCReadCheckTS)
variable.IsSandBoxModeEnabled.Store(!cfg.Security.DisconnectOnExpiredPassword)
atomic.StoreUint32(&variable.DDLSlowOprThreshold, cfg.Instance.DDLSlowOprThreshold)
atomic.StoreUint64(&variable.ExpensiveQueryTimeThreshold, cfg.Instance.ExpensiveQueryTimeThreshold)
atomic.StoreUint64(&variable.ExpensiveTxnTimeThreshold, cfg.Instance.ExpensiveTxnTimeThreshold)
vardef.ProcessGeneralLog.Store(cfg.Instance.TiDBGeneralLog)
vardef.EnablePProfSQLCPU.Store(cfg.Instance.EnablePProfSQLCPU)
vardef.EnableRCReadCheckTS.Store(cfg.Instance.TiDBRCReadCheckTS)
vardef.IsSandBoxModeEnabled.Store(!cfg.Security.DisconnectOnExpiredPassword)
atomic.StoreUint32(&vardef.DDLSlowOprThreshold, cfg.Instance.DDLSlowOprThreshold)
atomic.StoreUint64(&vardef.ExpensiveQueryTimeThreshold, cfg.Instance.ExpensiveQueryTimeThreshold)
atomic.StoreUint64(&vardef.ExpensiveTxnTimeThreshold, cfg.Instance.ExpensiveTxnTimeThreshold)

if len(cfg.ServerVersion) > 0 {
mysql.ServerVersion = cfg.ServerVersion
variable.SetSysVar(variable.Version, cfg.ServerVersion)
variable.SetSysVar(vardef.Version, cfg.ServerVersion)
}

if len(cfg.TiDBEdition) > 0 {
versioninfo.TiDBEdition = cfg.TiDBEdition
variable.SetSysVar(variable.VersionComment, "TiDB Server (Apache License 2.0) "+versioninfo.TiDBEdition+" Edition, MySQL 8.0 compatible")
variable.SetSysVar(vardef.VersionComment, "TiDB Server (Apache License 2.0) "+versioninfo.TiDBEdition+" Edition, MySQL 8.0 compatible")
}
if len(cfg.VersionComment) > 0 {
variable.SetSysVar(variable.VersionComment, cfg.VersionComment)
variable.SetSysVar(vardef.VersionComment, cfg.VersionComment)
}
if len(cfg.TiDBReleaseVersion) > 0 {
mysql.TiDBReleaseVersion = cfg.TiDBReleaseVersion
}

variable.SetSysVar(variable.TiDBForcePriority, mysql.Priority2Str[priority])
variable.SetSysVar(variable.TiDBOptDistinctAggPushDown, variable.BoolToOnOff(cfg.Performance.DistinctAggPushDown))
variable.SetSysVar(variable.TiDBOptProjectionPushDown, variable.BoolToOnOff(cfg.Performance.ProjectionPushDown))
variable.SetSysVar(variable.Port, fmt.Sprintf("%d", cfg.Port))
variable.SetSysVar(vardef.TiDBForcePriority, mysql.Priority2Str[priority])
variable.SetSysVar(vardef.TiDBOptDistinctAggPushDown, variable.BoolToOnOff(cfg.Performance.DistinctAggPushDown))
variable.SetSysVar(vardef.TiDBOptProjectionPushDown, variable.BoolToOnOff(cfg.Performance.ProjectionPushDown))
variable.SetSysVar(vardef.Port, fmt.Sprintf("%d", cfg.Port))
cfg.Socket = strings.Replace(cfg.Socket, "{Port}", fmt.Sprintf("%d", cfg.Port), 1)
variable.SetSysVar(variable.Socket, cfg.Socket)
variable.SetSysVar(variable.DataDir, cfg.Path)
variable.SetSysVar(variable.TiDBSlowQueryFile, cfg.Log.SlowQueryFile)
variable.SetSysVar(variable.TiDBIsolationReadEngines, strings.Join(cfg.IsolationRead.Engines, ","))
variable.SetSysVar(variable.TiDBEnforceMPPExecution, variable.BoolToOnOff(config.GetGlobalConfig().Performance.EnforceMPP))
variable.MemoryUsageAlarmRatio.Store(cfg.Instance.MemoryUsageAlarmRatio)
variable.SetSysVar(variable.TiDBConstraintCheckInPlacePessimistic, variable.BoolToOnOff(cfg.PessimisticTxn.ConstraintCheckInPlacePessimistic))
variable.SetSysVar(vardef.Socket, cfg.Socket)
variable.SetSysVar(vardef.DataDir, cfg.Path)
variable.SetSysVar(vardef.TiDBSlowQueryFile, cfg.Log.SlowQueryFile)
variable.SetSysVar(vardef.TiDBIsolationReadEngines, strings.Join(cfg.IsolationRead.Engines, ","))
variable.SetSysVar(vardef.TiDBEnforceMPPExecution, variable.BoolToOnOff(config.GetGlobalConfig().Performance.EnforceMPP))
vardef.MemoryUsageAlarmRatio.Store(cfg.Instance.MemoryUsageAlarmRatio)
variable.SetSysVar(vardef.TiDBConstraintCheckInPlacePessimistic, variable.BoolToOnOff(cfg.PessimisticTxn.ConstraintCheckInPlacePessimistic))
if hostname, err := os.Hostname(); err == nil {
variable.SetSysVar(variable.Hostname, hostname)
variable.SetSysVar(vardef.Hostname, hostname)
}
variable.GlobalLogMaxDays.Store(int32(config.GetGlobalConfig().Log.File.MaxDays))
vardef.GlobalLogMaxDays.Store(int32(config.GetGlobalConfig().Log.File.MaxDays))

if cfg.Security.EnableSEM {
sem.Enable()
}

// For CI environment we default enable prepare-plan-cache.
if config.CheckTableBeforeDrop { // only for test
variable.SetSysVar(variable.TiDBEnablePrepPlanCache, variable.BoolToOnOff(true))
variable.SetSysVar(vardef.TiDBEnablePrepPlanCache, variable.BoolToOnOff(true))
}
// use server-memory-quota as max-plan-cache-memory
plannercore.PreparedPlanCacheMaxMemory.Store(cfg.Performance.ServerMemoryQuota)
Expand Down Expand Up @@ -827,7 +828,7 @@ func setGlobalVars() {
chunk.InitChunkAllocSize(cfg.TiDBMaxReuseChunk, cfg.TiDBMaxReuseColumn)

if len(cfg.Instance.TiDBServiceScope) > 0 {
variable.ServiceScope.Store(strings.ToLower(cfg.Instance.TiDBServiceScope))
vardef.ServiceScope.Store(strings.ToLower(cfg.Instance.TiDBServiceScope))
}
}

Expand Down
21 changes: 11 additions & 10 deletions cmd/tidb-server/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (

"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/testkit/testsetup"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -51,33 +52,33 @@ func TestRunMain(t *testing.T) {

func TestSetGlobalVars(t *testing.T) {
defer view.Stop()
require.Equal(t, "tikv,tiflash,tidb", variable.GetSysVar(variable.TiDBIsolationReadEngines).Value)
require.Equal(t, "1073741824", variable.GetSysVar(variable.TiDBMemQuotaQuery).Value)
require.NotEqual(t, "test", variable.GetSysVar(variable.Version).Value)
require.Equal(t, "tikv,tiflash,tidb", variable.GetSysVar(vardef.TiDBIsolationReadEngines).Value)
require.Equal(t, "1073741824", variable.GetSysVar(vardef.TiDBMemQuotaQuery).Value)
require.NotEqual(t, "test", variable.GetSysVar(vardef.Version).Value)

config.UpdateGlobal(func(conf *config.Config) {
conf.IsolationRead.Engines = []string{"tikv", "tidb"}
conf.ServerVersion = "test"
})
setGlobalVars()

require.Equal(t, "tikv,tidb", variable.GetSysVar(variable.TiDBIsolationReadEngines).Value)
require.Equal(t, "test", variable.GetSysVar(variable.Version).Value)
require.Equal(t, variable.GetSysVar(variable.Version).Value, mysql.ServerVersion)
require.Equal(t, "tikv,tidb", variable.GetSysVar(vardef.TiDBIsolationReadEngines).Value)
require.Equal(t, "test", variable.GetSysVar(vardef.Version).Value)
require.Equal(t, variable.GetSysVar(vardef.Version).Value, mysql.ServerVersion)

config.UpdateGlobal(func(conf *config.Config) {
conf.ServerVersion = ""
})
setGlobalVars()

// variable.Version won't change when len(conf.ServerVersion) == 0
require.Equal(t, "test", variable.GetSysVar(variable.Version).Value)
require.Equal(t, variable.GetSysVar(variable.Version).Value, mysql.ServerVersion)
require.Equal(t, "test", variable.GetSysVar(vardef.Version).Value)
require.Equal(t, variable.GetSysVar(vardef.Version).Value, mysql.ServerVersion)

cfg := config.GetGlobalConfig()
require.Equal(t, cfg.Socket, variable.GetSysVar(variable.Socket).Value)
require.Equal(t, cfg.Socket, variable.GetSysVar(vardef.Socket).Value)

if hostname, err := os.Hostname(); err == nil {
require.Equal(t, variable.GetSysVar(variable.Hostname).Value, hostname)
require.Equal(t, variable.GetSysVar(vardef.Hostname).Value, hostname)
}
}
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -311,7 +311,7 @@ require (
google.golang.org/genproto/googleapis/rpc v0.0.0-20240515191416-fc5f0ca64291 // indirect
google.golang.org/protobuf v1.36.1
gopkg.in/inf.v0 v0.9.1 // indirect
gopkg.in/natefinch/lumberjack.v2 v2.2.1 // indirect
gopkg.in/natefinch/lumberjack.v2 v2.2.1
gopkg.in/yaml.v3 v3.0.1 // indirect
k8s.io/apimachinery v0.29.11 // indirect
k8s.io/klog/v2 v2.120.1 // indirect
Expand Down
2 changes: 1 addition & 1 deletion lightning/pkg/importer/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ go_library(
"//pkg/parser/mysql",
"//pkg/planner/core",
"//pkg/session",
"//pkg/sessionctx/variable",
"//pkg/sessionctx/vardef",
"//pkg/store/driver",
"//pkg/store/driver/txn",
"//pkg/table",
Expand Down
Loading

0 comments on commit cf4d252

Please sign in to comment.