mirror of https://github.com/apache/druid.git
Add ability to limit the number of segments killed in kill task (#14662)
### Description Previously, the `maxSegments` configured for auto kill could be ignored if an interval of data for a given datasource had more than this number of unused segments, causing the kill task spawned with the task of deleting unused segments in that given interval of data to delete more than the `maxSegments` configured. Now each kill task spawned by the auto kill coordinator duty, will kill at most `limit` segments. This is done by adding a new config property to the `KillUnusedSegmentTask` which allows users to specify this limit.
This commit is contained in:
parent
748874405c
commit
ba957a9b97
|
@ -96,15 +96,18 @@ The available grammar is:
|
||||||
"dataSource": <task_datasource>,
|
"dataSource": <task_datasource>,
|
||||||
"interval" : <all_unused_segments_in_this_interval_will_die!>,
|
"interval" : <all_unused_segments_in_this_interval_will_die!>,
|
||||||
"context": <task context>,
|
"context": <task context>,
|
||||||
"batchSize": <optional_batch size>
|
"batchSize": <optional_batch size>,
|
||||||
|
"limit": <the maximum number of segments to delete>
|
||||||
}
|
}
|
||||||
```
|
```
|
||||||
|
|
||||||
Some of the parameters used in the task payload are further explained below:
|
Some of the parameters used in the task payload are further explained below:
|
||||||
|
|
||||||
| Parameter |Default| Explanation |
|
| Parameter | Default | Explanation |
|
||||||
|--------------|-------|--------------------------------------------------------------------------------------------------------|
|
|-------------|-----------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
|
||||||
| `batchSize` |100 | Maximum number of segments that are deleted in one kill batch. Some operations on the Overlord may get stuck while a `kill` task is in progress due to concurrency constraints (such as in `TaskLockbox`). Thus, a `kill` task splits the list of unused segments to be deleted into smaller batches to yield the Overlord resources intermittently to other task operations.|
|
| `batchSize` |100 | Maximum number of segments that are deleted in one kill batch. Some operations on the Overlord may get stuck while a `kill` task is in progress due to concurrency constraints (such as in `TaskLockbox`). Thus, a `kill` task splits the list of unused segments to be deleted into smaller batches to yield the Overlord resources intermittently to other task operations.|
|
||||||
|
| `limit` | null - no limit | Maximum number of segments for the kill task to delete.|
|
||||||
|
|
||||||
|
|
||||||
**WARNING:** The `kill` task permanently removes all information about the affected segments from the metadata store and
|
**WARNING:** The `kill` task permanently removes all information about the affected segments from the metadata store and
|
||||||
deep storage. This operation cannot be undone.
|
deep storage. This operation cannot be undone.
|
||||||
|
|
|
@ -27,6 +27,8 @@ import org.apache.druid.indexing.common.task.Task;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
public class RetrieveUnusedSegmentsAction implements TaskAction<List<DataSegment>>
|
public class RetrieveUnusedSegmentsAction implements TaskAction<List<DataSegment>>
|
||||||
|
@ -37,14 +39,19 @@ public class RetrieveUnusedSegmentsAction implements TaskAction<List<DataSegment
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
private final Interval interval;
|
private final Interval interval;
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
private final Integer limit;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public RetrieveUnusedSegmentsAction(
|
public RetrieveUnusedSegmentsAction(
|
||||||
@JsonProperty("dataSource") String dataSource,
|
@JsonProperty("dataSource") String dataSource,
|
||||||
@JsonProperty("interval") Interval interval
|
@JsonProperty("interval") Interval interval,
|
||||||
|
@JsonProperty("limit") @Nullable Integer limit
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dataSource = dataSource;
|
this.dataSource = dataSource;
|
||||||
this.interval = interval;
|
this.interval = interval;
|
||||||
|
this.limit = limit;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -59,6 +66,13 @@ public class RetrieveUnusedSegmentsAction implements TaskAction<List<DataSegment
|
||||||
return interval;
|
return interval;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@JsonProperty
|
||||||
|
public Integer getLimit()
|
||||||
|
{
|
||||||
|
return limit;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TypeReference<List<DataSegment>> getReturnTypeReference()
|
public TypeReference<List<DataSegment>> getReturnTypeReference()
|
||||||
{
|
{
|
||||||
|
@ -68,7 +82,8 @@ public class RetrieveUnusedSegmentsAction implements TaskAction<List<DataSegment
|
||||||
@Override
|
@Override
|
||||||
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox)
|
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox)
|
||||||
{
|
{
|
||||||
return toolbox.getIndexerMetadataStorageCoordinator().retrieveUnusedSegmentsForInterval(dataSource, interval);
|
return toolbox.getIndexerMetadataStorageCoordinator()
|
||||||
|
.retrieveUnusedSegmentsForInterval(dataSource, interval, limit);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -83,6 +98,7 @@ public class RetrieveUnusedSegmentsAction implements TaskAction<List<DataSegment
|
||||||
return getClass().getSimpleName() + "{" +
|
return getClass().getSimpleName() + "{" +
|
||||||
"dataSource='" + dataSource + '\'' +
|
"dataSource='" + dataSource + '\'' +
|
||||||
", interval=" + interval +
|
", interval=" + interval +
|
||||||
|
", limit=" + limit +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -79,7 +79,7 @@ public class ArchiveTask extends AbstractFixedIntervalTask
|
||||||
// List unused segments
|
// List unused segments
|
||||||
final List<DataSegment> unusedSegments = toolbox
|
final List<DataSegment> unusedSegments = toolbox
|
||||||
.getTaskActionClient()
|
.getTaskActionClient()
|
||||||
.submit(new RetrieveUnusedSegmentsAction(myLock.getDataSource(), myLock.getInterval()));
|
.submit(new RetrieveUnusedSegmentsAction(myLock.getDataSource(), myLock.getInterval(), null));
|
||||||
|
|
||||||
// Verify none of these segments have versions > lock version
|
// Verify none of these segments have versions > lock version
|
||||||
for (final DataSegment unusedSegment : unusedSegments) {
|
for (final DataSegment unusedSegment : unusedSegments) {
|
||||||
|
|
|
@ -26,8 +26,8 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Lists;
|
|
||||||
import org.apache.druid.client.indexing.ClientKillUnusedSegmentsTaskQuery;
|
import org.apache.druid.client.indexing.ClientKillUnusedSegmentsTaskQuery;
|
||||||
|
import org.apache.druid.error.InvalidInput;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
import org.apache.druid.indexing.common.TaskLock;
|
import org.apache.druid.indexing.common.TaskLock;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
|
@ -37,6 +37,7 @@ import org.apache.druid.indexing.common.actions.SegmentNukeAction;
|
||||||
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
import org.apache.druid.indexing.common.actions.TaskActionClient;
|
||||||
import org.apache.druid.indexing.common.actions.TaskLocks;
|
import org.apache.druid.indexing.common.actions.TaskLocks;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.druid.server.security.ResourceAction;
|
import org.apache.druid.server.security.ResourceAction;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
|
@ -44,6 +45,8 @@ import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nonnull;
|
import javax.annotation.Nonnull;
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
@ -61,6 +64,7 @@ import java.util.stream.Collectors;
|
||||||
*/
|
*/
|
||||||
public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
||||||
{
|
{
|
||||||
|
public static final String TYPE = "kill";
|
||||||
private static final Logger LOG = new Logger(KillUnusedSegmentsTask.class);
|
private static final Logger LOG = new Logger(KillUnusedSegmentsTask.class);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -74,14 +78,16 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
||||||
private static final int DEFAULT_SEGMENT_NUKE_BATCH_SIZE = 100;
|
private static final int DEFAULT_SEGMENT_NUKE_BATCH_SIZE = 100;
|
||||||
|
|
||||||
private final boolean markAsUnused;
|
private final boolean markAsUnused;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Split processing to try and keep each nuke operation relatively short, in the case that either
|
* Split processing to try and keep each nuke operation relatively short, in the case that either
|
||||||
* the database or the storage layer is particularly slow.
|
* the database or the storage layer is particularly slow.
|
||||||
*/
|
*/
|
||||||
private final int batchSize;
|
private final int batchSize;
|
||||||
|
@Nullable private final Integer limit;
|
||||||
|
|
||||||
// counter included primarily for testing
|
|
||||||
|
// counters included primarily for testing
|
||||||
|
private int numSegmentsKilled = 0;
|
||||||
private long numBatchesProcessed = 0;
|
private long numBatchesProcessed = 0;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
|
@ -90,8 +96,9 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
||||||
@JsonProperty("dataSource") String dataSource,
|
@JsonProperty("dataSource") String dataSource,
|
||||||
@JsonProperty("interval") Interval interval,
|
@JsonProperty("interval") Interval interval,
|
||||||
@JsonProperty("context") Map<String, Object> context,
|
@JsonProperty("context") Map<String, Object> context,
|
||||||
@JsonProperty("markAsUnused") Boolean markAsUnused,
|
@JsonProperty("markAsUnused") @Deprecated Boolean markAsUnused,
|
||||||
@JsonProperty("batchSize") Integer batchSize
|
@JsonProperty("batchSize") Integer batchSize,
|
||||||
|
@JsonProperty("limit") @Nullable Integer limit
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(
|
super(
|
||||||
|
@ -103,6 +110,19 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
||||||
this.markAsUnused = markAsUnused != null && markAsUnused;
|
this.markAsUnused = markAsUnused != null && markAsUnused;
|
||||||
this.batchSize = (batchSize != null) ? batchSize : DEFAULT_SEGMENT_NUKE_BATCH_SIZE;
|
this.batchSize = (batchSize != null) ? batchSize : DEFAULT_SEGMENT_NUKE_BATCH_SIZE;
|
||||||
Preconditions.checkArgument(this.batchSize > 0, "batchSize should be greater than zero");
|
Preconditions.checkArgument(this.batchSize > 0, "batchSize should be greater than zero");
|
||||||
|
if (null != limit && limit <= 0) {
|
||||||
|
throw InvalidInput.exception(
|
||||||
|
"limit [%d] is invalid. It must be a positive integer.",
|
||||||
|
limit
|
||||||
|
);
|
||||||
|
}
|
||||||
|
if (limit != null && markAsUnused != null && markAsUnused) {
|
||||||
|
throw InvalidInput.exception(
|
||||||
|
"limit cannot be provided with markAsUnused.",
|
||||||
|
limit
|
||||||
|
);
|
||||||
|
}
|
||||||
|
this.limit = limit;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -119,10 +139,17 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
||||||
return batchSize;
|
return batchSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@JsonProperty
|
||||||
|
public Integer getLimit()
|
||||||
|
{
|
||||||
|
return limit;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getType()
|
public String getType()
|
||||||
{
|
{
|
||||||
return "kill";
|
return TYPE;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nonnull
|
@Nonnull
|
||||||
|
@ -140,6 +167,13 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
||||||
return numBatchesProcessed;
|
return numBatchesProcessed;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
@VisibleForTesting
|
||||||
|
long getNumSegmentsKilled()
|
||||||
|
{
|
||||||
|
return numSegmentsKilled;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public TaskStatus runTask(TaskToolbox toolbox) throws Exception
|
public TaskStatus runTask(TaskToolbox toolbox) throws Exception
|
||||||
{
|
{
|
||||||
|
@ -153,27 +187,29 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
||||||
}
|
}
|
||||||
|
|
||||||
// List unused segments
|
// List unused segments
|
||||||
final List<DataSegment> allUnusedSegments = toolbox
|
int nextBatchSize = computeNextBatchSize(numSegmentsKilled);
|
||||||
.getTaskActionClient()
|
@Nullable Integer numTotalBatches = getNumTotalBatches();
|
||||||
.submit(new RetrieveUnusedSegmentsAction(getDataSource(), getInterval()));
|
List<DataSegment> unusedSegments;
|
||||||
|
LOG.info(
|
||||||
|
"Starting kill with batchSize[%d], up to limit[%d] segments will be deleted%s",
|
||||||
|
batchSize,
|
||||||
|
limit,
|
||||||
|
numTotalBatches != null ? StringUtils.format(" in([%d] batches]).", numTotalBatches) : "."
|
||||||
|
);
|
||||||
|
do {
|
||||||
|
if (nextBatchSize <= 0) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
unusedSegments = toolbox
|
||||||
|
.getTaskActionClient()
|
||||||
|
.submit(new RetrieveUnusedSegmentsAction(getDataSource(), getInterval(), nextBatchSize));
|
||||||
|
|
||||||
final List<List<DataSegment>> unusedSegmentBatches = Lists.partition(allUnusedSegments, batchSize);
|
|
||||||
|
|
||||||
// The individual activities here on the toolbox have possibility to run for a longer period of time,
|
|
||||||
// since they involve calls to metadata storage and archival object storage. And, the tasks take hold of the
|
|
||||||
// task lockbox to run. By splitting the segment list into smaller batches, we have an opportunity to yield the
|
|
||||||
// lock to other activity that might need to happen using the overlord tasklockbox.
|
|
||||||
|
|
||||||
LOG.info("Running kill task[%s] for dataSource[%s] and interval[%s]. Killing total [%,d] unused segments in [%d] batches(batchSize = [%d]).",
|
|
||||||
getId(), getDataSource(), getInterval(), allUnusedSegments.size(), unusedSegmentBatches.size(), batchSize);
|
|
||||||
|
|
||||||
for (final List<DataSegment> unusedSegments : unusedSegmentBatches) {
|
|
||||||
if (!TaskLocks.isLockCoversSegments(taskLockMap, unusedSegments)) {
|
if (!TaskLocks.isLockCoversSegments(taskLockMap, unusedSegments)) {
|
||||||
throw new ISE(
|
throw new ISE(
|
||||||
"Locks[%s] for task[%s] can't cover segments[%s]",
|
"Locks[%s] for task[%s] can't cover segments[%s]",
|
||||||
taskLockMap.values().stream().flatMap(List::stream).collect(Collectors.toList()),
|
taskLockMap.values().stream().flatMap(List::stream).collect(Collectors.toList()),
|
||||||
getId(),
|
getId(),
|
||||||
unusedSegments
|
unusedSegments
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -186,19 +222,40 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
||||||
toolbox.getTaskActionClient().submit(new SegmentNukeAction(new HashSet<>(unusedSegments)));
|
toolbox.getTaskActionClient().submit(new SegmentNukeAction(new HashSet<>(unusedSegments)));
|
||||||
toolbox.getDataSegmentKiller().kill(unusedSegments);
|
toolbox.getDataSegmentKiller().kill(unusedSegments);
|
||||||
numBatchesProcessed++;
|
numBatchesProcessed++;
|
||||||
|
numSegmentsKilled += unusedSegments.size();
|
||||||
|
|
||||||
if (numBatchesProcessed % 10 == 0) {
|
LOG.info("Processed [%d] batches for kill task[%s].", numBatchesProcessed, getId());
|
||||||
LOG.info("Processed [%d/%d] batches for kill task[%s].",
|
|
||||||
numBatchesProcessed, unusedSegmentBatches.size(), getId());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
LOG.info("Finished kill task[%s] for dataSource[%s] and interval[%s]. Deleted total [%,d] unused segments in [%d] batches.",
|
nextBatchSize = computeNextBatchSize(numSegmentsKilled);
|
||||||
getId(), getDataSource(), getInterval(), allUnusedSegments.size(), unusedSegmentBatches.size());
|
} while (unusedSegments.size() != 0 && (null == numTotalBatches || numBatchesProcessed < numTotalBatches));
|
||||||
|
|
||||||
|
LOG.info("Finished kill task[%s] for dataSource[%s] and interval[%s]. Deleted total [%d] unused segments "
|
||||||
|
+ "in [%d] batches.",
|
||||||
|
getId(),
|
||||||
|
getDataSource(),
|
||||||
|
getInterval(),
|
||||||
|
numSegmentsKilled,
|
||||||
|
numBatchesProcessed
|
||||||
|
);
|
||||||
|
|
||||||
return TaskStatus.success(getId());
|
return TaskStatus.success(getId());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
@VisibleForTesting
|
||||||
|
@Nullable
|
||||||
|
Integer getNumTotalBatches()
|
||||||
|
{
|
||||||
|
return null != limit ? (int) Math.ceil((double) limit / batchSize) : null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
@VisibleForTesting
|
||||||
|
int computeNextBatchSize(int numSegmentsKilled)
|
||||||
|
{
|
||||||
|
return null != limit ? Math.min(limit - numSegmentsKilled, batchSize) : batchSize;
|
||||||
|
}
|
||||||
|
|
||||||
private NavigableMap<DateTime, List<TaskLock>> getTaskLockMap(TaskActionClient client) throws IOException
|
private NavigableMap<DateTime, List<TaskLock>> getTaskLockMap(TaskActionClient client) throws IOException
|
||||||
{
|
{
|
||||||
final NavigableMap<DateTime, List<TaskLock>> taskLockMap = new TreeMap<>();
|
final NavigableMap<DateTime, List<TaskLock>> taskLockMap = new TreeMap<>();
|
||||||
|
|
|
@ -87,7 +87,7 @@ public class MoveTask extends AbstractFixedIntervalTask
|
||||||
// List unused segments
|
// List unused segments
|
||||||
final List<DataSegment> unusedSegments = toolbox
|
final List<DataSegment> unusedSegments = toolbox
|
||||||
.getTaskActionClient()
|
.getTaskActionClient()
|
||||||
.submit(new RetrieveUnusedSegmentsAction(myLock.getDataSource(), myLock.getInterval()));
|
.submit(new RetrieveUnusedSegmentsAction(myLock.getDataSource(), myLock.getInterval(), null));
|
||||||
|
|
||||||
// Verify none of these segments have versions > lock version
|
// Verify none of these segments have versions > lock version
|
||||||
for (final DataSegment unusedSegment : unusedSegments) {
|
for (final DataSegment unusedSegment : unusedSegments) {
|
||||||
|
|
|
@ -80,7 +80,7 @@ public class RestoreTask extends AbstractFixedIntervalTask
|
||||||
// List unused segments
|
// List unused segments
|
||||||
final List<DataSegment> unusedSegments = toolbox
|
final List<DataSegment> unusedSegments = toolbox
|
||||||
.getTaskActionClient()
|
.getTaskActionClient()
|
||||||
.submit(new RetrieveUnusedSegmentsAction(myLock.getDataSource(), myLock.getInterval()));
|
.submit(new RetrieveUnusedSegmentsAction(myLock.getDataSource(), myLock.getInterval(), null));
|
||||||
|
|
||||||
// Verify none of these segments have versions > lock version
|
// Verify none of these segments have versions > lock version
|
||||||
for (final DataSegment unusedSegment : unusedSegments) {
|
for (final DataSegment unusedSegment : unusedSegments) {
|
||||||
|
|
|
@ -61,7 +61,7 @@ import java.util.Set;
|
||||||
*/
|
*/
|
||||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||||
@JsonSubTypes(value = {
|
@JsonSubTypes(value = {
|
||||||
@Type(name = "kill", value = KillUnusedSegmentsTask.class),
|
@Type(name = KillUnusedSegmentsTask.TYPE, value = KillUnusedSegmentsTask.class),
|
||||||
@Type(name = "move", value = MoveTask.class),
|
@Type(name = "move", value = MoveTask.class),
|
||||||
@Type(name = "archive", value = ArchiveTask.class),
|
@Type(name = "archive", value = ArchiveTask.class),
|
||||||
@Type(name = "restore", value = RestoreTask.class),
|
@Type(name = "restore", value = RestoreTask.class),
|
||||||
|
|
|
@ -104,7 +104,7 @@ public class RetrieveSegmentsActionsTest
|
||||||
@Test
|
@Test
|
||||||
public void testRetrieveUnusedSegmentsAction()
|
public void testRetrieveUnusedSegmentsAction()
|
||||||
{
|
{
|
||||||
final RetrieveUnusedSegmentsAction action = new RetrieveUnusedSegmentsAction(task.getDataSource(), INTERVAL);
|
final RetrieveUnusedSegmentsAction action = new RetrieveUnusedSegmentsAction(task.getDataSource(), INTERVAL, null);
|
||||||
final Set<DataSegment> resultSegments = new HashSet<>(action.perform(task, actionTestKit.getTaskActionToolbox()));
|
final Set<DataSegment> resultSegments = new HashSet<>(action.perform(task, actionTestKit.getTaskActionToolbox()));
|
||||||
Assert.assertEquals(expectedUnusedSegments, resultSegments);
|
Assert.assertEquals(expectedUnusedSegments, resultSegments);
|
||||||
}
|
}
|
||||||
|
|
|
@ -51,8 +51,9 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
||||||
"killTaskId",
|
"killTaskId",
|
||||||
"datasource",
|
"datasource",
|
||||||
Intervals.of("2020-01-01/P1D"),
|
Intervals.of("2020-01-01/P1D"),
|
||||||
true,
|
false,
|
||||||
99
|
99,
|
||||||
|
5
|
||||||
);
|
);
|
||||||
final byte[] json = objectMapper.writeValueAsBytes(taskQuery);
|
final byte[] json = objectMapper.writeValueAsBytes(taskQuery);
|
||||||
final KillUnusedSegmentsTask fromJson = (KillUnusedSegmentsTask) objectMapper.readValue(json, Task.class);
|
final KillUnusedSegmentsTask fromJson = (KillUnusedSegmentsTask) objectMapper.readValue(json, Task.class);
|
||||||
|
@ -61,6 +62,8 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
||||||
Assert.assertEquals(taskQuery.getInterval(), fromJson.getInterval());
|
Assert.assertEquals(taskQuery.getInterval(), fromJson.getInterval());
|
||||||
Assert.assertEquals(taskQuery.getMarkAsUnused(), fromJson.isMarkAsUnused());
|
Assert.assertEquals(taskQuery.getMarkAsUnused(), fromJson.isMarkAsUnused());
|
||||||
Assert.assertEquals(taskQuery.getBatchSize(), Integer.valueOf(fromJson.getBatchSize()));
|
Assert.assertEquals(taskQuery.getBatchSize(), Integer.valueOf(fromJson.getBatchSize()));
|
||||||
|
Assert.assertEquals(taskQuery.getLimit(), fromJson.getLimit());
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -71,6 +74,7 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
||||||
"datasource",
|
"datasource",
|
||||||
Intervals.of("2020-01-01/P1D"),
|
Intervals.of("2020-01-01/P1D"),
|
||||||
true,
|
true,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
final byte[] json = objectMapper.writeValueAsBytes(taskQuery);
|
final byte[] json = objectMapper.writeValueAsBytes(taskQuery);
|
||||||
|
@ -80,6 +84,7 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
||||||
Assert.assertEquals(taskQuery.getInterval(), fromJson.getInterval());
|
Assert.assertEquals(taskQuery.getInterval(), fromJson.getInterval());
|
||||||
Assert.assertEquals(taskQuery.getMarkAsUnused(), fromJson.isMarkAsUnused());
|
Assert.assertEquals(taskQuery.getMarkAsUnused(), fromJson.isMarkAsUnused());
|
||||||
Assert.assertEquals(100, fromJson.getBatchSize());
|
Assert.assertEquals(100, fromJson.getBatchSize());
|
||||||
|
Assert.assertNull(taskQuery.getLimit());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -91,7 +96,8 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
||||||
Intervals.of("2020-01-01/P1D"),
|
Intervals.of("2020-01-01/P1D"),
|
||||||
null,
|
null,
|
||||||
true,
|
true,
|
||||||
99
|
99,
|
||||||
|
null
|
||||||
);
|
);
|
||||||
final byte[] json = objectMapper.writeValueAsBytes(task);
|
final byte[] json = objectMapper.writeValueAsBytes(task);
|
||||||
final ClientKillUnusedSegmentsTaskQuery taskQuery = (ClientKillUnusedSegmentsTaskQuery) objectMapper.readValue(
|
final ClientKillUnusedSegmentsTaskQuery taskQuery = (ClientKillUnusedSegmentsTaskQuery) objectMapper.readValue(
|
||||||
|
@ -103,5 +109,6 @@ public class ClientKillUnusedSegmentsTaskQuerySerdeTest
|
||||||
Assert.assertEquals(task.getInterval(), taskQuery.getInterval());
|
Assert.assertEquals(task.getInterval(), taskQuery.getInterval());
|
||||||
Assert.assertEquals(task.isMarkAsUnused(), taskQuery.getMarkAsUnused());
|
Assert.assertEquals(task.isMarkAsUnused(), taskQuery.getMarkAsUnused());
|
||||||
Assert.assertEquals(Integer.valueOf(task.getBatchSize()), taskQuery.getBatchSize());
|
Assert.assertEquals(Integer.valueOf(task.getBatchSize()), taskQuery.getBatchSize());
|
||||||
|
Assert.assertNull(task.getLimit());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -81,6 +81,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
||||||
Intervals.of("2019-03-01/2019-04-01"),
|
Intervals.of("2019-03-01/2019-04-01"),
|
||||||
null,
|
null,
|
||||||
false,
|
false,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -97,7 +98,9 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
||||||
newSegment(Intervals.of("2019-01-01/2019-02-01"), version),
|
newSegment(Intervals.of("2019-01-01/2019-02-01"), version),
|
||||||
newSegment(Intervals.of("2019-04-01/2019-05-01"), version)
|
newSegment(Intervals.of("2019-04-01/2019-05-01"), version)
|
||||||
);
|
);
|
||||||
Assert.assertEquals(1L, task.getNumBatchesProcessed());
|
|
||||||
|
Assert.assertEquals(2L, task.getNumBatchesProcessed());
|
||||||
|
Assert.assertEquals(1, task.getNumSegmentsKilled());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -128,6 +131,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
||||||
Intervals.of("2019-03-01/2019-04-01"),
|
Intervals.of("2019-03-01/2019-04-01"),
|
||||||
null,
|
null,
|
||||||
true,
|
true,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -144,7 +148,8 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
||||||
newSegment(Intervals.of("2019-01-01/2019-02-01"), version),
|
newSegment(Intervals.of("2019-01-01/2019-02-01"), version),
|
||||||
newSegment(Intervals.of("2019-04-01/2019-05-01"), version)
|
newSegment(Intervals.of("2019-04-01/2019-05-01"), version)
|
||||||
);
|
);
|
||||||
Assert.assertEquals(1L, task.getNumBatchesProcessed());
|
Assert.assertEquals(2L, task.getNumBatchesProcessed());
|
||||||
|
Assert.assertEquals(1, task.getNumSegmentsKilled());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -157,13 +162,14 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
||||||
Intervals.of("2019-03-01/2019-04-01"),
|
Intervals.of("2019-03-01/2019-04-01"),
|
||||||
null,
|
null,
|
||||||
true,
|
true,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
Assert.assertTrue(task.getInputSourceResources().isEmpty());
|
Assert.assertTrue(task.getInputSourceResources().isEmpty());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testKillBatchSizeOne() throws Exception
|
public void testKillBatchSizeOneAndLimit4() throws Exception
|
||||||
{
|
{
|
||||||
final String version = DateTimes.nowUtc().toString();
|
final String version = DateTimes.nowUtc().toString();
|
||||||
final Set<DataSegment> segments = ImmutableSet.of(
|
final Set<DataSegment> segments = ImmutableSet.of(
|
||||||
|
@ -176,14 +182,23 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
||||||
|
|
||||||
Assert.assertEquals(segments, announced);
|
Assert.assertEquals(segments, announced);
|
||||||
|
|
||||||
|
Assert.assertEquals(
|
||||||
|
segments.size(),
|
||||||
|
getSegmentsMetadataManager().markAsUnusedSegmentsInInterval(
|
||||||
|
DATA_SOURCE,
|
||||||
|
Intervals.of("2018-01-01/2020-01-01")
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
final KillUnusedSegmentsTask task =
|
final KillUnusedSegmentsTask task =
|
||||||
new KillUnusedSegmentsTask(
|
new KillUnusedSegmentsTask(
|
||||||
null,
|
null,
|
||||||
DATA_SOURCE,
|
DATA_SOURCE,
|
||||||
Intervals.of("2018-01-01/2020-01-01"),
|
Intervals.of("2018-01-01/2020-01-01"),
|
||||||
null,
|
null,
|
||||||
true,
|
false,
|
||||||
1
|
1,
|
||||||
|
4
|
||||||
);
|
);
|
||||||
|
|
||||||
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode());
|
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode());
|
||||||
|
@ -195,6 +210,7 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
||||||
|
|
||||||
Assert.assertEquals(Collections.emptyList(), unusedSegments);
|
Assert.assertEquals(Collections.emptyList(), unusedSegments);
|
||||||
Assert.assertEquals(4L, task.getNumBatchesProcessed());
|
Assert.assertEquals(4L, task.getNumBatchesProcessed());
|
||||||
|
Assert.assertEquals(4, task.getNumSegmentsKilled());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -218,7 +234,8 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
||||||
Intervals.of("2018-01-01/2020-01-01"),
|
Intervals.of("2018-01-01/2020-01-01"),
|
||||||
null,
|
null,
|
||||||
true,
|
true,
|
||||||
3
|
3,
|
||||||
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode());
|
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode());
|
||||||
|
@ -229,7 +246,120 @@ public class KillUnusedSegmentsTaskTest extends IngestionTestBase
|
||||||
getMetadataStorageCoordinator().retrieveUnusedSegmentsForInterval(DATA_SOURCE, Intervals.of("2019/2020"));
|
getMetadataStorageCoordinator().retrieveUnusedSegmentsForInterval(DATA_SOURCE, Intervals.of("2019/2020"));
|
||||||
|
|
||||||
Assert.assertEquals(Collections.emptyList(), unusedSegments);
|
Assert.assertEquals(Collections.emptyList(), unusedSegments);
|
||||||
Assert.assertEquals(2L, task.getNumBatchesProcessed());
|
Assert.assertEquals(3L, task.getNumBatchesProcessed());
|
||||||
|
Assert.assertEquals(4, task.getNumSegmentsKilled());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testComputeNextBatchSizeDefault()
|
||||||
|
{
|
||||||
|
final KillUnusedSegmentsTask task =
|
||||||
|
new KillUnusedSegmentsTask(
|
||||||
|
null,
|
||||||
|
DATA_SOURCE,
|
||||||
|
Intervals.of("2018-01-01/2020-01-01"),
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
Assert.assertEquals(100, task.computeNextBatchSize(50));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testComputeNextBatchSizeWithBatchSizeLargerThanLimit()
|
||||||
|
{
|
||||||
|
final KillUnusedSegmentsTask task =
|
||||||
|
new KillUnusedSegmentsTask(
|
||||||
|
null,
|
||||||
|
DATA_SOURCE,
|
||||||
|
Intervals.of("2018-01-01/2020-01-01"),
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
10,
|
||||||
|
5
|
||||||
|
);
|
||||||
|
Assert.assertEquals(5, task.computeNextBatchSize(0));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testComputeNextBatchSizeWithBatchSizeSmallerThanLimit()
|
||||||
|
{
|
||||||
|
final KillUnusedSegmentsTask task =
|
||||||
|
new KillUnusedSegmentsTask(
|
||||||
|
null,
|
||||||
|
DATA_SOURCE,
|
||||||
|
Intervals.of("2018-01-01/2020-01-01"),
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
5,
|
||||||
|
10
|
||||||
|
);
|
||||||
|
Assert.assertEquals(5, task.computeNextBatchSize(0));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testComputeNextBatchSizeWithRemainingLessThanLimit()
|
||||||
|
{
|
||||||
|
final KillUnusedSegmentsTask task =
|
||||||
|
new KillUnusedSegmentsTask(
|
||||||
|
null,
|
||||||
|
DATA_SOURCE,
|
||||||
|
Intervals.of("2018-01-01/2020-01-01"),
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
5,
|
||||||
|
10
|
||||||
|
);
|
||||||
|
Assert.assertEquals(3, task.computeNextBatchSize(7));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetNumTotalBatchesDefault()
|
||||||
|
{
|
||||||
|
final KillUnusedSegmentsTask task =
|
||||||
|
new KillUnusedSegmentsTask(
|
||||||
|
null,
|
||||||
|
DATA_SOURCE,
|
||||||
|
Intervals.of("2018-01-01/2020-01-01"),
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
Assert.assertNull(task.getNumTotalBatches());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetNumTotalBatchesWithBatchSizeLargerThanLimit()
|
||||||
|
{
|
||||||
|
final KillUnusedSegmentsTask task =
|
||||||
|
new KillUnusedSegmentsTask(
|
||||||
|
null,
|
||||||
|
DATA_SOURCE,
|
||||||
|
Intervals.of("2018-01-01/2020-01-01"),
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
10,
|
||||||
|
5
|
||||||
|
);
|
||||||
|
Assert.assertEquals(1, (int) task.getNumTotalBatches());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetNumTotalBatchesWithBatchSizeSmallerThanLimit()
|
||||||
|
{
|
||||||
|
final KillUnusedSegmentsTask task =
|
||||||
|
new KillUnusedSegmentsTask(
|
||||||
|
null,
|
||||||
|
DATA_SOURCE,
|
||||||
|
Intervals.of("2018-01-01/2020-01-01"),
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
5,
|
||||||
|
10
|
||||||
|
);
|
||||||
|
Assert.assertEquals(2, (int) task.getNumTotalBatches());
|
||||||
}
|
}
|
||||||
|
|
||||||
private static DataSegment newSegment(Interval interval, String version)
|
private static DataSegment newSegment(Interval interval, String version)
|
||||||
|
|
|
@ -949,6 +949,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
||||||
Intervals.of("2011-04-01/P4D"),
|
Intervals.of("2011-04-01/P4D"),
|
||||||
null,
|
null,
|
||||||
false,
|
false,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -957,7 +958,7 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
||||||
Assert.assertEquals("merged statusCode", TaskState.SUCCESS, status.getStatusCode());
|
Assert.assertEquals("merged statusCode", TaskState.SUCCESS, status.getStatusCode());
|
||||||
Assert.assertEquals("num segments published", 0, mdc.getPublished().size());
|
Assert.assertEquals("num segments published", 0, mdc.getPublished().size());
|
||||||
Assert.assertEquals("num segments nuked", 3, mdc.getNuked().size());
|
Assert.assertEquals("num segments nuked", 3, mdc.getNuked().size());
|
||||||
Assert.assertEquals("delete segment batch call count", 1, mdc.getDeleteSegmentsCount());
|
Assert.assertEquals("delete segment batch call count", 2, mdc.getDeleteSegmentsCount());
|
||||||
Assert.assertTrue(
|
Assert.assertTrue(
|
||||||
"expected unused segments get killed",
|
"expected unused segments get killed",
|
||||||
expectedUnusedSegments.containsAll(mdc.getNuked()) && mdc.getNuked().containsAll(
|
expectedUnusedSegments.containsAll(mdc.getNuked()) && mdc.getNuked().containsAll(
|
||||||
|
@ -970,6 +971,104 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testKillUnusedSegmentsTaskWithMaxSegmentsToKill() throws Exception
|
||||||
|
{
|
||||||
|
final File tmpSegmentDir = temporaryFolder.newFolder();
|
||||||
|
|
||||||
|
List<DataSegment> expectedUnusedSegments = Lists.transform(
|
||||||
|
ImmutableList.of(
|
||||||
|
"2011-04-01/2011-04-02",
|
||||||
|
"2011-04-02/2011-04-03",
|
||||||
|
"2011-04-04/2011-04-05"
|
||||||
|
), new Function<String, DataSegment>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public DataSegment apply(String input)
|
||||||
|
{
|
||||||
|
final Interval interval = Intervals.of(input);
|
||||||
|
try {
|
||||||
|
return DataSegment.builder()
|
||||||
|
.dataSource("test_kill_task")
|
||||||
|
.interval(interval)
|
||||||
|
.loadSpec(
|
||||||
|
ImmutableMap.of(
|
||||||
|
"type",
|
||||||
|
"local",
|
||||||
|
"path",
|
||||||
|
tmpSegmentDir.getCanonicalPath()
|
||||||
|
+ "/druid/localStorage/wikipedia/"
|
||||||
|
+ interval.getStart()
|
||||||
|
+ "-"
|
||||||
|
+ interval.getEnd()
|
||||||
|
+ "/"
|
||||||
|
+ "2011-04-6T16:52:46.119-05:00"
|
||||||
|
+ "/0/index.zip"
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.version("2011-04-6T16:52:46.119-05:00")
|
||||||
|
.dimensions(ImmutableList.of())
|
||||||
|
.metrics(ImmutableList.of())
|
||||||
|
.shardSpec(NoneShardSpec.instance())
|
||||||
|
.binaryVersion(9)
|
||||||
|
.size(0)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
throw new ISE(e, "Error creating segments");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
|
||||||
|
mdc.setUnusedSegments(expectedUnusedSegments);
|
||||||
|
|
||||||
|
// manually create local segments files
|
||||||
|
List<File> segmentFiles = new ArrayList<>();
|
||||||
|
for (DataSegment segment : mdc.retrieveUnusedSegmentsForInterval("test_kill_task", Intervals.of("2011-04-01/P4D"))) {
|
||||||
|
File file = new File((String) segment.getLoadSpec().get("path"));
|
||||||
|
FileUtils.mkdirp(file.getParentFile());
|
||||||
|
Files.write(file.toPath(), ByteArrays.EMPTY_ARRAY);
|
||||||
|
segmentFiles.add(file);
|
||||||
|
}
|
||||||
|
|
||||||
|
final int maxSegmentsToKill = 2;
|
||||||
|
final Task killUnusedSegmentsTask =
|
||||||
|
new KillUnusedSegmentsTask(
|
||||||
|
null,
|
||||||
|
"test_kill_task",
|
||||||
|
Intervals.of("2011-04-01/P4D"),
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
maxSegmentsToKill
|
||||||
|
);
|
||||||
|
|
||||||
|
final TaskStatus status = runTask(killUnusedSegmentsTask);
|
||||||
|
Assert.assertEquals(taskLocation, status.getLocation());
|
||||||
|
Assert.assertEquals("merged statusCode", TaskState.SUCCESS, status.getStatusCode());
|
||||||
|
Assert.assertEquals("num segments published", 0, mdc.getPublished().size());
|
||||||
|
Assert.assertEquals("num segments nuked", maxSegmentsToKill, mdc.getNuked().size());
|
||||||
|
Assert.assertTrue(
|
||||||
|
"expected unused segments get killed",
|
||||||
|
expectedUnusedSegments.containsAll(mdc.getNuked())
|
||||||
|
);
|
||||||
|
|
||||||
|
int expectedNumOfSegmentsRemaining = segmentFiles.size() - maxSegmentsToKill;
|
||||||
|
int actualNumOfSegmentsRemaining = 0;
|
||||||
|
for (File file : segmentFiles) {
|
||||||
|
if (file.exists()) {
|
||||||
|
actualNumOfSegmentsRemaining++;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
Assert.assertEquals(
|
||||||
|
"Expected of segments deleted did not match expectations",
|
||||||
|
expectedNumOfSegmentsRemaining,
|
||||||
|
actualNumOfSegmentsRemaining
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRealtimeishTask() throws Exception
|
public void testRealtimeishTask() throws Exception
|
||||||
{
|
{
|
||||||
|
|
|
@ -42,6 +42,7 @@ import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataStorageCoordinator
|
public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataStorageCoordinator
|
||||||
{
|
{
|
||||||
|
@ -110,6 +111,22 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<DataSegment> retrieveUnusedSegmentsForInterval(String dataSource, Interval interval, @Nullable Integer limit)
|
||||||
|
{
|
||||||
|
synchronized (unusedSegments) {
|
||||||
|
Stream<DataSegment> resultStream = unusedSegments.stream();
|
||||||
|
|
||||||
|
resultStream = resultStream.filter(ds -> !nuked.contains(ds));
|
||||||
|
|
||||||
|
if (limit != null) {
|
||||||
|
resultStream = resultStream.limit(limit);
|
||||||
|
}
|
||||||
|
|
||||||
|
return ImmutableList.copyOf(resultStream.iterator());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int markSegmentsAsUnusedWithinInterval(String dataSource, Interval interval)
|
public int markSegmentsAsUnusedWithinInterval(String dataSource, Interval interval)
|
||||||
{
|
{
|
||||||
|
|
|
@ -24,6 +24,8 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -40,14 +42,16 @@ public class ClientKillUnusedSegmentsTaskQuery implements ClientTaskQuery
|
||||||
private final Interval interval;
|
private final Interval interval;
|
||||||
private final Boolean markAsUnused;
|
private final Boolean markAsUnused;
|
||||||
private final Integer batchSize;
|
private final Integer batchSize;
|
||||||
|
@Nullable private final Integer limit;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public ClientKillUnusedSegmentsTaskQuery(
|
public ClientKillUnusedSegmentsTaskQuery(
|
||||||
@JsonProperty("id") String id,
|
@JsonProperty("id") String id,
|
||||||
@JsonProperty("dataSource") String dataSource,
|
@JsonProperty("dataSource") String dataSource,
|
||||||
@JsonProperty("interval") Interval interval,
|
@JsonProperty("interval") Interval interval,
|
||||||
@JsonProperty("markAsUnused") Boolean markAsUnused,
|
@JsonProperty("markAsUnused") @Deprecated Boolean markAsUnused,
|
||||||
@JsonProperty("batchSize") Integer batchSize
|
@JsonProperty("batchSize") Integer batchSize,
|
||||||
|
@JsonProperty("limit") Integer limit
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.id = Preconditions.checkNotNull(id, "id");
|
this.id = Preconditions.checkNotNull(id, "id");
|
||||||
|
@ -55,6 +59,8 @@ public class ClientKillUnusedSegmentsTaskQuery implements ClientTaskQuery
|
||||||
this.interval = interval;
|
this.interval = interval;
|
||||||
this.markAsUnused = markAsUnused;
|
this.markAsUnused = markAsUnused;
|
||||||
this.batchSize = batchSize;
|
this.batchSize = batchSize;
|
||||||
|
Preconditions.checkArgument(limit == null || limit > 0, "limit must be > 0");
|
||||||
|
this.limit = limit;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -96,6 +102,14 @@ public class ClientKillUnusedSegmentsTaskQuery implements ClientTaskQuery
|
||||||
return batchSize;
|
return batchSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
@Nullable
|
||||||
|
public Integer getLimit()
|
||||||
|
{
|
||||||
|
return limit;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object o)
|
public boolean equals(Object o)
|
||||||
{
|
{
|
||||||
|
@ -110,12 +124,13 @@ public class ClientKillUnusedSegmentsTaskQuery implements ClientTaskQuery
|
||||||
&& Objects.equals(dataSource, that.dataSource)
|
&& Objects.equals(dataSource, that.dataSource)
|
||||||
&& Objects.equals(interval, that.interval)
|
&& Objects.equals(interval, that.interval)
|
||||||
&& Objects.equals(markAsUnused, that.markAsUnused)
|
&& Objects.equals(markAsUnused, that.markAsUnused)
|
||||||
&& Objects.equals(batchSize, that.batchSize);
|
&& Objects.equals(batchSize, that.batchSize)
|
||||||
|
&& Objects.equals(limit, that.limit);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
return Objects.hash(id, dataSource, interval, markAsUnused, batchSize);
|
return Objects.hash(id, dataSource, interval, markAsUnused, batchSize, limit);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -123,17 +123,30 @@ public interface IndexerMetadataStorageCoordinator
|
||||||
Segments visibility
|
Segments visibility
|
||||||
);
|
);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* see {@link #retrieveUnusedSegmentsForInterval(String, Interval, Integer)}
|
||||||
|
*/
|
||||||
|
default List<DataSegment> retrieveUnusedSegmentsForInterval(String dataSource, Interval interval)
|
||||||
|
{
|
||||||
|
return retrieveUnusedSegmentsForInterval(dataSource, interval, null);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retrieve all published segments which include ONLY data within the given interval and are marked as unused from the
|
* Retrieve all published segments which include ONLY data within the given interval and are marked as unused from the
|
||||||
* metadata store.
|
* metadata store.
|
||||||
*
|
*
|
||||||
* @param dataSource The data source the segments belong to
|
* @param dataSource The data source the segments belong to
|
||||||
* @param interval Filter the data segments to ones that include data in this interval exclusively.
|
* @param interval Filter the data segments to ones that include data in this interval exclusively.
|
||||||
|
* @param limit The maximum number of unused segments to retreive. If null, no limit is applied.
|
||||||
*
|
*
|
||||||
* @return DataSegments which include ONLY data within the requested interval and are marked as unused. Segments NOT
|
* @return DataSegments which include ONLY data within the requested interval and are marked as unused. Segments NOT
|
||||||
* returned here may include data in the interval
|
* returned here may include data in the interval
|
||||||
*/
|
*/
|
||||||
List<DataSegment> retrieveUnusedSegmentsForInterval(String dataSource, Interval interval);
|
List<DataSegment> retrieveUnusedSegmentsForInterval(
|
||||||
|
String dataSource,
|
||||||
|
Interval interval,
|
||||||
|
@Nullable Integer limit
|
||||||
|
);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mark as unused segments which include ONLY data within the given interval.
|
* Mark as unused segments which include ONLY data within the given interval.
|
||||||
|
|
|
@ -190,12 +190,22 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<DataSegment> retrieveUnusedSegmentsForInterval(final String dataSource, final Interval interval)
|
public List<DataSegment> retrieveUnusedSegmentsForInterval(final String dataSource, final Interval interval)
|
||||||
|
{
|
||||||
|
return retrieveUnusedSegmentsForInterval(dataSource, interval, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<DataSegment> retrieveUnusedSegmentsForInterval(
|
||||||
|
String dataSource,
|
||||||
|
Interval interval,
|
||||||
|
@Nullable Integer limit
|
||||||
|
)
|
||||||
{
|
{
|
||||||
final List<DataSegment> matchingSegments = connector.inReadOnlyTransaction(
|
final List<DataSegment> matchingSegments = connector.inReadOnlyTransaction(
|
||||||
(handle, status) -> {
|
(handle, status) -> {
|
||||||
try (final CloseableIterator<DataSegment> iterator =
|
try (final CloseableIterator<DataSegment> iterator =
|
||||||
SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper)
|
SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper)
|
||||||
.retrieveUnusedSegments(dataSource, Collections.singletonList(interval))) {
|
.retrieveUnusedSegments(dataSource, Collections.singletonList(interval), limit)) {
|
||||||
return ImmutableList.copyOf(iterator);
|
return ImmutableList.copyOf(iterator);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -574,7 +574,7 @@ public class SqlSegmentsMetadataManager implements SegmentsMetadataManager
|
||||||
}
|
}
|
||||||
|
|
||||||
try (final CloseableIterator<DataSegment> iterator =
|
try (final CloseableIterator<DataSegment> iterator =
|
||||||
queryTool.retrieveUnusedSegments(dataSourceName, intervals)) {
|
queryTool.retrieveUnusedSegments(dataSourceName, intervals, null)) {
|
||||||
while (iterator.hasNext()) {
|
while (iterator.hasNext()) {
|
||||||
final DataSegment dataSegment = iterator.next();
|
final DataSegment dataSegment = iterator.next();
|
||||||
timeline.addSegments(Iterators.singletonIterator(dataSegment));
|
timeline.addSegments(Iterators.singletonIterator(dataSegment));
|
||||||
|
|
|
@ -37,6 +37,8 @@ import org.skife.jdbi.v2.PreparedBatch;
|
||||||
import org.skife.jdbi.v2.Query;
|
import org.skife.jdbi.v2.Query;
|
||||||
import org.skife.jdbi.v2.ResultIterator;
|
import org.skife.jdbi.v2.ResultIterator;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -104,7 +106,7 @@ public class SqlSegmentsMetadataQuery
|
||||||
final Collection<Interval> intervals
|
final Collection<Interval> intervals
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return retrieveSegments(dataSource, intervals, IntervalMode.OVERLAPS, true);
|
return retrieveSegments(dataSource, intervals, IntervalMode.OVERLAPS, true, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -118,10 +120,11 @@ public class SqlSegmentsMetadataQuery
|
||||||
*/
|
*/
|
||||||
public CloseableIterator<DataSegment> retrieveUnusedSegments(
|
public CloseableIterator<DataSegment> retrieveUnusedSegments(
|
||||||
final String dataSource,
|
final String dataSource,
|
||||||
final Collection<Interval> intervals
|
final Collection<Interval> intervals,
|
||||||
|
@Nullable final Integer limit
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return retrieveSegments(dataSource, intervals, IntervalMode.CONTAINS, false);
|
return retrieveSegments(dataSource, intervals, IntervalMode.CONTAINS, false, limit);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -201,7 +204,7 @@ public class SqlSegmentsMetadataQuery
|
||||||
// Retrieve, then drop, since we can't write a WHERE clause directly.
|
// Retrieve, then drop, since we can't write a WHERE clause directly.
|
||||||
final List<SegmentId> segments = ImmutableList.copyOf(
|
final List<SegmentId> segments = ImmutableList.copyOf(
|
||||||
Iterators.transform(
|
Iterators.transform(
|
||||||
retrieveSegments(dataSource, Collections.singletonList(interval), IntervalMode.CONTAINS, true),
|
retrieveSegments(dataSource, Collections.singletonList(interval), IntervalMode.CONTAINS, true, null),
|
||||||
DataSegment::getId
|
DataSegment::getId
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
@ -213,7 +216,8 @@ public class SqlSegmentsMetadataQuery
|
||||||
final String dataSource,
|
final String dataSource,
|
||||||
final Collection<Interval> intervals,
|
final Collection<Interval> intervals,
|
||||||
final IntervalMode matchMode,
|
final IntervalMode matchMode,
|
||||||
final boolean used
|
final boolean used,
|
||||||
|
@Nullable final Integer limit
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
// Check if the intervals all support comparing as strings. If so, bake them into the SQL.
|
// Check if the intervals all support comparing as strings. If so, bake them into the SQL.
|
||||||
|
@ -259,6 +263,9 @@ public class SqlSegmentsMetadataQuery
|
||||||
.setFetchSize(connector.getStreamingFetchSize())
|
.setFetchSize(connector.getStreamingFetchSize())
|
||||||
.bind("used", used)
|
.bind("used", used)
|
||||||
.bind("dataSource", dataSource);
|
.bind("dataSource", dataSource);
|
||||||
|
if (null != limit) {
|
||||||
|
sql.setMaxRows(limit);
|
||||||
|
}
|
||||||
|
|
||||||
if (compareAsString) {
|
if (compareAsString) {
|
||||||
final Iterator<Interval> iterator = intervals.iterator();
|
final Iterator<Interval> iterator = intervals.iterator();
|
||||||
|
|
|
@ -84,9 +84,40 @@ public interface OverlordClient
|
||||||
* @return future with task ID
|
* @return future with task ID
|
||||||
*/
|
*/
|
||||||
default ListenableFuture<String> runKillTask(String idPrefix, String dataSource, Interval interval)
|
default ListenableFuture<String> runKillTask(String idPrefix, String dataSource, Interval interval)
|
||||||
|
{
|
||||||
|
return runKillTask(idPrefix, dataSource, interval, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run a "kill" task for a particular datasource and interval. Shortcut to {@link #runTask(String, Object)}.
|
||||||
|
*
|
||||||
|
* The kill task deletes all unused segment records from deep storage and the metadata store. The task runs
|
||||||
|
* asynchronously after the API call returns. The resolved future is the ID of the task, which can be used to
|
||||||
|
* monitor its progress through the {@link #taskStatus(String)} API.
|
||||||
|
*
|
||||||
|
* @param idPrefix Descriptive prefix to include at the start of task IDs
|
||||||
|
* @param dataSource Datasource to kill
|
||||||
|
* @param interval Interval to kill
|
||||||
|
* @param maxSegmentsToKill The maximum number of segments to kill
|
||||||
|
*
|
||||||
|
* @return future with task ID
|
||||||
|
*/
|
||||||
|
default ListenableFuture<String> runKillTask(
|
||||||
|
String idPrefix,
|
||||||
|
String dataSource,
|
||||||
|
Interval interval,
|
||||||
|
@Nullable Integer maxSegmentsToKill
|
||||||
|
)
|
||||||
{
|
{
|
||||||
final String taskId = IdUtils.newTaskId(idPrefix, ClientKillUnusedSegmentsTaskQuery.TYPE, dataSource, interval);
|
final String taskId = IdUtils.newTaskId(idPrefix, ClientKillUnusedSegmentsTaskQuery.TYPE, dataSource, interval);
|
||||||
final ClientTaskQuery taskQuery = new ClientKillUnusedSegmentsTaskQuery(taskId, dataSource, interval, false, null);
|
final ClientTaskQuery taskQuery = new ClientKillUnusedSegmentsTaskQuery(
|
||||||
|
taskId,
|
||||||
|
dataSource,
|
||||||
|
interval,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
maxSegmentsToKill
|
||||||
|
);
|
||||||
return FutureUtils.transform(runTask(taskId, taskQuery), ignored -> taskId);
|
return FutureUtils.transform(runTask(taskId, taskQuery), ignored -> taskId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -33,6 +33,8 @@ import org.apache.druid.utils.CollectionUtils;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
@ -130,7 +132,12 @@ public class KillUnusedSegments implements CoordinatorDuty
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
FutureUtils.getUnchecked(overlordClient.runKillTask("coordinator-issued", dataSource, intervalToKill), true);
|
FutureUtils.getUnchecked(overlordClient.runKillTask(
|
||||||
|
"coordinator-issued",
|
||||||
|
dataSource,
|
||||||
|
intervalToKill,
|
||||||
|
maxSegmentsToKill
|
||||||
|
), true);
|
||||||
++submittedTasks;
|
++submittedTasks;
|
||||||
}
|
}
|
||||||
catch (Exception ex) {
|
catch (Exception ex) {
|
||||||
|
@ -148,6 +155,7 @@ public class KillUnusedSegments implements CoordinatorDuty
|
||||||
/**
|
/**
|
||||||
* Calculates the interval for which segments are to be killed in a datasource.
|
* Calculates the interval for which segments are to be killed in a datasource.
|
||||||
*/
|
*/
|
||||||
|
@Nullable
|
||||||
private Interval findIntervalForKill(String dataSource)
|
private Interval findIntervalForKill(String dataSource)
|
||||||
{
|
{
|
||||||
final DateTime maxEndTime = ignoreRetainDuration
|
final DateTime maxEndTime = ignoreRetainDuration
|
||||||
|
|
|
@ -342,7 +342,7 @@ public class DataSourcesResource
|
||||||
}
|
}
|
||||||
final Interval theInterval = Intervals.of(interval.replace('_', '/'));
|
final Interval theInterval = Intervals.of(interval.replace('_', '/'));
|
||||||
try {
|
try {
|
||||||
FutureUtils.getUnchecked(overlordClient.runKillTask("api-issued", dataSourceName, theInterval), true);
|
FutureUtils.getUnchecked(overlordClient.runKillTask("api-issued", dataSourceName, theInterval, null), true);
|
||||||
return Response.ok().build();
|
return Response.ok().build();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
|
|
|
@ -35,13 +35,21 @@ public class ClientKillUnusedSegmentsTaskQueryTest
|
||||||
private static final Interval INTERVAL = new Interval(START, START.plus(1));
|
private static final Interval INTERVAL = new Interval(START, START.plus(1));
|
||||||
private static final Boolean MARK_UNUSED = true;
|
private static final Boolean MARK_UNUSED = true;
|
||||||
private static final Integer BATCH_SIZE = 999;
|
private static final Integer BATCH_SIZE = 999;
|
||||||
|
private static final Integer LIMIT = 1000;
|
||||||
|
|
||||||
ClientKillUnusedSegmentsTaskQuery clientKillUnusedSegmentsQuery;
|
ClientKillUnusedSegmentsTaskQuery clientKillUnusedSegmentsQuery;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp()
|
public void setUp()
|
||||||
{
|
{
|
||||||
clientKillUnusedSegmentsQuery = new ClientKillUnusedSegmentsTaskQuery("killTaskId", DATA_SOURCE, INTERVAL, true, BATCH_SIZE);
|
clientKillUnusedSegmentsQuery = new ClientKillUnusedSegmentsTaskQuery(
|
||||||
|
"killTaskId",
|
||||||
|
DATA_SOURCE,
|
||||||
|
INTERVAL,
|
||||||
|
true,
|
||||||
|
BATCH_SIZE,
|
||||||
|
LIMIT
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -80,12 +88,18 @@ public class ClientKillUnusedSegmentsTaskQueryTest
|
||||||
Assert.assertEquals(BATCH_SIZE, clientKillUnusedSegmentsQuery.getBatchSize());
|
Assert.assertEquals(BATCH_SIZE, clientKillUnusedSegmentsQuery.getBatchSize());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetLimit()
|
||||||
|
{
|
||||||
|
Assert.assertEquals(LIMIT, clientKillUnusedSegmentsQuery.getLimit());
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testEquals()
|
public void testEquals()
|
||||||
{
|
{
|
||||||
EqualsVerifier.forClass(ClientKillUnusedSegmentsTaskQuery.class)
|
EqualsVerifier.forClass(ClientKillUnusedSegmentsTaskQuery.class)
|
||||||
.usingGetClass()
|
.usingGetClass()
|
||||||
.withNonnullFields("id", "dataSource", "interval", "batchSize")
|
.withNonnullFields("id", "dataSource", "interval", "batchSize", "limit")
|
||||||
.verify();
|
.verify();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -932,6 +932,22 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSimpleUnusedListWithLimit() throws IOException
|
||||||
|
{
|
||||||
|
coordinator.announceHistoricalSegments(SEGMENTS);
|
||||||
|
markAllSegmentsUnused();
|
||||||
|
int limit = SEGMENTS.size() - 1;
|
||||||
|
Set<DataSegment> retreivedUnusedSegments = ImmutableSet.copyOf(
|
||||||
|
coordinator.retrieveUnusedSegmentsForInterval(
|
||||||
|
defaultSegment.getDataSource(),
|
||||||
|
defaultSegment.getInterval(),
|
||||||
|
limit
|
||||||
|
)
|
||||||
|
);
|
||||||
|
Assert.assertEquals(limit, retreivedUnusedSegments.size());
|
||||||
|
Assert.assertTrue(SEGMENTS.containsAll(retreivedUnusedSegments));
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testUsedOverlapLow() throws IOException
|
public void testUsedOverlapLow() throws IOException
|
||||||
|
|
|
@ -422,7 +422,14 @@ public class OverlordClientImplTest
|
||||||
public void test_taskPayload() throws ExecutionException, InterruptedException, JsonProcessingException
|
public void test_taskPayload() throws ExecutionException, InterruptedException, JsonProcessingException
|
||||||
{
|
{
|
||||||
final String taskID = "taskId_1";
|
final String taskID = "taskId_1";
|
||||||
final ClientTaskQuery clientTaskQuery = new ClientKillUnusedSegmentsTaskQuery(taskID, "test", null, null, null);
|
final ClientTaskQuery clientTaskQuery = new ClientKillUnusedSegmentsTaskQuery(
|
||||||
|
taskID,
|
||||||
|
"test",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
serviceClient.expectAndRespond(
|
serviceClient.expectAndRespond(
|
||||||
new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/task/" + taskID),
|
new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/task/" + taskID),
|
||||||
|
|
|
@ -216,11 +216,13 @@ public class KillUnusedSegmentsTest
|
||||||
|
|
||||||
private void runAndVerifyKillInterval(Interval expectedKillInterval)
|
private void runAndVerifyKillInterval(Interval expectedKillInterval)
|
||||||
{
|
{
|
||||||
|
int limit = config.getCoordinatorKillMaxSegments();
|
||||||
target.run(params);
|
target.run(params);
|
||||||
Mockito.verify(overlordClient, Mockito.times(1)).runKillTask(
|
Mockito.verify(overlordClient, Mockito.times(1)).runKillTask(
|
||||||
ArgumentMatchers.anyString(),
|
ArgumentMatchers.anyString(),
|
||||||
ArgumentMatchers.eq("DS1"),
|
ArgumentMatchers.eq("DS1"),
|
||||||
ArgumentMatchers.eq(expectedKillInterval)
|
ArgumentMatchers.eq(expectedKillInterval),
|
||||||
|
ArgumentMatchers.eq(limit)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -593,7 +593,7 @@ public class DataSourcesResourceTest
|
||||||
Interval theInterval = Intervals.of(interval.replace('_', '/'));
|
Interval theInterval = Intervals.of(interval.replace('_', '/'));
|
||||||
|
|
||||||
OverlordClient overlordClient = EasyMock.createStrictMock(OverlordClient.class);
|
OverlordClient overlordClient = EasyMock.createStrictMock(OverlordClient.class);
|
||||||
EasyMock.expect(overlordClient.runKillTask("api-issued", "datasource1", theInterval))
|
EasyMock.expect(overlordClient.runKillTask("api-issued", "datasource1", theInterval, null))
|
||||||
.andReturn(Futures.immediateFuture(null));
|
.andReturn(Futures.immediateFuture(null));
|
||||||
EasyMock.replay(overlordClient, server);
|
EasyMock.replay(overlordClient, server);
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue