pingcap / tidb

TiDB - the open-source, cloud-native, distributed SQL database designed for modern applications.
https://pingcap.com
Apache License 2.0
37.21k stars 5.84k forks source link

Audit API General events for Starting/Error have incorrect context #36109

Open morgo opened 2 years ago

morgo commented 2 years ago

Bug Report

Please answer these questions before submitting your issue. Thanks!

1. Minimal reproduce step (Required)

Write a simple plugin based on https://github.com/pingcap/tidb/tree/master/plugin/conn_ip_example

For the OnGeneralEvent use the following code:

func OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugin.GeneralEvent, cmd string) {
    fmt.Printf("#### event: %s sctx: %#v\n", event.String(), sctx)
}

Compile the server, and run two statements:

mysql> show tables;
Empty set (0.00 sec)

mysql> select 1;
+---+
| 1 |
+---+
| 1 |
+---+
1 row in set (0.00 sec)

2. What did you expect to see? (Required)

For the second statement, there should be a Starting and Completed event. The starting event should have the SQL of select 1.

3. What did you see instead (Required)

The Starting event incorrectly has all the context of the previously executed statement (show tables). There is no context about the statement which is select 1. This is the same for the event Error, where if I were to create an invalid statement, all the context refers to the previously executed statement.

Here is a printf:

