Skip to content

Commit

Permalink
Add stats to KillUnusedSegments coordinator duty (apache#14782)
Browse files Browse the repository at this point in the history
### Description

Added the following metrics, which are calculated from the `KillUnusedSegments` coordinatorDuty

`"killTask/availableSlot/count"`: calculates the number remaining task slots available for auto kill
`"killTask/maxSlot/count"`: calculates the maximum number of tasks available for auto kill
`"killTask/task/count"`: calculates the number of tasks submitted by auto kill. 

#### Release note
NEW: metrics added for auto kill

`"killTask/availableSlot/count"`: calculates the number remaining task slots available for auto kill
`"killTask/maxSlot/count"`: calculates the maximum number of tasks available for auto kill
`"killTask/task/count"`: calculates the number of tasks submitted by auto kill.
  • Loading branch information
zachjsh authored Aug 10, 2023
1 parent 23306c4 commit 82d82df
Show file tree
Hide file tree
Showing 7 changed files with 284 additions and 185 deletions.
3 changes: 3 additions & 0 deletions docs/operations/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -320,6 +320,9 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina
|`compact/task/count`|Number of tasks issued in the auto compaction run.| |Varies|
|`compactTask/maxSlot/count`|Maximum number of task slots available for auto compaction tasks in the auto compaction run.| |Varies|
|`compactTask/availableSlot/count`|Number of available task slots that can be used for auto compaction tasks in the auto compaction run. This is the max number of task slots minus any currently running compaction tasks.| |Varies|
|`killTask/availableSlot/count`| Number of available task slots that can be used for auto kill tasks in the auto kill run. This is the max number of task slots minus any currently running auto kill tasks. | |Varies|
|`killTask/maxSlot/count`| Maximum number of task slots available for auto kill tasks in the auto kill run. | |Varies|
|`kill/task/count`| Number of tasks issued in the auto kill run. | |Varies|
|`segment/waitCompact/bytes`|Total bytes of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction).|`dataSource`|Varies|
|`segment/waitCompact/count`|Total number of segments of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction).|`dataSource`|Varies|
|`interval/waitCompact/count`|Total number of intervals of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction).|`dataSource`|Varies|
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -175,5 +175,9 @@
"namespace/cache/numEntries" : { "dimensions" : [], "type" : "gauge" },
"namespace/cache/heapSizeInBytes" : { "dimensions" : [], "type" : "gauge" },

"service/heartbeat" : { "dimensions" : ["leader"], "type" : "count" }
"service/heartbeat" : { "dimensions" : ["leader"], "type" : "count" },

"killTask/availableSlot/count" : { "dimensions" : [], "type" : "count" },
"killTask/maxSlot/count" : { "dimensions" : [], "type" : "count" },
"killTask/task/count" : { "dimensions" : [], "type" : "count" }
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.inject.Inject;
import org.apache.druid.client.indexing.ClientCompactionIOConfig;
import org.apache.druid.client.indexing.ClientCompactionIntervalSpec;
Expand All @@ -32,7 +33,6 @@
import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig;
import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec;
import org.apache.druid.client.indexing.ClientTaskQuery;
import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo;
import org.apache.druid.client.indexing.TaskPayloadResponse;
import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.common.utils.IdUtils;
Expand All @@ -41,11 +41,8 @@
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.granularity.GranularityType;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.rpc.HttpResponseException;
import org.apache.druid.rpc.indexing.OverlordClient;
import org.apache.druid.server.coordinator.AutoCompactionSnapshot;
import org.apache.druid.server.coordinator.CompactionStatistics;
Expand All @@ -59,17 +56,14 @@
import org.apache.druid.server.coordinator.stats.Stats;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentTimeline;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.joda.time.Interval;

import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.stream.Collectors;
Expand All @@ -87,6 +81,9 @@ public class CompactSegments implements CoordinatorCustomDuty

private static final Logger LOG = new Logger(CompactSegments.class);

private static final Predicate<TaskStatusPlus> IS_COMPACTION_TASK =
status -> null != status && COMPACTION_TASK_TYPE.equals(status.getType());

private final CompactionSegmentSearchPolicy policy;
private final boolean skipLockedIntervals;
private final OverlordClient overlordClient;
Expand Down Expand Up @@ -152,9 +149,10 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)

