mirror of
https://github.com/apache/druid.git
synced 2025-02-17 15:35:56 +00:00
Some changes that make it possible to restart tasks on the same hardware.
This is done by killing and respawning the jvms rather than reconnecting to existing jvms, for a couple reasons. One is that it lets you restore tasks after server reboots too, and another is that it lets you upgrade all the software on a box at once by just restarting everything. The main changes are, 1) Add "canRestore" and "stopGracefully" methods to Tasks that say if a task can stop gracefully, and actually do a graceful stop. RealtimeIndexTask is the only one that currently implements this. 2) Add "stop" method to TaskRunners that attempts to do an orderly shutdown. ThreadPoolTaskRunner- call stopGracefully on restorable tasks, wait for exit ForkingTaskRunner- close output stream to restorable tasks, wait for exit RemoteTaskRunner- do nothing special, we actually don't want to shutdown 3) Add "restore" method to TaskRunners that attempts to bootstrap tasks from last run. Only ForkingTaskRunner does anything here. It maintains a "restore.json" file with a list of restorable tasks. 4) Have the CliPeon's ExecutorLifecycle lock the task base directory to avoid a restored task and a zombie old task from stomping on each other.
This commit is contained in:
parent
36569094ce
commit
501dcb43fa
@ -279,6 +279,8 @@ Additional peon configs include:
|
|||||||
|`druid.indexer.task.hadoopWorkingPath`|Temporary working directory for Hadoop tasks.|/tmp/druid-indexing|
|
|`druid.indexer.task.hadoopWorkingPath`|Temporary working directory for Hadoop tasks.|/tmp/druid-indexing|
|
||||||
|`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|50000|
|
|`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|50000|
|
||||||
|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.3.0|
|
|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.3.0|
|
||||||
|
|`druid.indexer.task.gracefulShutdownTimeout`|Wait this long on middleManager restart for restorable tasks to gracefully exit.|PT5M|
|
||||||
|
|`druid.indexer.task.directoryLockTimeout`|Wait this long for zombie peons to exit before giving up on their replacements.|PT10M|
|
||||||
|
|
||||||
If `druid.indexer.runner.separateIngestionEndpoint` is set to true then following configurations are available for the ingestion server at peon:
|
If `druid.indexer.runner.separateIngestionEndpoint` is set to true then following configurations are available for the ingestion server at peon:
|
||||||
|
|
||||||
|
@ -109,7 +109,7 @@ public class TaskToolboxFactory
|
|||||||
|
|
||||||
public TaskToolbox build(Task task)
|
public TaskToolbox build(Task task)
|
||||||
{
|
{
|
||||||
final File taskWorkDir = new File(new File(config.getBaseTaskDir(), task.getId()), "work");
|
final File taskWorkDir = config.getTaskWorkDir(task.getId());
|
||||||
|
|
||||||
return new TaskToolbox(
|
return new TaskToolbox(
|
||||||
config,
|
config,
|
||||||
|
@ -20,6 +20,7 @@ package io.druid.indexing.common.config;
|
|||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import org.joda.time.Period;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@ -30,6 +31,10 @@ public class TaskConfig
|
|||||||
"org.apache.hadoop:hadoop-client:2.3.0"
|
"org.apache.hadoop:hadoop-client:2.3.0"
|
||||||
);
|
);
|
||||||
|
|
||||||
|
private static final Period DEFAULT_DIRECTORY_LOCK_TIMEOUT = new Period("PT10M");
|
||||||
|
|
||||||
|
private static final Period DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT = new Period("PT5M");
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
private final String baseDir;
|
private final String baseDir;
|
||||||
|
|
||||||
@ -45,13 +50,21 @@ public class TaskConfig
|
|||||||
@JsonProperty
|
@JsonProperty
|
||||||
private final List<String> defaultHadoopCoordinates;
|
private final List<String> defaultHadoopCoordinates;
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
private final Period gracefulShutdownTimeout;
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
private final Period directoryLockTimeout;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public TaskConfig(
|
public TaskConfig(
|
||||||
@JsonProperty("baseDir") String baseDir,
|
@JsonProperty("baseDir") String baseDir,
|
||||||
@JsonProperty("baseTaskDir") String baseTaskDir,
|
@JsonProperty("baseTaskDir") String baseTaskDir,
|
||||||
@JsonProperty("hadoopWorkingPath") String hadoopWorkingPath,
|
@JsonProperty("hadoopWorkingPath") String hadoopWorkingPath,
|
||||||
@JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary,
|
@JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary,
|
||||||
@JsonProperty("defaultHadoopCoordinates") List<String> defaultHadoopCoordinates
|
@JsonProperty("defaultHadoopCoordinates") List<String> defaultHadoopCoordinates,
|
||||||
|
@JsonProperty("gracefulShutdownTimeout") Period gracefulShutdownTimeout,
|
||||||
|
@JsonProperty("directoryLockTimeout") Period directoryLockTimeout
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.baseDir = baseDir == null ? "/tmp" : baseDir;
|
this.baseDir = baseDir == null ? "/tmp" : baseDir;
|
||||||
@ -61,6 +74,12 @@ public class TaskConfig
|
|||||||
this.defaultHadoopCoordinates = defaultHadoopCoordinates == null
|
this.defaultHadoopCoordinates = defaultHadoopCoordinates == null
|
||||||
? DEFAULT_DEFAULT_HADOOP_COORDINATES
|
? DEFAULT_DEFAULT_HADOOP_COORDINATES
|
||||||
: defaultHadoopCoordinates;
|
: defaultHadoopCoordinates;
|
||||||
|
this.gracefulShutdownTimeout = gracefulShutdownTimeout == null
|
||||||
|
? DEFAULT_GRACEFUL_SHUTDOWN_TIMEOUT
|
||||||
|
: gracefulShutdownTimeout;
|
||||||
|
this.directoryLockTimeout = directoryLockTimeout == null
|
||||||
|
? DEFAULT_DIRECTORY_LOCK_TIMEOUT
|
||||||
|
: directoryLockTimeout;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
@ -75,6 +94,21 @@ public class TaskConfig
|
|||||||
return baseTaskDir;
|
return baseTaskDir;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public File getTaskDir(String taskId)
|
||||||
|
{
|
||||||
|
return new File(baseTaskDir, taskId);
|
||||||
|
}
|
||||||
|
|
||||||
|
public File getTaskWorkDir(String taskId)
|
||||||
|
{
|
||||||
|
return new File(getTaskDir(taskId), "work");
|
||||||
|
}
|
||||||
|
|
||||||
|
public File getTaskLockFile(String taskId)
|
||||||
|
{
|
||||||
|
return new File(getTaskDir(taskId), "lock");
|
||||||
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public String getHadoopWorkingPath()
|
public String getHadoopWorkingPath()
|
||||||
{
|
{
|
||||||
@ -93,6 +127,18 @@ public class TaskConfig
|
|||||||
return defaultHadoopCoordinates;
|
return defaultHadoopCoordinates;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public Period getGracefulShutdownTimeout()
|
||||||
|
{
|
||||||
|
return gracefulShutdownTimeout;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public Period getDirectoryLockTimeout()
|
||||||
|
{
|
||||||
|
return directoryLockTimeout;
|
||||||
|
}
|
||||||
|
|
||||||
private String defaultDir(String configParameter, final String defaultVal)
|
private String defaultDir(String configParameter, final String defaultVal)
|
||||||
{
|
{
|
||||||
if (configParameter == null) {
|
if (configParameter == null) {
|
||||||
|
@ -128,6 +128,19 @@ public abstract class AbstractTask implements Task
|
|||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean canRestore()
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void stopGracefully()
|
||||||
|
{
|
||||||
|
// Should not be called when canRestore = false.
|
||||||
|
throw new UnsupportedOperationException("Cannot stop gracefully");
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString()
|
public String toString()
|
||||||
{
|
{
|
||||||
|
@ -29,6 +29,7 @@ import com.metamx.common.parsers.ParseException;
|
|||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import io.druid.data.input.Committer;
|
import io.druid.data.input.Committer;
|
||||||
import io.druid.data.input.Firehose;
|
import io.druid.data.input.Firehose;
|
||||||
|
import io.druid.data.input.FirehoseFactory;
|
||||||
import io.druid.data.input.InputRow;
|
import io.druid.data.input.InputRow;
|
||||||
import io.druid.indexing.common.TaskLock;
|
import io.druid.indexing.common.TaskLock;
|
||||||
import io.druid.indexing.common.TaskStatus;
|
import io.druid.indexing.common.TaskStatus;
|
||||||
@ -48,6 +49,9 @@ import io.druid.segment.indexing.RealtimeTuningConfig;
|
|||||||
import io.druid.segment.realtime.FireDepartment;
|
import io.druid.segment.realtime.FireDepartment;
|
||||||
import io.druid.segment.realtime.RealtimeMetricsMonitor;
|
import io.druid.segment.realtime.RealtimeMetricsMonitor;
|
||||||
import io.druid.segment.realtime.SegmentPublisher;
|
import io.druid.segment.realtime.SegmentPublisher;
|
||||||
|
import io.druid.segment.realtime.firehose.ClippedFirehoseFactory;
|
||||||
|
import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory;
|
||||||
|
import io.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory;
|
||||||
import io.druid.segment.realtime.plumber.Committers;
|
import io.druid.segment.realtime.plumber.Committers;
|
||||||
import io.druid.segment.realtime.plumber.Plumber;
|
import io.druid.segment.realtime.plumber.Plumber;
|
||||||
import io.druid.segment.realtime.plumber.PlumberSchool;
|
import io.druid.segment.realtime.plumber.PlumberSchool;
|
||||||
@ -62,6 +66,7 @@ import java.io.File;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
|
||||||
public class RealtimeIndexTask extends AbstractTask
|
public class RealtimeIndexTask extends AbstractTask
|
||||||
{
|
{
|
||||||
@ -104,6 +109,12 @@ public class RealtimeIndexTask extends AbstractTask
|
|||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
private volatile Plumber plumber = null;
|
private volatile Plumber plumber = null;
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
private volatile Firehose firehose = null;
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
private volatile boolean stopped = false;
|
||||||
|
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null;
|
private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null;
|
||||||
|
|
||||||
@ -285,8 +296,6 @@ public class RealtimeIndexTask extends AbstractTask
|
|||||||
|
|
||||||
this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, fireDepartment.getMetrics());
|
this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, fireDepartment.getMetrics());
|
||||||
|
|
||||||
// Delay firehose connection to avoid claiming input resources while the plumber is starting up.
|
|
||||||
Firehose firehose = null;
|
|
||||||
Supplier<Committer> committerSupplier = null;
|
Supplier<Committer> committerSupplier = null;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
@ -295,12 +304,14 @@ public class RealtimeIndexTask extends AbstractTask
|
|||||||
// Set up metrics emission
|
// Set up metrics emission
|
||||||
toolbox.getMonitorScheduler().addMonitor(metricsMonitor);
|
toolbox.getMonitorScheduler().addMonitor(metricsMonitor);
|
||||||
|
|
||||||
// Set up firehose
|
// Delay firehose connection to avoid claiming input resources while the plumber is starting up.
|
||||||
firehose = spec.getIOConfig().getFirehoseFactory().connect(spec.getDataSchema().getParser());
|
final FirehoseFactory firehoseFactory = spec.getIOConfig().getFirehoseFactory();
|
||||||
|
final boolean firehoseDrainableByClosing = isFirehoseDrainableByClosing(firehoseFactory);
|
||||||
|
firehose = firehoseFactory.connect(spec.getDataSchema().getParser());
|
||||||
committerSupplier = Committers.supplierFromFirehose(firehose);
|
committerSupplier = Committers.supplierFromFirehose(firehose);
|
||||||
|
|
||||||
// Time to read data!
|
// Time to read data!
|
||||||
while (firehose.hasMore()) {
|
while ((!stopped || firehoseDrainableByClosing) && firehose.hasMore()) {
|
||||||
final InputRow inputRow;
|
final InputRow inputRow;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
@ -337,8 +348,38 @@ public class RealtimeIndexTask extends AbstractTask
|
|||||||
finally {
|
finally {
|
||||||
if (normalExit) {
|
if (normalExit) {
|
||||||
try {
|
try {
|
||||||
plumber.persist(committerSupplier.get());
|
if (!stopped) {
|
||||||
plumber.finishJob();
|
// Hand off all pending data
|
||||||
|
log.info("Persisting and handing off pending data.");
|
||||||
|
plumber.persist(committerSupplier.get());
|
||||||
|
plumber.finishJob();
|
||||||
|
} else {
|
||||||
|
log.info("Persisting pending data without handoff, in preparation for restart.");
|
||||||
|
final Committer committer = committerSupplier.get();
|
||||||
|
final CountDownLatch persistLatch = new CountDownLatch(1);
|
||||||
|
plumber.persist(
|
||||||
|
new Committer()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Object getMetadata()
|
||||||
|
{
|
||||||
|
return committer.getMetadata();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run()
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
committer.run();
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
persistLatch.countDown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
persistLatch.await();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.makeAlert(e, "Failed to finish realtime task").emit();
|
log.makeAlert(e, "Failed to finish realtime task").emit();
|
||||||
@ -352,15 +393,67 @@ public class RealtimeIndexTask extends AbstractTask
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
log.info("Job done!");
|
||||||
return TaskStatus.success(getId());
|
return TaskStatus.success(getId());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean canRestore()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void stopGracefully()
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
synchronized (this) {
|
||||||
|
if (!stopped) {
|
||||||
|
stopped = true;
|
||||||
|
log.info("Gracefully stopping.");
|
||||||
|
if (isFirehoseDrainableByClosing(spec.getIOConfig().getFirehoseFactory())) {
|
||||||
|
firehose.close();
|
||||||
|
} else {
|
||||||
|
log.debug("Cannot drain firehose[%s] by closing, so skipping closing.", firehose);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Public for tests.
|
||||||
|
*/
|
||||||
|
@JsonIgnore
|
||||||
|
public Firehose getFirehose()
|
||||||
|
{
|
||||||
|
return firehose;
|
||||||
|
}
|
||||||
|
|
||||||
@JsonProperty("spec")
|
@JsonProperty("spec")
|
||||||
public FireDepartment getRealtimeIngestionSchema()
|
public FireDepartment getRealtimeIngestionSchema()
|
||||||
{
|
{
|
||||||
return spec;
|
return spec;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Is a firehose from this factory drainable by closing it? If so, we should drain on stopGracefully rather than
|
||||||
|
* abruptly stopping.
|
||||||
|
* <p/>
|
||||||
|
* This is a hack to get around the fact that the Firehose and FirehoseFactory interfaces do not help us do this.
|
||||||
|
*/
|
||||||
|
private static boolean isFirehoseDrainableByClosing(FirehoseFactory firehoseFactory)
|
||||||
|
{
|
||||||
|
return firehoseFactory instanceof EventReceiverFirehoseFactory
|
||||||
|
|| (firehoseFactory instanceof TimedShutoffFirehoseFactory
|
||||||
|
&& isFirehoseDrainableByClosing(((TimedShutoffFirehoseFactory) firehoseFactory).getDelegateFactory()))
|
||||||
|
|| (firehoseFactory instanceof ClippedFirehoseFactory
|
||||||
|
&& isFirehoseDrainableByClosing(((ClippedFirehoseFactory) firehoseFactory).getDelegate()));
|
||||||
|
}
|
||||||
|
|
||||||
public static class TaskActionSegmentPublisher implements SegmentPublisher
|
public static class TaskActionSegmentPublisher implements SegmentPublisher
|
||||||
{
|
{
|
||||||
final Task task;
|
final Task task;
|
||||||
|
@ -62,6 +62,7 @@ public interface Task
|
|||||||
{
|
{
|
||||||
/**
|
/**
|
||||||
* Returns ID of this task. Must be unique across all tasks ever created.
|
* Returns ID of this task. Must be unique across all tasks ever created.
|
||||||
|
*
|
||||||
* @return task ID
|
* @return task ID
|
||||||
*/
|
*/
|
||||||
public String getId();
|
public String getId();
|
||||||
@ -69,6 +70,7 @@ public interface Task
|
|||||||
/**
|
/**
|
||||||
* Returns group ID of this task. Tasks with the same group ID can share locks. If tasks do not need to share locks,
|
* Returns group ID of this task. Tasks with the same group ID can share locks. If tasks do not need to share locks,
|
||||||
* a common convention is to set group ID equal to task ID.
|
* a common convention is to set group ID equal to task ID.
|
||||||
|
*
|
||||||
* @return task group ID
|
* @return task group ID
|
||||||
*/
|
*/
|
||||||
public String getGroupId();
|
public String getGroupId();
|
||||||
@ -76,12 +78,14 @@ public interface Task
|
|||||||
/**
|
/**
|
||||||
* Returns a {@link io.druid.indexing.common.task.TaskResource} for this task. Task resources define specific
|
* Returns a {@link io.druid.indexing.common.task.TaskResource} for this task. Task resources define specific
|
||||||
* worker requirements a task may require.
|
* worker requirements a task may require.
|
||||||
|
*
|
||||||
* @return {@link io.druid.indexing.common.task.TaskResource} for this task
|
* @return {@link io.druid.indexing.common.task.TaskResource} for this task
|
||||||
*/
|
*/
|
||||||
public TaskResource getTaskResource();
|
public TaskResource getTaskResource();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a descriptive label for this task type. Used for metrics emission and logging.
|
* Returns a descriptive label for this task type. Used for metrics emission and logging.
|
||||||
|
*
|
||||||
* @return task type label
|
* @return task type label
|
||||||
*/
|
*/
|
||||||
public String getType();
|
public String getType();
|
||||||
@ -90,7 +94,7 @@ public interface Task
|
|||||||
* Get the nodeType for if/when this task publishes on zookeeper.
|
* Get the nodeType for if/when this task publishes on zookeeper.
|
||||||
*
|
*
|
||||||
* @return the nodeType to use when publishing the server to zookeeper. null if the task doesn't expect to
|
* @return the nodeType to use when publishing the server to zookeeper. null if the task doesn't expect to
|
||||||
* publish to zookeeper.
|
* publish to zookeeper.
|
||||||
*/
|
*/
|
||||||
public String getNodeType();
|
public String getNodeType();
|
||||||
|
|
||||||
@ -102,7 +106,9 @@ public interface Task
|
|||||||
/**
|
/**
|
||||||
* Returns query runners for this task. If this task is not meant to answer queries over its datasource, this method
|
* Returns query runners for this task. If this task is not meant to answer queries over its datasource, this method
|
||||||
* should return null.
|
* should return null.
|
||||||
|
*
|
||||||
* @param <T> query result type
|
* @param <T> query result type
|
||||||
|
*
|
||||||
* @return query runners for this task
|
* @return query runners for this task
|
||||||
*/
|
*/
|
||||||
public <T> QueryRunner<T> getQueryRunner(Query<T> query);
|
public <T> QueryRunner<T> getQueryRunner(Query<T> query);
|
||||||
@ -117,7 +123,7 @@ public interface Task
|
|||||||
* Execute preflight actions for a task. This can be used to acquire locks, check preconditions, and so on. The
|
* Execute preflight actions for a task. This can be used to acquire locks, check preconditions, and so on. The
|
||||||
* actions must be idempotent, since this method may be executed multiple times. This typically runs on the
|
* actions must be idempotent, since this method may be executed multiple times. This typically runs on the
|
||||||
* coordinator. If this method throws an exception, the task should be considered a failure.
|
* coordinator. If this method throws an exception, the task should be considered a failure.
|
||||||
*
|
* <p/>
|
||||||
* This method must be idempotent, as it may be run multiple times per task.
|
* This method must be idempotent, as it may be run multiple times per task.
|
||||||
*
|
*
|
||||||
* @param taskActionClient action client for this task (not the full toolbox)
|
* @param taskActionClient action client for this task (not the full toolbox)
|
||||||
@ -128,6 +134,20 @@ public interface Task
|
|||||||
*/
|
*/
|
||||||
public boolean isReady(TaskActionClient taskActionClient) throws Exception;
|
public boolean isReady(TaskActionClient taskActionClient) throws Exception;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns whether or not this task can restore its progress from its on-disk working directory. Restorable tasks
|
||||||
|
* may be started with a non-empty working directory. Tasks that exit uncleanly may still have a chance to attempt
|
||||||
|
* restores, meaning that restorable tasks should be able to deal with potentially partially written on-disk state.
|
||||||
|
*/
|
||||||
|
public boolean canRestore();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Asks a task to arrange for its "run" method to exit promptly. This method will only be called if
|
||||||
|
* {@link #canRestore()} returns true. Tasks that take too long to stop gracefully will be terminated with
|
||||||
|
* extreme prejudice.
|
||||||
|
*/
|
||||||
|
public void stopGracefully();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Execute a task. This typically runs on a worker as determined by a TaskRunner, and will be run while
|
* Execute a task. This typically runs on a worker as determined by a TaskRunner, and will be run while
|
||||||
* holding a lock on our dataSource and implicit lock interval (if any). If this method throws an exception, the task
|
* holding a lock on our dataSource and implicit lock interval (if any). If this method throws an exception, the task
|
||||||
|
@ -17,6 +17,8 @@
|
|||||||
|
|
||||||
package io.druid.indexing.overlord;
|
package io.druid.indexing.overlord;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.CharMatcher;
|
import com.google.common.base.CharMatcher;
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
@ -28,9 +30,11 @@ import com.google.common.collect.ImmutableList;
|
|||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
|
import com.google.common.io.ByteSink;
|
||||||
import com.google.common.io.ByteSource;
|
import com.google.common.io.ByteSource;
|
||||||
import com.google.common.io.ByteStreams;
|
import com.google.common.io.ByteStreams;
|
||||||
import com.google.common.io.Closer;
|
import com.google.common.io.Closer;
|
||||||
|
import com.google.common.io.FileWriteMode;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||||
@ -40,6 +44,7 @@ import com.metamx.common.ISE;
|
|||||||
import com.metamx.common.Pair;
|
import com.metamx.common.Pair;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
|
import io.druid.concurrent.Execs;
|
||||||
import io.druid.guice.annotations.Self;
|
import io.druid.guice.annotations.Self;
|
||||||
import io.druid.indexing.common.TaskStatus;
|
import io.druid.indexing.common.TaskStatus;
|
||||||
import io.druid.indexing.common.config.TaskConfig;
|
import io.druid.indexing.common.config.TaskConfig;
|
||||||
@ -51,6 +56,8 @@ import io.druid.server.DruidNode;
|
|||||||
import io.druid.tasklogs.TaskLogPusher;
|
import io.druid.tasklogs.TaskLogPusher;
|
||||||
import io.druid.tasklogs.TaskLogStreamer;
|
import io.druid.tasklogs.TaskLogStreamer;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
@ -64,6 +71,7 @@ import java.util.Properties;
|
|||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Runs tasks in separate processes using the "internal peon" verb.
|
* Runs tasks in separate processes using the "internal peon" verb.
|
||||||
@ -72,6 +80,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
{
|
{
|
||||||
private static final EmittingLogger log = new EmittingLogger(ForkingTaskRunner.class);
|
private static final EmittingLogger log = new EmittingLogger(ForkingTaskRunner.class);
|
||||||
private static final String CHILD_PROPERTY_PREFIX = "druid.indexer.fork.property.";
|
private static final String CHILD_PROPERTY_PREFIX = "druid.indexer.fork.property.";
|
||||||
|
private static final String TASK_RESTORE_FILENAME = "restore.json";
|
||||||
private final ForkingTaskRunnerConfig config;
|
private final ForkingTaskRunnerConfig config;
|
||||||
private final TaskConfig taskConfig;
|
private final TaskConfig taskConfig;
|
||||||
private final Properties props;
|
private final Properties props;
|
||||||
@ -83,6 +92,8 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
|
|
||||||
private final Map<String, ForkingTaskRunnerWorkItem> tasks = Maps.newHashMap();
|
private final Map<String, ForkingTaskRunnerWorkItem> tasks = Maps.newHashMap();
|
||||||
|
|
||||||
|
private volatile boolean stopping = false;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public ForkingTaskRunner(
|
public ForkingTaskRunner(
|
||||||
ForkingTaskRunnerConfig config,
|
ForkingTaskRunnerConfig config,
|
||||||
@ -102,7 +113,51 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
this.node = node;
|
this.node = node;
|
||||||
this.portFinder = new PortFinder(config.getStartPort());
|
this.portFinder = new PortFinder(config.getStartPort());
|
||||||
|
|
||||||
this.exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(workerConfig.getCapacity()));
|
this.exec = MoreExecutors.listeningDecorator(
|
||||||
|
Execs.multiThreaded(workerConfig.getCapacity(), "forking-task-runner-%d")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<Pair<Task, ListenableFuture<TaskStatus>>> restore()
|
||||||
|
{
|
||||||
|
final File restoreFile = getRestoreFile();
|
||||||
|
final TaskRestoreInfo taskRestoreInfo;
|
||||||
|
if (restoreFile.exists()) {
|
||||||
|
try {
|
||||||
|
taskRestoreInfo = jsonMapper.readValue(restoreFile, TaskRestoreInfo.class);
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
log.error(e, "Failed to read restorable tasks from file[%s]. Skipping restore.", restoreFile);
|
||||||
|
return ImmutableList.of();
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
return ImmutableList.of();
|
||||||
|
}
|
||||||
|
|
||||||
|
final List<Pair<Task, ListenableFuture<TaskStatus>>> retVal = Lists.newArrayList();
|
||||||
|
for (final String taskId : taskRestoreInfo.getRunningTasks()) {
|
||||||
|
try {
|
||||||
|
final File taskFile = new File(taskConfig.getTaskDir(taskId), "task.json");
|
||||||
|
final Task task = jsonMapper.readValue(taskFile, Task.class);
|
||||||
|
|
||||||
|
if (!task.getId().equals(taskId)) {
|
||||||
|
throw new ISE("WTF?! Task[%s] restore file had wrong id[%s].", taskId, task.getId());
|
||||||
|
}
|
||||||
|
|
||||||
|
if (task.canRestore()) {
|
||||||
|
log.info("Restoring task[%s].", task.getId());
|
||||||
|
retVal.add(Pair.of(task, run(task)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
log.warn(e, "Failed to restore task[%s]. Trying to restore other tasks.", taskId);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
log.info("Restored %,d tasks.", retVal.size());
|
||||||
|
|
||||||
|
return retVal;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -113,7 +168,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
tasks.put(
|
tasks.put(
|
||||||
task.getId(),
|
task.getId(),
|
||||||
new ForkingTaskRunnerWorkItem(
|
new ForkingTaskRunnerWorkItem(
|
||||||
task.getId(),
|
task,
|
||||||
exec.submit(
|
exec.submit(
|
||||||
new Callable<TaskStatus>()
|
new Callable<TaskStatus>()
|
||||||
{
|
{
|
||||||
@ -121,7 +176,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
public TaskStatus call()
|
public TaskStatus call()
|
||||||
{
|
{
|
||||||
final String attemptUUID = UUID.randomUUID().toString();
|
final String attemptUUID = UUID.randomUUID().toString();
|
||||||
final File taskDir = new File(taskConfig.getBaseTaskDir(), task.getId());
|
final File taskDir = taskConfig.getTaskDir(task.getId());
|
||||||
final File attemptDir = new File(taskDir, attemptUUID);
|
final File attemptDir = new File(taskDir, attemptUUID);
|
||||||
|
|
||||||
final ProcessHolder processHolder;
|
final ProcessHolder processHolder;
|
||||||
@ -144,9 +199,9 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
throw new IOException(String.format("Could not create directories: %s", attemptDir));
|
throw new IOException(String.format("Could not create directories: %s", attemptDir));
|
||||||
}
|
}
|
||||||
|
|
||||||
final File taskFile = new File(attemptDir, "task.json");
|
final File taskFile = new File(taskDir, "task.json");
|
||||||
final File statusFile = new File(attemptDir, "status.json");
|
final File statusFile = new File(attemptDir, "status.json");
|
||||||
final File logFile = new File(attemptDir, "log");
|
final File logFile = new File(taskDir, "log");
|
||||||
|
|
||||||
// time to adjust process holders
|
// time to adjust process holders
|
||||||
synchronized (tasks) {
|
synchronized (tasks) {
|
||||||
@ -245,12 +300,18 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
command.add(String.format("-Ddruid.host=%s", childHost));
|
command.add(String.format("-Ddruid.host=%s", childHost));
|
||||||
command.add(String.format("-Ddruid.port=%d", childPort));
|
command.add(String.format("-Ddruid.port=%d", childPort));
|
||||||
|
|
||||||
if(config.isSeparateIngestionEndpoint()) {
|
if (config.isSeparateIngestionEndpoint()) {
|
||||||
command.add(String.format("-Ddruid.indexer.task.chathandler.service=%s", "placeholder/serviceName"));
|
command.add(String.format(
|
||||||
|
"-Ddruid.indexer.task.chathandler.service=%s",
|
||||||
|
"placeholder/serviceName"
|
||||||
|
));
|
||||||
// Actual serviceName will be passed by the EventReceiverFirehose when it registers itself with ChatHandlerProvider
|
// Actual serviceName will be passed by the EventReceiverFirehose when it registers itself with ChatHandlerProvider
|
||||||
// Thus, "placeholder/serviceName" will be ignored
|
// Thus, "placeholder/serviceName" will be ignored
|
||||||
command.add(String.format("-Ddruid.indexer.task.chathandler.host=%s", childHost));
|
command.add(String.format("-Ddruid.indexer.task.chathandler.host=%s", childHost));
|
||||||
command.add(String.format("-Ddruid.indexer.task.chathandler.port=%d", childChatHandlerPort));
|
command.add(String.format(
|
||||||
|
"-Ddruid.indexer.task.chathandler.port=%d",
|
||||||
|
childChatHandlerPort
|
||||||
|
));
|
||||||
}
|
}
|
||||||
|
|
||||||
command.add("io.druid.cli.Main");
|
command.add("io.druid.cli.Main");
|
||||||
@ -264,7 +325,9 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
command.add(nodeType);
|
command.add(nodeType);
|
||||||
}
|
}
|
||||||
|
|
||||||
jsonMapper.writeValue(taskFile, task);
|
if (!taskFile.exists()) {
|
||||||
|
jsonMapper.writeValue(taskFile, task);
|
||||||
|
}
|
||||||
|
|
||||||
log.info("Running command: %s", Joiner.on(" ").join(command));
|
log.info("Running command: %s", Joiner.on(" ").join(command));
|
||||||
taskWorkItem.processHolder = new ProcessHolder(
|
taskWorkItem.processHolder = new ProcessHolder(
|
||||||
@ -280,7 +343,8 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
log.info("Logging task %s output to: %s", task.getId(), logFile);
|
log.info("Logging task %s output to: %s", task.getId(), logFile);
|
||||||
boolean runFailed = true;
|
boolean runFailed = true;
|
||||||
|
|
||||||
try (final OutputStream toLogfile = Files.asByteSink(logFile).openStream()) {
|
final ByteSink logSink = Files.asByteSink(logFile, FileWriteMode.APPEND);
|
||||||
|
try (final OutputStream toLogfile = logSink.openStream()) {
|
||||||
ByteStreams.copy(processHolder.process.getInputStream(), toLogfile);
|
ByteStreams.copy(processHolder.process.getInputStream(), toLogfile);
|
||||||
final int statusCode = processHolder.process.waitFor();
|
final int statusCode = processHolder.process.waitFor();
|
||||||
log.info("Process exited with status[%d] for task: %s", statusCode, task.getId());
|
log.info("Process exited with status[%d] for task: %s", statusCode, task.getId());
|
||||||
@ -319,13 +383,27 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
if (taskWorkItem != null && taskWorkItem.processHolder != null) {
|
if (taskWorkItem != null && taskWorkItem.processHolder != null) {
|
||||||
taskWorkItem.processHolder.process.destroy();
|
taskWorkItem.processHolder.process.destroy();
|
||||||
}
|
}
|
||||||
|
if (!stopping) {
|
||||||
|
saveRunningTasks();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
portFinder.markPortUnused(childPort);
|
|
||||||
if(childChatHandlerPort > 0) {
|
if (childChatHandlerPort > 0) {
|
||||||
portFinder.markPortUnused(childChatHandlerPort);
|
portFinder.markPortUnused(childChatHandlerPort);
|
||||||
}
|
}
|
||||||
log.info("Removing temporary directory: %s", attemptDir);
|
|
||||||
FileUtils.deleteDirectory(attemptDir);
|
try {
|
||||||
|
if (!stopping && taskDir.exists()) {
|
||||||
|
log.info("Removing task directory: %s", taskDir);
|
||||||
|
FileUtils.deleteDirectory(taskDir);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
log.makeAlert(e, "Failed to delete task directory")
|
||||||
|
.addData("taskDir", taskDir.toString())
|
||||||
|
.addData("task", task.getId())
|
||||||
|
.emit();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.error(e, "Suppressing exception caught while cleaning up task");
|
log.error(e, "Suppressing exception caught while cleaning up task");
|
||||||
@ -337,7 +415,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
saveRunningTasks();
|
||||||
return tasks.get(task.getId()).getResult();
|
return tasks.get(task.getId()).getResult();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -345,16 +423,41 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
@LifecycleStop
|
@LifecycleStop
|
||||||
public void stop()
|
public void stop()
|
||||||
{
|
{
|
||||||
synchronized (tasks) {
|
stopping = true;
|
||||||
exec.shutdown();
|
exec.shutdown();
|
||||||
|
|
||||||
|
synchronized (tasks) {
|
||||||
for (ForkingTaskRunnerWorkItem taskWorkItem : tasks.values()) {
|
for (ForkingTaskRunnerWorkItem taskWorkItem : tasks.values()) {
|
||||||
if (taskWorkItem.processHolder != null) {
|
if (taskWorkItem.processHolder != null) {
|
||||||
log.info("Destroying process: %s", taskWorkItem.processHolder.process);
|
log.info("Closing output stream to task[%s].", taskWorkItem.getTask().getId());
|
||||||
taskWorkItem.processHolder.process.destroy();
|
try {
|
||||||
|
taskWorkItem.processHolder.process.getOutputStream().close();
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
log.warn(e, "Failed to close stdout to task[%s]. Destroying task.", taskWorkItem.getTask().getId());
|
||||||
|
taskWorkItem.processHolder.process.destroy();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final DateTime start = new DateTime();
|
||||||
|
final long timeout = new Interval(start, taskConfig.getGracefulShutdownTimeout()).toDurationMillis();
|
||||||
|
|
||||||
|
// Things should be terminating now. Wait for it to happen so logs can be uploaded and all that good stuff.
|
||||||
|
log.info("Waiting %,dms for shutdown.", timeout);
|
||||||
|
if (timeout > 0) {
|
||||||
|
try {
|
||||||
|
exec.awaitTermination(timeout, TimeUnit.MILLISECONDS);
|
||||||
|
log.info("Finished stopping in %,dms.", System.currentTimeMillis() - start.getMillis());
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
log.warn(e, "Interrupted while waiting for executor to finish.");
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
log.warn("Ran out of time, not waiting for executor to finish!");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -448,17 +551,68 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Save running tasks to a file, so they can potentially be restored on next startup. Suppresses exceptions that
|
||||||
|
// occur while saving.
|
||||||
|
private void saveRunningTasks()
|
||||||
|
{
|
||||||
|
final File restoreFile = getRestoreFile();
|
||||||
|
final List<String> theTasks = Lists.newArrayList();
|
||||||
|
for (ForkingTaskRunnerWorkItem forkingTaskRunnerWorkItem : tasks.values()) {
|
||||||
|
theTasks.add(forkingTaskRunnerWorkItem.getTaskId());
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
Files.createParentDirs(restoreFile);
|
||||||
|
jsonMapper.writeValue(restoreFile, new TaskRestoreInfo(theTasks));
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
log.warn(e, "Failed to save tasks to restore file[%s]. Skipping this save.", restoreFile);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private File getRestoreFile()
|
||||||
|
{
|
||||||
|
return new File(taskConfig.getBaseTaskDir(), TASK_RESTORE_FILENAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class TaskRestoreInfo
|
||||||
|
{
|
||||||
|
@JsonProperty
|
||||||
|
private final List<String> runningTasks;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public TaskRestoreInfo(
|
||||||
|
@JsonProperty("runningTasks") List<String> runningTasks
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.runningTasks = runningTasks;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<String> getRunningTasks()
|
||||||
|
{
|
||||||
|
return runningTasks;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private static class ForkingTaskRunnerWorkItem extends TaskRunnerWorkItem
|
private static class ForkingTaskRunnerWorkItem extends TaskRunnerWorkItem
|
||||||
{
|
{
|
||||||
|
private final Task task;
|
||||||
|
|
||||||
private volatile boolean shutdown = false;
|
private volatile boolean shutdown = false;
|
||||||
private volatile ProcessHolder processHolder = null;
|
private volatile ProcessHolder processHolder = null;
|
||||||
|
|
||||||
private ForkingTaskRunnerWorkItem(
|
private ForkingTaskRunnerWorkItem(
|
||||||
String taskId,
|
Task task,
|
||||||
ListenableFuture<TaskStatus> statusFuture
|
ListenableFuture<TaskStatus> statusFuture
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(taskId, statusFuture);
|
super(task.getId(), statusFuture);
|
||||||
|
this.task = task;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Task getTask()
|
||||||
|
{
|
||||||
|
return task;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -40,6 +40,7 @@ import com.google.common.util.concurrent.ListeningScheduledExecutorService;
|
|||||||
import com.google.common.util.concurrent.MoreExecutors;
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
import com.google.common.util.concurrent.SettableFuture;
|
import com.google.common.util.concurrent.SettableFuture;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
|
import com.metamx.common.Pair;
|
||||||
import com.metamx.common.RE;
|
import com.metamx.common.RE;
|
||||||
import com.metamx.common.lifecycle.LifecycleStart;
|
import com.metamx.common.lifecycle.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
@ -305,6 +306,12 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<Pair<Task, ListenableFuture<TaskStatus>>> restore()
|
||||||
|
{
|
||||||
|
return ImmutableList.of();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Collection<ZkWorker> getWorkers()
|
public Collection<ZkWorker> getWorkers()
|
||||||
{
|
{
|
||||||
|
@ -213,6 +213,9 @@ public class TaskQueue
|
|||||||
log.info("Beginning management in %s.", config.getStartDelay());
|
log.info("Beginning management in %s.", config.getStartDelay());
|
||||||
Thread.sleep(config.getStartDelay().getMillis());
|
Thread.sleep(config.getStartDelay().getMillis());
|
||||||
|
|
||||||
|
// Ignore return value- we'll get the IDs and futures from getKnownTasks later.
|
||||||
|
taskRunner.restore();
|
||||||
|
|
||||||
while (active) {
|
while (active) {
|
||||||
giant.lock();
|
giant.lock();
|
||||||
|
|
||||||
|
@ -18,16 +18,24 @@
|
|||||||
package io.druid.indexing.overlord;
|
package io.druid.indexing.overlord;
|
||||||
|
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
|
import com.metamx.common.Pair;
|
||||||
import io.druid.indexing.common.TaskStatus;
|
import io.druid.indexing.common.TaskStatus;
|
||||||
import io.druid.indexing.common.task.Task;
|
import io.druid.indexing.common.task.Task;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Interface for handing off tasks. Managed by a {@link io.druid.indexing.overlord.TaskQueue}.
|
* Interface for handing off tasks. Managed by a {@link io.druid.indexing.overlord.TaskQueue}.
|
||||||
*/
|
*/
|
||||||
public interface TaskRunner
|
public interface TaskRunner
|
||||||
{
|
{
|
||||||
|
/**
|
||||||
|
* Some task runners can restart previously-running tasks after being bounced. This method does that, and returns
|
||||||
|
* the list of tasks (and status futures).
|
||||||
|
*/
|
||||||
|
public List<Pair<Task, ListenableFuture<TaskStatus>>> restore();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Run a task. The returned status should be some kind of completed status.
|
* Run a task. The returned status should be some kind of completed status.
|
||||||
*
|
*
|
||||||
@ -45,6 +53,12 @@ public interface TaskRunner
|
|||||||
*/
|
*/
|
||||||
public void shutdown(String taskid);
|
public void shutdown(String taskid);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Stop this task runner. This may block until currently-running tasks can be gracefully stopped. After calling
|
||||||
|
* stopping, "run" will not accept further tasks.
|
||||||
|
*/
|
||||||
|
public void stop();
|
||||||
|
|
||||||
public Collection<? extends TaskRunnerWorkItem> getRunningTasks();
|
public Collection<? extends TaskRunnerWorkItem> getRunningTasks();
|
||||||
|
|
||||||
public Collection<? extends TaskRunnerWorkItem> getPendingTasks();
|
public Collection<? extends TaskRunnerWorkItem> getPendingTasks();
|
||||||
|
@ -28,53 +28,121 @@ import com.google.common.util.concurrent.ListenableFuture;
|
|||||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||||
import com.google.common.util.concurrent.MoreExecutors;
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
import com.metamx.common.Pair;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
|
import com.metamx.emitter.service.AlertEvent;
|
||||||
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
|
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||||
import io.druid.concurrent.Execs;
|
import io.druid.concurrent.Execs;
|
||||||
import io.druid.indexing.common.TaskStatus;
|
import io.druid.indexing.common.TaskStatus;
|
||||||
import io.druid.indexing.common.TaskToolbox;
|
import io.druid.indexing.common.TaskToolbox;
|
||||||
import io.druid.indexing.common.TaskToolboxFactory;
|
import io.druid.indexing.common.TaskToolboxFactory;
|
||||||
|
import io.druid.indexing.common.config.TaskConfig;
|
||||||
import io.druid.indexing.common.task.Task;
|
import io.druid.indexing.common.task.Task;
|
||||||
import io.druid.query.NoopQueryRunner;
|
import io.druid.query.NoopQueryRunner;
|
||||||
import io.druid.query.Query;
|
import io.druid.query.Query;
|
||||||
import io.druid.query.QueryRunner;
|
import io.druid.query.QueryRunner;
|
||||||
import io.druid.query.QueryRunnerFactoryConglomerate;
|
|
||||||
import io.druid.query.QuerySegmentWalker;
|
import io.druid.query.QuerySegmentWalker;
|
||||||
import io.druid.query.SegmentDescriptor;
|
import io.druid.query.SegmentDescriptor;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.io.File;
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.ConcurrentSkipListSet;
|
import java.util.concurrent.ConcurrentSkipListSet;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Runs tasks in a JVM thread using an ExecutorService.
|
* Runs tasks in a JVM thread using an ExecutorService.
|
||||||
*/
|
*/
|
||||||
public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
|
public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
|
||||||
{
|
{
|
||||||
|
private static final EmittingLogger log = new EmittingLogger(ThreadPoolTaskRunner.class);
|
||||||
|
|
||||||
private final TaskToolboxFactory toolboxFactory;
|
private final TaskToolboxFactory toolboxFactory;
|
||||||
|
private final TaskConfig taskConfig;
|
||||||
private final ListeningExecutorService exec;
|
private final ListeningExecutorService exec;
|
||||||
private final Set<ThreadPoolTaskRunnerWorkItem> runningItems = new ConcurrentSkipListSet<>();
|
private final Set<ThreadPoolTaskRunnerWorkItem> runningItems = new ConcurrentSkipListSet<>();
|
||||||
private final QueryRunnerFactoryConglomerate conglomerate;
|
private final ServiceEmitter emitter;
|
||||||
private static final EmittingLogger log = new EmittingLogger(ThreadPoolTaskRunner.class);
|
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public ThreadPoolTaskRunner(
|
public ThreadPoolTaskRunner(
|
||||||
TaskToolboxFactory toolboxFactory,
|
TaskToolboxFactory toolboxFactory,
|
||||||
QueryRunnerFactoryConglomerate conglomerate
|
TaskConfig taskConfig,
|
||||||
|
ServiceEmitter emitter
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.toolboxFactory = Preconditions.checkNotNull(toolboxFactory, "toolboxFactory");
|
this.toolboxFactory = Preconditions.checkNotNull(toolboxFactory, "toolboxFactory");
|
||||||
|
this.taskConfig = taskConfig;
|
||||||
this.exec = MoreExecutors.listeningDecorator(Execs.singleThreaded("task-runner-%d"));
|
this.exec = MoreExecutors.listeningDecorator(Execs.singleThreaded("task-runner-%d"));
|
||||||
this.conglomerate = conglomerate;
|
this.emitter = Preconditions.checkNotNull(emitter, "emitter");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<Pair<Task, ListenableFuture<TaskStatus>>> restore()
|
||||||
|
{
|
||||||
|
return ImmutableList.of();
|
||||||
}
|
}
|
||||||
|
|
||||||
@LifecycleStop
|
@LifecycleStop
|
||||||
public void stop()
|
public void stop()
|
||||||
{
|
{
|
||||||
|
exec.shutdown();
|
||||||
|
|
||||||
|
for (ThreadPoolTaskRunnerWorkItem item : runningItems) {
|
||||||
|
final Task task = item.getTask();
|
||||||
|
final long start = System.currentTimeMillis();
|
||||||
|
final boolean graceful;
|
||||||
|
final long elapsed;
|
||||||
|
boolean error = false;
|
||||||
|
|
||||||
|
if (task.canRestore()) {
|
||||||
|
// Attempt graceful shutdown.
|
||||||
|
graceful = true;
|
||||||
|
log.info("Starting graceful shutdown of task[%s].", task.getId());
|
||||||
|
|
||||||
|
try {
|
||||||
|
task.stopGracefully();
|
||||||
|
final TaskStatus taskStatus = item.getResult().get(
|
||||||
|
new Interval(new DateTime(start), taskConfig.getGracefulShutdownTimeout()).toDurationMillis(),
|
||||||
|
TimeUnit.MILLISECONDS
|
||||||
|
);
|
||||||
|
log.info(
|
||||||
|
"Graceful shutdown of task[%s] finished in %,dms with status[%s].",
|
||||||
|
task.getId(),
|
||||||
|
System.currentTimeMillis() - start,
|
||||||
|
taskStatus.getStatusCode()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
log.makeAlert(e, "Graceful task shutdown failed: %s", task.getDataSource())
|
||||||
|
.addData("taskId", task.getId())
|
||||||
|
.addData("dataSource", task.getDataSource())
|
||||||
|
.emit();
|
||||||
|
log.warn(e, "Graceful shutdown of task[%s] aborted with exception.");
|
||||||
|
error = true;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
graceful = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
elapsed = System.currentTimeMillis() - start;
|
||||||
|
|
||||||
|
final ServiceMetricEvent.Builder metricBuilder = ServiceMetricEvent
|
||||||
|
.builder()
|
||||||
|
.setDimension("task", task.getId())
|
||||||
|
.setDimension("dataSource", task.getDataSource())
|
||||||
|
.setDimension("graceful", String.valueOf(graceful))
|
||||||
|
.setDimension("error", String.valueOf(error));
|
||||||
|
|
||||||
|
emitter.emit(metricBuilder.build("task/interrupt/count", 1L));
|
||||||
|
emitter.emit(metricBuilder.build("task/interrupt/elapsed", elapsed));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Ok, now interrupt everything.
|
||||||
exec.shutdownNow();
|
exec.shutdownNow();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -210,7 +278,6 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
|
|||||||
public TaskStatus call()
|
public TaskStatus call()
|
||||||
{
|
{
|
||||||
final long startTime = System.currentTimeMillis();
|
final long startTime = System.currentTimeMillis();
|
||||||
final File taskDir = toolbox.getTaskWorkDir();
|
|
||||||
|
|
||||||
TaskStatus status;
|
TaskStatus status;
|
||||||
|
|
||||||
@ -231,19 +298,6 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
|
|||||||
throw Throwables.propagate(t);
|
throw Throwables.propagate(t);
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
|
||||||
if (taskDir.exists()) {
|
|
||||||
log.info("Removing task directory: %s", taskDir);
|
|
||||||
FileUtils.deleteDirectory(taskDir);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (Exception e) {
|
|
||||||
log.makeAlert(e, "Failed to delete task directory")
|
|
||||||
.addData("taskDir", taskDir.toString())
|
|
||||||
.addData("task", task.getId())
|
|
||||||
.emit();
|
|
||||||
}
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
return status.withDuration(System.currentTimeMillis() - startTime);
|
return status.withDuration(System.currentTimeMillis() - startTime);
|
||||||
}
|
}
|
||||||
|
@ -18,7 +18,10 @@
|
|||||||
package io.druid.indexing.worker;
|
package io.druid.indexing.worker;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.google.api.client.util.Sets;
|
||||||
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
import com.metamx.common.Pair;
|
||||||
import com.metamx.common.lifecycle.LifecycleStart;
|
import com.metamx.common.lifecycle.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
@ -28,18 +31,20 @@ import io.druid.indexing.common.task.Task;
|
|||||||
import io.druid.indexing.overlord.TaskRunner;
|
import io.druid.indexing.overlord.TaskRunner;
|
||||||
import io.druid.indexing.worker.config.WorkerConfig;
|
import io.druid.indexing.worker.config.WorkerConfig;
|
||||||
import org.apache.curator.framework.CuratorFramework;
|
import org.apache.curator.framework.CuratorFramework;
|
||||||
|
import org.apache.curator.framework.listen.Listenable;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The monitor watches ZK at a specified path for new tasks to appear. Upon starting the monitor, a listener will be
|
* The monitor watches ZK at a specified path for new tasks to appear. Upon starting the monitor, a listener will be
|
||||||
* created that waits for new tasks. Tasks are executed as soon as they are seen.
|
* created that waits for new tasks. Tasks are executed as soon as they are seen.
|
||||||
*
|
* <p/>
|
||||||
* The monitor implements {@link io.druid.query.QuerySegmentWalker} so tasks can offer up queryable data. This is useful for
|
* The monitor implements {@link io.druid.query.QuerySegmentWalker} so tasks can offer up queryable data. This is useful for
|
||||||
* realtime index tasks.
|
* realtime index tasks.
|
||||||
*/
|
*/
|
||||||
@ -84,14 +89,20 @@ public class WorkerTaskMonitor
|
|||||||
public void start()
|
public void start()
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
|
// restore restorable tasks
|
||||||
|
final List<Pair<Task, ListenableFuture<TaskStatus>>> restored = taskRunner.restore();
|
||||||
|
for (Pair<Task, ListenableFuture<TaskStatus>> pair : restored) {
|
||||||
|
submitTaskRunnable(pair.lhs, pair.rhs);
|
||||||
|
}
|
||||||
|
|
||||||
// cleanup any old running task announcements which are invalid after restart
|
// cleanup any old running task announcements which are invalid after restart
|
||||||
for (TaskAnnouncement announcement : workerCuratorCoordinator.getAnnouncements()){
|
for (TaskAnnouncement announcement : workerCuratorCoordinator.getAnnouncements()) {
|
||||||
if(announcement.getTaskStatus().isRunnable()) {
|
if (!isTaskRunning(announcement.getTaskStatus().getId()) && announcement.getTaskStatus().isRunnable()) {
|
||||||
workerCuratorCoordinator.updateAnnouncement(
|
workerCuratorCoordinator.updateAnnouncement(
|
||||||
TaskAnnouncement.create(
|
TaskAnnouncement.create(
|
||||||
announcement.getTaskId(),
|
announcement.getTaskStatus().getId(),
|
||||||
announcement.getTaskResource(),
|
announcement.getTaskResource(),
|
||||||
TaskStatus.failure(announcement.getTaskId())
|
TaskStatus.failure(announcement.getTaskStatus().getId())
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
@ -110,67 +121,7 @@ public class WorkerTaskMonitor
|
|||||||
Task.class
|
Task.class
|
||||||
);
|
);
|
||||||
|
|
||||||
if (isTaskRunning(task)) {
|
submitTaskRunnable(task, null);
|
||||||
log.warn(
|
|
||||||
"I can't build it. There's something in the way. Got task %s that I am already running...",
|
|
||||||
task.getId()
|
|
||||||
);
|
|
||||||
workerCuratorCoordinator.unannounceTask(task.getId());
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
log.info("Submitting runnable for task[%s]", task.getId());
|
|
||||||
|
|
||||||
exec.submit(
|
|
||||||
new Runnable()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void run()
|
|
||||||
{
|
|
||||||
final long startTime = System.currentTimeMillis();
|
|
||||||
|
|
||||||
log.info("Affirmative. Running task [%s]", task.getId());
|
|
||||||
running.add(task);
|
|
||||||
|
|
||||||
TaskStatus taskStatus;
|
|
||||||
try {
|
|
||||||
workerCuratorCoordinator.unannounceTask(task.getId());
|
|
||||||
workerCuratorCoordinator.announceTaskAnnouncement(
|
|
||||||
TaskAnnouncement.create(
|
|
||||||
task,
|
|
||||||
TaskStatus.running(task.getId())
|
|
||||||
)
|
|
||||||
);
|
|
||||||
taskStatus = taskRunner.run(task).get();
|
|
||||||
}
|
|
||||||
catch (Exception e) {
|
|
||||||
log.makeAlert(e, "I can't build there. Failed to run task")
|
|
||||||
.addData("task", task.getId())
|
|
||||||
.emit();
|
|
||||||
taskStatus = TaskStatus.failure(task.getId());
|
|
||||||
}
|
|
||||||
finally {
|
|
||||||
running.remove(task);
|
|
||||||
}
|
|
||||||
|
|
||||||
taskStatus = taskStatus.withDuration(System.currentTimeMillis() - startTime);
|
|
||||||
|
|
||||||
try {
|
|
||||||
workerCuratorCoordinator.updateAnnouncement(TaskAnnouncement.create(task, taskStatus));
|
|
||||||
log.info(
|
|
||||||
"Job's finished. Completed [%s] with status [%s]",
|
|
||||||
task.getId(),
|
|
||||||
taskStatus.getStatusCode()
|
|
||||||
);
|
|
||||||
}
|
|
||||||
catch (Exception e) {
|
|
||||||
log.makeAlert(e, "Failed to update task status")
|
|
||||||
.addData("task", task.getId())
|
|
||||||
.emit();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -184,10 +135,86 @@ public class WorkerTaskMonitor
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean isTaskRunning(final Task task)
|
private void submitTaskRunnable(final Task task, final ListenableFuture<TaskStatus> taskStatusAlreadySubmitted)
|
||||||
|
{
|
||||||
|
if (isTaskRunning(task.getId())) {
|
||||||
|
log.warn(
|
||||||
|
"I can't build it. There's something in the way. Got task %s that I am already running...",
|
||||||
|
task.getId()
|
||||||
|
);
|
||||||
|
workerCuratorCoordinator.unannounceTask(task.getId());
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
log.info("Submitting runnable for task[%s]", task.getId());
|
||||||
|
|
||||||
|
running.add(task);
|
||||||
|
|
||||||
|
exec.submit(
|
||||||
|
new Runnable()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void run()
|
||||||
|
{
|
||||||
|
final long startTime = System.currentTimeMillis();
|
||||||
|
|
||||||
|
TaskStatus taskStatus;
|
||||||
|
|
||||||
|
try {
|
||||||
|
workerCuratorCoordinator.updateAnnouncement(
|
||||||
|
TaskAnnouncement.create(
|
||||||
|
task,
|
||||||
|
TaskStatus.running(task.getId())
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
if (taskStatusAlreadySubmitted != null) {
|
||||||
|
log.info("Affirmative. Connecting to already-running task [%s]", task.getId());
|
||||||
|
taskStatus = taskStatusAlreadySubmitted.get();
|
||||||
|
} else {
|
||||||
|
log.info("Affirmative. Running task [%s]", task.getId());
|
||||||
|
workerCuratorCoordinator.unannounceTask(task.getId());
|
||||||
|
taskStatus = taskRunner.run(task).get();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
log.debug(e, "Interrupted while running task[%s], exiting.", task.getId());
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
log.makeAlert(e, "I can't build there. Failed to run task")
|
||||||
|
.addData("task", task.getId())
|
||||||
|
.emit();
|
||||||
|
taskStatus = TaskStatus.failure(task.getId());
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
running.remove(task);
|
||||||
|
}
|
||||||
|
|
||||||
|
taskStatus = taskStatus.withDuration(System.currentTimeMillis() - startTime);
|
||||||
|
|
||||||
|
try {
|
||||||
|
workerCuratorCoordinator.updateAnnouncement(TaskAnnouncement.create(task, taskStatus));
|
||||||
|
log.info(
|
||||||
|
"Job's finished. Completed [%s] with status [%s]",
|
||||||
|
task.getId(),
|
||||||
|
taskStatus.getStatusCode()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
log.makeAlert(e, "Failed to update task status")
|
||||||
|
.addData("task", task.getId())
|
||||||
|
.emit();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean isTaskRunning(final String taskId)
|
||||||
{
|
{
|
||||||
for (final Task runningTask : running) {
|
for (final Task runningTask : running) {
|
||||||
if (runningTask.getId().equals(task.getId())) {
|
if (runningTask.getId().equals(taskId)) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -200,7 +227,8 @@ public class WorkerTaskMonitor
|
|||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
pathChildrenCache.close();
|
pathChildrenCache.close();
|
||||||
exec.shutdown();
|
exec.shutdownNow();
|
||||||
|
taskRunner.stop();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.makeAlert(e, "Exception stopping WorkerTaskMonitor")
|
log.makeAlert(e, "Exception stopping WorkerTaskMonitor")
|
||||||
|
@ -31,12 +31,17 @@ import com.metamx.emitter.EmittingLogger;
|
|||||||
import io.druid.concurrent.Execs;
|
import io.druid.concurrent.Execs;
|
||||||
import io.druid.indexing.common.TaskStatus;
|
import io.druid.indexing.common.TaskStatus;
|
||||||
import io.druid.indexing.common.actions.TaskActionClientFactory;
|
import io.druid.indexing.common.actions.TaskActionClientFactory;
|
||||||
|
import io.druid.indexing.common.config.TaskConfig;
|
||||||
import io.druid.indexing.common.task.Task;
|
import io.druid.indexing.common.task.Task;
|
||||||
import io.druid.indexing.overlord.TaskRunner;
|
import io.druid.indexing.overlord.TaskRunner;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
|
import java.nio.channels.FileChannel;
|
||||||
|
import java.nio.channels.FileLock;
|
||||||
|
import java.nio.file.StandardOpenOption;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -47,37 +52,41 @@ public class ExecutorLifecycle
|
|||||||
{
|
{
|
||||||
private static final EmittingLogger log = new EmittingLogger(ExecutorLifecycle.class);
|
private static final EmittingLogger log = new EmittingLogger(ExecutorLifecycle.class);
|
||||||
|
|
||||||
private final ExecutorLifecycleConfig config;
|
private final ExecutorLifecycleConfig taskExecutorConfig;
|
||||||
|
private final TaskConfig taskConfig;
|
||||||
private final TaskActionClientFactory taskActionClientFactory;
|
private final TaskActionClientFactory taskActionClientFactory;
|
||||||
private final TaskRunner taskRunner;
|
private final TaskRunner taskRunner;
|
||||||
private final ObjectMapper jsonMapper;
|
private final ObjectMapper jsonMapper;
|
||||||
|
|
||||||
private final ExecutorService parentMonitorExec = Execs.singleThreaded("parent-monitor-%d");
|
private final ExecutorService parentMonitorExec = Execs.singleThreaded("parent-monitor-%d");
|
||||||
|
|
||||||
|
private volatile Task task = null;
|
||||||
private volatile ListenableFuture<TaskStatus> statusFuture = null;
|
private volatile ListenableFuture<TaskStatus> statusFuture = null;
|
||||||
|
private volatile FileChannel taskLockChannel;
|
||||||
|
private volatile FileLock taskLockFileLock;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public ExecutorLifecycle(
|
public ExecutorLifecycle(
|
||||||
ExecutorLifecycleConfig config,
|
ExecutorLifecycleConfig taskExecutorConfig,
|
||||||
|
TaskConfig taskConfig,
|
||||||
TaskActionClientFactory taskActionClientFactory,
|
TaskActionClientFactory taskActionClientFactory,
|
||||||
TaskRunner taskRunner,
|
TaskRunner taskRunner,
|
||||||
ObjectMapper jsonMapper
|
ObjectMapper jsonMapper
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.config = config;
|
this.taskExecutorConfig = taskExecutorConfig;
|
||||||
|
this.taskConfig = taskConfig;
|
||||||
this.taskActionClientFactory = taskActionClientFactory;
|
this.taskActionClientFactory = taskActionClientFactory;
|
||||||
this.taskRunner = taskRunner;
|
this.taskRunner = taskRunner;
|
||||||
this.jsonMapper = jsonMapper;
|
this.jsonMapper = jsonMapper;
|
||||||
}
|
}
|
||||||
|
|
||||||
@LifecycleStart
|
@LifecycleStart
|
||||||
public void start()
|
public void start() throws InterruptedException
|
||||||
{
|
{
|
||||||
final File taskFile = Preconditions.checkNotNull(config.getTaskFile(), "taskFile");
|
final File taskFile = Preconditions.checkNotNull(taskExecutorConfig.getTaskFile(), "taskFile");
|
||||||
final File statusFile = Preconditions.checkNotNull(config.getStatusFile(), "statusFile");
|
final File statusFile = Preconditions.checkNotNull(taskExecutorConfig.getStatusFile(), "statusFile");
|
||||||
final InputStream parentStream = Preconditions.checkNotNull(config.getParentStream(), "parentStream");
|
final InputStream parentStream = Preconditions.checkNotNull(taskExecutorConfig.getParentStream(), "parentStream");
|
||||||
|
|
||||||
final Task task;
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
task = jsonMapper.readValue(taskFile, Task.class);
|
task = jsonMapper.readValue(taskFile, Task.class);
|
||||||
@ -91,6 +100,43 @@ public class ExecutorLifecycle
|
|||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Avoid running the same task twice on the same machine by locking the task base directory.
|
||||||
|
|
||||||
|
final File taskLockFile = taskConfig.getTaskLockFile(task.getId());
|
||||||
|
|
||||||
|
try {
|
||||||
|
synchronized (this) {
|
||||||
|
if (taskLockChannel == null && taskLockFileLock == null) {
|
||||||
|
taskLockChannel = FileChannel.open(
|
||||||
|
taskLockFile.toPath(),
|
||||||
|
StandardOpenOption.CREATE,
|
||||||
|
StandardOpenOption.WRITE
|
||||||
|
);
|
||||||
|
|
||||||
|
log.info("Attempting to lock file[%s].", taskLockFile);
|
||||||
|
final long startLocking = System.currentTimeMillis();
|
||||||
|
final long timeout = new DateTime(startLocking).plus(taskConfig.getDirectoryLockTimeout()).getMillis();
|
||||||
|
while (taskLockFileLock == null && System.currentTimeMillis() < timeout) {
|
||||||
|
taskLockFileLock = taskLockChannel.tryLock();
|
||||||
|
if (taskLockFileLock == null) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (taskLockFileLock == null) {
|
||||||
|
throw new ISE("Could not acquire lock file[%s] within %,dms.", taskLockFile, timeout - startLocking);
|
||||||
|
} else {
|
||||||
|
log.info("Acquired lock file[%s] in %,dms.", taskLockFile, System.currentTimeMillis() - startLocking);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
throw new ISE("Already started!");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
|
||||||
// Spawn monitor thread to keep a watch on parent's stdin
|
// Spawn monitor thread to keep a watch on parent's stdin
|
||||||
// If stdin reaches eof, the parent is gone, and we should shut down
|
// If stdin reaches eof, the parent is gone, and we should shut down
|
||||||
parentMonitorExec.submit(
|
parentMonitorExec.submit(
|
||||||
@ -120,7 +166,8 @@ public class ExecutorLifecycle
|
|||||||
if (!task.isReady(taskActionClientFactory.create(task))) {
|
if (!task.isReady(taskActionClientFactory.create(task))) {
|
||||||
throw new ISE("Task is not ready to run yet!", task.getId());
|
throw new ISE("Task is not ready to run yet!", task.getId());
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
}
|
||||||
|
catch (Exception e) {
|
||||||
throw new ISE(e, "Failed to run isReady", task.getId());
|
throw new ISE(e, "Failed to run isReady", task.getId());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -164,8 +211,18 @@ public class ExecutorLifecycle
|
|||||||
}
|
}
|
||||||
|
|
||||||
@LifecycleStop
|
@LifecycleStop
|
||||||
public void stop()
|
public void stop() throws Exception
|
||||||
{
|
{
|
||||||
parentMonitorExec.shutdown();
|
parentMonitorExec.shutdown();
|
||||||
|
|
||||||
|
synchronized (this) {
|
||||||
|
if (taskLockFileLock != null) {
|
||||||
|
taskLockFileLock.release();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (taskLockChannel != null) {
|
||||||
|
taskLockChannel.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -89,7 +89,7 @@ public class TaskToolboxTest
|
|||||||
EasyMock.replay(task);
|
EasyMock.replay(task);
|
||||||
|
|
||||||
taskToolbox = new TaskToolboxFactory(
|
taskToolbox = new TaskToolboxFactory(
|
||||||
new TaskConfig(temporaryFolder.newFile().toString(), null, null, 50000, null),
|
new TaskConfig(temporaryFolder.newFile().toString(), null, null, 50000, null, null, null),
|
||||||
mockTaskActionClientFactory,
|
mockTaskActionClientFactory,
|
||||||
mockEmitter,
|
mockEmitter,
|
||||||
mockSegmentPusher,
|
mockSegmentPusher,
|
||||||
|
@ -19,12 +19,138 @@
|
|||||||
|
|
||||||
package io.druid.indexing.common.task;
|
package io.druid.indexing.common.task;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.google.api.client.util.Charsets;
|
||||||
|
import com.google.api.client.util.Sets;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
|
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||||
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
|
import com.metamx.common.Granularity;
|
||||||
|
import com.metamx.common.ISE;
|
||||||
|
import com.metamx.common.guava.Sequences;
|
||||||
|
import com.metamx.common.logger.Logger;
|
||||||
|
import com.metamx.emitter.EmittingLogger;
|
||||||
|
import com.metamx.emitter.core.LoggingEmitter;
|
||||||
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
|
import com.metamx.metrics.MonitorScheduler;
|
||||||
|
import io.druid.client.cache.CacheConfig;
|
||||||
|
import io.druid.client.cache.MapCache;
|
||||||
|
import io.druid.concurrent.Execs;
|
||||||
|
import io.druid.data.input.InputRow;
|
||||||
|
import io.druid.data.input.MapBasedInputRow;
|
||||||
|
import io.druid.granularity.QueryGranularity;
|
||||||
|
import io.druid.indexing.common.SegmentLoaderFactory;
|
||||||
|
import io.druid.indexing.common.TaskStatus;
|
||||||
|
import io.druid.indexing.common.TaskToolbox;
|
||||||
|
import io.druid.indexing.common.TaskToolboxFactory;
|
||||||
|
import io.druid.indexing.common.TestUtils;
|
||||||
|
import io.druid.indexing.common.actions.LocalTaskActionClientFactory;
|
||||||
|
import io.druid.indexing.common.actions.TaskActionClientFactory;
|
||||||
|
import io.druid.indexing.common.actions.TaskActionToolbox;
|
||||||
|
import io.druid.indexing.common.config.TaskConfig;
|
||||||
|
import io.druid.indexing.common.config.TaskStorageConfig;
|
||||||
|
import io.druid.indexing.overlord.HeapMemoryTaskStorage;
|
||||||
|
import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
|
||||||
|
import io.druid.indexing.overlord.TaskLockbox;
|
||||||
|
import io.druid.indexing.overlord.TaskStorage;
|
||||||
|
import io.druid.indexing.test.TestDataSegmentAnnouncer;
|
||||||
|
import io.druid.indexing.test.TestDataSegmentKiller;
|
||||||
|
import io.druid.indexing.test.TestDataSegmentPusher;
|
||||||
|
import io.druid.indexing.test.TestIndexerMetadataStorageCoordinator;
|
||||||
|
import io.druid.indexing.test.TestServerView;
|
||||||
|
import io.druid.jackson.DefaultObjectMapper;
|
||||||
|
import io.druid.query.DefaultQueryRunnerFactoryConglomerate;
|
||||||
|
import io.druid.query.Druids;
|
||||||
|
import io.druid.query.IntervalChunkingQueryRunnerDecorator;
|
||||||
|
import io.druid.query.Query;
|
||||||
|
import io.druid.query.QueryRunner;
|
||||||
|
import io.druid.query.QueryRunnerFactory;
|
||||||
|
import io.druid.query.QueryRunnerFactoryConglomerate;
|
||||||
|
import io.druid.query.QueryToolChest;
|
||||||
|
import io.druid.query.QueryWatcher;
|
||||||
|
import io.druid.query.Result;
|
||||||
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
|
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||||
|
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||||
|
import io.druid.query.timeseries.TimeseriesQuery;
|
||||||
|
import io.druid.query.timeseries.TimeseriesQueryEngine;
|
||||||
|
import io.druid.query.timeseries.TimeseriesQueryQueryToolChest;
|
||||||
|
import io.druid.query.timeseries.TimeseriesQueryRunnerFactory;
|
||||||
|
import io.druid.query.timeseries.TimeseriesResultValue;
|
||||||
|
import io.druid.segment.indexing.DataSchema;
|
||||||
|
import io.druid.segment.indexing.RealtimeIOConfig;
|
||||||
|
import io.druid.segment.indexing.RealtimeTuningConfig;
|
||||||
|
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||||
|
import io.druid.segment.loading.SegmentLoaderConfig;
|
||||||
|
import io.druid.segment.loading.SegmentLoaderLocalCacheManager;
|
||||||
|
import io.druid.segment.loading.StorageLocationConfig;
|
||||||
|
import io.druid.segment.realtime.FireDepartment;
|
||||||
|
import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory;
|
||||||
|
import io.druid.segment.realtime.firehose.NoopChatHandlerProvider;
|
||||||
|
import io.druid.server.coordination.DruidServerMetadata;
|
||||||
|
import io.druid.server.metrics.EventReceiverFirehoseRegister;
|
||||||
|
import io.druid.timeline.DataSegment;
|
||||||
|
import org.easymock.EasyMock;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.Period;
|
||||||
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.nio.file.Files;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
|
|
||||||
public class RealtimeIndexTaskTest
|
public class RealtimeIndexTaskTest
|
||||||
{
|
{
|
||||||
|
private static final Logger log = new Logger(RealtimeIndexTaskTest.class);
|
||||||
|
private static final DruidServerMetadata dummyServer = new DruidServerMetadata(
|
||||||
|
"dummy",
|
||||||
|
"dummy_host",
|
||||||
|
0,
|
||||||
|
"historical",
|
||||||
|
"dummy_tier",
|
||||||
|
0
|
||||||
|
);
|
||||||
|
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||||
|
private static final ServiceEmitter emitter = new ServiceEmitter(
|
||||||
|
"service",
|
||||||
|
"host",
|
||||||
|
new LoggingEmitter(
|
||||||
|
log,
|
||||||
|
LoggingEmitter.Level.ERROR,
|
||||||
|
jsonMapper
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public final TemporaryFolder tempFolder = new TemporaryFolder();
|
||||||
|
|
||||||
|
private DateTime now;
|
||||||
|
private ListeningExecutorService taskExec;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp()
|
||||||
|
{
|
||||||
|
EmittingLogger.registerEmitter(emitter);
|
||||||
|
emitter.start();
|
||||||
|
taskExec = MoreExecutors.listeningDecorator(Execs.singleThreaded("realtime-index-task-test-%d"));
|
||||||
|
now = new DateTime();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown()
|
||||||
|
{
|
||||||
|
taskExec.shutdownNow();
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMakeTaskId() throws Exception
|
public void testMakeTaskId() throws Exception
|
||||||
@ -34,4 +160,386 @@ public class RealtimeIndexTaskTest
|
|||||||
RealtimeIndexTask.makeTaskId("test", 0, new DateTime("2015-01-02"), 0x76543210)
|
RealtimeIndexTask.makeTaskId("test", 0, new DateTime("2015-01-02"), 0x76543210)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 10000L)
|
||||||
|
public void testBasics() throws Exception
|
||||||
|
{
|
||||||
|
final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator();
|
||||||
|
final RealtimeIndexTask task = makeRealtimeTask(null);
|
||||||
|
final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder());
|
||||||
|
final ListenableFuture<TaskStatus> statusFuture = runTask(task, taskToolbox);
|
||||||
|
|
||||||
|
// Wait for firehose to show up, it starts off null.
|
||||||
|
while (task.getFirehose() == null) {
|
||||||
|
Thread.sleep(50);
|
||||||
|
}
|
||||||
|
|
||||||
|
final EventReceiverFirehoseFactory.EventReceiverFirehose firehose =
|
||||||
|
(EventReceiverFirehoseFactory.EventReceiverFirehose) task.getFirehose();
|
||||||
|
|
||||||
|
firehose.addRows(
|
||||||
|
ImmutableList.<InputRow>of(
|
||||||
|
new MapBasedInputRow(
|
||||||
|
now,
|
||||||
|
ImmutableList.of("dim1"),
|
||||||
|
ImmutableMap.<String, Object>of("dim1", "foo")
|
||||||
|
),
|
||||||
|
new MapBasedInputRow(
|
||||||
|
now,
|
||||||
|
ImmutableList.of("dim2"),
|
||||||
|
ImmutableMap.<String, Object>of("dim2", "bar")
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
// Stop the firehose, this will drain out existing events.
|
||||||
|
firehose.close();
|
||||||
|
|
||||||
|
// Wait for publish.
|
||||||
|
while (mdc.getPublished().isEmpty()) {
|
||||||
|
Thread.sleep(50);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Do a query.
|
||||||
|
Assert.assertEquals(2, countEvents(task));
|
||||||
|
|
||||||
|
// Simulate handoff.
|
||||||
|
for (DataSegment segment : mdc.getPublished()) {
|
||||||
|
((TestServerView) taskToolbox.getNewSegmentServerView()).segmentAdded(dummyServer, segment);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Wait for the task to finish.
|
||||||
|
final TaskStatus taskStatus = statusFuture.get();
|
||||||
|
Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 10000L)
|
||||||
|
public void testRestore() throws Exception
|
||||||
|
{
|
||||||
|
final File directory = tempFolder.newFolder();
|
||||||
|
final RealtimeIndexTask task1 = makeRealtimeTask(null);
|
||||||
|
|
||||||
|
// First run:
|
||||||
|
{
|
||||||
|
final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator();
|
||||||
|
final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory);
|
||||||
|
final ListenableFuture<TaskStatus> statusFuture = runTask(task1, taskToolbox);
|
||||||
|
|
||||||
|
// Wait for firehose to show up, it starts off null.
|
||||||
|
while (task1.getFirehose() == null) {
|
||||||
|
Thread.sleep(50);
|
||||||
|
}
|
||||||
|
|
||||||
|
final EventReceiverFirehoseFactory.EventReceiverFirehose firehose =
|
||||||
|
(EventReceiverFirehoseFactory.EventReceiverFirehose) task1.getFirehose();
|
||||||
|
|
||||||
|
firehose.addRows(
|
||||||
|
ImmutableList.<InputRow>of(
|
||||||
|
new MapBasedInputRow(
|
||||||
|
now,
|
||||||
|
ImmutableList.of("dim1"),
|
||||||
|
ImmutableMap.<String, Object>of("dim1", "foo")
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
// Trigger graceful shutdown.
|
||||||
|
task1.stopGracefully();
|
||||||
|
|
||||||
|
// Wait for the task to finish. The status doesn't really matter, but we'll check it anyway.
|
||||||
|
final TaskStatus taskStatus = statusFuture.get();
|
||||||
|
Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode());
|
||||||
|
|
||||||
|
// Nothing should be published.
|
||||||
|
Assert.assertEquals(Sets.newHashSet(), mdc.getPublished());
|
||||||
|
}
|
||||||
|
|
||||||
|
// Second run:
|
||||||
|
{
|
||||||
|
final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator();
|
||||||
|
final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId());
|
||||||
|
final TaskToolbox taskToolbox = makeToolbox(task2, mdc, directory);
|
||||||
|
final ListenableFuture<TaskStatus> statusFuture = runTask(task2, taskToolbox);
|
||||||
|
|
||||||
|
// Wait for firehose to show up, it starts off null.
|
||||||
|
while (task2.getFirehose() == null) {
|
||||||
|
Thread.sleep(50);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Do a query, at this point the previous data should be loaded.
|
||||||
|
Assert.assertEquals(1, countEvents(task2));
|
||||||
|
|
||||||
|
final EventReceiverFirehoseFactory.EventReceiverFirehose firehose =
|
||||||
|
(EventReceiverFirehoseFactory.EventReceiverFirehose) task2.getFirehose();
|
||||||
|
|
||||||
|
firehose.addRows(
|
||||||
|
ImmutableList.<InputRow>of(
|
||||||
|
new MapBasedInputRow(
|
||||||
|
now,
|
||||||
|
ImmutableList.of("dim2"),
|
||||||
|
ImmutableMap.<String, Object>of("dim2", "bar")
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
// Stop the firehose, this will drain out existing events.
|
||||||
|
firehose.close();
|
||||||
|
|
||||||
|
// Wait for publish.
|
||||||
|
while (mdc.getPublished().isEmpty()) {
|
||||||
|
Thread.sleep(50);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Do a query.
|
||||||
|
Assert.assertEquals(2, countEvents(task2));
|
||||||
|
|
||||||
|
// Simulate handoff.
|
||||||
|
for (DataSegment segment : mdc.getPublished()) {
|
||||||
|
((TestServerView) taskToolbox.getNewSegmentServerView()).segmentAdded(dummyServer, segment);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Wait for the task to finish.
|
||||||
|
final TaskStatus taskStatus = statusFuture.get();
|
||||||
|
Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 10000L)
|
||||||
|
public void testRestoreCorruptData() throws Exception
|
||||||
|
{
|
||||||
|
final File directory = tempFolder.newFolder();
|
||||||
|
final RealtimeIndexTask task1 = makeRealtimeTask(null);
|
||||||
|
|
||||||
|
// First run:
|
||||||
|
{
|
||||||
|
final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator();
|
||||||
|
final TaskToolbox taskToolbox = makeToolbox(task1, mdc, directory);
|
||||||
|
final ListenableFuture<TaskStatus> statusFuture = runTask(task1, taskToolbox);
|
||||||
|
|
||||||
|
// Wait for firehose to show up, it starts off null.
|
||||||
|
while (task1.getFirehose() == null) {
|
||||||
|
Thread.sleep(50);
|
||||||
|
}
|
||||||
|
|
||||||
|
final EventReceiverFirehoseFactory.EventReceiverFirehose firehose =
|
||||||
|
(EventReceiverFirehoseFactory.EventReceiverFirehose) task1.getFirehose();
|
||||||
|
|
||||||
|
firehose.addRows(
|
||||||
|
ImmutableList.<InputRow>of(
|
||||||
|
new MapBasedInputRow(
|
||||||
|
now,
|
||||||
|
ImmutableList.of("dim1"),
|
||||||
|
ImmutableMap.<String, Object>of("dim1", "foo")
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
// Trigger graceful shutdown.
|
||||||
|
task1.stopGracefully();
|
||||||
|
|
||||||
|
// Wait for the task to finish. The status doesn't really matter, but we'll check it anyway.
|
||||||
|
final TaskStatus taskStatus = statusFuture.get();
|
||||||
|
Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode());
|
||||||
|
|
||||||
|
// Nothing should be published.
|
||||||
|
Assert.assertEquals(Sets.newHashSet(), mdc.getPublished());
|
||||||
|
}
|
||||||
|
|
||||||
|
// Corrupt the data:
|
||||||
|
final File smooshFile = new File(
|
||||||
|
String.format(
|
||||||
|
"%s/persistent/task/%s/work/persist/%s/%s_%s/0/00000.smoosh",
|
||||||
|
directory,
|
||||||
|
task1.getId(),
|
||||||
|
task1.getDataSource(),
|
||||||
|
Granularity.DAY.truncate(now),
|
||||||
|
Granularity.DAY.increment(Granularity.DAY.truncate(now))
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
Files.write(smooshFile.toPath(), "oops!".getBytes(Charsets.UTF_8));
|
||||||
|
|
||||||
|
// Second run:
|
||||||
|
{
|
||||||
|
final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator();
|
||||||
|
final RealtimeIndexTask task2 = makeRealtimeTask(task1.getId());
|
||||||
|
final TaskToolbox taskToolbox = makeToolbox(task2, mdc, directory);
|
||||||
|
final ListenableFuture<TaskStatus> statusFuture = runTask(task2, taskToolbox);
|
||||||
|
|
||||||
|
// Wait for the task to finish.
|
||||||
|
boolean caught = false;
|
||||||
|
try {
|
||||||
|
statusFuture.get();
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
caught = true;
|
||||||
|
}
|
||||||
|
Assert.assertTrue("expected exception", caught);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private ListenableFuture<TaskStatus> runTask(final Task task, final TaskToolbox toolbox)
|
||||||
|
{
|
||||||
|
return taskExec.submit(
|
||||||
|
new Callable<TaskStatus>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public TaskStatus call() throws Exception
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
if (task.isReady(toolbox.getTaskActionClient())) {
|
||||||
|
return task.run(toolbox);
|
||||||
|
} else {
|
||||||
|
throw new ISE("Task is not ready");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
log.warn(e, "Task failed");
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private RealtimeIndexTask makeRealtimeTask(final String taskId)
|
||||||
|
{
|
||||||
|
ObjectMapper objectMapper = new DefaultObjectMapper();
|
||||||
|
DataSchema dataSchema = new DataSchema(
|
||||||
|
"test_ds",
|
||||||
|
null,
|
||||||
|
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
|
||||||
|
new UniformGranularitySpec(Granularity.DAY, QueryGranularity.NONE, null),
|
||||||
|
objectMapper
|
||||||
|
);
|
||||||
|
RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig(
|
||||||
|
new EventReceiverFirehoseFactory(
|
||||||
|
"foo",
|
||||||
|
100,
|
||||||
|
new NoopChatHandlerProvider(),
|
||||||
|
objectMapper,
|
||||||
|
null,
|
||||||
|
new EventReceiverFirehoseRegister()
|
||||||
|
),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig(
|
||||||
|
1000,
|
||||||
|
new Period("P1Y"),
|
||||||
|
new Period("PT10M"),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
return new RealtimeIndexTask(
|
||||||
|
taskId,
|
||||||
|
null,
|
||||||
|
new FireDepartment(dataSchema, realtimeIOConfig, realtimeTuningConfig),
|
||||||
|
null
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private TaskToolbox makeToolbox(final Task task, final IndexerMetadataStorageCoordinator mdc, final File directory)
|
||||||
|
{
|
||||||
|
final TaskStorage taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null));
|
||||||
|
final TaskConfig taskConfig = new TaskConfig(directory.getPath(), null, null, 50000, null, null, null);
|
||||||
|
final TaskLockbox taskLockbox = new TaskLockbox(taskStorage);
|
||||||
|
final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
|
||||||
|
taskLockbox,
|
||||||
|
mdc,
|
||||||
|
emitter
|
||||||
|
);
|
||||||
|
final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
|
||||||
|
taskStorage,
|
||||||
|
taskActionToolbox
|
||||||
|
);
|
||||||
|
final QueryRunnerFactoryConglomerate conglomerate = new DefaultQueryRunnerFactoryConglomerate(
|
||||||
|
ImmutableMap.<Class<? extends Query>, QueryRunnerFactory>of(
|
||||||
|
TimeseriesQuery.class,
|
||||||
|
new TimeseriesQueryRunnerFactory(
|
||||||
|
new TimeseriesQueryQueryToolChest(
|
||||||
|
new IntervalChunkingQueryRunnerDecorator(null, null, null)
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public <T> QueryRunner<T> decorate(
|
||||||
|
QueryRunner<T> delegate, QueryToolChest<T, ? extends Query<T>> toolChest
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return delegate;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
),
|
||||||
|
new TimeseriesQueryEngine(),
|
||||||
|
new QueryWatcher()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void registerQuery(Query query, ListenableFuture future)
|
||||||
|
{
|
||||||
|
// do nothing
|
||||||
|
}
|
||||||
|
}
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
final TestUtils testUtils = new TestUtils();
|
||||||
|
final TaskToolboxFactory toolboxFactory = new TaskToolboxFactory(
|
||||||
|
taskConfig,
|
||||||
|
taskActionClientFactory,
|
||||||
|
emitter,
|
||||||
|
new TestDataSegmentPusher(),
|
||||||
|
new TestDataSegmentKiller(),
|
||||||
|
null, // DataSegmentMover
|
||||||
|
null, // DataSegmentArchiver
|
||||||
|
new TestDataSegmentAnnouncer(),
|
||||||
|
new TestServerView(),
|
||||||
|
conglomerate,
|
||||||
|
MoreExecutors.sameThreadExecutor(), // queryExecutorService
|
||||||
|
EasyMock.createMock(MonitorScheduler.class),
|
||||||
|
new SegmentLoaderFactory(
|
||||||
|
new SegmentLoaderLocalCacheManager(
|
||||||
|
null,
|
||||||
|
new SegmentLoaderConfig()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public List<StorageLocationConfig> getLocations()
|
||||||
|
{
|
||||||
|
return Lists.newArrayList();
|
||||||
|
}
|
||||||
|
}, testUtils.getTestObjectMapper()
|
||||||
|
)
|
||||||
|
),
|
||||||
|
testUtils.getTestObjectMapper(),
|
||||||
|
testUtils.getTestIndexMerger(),
|
||||||
|
testUtils.getTestIndexIO(),
|
||||||
|
MapCache.create(1024),
|
||||||
|
new CacheConfig()
|
||||||
|
);
|
||||||
|
|
||||||
|
taskLockbox.add(task);
|
||||||
|
return toolboxFactory.build(task);
|
||||||
|
}
|
||||||
|
|
||||||
|
public long countEvents(final Task task) throws Exception
|
||||||
|
{
|
||||||
|
// Do a query.
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("test_ds")
|
||||||
|
.aggregators(
|
||||||
|
ImmutableList.<AggregatorFactory>of(
|
||||||
|
new LongSumAggregatorFactory("rows", "rows")
|
||||||
|
)
|
||||||
|
).granularity(QueryGranularity.ALL)
|
||||||
|
.intervals("2000/3000")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
ArrayList<Result<TimeseriesResultValue>> results = Sequences.toList(
|
||||||
|
task.getQueryRunner(query).run(query, ImmutableMap.<String, Object>of()),
|
||||||
|
Lists.<Result<TimeseriesResultValue>>newArrayList()
|
||||||
|
);
|
||||||
|
return results.get(0).getValue().getLongMetric("rows");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -192,7 +192,7 @@ public class IngestSegmentFirehoseFactoryTest
|
|||||||
);
|
);
|
||||||
|
|
||||||
final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory(
|
final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory(
|
||||||
new TaskConfig(tmpDir.getAbsolutePath(), null, null, 50000, null),
|
new TaskConfig(tmpDir.getAbsolutePath(), null, null, 50000, null, null, null),
|
||||||
tac,
|
tac,
|
||||||
newMockEmitter(),
|
newMockEmitter(),
|
||||||
new DataSegmentPusher()
|
new DataSegmentPusher()
|
||||||
|
@ -294,7 +294,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory(
|
final TaskToolboxFactory taskToolboxFactory = new TaskToolboxFactory(
|
||||||
new TaskConfig(testCase.tmpDir.getAbsolutePath(), null, null, 50000, null),
|
new TaskConfig(testCase.tmpDir.getAbsolutePath(), null, null, 50000, null, null, null),
|
||||||
new TaskActionClientFactory()
|
new TaskActionClientFactory()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -32,7 +32,6 @@ import com.google.common.collect.ImmutableSet;
|
|||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
import com.google.common.collect.Sets;
|
|
||||||
import com.metamx.common.Granularity;
|
import com.metamx.common.Granularity;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.guava.Comparators;
|
import com.metamx.common.guava.Comparators;
|
||||||
@ -72,8 +71,8 @@ import io.druid.indexing.common.task.RealtimeIndexTaskTest;
|
|||||||
import io.druid.indexing.common.task.Task;
|
import io.druid.indexing.common.task.Task;
|
||||||
import io.druid.indexing.common.task.TaskResource;
|
import io.druid.indexing.common.task.TaskResource;
|
||||||
import io.druid.indexing.overlord.config.TaskQueueConfig;
|
import io.druid.indexing.overlord.config.TaskQueueConfig;
|
||||||
|
import io.druid.indexing.test.TestIndexerMetadataStorageCoordinator;
|
||||||
import io.druid.jackson.DefaultObjectMapper;
|
import io.druid.jackson.DefaultObjectMapper;
|
||||||
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
|
|
||||||
import io.druid.metadata.SQLMetadataStorageActionHandlerFactory;
|
import io.druid.metadata.SQLMetadataStorageActionHandlerFactory;
|
||||||
import io.druid.metadata.TestDerbyConnector;
|
import io.druid.metadata.TestDerbyConnector;
|
||||||
import io.druid.query.QueryRunnerFactoryConglomerate;
|
import io.druid.query.QueryRunnerFactoryConglomerate;
|
||||||
@ -183,6 +182,7 @@ public class TaskLifecycleTest
|
|||||||
|
|
||||||
private final String taskStorageType;
|
private final String taskStorageType;
|
||||||
|
|
||||||
|
|
||||||
private ObjectMapper mapper;
|
private ObjectMapper mapper;
|
||||||
private TaskStorageQueryAdapter tsqa = null;
|
private TaskStorageQueryAdapter tsqa = null;
|
||||||
private File tmpDir = null;
|
private File tmpDir = null;
|
||||||
@ -190,7 +190,7 @@ public class TaskLifecycleTest
|
|||||||
private TaskLockbox tl = null;
|
private TaskLockbox tl = null;
|
||||||
private TaskQueue tq = null;
|
private TaskQueue tq = null;
|
||||||
private TaskRunner tr = null;
|
private TaskRunner tr = null;
|
||||||
private MockIndexerMetadataStorageCoordinator mdc = null;
|
private TestIndexerMetadataStorageCoordinator mdc = null;
|
||||||
private TaskActionClientFactory tac = null;
|
private TaskActionClientFactory tac = null;
|
||||||
private TaskToolboxFactory tb = null;
|
private TaskToolboxFactory tb = null;
|
||||||
private IndexSpec indexSpec;
|
private IndexSpec indexSpec;
|
||||||
@ -205,9 +205,18 @@ public class TaskLifecycleTest
|
|||||||
private TestDerbyConnector testDerbyConnector;
|
private TestDerbyConnector testDerbyConnector;
|
||||||
private List<ServerView.SegmentCallback> segmentCallbacks = new ArrayList<>();
|
private List<ServerView.SegmentCallback> segmentCallbacks = new ArrayList<>();
|
||||||
|
|
||||||
private static MockIndexerMetadataStorageCoordinator newMockMDC()
|
private static TestIndexerMetadataStorageCoordinator newMockMDC()
|
||||||
{
|
{
|
||||||
return new MockIndexerMetadataStorageCoordinator();
|
return new TestIndexerMetadataStorageCoordinator()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Set<DataSegment> announceHistoricalSegments(Set<DataSegment> segments)
|
||||||
|
{
|
||||||
|
Set<DataSegment> retVal = super.announceHistoricalSegments(segments);
|
||||||
|
publishCountDown.countDown();
|
||||||
|
return retVal;
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
private static ServiceEmitter newMockEmitter()
|
private static ServiceEmitter newMockEmitter()
|
||||||
@ -373,7 +382,11 @@ public class TaskLifecycleTest
|
|||||||
ts = new MetadataTaskStorage(
|
ts = new MetadataTaskStorage(
|
||||||
testDerbyConnector,
|
testDerbyConnector,
|
||||||
new TaskStorageConfig(null),
|
new TaskStorageConfig(null),
|
||||||
new SQLMetadataStorageActionHandlerFactory(testDerbyConnector, derbyConnectorRule.metadataTablesConfigSupplier().get(), mapper)
|
new SQLMetadataStorageActionHandlerFactory(
|
||||||
|
testDerbyConnector,
|
||||||
|
derbyConnectorRule.metadataTablesConfigSupplier().get(),
|
||||||
|
mapper
|
||||||
|
)
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
throw new RuntimeException(String.format("Unknown task storage type [%s]", taskStorageType));
|
throw new RuntimeException(String.format("Unknown task storage type [%s]", taskStorageType));
|
||||||
@ -408,13 +421,15 @@ public class TaskLifecycleTest
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void setUpAndStartTaskQueue(DataSegmentPusher dataSegmentPusher) {
|
private void setUpAndStartTaskQueue(DataSegmentPusher dataSegmentPusher)
|
||||||
|
{
|
||||||
|
final TaskConfig taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, null, null);
|
||||||
tsqa = new TaskStorageQueryAdapter(ts);
|
tsqa = new TaskStorageQueryAdapter(ts);
|
||||||
tl = new TaskLockbox(ts);
|
tl = new TaskLockbox(ts);
|
||||||
mdc = newMockMDC();
|
mdc = newMockMDC();
|
||||||
tac = new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tl, mdc, newMockEmitter()));
|
tac = new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tl, mdc, newMockEmitter()));
|
||||||
tb = new TaskToolboxFactory(
|
tb = new TaskToolboxFactory(
|
||||||
new TaskConfig(tmpDir.toString(), null, null, 50000, null),
|
taskConfig,
|
||||||
tac,
|
tac,
|
||||||
newMockEmitter(),
|
newMockEmitter(),
|
||||||
dataSegmentPusher,
|
dataSegmentPusher,
|
||||||
@ -491,7 +506,7 @@ public class TaskLifecycleTest
|
|||||||
MapCache.create(0),
|
MapCache.create(0),
|
||||||
FireDepartmentTest.NO_CACHE_CONFIG
|
FireDepartmentTest.NO_CACHE_CONFIG
|
||||||
);
|
);
|
||||||
tr = new ThreadPoolTaskRunner(tb, null);
|
tr = new ThreadPoolTaskRunner(tb, taskConfig, emitter);
|
||||||
tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter);
|
tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter);
|
||||||
tq.start();
|
tq.start();
|
||||||
}
|
}
|
||||||
@ -821,7 +836,7 @@ public class TaskLifecycleTest
|
|||||||
Assert.assertEquals("segments nuked", 0, mdc.getNuked().size());
|
Assert.assertEquals("segments nuked", 0, mdc.getNuked().size());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test (timeout = 4000L)
|
@Test(timeout = 4000L)
|
||||||
public void testRealtimeIndexTask() throws Exception
|
public void testRealtimeIndexTask() throws Exception
|
||||||
{
|
{
|
||||||
monitorScheduler.addMonitor(EasyMock.anyObject(Monitor.class));
|
monitorScheduler.addMonitor(EasyMock.anyObject(Monitor.class));
|
||||||
@ -870,7 +885,7 @@ public class TaskLifecycleTest
|
|||||||
EasyMock.verify(monitorScheduler, queryRunnerFactoryConglomerate);
|
EasyMock.verify(monitorScheduler, queryRunnerFactoryConglomerate);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test (timeout = 4000L)
|
@Test(timeout = 4000L)
|
||||||
public void testRealtimeIndexTaskFailure() throws Exception
|
public void testRealtimeIndexTaskFailure() throws Exception
|
||||||
{
|
{
|
||||||
setUpAndStartTaskQueue(
|
setUpAndStartTaskQueue(
|
||||||
@ -1012,7 +1027,8 @@ public class TaskLifecycleTest
|
|||||||
return retVal;
|
return retVal;
|
||||||
}
|
}
|
||||||
|
|
||||||
private RealtimeIndexTask giveMeARealtimeIndexTask() {
|
private RealtimeIndexTask giveMeARealtimeIndexTask()
|
||||||
|
{
|
||||||
String taskId = String.format("rt_task_%s", System.currentTimeMillis());
|
String taskId = String.format("rt_task_%s", System.currentTimeMillis());
|
||||||
DataSchema dataSchema = new DataSchema(
|
DataSchema dataSchema = new DataSchema(
|
||||||
"test_ds",
|
"test_ds",
|
||||||
@ -1023,7 +1039,8 @@ public class TaskLifecycleTest
|
|||||||
);
|
);
|
||||||
RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig(
|
RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig(
|
||||||
new MockFirehoseFactory(true),
|
new MockFirehoseFactory(true),
|
||||||
null, // PlumberSchool - Realtime Index Task always uses RealtimePlumber which is hardcoded in RealtimeIndexTask class
|
null,
|
||||||
|
// PlumberSchool - Realtime Index Task always uses RealtimePlumber which is hardcoded in RealtimeIndexTask class
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig(
|
RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig(
|
||||||
@ -1045,64 +1062,4 @@ public class TaskLifecycleTest
|
|||||||
null
|
null
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class MockIndexerMetadataStorageCoordinator extends IndexerSQLMetadataStorageCoordinator
|
|
||||||
{
|
|
||||||
final private Set<DataSegment> published = Sets.newHashSet();
|
|
||||||
final private Set<DataSegment> nuked = Sets.newHashSet();
|
|
||||||
|
|
||||||
private List<DataSegment> unusedSegments;
|
|
||||||
|
|
||||||
private MockIndexerMetadataStorageCoordinator()
|
|
||||||
{
|
|
||||||
super(null, null, null);
|
|
||||||
unusedSegments = Lists.newArrayList();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<DataSegment> getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException
|
|
||||||
{
|
|
||||||
return ImmutableList.of();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<DataSegment> getUnusedSegmentsForInterval(String dataSource, Interval interval)
|
|
||||||
{
|
|
||||||
return unusedSegments;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Set<DataSegment> announceHistoricalSegments(Set<DataSegment> segments)
|
|
||||||
{
|
|
||||||
Set<DataSegment> added = Sets.newHashSet();
|
|
||||||
for (final DataSegment segment : segments) {
|
|
||||||
if (published.add(segment)) {
|
|
||||||
added.add(segment);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
TaskLifecycleTest.publishCountDown.countDown();
|
|
||||||
return ImmutableSet.copyOf(added);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void deleteSegments(Set<DataSegment> segments)
|
|
||||||
{
|
|
||||||
nuked.addAll(segments);
|
|
||||||
}
|
|
||||||
|
|
||||||
public Set<DataSegment> getPublished()
|
|
||||||
{
|
|
||||||
return ImmutableSet.copyOf(published);
|
|
||||||
}
|
|
||||||
|
|
||||||
public Set<DataSegment> getNuked()
|
|
||||||
{
|
|
||||||
return ImmutableSet.copyOf(nuked);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setUnusedSegments(List<DataSegment> unusedSegments)
|
|
||||||
{
|
|
||||||
this.unusedSegments = unusedSegments;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap;
|
|||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import com.google.common.util.concurrent.MoreExecutors;
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
|
import com.metamx.common.Pair;
|
||||||
import com.metamx.common.concurrent.ScheduledExecutorFactory;
|
import com.metamx.common.concurrent.ScheduledExecutorFactory;
|
||||||
import com.metamx.common.guava.CloseQuietly;
|
import com.metamx.common.guava.CloseQuietly;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
@ -294,6 +295,18 @@ public class OverlordResourceTest
|
|||||||
this.runningTasks = new ArrayList<>();
|
this.runningTasks = new ArrayList<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<Pair<Task, ListenableFuture<TaskStatus>>> restore()
|
||||||
|
{
|
||||||
|
return ImmutableList.of();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void stop()
|
||||||
|
{
|
||||||
|
// Do nothing
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized ListenableFuture<TaskStatus> run(final Task task)
|
public synchronized ListenableFuture<TaskStatus> run(final Task task)
|
||||||
{
|
{
|
||||||
|
@ -0,0 +1,66 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets 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 io.druid.indexing.test;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
import io.druid.server.coordination.DataSegmentAnnouncer;
|
||||||
|
import io.druid.timeline.DataSegment;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class TestDataSegmentAnnouncer implements DataSegmentAnnouncer
|
||||||
|
{
|
||||||
|
public Set<DataSegment> announcedSegments = Sets.newConcurrentHashSet();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void announceSegment(DataSegment segment) throws IOException
|
||||||
|
{
|
||||||
|
announcedSegments.add(segment);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void unannounceSegment(DataSegment segment) throws IOException
|
||||||
|
{
|
||||||
|
announcedSegments.remove(segment);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void announceSegments(Iterable<DataSegment> segments) throws IOException
|
||||||
|
{
|
||||||
|
for (DataSegment segment : segments) {
|
||||||
|
announcedSegments.add(segment);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void unannounceSegments(Iterable<DataSegment> segments) throws IOException
|
||||||
|
{
|
||||||
|
for (DataSegment segment : segments) {
|
||||||
|
announcedSegments.remove(segment);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Set<DataSegment> getAnnouncedSegments()
|
||||||
|
{
|
||||||
|
return ImmutableSet.copyOf(announcedSegments);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,44 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets 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 io.druid.indexing.test;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
import io.druid.segment.loading.DataSegmentKiller;
|
||||||
|
import io.druid.segment.loading.SegmentLoadingException;
|
||||||
|
import io.druid.timeline.DataSegment;
|
||||||
|
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class TestDataSegmentKiller implements DataSegmentKiller
|
||||||
|
{
|
||||||
|
private final Set<DataSegment> killedSegments = Sets.newConcurrentHashSet();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void kill(DataSegment segment) throws SegmentLoadingException
|
||||||
|
{
|
||||||
|
killedSegments.add(segment);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Set<DataSegment> getKilledSegments()
|
||||||
|
{
|
||||||
|
return ImmutableSet.copyOf(killedSegments);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,52 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets 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 io.druid.indexing.test;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
import io.druid.segment.loading.DataSegmentPusher;
|
||||||
|
import io.druid.timeline.DataSegment;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class TestDataSegmentPusher implements DataSegmentPusher
|
||||||
|
{
|
||||||
|
private final Set<DataSegment> pushedSegments = Sets.newConcurrentHashSet();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getPathForHadoop(String dataSource)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DataSegment push(File file, DataSegment segment) throws IOException
|
||||||
|
{
|
||||||
|
pushedSegments.add(segment);
|
||||||
|
return segment;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Set<DataSegment> getPushedSegments()
|
||||||
|
{
|
||||||
|
return ImmutableSet.copyOf(pushedSegments);
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,113 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets 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 io.druid.indexing.test;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
|
||||||
|
import io.druid.segment.realtime.appenderator.SegmentIdentifier;
|
||||||
|
import io.druid.timeline.DataSegment;
|
||||||
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataStorageCoordinator
|
||||||
|
{
|
||||||
|
final private Set<DataSegment> published = Sets.newConcurrentHashSet();
|
||||||
|
final private Set<DataSegment> nuked = Sets.newConcurrentHashSet();
|
||||||
|
final private List<DataSegment> unusedSegments;
|
||||||
|
|
||||||
|
public TestIndexerMetadataStorageCoordinator()
|
||||||
|
{
|
||||||
|
unusedSegments = Lists.newArrayList();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<DataSegment> getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException
|
||||||
|
{
|
||||||
|
return ImmutableList.of();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<DataSegment> getUnusedSegmentsForInterval(String dataSource, Interval interval)
|
||||||
|
{
|
||||||
|
synchronized (unusedSegments) {
|
||||||
|
return ImmutableList.copyOf(unusedSegments);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<DataSegment> announceHistoricalSegments(Set<DataSegment> segments)
|
||||||
|
{
|
||||||
|
Set<DataSegment> added = Sets.newHashSet();
|
||||||
|
for (final DataSegment segment : segments) {
|
||||||
|
if (published.add(segment)) {
|
||||||
|
added.add(segment);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return ImmutableSet.copyOf(added);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SegmentIdentifier allocatePendingSegment(
|
||||||
|
String dataSource,
|
||||||
|
String sequenceName,
|
||||||
|
String previousSegmentId,
|
||||||
|
Interval interval,
|
||||||
|
String maxVersion
|
||||||
|
) throws IOException
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void deleteSegments(Set<DataSegment> segments)
|
||||||
|
{
|
||||||
|
nuked.addAll(segments);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateSegmentMetadata(Set<DataSegment> segments) throws IOException
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Set<DataSegment> getPublished()
|
||||||
|
{
|
||||||
|
return ImmutableSet.copyOf(published);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Set<DataSegment> getNuked()
|
||||||
|
{
|
||||||
|
return ImmutableSet.copyOf(nuked);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setUnusedSegments(List<DataSegment> newUnusedSegments)
|
||||||
|
{
|
||||||
|
synchronized (unusedSegments) {
|
||||||
|
unusedSegments.clear();
|
||||||
|
unusedSegments.addAll(newUnusedSegments);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,116 @@
|
|||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets 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 io.druid.indexing.test;
|
||||||
|
|
||||||
|
import com.google.api.client.util.Lists;
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
|
import com.metamx.common.Pair;
|
||||||
|
import io.druid.client.FilteredServerView;
|
||||||
|
import io.druid.client.ServerView;
|
||||||
|
import io.druid.server.coordination.DruidServerMetadata;
|
||||||
|
import io.druid.timeline.DataSegment;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.ConcurrentMap;
|
||||||
|
import java.util.concurrent.Executor;
|
||||||
|
|
||||||
|
public class TestServerView implements FilteredServerView, ServerView.SegmentCallback
|
||||||
|
{
|
||||||
|
final ConcurrentMap<ServerView.SegmentCallback, Pair<Predicate<DataSegment>, Executor>> callbacks = Maps.newConcurrentMap();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void registerSegmentCallback(
|
||||||
|
final Executor exec,
|
||||||
|
final ServerView.SegmentCallback callback,
|
||||||
|
final Predicate<DataSegment> filter
|
||||||
|
)
|
||||||
|
{
|
||||||
|
callbacks.put(callback, Pair.of(filter, exec));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ServerView.CallbackAction segmentAdded(
|
||||||
|
final DruidServerMetadata server,
|
||||||
|
final DataSegment segment
|
||||||
|
)
|
||||||
|
{
|
||||||
|
for (final Map.Entry<ServerView.SegmentCallback, Pair<Predicate<DataSegment>, Executor>> entry : callbacks.entrySet()) {
|
||||||
|
if (entry.getValue().lhs.apply(segment)) {
|
||||||
|
entry.getValue().rhs.execute(
|
||||||
|
new Runnable()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void run()
|
||||||
|
{
|
||||||
|
entry.getKey().segmentAdded(server, segment);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return ServerView.CallbackAction.CONTINUE;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ServerView.CallbackAction segmentRemoved(
|
||||||
|
final DruidServerMetadata server,
|
||||||
|
final DataSegment segment
|
||||||
|
)
|
||||||
|
{
|
||||||
|
for (final Map.Entry<ServerView.SegmentCallback, Pair<Predicate<DataSegment>, Executor>> entry : callbacks.entrySet()) {
|
||||||
|
if (entry.getValue().lhs.apply(segment)) {
|
||||||
|
entry.getValue().rhs.execute(
|
||||||
|
new Runnable()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void run()
|
||||||
|
{
|
||||||
|
entry.getKey().segmentRemoved(server, segment);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return ServerView.CallbackAction.CONTINUE;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ServerView.CallbackAction segmentViewInitialized()
|
||||||
|
{
|
||||||
|
for (final Map.Entry<ServerView.SegmentCallback, Pair<Predicate<DataSegment>, Executor>> entry : callbacks.entrySet()) {
|
||||||
|
entry.getValue().rhs.execute(
|
||||||
|
new Runnable()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void run()
|
||||||
|
{
|
||||||
|
entry.getKey().segmentViewInitialized();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
return ServerView.CallbackAction.CONTINUE;
|
||||||
|
}
|
||||||
|
}
|
@ -41,6 +41,7 @@ import io.druid.segment.loading.SegmentLoaderLocalCacheManager;
|
|||||||
import io.druid.segment.loading.StorageLocationConfig;
|
import io.druid.segment.loading.StorageLocationConfig;
|
||||||
import io.druid.server.initialization.IndexerZkConfig;
|
import io.druid.server.initialization.IndexerZkConfig;
|
||||||
import io.druid.server.initialization.ZkPathsConfig;
|
import io.druid.server.initialization.ZkPathsConfig;
|
||||||
|
import io.druid.server.metrics.NoopServiceEmitter;
|
||||||
import org.apache.curator.framework.CuratorFramework;
|
import org.apache.curator.framework.CuratorFramework;
|
||||||
import org.apache.curator.framework.CuratorFrameworkFactory;
|
import org.apache.curator.framework.CuratorFrameworkFactory;
|
||||||
import org.apache.curator.retry.ExponentialBackoffRetry;
|
import org.apache.curator.retry.ExponentialBackoffRetry;
|
||||||
@ -133,13 +134,14 @@ public class WorkerTaskMonitorTest
|
|||||||
|
|
||||||
private WorkerTaskMonitor createTaskMonitor()
|
private WorkerTaskMonitor createTaskMonitor()
|
||||||
{
|
{
|
||||||
|
final TaskConfig taskConfig = new TaskConfig(Files.createTempDir().toString(), null, null, 0, null, null, null);
|
||||||
return new WorkerTaskMonitor(
|
return new WorkerTaskMonitor(
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
cf,
|
cf,
|
||||||
workerCuratorCoordinator,
|
workerCuratorCoordinator,
|
||||||
new ThreadPoolTaskRunner(
|
new ThreadPoolTaskRunner(
|
||||||
new TaskToolboxFactory(
|
new TaskToolboxFactory(
|
||||||
new TaskConfig(Files.createTempDir().toString(), null, null, 0, null),
|
taskConfig,
|
||||||
null, null, null, null, null, null, null, null, null, null, null, new SegmentLoaderFactory(
|
null, null, null, null, null, null, null, null, null, null, null, new SegmentLoaderFactory(
|
||||||
new SegmentLoaderLocalCacheManager(
|
new SegmentLoaderLocalCacheManager(
|
||||||
null,
|
null,
|
||||||
@ -160,7 +162,8 @@ public class WorkerTaskMonitorTest
|
|||||||
null,
|
null,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
null
|
taskConfig,
|
||||||
|
new NoopServiceEmitter()
|
||||||
),
|
),
|
||||||
new WorkerConfig().setCapacity(1)
|
new WorkerConfig().setCapacity(1)
|
||||||
);
|
);
|
||||||
|
@ -177,17 +177,7 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory<MapInputRow
|
|||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
for (final InputRow row : rows) {
|
addRows(rows);
|
||||||
boolean added = false;
|
|
||||||
while (!closed && !added) {
|
|
||||||
added = buffer.offer(row, 500, TimeUnit.MILLISECONDS);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!added) {
|
|
||||||
throw new IllegalStateException("Cannot add events to closed firehose!");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return Response.ok(
|
return Response.ok(
|
||||||
objectMapper.writeValueAsString(ImmutableMap.of("eventCount", events.size())),
|
objectMapper.writeValueAsString(ImmutableMap.of("eventCount", events.size())),
|
||||||
contentType
|
contentType
|
||||||
@ -267,11 +257,29 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory<MapInputRow
|
|||||||
@Override
|
@Override
|
||||||
public void close() throws IOException
|
public void close() throws IOException
|
||||||
{
|
{
|
||||||
log.info("Firehose closing.");
|
if (!closed) {
|
||||||
closed = true;
|
log.info("Firehose closing.");
|
||||||
eventReceiverFirehoseRegister.unregister(serviceName);
|
closed = true;
|
||||||
if (chatHandlerProvider.isPresent()) {
|
|
||||||
chatHandlerProvider.get().unregister(serviceName);
|
eventReceiverFirehoseRegister.unregister(serviceName);
|
||||||
|
if (chatHandlerProvider.isPresent()) {
|
||||||
|
chatHandlerProvider.get().unregister(serviceName);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// public for tests
|
||||||
|
public void addRows(Iterable<InputRow> rows) throws InterruptedException
|
||||||
|
{
|
||||||
|
for (final InputRow row : rows) {
|
||||||
|
boolean added = false;
|
||||||
|
while (!closed && !added) {
|
||||||
|
added = buffer.offer(row, 500, TimeUnit.MILLISECONDS);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!added) {
|
||||||
|
throw new IllegalStateException("Cannot add events to closed firehose!");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user