#### event: STARTING sctx: &variable.SessionVars{Concurrency:variable.Concurrency{indexLookupConcurrency:-1, indexLookupJoinConcurrency:-1, distSQLScanConcurrency:15, hashJoinConcurrency:-1, projectionConcurrency:-1, hashAggPartialConcurrency:-1, hashAggFinalConcurrency:-1, windowConcurrency:-1, mergeJoinConcurrency:1, streamAggConcurrency:1, indexSerialScanConcurrency:1, ExecutorConcurrency:5, SourceAddr:net.TCPAddr{IP:net.IP(nil), Port:0, Zone:""}}, MemQuota:variable.MemQuota{MemQuotaQuery:1073741824, MemQuotaApplyCache:33554432}, BatchSize:variable.BatchSize{IndexJoinBatchSize:25000, IndexLookupSize:20000, InitChunkSize:32, MaxChunkSize:1024}, DMLBatchSize:0, RetryLimit:10, DisableTxnAutoRetry:true, UsersLock:sync.RWMutex{w:sync.Mutex{state:0, sema:0x0}, writerSem:0x0, readerSem:0x0, readerCount:0, readerWait:0}, Users:map[string]types.Datum{}, UserVarTypes:map[string]*types.FieldType{}, systems:map[string]string{"allow_auto_random_explicit_insert":"OFF", "auto_increment_increment":"1", "auto_increment_offset":"1", "autocommit":"ON", "block_encryption_mode":"aes-128-ecb", "character_set_client":"utf8mb4", "character_set_connection":"utf8mb4", "character_set_database":"utf8mb4", "character_set_results":"utf8mb4", "character_set_server":"utf8mb4", "collation_connection":"utf8mb4_bin", "collation_database":"utf8mb4_bin", "collation_server":"utf8mb4_bin", "cte_max_recursion_depth":"1000", "default_week_format":"0", "innodb_lock_wait_timeout":"50", "interactive_timeout":"28800", "max_allowed_packet":"67108864", "max_execution_time":"0", "sql_mode":"ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", "sql_select_limit":"18446744073709551615", "tidb_allow_batch_cop":"1", "tidb_allow_fallback_to_tikv":"", "tidb_allow_mpp":"ON", "tidb_analyze_version":"2", "tidb_backoff_lock_fast":"10", "tidb_backoff_weight":"2", "tidb_batch_pending_tiflash_count":"4000", "tidb_broadcast_join_threshold_count":"10240", "tidb_broadcast_join_threshold_size":"104857600", "tidb_constraint_check_in_place":"OFF", "tidb_cost_model_version":"1", "tidb_disable_txn_auto_retry":"ON", "tidb_distsql_scan_concurrency":"15", "tidb_dml_batch_size":"0", "tidb_enable_1pc":"OFF", "tidb_enable_amend_pessimistic_txn":"OFF", "tidb_enable_async_commit":"OFF", "tidb_enable_auto_increment_in_generated":"OFF", "tidb_enable_cascades_planner":"OFF", "tidb_enable_chunk_rpc":"1", "tidb_enable_clustered_index":"INT_ONLY", "tidb_enable_exchange_partition":"OFF", "tidb_enable_extended_stats":"OFF", "tidb_enable_fast_analyze":"OFF", "tidb_enable_index_merge":"ON", "tidb_enable_index_merge_join":"OFF", "tidb_enable_legacy_instance_scope":"ON", "tidb_enable_list_partition":"ON", "tidb_enable_mutation_checker":"ON", "tidb_enable_new_cost_interface":"OFF", "tidb_enable_new_only_full_group_by_check":"OFF", "tidb_enable_noop_functions":"OFF", "tidb_enable_ordered_result_mode":"OFF", "tidb_enable_paging":"ON", "tidb_enable_parallel_apply":"OFF", "tidb_enable_pipelined_window_function":"ON", "tidb_enable_pseudo_for_outdated_stats":"ON", "tidb_enable_rate_limit_action":"ON", "tidb_enable_strict_double_type_check":"ON", "tidb_enable_table_partition":"ON", "tidb_enable_vectorized_expression":"ON", "tidb_enable_window_function":"ON", "tidb_enforce_mpp":"OFF", "tidb_evolve_plan_baselines":"OFF", "tidb_executor_concurrency":"5", "tidb_guarantee_linearizability":"ON", "tidb_hash_exchange_with_new_collation":"ON", "tidb_hash_join_concurrency":"-1", "tidb_hashagg_final_concurrency":"-1", "tidb_hashagg_partial_concurrency":"-1", "tidb_ignore_prepared_cache_close_stmt":"OFF", "tidb_index_join_batch_size":"25000", "tidb_index_lookup_concurrency":"-1", "tidb_index_lookup_join_concurrency":"-1", "tidb_index_lookup_size":"20000", "tidb_index_serial_scan_concurrency":"1", "tidb_init_chunk_size":"32", "tidb_isolation_read_engines":"tikv,tiflash,tidb", "tidb_max_chunk_size":"1024", "tidb_max_tiflash_threads":"-1", "tidb_mem_quota_apply_cache":"33554432", "tidb_mem_quota_query":"1073741824", "tidb_merge_join_concurrency":"1", "tidb_mpp_store_fail_ttl":"60s", "tidb_multi_statement_mode":"OFF", "tidb_nontransactional_ignore_error":"OFF", "tidb_opt_agg_push_down":"OFF", "tidb_opt_broadcast_cartesian_join":"1", "tidb_opt_concurrency_factor":"3", "tidb_opt_concurrency_factor_v2":"3", "tidb_opt_copcpu_factor":"3", "tidb_opt_copcpu_factor_v2":"30", "tidb_opt_correlation_exp_factor":"1", "tidb_opt_correlation_threshold":"0.9", "tidb_opt_cpu_factor":"3", "tidb_opt_cpu_factor_v2":"30", "tidb_opt_desc_factor":"3", "tidb_opt_desc_factor_v2":"150", "tidb_opt_disk_factor":"1.5", "tidb_opt_disk_factor_v2":"1.5", "tidb_opt_enable_correlation_adjustment":"ON", "tidb_opt_insubq_to_join_and_agg":"ON", "tidb_opt_limit_push_down_threshold":"100", "tidb_opt_memory_factor":"0.001", "tidb_opt_memory_factor_v2":"0.001", "tidb_opt_mpp_outer_join_fixed_build_side":"OFF", "tidb_opt_network_factor":"1", "tidb_opt_network_factor_v2":"4", "tidb_opt_prefer_range_scan":"OFF", "tidb_opt_projection_push_down":"OFF", "tidb_opt_scan_factor":"1.5", "tidb_opt_scan_factor_v2":"100", "tidb_opt_seek_factor_v2":"9500000", "tidb_opt_tiflash_cpu_factor_v2":"2", "tidb_opt_tiflash_scan_factor_v2":"15", "tidb_partition_prune_mode":"static", "tidb_placement_mode":"STRICT", "tidb_projection_concurrency":"-1", "tidb_rc_read_check_ts":"OFF", "tidb_read_consistency":"strict", "tidb_read_staleness":"0", "tidb_redact_log":"OFF", "tidb_regard_null_as_point":"ON", "tidb_remove_orderby_in_subquery":"OFF", "tidb_replica_read":"leader", "tidb_retry_limit":"10", "tidb_row_format_version":"2", "tidb_shard_allocate_step":"9223372036854775807", "tidb_skip_ascii_check":"OFF", "tidb_skip_isolation_level_check":"OFF", "tidb_skip_utf8_check":"OFF", "tidb_streamagg_concurrency":"1", "tidb_sysdate_is_now":"OFF", "tidb_tmp_table_max_size":"67108864", "tidb_track_aggregate_memory_usage":"ON", "tidb_txn_assertion_level":"FAST", "tidb_txn_mode":"", "tidb_use_plan_baselines":"ON", "tidb_window_concurrency":"-1", "tiflash_fine_grained_shuffle_batch_size":"8192", "tiflash_fine_grained_shuffle_stream_count":"-1", "time_zone":"SYSTEM", "transaction_isolation":"REPEATABLE-READ", "tx_isolation":"REPEATABLE-READ", "tx_read_ts":"", "wait_timeout":"28800", "windowing_use_high_precision":"ON"}, stmtVars:map[string]string{}, SysWarningCount:0, SysErrorCount:0x0, PreparedStmts:map[uint32]interface {}{}, PreparedStmtNameToID:map[string]uint32{}, preparedStmtID:0x0, PreparedParams:variable.PreparedParams{}, LastUpdateTime4PC:types.Time{coreTime:0x0}, ActiveRoles:[]*auth.RoleIdentity{}, RetryInfo:(*variable.RetryInfo)(0x140121c4b60), TxnCtx:(*variable.TransactionContext)(0x14011bd9300), TxnManager:(*session.txnManager)(0x140121a9920), KVVars:(*kv.Variables)(0x14011f68d68), txnIsolationLevelOneShot:struct { state variable.txnIsolationLevelOneShotState; value string }{state:0x0, value:""}, mppTaskIDAllocator:struct { mu sync.Mutex; lastTS uint64; taskID int64 }{mu:sync.Mutex{state:0, sema:0x0}, lastTS:0x0, taskID:0}, Status:0x2, ClientCapability:0x1ba685, TLSConnectionState:(*tls.ConnectionState)(nil), ConnectionID:0x20000000193, PlanID:2, PlanColumnID:1, MapHashCode2UniqueID4ExtendedCol:map[string]int(nil), User:(*auth.UserIdentity)(0x1401219db30), Port:"49641", CurrentDB:"test", CurrentDBChanged:true, StrictSQLMode:true, CommonGlobalLoaded:true, InRestrictedSQL:false, SnapshotTS:0x0, TxnReadTS:(*variable.TxnReadTS)(0x14012216800), SnapshotInfoschema:interface {}(nil), BinlogClient:(*client.PumpsClient)(nil), GlobalVarsAccessor:(*session.session)(0x140122126c0), LastFoundRows:0x0, StmtCtx:(*stmtctx.StatementContext)(0x14012284d38), AllowAggPushDown:false, AllowCartesianBCJ:1, MPPOuterJoinFixedBuildSide:false, AllowDistinctAggPushDown:false, MultiStatementMode:0, AllowWriteRowID:false, AllowBatchCop:1, allowMPPExecution:true, HashExchangeWithNewCollation:true, enforceMPPExecution:false, TiFlashMaxThreads:-1, AllowAutoRandExplicitInsert:false, BroadcastJoinThresholdSize:104857600, BroadcastJoinThresholdCount:10240, LimitPushDownThreshold:100, CorrelationThreshold:0.9, EnableCorrelationAdjustment:true, CorrelationExpFactor:1, cpuFactor:3, copCPUFactor:3, networkFactor:1, scanFactor:1.5, descScanFactor:3, seekFactor:20, memoryFactor:0.001, diskFactor:1.5, concurrencyFactor:3, cpuFactorV2:30, copCPUFactorV2:30, tiflashCPUFactorV2:2, networkFactorV2:4, scanFactorV2:100, descScanFactorV2:150, tiflashScanFactorV2:15, seekFactorV2:9.5e+06, memoryFactorV2:0.001, diskFactorV2:1.5, concurrencyFactorV2:3, CopTiFlashConcurrencyFactor:24, CurrInsertValues:chunk.Row{c:(*chunk.Chunk)(nil), idx:0}, CurrInsertBatchExtraCols:[][]types.Datum(nil), TimeZone:(*time.Location)(0x14011552fc0), SQLMode:1436549152, AutoIncrementIncrement:1, AutoIncrementOffset:1, SkipASCIICheck:false, SkipUTF8Check:false, BatchInsert:false, BatchDelete:false, BatchCommit:false, IDAllocator:autoid.Allocator(nil), OptimizerSelectivityLevel:0, OptimizerEnableNewOnlyFullGroupByCheck:false, EnableOuterJoinReorder:true, EnableTablePartition:"ON", EnableListTablePartition:true, EnableCascadesPlanner:false, EnableWindowFunction:true, EnablePipelinedWindowExec:true, AllowProjectionPushDown:false, EnableStrictDoubleTypeCheck:true, EnableVectorizedExpression:true, DDLReorgPriority:1, EnableAutoIncrementInGenerated:false, EnablePointGetCache:false, PlacementMode:"STRICT", WaitSplitRegionFinish:true, WaitSplitRegionTimeout:0x12c, EnableChunkRPC:true, writeStmtBufs:variable.WriteStmtBufs{RowValBuf:[]uint8(nil), AddRowValues:[]types.Datum(nil), IndexValsBuf:[]types.Datum(nil), IndexKeyBuf:[]uint8(nil)}, ConstraintCheckInPlace:false, CommandValue:0x3, TiDBOptJoinReorderThreshold:0, SlowQueryFile:"tidb-slow.log", EnableFastAnalyze:false, TxnMode:"", LowResolutionTSO:false, MaxExecutionTime:0x0, Killed:0x0, ConnectionInfo:(*variable.ConnectionInfo)(0x1401214e540), NoopFuncsMode:0, StartTime:time.Date(2022, time.July, 11, 11, 21, 2, 7004000, time.Local), DurationParse:292834, DurationCompile:63417, RewritePhaseInfo:variable.RewritePhaseInfo{DurationRewrite:18833, DurationPreprocessSubQuery:0, PreprocessSubQueries:0}, DurationOptimization:13208, DurationWaitTS:0, PrevStmt:(stringutil.StringerFunc)(0x101c99af0), prevStmtDigest:"be021a0b3956334563aa6949abdacb23a012818b1d159432132f69c9e5a5f914", AllowRemoveAutoInc:false, UsePlanBaselines:true, EvolvePlanBaselines:false, EnableExtendedStats:false, allowInSubqToJoinAndAgg:true, preferRangeScan:false, enableIndexMerge:true, replicaRead:0x0, IsolationReadEngines:map[kv.StoreType]struct {}{0x0:struct {}{}, 0x1:struct {}{}, 0x2:struct {}{}}, PlannerSelectBlockAsName:[]ast.HintTable{ast.HintTable{DBName:model.CIStr{O:"", L:""}, TableName:model.CIStr{O:"", L:""}, QBName:model.CIStr{O:"", L:""}, PartitionList:[]model.CIStr(nil)}}, LockWaitTimeout:50000, MetricSchemaStep:60, MetricSchemaRangeDuration:60, InspectionTableCache:map[string]variable.TableSnapshot(nil), RowEncoder:rowcodec.Encoder{row:rowcodec.row{large:false, numNotNullCols:0x0, numNullCols:0x0, colIDs:[]uint8(nil), offsets:[]uint16(nil), data:[]uint8(nil), colIDs32:[]uint32(nil), offsets32:[]uint32(nil)}, tempColIDs:[]int64(nil), values:[]*types.Datum(nil), Enable:true}, SequenceState:(*variable.SequenceState)(0x140121c65d0), WindowingUseHighPrecision:true, FoundInPlanCache:false, PrevFoundInPlanCache:false, FoundInBinding:false, PrevFoundInBinding:false, OptimizerUseInvisibleIndexes:false, SelectLimit:0xffffffffffffffff, EnableClusteredIndex:0, PresumeKeyNotExists:false, EnableParallelApply:false, EnableRedactLog:false, ShardAllocateStep:9223372036854775807, EnableAmendPessimisticTxn:false, LastTxnInfo:"", LastQueryInfo:sessionstates.QueryInfo{TxnScope:"global", StartTS:0x0, ForUpdateTS:0x0, ErrMsg:""}, LastDDLInfo:sessionstates.LastDDLInfo{Query:"", SeqNum:0x0}, PartitionPruneMode:atomic.String{_:atomic.nocmp{}, v:atomic.Value{Value:atomic.Value{v:"static"}, _:atomic.nocmp{}}}, TxnScope:kv.TxnScopeVar{varValue:"global", txnScope:"global"}, EnabledRateLimitAction:true, EnableAsyncCommit:false, Enable1PC:false, GuaranteeLinearizability:true, AnalyzeVersion:2, EnableIndexMergeJoin:false, TrackAggregateMemoryUsage:true, TiDBEnableExchangePartition:false, AllowFallbackToTiKV:map[kv.StoreType]struct {}{}, CTEMaxRecursionDepth:1000, TMPTableSize:67108864, EnableStableResultMode:false, EnablePseudoForOutdatedStats:true, RegardNULLAsPoint:true, LocalTemporaryTables:(*infoschema.LocalTemporaryTables)(0x140121c6700), TemporaryTableData:variable.TemporaryTableData(nil), MPPStoreLastFailTime:map[string]time.Time{}, MPPStoreFailTTL:"60s", ReadStaleness:0, cached:struct { curr int8; data [2]stmtctx.StatementContext }{curr:1, data:[2]stmtctx.StatementContext{stmtctx.StatementContext{StmtHints:stmtctx.StmtHints{MemQuotaQuery:0, ApplyCacheCapacity:0, MaxExecutionTime:0x0, ReplicaRead:0x0, AllowInSubqToJoinAndAgg:false, NoIndexMergeHint:false, StraightJoinOrder:false, EnableCascadesPlanner:false, ForceNthPlan:0, HasAllowInSubqToJoinAndAggHint:false, HasMemQuotaHint:false, HasReplicaReadHint:false, HasMaxExecutionTime:false, HasEnableCascadesPlannerHint:false, SetVars:map[string]string(nil), OriginalTableHints:[]*ast.TableOptimizerHint(nil)}, IsDDLJobInQueue:false, DDLJobID:0, InInsertStmt:false, InUpdateStmt:false, InDeleteStmt:false, InSelectStmt:true, InLoadDataStmt:false, InExplainStmt:false, InCreateOrAlterStmt:false, InSetSessionStatesStmt:false, InPreparedPlanBuilding:false, IgnoreTruncate:false, IgnoreZeroInDate:true, NoZeroDate:false, DupKeyAsWarning:false, BadNullAsWarning:false, DividedByZeroAsWarning:false, TruncateAsWarning:true, OverflowAsWarning:true, InShowWarning:false, UseCache:false, BatchCheck:false, InNullRejectCheck:false, AllowInvalidDate:false, IgnoreNoPartition:false, SkipPlanCache:false, IgnoreExplainIDSuffix:false, SkipUTF8Check:false, SkipASCIICheck:false, SkipUTF8MB4Check:false, MultiSchemaInfo:(*model.MultiSchemaInfo)(nil), IsStaleness:false, InRestrictedSQL:false, mu:struct { sync.Mutex; affectedRows uint64; foundRows uint64; records uint64; deleted uint64; updated uint64; copied uint64; touched uint64; message string; warnings []stmtctx.SQLWarn; errorCount uint16; execDetails execdetails.ExecDetails; allExecDetails []*execdetails.ExecDetails }{Mutex:sync.Mutex{state:0, sema:0x0}, affectedRows:0x0, foundRows:0x1, records:0x0, deleted:0x0, updated:0x0, copied:0x0, touched:0x0, message:"", warnings:[]stmtctx.SQLWarn(nil), errorCount:0x0, execDetails:execdetails.ExecDetails{CalleeAddress:"", CopTime:0, BackoffTime:0, LockKeysDuration:0, BackoffSleep:map[string]time.Duration(nil), BackoffTimes:map[string]int(nil), RequestCount:0, CommitDetail:(*util.CommitDetails)(nil), LockKeysDetail:(*util.LockKeysDetails)(nil), ScanDetail:(*util.ScanDetail)(nil), TimeDetail:util.TimeDetail{ProcessTime:0, WaitTime:0, KvReadWallTimeMs:0}}, allExecDetails:[]*execdetails.ExecDetails(nil)}, PrevAffectedRows:0, PrevLastInsertID:0x0, LastInsertID:0x0, InsertID:0x0, BaseRowID:0, MaxRowID:0, TimeZone:(*time.Location)(0x14011552fc0), Priority:0, NotFillCache:false, MemTracker:(*memory.Tracker)(0x14012284b18), DiskTracker:(*memory.Tracker)(0x14012284b98), IsTiFlash:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}, RuntimeStatsColl:(*execdetails.RuntimeStatsColl)(0x14011f68f30), TableIDs:[]int64(nil), IndexNames:[]string(nil), StmtType:"Select", OriginalSQL:"select 1", digestMemo:struct { sync.Once; normalized string; digest *parser.Digest }{Once:sync.Once{done:0x1, m:sync.Mutex{state:0, sema:0x0}}, normalized:"select ?", digest:(*parser.Digest)(0x140125ddce0)}, BindSQL:"", planNormalized:"0\t3\t0\t?\n1\t25\t0\t\n", planDigest:(*parser.Digest)(0x1401241eb40), encodedPlan:"e/BbMAkzXzMJMAkxCTEtPkNvbHVtbiMxCTEJdGltZTo4Ljc5wrVzLCBsb29wczoyLCBDb25jdXJyZW5jeTpPRkYJMCBCeXRlcwlOL0EKMQkyNV80CTAJMQlyb3dzOjERRww3NTBuGUUgCU4vQQlOL0EK", planHint:"", planHintSet:true, Tables:[]stmtctx.TableEntry(nil), PointExec:false, lockWaitStartTime:0, PessimisticLockWaited:0, LockKeysDuration:0, LockKeysCount:0, LockTableIDs:map[int64]struct {}{}, TblInfo2UnionScan:map[*model.TableInfo]bool{}, TaskID:0x88, TaskMapBakTS:0x2, stmtCache:map[stmtctx.StmtCacheKey]interface {}(nil), CTEStorageMap:interface {}(nil), ReadFromTableCache:false, cache:struct { execdetails.RuntimeStatsColl; MemTracker memory.Tracker; DiskTracker memory.Tracker; LogOnExceed [2]memory.LogOnExceed }{RuntimeStatsColl:execdetails.RuntimeStatsColl{mu:sync.Mutex{state:0, sema:0x0}, rootStats:map[int]*execdetails.RootRuntimeStats(nil), copStats:map[int]*execdetails.CopRuntimeStats(nil)}, MemTracker:memory.Tracker{mu:struct { sync.Mutex; children map[int][]*memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, children:map[int][]*memory.Tracker{3:[]*memory.Tracker{(*memory.Tracker)(0x140125af800)}}}, actionMuForHardLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:(*memory.PanicOnExceed)(0x140124e62c0)}, actionMuForSoftLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:memory.ActionOnExceed(nil)}, parMu:struct { sync.Mutex; parent *memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, parent:(*memory.Tracker)(nil)}, label:-1, bytesConsumed:0, bytesLimit:atomic.Value{v:(*memory.bytesLimits)(0x140124bff60)}, maxConsumed:atomic.Int64{_:atomic.nocmp{}, v:0}, isGlobal:false}, DiskTracker:memory.Tracker{mu:struct { sync.Mutex; children map[int][]*memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, children:map[int][]*memory.Tracker(nil)}, actionMuForHardLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:(*memory.LogOnExceed)(0x14012284c18)}, actionMuForSoftLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:memory.ActionOnExceed(nil)}, parMu:struct { sync.Mutex; parent *memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, parent:(*memory.Tracker)(nil)}, label:-1, bytesConsumed:0, bytesLimit:atomic.Value{v:(*memory.bytesLimits)(0x140124bff70)}, maxConsumed:atomic.Int64{_:atomic.nocmp{}, v:0}, isGlobal:false}, LogOnExceed:[2]memory.LogOnExceed{memory.LogOnExceed{BaseOOMAction:memory.BaseOOMAction{fallbackAction:memory.ActionOnExceed(nil), finished:0}, mutex:sync.Mutex{state:0, sema:0x0}, acted:false, ConnID:0x0, logHook:(func(uint64))(nil)}, memory.LogOnExceed{BaseOOMAction:memory.BaseOOMAction{fallbackAction:memory.ActionOnExceed(nil), finished:0}, mutex:sync.Mutex{state:0, sema:0x0}, acted:false, ConnID:0x0, logHook:(func(uint64))(nil)}}}, OptimInfo:map[int]string(nil), InVerboseExplain:false, EnableOptimizeTrace:false, OptimizeTracer:(*tracing.OptimizeTracer)(nil), EnableOptimizerCETrace:false, OptimizerCETrace:[]*tracing.CETraceRecord(nil), WaitLockLeaseTime:0, KvExecCounter:(*stmtstats.KvExecCounter)(nil), WeakConsistency:false, StatsLoad:struct { Timeout time.Duration; NeededItems []model.TableItemID; ResultCh chan model.TableItemID; Fallback bool; LoadStartTime time.Time }{Timeout:0, NeededItems:[]model.TableItemID(nil), ResultCh:(chan model.TableItemID)(nil), Fallback:false, LoadStartTime:time.Time{wall:0x0, ext:0, loc:(*time.Location)(nil)}}, SysdateIsNow:false, RCCheckTS:false, IsSQLRegistered:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}, IsSQLAndPlanRegistered:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}}, stmtctx.StatementContext{StmtHints:stmtctx.StmtHints{MemQuotaQuery:0, ApplyCacheCapacity:0, MaxExecutionTime:0x0, ReplicaRead:0x0, AllowInSubqToJoinAndAgg:false, NoIndexMergeHint:false, StraightJoinOrder:false, EnableCascadesPlanner:false, ForceNthPlan:0, HasAllowInSubqToJoinAndAggHint:false, HasMemQuotaHint:false, HasReplicaReadHint:false, HasMaxExecutionTime:false, HasEnableCascadesPlannerHint:false, SetVars:map[string]string(nil), OriginalTableHints:[]*ast.TableOptimizerHint(nil)}, IsDDLJobInQueue:false, DDLJobID:0, InInsertStmt:false, InUpdateStmt:false, InDeleteStmt:false, InSelectStmt:false, InLoadDataStmt:false, InExplainStmt:false, InCreateOrAlterStmt:false, InSetSessionStatesStmt:false, InPreparedPlanBuilding:false, IgnoreTruncate:true, IgnoreZeroInDate:true, NoZeroDate:false, DupKeyAsWarning:false, BadNullAsWarning:false, DividedByZeroAsWarning:false, TruncateAsWarning:false, OverflowAsWarning:false, InShowWarning:false, UseCache:false, BatchCheck:false, InNullRejectCheck:false, AllowInvalidDate:false, IgnoreNoPartition:false, SkipPlanCache:false, IgnoreExplainIDSuffix:false, SkipUTF8Check:false, SkipASCIICheck:false, SkipUTF8MB4Check:false, MultiSchemaInfo:(*model.MultiSchemaInfo)(nil), IsStaleness:false, InRestrictedSQL:false, mu:struct { sync.Mutex; affectedRows uint64; foundRows uint64; records uint64; deleted uint64; updated uint64; copied uint64; touched uint64; message string; warnings []stmtctx.SQLWarn; errorCount uint16; execDetails execdetails.ExecDetails; allExecDetails []*execdetails.ExecDetails }{Mutex:sync.Mutex{state:0, sema:0x0}, affectedRows:0x0, foundRows:0x0, records:0x0, deleted:0x0, updated:0x0, copied:0x0, touched:0x0, message:"", warnings:[]stmtctx.SQLWarn(nil), errorCount:0x0, execDetails:execdetails.ExecDetails{CalleeAddress:"", CopTime:0, BackoffTime:0, LockKeysDuration:0, BackoffSleep:map[string]time.Duration(nil), BackoffTimes:map[string]int(nil), RequestCount:0, CommitDetail:(*util.CommitDetails)(nil), LockKeysDetail:(*util.LockKeysDetails)(nil), ScanDetail:(*util.ScanDetail)(nil), TimeDetail:util.TimeDetail{ProcessTime:0, WaitTime:0, KvReadWallTimeMs:0}}, allExecDetails:[]*execdetails.ExecDetails(nil)}, PrevAffectedRows:-1, PrevLastInsertID:0x0, LastInsertID:0x0, InsertID:0x0, BaseRowID:0, MaxRowID:0, TimeZone:(*time.Location)(0x14011552fc0), Priority:0, NotFillCache:false, MemTracker:(*memory.Tracker)(0x14012285088), DiskTracker:(*memory.Tracker)(0x14012285108), IsTiFlash:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}, RuntimeStatsColl:(*execdetails.RuntimeStatsColl)(0x14011f68f30), TableIDs:[]int64(nil), IndexNames:[]string(nil), StmtType:"Show", OriginalSQL:"show tables", digestMemo:struct { sync.Once; normalized string; digest *parser.Digest }{Once:sync.Once{done:0x1, m:sync.Mutex{state:0, sema:0x0}}, normalized:"show tables", digest:(*parser.Digest)(0x1401482ed80)}, BindSQL:"", planNormalized:"0\t7\t0\t\n", planDigest:(*parser.Digest)(0x1401482efc0), encodedPlan:"", planHint:"", planHintSet:false, Tables:[]stmtctx.TableEntry(nil), PointExec:false, lockWaitStartTime:0, PessimisticLockWaited:0, LockKeysDuration:0, LockKeysCount:0, LockTableIDs:map[int64]struct {}{}, TblInfo2UnionScan:map[*model.TableInfo]bool{}, TaskID:0xb2, TaskMapBakTS:0x0, stmtCache:map[stmtctx.StmtCacheKey]interface {}(nil), CTEStorageMap:interface {}(nil), ReadFromTableCache:false, cache:struct { execdetails.RuntimeStatsColl; MemTracker memory.Tracker; DiskTracker memory.Tracker; LogOnExceed [2]memory.LogOnExceed }{RuntimeStatsColl:execdetails.RuntimeStatsColl{mu:sync.Mutex{state:0, sema:0x0}, rootStats:map[int]*execdetails.RootRuntimeStats(nil), copStats:map[int]*execdetails.CopRuntimeStats(nil)}, MemTracker:memory.Tracker{mu:struct { sync.Mutex; children map[int][]*memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, children:map[int][]*memory.Tracker(nil)}, actionMuForHardLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:(*memory.PanicOnExceed)(0x1401489a7c0)}, actionMuForSoftLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:memory.ActionOnExceed(nil)}, parMu:struct { sync.Mutex; parent *memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, parent:(*memory.Tracker)(nil)}, label:-1, bytesConsumed:0, bytesLimit:atomic.Value{v:(*memory.bytesLimits)(0x140129589f0)}, maxConsumed:atomic.Int64{_:atomic.nocmp{}, v:0}, isGlobal:false}, DiskTracker:memory.Tracker{mu:struct { sync.Mutex; children map[int][]*memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, children:map[int][]*memory.Tracker(nil)}, actionMuForHardLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:(*memory.LogOnExceed)(0x14012285188)}, actionMuForSoftLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:memory.ActionOnExceed(nil)}, parMu:struct { sync.Mutex; parent *memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, parent:(*memory.Tracker)(nil)}, label:-1, bytesConsumed:0, bytesLimit:atomic.Value{v:(*memory.bytesLimits)(0x14012958a00)}, maxConsumed:atomic.Int64{_:atomic.nocmp{}, v:0}, isGlobal:false}, LogOnExceed:[2]memory.LogOnExceed{memory.LogOnExceed{BaseOOMAction:memory.BaseOOMAction{fallbackAction:memory.ActionOnExceed(nil), finished:0}, mutex:sync.Mutex{state:0, sema:0x0}, acted:false, ConnID:0x0, logHook:(func(uint64))(nil)}, memory.LogOnExceed{BaseOOMAction:memory.BaseOOMAction{fallbackAction:memory.ActionOnExceed(nil), finished:0}, mutex:sync.Mutex{state:0, sema:0x0}, acted:false, ConnID:0x0, logHook:(func(uint64))(nil)}}}, OptimInfo:map[int]string(nil), InVerboseExplain:false, EnableOptimizeTrace:false, OptimizeTracer:(*tracing.OptimizeTracer)(nil), EnableOptimizerCETrace:false, OptimizerCETrace:[]*tracing.CETraceRecord(nil), WaitLockLeaseTime:0, KvExecCounter:(*stmtstats.KvExecCounter)(0x1401489a880), WeakConsistency:false, StatsLoad:struct { Timeout time.Duration; NeededItems []model.TableItemID; ResultCh chan model.TableItemID; Fallback bool; LoadStartTime time.Time }{Timeout:0, NeededItems:[]model.TableItemID(nil), ResultCh:(chan model.TableItemID)(nil), Fallback:false, LoadStartTime:time.Time{wall:0x0, ext:0, loc:(*time.Location)(nil)}}, SysdateIsNow:false, RCCheckTS:false, IsSQLRegistered:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}, IsSQLAndPlanRegistered:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}}}}, Rng:(*mathutil.MysqlRng)(0x140121c65f0), EnablePaging:true, EnableLegacyInstanceScope:true, ReadConsistency:"strict", StatsLoadSyncWait:0, SysdateIsNow:false, EnableMutationChecker:true, AssertionLevel:1, IgnorePreparedCacheCloseStmt:false, EnableNewCostInterface:false, CostModelVersion:1, BatchPendingTiFlashCount:4000, RcReadCheckTS:false, RemoveOrderbyInSubquery:false, NonTransactionalIgnoreError:false, MaxAllowedPacket:0x4000000, TiFlashFineGrainedShuffleStreamCount:-1, TiFlashFineGrainedShuffleBatchSize:0x2000, RequestSourceType:"Show"}
#### event: COMPLETED sctx: &variable.SessionVars{Concurrency:variable.Concurrency{indexLookupConcurrency:-1, indexLookupJoinConcurrency:-1, distSQLScanConcurrency:15, hashJoinConcurrency:-1, projectionConcurrency:-1, hashAggPartialConcurrency:-1, hashAggFinalConcurrency:-1, windowConcurrency:-1, mergeJoinConcurrency:1, streamAggConcurrency:1, indexSerialScanConcurrency:1, ExecutorConcurrency:5, SourceAddr:net.TCPAddr{IP:net.IP(nil), Port:0, Zone:""}}, MemQuota:variable.MemQuota{MemQuotaQuery:1073741824, MemQuotaApplyCache:33554432}, BatchSize:variable.BatchSize{IndexJoinBatchSize:25000, IndexLookupSize:20000, InitChunkSize:32, MaxChunkSize:1024}, DMLBatchSize:0, RetryLimit:10, DisableTxnAutoRetry:true, UsersLock:sync.RWMutex{w:sync.Mutex{state:0, sema:0x0}, writerSem:0x0, readerSem:0x0, readerCount:0, readerWait:0}, Users:map[string]types.Datum{}, UserVarTypes:map[string]*types.FieldType{}, systems:map[string]string{"allow_auto_random_explicit_insert":"OFF", "auto_increment_increment":"1", "auto_increment_offset":"1", "autocommit":"ON", "block_encryption_mode":"aes-128-ecb", "character_set_client":"utf8mb4", "character_set_connection":"utf8mb4", "character_set_database":"utf8mb4", "character_set_results":"utf8mb4", "character_set_server":"utf8mb4", "collation_connection":"utf8mb4_bin", "collation_database":"utf8mb4_bin", "collation_server":"utf8mb4_bin", "cte_max_recursion_depth":"1000", "default_week_format":"0", "innodb_lock_wait_timeout":"50", "interactive_timeout":"28800", "max_allowed_packet":"67108864", "max_execution_time":"0", "sql_mode":"ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION", "sql_select_limit":"18446744073709551615", "tidb_allow_batch_cop":"1", "tidb_allow_fallback_to_tikv":"", "tidb_allow_mpp":"ON", "tidb_analyze_version":"2", "tidb_backoff_lock_fast":"10", "tidb_backoff_weight":"2", "tidb_batch_pending_tiflash_count":"4000", "tidb_broadcast_join_threshold_count":"10240", "tidb_broadcast_join_threshold_size":"104857600", "tidb_constraint_check_in_place":"OFF", "tidb_cost_model_version":"1", "tidb_disable_txn_auto_retry":"ON", "tidb_distsql_scan_concurrency":"15", "tidb_dml_batch_size":"0", "tidb_enable_1pc":"OFF", "tidb_enable_amend_pessimistic_txn":"OFF", "tidb_enable_async_commit":"OFF", "tidb_enable_auto_increment_in_generated":"OFF", "tidb_enable_cascades_planner":"OFF", "tidb_enable_chunk_rpc":"1", "tidb_enable_clustered_index":"INT_ONLY", "tidb_enable_exchange_partition":"OFF", "tidb_enable_extended_stats":"OFF", "tidb_enable_fast_analyze":"OFF", "tidb_enable_index_merge":"ON", "tidb_enable_index_merge_join":"OFF", "tidb_enable_legacy_instance_scope":"ON", "tidb_enable_list_partition":"ON", "tidb_enable_mutation_checker":"ON", "tidb_enable_new_cost_interface":"OFF", "tidb_enable_new_only_full_group_by_check":"OFF", "tidb_enable_noop_functions":"OFF", "tidb_enable_ordered_result_mode":"OFF", "tidb_enable_paging":"ON", "tidb_enable_parallel_apply":"OFF", "tidb_enable_pipelined_window_function":"ON", "tidb_enable_pseudo_for_outdated_stats":"ON", "tidb_enable_rate_limit_action":"ON", "tidb_enable_strict_double_type_check":"ON", "tidb_enable_table_partition":"ON", "tidb_enable_vectorized_expression":"ON", "tidb_enable_window_function":"ON", "tidb_enforce_mpp":"OFF", "tidb_evolve_plan_baselines":"OFF", "tidb_executor_concurrency":"5", "tidb_guarantee_linearizability":"ON", "tidb_hash_exchange_with_new_collation":"ON", "tidb_hash_join_concurrency":"-1", "tidb_hashagg_final_concurrency":"-1", "tidb_hashagg_partial_concurrency":"-1", "tidb_ignore_prepared_cache_close_stmt":"OFF", "tidb_index_join_batch_size":"25000", "tidb_index_lookup_concurrency":"-1", "tidb_index_lookup_join_concurrency":"-1", "tidb_index_lookup_size":"20000", "tidb_index_serial_scan_concurrency":"1", "tidb_init_chunk_size":"32", "tidb_isolation_read_engines":"tikv,tiflash,tidb", "tidb_max_chunk_size":"1024", "tidb_max_tiflash_threads":"-1", "tidb_mem_quota_apply_cache":"33554432", "tidb_mem_quota_query":"1073741824", "tidb_merge_join_concurrency":"1", "tidb_mpp_store_fail_ttl":"60s", "tidb_multi_statement_mode":"OFF", "tidb_nontransactional_ignore_error":"OFF", "tidb_opt_agg_push_down":"OFF", "tidb_opt_broadcast_cartesian_join":"1", "tidb_opt_concurrency_factor":"3", "tidb_opt_concurrency_factor_v2":"3", "tidb_opt_copcpu_factor":"3", "tidb_opt_copcpu_factor_v2":"30", "tidb_opt_correlation_exp_factor":"1", "tidb_opt_correlation_threshold":"0.9", "tidb_opt_cpu_factor":"3", "tidb_opt_cpu_factor_v2":"30", "tidb_opt_desc_factor":"3", "tidb_opt_desc_factor_v2":"150", "tidb_opt_disk_factor":"1.5", "tidb_opt_disk_factor_v2":"1.5", "tidb_opt_enable_correlation_adjustment":"ON", "tidb_opt_insubq_to_join_and_agg":"ON", "tidb_opt_limit_push_down_threshold":"100", "tidb_opt_memory_factor":"0.001", "tidb_opt_memory_factor_v2":"0.001", "tidb_opt_mpp_outer_join_fixed_build_side":"OFF", "tidb_opt_network_factor":"1", "tidb_opt_network_factor_v2":"4", "tidb_opt_prefer_range_scan":"OFF", "tidb_opt_projection_push_down":"OFF", "tidb_opt_scan_factor":"1.5", "tidb_opt_scan_factor_v2":"100", "tidb_opt_seek_factor_v2":"9500000", "tidb_opt_tiflash_cpu_factor_v2":"2", "tidb_opt_tiflash_scan_factor_v2":"15", "tidb_partition_prune_mode":"static", "tidb_placement_mode":"STRICT", "tidb_projection_concurrency":"-1", "tidb_rc_read_check_ts":"OFF", "tidb_read_consistency":"strict", "tidb_read_staleness":"0", "tidb_redact_log":"OFF", "tidb_regard_null_as_point":"ON", "tidb_remove_orderby_in_subquery":"OFF", "tidb_replica_read":"leader", "tidb_retry_limit":"10", "tidb_row_format_version":"2", "tidb_shard_allocate_step":"9223372036854775807", "tidb_skip_ascii_check":"OFF", "tidb_skip_isolation_level_check":"OFF", "tidb_skip_utf8_check":"OFF", "tidb_streamagg_concurrency":"1", "tidb_sysdate_is_now":"OFF", "tidb_tmp_table_max_size":"67108864", "tidb_track_aggregate_memory_usage":"ON", "tidb_txn_assertion_level":"FAST", "tidb_txn_mode":"", "tidb_use_plan_baselines":"ON", "tidb_window_concurrency":"-1", "tiflash_fine_grained_shuffle_batch_size":"8192", "tiflash_fine_grained_shuffle_stream_count":"-1", "time_zone":"SYSTEM", "transaction_isolation":"REPEATABLE-READ", "tx_isolation":"REPEATABLE-READ", "tx_read_ts":"", "wait_timeout":"28800", "windowing_use_high_precision":"ON"}, stmtVars:map[string]string{}, SysWarningCount:0, SysErrorCount:0x0, PreparedStmts:map[uint32]interface {}{}, PreparedStmtNameToID:map[string]uint32{}, preparedStmtID:0x0, PreparedParams:variable.PreparedParams{}, LastUpdateTime4PC:types.Time{coreTime:0x0}, ActiveRoles:[]*auth.RoleIdentity{}, RetryInfo:(*variable.RetryInfo)(0x140121c4b60), TxnCtx:(*variable.TransactionContext)(0x1401490ce00), TxnManager:(*session.txnManager)(0x140121a9920), KVVars:(*kv.Variables)(0x14011f68d68), txnIsolationLevelOneShot:struct { state variable.txnIsolationLevelOneShotState; value string }{state:0x0, value:""}, mppTaskIDAllocator:struct { mu sync.Mutex; lastTS uint64; taskID int64 }{mu:sync.Mutex{state:0, sema:0x0}, lastTS:0x0, taskID:0}, Status:0x2, ClientCapability:0x1ba685, TLSConnectionState:(*tls.ConnectionState)(nil), ConnectionID:0x20000000193, PlanID:4, PlanColumnID:1, MapHashCode2UniqueID4ExtendedCol:map[string]int(nil), User:(*auth.UserIdentity)(0x1401219db30), Port:"49641", CurrentDB:"test", CurrentDBChanged:true, StrictSQLMode:true, CommonGlobalLoaded:true, InRestrictedSQL:false, SnapshotTS:0x0, TxnReadTS:(*variable.TxnReadTS)(0x14012216800), SnapshotInfoschema:interface {}(nil), BinlogClient:(*client.PumpsClient)(nil), GlobalVarsAccessor:(*session.session)(0x140122126c0), LastFoundRows:0x1, StmtCtx:(*stmtctx.StatementContext)(0x140122847c8), AllowAggPushDown:false, AllowCartesianBCJ:1, MPPOuterJoinFixedBuildSide:false, AllowDistinctAggPushDown:false, MultiStatementMode:0, AllowWriteRowID:false, AllowBatchCop:1, allowMPPExecution:true, HashExchangeWithNewCollation:true, enforceMPPExecution:false, TiFlashMaxThreads:-1, AllowAutoRandExplicitInsert:false, BroadcastJoinThresholdSize:104857600, BroadcastJoinThresholdCount:10240, LimitPushDownThreshold:100, CorrelationThreshold:0.9, EnableCorrelationAdjustment:true, CorrelationExpFactor:1, cpuFactor:3, copCPUFactor:3, networkFactor:1, scanFactor:1.5, descScanFactor:3, seekFactor:20, memoryFactor:0.001, diskFactor:1.5, concurrencyFactor:3, cpuFactorV2:30, copCPUFactorV2:30, tiflashCPUFactorV2:2, networkFactorV2:4, scanFactorV2:100, descScanFactorV2:150, tiflashScanFactorV2:15, seekFactorV2:9.5e+06, memoryFactorV2:0.001, diskFactorV2:1.5, concurrencyFactorV2:3, CopTiFlashConcurrencyFactor:24, CurrInsertValues:chunk.Row{c:(*chunk.Chunk)(nil), idx:0}, CurrInsertBatchExtraCols:[][]types.Datum(nil), TimeZone:(*time.Location)(0x14011552fc0), SQLMode:1436549152, AutoIncrementIncrement:1, AutoIncrementOffset:1, SkipASCIICheck:false, SkipUTF8Check:false, BatchInsert:false, BatchDelete:false, BatchCommit:false, IDAllocator:autoid.Allocator(nil), OptimizerSelectivityLevel:0, OptimizerEnableNewOnlyFullGroupByCheck:false, EnableOuterJoinReorder:true, EnableTablePartition:"ON", EnableListTablePartition:true, EnableCascadesPlanner:false, EnableWindowFunction:true, EnablePipelinedWindowExec:true, AllowProjectionPushDown:false, EnableStrictDoubleTypeCheck:true, EnableVectorizedExpression:true, DDLReorgPriority:1, EnableAutoIncrementInGenerated:false, EnablePointGetCache:false, PlacementMode:"STRICT", WaitSplitRegionFinish:true, WaitSplitRegionTimeout:0x12c, EnableChunkRPC:true, writeStmtBufs:variable.WriteStmtBufs{RowValBuf:[]uint8(nil), AddRowValues:[]types.Datum(nil), IndexValsBuf:[]types.Datum(nil), IndexKeyBuf:[]uint8(nil)}, ConstraintCheckInPlace:false, CommandValue:0x3, TiDBOptJoinReorderThreshold:0, SlowQueryFile:"tidb-slow.log", EnableFastAnalyze:false, TxnMode:"", LowResolutionTSO:false, MaxExecutionTime:0x0, Killed:0x0, ConnectionInfo:(*variable.ConnectionInfo)(0x1401214e540), NoopFuncsMode:0, StartTime:time.Date(2022, time.July, 11, 11, 21, 9, 323668000, time.Local), DurationParse:0, DurationCompile:222583, RewritePhaseInfo:variable.RewritePhaseInfo{DurationRewrite:130709, DurationPreprocessSubQuery:0, PreprocessSubQueries:0}, DurationOptimization:18125, DurationWaitTS:0, PrevStmt:(stringutil.StringerFunc)(0x101c99af0), prevStmtDigest:"e1c71d1661ae46e09b7aaec1c390957f0d6260410df4e4bc71b9c8d681021471", AllowRemoveAutoInc:false, UsePlanBaselines:true, EvolvePlanBaselines:false, EnableExtendedStats:false, allowInSubqToJoinAndAgg:true, preferRangeScan:false, enableIndexMerge:true, replicaRead:0x0, IsolationReadEngines:map[kv.StoreType]struct {}{0x0:struct {}{}, 0x1:struct {}{}, 0x2:struct {}{}}, PlannerSelectBlockAsName:[]ast.HintTable{ast.HintTable{DBName:model.CIStr{O:"", L:""}, TableName:model.CIStr{O:"", L:""}, QBName:model.CIStr{O:"", L:""}, PartitionList:[]model.CIStr(nil)}, ast.HintTable{DBName:model.CIStr{O:"", L:""}, TableName:model.CIStr{O:"", L:""}, QBName:model.CIStr{O:"", L:""}, PartitionList:[]model.CIStr(nil)}}, LockWaitTimeout:50000, MetricSchemaStep:60, MetricSchemaRangeDuration:60, InspectionTableCache:map[string]variable.TableSnapshot(nil), RowEncoder:rowcodec.Encoder{row:rowcodec.row{large:false, numNotNullCols:0x0, numNullCols:0x0, colIDs:[]uint8(nil), offsets:[]uint16(nil), data:[]uint8(nil), colIDs32:[]uint32(nil), offsets32:[]uint32(nil)}, tempColIDs:[]int64(nil), values:[]*types.Datum(nil), Enable:true}, SequenceState:(*variable.SequenceState)(0x140121c65d0), WindowingUseHighPrecision:true, FoundInPlanCache:false, PrevFoundInPlanCache:false, FoundInBinding:false, PrevFoundInBinding:false, OptimizerUseInvisibleIndexes:false, SelectLimit:0xffffffffffffffff, EnableClusteredIndex:0, PresumeKeyNotExists:false, EnableParallelApply:false, EnableRedactLog:false, ShardAllocateStep:9223372036854775807, EnableAmendPessimisticTxn:false, LastTxnInfo:"", LastQueryInfo:sessionstates.QueryInfo{TxnScope:"global", StartTS:0x0, ForUpdateTS:0x0, ErrMsg:""}, LastDDLInfo:sessionstates.LastDDLInfo{Query:"", SeqNum:0x0}, PartitionPruneMode:atomic.String{_:atomic.nocmp{}, v:atomic.Value{Value:atomic.Value{v:"static"}, _:atomic.nocmp{}}}, TxnScope:kv.TxnScopeVar{varValue:"global", txnScope:"global"}, EnabledRateLimitAction:true, EnableAsyncCommit:false, Enable1PC:false, GuaranteeLinearizability:true, AnalyzeVersion:2, EnableIndexMergeJoin:false, TrackAggregateMemoryUsage:true, TiDBEnableExchangePartition:false, AllowFallbackToTiKV:map[kv.StoreType]struct {}{}, CTEMaxRecursionDepth:1000, TMPTableSize:67108864, EnableStableResultMode:false, EnablePseudoForOutdatedStats:true, RegardNULLAsPoint:true, LocalTemporaryTables:(*infoschema.LocalTemporaryTables)(0x140121c6700), TemporaryTableData:variable.TemporaryTableData(nil), MPPStoreLastFailTime:map[string]time.Time{}, MPPStoreFailTTL:"60s", ReadStaleness:0, cached:struct { curr int8; data [2]stmtctx.StatementContext }{curr:0, data:[2]stmtctx.StatementContext{stmtctx.StatementContext{StmtHints:stmtctx.StmtHints{MemQuotaQuery:0, ApplyCacheCapacity:0, MaxExecutionTime:0x0, ReplicaRead:0x0, AllowInSubqToJoinAndAgg:false, NoIndexMergeHint:false, StraightJoinOrder:false, EnableCascadesPlanner:false, ForceNthPlan:0, HasAllowInSubqToJoinAndAggHint:false, HasMemQuotaHint:false, HasReplicaReadHint:false, HasMaxExecutionTime:false, HasEnableCascadesPlannerHint:false, SetVars:map[string]string(nil), OriginalTableHints:[]*ast.TableOptimizerHint(nil)}, IsDDLJobInQueue:false, DDLJobID:0, InInsertStmt:false, InUpdateStmt:false, InDeleteStmt:false, InSelectStmt:true, InLoadDataStmt:false, InExplainStmt:false, InCreateOrAlterStmt:false, InSetSessionStatesStmt:false, InPreparedPlanBuilding:false, IgnoreTruncate:false, IgnoreZeroInDate:true, NoZeroDate:false, DupKeyAsWarning:false, BadNullAsWarning:false, DividedByZeroAsWarning:false, TruncateAsWarning:true, OverflowAsWarning:true, InShowWarning:false, UseCache:false, BatchCheck:false, InNullRejectCheck:false, AllowInvalidDate:false, IgnoreNoPartition:false, SkipPlanCache:false, IgnoreExplainIDSuffix:false, SkipUTF8Check:false, SkipASCIICheck:false, SkipUTF8MB4Check:false, MultiSchemaInfo:(*model.MultiSchemaInfo)(nil), IsStaleness:false, InRestrictedSQL:false, mu:struct { sync.Mutex; affectedRows uint64; foundRows uint64; records uint64; deleted uint64; updated uint64; copied uint64; touched uint64; message string; warnings []stmtctx.SQLWarn; errorCount uint16; execDetails execdetails.ExecDetails; allExecDetails []*execdetails.ExecDetails }{Mutex:sync.Mutex{state:0, sema:0x0}, affectedRows:0x0, foundRows:0x1, records:0x0, deleted:0x0, updated:0x0, copied:0x0, touched:0x0, message:"", warnings:[]stmtctx.SQLWarn(nil), errorCount:0x0, execDetails:execdetails.ExecDetails{CalleeAddress:"", CopTime:0, BackoffTime:0, LockKeysDuration:0, BackoffSleep:map[string]time.Duration(nil), BackoffTimes:map[string]int(nil), RequestCount:0, CommitDetail:(*util.CommitDetails)(nil), LockKeysDetail:(*util.LockKeysDetails)(nil), ScanDetail:(*util.ScanDetail)(nil), TimeDetail:util.TimeDetail{ProcessTime:0, WaitTime:0, KvReadWallTimeMs:0}}, allExecDetails:[]*execdetails.ExecDetails(nil)}, PrevAffectedRows:0, PrevLastInsertID:0x0, LastInsertID:0x0, InsertID:0x0, BaseRowID:0, MaxRowID:0, TimeZone:(*time.Location)(0x14011552fc0), Priority:0, NotFillCache:false, MemTracker:(*memory.Tracker)(0x14012284b18), DiskTracker:(*memory.Tracker)(0x14012284b98), IsTiFlash:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}, RuntimeStatsColl:(*execdetails.RuntimeStatsColl)(0x14011f68f30), TableIDs:[]int64(nil), IndexNames:[]string(nil), StmtType:"Select", OriginalSQL:"select 1", digestMemo:struct { sync.Once; normalized string; digest *parser.Digest }{Once:sync.Once{done:0x1, m:sync.Mutex{state:0, sema:0x0}}, normalized:"select ?", digest:(*parser.Digest)(0x14014b2d320)}, BindSQL:"", planNormalized:"0\t3\t0\t?\n1\t25\t0\t\n", planDigest:(*parser.Digest)(0x14014b2d770), encodedPlan:"", planHint:"", planHintSet:false, Tables:[]stmtctx.TableEntry(nil), PointExec:false, lockWaitStartTime:0, PessimisticLockWaited:0, LockKeysDuration:0, LockKeysCount:0, LockTableIDs:map[int64]struct {}{}, TblInfo2UnionScan:map[*model.TableInfo]bool{}, TaskID:0xbf, TaskMapBakTS:0x2, stmtCache:map[stmtctx.StmtCacheKey]interface {}(nil), CTEStorageMap:map[int]*executor.CTEStorages{}, ReadFromTableCache:false, cache:struct { execdetails.RuntimeStatsColl; MemTracker memory.Tracker; DiskTracker memory.Tracker; LogOnExceed [2]memory.LogOnExceed }{RuntimeStatsColl:execdetails.RuntimeStatsColl{mu:sync.Mutex{state:0, sema:0x0}, rootStats:map[int]*execdetails.RootRuntimeStats(nil), copStats:map[int]*execdetails.CopRuntimeStats(nil)}, MemTracker:memory.Tracker{mu:struct { sync.Mutex; children map[int][]*memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, children:map[int][]*memory.Tracker{3:[]*memory.Tracker{(*memory.Tracker)(0x14014853800)}}}, actionMuForHardLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:(*memory.PanicOnExceed)(0x140148e3240)}, actionMuForSoftLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:memory.ActionOnExceed(nil)}, parMu:struct { sync.Mutex; parent *memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, parent:(*memory.Tracker)(nil)}, label:-1, bytesConsumed:0, bytesLimit:atomic.Value{v:(*memory.bytesLimits)(0x140148f1e70)}, maxConsumed:atomic.Int64{_:atomic.nocmp{}, v:0}, isGlobal:false}, DiskTracker:memory.Tracker{mu:struct { sync.Mutex; children map[int][]*memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, children:map[int][]*memory.Tracker(nil)}, actionMuForHardLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:(*memory.LogOnExceed)(0x14012284c18)}, actionMuForSoftLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:memory.ActionOnExceed(nil)}, parMu:struct { sync.Mutex; parent *memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, parent:(*memory.Tracker)(nil)}, label:-1, bytesConsumed:0, bytesLimit:atomic.Value{v:(*memory.bytesLimits)(0x140148f1e80)}, maxConsumed:atomic.Int64{_:atomic.nocmp{}, v:0}, isGlobal:false}, LogOnExceed:[2]memory.LogOnExceed{memory.LogOnExceed{BaseOOMAction:memory.BaseOOMAction{fallbackAction:memory.ActionOnExceed(nil), finished:0}, mutex:sync.Mutex{state:0, sema:0x0}, acted:false, ConnID:0x0, logHook:(func(uint64))(nil)}, memory.LogOnExceed{BaseOOMAction:memory.BaseOOMAction{fallbackAction:memory.ActionOnExceed(nil), finished:0}, mutex:sync.Mutex{state:0, sema:0x0}, acted:false, ConnID:0x0, logHook:(func(uint64))(nil)}}}, OptimInfo:map[int]string(nil), InVerboseExplain:false, EnableOptimizeTrace:false, OptimizeTracer:(*tracing.OptimizeTracer)(nil), EnableOptimizerCETrace:false, OptimizerCETrace:[]*tracing.CETraceRecord(nil), WaitLockLeaseTime:0, KvExecCounter:(*stmtstats.KvExecCounter)(nil), WeakConsistency:false, StatsLoad:struct { Timeout time.Duration; NeededItems []model.TableItemID; ResultCh chan model.TableItemID; Fallback bool; LoadStartTime time.Time }{Timeout:0, NeededItems:[]model.TableItemID(nil), ResultCh:(chan model.TableItemID)(nil), Fallback:false, LoadStartTime:time.Time{wall:0x0, ext:0, loc:(*time.Location)(nil)}}, SysdateIsNow:false, RCCheckTS:false, IsSQLRegistered:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}, IsSQLAndPlanRegistered:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}}, stmtctx.StatementContext{StmtHints:stmtctx.StmtHints{MemQuotaQuery:0, ApplyCacheCapacity:0, MaxExecutionTime:0x0, ReplicaRead:0x0, AllowInSubqToJoinAndAgg:false, NoIndexMergeHint:false, StraightJoinOrder:false, EnableCascadesPlanner:false, ForceNthPlan:0, HasAllowInSubqToJoinAndAggHint:false, HasMemQuotaHint:false, HasReplicaReadHint:false, HasMaxExecutionTime:false, HasEnableCascadesPlannerHint:false, SetVars:map[string]string(nil), OriginalTableHints:[]*ast.TableOptimizerHint(nil)}, IsDDLJobInQueue:false, DDLJobID:0, InInsertStmt:false, InUpdateStmt:false, InDeleteStmt:false, InSelectStmt:false, InLoadDataStmt:false, InExplainStmt:false, InCreateOrAlterStmt:false, InSetSessionStatesStmt:false, InPreparedPlanBuilding:false, IgnoreTruncate:true, IgnoreZeroInDate:true, NoZeroDate:false, DupKeyAsWarning:false, BadNullAsWarning:false, DividedByZeroAsWarning:false, TruncateAsWarning:false, OverflowAsWarning:false, InShowWarning:false, UseCache:false, BatchCheck:false, InNullRejectCheck:false, AllowInvalidDate:false, IgnoreNoPartition:false, SkipPlanCache:false, IgnoreExplainIDSuffix:false, SkipUTF8Check:false, SkipASCIICheck:false, SkipUTF8MB4Check:false, MultiSchemaInfo:(*model.MultiSchemaInfo)(nil), IsStaleness:false, InRestrictedSQL:false, mu:struct { sync.Mutex; affectedRows uint64; foundRows uint64; records uint64; deleted uint64; updated uint64; copied uint64; touched uint64; message string; warnings []stmtctx.SQLWarn; errorCount uint16; execDetails execdetails.ExecDetails; allExecDetails []*execdetails.ExecDetails }{Mutex:sync.Mutex{state:0, sema:0x0}, affectedRows:0x0, foundRows:0x0, records:0x0, deleted:0x0, updated:0x0, copied:0x0, touched:0x0, message:"", warnings:[]stmtctx.SQLWarn(nil), errorCount:0x0, execDetails:execdetails.ExecDetails{CalleeAddress:"", CopTime:0, BackoffTime:0, LockKeysDuration:0, BackoffSleep:map[string]time.Duration(nil), BackoffTimes:map[string]int(nil), RequestCount:0, CommitDetail:(*util.CommitDetails)(nil), LockKeysDetail:(*util.LockKeysDetails)(nil), ScanDetail:(*util.ScanDetail)(nil), TimeDetail:util.TimeDetail{ProcessTime:0, WaitTime:0, KvReadWallTimeMs:0}}, allExecDetails:[]*execdetails.ExecDetails(nil)}, PrevAffectedRows:-1, PrevLastInsertID:0x0, LastInsertID:0x0, InsertID:0x0, BaseRowID:0, MaxRowID:0, TimeZone:(*time.Location)(0x14011552fc0), Priority:0, NotFillCache:false, MemTracker:(*memory.Tracker)(0x14012285088), DiskTracker:(*memory.Tracker)(0x14012285108), IsTiFlash:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}, RuntimeStatsColl:(*execdetails.RuntimeStatsColl)(0x14011f68f30), TableIDs:[]int64(nil), IndexNames:[]string(nil), StmtType:"Show", OriginalSQL:"show tables", digestMemo:struct { sync.Once; normalized string; digest *parser.Digest }{Once:sync.Once{done:0x1, m:sync.Mutex{state:0, sema:0x0}}, normalized:"show tables", digest:(*parser.Digest)(0x1401482ed80)}, BindSQL:"", planNormalized:"0\t7\t0\t\n", planDigest:(*parser.Digest)(0x1401482efc0), encodedPlan:"", planHint:"", planHintSet:false, Tables:[]stmtctx.TableEntry(nil), PointExec:false, lockWaitStartTime:0, PessimisticLockWaited:0, LockKeysDuration:0, LockKeysCount:0, LockTableIDs:map[int64]struct {}{}, TblInfo2UnionScan:map[*model.TableInfo]bool{}, TaskID:0xb2, TaskMapBakTS:0x0, stmtCache:map[stmtctx.StmtCacheKey]interface {}(nil), CTEStorageMap:interface {}(nil), ReadFromTableCache:false, cache:struct { execdetails.RuntimeStatsColl; MemTracker memory.Tracker; DiskTracker memory.Tracker; LogOnExceed [2]memory.LogOnExceed }{RuntimeStatsColl:execdetails.RuntimeStatsColl{mu:sync.Mutex{state:0, sema:0x0}, rootStats:map[int]*execdetails.RootRuntimeStats(nil), copStats:map[int]*execdetails.CopRuntimeStats(nil)}, MemTracker:memory.Tracker{mu:struct { sync.Mutex; children map[int][]*memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, children:map[int][]*memory.Tracker(nil)}, actionMuForHardLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:(*memory.PanicOnExceed)(0x1401489a7c0)}, actionMuForSoftLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:memory.ActionOnExceed(nil)}, parMu:struct { sync.Mutex; parent *memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, parent:(*memory.Tracker)(nil)}, label:-1, bytesConsumed:0, bytesLimit:atomic.Value{v:(*memory.bytesLimits)(0x140129589f0)}, maxConsumed:atomic.Int64{_:atomic.nocmp{}, v:0}, isGlobal:false}, DiskTracker:memory.Tracker{mu:struct { sync.Mutex; children map[int][]*memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, children:map[int][]*memory.Tracker(nil)}, actionMuForHardLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:(*memory.LogOnExceed)(0x14012285188)}, actionMuForSoftLimit:memory.actionMu{Mutex:sync.Mutex{state:0, sema:0x0}, actionOnExceed:memory.ActionOnExceed(nil)}, parMu:struct { sync.Mutex; parent *memory.Tracker }{Mutex:sync.Mutex{state:0, sema:0x0}, parent:(*memory.Tracker)(nil)}, label:-1, bytesConsumed:0, bytesLimit:atomic.Value{v:(*memory.bytesLimits)(0x14012958a00)}, maxConsumed:atomic.Int64{_:atomic.nocmp{}, v:0}, isGlobal:false}, LogOnExceed:[2]memory.LogOnExceed{memory.LogOnExceed{BaseOOMAction:memory.BaseOOMAction{fallbackAction:memory.ActionOnExceed(nil), finished:0}, mutex:sync.Mutex{state:0, sema:0x0}, acted:false, ConnID:0x0, logHook:(func(uint64))(nil)}, memory.LogOnExceed{BaseOOMAction:memory.BaseOOMAction{fallbackAction:memory.ActionOnExceed(nil), finished:0}, mutex:sync.Mutex{state:0, sema:0x0}, acted:false, ConnID:0x0, logHook:(func(uint64))(nil)}}}, OptimInfo:map[int]string(nil), InVerboseExplain:false, EnableOptimizeTrace:false, OptimizeTracer:(*tracing.OptimizeTracer)(nil), EnableOptimizerCETrace:false, OptimizerCETrace:[]*tracing.CETraceRecord(nil), WaitLockLeaseTime:0, KvExecCounter:(*stmtstats.KvExecCounter)(0x1401489a880), WeakConsistency:false, StatsLoad:struct { Timeout time.Duration; NeededItems []model.TableItemID; ResultCh chan model.TableItemID; Fallback bool; LoadStartTime time.Time }{Timeout:0, NeededItems:[]model.TableItemID(nil), ResultCh:(chan model.TableItemID)(nil), Fallback:false, LoadStartTime:time.Time{wall:0x0, ext:0, loc:(*time.Location)(nil)}}, SysdateIsNow:false, RCCheckTS:false, IsSQLRegistered:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}, IsSQLAndPlanRegistered:atomic.Bool{_:atomic.nocmp{}, v:atomic.Uint32{_:atomic.nocmp{}, v:0x0}}}}}, Rng:(*mathutil.MysqlRng)(0x140121c65f0), EnablePaging:true, EnableLegacyInstanceScope:true, ReadConsistency:"strict", StatsLoadSyncWait:0, SysdateIsNow:false, EnableMutationChecker:true, AssertionLevel:1, IgnorePreparedCacheCloseStmt:false, EnableNewCostInterface:false, CostModelVersion:1, BatchPendingTiFlashCount:4000, RcReadCheckTS:false, RemoveOrderbyInSubquery:false, NonTransactionalIgnoreError:false, MaxAllowedPacket:0x4000000, TiFlashFineGrainedShuffleStreamCount:-1, TiFlashFineGrainedShuffleBatchSize:0x2000, RequestSourceType:"Select"}

4. What is your TiDB version? (Required)

master

lcwangchao commented 2 years ago

@morgo Audit log are using StmtCtx to read the SQL, StmtType, etc.. However the ResetContextOfStmt is called in ExecuteStmt so when we call OnGeneralEvent with event Starting , the StmtCtx is not properly set yet.

It is not easy to move ResetContextOfStmt from ExecuteStmt to handleStmt because ExecuteStmt is also called by other methods like ExecRestrictedStmt or ExecuteInternal . If we move it, the modification will be huge.

Maybe it is not a good idea to force the audit log plugin to read the context from StmtCtx. I think we can decouple audit log with StmtCtx and pass the context to audit log according to the environment.

The current plugin API of tidb is also not well designed and does not provide a clear semantic for statement's lifecycle. So we are considering to redesign the audit log in the future.