Skip to content

Commit

Permalink
br: merge into feature branch (#31045)
Browse files Browse the repository at this point in the history
* br: fix the integration tests (#30423)

* util, cmd: remove unused filesort (#30438)

* *: update client-go for small backoff time (#30436)

* server: Fix unstable tests with FakeAuthSwitch (#30287)

* dumpling: fix dump failed when sequence exists (#30164)

* *: replace compareDatum by compare (#30421)

* lightning: fix gcs max key limit (#30393)

* expression, parser: add built-in func is_uuid (#30318)

* expression: migrate test-infra to testify for constant_fold_test.go (#30424)

* executor: fix pipelined window invalid memory address (#30418)

* makefile: add gotestsum for verify ci (#29848)

* server: close sql rows to fix unstable test (#30306)

* Makefile: add coverage record for BR and Dumpling (#30457)

* executor: track the mem usage of IndexMergeReader (#30210)

* infosync: close body when ReadAll encounters error (#30462)

* planner: show accessed partition when explain mpp query over partition table (#30367)

* *: Fix use of user identity in SHOW GRANTS + error messages (#30294)

* ddl: add not null flag for auto_increment column  (#30477)

* expression: make some unstable test serial (#30323)

* expression: migrate test-infra to testify for constant_propagation_test.go (#30430)

* executor: stable test TestSetDDLReorgBatchSize and TestSetDDLReorgWorkerCnt (#30480)

* statistics, util/ranger: add cardinality estimation trace for `GetRowCountBy...` (#30321)

* *: skip mysql client goroutine leak detection in integration ddl (#30467)

* executor,util: write slow query to slow log no matter what log level (#30461)

* executor: enable index_merge used in transaction. (#29875)

* logutil: add testcase for SlowQueryLogger.MaxDays/MaxSize/MaxBackups (#30316)

* expression: fix data race in builtin_other_vec_generated_test.go (#30503)

* expression: fix data race in the collationInfo (#30490)

* planner/core, session: fix error message of wrong variable scope (#30510)

* lightning: support Re/ReregisterMySQL by different tls name (#30463)

* executor: TestBatchGetandPointGetwithHashPartition test typo (#29669) (#29671)

* mockstore: improve log to avoid panic for nil pointer (#30513)

* *: replace compareDatum by compare, PR 10 (#30456)

* planner: Disable dynamic partition prune mode for all non-autocommit (#27532) (#30505)

* expression: change the log level of an confusing log from warn to debug (#30484)

* br: Check crypter.key valid before backup (#29991)

* *: replace compareDatum by compare, PR 11 (#30465)

* dumpling: fix default column collation with upstream when dump table (#30531)

* server: fix prepared cursor select (#30285)

* executor: HashJoinExec checks the buildError even if the probeSide is empty (#30471)

* parser, expression: follow mysql, increase interval precedence (#30528)

* makefile: set timeout 25m for make race (#30555)

* planner: fix the unstable test TestAnalyzeGlobalStatsWithOpts/2 (#30576)

* expression,types: Adjusts UNIX_TIMESTAMP() for non-existing DST values (#28739) (#30405)

* br: add res.Body.close to avoid leak (#30545)

* lightning: add back integration test lightning_error_summary (#30547)

* sessionctx/variable: small refactor (split large file) (#30511)

* ddl: let `admin cancel ddl jobs` run in a new transaction (#30549)

* *: Retry when placement PutBundles failed (#30590)

* dumpling: delete unit test in github actions (#30562)

* *: support trace plan target='estimation' statement (#30491)

* expression: migrate test-infra to testify for integration_test.go (#30548)

* planner: support trace for min/max eliminate (#30441)

* support min/max trace

Signed-off-by: yisaer <[email protected]>

* address the comment

Signed-off-by: yisaer <[email protected]>

Co-authored-by: Ti Chi Robot <[email protected]>

* br: remove cdclog in br (#30573)

* *: show cmd to check if all needed histograms are loaded (#29672)

* expression: clone repertoire when clone the scalar function (#30602)

* *: use the real StateRemote interface implementation for cached table (#30066)

* *: query failed after add index / timestamp out-of-range (#28424) (#29323)

* planner: implement collecting predicate columns from logical plan (#29878)

* *: show PK name when decoding the clustered index row key (#30623)

* ddl/callback_test.go: migrate test-infra to testify (#30317)

* *: Rename some names of placement ddl operation (#30622)

* executor: fix data race in the index_lookup_hash_join (#30619)

* ddl: remove unnecessary locking when adding an index (#29772)

* server: try to make `TidbTestSuite` more stable (#30643)

* *: Add some PD tests for placement and fix some bug found (#30621)

* *: migrate sync.WaitGroup to util.WaitGroupWrapper (#30644)

* planner: add trace for join eliminate rule (#30343)

* executor: migrate test-infra to testify for executor/shuffle_test.go (#30514)

* planner: make (*AccessPath).OnlyPointRange more succinct (#30520)

* planner: add trace for join reorder (#30394)

* executor: migrate test-infra to testify for executor/union_scan_test.go (#30525)

* expression: make cast return error if cast binary literal to another character set (#30537)

* *: update tikv client (#30670)

* *: update sysutil in go.mod to fix panic when search log (#30523)

* topsql: shouldn't evict the SQL meta, since the evicted SQL can be appear on Other components (TiKV) TopN records (#27050)

* testify: migrate test-infra to testify for analyze_test.go (#30640)

* util: replace compareDatum by compare,  point part (#30575)

* test: make all the tests run in serial (#30692)

* statistics: add mutex for Handle.globalMap and Handle.feedback (#30550)

* executor: fix regular expression in json so that it could match identifer start with '$' (#29750)

* util/testkit/testkit.go: fix typo (#30638)

* planner: Introduce a new global variable to control the historical statistics feature (#30646)

* topsql: introduce datasink interface (#30662)

* planner: unify the argument of stats functions to use SessionCtx instead of StatementContext (#30668)

* metrics: fix the Max SafeTS Gap metrics (#30689)

* lightning: Add source dir existence check for s3 (#30674)

* golangci-lint: support durationcheck (#30027)

* executor: fix data race on IndexHashJoin.cancelFunc (#30701)

* sessionctx/variable: change tidb_store_limit to global only (#30522)

* statistics: remove reassignment of Handle.pool in NewHandle (#30675)

* br: fix some unstable unit test cases. (#30716)

* bindinfo: fix the comment typo (#30616)

* server: support decoding prepared string args to character_set_client (#30723)

* expression: fix enum type join binary get wrong result (#30445)

* cmd/explaintest: fix wrong result comparison for explain test (#30717)

* parallel create tables in br

* metrics: fix copr-cache metrics (#30712)

* test: merge executor's serial tests to other tests (#30711)

* statistics: avoid deadlock when create/drop extended stats and analyze at the same time (#30566)

* ddl: add batch create table api

Signed-off-by: xhe <[email protected]>

* ddl: add unit tests

Signed-off-by: xhe <[email protected]>

* ddl: fix fmt

Signed-off-by: xhe <[email protected]>

* ddl: typo

Co-authored-by: Arenatlx <[email protected]>

* ddl: fix tests

Signed-off-by: xhe <[email protected]>

* ddl: rename to BatchCreateTableWithInfo

Signed-off-by: xhe <[email protected]>

* ddl: trace the error

Signed-off-by: xhe <[email protected]>

* ddl: comments

Signed-off-by: xhe <[email protected]>

* ddl: cancle the job right

Signed-off-by: xhe <[email protected]>

* ddl: cancel the job right 2

Signed-off-by: xhe <[email protected]>

* ddl: report error if entry too large

Signed-off-by: xhe <[email protected]>

* ddl: report error when table is duplicated

Signed-off-by: xhe <[email protected]>

* ddl: go fmt

Signed-off-by: xhe <[email protected]>

* infoschema: improve batch memory perf

Signed-off-by: xhe <[email protected]>

* ddl: retain ID

Signed-off-by: xhe <[email protected]>

* sessionctx: fix the value of analyze_version when upgrading 4.x to 5.… (#30743)

* ddl: reduce log frequency

Signed-off-by: xhe <[email protected]>

* ddl: fix tests

Signed-off-by: xhe <[email protected]>

* server: disable socket listener for `basicHTTPHandlerTestSuite` (#30680)

* planner: support the plan cache aware of bindings (#30169)

* planner: fix early set of plan's statisticsTable (#30754)

* *: implement renew write lock lease for cached table (#30206)

* *: Modify placement rule index to reserve some indexes for future work (#30737)

* executor: add an unit test case for unreasonable invoking Close (#30696)

* planner: fix wrong subquery's coercibility  (#30750)

* executor: add more testcases for index merge (#30497)

* server: add grpc server config for a suitable behavior (#30774)

* config, charset: make charset config not affected by collation config (#30572)

* lightning: emit tidb log by change FilterCore to only allow matched packages (#30700)

* topsql: a centralized place to generate tipb report data (#30781)

* planner: add trace for partition pruning (#30546)

* planner: refine collation handling for between (#30793)

* test: merge serial tests in bindinfo, expression, parser and statistics (#30749)

* br: update log description for split check (#30763)

* *: replace compareDatum by compare, range part (#30710)

* *: placement policy ref will be converted to direct options when recover or flashback table (#30705)

* ddl: handle the incorrect number of placement followers (#30715)

* ddl: revert "ddl: remove unnecessary locking when adding an index" (#30667)

* br/pkg/task: migrate test-infra to testify (#30605)

* *: fix the flen type datetime for union/case-when/control-funcs (#30588)

* types, util: clean up compareDatum (#30815)

* ddl: add helper function to set and query TiFlash's sync status (#30473)

* dumpling: fix more dumpling log level query template (#30176)

* parser: support `admin flush plan_cache` (#30747)

* topsql: support multiple datasinks (#30808)

* br: update permission, so tikv can write to folder when backup to local (#30396)

* session: fix bootstrap to only persist global variables (#30593)

close #28667

* docs/design: update collation compatibility issues in charsets doc (#30806)

* executor: improve SET sysvar=DEFAULT handling (#29680)

close #29670

* br: add error handling for group context cancel when restore file is corrupted (#30190)

close #30135

* executor: buildWindow cannot call typeInfer twice (#30773)

close #30402

* *: refactor encoding and uniform usages (#30288)

* lightning: optimize region split check logic (#30428)

close #30018

* br: ignore mock directory when gcov in br (#30586)

* *: forbid set tiflash replica count for a placement table (#30844)

close #30741

* execute: don't transform charset in internal sql (#30843)

close #30789

* planner: update PlanBuilder.windowSpecs when building subquery (#30878)

close #30804

* br: fix S3 backup endpoint suffix (#30530)

close #30104

* lightning: make pre-check output message clearer (#30439)

close #30395

* expression: wrap to_binary and from_binary for cast function's argument (#30706)

* executor: fix bug when using IndexMerge in transaction (#30719)

close #30685

* ddl: migrate test-infra to testify for ddl/foreign_key_test.go (#30853)

close #29103

* expression: fix wrong retType for reverse function (#30829)

close #30809

* planner: support trace topn push down (#30800)

ref #29661

* github: add issue requirement to pull request template (#30817)

close #30814

* fix merge issue

* topsql: introduce stmtstats and sql execution count (#30277)

* topsql: add pubsub datasink (#30860)

* executor: fix the incorrect untouch used in optimistic transactions (#30447)

close #30410

* expression, cmd: let crc32() support gbk (#30900)

close #30898

* server: Add uptime status var and statistics (#29790)

close #8842

* br: error log optimization (#29640)

close #27015

* planner: fix wrong collation when rewrite in condition (#30492)

close #30486

* planner: add extractor for tikv_region_peers (#30656)

* fix issue that loss table restore

* lightning: add back table empty check and add a switch config (#30887)

close #27919

* br: improve backoff unit test (#30892)

* *: add TxnManager to manage txn in session (#30574)

* *: add TxnManager to manage txn in session

* modify

* add tests

* move failpoint content to a single file

* Makefile: add `t.Parallel` check to ensure tests are run in serial (#30869)

* refactoring code

* refactoring code

* placement: remove isolationlevel (#30859)

close #30858

* planner: revise the optimize trace output (#30882)

* table: set the datum collation correctly in CastValue() (#30931)

close #30930

* *: Use TxnManager.GetTxnInfoSchema() to get the txn infoschema (#30934)

close #30933

* parser: add IsValid() to Encoding to speed up string validation for UTF-8 (#30937)

close #30936

* planner: rename pstmtPlanCacheXX to PlanCacheXX (#30909)

* table/tables: make CI TestCacheTableBasicReadAndWrite more stable (#30924)

close #30922

* restore: use new ScatterRegions API (#30899)

close #30425

* *: when placement conflicts with tiflash, cancel the job (#30945)

* Makefile,tools: make CI great again! (#30828)

close #30822

* br/pkg/membuf: remove global buffer pool (#29934)

* ddl: add format error for incorrect dict syntax in the placement rule (#30919)

close #30454

* planner: fix index merge plan when expr cannot be pushed to tikv (#30341)

close #30200

* executor: display 'show create table' and INFOSCHEMA for cached table correctly (#30951)

close #30950

* br: extend the timeout for scan region since 3 seconds is not enough (#30889)

close #30720

* planner: remove bindSQL from planCacheKey to planCacheValue (#30916)

* execution: refine precision of cast as decimal in agg func (#30805)

* *: fix data race in the tikv_client (#30964)

close #30658

* ddl: migrate test-infra to testify for ddl/db_partition_test.go (#30952)

close #28635

* planner: fix `AccessPath.TableFilters` got modified unexpectedly (#30966)

close #30965

* test: merge serial tests in ddl, infoschema, session, store, table, telemetry and types (#30874)

* executor: fix the returned field count of the prepare statement (#30981)

close #30971

* binlog: allow multiple ddl targets (#30904)

* planner: trace predicate push down  (#30902)

ref #29661

* placement: give default 2 followers for non-sugar syntax (#31000)

* flatten the json output (#30905)

Signed-off-by: yisaer <[email protected]>

Co-authored-by: Ti Chi Robot <[email protected]>

* test: control log level with environment variables (#30871)

* planner: add usage of paging copr in optimizer (#30536)

close #30578

* test: merge serial tests in cmd, planner, server, util (#31003)

* planner: change predicateColumnCollector to columnStatsUsageCollector and collect histogram-needed columns (#30671)

* executor: migrate test-infra to testify for distsql_test.go (#31023)

close #28574

* remote uncessary package errors

* reused the retry code from lightning

* refactoring retryable

* ddl: add batch create table api

Signed-off-by: xhe <[email protected]>

* ddl: add unit tests

Signed-off-by: xhe <[email protected]>

* ddl: fix fmt

Signed-off-by: xhe <[email protected]>

* br ddl code

* parallel create tables in br

* ddl: add batch create table api

Signed-off-by: xhe <[email protected]>

* ddl: add unit tests

Signed-off-by: xhe <[email protected]>

* ddl: fix fmt

Signed-off-by: xhe <[email protected]>

* ddl: typo

Co-authored-by: Arenatlx <[email protected]>

* ddl: fix tests

Signed-off-by: xhe <[email protected]>

* ddl: rename to BatchCreateTableWithInfo

Signed-off-by: xhe <[email protected]>

* ddl: trace the error

Signed-off-by: xhe <[email protected]>

* ddl: comments

Signed-off-by: xhe <[email protected]>

* ddl: cancle the job right

Signed-off-by: xhe <[email protected]>

* ddl: cancel the job right 2

Signed-off-by: xhe <[email protected]>

* ddl: report error if entry too large

Signed-off-by: xhe <[email protected]>

* ddl: report error when table is duplicated

Signed-off-by: xhe <[email protected]>

* ddl: go fmt

Signed-off-by: xhe <[email protected]>

* infoschema: improve batch memory perf

Signed-off-by: xhe <[email protected]>

* ddl: retain ID

Signed-off-by: xhe <[email protected]>

* ddl: reduce log frequency

Signed-off-by: xhe <[email protected]>

* ddl: fix tests

Signed-off-by: xhe <[email protected]>

* ddl: remove retainID from the interface

Signed-off-by: xhe <[email protected]>

* ddl: fix tests

Signed-off-by: xhe <[email protected]>

* executor: fix rebasing problem

Signed-off-by: xhe <[email protected]>

* sessionctx: enable IndexMerge by default (#30650)

close #29597

* br: Enable lint `gosec` in br (#30895)

close #30699

* planner: support 'admin flush plan cache' (#30370)

* merge from tidb batch_1

Co-authored-by: 3pointer <[email protected]>
Co-authored-by: wjHuang <[email protected]>
Co-authored-by: Lei Zhao <[email protected]>
Co-authored-by: Daniël van Eeden <[email protected]>
Co-authored-by: sylzd <[email protected]>
Co-authored-by: Jianjun Liao <[email protected]>
Co-authored-by: unconsolable <[email protected]>
Co-authored-by: tison <[email protected]>
Co-authored-by: Shenghui Wu <[email protected]>
Co-authored-by: tangenta <[email protected]>
Co-authored-by: Weizhen Wang <[email protected]>
Co-authored-by: guo-shaoge <[email protected]>
Co-authored-by: Ryan Leung <[email protected]>
Co-authored-by: xufei <[email protected]>
Co-authored-by: Morgan Tocker <[email protected]>
Co-authored-by: Zhou Kunqin <[email protected]>
Co-authored-by: 王超 <[email protected]>
Co-authored-by: TonsnakeLin <[email protected]>
Co-authored-by: Ehco <[email protected]>
Co-authored-by: Mattias Jonsson <[email protected]>
Co-authored-by: HuaiyuXu <[email protected]>
Co-authored-by: Zak Zhao <[email protected]>
Co-authored-by: WizardXiao <[email protected]>
Co-authored-by: xhe <[email protected]>
Co-authored-by: Hangjie Mo <[email protected]>
Co-authored-by: Yuanjia Zhang <[email protected]>
Co-authored-by: glorv <[email protected]>
Co-authored-by: djshow832 <[email protected]>
Co-authored-by: Chunzhu Li <[email protected]>
Co-authored-by: Song Gao <[email protected]>
Co-authored-by: Ti Chi Robot <[email protected]>
Co-authored-by: Xiaoju Wu <[email protected]>
Co-authored-by: xiongjiwei <[email protected]>
Co-authored-by: tiancaiamao <[email protected]>
Co-authored-by: Yifan Xu <[email protected]>
Co-authored-by: JmPotato <[email protected]>
Co-authored-by: Zach <[email protected]>
Co-authored-by: bb7133 <[email protected]>
Co-authored-by: lvtu <[email protected]>
Co-authored-by: crazycs <[email protected]>
Co-authored-by: znhh6018 <[email protected]>
Co-authored-by: eddie lin <[email protected]>
Co-authored-by: dongjunduo <[email protected]>
Co-authored-by: Zhenchi <[email protected]>
Co-authored-by: wangggong <[email protected]>
Co-authored-by: zhangjinpeng1987 <[email protected]>
Co-authored-by: Jack Yu <[email protected]>
Co-authored-by: Arenatlx <[email protected]>
Co-authored-by: Yiding Cui <[email protected]>
Co-authored-by: Chengpeng Yan <[email protected]>
Co-authored-by: bestwoody <[email protected]>
Co-authored-by: Calvin Neo <[email protected]>
Co-authored-by: Lynn <[email protected]>
Co-authored-by: Zhuhe Fang <[email protected]>
Co-authored-by: Mini256 <[email protected]>
Co-authored-by: Xiang Zhang <[email protected]>
Co-authored-by: Yexiang Zhang <[email protected]>
Co-authored-by: cfzjywxk <[email protected]>
Co-authored-by: db <[email protected]>
Co-authored-by: 山岚 <[email protected]>
Co-authored-by: Yujie Xia <[email protected]>
Co-authored-by: Yilong Li <[email protected]>
Co-authored-by: tuuuuuu <[email protected]>
Co-authored-by: qupeng <[email protected]>
Co-authored-by: you06 <[email protected]>
  • Loading branch information
Show file tree
Hide file tree
Showing 26 changed files with 607 additions and 132 deletions.
9 changes: 8 additions & 1 deletion .golangci_br.yml
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ linters:
- exhaustivestruct
- exhaustive
- godot
- gosec
- errorlint
- wrapcheck
- gomoddirectives
Expand Down Expand Up @@ -81,3 +80,11 @@ linters-settings:

issues:
exclude-rules:
- path: br/tests/
linters:
- gosec
- errcheck
- path: _test\.go
linters:
- gosec

4 changes: 2 additions & 2 deletions br/pkg/lightning/backend/kv/sql2kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,15 +79,15 @@ func NewTableKVEncoder(tbl table.Table, options *SessionOptions) (Encoder, error
for _, col := range cols {
if mysql.HasPriKeyFlag(col.Flag) {
incrementalBits := autoRandomIncrementBits(col, int(meta.AutoRandomBits))
autoRandomBits := rand.New(rand.NewSource(options.AutoRandomSeed)).Int63n(1<<meta.AutoRandomBits) << incrementalBits
autoRandomBits := rand.New(rand.NewSource(options.AutoRandomSeed)).Int63n(1<<meta.AutoRandomBits) << incrementalBits // nolint:gosec
autoIDFn = func(id int64) int64 {
return autoRandomBits | id
}
break
}
}
} else if meta.ShardRowIDBits > 0 {
rd := rand.New(rand.NewSource(options.AutoRandomSeed))
rd := rand.New(rand.NewSource(options.AutoRandomSeed)) // nolint:gosec
mask := int64(1)<<meta.ShardRowIDBits - 1
shift := autoid.RowIDBitLength - meta.ShardRowIDBits - 1
autoIDFn = func(id int64) int64 {
Expand Down
6 changes: 4 additions & 2 deletions br/pkg/lightning/checkpoints/checkpoints.go
Original file line number Diff line number Diff line change
Expand Up @@ -990,7 +990,7 @@ func (cpdb *FileCheckpointsDB) save() error {
// because `os.WriteFile` is not atomic, directly write into it may reset the file
// to an empty file if write is not finished.
tmpPath := cpdb.path + ".tmp"
if err := os.WriteFile(tmpPath, serialized, 0o644); err != nil {
if err := os.WriteFile(tmpPath, serialized, 0o644); err != nil { // nolint:gosec
return errors.Trace(err)
}
if err := os.Rename(tmpPath, cpdb.path); err != nil {
Expand Down Expand Up @@ -1301,6 +1301,8 @@ func (cpdb *MySQLCheckpointsDB) GetLocalStoringTables(ctx context.Context) (map[
// 1. table status is earlier than CheckpointStatusIndexImported, and
// 2. engine status is earlier than CheckpointStatusImported, and
// 3. chunk has been read

// nolint:gosec
query := fmt.Sprintf(`
SELECT DISTINCT t.table_name, c.engine_id
FROM %s.%s t, %s.%s c, %s.%s e
Expand Down Expand Up @@ -1386,7 +1388,7 @@ func (cpdb *MySQLCheckpointsDB) DestroyErrorCheckpoint(ctx context.Context, tabl
colName = columnTableName
aliasedColName = "t.table_name"
}

// nolint:gosec
selectQuery := fmt.Sprintf(`
SELECT
t.table_name,
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/lightning/common/security.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ func ToTLSConfig(caPath, certPath, keyPath string) (*tls.Config, error) {
return nil, errors.New("failed to append ca certs")
}

return &tls.Config{
return &tls.Config{ // nolint:gosec
Certificates: certificates,
RootCAs: certPool,
NextProtos: []string{"h2", "http/1.1"}, // specify `h2` to let Go use HTTP/2.
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/lightning/lightning.go
Original file line number Diff line number Diff line change
Expand Up @@ -789,7 +789,7 @@ func CleanupMetas(ctx context.Context, cfg *config.Config, tableName string) err
func UnsafeCloseEngine(ctx context.Context, importer backend.Backend, engine string) (*backend.ClosedEngine, error) {
if index := strings.LastIndexByte(engine, ':'); index >= 0 {
tableName := engine[:index]
engineID, err := strconv.Atoi(engine[index+1:])
engineID, err := strconv.Atoi(engine[index+1:]) // nolint:gosec
if err != nil {
return nil, errors.Trace(err)
}
Expand Down
11 changes: 6 additions & 5 deletions br/pkg/lightning/restore/meta_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ func (m *dbTableMetaMgr) AllocTableRowIDs(ctx context.Context, rawRowIDMax int64
}
needAutoID := common.TableHasAutoRowID(m.tr.tableInfo.Core) || m.tr.tableInfo.Core.GetAutoIncrementColInfo() != nil || m.tr.tableInfo.Core.ContainsAutoRandomBits()
err = exec.Transact(ctx, "init table allocator base", func(ctx context.Context, tx *sql.Tx) error {
query := fmt.Sprintf("SELECT task_id, row_id_base, row_id_max, total_kvs_base, total_bytes_base, checksum_base, status from %s WHERE table_id = ? FOR UPDATE", m.tableName)
query := fmt.Sprintf("SELECT task_id, row_id_base, row_id_max, total_kvs_base, total_bytes_base, checksum_base, status from %s WHERE table_id = ? FOR UPDATE", m.tableName) // nolint:gosec
rows, err := tx.QueryContext(ctx, query, m.tr.tableInfo.ID)
if err != nil {
return errors.Trace(err)
Expand Down Expand Up @@ -381,6 +381,7 @@ func (m *dbTableMetaMgr) CheckAndUpdateLocalChecksum(ctx context.Context, checks
needChecksum = true
needRemoteDupe = true
err = exec.Transact(ctx, "checksum pre-check", func(ctx context.Context, tx *sql.Tx) error {
// nolint:gosec
query := fmt.Sprintf("SELECT task_id, total_kvs_base, total_bytes_base, checksum_base, total_kvs, total_bytes, checksum, status, has_duplicates from %s WHERE table_id = ? FOR UPDATE", m.tableName)
rows, err := tx.QueryContext(ctx, query, m.tr.tableInfo.ID)
if err != nil {
Expand Down Expand Up @@ -593,7 +594,7 @@ func (m *dbTaskMetaMgr) CheckTaskExist(ctx context.Context) (bool, error) {
// avoid override existing metadata if the meta is already inserted.
exist := false
err := exec.Transact(ctx, "check whether this task has started before", func(ctx context.Context, tx *sql.Tx) error {
query := fmt.Sprintf("SELECT task_id from %s WHERE task_id = %d", m.tableName, m.taskID)
query := fmt.Sprintf("SELECT task_id from %s WHERE task_id = %d", m.tableName, m.taskID) // nolint:gosec
rows, err := tx.QueryContext(ctx, query)
if err != nil {
return errors.Annotate(err, "fetch task meta failed")
Expand Down Expand Up @@ -635,7 +636,7 @@ func (m *dbTaskMetaMgr) CheckTasksExclusively(ctx context.Context, action func(t
return errors.Annotate(err, "enable pessimistic transaction failed")
}
return exec.Transact(ctx, "check tasks exclusively", func(ctx context.Context, tx *sql.Tx) error {
query := fmt.Sprintf("SELECT task_id, pd_cfgs, status, state, source_bytes, cluster_avail from %s FOR UPDATE", m.tableName)
query := fmt.Sprintf("SELECT task_id, pd_cfgs, status, state, source_bytes, cluster_avail from %s FOR UPDATE", m.tableName) // nolint:gosec
rows, err := tx.QueryContext(ctx, query)
if err != nil {
return errors.Annotate(err, "fetch task metas failed")
Expand Down Expand Up @@ -695,7 +696,7 @@ func (m *dbTaskMetaMgr) CheckAndPausePdSchedulers(ctx context.Context) (pdutil.U
paused := false
var pausedCfg storedCfgs
err = exec.Transact(ctx, "check and pause schedulers", func(ctx context.Context, tx *sql.Tx) error {
query := fmt.Sprintf("SELECT task_id, pd_cfgs, status, state from %s FOR UPDATE", m.tableName)
query := fmt.Sprintf("SELECT task_id, pd_cfgs, status, state from %s FOR UPDATE", m.tableName) // nolint:gosec
rows, err := tx.QueryContext(ctx, query)
if err != nil {
return errors.Annotate(err, "fetch task meta failed")
Expand Down Expand Up @@ -821,7 +822,7 @@ func (m *dbTaskMetaMgr) CheckAndFinishRestore(ctx context.Context, finished bool
switchBack := true
allFinished := finished
err = exec.Transact(ctx, "check and finish schedulers", func(ctx context.Context, tx *sql.Tx) error {
query := fmt.Sprintf("SELECT task_id, status, state from %s FOR UPDATE", m.tableName)
query := fmt.Sprintf("SELECT task_id, status, state from %s FOR UPDATE", m.tableName) // nolint:gosec
rows, err := tx.QueryContext(ctx, query)
if err != nil {
return errors.Annotate(err, "fetch task meta failed")
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/mock/mock_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -207,7 +207,7 @@ func waitUntilServerOnline(addr string, statusPort uint) string {
// connect http status
statusURL := fmt.Sprintf("http://127.0.0.1:%d/status", statusPort)
for retry = 0; retry < retryTime; retry++ {
resp, err := http.Get(statusURL) // nolint:noctx
resp, err := http.Get(statusURL) // nolint:noctx,gosec
if err == nil {
// Ignore errors.
_, _ = io.ReadAll(resp.Body)
Expand Down
1 change: 1 addition & 0 deletions br/pkg/storage/hdfs.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ func dfsCommand(args ...string) (*exec.Cmd, error) {
}
cmd = append(cmd, bin, "dfs")
cmd = append(cmd, args...)
//nolint:gosec
return exec.Command(cmd[0], cmd[1:]...), nil
}

Expand Down
2 changes: 1 addition & 1 deletion br/pkg/utils/pprof.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ import (
// #nosec
// register HTTP handler for /debug/pprof
"net/http"
_ "net/http/pprof"
_ "net/http/pprof" // nolint:gosec

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
Expand Down
22 changes: 13 additions & 9 deletions cmd/explaintest/r/explain_indexmerge.result
Original file line number Diff line number Diff line change
Expand Up @@ -6,19 +6,23 @@ create index td on t (d);
load stats 's/explain_indexmerge_stats_t.json';
explain format = 'brief' select * from t where a < 50 or b < 50;
id estRows task access object operator info
TableReader 98.00 root data:Selection
└─Selection 98.00 cop[tikv] or(lt(test.t.a, 50), lt(test.t.b, 50))
└─TableFullScan 5000000.00 cop[tikv] table:t keep order:false
IndexMerge 98.00 root
├─TableRangeScan(Build) 49.00 cop[tikv] table:t range:[-inf,50), keep order:false
├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tb(b) range:[-inf,50), keep order:false
└─TableRowIDScan(Probe) 98.00 cop[tikv] table:t keep order:false
explain format = 'brief' select * from t where (a < 50 or b < 50) and f > 100;
id estRows task access object operator info
TableReader 98.00 root data:Selection
└─Selection 98.00 cop[tikv] gt(test.t.f, 100), or(lt(test.t.a, 50), lt(test.t.b, 50))
└─TableFullScan 5000000.00 cop[tikv] table:t keep order:false
IndexMerge 98.00 root
├─TableRangeScan(Build) 49.00 cop[tikv] table:t range:[-inf,50), keep order:false
├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tb(b) range:[-inf,50), keep order:false
└─Selection(Probe) 98.00 cop[tikv] gt(test.t.f, 100)
└─TableRowIDScan 98.00 cop[tikv] table:t keep order:false
explain format = 'brief' select * from t where b < 50 or c < 50;
id estRows task access object operator info
TableReader 98.00 root data:Selection
└─Selection 98.00 cop[tikv] or(lt(test.t.b, 50), lt(test.t.c, 50))
└─TableFullScan 5000000.00 cop[tikv] table:t keep order:false
IndexMerge 98.00 root
├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tb(b) range:[-inf,50), keep order:false
├─IndexRangeScan(Build) 49.00 cop[tikv] table:t, index:tc(c) range:[-inf,50), keep order:false
└─TableRowIDScan(Probe) 98.00 cop[tikv] table:t keep order:false
set session tidb_enable_index_merge = on;
explain format = 'brief' select * from t where a < 50 or b < 50;
id estRows task access object operator info
Expand Down
19 changes: 19 additions & 0 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/telemetry"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/dbterror"
"github.com/pingcap/tidb/util/domainutil"
Expand Down Expand Up @@ -89,6 +90,7 @@ type Domain struct {
cancel context.CancelFunc
indexUsageSyncLease time.Duration
planReplayer *planReplayer
expiredTimeStamp4PC types.Time

serverID uint64
serverIDSession *concurrency.Session
Expand Down Expand Up @@ -335,6 +337,22 @@ func (do *Domain) GetSnapshotMeta(startTS uint64) (*meta.Meta, error) {
return meta.NewSnapshotMeta(snapshot), nil
}

// ExpiredTimeStamp4PC gets expiredTimeStamp4PC from domain.
func (do *Domain) ExpiredTimeStamp4PC() types.Time {
do.m.Lock()
defer do.m.Unlock()

return do.expiredTimeStamp4PC
}

// SetExpiredTimeStamp4PC sets the expiredTimeStamp4PC from domain.
func (do *Domain) SetExpiredTimeStamp4PC(time types.Time) {
do.m.Lock()
defer do.m.Unlock()

do.expiredTimeStamp4PC = time
}

// DDL gets DDL from domain.
func (do *Domain) DDL() ddl.DDL {
return do.ddl
Expand Down Expand Up @@ -712,6 +730,7 @@ func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duratio
planReplayer: &planReplayer{planReplayerGCLease: planReplayerGCLease},
onClose: onClose,
renewLeaseCh: make(chan func(), 10),
expiredTimeStamp4PC: types.NewTime(types.ZeroCoreTime, mysql.TypeTimestamp, types.DefaultFsp),
}

do.SchemaValidator = NewSchemaValidator(ddlLease, do)
Expand Down
35 changes: 34 additions & 1 deletion executor/simple.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import (
"github.com/pingcap/tidb/privilege"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
Expand Down Expand Up @@ -160,7 +161,7 @@ func (e *SimpleExec) Next(ctx context.Context, req *chunk.Chunk) (err error) {
case *ast.ShutdownStmt:
err = e.executeShutdown(x)
case *ast.AdminStmt:
err = e.executeAdminReloadStatistics(x)
err = e.executeAdmin(x)
}
e.done = true
return err
Expand Down Expand Up @@ -1659,6 +1660,16 @@ func asyncDelayShutdown(p *os.Process, delay time.Duration) {
}
}

func (e *SimpleExec) executeAdmin(s *ast.AdminStmt) error {
switch s.Tp {
case ast.AdminReloadStatistics:
return e.executeAdminReloadStatistics(s)
case ast.AdminFlushPlanCache:
return e.executeAdminFlushPlanCache(s)
}
return nil
}

func (e *SimpleExec) executeAdminReloadStatistics(s *ast.AdminStmt) error {
if s.Tp != ast.AdminReloadStatistics {
return errors.New("This AdminStmt is not ADMIN RELOAD STATS_EXTENDED")
Expand All @@ -1668,3 +1679,25 @@ func (e *SimpleExec) executeAdminReloadStatistics(s *ast.AdminStmt) error {
}
return domain.GetDomain(e.ctx).StatsHandle().ReloadExtendedStatistics()
}

func (e *SimpleExec) executeAdminFlushPlanCache(s *ast.AdminStmt) error {
if s.Tp != ast.AdminFlushPlanCache {
return errors.New("This AdminStmt is not ADMIN FLUSH PLAN_CACHE")
}
if s.StatementScope == ast.StatementScopeGlobal {
return errors.New("Do not support the 'admin flush global scope.'")
}
if !plannercore.PreparedPlanCacheEnabled() {
e.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("The plan cache is disable. So there no need to flush the plan cache"))
return nil
}
now := types.NewTime(types.FromGoTime(time.Now().In(e.ctx.GetSessionVars().StmtCtx.TimeZone)), mysql.TypeTimestamp, 3)
e.ctx.GetSessionVars().LastUpdateTime4PC = now
e.ctx.PreparedPlanCache().DeleteAll()
if s.StatementScope == ast.StatementScopeInstance {
// Record the timestamp. When other sessions want to use the plan cache,
// it will check the timestamp first to decide whether the plan cache should be flushed.
domain.GetDomain(e.ctx).SetExpiredTimeStamp4PC(now)
}
return nil
}
11 changes: 11 additions & 0 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -294,6 +294,16 @@ func (e *Execute) OptimizePreparedPlan(ctx context.Context, sctx sessionctx.Cont
}
prepared.SchemaVersion = is.SchemaMetaVersion()
}
// If the lastUpdateTime less than expiredTimeStamp4PC,
// it means other sessions have executed 'admin flush instance plan_cache'.
// So we need to clear the current session's plan cache.
// And update lastUpdateTime to the newest one.
expiredTimeStamp4PC := domain.GetDomain(sctx).ExpiredTimeStamp4PC()
if prepared.UseCache && expiredTimeStamp4PC.Compare(vars.LastUpdateTime4PC) > 0 {
sctx.PreparedPlanCache().DeleteAll()
prepared.CachedPlan = nil
vars.LastUpdateTime4PC = expiredTimeStamp4PC
}
err = e.getPhysicalPlan(ctx, sctx, is, preparedObj)
if err != nil {
return err
Expand Down Expand Up @@ -401,6 +411,7 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context,
}
stmtCtx.UseCache = prepared.UseCache

var bindSQL string
if prepared.UseCache {
bindSQL = GetBindSQL4PlanCache(sctx, prepared.Stmt)
cacheKey = NewPlanCacheKey(sctx.GetSessionVars(), e.ExecID, prepared.SchemaVersion)
Expand Down
4 changes: 4 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2203,6 +2203,10 @@ func (s *testIntegrationSuite) TestOptimizeHintOnPartitionTable(c *C) {
partition p1 values less than(11),
partition p2 values less than(16));`)
tk.MustExec(`insert into t values (1,1,"1"), (2,2,"2"), (8,8,"8"), (11,11,"11"), (15,15,"15")`)
tk.MustExec("set @@tidb_enable_index_merge = off")
defer func() {
tk.MustExec("set @@tidb_enable_index_merge = on")
}()

// Create virtual tiflash replica info.
dom := domain.GetDomain(tk.Se)
Expand Down
2 changes: 2 additions & 0 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1380,6 +1380,8 @@ func (b *PlanBuilder) buildAdmin(ctx context.Context, as *ast.AdminStmt) (Plan,
return &AdminResetTelemetryID{}, nil
case ast.AdminReloadStatistics:
return &Simple{Statement: as}, nil
case ast.AdminFlushPlanCache:
return &Simple{Statement: as}, nil
default:
return nil, ErrUnsupportedType.GenWithStack("Unsupported ast.AdminStmt(%T) for buildAdmin", as)
}
Expand Down
Loading

0 comments on commit 591a7d0

Please sign in to comment.