Skip to content
Merged
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 @@ -25,7 +25,7 @@
import java.util.ArrayList;
import java.util.List;

class StubServiceEmitter extends ServiceEmitter
public class StubServiceEmitter extends ServiceEmitter
{
private List<Event> events = new ArrayList<>();

Expand Down
5 changes: 5 additions & 0 deletions docs/content/operations/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,11 @@ Note: If the JVM does not support CPU time measurement for the current thread, i
|`segment/added/bytes`|Size in bytes of new segments created.|dataSource, taskId, taskType, interval.|Varies.|
|`segment/moved/bytes`|Size in bytes of segments moved/archived via the Move Task.|dataSource, taskId, taskType, interval.|Varies.|
|`segment/nuked/bytes`|Size in bytes of segments deleted via the Kill Task.|dataSource, taskId, taskType, interval.|Varies.|
|`task/success/count`|Number of successful tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.|dataSource.|Varies.|
|`task/failed/count`|Number of failed tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.|dataSource.|Varies.|
|`task/running/count`|Number of current running tasks. This metric is only available if the TaskCountStatsMonitor module is included.|dataSource.|Varies.|
|`task/pending/count`|Number of current pending tasks. This metric is only available if the TaskCountStatsMonitor module is included.|dataSource.|Varies.|
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Can we also add a metric for waiting tasks? It will be useful too.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Good, done.

|`task/waiting/count`|Number of current waiting tasks. This metric is only available if the TaskCountStatsMonitor module is included.|dataSource.|Varies.|

## Coordination

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,14 +41,16 @@
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.server.DruidNode;
import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig;
import org.apache.druid.server.metrics.TaskCountStatsProvider;

import java.util.Map;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantLock;

/**
* Encapsulates the indexer leadership lifecycle.
*/
public class TaskMaster
public class TaskMaster implements TaskCountStatsProvider
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Does TaskQueue also need to be TaskCountStatsProvider?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Hmm, TaskCountStatsProvider is bind to TaskMaster, there is no need to let TaskQueue be TaskCountStatsProvider.

{
private static final EmittingLogger log = new EmittingLogger(TaskMaster.class);

Expand Down Expand Up @@ -265,4 +267,59 @@ public Optional<SupervisorManager> getSupervisorManager()
return Optional.absent();
}
}

@Override
public Map<String, Long> getSuccessfulTaskCount()
{
Optional<TaskQueue> taskQueue = getTaskQueue();
if (taskQueue.isPresent()) {
return taskQueue.get().getSuccessfulTaskCount();
} else {
return null;
}
}

@Override
public Map<String, Long> getFailedTaskCount()
{
Optional<TaskQueue> taskQueue = getTaskQueue();
if (taskQueue.isPresent()) {
return taskQueue.get().getFailedTaskCount();
} else {
return null;
}
}

@Override
public Map<String, Long> getRunningTaskCount()
{
Optional<TaskQueue> taskQueue = getTaskQueue();
if (taskQueue.isPresent()) {
return taskQueue.get().getRunningTaskCount();
} else {
return null;
}
}

@Override
public Map<String, Long> getPendingTaskCount()
{
Optional<TaskQueue> taskQueue = getTaskQueue();
if (taskQueue.isPresent()) {
return taskQueue.get().getPendingTaskCount();
} else {
return null;
}
}

