Skip to content

Commit

Permalink
replace the ambiguous name 'external consistency'
Browse files Browse the repository at this point in the history
Signed-off-by: ekexium <[email protected]>
  • Loading branch information
ekexium committed Feb 3, 2021
1 parent 7ede31c commit 044d27a
Show file tree
Hide file tree
Showing 10 changed files with 236 additions and 230 deletions.
2 changes: 1 addition & 1 deletion executor/simple.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}

Expand Down
2 changes: 0 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -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=
Expand Down
4 changes: 2 additions & 2 deletions kv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
8 changes: 4 additions & 4 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down Expand Up @@ -2557,7 +2557,7 @@ var builtinGlobalVariable = []string{
variable.TiDBEnableRateLimitAction,
variable.TiDBEnableAsyncCommit,
variable.TiDBEnable1PC,
variable.TiDBGuaranteeExternalConsistency,
variable.TiDBGuaranteeLinearizability,
variable.TiDBAnalyzeVersion,
variable.TiDBEnableIndexMergeJoin,
variable.TiDBTrackAggregateMemoryUsage,
Expand Down
166 changes: 83 additions & 83 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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{
Expand Down Expand Up @@ -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:
Expand Down
2 changes: 1 addition & 1 deletion sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -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},
Expand Down
Loading

0 comments on commit 044d27a

Please sign in to comment.