planner, gcworker, executor: remove unused code (#64496)

close pingcap/tidb#61967
This commit is contained in:
Lynn
2025-11-19 18:57:44 +08:00
committed by GitHub
parent 17945b9a23
commit a51c19c1ff
5 changed files with 0 additions and 88 deletions

View File

@ -2937,10 +2937,8 @@ func checkDuplicateForUniqueIndex(ctx context.Context, t table.Table, reorgInfo
// TaskKeyBuilder is used to build task key for the backfill job.
type TaskKeyBuilder struct {
keyspace string
multiSchemaSeq int32
mergeTempIdx bool
jobID int64
}
// NewTaskKeyBuilder creates a new TaskKeyBuilder.

View File

@ -17,7 +17,6 @@ package importer
import (
"context"
"fmt"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/tidb/br/pkg/storage"
@ -32,10 +31,6 @@ import (
"github.com/pingcap/tidb/pkg/util/sqlexec"
)
const (
etcdDialTimeout = 5 * time.Second
)
// GetEtcdClient returns an etcd client.
// exported for testing.
var GetEtcdClient = store.NewEtcdCli

View File

@ -3669,16 +3669,6 @@ func (b *PlanBuilder) addAliasName(ctx context.Context, selectStmt *ast.SelectSt
return resultList, nil
}
func (b *PlanBuilder) pushHintWithoutTableWarning(hint *ast.TableOptimizerHint) {
var sb strings.Builder
ctx := format.NewRestoreCtx(0, &sb)
if err := hint.Restore(ctx); err != nil {
return
}
b.ctx.GetSessionVars().StmtCtx.SetHintWarning(
fmt.Sprintf("Hint %s is inapplicable. Please specify the table names in the arguments.", sb.String()))
}
func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLevel int) {
hints = b.hintProcessor.GetCurrentStmtHints(hints, currentLevel)
sessionVars := b.ctx.GetSessionVars()

View File

@ -190,11 +190,6 @@ func (s *ScalarSubQueryExpr) Decorrelate(*expression.Schema) expression.Expressi
return s
}
// resolveIndices implements the Expression interface.
func (*ScalarSubQueryExpr) resolveIndices(*expression.Schema) error {
return nil
}
// ResolveIndices implements the Expression interface.
func (s *ScalarSubQueryExpr) ResolveIndices(_ *expression.Schema) (expression.Expression, error) {
return s, nil
@ -205,11 +200,6 @@ func (s *ScalarSubQueryExpr) ResolveIndicesByVirtualExpr(_ expression.EvalContex
return s, false
}
// resolveIndicesByVirtualExpr implements the Expression interface.
func (*ScalarSubQueryExpr) resolveIndicesByVirtualExpr(_ expression.EvalContext, _ *expression.Schema) bool {
return false
}
// RemapColumn implements the Expression interface.
func (s *ScalarSubQueryExpr) RemapColumn(_ map[int64]*expression.Column) (expression.Expression, error) {
return s, nil

View File

@ -15,7 +15,6 @@
package gcworker
import (
"bytes"
"context"
"encoding/hex"
"fmt"
@ -1143,20 +1142,6 @@ func (w *GCWorker) getStoresForGC(ctx context.Context) ([]*metapb.Store, error)
return upStores, nil
}
func (w *GCWorker) getStoresMapForGC(ctx context.Context) (map[uint64]*metapb.Store, error) {
stores, err := w.getStoresForGC(ctx)
if err != nil {
return nil, err
}
storesMap := make(map[uint64]*metapb.Store, len(stores))
for _, store := range stores {
storesMap[store.Id] = store
}
return storesMap, nil
}
func (w *GCWorker) loadGCConcurrencyWithDefault() (int, error) {
str, err := w.loadValueFromSysTable(gcConcurrencyKey)
if err != nil {
@ -1409,52 +1394,6 @@ func (w *GCWorker) broadcastGCSafePoint(ctx context.Context, gcSafePoint uint64)
return nil
}
func (w *GCWorker) doGCForRange(ctx context.Context, startKey []byte, endKey []byte, safePoint uint64) (rangetask.TaskStat, error) {
var stat rangetask.TaskStat
defer func() {
metrics.GCActionRegionResultCounter.WithLabelValues("success").Add(float64(stat.CompletedRegions))
metrics.GCActionRegionResultCounter.WithLabelValues("fail").Add(float64(stat.FailedRegions))
}()
key := startKey
for {
bo := tikv.NewBackofferWithVars(ctx, gcOneRegionMaxBackoff, nil)
loc, err := w.tikvStore.GetRegionCache().LocateKey(bo, key)
if err != nil {
return stat, errors.Trace(err)
}
var regionErr *errorpb.Error
regionErr, err = w.doGCForRegion(bo, safePoint, loc.Region)
// we check regionErr here first, because we know 'regionErr' and 'err' should not return together, to keep it to
// make the process correct.
if regionErr != nil {
err = bo.Backoff(tikv.BoRegionMiss(), errors.New(regionErr.String()))
if err == nil {
continue
}
}
if err != nil {
logutil.BgLogger().Warn("[gc worker]",
zap.String("uuid", w.uuid),
zap.String("gc for range", fmt.Sprintf("[%d, %d)", startKey, endKey)),
zap.Uint64("safePoint", safePoint),
zap.Error(err))
stat.FailedRegions++
} else {
stat.CompletedRegions++
}
key = loc.EndKey
if len(key) == 0 || bytes.Compare(key, endKey) >= 0 {
break
}
}
return stat, nil
}
// doGCForRegion used for gc for region.
// these two errors should not return together, for more, see the func 'doGC'
func (w *GCWorker) doGCForRegion(bo *tikv.Backoffer, safePoint uint64, region tikv.RegionVerID) (*errorpb.Error, error) {