@Override
public Map<String, Long> getWaitingTaskCount()
{
Optional<TaskQueue> taskQueue = getTaskQueue();
if (taskQueue.isPresent()) {
return taskQueue.get().getWaitingTaskCount();
} else {
return null;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -53,12 +53,15 @@
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;

/**
* Interface between task producers and the task runner.
Expand Down Expand Up @@ -100,6 +103,11 @@ public class TaskQueue

private static final EmittingLogger log = new EmittingLogger(TaskQueue.class);

private final Map<String, AtomicLong> totalSuccessfulTaskCount = new ConcurrentHashMap<>();
private final Map<String, AtomicLong> totalFailedTaskCount = new ConcurrentHashMap<>();
private Map<String, Long> prevTotalSuccessfulTaskCount = new HashMap<>();
private Map<String, Long> prevTotalFailedTaskCount = new HashMap<>();

@Inject
public TaskQueue(
TaskQueueConfig config,
Expand Down Expand Up @@ -510,6 +518,14 @@ private void handleStatus(final TaskStatus status)
task,
status.getDuration()
);

if (status.isSuccess()) {
totalSuccessfulTaskCount.computeIfAbsent(task.getDataSource(), k -> new AtomicLong())
.incrementAndGet();
} else {
totalFailedTaskCount.computeIfAbsent(task.getDataSource(), k -> new AtomicLong())
.incrementAndGet();
}
}
}
catch (Exception e) {
Expand Down Expand Up @@ -586,4 +602,70 @@ private static Map<String, Task> toTaskIDMap(List<Task> taskList)
return rv;
}

private Map<String, Long> getDeltaValues(Map<String, Long> total, Map<String, Long> prev)
{
return total.entrySet()
.stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue() - prev.getOrDefault(e.getKey(), 0L)));
}

public Map<String, Long> getSuccessfulTaskCount()
{
Map<String, Long> total = totalSuccessfulTaskCount.entrySet()
.stream()
.collect(Collectors.toMap(
Map.Entry::getKey,
e -> e.getValue().get()
));
Map<String, Long> delta = getDeltaValues(total, prevTotalSuccessfulTaskCount);
prevTotalSuccessfulTaskCount = total;
return delta;
}

public Map<String, Long> getFailedTaskCount()
{
Map<String, Long> total = totalFailedTaskCount.entrySet()
.stream()
.collect(Collectors.toMap(
Map.Entry::getKey,
e -> e.getValue().get()
));
Map<String, Long> delta = getDeltaValues(total, prevTotalFailedTaskCount);
prevTotalFailedTaskCount = total;
return delta;
}

public Map<String, Long> getRunningTaskCount()
{
Map<String, String> taskDatasources = tasks.stream().collect(Collectors.toMap(Task::getId, Task::getDataSource));
return taskRunner.getRunningTasks()
.stream()
.collect(Collectors.toMap(
e -> taskDatasources.getOrDefault(e.getTaskId(), ""),
e -> 1L,
Long::sum
));
}

public Map<String, Long> getPendingTaskCount()
{
Map<String, String> taskDatasources = tasks.stream().collect(Collectors.toMap(Task::getId, Task::getDataSource));
return taskRunner.getPendingTasks()
.stream()
.collect(Collectors.toMap(
e -> taskDatasources.getOrDefault(e.getTaskId(), ""),
e -> 1L,
Long::sum
));
}

public Map<String, Long> getWaitingTaskCount()
{
Set<String> runnerKnownTaskIds = taskRunner.getKnownTasks()
.stream()
.map(TaskRunnerWorkItem::getTaskId)
.collect(Collectors.toSet());
return tasks.stream().filter(task -> !runnerKnownTaskIds.contains(task.getId()))
.collect(Collectors.toMap(Task::getDataSource, task -> 1L, Long::sum));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* 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.druid.server.metrics;

import com.google.inject.Inject;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.java.util.metrics.AbstractMonitor;

import java.util.Map;

public class TaskCountStatsMonitor extends AbstractMonitor
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Would you please add a unit test? I think it should verify the values emitted from this monitor.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Done.

{
private final TaskCountStatsProvider statsProvider;

@Inject
public TaskCountStatsMonitor(
TaskCountStatsProvider statsProvider
)
{
this.statsProvider = statsProvider;
}

@Override
public boolean doMonitor(ServiceEmitter emitter)
{
emit(emitter, "task/success/count", statsProvider.getSuccessfulTaskCount());
emit(emitter, "task/failed/count", statsProvider.getFailedTaskCount());
emit(emitter, "task/running/count", statsProvider.getRunningTaskCount());
emit(emitter, "task/pending/count", statsProvider.getPendingTaskCount());
emit(emitter, "task/waiting/count", statsProvider.getWaitingTaskCount());
return true;
}

private void emit(ServiceEmitter emitter, String key, Map<String, Long> counts)
{
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
if (counts != null) {
counts.forEach((k, v) -> {
builder.setDimension("dataSource", k);
emitter.emit(builder.build(key, v));
});
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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.druid.server.metrics;

import java.util.Map;

public interface TaskCountStatsProvider
{
/**
* Return the number of successful tasks for each datasource during emission period.
*/
Map<String, Long> getSuccessfulTaskCount();

/**
* Return the number of failed tasks for each datasource during emission period.
*/
Map<String, Long> getFailedTaskCount();

/**
* Return the number of current running tasks for each datasource.
*/
Map<String, Long> getRunningTaskCount();

/**
* Return the number of current pending tasks for each datasource.
*/
Map<String, Long> getPendingTaskCount();

/**
* Return the number of current waiting tasks for each datasource.
*/
Map<String, Long> getWaitingTaskCount();
}
Loading