From b1acc730b48b2194fe996d0aff937be361f0450b Mon Sep 17 00:00:00 2001 From: ekexium Date: Wed, 27 Jan 2021 15:34:03 +0800 Subject: [PATCH 01/15] sessionctx: remove tidb_guarantee_external_consistency variable Signed-off-by: ekexium --- session/session.go | 3 +- sessionctx/variable/session.go | 163 ++++++++++----------- sessionctx/variable/sysvar.go | 1 - sessionctx/variable/tidb_vars.go | 242 +++++++++++++++---------------- 4 files changed, 199 insertions(+), 210 deletions(-) diff --git a/session/session.go b/session/session.go index 01e546072ac93..6df228357b32f 100644 --- a/session/session.go +++ b/session/session.go @@ -508,7 +508,7 @@ func (s *session) doCommit(ctx context.Context) error { } s.txn.SetOption(kv.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) s.txn.SetOption(kv.Enable1PC, s.GetSessionVars().Enable1PC) - s.txn.SetOption(kv.GuaranteeExternalConsistency, s.GetSessionVars().GuaranteeExternalConsistency) + s.txn.SetOption(kv.GuaranteeExternalConsistency, true) return s.txn.Commit(sessionctx.SetCommitCtx(ctx, s)) } @@ -2554,7 +2554,6 @@ var builtinGlobalVariable = []string{ variable.TiDBEnableRateLimitAction, variable.TiDBEnableAsyncCommit, variable.TiDBEnable1PC, - variable.TiDBGuaranteeExternalConsistency, variable.TiDBAnalyzeVersion, variable.TiDBEnableIndexMergeJoin, variable.TiDBTrackAggregateMemoryUsage, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 322c7755731b0..fe5c47a6c1144 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -795,8 +795,6 @@ type SessionVars struct { // Enable1PC indicates whether to enable the one-phase commit feature. Enable1PC bool - GuaranteeExternalConsistency bool - // AnalyzeVersion indicates how TiDB collect and use analyzed statistics. AnalyzeVersion int @@ -883,86 +881,85 @@ 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, + AnalyzeVersion: DefTiDBAnalyzeVersion, + EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, } vars.KVVars = kv.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ @@ -1681,8 +1678,6 @@ 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 TiDBAnalyzeVersion: s.AnalyzeVersion = tidbOptPositiveInt32(val, DefTiDBAnalyzeVersion) case TiDBEnableIndexMergeJoin: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 68c4a20f711aa..3ee2b09d08285 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -746,7 +746,6 @@ 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: 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 5986c633b0c14..deed0f38f6c78 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -509,9 +509,6 @@ 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" - // TiDBAnalyzeVersion indicates the how tidb collects the analyzed statistics and how use to it. TiDBAnalyzeVersion = "tidb_analyze_version" @@ -539,126 +536,125 @@ 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 = false - 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 + DefTiDBAnalyzeVersion = 1 + DefTiDBEnableIndexMergeJoin = false + DefTiDBTrackAggregateMemoryUsage = false ) // Process global variables. From 1c75da539a93cf8e7c71c37e9e9faa6d06c51b23 Mon Sep 17 00:00:00 2001 From: ekexium Date: Thu, 28 Jan 2021 15:49:12 +0800 Subject: [PATCH 02/15] replace pingcap/parser Signed-off-by: ekexium --- go.mod | 2 ++ go.sum | 2 ++ 2 files changed, 4 insertions(+) diff --git a/go.mod b/go.mod index 49db0b780b613..e524347acfa08 100644 --- a/go.mod +++ b/go.mod @@ -85,3 +85,5 @@ require ( ) go 1.13 + +replace github.com/pingcap/parser => github.com/ekexium/parser v0.0.0-20210127083829-8b5443f3aa8b diff --git a/go.sum b/go.sum index e51fe54a7a1dc..c5fa5ff8c4eb7 100644 --- a/go.sum +++ b/go.sum @@ -220,6 +220,8 @@ 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-20210127083829-8b5443f3aa8b h1:6YQSgGiTZh4AWL1eZXYsnF+3OpqESDy0Hbu5C8HJ2e4= +github.com/ekexium/parser v0.0.0-20210127083829-8b5443f3aa8b/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= github.com/elazarl/go-bindata-assetfs v1.0.0/go.mod h1:v+YaWX3bdea5J/mo8dSETolEo7R71Vk1u8bnjau5yw4= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= From 8315d6e9b209cd76333783e938cfc339790e26c1 Mon Sep 17 00:00:00 2001 From: ekexium Date: Thu, 28 Jan 2021 16:11:05 +0800 Subject: [PATCH 03/15] executor: set external consistency option Signed-off-by: ekexium --- executor/simple.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/executor/simple.go b/executor/simple.go index 48f4b5264c79d..0bef64f7ca718 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -559,6 +559,7 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { if s.ReadOnly && s.Bound != nil { return e.executeStartTransactionReadOnlyWithTimestampBound(ctx, s) } + // If BEGIN is the first statement in TxnCtx, we can reuse the existing transaction, without the // need to call NewTxn, which commits the existing transaction and begins a new one. // If the last un-committed/un-rollback transaction is a time-bounded read-only transaction, we should @@ -589,6 +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.WithoutExternalConsistency) return nil } From 0929cced85a9f0ce307c69c9aec645b2aa2c57c2 Mon Sep 17 00:00:00 2001 From: ekexium Date: Thu, 28 Jan 2021 17:06:21 +0800 Subject: [PATCH 04/15] make tidy Signed-off-by: ekexium --- go.sum | 4 ---- 1 file changed, 4 deletions(-) diff --git a/go.sum b/go.sum index c5fa5ff8c4eb7..656254f008777 100644 --- a/go.sum +++ b/go.sum @@ -653,10 +653,6 @@ github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8 h1:M+DNpOu/I3uDmwee6vcnoPd6GgSMqND4gxvDQ/W584U= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20190506092653-e336082eb825/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= -github.com/pingcap/parser v0.0.0-20200422082501-7329d80eaf2c/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/pingcap/parser v0.0.0-20210125075924-ffe0fda947cb h1:16lZkJzsAOIY781DmUgSU0El+GpV8fl3BohzVmHO1rE= -github.com/pingcap/parser v0.0.0-20210125075924-ffe0fda947cb/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= github.com/pingcap/pd v2.1.5+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= From 4a31085bd81e7af9dffdeed911ff79438c2e0f01 Mon Sep 17 00:00:00 2001 From: ekexium Date: Thu, 28 Jan 2021 18:36:18 +0800 Subject: [PATCH 05/15] resume tidb_guarantee_external_consistency variable Signed-off-by: ekexium --- session/session.go | 3 +- sessionctx/variable/session.go | 163 +++++++++++---------- sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 242 ++++++++++++++++--------------- store/tikv/2pc.go | 6 +- 5 files changed, 215 insertions(+), 200 deletions(-) diff --git a/session/session.go b/session/session.go index 6df228357b32f..01e546072ac93 100644 --- a/session/session.go +++ b/session/session.go @@ -508,7 +508,7 @@ func (s *session) doCommit(ctx context.Context) error { } s.txn.SetOption(kv.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) s.txn.SetOption(kv.Enable1PC, s.GetSessionVars().Enable1PC) - s.txn.SetOption(kv.GuaranteeExternalConsistency, true) + s.txn.SetOption(kv.GuaranteeExternalConsistency, s.GetSessionVars().GuaranteeExternalConsistency) return s.txn.Commit(sessionctx.SetCommitCtx(ctx, s)) } @@ -2554,6 +2554,7 @@ var builtinGlobalVariable = []string{ variable.TiDBEnableRateLimitAction, variable.TiDBEnableAsyncCommit, variable.TiDBEnable1PC, + variable.TiDBGuaranteeExternalConsistency, variable.TiDBAnalyzeVersion, variable.TiDBEnableIndexMergeJoin, variable.TiDBTrackAggregateMemoryUsage, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index fe5c47a6c1144..322c7755731b0 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -795,6 +795,8 @@ type SessionVars struct { // Enable1PC indicates whether to enable the one-phase commit feature. Enable1PC bool + GuaranteeExternalConsistency bool + // AnalyzeVersion indicates how TiDB collect and use analyzed statistics. AnalyzeVersion int @@ -881,85 +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, - 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, + GuaranteeExternalConsistency: DefTiDBGuaranteeExternalConsistency, + AnalyzeVersion: DefTiDBAnalyzeVersion, + EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, } vars.KVVars = kv.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ @@ -1678,6 +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 TiDBAnalyzeVersion: s.AnalyzeVersion = tidbOptPositiveInt32(val, DefTiDBAnalyzeVersion) case TiDBEnableIndexMergeJoin: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 3ee2b09d08285..68c4a20f711aa 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -746,6 +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: 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 deed0f38f6c78..5986c633b0c14 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -509,6 +509,9 @@ 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" + // TiDBAnalyzeVersion indicates the how tidb collects the analyzed statistics and how use to it. TiDBAnalyzeVersion = "tidb_analyze_version" @@ -536,125 +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 - 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 + DefTiDBGuaranteeExternalConsistency = false + DefTiDBAnalyzeVersion = 1 + DefTiDBEnableIndexMergeJoin = false + DefTiDBTrackAggregateMemoryUsage = false ) // Process global variables. diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 652e5f71ec133..04f2e757c06f4 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -1000,7 +1000,11 @@ func (c *twoPhaseCommitter) checkOnePC() bool { func (c *twoPhaseCommitter) needExternalConsistency() bool { guaranteeExternalConsistencyOption := c.txn.us.GetOption(kv.GuaranteeExternalConsistency) - return guaranteeExternalConsistencyOption != nil && guaranteeExternalConsistencyOption.(bool) + if guaranteeExternalConsistencyOption == nil { + // by default, guarantee + return true + } + return guaranteeExternalConsistencyOption.(bool) } func (c *twoPhaseCommitter) isAsyncCommit() bool { From 96c52f4fc7a91236d85353275473618ec3fc96c2 Mon Sep 17 00:00:00 2001 From: ekexium Date: Wed, 3 Feb 2021 14:59:16 +0800 Subject: [PATCH 06/15] sysvar: enable tidb_guarantee_external_consistency by default Signed-off-by: ekexium --- sessionctx/variable/tidb_vars.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 5986c633b0c14..8e971fbd785f2 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -655,7 +655,7 @@ const ( DefTiDBEnableRateLimitAction = true DefTiDBEnableAsyncCommit = false DefTiDBEnable1PC = false - DefTiDBGuaranteeExternalConsistency = false + DefTiDBGuaranteeExternalConsistency = true DefTiDBAnalyzeVersion = 1 DefTiDBEnableIndexMergeJoin = false DefTiDBTrackAggregateMemoryUsage = false From 4c92eace7a950902b6278242751837ada4598638 Mon Sep 17 00:00:00 2001 From: ekexium Date: Wed, 3 Feb 2021 15:03:31 +0800 Subject: [PATCH 07/15] hide async-commit related sysvars Signed-off-by: ekexium --- go.mod | 8 ++++---- go.sum | 24 ++++++++++++------------ sessionctx/variable/tidb_vars.go | 2 +- 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/go.mod b/go.mod index e524347acfa08..0c2dd70b5e25f 100644 --- a/go.mod +++ b/go.mod @@ -70,16 +70,16 @@ require ( go.uber.org/multierr v1.6.0 // indirect go.uber.org/zap v1.16.0 golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a // indirect - golang.org/x/net v0.0.0-20200904194848-62affa334b73 + golang.org/x/net v0.0.0-20201021035429-f5854403a974 golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9 - golang.org/x/sys v0.0.0-20200819171115-d785dc25833f + golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4 golang.org/x/text v0.3.4 - golang.org/x/tools v0.0.0-20200820010801-b793a1359eac + golang.org/x/tools v0.1.0 google.golang.org/grpc v1.27.1 gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0 gopkg.in/yaml.v2 v2.3.0 // indirect - honnef.co/go/tools v0.1.0 // indirect + honnef.co/go/tools v0.1.1 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) diff --git a/go.sum b/go.sum index 656254f008777..1e5b14166d4d2 100644 --- a/go.sum +++ b/go.sum @@ -887,7 +887,7 @@ github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1: github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/gopher-lua v0.0.0-20181031023651-12c4817b42c5/go.mod h1:aEV29XrmTYFr3CiRxZeGHpkvbwq+prZduBqMaascyCU= github.com/zaf/temp v0.0.0-20170209143821-94e385923345/go.mod h1:sXsZgXwh6DB0qlskmZVB4HE93e5YrktMrgUDPy9iYmY= github.com/zhangjinpeng1987/raft v0.0.0-20200819064223-df31bb68a018 h1:T3OrqVdcH6z6SakR7WkECvGpdkfB0MAur/6zf66GPxQ= @@ -1017,9 +1017,8 @@ golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200421231249-e086a090c8fd/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= -golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= -golang.org/x/net v0.0.0-20200904194848-62affa334b73 h1:MXfv8rhZWmFeqX3GNZRsd6vOLoaCHjYEX3qkRo3YBUA= -golang.org/x/net v0.0.0-20200904194848-62affa334b73/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20201021035429-f5854403a974 h1:IX6qOQeG5uLjB/hjjwjedwfjND0hgjPMMyO1RoIXQNI= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1032,7 +1031,6 @@ golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9 h1:SQFwaSi55rU7vdNs9Yr0Z324VNlrF+0wMqRXT4St8ck= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1077,8 +1075,9 @@ golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200413165638-669c56c373c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200420163511-1957bb5e6d1f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200819171115-d785dc25833f h1:KJuwZVtZBVzDmEDtB2zro9CXkD9O0dpCv4o2LHbQIAw= -golang.org/x/sys v0.0.0-20200819171115-d785dc25833f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4 h1:myAQVi0cGEoqQVR5POX+8RR2mrocKqNN1hmeMqhX27k= +golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= @@ -1135,13 +1134,14 @@ golang.org/x/tools v0.0.0-20200325010219-a49f79bcc224/go.mod h1:Sl4aGygMT6LrqrWc golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= golang.org/x/tools v0.0.0-20200422205258-72e4a01eba43/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200527183253-8e7acdbce89d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200609164405-eb789aa7ce50/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200820010801-b793a1359eac h1:DugppSxw0LSF8lcjaODPJZoDzq0ElTGskTst3ZaBkHI= -golang.org/x/tools v0.0.0-20200820010801-b793a1359eac/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.1.0 h1:po9/4sTYwZU9lPhi1tOrb4hCv3qrhiQ77LZfGa2OjwY= +golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.6.0/go.mod h1:9mxDZsDKxgMAuccQkewq682L+0eCu4dCN2yonUJTCLU= @@ -1253,8 +1253,8 @@ honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.1.0 h1:AWNL1W1i7f0wNZ8VwOKNJ0sliKvOF/adn0EHenfUh+c= -honnef.co/go/tools v0.1.0/go.mod h1:XtegFAyX/PfluP4921rXU5IkjkqBCDnUq4W8VCIoKvM= +honnef.co/go/tools v0.1.1 h1:EVDuO03OCZwpV2t/tLLxPmPiomagMoBOgfPt0FM+4IY= +honnef.co/go/tools v0.1.1/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= modernc.org/b v1.0.0/go.mod h1:uZWcZfRj1BpYzfN9JTerzlNUnnPsV9O2ZA8JsRcubNg= modernc.org/db v1.0.0/go.mod h1:kYD/cO29L/29RM0hXYl4i3+Q5VojL31kTUVpVJDw0s8= diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 8e971fbd785f2..e95374df9974d 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -687,7 +687,7 @@ 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} +var FeatureSwitchVariables = []string{TiDBEnableChangeColumnType, TiDBEnablePointGetCache, TiDBEnableAlterPlacement, TiDBEnableChangeMultiSchema, TiDBEnableAsyncCommit, TiDBEnable1PC, TiDBGuaranteeExternalConsistency} // FilterImplicitFeatureSwitch is used to filter result of show variables, these switches should be turn blind to users. func FilterImplicitFeatureSwitch(sysVar *SysVar) bool { From b6391000a12916e3ff197d8c568cd9496f85e1c8 Mon Sep 17 00:00:00 2001 From: ekexium Date: Wed, 3 Feb 2021 15:11:03 +0800 Subject: [PATCH 08/15] new syntax overrides sysvar Signed-off-by: ekexium --- session/session.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/session/session.go b/session/session.go index 01e546072ac93..afa1a4f4ad704 100644 --- a/session/session.go +++ b/session/session.go @@ -508,7 +508,10 @@ func (s *session) doCommit(ctx context.Context) error { } s.txn.SetOption(kv.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) s.txn.SetOption(kv.Enable1PC, s.GetSessionVars().Enable1PC) - s.txn.SetOption(kv.GuaranteeExternalConsistency, s.GetSessionVars().GuaranteeExternalConsistency) + // 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) + } return s.txn.Commit(sessionctx.SetCommitCtx(ctx, s)) } From 7ede31cb9d9475d000409d0f983b8d36340427fe Mon Sep 17 00:00:00 2001 From: ekexium Date: Wed, 3 Feb 2021 15:13:01 +0800 Subject: [PATCH 09/15] update parser dependency Signed-off-by: ekexium --- executor/simple.go | 2 +- go.mod | 2 +- go.sum | 6 ++++-- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/executor/simple.go b/executor/simple.go index 0bef64f7ca718..74ca7d2d0405a 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.WithoutExternalConsistency) + txn.SetOption(kv.GuaranteeExternalConsistency, !s.CausalConsistencyOnly) return nil } diff --git a/go.mod b/go.mod index 0c2dd70b5e25f..fae96de6b4b4f 100644 --- a/go.mod +++ b/go.mod @@ -86,4 +86,4 @@ require ( go 1.13 -replace github.com/pingcap/parser => github.com/ekexium/parser v0.0.0-20210127083829-8b5443f3aa8b +replace github.com/pingcap/parser => github.com/ekexium/parser v0.0.0-20210203081914-18a31cd39b53 diff --git a/go.sum b/go.sum index 1e5b14166d4d2..53280b25f6da3 100644 --- a/go.sum +++ b/go.sum @@ -220,8 +220,10 @@ 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-20210127083829-8b5443f3aa8b h1:6YQSgGiTZh4AWL1eZXYsnF+3OpqESDy0Hbu5C8HJ2e4= -github.com/ekexium/parser v0.0.0-20210127083829-8b5443f3aa8b/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= +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= github.com/elazarl/go-bindata-assetfs v1.0.0/go.mod h1:v+YaWX3bdea5J/mo8dSETolEo7R71Vk1u8bnjau5yw4= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= From 044d27ab436190217a21b07d3cdd5f370361f228 Mon Sep 17 00:00:00 2001 From: ekexium Date: Wed, 3 Feb 2021 16:32:03 +0800 Subject: [PATCH 10/15] 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")) From de28dd75e5c5edb7f6bb09debe4da77869c1ec67 Mon Sep 17 00:00:00 2001 From: ekexium Date: Wed, 3 Feb 2021 16:38:11 +0800 Subject: [PATCH 11/15] update comment Signed-off-by: ekexium --- session/session.go | 2 +- sessionctx/variable/session.go | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/session/session.go b/session/session.go index 377f366548887..21a22b988dfc4 100644 --- a/session/session.go +++ b/session/session.go @@ -508,7 +508,7 @@ func (s *session) doCommit(ctx context.Context) error { } s.txn.SetOption(kv.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) s.txn.SetOption(kv.Enable1PC, s.GetSessionVars().Enable1PC) - // priority of the sysvar is lower than `start transaction with causal consistency` + // priority of the sysvar is lower than `start transaction with causal consistency only` if s.txn.GetUnionStore().GetOption(kv.GuaranteeLinearizability) == nil { s.txn.SetOption(kv.GuaranteeLinearizability, s.GetSessionVars().GuaranteeLinearizability) } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 43ede082f2519..6fa44191c412c 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -795,6 +795,7 @@ type SessionVars struct { // Enable1PC indicates whether to enable the one-phase commit feature. Enable1PC bool + // GuaranteeLinearizability indicates whether to guarantee linearizability GuaranteeLinearizability bool // AnalyzeVersion indicates how TiDB collect and use analyzed statistics. From cc648a42e386ac84acbc9ec0405bda7107a3a19d Mon Sep 17 00:00:00 2001 From: ekexium Date: Wed, 3 Feb 2021 16:56:11 +0800 Subject: [PATCH 12/15] fix: normal 'start transaction' won't override sysvar Signed-off-by: ekexium --- executor/simple.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/executor/simple.go b/executor/simple.go index 8dacf5a05471a..7d4c02325392e 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -590,7 +590,9 @@ 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.GuaranteeLinearizability, !s.CausalConsistencyOnly) + if s.CausalConsistencyOnly { + txn.SetOption(kv.GuaranteeLinearizability, false) + } return nil } From e22085b5ec68850bca0b59154bef191dff5de503 Mon Sep 17 00:00:00 2001 From: ekexium Date: Thu, 4 Feb 2021 10:50:34 +0800 Subject: [PATCH 13/15] simplify code Signed-off-by: ekexium --- store/tikv/2pc.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 8562821a953a1..3d60e80a975f0 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -1000,11 +1000,8 @@ func (c *twoPhaseCommitter) checkOnePC() bool { func (c *twoPhaseCommitter) needLinearizability() bool { GuaranteeLinearizabilityOption := c.txn.us.GetOption(kv.GuaranteeLinearizability) - if GuaranteeLinearizabilityOption == nil { - // by default, guarantee - return true - } - return GuaranteeLinearizabilityOption.(bool) + // by default, guarantee + return GuaranteeLinearizabilityOption == nil || GuaranteeLinearizabilityOption.(bool) } func (c *twoPhaseCommitter) isAsyncCommit() bool { From 20cf29f07840b937a3833993816c6ddd7df4ca61 Mon Sep 17 00:00:00 2001 From: ekexium Date: Thu, 4 Feb 2021 10:55:42 +0800 Subject: [PATCH 14/15] session: add a GetOption method for Transaction Signed-off-by: ekexium --- kv/kv.go | 2 ++ session/session.go | 2 +- store/tikv/txn.go | 4 ++++ 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/kv/kv.go b/kv/kv.go index 90714f747b0a3..182c83cc9a772 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -265,6 +265,8 @@ type Transaction interface { // SetOption sets an option with a value, when val is nil, uses the default // value of this option. SetOption(opt Option, val interface{}) + // GetOption returns the option + GetOption(opt Option) interface{} // DelOption deletes an option. DelOption(opt Option) // IsReadOnly checks if the transaction has only performed read operations. diff --git a/session/session.go b/session/session.go index 21a22b988dfc4..db91756ff2599 100644 --- a/session/session.go +++ b/session/session.go @@ -509,7 +509,7 @@ func (s *session) doCommit(ctx context.Context) error { s.txn.SetOption(kv.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) s.txn.SetOption(kv.Enable1PC, s.GetSessionVars().Enable1PC) // priority of the sysvar is lower than `start transaction with causal consistency only` - if s.txn.GetUnionStore().GetOption(kv.GuaranteeLinearizability) == nil { + if s.txn.GetOption(kv.GuaranteeLinearizability) == nil { s.txn.SetOption(kv.GuaranteeLinearizability, s.GetSessionVars().GuaranteeLinearizability) } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 263de5ef81cfa..ae41db4e0f794 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -206,6 +206,10 @@ func (txn *tikvTxn) SetOption(opt kv.Option, val interface{}) { } } +func (txn *tikvTxn) GetOption(opt kv.Option) interface{} { + return txn.us.GetOption(opt) +} + func (txn *tikvTxn) DelOption(opt kv.Option) { txn.us.DelOption(opt) } From cdd3d515238d686683f4704a6e8b424a70e7039c Mon Sep 17 00:00:00 2001 From: ekexium Date: Thu, 4 Feb 2021 10:56:11 +0800 Subject: [PATCH 15/15] resume parser dependency Signed-off-by: ekexium --- go.mod | 6 ++---- go.sum | 10 ++++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/go.mod b/go.mod index fae96de6b4b4f..b2228d2d83b5b 100644 --- a/go.mod +++ b/go.mod @@ -46,7 +46,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20201215060142-f3dafca4c7fd github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8 - github.com/pingcap/parser v0.0.0-20210125075924-ffe0fda947cb + github.com/pingcap/parser v0.0.0-20210203141130-32ef3e0e2e13 github.com/pingcap/sysutil v0.0.0-20201130064824-f0c8aa6a6966 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible github.com/pingcap/tipb v0.0.0-20201229060814-148bc717ce4c @@ -73,7 +73,7 @@ require ( golang.org/x/net v0.0.0-20201021035429-f5854403a974 golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9 golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4 - golang.org/x/text v0.3.4 + golang.org/x/text v0.3.5 golang.org/x/tools v0.1.0 google.golang.org/grpc v1.27.1 gopkg.in/check.v1 v1.0.0-20200902074654-038fdea0a05b // indirect @@ -85,5 +85,3 @@ require ( ) go 1.13 - -replace github.com/pingcap/parser => github.com/ekexium/parser v0.0.0-20210203081914-18a31cd39b53 diff --git a/go.sum b/go.sum index 88cc7eae3ee62..4cfcd00c6c0ef 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-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= github.com/elazarl/go-bindata-assetfs v1.0.0/go.mod h1:v+YaWX3bdea5J/mo8dSETolEo7R71Vk1u8bnjau5yw4= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= @@ -653,6 +651,10 @@ github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8 h1:M+DNpOu/I3uDmwee6vcnoPd6GgSMqND4gxvDQ/W584U= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= +github.com/pingcap/parser v0.0.0-20190506092653-e336082eb825/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20200422082501-7329d80eaf2c/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/pingcap/parser v0.0.0-20210203141130-32ef3e0e2e13 h1:hIbRR2siRrJniSRrMU+XgZl3reOWiPZk0P+7PwKXuEQ= +github.com/pingcap/parser v0.0.0-20210203141130-32ef3e0e2e13/go.mod h1:GbEr2PgY72/4XqPZzmzstlOU/+il/wrjeTNFs6ihsSE= github.com/pingcap/pd v2.1.5+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= @@ -1082,8 +1084,8 @@ golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.3.4 h1:0YWbFKbhXG/wIiuHDSKpS0Iy7FSA+u45VtBMfQcFTTc= -golang.org/x/text v0.3.4/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.5 h1:i6eZZ+zk0SOf0xgBpEpPD18qWcJda6q1sxt3S0kzyUQ= +golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=