planner, gcworker, executor: remove unused code (#64496)
close pingcap/tidb#61967
This commit is contained in:
@ -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.
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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) {
|
||||
|
||||
Reference in New Issue
Block a user