mirror of https://github.com/apache/druid.git
Index service: Fix kill task
This commit is contained in:
parent
553738e1d8
commit
82d77a8b72
|
@ -56,6 +56,6 @@ public class KillTask extends AbstractTask
|
|||
.kill(getDataSource(), getInterval())
|
||||
);
|
||||
|
||||
return TaskStatus.success(getId(), segmentsToKill);
|
||||
return TaskStatus.success(getId()).withSegmentsNuked(segmentsToKill);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,7 +39,7 @@ public class LocalTaskRunner implements TaskRunner
|
|||
private final TaskToolbox toolbox;
|
||||
private final ExecutorService exec;
|
||||
|
||||
private static final Logger log = new Logger(TaskQueue.class);
|
||||
private static final Logger log = new Logger(LocalTaskRunner.class);
|
||||
|
||||
public LocalTaskRunner(
|
||||
TaskToolbox toolbox,
|
||||
|
@ -66,11 +66,11 @@ public class LocalTaskRunner implements TaskRunner
|
|||
public void run()
|
||||
{
|
||||
final long startTime = System.currentTimeMillis();
|
||||
final File taskDir = toolbox.getConfig().getTaskDir(task);
|
||||
|
||||
TaskStatus status;
|
||||
|
||||
try {
|
||||
log.info("Running task: %s", task.getId());
|
||||
status = task.run(context, toolbox, callback);
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
|
@ -87,13 +87,15 @@ public class LocalTaskRunner implements TaskRunner
|
|||
}
|
||||
|
||||
try {
|
||||
final File taskDir = toolbox.getConfig().getTaskDir(task);
|
||||
|
||||
if (taskDir.exists()) {
|
||||
log.info("Removing task directory: %s", taskDir);
|
||||
FileUtils.deleteDirectory(taskDir);
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(e, "Failed to delete task directory[%s]", taskDir.toString());
|
||||
log.error(e, "Failed to delete task directory: %s", task.getId());
|
||||
}
|
||||
|
||||
try {
|
||||
|
|
|
@ -21,14 +21,13 @@ package com.metamx.druid.merger.coordinator.exec;
|
|||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.coordinator.TaskContext;
|
||||
import com.metamx.druid.merger.coordinator.TaskQueue;
|
||||
import com.metamx.druid.merger.coordinator.TaskRunner;
|
||||
|
@ -37,7 +36,6 @@ import com.metamx.emitter.EmittingLogger;
|
|||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
public class TaskConsumer implements Runnable
|
||||
|
@ -96,7 +94,7 @@ public class TaskConsumer implements Runnable
|
|||
}
|
||||
catch (InterruptedException e) {
|
||||
log.info(e, "Interrupted while waiting for new work");
|
||||
throw Throwables.propagate(e);
|
||||
throw e;
|
||||
}
|
||||
|
||||
try {
|
||||
|
@ -117,10 +115,10 @@ public class TaskConsumer implements Runnable
|
|||
}
|
||||
}
|
||||
}
|
||||
catch (Throwable t) {
|
||||
catch (Exception e) {
|
||||
// exit thread
|
||||
log.error(t, "Uncaught Throwable while consuming tasks");
|
||||
throw Throwables.propagate(t);
|
||||
log.error(e, "Uncaught exception while consuming tasks");
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -242,7 +240,18 @@ public class TaskConsumer implements Runnable
|
|||
private void deleteSegments(Task task, TaskContext context, Set<DataSegment> segments) throws Exception
|
||||
{
|
||||
for (DataSegment segment : segments) {
|
||||
verifySegment(task, context, segment);
|
||||
verifyDataSourceAndInterval(task, context, segment);
|
||||
|
||||
// Verify version (must be less than our context version)
|
||||
if (segment.getVersion().compareTo(context.getVersion()) >= 0) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Segment-to-nuke for task[%s] has invalid version: %s",
|
||||
task.getId(),
|
||||
segment.getIdentifier()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
log.info("Deleting segment[%s] for task[%s]", segment.getIdentifier(), task.getId());
|
||||
mergerDBCoordinator.deleteSegment(segment);
|
||||
|
@ -252,14 +261,25 @@ public class TaskConsumer implements Runnable
|
|||
private void publishSegments(Task task, TaskContext context, Set<DataSegment> segments) throws Exception
|
||||
{
|
||||
for (DataSegment segment : segments) {
|
||||
verifySegment(task, context, segment);
|
||||
verifyDataSourceAndInterval(task, context, segment);
|
||||
|
||||
// Verify version (must be equal to our context version)
|
||||
if (!context.getVersion().equals(segment.getVersion())) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Segment for task[%s] has invalid version: %s",
|
||||
task.getId(),
|
||||
segment.getIdentifier()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), task.getId());
|
||||
mergerDBCoordinator.announceHistoricalSegment(segment);
|
||||
}
|
||||
}
|
||||
|
||||
private void verifySegment(Task task, TaskContext context, DataSegment segment)
|
||||
private void verifyDataSourceAndInterval(Task task, TaskContext context, DataSegment segment)
|
||||
{
|
||||
if (!task.getDataSource().equals(segment.getDataSource())) {
|
||||
throw new IllegalStateException(
|
||||
|
@ -280,15 +300,5 @@ public class TaskConsumer implements Runnable
|
|||
)
|
||||
);
|
||||
}
|
||||
|
||||
if (!context.getVersion().equals(segment.getVersion())) {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Segment for task[%s] has invalid version: %s",
|
||||
task.getId(),
|
||||
segment.getIdentifier()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue