-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Add TaskCountStatsMonitor to monitor task count stats #6657
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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 | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Does
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm, |
||
| { | ||
| private static final EmittingLogger log = new EmittingLogger(TaskMaster.class); | ||
|
|
||
|
|
@@ -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 |
|---|---|---|
| @@ -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 | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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(); | ||
| } |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good, done.