diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index 51afb07ae65497336c810987468a548b46e491ac..697e362b827fea0da5f49e71c0f920827d691868 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -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. diff --git a/docs/content/Indexing-Service.md b/docs/content/Indexing-Service.md index aed200a7c8f40e3272b0eb9812ad3cf8417b5cf1..0d1cc38621e1d3fcdef0be8c175a06d70b2ba9c3 100644 --- a/docs/content/Indexing-Service.md +++ b/docs/content/Indexing-Service.md @@ -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: diff --git a/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java b/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java index 703179f45427def260bf6d5c1ae5592dc8a67ea4..ed9f628452ce3655bf643da20dccf65b7f02eb94 100644 --- a/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java +++ b/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java @@ -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); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java index 5730e3be082957c140eb64ce9a4d12fca5a0dc80..5d600dcd3694f01479bf0a0d7f54b063a6f905b7 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockAcquireAction.java @@ -68,7 +68,7 @@ public class LockAcquireAction implements TaskAction @Override public boolean isAudited() { - return true; + return false; } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockReleaseAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockReleaseAction.java index 97397666d2b8ff0f275c8035485f6dd81f19d2f3..6179c5ee658dfcdfea6a42ef38840e6af7379f50 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LockReleaseAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockReleaseAction.java @@ -60,7 +60,7 @@ public class LockReleaseAction implements TaskAction @Override public boolean isAudited() { - return true; + return false; } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/SpawnTasksAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java similarity index 64% rename from indexing-service/src/main/java/io/druid/indexing/common/actions/SpawnTasksAction.java rename to indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java index 85b1a53c275c2a7fa1cbafa067f4d75e0d849d8e..699460af82f0460530262c487bc5f6c66a73d6ff 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/SpawnTasksAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LockTryAcquireAction.java @@ -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 +public class LockTryAcquireAction implements TaskAction> { @JsonIgnore - private final List newTasks; + private final Interval interval; @JsonCreator - public SpawnTasksAction( - @JsonProperty("newTasks") List newTasks + public LockTryAcquireAction( + @JsonProperty("interval") Interval interval ) { - this.newTasks = ImmutableList.copyOf(newTasks); + this.interval = interval; } @JsonProperty - public List getNewTasks() + public Interval getInterval() { - return newTasks; + return interval; } - public TypeReference getReturnTypeReference() + public TypeReference> getReturnTypeReference() { - return new TypeReference() {}; + return new TypeReference>() + { + }; } @Override - public Void perform(Task task, TaskActionToolbox toolbox) + public Optional 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 + '}'; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java index 37d4346247e41b1b84fc3c111d4f5ab420b0400e..d9bdfe5b694ab008a81132c5503fd606c490ceb1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskAction.java @@ -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 { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java index b7e78e0c2be7ac86bffb347a8c1a2f3810550d67..a0b41e58a630582a145a40db2d3d07350b71ad08 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java @@ -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; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/config/EventReceiverFirehoseFactoryConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/config/EventReceiverFirehoseFactoryConfig.java deleted file mode 100644 index 5822bdb622a2fdbc7b1d8f7301d1306d756e88dd..0000000000000000000000000000000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/config/EventReceiverFirehoseFactoryConfig.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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(); -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/config/IndexerZkConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/config/IndexerZkConfig.java deleted file mode 100644 index 67a750ab535bafc838d6724677477ee08ee35518..0000000000000000000000000000000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/config/IndexerZkConfig.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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(); -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskLogConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskLogConfig.java deleted file mode 100644 index 787878e8eb805450538348c5c00598bd3002303f..0000000000000000000000000000000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskLogConfig.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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(); -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/EventReceiverFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/index/EventReceiverFirehoseFactory.java index 0f3732ed386c2c950527da31998bb14ae0d0bf49..b9e420f70200f0880be3ed58b5389859e589a390 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/EventReceiverFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/EventReceiverFirehoseFactory.java @@ -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; - @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() { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractFixedIntervalTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractFixedIntervalTask.java new file mode 100644 index 0000000000000000000000000000000000000000..d1d494f5c833ec61cd7826ab6d5ac5c9f8196ceb --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractFixedIntervalTask.java @@ -0,0 +1,75 @@ +/* + * 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.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.indexing.common.actions.LockTryAcquireAction; +import io.druid.indexing.common.actions.TaskActionClient; +import org.joda.time.Interval; + +public abstract class AbstractFixedIntervalTask extends AbstractTask +{ + @JsonIgnore + private final Interval interval; + + protected AbstractFixedIntervalTask( + String id, + String dataSource, + Interval interval + ) + { + this(id, id, new TaskResource(id, 1), dataSource, interval); + } + + 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; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java index 1944243e7fe772c4904eb5e44c1a2efc9a9c4060..eaff1b9b46f5bd35a69b973b8ec323cba316b23a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AbstractTask.java @@ -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; - - 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 getImplicitLockInterval() - { - return interval; - } - @Override public QueryRunner getQueryRunner(Query 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 getTaskLocks(TaskToolbox toolbox) throws IOException { - final List 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()); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java index 32d3e49e61838b2687e4bf8c8649bae2fb0237e9..872ac3507bd4ec2840f550114084748a9a64c1a6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/DeleteTask.java @@ -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(); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index e687875433b3904794d3b084579a3eba14212c6b..400b088a69303d37930bc5eea81b3925cbf3560f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -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() { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexDeterminePartitionsTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexDeterminePartitionsTask.java deleted file mode 100644 index a89cd475ff094e87d8b17dda0d161d4dd2db327b..0000000000000000000000000000000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexDeterminePartitionsTask.java +++ /dev/null @@ -1,299 +0,0 @@ -/* - * 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 unusableDimensions = Sets.newHashSet(); - - // Track values of all non-blacklisted dimensions - final Map> 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 dimValues = inputRow.getDimension(dim); - - if (!unusableDimensions.contains(dim)) { - - if (dimValues.size() == 1) { - - // Track this value - TreeMultiset 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 shardSpecs = Lists.newArrayList(); - - // Select highest-cardinality dimension - Ordering>> byCardinalityOrdering = new Ordering>>() - { - @Override - public int compare( - Map.Entry> left, - Map.Entry> 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> partitionEntry = - byCardinalityOrdering.max(dimensionValueMultisets.entrySet()); - - final String partitionDim = partitionEntry.getKey(); - final TreeMultiset 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 nextTasks = Lists.transform( - shardSpecs, - new Function() - { - @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; - } -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexGeneratorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexGeneratorTask.java deleted file mode 100644 index 84f6211ad524169ccd892e0aecba61a8c1ae2301..0000000000000000000000000000000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexGeneratorTask.java +++ /dev/null @@ -1,240 +0,0 @@ -/* - * 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 pushedSegments = new CopyOnWriteArrayList(); - 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; - } -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 6e7d9f61f17c2f2c33bf3a0482f3f8735701258e..19cb791b77e3fb211ce117238e7995fc14f21668 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -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 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 toSubtasks() + @Override + public String getType() { - final List 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 segments = Sets.newHashSet(); + for (final Interval bucket : granularitySpec.bucketIntervals()) { + final List 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.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 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 unusableDimensions = com.google.common.collect.Sets.newHashSet(); + // Track values of all non-blacklisted dimensions + final Map> 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 dimValues = inputRow.getDimension(dim); + if (!unusableDimensions.contains(dim)) { + if (dimValues.size() == 1) { + // Track this value + TreeMultiset 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 shardSpecs = Lists.newArrayList(); + + // Select highest-cardinality dimension + Ordering>> byCardinalityOrdering = new Ordering>>() + { + @Override + public int compare( + Map.Entry> left, + Map.Entry> 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> partitionEntry = + byCardinalityOrdering.max(dimensionValueMultisets.entrySet()); + + final String partitionDim = partitionEntry.getKey(); + final TreeMultiset 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 pushedSegments = new CopyOnWriteArrayList(); + 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 getSpatialDimensions() + { + return spatialDimensions; + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java index 8f4068a5e46adb97e3d5433911abacd5c6a810be..c77ddb21d96f71cc0ab5d8bc9a0f2cf71a922a82 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/KillTask.java @@ -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(), diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java index 750509f9cecd6abf6c59db7931a29b8e1dd42d6f..d49d74b355b2abc601cc29ff27077b5306d11bc5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java @@ -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 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 toIdentifier = new Function() { @Override @@ -199,7 +199,10 @@ public abstract class MergeTaskBase extends AbstractTask }; final Set current = ImmutableSet.copyOf( - Iterables.transform(taskActionClient.submit(defaultListUsedAction()), toIdentifier) + Iterables.transform( + taskActionClient.submit(new SegmentListUsedAction(getDataSource(), getInterval())), + toIdentifier + ) ); final Set 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(); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java index c6291fdb4c9e59dac5b01bbf60bab56c080dda14..001e1520eaaeee625127d5e58b3f6b2a995edd80 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/NoopTask.java @@ -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 { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index bb30c3517320f1e68990b516b7d176f1d2730d2e..f28d83c1cafa97feebfec7f7043f776edfcf7327 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -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 { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index a3e422328659a3387487617fe58e29bf70be38a4..4d6afd2ebf60c5ee21811d46c4f8557c26378155 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -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: *
    - *
  • Tasks must operate on a single datasource.
  • - *
  • Tasks should be immutable, since the task ID is used as a proxy for the task in many locations.
  • - *
  • Task IDs must be unique. This can be done by naming them using UUIDs or the current timestamp.
  • - *
  • 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.
  • - *
  • 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.
  • - *
  • Tasks do not need to explicitly release locks; they are released upon task completion. Tasks may choose - * to release locks early if they desire.
  • + *
  • Tasks must operate on a single datasource.
  • + *
  • Tasks should be immutable, since the task ID is used as a proxy for the task in many locations.
  • + *
  • Task IDs must be unique. This can be done by naming them using UUIDs or the current timestamp.
  • + *
  • 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.
  • + *
  • Tasks do not need to explicitly release locks; they are released upon task completion. Tasks may choose + * to release locks early if they desire.
  • *
*/ @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 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 QueryRunner getQueryRunner(Query 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 diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java index 5085bdbd2e3d5024564db55f36a0f729a590d14d..52807861a1219993a60b9659ffd0035e07368354 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java @@ -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 segments = taskActionClient.submit(defaultListUsedAction()); - - final FunctionalIterable tasks = FunctionalIterable - .create(segments) - .keep( - new Function() - { - @Override - public Task apply(DataSegment segment) + final List segments = toolbox.getTaskActionClient().submit( + new SegmentListUsedAction( + getDataSource(), + getInterval() + ) + ); + final FunctionalIterable tasks = FunctionalIterable + .create(segments) + .keep( + new Function() { - 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; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/DbTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/DbTaskStorage.java index 92e1043c8ebc7a4bf7ae47e9fcd3de3d7c13dfcc..7d3ad05512ef28a0cf2d0126dd40c7305978a353 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/DbTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/DbTaskStorage.java @@ -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() { @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() { @Override @@ -162,7 +173,7 @@ public class DbTaskStorage implements TaskStorage @Override public Optional getTask(final String taskid) { - return dbi.withHandle( + return retryingHandle( new HandleCallback>() { @Override @@ -192,7 +203,7 @@ public class DbTaskStorage implements TaskStorage @Override public Optional getStatus(final String taskid) { - return dbi.withHandle( + return retryingHandle( new HandleCallback>() { @Override @@ -222,7 +233,7 @@ public class DbTaskStorage implements TaskStorage @Override public List getActiveTasks() { - return dbi.withHandle( + return retryingHandle( new HandleCallback>() { @Override @@ -231,7 +242,7 @@ public class DbTaskStorage implements TaskStorage final List> 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() { @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() { @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() { @Override @@ -376,7 +387,7 @@ public class DbTaskStorage implements TaskStorage @Override public List getAuditLogs(final String taskid) { - return dbi.withHandle( + return retryingHandle( new HandleCallback>() { @Override @@ -392,21 +403,18 @@ public class DbTaskStorage implements TaskStorage .bind("task_id", taskid) .list(); - return Lists.transform( - dbTaskLogs, new Function, TaskAction>() - { - @Override - public TaskAction apply(Map row) - { - try { - return jsonMapper.readValue((byte[]) row.get("log_payload"), TaskAction.class); - } - catch (Exception e) { - throw Throwables.propagate(e); - } + final List retList = Lists.newArrayList(); + for (final Map 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 getLocksWithIds(final String taskid) { - return dbi.withHandle( + return retryingHandle( new HandleCallback>() { @Override @@ -439,4 +447,45 @@ public class DbTaskStorage implements TaskStorage } ); } + + /** + * Retry SQL operations + */ + private T retryingHandle(final HandleCallback callback) { + final Callable call = new Callable() + { + @Override + public T call() throws Exception + { + return dbi.withHandle(callback); + } + }; + final Predicate shouldRetry = new Predicate() + { + @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()))); + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index 759149b7bd45b18e71f56ff1a3f58ee141709234..fce401c66414bf200922a0e532ca898455f80562 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -101,12 +101,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer this.exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(workerConfig.getCapacity())); } - @Override - public void bootstrap(List tasks) - { - // do nothing - } - @Override public ListenableFuture 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() { @@ -358,6 +352,14 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer } } + @Override + public Collection getKnownTasks() + { + synchronized (tasks) { + return Lists.newArrayList(tasks.values()); + } + } + @Override public Collection getWorkers() { @@ -425,11 +427,11 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer private volatile ProcessHolder processHolder = null; private ForkingTaskRunnerWorkItem( - Task task, + String taskId, ListenableFuture statusFuture ) { - super(task, statusFuture); + super(taskId, statusFuture); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index be87cef72552b4d3fad0ffdbb2a4730b7da75b12..8b8234443191caefb54691ad01be7e31552d1605 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -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 zkWorkers = new ConcurrentHashMap(); + private final ConcurrentMap zkWorkers = new ConcurrentHashMap<>(); + // payloads of pending tasks, which we remember just long enough to assign to workers + private final ConcurrentMap 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() + { + @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 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 tasks) - { - try { - if (!started) { - throw new ISE("Must start RTR first before calling bootstrap!"); - } - - Map 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.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 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.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.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 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 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 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.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 result = taskRunnerWorkItem.getResult(); - if (result != null) { - ((SettableFuture) 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 result = taskRunnerWorkItem.getResult(); + if (result != null) { + ((SettableFuture) result).set(taskStatus); + } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java index 1c1dc7a17a99470430f5c9d7ea9c8579bf5a589a..76d373a049af67f75d6d4aab1062ea55148c1848 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java @@ -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 result, Worker worker ) { - super(task, result); + super(taskId, result); this.result = result; this.worker = worker; } public RemoteTaskRunnerWorkItem( - Task task, + String taskId, SettableFuture 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); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index 3dc024530b3fe8651c4b7cdf4cd69b3c4246aafd..d486f37c0fceaf7cb74bcea89f7b2f3cbd158531 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -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> 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> byVersionOrdering = new Ordering>() + { + @Override + public int compare(Pair left, Pair 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 uniqueTaskIds = Sets.newHashSet(); + int taskLockCount = 0; + for (final Pair taskAndLock : byVersionOrdering.sortedCopy(storedLocks)) { + final Task task = taskAndLock.lhs; + final TaskLock savedTaskLock = taskAndLock.rhs; + uniqueTaskIds.add(task.getId()); + final Optional 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 tryLock(final Task task, final Interval interval, final Optional preferredVersion) + private Optional tryLock(final Task task, final Interval interval, final Optional 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 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 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 dsRunning = running.get(task.getDataSource()); + if(dsRunning == null) { + searchSpace = ImmutableList.of(); } else { - // Scan through all locks for this datasource - final NavigableMap dsRunning = running.get(task.getDataSource()); - if(dsRunning == null) { - searchSpace = ImmutableList.of(); - } else { - searchSpace = dsRunning.values(); - } + searchSpace = dsRunning.values(); } return ImmutableList.copyOf( diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java index 5d5cba4f200d26a95ee88e40473fe9c836fe43f5..96183d5ae64280146c06dbd7dce59bc0957d5ac8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java @@ -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 leaderLifecycleRef = new AtomicReference(null); + private final AtomicReference 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()); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java index a78aef84f6e1ae5a2ec9f39dbbf2bfafdae56271..77e4b26e372209f09e6f39eab3f5a6b3e156a5d0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java @@ -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. *

- * 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). *

- * 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. - *

- * For persistence, the queue saves new tasks from {@link #add} and task status updates from {@link #notify} using a - * {@link TaskStorage} obj - *

- * 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 queue = Lists.newLinkedList(); + private final List tasks = Lists.newArrayList(); + private final Map> 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 tasksByLock = ArrayListMultimap.create(); - - for (final Task task : taskStorage.getActiveTasks()) { - try { - final List 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> byVersionOrdering = new Ordering>() - { - @Override - public int compare(Map.Entry left, Map.Entry right) - { - return left.getKey().getVersion().compareTo(right.getKey().getVersion()); - } - }; - - // Acquire as many locks as possible, in version order - for(final Map.Entry taskAndLock : byVersionOrdering.sortedCopy(tasksByLock.entries())) { - final Task task = taskAndLock.getValue(); - final TaskLock savedTaskLock = taskAndLock.getKey(); - - final Optional 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() + { + @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 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> 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 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 tasksToKill = Sets.difference( + runnerTaskFutures.keySet(), + ImmutableSet.copyOf( + Lists.transform( + tasks, + new Function() + { + @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 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 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 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 attachCallbacks(final Task task, final ListenableFuture statusFuture) + { + final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() + .setUser2(task.getDataSource()) + .setUser4(task.getType()); + Futures.addCallback( + statusFuture, + new FutureCallback() + { + @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 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(); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java index 6509c975cdfab2ba095d64d74df52975bb6828fd..0b4b5e3ff896cd784f6198a1c3c1e2a8f203a1c2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java @@ -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 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 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 getPendingTasks(); + public Collection getKnownTasks(); + public Collection getWorkers(); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java index 4d4cac6ef70995239e3a1d29b0597c5c3e53cdef..a78faa24d030ce5e4ac980c95c438c93508afa82 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java @@ -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 { - private final Task task; + private final String taskId; private final ListenableFuture result; private final DateTime createdTime; private final DateTime queueInsertionTime; public TaskRunnerWorkItem( - Task task, + String taskId, ListenableFuture result ) { - this(task, result, new DateTime(), new DateTime()); + this(taskId, result, new DateTime(), new DateTime()); } public TaskRunnerWorkItem( - Task task, + String taskId, ListenableFuture 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 getResult() @@ -69,13 +66,11 @@ public class TaskRunnerWorkItem implements Comparable return result; } - @JsonProperty public DateTime getCreatedTime() { return createdTime; } - @JsonProperty public DateTime getQueueInsertionTime() { return queueInsertionTime; @@ -83,7 +78,7 @@ public class TaskRunnerWorkItem implements Comparable 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 { 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 public String toString() { return "TaskRunnerWorkItem{" + - "task=" + task + + "taskId='" + taskId + '\'' + ", result=" + result + ", createdTime=" + createdTime + + ", queueInsertionTime=" + queueInsertionTime + '}'; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java index 3e3cbf46cdc63336e0192b479f76bb7096837d70..3a2145627dfadce4ea6ff6c67abac831a314ef59 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java @@ -77,7 +77,8 @@ public interface TaskStorage public List 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 getActiveTasks(); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java index db03ab67ff79c492b3d5a88996455570d14b98cf..e9a2a8d5d7c8d77cbfc06defe04fec27ccce97cc 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -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> getSameGroupChildStatuses(final String taskid) - { - final Optional taskOptional = storage.getTask(taskid); - final Optional statusOptional = storage.getStatus(taskid); - final ImmutableMap.Builder> resultBuilder = ImmutableMap.builder(); - - resultBuilder.put(taskid, statusOptional); - - final Iterable nextTasks = FunctionalIterable - .create(storage.getAuditLogs(taskid)).filter( - new Predicate() - { - @Override - public boolean apply(TaskAction taskAction) - { - return taskAction instanceof SpawnTasksAction; - } - } - ).transformCat( - new Function>() - { - @Override - public Iterable 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 getSameGroupMergedStatus(final String taskid) - { - final Map> statuses = getSameGroupChildStatuses(taskid); - - int nSuccesses = 0; - int nFailures = 0; - int nTotal = 0; - int nPresent = 0; - - for(final Optional statusOption : statuses.values()) { - nTotal ++; - - if(statusOption.isPresent()) { - nPresent ++; - - final TaskStatus status = statusOption.get(); - - if(status.isSuccess()) { - nSuccesses ++; - } else if(status.isFailure()) { - nFailures ++; - } - } - } - - final Optional 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 getSameGroupNewSegments(final String taskid) + public Set getInsertedSegments(final String taskid) { - final Optional taskOptional = storage.getTask(taskid); final Set segments = Sets.newHashSet(); - final List 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; } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java index 78e4b9e30ec4219c107e4c3a75fb008f16b6e73d..2cc94ac740052382abf116664177f2c0156ef5e3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java @@ -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 runningItems = new ConcurrentSkipListSet(); + private final Set 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 tasks) - { - // do nothing - } - @Override public ListenableFuture run(final Task task) { final TaskToolbox toolbox = toolboxFactory.build(task); - final ListenableFuture statusFuture = exec.submit(new ExecutorServiceTaskRunnerCallable(task, toolbox)); - - final TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem(task, statusFuture); + final ListenableFuture statusFuture = exec.submit(new ThreadPoolTaskRunnerCallable(task, toolbox)); + final ThreadPoolTaskRunnerWorkItem taskRunnerWorkItem = new ThreadPoolTaskRunnerWorkItem(task, statusFuture); runningItems.add(taskRunnerWorkItem); Futures.addCallback( statusFuture, new FutureCallback() @@ -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 getRunningTasks() { - return ImmutableList.copyOf(runningItems); + return ImmutableList.copyOf(runningItems); } @Override @@ -133,6 +125,12 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker return ImmutableList.of(); } + @Override + public Collection getKnownTasks() + { + return ImmutableList.copyOf(runningItems); + } + @Override public Collection getWorkers() { @@ -155,18 +153,8 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker { QueryRunner queryRunner = null; - final List runningTasks = Lists.transform( - ImmutableList.copyOf(getRunningTasks()), new Function() - { - @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 taskQueryRunner = task.getQueryRunner(query); @@ -185,12 +173,31 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker return queryRunner == null ? new NoopQueryRunner() : queryRunner; } - private static class ExecutorServiceTaskRunnerCallable implements Callable + private static class ThreadPoolTaskRunnerWorkItem extends TaskRunnerWorkItem + { + private final Task task; + + private ThreadPoolTaskRunnerWorkItem( + Task task, + ListenableFuture result + ) + { + super(task.getId(), result); + this.task = task; + } + + public Task getTask() + { + return task; + } + } + + private static class ThreadPoolTaskRunnerCallable implements Callable { 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); - } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java index edb0c3df68516584b2104427dceb0dad909d74f7..335b5fa583dbbb2109b7f10a732c5f0e4ff5bbd7 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java @@ -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) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/config/IndexerDbConnectorConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/config/IndexerDbConnectorConfig.java deleted file mode 100644 index 3318975c7caed92939094d2ac8b3f53123643405..0000000000000000000000000000000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/config/IndexerDbConnectorConfig.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.config; - -import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.db.DbConnectorConfig; -import org.skife.config.Config; - -public abstract class IndexerDbConnectorConfig extends DbConnectorConfig -{ - @JsonProperty("taskTable") - @Config("druid.database.taskTable") - public abstract String getTaskTable(); - - @JsonProperty("taskLockTable") - @Config("druid.database.taskLockTable") - public abstract String getTaskLockTable(); - - @JsonProperty("taskLogTable") - @Config("druid.database.taskLogTable") - public abstract String getTaskLogTable(); -} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/config/TaskQueueConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/config/TaskQueueConfig.java new file mode 100644 index 0000000000000000000000000000000000000000..ac46f2cc60fb72a1b9b1ba204a6ee62d0d26228f --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/config/TaskQueueConfig.java @@ -0,0 +1,79 @@ +/* + * 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.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Duration; +import org.joda.time.Period; + +public class TaskQueueConfig +{ + @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) + { + return (period == null ? new Period(theDefault) : period).toStandardDuration(); + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/exec/TaskConsumer.java b/indexing-service/src/main/java/io/druid/indexing/overlord/exec/TaskConsumer.java deleted file mode 100644 index d75cad14f08e879924f4942d93789d667d3b936a..0000000000000000000000000000000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/exec/TaskConsumer.java +++ /dev/null @@ -1,204 +0,0 @@ -/* - * 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 status = runner.run(task); - - Futures.addCallback( - status, new FutureCallback() - { - @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(); - } - } - } - ); - } -} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OldOverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OldOverlordResource.java deleted file mode 100644 index 6897490624acb77540f0c0f11438c07a1bd61278..0000000000000000000000000000000000000000 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OldOverlordResource.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.http; - -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 javax.ws.rs.Path; - -/** - */ -@Deprecated -@Path("/mmx/merger/v1") -public class OldOverlordResource extends OverlordResource -{ - @Inject - public OldOverlordResource( - TaskMaster taskMaster, - TaskStorageQueryAdapter taskStorageQueryAdapter, - TaskLogStreamer taskLogStreamer, - JacksonConfigManager configManager - ) throws Exception - { - super(taskMaster, taskStorageQueryAdapter, taskLogStreamer, configManager); - } -} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index ef195b6f6ccae39e713507315df81df17fb090fc..d3d58bef03d6645385c8ad5514cf709835db419a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -70,14 +70,7 @@ public class OverlordResource public Map apply(TaskRunnerWorkItem input) { return new ImmutableMap.Builder() - .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 segments = taskStorageQueryAdapter.getSameGroupNewSegments(taskid); + final Set 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() + taskMaster.getTaskQueue(), + new Function() { @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 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 { diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java index bc8879b6960a9f63f96831142f0c2d74ce43f54f..17d889b5143d13b8bd46c22dc23ea1afc06e49bd 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java @@ -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() - { - @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() + { + @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); + } + } + } ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java index 577d9f4e0a6c5ba61a56127d75eba821e676f335..7e8dd86f5a0a88970ebb6e6355e44eec9dc45fe6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java @@ -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 diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 36946e196d7933bee54bfa7ed35e320238544294..485b3202b5463cfeb183eb8efd8a288e46cf1032 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -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.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.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()); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java index b6f93f047043294eede347be3b2b5304584b9cda..e4ec6d54aa111359fde81f5303472a161b33f884 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java @@ -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 { diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java index f368e69758b59674629f1030c5bfaf1e7c747af1..fcf9715fe62a798c4022b6f530ecaa641b8d17a0 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -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 existingTasks = Sets.newHashSet(); + final Set 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.asList(TestMergeTask.createDummyTask("second"))); - - Set runningTasks = Sets.newHashSet( + final Set runningTasks = Sets.newHashSet( Iterables.transform( remoteTaskRunner.getRunningTasks(), new Function() @@ -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.asList(task)); - ListenableFuture future = remoteTaskRunner.run(task); TaskStatus status = future.get(); @@ -356,7 +345,6 @@ public class RemoteTaskRunnerTest public void testWorkerRemoved() throws Exception { doSetup(); - remoteTaskRunner.bootstrap(Lists.newArrayList()); Future future = remoteTaskRunner.run(task); Assert.assertTrue(taskAnnounced(task.getId())); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 0b3a4e9ed97dbca34fba9b4eafffcc029ac8cc6c..ac90c1e12f008298c9d72cfd9a6583ea869358e2 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -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 byIntervalOrdering = new Ordering() @@ -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 preRunTaskStatus = tsqa.getSameGroupMergedStatus(indexTask.getId()); + final Optional 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 publishedSegments = byIntervalOrdering.sortedCopy(mdc.getPublished()); - final List loggedSegments = byIntervalOrdering.sortedCopy(tsqa.getSameGroupNewSegments(indexTask.getId())); + final List 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()); } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskQueueTest.java deleted file mode 100644 index 2b1d0e560c111b28b7258e85221dba4cbbf29bd5..0000000000000000000000000000000000000000 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskQueueTest.java +++ /dev/null @@ -1,438 +0,0 @@ -/* - * 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 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.of( - "local", - new LocalDataSegmentPuller() - ), - null, - new SegmentLoaderConfig() - { - @Override - public List 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.of( - "local", - new LocalDataSegmentPuller() - ), - null, - new SegmentLoaderConfig() - { - @Override - public List 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 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 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); - } - }; - } -} diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java index 02ac9a21778ec80a70ffb95ee053da6c2b47f9a6..1f3f4a44eee000dcc9826492f4635d7ad97929da 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java @@ -111,7 +111,7 @@ public class SimpleResourceManagementStrategyTest boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -139,7 +139,7 @@ public class SimpleResourceManagementStrategyTest boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -155,7 +155,7 @@ public class SimpleResourceManagementStrategyTest provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -196,7 +196,7 @@ public class SimpleResourceManagementStrategyTest boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -214,7 +214,7 @@ public class SimpleResourceManagementStrategyTest provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -248,7 +248,7 @@ public class SimpleResourceManagementStrategyTest boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -278,7 +278,7 @@ public class SimpleResourceManagementStrategyTest boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -293,7 +293,7 @@ public class SimpleResourceManagementStrategyTest terminatedSomething = simpleResourceManagementStrategy.doTerminate( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) diff --git a/pom.xml b/pom.xml index 702f796bff1ba4f317371a17ac07f7ad8e5934ac..db67db72a7573dcdff3e7303dce61824dbb97de1 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ UTF-8 - 0.25.0 + 0.25.1 2.1.0-incubating 0.1.5 diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java index ff286e262acb2894c279bb0e3aa76d8677b5b595..1e310a9b90f2259ce68665f018ec0709cafb8bad 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java @@ -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 diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java index 735a93656285c7a00e29b1a601af895b3a9b57e1..e8b1a99710f692b732353d8a1795049d2ac8a3cc 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java @@ -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); } } diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java index 598153ec778a76c6e79b10603a920fe919f5a6c4..619ad737cfe9af98b6ff53d9ba418cb265de8673 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java @@ -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 retryS3Operation(Callable f) throws IOException, S3ServiceException, InterruptedException + public static T retryS3Operation(Callable 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 shouldRetry = new Predicate() + { + @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; } diff --git a/server/src/main/java/io/druid/guice/HttpClientModule.java b/server/src/main/java/io/druid/guice/HttpClientModule.java index 8bf7c70b6980192cbbf62dab72f57813d8cc0cff..700aff7baee66235f1ccb3b50b9f3cf69249a1ec 100644 --- a/server/src/main/java/io/druid/guice/HttpClientModule.java +++ b/server/src/main/java/io/druid/guice/HttpClientModule.java @@ -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() { diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 3e1313c51ba96fd724c0f89dfa6b327e255a0a73..18609f7aea69f3d7e9496669b1ee369d3ac08993 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -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); } diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index db60016fc4a97aa638059dff102d66f136fa4e11..e8a5b985bd4cce92810be9f8b6cc1ee16f377de2 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -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")