disttask: refactor metrics, collect metrics in scheduler manager (#50634)
close pingcap/tidb#49615
This commit is contained in:
@ -323,6 +323,21 @@ func (mr *MockTaskManagerMockRecorder) GetAllNodes(arg0 any) *gomock.Call {
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetAllNodes", reflect.TypeOf((*MockTaskManager)(nil).GetAllNodes), arg0)
|
||||
}
|
||||
|
||||
// GetAllSubtasks mocks base method.
|
||||
func (m *MockTaskManager) GetAllSubtasks(arg0 context.Context) ([]*proto.Subtask, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "GetAllSubtasks", arg0)
|
||||
ret0, _ := ret[0].([]*proto.Subtask)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// GetAllSubtasks indicates an expected call of GetAllSubtasks.
|
||||
func (mr *MockTaskManagerMockRecorder) GetAllSubtasks(arg0 any) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetAllSubtasks", reflect.TypeOf((*MockTaskManager)(nil).GetAllSubtasks), arg0)
|
||||
}
|
||||
|
||||
// GetAllSubtasksByStepAndState mocks base method.
|
||||
func (m *MockTaskManager) GetAllSubtasksByStepAndState(arg0 context.Context, arg1 int64, arg2 proto.Step, arg3 proto.SubtaskState) ([]*proto.Subtask, error) {
|
||||
m.ctrl.T.Helper()
|
||||
|
||||
@ -51,6 +51,16 @@ const (
|
||||
SubtaskStatePaused SubtaskState = "paused"
|
||||
)
|
||||
|
||||
// AllSubtaskStates is all subtask state.
|
||||
var AllSubtaskStates = []SubtaskState{
|
||||
SubtaskStatePending,
|
||||
SubtaskStateRunning,
|
||||
SubtaskStateSucceed,
|
||||
SubtaskStateFailed,
|
||||
SubtaskStateCanceled,
|
||||
SubtaskStatePaused,
|
||||
}
|
||||
|
||||
type (
|
||||
// SubtaskState is the state of subtask.
|
||||
SubtaskState string
|
||||
|
||||
@ -4,6 +4,7 @@ go_library(
|
||||
name = "scheduler",
|
||||
srcs = [
|
||||
"balancer.go",
|
||||
"collector.go",
|
||||
"interface.go",
|
||||
"nodes.go",
|
||||
"scheduler.go",
|
||||
@ -32,6 +33,7 @@ go_library(
|
||||
"@com_github_pingcap_errors//:errors",
|
||||
"@com_github_pingcap_failpoint//:failpoint",
|
||||
"@com_github_pingcap_log//:log",
|
||||
"@com_github_prometheus_client_golang//prometheus",
|
||||
"@org_uber_go_zap//:zap",
|
||||
],
|
||||
)
|
||||
|
||||
126
pkg/disttask/framework/scheduler/collector.go
Normal file
126
pkg/disttask/framework/scheduler/collector.go
Normal file
@ -0,0 +1,126 @@
|
||||
// Copyright 2024 PingCAP, Inc.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package scheduler
|
||||
|
||||
import (
|
||||
"strconv"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/tidb/pkg/disttask/framework/proto"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
)
|
||||
|
||||
var subtaskCollector = newCollector()
|
||||
|
||||
func init() {
|
||||
prometheus.MustRegister(subtaskCollector)
|
||||
}
|
||||
|
||||
// Because the exec_id of a subtask may change, after all tasks
|
||||
// are successful, subtasks will be migrated from tidb_subtask_background
|
||||
// to tidb_subtask_background_history. In the above situation,
|
||||
// the built-in collector of Prometheus needs to delete the previously
|
||||
// added metrics, which is quite troublesome.
|
||||
// Therefore, a custom collector is used.
|
||||
type collector struct {
|
||||
subtaskInfo atomic.Pointer[[]*proto.Subtask]
|
||||
|
||||
subtasks *prometheus.Desc
|
||||
subtaskDuration *prometheus.Desc
|
||||
}
|
||||
|
||||
func newCollector() *collector {
|
||||
return &collector{
|
||||
subtasks: prometheus.NewDesc(
|
||||
"tidb_disttask_subtasks",
|
||||
"Number of subtasks.",
|
||||
[]string{"task_type", "task_id", "status", "exec_id"}, nil,
|
||||
),
|
||||
subtaskDuration: prometheus.NewDesc(
|
||||
"tidb_disttask_subtask_duration",
|
||||
"Duration of subtasks in different states.",
|
||||
[]string{"task_type", "task_id", "status", "subtask_id", "exec_id"}, nil,
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
// Describe implements the prometheus.Collector interface.
|
||||
func (c *collector) Describe(ch chan<- *prometheus.Desc) {
|
||||
ch <- c.subtasks
|
||||
ch <- c.subtaskDuration
|
||||
}
|
||||
|
||||
// Collect implements the prometheus.Collector interface.
|
||||
func (c *collector) Collect(ch chan<- prometheus.Metric) {
|
||||
p := c.subtaskInfo.Load()
|
||||
if p == nil {
|
||||
return
|
||||
}
|
||||
subtasks := *p
|
||||
|
||||
// taskID => execID => state => cnt
|
||||
subtaskCnt := make(map[int64]map[string]map[proto.SubtaskState]int)
|
||||
taskType := make(map[int64]proto.TaskType)
|
||||
for _, subtask := range subtasks {
|
||||
if _, ok := subtaskCnt[subtask.TaskID]; !ok {
|
||||
subtaskCnt[subtask.TaskID] = make(map[string]map[proto.SubtaskState]int)
|
||||
}
|
||||
if _, ok := subtaskCnt[subtask.TaskID][subtask.ExecID]; !ok {
|
||||
subtaskCnt[subtask.TaskID][subtask.ExecID] = make(map[proto.SubtaskState]int)
|
||||
}
|
||||
|
||||
subtaskCnt[subtask.TaskID][subtask.ExecID][subtask.State]++
|
||||
taskType[subtask.TaskID] = subtask.Type
|
||||
|
||||
c.setDistSubtaskDuration(ch, subtask)
|
||||
}
|
||||
for taskID, execIDMap := range subtaskCnt {
|
||||
for execID, stateMap := range execIDMap {
|
||||
for state, cnt := range stateMap {
|
||||
ch <- prometheus.MustNewConstMetric(c.subtasks, prometheus.GaugeValue,
|
||||
float64(cnt),
|
||||
taskType[taskID].String(),
|
||||
strconv.Itoa(int(taskID)),
|
||||
state.String(),
|
||||
execID,
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *collector) setDistSubtaskDuration(ch chan<- prometheus.Metric, subtask *proto.Subtask) {
|
||||
switch subtask.State {
|
||||
case proto.SubtaskStatePending:
|
||||
ch <- prometheus.MustNewConstMetric(c.subtaskDuration, prometheus.GaugeValue,
|
||||
time.Since(subtask.CreateTime).Seconds(),
|
||||
subtask.Type.String(),
|
||||
strconv.Itoa(int(subtask.TaskID)),
|
||||
subtask.State.String(),
|
||||
strconv.Itoa(int(subtask.ID)),
|
||||
subtask.ExecID,
|
||||
)
|
||||
case proto.SubtaskStateRunning:
|
||||
ch <- prometheus.MustNewConstMetric(c.subtaskDuration, prometheus.GaugeValue,
|
||||
time.Since(subtask.StartTime).Seconds(),
|
||||
subtask.Type.String(),
|
||||
strconv.Itoa(int(subtask.TaskID)),
|
||||
subtask.State.String(),
|
||||
strconv.Itoa(int(subtask.ID)),
|
||||
subtask.ExecID,
|
||||
)
|
||||
}
|
||||
}
|
||||
@ -30,6 +30,8 @@ type TaskManager interface {
|
||||
// The returned tasks are sorted by task order, see proto.Task, and only contains
|
||||
// some fields, see row2TaskBasic.
|
||||
GetTopUnfinishedTasks(ctx context.Context) ([]*proto.Task, error)
|
||||
// GetAllSubtasks gets all subtasks with basic columns.
|
||||
GetAllSubtasks(ctx context.Context) ([]*proto.Subtask, error)
|
||||
GetTasksInStates(ctx context.Context, states ...any) (task []*proto.Task, err error)
|
||||
GetTaskByID(ctx context.Context, taskID int64) (task *proto.Task, err error)
|
||||
GCSubtasks(ctx context.Context) error
|
||||
|
||||
@ -37,7 +37,8 @@ var (
|
||||
// defaultHistorySubtaskTableGcInterval is the interval of gc history subtask table.
|
||||
defaultHistorySubtaskTableGcInterval = 24 * time.Hour
|
||||
// DefaultCleanUpInterval is the interval of cleanup routine.
|
||||
DefaultCleanUpInterval = 10 * time.Minute
|
||||
DefaultCleanUpInterval = 10 * time.Minute
|
||||
defaultCollectMetricsInterval = 5 * time.Second
|
||||
)
|
||||
|
||||
// WaitTaskFinished is used to sync the test.
|
||||
@ -162,6 +163,7 @@ func (sm *Manager) Start() {
|
||||
sm.wg.Run(sm.scheduleTaskLoop)
|
||||
sm.wg.Run(sm.gcSubtaskHistoryTableLoop)
|
||||
sm.wg.Run(sm.cleanupTaskLoop)
|
||||
sm.wg.Run(sm.collectLoop)
|
||||
sm.wg.Run(func() {
|
||||
sm.nodeMgr.maintainLiveNodesLoop(sm.ctx, sm.taskMgr)
|
||||
})
|
||||
@ -419,3 +421,28 @@ func (sm *Manager) MockScheduler(task *proto.Task) *BaseScheduler {
|
||||
serverID: sm.serverID,
|
||||
})
|
||||
}
|
||||
|
||||
func (sm *Manager) collectLoop() {
|
||||
sm.logger.Info("collect loop start")
|
||||
ticker := time.NewTicker(defaultCollectMetricsInterval)
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-sm.ctx.Done():
|
||||
sm.logger.Info("collect loop exits")
|
||||
return
|
||||
case <-ticker.C:
|
||||
sm.collect()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (sm *Manager) collect() {
|
||||
subtasks, err := sm.taskMgr.GetAllSubtasks(sm.ctx)
|
||||
if err != nil {
|
||||
sm.logger.Warn("get all subtasks failed", zap.Error(err))
|
||||
return
|
||||
}
|
||||
|
||||
subtaskCollector.subtaskInfo.Store(&subtasks)
|
||||
}
|
||||
|
||||
@ -79,6 +79,15 @@ func row2BasicSubTask(r chunk.Row) *proto.Subtask {
|
||||
if !r.IsNull(8) {
|
||||
ordinal = int(r.GetInt64(8))
|
||||
}
|
||||
|
||||
// subtask defines start time as bigint, to ensure backward compatible,
|
||||
// we keep it that way, and we convert it here.
|
||||
var startTime time.Time
|
||||
if !r.IsNull(9) {
|
||||
ts := r.GetInt64(9)
|
||||
startTime = time.Unix(ts, 0)
|
||||
}
|
||||
|
||||
subtask := &proto.Subtask{
|
||||
ID: r.GetInt64(0),
|
||||
Step: proto.Step(r.GetInt64(1)),
|
||||
@ -89,6 +98,7 @@ func row2BasicSubTask(r chunk.Row) *proto.Subtask {
|
||||
Concurrency: int(r.GetInt64(6)),
|
||||
CreateTime: createTime,
|
||||
Ordinal: ordinal,
|
||||
StartTime: startTime,
|
||||
}
|
||||
return subtask
|
||||
}
|
||||
@ -96,18 +106,15 @@ func row2BasicSubTask(r chunk.Row) *proto.Subtask {
|
||||
// Row2SubTask converts a row to a subtask.
|
||||
func Row2SubTask(r chunk.Row) *proto.Subtask {
|
||||
subtask := row2BasicSubTask(r)
|
||||
// subtask defines start/update time as bigint, to ensure backward compatible,
|
||||
|
||||
// subtask defines update time as bigint, to ensure backward compatible,
|
||||
// we keep it that way, and we convert it here.
|
||||
var startTime, updateTime time.Time
|
||||
if !r.IsNull(9) {
|
||||
ts := r.GetInt64(9)
|
||||
startTime = time.Unix(ts, 0)
|
||||
}
|
||||
var updateTime time.Time
|
||||
if !r.IsNull(10) {
|
||||
ts := r.GetInt64(10)
|
||||
updateTime = time.Unix(ts, 0)
|
||||
}
|
||||
subtask.StartTime = startTime
|
||||
|
||||
subtask.UpdateTime = updateTime
|
||||
subtask.Meta = r.GetBytes(11)
|
||||
subtask.Summary = r.GetJSON(12).String()
|
||||
|
||||
@ -123,7 +123,7 @@ func (mgr *TaskManager) RunningSubtasksBack2Pending(ctx context.Context, subtask
|
||||
for _, subtask := range subtasks {
|
||||
_, err := sqlexec.ExecSQL(ctx, se, `
|
||||
update mysql.tidb_background_subtask
|
||||
set state = %?, state_update_time = CURRENT_TIMESTAMP()
|
||||
set state = %?, state_update_time = unix_timestamp()
|
||||
where id = %? and exec_id = %? and state = %?`,
|
||||
proto.SubtaskStatePending, subtask.ID, subtask.ExecID, proto.SubtaskStateRunning)
|
||||
if err != nil {
|
||||
|
||||
@ -42,9 +42,9 @@ const (
|
||||
TaskColumns = basicTaskColumns + `, t.start_time, t.state_update_time, t.meta, t.dispatcher_id, t.error`
|
||||
// InsertTaskColumns is the columns used in insert task.
|
||||
InsertTaskColumns = `task_key, type, state, priority, concurrency, step, meta, create_time`
|
||||
basicSubtaskColumns = `id, step, task_key, type, exec_id, state, concurrency, create_time, ordinal`
|
||||
basicSubtaskColumns = `id, step, task_key, type, exec_id, state, concurrency, create_time, ordinal, start_time`
|
||||
// SubtaskColumns is the columns for subtask.
|
||||
SubtaskColumns = basicSubtaskColumns + `, start_time, state_update_time, meta, summary`
|
||||
SubtaskColumns = basicSubtaskColumns + `, state_update_time, meta, summary`
|
||||
// InsertSubtaskColumns is the columns used in insert subtask.
|
||||
InsertSubtaskColumns = `step, task_key, exec_id, meta, state, type, concurrency, ordinal, create_time, checkpoint, summary`
|
||||
)
|
||||
@ -730,3 +730,19 @@ func (mgr *TaskManager) GetSubtasksWithHistory(ctx context.Context, taskID int64
|
||||
}
|
||||
return subtasks, nil
|
||||
}
|
||||
|
||||
// GetAllSubtasks gets all subtasks with basic columns.
|
||||
func (mgr *TaskManager) GetAllSubtasks(ctx context.Context) ([]*proto.Subtask, error) {
|
||||
rs, err := mgr.ExecuteSQLWithNewSession(ctx, `select `+basicSubtaskColumns+` from mysql.tidb_background_subtask`)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(rs) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
subtasks := make([]*proto.Subtask, 0, len(rs))
|
||||
for _, r := range rs {
|
||||
subtasks = append(subtasks, row2BasicSubTask(r))
|
||||
}
|
||||
return subtasks, nil
|
||||
}
|
||||
|
||||
@ -31,7 +31,6 @@ import (
|
||||
"github.com/pingcap/tidb/pkg/disttask/framework/storage"
|
||||
"github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/execute"
|
||||
"github.com/pingcap/tidb/pkg/domain/infosync"
|
||||
"github.com/pingcap/tidb/pkg/metrics"
|
||||
"github.com/pingcap/tidb/pkg/util"
|
||||
"github.com/pingcap/tidb/pkg/util/backoff"
|
||||
"github.com/pingcap/tidb/pkg/util/gctuner"
|
||||
@ -143,7 +142,7 @@ func (e *BaseTaskExecutor) checkBalanceSubtask(ctx context.Context) {
|
||||
continue
|
||||
}
|
||||
if !e.IsIdempotent(st) {
|
||||
e.updateSubtaskStateAndError(ctx, st, proto.SubtaskStateFailed, ErrNonIdempotentSubtask)
|
||||
e.updateSubtaskStateAndErrorImpl(ctx, st.ExecID, st.ID, proto.SubtaskStateFailed, ErrNonIdempotentSubtask)
|
||||
return
|
||||
}
|
||||
extraRunningSubtasks = append(extraRunningSubtasks, st)
|
||||
@ -316,18 +315,6 @@ func (e *BaseTaskExecutor) runStep(resource *proto.StepResource) (resErr error)
|
||||
}
|
||||
}()
|
||||
|
||||
subtasks, err := e.taskTable.GetSubtasksByExecIDAndStepAndStates(
|
||||
runStepCtx, e.id, task.ID, task.Step,
|
||||
proto.SubtaskStatePending, proto.SubtaskStateRunning)
|
||||
if err != nil {
|
||||
e.onError(err)
|
||||
return e.getError()
|
||||
}
|
||||
for _, subtask := range subtasks {
|
||||
metrics.IncDistTaskSubTaskCnt(subtask)
|
||||
metrics.StartDistTaskSubTask(subtask)
|
||||
}
|
||||
|
||||
for {
|
||||
// check if any error occurs.
|
||||
if err := e.getError(); err != nil {
|
||||
@ -352,15 +339,15 @@ func (e *BaseTaskExecutor) runStep(resource *proto.StepResource) (resErr error)
|
||||
e.logger.Info("subtask in running state and is not idempotent, fail it",
|
||||
zap.Int64("subtask-id", subtask.ID))
|
||||
e.onError(ErrNonIdempotentSubtask)
|
||||
e.updateSubtaskStateAndError(runStepCtx, subtask, proto.SubtaskStateFailed, ErrNonIdempotentSubtask)
|
||||
e.updateSubtaskStateAndErrorImpl(runStepCtx, subtask.ExecID, subtask.ID, proto.SubtaskStateFailed, ErrNonIdempotentSubtask)
|
||||
e.markErrorHandled()
|
||||
break
|
||||
}
|
||||
} else {
|
||||
// subtask.State == proto.SubtaskStatePending
|
||||
err := e.startSubtaskAndUpdateState(runStepCtx, subtask)
|
||||
err := e.startSubtask(runStepCtx, subtask.ID)
|
||||
if err != nil {
|
||||
e.logger.Warn("startSubtaskAndUpdateState meets error", zap.Error(err))
|
||||
e.logger.Warn("startSubtask meets error", zap.Error(err))
|
||||
// should ignore ErrSubtaskNotFound
|
||||
// since the err only indicate that the subtask not owned by current task executor.
|
||||
if err == storage.ErrSubtaskNotFound {
|
||||
@ -496,7 +483,7 @@ func (e *BaseTaskExecutor) onSubtaskFinished(ctx context.Context, executor execu
|
||||
return
|
||||
}
|
||||
|
||||
e.finishSubtaskAndUpdateState(ctx, subtask)
|
||||
e.finishSubtask(ctx, subtask)
|
||||
|
||||
finished = e.markSubTaskCanceledOrFailed(ctx, subtask)
|
||||
if finished {
|
||||
@ -531,7 +518,7 @@ func (e *BaseTaskExecutor) Rollback() error {
|
||||
break
|
||||
}
|
||||
|
||||
e.updateSubtaskStateAndError(e.ctx, subtask, proto.SubtaskStateCanceled, nil)
|
||||
e.updateSubtaskStateAndErrorImpl(e.ctx, subtask.ExecID, subtask.ID, proto.SubtaskStateCanceled, nil)
|
||||
if err = e.getError(); err != nil {
|
||||
return err
|
||||
}
|
||||
@ -653,18 +640,6 @@ func (e *BaseTaskExecutor) resetError() {
|
||||
e.mu.handled = false
|
||||
}
|
||||
|
||||
func (e *BaseTaskExecutor) startSubtaskAndUpdateState(ctx context.Context, subtask *proto.Subtask) error {
|
||||
err := e.startSubtask(ctx, subtask.ID)
|
||||
if err == nil {
|
||||
metrics.DecDistTaskSubTaskCnt(subtask)
|
||||
metrics.EndDistTaskSubTask(subtask)
|
||||
subtask.State = proto.SubtaskStateRunning
|
||||
metrics.IncDistTaskSubTaskCnt(subtask)
|
||||
metrics.StartDistTaskSubTask(subtask)
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (e *BaseTaskExecutor) updateSubtaskStateAndErrorImpl(ctx context.Context, execID string, subtaskID int64, state proto.SubtaskState, subTaskErr error) {
|
||||
// retry for 3+6+12+24+(30-4)*30 ~= 825s ~= 14 minutes
|
||||
backoffer := backoff.NewExponential(scheduler.RetrySQLInterval, 2, scheduler.RetrySQLMaxInterval)
|
||||
@ -708,25 +683,6 @@ func (e *BaseTaskExecutor) finishSubtask(ctx context.Context, subtask *proto.Sub
|
||||
}
|
||||
}
|
||||
|
||||
func (e *BaseTaskExecutor) updateSubtaskStateAndError(ctx context.Context, subtask *proto.Subtask, state proto.SubtaskState, subTaskErr error) {
|
||||
metrics.DecDistTaskSubTaskCnt(subtask)
|
||||
metrics.EndDistTaskSubTask(subtask)
|
||||
e.updateSubtaskStateAndErrorImpl(ctx, subtask.ExecID, subtask.ID, state, subTaskErr)
|
||||
subtask.State = state
|
||||
metrics.IncDistTaskSubTaskCnt(subtask)
|
||||
if !subtask.IsDone() {
|
||||
metrics.StartDistTaskSubTask(subtask)
|
||||
}
|
||||
}
|
||||
|
||||
func (e *BaseTaskExecutor) finishSubtaskAndUpdateState(ctx context.Context, subtask *proto.Subtask) {
|
||||
metrics.DecDistTaskSubTaskCnt(subtask)
|
||||
metrics.EndDistTaskSubTask(subtask)
|
||||
e.finishSubtask(ctx, subtask)
|
||||
subtask.State = proto.SubtaskStateSucceed
|
||||
metrics.IncDistTaskSubTaskCnt(subtask)
|
||||
}
|
||||
|
||||
// markSubTaskCanceledOrFailed check the error type and decide the subtasks' state.
|
||||
// 1. Only cancel subtasks when meet ErrCancelSubtask.
|
||||
// 2. Only fail subtasks when meet non retryable error.
|
||||
@ -736,14 +692,14 @@ func (e *BaseTaskExecutor) markSubTaskCanceledOrFailed(ctx context.Context, subt
|
||||
err := errors.Cause(err)
|
||||
if ctx.Err() != nil && context.Cause(ctx) == ErrCancelSubtask {
|
||||
e.logger.Warn("subtask canceled", zap.Error(err))
|
||||
e.updateSubtaskStateAndError(e.ctx, subtask, proto.SubtaskStateCanceled, nil)
|
||||
e.updateSubtaskStateAndErrorImpl(e.ctx, subtask.ExecID, subtask.ID, proto.SubtaskStateCanceled, nil)
|
||||
} else if e.IsRetryableError(err) {
|
||||
e.logger.Warn("meet retryable error", zap.Error(err))
|
||||
} else if common.IsContextCanceledError(err) {
|
||||
e.logger.Info("meet context canceled for gracefully shutdown", zap.Error(err))
|
||||
} else {
|
||||
e.logger.Warn("subtask failed", zap.Error(err))
|
||||
e.updateSubtaskStateAndError(e.ctx, subtask, proto.SubtaskStateFailed, err)
|
||||
e.updateSubtaskStateAndErrorImpl(e.ctx, subtask.ExecID, subtask.ID, proto.SubtaskStateFailed, err)
|
||||
}
|
||||
e.markErrorHandled()
|
||||
return true
|
||||
|
||||
@ -78,9 +78,6 @@ func TestTaskExecutorRun(t *testing.T) {
|
||||
// 3. run subtask failed
|
||||
runSubtaskErr := errors.New("run subtask error")
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}}, nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(&proto.Subtask{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}, nil)
|
||||
@ -95,9 +92,6 @@ func TestTaskExecutorRun(t *testing.T) {
|
||||
|
||||
// 4. run subtask success
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}}, nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(&proto.Subtask{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}, nil)
|
||||
@ -114,12 +108,6 @@ func TestTaskExecutorRun(t *testing.T) {
|
||||
|
||||
// 5. run subtask one by one
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(
|
||||
[]*proto.Subtask{
|
||||
{ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"},
|
||||
{ID: 2, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"},
|
||||
}, nil)
|
||||
// first round of the run loop
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(&proto.Subtask{
|
||||
@ -148,8 +136,6 @@ func TestTaskExecutorRun(t *testing.T) {
|
||||
// idempotent, so fail it.
|
||||
subtaskID := int64(2)
|
||||
theSubtask := &proto.Subtask{ID: subtaskID, Type: tp, Step: proto.StepOne, State: proto.SubtaskStateRunning, ExecID: "id"}
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{theSubtask}, nil)
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(theSubtask, nil)
|
||||
@ -163,8 +149,6 @@ func TestTaskExecutorRun(t *testing.T) {
|
||||
// run previous left subtask in running state again, but the subtask idempotent,
|
||||
// run it again.
|
||||
theSubtask = &proto.Subtask{ID: subtaskID, Type: tp, Step: proto.StepOne, State: proto.SubtaskStateRunning, ExecID: "id"}
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{theSubtask}, nil)
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
// first round of the run loop
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
@ -182,9 +166,6 @@ func TestTaskExecutorRun(t *testing.T) {
|
||||
require.True(t, ctrl.Satisfied())
|
||||
|
||||
// 6. cancel
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{
|
||||
ID: 2, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}}, nil)
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(&proto.Subtask{
|
||||
@ -198,9 +179,6 @@ func TestTaskExecutorRun(t *testing.T) {
|
||||
require.True(t, ctrl.Satisfied())
|
||||
|
||||
// 7. RunSubtask return context.Canceled
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{
|
||||
ID: 2, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}}, nil)
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(&proto.Subtask{
|
||||
@ -213,9 +191,6 @@ func TestTaskExecutorRun(t *testing.T) {
|
||||
require.True(t, ctrl.Satisfied())
|
||||
|
||||
// 8. grpc cancel
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{
|
||||
ID: 2, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}}, nil)
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(&proto.Subtask{
|
||||
@ -229,9 +204,6 @@ func TestTaskExecutorRun(t *testing.T) {
|
||||
require.True(t, ctrl.Satisfied())
|
||||
|
||||
// 9. annotate grpc cancel
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{
|
||||
ID: 2, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}}, nil)
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(&proto.Subtask{
|
||||
@ -251,9 +223,6 @@ func TestTaskExecutorRun(t *testing.T) {
|
||||
|
||||
// 10. subtask owned by other executor
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}}, nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(&proto.Subtask{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}, nil)
|
||||
@ -300,8 +269,6 @@ func TestTaskExecutorRun(t *testing.T) {
|
||||
mockSubtaskTable.EXPECT().HasSubtasksInStates(gomock.Any(), "id", task1.ID, task1.Step,
|
||||
unfinishedNormalSubtaskStates...).Return(true, nil)
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(nil, nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task1.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(nil, nil)
|
||||
mockStepExecutor.EXPECT().Cleanup(gomock.Any()).Return(nil)
|
||||
@ -406,8 +373,6 @@ func TestTaskExecutor(t *testing.T) {
|
||||
subtasks := []*proto.Subtask{
|
||||
{ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"},
|
||||
}
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", taskID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(subtasks, nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(subtasks[0], nil)
|
||||
mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), taskID, "id").Return(nil)
|
||||
@ -426,8 +391,6 @@ func TestTaskExecutor(t *testing.T) {
|
||||
|
||||
// 3. run one subtask, then task moved to history(ErrTaskNotFound).
|
||||
mockStepExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "id", taskID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(subtasks, nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(subtasks[0], nil)
|
||||
mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), taskID, "id").Return(nil)
|
||||
@ -463,8 +426,6 @@ func TestRunStepCurrentSubtaskScheduledAway(t *testing.T) {
|
||||
// mock for runStep
|
||||
mockExtension.EXPECT().GetStepExecutor(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockStepExecutor, nil)
|
||||
mockExtension.EXPECT().IsRetryableError(gomock.Any()).Return(false)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(gomock.Any(), "tidb1", task.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(subtasks, nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "tidb1", task.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(subtasks[0], nil)
|
||||
mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), task.ID, "tidb1").Return(nil)
|
||||
@ -581,47 +542,10 @@ func TestExecutorErrHandling(t *testing.T) {
|
||||
require.NoError(t, taskExecutor.RunStep(nil))
|
||||
require.True(t, ctrl.Satisfied())
|
||||
|
||||
// 5. GetSubtasksByStepAndStates meet retryable error.
|
||||
getSubtasksByExecIDAndStepAndStatesErr := errors.New("get subtasks err")
|
||||
mockExtension.EXPECT().GetStepExecutor(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockSubtaskExecutor, nil)
|
||||
mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(
|
||||
gomock.Any(),
|
||||
taskExecutor.id,
|
||||
gomock.Any(),
|
||||
proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(nil, getSubtasksByExecIDAndStepAndStatesErr)
|
||||
mockSubtaskExecutor.EXPECT().Cleanup(gomock.Any()).Return(nil)
|
||||
mockExtension.EXPECT().IsRetryableError(gomock.Any()).Return(true)
|
||||
require.NoError(t, taskExecutor.RunStep(nil))
|
||||
require.True(t, ctrl.Satisfied())
|
||||
|
||||
// 6. GetSubtasksByExecIDAndStepAndStates meet non retryable error.
|
||||
mockExtension.EXPECT().GetStepExecutor(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockSubtaskExecutor, nil)
|
||||
mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(
|
||||
gomock.Any(),
|
||||
taskExecutor.id,
|
||||
gomock.Any(),
|
||||
proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(nil, getSubtasksByExecIDAndStepAndStatesErr)
|
||||
mockSubtaskExecutor.EXPECT().Cleanup(gomock.Any()).Return(nil)
|
||||
mockExtension.EXPECT().IsRetryableError(gomock.Any()).Return(false)
|
||||
mockSubtaskTable.EXPECT().FailSubtask(taskExecutor.ctx, taskExecutor.id, gomock.Any(), getSubtasksByExecIDAndStepAndStatesErr)
|
||||
require.NoError(t, taskExecutor.RunStep(nil))
|
||||
require.True(t, ctrl.Satisfied())
|
||||
|
||||
// 7. Cleanup meet retryable error.
|
||||
// 5. Cleanup meet retryable error.
|
||||
cleanupErr := errors.New("cleanup err")
|
||||
mockExtension.EXPECT().GetStepExecutor(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockSubtaskExecutor, nil)
|
||||
mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(
|
||||
gomock.Any(),
|
||||
taskExecutor.id,
|
||||
gomock.Any(),
|
||||
proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}}, nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(&proto.Subtask{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}, nil)
|
||||
@ -636,16 +560,9 @@ func TestExecutorErrHandling(t *testing.T) {
|
||||
require.NoError(t, taskExecutor.RunStep(nil))
|
||||
require.True(t, ctrl.Satisfied())
|
||||
|
||||
// 8. Cleanup meet non retryable error.
|
||||
// 6. Cleanup meet non retryable error.
|
||||
mockExtension.EXPECT().GetStepExecutor(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockSubtaskExecutor, nil)
|
||||
mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(
|
||||
gomock.Any(),
|
||||
taskExecutor.id,
|
||||
gomock.Any(),
|
||||
proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}}, nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(&proto.Subtask{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}, nil)
|
||||
@ -661,29 +578,22 @@ func TestExecutorErrHandling(t *testing.T) {
|
||||
require.NoError(t, taskExecutor.RunStep(nil))
|
||||
require.True(t, ctrl.Satisfied())
|
||||
|
||||
// 9. runSummaryCollectLoop meet retryable error.
|
||||
// 7. runSummaryCollectLoop meet retryable error.
|
||||
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockSummaryCollectErr", "return()"))
|
||||
mockExtension.EXPECT().IsRetryableError(gomock.Any()).Return(true)
|
||||
require.NoError(t, taskExecutor.RunStep(nil))
|
||||
require.True(t, ctrl.Satisfied())
|
||||
|
||||
// 10. runSummaryCollectLoop meet non retryable error.
|
||||
// 8. runSummaryCollectLoop meet non retryable error.
|
||||
mockExtension.EXPECT().IsRetryableError(gomock.Any()).Return(false)
|
||||
mockSubtaskTable.EXPECT().FailSubtask(taskExecutor.ctx, taskExecutor.id, gomock.Any(), gomock.Any())
|
||||
require.NoError(t, taskExecutor.RunStep(nil))
|
||||
require.True(t, ctrl.Satisfied())
|
||||
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockSummaryCollectErr"))
|
||||
|
||||
// 13. subtask succeed.
|
||||
// 9. subtask succeed.
|
||||
mockExtension.EXPECT().GetStepExecutor(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockSubtaskExecutor, nil)
|
||||
mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil)
|
||||
mockSubtaskTable.EXPECT().GetSubtasksByExecIDAndStepAndStates(
|
||||
gomock.Any(),
|
||||
taskExecutor.id,
|
||||
gomock.Any(),
|
||||
proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}}, nil)
|
||||
mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", task.ID, proto.StepOne,
|
||||
unfinishedNormalSubtaskStates...).Return(&proto.Subtask{
|
||||
ID: 1, Type: tp, Step: proto.StepOne, State: proto.SubtaskStatePending, ExecID: "id"}, nil)
|
||||
|
||||
@ -16,7 +16,6 @@ package metrics
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/pingcap/tidb/pkg/disttask/framework/proto"
|
||||
@ -28,6 +27,7 @@ const (
|
||||
lblTaskType = "task_type"
|
||||
lblTaskID = "task_id"
|
||||
lblSubTaskID = "subtask_id"
|
||||
lblExecID = "exec_id"
|
||||
)
|
||||
|
||||
// status for task
|
||||
@ -45,8 +45,8 @@ var (
|
||||
DistTaskStarttimeGauge *prometheus.GaugeVec
|
||||
// DistTaskSubTaskCntGauge is the gauge of dist task subtask count.
|
||||
DistTaskSubTaskCntGauge *prometheus.GaugeVec
|
||||
// DistTaskSubTaskStartTimeGauge is the gauge of dist task subtask start time.
|
||||
DistTaskSubTaskStartTimeGauge *prometheus.GaugeVec
|
||||
// DistTaskSubTaskDurationGauge is the gauge of dist task subtask duration.
|
||||
DistTaskSubTaskDurationGauge *prometheus.GaugeVec
|
||||
)
|
||||
|
||||
// InitDistTaskMetrics initializes disttask metrics.
|
||||
@ -66,60 +66,6 @@ func InitDistTaskMetrics() {
|
||||
Name: "start_time",
|
||||
Help: "Gauge of start_time of disttask.",
|
||||
}, []string{lblTaskType, lblTaskStatus, lblTaskID})
|
||||
|
||||
DistTaskSubTaskCntGauge = NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Namespace: "tidb",
|
||||
Subsystem: "disttask",
|
||||
Name: "subtask_cnt",
|
||||
Help: "Gauge of subtask count.",
|
||||
}, []string{lblTaskType, lblTaskID, lblTaskStatus})
|
||||
|
||||
DistTaskSubTaskStartTimeGauge = NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Namespace: "tidb",
|
||||
Subsystem: "disttask",
|
||||
Name: "subtask_start_time",
|
||||
Help: "Gauge of subtask start time.",
|
||||
}, []string{lblTaskType, lblTaskID, lblTaskStatus, lblSubTaskID})
|
||||
}
|
||||
|
||||
// IncDistTaskSubTaskCnt increases the count of dist task subtask.
|
||||
func IncDistTaskSubTaskCnt(subtask *proto.Subtask) {
|
||||
DistTaskSubTaskCntGauge.WithLabelValues(
|
||||
subtask.Type.String(),
|
||||
strconv.Itoa(int(subtask.TaskID)),
|
||||
subtask.State.String(),
|
||||
).Inc()
|
||||
}
|
||||
|
||||
// DecDistTaskSubTaskCnt decreases the count of dist task subtask.
|
||||
func DecDistTaskSubTaskCnt(subtask *proto.Subtask) {
|
||||
DistTaskSubTaskCntGauge.WithLabelValues(
|
||||
subtask.Type.String(),
|
||||
strconv.Itoa(int(subtask.TaskID)),
|
||||
subtask.State.String(),
|
||||
).Dec()
|
||||
}
|
||||
|
||||
// StartDistTaskSubTask sets the start time of dist task subtask.
|
||||
func StartDistTaskSubTask(subtask *proto.Subtask) {
|
||||
DistTaskSubTaskStartTimeGauge.WithLabelValues(
|
||||
subtask.Type.String(),
|
||||
strconv.Itoa(int(subtask.TaskID)),
|
||||
subtask.State.String(),
|
||||
strconv.Itoa(int(subtask.ID)),
|
||||
).SetToCurrentTime()
|
||||
}
|
||||
|
||||
// EndDistTaskSubTask deletes the start time of dist task subtask.
|
||||
func EndDistTaskSubTask(subtask *proto.Subtask) {
|
||||
DistTaskSubTaskStartTimeGauge.DeleteLabelValues(
|
||||
subtask.Type.String(),
|
||||
strconv.Itoa(int(subtask.TaskID)),
|
||||
subtask.State.String(),
|
||||
strconv.Itoa(int(subtask.ID)),
|
||||
)
|
||||
}
|
||||
|
||||
// UpdateMetricsForAddTask update metrics when a task is added
|
||||
|
||||
@ -14584,7 +14584,7 @@
|
||||
"h": 8,
|
||||
"w": 12,
|
||||
"x": 0,
|
||||
"y": 1
|
||||
"y": 20
|
||||
},
|
||||
"hiddenSeries": false,
|
||||
"id": 322,
|
||||
@ -14624,7 +14624,7 @@
|
||||
"expr": "sum(tidb_disttask_task_status{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\"}) by (status, task_type)",
|
||||
"hide": false,
|
||||
"interval": "",
|
||||
"legendFormat": "{{task_type}}_{{status}}",
|
||||
"legendFormat": "{{task_type}}-{{status}}",
|
||||
"queryType": "randomWalk",
|
||||
"refId": "A"
|
||||
}
|
||||
@ -14685,7 +14685,7 @@
|
||||
"gridPos": {
|
||||
"h": 8,
|
||||
"w": 12,
|
||||
"x": 12,
|
||||
"x": 0,
|
||||
"y": 20
|
||||
},
|
||||
"hiddenSeries": false,
|
||||
@ -14717,17 +14717,17 @@
|
||||
"targets": [
|
||||
{
|
||||
"exemplar": true,
|
||||
"expr": "sum(tidb_disttask_subtask_cnt{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", status=~\"succeed|failed|canceled|reverted|revert_failed\"}) by (task_id, task_type)",
|
||||
"expr": "sum(tidb_disttask_subtasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", status=~\"succeed|failed|canceled|reverted|revert_failed\"}) by (task_id, task_type)",
|
||||
"interval": "",
|
||||
"legendFormat": "{{task_type}}_{{task_id}}_completed_cnt",
|
||||
"legendFormat": "{{task_type}}-task{{task_id}}-completed",
|
||||
"queryType": "randomWalk",
|
||||
"refId": "A"
|
||||
},
|
||||
{
|
||||
"exemplar": true,
|
||||
"expr": "sum(tidb_disttask_subtask_cnt{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (task_id, task_type)",
|
||||
"expr": "sum(tidb_disttask_subtasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (task_id, task_type)",
|
||||
"interval": "",
|
||||
"legendFormat": "{{task_type}}_{{task_id}}_total_cnt",
|
||||
"legendFormat": "{{task_type}}-task{{task_id}}-total",
|
||||
"queryType": "randomWalk",
|
||||
"refId": "B"
|
||||
}
|
||||
@ -14736,7 +14736,7 @@
|
||||
"timeFrom": null,
|
||||
"timeRegions": [],
|
||||
"timeShift": null,
|
||||
"title": "Dist Task Subtasks",
|
||||
"title": "Completed/Total Subtask Count",
|
||||
"tooltip": {
|
||||
"shared": true,
|
||||
"sort": 0,
|
||||
@ -14773,6 +14773,357 @@
|
||||
"alignLevel": null
|
||||
}
|
||||
},
|
||||
{
|
||||
"aliasColors": {},
|
||||
"bars": false,
|
||||
"dashLength": 10,
|
||||
"dashes": false,
|
||||
"datasource": "${DS_TEST-CLUSTER}",
|
||||
"fieldConfig": {
|
||||
"defaults": {},
|
||||
"overrides": []
|
||||
},
|
||||
"fill": 1,
|
||||
"fillGradient": 0,
|
||||
"gridPos": {
|
||||
"h": 8,
|
||||
"w": 12,
|
||||
"x": 0,
|
||||
"y": 30
|
||||
},
|
||||
"hiddenSeries": false,
|
||||
"id": 329,
|
||||
"legend": {
|
||||
"avg": false,
|
||||
"current": false,
|
||||
"max": false,
|
||||
"min": false,
|
||||
"show": true,
|
||||
"total": false,
|
||||
"values": false
|
||||
},
|
||||
"lines": true,
|
||||
"linewidth": 1,
|
||||
"nullPointMode": "null",
|
||||
"options": {
|
||||
"alertThreshold": true
|
||||
},
|
||||
"percentage": false,
|
||||
"pluginVersion": "7.5.11",
|
||||
"pointradius": 2,
|
||||
"points": false,
|
||||
"renderer": "flot",
|
||||
"seriesOverrides": [],
|
||||
"spaceLength": 10,
|
||||
"stack": false,
|
||||
"steppedLine": false,
|
||||
"targets": [
|
||||
{
|
||||
"exemplar": true,
|
||||
"expr": "sum(tidb_disttask_subtasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", status=~\"pending\"}) by (exec_id)",
|
||||
"interval": "",
|
||||
"legendFormat": "{{exec_id}}",
|
||||
"queryType": "randomWalk",
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
"thresholds": [],
|
||||
"timeFrom": null,
|
||||
"timeRegions": [],
|
||||
"timeShift": null,
|
||||
"title": "Pending Subtask Count",
|
||||
"tooltip": {
|
||||
"shared": true,
|
||||
"sort": 0,
|
||||
"value_type": "individual"
|
||||
},
|
||||
"type": "graph",
|
||||
"xaxis": {
|
||||
"buckets": null,
|
||||
"mode": "time",
|
||||
"name": null,
|
||||
"show": true,
|
||||
"values": []
|
||||
},
|
||||
"yaxes": [
|
||||
{
|
||||
"format": "short",
|
||||
"label": null,
|
||||
"logBase": 1,
|
||||
"max": null,
|
||||
"min": null,
|
||||
"show": true
|
||||
},
|
||||
{
|
||||
"format": "short",
|
||||
"label": null,
|
||||
"logBase": 1,
|
||||
"max": null,
|
||||
"min": null,
|
||||
"show": true
|
||||
}
|
||||
],
|
||||
"yaxis": {
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
}
|
||||
},
|
||||
{
|
||||
"aliasColors": {},
|
||||
"bars": false,
|
||||
"dashLength": 10,
|
||||
"dashes": false,
|
||||
"datasource": "${DS_TEST-CLUSTER}",
|
||||
"fieldConfig": {
|
||||
"defaults": {},
|
||||
"overrides": []
|
||||
},
|
||||
"fill": 1,
|
||||
"fillGradient": 0,
|
||||
"gridPos": {
|
||||
"h": 8,
|
||||
"w": 12,
|
||||
"x": 12,
|
||||
"y": 28
|
||||
},
|
||||
"hiddenSeries": false,
|
||||
"id": 330,
|
||||
"legend": {
|
||||
"avg": false,
|
||||
"current": false,
|
||||
"max": false,
|
||||
"min": false,
|
||||
"show": true,
|
||||
"total": false,
|
||||
"values": false
|
||||
},
|
||||
"lines": true,
|
||||
"linewidth": 1,
|
||||
"nullPointMode": "null",
|
||||
"options": {
|
||||
"alertThreshold": true
|
||||
},
|
||||
"percentage": false,
|
||||
"pluginVersion": "7.5.11",
|
||||
"pointradius": 2,
|
||||
"points": false,
|
||||
"renderer": "flot",
|
||||
"seriesOverrides": [],
|
||||
"spaceLength": 10,
|
||||
"stack": false,
|
||||
"steppedLine": false,
|
||||
"targets": [
|
||||
{
|
||||
"exemplar": true,
|
||||
"expr": "tidb_disttask_subtask_duration{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", status=~\"running\"}",
|
||||
"interval": "",
|
||||
"legendFormat": "{{task_type}}-task{{task_id}}-subtask{{subtask_id}}",
|
||||
"queryType": "randomWalk",
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
"thresholds": [],
|
||||
"timeFrom": null,
|
||||
"timeRegions": [],
|
||||
"timeShift": null,
|
||||
"title": "SubTask Running Duration",
|
||||
"tooltip": {
|
||||
"shared": true,
|
||||
"sort": 0,
|
||||
"value_type": "individual"
|
||||
},
|
||||
"type": "graph",
|
||||
"xaxis": {
|
||||
"buckets": null,
|
||||
"mode": "time",
|
||||
"name": null,
|
||||
"show": true,
|
||||
"values": []
|
||||
},
|
||||
"yaxes": [
|
||||
{
|
||||
"format": "s",
|
||||
"label": null,
|
||||
"logBase": 1,
|
||||
"max": null,
|
||||
"min": null,
|
||||
"show": true
|
||||
},
|
||||
{
|
||||
"format": "short",
|
||||
"label": null,
|
||||
"logBase": 1,
|
||||
"max": null,
|
||||
"min": null,
|
||||
"show": true
|
||||
}
|
||||
],
|
||||
"yaxis": {
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
}
|
||||
},
|
||||
{
|
||||
"datasource": "${DS_TEST-CLUSTER}",
|
||||
"fieldConfig": {
|
||||
"defaults": {
|
||||
"custom": {
|
||||
"align": null,
|
||||
"filterable": false
|
||||
},
|
||||
"thresholds": {
|
||||
"mode": "absolute",
|
||||
"steps": [
|
||||
{
|
||||
"color": "green",
|
||||
"value": null
|
||||
},
|
||||
{
|
||||
"color": "red",
|
||||
"value": 80
|
||||
}
|
||||
]
|
||||
},
|
||||
"mappings": [
|
||||
{
|
||||
"from": "",
|
||||
"id": 1,
|
||||
"text": "",
|
||||
"to": "",
|
||||
"type": 1,
|
||||
"value": ""
|
||||
}
|
||||
],
|
||||
"color": {
|
||||
"mode": "thresholds"
|
||||
}
|
||||
},
|
||||
"overrides": [
|
||||
{
|
||||
"matcher": {
|
||||
"id": "byName",
|
||||
"options": "status"
|
||||
},
|
||||
"properties": [
|
||||
{
|
||||
"id": "custom.width",
|
||||
"value": 84
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"matcher": {
|
||||
"id": "byName",
|
||||
"options": "exec_id"
|
||||
},
|
||||
"properties": [
|
||||
{
|
||||
"id": "custom.width",
|
||||
"value": 112
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"matcher": {
|
||||
"id": "byName",
|
||||
"options": "subtask_id"
|
||||
},
|
||||
"properties": [
|
||||
{
|
||||
"id": "custom.width",
|
||||
"value": 91
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"matcher": {
|
||||
"id": "byName",
|
||||
"options": "task_id"
|
||||
},
|
||||
"properties": [
|
||||
{
|
||||
"id": "custom.width",
|
||||
"value": 76
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"matcher": {
|
||||
"id": "byName",
|
||||
"options": "task_type"
|
||||
},
|
||||
"properties": [
|
||||
{
|
||||
"id": "custom.width",
|
||||
"value": 108
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"matcher": {
|
||||
"id": "byName",
|
||||
"options": "pending_duration"
|
||||
},
|
||||
"properties": [
|
||||
{
|
||||
"id": "unit",
|
||||
"value": "s"
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
},
|
||||
"gridPos": {
|
||||
"h": 8,
|
||||
"w": 12,
|
||||
"x": 12,
|
||||
"y": 36
|
||||
},
|
||||
"id": 331,
|
||||
"options": {
|
||||
"showHeader": true,
|
||||
"sortBy": [
|
||||
{
|
||||
"desc": true,
|
||||
"displayName": "pending_duration"
|
||||
}
|
||||
]
|
||||
},
|
||||
"pluginVersion": "7.5.11",
|
||||
"targets": [
|
||||
{
|
||||
"exemplar": true,
|
||||
"expr": "tidb_disttask_subtask_duration{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", status=\"pending\"}",
|
||||
"format": "table",
|
||||
"instant": true,
|
||||
"interval": "",
|
||||
"legendFormat": "",
|
||||
"queryType": "randomWalk",
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
"title": "Subtask Pending Duration",
|
||||
"transformations": [
|
||||
{
|
||||
"id": "organize",
|
||||
"options": {
|
||||
"excludeByName": {
|
||||
"Time": true,
|
||||
"__name__": true,
|
||||
"instance": true,
|
||||
"job": true
|
||||
},
|
||||
"indexByName": {},
|
||||
"renameByName": {
|
||||
"Time": "",
|
||||
"Value": "pending_duration",
|
||||
"__name__": ""
|
||||
}
|
||||
}
|
||||
}
|
||||
],
|
||||
"type": "table"
|
||||
},
|
||||
{
|
||||
"datasource": "${DS_TEST-CLUSTER}",
|
||||
"fieldConfig": {
|
||||
@ -14781,6 +15132,7 @@
|
||||
"mode": "palette-classic"
|
||||
},
|
||||
"mappings": [],
|
||||
"nullValueMode": "null as zero",
|
||||
"thresholds": {
|
||||
"mode": "absolute",
|
||||
"steps": [
|
||||
@ -14801,10 +15153,10 @@
|
||||
"gridPos": {
|
||||
"h": 8,
|
||||
"w": 12,
|
||||
"x": 0,
|
||||
"y": 36
|
||||
"x": 12,
|
||||
"y": 20
|
||||
},
|
||||
"id": 23763571993,
|
||||
"id": 332,
|
||||
"options": {
|
||||
"displayLabels": [],
|
||||
"legend": {
|
||||
@ -14815,7 +15167,7 @@
|
||||
"pieType": "pie",
|
||||
"reduceOptions": {
|
||||
"calcs": [
|
||||
"lastNotNull"
|
||||
"last"
|
||||
],
|
||||
"fields": "",
|
||||
"values": false
|
||||
@ -14826,14 +15178,14 @@
|
||||
"targets": [
|
||||
{
|
||||
"exemplar": true,
|
||||
"expr": "sum(tidb_disttask_subtask_cnt{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", status=~\"pending|running|reverting|revert_pending|paused\"}) by (instance)",
|
||||
"expr": "sum(tidb_disttask_subtasks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", status=~\"pending|running|reverting|revert_pending|paused\"}) by (exec_id)",
|
||||
"interval": "",
|
||||
"legendFormat": "",
|
||||
"queryType": "randomWalk",
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
"title": "Distributed Task Running Subtask Distribution on TiDB Nodes",
|
||||
"title": "Uncompleted Subtask Distribution on TiDB Nodes",
|
||||
"type": "piechart"
|
||||
}
|
||||
],
|
||||
|
||||
@ -1517,314 +1517,6 @@
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
}
|
||||
},
|
||||
{
|
||||
"aliasColors": {},
|
||||
"bars": false,
|
||||
"dashLength": 10,
|
||||
"dashes": false,
|
||||
"datasource": "${DS_TEST-CLUSTER}",
|
||||
"description": "",
|
||||
"fieldConfig": {
|
||||
"defaults": {},
|
||||
"overrides": []
|
||||
},
|
||||
"fill": 1,
|
||||
"fillGradient": 0,
|
||||
"gridPos": {
|
||||
"h": 7,
|
||||
"w": 12,
|
||||
"x": 0,
|
||||
"y": 50
|
||||
},
|
||||
"hiddenSeries": false,
|
||||
"id": 31,
|
||||
"legend": {
|
||||
"avg": false,
|
||||
"current": false,
|
||||
"max": false,
|
||||
"min": false,
|
||||
"show": true,
|
||||
"total": false,
|
||||
"values": false
|
||||
},
|
||||
"lines": true,
|
||||
"linewidth": 1,
|
||||
"nullPointMode": "null",
|
||||
"options": {
|
||||
"alertThreshold": true
|
||||
},
|
||||
"percentage": false,
|
||||
"pluginVersion": "7.5.11",
|
||||
"pointradius": 2,
|
||||
"points": false,
|
||||
"renderer": "flot",
|
||||
"seriesOverrides": [],
|
||||
"spaceLength": 10,
|
||||
"stack": false,
|
||||
"steppedLine": false,
|
||||
"targets": [
|
||||
{
|
||||
"exemplar": true,
|
||||
"expr": "time()-tidb_disttask_subtask_start_time{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", status=\"pending\"}",
|
||||
"interval": "",
|
||||
"legendFormat": "{{status}}_subtask_id_{{subtask_id}}",
|
||||
"queryType": "randomWalk",
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
"thresholds": [],
|
||||
"timeFrom": null,
|
||||
"timeRegions": [],
|
||||
"timeShift": null,
|
||||
"title": "Distributed Task SubTask Pending Duration",
|
||||
"tooltip": {
|
||||
"shared": true,
|
||||
"sort": 0,
|
||||
"value_type": "individual"
|
||||
},
|
||||
"type": "graph",
|
||||
"xaxis": {
|
||||
"buckets": null,
|
||||
"mode": "time",
|
||||
"name": null,
|
||||
"show": true,
|
||||
"values": []
|
||||
},
|
||||
"yaxes": [
|
||||
{
|
||||
"format": "s",
|
||||
"label": null,
|
||||
"logBase": 1,
|
||||
"max": null,
|
||||
"min": null,
|
||||
"show": true
|
||||
},
|
||||
{
|
||||
"format": "short",
|
||||
"label": null,
|
||||
"logBase": 1,
|
||||
"max": null,
|
||||
"min": null,
|
||||
"show": true
|
||||
}
|
||||
],
|
||||
"yaxis": {
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
},
|
||||
"scopedVars": {
|
||||
"instance": {
|
||||
"text": "127.0.0.1:10080",
|
||||
"value": "127.0.0.1:10080",
|
||||
"selected": false
|
||||
}
|
||||
}
|
||||
},
|
||||
{
|
||||
"aliasColors": {},
|
||||
"bars": false,
|
||||
"dashLength": 10,
|
||||
"dashes": false,
|
||||
"datasource": "${DS_TEST-CLUSTER}",
|
||||
"fieldConfig": {
|
||||
"defaults": {},
|
||||
"overrides": []
|
||||
},
|
||||
"fill": 1,
|
||||
"fillGradient": 0,
|
||||
"gridPos": {
|
||||
"h": 7,
|
||||
"w": 12,
|
||||
"x": 12,
|
||||
"y": 43
|
||||
},
|
||||
"hiddenSeries": false,
|
||||
"id": 32,
|
||||
"legend": {
|
||||
"avg": false,
|
||||
"current": false,
|
||||
"max": false,
|
||||
"min": false,
|
||||
"show": true,
|
||||
"total": false,
|
||||
"values": false
|
||||
},
|
||||
"lines": true,
|
||||
"linewidth": 1,
|
||||
"nullPointMode": "null",
|
||||
"options": {
|
||||
"alertThreshold": true
|
||||
},
|
||||
"percentage": false,
|
||||
"pluginVersion": "7.5.11",
|
||||
"pointradius": 2,
|
||||
"points": false,
|
||||
"renderer": "flot",
|
||||
"seriesOverrides": [],
|
||||
"spaceLength": 10,
|
||||
"stack": false,
|
||||
"steppedLine": false,
|
||||
"targets": [
|
||||
{
|
||||
"exemplar": true,
|
||||
"expr": "tidb_disttask_subtask_cnt{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", status=\"pending\"}",
|
||||
"interval": "",
|
||||
"legendFormat": "pending_subtasks",
|
||||
"queryType": "randomWalk",
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
"thresholds": [],
|
||||
"timeFrom": null,
|
||||
"timeRegions": [],
|
||||
"timeShift": null,
|
||||
"title": "Distributed Task Pending SubTask Cnt",
|
||||
"tooltip": {
|
||||
"shared": true,
|
||||
"sort": 0,
|
||||
"value_type": "individual"
|
||||
},
|
||||
"type": "graph",
|
||||
"xaxis": {
|
||||
"buckets": null,
|
||||
"mode": "time",
|
||||
"name": null,
|
||||
"show": true,
|
||||
"values": []
|
||||
},
|
||||
"yaxes": [
|
||||
{
|
||||
"format": "short",
|
||||
"label": null,
|
||||
"logBase": 1,
|
||||
"max": null,
|
||||
"min": null,
|
||||
"show": true
|
||||
},
|
||||
{
|
||||
"format": "short",
|
||||
"label": null,
|
||||
"logBase": 1,
|
||||
"max": null,
|
||||
"min": null,
|
||||
"show": true
|
||||
}
|
||||
],
|
||||
"yaxis": {
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
},
|
||||
"scopedVars": {
|
||||
"instance": {
|
||||
"text": "127.0.0.1:10080",
|
||||
"value": "127.0.0.1:10080",
|
||||
"selected": false
|
||||
}
|
||||
}
|
||||
},
|
||||
{
|
||||
"aliasColors": {},
|
||||
"bars": false,
|
||||
"dashLength": 10,
|
||||
"dashes": false,
|
||||
"datasource": "${DS_TEST-CLUSTER}",
|
||||
"description": "",
|
||||
"fieldConfig": {
|
||||
"defaults": {},
|
||||
"overrides": []
|
||||
},
|
||||
"fill": 1,
|
||||
"fillGradient": 0,
|
||||
"gridPos": {
|
||||
"h": 7,
|
||||
"w": 12,
|
||||
"x": 12,
|
||||
"y": 50
|
||||
},
|
||||
"hiddenSeries": false,
|
||||
"id": 33,
|
||||
"legend": {
|
||||
"avg": false,
|
||||
"current": false,
|
||||
"max": false,
|
||||
"min": false,
|
||||
"show": true,
|
||||
"total": false,
|
||||
"values": false
|
||||
},
|
||||
"lines": true,
|
||||
"linewidth": 1,
|
||||
"nullPointMode": "null",
|
||||
"options": {
|
||||
"alertThreshold": true
|
||||
},
|
||||
"percentage": false,
|
||||
"pluginVersion": "7.5.11",
|
||||
"pointradius": 2,
|
||||
"points": false,
|
||||
"renderer": "flot",
|
||||
"seriesOverrides": [],
|
||||
"spaceLength": 10,
|
||||
"stack": false,
|
||||
"steppedLine": false,
|
||||
"targets": [
|
||||
{
|
||||
"exemplar": true,
|
||||
"expr": "time()-tidb_disttask_subtask_start_time{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", status=\"running\"}",
|
||||
"interval": "",
|
||||
"legendFormat": "{{status}}_subtask_id_{{subtask_id}}",
|
||||
"queryType": "randomWalk",
|
||||
"refId": "A"
|
||||
}
|
||||
],
|
||||
"thresholds": [],
|
||||
"timeFrom": null,
|
||||
"timeRegions": [],
|
||||
"timeShift": null,
|
||||
"title": "Distributed Task SubTask Running Duration",
|
||||
"tooltip": {
|
||||
"shared": true,
|
||||
"sort": 0,
|
||||
"value_type": "individual"
|
||||
},
|
||||
"type": "graph",
|
||||
"xaxis": {
|
||||
"buckets": null,
|
||||
"mode": "time",
|
||||
"name": null,
|
||||
"show": true,
|
||||
"values": []
|
||||
},
|
||||
"yaxes": [
|
||||
{
|
||||
"format": "s",
|
||||
"label": null,
|
||||
"logBase": 1,
|
||||
"max": null,
|
||||
"min": null,
|
||||
"show": true
|
||||
},
|
||||
{
|
||||
"format": "short",
|
||||
"label": null,
|
||||
"logBase": 1,
|
||||
"max": null,
|
||||
"min": null,
|
||||
"show": true
|
||||
}
|
||||
],
|
||||
"yaxis": {
|
||||
"align": false,
|
||||
"alignLevel": null
|
||||
},
|
||||
"scopedVars": {
|
||||
"instance": {
|
||||
"text": "127.0.0.1:10080",
|
||||
"value": "127.0.0.1:10080",
|
||||
"selected": false
|
||||
}
|
||||
}
|
||||
}
|
||||
],
|
||||
"repeat": "instance",
|
||||
|
||||
@ -270,9 +270,6 @@ func RegisterMetrics() {
|
||||
|
||||
prometheus.MustRegister(DistTaskGauge)
|
||||
prometheus.MustRegister(DistTaskStarttimeGauge)
|
||||
prometheus.MustRegister(DistTaskSubTaskCntGauge)
|
||||
prometheus.MustRegister(DistTaskSubTaskStartTimeGauge)
|
||||
|
||||
prometheus.MustRegister(RunawayCheckerCounter)
|
||||
prometheus.MustRegister(GlobalSortWriteToCloudStorageDuration)
|
||||
prometheus.MustRegister(GlobalSortWriteToCloudStorageRate)
|
||||
|
||||
Reference in New Issue
Block a user