// Fetch currently running compaction tasks
int busyCompactionTaskSlots = 0;
final CloseableIterator<TaskStatusPlus> activeTasks =
FutureUtils.getUnchecked(overlordClient.taskStatuses(null, null, 0), true);
final List<TaskStatusPlus> compactionTasks = filterNonCompactionTasks(activeTasks);
final List<TaskStatusPlus> compactionTasks = CoordinatorDutyUtils.getNumActiveTaskSlots(
overlordClient,
IS_COMPACTION_TASK
);
for (TaskStatusPlus status : compactionTasks) {
final TaskPayloadResponse response =
FutureUtils.getUnchecked(overlordClient.taskPayload(status.getId()), true);
Expand Down Expand Up @@ -336,62 +334,9 @@ private static boolean useRangePartitions(ClientCompactionTaskQueryTuningConfig
return tuningConfig.getPartitionsSpec() instanceof DimensionRangePartitionsSpec;
}

private static List<TaskStatusPlus> filterNonCompactionTasks(CloseableIterator<TaskStatusPlus> taskStatuses)
{
final List<TaskStatusPlus> retVal = new ArrayList<>();

try (final Closer closer = Closer.create()) {
closer.register(taskStatuses);
while (taskStatuses.hasNext()) {
final TaskStatusPlus status = taskStatuses.next();

// taskType can be null if middleManagers are running with an older version. Here, we consevatively regard
// the tasks of the unknown taskType as the compactionTask. This is because it's important to not run
// compactionTasks more than the configured limit at any time which might impact to the ingestion
// performance.
if (status.getType() == null || COMPACTION_TASK_TYPE.equals(status.getType())) {
retVal.add(status);
}
}
}
catch (IOException e) {
throw new RuntimeException(e);
}

return retVal;
}

private int getCompactionTaskCapacity(CoordinatorCompactionConfig dynamicConfig)
{
int totalWorkerCapacity;
try {
final IndexingTotalWorkerCapacityInfo workerCapacityInfo =
FutureUtils.get(overlordClient.getTotalWorkerCapacity(), true);

if (dynamicConfig.isUseAutoScaleSlots() && workerCapacityInfo.getMaximumCapacityWithAutoScale() > 0) {
totalWorkerCapacity = workerCapacityInfo.getMaximumCapacityWithAutoScale();
} else {
totalWorkerCapacity = workerCapacityInfo.getCurrentClusterCapacity();
}
}
catch (ExecutionException e) {
// Call to getTotalWorkerCapacity may fail during a rolling upgrade: API was added in 0.23.0.
if (e.getCause() instanceof HttpResponseException
&& ((HttpResponseException) e.getCause()).getResponse().getStatus().equals(HttpResponseStatus.NOT_FOUND)) {
LOG.noStackTrace().warn(e, "Call to getTotalWorkerCapacity failed. Falling back to getWorkers.");
totalWorkerCapacity =
FutureUtils.getUnchecked(overlordClient.getWorkers(), true)
.stream()
.mapToInt(worker -> worker.getWorker().getCapacity())
.sum();
} else {
throw new RuntimeException(e.getCause());
}
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
int totalWorkerCapacity = CoordinatorDutyUtils.getTotalWorkerCapacity(overlordClient);

return Math.min(
(int) (totalWorkerCapacity * dynamicConfig.getCompactionTaskSlotRatio()),
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,129 @@
/*
* 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.coordinator.duty;

import com.google.common.base.Predicate;
import org.apache.druid.client.indexing.IndexingTotalWorkerCapacityInfo;
import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.indexer.TaskStatusPlus;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.rpc.HttpResponseException;
import org.apache.druid.rpc.indexing.OverlordClient;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;

import javax.annotation.Nonnull;

import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.ExecutionException;

/**
* utilty methods that are useful for coordinator duties
*/
public class CoordinatorDutyUtils
{

private static final Logger LOG = new Logger(CoordinatorDutyUtils.class);

/**
* Returns the total worker capacity in the cluster, including autoscaling, if enabled.
*
* @param overlordClient The overlord client used to get worker capacity info.
*
* @return the total worker capacity in the cluster, including autoscaling, if enabled.
*/
public static int getTotalWorkerCapacity(@Nonnull final OverlordClient overlordClient)
{
int totalWorkerCapacity;
try {
final IndexingTotalWorkerCapacityInfo workerCapacityInfo =
FutureUtils.get(overlordClient.getTotalWorkerCapacity(), true);
totalWorkerCapacity = workerCapacityInfo.getMaximumCapacityWithAutoScale();
if (totalWorkerCapacity < 0) {
totalWorkerCapacity = workerCapacityInfo.getCurrentClusterCapacity();
}
}
catch (ExecutionException e) {
// Call to getTotalWorkerCapacity may fail during a rolling upgrade: API was added in 0.23.0.
if (e.getCause() instanceof HttpResponseException
&& ((HttpResponseException) e.getCause()).getResponse().getStatus().equals(HttpResponseStatus.NOT_FOUND)) {
LOG.noStackTrace().warn(e, "Call to getTotalWorkerCapacity failed. Falling back to getWorkers.");
totalWorkerCapacity =
FutureUtils.getUnchecked(overlordClient.getWorkers(), true)
.stream()
.mapToInt(worker -> worker.getWorker().getCapacity())
.sum();
} else {
throw new RuntimeException(e.getCause());
}
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}

return totalWorkerCapacity;
}

/**
* Return the number of active tasks that match the task predicate provided. The number of active tasks returned
* may be an overestimate, as tasks that return status's with null types will be conservatively counted to match the
* predicate provided.
*
* @param overlordClient The overlord client to use to retrieve the list of active tasks.
* @param taskPredicate The predicate to match against the list of retreived task status.
* This predicate will never be called with a null task status.
*
* @return the number of active tasks that match the task predicate provided
*/
public static List<TaskStatusPlus> getNumActiveTaskSlots(
@Nonnull final OverlordClient overlordClient,
final Predicate<TaskStatusPlus> taskPredicate
)
{
final CloseableIterator<TaskStatusPlus> activeTasks =
FutureUtils.getUnchecked(overlordClient.taskStatuses(null, null, 0), true);
// Fetch currently running tasks that match the predicate
List<TaskStatusPlus> taskStatuses = new ArrayList<>();

try (final Closer closer = Closer.create()) {
closer.register(activeTasks);
while (activeTasks.hasNext()) {
final TaskStatusPlus status = activeTasks.next();

// taskType can be null if middleManagers are running with an older version. Here, we consevatively regard
// the tasks of the unknown taskType as the killTask. This is because it's important to not run
// killTasks more than the configured limit at any time which might impact to the ingestion
// performance.
if (null != status && (null == status.getType() || (taskPredicate.apply(status)))) {
taskStatuses.add(status);
}
}
}
catch (IOException e) {
throw new RuntimeException(e);
}

return taskStatuses;
}
}
Loading

0 comments on commit 82d82df

Please sign in to comment.