From 17a30f28745ef3acde7c802dc6ce09edf2cb517c Mon Sep 17 00:00:00 2001 From: gengliqi Date: Wed, 31 Aug 2022 16:49:01 +0800 Subject: [PATCH 1/6] add TiFlashReadForWriteStmt Signed-off-by: gengliqi --- planner/optimize.go | 4 +--- sessionctx/variable/session.go | 3 +++ sessionctx/variable/sysvar.go | 4 ++++ sessionctx/variable/tidb_vars.go | 4 ++++ 4 files changed, 12 insertions(+), 3 deletions(-) diff --git a/planner/optimize.go b/planner/optimize.go index e859d78fd5b62..87aafab6c53e5 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -88,9 +88,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in } } - // Because for write stmt, TiFlash has a different results when lock the data in point get plan. We ban the TiFlash - // engine in not read only stmt. - if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(node, sessVars) { + if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !sessVars.TiFlashReadForWriteStmt && !IsReadOnly(node, sessVars) { delete(sessVars.IsolationReadEngines, kv.TiFlash) defer func() { sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 0dc6eae68fdb8..5ce81ec2708fc 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1230,6 +1230,9 @@ type SessionVars struct { // GeneralPlanCacheSize controls the size of general plan cache. GeneralPlanCacheSize uint64 + + // TiFlashReadForWriteStmt indicates whether to enable write stmt read on TiFlash. + TiFlashReadForWriteStmt bool } // GetPreparedStmtByName returns the prepared statement specified by stmtName. diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 7972f9d24a008..fcd933692bbcf 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1783,6 +1783,10 @@ var defaultSysVars = []*SysVar{ DDLDiskQuota.Store(TidbOptInt64(val, DefTiDBDDLDiskQuota)) return nil }}, + {Scope: ScopeSession, Name: TiFlashReadForWriteStmt, Value: BoolToOnOff(DefTiFlashReadForWriteStmt), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.TiFlashReadForWriteStmt = TiDBOptOn(val) + return nil + }}, } // FeedbackProbability points to the FeedbackProbability in statistics package. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 782e3a06161c7..fa3c2b528e673 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -234,6 +234,9 @@ const ( // TiFlashFastScan indicates whether use fast scan in tiflash . TiFlashFastScan = "tiflash_fastscan" + + // TiFlashReadForWriteStmt indicates whether to enable write stmt read on TiFlash. + TiFlashReadForWriteStmt = "tiflash_read_for_write_stmt" ) // TiDB system variable names that both in session and global scope. @@ -1020,6 +1023,7 @@ const ( DefExecutorConcurrency = 5 DefTiDBEnableGeneralPlanCache = false DefTiDBGeneralPlanCacheSize = 100 + DefTiFlashReadForWriteStmt = false // MaxDDLReorgBatchSize is exported for testing. MaxDDLReorgBatchSize int32 = 10240 MinDDLReorgBatchSize int32 = 32 From c7750c0aea80f0b1a6ca081138773c03549e18a5 Mon Sep 17 00:00:00 2001 From: gengliqi Date: Wed, 31 Aug 2022 17:38:58 +0800 Subject: [PATCH 2/6] add default value Signed-off-by: gengliqi --- sessionctx/variable/session.go | 3 ++- sessionctx/variable/tidb_vars.go | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 5ce81ec2708fc..3bb7fc8adeb84 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1231,7 +1231,7 @@ type SessionVars struct { // GeneralPlanCacheSize controls the size of general plan cache. GeneralPlanCacheSize uint64 - // TiFlashReadForWriteStmt indicates whether to enable write stmt read on TiFlash. + // TiFlashReadForWriteStmt indicates whether to enable write stmts to read on TiFlash. TiFlashReadForWriteStmt bool } @@ -1519,6 +1519,7 @@ func NewSessionVars() *SessionVars { EnableSkewDistinctAgg: DefTiDBSkewDistinctAgg, MaxAllowedPacket: DefMaxAllowedPacket, TiFlashFastScan: DefTiFlashFastScan, + TiFlashReadForWriteStmt: DefTiFlashReadForWriteStmt, } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index fa3c2b528e673..0f815322484a4 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -235,7 +235,7 @@ const ( // TiFlashFastScan indicates whether use fast scan in tiflash . TiFlashFastScan = "tiflash_fastscan" - // TiFlashReadForWriteStmt indicates whether to enable write stmt read on TiFlash. + // TiFlashReadForWriteStmt indicates whether to enable write stmts to read on TiFlash. TiFlashReadForWriteStmt = "tiflash_read_for_write_stmt" ) From 3269718c391d55305bf1d203edd056cd944ae05b Mon Sep 17 00:00:00 2001 From: gengliqi Date: Wed, 31 Aug 2022 18:59:16 +0800 Subject: [PATCH 3/6] add TestInsertSelectPushDownToTiFlash Signed-off-by: gengliqi --- planner/core/integration_test.go | 49 ++++++++++++++++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index eade71c9bd3a4..099a4e6b0fa9e 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -7130,3 +7130,52 @@ func TestCastTimeAsDurationToTiFlash(t *testing.T) { } tk.MustQuery("explain select cast(a as time), cast(b as time) from t;").CheckAt([]int{0, 2, 4}, rows) } + +func TestInsertSelectPushDownToTiFlash(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into t values(1, 2)") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2(a int)") + tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@tiflash_read_for_write_stmt = ON") + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + tbl2, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t2", L: "t2"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl2.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + rows := [][]interface{}{ + {"Insert_1", "root", "N/A"}, + {"└─TableReader_11", "root", "data:ExchangeSender_10"}, + {" └─ExchangeSender_10", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_6", "mpp[tiflash]", "plus(test.t.a, test.t.b)->Column#5"}, + {" └─TableFullScan_9", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain insert into t2 select a+b from t;").CheckAt([]int{0, 2, 4}, rows) + + rows = [][]interface{}{ + {"Insert_1", "root", "N/A"}, + {"└─TableReader_30", "root", "data:ExchangeSender_29"}, + {" └─ExchangeSender_29", "mpp[tiflash]", "ExchangeType: PassThrough"}, + {" └─Projection_10", "mpp[tiflash]", "test.t.a"}, + {" └─HashJoin_28", "mpp[tiflash]", "inner join, equal:[eq(test.t2.a, test.t.a)]"}, + {" ├─ExchangeReceiver_17(Build)", "mpp[tiflash]", ""}, + {" │ └─ExchangeSender_16", "mpp[tiflash]", "ExchangeType: Broadcast"}, + {" │ └─Selection_15", "mpp[tiflash]", "not(isnull(test.t2.a))"}, + {" │ └─TableFullScan_14", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + {" └─Selection_19(Probe)", "mpp[tiflash]", "not(isnull(test.t.a))"}, + {" └─TableFullScan_18", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain insert into t2 select t.a from t2 join t on t2.a = t.a;").CheckAt([]int{0, 2, 4}, rows) +} From 9be31ce02d3bb9cbcfa793c0dd191b7e687cee3a Mon Sep 17 00:00:00 2001 From: gengliqi Date: Fri, 2 Sep 2022 12:57:46 +0800 Subject: [PATCH 4/6] change name to tidb_enable_tiflash_read_for_write_stmt Signed-off-by: gengliqi --- planner/core/integration_test.go | 48 ++++---- planner/optimize.go | 2 +- sessionctx/variable/session.go | 182 +++++++++++++++---------------- sessionctx/variable/sysvar.go | 4 +- sessionctx/variable/tidb_vars.go | 8 +- 5 files changed, 123 insertions(+), 121 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 099a4e6b0fa9e..8087bde65650a 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -7131,7 +7131,7 @@ func TestCastTimeAsDurationToTiFlash(t *testing.T) { tk.MustQuery("explain select cast(a as time), cast(b as time) from t;").CheckAt([]int{0, 2, 4}, rows) } -func TestInsertSelectPushDownToTiFlash(t *testing.T) { +func TestEnableTiFlashReadForWriteStmt(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -7143,7 +7143,7 @@ func TestInsertSelectPushDownToTiFlash(t *testing.T) { tk.MustExec("create table t2(a int)") tk.MustExec("set @@tidb_allow_mpp=1; set @@tidb_enforce_mpp=1") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@tiflash_read_for_write_stmt = ON") + tk.MustExec("set @@tidb_enable_tiflash_read_for_write_stmt = ON") tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) require.NoError(t, err) @@ -7155,27 +7155,29 @@ func TestInsertSelectPushDownToTiFlash(t *testing.T) { // Set the hacked TiFlash replica for explain tests. tbl2.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - rows := [][]interface{}{ - {"Insert_1", "root", "N/A"}, - {"└─TableReader_11", "root", "data:ExchangeSender_10"}, - {" └─ExchangeSender_10", "mpp[tiflash]", "ExchangeType: PassThrough"}, - {" └─Projection_6", "mpp[tiflash]", "plus(test.t.a, test.t.b)->Column#5"}, - {" └─TableFullScan_9", "mpp[tiflash]", "keep order:false, stats:pseudo"}, + check_mpp := func(r [][]interface{}) { + contain_mpp := false + for i := range r { + if r[i][2] == "mpp[tiflash]" { + contain_mpp = true + break + } + } + require.Equal(t, contain_mpp, true) } - tk.MustQuery("explain insert into t2 select a+b from t;").CheckAt([]int{0, 2, 4}, rows) - rows = [][]interface{}{ - {"Insert_1", "root", "N/A"}, - {"└─TableReader_30", "root", "data:ExchangeSender_29"}, - {" └─ExchangeSender_29", "mpp[tiflash]", "ExchangeType: PassThrough"}, - {" └─Projection_10", "mpp[tiflash]", "test.t.a"}, - {" └─HashJoin_28", "mpp[tiflash]", "inner join, equal:[eq(test.t2.a, test.t.a)]"}, - {" ├─ExchangeReceiver_17(Build)", "mpp[tiflash]", ""}, - {" │ └─ExchangeSender_16", "mpp[tiflash]", "ExchangeType: Broadcast"}, - {" │ └─Selection_15", "mpp[tiflash]", "not(isnull(test.t2.a))"}, - {" │ └─TableFullScan_14", "mpp[tiflash]", "keep order:false, stats:pseudo"}, - {" └─Selection_19(Probe)", "mpp[tiflash]", "not(isnull(test.t.a))"}, - {" └─TableFullScan_18", "mpp[tiflash]", "keep order:false, stats:pseudo"}, - } - tk.MustQuery("explain insert into t2 select t.a from t2 join t on t2.a = t.a;").CheckAt([]int{0, 2, 4}, rows) + // Insert into ... select + rs := tk.MustQuery("explain insert into t2 select a+b from t").Rows() + check_mpp(rs) + + rs = tk.MustQuery("explain insert into t2 select t.a from t2 join t on t2.a = t.a").Rows() + check_mpp(rs) + + // Replace into ... select + rs = tk.MustQuery("explain replace into t2 select a+b from t").Rows() + check_mpp(rs) + + // CTE + rs = tk.MustQuery("explain update t set a=a+1 where b in (select a from t2 where t.a > t2.a)").Rows() + check_mpp(rs) } diff --git a/planner/optimize.go b/planner/optimize.go index 87aafab6c53e5..b3a0ef16b039e 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -88,7 +88,7 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in } } - if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !sessVars.TiFlashReadForWriteStmt && !IsReadOnly(node, sessVars) { + if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !sessVars.EnableTiFlashReadForWriteStmt && !IsReadOnly(node, sessVars) { delete(sessVars.IsolationReadEngines, kv.TiFlash) defer func() { sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 3bb7fc8adeb84..fe17a3e5cc354 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1231,8 +1231,8 @@ type SessionVars struct { // GeneralPlanCacheSize controls the size of general plan cache. GeneralPlanCacheSize uint64 - // TiFlashReadForWriteStmt indicates whether to enable write stmts to read on TiFlash. - TiFlashReadForWriteStmt bool + // EnableTiFlashReadForWriteStmt indicates whether to enable TiFlash to read for write statements. + EnableTiFlashReadForWriteStmt bool } // GetPreparedStmtByName returns the prepared statement specified by stmtName. @@ -1431,95 +1431,95 @@ func NewSessionVars() *SessionVars { values: make(map[string]types.Datum), types: 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, - AllowCartesianBCJ: DefOptCartesianBCJ, - MPPOuterJoinFixedBuildSide: DefOptMPPOuterJoinFixedBuildSide, - BroadcastJoinThresholdSize: DefBroadcastJoinThresholdSize, - BroadcastJoinThresholdCount: DefBroadcastJoinThresholdSize, - OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, - EnableOuterJoinReorder: DefTiDBEnableOuterJoinReorder, - RetryLimit: DefTiDBRetryLimit, - DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, - DDLReorgPriority: kv.PriorityLow, - allowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, - preferRangeScan: DefOptPreferRangeScan, - EnableCorrelationAdjustment: DefOptEnableCorrelationAdjustment, - LimitPushDownThreshold: DefOptLimitPushDownThreshold, - CorrelationThreshold: DefOptCorrelationThreshold, - CorrelationExpFactor: DefOptCorrelationExpFactor, - cpuFactor: DefOptCPUFactor, - copCPUFactor: DefOptCopCPUFactor, - CopTiFlashConcurrencyFactor: DefOptTiFlashConcurrencyFactor, - networkFactor: DefOptNetworkFactor, - scanFactor: DefOptScanFactor, - descScanFactor: DefOptDescScanFactor, - seekFactor: DefOptSeekFactor, - memoryFactor: DefOptMemoryFactor, - diskFactor: DefOptDiskFactor, - concurrencyFactor: DefOptConcurrencyFactor, - EnableVectorizedExpression: DefEnableVectorizedExpression, - CommandValue: uint32(mysql.ComSleep), - TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, - SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, - WaitSplitRegionFinish: DefTiDBWaitSplitRegionFinish, - WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, - enableIndexMerge: DefTiDBEnableIndexMerge, - NoopFuncsMode: TiDBOptOnOffWarn(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, - EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn, - PartitionPruneMode: *atomic2.NewString(DefTiDBPartitionPruneMode), - TxnScope: kv.NewDefaultTxnScopeVar(), - EnabledRateLimitAction: DefTiDBEnableRateLimitAction, - EnableAsyncCommit: DefTiDBEnableAsyncCommit, - Enable1PC: DefTiDBEnable1PC, - GuaranteeLinearizability: DefTiDBGuaranteeLinearizability, - AnalyzeVersion: DefTiDBAnalyzeVersion, - EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, - AllowFallbackToTiKV: make(map[kv.StoreType]struct{}), - CTEMaxRecursionDepth: DefCTEMaxRecursionDepth, - TMPTableSize: DefTiDBTmpTableMaxSize, - MPPStoreLastFailTime: make(map[string]time.Time), - MPPStoreFailTTL: DefTiDBMPPStoreFailTTL, - Rng: mathutil.NewWithTime(), - StatsLoadSyncWait: StatsLoadSyncWait.Load(), - EnableLegacyInstanceScope: DefEnableLegacyInstanceScope, - RemoveOrderbyInSubquery: DefTiDBRemoveOrderbyInSubquery, - EnableSkewDistinctAgg: DefTiDBSkewDistinctAgg, - MaxAllowedPacket: DefMaxAllowedPacket, - TiFlashFastScan: DefTiFlashFastScan, - TiFlashReadForWriteStmt: DefTiFlashReadForWriteStmt, + 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, + AllowCartesianBCJ: DefOptCartesianBCJ, + MPPOuterJoinFixedBuildSide: DefOptMPPOuterJoinFixedBuildSide, + BroadcastJoinThresholdSize: DefBroadcastJoinThresholdSize, + BroadcastJoinThresholdCount: DefBroadcastJoinThresholdSize, + OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, + EnableOuterJoinReorder: DefTiDBEnableOuterJoinReorder, + RetryLimit: DefTiDBRetryLimit, + DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, + DDLReorgPriority: kv.PriorityLow, + allowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, + preferRangeScan: DefOptPreferRangeScan, + EnableCorrelationAdjustment: DefOptEnableCorrelationAdjustment, + LimitPushDownThreshold: DefOptLimitPushDownThreshold, + CorrelationThreshold: DefOptCorrelationThreshold, + CorrelationExpFactor: DefOptCorrelationExpFactor, + cpuFactor: DefOptCPUFactor, + copCPUFactor: DefOptCopCPUFactor, + CopTiFlashConcurrencyFactor: DefOptTiFlashConcurrencyFactor, + networkFactor: DefOptNetworkFactor, + scanFactor: DefOptScanFactor, + descScanFactor: DefOptDescScanFactor, + seekFactor: DefOptSeekFactor, + memoryFactor: DefOptMemoryFactor, + diskFactor: DefOptDiskFactor, + concurrencyFactor: DefOptConcurrencyFactor, + EnableVectorizedExpression: DefEnableVectorizedExpression, + CommandValue: uint32(mysql.ComSleep), + TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, + SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, + WaitSplitRegionFinish: DefTiDBWaitSplitRegionFinish, + WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, + enableIndexMerge: DefTiDBEnableIndexMerge, + NoopFuncsMode: TiDBOptOnOffWarn(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, + EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn, + PartitionPruneMode: *atomic2.NewString(DefTiDBPartitionPruneMode), + TxnScope: kv.NewDefaultTxnScopeVar(), + EnabledRateLimitAction: DefTiDBEnableRateLimitAction, + EnableAsyncCommit: DefTiDBEnableAsyncCommit, + Enable1PC: DefTiDBEnable1PC, + GuaranteeLinearizability: DefTiDBGuaranteeLinearizability, + AnalyzeVersion: DefTiDBAnalyzeVersion, + EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, + AllowFallbackToTiKV: make(map[kv.StoreType]struct{}), + CTEMaxRecursionDepth: DefCTEMaxRecursionDepth, + TMPTableSize: DefTiDBTmpTableMaxSize, + MPPStoreLastFailTime: make(map[string]time.Time), + MPPStoreFailTTL: DefTiDBMPPStoreFailTTL, + Rng: mathutil.NewWithTime(), + StatsLoadSyncWait: StatsLoadSyncWait.Load(), + EnableLegacyInstanceScope: DefEnableLegacyInstanceScope, + RemoveOrderbyInSubquery: DefTiDBRemoveOrderbyInSubquery, + EnableSkewDistinctAgg: DefTiDBSkewDistinctAgg, + MaxAllowedPacket: DefMaxAllowedPacket, + TiFlashFastScan: DefTiFlashFastScan, + EnableTiFlashReadForWriteStmt: DefTiDBEnableTiFlashReadForWriteStmt, } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index fcd933692bbcf..d14c5523f0195 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1783,8 +1783,8 @@ var defaultSysVars = []*SysVar{ DDLDiskQuota.Store(TidbOptInt64(val, DefTiDBDDLDiskQuota)) return nil }}, - {Scope: ScopeSession, Name: TiFlashReadForWriteStmt, Value: BoolToOnOff(DefTiFlashReadForWriteStmt), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { - s.TiFlashReadForWriteStmt = TiDBOptOn(val) + {Scope: ScopeSession, Name: TiDBEnableTiFlashReadForWriteStmt, Value: BoolToOnOff(DefTiDBEnableTiFlashReadForWriteStmt), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.EnableTiFlashReadForWriteStmt = TiDBOptOn(val) return nil }}, } diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 1bb472e2dbfc8..4003b4959db53 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -232,11 +232,11 @@ const ( // RequireSecureTransport indicates the secure mode for data transport RequireSecureTransport = "require_secure_transport" - // TiFlashFastScan indicates whether use fast scan in tiflash . + // TiFlashFastScan indicates whether use fast scan in tiflash. TiFlashFastScan = "tiflash_fastscan" - // TiFlashReadForWriteStmt indicates whether to enable write stmts to read on TiFlash. - TiFlashReadForWriteStmt = "tiflash_read_for_write_stmt" + // TiDBEnableTiFlashReadForWriteStmt indicates whether to enable TiFlash to read for write statements. + TiDBEnableTiFlashReadForWriteStmt = "tidb_enable_tiflash_read_for_write_stmt" ) // TiDB system variable names that both in session and global scope. @@ -1023,7 +1023,7 @@ const ( DefExecutorConcurrency = 5 DefTiDBEnableGeneralPlanCache = false DefTiDBGeneralPlanCacheSize = 100 - DefTiFlashReadForWriteStmt = false + DefTiDBEnableTiFlashReadForWriteStmt = false // MaxDDLReorgBatchSize is exported for testing. MaxDDLReorgBatchSize int32 = 10240 MinDDLReorgBatchSize int32 = 32 From 75f8840d8c1455668db01d35e0128318bd5374d4 Mon Sep 17 00:00:00 2001 From: gengliqi Date: Fri, 2 Sep 2022 13:09:08 +0800 Subject: [PATCH 5/6] fix lint Signed-off-by: gengliqi --- planner/core/integration_test.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 8087bde65650a..075972aa23a01 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -7155,29 +7155,29 @@ func TestEnableTiFlashReadForWriteStmt(t *testing.T) { // Set the hacked TiFlash replica for explain tests. tbl2.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} - check_mpp := func(r [][]interface{}) { - contain_mpp := false + checkMpp := func(r [][]interface{}) { + check := false for i := range r { if r[i][2] == "mpp[tiflash]" { - contain_mpp = true + check = true break } } - require.Equal(t, contain_mpp, true) + require.Equal(t, check, true) } // Insert into ... select rs := tk.MustQuery("explain insert into t2 select a+b from t").Rows() - check_mpp(rs) + checkMpp(rs) rs = tk.MustQuery("explain insert into t2 select t.a from t2 join t on t2.a = t.a").Rows() - check_mpp(rs) + checkMpp(rs) // Replace into ... select rs = tk.MustQuery("explain replace into t2 select a+b from t").Rows() - check_mpp(rs) + checkMpp(rs) // CTE rs = tk.MustQuery("explain update t set a=a+1 where b in (select a from t2 where t.a > t2.a)").Rows() - check_mpp(rs) + checkMpp(rs) } From f4d81206105ca98d2a1bc27a57a79ce784a63c68 Mon Sep 17 00:00:00 2001 From: gengliqi Date: Fri, 2 Sep 2022 15:55:57 +0800 Subject: [PATCH 6/6] add global scope Signed-off-by: gengliqi --- sessionctx/variable/sysvar.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index bb7fb26185e55..192b81b0b442d 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1791,7 +1791,7 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeSession, Name: TiDBEnableTiFlashReadForWriteStmt, Value: BoolToOnOff(DefTiDBEnableTiFlashReadForWriteStmt), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTiFlashReadForWriteStmt, Value: BoolToOnOff(DefTiDBEnableTiFlashReadForWriteStmt), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableTiFlashReadForWriteStmt = TiDBOptOn(val) return nil }},