-
Notifications
You must be signed in to change notification settings - Fork 5.8k
/
session.go
2338 lines (1996 loc) · 83.7 KB
/
session.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2015 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package variable
import (
"bytes"
"crypto/tls"
"encoding/binary"
"fmt"
"math"
"math/rand"
"net"
"sort"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/pingcap/errors"
pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/charset"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/rowcodec"
"github.com/pingcap/tidb/util/stringutil"
"github.com/pingcap/tidb/util/tableutil"
"github.com/pingcap/tidb/util/timeutil"
tikvstore "github.com/tikv/client-go/v2/kv"
"github.com/tikv/client-go/v2/oracle"
"github.com/twmb/murmur3"
atomic2 "go.uber.org/atomic"
)
// PreparedStmtCount is exported for test.
var PreparedStmtCount int64
// RetryInfo saves retry information.
type RetryInfo struct {
Retrying bool
DroppedPreparedStmtIDs []uint32
autoIncrementIDs retryInfoAutoIDs
autoRandomIDs retryInfoAutoIDs
}
// Clean does some clean work.
func (r *RetryInfo) Clean() {
r.autoIncrementIDs.clean()
r.autoRandomIDs.clean()
if len(r.DroppedPreparedStmtIDs) > 0 {
r.DroppedPreparedStmtIDs = r.DroppedPreparedStmtIDs[:0]
}
}
// ResetOffset resets the current retry offset.
func (r *RetryInfo) ResetOffset() {
r.autoIncrementIDs.resetOffset()
r.autoRandomIDs.resetOffset()
}
// AddAutoIncrementID adds id to autoIncrementIDs.
func (r *RetryInfo) AddAutoIncrementID(id int64) {
r.autoIncrementIDs.autoIDs = append(r.autoIncrementIDs.autoIDs, id)
}
// GetCurrAutoIncrementID gets current autoIncrementID.
func (r *RetryInfo) GetCurrAutoIncrementID() (int64, bool) {
return r.autoIncrementIDs.getCurrent()
}
// AddAutoRandomID adds id to autoRandomIDs.
func (r *RetryInfo) AddAutoRandomID(id int64) {
r.autoRandomIDs.autoIDs = append(r.autoRandomIDs.autoIDs, id)
}
// GetCurrAutoRandomID gets current AutoRandomID.
func (r *RetryInfo) GetCurrAutoRandomID() (int64, bool) {
return r.autoRandomIDs.getCurrent()
}
type retryInfoAutoIDs struct {
currentOffset int
autoIDs []int64
}
func (r *retryInfoAutoIDs) resetOffset() {
r.currentOffset = 0
}
func (r *retryInfoAutoIDs) clean() {
r.currentOffset = 0
if len(r.autoIDs) > 0 {
r.autoIDs = r.autoIDs[:0]
}
}
func (r *retryInfoAutoIDs) getCurrent() (int64, bool) {
if r.currentOffset >= len(r.autoIDs) {
return 0, false
}
id := r.autoIDs[r.currentOffset]
r.currentOffset++
return id, true
}
// TransactionContext is used to store variables that has transaction scope.
type TransactionContext struct {
forUpdateTS uint64
stmtFuture oracle.Future
Binlog interface{}
InfoSchema interface{}
History interface{}
StartTS uint64
// ShardStep indicates the max size of continuous rowid shard in one transaction.
ShardStep int
shardRemain int
currentShard int64
shardRand *rand.Rand
// TableDeltaMap is used in the schema validator for DDL changes in one table not to block others.
// It's also used in the statistics updating.
// Note: for the partitioned table, it stores all the partition IDs.
TableDeltaMap map[int64]TableDelta
// unchangedRowKeys is used to store the unchanged rows that needs to lock for pessimistic transaction.
unchangedRowKeys map[string]struct{}
// pessimisticLockCache is the cache for pessimistic locked keys,
// The value never changes during the transaction.
pessimisticLockCache map[string][]byte
PessimisticCacheHit int
// CreateTime For metrics.
CreateTime time.Time
StatementCount int
CouldRetry bool
IsPessimistic bool
// IsStaleness indicates whether the txn is read only staleness txn.
IsStaleness bool
// IsExplicit indicates whether the txn is an interactive txn, which is typically started with a BEGIN
// or START TRANSACTION statement, or by setting autocommit to 0.
IsExplicit bool
Isolation string
LockExpire uint32
ForUpdate uint32
// TxnScope indicates the value of txn_scope
TxnScope string
// TableDeltaMap lock to prevent potential data race
tdmLock sync.Mutex
// TemporaryTables is used to store transaction-specific information for global temporary tables.
// It can also be stored in sessionCtx with local temporary tables, but it's easier to clean this data after transaction ends.
TemporaryTables map[int64]tableutil.TempTable
}
// GetShard returns the shard prefix for the next `count` rowids.
func (tc *TransactionContext) GetShard(shardRowIDBits uint64, typeBitsLength uint64, reserveSignBit bool, count int) int64 {
if shardRowIDBits == 0 {
return 0
}
if tc.shardRand == nil {
tc.shardRand = rand.New(rand.NewSource(int64(tc.StartTS))) // #nosec G404
}
if tc.shardRemain <= 0 {
tc.updateShard()
tc.shardRemain = tc.ShardStep
}
tc.shardRemain -= count
var signBitLength uint64
if reserveSignBit {
signBitLength = 1
}
return (tc.currentShard & (1<<shardRowIDBits - 1)) << (typeBitsLength - shardRowIDBits - signBitLength)
}
func (tc *TransactionContext) updateShard() {
var buf [8]byte
binary.LittleEndian.PutUint64(buf[:], tc.shardRand.Uint64())
tc.currentShard = int64(murmur3.Sum32(buf[:]))
}
// AddUnchangedRowKey adds an unchanged row key in update statement for pessimistic lock.
func (tc *TransactionContext) AddUnchangedRowKey(key []byte) {
if tc.unchangedRowKeys == nil {
tc.unchangedRowKeys = map[string]struct{}{}
}
tc.unchangedRowKeys[string(key)] = struct{}{}
}
// CollectUnchangedRowKeys collects unchanged row keys for pessimistic lock.
func (tc *TransactionContext) CollectUnchangedRowKeys(buf []kv.Key) []kv.Key {
for key := range tc.unchangedRowKeys {
buf = append(buf, kv.Key(key))
}
tc.unchangedRowKeys = nil
return buf
}
// UpdateDeltaForTable updates the delta info for some table.
func (tc *TransactionContext) UpdateDeltaForTable(physicalTableID int64, delta int64, count int64, colSize map[int64]int64) {
tc.tdmLock.Lock()
defer tc.tdmLock.Unlock()
if tc.TableDeltaMap == nil {
tc.TableDeltaMap = make(map[int64]TableDelta)
}
item := tc.TableDeltaMap[physicalTableID]
if item.ColSize == nil && colSize != nil {
item.ColSize = make(map[int64]int64, len(colSize))
}
item.Delta += delta
item.Count += count
item.TableID = physicalTableID
for key, val := range colSize {
item.ColSize[key] += val
}
tc.TableDeltaMap[physicalTableID] = item
}
// GetKeyInPessimisticLockCache gets a key in pessimistic lock cache.
func (tc *TransactionContext) GetKeyInPessimisticLockCache(key kv.Key) (val []byte, ok bool) {
if tc.pessimisticLockCache == nil {
return nil, false
}
val, ok = tc.pessimisticLockCache[string(key)]
if ok {
tc.PessimisticCacheHit++
}
return
}
// SetPessimisticLockCache sets a key value pair into pessimistic lock cache.
func (tc *TransactionContext) SetPessimisticLockCache(key kv.Key, val []byte) {
if tc.pessimisticLockCache == nil {
tc.pessimisticLockCache = map[string][]byte{}
}
tc.pessimisticLockCache[string(key)] = val
}
// Cleanup clears up transaction info that no longer use.
func (tc *TransactionContext) Cleanup() {
// tc.InfoSchema = nil; we cannot do it now, because some operation like handleFieldList depend on this.
tc.Binlog = nil
tc.History = nil
tc.tdmLock.Lock()
tc.TableDeltaMap = nil
tc.tdmLock.Unlock()
tc.pessimisticLockCache = nil
tc.IsStaleness = false
}
// ClearDelta clears the delta map.
func (tc *TransactionContext) ClearDelta() {
tc.tdmLock.Lock()
tc.TableDeltaMap = nil
tc.tdmLock.Unlock()
}
// GetForUpdateTS returns the ts for update.
func (tc *TransactionContext) GetForUpdateTS() uint64 {
if tc.forUpdateTS > tc.StartTS {
return tc.forUpdateTS
}
return tc.StartTS
}
// SetForUpdateTS sets the ts for update.
func (tc *TransactionContext) SetForUpdateTS(forUpdateTS uint64) {
if forUpdateTS > tc.forUpdateTS {
tc.forUpdateTS = forUpdateTS
}
}
// SetStmtFutureForRC sets the stmtFuture .
func (tc *TransactionContext) SetStmtFutureForRC(future oracle.Future) {
tc.stmtFuture = future
}
// GetStmtFutureForRC gets the stmtFuture.
func (tc *TransactionContext) GetStmtFutureForRC() oracle.Future {
return tc.stmtFuture
}
// WriteStmtBufs can be used by insert/replace/delete/update statement.
// TODO: use a common memory pool to replace this.
type WriteStmtBufs struct {
// RowValBuf is used by tablecodec.EncodeRow, to reduce runtime.growslice.
RowValBuf []byte
// AddRowValues use to store temp insert rows value, to reduce memory allocations when importing data.
AddRowValues []types.Datum
// IndexValsBuf is used by index.FetchValues
IndexValsBuf []types.Datum
// IndexKeyBuf is used by index.GenIndexKey
IndexKeyBuf []byte
}
func (ib *WriteStmtBufs) clean() {
ib.RowValBuf = nil
ib.AddRowValues = nil
ib.IndexValsBuf = nil
ib.IndexKeyBuf = nil
}
// TableSnapshot represents a data snapshot of the table contained in `information_schema`.
type TableSnapshot struct {
Rows [][]types.Datum
Err error
}
type txnIsolationLevelOneShotState uint
// RewritePhaseInfo records some information about the rewrite phase
type RewritePhaseInfo struct {
// DurationRewrite is the duration of rewriting the SQL.
DurationRewrite time.Duration
// DurationPreprocessSubQuery is the duration of pre-processing sub-queries.
DurationPreprocessSubQuery time.Duration
// PreprocessSubQueries is the number of pre-processed sub-queries.
PreprocessSubQueries int
}
// Reset resets all fields in RewritePhaseInfo.
func (r *RewritePhaseInfo) Reset() {
r.DurationRewrite = 0
r.DurationPreprocessSubQuery = 0
r.PreprocessSubQueries = 0
}
// TemporaryTableData is a interface to maintain temporary data in session
type TemporaryTableData interface {
kv.Retriever
// Staging create a new staging buffer inside the MemBuffer.
// Subsequent writes will be temporarily stored in this new staging buffer.
// When you think all modifications looks good, you can call `Release` to public all of them to the upper level buffer.
Staging() kv.StagingHandle
// Release publish all modifications in the latest staging buffer to upper level.
Release(kv.StagingHandle)
// Cleanup cleanups the resources referenced by the StagingHandle.
// If the changes are not published by `Release`, they will be discarded.
Cleanup(kv.StagingHandle)
// GetTableSize get the size of a table
GetTableSize(tblID int64) int64
// DeleteTableKey removes the entry for key k from table
DeleteTableKey(tblID int64, k kv.Key) error
// SetTableKey sets the entry for k from table
SetTableKey(tblID int64, k kv.Key, val []byte) error
}
// temporaryTableData is used for store temporary table data in session
type temporaryTableData struct {
kv.MemBuffer
tblSize map[int64]int64
}
// NewTemporaryTableData creates a new TemporaryTableData
func NewTemporaryTableData(memBuffer kv.MemBuffer) TemporaryTableData {
return &temporaryTableData{
MemBuffer: memBuffer,
tblSize: make(map[int64]int64),
}
}
// GetTableSize get the size of a table
func (d *temporaryTableData) GetTableSize(tblID int64) int64 {
if tblSize, ok := d.tblSize[tblID]; ok {
return tblSize
}
return 0
}
// DeleteTableKey removes the entry for key k from table
func (d *temporaryTableData) DeleteTableKey(tblID int64, k kv.Key) error {
bufferSize := d.MemBuffer.Size()
defer d.updateTblSize(tblID, bufferSize)
return d.MemBuffer.Delete(k)
}
// SetTableKey sets the entry for k from table
func (d *temporaryTableData) SetTableKey(tblID int64, k kv.Key, val []byte) error {
bufferSize := d.MemBuffer.Size()
defer d.updateTblSize(tblID, bufferSize)
return d.MemBuffer.Set(k, val)
}
func (d *temporaryTableData) updateTblSize(tblID int64, beforeSize int) {
delta := int64(d.MemBuffer.Size() - beforeSize)
d.tblSize[tblID] = d.GetTableSize(tblID) + delta
}
const (
// oneShotDef means default, that is tx_isolation_one_shot not set.
oneShotDef txnIsolationLevelOneShotState = iota
// oneShotSet means it's set in current transaction.
oneShotSet
// onsShotUse means it should be used in current transaction.
oneShotUse
)
// SessionVars is to handle user-defined or global variables in the current session.
type SessionVars struct {
Concurrency
MemQuota
BatchSize
// DMLBatchSize indicates the number of rows batch-committed for a statement.
// It will be used when using LOAD DATA or BatchInsert or BatchDelete is on.
DMLBatchSize int
RetryLimit int64
DisableTxnAutoRetry bool
// UsersLock is a lock for user defined variables.
UsersLock sync.RWMutex
// Users are user defined variables.
Users map[string]types.Datum
// UserVarTypes stores the FieldType for user variables, it cannot be inferred from Users when Users have not been set yet.
// It is read/write protected by UsersLock.
UserVarTypes map[string]*types.FieldType
// systems variables, don't modify it directly, use GetSystemVar/SetSystemVar method.
systems map[string]string
// stmtVars variables are temporarily set by SET_VAR hint
// It only take effect for the duration of a single statement
stmtVars map[string]string
// SysWarningCount is the system variable "warning_count", because it is on the hot path, so we extract it from the systems
SysWarningCount int
// SysErrorCount is the system variable "error_count", because it is on the hot path, so we extract it from the systems
SysErrorCount uint16
// PreparedStmts stores prepared statement.
PreparedStmts map[uint32]interface{}
PreparedStmtNameToID map[string]uint32
// preparedStmtID is id of prepared statement.
preparedStmtID uint32
// PreparedParams params for prepared statements
PreparedParams PreparedParams
// ActiveRoles stores active roles for current user
ActiveRoles []*auth.RoleIdentity
RetryInfo *RetryInfo
// TxnCtx Should be reset on transaction finished.
TxnCtx *TransactionContext
// KVVars is the variables for KV storage.
KVVars *tikvstore.Variables
// txnIsolationLevelOneShot is used to implements "set transaction isolation level ..."
txnIsolationLevelOneShot struct {
state txnIsolationLevelOneShotState
value string
}
// mppTaskIDAllocator is used to allocate mpp task id for a session.
mppTaskIDAllocator struct {
mu sync.Mutex
lastTS uint64
taskID int64
}
// Status stands for the session status. e.g. in transaction or not, auto commit is on or off, and so on.
Status uint16
// ClientCapability is client's capability.
ClientCapability uint32
// TLSConnectionState is the TLS connection state (nil if not using TLS).
TLSConnectionState *tls.ConnectionState
// ConnectionID is the connection id of the current session.
ConnectionID uint64
// PlanID is the unique id of logical and physical plan.
PlanID int
// PlanColumnID is the unique id for column when building plan.
PlanColumnID int64
// User is the user identity with which the session login.
User *auth.UserIdentity
// Port is the port of the connected socket
Port string
// CurrentDB is the default database of this session.
CurrentDB string
// CurrentDBChanged indicates if the CurrentDB has been updated, and if it is we should print it into
// the slow log to make it be compatible with MySQL, https://github.com/pingcap/tidb/issues/17846.
CurrentDBChanged bool
// StrictSQLMode indicates if the session is in strict mode.
StrictSQLMode bool
// CommonGlobalLoaded indicates if common global variable has been loaded for this session.
CommonGlobalLoaded bool
// InRestrictedSQL indicates if the session is handling restricted SQL execution.
InRestrictedSQL bool
// SnapshotTS is used for reading history data. For simplicity, SnapshotTS only supports distsql request.
SnapshotTS uint64
// TxnReadTS is used for staleness transaction, it provides next staleness transaction startTS.
TxnReadTS *TxnReadTS
// SnapshotInfoschema is used with SnapshotTS, when the schema version at snapshotTS less than current schema
// version, we load an old version schema for query.
SnapshotInfoschema interface{}
// BinlogClient is used to write binlog.
BinlogClient *pumpcli.PumpsClient
// GlobalVarsAccessor is used to set and get global variables.
GlobalVarsAccessor GlobalVarAccessor
// LastFoundRows is the number of found rows of last query statement
LastFoundRows uint64
// StmtCtx holds variables for current executing statement.
StmtCtx *stmtctx.StatementContext
// AllowAggPushDown can be set to false to forbid aggregation push down.
AllowAggPushDown bool
// AllowBCJ means allow broadcast join.
AllowBCJ bool
// AllowCartesianBCJ means allow broadcast CARTESIAN join, 0 means not allow, 1 means allow broadcast CARTESIAN join
// but the table size should under the broadcast threshold, 2 means allow broadcast CARTESIAN join even if the table
// size exceeds the broadcast threshold
AllowCartesianBCJ int
// MPPOuterJoinFixedBuildSide means in MPP plan, always use right(left) table as build side for left(right) out join
MPPOuterJoinFixedBuildSide bool
// AllowDistinctAggPushDown can be set true to allow agg with distinct push down to tikv/tiflash.
AllowDistinctAggPushDown bool
// MultiStatementMode permits incorrect client library usage. Not recommended to be turned on.
MultiStatementMode int
// AllowWriteRowID variable is currently not recommended to be turned on.
AllowWriteRowID bool
// AllowBatchCop means if we should send batch coprocessor to TiFlash. Default value is 1, means to use batch cop in case of aggregation and join.
// Value set to 2 means to force to send batch cop for any query. Value set to 0 means never use batch cop.
AllowBatchCop int
// allowMPPExecution means if we should use mpp way to execute query.
// Default value is `true`, means to be determined by the optimizer.
// Value set to `false` means never use mpp.
allowMPPExecution bool
// HashExchangeWithNewCollation means if we support hash exchange when new collation is enabled.
// Default value is `true`, means support hash exchange when new collation is enabled.
// Value set to `false` means not use hash exchange when new collation is enabled.
HashExchangeWithNewCollation bool
// enforceMPPExecution means if we should enforce mpp way to execute query.
// Default value is `false`, means to be determined by variable `allowMPPExecution`.
// Value set to `true` means enforce use mpp.
// Note if you want to set `enforceMPPExecution` to `true`, you must set `allowMPPExecution` to `true` first.
enforceMPPExecution bool
// TiDBAllowAutoRandExplicitInsert indicates whether explicit insertion on auto_random column is allowed.
AllowAutoRandExplicitInsert bool
// BroadcastJoinThresholdSize is used to limit the size of smaller table.
// It's unit is bytes, if the size of small table is larger than it, we will not use bcj.
BroadcastJoinThresholdSize int64
// BroadcastJoinThresholdCount is used to limit the total count of smaller table.
// If we can't estimate the size of one side of join child, we will check if its row number exceeds this limitation.
BroadcastJoinThresholdCount int64
// LimitPushDownThreshold determines if push Limit or TopN down to TiKV forcibly.
LimitPushDownThreshold int64
// CorrelationThreshold is the guard to enable row count estimation using column order correlation.
CorrelationThreshold float64
// EnableCorrelationAdjustment is used to indicate if correlation adjustment is enabled.
EnableCorrelationAdjustment bool
// CorrelationExpFactor is used to control the heuristic approach of row count estimation when CorrelationThreshold is not met.
CorrelationExpFactor int
// CPUFactor is the CPU cost of processing one expression for one row.
CPUFactor float64
// CopCPUFactor is the CPU cost of processing one expression for one row in coprocessor.
CopCPUFactor float64
// CopTiFlashConcurrencyFactor is the concurrency number of computation in tiflash coprocessor.
CopTiFlashConcurrencyFactor float64
// networkFactor is the network cost of transferring 1 byte data.
networkFactor float64
// ScanFactor is the IO cost of scanning 1 byte data on TiKV and TiFlash.
scanFactor float64
// descScanFactor is the IO cost of scanning 1 byte data on TiKV and TiFlash in desc order.
descScanFactor float64
// seekFactor is the IO cost of seeking the start value of a range in TiKV or TiFlash.
seekFactor float64
// MemoryFactor is the memory cost of storing one tuple.
MemoryFactor float64
// DiskFactor is the IO cost of reading/writing one byte to temporary disk.
DiskFactor float64
// ConcurrencyFactor is the CPU cost of additional one goroutine.
ConcurrencyFactor float64
// CurrInsertValues is used to record current ValuesExpr's values.
// See http://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values
CurrInsertValues chunk.Row
// In https://github.com/pingcap/tidb/issues/14164, we can see that MySQL can enter the column that is not in the insert's SELECT's output.
// We store the extra columns in this variable.
CurrInsertBatchExtraCols [][]types.Datum
// Per-connection time zones. Each client that connects has its own time zone setting, given by the session time_zone variable.
// See https://dev.mysql.com/doc/refman/5.7/en/time-zone-support.html
TimeZone *time.Location
SQLMode mysql.SQLMode
// AutoIncrementIncrement and AutoIncrementOffset indicates the autoID's start value and increment.
AutoIncrementIncrement int
AutoIncrementOffset int
/* TiDB system variables */
// SkipASCIICheck check on input value.
SkipASCIICheck bool
// SkipUTF8Check check on input value.
SkipUTF8Check bool
// BatchInsert indicates if we should split insert data into multiple batches.
BatchInsert bool
// BatchDelete indicates if we should split delete data into multiple batches.
BatchDelete bool
// BatchCommit indicates if we should split the transaction into multiple batches.
BatchCommit bool
// IDAllocator is provided by kvEncoder, if it is provided, we will use it to alloc auto id instead of using
// Table.alloc.
IDAllocator autoid.Allocator
// OptimizerSelectivityLevel defines the level of the selectivity estimation in plan.
OptimizerSelectivityLevel int
// EnableTablePartition enables table partition feature.
EnableTablePartition string
// EnableListTablePartition enables list table partition feature.
EnableListTablePartition bool
// EnableCascadesPlanner enables the cascades planner.
EnableCascadesPlanner bool
// EnableWindowFunction enables the window function.
EnableWindowFunction bool
// EnablePipelinedWindowExec enables executing window functions in a pipelined manner.
EnablePipelinedWindowExec bool
// EnableStrictDoubleTypeCheck enables table field double type check.
EnableStrictDoubleTypeCheck bool
// EnableVectorizedExpression enables the vectorized expression evaluation.
EnableVectorizedExpression bool
// DDLReorgPriority is the operation priority of adding indices.
DDLReorgPriority int
// EnableChangeMultiSchema is used to control whether to enable the multi schema change.
EnableChangeMultiSchema bool
// EnableAutoIncrementInGenerated is used to control whether to allow auto incremented columns in generated columns.
EnableAutoIncrementInGenerated bool
// EnablePointGetCache is used to cache value for point get for read only scenario.
EnablePointGetCache bool
// EnableAlterPlacement indicates whether a user can alter table partition placement rules.
EnableAlterPlacement bool
// EnablePlacementChecks indicates whether a user can check validation of placement.
EnablePlacementChecks bool
// WaitSplitRegionFinish defines the split region behaviour is sync or async.
WaitSplitRegionFinish bool
// WaitSplitRegionTimeout defines the split region timeout.
WaitSplitRegionTimeout uint64
// EnableStreaming indicates whether the coprocessor request can use streaming API.
// TODO: remove this after tidb-server configuration "enable-streaming' removed.
EnableStreaming bool
// EnableChunkRPC indicates whether the coprocessor request can use chunk API.
EnableChunkRPC bool
writeStmtBufs WriteStmtBufs
// ConstraintCheckInPlace indicates whether to check the constraint when the SQL executing.
ConstraintCheckInPlace bool
// CommandValue indicates which command current session is doing.
CommandValue uint32
// TiDBOptJoinReorderThreshold defines the minimal number of join nodes
// to use the greedy join reorder algorithm.
TiDBOptJoinReorderThreshold int
// SlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse.
SlowQueryFile string
// EnableFastAnalyze indicates whether to take fast analyze.
EnableFastAnalyze bool
// TxnMode indicates should be pessimistic or optimistic.
TxnMode string
// LowResolutionTSO is used for reading data with low resolution TSO which is updated once every two seconds.
LowResolutionTSO bool
// MaxExecutionTime is the timeout for select statement, in milliseconds.
// If the value is 0, timeouts are not enabled.
// See https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_max_execution_time
MaxExecutionTime uint64
// Killed is a flag to indicate that this query is killed.
Killed uint32
// ConnectionInfo indicates current connection info used by current session, only be lazy assigned by plugin.
ConnectionInfo *ConnectionInfo
// NoopFuncsMode allows OFF/ON/WARN values as 0/1/2.
NoopFuncsMode int
// StartTime is the start time of the last query.
StartTime time.Time
// DurationParse is the duration of parsing SQL string to AST of the last query.
DurationParse time.Duration
// DurationCompile is the duration of compiling AST to execution plan of the last query.
DurationCompile time.Duration
// RewritePhaseInfo records all information about the rewriting phase.
RewritePhaseInfo
// DurationOptimization is the duration of optimizing a query.
DurationOptimization time.Duration
// DurationWaitTS is the duration of waiting for a snapshot TS
DurationWaitTS time.Duration
// PrevStmt is used to store the previous executed statement in the current session.
PrevStmt fmt.Stringer
// prevStmtDigest is used to store the digest of the previous statement in the current session.
prevStmtDigest string
// AllowRemoveAutoInc indicates whether a user can drop the auto_increment column attribute or not.
AllowRemoveAutoInc bool
// UsePlanBaselines indicates whether we will use plan baselines to adjust plan.
UsePlanBaselines bool
// EvolvePlanBaselines indicates whether we will evolve the plan baselines.
EvolvePlanBaselines bool
// EnableExtendedStats indicates whether we enable the extended statistics feature.
EnableExtendedStats bool
// Unexported fields should be accessed and set through interfaces like GetReplicaRead() and SetReplicaRead().
// allowInSubqToJoinAndAgg can be set to false to forbid rewriting the semi join to inner join with agg.
allowInSubqToJoinAndAgg bool
// preferRangeScan allows optimizer to always prefer range scan over table scan.
preferRangeScan bool
// EnableIndexMerge enables the generation of IndexMergePath.
enableIndexMerge bool
// replicaRead is used for reading data from replicas, only follower is supported at this time.
replicaRead kv.ReplicaReadType
// IsolationReadEngines is used to isolation read, tidb only read from the stores whose engine type is in the engines.
IsolationReadEngines map[kv.StoreType]struct{}
PlannerSelectBlockAsName []ast.HintTable
// LockWaitTimeout is the duration waiting for pessimistic lock in milliseconds
LockWaitTimeout int64
// MetricSchemaStep indicates the step when query metric schema.
MetricSchemaStep int64
// MetricSchemaRangeDuration indicates the step when query metric schema.
MetricSchemaRangeDuration int64
// Some data of cluster-level memory tables will be retrieved many times in different inspection rules,
// and the cost of retrieving some data is expensive. We use the `TableSnapshot` to cache those data
// and obtain them lazily, and provide a consistent view of inspection tables for each inspection rules.
// All cached snapshots will be released at the end of retrieving
InspectionTableCache map[string]TableSnapshot
// RowEncoder is reused in session for encode row data.
RowEncoder rowcodec.Encoder
// SequenceState cache all sequence's latest value accessed by lastval() builtins. It's a session scoped
// variable, and all public methods of SequenceState are currently-safe.
SequenceState *SequenceState
// WindowingUseHighPrecision determines whether to compute window operations without loss of precision.
// see https://dev.mysql.com/doc/refman/8.0/en/window-function-optimization.html for more details.
WindowingUseHighPrecision bool
// FoundInPlanCache indicates whether this statement was found in plan cache.
FoundInPlanCache bool
// PrevFoundInPlanCache indicates whether the last statement was found in plan cache.
PrevFoundInPlanCache bool
// FoundInBinding indicates whether the execution plan is matched with the hints in the binding.
FoundInBinding bool
// PrevFoundInBinding indicates whether the last execution plan is matched with the hints in the binding.
PrevFoundInBinding bool
// OptimizerUseInvisibleIndexes indicates whether optimizer can use invisible index
OptimizerUseInvisibleIndexes bool
// SelectLimit limits the max counts of select statement's output
SelectLimit uint64
// EnableClusteredIndex indicates whether to enable clustered index when creating a new table.
EnableClusteredIndex ClusteredIndexDefMode
// PresumeKeyNotExists indicates lazy existence checking is enabled.
PresumeKeyNotExists bool
// EnableParallelApply indicates that thether to use parallel apply.
EnableParallelApply bool
// EnableRedactLog indicates that whether redact log.
EnableRedactLog bool
// ShardAllocateStep indicates the max size of continuous rowid shard in one transaction.
ShardAllocateStep int64
// EnableAmendPessimisticTxn indicates if schema change amend is enabled for pessimistic transactions.
EnableAmendPessimisticTxn bool
// LastTxnInfo keeps track the info of last committed transaction.
LastTxnInfo string
// LastQueryInfo keeps track the info of last query.
LastQueryInfo QueryInfo
// PartitionPruneMode indicates how and when to prune partitions.
PartitionPruneMode atomic2.String
// TxnScope indicates the scope of the transactions. It should be `global` or equal to the value of key `zone` in config.Labels.
TxnScope kv.TxnScopeVar
// EnabledRateLimitAction indicates whether enabled ratelimit action during coprocessor
EnabledRateLimitAction bool
// EnableAsyncCommit indicates whether to enable the async commit feature.
EnableAsyncCommit bool
// 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.
AnalyzeVersion int
// EnableIndexMergeJoin indicates whether to enable index merge join.
EnableIndexMergeJoin bool
// TrackAggregateMemoryUsage indicates whether to track the memory usage of aggregate function.
TrackAggregateMemoryUsage bool
// TiDBEnableExchangePartition indicates whether to enable exchange partition
TiDBEnableExchangePartition bool
// AllowFallbackToTiKV indicates the engine types whose unavailability triggers fallback to TiKV.
// Now we only support TiFlash.
AllowFallbackToTiKV map[kv.StoreType]struct{}
// CTEMaxRecursionDepth indicates The common table expression (CTE) maximum recursion depth.
// see https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_cte_max_recursion_depth
CTEMaxRecursionDepth int
// The temporary table size threshold, which is different from MySQL. See https://github.com/pingcap/tidb/issues/28691.
TMPTableSize int64
// EnableStableResultMode if stabilize query results.
EnableStableResultMode bool
// EnablePseudoForOutdatedStats if using pseudo for outdated stats
EnablePseudoForOutdatedStats bool
// LocalTemporaryTables is *infoschema.LocalTemporaryTables, use interface to avoid circle dependency.
// It's nil if there is no local temporary table.
LocalTemporaryTables interface{}
// TemporaryTableData stores committed kv values for temporary table for current session.
TemporaryTableData TemporaryTableData
// MPPStoreLastFailTime records the lastest fail time that a TiFlash store failed.
MPPStoreLastFailTime map[string]time.Time
// MPPStoreFailTTL indicates the duration that protect TiDB from sending task to a new recovered TiFlash.
MPPStoreFailTTL string
// ReadStaleness indicates the staleness duration for the following query
ReadStaleness time.Duration
// cached is used to optimze the object allocation.
cached struct {
curr int8
data [2]stmtctx.StatementContext
}
}
// InitStatementContext initializes a StatementContext, the object is reused to reduce allocation.
func (s *SessionVars) InitStatementContext() *stmtctx.StatementContext {
s.cached.curr = (s.cached.curr + 1) % 2
s.cached.data[s.cached.curr] = stmtctx.StatementContext{}
return &s.cached.data[s.cached.curr]
}
// AllocMPPTaskID allocates task id for mpp tasks. It will reset the task id if the query's
// startTs is different.
func (s *SessionVars) AllocMPPTaskID(startTS uint64) int64 {
s.mppTaskIDAllocator.mu.Lock()
defer s.mppTaskIDAllocator.mu.Unlock()
if s.mppTaskIDAllocator.lastTS == startTS {
s.mppTaskIDAllocator.taskID++
return s.mppTaskIDAllocator.taskID
}
s.mppTaskIDAllocator.lastTS = startTS
s.mppTaskIDAllocator.taskID = 1
return 1
}
// IsMPPAllowed returns whether mpp execution is allowed.
func (s *SessionVars) IsMPPAllowed() bool {
return s.allowMPPExecution
}
// IsMPPEnforced returns whether mpp execution is enforced.
func (s *SessionVars) IsMPPEnforced() bool {
return s.allowMPPExecution && s.enforceMPPExecution
}
// RaiseWarningWhenMPPEnforced will raise a warning when mpp mode is enforced and executing explain statement.
// TODO: Confirm whether this function will be inlined and
// omit the overhead of string construction when calling with false condition.
func (s *SessionVars) RaiseWarningWhenMPPEnforced(warning string) {
if s.IsMPPEnforced() && s.StmtCtx.InExplainStmt {
s.StmtCtx.AppendWarning(errors.New(warning))
}
}
// CheckAndGetTxnScope will return the transaction scope we should use in the current session.