From 044d27ab436190217a21b07d3cdd5f370361f228 Mon Sep 17 00:00:00 2001 From: ekexium Date: Wed, 3 Feb 2021 16:32:03 +0800 Subject: [PATCH] replace the ambiguous name 'external consistency' Signed-off-by: ekexium --- executor/simple.go | 2 +- go.sum | 2 - kv/kv.go | 4 +- session/session.go | 8 +- sessionctx/variable/session.go | 166 ++++++++++---------- sessionctx/variable/sysvar.go | 2 +- sessionctx/variable/tidb_vars.go | 254 ++++++++++++++++--------------- store/tikv/1pc_test.go | 4 +- store/tikv/2pc.go | 12 +- store/tikv/async_commit_test.go | 12 +- 10 files changed, 236 insertions(+), 230 deletions(-) diff --git a/executor/simple.go b/executor/simple.go index 74ca7d2d0405a..8dacf5a05471a 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -590,7 +590,7 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { if e.ctx.GetSessionVars().TxnCtx.IsPessimistic { txn.SetOption(kv.Pessimistic, true) } - txn.SetOption(kv.GuaranteeExternalConsistency, !s.CausalConsistencyOnly) + txn.SetOption(kv.GuaranteeLinearizability, !s.CausalConsistencyOnly) return nil } diff --git a/go.sum b/go.sum index 53280b25f6da3..88cc7eae3ee62 100644 --- a/go.sum +++ b/go.sum @@ -220,8 +220,6 @@ github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4 github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/eclipse/paho.mqtt.golang v1.2.0/go.mod h1:H9keYFcgq3Qr5OUJm/JZI/i6U7joQ8SYLhZwfeOo6Ts= github.com/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaBNrHW5M= -github.com/ekexium/parser v0.0.0-20210202083354-07aa34a5ca3c h1:aQkO2+Ua2sxfg8dJ4B7RaEVv/E/6zSD92EoyJom2qwk= -github.com/ekexium/parser v0.0.0-20210202083354-07aa34a5ca3c/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= github.com/ekexium/parser v0.0.0-20210203081914-18a31cd39b53 h1:F+WtCF9ygyJvqIO+ogDh21IHeeZLpc1d2YvEVNy7+wY= github.com/ekexium/parser v0.0.0-20210203081914-18a31cd39b53/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= diff --git a/kv/kv.go b/kv/kv.go index b46f2b0a1e959..90714f747b0a3 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -62,8 +62,8 @@ const ( EnableAsyncCommit // Enable1PC indicates whether one-phase commit is enabled Enable1PC - // GuaranteeExternalConsistency indicates whether to guarantee external consistency at the cost of an extra tso request before prewrite - GuaranteeExternalConsistency + // GuaranteeLinearizability indicates whether to guarantee linearizability at the cost of an extra tso request before prewrite + GuaranteeLinearizability // TxnScope indicates which @@txn_scope this transaction will work with. TxnScope // StalenessReadOnly indicates whether the transaction is staleness read only transaction diff --git a/session/session.go b/session/session.go index afa1a4f4ad704..377f366548887 100644 --- a/session/session.go +++ b/session/session.go @@ -508,9 +508,9 @@ func (s *session) doCommit(ctx context.Context) error { } s.txn.SetOption(kv.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) s.txn.SetOption(kv.Enable1PC, s.GetSessionVars().Enable1PC) - // priorify of the sysvar is lower than `start transaction with causal consistency` - if s.txn.GetUnionStore().GetOption(kv.GuaranteeExternalConsistency) == nil { - s.txn.SetOption(kv.GuaranteeExternalConsistency, s.GetSessionVars().GuaranteeExternalConsistency) + // priority of the sysvar is lower than `start transaction with causal consistency` + if s.txn.GetUnionStore().GetOption(kv.GuaranteeLinearizability) == nil { + s.txn.SetOption(kv.GuaranteeLinearizability, s.GetSessionVars().GuaranteeLinearizability) } return s.txn.Commit(sessionctx.SetCommitCtx(ctx, s)) @@ -2557,7 +2557,7 @@ var builtinGlobalVariable = []string{ variable.TiDBEnableRateLimitAction, variable.TiDBEnableAsyncCommit, variable.TiDBEnable1PC, - variable.TiDBGuaranteeExternalConsistency, + variable.TiDBGuaranteeLinearizability, variable.TiDBAnalyzeVersion, variable.TiDBEnableIndexMergeJoin, variable.TiDBTrackAggregateMemoryUsage, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 322c7755731b0..43ede082f2519 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -795,7 +795,7 @@ type SessionVars struct { // Enable1PC indicates whether to enable the one-phase commit feature. Enable1PC bool - GuaranteeExternalConsistency bool + GuaranteeLinearizability bool // AnalyzeVersion indicates how TiDB collect and use analyzed statistics. AnalyzeVersion int @@ -883,86 +883,86 @@ type ConnectionInfo struct { // NewSessionVars creates a session vars object. func NewSessionVars() *SessionVars { vars := &SessionVars{ - Users: make(map[string]types.Datum), - UserVarTypes: make(map[string]*types.FieldType), - systems: make(map[string]string), - stmtVars: make(map[string]string), - PreparedStmts: make(map[uint32]interface{}), - PreparedStmtNameToID: make(map[string]uint32), - PreparedParams: make([]types.Datum, 0, 10), - TxnCtx: &TransactionContext{}, - RetryInfo: &RetryInfo{}, - ActiveRoles: make([]*auth.RoleIdentity, 0, 10), - StrictSQLMode: true, - AutoIncrementIncrement: DefAutoIncrementIncrement, - AutoIncrementOffset: DefAutoIncrementOffset, - Status: mysql.ServerStatusAutocommit, - StmtCtx: new(stmtctx.StatementContext), - AllowAggPushDown: false, - AllowBCJ: false, - BroadcastJoinThresholdSize: DefBroadcastJoinThresholdSize, - BroadcastJoinThresholdCount: DefBroadcastJoinThresholdSize, - OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, - RetryLimit: DefTiDBRetryLimit, - DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, - DDLReorgPriority: kv.PriorityLow, - allowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, - preferRangeScan: DefOptPreferRangeScan, - CorrelationThreshold: DefOptCorrelationThreshold, - CorrelationExpFactor: DefOptCorrelationExpFactor, - CPUFactor: DefOptCPUFactor, - CopCPUFactor: DefOptCopCPUFactor, - CopTiFlashConcurrencyFactor: DefOptTiFlashConcurrencyFactor, - NetworkFactor: DefOptNetworkFactor, - ScanFactor: DefOptScanFactor, - DescScanFactor: DefOptDescScanFactor, - SeekFactor: DefOptSeekFactor, - MemoryFactor: DefOptMemoryFactor, - DiskFactor: DefOptDiskFactor, - ConcurrencyFactor: DefOptConcurrencyFactor, - EnableRadixJoin: false, - EnableVectorizedExpression: DefEnableVectorizedExpression, - L2CacheSize: cpuid.CPU.Cache.L2, - CommandValue: uint32(mysql.ComSleep), - TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, - SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, - WaitSplitRegionFinish: DefTiDBWaitSplitRegionFinish, - WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, - enableIndexMerge: false, - EnableNoopFuncs: DefTiDBEnableNoopFuncs, - replicaRead: kv.ReplicaReadLeader, - AllowRemoveAutoInc: DefTiDBAllowRemoveAutoInc, - UsePlanBaselines: DefTiDBUsePlanBaselines, - EvolvePlanBaselines: DefTiDBEvolvePlanBaselines, - EnableExtendedStats: false, - IsolationReadEngines: make(map[kv.StoreType]struct{}), - LockWaitTimeout: DefInnodbLockWaitTimeout * 1000, - MetricSchemaStep: DefTiDBMetricSchemaStep, - MetricSchemaRangeDuration: DefTiDBMetricSchemaRangeDuration, - SequenceState: NewSequenceState(), - WindowingUseHighPrecision: true, - PrevFoundInPlanCache: DefTiDBFoundInPlanCache, - FoundInPlanCache: DefTiDBFoundInPlanCache, - PrevFoundInBinding: DefTiDBFoundInBinding, - FoundInBinding: DefTiDBFoundInBinding, - SelectLimit: math.MaxUint64, - AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert, - EnableClusteredIndex: DefTiDBEnableClusteredIndex, - EnableParallelApply: DefTiDBEnableParallelApply, - ShardAllocateStep: DefTiDBShardAllocateStep, - EnableChangeColumnType: DefTiDBChangeColumnType, - EnableChangeMultiSchema: DefTiDBChangeMultiSchema, - EnablePointGetCache: DefTiDBPointGetCache, - EnableAlterPlacement: DefTiDBEnableAlterPlacement, - EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn, - PartitionPruneMode: *atomic2.NewString(DefTiDBPartitionPruneMode), - TxnScope: config.GetGlobalConfig().TxnScope, - EnabledRateLimitAction: DefTiDBEnableRateLimitAction, - EnableAsyncCommit: DefTiDBEnableAsyncCommit, - Enable1PC: DefTiDBEnable1PC, - GuaranteeExternalConsistency: DefTiDBGuaranteeExternalConsistency, - AnalyzeVersion: DefTiDBAnalyzeVersion, - EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, + Users: make(map[string]types.Datum), + UserVarTypes: make(map[string]*types.FieldType), + systems: make(map[string]string), + stmtVars: make(map[string]string), + PreparedStmts: make(map[uint32]interface{}), + PreparedStmtNameToID: make(map[string]uint32), + PreparedParams: make([]types.Datum, 0, 10), + TxnCtx: &TransactionContext{}, + RetryInfo: &RetryInfo{}, + ActiveRoles: make([]*auth.RoleIdentity, 0, 10), + StrictSQLMode: true, + AutoIncrementIncrement: DefAutoIncrementIncrement, + AutoIncrementOffset: DefAutoIncrementOffset, + Status: mysql.ServerStatusAutocommit, + StmtCtx: new(stmtctx.StatementContext), + AllowAggPushDown: false, + AllowBCJ: false, + BroadcastJoinThresholdSize: DefBroadcastJoinThresholdSize, + BroadcastJoinThresholdCount: DefBroadcastJoinThresholdSize, + OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, + RetryLimit: DefTiDBRetryLimit, + DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, + DDLReorgPriority: kv.PriorityLow, + allowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, + preferRangeScan: DefOptPreferRangeScan, + CorrelationThreshold: DefOptCorrelationThreshold, + CorrelationExpFactor: DefOptCorrelationExpFactor, + CPUFactor: DefOptCPUFactor, + CopCPUFactor: DefOptCopCPUFactor, + CopTiFlashConcurrencyFactor: DefOptTiFlashConcurrencyFactor, + NetworkFactor: DefOptNetworkFactor, + ScanFactor: DefOptScanFactor, + DescScanFactor: DefOptDescScanFactor, + SeekFactor: DefOptSeekFactor, + MemoryFactor: DefOptMemoryFactor, + DiskFactor: DefOptDiskFactor, + ConcurrencyFactor: DefOptConcurrencyFactor, + EnableRadixJoin: false, + EnableVectorizedExpression: DefEnableVectorizedExpression, + L2CacheSize: cpuid.CPU.Cache.L2, + CommandValue: uint32(mysql.ComSleep), + TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, + SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, + WaitSplitRegionFinish: DefTiDBWaitSplitRegionFinish, + WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, + enableIndexMerge: false, + EnableNoopFuncs: DefTiDBEnableNoopFuncs, + replicaRead: kv.ReplicaReadLeader, + AllowRemoveAutoInc: DefTiDBAllowRemoveAutoInc, + UsePlanBaselines: DefTiDBUsePlanBaselines, + EvolvePlanBaselines: DefTiDBEvolvePlanBaselines, + EnableExtendedStats: false, + IsolationReadEngines: make(map[kv.StoreType]struct{}), + LockWaitTimeout: DefInnodbLockWaitTimeout * 1000, + MetricSchemaStep: DefTiDBMetricSchemaStep, + MetricSchemaRangeDuration: DefTiDBMetricSchemaRangeDuration, + SequenceState: NewSequenceState(), + WindowingUseHighPrecision: true, + PrevFoundInPlanCache: DefTiDBFoundInPlanCache, + FoundInPlanCache: DefTiDBFoundInPlanCache, + PrevFoundInBinding: DefTiDBFoundInBinding, + FoundInBinding: DefTiDBFoundInBinding, + SelectLimit: math.MaxUint64, + AllowAutoRandExplicitInsert: DefTiDBAllowAutoRandExplicitInsert, + EnableClusteredIndex: DefTiDBEnableClusteredIndex, + EnableParallelApply: DefTiDBEnableParallelApply, + ShardAllocateStep: DefTiDBShardAllocateStep, + EnableChangeColumnType: DefTiDBChangeColumnType, + EnableChangeMultiSchema: DefTiDBChangeMultiSchema, + EnablePointGetCache: DefTiDBPointGetCache, + EnableAlterPlacement: DefTiDBEnableAlterPlacement, + EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn, + PartitionPruneMode: *atomic2.NewString(DefTiDBPartitionPruneMode), + TxnScope: config.GetGlobalConfig().TxnScope, + EnabledRateLimitAction: DefTiDBEnableRateLimitAction, + EnableAsyncCommit: DefTiDBEnableAsyncCommit, + Enable1PC: DefTiDBEnable1PC, + GuaranteeLinearizability: DefTiDBGuaranteeLinearizability, + AnalyzeVersion: DefTiDBAnalyzeVersion, + EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, } vars.KVVars = kv.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ @@ -1681,8 +1681,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableAsyncCommit = TiDBOptOn(val) case TiDBEnable1PC: s.Enable1PC = TiDBOptOn(val) - case TiDBGuaranteeExternalConsistency: - s.GuaranteeExternalConsistency = TiDBOptOn(val) + case TiDBGuaranteeLinearizability: + s.GuaranteeLinearizability = TiDBOptOn(val) case TiDBAnalyzeVersion: s.AnalyzeVersion = tidbOptPositiveInt32(val, DefTiDBAnalyzeVersion) case TiDBEnableIndexMergeJoin: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 68c4a20f711aa..86673e68469f9 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -746,7 +746,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableAmendPessimisticTxn, Value: BoolToOnOff(DefTiDBEnableAmendPessimisticTxn), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableAsyncCommit, Value: BoolToOnOff(DefTiDBEnableAsyncCommit), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnable1PC, Value: BoolToOnOff(DefTiDBEnable1PC), Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBGuaranteeExternalConsistency, Value: BoolToOnOff(DefTiDBGuaranteeExternalConsistency), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBGuaranteeLinearizability, Value: BoolToOnOff(DefTiDBGuaranteeLinearizability), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Type: TypeInt, MinValue: 1, MaxValue: 2}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableIndexMergeJoin, Value: BoolToOnOff(DefTiDBEnableIndexMergeJoin), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBTrackAggregateMemoryUsage, Value: BoolToOnOff(DefTiDBTrackAggregateMemoryUsage), Type: TypeBool}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index e95374df9974d..b60c0f1b27331 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -509,8 +509,8 @@ const ( // TiDBEnable1PC indicates whether to enable the one-phase commit feature. TiDBEnable1PC = "tidb_enable_1pc" - // TiDBGuaranteeExternalConsistency indicates whether maintain the external consistency. - TiDBGuaranteeExternalConsistency = "tidb_guarantee_external_consistency" + // TiDBGuaranteeLinearizability indicates whether to guarantee linearizability. + TiDBGuaranteeLinearizability = "tidb_guarantee_linearizability" // TiDBAnalyzeVersion indicates the how tidb collects the analyzed statistics and how use to it. TiDBAnalyzeVersion = "tidb_analyze_version" @@ -539,126 +539,126 @@ const ( // Default TiDB system variable values. const ( - DefHostname = "localhost" - DefIndexLookupConcurrency = ConcurrencyUnset - DefIndexLookupJoinConcurrency = ConcurrencyUnset - DefIndexSerialScanConcurrency = 1 - DefIndexJoinBatchSize = 25000 - DefIndexLookupSize = 20000 - DefDistSQLScanConcurrency = 15 - DefBuildStatsConcurrency = 4 - DefAutoAnalyzeRatio = 0.5 - DefAutoAnalyzeStartTime = "00:00 +0000" - DefAutoAnalyzeEndTime = "23:59 +0000" - DefAutoIncrementIncrement = 1 - DefAutoIncrementOffset = 1 - DefChecksumTableConcurrency = 4 - DefSkipUTF8Check = false - DefSkipASCIICheck = false - DefOptAggPushDown = false - DefOptBCJ = false - DefOptWriteRowID = false - DefOptCorrelationThreshold = 0.9 - DefOptCorrelationExpFactor = 1 - DefOptCPUFactor = 3.0 - DefOptCopCPUFactor = 3.0 - DefOptTiFlashConcurrencyFactor = 24.0 - DefOptNetworkFactor = 1.0 - DefOptScanFactor = 1.5 - DefOptDescScanFactor = 3.0 - DefOptSeekFactor = 20.0 - DefOptMemoryFactor = 0.001 - DefOptDiskFactor = 1.5 - DefOptConcurrencyFactor = 3.0 - DefOptInSubqToJoinAndAgg = true - DefOptPreferRangeScan = false - DefBatchInsert = false - DefBatchDelete = false - DefBatchCommit = false - DefCurretTS = 0 - DefInitChunkSize = 32 - DefMaxChunkSize = 1024 - DefDMLBatchSize = 0 - DefMaxPreparedStmtCount = -1 - DefWaitTimeout = 0 - DefTiDBMemQuotaApplyCache = 32 << 20 // 32MB. - DefTiDBMemQuotaHashJoin = 32 << 30 // 32GB. - DefTiDBMemQuotaMergeJoin = 32 << 30 // 32GB. - DefTiDBMemQuotaSort = 32 << 30 // 32GB. - DefTiDBMemQuotaTopn = 32 << 30 // 32GB. - DefTiDBMemQuotaIndexLookupReader = 32 << 30 // 32GB. - DefTiDBMemQuotaIndexLookupJoin = 32 << 30 // 32GB. - DefTiDBMemQuotaDistSQL = 32 << 30 // 32GB. - DefTiDBGeneralLog = false - DefTiDBPProfSQLCPU = 0 - DefTiDBRetryLimit = 10 - DefTiDBDisableTxnAutoRetry = true - DefTiDBConstraintCheckInPlace = false - DefTiDBHashJoinConcurrency = ConcurrencyUnset - DefTiDBProjectionConcurrency = ConcurrencyUnset - DefBroadcastJoinThresholdSize = 100 * 1024 * 1024 - DefBroadcastJoinThresholdCount = 10 * 1024 - DefTiDBOptimizerSelectivityLevel = 0 - DefTiDBAllowBatchCop = 1 - DefTiDBAllowMPPExecution = false - DefTiDBTxnMode = "" - DefTiDBRowFormatV1 = 1 - DefTiDBRowFormatV2 = 2 - DefTiDBDDLReorgWorkerCount = 4 - DefTiDBDDLReorgBatchSize = 256 - DefTiDBDDLErrorCountLimit = 512 - DefTiDBMaxDeltaSchemaCount = 1024 - DefTiDBChangeColumnType = false - DefTiDBChangeMultiSchema = false - DefTiDBPointGetCache = false - DefTiDBEnableAlterPlacement = false - DefTiDBHashAggPartialConcurrency = ConcurrencyUnset - DefTiDBHashAggFinalConcurrency = ConcurrencyUnset - DefTiDBWindowConcurrency = ConcurrencyUnset - DefTiDBMergeJoinConcurrency = 1 // disable optimization by default - DefTiDBStreamAggConcurrency = 1 - DefTiDBForcePriority = mysql.NoPriority - DefTiDBUseRadixJoin = false - DefEnableWindowFunction = true - DefEnableStrictDoubleTypeCheck = true - DefEnableVectorizedExpression = true - DefTiDBOptJoinReorderThreshold = 0 - DefTiDBDDLSlowOprThreshold = 300 - DefTiDBUseFastAnalyze = false - DefTiDBSkipIsolationLevelCheck = false - DefTiDBExpensiveQueryTimeThreshold = 60 // 60s - DefTiDBScatterRegion = false - DefTiDBWaitSplitRegionFinish = true - DefWaitSplitRegionTimeout = 300 // 300s - DefTiDBEnableNoopFuncs = false - DefTiDBAllowRemoveAutoInc = false - DefTiDBUsePlanBaselines = true - DefTiDBEvolvePlanBaselines = false - DefTiDBEvolvePlanTaskMaxTime = 600 // 600s - DefTiDBEvolvePlanTaskStartTime = "00:00 +0000" - DefTiDBEvolvePlanTaskEndTime = "23:59 +0000" - DefInnodbLockWaitTimeout = 50 // 50s - DefTiDBStoreLimit = 0 - DefTiDBMetricSchemaStep = 60 // 60s - DefTiDBMetricSchemaRangeDuration = 60 // 60s - DefTiDBFoundInPlanCache = false - DefTiDBFoundInBinding = false - DefTiDBEnableCollectExecutionInfo = true - DefTiDBAllowAutoRandExplicitInsert = false - DefTiDBEnableClusteredIndex = false - DefTiDBRedactLog = false - DefTiDBShardAllocateStep = math.MaxInt64 - DefTiDBEnableTelemetry = true - DefTiDBEnableParallelApply = false - DefTiDBEnableAmendPessimisticTxn = false - DefTiDBPartitionPruneMode = "static-only" - DefTiDBEnableRateLimitAction = true - DefTiDBEnableAsyncCommit = false - DefTiDBEnable1PC = false - DefTiDBGuaranteeExternalConsistency = true - DefTiDBAnalyzeVersion = 1 - DefTiDBEnableIndexMergeJoin = false - DefTiDBTrackAggregateMemoryUsage = false + DefHostname = "localhost" + DefIndexLookupConcurrency = ConcurrencyUnset + DefIndexLookupJoinConcurrency = ConcurrencyUnset + DefIndexSerialScanConcurrency = 1 + DefIndexJoinBatchSize = 25000 + DefIndexLookupSize = 20000 + DefDistSQLScanConcurrency = 15 + DefBuildStatsConcurrency = 4 + DefAutoAnalyzeRatio = 0.5 + DefAutoAnalyzeStartTime = "00:00 +0000" + DefAutoAnalyzeEndTime = "23:59 +0000" + DefAutoIncrementIncrement = 1 + DefAutoIncrementOffset = 1 + DefChecksumTableConcurrency = 4 + DefSkipUTF8Check = false + DefSkipASCIICheck = false + DefOptAggPushDown = false + DefOptBCJ = false + DefOptWriteRowID = false + DefOptCorrelationThreshold = 0.9 + DefOptCorrelationExpFactor = 1 + DefOptCPUFactor = 3.0 + DefOptCopCPUFactor = 3.0 + DefOptTiFlashConcurrencyFactor = 24.0 + DefOptNetworkFactor = 1.0 + DefOptScanFactor = 1.5 + DefOptDescScanFactor = 3.0 + DefOptSeekFactor = 20.0 + DefOptMemoryFactor = 0.001 + DefOptDiskFactor = 1.5 + DefOptConcurrencyFactor = 3.0 + DefOptInSubqToJoinAndAgg = true + DefOptPreferRangeScan = false + DefBatchInsert = false + DefBatchDelete = false + DefBatchCommit = false + DefCurretTS = 0 + DefInitChunkSize = 32 + DefMaxChunkSize = 1024 + DefDMLBatchSize = 0 + DefMaxPreparedStmtCount = -1 + DefWaitTimeout = 0 + DefTiDBMemQuotaApplyCache = 32 << 20 // 32MB. + DefTiDBMemQuotaHashJoin = 32 << 30 // 32GB. + DefTiDBMemQuotaMergeJoin = 32 << 30 // 32GB. + DefTiDBMemQuotaSort = 32 << 30 // 32GB. + DefTiDBMemQuotaTopn = 32 << 30 // 32GB. + DefTiDBMemQuotaIndexLookupReader = 32 << 30 // 32GB. + DefTiDBMemQuotaIndexLookupJoin = 32 << 30 // 32GB. + DefTiDBMemQuotaDistSQL = 32 << 30 // 32GB. + DefTiDBGeneralLog = false + DefTiDBPProfSQLCPU = 0 + DefTiDBRetryLimit = 10 + DefTiDBDisableTxnAutoRetry = true + DefTiDBConstraintCheckInPlace = false + DefTiDBHashJoinConcurrency = ConcurrencyUnset + DefTiDBProjectionConcurrency = ConcurrencyUnset + DefBroadcastJoinThresholdSize = 100 * 1024 * 1024 + DefBroadcastJoinThresholdCount = 10 * 1024 + DefTiDBOptimizerSelectivityLevel = 0 + DefTiDBAllowBatchCop = 1 + DefTiDBAllowMPPExecution = false + DefTiDBTxnMode = "" + DefTiDBRowFormatV1 = 1 + DefTiDBRowFormatV2 = 2 + DefTiDBDDLReorgWorkerCount = 4 + DefTiDBDDLReorgBatchSize = 256 + DefTiDBDDLErrorCountLimit = 512 + DefTiDBMaxDeltaSchemaCount = 1024 + DefTiDBChangeColumnType = false + DefTiDBChangeMultiSchema = false + DefTiDBPointGetCache = false + DefTiDBEnableAlterPlacement = false + DefTiDBHashAggPartialConcurrency = ConcurrencyUnset + DefTiDBHashAggFinalConcurrency = ConcurrencyUnset + DefTiDBWindowConcurrency = ConcurrencyUnset + DefTiDBMergeJoinConcurrency = 1 // disable optimization by default + DefTiDBStreamAggConcurrency = 1 + DefTiDBForcePriority = mysql.NoPriority + DefTiDBUseRadixJoin = false + DefEnableWindowFunction = true + DefEnableStrictDoubleTypeCheck = true + DefEnableVectorizedExpression = true + DefTiDBOptJoinReorderThreshold = 0 + DefTiDBDDLSlowOprThreshold = 300 + DefTiDBUseFastAnalyze = false + DefTiDBSkipIsolationLevelCheck = false + DefTiDBExpensiveQueryTimeThreshold = 60 // 60s + DefTiDBScatterRegion = false + DefTiDBWaitSplitRegionFinish = true + DefWaitSplitRegionTimeout = 300 // 300s + DefTiDBEnableNoopFuncs = false + DefTiDBAllowRemoveAutoInc = false + DefTiDBUsePlanBaselines = true + DefTiDBEvolvePlanBaselines = false + DefTiDBEvolvePlanTaskMaxTime = 600 // 600s + DefTiDBEvolvePlanTaskStartTime = "00:00 +0000" + DefTiDBEvolvePlanTaskEndTime = "23:59 +0000" + DefInnodbLockWaitTimeout = 50 // 50s + DefTiDBStoreLimit = 0 + DefTiDBMetricSchemaStep = 60 // 60s + DefTiDBMetricSchemaRangeDuration = 60 // 60s + DefTiDBFoundInPlanCache = false + DefTiDBFoundInBinding = false + DefTiDBEnableCollectExecutionInfo = true + DefTiDBAllowAutoRandExplicitInsert = false + DefTiDBEnableClusteredIndex = false + DefTiDBRedactLog = false + DefTiDBShardAllocateStep = math.MaxInt64 + DefTiDBEnableTelemetry = true + DefTiDBEnableParallelApply = false + DefTiDBEnableAmendPessimisticTxn = false + DefTiDBPartitionPruneMode = "static-only" + DefTiDBEnableRateLimitAction = true + DefTiDBEnableAsyncCommit = false + DefTiDBEnable1PC = false + DefTiDBGuaranteeLinearizability = true + DefTiDBAnalyzeVersion = 1 + DefTiDBEnableIndexMergeJoin = false + DefTiDBTrackAggregateMemoryUsage = false ) // Process global variables. @@ -687,7 +687,15 @@ var ( ) // FeatureSwitchVariables is used to filter result of show variables, these switches should be turn blind to users. -var FeatureSwitchVariables = []string{TiDBEnableChangeColumnType, TiDBEnablePointGetCache, TiDBEnableAlterPlacement, TiDBEnableChangeMultiSchema, TiDBEnableAsyncCommit, TiDBEnable1PC, TiDBGuaranteeExternalConsistency} +var FeatureSwitchVariables = []string{ + TiDBEnableChangeColumnType, + TiDBEnablePointGetCache, + TiDBEnableAlterPlacement, + TiDBEnableChangeMultiSchema, + TiDBEnableAsyncCommit, + TiDBEnable1PC, + TiDBGuaranteeLinearizability, +} // FilterImplicitFeatureSwitch is used to filter result of show variables, these switches should be turn blind to users. func FilterImplicitFeatureSwitch(sysVar *SysVar) bool { diff --git a/store/tikv/1pc_test.go b/store/tikv/1pc_test.go index 159049b0f2fd8..eef4315266dd2 100644 --- a/store/tikv/1pc_test.go +++ b/store/tikv/1pc_test.go @@ -224,9 +224,9 @@ func (s *testOnePCSuite) Test1PCDisallowMultiRegion(c *C) { } } -// It's just a simple validation of external consistency. +// It's just a simple validation of linearizability. // Extra tests are needed to test this feature with the control of the TiKV cluster. -func (s *testOnePCSuite) Test1PCExternalConsistency(c *C) { +func (s *testOnePCSuite) Test1PCLinearizability(c *C) { t1, err := s.store.Begin() c.Assert(err, IsNil) t2, err := s.store.Begin() diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 04f2e757c06f4..8562821a953a1 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -998,13 +998,13 @@ func (c *twoPhaseCommitter) checkOnePC() bool { return c.connID > 0 && !c.shouldWriteBinlog() && enable1PCOption != nil && enable1PCOption.(bool) } -func (c *twoPhaseCommitter) needExternalConsistency() bool { - guaranteeExternalConsistencyOption := c.txn.us.GetOption(kv.GuaranteeExternalConsistency) - if guaranteeExternalConsistencyOption == nil { +func (c *twoPhaseCommitter) needLinearizability() bool { + GuaranteeLinearizabilityOption := c.txn.us.GetOption(kv.GuaranteeLinearizability) + if GuaranteeLinearizabilityOption == nil { // by default, guarantee return true } - return guaranteeExternalConsistencyOption.(bool) + return GuaranteeLinearizabilityOption.(bool) } func (c *twoPhaseCommitter) isAsyncCommit() bool { @@ -1118,11 +1118,11 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { commitTSMayBeCalculated = true c.setOnePC(true) } - // If we want to use async commit or 1PC and also want external consistency across + // If we want to use async commit or 1PC and also want linearizability across // all nodes, we have to make sure the commit TS of this transaction is greater // than the snapshot TS of all existent readers. So we get a new timestamp // from PD as our MinCommitTS. - if commitTSMayBeCalculated && c.needExternalConsistency() { + if commitTSMayBeCalculated && c.needLinearizability() { minCommitTS, err := c.store.oracle.GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) // If we fail to get a timestamp from PD, we just propagate the failure // instead of falling back to the normal 2PC because a normal 2PC will diff --git a/store/tikv/async_commit_test.go b/store/tikv/async_commit_test.go index ff296b33bf5e3..01d15c648d9b5 100644 --- a/store/tikv/async_commit_test.go +++ b/store/tikv/async_commit_test.go @@ -115,9 +115,9 @@ func (s *testAsyncCommitCommon) mustGetNoneFromSnapshot(c *C, version uint64, ke c.Assert(errors.Cause(err), Equals, kv.ErrNotExist) } -func (s *testAsyncCommitCommon) beginAsyncCommitWithExternalConsistency(c *C) *tikvTxn { +func (s *testAsyncCommitCommon) beginAsyncCommitWithLinearizability(c *C) *tikvTxn { txn := s.beginAsyncCommit(c) - txn.SetOption(kv.GuaranteeExternalConsistency, true) + txn.SetOption(kv.GuaranteeLinearizability, true) return txn } @@ -366,11 +366,11 @@ func (s *testAsyncCommitSuite) TestRepeatableRead(c *C) { test(true) } -// It's just a simple validation of external consistency. +// It's just a simple validation of linearizability. // Extra tests are needed to test this feature with the control of the TiKV cluster. -func (s *testAsyncCommitSuite) TestAsyncCommitExternalConsistency(c *C) { - t1 := s.beginAsyncCommitWithExternalConsistency(c) - t2 := s.beginAsyncCommitWithExternalConsistency(c) +func (s *testAsyncCommitSuite) TestAsyncCommitLinearizability(c *C) { + t1 := s.beginAsyncCommitWithLinearizability(c) + t2 := s.beginAsyncCommitWithLinearizability(c) err := t1.Set([]byte("a"), []byte("a1")) c.Assert(err, IsNil) err = t2.Set([]byte("b"), []byte("b1"))