提交 2d21aea2 编写于 作者: F fjy

Merge pull request #323 from metamx/indexing-service-stuff

Indexing service stuff
......@@ -308,21 +308,29 @@ This module is used to configure the [Indexing Service](Indexing-Service.html) t
|Property|Description|Default|
|--------|-----------|-------|
|`druid.indexer.logs.type`|Choices:noop, S3. Where to store task logs|noop|
|`druid.indexer.logs.type`|Choices:noop, s3, file. Where to store task logs|file|
#### Noop Task Logs
#### File Task Logs
No task logs are actually stored.
Store task logs in the local filesystem.
|Property|Description|Default|
|--------|-----------|-------|
|`druid.indexer.logs.directory`|Local filesystem path.|log|
#### S3 Task Logs
Store Task Logs in S3.
Store task logs in S3.
|Property|Description|Default|
|--------|-----------|-------|
|`druid.indexer.logs.s3Bucket`|S3 bucket name.|none|
|`druid.indexer.logs.s3Prefix`|S3 key prefix.|none|
#### Noop Task Logs
No task logs are actually stored.
### Firehose Module
The Firehose module lists all available firehoses. There are no configurations.
......@@ -110,12 +110,16 @@ If autoscaling is enabled, new middle managers may be added when a task has been
#### JVM Configuration
In addition to the configuration of some of the default modules in [Configuration](Configuration.html), the overlord module requires the following basic configs to run in remote mode:
In addition to the configuration of some of the default modules in [Configuration](Configuration.html), the overlord has the following basic configs:
|Property|Description|Default|
|--------|-----------|-------|
|`druid.indexer.runner.type`|Choices "local" or "remote". Indicates whether tasks should be run locally or in a distributed environment.|local|
|`druid.indexer.storage.type`|Choices are "local" or "db". Indicates whether incoming tasks should be stored locally (in heap) or in a database. Storing incoming tasks in a database allows for tasks to be bootstrapped if the overlord should fail.|local|
|`druid.indexer.storage.type`|Choices are "local" or "db". Indicates whether incoming tasks should be stored locally (in heap) or in a database. Storing incoming tasks in a database allows for tasks to be resumed if the overlord should fail.|local|
|`druid.indexer.queue.maxSize`|Maximum number of active tasks at one time.|Integer.MAX_VALUE|
|`druid.indexer.queue.startDelay`|Sleep this long before starting overlord queue management. This can be useful to give a cluster time to re-orient itself after e.g. a widespread network issue.|PT1M|
|`druid.indexer.queue.restartDelay`|Sleep this long when overlord queue management throws an exception before trying again.|PT30S|
|`druid.indexer.queue.storageSyncRate`|Sync overlord state this often with an underlying task persistence mechanism.|PT1M|
The following configs only apply if the overlord is running in remote mode:
......
......@@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import io.druid.indexing.common.config.EventReceiverFirehoseFactoryConfig;
import io.druid.indexing.common.index.EventReceiverFirehoseFactory;
import io.druid.initialization.DruidModule;
......@@ -46,7 +45,5 @@ public class IndexingServiceFirehoseModule implements DruidModule
@Override
public void configure(Binder binder)
{
// backwards compatibility
ConfigProvider.bind(binder, EventReceiverFirehoseFactoryConfig.class);
}
}
......@@ -68,7 +68,7 @@ public class LockAcquireAction implements TaskAction<TaskLock>
@Override
public boolean isAudited()
{
return true;
return false;
}
@Override
......
......@@ -60,7 +60,7 @@ public class LockReleaseAction implements TaskAction<Void>
@Override
public boolean isAudited()
{
return true;
return false;
}
@Override
......
......@@ -23,56 +23,54 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableList;
import com.google.common.base.Optional;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.task.Task;
import org.joda.time.Interval;
import java.util.List;
public class SpawnTasksAction implements TaskAction<Void>
public class LockTryAcquireAction implements TaskAction<Optional<TaskLock>>
{
@JsonIgnore
private final List<Task> newTasks;
private final Interval interval;
@JsonCreator
public SpawnTasksAction(
@JsonProperty("newTasks") List<Task> newTasks
public LockTryAcquireAction(
@JsonProperty("interval") Interval interval
)
{
this.newTasks = ImmutableList.copyOf(newTasks);
this.interval = interval;
}
@JsonProperty
public List<Task> getNewTasks()
public Interval getInterval()
{
return newTasks;
return interval;
}
public TypeReference<Void> getReturnTypeReference()
public TypeReference<Optional<TaskLock>> getReturnTypeReference()
{
return new TypeReference<Void>() {};
return new TypeReference<Optional<TaskLock>>()
{
};
}
@Override
public Void perform(Task task, TaskActionToolbox toolbox)
public Optional<TaskLock> perform(Task task, TaskActionToolbox toolbox)
{
for(final Task newTask : newTasks) {
toolbox.getTaskQueue().add(newTask);
}
return null;
return toolbox.getTaskLockbox().tryLock(task, interval);
}
@Override
public boolean isAudited()
{
return true;
return false;
}
@Override
public String toString()
{
return "SpawnTasksAction{" +
"newTasks=" + newTasks +
return "LockTryAcquireAction{" +
"interval=" + interval +
'}';
}
}
......@@ -29,13 +29,13 @@ import java.io.IOException;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "lockAcquire", value = LockAcquireAction.class),
@JsonSubTypes.Type(name = "lockTryAcquire", value = LockTryAcquireAction.class),
@JsonSubTypes.Type(name = "lockList", value = LockListAction.class),
@JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class),
@JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class),
@JsonSubTypes.Type(name = "segmentListUsed", value = SegmentListUsedAction.class),
@JsonSubTypes.Type(name = "segmentListUnused", value = SegmentListUnusedAction.class),
@JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class),
@JsonSubTypes.Type(name = "spawnTasks", value = SpawnTasksAction.class)
@JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class)
})
public interface TaskAction<RetType>
{
......
......@@ -27,7 +27,6 @@ import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.IndexerDBCoordinator;
import io.druid.indexing.overlord.TaskLockbox;
import io.druid.indexing.overlord.TaskQueue;
import io.druid.timeline.DataSegment;
import java.util.List;
......@@ -35,30 +34,22 @@ import java.util.Set;
public class TaskActionToolbox
{
private final TaskQueue taskQueue;
private final TaskLockbox taskLockbox;
private final IndexerDBCoordinator indexerDBCoordinator;
private final ServiceEmitter emitter;
@Inject
public TaskActionToolbox(
TaskQueue taskQueue,
TaskLockbox taskLockbox,
IndexerDBCoordinator indexerDBCoordinator,
ServiceEmitter emitter
)
{
this.taskQueue = taskQueue;
this.taskLockbox = taskLockbox;
this.indexerDBCoordinator = indexerDBCoordinator;
this.emitter = emitter;
}
public TaskQueue getTaskQueue()
{
return taskQueue;
}
public TaskLockbox getTaskLockbox()
{
return taskLockbox;
......
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.common.config;
import org.skife.config.Config;
/**
*/
@Deprecated
public abstract class EventReceiverFirehoseFactoryConfig
{
@Config("druid.indexer.firehoseId.prefix")
public abstract String getFirehoseIdPrefix();
}
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.common.config;
import io.druid.server.initialization.ZkPathsConfig;
import org.skife.config.Config;
import org.skife.config.Default;
/**
*/
public abstract class IndexerZkConfig extends ZkPathsConfig
{
@Config("druid.zk.maxNumBytes")
@Default("512000")
public abstract long getMaxNumBytes();
}
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.common.config;
import org.skife.config.Config;
import org.skife.config.Default;
import org.skife.config.DefaultNull;
public abstract class TaskLogConfig
{
@Config("druid.indexer.logs.type")
@Default("noop")
public abstract String getLogType();
@Config("druid.indexer.logs.s3bucket")
@DefaultNull
public abstract String getLogStorageBucket();
@Config("druid.indexer.logs.s3prefix")
@DefaultNull
public abstract String getLogStoragePrefix();
}
......@@ -33,7 +33,6 @@ import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.InputRow;
import io.druid.data.input.impl.MapInputRowParser;
import io.druid.indexing.common.config.EventReceiverFirehoseFactoryConfig;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
......@@ -63,31 +62,15 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory
private final MapInputRowParser parser;
private final Optional<ChatHandlerProvider> chatHandlerProvider;
@Deprecated
private final EventReceiverFirehoseFactoryConfig config;
@JsonCreator
public EventReceiverFirehoseFactory(
@JsonProperty("serviceName") String serviceName,
@JsonProperty("firehoseId") String firehoseId,
@JsonProperty("bufferSize") Integer bufferSize,
@JsonProperty("parser") MapInputRowParser parser,
@JacksonInject ChatHandlerProvider chatHandlerProvider,
@JacksonInject EventReceiverFirehoseFactoryConfig config
@JacksonInject ChatHandlerProvider chatHandlerProvider
)
{
// This code is here for backwards compatibility
if (serviceName == null) {
this.serviceName = String.format(
"%s:%s",
config.getFirehoseIdPrefix(),
Preconditions.checkNotNull(firehoseId, "firehoseId")
);
} else {
this.serviceName = serviceName;
}
this.config = config;
this.serviceName = Preconditions.checkNotNull(serviceName, "serviceName");
this.bufferSize = bufferSize == null || bufferSize <= 0 ? DEFAULT_BUFFER_SIZE : bufferSize;
this.parser = Preconditions.checkNotNull(parser, "parser");
this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider);
......@@ -117,13 +100,6 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory
return serviceName;
}
@Deprecated
@JsonProperty("firehoseId")
public String getFirehoseId()
{
return serviceName.replaceFirst(String.format("%s:", config.getFirehoseIdPrefix()), "");
}
@JsonProperty
public int getBufferSize()
{
......
......@@ -17,23 +17,59 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.overlord.config;
package io.druid.indexing.common.task;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.db.DbConnectorConfig;
import org.skife.config.Config;
import io.druid.indexing.common.actions.LockTryAcquireAction;
import io.druid.indexing.common.actions.TaskActionClient;
import org.joda.time.Interval;
public abstract class IndexerDbConnectorConfig extends DbConnectorConfig
public abstract class AbstractFixedIntervalTask extends AbstractTask
{
@JsonProperty("taskTable")
@Config("druid.database.taskTable")
public abstract String getTaskTable();
@JsonIgnore
private final Interval interval;
@JsonProperty("taskLockTable")
@Config("druid.database.taskLockTable")
public abstract String getTaskLockTable();
protected AbstractFixedIntervalTask(
String id,
String dataSource,
Interval interval
)
{
this(id, id, new TaskResource(id, 1), dataSource, interval);
}
@JsonProperty("taskLogTable")
@Config("druid.database.taskLogTable")
public abstract String getTaskLogTable();
protected AbstractFixedIntervalTask(
String id,
String groupId,
String dataSource,
Interval interval
)
{
this(id, groupId, new TaskResource(id, 1), dataSource, interval);
}
protected AbstractFixedIntervalTask(
String id,
String groupId,
TaskResource taskResource,
String dataSource,
Interval interval
)
{
super(id, groupId, taskResource, dataSource);
this.interval = interval;
}
@Override
public boolean isReady(TaskActionClient taskActionClient) throws Exception
{
return taskActionClient.submit(new LockTryAcquireAction(interval)).isPresent();
}
@JsonProperty
public Interval getInterval()
{
return interval;
}
}
......@@ -23,21 +23,15 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Joiner;
import com.google.common.base.Objects;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.LockAcquireAction;
import io.druid.indexing.common.actions.LockListAction;
import io.druid.indexing.common.actions.SegmentListUsedAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import org.joda.time.Interval;
import java.io.IOException;
import java.util.List;
public abstract class AbstractTask implements Task
{
......@@ -55,26 +49,22 @@ public abstract class AbstractTask implements Task
@JsonIgnore
private final String dataSource;
@JsonIgnore
private final Optional<Interval> interval;
protected AbstractTask(String id, String dataSource, Interval interval)
protected AbstractTask(String id, String dataSource)
{
this(id, id, new TaskResource(id, 1), dataSource, interval);
this(id, id, new TaskResource(id, 1), dataSource);
}
protected AbstractTask(String id, String groupId, String dataSource, Interval interval)
protected AbstractTask(String id, String groupId, String dataSource)
{
this(id, groupId, new TaskResource(id, 1), dataSource, interval);
this(id, groupId, new TaskResource(id, 1), dataSource);
}
protected AbstractTask(String id, String groupId, TaskResource taskResource, String dataSource, Interval interval)
protected AbstractTask(String id, String groupId, TaskResource taskResource, String dataSource)
{
this.id = Preconditions.checkNotNull(id, "id");
this.groupId = Preconditions.checkNotNull(groupId, "groupId");
this.taskResource = Preconditions.checkNotNull(taskResource, "resource");
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
this.interval = Optional.fromNullable(interval);
}
@JsonProperty
......@@ -111,25 +101,12 @@ public abstract class AbstractTask implements Task
return dataSource;
}
@JsonProperty("interval")
@Override
public Optional<Interval> getImplicitLockInterval()
{
return interval;
}
@Override
public <T> QueryRunner<T> getQueryRunner(Query<T> query)
{
return null;
}
@Override
public TaskStatus preflight(TaskActionClient taskActionClient) throws Exception
{
return TaskStatus.running(id);
}
@Override
public String toString()
{
......@@ -137,7 +114,6 @@ public abstract class AbstractTask implements Task
.add("id", id)
.add("type", getType())
.add("dataSource", dataSource)
.add("interval", getImplicitLockInterval())
.toString();
}
......@@ -149,11 +125,6 @@ public abstract class AbstractTask implements Task
return ID_JOINER.join(objects);
}
public SegmentListUsedAction defaultListUsedAction()
{
return new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get());
}
public TaskStatus success()
{
return TaskStatus.success(getId());
......@@ -186,14 +157,6 @@ public abstract class AbstractTask implements Task
protected Iterable<TaskLock> getTaskLocks(TaskToolbox toolbox) throws IOException
{
final List<TaskLock> locks = toolbox.getTaskActionClient().submit(new LockListAction());
if (locks.isEmpty() && getImplicitLockInterval().isPresent()) {
// In the Peon's local mode, the implicit lock interval is not pre-acquired, so we need to try it here.
toolbox.getTaskActionClient().submit(new LockAcquireAction(getImplicitLockInterval().get()));
return toolbox.getTaskActionClient().submit(new LockListAction());
} else {
return locks;
}
return toolbox.getTaskActionClient().submit(new LockListAction());
}
}
......@@ -30,7 +30,6 @@ import io.druid.granularity.QueryGranularity;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.LockListAction;
import io.druid.indexing.common.actions.SegmentInsertAction;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.IndexMerger;
......@@ -44,7 +43,7 @@ import org.joda.time.Interval;
import java.io.File;
public class DeleteTask extends AbstractTask
public class DeleteTask extends AbstractFixedIntervalTask
{
private static final Logger log = new Logger(DeleteTask.class);
......@@ -78,16 +77,15 @@ public class DeleteTask extends AbstractTask
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
// Strategy: Create an empty segment covering the interval to be deleted
final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox));
final Interval interval = this.getImplicitLockInterval().get();
final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox));
final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty);
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(getInterval(), empty);
// Create DataSegment
final DataSegment segment =
DataSegment.builder()
.dataSource(this.getDataSource())
.interval(interval)
.interval(getInterval())
.version(myLock.getVersion())
.shardSpec(new NoneShardSpec())
.build();
......
......@@ -37,12 +37,15 @@ import io.druid.indexer.HadoopDruidIndexerSchema;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.LockTryAcquireAction;
import io.druid.indexing.common.actions.SegmentInsertAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.initialization.Initialization;
import io.druid.server.initialization.ExtensionsConfig;
import io.druid.timeline.DataSegment;
import io.tesla.aether.internal.DefaultTeslaAether;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.io.File;
import java.lang.reflect.Method;
......@@ -51,7 +54,7 @@ import java.net.URLClassLoader;
import java.util.Arrays;
import java.util.List;
public class HadoopIndexTask extends AbstractTask
public class HadoopIndexTask extends AbstractFixedIntervalTask
{
private static final Logger log = new Logger(HadoopIndexTask.class);
private static String defaultHadoopCoordinates = "org.apache.hadoop:hadoop-core:1.0.3";
......@@ -88,10 +91,14 @@ public class HadoopIndexTask extends AbstractTask
super(
id != null ? id : String.format("index_hadoop_%s_%s", schema.getDataSource(), new DateTime()),
schema.getDataSource(),
JodaUtils.umbrellaInterval(JodaUtils.condenseIntervals(schema.getGranularitySpec().bucketIntervals()))
JodaUtils.umbrellaInterval(
JodaUtils.condenseIntervals(
schema.getGranularitySpec()
.bucketIntervals()
)
)
);
// Some HadoopDruidIndexerSchema stuff doesn't make sense in the context of the indexing service
Preconditions.checkArgument(schema.getSegmentOutputPath() == null, "segmentOutputPath must be absent");
Preconditions.checkArgument(schema.getWorkingPath() == null, "workingPath must be absent");
......@@ -107,7 +114,6 @@ public class HadoopIndexTask extends AbstractTask
return "index_hadoop";
}
@JsonProperty("config")
public HadoopDruidIndexerSchema getSchema()
{
......
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.common.collect.TreeMultiset;
import com.google.common.primitives.Ints;
import com.metamx.common.logger.Logger;
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.InputRow;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.SpawnTasksAction;
import io.druid.segment.realtime.Schema;
import io.druid.timeline.partition.NoneShardSpec;
import io.druid.timeline.partition.ShardSpec;
import io.druid.timeline.partition.SingleDimensionShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.util.List;
import java.util.Map;
import java.util.Set;
public class IndexDeterminePartitionsTask extends AbstractTask
{
private static String makeTaskId(String groupId, DateTime start, DateTime end)
{
return String.format(
"%s_partitions_%s_%s",
groupId,
start,
end
);
}
@JsonIgnore
private final FirehoseFactory firehoseFactory;
@JsonIgnore
private final Schema schema;
@JsonIgnore
private final long targetPartitionSize;
@JsonIgnore
private final int rowFlushBoundary;
private static final Logger log = new Logger(IndexTask.class);
@JsonCreator
public IndexDeterminePartitionsTask(
@JsonProperty("id") String id,
@JsonProperty("groupId") String groupId,
@JsonProperty("interval") Interval interval,
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
@JsonProperty("schema") Schema schema,
@JsonProperty("targetPartitionSize") long targetPartitionSize,
@JsonProperty("rowFlushBoundary") int rowFlushBoundary
)
{
super(
id != null ? id : makeTaskId(groupId, interval.getStart(), interval.getEnd()),
groupId,
schema.getDataSource(),
Preconditions.checkNotNull(interval, "interval")
);
this.firehoseFactory = firehoseFactory;
this.schema = schema;
this.targetPartitionSize = targetPartitionSize;
this.rowFlushBoundary = rowFlushBoundary;
}
@Override
public String getType()
{
return "index_partitions";
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
log.info("Running with targetPartitionSize[%d]", targetPartitionSize);
// The implementation of this determine partitions stuff is less than optimal. Should be done better.
// We know this exists
final Interval interval = getImplicitLockInterval().get();
// Blacklist dimensions that have multiple values per row
final Set<String> unusableDimensions = Sets.newHashSet();
// Track values of all non-blacklisted dimensions
final Map<String, TreeMultiset<String>> dimensionValueMultisets = Maps.newHashMap();
// Load data
final Firehose firehose = firehoseFactory.connect();
try {
while (firehose.hasMore()) {
final InputRow inputRow = firehose.nextRow();
if (interval.contains(inputRow.getTimestampFromEpoch())) {
// Extract dimensions from event
for (final String dim : inputRow.getDimensions()) {
final List<String> dimValues = inputRow.getDimension(dim);
if (!unusableDimensions.contains(dim)) {
if (dimValues.size() == 1) {
// Track this value
TreeMultiset<String> dimensionValueMultiset = dimensionValueMultisets.get(dim);
if (dimensionValueMultiset == null) {
dimensionValueMultiset = TreeMultiset.create();
dimensionValueMultisets.put(dim, dimensionValueMultiset);
}
dimensionValueMultiset.add(dimValues.get(0));
} else {
// Only single-valued dimensions can be used for partitions
unusableDimensions.add(dim);
dimensionValueMultisets.remove(dim);
}
}
}
}
}
}
finally {
firehose.close();
}
// ShardSpecs for index generator tasks
final List<ShardSpec> shardSpecs = Lists.newArrayList();
// Select highest-cardinality dimension
Ordering<Map.Entry<String, TreeMultiset<String>>> byCardinalityOrdering = new Ordering<Map.Entry<String, TreeMultiset<String>>>()
{
@Override
public int compare(
Map.Entry<String, TreeMultiset<String>> left,
Map.Entry<String, TreeMultiset<String>> right
)
{
return Ints.compare(left.getValue().elementSet().size(), right.getValue().elementSet().size());
}
};
if (dimensionValueMultisets.isEmpty()) {
// No suitable partition dimension. We'll make one big segment and hope for the best.
log.info("No suitable partition dimension found");
shardSpecs.add(new NoneShardSpec());
} else {
// Find best partition dimension (heuristic: highest cardinality).
final Map.Entry<String, TreeMultiset<String>> partitionEntry =
byCardinalityOrdering.max(dimensionValueMultisets.entrySet());
final String partitionDim = partitionEntry.getKey();
final TreeMultiset<String> partitionDimValues = partitionEntry.getValue();
log.info(
"Partitioning on dimension[%s] with cardinality[%d] over rows[%d]",
partitionDim,
partitionDimValues.elementSet().size(),
partitionDimValues.size()
);
// Iterate over unique partition dimension values in sorted order
String currentPartitionStart = null;
int currentPartitionSize = 0;
for (final String partitionDimValue : partitionDimValues.elementSet()) {
currentPartitionSize += partitionDimValues.count(partitionDimValue);
if (currentPartitionSize >= targetPartitionSize) {
final ShardSpec shardSpec = new SingleDimensionShardSpec(
partitionDim,
currentPartitionStart,
partitionDimValue,
shardSpecs.size()
);
log.info("Adding shard: %s", shardSpec);
shardSpecs.add(shardSpec);
currentPartitionSize = partitionDimValues.count(partitionDimValue);
currentPartitionStart = partitionDimValue;
}
}
if (currentPartitionSize > 0) {
// One last shard to go
final ShardSpec shardSpec;
if (shardSpecs.isEmpty()) {
shardSpec = new NoneShardSpec();
} else {
shardSpec = new SingleDimensionShardSpec(
partitionDim,
currentPartitionStart,
null,
shardSpecs.size()
);
}
log.info("Adding shard: %s", shardSpec);
shardSpecs.add(shardSpec);
}
}
List<Task> nextTasks = Lists.transform(
shardSpecs,
new Function<ShardSpec, Task>()
{
@Override
public Task apply(ShardSpec shardSpec)
{
return new IndexGeneratorTask(
null,
getGroupId(),
getImplicitLockInterval().get(),
firehoseFactory,
new Schema(
schema.getDataSource(),
schema.getSpatialDimensions(),
schema.getAggregators(),
schema.getIndexGranularity(),
shardSpec
),
rowFlushBoundary
);
}
}
);
toolbox.getTaskActionClient().submit(new SpawnTasksAction(nextTasks));
return TaskStatus.success(getId());
}
@JsonProperty
public FirehoseFactory getFirehoseFactory()
{
return firehoseFactory;
}
@JsonProperty
public Schema getSchema()
{
return schema;
}
@JsonProperty
public long getTargetPartitionSize()
{
return targetPartitionSize;
}
@JsonProperty
public int getRowFlushBoundary()
{
return rowFlushBoundary;
}
}
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.metamx.common.logger.Logger;
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.InputRow;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.SegmentInsertAction;
import io.druid.indexing.common.index.YeOldePlumberSchool;
import io.druid.segment.loading.DataSegmentPusher;
import io.druid.segment.realtime.FireDepartmentMetrics;
import io.druid.segment.realtime.Schema;
import io.druid.segment.realtime.plumber.Plumber;
import io.druid.segment.realtime.plumber.Sink;
import io.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
public class IndexGeneratorTask extends AbstractTask
{
@JsonIgnore
private final FirehoseFactory firehoseFactory;
@JsonIgnore
private final Schema schema;
@JsonIgnore
private final int rowFlushBoundary;
private static final Logger log = new Logger(IndexTask.class);
@JsonCreator
public IndexGeneratorTask(
@JsonProperty("id") String id,
@JsonProperty("groupId") String groupId,
@JsonProperty("interval") Interval interval,
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
@JsonProperty("schema") Schema schema,
@JsonProperty("rowFlushBoundary") int rowFlushBoundary
)
{
super(
id != null
? id
: String.format(
"%s_generator_%s_%s_%s",
groupId,
interval.getStart(),
interval.getEnd(),
schema.getShardSpec().getPartitionNum()
),
groupId,
schema.getDataSource(),
Preconditions.checkNotNull(interval, "interval")
);
this.firehoseFactory = firehoseFactory;
this.schema = schema;
this.rowFlushBoundary = rowFlushBoundary;
}
@Override
public String getType()
{
return "index_generator";
}
@Override
public TaskStatus run(final TaskToolbox toolbox) throws Exception
{
// We should have a lock from before we started running
final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox));
// We know this exists
final Interval interval = getImplicitLockInterval().get();
// Set up temporary directory for indexing
final File tmpDir = new File(
toolbox.getTaskWorkDir(),
String.format(
"%s_%s_%s_%s_%s",
this.getDataSource(),
interval.getStart(),
interval.getEnd(),
myLock.getVersion(),
schema.getShardSpec().getPartitionNum()
)
);
// We need to track published segments.
final List<DataSegment> pushedSegments = new CopyOnWriteArrayList<DataSegment>();
final DataSegmentPusher wrappedDataSegmentPusher = new DataSegmentPusher()
{
@Override
public String getPathForHadoop(String dataSource)
{
return toolbox.getSegmentPusher().getPathForHadoop(dataSource);
}
@Override
public DataSegment push(File file, DataSegment segment) throws IOException
{
final DataSegment pushedSegment = toolbox.getSegmentPusher().push(file, segment);
pushedSegments.add(pushedSegment);
return pushedSegment;
}
};
// Create firehose + plumber
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
final Firehose firehose = firehoseFactory.connect();
final Plumber plumber = new YeOldePlumberSchool(
interval,
myLock.getVersion(),
wrappedDataSegmentPusher,
tmpDir
).findPlumber(schema, metrics);
// rowFlushBoundary for this job
final int myRowFlushBoundary = this.rowFlushBoundary > 0
? rowFlushBoundary
: toolbox.getConfig().getDefaultRowFlushBoundary();
try {
while (firehose.hasMore()) {
final InputRow inputRow = firehose.nextRow();
if (shouldIndex(inputRow)) {
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
if (sink == null) {
throw new NullPointerException(
String.format(
"Was expecting non-null sink for timestamp[%s]",
new DateTime(inputRow.getTimestampFromEpoch())
)
);
}
int numRows = sink.add(inputRow);
metrics.incrementProcessed();
if (numRows >= myRowFlushBoundary) {
plumber.persist(firehose.commit());
}
} else {
metrics.incrementThrownAway();
}
}
}
finally {
firehose.close();
}
plumber.persist(firehose.commit());
plumber.finishJob();
// Output metrics
log.info(
"Task[%s] took in %,d rows (%,d processed, %,d unparseable, %,d thrown away) and output %,d rows",
getId(),
metrics.processed() + metrics.unparseable() + metrics.thrownAway(),
metrics.processed(),
metrics.unparseable(),
metrics.thrownAway(),
metrics.rowOutput()
);
// Request segment pushes
toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(pushedSegments)));
// Done
return TaskStatus.success(getId());
}
/**
* Should we index this inputRow? Decision is based on our interval and shardSpec.
*
* @param inputRow the row to check
*
* @return true or false
*/
private boolean shouldIndex(InputRow inputRow)
{
if (getImplicitLockInterval().get().contains(inputRow.getTimestampFromEpoch())) {
return schema.getShardSpec().isInChunk(inputRow);
} else {
return false;
}
}
@JsonProperty("firehose")
public FirehoseFactory getFirehoseFactory()
{
return firehoseFactory;
}
@JsonProperty
public Schema getSchema()
{
return schema;
}
@JsonProperty
public int getRowFlushBoundary()
{
return rowFlushBoundary;
}
}
......@@ -22,26 +22,48 @@ package io.druid.indexing.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.api.client.util.Sets;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.collect.TreeMultiset;
import com.google.common.primitives.Ints;
import com.metamx.common.logger.Logger;
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.InputRow;
import io.druid.data.input.impl.SpatialDimensionSchema;
import io.druid.granularity.QueryGranularity;
import io.druid.indexer.granularity.GranularitySpec;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.SpawnTasksAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.actions.SegmentInsertAction;
import io.druid.indexing.common.index.YeOldePlumberSchool;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.loading.DataSegmentPusher;
import io.druid.segment.realtime.FireDepartmentMetrics;
import io.druid.segment.realtime.Schema;
import io.druid.segment.realtime.plumber.Plumber;
import io.druid.segment.realtime.plumber.Sink;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import io.druid.timeline.partition.ShardSpec;
import io.druid.timeline.partition.SingleDimensionShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
public class IndexTask extends AbstractTask
public class IndexTask extends AbstractFixedIntervalTask
{
private static final Logger log = new Logger(IndexTask.class);
......@@ -58,7 +80,7 @@ public class IndexTask extends AbstractTask
private final QueryGranularity indexGranularity;
@JsonIgnore
private final long targetPartitionSize;
private final int targetPartitionSize;
@JsonIgnore
private final FirehoseFactory firehoseFactory;
......@@ -74,7 +96,7 @@ public class IndexTask extends AbstractTask
@JsonProperty("spatialDimensions") List<SpatialDimensionSchema> spatialDimensions,
@JsonProperty("aggregators") AggregatorFactory[] aggregators,
@JsonProperty("indexGranularity") QueryGranularity indexGranularity,
@JsonProperty("targetPartitionSize") long targetPartitionSize,
@JsonProperty("targetPartitionSize") int targetPartitionSize,
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
@JsonProperty("rowFlushBoundary") int rowFlushBoundary
)
......@@ -96,75 +118,283 @@ public class IndexTask extends AbstractTask
this.aggregators = aggregators;
this.indexGranularity = (indexGranularity == null) ? QueryGranularity.NONE : indexGranularity;
this.targetPartitionSize = targetPartitionSize;
this.firehoseFactory = firehoseFactory;
this.firehoseFactory = Preconditions.checkNotNull(firehoseFactory, "firehoseFactory");
this.rowFlushBoundary = rowFlushBoundary;
}
public List<Task> toSubtasks()
@Override
public String getType()
{
final List<Task> retVal = Lists.newArrayList();
return "index";
}
for (final Interval interval : granularitySpec.bucketIntervals()) {
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox));
final Set<DataSegment> segments = Sets.newHashSet();
for (final Interval bucket : granularitySpec.bucketIntervals()) {
final List<ShardSpec> shardSpecs;
if (targetPartitionSize > 0) {
// Need to do one pass over the data before indexing in order to determine good partitions
retVal.add(
new IndexDeterminePartitionsTask(
null,
getGroupId(),
interval,
firehoseFactory,
new Schema(
getDataSource(),
spatialDimensions,
aggregators,
indexGranularity,
new NoneShardSpec()
),
targetPartitionSize,
rowFlushBoundary
)
);
shardSpecs = determinePartitions(bucket, targetPartitionSize);
} else {
// Jump straight into indexing
retVal.add(
new IndexGeneratorTask(
null,
getGroupId(),
interval,
firehoseFactory,
new Schema(
getDataSource(),
spatialDimensions,
aggregators,
indexGranularity,
new NoneShardSpec()
),
rowFlushBoundary
)
shardSpecs = ImmutableList.<ShardSpec>of(new NoneShardSpec());
}
for (final ShardSpec shardSpec : shardSpecs) {
final DataSegment segment = generateSegment(
toolbox,
new Schema(
getDataSource(),
spatialDimensions,
aggregators,
indexGranularity,
shardSpec
),
bucket,
myLock.getVersion()
);
segments.add(segment);
}
}
return retVal;
toolbox.getTaskActionClient().submit(new SegmentInsertAction(segments));
return TaskStatus.success(getId());
}
@Override
public String getType()
private List<ShardSpec> determinePartitions(
final Interval interval,
final int targetPartitionSize
) throws IOException
{
return "index";
log.info("Determining partitions for interval[%s] with targetPartitionSize[%d]", interval, targetPartitionSize);
// The implementation of this determine partitions stuff is less than optimal. Should be done better.
// Blacklist dimensions that have multiple values per row
final Set<String> unusableDimensions = com.google.common.collect.Sets.newHashSet();
// Track values of all non-blacklisted dimensions
final Map<String, TreeMultiset<String>> dimensionValueMultisets = Maps.newHashMap();
// Load data
try (Firehose firehose = firehoseFactory.connect()) {
while (firehose.hasMore()) {
final InputRow inputRow = firehose.nextRow();
if (interval.contains(inputRow.getTimestampFromEpoch())) {
// Extract dimensions from event
for (final String dim : inputRow.getDimensions()) {
final List<String> dimValues = inputRow.getDimension(dim);
if (!unusableDimensions.contains(dim)) {
if (dimValues.size() == 1) {
// Track this value
TreeMultiset<String> dimensionValueMultiset = dimensionValueMultisets.get(dim);
if (dimensionValueMultiset == null) {
dimensionValueMultiset = TreeMultiset.create();
dimensionValueMultisets.put(dim, dimensionValueMultiset);
}
dimensionValueMultiset.add(dimValues.get(0));
} else {
// Only single-valued dimensions can be used for partitions
unusableDimensions.add(dim);
dimensionValueMultisets.remove(dim);
}
}
}
}
}
}
// ShardSpecs we will return
final List<ShardSpec> shardSpecs = Lists.newArrayList();
// Select highest-cardinality dimension
Ordering<Map.Entry<String, TreeMultiset<String>>> byCardinalityOrdering = new Ordering<Map.Entry<String, TreeMultiset<String>>>()
{
@Override
public int compare(
Map.Entry<String, TreeMultiset<String>> left,
Map.Entry<String, TreeMultiset<String>> right
)
{
return Ints.compare(left.getValue().elementSet().size(), right.getValue().elementSet().size());
}
};
if (dimensionValueMultisets.isEmpty()) {
// No suitable partition dimension. We'll make one big segment and hope for the best.
log.info("No suitable partition dimension found");
shardSpecs.add(new NoneShardSpec());
} else {
// Find best partition dimension (heuristic: highest cardinality).
final Map.Entry<String, TreeMultiset<String>> partitionEntry =
byCardinalityOrdering.max(dimensionValueMultisets.entrySet());
final String partitionDim = partitionEntry.getKey();
final TreeMultiset<String> partitionDimValues = partitionEntry.getValue();
log.info(
"Partitioning on dimension[%s] with cardinality[%d] over rows[%d]",
partitionDim,
partitionDimValues.elementSet().size(),
partitionDimValues.size()
);
// Iterate over unique partition dimension values in sorted order
String currentPartitionStart = null;
int currentPartitionSize = 0;
for (final String partitionDimValue : partitionDimValues.elementSet()) {
currentPartitionSize += partitionDimValues.count(partitionDimValue);
if (currentPartitionSize >= targetPartitionSize) {
final ShardSpec shardSpec = new SingleDimensionShardSpec(
partitionDim,
currentPartitionStart,
partitionDimValue,
shardSpecs.size()
);
log.info("Adding shard: %s", shardSpec);
shardSpecs.add(shardSpec);
currentPartitionSize = partitionDimValues.count(partitionDimValue);
currentPartitionStart = partitionDimValue;
}
}
if (currentPartitionSize > 0) {
// One last shard to go
final ShardSpec shardSpec;
if (shardSpecs.isEmpty()) {
shardSpec = new NoneShardSpec();
} else {
shardSpec = new SingleDimensionShardSpec(
partitionDim,
currentPartitionStart,
null,
shardSpecs.size()
);
}
log.info("Adding shard: %s", shardSpec);
shardSpecs.add(shardSpec);
}
}
return shardSpecs;
}
@Override
public TaskStatus preflight(TaskActionClient taskActionClient) throws Exception
private DataSegment generateSegment(
final TaskToolbox toolbox,
final Schema schema,
final Interval interval,
final String version
) throws IOException
{
taskActionClient.submit(new SpawnTasksAction(toSubtasks()));
return TaskStatus.success(getId());
// Set up temporary directory.
final File tmpDir = new File(
toolbox.getTaskWorkDir(),
String.format(
"%s_%s_%s_%s_%s",
this.getDataSource(),
interval.getStart(),
interval.getEnd(),
version,
schema.getShardSpec().getPartitionNum()
)
);
// We need to track published segments.
final List<DataSegment> pushedSegments = new CopyOnWriteArrayList<DataSegment>();
final DataSegmentPusher wrappedDataSegmentPusher = new DataSegmentPusher()
{
@Override
public String getPathForHadoop(String dataSource)
{
return toolbox.getSegmentPusher().getPathForHadoop(dataSource);
}
@Override
public DataSegment push(File file, DataSegment segment) throws IOException
{
final DataSegment pushedSegment = toolbox.getSegmentPusher().push(file, segment);
pushedSegments.add(pushedSegment);
return pushedSegment;
}
};
// Create firehose + plumber
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
final Firehose firehose = firehoseFactory.connect();
final Plumber plumber = new YeOldePlumberSchool(
interval,
version,
wrappedDataSegmentPusher,
tmpDir
).findPlumber(schema, metrics);
// rowFlushBoundary for this job
final int myRowFlushBoundary = this.rowFlushBoundary > 0
? rowFlushBoundary
: toolbox.getConfig().getDefaultRowFlushBoundary();
try {
plumber.startJob();
while (firehose.hasMore()) {
final InputRow inputRow = firehose.nextRow();
if (shouldIndex(schema, interval, inputRow)) {
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
if (sink == null) {
throw new NullPointerException(
String.format(
"Was expecting non-null sink for timestamp[%s]",
new DateTime(inputRow.getTimestampFromEpoch())
)
);
}
int numRows = sink.add(inputRow);
metrics.incrementProcessed();
if (numRows >= myRowFlushBoundary) {
plumber.persist(firehose.commit());
}
} else {
metrics.incrementThrownAway();
}
}
}
finally {
firehose.close();
}
plumber.persist(firehose.commit());
plumber.finishJob();
// Output metrics
log.info(
"Task[%s] took in %,d rows (%,d processed, %,d unparseable, %,d thrown away) and output %,d rows",
getId(),
metrics.processed() + metrics.unparseable() + metrics.thrownAway(),
metrics.processed(),
metrics.unparseable(),
metrics.thrownAway(),
metrics.rowOutput()
);
// We expect a single segment to have been created.
return Iterables.getOnlyElement(pushedSegments);
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
/**
* Should we index this inputRow? Decision is based on our interval and shardSpec.
*
* @param inputRow the row to check
*
* @return true or false
*/
private boolean shouldIndex(final Schema schema, final Interval interval, final InputRow inputRow)
{
throw new IllegalStateException("IndexTasks should not be run!");
return interval.contains(inputRow.getTimestampFromEpoch()) && schema.getShardSpec().isInChunk(inputRow);
}
@JsonProperty
......@@ -191,7 +421,7 @@ public class IndexTask extends AbstractTask
return targetPartitionSize;
}
@JsonProperty
@JsonProperty("firehose")
public FirehoseFactory getFirehoseFactory()
{
return firehoseFactory;
......@@ -202,4 +432,10 @@ public class IndexTask extends AbstractTask
{
return rowFlushBoundary;
}
@JsonProperty
public List<SpatialDimensionSchema> getSpatialDimensions()
{
return spatialDimensions;
}
}
......@@ -28,7 +28,6 @@ import com.metamx.common.logger.Logger;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.LockListAction;
import io.druid.indexing.common.actions.SegmentListUnusedAction;
import io.druid.indexing.common.actions.SegmentNukeAction;
import io.druid.timeline.DataSegment;
......@@ -38,7 +37,7 @@ import java.util.List;
/**
*/
public class KillTask extends AbstractTask
public class KillTask extends AbstractFixedIntervalTask
{
private static final Logger log = new Logger(KillTask.class);
......@@ -68,12 +67,12 @@ public class KillTask extends AbstractTask
// Confirm we have a lock (will throw if there isn't exactly one element)
final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox));
if(!myLock.getDataSource().equals(getDataSource())) {
if (!myLock.getDataSource().equals(getDataSource())) {
throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource());
}
if(!myLock.getInterval().equals(getImplicitLockInterval().get())) {
throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getImplicitLockInterval().get());
if (!myLock.getInterval().equals(getInterval())) {
throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getInterval());
}
// List unused segments
......@@ -82,8 +81,8 @@ public class KillTask extends AbstractTask
.submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval()));
// Verify none of these segments have versions > lock version
for(final DataSegment unusedSegment : unusedSegments) {
if(unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
for (final DataSegment unusedSegment : unusedSegments) {
if (unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
unusedSegment.getIdentifier(),
......
......@@ -27,7 +27,6 @@ import com.google.common.base.Joiner;
import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
......@@ -41,9 +40,8 @@ import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.LockAcquireAction;
import io.druid.indexing.common.actions.LockListAction;
import io.druid.indexing.common.actions.SegmentInsertAction;
import io.druid.indexing.common.actions.SegmentListUsedAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.segment.IndexIO;
import io.druid.timeline.DataSegment;
......@@ -53,14 +51,13 @@ import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
*/
public abstract class MergeTaskBase extends AbstractTask
public abstract class MergeTaskBase extends AbstractFixedIntervalTask
{
@JsonIgnore
private final List<DataSegment> segments;
......@@ -186,9 +183,12 @@ public abstract class MergeTaskBase extends AbstractTask
* we are operating on every segment that overlaps the chosen interval.
*/
@Override
public TaskStatus preflight(TaskActionClient taskActionClient)
public boolean isReady(TaskActionClient taskActionClient) throws Exception
{
try {
// Try to acquire lock
if (!super.isReady(taskActionClient)) {
return false;
} else {
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
{
@Override
......@@ -199,7 +199,10 @@ public abstract class MergeTaskBase extends AbstractTask
};
final Set<String> current = ImmutableSet.copyOf(
Iterables.transform(taskActionClient.submit(defaultListUsedAction()), toIdentifier)
Iterables.transform(
taskActionClient.submit(new SegmentListUsedAction(getDataSource(), getInterval())),
toIdentifier
)
);
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
......@@ -219,10 +222,7 @@ public abstract class MergeTaskBase extends AbstractTask
);
}
return TaskStatus.running(getId());
}
catch (IOException e) {
throw Throwables.propagate(e);
return true;
}
}
......@@ -241,7 +241,7 @@ public abstract class MergeTaskBase extends AbstractTask
return Objects.toStringHelper(this)
.add("id", getId())
.add("dataSource", getDataSource())
.add("interval", getImplicitLockInterval())
.add("interval", getInterval())
.add("segments", segments)
.toString();
}
......
......@@ -25,9 +25,8 @@ import com.metamx.common.logger.Logger;
import io.druid.data.input.FirehoseFactory;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.TaskActionClient;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.Period;
/**
*/
......@@ -42,19 +41,16 @@ public class NoopTask extends AbstractTask
@JsonCreator
public NoopTask(
@JsonProperty("id") String id,
@JsonProperty("interval") Interval interval,
@JsonProperty("runTime") int runTime,
@JsonProperty("firehose") FirehoseFactory firehoseFactory
)
{
super(
id == null ? String.format("noop_%s", new DateTime()) : id,
"none",
interval == null ? new Interval(Period.days(1), new DateTime()) : interval
"none"
);
this.runTime = (runTime == 0) ? defaultRunTime : runTime;
this.firehoseFactory = firehoseFactory;
}
......@@ -76,6 +72,12 @@ public class NoopTask extends AbstractTask
return firehoseFactory;
}
@Override
public boolean isReady(TaskActionClient taskActionClient) throws Exception
{
return true;
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
......
......@@ -38,6 +38,7 @@ import io.druid.indexing.common.actions.LockAcquireAction;
import io.druid.indexing.common.actions.LockListAction;
import io.druid.indexing.common.actions.LockReleaseAction;
import io.druid.indexing.common.actions.SegmentInsertAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
......@@ -130,8 +131,7 @@ public class RealtimeIndexTask extends AbstractTask
), 1
)
: taskResource,
schema.getDataSource(),
null
schema.getDataSource()
);
this.schema = schema;
......@@ -167,6 +167,12 @@ public class RealtimeIndexTask extends AbstractTask
}
}
@Override
public boolean isReady(TaskActionClient taskActionClient) throws Exception
{
return true;
}
@Override
public TaskStatus run(final TaskToolbox toolbox) throws Exception
{
......
......@@ -21,27 +21,22 @@ package io.druid.indexing.common.task;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.common.base.Optional;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import org.joda.time.Interval;
/**
* Represents a task that can run on a worker. The general contracts surrounding Tasks are:
* <ul>
* <li>Tasks must operate on a single datasource.</li>
* <li>Tasks should be immutable, since the task ID is used as a proxy for the task in many locations.</li>
* <li>Task IDs must be unique. This can be done by naming them using UUIDs or the current timestamp.</li>
* <li>Tasks are each part of a "task group", which is a set of tasks that can share interval locks. These are
* useful for producing sharded segments.</li>
* <li>Tasks can optionally have an "implicit lock interval". Tasks with this property are guaranteed to have
* a lock on that interval during their {@link #preflight(io.druid.indexing.common.actions.TaskActionClient)}
* and {@link #run(io.druid.indexing.common.TaskToolbox)} methods.</li>
* <li>Tasks do not need to explicitly release locks; they are released upon task completion. Tasks may choose
* to release locks early if they desire.</li>
* <li>Tasks must operate on a single datasource.</li>
* <li>Tasks should be immutable, since the task ID is used as a proxy for the task in many locations.</li>
* <li>Task IDs must be unique. This can be done by naming them using UUIDs or the current timestamp.</li>
* <li>Tasks are each part of a "task group", which is a set of tasks that can share interval locks. These are
* useful for producing sharded segments.</li>
* <li>Tasks do not need to explicitly release locks; they are released upon task completion. Tasks may choose
* to release locks early if they desire.</li>
* </ul>
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
......@@ -51,8 +46,6 @@ import org.joda.time.Interval;
@JsonSubTypes.Type(name = "delete", value = DeleteTask.class),
@JsonSubTypes.Type(name = "kill", value = KillTask.class),
@JsonSubTypes.Type(name = "index", value = IndexTask.class),
@JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class),
@JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class),
@JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class),
@JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class),
@JsonSubTypes.Type(name = "noop", value = NoopTask.class),
......@@ -96,12 +89,6 @@ public interface Task
*/
public String getDataSource();
/**
* Returns implicit lock interval for this task, if any. Tasks without implicit lock intervals are not granted locks
* when started and must explicitly request them.
*/
public Optional<Interval> getImplicitLockInterval();
/**
* Returns query runners for this task. If this task is not meant to answer queries over its datasource, this method
* should return null.
......@@ -109,18 +96,19 @@ public interface Task
public <T> QueryRunner<T> getQueryRunner(Query<T> query);
/**
* Execute preflight checks for a task. This typically runs on the coordinator, 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 should be considered a failure.
* 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
* coordinator. If this method throws an exception, the task should be considered a failure.
*
* 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)
*
* @return Some kind of status (runnable means continue on to a worker, non-runnable means we completed without
* using a worker).
* @return true if ready, false if not ready yet
*
* @throws Exception
* @throws Exception if the task should be considered a failure
*/
public TaskStatus preflight(TaskActionClient taskActionClient) throws Exception;
public boolean isReady(TaskActionClient taskActionClient) throws Exception;
/**
* Execute a task. This typically runs on a worker as determined by a TaskRunner, and will be run while
......
......@@ -23,16 +23,14 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.collect.Lists;
import com.google.common.base.Preconditions;
import com.google.common.collect.Sets;
import com.metamx.common.ISE;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.logger.Logger;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.SegmentInsertAction;
import io.druid.indexing.common.actions.SegmentListUsedAction;
import io.druid.indexing.common.actions.SpawnTasksAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.segment.IndexIO;
import io.druid.segment.loading.SegmentLoadingException;
......@@ -48,10 +46,10 @@ import java.util.Map;
/**
*/
public class VersionConverterTask extends AbstractTask
public class VersionConverterTask extends AbstractFixedIntervalTask
{
private static final String TYPE = "version_converter";
private static final Integer CURR_VERSION_INTEGER = new Integer(IndexIO.CURRENT_VERSION_ID);
private static final Integer CURR_VERSION_INTEGER = IndexIO.CURRENT_VERSION_ID;
private static final Logger log = new Logger(VersionConverterTask.class);
......@@ -74,6 +72,8 @@ public class VersionConverterTask extends AbstractTask
private static String makeId(String dataSource, Interval interval)
{
Preconditions.checkNotNull(dataSource, "dataSource");
Preconditions.checkNotNull(interval, "interval");
return joinId(TYPE, dataSource, interval.getStart(), interval.getEnd(), new DateTime());
}
......@@ -105,7 +105,6 @@ public class VersionConverterTask extends AbstractTask
)
{
super(id, groupId, dataSource, interval);
this.segment = segment;
}
......@@ -125,45 +124,43 @@ public class VersionConverterTask extends AbstractTask
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
if (segment == null) {
throw new ISE("Segment was null, this should never run.", this.getClass().getSimpleName());
}
log.info("I'm in a subless mood.");
convertSegment(toolbox, segment);
return success();
}
@Override
public TaskStatus preflight(TaskActionClient taskActionClient) throws Exception
{
if (segment != null) {
return super.preflight(taskActionClient);
}
List<DataSegment> segments = taskActionClient.submit(defaultListUsedAction());
final FunctionalIterable<Task> tasks = FunctionalIterable
.create(segments)
.keep(
new Function<DataSegment, Task>()
{
@Override
public Task apply(DataSegment segment)
final List<DataSegment> segments = toolbox.getTaskActionClient().submit(
new SegmentListUsedAction(
getDataSource(),
getInterval()
)
);
final FunctionalIterable<Task> tasks = FunctionalIterable
.create(segments)
.keep(
new Function<DataSegment, Task>()
{
final Integer segmentVersion = segment.getBinaryVersion();
if (!CURR_VERSION_INTEGER.equals(segmentVersion)) {
return new SubTask(getGroupId(), segment);
@Override
public Task apply(DataSegment segment)
{
final Integer segmentVersion = segment.getBinaryVersion();
if (!CURR_VERSION_INTEGER.equals(segmentVersion)) {
return new SubTask(getGroupId(), segment);
}
log.info("Skipping[%s], already version[%s]", segment.getIdentifier(), segmentVersion);
return null;
}
log.info("Skipping[%s], already version[%s]", segment.getIdentifier(), segmentVersion);
return null;
}
}
);
taskActionClient.submit(new SpawnTasksAction(Lists.newArrayList(tasks)));
return TaskStatus.success(getId());
);
// Vestigial from a past time when this task spawned subtasks.
for (final Task subTask : tasks) {
final TaskStatus status = subTask.run(toolbox);
if (!status.isSuccess()) {
return status;
}
}
} else {
log.info("I'm in a subless mood.");
convertSegment(toolbox, segment);
}
return success();
}
@Override
......@@ -185,7 +182,7 @@ public class VersionConverterTask extends AbstractTask
return super.equals(o);
}
public static class SubTask extends AbstractTask
public static class SubTask extends AbstractFixedIntervalTask
{
@JsonIgnore
private final DataSegment segment;
......
......@@ -23,15 +23,19 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.inject.Inject;
import com.metamx.common.RetryUtils;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.emitter.EmittingLogger;
import com.mysql.jdbc.exceptions.MySQLTimeoutException;
import com.mysql.jdbc.exceptions.MySQLTransientException;
import io.druid.db.DbConnector;
import io.druid.db.DbTablesConfig;
import io.druid.indexing.common.TaskLock;
......@@ -41,11 +45,18 @@ import io.druid.indexing.common.task.Task;
import org.joda.time.DateTime;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.exceptions.CallbackFailedException;
import org.skife.jdbi.v2.exceptions.DBIException;
import org.skife.jdbi.v2.exceptions.StatementException;
import org.skife.jdbi.v2.exceptions.UnableToObtainConnectionException;
import org.skife.jdbi.v2.tweak.HandleCallback;
import java.sql.SQLException;
import java.sql.SQLRecoverableException;
import java.sql.SQLTransientException;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
public class DbTaskStorage implements TaskStorage
{
......@@ -92,7 +103,7 @@ public class DbTaskStorage implements TaskStorage
log.info("Inserting task %s with status: %s", task.getId(), status);
try {
dbi.withHandle(
retryingHandle(
new HandleCallback<Void>()
{
@Override
......@@ -134,7 +145,7 @@ public class DbTaskStorage implements TaskStorage
log.info("Updating task %s to status: %s", status.getId(), status);
int updated = dbi.withHandle(
int updated = retryingHandle(
new HandleCallback<Integer>()
{
@Override
......@@ -162,7 +173,7 @@ public class DbTaskStorage implements TaskStorage
@Override
public Optional<Task> getTask(final String taskid)
{
return dbi.withHandle(
return retryingHandle(
new HandleCallback<Optional<Task>>()
{
@Override
......@@ -192,7 +203,7 @@ public class DbTaskStorage implements TaskStorage
@Override
public Optional<TaskStatus> getStatus(final String taskid)
{
return dbi.withHandle(
return retryingHandle(
new HandleCallback<Optional<TaskStatus>>()
{
@Override
......@@ -222,7 +233,7 @@ public class DbTaskStorage implements TaskStorage
@Override
public List<Task> getActiveTasks()
{
return dbi.withHandle(
return retryingHandle(
new HandleCallback<List<Task>>()
{
@Override
......@@ -231,7 +242,7 @@ public class DbTaskStorage implements TaskStorage
final List<Map<String, Object>> dbTasks =
handle.createQuery(
String.format(
"SELECT id, payload, status_payload FROM %s WHERE active = 1",
"SELECT id, payload, status_payload FROM %s WHERE active = 1 ORDER BY created_date",
dbTables.getTasksTable()
)
)
......@@ -273,7 +284,7 @@ public class DbTaskStorage implements TaskStorage
taskid
);
dbi.withHandle(
retryingHandle(
new HandleCallback<Integer>()
{
@Override
......@@ -308,7 +319,7 @@ public class DbTaskStorage implements TaskStorage
if (taskLock.equals(taskLockToRemove)) {
log.info("Deleting TaskLock with id[%d]: %s", id, taskLock);
dbi.withHandle(
retryingHandle(
new HandleCallback<Integer>()
{
@Override
......@@ -353,7 +364,7 @@ public class DbTaskStorage implements TaskStorage
log.info("Logging action for task[%s]: %s", task.getId(), taskAction);
dbi.withHandle(
retryingHandle(
new HandleCallback<Integer>()
{
@Override
......@@ -376,7 +387,7 @@ public class DbTaskStorage implements TaskStorage
@Override
public List<TaskAction> getAuditLogs(final String taskid)
{
return dbi.withHandle(
return retryingHandle(
new HandleCallback<List<TaskAction>>()
{
@Override
......@@ -392,21 +403,18 @@ public class DbTaskStorage implements TaskStorage
.bind("task_id", taskid)
.list();
return Lists.transform(
dbTaskLogs, new Function<Map<String, Object>, TaskAction>()
{
@Override
public TaskAction apply(Map<String, Object> row)
{
try {
return jsonMapper.readValue((byte[]) row.get("log_payload"), TaskAction.class);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
final List<TaskAction> retList = Lists.newArrayList();
for (final Map<String, Object> dbTaskLog : dbTaskLogs) {
try {
retList.add(jsonMapper.readValue((byte[]) dbTaskLog.get("log_payload"), TaskAction.class));
} catch (Exception e) {
log.makeAlert(e, "Failed to deserialize TaskLog")
.addData("task", taskid)
.addData("logPayload", dbTaskLog)
.emit();
}
}
);
return retList;
}
}
);
......@@ -414,7 +422,7 @@ public class DbTaskStorage implements TaskStorage
private Map<Long, TaskLock> getLocksWithIds(final String taskid)
{
return dbi.withHandle(
return retryingHandle(
new HandleCallback<Map<Long, TaskLock>>()
{
@Override
......@@ -439,4 +447,45 @@ public class DbTaskStorage implements TaskStorage
}
);
}
/**
* Retry SQL operations
*/
private <T> T retryingHandle(final HandleCallback<T> callback) {
final Callable<T> call = new Callable<T>()
{
@Override
public T call() throws Exception
{
return dbi.withHandle(callback);
}
};
final Predicate<Throwable> shouldRetry = new Predicate<Throwable>()
{
@Override
public boolean apply(Throwable e)
{
return shouldRetryException(e);
}
};
final int maxTries = 10;
try {
return RetryUtils.retry(call, shouldRetry, maxTries);
} catch (RuntimeException e) {
throw Throwables.propagate(e);
} catch (Exception e) {
throw new CallbackFailedException(e);
}
}
private static boolean shouldRetryException(final Throwable e)
{
return e != null && (e instanceof SQLTransientException
|| e instanceof MySQLTransientException
|| e instanceof SQLRecoverableException
|| e instanceof UnableToObtainConnectionException
|| (e instanceof SQLException && ((SQLException) e).getErrorCode() == 1317)
|| (e instanceof SQLException && shouldRetryException(e.getCause()))
|| (e instanceof DBIException && shouldRetryException(e.getCause())));
}
}
......@@ -101,12 +101,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
this.exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(workerConfig.getCapacity()));
}
@Override
public void bootstrap(List<Task> tasks)
{
// do nothing
}
@Override
public ListenableFuture<TaskStatus> run(final Task task)
{
......@@ -115,7 +109,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
tasks.put(
task.getId(),
new ForkingTaskRunnerWorkItem(
task,
task.getId(),
exec.submit(
new Callable<TaskStatus>()
{
......@@ -358,6 +352,14 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
}
@Override
public Collection<TaskRunnerWorkItem> getKnownTasks()
{
synchronized (tasks) {
return Lists.<TaskRunnerWorkItem>newArrayList(tasks.values());
}
}
@Override
public Collection<ZkWorker> getWorkers()
{
......@@ -425,11 +427,11 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
private volatile ProcessHolder processHolder = null;
private ForkingTaskRunnerWorkItem(
Task task,
String taskId,
ListenableFuture<TaskStatus> statusFuture
)
{
super(task, statusFuture);
super(taskId, statusFuture);
}
}
......
......@@ -27,11 +27,13 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Stopwatch;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.io.InputSupplier;
import com.google.common.primitives.Ints;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import com.metamx.common.ISE;
......@@ -51,8 +53,8 @@ import io.druid.indexing.worker.TaskAnnouncement;
import io.druid.indexing.worker.Worker;
import io.druid.server.initialization.ZkPathsConfig;
import io.druid.tasklogs.TaskLogStreamer;
import org.apache.commons.lang.mutable.MutableInt;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
......@@ -72,6 +74,7 @@ import java.util.Map;
import java.util.TreeSet;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
......@@ -108,11 +111,15 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
private final HttpClient httpClient;
// all workers that exist in ZK
private final Map<String, ZkWorker> zkWorkers = new ConcurrentHashMap<String, ZkWorker>();
private final ConcurrentMap<String, ZkWorker> zkWorkers = new ConcurrentHashMap<>();
// payloads of pending tasks, which we remember just long enough to assign to workers
private final ConcurrentMap<String, Task> pendingTaskPayloads = new ConcurrentHashMap<>();
// tasks that have not yet been assigned to a worker
private final RemoteTaskRunnerWorkQueue pendingTasks = new RemoteTaskRunnerWorkQueue();
// all tasks that have been assigned to a worker
private final RemoteTaskRunnerWorkQueue runningTasks = new RemoteTaskRunnerWorkQueue();
// tasks that have not yet run
private final RemoteTaskRunnerWorkQueue pendingTasks = new RemoteTaskRunnerWorkQueue();
// tasks that are complete but not cleaned up yet
private final RemoteTaskRunnerWorkQueue completeTasks = new RemoteTaskRunnerWorkQueue();
private final ExecutorService runPendingTasksExec = Executors.newSingleThreadExecutor();
......@@ -148,6 +155,9 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
return;
}
final MutableInt waitingFor = new MutableInt(1);
final Object waitingForMonitor = new Object();
// Add listener for creation/deletion of workers
workerPathCache.getListenable().addListener(
new PathChildrenCacheListener()
......@@ -162,7 +172,32 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
event.getData().getData(),
Worker.class
);
addWorker(worker, PathChildrenCache.StartMode.NORMAL);
synchronized (waitingForMonitor) {
waitingFor.increment();
}
Futures.addCallback(
addWorker(worker),
new FutureCallback<ZkWorker>()
{
@Override
public void onSuccess(ZkWorker zkWorker)
{
synchronized (waitingForMonitor) {
waitingFor.decrement();
waitingForMonitor.notifyAll();
}
}
@Override
public void onFailure(Throwable throwable)
{
synchronized (waitingForMonitor) {
waitingFor.decrement();
waitingForMonitor.notifyAll();
}
}
}
);
break;
case CHILD_REMOVED:
worker = jsonMapper.readValue(
......@@ -171,22 +206,23 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
);
removeWorker(worker);
break;
case INITIALIZED:
synchronized (waitingForMonitor) {
waitingFor.decrement();
waitingForMonitor.notifyAll();
}
default:
break;
}
}
}
);
workerPathCache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE);
for (ChildData childData : workerPathCache.getCurrentData()) {
final Worker worker = jsonMapper.readValue(
childData.getData(),
Worker.class
);
addWorker(worker, PathChildrenCache.StartMode.BUILD_INITIAL_CACHE);
workerPathCache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT);
synchronized (waitingForMonitor) {
while (waitingFor.intValue() > 0) {
waitingForMonitor.wait();
}
}
started = true;
}
catch (Exception e) {
......@@ -201,7 +237,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
if (!started) {
return;
}
started = false;
for (ZkWorker zkWorker : zkWorkers.values()) {
zkWorker.close();
}
......@@ -210,9 +246,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
catch (Exception e) {
throw Throwables.propagate(e);
}
finally {
started = false;
}
}
@Override
......@@ -233,6 +266,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
return pendingTasks.values();
}
@Override
public Collection<RemoteTaskRunnerWorkItem> getKnownTasks()
{
// Racey, since there is a period of time during assignment when a task is neither pending nor running
return Lists.newArrayList(Iterables.concat(pendingTasks.values(), runningTasks.values(), completeTasks.values()));
}
public ZkWorker findWorkerRunningTask(String taskId)
{
for (ZkWorker zkWorker : zkWorkers.values()) {
......@@ -243,46 +283,10 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
return null;
}
public boolean isWorkerRunningTask(Worker worker, Task task)
public boolean isWorkerRunningTask(Worker worker, String taskId)
{
ZkWorker zkWorker = zkWorkers.get(worker.getHost());
return (zkWorker != null && zkWorker.isRunningTask(task.getId()));
}
@Override
public void bootstrap(List<Task> tasks)
{
try {
if (!started) {
throw new ISE("Must start RTR first before calling bootstrap!");
}
Map<String, Worker> existingTasks = Maps.newHashMap();
for (ZkWorker zkWorker : zkWorkers.values()) {
for (String runningTask : zkWorker.getRunningTasks().keySet()) {
existingTasks.put(runningTask, zkWorker.getWorker());
}
}
for (Task task : tasks) {
Worker worker = existingTasks.get(task.getId());
if (worker != null) {
log.info("Bootstrap found [%s] running on [%s].", task.getId(), worker.getHost());
runningTasks.put(
task.getId(),
new RemoteTaskRunnerWorkItem(
task,
SettableFuture.<TaskStatus>create(),
worker
)
);
}
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
return (zkWorker != null && zkWorker.isRunningTask(taskId));
}
/**
......@@ -293,8 +297,11 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
@Override
public ListenableFuture<TaskStatus> run(final Task task)
{
RemoteTaskRunnerWorkItem runningTask = runningTasks.get(task.getId());
if (runningTask != null) {
final RemoteTaskRunnerWorkItem completeTask, runningTask, pendingTask;
if ((pendingTask = pendingTasks.get(task.getId())) != null) {
log.info("Assigned a task[%s] that is already pending, not doing anything", task.getId());
return pendingTask.getResult();
} else if ((runningTask = runningTasks.get(task.getId())) != null) {
ZkWorker zkWorker = findWorkerRunningTask(task.getId());
if (zkWorker == null) {
log.warn("Told to run task[%s], but no worker has started running it yet.", task.getId());
......@@ -302,26 +309,15 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
log.info("Task[%s] already running on %s.", task.getId(), zkWorker.getWorker().getHost());
TaskAnnouncement announcement = zkWorker.getRunningTasks().get(task.getId());
if (announcement.getTaskStatus().isComplete()) {
taskComplete(runningTask, zkWorker, task.getId(), announcement.getTaskStatus());
taskComplete(runningTask, zkWorker, announcement.getTaskStatus());
}
}
return runningTask.getResult();
} else if ((completeTask = completeTasks.get(task.getId())) != null) {
return completeTask.getResult();
} else {
return addPendingTask(task).getResult();
}
RemoteTaskRunnerWorkItem pendingTask = pendingTasks.get(task.getId());
if (pendingTask != null) {
log.info("Assigned a task[%s] that is already pending, not doing anything", task.getId());
return pendingTask.getResult();
}
RemoteTaskRunnerWorkItem taskRunnerWorkItem = new RemoteTaskRunnerWorkItem(
task,
SettableFuture.<TaskStatus>create(),
null
);
addPendingTask(taskRunnerWorkItem);
return taskRunnerWorkItem.getResult();
}
/**
......@@ -330,39 +326,43 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
* @param taskId - task id to shutdown
*/
@Override
public void shutdown(String taskId)
public void shutdown(final String taskId)
{
if (pendingTasks.containsKey(taskId)) {
pendingTasks.remove(taskId);
return;
}
final ZkWorker zkWorker = findWorkerRunningTask(taskId);
if (!started) {
log.info("This TaskRunner is stopped. Ignoring shutdown command for task: %s", taskId);
} else if (pendingTasks.remove(taskId) != null) {
pendingTaskPayloads.remove(taskId);
log.info("Removed task from pending queue: %s", taskId);
} else if (completeTasks.containsKey(taskId)) {
cleanup(completeTasks.get(taskId).getWorker().getHost(), taskId);
} else {
final ZkWorker zkWorker = findWorkerRunningTask(taskId);
if (zkWorker == null) {
log.info("Can't shutdown! No worker running task %s", taskId);
return;
}
if (zkWorker == null) {
log.info("Can't shutdown! No worker running task %s", taskId);
return;
}
try {
final URL url = makeWorkerURL(zkWorker.getWorker(), String.format("/task/%s/shutdown", taskId));
final StatusResponseHolder response = httpClient.post(url)
.go(RESPONSE_HANDLER)
.get();
log.info(
"Sent shutdown message to worker: %s, status %s, response: %s",
zkWorker.getWorker().getHost(),
response.getStatus(),
response.getContent()
);
try {
final URL url = makeWorkerURL(zkWorker.getWorker(), String.format("/task/%s/shutdown", taskId));
final StatusResponseHolder response = httpClient.post(url)
.go(RESPONSE_HANDLER)
.get();
log.info(
"Sent shutdown message to worker: %s, status %s, response: %s",
zkWorker.getWorker().getHost(),
response.getStatus(),
response.getContent()
);
if (!response.getStatus().equals(HttpResponseStatus.ACCEPTED)) {
log.error("Shutdown failed for %s! Are you sure the task was running?", taskId);
if (!response.getStatus().equals(HttpResponseStatus.ACCEPTED)) {
log.error("Shutdown failed for %s! Are you sure the task was running?", taskId);
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
......@@ -417,12 +417,18 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
/**
* Adds a task to the pending queue
*/
private void addPendingTask(final RemoteTaskRunnerWorkItem taskRunnerWorkItem)
private RemoteTaskRunnerWorkItem addPendingTask(final Task task)
{
log.info("Added pending task %s", taskRunnerWorkItem.getTask().getId());
pendingTasks.put(taskRunnerWorkItem.getTask().getId(), taskRunnerWorkItem);
log.info("Added pending task %s", task.getId());
final RemoteTaskRunnerWorkItem taskRunnerWorkItem = new RemoteTaskRunnerWorkItem(
task.getId(),
SettableFuture.<TaskStatus>create(),
null
);
pendingTaskPayloads.put(task.getId(), task);
pendingTasks.put(task.getId(), taskRunnerWorkItem);
runPendingTasks();
return taskRunnerWorkItem;
}
/**
......@@ -439,11 +445,14 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
public Void call() throws Exception
{
try {
// make a copy of the pending tasks because assignTask may delete tasks from pending and move them
// make a copy of the pending tasks because tryAssignTask may delete tasks from pending and move them
// into running status
List<RemoteTaskRunnerWorkItem> copy = Lists.newArrayList(pendingTasks.values());
for (RemoteTaskRunnerWorkItem taskWrapper : copy) {
assignTask(taskWrapper);
for (RemoteTaskRunnerWorkItem taskRunnerWorkItem : copy) {
String taskId = taskRunnerWorkItem.getTaskId();
if (tryAssignTask(pendingTaskPayloads.get(taskId), taskRunnerWorkItem)) {
pendingTaskPayloads.remove(taskId);
}
}
}
catch (Exception e) {
......@@ -457,21 +466,30 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
}
/**
* Removes a task from the running queue and clears out the ZK status path of the task.
* Removes a task from the complete queue and clears out the ZK status path of the task.
*
* @param workerId - the worker that was previously running the task
* @param taskId - the task to cleanup
*/
private void cleanup(final String workerId, final String taskId)
{
log.info("Cleaning up [%s]", taskId);
runningTasks.remove(taskId);
final String statusPath = JOINER.join(zkPaths.getIndexerStatusPath(), workerId, taskId);
try {
cf.delete().guaranteed().forPath(statusPath);
if (!started) {
return;
}
catch (Exception e) {
log.info("Tried to delete status path[%s] that didn't exist! Must've gone away already?", statusPath);
if (completeTasks.remove(taskId) == null) {
log.makeAlert("WTF?! Asked to cleanup nonexistent task")
.addData("workerId", workerId)
.addData("taskId", taskId)
.emit();
} else {
log.info("Cleaning up task[%s] on worker[%s]", taskId, workerId);
final String statusPath = JOINER.join(zkPaths.getIndexerStatusPath(), workerId, taskId);
try {
cf.delete().guaranteed().forPath(statusPath);
}
catch (Exception e) {
log.info("Tried to delete status path[%s] that didn't exist! Must've gone away already?", statusPath);
}
}
}
......@@ -481,26 +499,34 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
* needs to bootstrap after a restart.
*
* @param taskRunnerWorkItem - the task to assign
* @return true iff the task is now assigned
*/
private void assignTask(RemoteTaskRunnerWorkItem taskRunnerWorkItem)
private boolean tryAssignTask(final Task task, final RemoteTaskRunnerWorkItem taskRunnerWorkItem)
{
try {
final String taskId = taskRunnerWorkItem.getTask().getId();
Preconditions.checkNotNull(task, "task");
Preconditions.checkNotNull(taskRunnerWorkItem, "taskRunnerWorkItem");
Preconditions.checkArgument(task.getId().equals(taskRunnerWorkItem.getTaskId()), "task id != workItem id");
if (runningTasks.containsKey(taskId) || findWorkerRunningTask(taskId) != null) {
log.info("Task[%s] already running.", taskId);
if (runningTasks.containsKey(task.getId()) || findWorkerRunningTask(task.getId()) != null) {
log.info("Task[%s] already running.", task.getId());
return true;
} else {
// Nothing running this task, announce it in ZK for a worker to run it
ZkWorker zkWorker = findWorkerForTask(taskRunnerWorkItem.getTask());
ZkWorker zkWorker = findWorkerForTask(task);
if (zkWorker != null) {
announceTask(zkWorker, taskRunnerWorkItem);
announceTask(task, zkWorker, taskRunnerWorkItem);
return true;
} else {
return false;
}
}
}
catch (Exception e) {
log.makeAlert(e, "Exception while trying to run task")
.addData("taskId", taskRunnerWorkItem.getTask().getId())
.addData("taskId", taskRunnerWorkItem.getTaskId())
.emit();
return false;
}
}
......@@ -511,9 +537,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
* @param theZkWorker The worker the task is assigned to
* @param taskRunnerWorkItem The task to be assigned
*/
private void announceTask(ZkWorker theZkWorker, RemoteTaskRunnerWorkItem taskRunnerWorkItem) throws Exception
private void announceTask(
final Task task,
final ZkWorker theZkWorker,
final RemoteTaskRunnerWorkItem taskRunnerWorkItem
) throws Exception
{
final Task task = taskRunnerWorkItem.getTask();
Preconditions.checkArgument(task.getId().equals(taskRunnerWorkItem.getTaskId()), "task id != workItem id");
final Worker theWorker = theZkWorker.getWorker();
log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId());
......@@ -550,7 +580,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
Stopwatch timeoutStopwatch = new Stopwatch();
timeoutStopwatch.start();
synchronized (statusLock) {
while (!isWorkerRunningTask(theWorker, task)) {
while (!isWorkerRunningTask(theWorker, task.getId())) {
final long waitMs = config.getTaskAssignmentTimeout().toStandardDuration().getMillis();
statusLock.wait(waitMs);
long elapsed = timeoutStopwatch.elapsed(TimeUnit.MILLISECONDS);
......@@ -563,7 +593,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
config.getTaskAssignmentTimeout()
);
taskComplete(taskRunnerWorkItem, theZkWorker, task.getId(), TaskStatus.failure(task.getId()));
taskComplete(taskRunnerWorkItem, theZkWorker, TaskStatus.failure(task.getId()));
break;
}
}
......@@ -575,15 +605,17 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
* the worker. Status changes indicate the creation or completion of a task.
* The RemoteTaskRunner updates state according to these changes.
*
* @param worker - contains metadata for a worker that has appeared in ZK
* @param worker contains metadata for a worker that has appeared in ZK
* @return future that will contain a fully initialized worker
*/
private ZkWorker addWorker(final Worker worker, PathChildrenCache.StartMode startMode)
private ListenableFuture<ZkWorker> addWorker(final Worker worker)
{
log.info("Worker[%s] reportin' for duty!", worker.getHost());
try {
final String workerStatusPath = JOINER.join(zkPaths.getIndexerStatusPath(), worker.getHost());
final PathChildrenCache statusCache = pathChildrenCacheFactory.make(cf, workerStatusPath);
final SettableFuture<ZkWorker> retVal = SettableFuture.create();
final ZkWorker zkWorker = new ZkWorker(
worker,
statusCache,
......@@ -597,8 +629,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
@Override
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
{
String taskId;
RemoteTaskRunnerWorkItem taskRunnerWorkItem;
final String taskId;
final RemoteTaskRunnerWorkItem taskRunnerWorkItem;
synchronized (statusLock) {
try {
switch (event.getType()) {
......@@ -617,19 +649,27 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
);
// Synchronizing state with ZK
statusLock.notify();
statusLock.notifyAll();
taskRunnerWorkItem = runningTasks.get(taskId);
if (taskRunnerWorkItem == null) {
final RemoteTaskRunnerWorkItem tmp;
if ((tmp = runningTasks.get(taskId)) != null) {
taskRunnerWorkItem = tmp;
} else {
log.warn(
"WTF?! Worker[%s] announcing a status for a task I didn't know about: %s",
"Worker[%s] announced a status for a task I didn't know about, adding to runningTasks: %s",
zkWorker.getWorker().getHost(),
taskId
);
taskRunnerWorkItem = new RemoteTaskRunnerWorkItem(
taskId,
SettableFuture.<TaskStatus>create(),
zkWorker.getWorker()
);
runningTasks.put(taskId, taskRunnerWorkItem);
}
if (taskStatus.isComplete()) {
taskComplete(taskRunnerWorkItem, zkWorker, taskId, taskStatus);
taskComplete(taskRunnerWorkItem, zkWorker, taskStatus);
runPendingTasks();
}
break;
......@@ -638,11 +678,26 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
taskRunnerWorkItem = runningTasks.remove(taskId);
if (taskRunnerWorkItem != null) {
log.info("Task[%s] just disappeared!", taskId);
taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId()));
taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTaskId()));
} else {
log.info("Task[%s] went bye bye.", taskId);
}
break;
case INITIALIZED:
if (zkWorkers.putIfAbsent(worker.getHost(), zkWorker) == null) {
retVal.set(zkWorker);
} else {
final String message = String.format(
"WTF?! Tried to add already-existing worker[%s]",
worker.getHost()
);
log.makeAlert(message)
.addData("workerHost", worker.getHost())
.addData("workerIp", worker.getIp())
.emit();
retVal.setException(new IllegalStateException(message));
}
runPendingTasks();
}
}
catch (Exception e) {
......@@ -655,13 +710,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
}
}
);
zkWorker.start(startMode);
zkWorkers.put(worker.getHost(), zkWorker);
runPendingTasks();
return zkWorker;
zkWorker.start();
return retVal;
}
catch (Exception e) {
throw Throwables.propagate(e);
......@@ -707,7 +757,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
}
log.info("Failing task[%s]", assignedTask);
taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId()));
taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTaskId()));
} else {
log.warn("RemoteTaskRunner has no knowledge of task[%s]", assignedTask);
}
......@@ -763,19 +813,27 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
private void taskComplete(
RemoteTaskRunnerWorkItem taskRunnerWorkItem,
ZkWorker zkWorker,
String taskId,
TaskStatus taskStatus
)
{
if (taskRunnerWorkItem != null) {
final ListenableFuture<TaskStatus> result = taskRunnerWorkItem.getResult();
if (result != null) {
((SettableFuture<TaskStatus>) result).set(taskStatus);
}
}
Preconditions.checkNotNull(taskRunnerWorkItem, "taskRunnerWorkItem");
Preconditions.checkNotNull(zkWorker, "zkWorker");
Preconditions.checkNotNull(taskStatus, "taskStatus");
log.info(
"Worker[%s] completed task[%s] with status[%s]",
zkWorker.getWorker().getHost(),
taskStatus.getId(),
taskStatus.getStatusCode()
);
// Worker is done with this task
zkWorker.setLastCompletedTaskTime(new DateTime());
cleanup(zkWorker.getWorker().getHost(), taskId);
// Move from running -> complete
completeTasks.put(taskStatus.getId(), taskRunnerWorkItem);
runningTasks.remove(taskStatus.getId());
// Notify interested parties
final ListenableFuture<TaskStatus> result = taskRunnerWorkItem.getResult();
if (result != null) {
((SettableFuture<TaskStatus>) result).set(taskStatus);
}
}
}
......@@ -21,7 +21,6 @@ package io.druid.indexing.overlord;
import com.google.common.util.concurrent.SettableFuture;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.worker.Worker;
import org.joda.time.DateTime;
......@@ -33,25 +32,25 @@ public class RemoteTaskRunnerWorkItem extends TaskRunnerWorkItem
private final Worker worker;
public RemoteTaskRunnerWorkItem(
Task task,
String taskId,
SettableFuture<TaskStatus> result,
Worker worker
)
{
super(task, result);
super(taskId, result);
this.result = result;
this.worker = worker;
}
public RemoteTaskRunnerWorkItem(
Task task,
String taskId,
SettableFuture<TaskStatus> result,
DateTime createdTime,
DateTime queueInsertionTime,
Worker worker
)
{
super(task, result, createdTime, queueInsertionTime);
super(taskId, result, createdTime, queueInsertionTime);
this.result = result;
this.worker = worker;
}
......@@ -69,11 +68,11 @@ public class RemoteTaskRunnerWorkItem extends TaskRunnerWorkItem
@Override
public RemoteTaskRunnerWorkItem withQueueInsertionTime(DateTime time)
{
return new RemoteTaskRunnerWorkItem(getTask(), result, getCreatedTime(), time, worker);
return new RemoteTaskRunnerWorkItem(getTaskId(), result, getCreatedTime(), time, worker);
}
public RemoteTaskRunnerWorkItem withWorker(Worker theWorker)
{
return new RemoteTaskRunnerWorkItem(getTask(), result, getCreatedTime(), getQueueInsertionTime(), theWorker);
return new RemoteTaskRunnerWorkItem(getTaskId(), result, getCreatedTime(), getQueueInsertionTime(), theWorker);
}
}
......@@ -23,13 +23,15 @@ import com.google.common.base.Function;
import com.google.common.base.Objects;
import com.google.common.base.Optional;
import com.google.common.base.Predicate;
import com.google.common.collect.ComparisonChain;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.inject.Inject;
import com.metamx.common.IAE;
import com.metamx.common.Pair;
import com.metamx.common.guava.Comparators;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.emitter.EmittingLogger;
......@@ -73,8 +75,86 @@ public class TaskLockbox
}
/**
* Locks a task without removing it from the queue. Blocks until the lock is acquired. Throws an exception
* if the lock cannot be acquired.
* Wipe out our current in-memory state and resync it from our bundled {@link io.druid.indexing.overlord.TaskStorage}.
*/
public void syncFromStorage()
{
giant.lock();
try {
// Load stuff from taskStorage first. If this fails, we don't want to lose all our locks.
final List<Pair<Task, TaskLock>> storedLocks = Lists.newArrayList();
for (final Task task : taskStorage.getActiveTasks()) {
for (final TaskLock taskLock : taskStorage.getLocks(task.getId())) {
storedLocks.add(Pair.of(task, taskLock));
}
}
// Sort locks by version, so we add them back in the order they were acquired.
final Ordering<Pair<Task, TaskLock>> byVersionOrdering = new Ordering<Pair<Task, TaskLock>>()
{
@Override
public int compare(Pair<Task, TaskLock> left, Pair<Task, TaskLock> right)
{
// The second compare shouldn't be necessary, but, whatever.
return ComparisonChain.start()
.compare(left.rhs.getVersion(), right.rhs.getVersion())
.compare(left.lhs.getId(), right.lhs.getId())
.result();
}
};
running.clear();
// Bookkeeping for a log message at the end
final Set<String> uniqueTaskIds = Sets.newHashSet();
int taskLockCount = 0;
for (final Pair<Task, TaskLock> taskAndLock : byVersionOrdering.sortedCopy(storedLocks)) {
final Task task = taskAndLock.lhs;
final TaskLock savedTaskLock = taskAndLock.rhs;
uniqueTaskIds.add(task.getId());
final Optional<TaskLock> acquiredTaskLock = tryLock(
task,
savedTaskLock.getInterval(),
Optional.of(savedTaskLock.getVersion())
);
if (acquiredTaskLock.isPresent() && savedTaskLock.getVersion().equals(acquiredTaskLock.get().getVersion())) {
taskLockCount ++;
log.info(
"Reacquired lock on interval[%s] version[%s] for task: %s",
savedTaskLock.getInterval(),
savedTaskLock.getVersion(),
task.getId()
);
} else if (acquiredTaskLock.isPresent()) {
taskLockCount ++;
log.info(
"Could not reacquire lock on interval[%s] version[%s] (got version[%s] instead) for task: %s",
savedTaskLock.getInterval(),
savedTaskLock.getVersion(),
acquiredTaskLock.get().getVersion(),
task.getId()
);
} else {
log.info(
"Could not reacquire lock on interval[%s] version[%s] for task: %s",
savedTaskLock.getInterval(),
savedTaskLock.getVersion(),
task.getId()
);
}
}
log.info(
"Synced %,d locks for %,d tasks from storage (%,d locks ignored).",
taskLockCount,
uniqueTaskIds.size(),
storedLocks.size() - taskLockCount
);
} finally {
giant.unlock();
}
}
/**
* Acquires a lock on behalf of a task. Blocks until the lock is acquired. Throws an exception if the lock
* cannot be acquired.
*/
public TaskLock lock(final Task task, final Interval interval) throws InterruptedException
{
......@@ -97,7 +177,8 @@ public class TaskLockbox
* Attempt to lock a task, without removing it from the queue. Equivalent to the long form of {@code tryLock}
* with no preferred version.
*
* @param task task to attempt to lock
* @param task task that wants a lock
* @param interval interval to lock
*
* @return lock version if lock was acquired, absent otherwise
*/
......@@ -113,22 +194,17 @@ public class TaskLockbox
* is only mostly guaranteed, however; we assume clock monotonicity and we assume that callers specifying
* {@code preferredVersion} are doing the right thing.
*
* @param task task to attempt to lock
* @param task task that wants a lock
* @param interval interval to lock
* @param preferredVersion use this version string if one has not yet been assigned
*
* @return lock version if lock was acquired, absent otherwise
*/
public Optional<TaskLock> tryLock(final Task task, final Interval interval, final Optional<String> preferredVersion)
private Optional<TaskLock> tryLock(final Task task, final Interval interval, final Optional<String> preferredVersion)
{
giant.lock();
try {
if(task.getImplicitLockInterval().isPresent() && !task.getImplicitLockInterval().get().equals(interval)) {
// Task may only lock its fixed interval, if present
throw new IAE("Task must lock its fixed interval: %s", task.getId());
}
final String dataSource = task.getDataSource();
final List<TaskLockPosse> foundPosses = findLockPossesForInterval(dataSource, interval);
final TaskLockPosse posseToUse;
......@@ -184,9 +260,10 @@ public class TaskLockbox
if (posseToUse.getTaskIds().add(task.getId())) {
log.info("Added task[%s] to TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId());
// Best effort to update task storage facility
// Update task storage facility. If it fails, revoke the lock.
try {
taskStorage.addLock(task.getId(), posseToUse.getTaskLock());
return Optional.of(posseToUse.getTaskLock());
} catch(Exception e) {
log.makeAlert("Failed to persist lock in storage")
.addData("task", task.getId())
......@@ -194,12 +271,13 @@ public class TaskLockbox
.addData("interval", posseToUse.getTaskLock().getInterval())
.addData("version", posseToUse.getTaskLock().getVersion())
.emit();
unlock(task, interval);
return Optional.absent();
}
} else {
log.info("Task[%s] already present in TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId());
return Optional.of(posseToUse.getTaskLock());
}
return Optional.of(posseToUse.getTaskLock());
}
finally {
giant.unlock();
......@@ -271,7 +349,7 @@ public class TaskLockbox
// Wake up blocking-lock waiters
lockReleaseCondition.signalAll();
// Best effort to remove lock from storage
// Remove lock from storage. If it cannot be removed, just ignore the failure.
try {
taskStorage.removeLock(task.getId(), taskLock);
} catch(Exception e) {
......@@ -315,20 +393,6 @@ public class TaskLockbox
}
}
/**
* Removes all locks from this lockbox.
*/
public void clear()
{
giant.lock();
try {
running.clear();
} finally {
giant.unlock();
}
}
/**
* Return the currently-active lock posses for some task.
*
......@@ -341,17 +405,12 @@ public class TaskLockbox
try {
final Iterable<TaskLockPosse> searchSpace;
if (task.getImplicitLockInterval().isPresent()) {
// Narrow down search using findLockPossesForInterval
searchSpace = findLockPossesForInterval(task.getDataSource(), task.getImplicitLockInterval().get());
// Scan through all locks for this datasource
final NavigableMap<Interval, TaskLockPosse> dsRunning = running.get(task.getDataSource());
if(dsRunning == null) {
searchSpace = ImmutableList.of();
} else {
// Scan through all locks for this datasource
final NavigableMap<Interval, TaskLockPosse> dsRunning = running.get(task.getDataSource());
if(dsRunning == null) {
searchSpace = ImmutableList.of();
} else {
searchSpace = dsRunning.values();
}
searchSpace = dsRunning.values();
}
return ImmutableList.copyOf(
......
......@@ -34,7 +34,7 @@ import io.druid.guice.annotations.Self;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.exec.TaskConsumer;
import io.druid.indexing.overlord.config.TaskQueueConfig;
import io.druid.indexing.overlord.scaling.ResourceManagementScheduler;
import io.druid.indexing.overlord.scaling.ResourceManagementSchedulerFactory;
import io.druid.server.DruidNode;
......@@ -56,20 +56,22 @@ public class TaskMaster
private final LeaderSelector leaderSelector;
private final ReentrantLock giant = new ReentrantLock();
private final Condition mayBeStopped = giant.newCondition();
private final TaskQueue taskQueue;
private final TaskActionClientFactory taskActionClientFactory;
private final AtomicReference<Lifecycle> leaderLifecycleRef = new AtomicReference<Lifecycle>(null);
private final AtomicReference<Lifecycle> leaderLifecycleRef = new AtomicReference<>(null);
private volatile boolean leading = false;
private volatile TaskRunner taskRunner;
private volatile TaskQueue taskQueue;
private volatile ResourceManagementScheduler resourceManagementScheduler;
private static final EmittingLogger log = new EmittingLogger(TaskMaster.class);
@Inject
public TaskMaster(
final TaskQueue taskQueue,
final TaskQueueConfig taskQueueConfig,
final TaskLockbox taskLockbox,
final TaskStorage taskStorage,
final TaskActionClientFactory taskActionClientFactory,
@Self final DruidNode node,
final ZkPathsConfig zkPaths,
......@@ -80,118 +82,99 @@ public class TaskMaster
final ServiceEmitter emitter
)
{
this.taskQueue = taskQueue;
this.taskActionClientFactory = taskActionClientFactory;
this.leaderSelector = new LeaderSelector(
curator, zkPaths.getIndexerLeaderLatchPath(), new LeaderSelectorListener()
{
@Override
public void takeLeadership(CuratorFramework client) throws Exception
{
giant.lock();
try {
log.info("By the power of Grayskull, I have the power!");
taskRunner = runnerFactory.build();
final TaskConsumer taskConsumer = new TaskConsumer(
taskQueue,
taskRunner,
taskActionClientFactory,
emitter
);
// Bootstrap task queue and task lockbox (load state stuff from the database)
taskQueue.bootstrap();
// Sensible order to start stuff:
final Lifecycle leaderLifecycle = new Lifecycle();
if (leaderLifecycleRef.getAndSet(leaderLifecycle) != null) {
log.makeAlert("TaskMaster set a new Lifecycle without the old one being cleared! Race condition")
.emit();
}
leaderLifecycle.addManagedInstance(taskRunner);
leaderLifecycle.addHandler(
new Lifecycle.Handler()
{
@Override
public void start() throws Exception
{
taskRunner.bootstrap(taskQueue.snapshot());
}
@Override
public void stop()
{
}
curator,
zkPaths.getIndexerLeaderLatchPath(),
new LeaderSelectorListener()
{
@Override
public void takeLeadership(CuratorFramework client) throws Exception
{
giant.lock();
try {
// Make sure the previous leadership cycle is really, really over.
stopLeading();
// I AM THE MASTER OF THE UNIVERSE.
log.info("By the power of Grayskull, I have the power!");
taskLockbox.syncFromStorage();
taskRunner = runnerFactory.build();
taskQueue = new TaskQueue(
taskQueueConfig,
taskStorage,
taskRunner,
taskActionClientFactory,
taskLockbox,
emitter
);
// Sensible order to start stuff:
final Lifecycle leaderLifecycle = new Lifecycle();
if (leaderLifecycleRef.getAndSet(leaderLifecycle) != null) {
log.makeAlert("TaskMaster set a new Lifecycle without the old one being cleared! Race condition")
.emit();
}
);
leaderLifecycle.addManagedInstance(taskQueue);
leaderLifecycle.addHandler(
new Lifecycle.Handler()
{
@Override
public void start() throws Exception
{
serviceAnnouncer.announce(node);
}
@Override
public void stop()
{
serviceAnnouncer.unannounce(node);
leaderLifecycle.addManagedInstance(taskRunner);
if (taskRunner instanceof RemoteTaskRunner) {
final ScheduledExecutorFactory executorFactory = ScheduledExecutors.createFactory(leaderLifecycle);
resourceManagementScheduler = managementSchedulerFactory.build(
(RemoteTaskRunner) taskRunner,
executorFactory
);
leaderLifecycle.addManagedInstance(resourceManagementScheduler);
}
leaderLifecycle.addManagedInstance(taskQueue);
leaderLifecycle.addHandler(
new Lifecycle.Handler()
{
@Override
public void start() throws Exception
{
serviceAnnouncer.announce(node);
}
@Override
public void stop()
{
serviceAnnouncer.unannounce(node);
}
}
);
try {
leaderLifecycle.start();
leading = true;
while (leading && !Thread.currentThread().isInterrupted()) {
mayBeStopped.await();
}
}
);
leaderLifecycle.addManagedInstance(taskConsumer);
if (taskRunner instanceof RemoteTaskRunner) {
final ScheduledExecutorFactory executorFactory = ScheduledExecutors.createFactory(leaderLifecycle);
resourceManagementScheduler = managementSchedulerFactory.build(
(RemoteTaskRunner) taskRunner,
executorFactory
);
leaderLifecycle.addManagedInstance(resourceManagementScheduler);
catch (InterruptedException e) {
// Suppress so we can bow out gracefully
}
finally {
log.info("Bowing out!");
stopLeading();
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to lead").emit();
throw Throwables.propagate(e);
}
finally {
giant.unlock();
}
}
try {
leaderLifecycle.start();
leading = true;
while (leading && !Thread.currentThread().isInterrupted()) {
mayBeStopped.await();
@Override
public void stateChanged(CuratorFramework client, ConnectionState newState)
{
if (newState == ConnectionState.LOST || newState == ConnectionState.SUSPENDED) {
// disconnected from zk. assume leadership is gone
stopLeading();
}
}
catch (InterruptedException e) {
// Suppress so we can bow out gracefully
}
finally {
log.info("Bowing out!");
stopLeading();
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to lead").emit();
throw Throwables.propagate(e);
}
finally {
giant.unlock();
}
}
@Override
public void stateChanged(CuratorFramework client, ConnectionState newState)
{
if (newState == ConnectionState.LOST || newState == ConnectionState.SUSPENDED) {
// disconnected from zk. assume leadership is gone
stopLeading();
}
}
}
);
leaderSelector.setId(node.getHost());
......
......@@ -19,183 +19,187 @@
package io.druid.indexing.overlord;
import com.google.api.client.util.Maps;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.google.common.collect.Multimap;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.inject.Inject;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.emitter.EmittingLogger;
import io.druid.indexing.common.TaskLock;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.config.TaskQueueConfig;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
/**
* Interface between task producers and task consumers.
* Interface between task producers and the task runner.
* <p/>
* The queue accepts tasks from producers using {@link #add} and delivers tasks to consumers using either
* {@link #take} or {@link #poll}. Ordering is mostly-FIFO, with deviations when the natural next task would conflict
* with a currently-running task. In that case, tasks are skipped until a runnable one is found.
* This object accepts tasks from producers using {@link #add} and manages delivery of these tasks to a
* {@link TaskRunner}. Tasks will run in a mostly-FIFO order, with deviations when the natural next task is not ready
* in time (based on its {@link Task#isReady} method).
* <p/>
* To manage locking, the queue keeps track of currently-running tasks as {@link io.druid.indexing.common.TaskLock} objects. The idea is that
* only one TaskLock can be running on a particular dataSource + interval, and that TaskLock has a single version
* string that all tasks in the group must use to publish segments. Tasks in the same TaskLock may run concurrently.
* <p/>
* For persistence, the queue saves new tasks from {@link #add} and task status updates from {@link #notify} using a
* {@link TaskStorage} obj
* <p/>
* To support leader election of our containing system, the queue can be stopped (in which case it will not accept
* any new tasks, or hand out any more tasks, until started again).
* For persistence, we save all new tasks and task status changes using a {@link TaskStorage} object.
*/
public class TaskQueue
{
private final List<Task> queue = Lists.newLinkedList();
private final List<Task> tasks = Lists.newArrayList();
private final Map<String, ListenableFuture<TaskStatus>> taskFutures = Maps.newHashMap();
private final TaskQueueConfig config;
private final TaskStorage taskStorage;
private final TaskRunner taskRunner;
private final TaskActionClientFactory taskActionClientFactory;
private final TaskLockbox taskLockbox;
private final ServiceEmitter emitter;
private final ReentrantLock giant = new ReentrantLock();
private final Condition workMayBeAvailable = giant.newCondition();
private final Condition managementMayBeNecessary = giant.newCondition();
private final ExecutorService managerExec = Executors.newSingleThreadExecutor(
new ThreadFactoryBuilder()
.setDaemon(false)
.setNameFormat("TaskQueue-Manager").build()
);
private final ScheduledExecutorService storageSyncExec = Executors.newSingleThreadScheduledExecutor(
new ThreadFactoryBuilder()
.setDaemon(false)
.setNameFormat("TaskQueue-StorageSync").build()
);
private volatile boolean active = false;
private static final EmittingLogger log = new EmittingLogger(TaskQueue.class);
@Inject
public TaskQueue(TaskStorage taskStorage, TaskLockbox taskLockbox)
public TaskQueue(
TaskQueueConfig config,
TaskStorage taskStorage,
TaskRunner taskRunner,
TaskActionClientFactory taskActionClientFactory,
TaskLockbox taskLockbox,
ServiceEmitter emitter
)
{
this.config = Preconditions.checkNotNull(config, "config");
this.taskStorage = Preconditions.checkNotNull(taskStorage, "taskStorage");
this.taskRunner = Preconditions.checkNotNull(taskRunner, "taskRunner");
this.taskActionClientFactory = Preconditions.checkNotNull(taskActionClientFactory, "taskActionClientFactory");
this.taskLockbox = Preconditions.checkNotNull(taskLockbox, "taskLockbox");
this.emitter = Preconditions.checkNotNull(emitter, "emitter");
}
/**
* Bootstraps this task queue and associated task lockbox. Clears the lockbox before running. Should be called
* while the queue is stopped. It is not a good idea to start the queue if this method fails.
* Starts this task queue. Allows {@link #add(Task)} to accept new tasks.
*/
public void bootstrap()
@LifecycleStart
public void start()
{
// NOTE: Bootstraps can resurrect bogus stuff caused by leader races or whatevs.
// We may want to periodically fixup the database to refer to what we think is happening, to prevent
// this from occurring and also so that bogus stuff is detected by clients in a timely manner.
giant.lock();
try {
Preconditions.checkState(!active, "queue must be stopped");
log.info("Bootstrapping queue (and associated lockbox)");
queue.clear();
taskLockbox.clear();
// Get all running tasks and their locks
final Multimap<TaskLock, Task> tasksByLock = ArrayListMultimap.create();
for (final Task task : taskStorage.getActiveTasks()) {
try {
final List<TaskLock> taskLocks = taskStorage.getLocks(task.getId());
queue.add(task);
for (final TaskLock taskLock : taskLocks) {
tasksByLock.put(taskLock, task);
active = true;
syncFromStorage();
managerExec.submit(
new Runnable()
{
@Override
public void run()
{
while (true) {
try {
manage();
break;
}
catch (InterruptedException e) {
log.info("Interrupted, exiting!");
break;
}
catch (Exception e) {
final long restartDelay = config.getRestartDelay().getMillis();
log.makeAlert(e, "Failed to manage").addData("restartDelay", restartDelay).emit();
try {
Thread.sleep(restartDelay);
}
catch (InterruptedException e2) {
log.info("Interrupted, exiting!");
break;
}
}
}
}
}
}
catch (Exception e) {
log.makeAlert("Failed to bootstrap task").addData("task", task.getId()).emit();
throw Throwables.propagate(e);
}
}
// Sort locks by version
final Ordering<Map.Entry<TaskLock, Task>> byVersionOrdering = new Ordering<Map.Entry<TaskLock, Task>>()
{
@Override
public int compare(Map.Entry<TaskLock, Task> left, Map.Entry<TaskLock, Task> right)
{
return left.getKey().getVersion().compareTo(right.getKey().getVersion());
}
};
// Acquire as many locks as possible, in version order
for(final Map.Entry<TaskLock, Task> taskAndLock : byVersionOrdering.sortedCopy(tasksByLock.entries())) {
final Task task = taskAndLock.getValue();
final TaskLock savedTaskLock = taskAndLock.getKey();
final Optional<TaskLock> acquiredTaskLock = taskLockbox.tryLock(
task,
savedTaskLock.getInterval(),
Optional.of(savedTaskLock.getVersion())
);
if(acquiredTaskLock.isPresent() && savedTaskLock.getVersion().equals(acquiredTaskLock.get().getVersion())) {
log.info(
"Reacquired lock on interval[%s] version[%s] for task: %s",
savedTaskLock.getInterval(),
savedTaskLock.getVersion(),
task.getId()
);
} else if(acquiredTaskLock.isPresent()) {
log.info(
"Could not reacquire lock on interval[%s] version[%s] (got version[%s] instead) for task: %s",
savedTaskLock.getInterval(),
savedTaskLock.getVersion(),
acquiredTaskLock.get().getVersion(),
task.getId()
);
} else {
log.info(
"Could not reacquire lock on interval[%s] version[%s] for task: %s",
savedTaskLock.getInterval(),
savedTaskLock.getVersion(),
task.getId()
);
}
}
log.info("Bootstrapped %,d tasks with %,d locks. Ready to go!", queue.size(), tasksByLock.keySet().size());
} finally {
giant.unlock();
);
ScheduledExecutors.scheduleAtFixedRate(
storageSyncExec,
config.getStorageSyncRate(),
new Callable<ScheduledExecutors.Signal>()
{
@Override
public ScheduledExecutors.Signal call()
{
try {
syncFromStorage();
}
catch (Exception e) {
if (active) {
log.makeAlert(e, "Failed to sync with storage").emit();
}
}
if (active) {
return ScheduledExecutors.Signal.REPEAT;
} else {
return ScheduledExecutors.Signal.STOP;
}
}
}
);
managementMayBeNecessary.signalAll();
}
}
/**
* Returns an immutable snapshot of the current status of this queue.
*/
public List<Task> snapshot()
{
giant.lock();
try {
return ImmutableList.copyOf(queue);
} finally {
finally {
giant.unlock();
}
}
/**
* Starts this task queue. Allows {@link #add(Task)} to accept new tasks. This should not be called on
* an already-started queue.
* Shuts down the queue.
*/
@LifecycleStart
public void start()
@LifecycleStop
public void stop()
{
giant.lock();
try {
Preconditions.checkState(!active, "queue must be stopped");
active = true;
workMayBeAvailable.signalAll();
tasks.clear();
taskFutures.clear();
active = false;
managerExec.shutdownNow();
storageSyncExec.shutdownNow();
managementMayBeNecessary.signalAll();
}
finally {
giant.unlock();
......@@ -203,22 +207,83 @@ public class TaskQueue
}
/**
* Shuts down the queue, for now. This may safely be called on an already-stopped queue. The queue may be restarted
* if desired.
* Main task runner management loop. Meant to run forever, or, at least until we're stopped.
*/
@LifecycleStop
public void stop()
private void manage() throws InterruptedException
{
giant.lock();
log.info("Beginning management in %s.", config.getStartDelay());
Thread.sleep(config.getStartDelay().getMillis());
try {
log.info("Naptime! Shutting down until we are started again.");
queue.clear();
taskLockbox.clear();
active = false;
}
finally {
giant.unlock();
while (active) {
giant.lock();
try {
// Task futures available from the taskRunner
final Map<String, ListenableFuture<TaskStatus>> runnerTaskFutures = Maps.newHashMap();
for (final TaskRunnerWorkItem workItem : taskRunner.getKnownTasks()) {
runnerTaskFutures.put(workItem.getTaskId(), workItem.getResult());
}
// Attain futures for all active tasks (assuming they are ready to run).
for (final Task task : tasks) {
if (!taskFutures.containsKey(task.getId())) {
final ListenableFuture<TaskStatus> runnerTaskFuture;
if (runnerTaskFutures.containsKey(task.getId())) {
runnerTaskFuture = runnerTaskFutures.get(task.getId());
} else {
// Task should be running, so run it.
final boolean taskIsReady;
try {
taskIsReady = task.isReady(taskActionClientFactory.create(task));
}
catch (Exception e) {
log.makeAlert(e, "Exception thrown during isReady").addData("task", task.getId()).emit();
notifyStatus(task, TaskStatus.failure(task.getId()));
continue;
}
if (taskIsReady) {
log.info("Asking taskRunner to run: %s", task.getId());
runnerTaskFuture = taskRunner.run(task);
} else {
continue;
}
}
taskFutures.put(task.getId(), attachCallbacks(task, runnerTaskFuture));
}
}
// Kill tasks that shouldn't be running
final Set<String> tasksToKill = Sets.difference(
runnerTaskFutures.keySet(),
ImmutableSet.copyOf(
Lists.transform(
tasks,
new Function<Task, Object>()
{
@Override
public String apply(Task task)
{
return task.getId();
}
}
)
)
);
if (!tasksToKill.isEmpty()) {
log.info("Asking taskRunner to clean up %,d tasks.", tasksToKill.size());
for (final String taskId : tasksToKill) {
try {
taskRunner.shutdown(taskId);
} catch (Exception e) {
log.warn(e, "TaskRunner failed to clean up task: %s", taskId);
}
}
}
// awaitNanos because management may become necessary without this condition signalling,
// due to e.g. tasks becoming ready when other folks mess with the TaskLockbox.
managementMayBeNecessary.awaitNanos(60000000000L /* 60 seconds */);
}
finally {
giant.unlock();
}
}
}
......@@ -236,26 +301,20 @@ public class TaskQueue
try {
Preconditions.checkState(active, "Queue is not active!");
Preconditions.checkNotNull(task, "task");
Preconditions.checkState(tasks.size() < config.getMaxSize(), "Too many tasks (max = %,d)", config.getMaxSize());
// If this throws with any sort of exception, including TaskExistsException, we don't want to
// insert the task into our queue.
try {
taskStorage.insert(task, TaskStatus.running(task.getId()));
} catch (TaskExistsException e) {
}
catch (TaskExistsException e) {
log.warn("Attempt to add task twice: %s", task.getId());
throw Throwables.propagate(e);
}
queue.add(task);
workMayBeAvailable.signalAll();
// Attempt to add this task to a running task group. Silently continue if this is not possible.
// The main reason this is here is so when subtasks are added, they end up in the same task group
// as their parent whenever possible.
if(task.getImplicitLockInterval().isPresent()) {
taskLockbox.tryLock(task, task.getImplicitLockInterval().get());
}
tasks.add(task);
managementMayBeNecessary.signalAll();
return true;
}
finally {
......@@ -264,62 +323,22 @@ public class TaskQueue
}
/**
* Locks and returns next doable work from the queue. Blocks if there is no doable work.
*
* @return runnable task
* Shuts down a task if it has not yet finished.
* @param taskId task to kill
*/
public Task take() throws InterruptedException
public void shutdown(final String taskId)
{
giant.lock();
try {
Task task;
log.info("Waiting for work...");
while ((task = poll()) == null) {
// awaitNanos because work may become available without this condition signalling,
// due to other folks messing with the taskLockbox
workMayBeAvailable.awaitNanos(1000000000L /* 1 second */);
}
return task;
}
finally {
giant.unlock();
}
}
/**
* Locks and removes next doable work from the queue. Returns null if there is no doable work.
*
* @return runnable task or null
*/
public Task poll()
{
giant.lock();
try {
for (final Task task : queue) {
if(task.getImplicitLockInterval().isPresent()) {
// If this task has a fixed interval, attempt to lock it right now.
final Optional<TaskLock> maybeLock = taskLockbox.tryLock(task, task.getImplicitLockInterval().get());
if(maybeLock.isPresent()) {
log.info("Task claimed with fixed interval lock: %s", task.getId());
queue.remove(task);
return task;
}
} else {
// No fixed interval. Let's just run this and see what happens.
log.info("Task claimed with no fixed interval lock: %s", task.getId());
queue.remove(task);
return task;
Preconditions.checkNotNull(taskId, "taskId");
for (final Task task : tasks) {
if (task.getId().equals(taskId)) {
notifyStatus(task, TaskStatus.failure(taskId));
break;
}
}
return null;
}
finally {
} finally {
giant.unlock();
}
}
......@@ -329,14 +348,14 @@ public class TaskQueue
* the task storage facility. If the status is a completed status, the task will be unlocked and no further
* updates will be accepted.
*
* @param task task to update
* @param task task to update
* @param taskStatus new task status
*
* @throws NullPointerException if task or status is null
* @throws IllegalArgumentException if the task ID does not match the status ID
* @throws IllegalStateException if this queue is currently shut down
*/
public void notify(final Task task, final TaskStatus taskStatus)
private void notifyStatus(final Task task, final TaskStatus taskStatus)
{
giant.lock();
......@@ -350,36 +369,154 @@ public class TaskQueue
task.getId(),
taskStatus.getId()
);
// Save status to DB
boolean didPersistStatus = false;
// Inform taskRunner that this task can be shut down
try {
final Optional<TaskStatus> previousStatus = taskStorage.getStatus(task.getId());
if (!previousStatus.isPresent() || !previousStatus.get().isRunnable()) {
log.makeAlert("Ignoring notification for dead task").addData("task", task.getId()).emit();
return;
} else {
taskStorage.setStatus(taskStatus);
didPersistStatus = true;
taskRunner.shutdown(task.getId());
} catch (Exception e) {
log.warn(e, "TaskRunner failed to cleanup task after completion: %s", task.getId());
}
// Remove from running tasks
int removed = 0;
for (int i = tasks.size() - 1 ; i >= 0 ; i--) {
if (tasks.get(i).getId().equals(task.getId())) {
removed ++;
tasks.remove(i);
break;
}
} catch(Exception e) {
log.makeAlert(e, "Failed to persist status for task")
.addData("task", task.getId())
.addData("statusCode", taskStatus.getStatusCode())
.emit();
}
if (removed == 0) {
log.warn("Unknown task completed: %s", task.getId());
} else if (removed > 1) {
log.makeAlert("Removed multiple copies of task").addData("count", removed).addData("task", task.getId()).emit();
}
// Remove from futures list
taskFutures.remove(task.getId());
if (removed > 0) {
// If we thought this task should be running, save status to DB
try {
final Optional<TaskStatus> previousStatus = taskStorage.getStatus(task.getId());
if (!previousStatus.isPresent() || !previousStatus.get().isRunnable()) {
log.makeAlert("Ignoring notification for already-complete task").addData("task", task.getId()).emit();
} else {
taskStorage.setStatus(taskStatus);
taskLockbox.unlock(task);
log.info("Task done: %s", task);
managementMayBeNecessary.signalAll();
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to persist status for task")
.addData("task", task.getId())
.addData("statusCode", taskStatus.getStatusCode())
.emit();
}
}
}
finally {
giant.unlock();
}
}
if(taskStatus.isComplete()) {
if(didPersistStatus) {
log.info("Task done: %s", task);
taskLockbox.unlock(task);
} else {
log.warn("Status could not be persisted! Reinserting task: %s", task.getId());
queue.add(task);
/**
* Attach success and failure handlers to a task status future, such that when it completes, we perform the
* appropriate updates.
*
* @param statusFuture a task status future
*
* @return the same future, for convenience
*/
private ListenableFuture<TaskStatus> attachCallbacks(final Task task, final ListenableFuture<TaskStatus> statusFuture)
{
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
.setUser2(task.getDataSource())
.setUser4(task.getType());
Futures.addCallback(
statusFuture,
new FutureCallback<TaskStatus>()
{
@Override
public void onSuccess(final TaskStatus status)
{
log.info("Received %s status for task: %s", status.getStatusCode(), status.getId());
handleStatus(status);
}
@Override
public void onFailure(final Throwable t)
{
log.makeAlert(t, "Failed to run task")
.addData("task", task.getId())
.addData("type", task.getType())
.addData("dataSource", task.getDataSource())
.emit();
handleStatus(TaskStatus.failure(task.getId()));
}
private void handleStatus(final TaskStatus status)
{
try {
// If we're not supposed to be running anymore, don't do anything. Somewhat racey if the flag gets set
// after we check and before we commit the database transaction, but better than nothing.
if (!active) {
log.info("Abandoning task due to shutdown: %s", task.getId());
return;
}
notifyStatus(task, status);
// Emit event and log, if the task is done
if (status.isComplete()) {
metricBuilder.setUser3(status.getStatusCode().toString());
emitter.emit(metricBuilder.build("indexer/time/run/millis", status.getDuration()));
log.info(
"Task %s: %s (%d run duration)",
status.getStatusCode(),
task,
status.getDuration()
);
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to handle task status")
.addData("task", task.getId())
.addData("statusCode", status.getStatusCode())
.emit();
}
}
}
workMayBeAvailable.signalAll();
);
return statusFuture;
}
/**
* Resync the contents of this task queue with our storage facility. Useful to make sure our in-memory state
* corresponds to the storage facility even if the latter is manually modified.
*/
private void syncFromStorage()
{
giant.lock();
try {
if (active) {
final List<Task> newTasks = taskStorage.getActiveTasks();
log.info(
"Synced %,d tasks from storage (%,d tasks added, %,d tasks removed).",
newTasks.size(),
Sets.difference(Sets.newHashSet(newTasks), Sets.newHashSet(tasks)).size(),
Sets.difference(Sets.newHashSet(tasks), Sets.newHashSet(newTasks)).size()
);
tasks.clear();
tasks.addAll(newTasks);
managementMayBeNecessary.signalAll();
} else {
log.info("Not active. Skipping storage sync.");
}
}
catch (Exception e) {
log.warn(e, "Failed to sync tasks from storage!");
throw Throwables.propagate(e);
}
finally {
giant.unlock();
}
......
......@@ -24,34 +24,24 @@ import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.task.Task;
import java.util.Collection;
import java.util.List;
/**
* Interface for handing off tasks. Used by a {@link io.druid.indexing.overlord.exec.TaskConsumer} to
* run tasks that have been locked.
* Interface for handing off tasks. Managed by a {@link io.druid.indexing.overlord.TaskQueue}.
*/
public interface TaskRunner
{
/**
* Provide a new task runner with a list of tasks that may already be running. Will be called once shortly
* after instantiation and before any calls to {@link #run}. Bootstrapping should not be construed as a command
* to run the tasks; they will be passed to {@link #run} one-by-one when this is desired. Some bootstrapped tasks
* may not actually be running (for example, if they are currently held back due to not having a lock).
*
* @param tasks the tasks
*/
public void bootstrap(List<Task> tasks);
/**
* Run a task. The returned status should be some kind of completed status.
*
* @param task task to run
*
* @return task status, eventually
*/
public ListenableFuture<TaskStatus> run(Task task);
/**
* Best-effort task shutdown. May or may not do anything.
* Inform the task runner it can clean up any resources associated with a task. This implies shutdown of any
* currently-running tasks.
*/
public void shutdown(String taskid);
......@@ -59,5 +49,7 @@ public interface TaskRunner
public Collection<? extends TaskRunnerWorkItem> getPendingTasks();
public Collection<? extends TaskRunnerWorkItem> getKnownTasks();
public Collection<ZkWorker> getWorkers();
}
......@@ -19,11 +19,9 @@
package io.druid.indexing.overlord;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ComparisonChain;
import com.google.common.util.concurrent.ListenableFuture;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.task.Task;
import org.joda.time.DateTime;
import org.joda.time.DateTimeComparator;
......@@ -32,36 +30,35 @@ import org.joda.time.DateTimeComparator;
*/
public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
{
private final Task task;
private final String taskId;
private final ListenableFuture<TaskStatus> result;
private final DateTime createdTime;
private final DateTime queueInsertionTime;
public TaskRunnerWorkItem(
Task task,
String taskId,
ListenableFuture<TaskStatus> result
)
{
this(task, result, new DateTime(), new DateTime());
this(taskId, result, new DateTime(), new DateTime());
}
public TaskRunnerWorkItem(
Task task,
String taskId,
ListenableFuture<TaskStatus> result,
DateTime createdTime,
DateTime queueInsertionTime
)
{
this.task = task;
this.taskId = taskId;
this.result = result;
this.createdTime = createdTime;
this.queueInsertionTime = queueInsertionTime;
}
@JsonProperty
public Task getTask()
public String getTaskId()
{
return task;
return taskId;
}
public ListenableFuture<TaskStatus> getResult()
......@@ -69,13 +66,11 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
return result;
}
@JsonProperty
public DateTime getCreatedTime()
{
return createdTime;
}
@JsonProperty
public DateTime getQueueInsertionTime()
{
return queueInsertionTime;
......@@ -83,7 +78,7 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
public TaskRunnerWorkItem withQueueInsertionTime(DateTime time)
{
return new TaskRunnerWorkItem(task, result, createdTime, time);
return new TaskRunnerWorkItem(taskId, result, createdTime, time);
}
@Override
......@@ -91,7 +86,7 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
{
return ComparisonChain.start()
.compare(createdTime, taskRunnerWorkItem.getCreatedTime(), DateTimeComparator.getInstance())
.compare(task.getId(), taskRunnerWorkItem.getTask().getId())
.compare(taskId, taskRunnerWorkItem.getTaskId())
.result();
}
......@@ -99,9 +94,10 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
public String toString()
{
return "TaskRunnerWorkItem{" +
"task=" + task +
"taskId='" + taskId + '\'' +
", result=" + result +
", createdTime=" + createdTime +
", queueInsertionTime=" + queueInsertionTime +
'}';
}
}
......@@ -77,7 +77,8 @@ public interface TaskStorage
public List<TaskAction> getAuditLogs(String taskid);
/**
* Returns a list of currently running or pending tasks as stored in the storage facility, in no particular order.
* Returns a list of currently running or pending tasks as stored in the storage facility. No particular order
* is guaranteed, but implementations are encouraged to return tasks in ascending order of creation.
*/
public List<Task> getActiveTasks();
......
......@@ -19,23 +19,14 @@
package io.druid.indexing.overlord;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.google.inject.Inject;
import com.metamx.common.guava.FunctionalIterable;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.actions.SegmentInsertAction;
import io.druid.indexing.common.actions.SpawnTasksAction;
import io.druid.indexing.common.actions.TaskAction;
import io.druid.indexing.common.task.Task;
import io.druid.timeline.DataSegment;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
......@@ -57,126 +48,20 @@ public class TaskStorageQueryAdapter
}
/**
* Returns all recursive task statuses for a particular task, staying within the same task group. Includes that
* task, plus any tasks it spawned, and so on. Does not include spawned tasks that ended up in a different task
* group. Does not include this task's parents or siblings.
*/
public Map<String, Optional<TaskStatus>> getSameGroupChildStatuses(final String taskid)
{
final Optional<Task> taskOptional = storage.getTask(taskid);
final Optional<TaskStatus> statusOptional = storage.getStatus(taskid);
final ImmutableMap.Builder<String, Optional<TaskStatus>> resultBuilder = ImmutableMap.builder();
resultBuilder.put(taskid, statusOptional);
final Iterable<Task> nextTasks = FunctionalIterable
.create(storage.getAuditLogs(taskid)).filter(
new Predicate<TaskAction>()
{
@Override
public boolean apply(TaskAction taskAction)
{
return taskAction instanceof SpawnTasksAction;
}
}
).transformCat(
new Function<TaskAction, Iterable<Task>>()
{
@Override
public Iterable<Task> apply(TaskAction taskAction)
{
return ((SpawnTasksAction) taskAction).getNewTasks();
}
}
);
if(taskOptional.isPresent() && statusOptional.isPresent()) {
for(final Task nextTask : nextTasks) {
if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) {
resultBuilder.putAll(getSameGroupChildStatuses(nextTask.getId()));
}
}
}
return resultBuilder.build();
}
/**
* Like {@link #getSameGroupChildStatuses}, but flattens the recursive statuses into a single, merged status.
*/
public Optional<TaskStatus> getSameGroupMergedStatus(final String taskid)
{
final Map<String, Optional<TaskStatus>> statuses = getSameGroupChildStatuses(taskid);
int nSuccesses = 0;
int nFailures = 0;
int nTotal = 0;
int nPresent = 0;
for(final Optional<TaskStatus> statusOption : statuses.values()) {
nTotal ++;
if(statusOption.isPresent()) {
nPresent ++;
final TaskStatus status = statusOption.get();
if(status.isSuccess()) {
nSuccesses ++;
} else if(status.isFailure()) {
nFailures ++;
}
}
}
final Optional<TaskStatus> status;
if(nPresent == 0) {
status = Optional.absent();
} else if(nSuccesses == nTotal) {
status = Optional.of(TaskStatus.success(taskid));
} else if(nFailures > 0) {
status = Optional.of(TaskStatus.failure(taskid));
} else {
status = Optional.of(TaskStatus.running(taskid));
}
return status;
}
/**
* Returns all segments created by descendants for a particular task that stayed within the same task group. Includes
* that task, plus any tasks it spawned, and so on. Does not include spawned tasks that ended up in a different task
* group. Does not include this task's parents or siblings.
* Returns all segments created by this task.
*
* This method is useful when you want to figure out all of the things a single task spawned. It does pose issues
* with the result set perhaps growing boundlessly and we do not do anything to protect against that. Use at your
* own risk and know that at some point, we might adjust this to actually enforce some sort of limits.
*/
public Set<DataSegment> getSameGroupNewSegments(final String taskid)
public Set<DataSegment> getInsertedSegments(final String taskid)
{
final Optional<Task> taskOptional = storage.getTask(taskid);
final Set<DataSegment> segments = Sets.newHashSet();
final List<Task> nextTasks = Lists.newArrayList();
for(final TaskAction action : storage.getAuditLogs(taskid)) {
if(action instanceof SpawnTasksAction) {
nextTasks.addAll(((SpawnTasksAction) action).getNewTasks());
}
if(action instanceof SegmentInsertAction) {
for (final TaskAction action : storage.getAuditLogs(taskid)) {
if (action instanceof SegmentInsertAction) {
segments.addAll(((SegmentInsertAction) action).getSegments());
}
}
if(taskOptional.isPresent()) {
for(final Task nextTask : nextTasks) {
if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) {
segments.addAll(getSameGroupNewSegments(nextTask.getId()));
}
}
}
return segments;
}
}
......@@ -19,7 +19,7 @@
package io.druid.indexing.overlord;
import com.google.common.base.Function;
import com.google.api.client.repackaged.com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
......@@ -46,7 +46,6 @@ import org.joda.time.Interval;
import java.io.File;
import java.util.Collection;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentSkipListSet;
......@@ -58,7 +57,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
{
private final TaskToolboxFactory toolboxFactory;
private final ListeningExecutorService exec;
private final Set<TaskRunnerWorkItem> runningItems = new ConcurrentSkipListSet<TaskRunnerWorkItem>();
private final Set<ThreadPoolTaskRunnerWorkItem> runningItems = new ConcurrentSkipListSet<>();
private static final EmittingLogger log = new EmittingLogger(ThreadPoolTaskRunner.class);
......@@ -67,7 +66,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
TaskToolboxFactory toolboxFactory
)
{
this.toolboxFactory = toolboxFactory;
this.toolboxFactory = Preconditions.checkNotNull(toolboxFactory, "toolboxFactory");
this.exec = MoreExecutors.listeningDecorator(Execs.singleThreaded("task-runner-%d"));
}
......@@ -77,19 +76,12 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
exec.shutdownNow();
}
@Override
public void bootstrap(List<Task> tasks)
{
// do nothing
}
@Override
public ListenableFuture<TaskStatus> run(final Task task)
{
final TaskToolbox toolbox = toolboxFactory.build(task);
final ListenableFuture<TaskStatus> statusFuture = exec.submit(new ExecutorServiceTaskRunnerCallable(task, toolbox));
final TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem(task, statusFuture);
final ListenableFuture<TaskStatus> statusFuture = exec.submit(new ThreadPoolTaskRunnerCallable(task, toolbox));
final ThreadPoolTaskRunnerWorkItem taskRunnerWorkItem = new ThreadPoolTaskRunnerWorkItem(task, statusFuture);
runningItems.add(taskRunnerWorkItem);
Futures.addCallback(
statusFuture, new FutureCallback<TaskStatus>()
......@@ -115,7 +107,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
public void shutdown(final String taskid)
{
for (final TaskRunnerWorkItem runningItem : runningItems) {
if (runningItem.getTask().getId().equals(taskid)) {
if (runningItem.getTaskId().equals(taskid)) {
runningItem.getResult().cancel(true);
}
}
......@@ -124,7 +116,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
@Override
public Collection<TaskRunnerWorkItem> getRunningTasks()
{
return ImmutableList.copyOf(runningItems);
return ImmutableList.<TaskRunnerWorkItem>copyOf(runningItems);
}
@Override
......@@ -133,6 +125,12 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
return ImmutableList.of();
}
@Override
public Collection<TaskRunnerWorkItem> getKnownTasks()
{
return ImmutableList.<TaskRunnerWorkItem>copyOf(runningItems);
}
@Override
public Collection<ZkWorker> getWorkers()
{
......@@ -155,18 +153,8 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
{
QueryRunner<T> queryRunner = null;
final List<Task> runningTasks = Lists.transform(
ImmutableList.copyOf(getRunningTasks()), new Function<TaskRunnerWorkItem, Task>()
{
@Override
public Task apply(TaskRunnerWorkItem o)
{
return o.getTask();
}
}
);
for (final Task task : runningTasks) {
for (final ThreadPoolTaskRunnerWorkItem taskRunnerWorkItem : ImmutableList.copyOf(runningItems)) {
final Task task = taskRunnerWorkItem.getTask();
if (task.getDataSource().equals(query.getDataSource())) {
final QueryRunner<T> taskQueryRunner = task.getQueryRunner(query);
......@@ -185,12 +173,31 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
return queryRunner == null ? new NoopQueryRunner<T>() : queryRunner;
}
private static class ExecutorServiceTaskRunnerCallable implements Callable<TaskStatus>
private static class ThreadPoolTaskRunnerWorkItem extends TaskRunnerWorkItem
{
private final Task task;
private ThreadPoolTaskRunnerWorkItem(
Task task,
ListenableFuture<TaskStatus> result
)
{
super(task.getId(), result);
this.task = task;
}
public Task getTask()
{
return task;
}
}
private static class ThreadPoolTaskRunnerCallable implements Callable<TaskStatus>
{
private final Task task;
private final TaskToolbox toolbox;
public ExecutorServiceTaskRunnerCallable(Task task, TaskToolbox toolbox)
public ThreadPoolTaskRunnerCallable(Task task, TaskToolbox toolbox)
{
this.task = task;
this.toolbox = toolbox;
......@@ -242,10 +249,5 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
throw Throwables.propagate(e);
}
}
public TaskRunnerWorkItem getTaskRunnerWorkItem()
{
return new TaskRunnerWorkItem(task, null);
}
}
}
......@@ -71,9 +71,9 @@ public class ZkWorker implements Closeable
};
}
public void start(PathChildrenCache.StartMode startMode) throws Exception
public void start() throws Exception
{
statusCache.start(startMode);
statusCache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT);
}
public void addListener(PathChildrenCacheListener listener)
......
......@@ -17,30 +17,63 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.overlord.http;
package io.druid.indexing.overlord.config;
import com.google.inject.Inject;
import io.druid.common.config.JacksonConfigManager;
import io.druid.indexing.overlord.TaskMaster;
import io.druid.indexing.overlord.TaskStorageQueryAdapter;
import io.druid.tasklogs.TaskLogStreamer;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Duration;
import org.joda.time.Period;
import javax.ws.rs.Path;
/**
*/
@Deprecated
@Path("/mmx/merger/v1")
public class OldOverlordResource extends OverlordResource
public class TaskQueueConfig
{
@Inject
public OldOverlordResource(
TaskMaster taskMaster,
TaskStorageQueryAdapter taskStorageQueryAdapter,
TaskLogStreamer taskLogStreamer,
JacksonConfigManager configManager
) throws Exception
@JsonProperty
private int maxSize;
@JsonProperty
private Duration startDelay;
@JsonProperty
private Duration restartDelay;
@JsonProperty
private Duration storageSyncRate;
@JsonCreator
public TaskQueueConfig(
@JsonProperty("maxSize") final Integer maxSize,
@JsonProperty("startDelay") final Period startDelay,
@JsonProperty("restartDelay") final Period restartDelay,
@JsonProperty("storageSyncRate") final Period storageSyncRate
)
{
this.maxSize = maxSize == null ? Integer.MAX_VALUE : maxSize;
this.startDelay = defaultDuration(startDelay, "PT1M");
this.restartDelay = defaultDuration(restartDelay, "PT30S");
this.storageSyncRate = defaultDuration(storageSyncRate, "PT1M");
}
public int getMaxSize()
{
return maxSize;
}
public Duration getStartDelay()
{
return startDelay;
}
public Duration getRestartDelay()
{
return restartDelay;
}
public Duration getStorageSyncRate()
{
return storageSyncRate;
}
private static Duration defaultDuration(final Period period, final String theDefault)
{
super(taskMaster, taskStorageQueryAdapter, taskLogStreamer, configManager);
return (period == null ? new Period(theDefault) : period).toStandardDuration();
}
}
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.overlord.exec;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.TaskQueue;
import io.druid.indexing.overlord.TaskRunner;
public class TaskConsumer implements Runnable
{
private final TaskQueue queue;
private final TaskRunner runner;
private final TaskActionClientFactory taskActionClientFactory;
private final ServiceEmitter emitter;
private final Thread thready;
private volatile boolean shutdown = false;
private static final EmittingLogger log = new EmittingLogger(TaskConsumer.class);
public TaskConsumer(
TaskQueue queue,
TaskRunner runner,
TaskActionClientFactory taskActionClientFactory,
ServiceEmitter emitter
)
{
this.queue = queue;
this.runner = runner;
this.taskActionClientFactory = taskActionClientFactory;
this.emitter = emitter;
this.thready = new Thread(this);
}
@LifecycleStart
public void start()
{
thready.start();
}
@LifecycleStop
public void stop()
{
shutdown = true;
thready.interrupt();
}
@Override
public void run()
{
try {
while (!Thread.currentThread().isInterrupted()) {
final Task task;
try {
task = queue.take();
}
catch (InterruptedException e) {
log.info("Interrupted while waiting for new work");
Thread.currentThread().interrupt();
break;
}
try {
handoff(task);
}
catch (Exception e) {
log.makeAlert(e, "Failed to hand off task")
.addData("task", task.getId())
.addData("type", task.getType())
.addData("dataSource", task.getDataSource())
.addData("interval", task.getImplicitLockInterval())
.emit();
// Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now.
if (!shutdown) {
queue.notify(task, TaskStatus.failure(task.getId()));
}
}
}
}
catch (Exception e) {
// exit thread
log.error(e, "Uncaught exception while consuming tasks");
throw Throwables.propagate(e);
}
}
private void handoff(final Task task) throws Exception
{
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
.setUser2(task.getDataSource())
.setUser4(task.getType())
.setUser5(task.getImplicitLockInterval().toString());
// Run preflight checks
TaskStatus preflightStatus;
try {
preflightStatus = task.preflight(taskActionClientFactory.create(task));
log.info("Preflight done for task: %s", task.getId());
}
catch (Exception e) {
preflightStatus = TaskStatus.failure(task.getId());
log.error(e, "Exception thrown during preflight for task: %s", task.getId());
}
if (!preflightStatus.isRunnable()) {
log.info("Task finished during preflight: %s", task.getId());
queue.notify(task, preflightStatus);
return;
}
// Hand off work to TaskRunner, with a callback
final ListenableFuture<TaskStatus> status = runner.run(task);
Futures.addCallback(
status, new FutureCallback<TaskStatus>()
{
@Override
public void onSuccess(final TaskStatus status)
{
log.info("Received %s status for task: %s", status.getStatusCode(), task);
handleStatus(status);
}
@Override
public void onFailure(Throwable t)
{
log.makeAlert(t, "Failed to run task")
.addData("task", task.getId())
.addData("type", task.getType())
.addData("dataSource", task.getDataSource())
.addData("interval", task.getImplicitLockInterval())
.emit();
handleStatus(TaskStatus.failure(task.getId()));
}
private void handleStatus(TaskStatus status)
{
try {
// If we're not supposed to be running anymore, don't do anything. Somewhat racey if the flag gets set after
// we check and before we commit the database transaction, but better than nothing.
if (shutdown) {
log.info("Abandoning task due to shutdown: %s", task.getId());
return;
}
queue.notify(task, status);
// Emit event and log, if the task is done
if (status.isComplete()) {
metricBuilder.setUser3(status.getStatusCode().toString());
emitter.emit(metricBuilder.build("indexer/time/run/millis", status.getDuration()));
log.info(
"Task %s: %s (%d run duration)",
status.getStatusCode(),
task,
status.getDuration()
);
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to handle task status")
.addData("task", task.getId())
.addData("statusCode", status.getStatusCode())
.emit();
}
}
}
);
}
}
......@@ -70,14 +70,7 @@ public class OverlordResource
public Map<String, Object> apply(TaskRunnerWorkItem input)
{
return new ImmutableMap.Builder<String, Object>()
.put("id", input.getTask().getId())
.put("dataSource", input.getTask().getDataSource())
.put("interval",
!input.getTask().getImplicitLockInterval().isPresent()
? ""
: input.getTask().getImplicitLockInterval().get()
)
.put("nodeType", input.getTask().getNodeType() == null ? "" : input.getTask().getNodeType())
.put("id", input.getTaskId())
.put("createdTime", input.getCreatedTime())
.put("queueInsertionTime", input.getQueueInsertionTime())
.build();
......@@ -151,7 +144,7 @@ public class OverlordResource
@Produces("application/json")
public Response getTaskStatus(@PathParam("taskid") String taskid)
{
return optionalTaskResponse(taskid, "status", taskStorageQueryAdapter.getSameGroupMergedStatus(taskid));
return optionalTaskResponse(taskid, "status", taskStorageQueryAdapter.getStatus(taskid));
}
@GET
......@@ -159,7 +152,7 @@ public class OverlordResource
@Produces("application/json")
public Response getTaskSegments(@PathParam("taskid") String taskid)
{
final Set<DataSegment> segments = taskStorageQueryAdapter.getSameGroupNewSegments(taskid);
final Set<DataSegment> segments = taskStorageQueryAdapter.getInsertedSegments(taskid);
return Response.ok().entity(segments).build();
}
......@@ -169,13 +162,13 @@ public class OverlordResource
public Response doShutdown(@PathParam("taskid") final String taskid)
{
return asLeaderWith(
taskMaster.getTaskRunner(),
new Function<TaskRunner, Response>()
taskMaster.getTaskQueue(),
new Function<TaskQueue, Response>()
{
@Override
public Response apply(TaskRunner taskRunner)
public Response apply(TaskQueue taskQueue)
{
taskRunner.shutdown(taskid);
taskQueue.shutdown(taskid);
return Response.ok(ImmutableMap.of("task", taskid)).build();
}
}
......@@ -225,7 +218,7 @@ public class OverlordResource
final Map<String, Object> retMap;
// It would be great to verify that this worker is actually supposed to be running the task before
// actually doing the task. Some ideas for how that could be done would be using some sort of attempt_id
// actually doing the action. Some ideas for how that could be done would be using some sort of attempt_id
// or token that gets passed around.
try {
......
......@@ -20,16 +20,19 @@
package io.druid.indexing.worker.executor;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.api.client.repackaged.com.google.common.base.Preconditions;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.inject.Inject;
import com.metamx.common.ISE;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.emitter.EmittingLogger;
import io.druid.concurrent.Execs;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.TaskRunner;
......@@ -47,6 +50,7 @@ public class ExecutorLifecycle
private static final EmittingLogger log = new EmittingLogger(ExecutorLifecycle.class);
private final ExecutorLifecycleConfig config;
private final TaskActionClientFactory taskActionClientFactory;
private final TaskRunner taskRunner;
private final ObjectMapper jsonMapper;
......@@ -57,11 +61,13 @@ public class ExecutorLifecycle
@Inject
public ExecutorLifecycle(
ExecutorLifecycleConfig config,
TaskActionClientFactory taskActionClientFactory,
TaskRunner taskRunner,
ObjectMapper jsonMapper
)
{
this.config = config;
this.taskActionClientFactory = taskActionClientFactory;
this.taskRunner = taskRunner;
this.jsonMapper = jsonMapper;
}
......@@ -69,9 +75,9 @@ public class ExecutorLifecycle
@LifecycleStart
public void start()
{
final File taskFile = config.getTaskFile();
final File statusFile = config.getStatusFile();
final InputStream parentStream = config.getParentStream();
final File taskFile = Preconditions.checkNotNull(config.getTaskFile(), "taskFile");
final File statusFile = Preconditions.checkNotNull(config.getStatusFile(), "statusFile");
final InputStream parentStream = Preconditions.checkNotNull(config.getParentStream(), "parentStream");
final Task task;
......@@ -111,28 +117,41 @@ public class ExecutorLifecycle
}
);
statusFuture = Futures.transform(
taskRunner.run(task), new Function<TaskStatus, TaskStatus>()
{
@Override
public TaskStatus apply(TaskStatus taskStatus)
{
try {
log.info(
"Task completed with status: %s",
jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(taskStatus)
);
statusFile.getParentFile().mkdirs();
jsonMapper.writeValue(statusFile, taskStatus);
return taskStatus;
}
catch (Exception e) {
throw Throwables.propagate(e);
}
// Won't hurt in remote mode, and is required for setting up locks in local mode:
try {
if (!task.isReady(taskActionClientFactory.create(task))) {
throw new ISE("Task is not ready to run yet!", task.getId());
}
} catch (Exception e) {
throw new ISE(e, "Failed to run isReady", task.getId());
}
statusFuture = Futures.transform(
taskRunner.run(task),
new Function<TaskStatus, TaskStatus>()
{
@Override
public TaskStatus apply(TaskStatus taskStatus)
{
try {
log.info(
"Task completed with status: %s",
jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(taskStatus)
);
final File statusFileParent = statusFile.getParentFile();
if (statusFileParent != null) {
statusFileParent.mkdirs();
}
jsonMapper.writeValue(statusFile, taskStatus);
return taskStatus;
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
);
}
......
......@@ -67,7 +67,7 @@ public class MergeTaskBaseTest
@Test
public void testInterval()
{
Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTaskBase.getImplicitLockInterval().get());
Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTaskBase.getInterval());
}
@Test
......
......@@ -19,14 +19,15 @@
package io.druid.indexing.common.task;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Optional;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.metamx.common.Granularity;
import io.druid.data.input.impl.JSONDataSpec;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.granularity.QueryGranularity;
import io.druid.guice.FirehoseModule;
import io.druid.indexer.HadoopDruidIndexerSchema;
import io.druid.indexer.granularity.UniformGranularitySpec;
import io.druid.indexer.rollup.DataRollupSpec;
......@@ -36,6 +37,7 @@ import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.segment.IndexGranularity;
import io.druid.segment.realtime.Schema;
import io.druid.segment.realtime.firehose.LocalFirehoseFactory;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import junit.framework.Assert;
......@@ -43,12 +45,14 @@ import org.joda.time.Interval;
import org.joda.time.Period;
import org.junit.Test;
import java.io.File;
public class TaskSerdeTest
{
@Test
public void testIndexTaskSerde() throws Exception
{
final Task task = new IndexTask(
final IndexTask task = new IndexTask(
null,
"foo",
new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P2D"))),
......@@ -56,62 +60,34 @@ public class TaskSerdeTest
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
QueryGranularity.NONE,
10000,
null,
new LocalFirehoseFactory(new File("lol"), "rofl", null),
-1
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) {
jsonMapper.registerModule(jacksonModule);
}
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final Task task2 = jsonMapper.readValue(json, Task.class);
final IndexTask task2 = (IndexTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.of(new Interval("2010-01-01/P2D")), task.getImplicitLockInterval());
Assert.assertEquals(new Interval("2010-01-01/P2D"), task.getInterval());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
}
@Test
public void testIndexGeneratorTaskSerde() throws Exception
{
final Task task = new IndexGeneratorTask(
null,
"foo",
new Interval("2010-01-01/P1D"),
null,
new Schema(
"foo",
null,
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
QueryGranularity.NONE,
new NoneShardSpec()
),
-1
);
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final Task task2 = jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
Assert.assertEquals(task.getInterval(), task2.getInterval());
Assert.assertTrue(task.getFirehoseFactory() instanceof LocalFirehoseFactory);
Assert.assertTrue(task2.getFirehoseFactory() instanceof LocalFirehoseFactory);
}
@Test
public void testMergeTaskSerde() throws Exception
{
final Task task = new MergeTask(
final MergeTask task = new MergeTask(
null,
"foo",
ImmutableList.<DataSegment>of(
......@@ -126,26 +102,26 @@ public class TaskSerdeTest
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final Task task2 = jsonMapper.readValue(json, Task.class);
final MergeTask task2 = (MergeTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
Assert.assertEquals(((MergeTask) task).getSegments(), ((MergeTask) task2).getSegments());
Assert.assertEquals(task.getInterval(), task2.getInterval());
Assert.assertEquals(task.getSegments(), task2.getSegments());
Assert.assertEquals(
((MergeTask) task).getAggregators().get(0).getName(),
((MergeTask) task2).getAggregators().get(0).getName()
task.getAggregators().get(0).getName(),
task2.getAggregators().get(0).getName()
);
}
@Test
public void testKillTaskSerde() throws Exception
{
final Task task = new KillTask(
final KillTask task = new KillTask(
null,
"foo",
new Interval("2010-01-01/P1D")
......@@ -155,21 +131,21 @@ public class TaskSerdeTest
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final Task task2 = jsonMapper.readValue(json, Task.class);
final KillTask task2 = (KillTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
Assert.assertEquals(task.getInterval(), task2.getInterval());
}
@Test
public void testVersionConverterTaskSerde() throws Exception
{
final Task task = VersionConverterTask.create(
final VersionConverterTask task = VersionConverterTask.create(
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
);
......@@ -177,22 +153,22 @@ public class TaskSerdeTest
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final Task task2 = jsonMapper.readValue(json, Task.class);
final VersionConverterTask task2 = (VersionConverterTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
Assert.assertEquals(((VersionConverterTask) task).getSegment(), ((VersionConverterTask) task).getSegment());
Assert.assertEquals(task.getInterval(), task2.getInterval());
Assert.assertEquals(task.getSegment(), task.getSegment());
}
@Test
public void testVersionConverterSubTaskSerde() throws Exception
{
final Task task = new VersionConverterTask.SubTask(
final VersionConverterTask.SubTask task = new VersionConverterTask.SubTask(
"myGroupId",
DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build()
);
......@@ -201,26 +177,21 @@ public class TaskSerdeTest
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final Task task2 = jsonMapper.readValue(json, Task.class);
final VersionConverterTask.SubTask task2 = (VersionConverterTask.SubTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
Assert.assertEquals("myGroupId", task.getGroupId());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
Assert.assertEquals(
((VersionConverterTask.SubTask) task).getSegment(),
((VersionConverterTask.SubTask) task).getSegment()
);
Assert.assertEquals(task.getSegment(), task2.getSegment());
}
@Test
public void testRealtimeIndexTaskSerde() throws Exception
{
final Task task = new RealtimeIndexTask(
final RealtimeIndexTask task = new RealtimeIndexTask(
null,
new TaskResource("rofl", 2),
new Schema("foo", null, new AggregatorFactory[0], QueryGranularity.NONE, new NoneShardSpec()),
......@@ -235,32 +206,27 @@ public class TaskSerdeTest
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final Task task2 = jsonMapper.readValue(json, Task.class);
final RealtimeIndexTask task2 = (RealtimeIndexTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.<Interval>absent(), task.getImplicitLockInterval());
Assert.assertEquals(2, task.getTaskResource().getRequiredCapacity());
Assert.assertEquals("rofl", task.getTaskResource().getAvailabilityGroup());
Assert.assertEquals(new Period("PT10M"), ((RealtimeIndexTask) task).getWindowPeriod());
Assert.assertEquals(IndexGranularity.HOUR, ((RealtimeIndexTask) task).getSegmentGranularity());
Assert.assertEquals(new Period("PT10M"), task.getWindowPeriod());
Assert.assertEquals(IndexGranularity.HOUR, task.getSegmentGranularity());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
Assert.assertEquals(task.getTaskResource().getRequiredCapacity(), task2.getTaskResource().getRequiredCapacity());
Assert.assertEquals(task.getTaskResource().getAvailabilityGroup(), task2.getTaskResource().getAvailabilityGroup());
Assert.assertEquals(((RealtimeIndexTask) task).getWindowPeriod(), ((RealtimeIndexTask) task2).getWindowPeriod());
Assert.assertEquals(
((RealtimeIndexTask) task).getSegmentGranularity(),
((RealtimeIndexTask) task2).getSegmentGranularity()
);
Assert.assertEquals(task.getWindowPeriod(), task2.getWindowPeriod());
Assert.assertEquals(task.getSegmentGranularity(), task2.getSegmentGranularity());
}
@Test
public void testDeleteTaskSerde() throws Exception
{
final Task task = new DeleteTask(
final DeleteTask task = new DeleteTask(
null,
"foo",
new Interval("2010-01-01/P1D")
......@@ -270,46 +236,44 @@ public class TaskSerdeTest
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final Task task2 = jsonMapper.readValue(json, Task.class);
final DeleteTask task2 = (DeleteTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
Assert.assertEquals(task.getImplicitLockInterval().get(), task2.getImplicitLockInterval().get());
Assert.assertEquals(task.getInterval(), task2.getInterval());
}
@Test
public void testDeleteTaskFromJson() throws Exception
{
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final Task task = jsonMapper.readValue(
final DeleteTask task = (DeleteTask) jsonMapper.readValue(
"{\"type\":\"delete\",\"dataSource\":\"foo\",\"interval\":\"2010-01-01/P1D\"}",
Task.class
);
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final Task task2 = jsonMapper.readValue(json, Task.class);
final DeleteTask task2 = (DeleteTask) jsonMapper.readValue(json, Task.class);
Assert.assertNotNull(task.getId());
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
Assert.assertEquals(task.getImplicitLockInterval().get(), task2.getImplicitLockInterval().get());
Assert.assertEquals(task.getInterval(), task2.getInterval());
}
@Test
public void testAppendTaskSerde() throws Exception
{
final Task task = new AppendTask(
final AppendTask task = new AppendTask(
null,
"foo",
ImmutableList.of(
......@@ -321,17 +285,16 @@ public class TaskSerdeTest
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
final Task task2 = jsonMapper.readValue(json, Task.class);
final AppendTask task2 = (AppendTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
Assert.assertEquals(task.getImplicitLockInterval().get(), task2.getImplicitLockInterval().get());
Assert.assertEquals(((AppendTask) task).getSegments(), ((AppendTask) task2).getSegments());
Assert.assertEquals(task.getInterval(), task2.getInterval());
Assert.assertEquals(task.getSegments(), ((AppendTask) task2).getSegments());
}
@Test
......@@ -364,14 +327,14 @@ public class TaskSerdeTest
final ObjectMapper jsonMapper = new DefaultObjectMapper();
final String json = jsonMapper.writeValueAsString(task);
final Task task2 = jsonMapper.readValue(json, Task.class);
final HadoopIndexTask task2 = (HadoopIndexTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Optional.of(new Interval("2010-01-01/P1D")), task.getImplicitLockInterval());
Assert.assertEquals(new Interval("2010-01-01/P1D"), task.getInterval());
Assert.assertEquals(task.getId(), task2.getId());
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getImplicitLockInterval(), task2.getImplicitLockInterval());
Assert.assertEquals(task.getInterval(), task2.getInterval());
}
}
......@@ -28,6 +28,7 @@ import io.druid.indexing.common.actions.LockAcquireAction;
import io.druid.indexing.common.actions.LockListAction;
import io.druid.indexing.common.actions.LockReleaseAction;
import io.druid.indexing.common.actions.SegmentInsertAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.task.AbstractTask;
import io.druid.indexing.common.task.TaskResource;
import io.druid.timeline.DataSegment;
......@@ -42,12 +43,12 @@ public class RealtimeishTask extends AbstractTask
{
public RealtimeishTask()
{
super("rt1", "rt", new TaskResource("rt1", 1), "foo", null);
super("rt1", "rt", new TaskResource("rt1", 1), "foo");
}
public RealtimeishTask(String id, String groupId, TaskResource taskResource, String dataSource, Interval interval)
public RealtimeishTask(String id, String groupId, TaskResource taskResource, String dataSource)
{
super(id, groupId, taskResource, dataSource, interval);
super(id, groupId, taskResource, dataSource);
}
@Override
......@@ -56,6 +57,12 @@ public class RealtimeishTask extends AbstractTask
return "realtime_test";
}
@Override
public boolean isReady(TaskActionClient taskActionClient) throws Exception
{
return true;
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
......
......@@ -23,8 +23,8 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.api.client.repackaged.com.google.common.base.Throwables;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListenableFuture;
import com.metamx.emitter.EmittingLogger;
......@@ -55,7 +55,6 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.Arrays;
import java.util.Set;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicReference;
......@@ -219,7 +218,7 @@ public class RemoteTaskRunnerTest
)
);
Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTask().getId().equals("rt2"));
Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTaskId().equals("rt2"));
}
@Test
......@@ -266,7 +265,7 @@ public class RemoteTaskRunnerTest
)
);
Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTask().getId().equals("rt2"));
Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTaskId().equals("rt2"));
}
@Test
......@@ -280,7 +279,7 @@ public class RemoteTaskRunnerTest
Assert.assertTrue(workerRunningTask(task.getId()));
Assert.assertTrue(remoteTaskRunner.getRunningTasks().iterator().next().getTask().getId().equals("task"));
Assert.assertTrue(remoteTaskRunner.getRunningTasks().iterator().next().getTaskId().equals("task"));
cf.delete().forPath(joiner.join(statusPath, task.getId()));
......@@ -303,18 +302,13 @@ public class RemoteTaskRunnerTest
doSetup();
Set<String> existingTasks = Sets.newHashSet();
final Set<String> existingTasks = Sets.newHashSet();
for (ZkWorker zkWorker : remoteTaskRunner.getWorkers()) {
existingTasks.addAll(zkWorker.getRunningTasks().keySet());
}
Assert.assertEquals("existingTasks", ImmutableSet.of("first", "second"), existingTasks);
Assert.assertTrue(existingTasks.size() == 2);
Assert.assertTrue(existingTasks.contains("first"));
Assert.assertTrue(existingTasks.contains("second"));
remoteTaskRunner.bootstrap(Arrays.<Task>asList(TestMergeTask.createDummyTask("second")));
Set<String> runningTasks = Sets.newHashSet(
final Set<String> runningTasks = Sets.newHashSet(
Iterables.transform(
remoteTaskRunner.getRunningTasks(),
new Function<RemoteTaskRunnerWorkItem, String>()
......@@ -322,15 +316,12 @@ public class RemoteTaskRunnerTest
@Override
public String apply(RemoteTaskRunnerWorkItem input)
{
return input.getTask().getId();
return input.getTaskId();
}
}
)
);
Assert.assertTrue(runningTasks.size() == 1);
Assert.assertTrue(runningTasks.contains("second"));
Assert.assertFalse(runningTasks.contains("first"));
Assert.assertEquals("runningTasks", ImmutableSet.of("first", "second"), runningTasks);
}
@Test
......@@ -343,8 +334,6 @@ public class RemoteTaskRunnerTest
doSetup();
remoteTaskRunner.bootstrap(Arrays.<Task>asList(task));
ListenableFuture<TaskStatus> future = remoteTaskRunner.run(task);
TaskStatus status = future.get();
......@@ -356,7 +345,6 @@ public class RemoteTaskRunnerTest
public void testWorkerRemoved() throws Exception
{
doSetup();
remoteTaskRunner.bootstrap(Lists.<Task>newArrayList());
Future<TaskStatus> future = remoteTaskRunner.run(task);
Assert.assertTrue(taskAnnounced(task.getId()));
......
......@@ -53,12 +53,12 @@ import io.druid.indexing.common.actions.SegmentInsertAction;
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.task.AbstractTask;
import io.druid.indexing.common.task.AbstractFixedIntervalTask;
import io.druid.indexing.common.task.IndexTask;
import io.druid.indexing.common.task.KillTask;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.common.task.TaskResource;
import io.druid.indexing.overlord.exec.TaskConsumer;
import io.druid.indexing.overlord.config.TaskQueueConfig;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
......@@ -96,7 +96,6 @@ public class TaskLifecycleTest
private MockIndexerDBCoordinator mdc = null;
private TaskActionClientFactory tac = null;
private TaskToolboxFactory tb = null;
private TaskConsumer tc = null;
TaskStorageQueryAdapter tsqa = null;
private static final Ordering<DataSegment> byIntervalOrdering = new Ordering<DataSegment>()
......@@ -109,18 +108,19 @@ public class TaskLifecycleTest
};
@Before
public void setUp()
public void setUp() throws Exception
{
EmittingLogger.registerEmitter(EasyMock.createMock(ServiceEmitter.class));
final ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class);
EmittingLogger.registerEmitter(emitter);
tmp = Files.createTempDir();
final TaskQueueConfig tqc = new DefaultObjectMapper().readValue("{\"startDelay\":\"PT0S\"}", TaskQueueConfig.class);
ts = new HeapMemoryTaskStorage();
tsqa = new TaskStorageQueryAdapter(ts);
tl = new TaskLockbox(ts);
tq = new TaskQueue(ts, tl);
mdc = newMockMDC();
tac = new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter()));
tac = new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tl, mdc, newMockEmitter()));
tb = new TaskToolboxFactory(
new TaskConfig(tmp.toString(), null, null, 50000),
tac,
......@@ -171,14 +171,9 @@ public class TaskLifecycleTest
),
new DefaultObjectMapper()
);
tr = new ThreadPoolTaskRunner(tb);
tc = new TaskConsumer(tq, tr, tac, newMockEmitter());
tsqa = new TaskStorageQueryAdapter(ts);
tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter);
tq.start();
tc.start();
}
@After
......@@ -190,7 +185,6 @@ public class TaskLifecycleTest
catch (Exception e) {
// suppress
}
tc.stop();
tq.stop();
}
......@@ -216,13 +210,13 @@ public class TaskLifecycleTest
-1
);
final Optional<TaskStatus> preRunTaskStatus = tsqa.getSameGroupMergedStatus(indexTask.getId());
final Optional<TaskStatus> preRunTaskStatus = tsqa.getStatus(indexTask.getId());
Assert.assertTrue("pre run task status not present", !preRunTaskStatus.isPresent());
final TaskStatus mergedStatus = runTask(indexTask);
final TaskStatus status = ts.getStatus(indexTask.getId()).get();
final List<DataSegment> publishedSegments = byIntervalOrdering.sortedCopy(mdc.getPublished());
final List<DataSegment> loggedSegments = byIntervalOrdering.sortedCopy(tsqa.getSameGroupNewSegments(indexTask.getId()));
final List<DataSegment> loggedSegments = byIntervalOrdering.sortedCopy(tsqa.getInsertedSegments(indexTask.getId()));
Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode());
......@@ -264,11 +258,9 @@ public class TaskLifecycleTest
-1
);
final TaskStatus mergedStatus = runTask(indexTask);
final TaskStatus status = ts.getStatus(indexTask.getId()).get();
final TaskStatus status = runTask(indexTask);
Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
Assert.assertEquals("merged statusCode", TaskStatus.Status.FAILED, mergedStatus.getStatusCode());
Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode());
Assert.assertEquals("num segments published", 0, mdc.getPublished().size());
Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size());
}
......@@ -300,7 +292,13 @@ public class TaskLifecycleTest
@Test
public void testSimple() throws Exception
{
final Task task = new AbstractTask("id1", "id1", new TaskResource("id1", 1), "ds", new Interval("2012-01-01/P1D"))
final Task task = new AbstractFixedIntervalTask(
"id1",
"id1",
new TaskResource("id1", 1),
"ds",
new Interval("2012-01-01/P1D")
)
{
@Override
public String getType()
......@@ -337,7 +335,7 @@ public class TaskLifecycleTest
@Test
public void testBadInterval() throws Exception
{
final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
final Task task = new AbstractFixedIntervalTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
{
@Override
public String getType()
......@@ -371,7 +369,7 @@ public class TaskLifecycleTest
@Test
public void testBadVersion() throws Exception
{
final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
final Task task = new AbstractFixedIntervalTask("id1", "id1", "ds", new Interval("2012-01-01/P1D"))
{
@Override
public String getType()
......@@ -411,7 +409,7 @@ public class TaskLifecycleTest
TaskStatus status;
try {
while ((status = tsqa.getSameGroupMergedStatus(task.getId()).get()).isRunnable()) {
while ((status = tsqa.getStatus(task.getId()).get()).isRunnable()) {
if (System.currentTimeMillis() > startTime + 10 * 1000) {
throw new ISE("Where did the task go?!: %s", task.getId());
}
......
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.overlord;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.TaskToolboxFactory;
import io.druid.indexing.common.actions.LocalTaskActionClientFactory;
import io.druid.indexing.common.actions.SpawnTasksAction;
import io.druid.indexing.common.actions.TaskActionToolbox;
import io.druid.indexing.common.config.TaskConfig;
import io.druid.indexing.common.task.AbstractTask;
import io.druid.indexing.common.task.Task;
import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.LocalDataSegmentPuller;
import io.druid.segment.loading.OmniSegmentLoader;
import io.druid.segment.loading.SegmentLoaderConfig;
import io.druid.segment.loading.StorageLocationConfig;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import java.util.List;
import java.util.Set;
public class TaskQueueTest
{
@Test
public void testEmptyQueue() throws Exception
{
final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl);
// get task status for nonexistent task
Assert.assertFalse("getStatus", ts.getStatus("foo").isPresent());
// poll on empty queue
Assert.assertNull("poll", tq.poll());
}
public static TaskQueue newTaskQueue(TaskStorage storage, TaskLockbox lockbox)
{
final TaskQueue tq = new TaskQueue(storage, lockbox);
tq.bootstrap();
tq.start();
return tq;
}
@Test
public void testAddRemove() throws Exception
{
final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl);
final Task[] tasks = {
newTask("T0", "G0", "bar", new Interval("2011/P1Y")),
newTask("T1", "G1", "bar", new Interval("2011-03-01/P1D")),
newTask("T2", "G2", "foo", new Interval("2011-03-01/P1D")),
newTask("T3", "G3", "foo", new Interval("2011/P1Y")),
newTask("T4", "G4", "foo", new Interval("2012-01-02/P1D")),
newTask("T5", "G5", "foo", new Interval("2012-02-01/PT1H"))
};
Throwable thrown;
for (Task task : tasks) {
tq.add(task);
}
// get task status for in-progress task
Assert.assertEquals(
"T2 status (before finishing)",
TaskStatus.Status.RUNNING,
ts.getStatus(tasks[2].getId()).get().getStatusCode()
);
// Can't add tasks with the same id
thrown = null;
try {
tq.add(newTask("T5", "G5", "baz", new Interval("2013-02-01/PT1H")));
}
catch (TaskExistsException e) {
thrown = e;
}
Assert.assertNotNull("Exception on duplicate task id", thrown);
// take max number of tasks
final List<Task> taken = Lists.newArrayList();
while (true) {
final Task task = tq.poll();
if (task != null) {
taken.add(task);
} else {
break;
}
}
// check them
Assert.assertEquals(
"Taken tasks (round 1)",
Lists.newArrayList(
tasks[0], tasks[2], tasks[4], tasks[5]
),
taken
);
// mark one done
tq.notify(tasks[2], tasks[2].run(null));
// get its status back
Assert.assertEquals(
"T2 status (after finishing)",
TaskStatus.Status.SUCCESS,
ts.getStatus(tasks[2].getId()).get().getStatusCode()
);
// We should be able to get one more task now
taken.clear();
while (true) {
final Task task = tq.poll();
if (task != null) {
taken.add(task);
} else {
break;
}
}
// check it
Assert.assertEquals(
"Taken tasks (round 2)",
Lists.newArrayList(
tasks[3]
),
taken
);
// there should be no more tasks to get
Assert.assertNull("poll queue with no tasks available", tq.poll());
}
@Test
public void testContinues() throws Exception
{
final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl);
final TaskToolboxFactory tb = new TaskToolboxFactory(
new TaskConfig(null, null, null, null),
new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)),
null,
null,
null,
null,
null,
null,
null,
null,
new SegmentLoaderFactory(
new OmniSegmentLoader(
ImmutableMap.<String, DataSegmentPuller>of(
"local",
new LocalDataSegmentPuller()
),
null,
new SegmentLoaderConfig()
{
@Override
public List<StorageLocationConfig> getLocations()
{
return Lists.newArrayList();
}
}
)
),
null
);
final Task t0 = newTask("T0", "G0", "bar", new Interval("2011/P1Y"));
final Task t1 = newContinuedTask("T1", "G1", "bar", new Interval("2013/P1Y"), Lists.newArrayList(t0));
tq.add(t1);
Assert.assertTrue("T0 isPresent (#1)", !ts.getStatus("T0").isPresent());
Assert.assertTrue("T1 isPresent (#1)", ts.getStatus("T1").isPresent());
Assert.assertTrue("T1 isRunnable (#1)", ts.getStatus("T1").get().isRunnable());
Assert.assertTrue("T1 isComplete (#1)", !ts.getStatus("T1").get().isComplete());
// should be able to get t1 out
Assert.assertEquals("poll #1", "T1", tq.poll().getId());
Assert.assertNull("poll #2", tq.poll());
// report T1 done. Should cause T0 to be created
tq.notify(t1, t1.run(tb.build(t1)));
Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent());
Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable());
Assert.assertTrue("T0 isComplete (#2)", !ts.getStatus("T0").get().isComplete());
Assert.assertTrue("T1 isPresent (#2)", ts.getStatus("T1").isPresent());
Assert.assertTrue("T1 isRunnable (#2)", !ts.getStatus("T1").get().isRunnable());
Assert.assertTrue("T1 isComplete (#2)", ts.getStatus("T1").get().isComplete());
// should be able to get t0 out
Assert.assertEquals("poll #3", "T0", tq.poll().getId());
Assert.assertNull("poll #4", tq.poll());
// report T0 done. Should cause T0, T1 to be marked complete
tq.notify(t0, t0.run(tb.build(t0)));
Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent());
Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable());
Assert.assertTrue("T0 isComplete (#3)", ts.getStatus("T0").get().isComplete());
Assert.assertTrue("T1 isPresent (#3)", ts.getStatus("T1").isPresent());
Assert.assertTrue("T1 isRunnable (#3)", !ts.getStatus("T1").get().isRunnable());
Assert.assertTrue("T1 isComplete (#3)", ts.getStatus("T1").get().isComplete());
// should be no more events available for polling
Assert.assertNull("poll #5", tq.poll());
}
@Test
public void testConcurrency() throws Exception
{
final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl);
final TaskToolboxFactory tb = new TaskToolboxFactory(
new TaskConfig(null, null, null, null),
new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)),
null,
null,
null,
null,
null,
null,
null,
null,
new SegmentLoaderFactory(
new OmniSegmentLoader(
ImmutableMap.<String, DataSegmentPuller>of(
"local",
new LocalDataSegmentPuller()
),
null,
new SegmentLoaderConfig()
{
@Override
public List<StorageLocationConfig> getLocations()
{
return Lists.newArrayList();
}
}
)
),
null
);
// Imagine a larger task that splits itself up into pieces
final Task t1 = newTask("T1", "G0", "bar", new Interval("2011-01-01/P1D"));
final Task t2 = newTask("T2", "G1", "bar", new Interval("2011-01-02/P1D")); // Task group different from original
final Task t3 = newTask("T3", "G0", "bar", new Interval("2011-01-03/P1D"));
final Task t4 = newTask("T4", "G0", "bar", new Interval("2011-01-02/P5D")); // Interval wider than original
final Task t0 = newContinuedTask(
"T0",
"G0",
"bar",
new Interval("2011-01-01/P3D"),
ImmutableList.of(t1, t2, t3, t4)
);
tq.add(t0);
final Task wt0 = tq.poll();
final TaskLock wt0Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt0));
Assert.assertEquals("wt0 task id", "T0", wt0.getId());
Assert.assertNull("null poll #1", tq.poll());
// Sleep a bit to avoid false test passes
Thread.sleep(5);
// Finish t0
tq.notify(t0, t0.run(tb.build(t0)));
// take max number of tasks
final Set<String> taken = Sets.newHashSet();
while (true) {
// Sleep a bit to avoid false test passes
Thread.sleep(5);
final Task task = tq.poll();
if (task != null) {
final TaskLock taskLock = Iterables.getOnlyElement(tl.findLocksForTask(task));
Assert.assertEquals(
String.format("%s version", task.getId()),
wt0Lock.getVersion(),
taskLock.getVersion()
);
taken.add(task.getId());
} else {
break;
}
}
Assert.assertEquals("taken", Sets.newHashSet("T1", "T3"), taken);
// Finish t1
tq.notify(t1, t1.run(null));
Assert.assertNull("null poll #2", tq.poll());
// Finish t3
tq.notify(t3, t3.run(tb.build(t3)));
// We should be able to get t2 now
final Task wt2 = tq.poll();
final TaskLock wt2Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt2));
Assert.assertEquals("wt2 task id", "T2", wt2.getId());
Assert.assertEquals("wt2 group id", "G1", wt2.getGroupId());
Assert.assertNotSame("wt2 version", wt0Lock.getVersion(), wt2Lock.getVersion());
Assert.assertNull("null poll #3", tq.poll());
// Finish t2
tq.notify(t2, t2.run(tb.build(t2)));
// We should be able to get t4
// And it should be in group G0, but that group should have a different version than last time
// (Since the previous transaction named "G0" has ended and transaction names are not necessarily tied to
// one version if they end and are re-started)
final Task wt4 = tq.poll();
final TaskLock wt4Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt4));
Assert.assertEquals("wt4 task id", "T4", wt4.getId());
Assert.assertEquals("wt4 group id", "G0", wt4.getGroupId());
Assert.assertNotSame("wt4 version", wt0Lock.getVersion(), wt4Lock.getVersion());
Assert.assertNotSame("wt4 version", wt2Lock.getVersion(), wt4Lock.getVersion());
// Kind of done testing at this point, but let's finish t4 anyway
tq.notify(t4, t4.run(tb.build(t4)));
Assert.assertNull("null poll #4", tq.poll());
}
@Test
public void testBootstrap() throws Exception
{
final TaskStorage storage = new HeapMemoryTaskStorage();
final TaskLockbox lockbox = new TaskLockbox(storage);
storage.insert(newTask("T1", "G1", "bar", new Interval("2011-01-01/P1D")), TaskStatus.running("T1"));
storage.insert(newTask("T2", "G2", "bar", new Interval("2011-02-01/P1D")), TaskStatus.running("T2"));
storage.addLock("T1", new TaskLock("G1", "bar", new Interval("2011-01-01/P1D"), "1234"));
final TaskQueue tq = newTaskQueue(storage, lockbox);
final Task vt1 = tq.poll();
final TaskLock vt1Lock = Iterables.getOnlyElement(lockbox.findLocksForTask(vt1));
Assert.assertEquals("vt1 id", "T1", vt1.getId());
Assert.assertEquals("vt1 version", "1234", vt1Lock.getVersion());
tq.notify(vt1, TaskStatus.success("T1"));
// re-bootstrap
tq.stop();
storage.setStatus(TaskStatus.failure("T2"));
tq.bootstrap();
tq.start();
Assert.assertNull("null poll", tq.poll());
}
private static Task newTask(final String id, final String groupId, final String dataSource, final Interval interval)
{
return new AbstractTask(id, groupId, dataSource, interval)
{
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
return TaskStatus.success(id);
}
@Override
public String getType()
{
return "null";
}
};
}
private static Task newContinuedTask(
final String id,
final String groupId,
final String dataSource,
final Interval interval,
final List<Task> nextTasks
)
{
return new AbstractTask(id, groupId, dataSource, interval)
{
@Override
public String getType()
{
return "null";
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
toolbox.getTaskActionClient().submit(new SpawnTasksAction(nextTasks));
return TaskStatus.success(id);
}
};
}
}
......@@ -111,7 +111,7 @@ public class SimpleResourceManagementStrategyTest
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
Arrays.<RemoteTaskRunnerWorkItem>asList(
new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime())
new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime())
),
Arrays.<ZkWorker>asList(
new TestZkWorker(testTask)
......@@ -139,7 +139,7 @@ public class SimpleResourceManagementStrategyTest
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
Arrays.<RemoteTaskRunnerWorkItem>asList(
new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime())
new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime())
),
Arrays.<ZkWorker>asList(
new TestZkWorker(testTask)
......@@ -155,7 +155,7 @@ public class SimpleResourceManagementStrategyTest
provisionedSomething = simpleResourceManagementStrategy.doProvision(
Arrays.<RemoteTaskRunnerWorkItem>asList(
new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime())
new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime())
),
Arrays.<ZkWorker>asList(
new TestZkWorker(testTask)
......@@ -196,7 +196,7 @@ public class SimpleResourceManagementStrategyTest
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
Arrays.<RemoteTaskRunnerWorkItem>asList(
new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime())
new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime())
),
Arrays.<ZkWorker>asList(
new TestZkWorker(testTask)
......@@ -214,7 +214,7 @@ public class SimpleResourceManagementStrategyTest
provisionedSomething = simpleResourceManagementStrategy.doProvision(
Arrays.<RemoteTaskRunnerWorkItem>asList(
new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime())
new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime())
),
Arrays.<ZkWorker>asList(
new TestZkWorker(testTask)
......@@ -248,7 +248,7 @@ public class SimpleResourceManagementStrategyTest
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
Arrays.<RemoteTaskRunnerWorkItem>asList(
new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime())
new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime())
),
Arrays.<ZkWorker>asList(
new TestZkWorker(null)
......@@ -278,7 +278,7 @@ public class SimpleResourceManagementStrategyTest
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
Arrays.<RemoteTaskRunnerWorkItem>asList(
new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime())
new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime())
),
Arrays.<ZkWorker>asList(
new TestZkWorker(null)
......@@ -293,7 +293,7 @@ public class SimpleResourceManagementStrategyTest
terminatedSomething = simpleResourceManagementStrategy.doTerminate(
Arrays.<RemoteTaskRunnerWorkItem>asList(
new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime())
new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime())
),
Arrays.<ZkWorker>asList(
new TestZkWorker(null)
......
......@@ -39,7 +39,7 @@
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<metamx.java-util.version>0.25.0</metamx.java-util.version>
<metamx.java-util.version>0.25.1</metamx.java-util.version>
<apache.curator.version>2.1.0-incubating</apache.curator.version>
<druid.api.version>0.1.5</druid.api.version>
</properties>
......
......@@ -160,14 +160,11 @@ public class S3DataSegmentPuller implements DataSegmentPuller
}
);
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
}
catch (IOException e) {
catch (S3ServiceException | IOException e) {
throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords);
}
catch (S3ServiceException e) {
throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords);
catch (Exception e) {
throw Throwables.propagate(e);
}
}
......@@ -188,12 +185,12 @@ public class S3DataSegmentPuller implements DataSegmentPuller
);
return objDetails.getLastModifiedDate().getTime();
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
}
catch (S3ServiceException | IOException e) {
throw new SegmentLoadingException(e, e.getMessage());
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
private static class S3Coords
......
......@@ -138,7 +138,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher
catch (ServiceException e) {
throw new IOException(e);
}
catch (InterruptedException e) {
catch (Exception e) {
throw Throwables.propagate(e);
}
}
......
......@@ -19,15 +19,14 @@
package io.druid.storage.s3;
import com.google.common.base.Throwables;
import com.metamx.common.logger.Logger;
import com.google.common.base.Predicate;
import com.metamx.common.RetryUtils;
import org.jets3t.service.S3ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.model.S3Bucket;
import org.jets3t.service.model.S3Object;
import java.io.IOException;
import java.util.Random;
import java.util.concurrent.Callable;
/**
......@@ -35,8 +34,6 @@ import java.util.concurrent.Callable;
*/
public class S3Utils
{
private static final Logger log = new Logger(S3Utils.class);
public static void closeStreamsQuietly(S3Object s3Obj)
{
if (s3Obj == null) {
......@@ -55,68 +52,47 @@ public class S3Utils
* Retries S3 operations that fail due to io-related exceptions. Service-level exceptions (access denied, file not
* found, etc) are not retried.
*/
public static <T> T retryS3Operation(Callable<T> f) throws IOException, S3ServiceException, InterruptedException
public static <T> T retryS3Operation(Callable<T> f) throws Exception
{
int nTry = 0;
final int maxTries = 10;
while (true) {
try {
nTry++;
return f.call();
}
catch (IOException e) {
if (nTry <= maxTries) {
awaitNextRetry(e, nTry);
} else {
throw e;
}
}
catch (S3ServiceException e) {
if (nTry <= maxTries &&
(e.getCause() instanceof IOException ||
(e.getS3ErrorCode() != null && e.getS3ErrorCode().equals("RequestTimeout")))) {
awaitNextRetry(e, nTry);
final Predicate<Throwable> shouldRetry = new Predicate<Throwable>()
{
@Override
public boolean apply(Throwable e)
{
if (e instanceof IOException) {
return true;
} else if (e instanceof S3ServiceException) {
final boolean isIOException = e.getCause() instanceof IOException;
final boolean isTimeout = "RequestTimeout".equals(((S3ServiceException) e).getS3ErrorCode());
return isIOException || isTimeout;
} else {
throw e;
return false;
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
private static void awaitNextRetry(Exception e, int nTry) throws InterruptedException
{
final long baseSleepMillis = 1000;
final long maxSleepMillis = 60000;
final double fuzzyMultiplier = Math.min(Math.max(1 + 0.2 * new Random().nextGaussian(), 0), 2);
final long sleepMillis = (long) (Math.min(maxSleepMillis, baseSleepMillis * Math.pow(2, nTry)) * fuzzyMultiplier);
log.warn("S3 fail on try %d, retrying in %,dms.", nTry, sleepMillis);
Thread.sleep(sleepMillis);
};
final int maxTries = 10;
return RetryUtils.retry(f, shouldRetry, maxTries);
}
public static boolean isObjectInBucket(RestS3Service s3Client, String bucketName, String objectKey)
throws S3ServiceException
{
try {
s3Client.getObjectDetails(new S3Bucket(bucketName), objectKey);
try {
s3Client.getObjectDetails(new S3Bucket(bucketName), objectKey);
}
catch (S3ServiceException e) {
if (404 == e.getResponseCode()
|| "NoSuchKey".equals(e.getS3ErrorCode())
|| "NoSuchBucket".equals(e.getS3ErrorCode())) {
return false;
}
catch (S3ServiceException e) {
if (404 == e.getResponseCode()
|| "NoSuchKey".equals(e.getS3ErrorCode())
|| "NoSuchBucket".equals(e.getS3ErrorCode()))
{
return false;
}
if ("AccessDenied".equals(e.getS3ErrorCode()))
{
// Object is inaccessible to current user, but does exist.
return true;
}
// Something else has gone wrong
throw e;
if ("AccessDenied".equals(e.getS3ErrorCode())) {
// Object is inaccessible to current user, but does exist.
return true;
}
// Something else has gone wrong
throw e;
}
return true;
}
......
......@@ -103,7 +103,7 @@ public class HttpClientModule implements Module
private int numConnections = 5;
@JsonProperty
private Period readTimeout = null;
private Period readTimeout = new Period("PT5M");
public int getNumConnections()
{
......
......@@ -44,6 +44,7 @@ import io.druid.guice.PolyBind;
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.index.ChatHandlerProvider;
import io.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer;
import io.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer;
......@@ -54,11 +55,10 @@ import io.druid.indexing.overlord.IndexerDBCoordinator;
import io.druid.indexing.overlord.RemoteTaskRunnerFactory;
import io.druid.indexing.overlord.TaskLockbox;
import io.druid.indexing.overlord.TaskMaster;
import io.druid.indexing.overlord.TaskQueue;
import io.druid.indexing.overlord.TaskRunnerFactory;
import io.druid.indexing.overlord.TaskStorage;
import io.druid.indexing.overlord.TaskStorageQueryAdapter;
import io.druid.indexing.overlord.http.OldOverlordResource;
import io.druid.indexing.overlord.config.TaskQueueConfig;
import io.druid.indexing.overlord.http.OverlordRedirectInfo;
import io.druid.indexing.overlord.http.OverlordResource;
import io.druid.indexing.overlord.scaling.AutoScalingStrategy;
......@@ -115,6 +115,8 @@ public class CliOverlord extends ServerRunnable
@Override
public void configure(Binder binder)
{
JsonConfigProvider.bind(binder, "druid.indexer.queue", TaskQueueConfig.class);
binder.bind(TaskMaster.class).in(ManageLifecycle.class);
binder.bind(TaskLogStreamer.class).to(SwitchingTaskLogStreamer.class).in(LazySingleton.class);
......@@ -128,7 +130,6 @@ public class CliOverlord extends ServerRunnable
binder.bind(TaskActionClientFactory.class).to(LocalTaskActionClientFactory.class).in(LazySingleton.class);
binder.bind(TaskActionToolbox.class).in(LazySingleton.class);
binder.bind(TaskQueue.class).in(LazySingleton.class); // Lifecycle managed by TaskMaster instead
binder.bind(IndexerDBCoordinator.class).in(LazySingleton.class);
binder.bind(TaskLockbox.class).in(LazySingleton.class);
binder.bind(TaskStorageQueryAdapter.class).in(LazySingleton.class);
......@@ -147,7 +148,6 @@ public class CliOverlord extends ServerRunnable
binder.bind(JettyServerInitializer.class).toInstance(new OverlordJettyServerInitializer());
Jerseys.addResource(binder, OverlordResource.class);
Jerseys.addResource(binder, OldOverlordResource.class);
LifecycleModule.register(binder, Server.class);
}
......
......@@ -53,7 +53,6 @@ import io.druid.indexing.common.index.NoopChatHandlerProvider;
import io.druid.indexing.common.index.ServiceAnnouncingChatHandlerProvider;
import io.druid.indexing.overlord.HeapMemoryTaskStorage;
import io.druid.indexing.overlord.IndexerDBCoordinator;
import io.druid.indexing.overlord.TaskQueue;
import io.druid.indexing.overlord.TaskRunner;
import io.druid.indexing.overlord.TaskStorage;
import io.druid.indexing.overlord.ThreadPoolTaskRunner;
......@@ -174,7 +173,6 @@ public class CliPeon extends GuiceRunnable
.to(LocalTaskActionClientFactory.class).in(LazySingleton.class);
// all of these bindings are so that we can run the peon in local mode
binder.bind(TaskStorage.class).to(HeapMemoryTaskStorage.class).in(LazySingleton.class);
binder.bind(TaskQueue.class).in(LazySingleton.class);
binder.bind(TaskActionToolbox.class).in(LazySingleton.class);
binder.bind(IndexerDBCoordinator.class).in(LazySingleton.class);
taskActionBinder.addBinding("remote")
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册