Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ public void handle(final ITaskStateAction taskStateAction,
// The task instance is not active, means it is already finished.
return;
}
org.apache.dolphinscheduler.server.master.metrics.TaskMetrics.incTaskInstanceByState("timeout");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
org.apache.dolphinscheduler.server.master.metrics.TaskMetrics.incTaskInstanceByState("timeout");
TaskMetrics.incTaskInstanceByState("timeout");

final String taskName = taskExecutionRunnable.getName();
final TaskTimeoutStrategy timeoutNotifyStrategy = taskTimeoutLifecycleEvent.getTimeoutStrategy();
if (timeoutNotifyStrategy == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import org.apache.dolphinscheduler.server.master.engine.task.runnable.ITaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowTopologyLogicalTransitionWithTaskFinishLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;

import org.apache.commons.lang3.StringUtils;

Expand Down Expand Up @@ -109,6 +110,7 @@ public void onFatalEvent(final IWorkflowExecutionRunnable workflowExecutionRunna

if (taskExecutionRunnable.isTaskInstanceCanRetry()) {
taskExecutionRunnable.getWorkflowEventBus().publish(TaskRetryLifecycleEvent.of(taskExecutionRunnable));
TaskMetrics.incTaskInstanceByState("retry");
return;
}

Expand All @@ -131,6 +133,7 @@ private void persistentTaskInstanceFatalEventToDB(final ITaskExecutionRunnable t
taskInstance.setState(TaskExecutionStatus.FAILURE);
taskInstance.setEndTime(taskFatalEvent.getEndTime());
taskInstanceDao.updateById(taskInstance);
TaskMetrics.incTaskInstanceByState("fail");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
TaskMetrics.incTaskInstanceByState("fail");
TaskMetrics.incTaskInstanceByState(TaskExecutionStatus.FAILURE);

}

@Override
Expand All @@ -141,6 +144,7 @@ public void onDispatchedEvent(final IWorkflowExecutionRunnable workflowExecution
taskInstance.setState(DISPATCH);
taskInstance.setHost(taskDispatchedEvent.getExecutorHost());
taskInstanceDao.updateById(taskInstance);
TaskMetrics.incTaskInstanceByState("dispatch");
}

@Override
Expand Down Expand Up @@ -196,7 +200,7 @@ private void persistentTaskInstanceKilledEventToDB(final ITaskExecutionRunnable
taskInstance.setState(TaskExecutionStatus.KILL);
taskInstance.setEndTime(taskKilledEvent.getEndTime());
taskInstanceDao.updateById(taskInstance);

TaskMetrics.incTaskInstanceByState("kill");
}

@Override
Expand All @@ -208,6 +212,7 @@ public void onFailedEvent(final IWorkflowExecutionRunnable workflowExecutionRunn

if (taskExecutionRunnable.isTaskInstanceCanRetry()) {
taskExecutionRunnable.getWorkflowEventBus().publish(TaskRetryLifecycleEvent.of(taskExecutionRunnable));
TaskMetrics.incTaskInstanceByState("retry");
return;
}
// If all successors are condition tasks, then the task will not be marked as failure.
Expand All @@ -229,6 +234,7 @@ private void persistentTaskInstanceFailedEventToDB(final ITaskExecutionRunnable
taskInstance.setState(TaskExecutionStatus.FAILURE);
taskInstance.setEndTime(taskFailedEvent.getEndTime());
taskInstanceDao.updateById(taskInstance);
TaskMetrics.incTaskInstanceByState("fail");
}

@Override
Expand Down Expand Up @@ -259,6 +265,7 @@ protected void persistentTaskInstanceSuccessEventToDB(final ITaskExecutionRunnab
JSONUtils.toJsonString(taskSuccessEvent.getVarPool()));
taskInstance.setVarPool(VarPoolUtils.serializeVarPool(finalVarPool));
taskInstanceDao.updateById(taskInstance);
TaskMetrics.incTaskInstanceByState("success");
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.server.master.engine.command.ICommandHandler;
import org.apache.dolphinscheduler.server.master.engine.exceptions.CommandDuplicateHandleException;
import org.apache.dolphinscheduler.server.master.metrics.WorkflowInstanceMetrics;

import java.util.List;

Expand Down Expand Up @@ -52,8 +53,11 @@ public class WorkflowExecutionRunnableFactory {
*/
@Transactional
public IWorkflowExecutionRunnable createWorkflowExecuteRunnable(Command command) {
long startTime = System.currentTimeMillis();
deleteCommandOrThrow(command);
return doCreateWorkflowExecutionRunnable(command);
IWorkflowExecutionRunnable workflowExecutionRunnable = doCreateWorkflowExecutionRunnable(command);
WorkflowInstanceMetrics.recordWorkflowInstanceGenerateTime(System.currentTimeMillis() - startTime);
return workflowExecutionRunnable;
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowTopologyLogicalTransitionWithTaskFinishLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.policy.IWorkflowFailureStrategy;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.metrics.WorkflowInstanceMetrics;
import org.apache.dolphinscheduler.server.master.utils.WorkflowInstanceUtils;
import org.apache.dolphinscheduler.service.alert.WorkflowAlertManager;

Expand Down Expand Up @@ -172,6 +173,9 @@ protected void transformWorkflowInstanceState(final IWorkflowExecutionRunnable w
workflowInstanceDao.updateById(workflowInstance);
log.info("Success set WorkflowExecuteRunnable: {} state from: {} to {}",
workflowInstance.getName(), originState.name(), targetState.name());
WorkflowInstanceMetrics.incWorkflowInstanceByStateAndWorkflowDefinitionCode(
targetState,
String.valueOf(workflowInstance.getWorkflowDefinitionCode()));
} catch (Exception ex) {
workflowInstance.setState(originState);
throw ex;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,39 +50,12 @@ public class TaskMetrics {

}

private final Counter taskDispatchCounter =
Counter.builder("ds.task.dispatch.count")
.description("Task dispatch count")
.register(Metrics.globalRegistry);

private final Counter taskDispatchFailCounter =
Counter.builder("ds.task.dispatch.failure.count")
.description("Task dispatch failures count, retried ones included")
.register(Metrics.globalRegistry);

private final Counter taskDispatchErrorCounter =
Counter.builder("ds.task.dispatch.error.count")
.description("Number of errors during task dispatch")
.register(Metrics.globalRegistry);

public synchronized void registerTaskPrepared(Supplier<Number> consumer) {
Gauge.builder("ds.task.prepared", consumer)
.description("Task prepared count")
.register(Metrics.globalRegistry);
}

public void incTaskDispatchFailed(int failedCount) {
taskDispatchFailCounter.increment(failedCount);
}

public void incTaskDispatchError() {
taskDispatchErrorCounter.increment();
}

public void incTaskDispatch() {
taskDispatchCounter.increment();
}

public void incTaskInstanceByState(final String state) {
if (taskInstanceCounters.get(state) == null) {
return;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.dolphinscheduler.server.master.metrics;

import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;

import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
Expand All @@ -36,7 +38,7 @@
public class WorkflowInstanceMetrics {

private final Set<String> workflowInstanceStates = ImmutableSet.of(
"submit", "timeout", "finish", "failover", "success", "fail", "stop");
"submit", "timeout", "finish", "failover", "success", "fail", "stop", "pause");

static {
for (final String state : workflowInstanceStates) {
Expand Down Expand Up @@ -78,18 +80,34 @@ public synchronized void registerWorkflowInstanceResubmitGauge(Supplier<Number>
.register(Metrics.globalRegistry);
}

public void incWorkflowInstanceByStateAndWorkflowDefinitionCode(final String state,
final String workflowDefinitionCode) {
// When tags need to be determined from local context,
// you have no choice but to construct or lookup the Meter inside your method body.
// The lookup cost is just a single hash lookup, so it is acceptable for most use cases.
public void incWorkflowInstanceByStateAndWorkflowDefinitionCode(WorkflowExecutionStatus state,
String workflowDefinitionCode) {
Metrics.globalRegistry.counter(
"ds.workflow.instance.count",
"state", state,
"state", getMetricState(state),
"workflow.definition.code", workflowDefinitionCode)
.increment();
}

private String getMetricState(WorkflowExecutionStatus state) {
switch (state) {
case SUBMITTED_SUCCESS:
return "submit";
case FAILURE:
return "fail";
case SUCCESS:
return "success";
case STOP:
return "stop";
case PAUSE:
return "pause";
case FAILOVER:
return "failover";
default:
return state.name().toLowerCase();
}
}

public void cleanUpWorkflowInstanceCountMetricsByDefinitionCode(final Long workflowDefinitionCode) {
for (final String state : workflowInstanceStates) {
final Counter counter = Metrics.globalRegistry.counter(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.dolphinscheduler.server.master.metrics;

import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;

import java.util.Arrays;
import java.util.List;

import org.junit.jupiter.api.Test;

import io.micrometer.core.instrument.Counter;
import io.micrometer.core.instrument.Metrics;

class TaskMetricsTest {

@Test
void testIncTaskInstanceByState_validStates() {
List<String> validStates = Arrays.asList(
"submit", "timeout", "finish", "failover", "retry", "dispatch", "success", "kill", "fail", "stop");

for (String state : validStates) {
Counter counter = Metrics.globalRegistry.find("ds.task.instance.count")
.tag("state", state)
.counter();
assertNotNull(counter, "Counter should exist for state: " + state);
double before = counter.count();
TaskMetrics.incTaskInstanceByState(state);
assertEquals(before + 1, counter.count(), 0.001,
"Counter should be incremented for state: " + state);
}
}

@Test
void testIncTaskInstanceByState_invalidState() {
TaskMetrics.incTaskInstanceByState("nonexistent_state");
Counter counter = Metrics.globalRegistry.find("ds.task.instance.count")
.tag("state", "nonexistent_state")
.counter();
assertTrue(counter == null || counter.count() == 0,
"Counter should not exist or be zero for invalid state");
}

@Test
void testIncTaskInstanceByState_multipleIncrements() {
Counter counter = Metrics.globalRegistry.find("ds.task.instance.count")
.tag("state", "submit")
.counter();
assertNotNull(counter);
double before = counter.count();

TaskMetrics.incTaskInstanceByState("submit");
TaskMetrics.incTaskInstanceByState("submit");
TaskMetrics.incTaskInstanceByState("submit");

assertEquals(before + 3, counter.count(), 0.001,
"Counter should be incremented by 3 after three calls");
}

@Test
void testRegisterTaskPrepared() {
TaskMetrics.registerTaskPrepared(() -> 5);
assertNotNull(Metrics.globalRegistry.find("ds.task.prepared").gauge(),
"Task prepared gauge should be registered");
}

}
Loading
Loading