From a6787ab2016c3b20931bc5f0b260774ad23af7be Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 27 Jun 2013 16:40:17 -0700 Subject: [PATCH 01/28] fix up configs --- .../druid/client/DruidServerConfig.java | 1 + .../indexing/common/config/TaskConfig.java | 2 + .../indexing/common/config/TaskLogConfig.java | 5 + .../indexing/common/index/ChatHandler.java | 21 +++- .../common/index/ChatHandlerProvider.java | 96 ++++------------ .../index/EventReceiverFirehoseFactory.java | 25 ++++- .../EventReceivingChatHandlerProvider.java | 105 ++++++++++++++++++ .../common/index/NoopChatHandlerProvider.java | 45 ++++++++ .../config/ForkingTaskRunnerConfig.java | 1 + .../config/IndexerCoordinatorConfig.java | 2 +- .../http/IndexerCoordinatorNode.java | 7 +- .../indexing/worker/config/WorkerConfig.java | 4 - .../worker/executor/ChatHandlerResource.java | 6 +- .../worker/executor/ExecutorNode.java | 59 ++++++---- .../indexing/worker/http/WorkerNode.java | 2 +- pom.xml | 2 +- .../druid/metrics/NoopServiceEmitter.java | 3 +- 17 files changed, 272 insertions(+), 114 deletions(-) create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/common/index/NoopChatHandlerProvider.java diff --git a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java index 89028021c18..6ad178b5622 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java @@ -33,6 +33,7 @@ public abstract class DruidServerConfig public abstract String getHost(); @Config("druid.server.maxSize") + @Default("0") public abstract long getMaxSize(); @Config("druid.server.tier") diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java index 9de8bcc9302..8f7b20baee5 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java @@ -27,6 +27,7 @@ import java.io.File; public abstract class TaskConfig { @Config("druid.indexer.taskDir") + @Default("/tmp/persistent/task") public abstract File getBaseTaskDir(); @Config("druid.indexer.rowFlushBoundary") @@ -34,5 +35,6 @@ public abstract class TaskConfig public abstract int getDefaultRowFlushBoundary(); @Config("druid.indexer.hadoopWorkingPath") + @Default("/tmp/druid-indexing") public abstract String getHadoopWorkingPath(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskLogConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskLogConfig.java index a80fdcd76c8..59030c99807 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskLogConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskLogConfig.java @@ -1,10 +1,15 @@ package com.metamx.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(); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandler.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandler.java index 3458c931de4..95c66028e52 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandler.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandler.java @@ -1,7 +1,26 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 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 com.metamx.druid.indexing.common.index; /** - * Objects that can be registered with a {@link ChatHandlerProvider} and provide http endpoints for indexing-related + * Objects that can be registered with a {@link EventReceivingChatHandlerProvider} and provide http endpoints for indexing-related * objects. This interface is empty because it only exists to signal intent. The actual http endpoints are provided * through JAX-RS annotations on the {@link ChatHandler} objects. */ diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java index 6d67a7a5fb1..1662abc7d9b 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java @@ -1,83 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 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 com.metamx.druid.indexing.common.index; import com.google.common.base.Optional; -import com.google.common.collect.Maps; -import com.metamx.common.ISE; -import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; - -import java.util.concurrent.ConcurrentMap; /** - * Provides a way for the outside world to talk to objects in the indexing service. The {@link #get(String)} method - * allows anyone with a reference to this object to obtain a particular {@link ChatHandler}. An embedded - * {@link ServiceAnnouncer} will be used to advertise handlers on this host. */ -public class ChatHandlerProvider +public interface ChatHandlerProvider { - private static final Logger log = new Logger(ChatHandlerProvider.class); + public void register(final String key, ChatHandler handler); - private final ChatHandlerProviderConfig config; - private final ServiceAnnouncer serviceAnnouncer; - private final ConcurrentMap handlers; + public void unregister(final String key); - public ChatHandlerProvider( - ChatHandlerProviderConfig config, - ServiceAnnouncer serviceAnnouncer - ) - { - this.config = config; - this.serviceAnnouncer = serviceAnnouncer; - this.handlers = Maps.newConcurrentMap(); - } - - public void register(final String key, ChatHandler handler) - { - final String service = serviceName(key); - log.info("Registering Eventhandler: %s", key); - - if (handlers.putIfAbsent(key, handler) != null) { - throw new ISE("handler already registered for key: %s", key); - } - - try { - serviceAnnouncer.announce(service); - } - catch (Exception e) { - log.warn(e, "Failed to register service: %s", service); - handlers.remove(key, handler); - } - } - - public void unregister(final String key) - { - final String service = serviceName(key); - - log.info("Unregistering chat handler: %s", key); - - final ChatHandler handler = handlers.get(key); - if (handler == null) { - log.warn("handler not currently registered, ignoring: %s", key); - } - - try { - serviceAnnouncer.unannounce(service); - } - catch (Exception e) { - log.warn(e, "Failed to unregister service: %s", service); - } - - handlers.remove(key, handler); - } - - public Optional get(final String key) - { - return Optional.fromNullable(handlers.get(key)); - } - - private String serviceName(String key) - { - return String.format(config.getServiceFormat(), key); - } + public Optional get(final String key); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiverFirehoseFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiverFirehoseFactory.java index 5c437eaa356..1c235cf43c9 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiverFirehoseFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiverFirehoseFactory.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 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 com.metamx.druid.indexing.common.index; import com.fasterxml.jackson.annotation.JacksonInject; @@ -30,7 +49,7 @@ import java.util.concurrent.TimeUnit; /** * Builds firehoses that accept events through the {@link EventReceiver} interface. Can also register these - * firehoses with an {@link ChatHandlerProvider}. + * firehoses with an {@link EventReceivingChatHandlerProvider}. */ @JsonTypeName("receiver") public class EventReceiverFirehoseFactory implements FirehoseFactory @@ -41,14 +60,14 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory private final String firehoseId; private final int bufferSize; private final MapInputRowParser parser; - private final Optional chatHandlerProvider; + private final Optional chatHandlerProvider; @JsonCreator public EventReceiverFirehoseFactory( @JsonProperty("firehoseId") String firehoseId, @JsonProperty("bufferSize") Integer bufferSize, @JsonProperty("parser") MapInputRowParser parser, - @JacksonInject("chatHandlerProvider") ChatHandlerProvider chatHandlerProvider + @JacksonInject("chatHandlerProvider") EventReceivingChatHandlerProvider chatHandlerProvider ) { this.firehoseId = Preconditions.checkNotNull(firehoseId, "firehoseId"); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java new file mode 100644 index 00000000000..f18a79c241c --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java @@ -0,0 +1,105 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 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 com.metamx.druid.indexing.common.index; + +import com.google.common.base.Optional; +import com.google.common.collect.Maps; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import com.metamx.druid.curator.discovery.ServiceAnnouncer; +import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; + +import java.util.concurrent.ConcurrentMap; + +/** + * Provides a way for the outside world to talk to objects in the indexing service. The {@link #get(String)} method + * allows anyone with a reference to this object to obtain a particular {@link ChatHandler}. An embedded + * {@link ServiceAnnouncer} will be used to advertise handlers on this host. + */ +public class EventReceivingChatHandlerProvider implements ChatHandlerProvider +{ + private static final Logger log = new Logger(EventReceivingChatHandlerProvider.class); + + private final ChatHandlerProviderConfig config; + private final ServiceAnnouncer serviceAnnouncer; + private final ConcurrentMap handlers; + + public EventReceivingChatHandlerProvider( + ChatHandlerProviderConfig config, + ServiceAnnouncer serviceAnnouncer + ) + { + this.config = config; + this.serviceAnnouncer = serviceAnnouncer; + this.handlers = Maps.newConcurrentMap(); + } + + @Override + public void register(final String key, ChatHandler handler) + { + final String service = serviceName(key); + log.info("Registering Eventhandler: %s", key); + + if (handlers.putIfAbsent(key, handler) != null) { + throw new ISE("handler already registered for key: %s", key); + } + + try { + serviceAnnouncer.announce(service); + } + catch (Exception e) { + log.warn(e, "Failed to register service: %s", service); + handlers.remove(key, handler); + } + } + + @Override + public void unregister(final String key) + { + final String service = serviceName(key); + + log.info("Unregistering chat handler: %s", key); + + final ChatHandler handler = handlers.get(key); + if (handler == null) { + log.warn("handler not currently registered, ignoring: %s", key); + } + + try { + serviceAnnouncer.unannounce(service); + } + catch (Exception e) { + log.warn(e, "Failed to unregister service: %s", service); + } + + handlers.remove(key, handler); + } + + @Override + public Optional get(final String key) + { + return Optional.fromNullable(handlers.get(key)); + } + + private String serviceName(String key) + { + return String.format(config.getServiceFormat(), key); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/NoopChatHandlerProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/NoopChatHandlerProvider.java new file mode 100644 index 00000000000..bf9a36afdb5 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/NoopChatHandlerProvider.java @@ -0,0 +1,45 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 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 com.metamx.druid.indexing.common.index; + +import com.google.common.base.Optional; + +/** + */ +public class NoopChatHandlerProvider implements ChatHandlerProvider +{ + @Override + public void register(String key, ChatHandler handler) + { + // do nothing + } + + @Override + public void unregister(String key) + { + // do nothing + } + + @Override + public Optional get(String key) + { + return Optional.absent(); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java index 4ff499dbd92..25233ed3300 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java @@ -13,6 +13,7 @@ import java.util.Set; public abstract class ForkingTaskRunnerConfig { @Config("druid.indexer.taskDir") + @Default("/tmp/persistent") public abstract File getBaseTaskDir(); @Config("druid.indexer.fork.java") diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java index 9c3baf06c24..f8d2cf8c3dc 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java @@ -41,7 +41,7 @@ public abstract class IndexerCoordinatorConfig extends ZkPathsConfig public abstract int getNumLocalThreads(); @Config("druid.indexer.runner") - @Default("remote") + @Default("local") public abstract String getRunnerImpl(); @Config("druid.indexer.storage") diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index b07ae5f1792..f850ee5f5f3 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -34,6 +34,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; +import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; @@ -386,15 +387,17 @@ public class IndexerCoordinatorNode extends QueryableNode private void initializeS3Service() throws S3ServiceException { - if(s3Service == null) { + if (s3Service == null) { s3Service = new RestS3Service( new AWSCredentials( PropUtils.getProperty(props, "com.metamx.aws.accessKey"), @@ -430,17 +432,15 @@ public class ExecutorNode extends BaseServerNode { if (chatHandlerProvider == null) { final ChatHandlerProviderConfig config = configFactory.build(ChatHandlerProviderConfig.class); - final ServiceAnnouncer myServiceAnnouncer; if (config.getServiceFormat() == null) { - log.info("ChatHandlerProvider: Using NoopServiceAnnouncer. Good luck finding your firehoses!"); - myServiceAnnouncer = new NoopServiceAnnouncer(); + log.info("ChatHandlerProvider: Using NoopChatHandlerProvider. Good luck finding your firehoses!"); + this.chatHandlerProvider = new NoopChatHandlerProvider(); } else { - myServiceAnnouncer = serviceAnnouncer; + this.chatHandlerProvider = new EventReceivingChatHandlerProvider( + config, + serviceAnnouncer + ); } - this.chatHandlerProvider = new ChatHandlerProvider( - config, - myServiceAnnouncer - ); } } @@ -482,9 +482,12 @@ public class ExecutorNode extends BaseServerNode jsonMapper = new DefaultObjectMapper(); smileMapper = new DefaultObjectMapper(new SmileFactory()); smileMapper.getJsonFactory().setCodec(smileMapper); - } - else if (jsonMapper == null || smileMapper == null) { - throw new ISE("Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", jsonMapper, smileMapper); + } else if (jsonMapper == null || smileMapper == null) { + throw new ISE( + "Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", + jsonMapper, + smileMapper + ); } if (lifecycle == null) { @@ -499,7 +502,15 @@ public class ExecutorNode extends BaseServerNode configFactory = Config.createFactory(props); } - return new ExecutorNode(nodeType, props, lifecycle, jsonMapper, smileMapper, configFactory, executorLifecycleFactory); + return new ExecutorNode( + nodeType, + props, + lifecycle, + jsonMapper, + smileMapper, + configFactory, + executorLifecycleFactory + ); } } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java index 1ccf32e0795..4bd9cb8cb51 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java @@ -392,7 +392,7 @@ public class WorkerNode extends QueryableNode public void initializeWorkerTaskMonitor() { if (workerTaskMonitor == null) { - final ExecutorService workerExec = Executors.newFixedThreadPool(workerConfig.getNumThreads()); + final ExecutorService workerExec = Executors.newFixedThreadPool(workerConfig.getCapacity()); final CuratorFramework curatorFramework = getCuratorFramework(); final PathChildrenCache pathChildrenCache = new PathChildrenCache( diff --git a/pom.xml b/pom.xml index 83dcd517782..743fe894033 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ UTF-8 - 0.22.3 + 0.22.4 2.0.2-21-22 diff --git a/server/src/test/java/com/metamx/druid/metrics/NoopServiceEmitter.java b/server/src/test/java/com/metamx/druid/metrics/NoopServiceEmitter.java index 54c158495f9..f980bc4885c 100644 --- a/server/src/test/java/com/metamx/druid/metrics/NoopServiceEmitter.java +++ b/server/src/test/java/com/metamx/druid/metrics/NoopServiceEmitter.java @@ -22,7 +22,8 @@ package com.metamx.druid.metrics; import com.metamx.emitter.core.Event; import com.metamx.emitter.service.ServiceEmitter; -public class NoopServiceEmitter extends ServiceEmitter +public class + NoopServiceEmitter extends ServiceEmitter { public NoopServiceEmitter() { From b9578a1adab0901e69b725a5039efff11bd6f06d Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 12 Jul 2013 12:51:12 -0700 Subject: [PATCH 02/28] 1) remove retry logic from RTR 2) simplify configs 3) introduce task resource 4) make worker versions match coordinator version by default --- .../com/metamx/druid/client/DruidServer.java | 1 + .../druid/client/DruidServerConfig.java | 2 +- .../indexing/common/task/AbstractTask.java | 31 +- .../task/IndexDeterminePartitionsTask.java | 1 - .../common/task/RealtimeIndexTask.java | 14 +- .../druid/indexing/common/task/Task.java | 7 +- .../indexing/common/task/TaskResource.java | 39 ++ .../common/task/VersionConverterTask.java | 9 +- .../coordinator/ForkingTaskRunner.java | 6 + .../coordinator/RemoteTaskRunner.java | 367 ++++++++---------- .../indexing/coordinator/TaskRunner.java | 3 + .../coordinator/TaskRunnerWorkItem.java | 9 - .../coordinator/ThreadPoolTaskRunner.java | 9 +- .../druid/indexing/coordinator/ZkWorker.java | 94 +++-- .../config/EC2AutoScalingStrategyConfig.java | 5 + .../http/IndexerCoordinatorNode.java | 17 - .../scaling/EC2AutoScalingStrategy.java | 8 +- .../coordinator/setup/GalaxyUserData.java | 5 + .../indexing/coordinator/RealtimeishTask.java | 7 +- .../coordinator/RemoteTaskRunnerTest.java | 74 ++-- .../coordinator/TaskLifecycleTest.java | 7 +- .../indexing/coordinator/TaskQueueTest.java | 4 +- .../coordinator/TestRealtimeTask.java | 5 +- .../scaling/EC2AutoScalingStrategyTest.java | 6 + .../SimpleResourceManagementStrategyTest.java | 23 +- pom.xml | 2 +- 26 files changed, 413 insertions(+), 342 deletions(-) create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskResource.java diff --git a/client/src/main/java/com/metamx/druid/client/DruidServer.java b/client/src/main/java/com/metamx/druid/client/DruidServer.java index ebd078e9383..e128a391ca8 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServer.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServer.java @@ -33,6 +33,7 @@ import java.util.concurrent.ConcurrentMap; */ public class DruidServer implements Comparable { + public static final String DEFAULT_TIER = "_default_tier"; private static final Logger log = new Logger(DruidServer.class); private final Object lock = new Object(); diff --git a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java index 6ad178b5622..d66ef146db4 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java @@ -37,6 +37,6 @@ public abstract class DruidServerConfig public abstract long getMaxSize(); @Config("druid.server.tier") - @Default("_default_tier") + @Default(DruidServer.DEFAULT_TIER) public abstract String getTier(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java index 07d11582982..b240d55e6db 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java @@ -43,7 +43,7 @@ public abstract class AbstractTask implements Task private final String groupId; @JsonIgnore - private final String availabilityGroup; + private final TaskResource taskResource; @JsonIgnore private final String dataSource; @@ -53,23 +53,23 @@ public abstract class AbstractTask implements Task protected AbstractTask(String id, String dataSource, Interval interval) { - this(id, id, id, dataSource, interval); + this(id, id, new TaskResource(id, 1), dataSource, interval); } protected AbstractTask(String id, String groupId, String dataSource, Interval interval) { this.id = Preconditions.checkNotNull(id, "id"); this.groupId = Preconditions.checkNotNull(groupId, "groupId"); - this.availabilityGroup = id; + this.taskResource = new TaskResource(id, 1); this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); this.interval = Optional.fromNullable(interval); } - protected AbstractTask(String id, String groupId, String availabilityGroup, String dataSource, Interval interval) + protected AbstractTask(String id, String groupId, TaskResource taskResource, String dataSource, Interval interval) { this.id = Preconditions.checkNotNull(id, "id"); this.groupId = Preconditions.checkNotNull(groupId, "groupId"); - this.availabilityGroup = Preconditions.checkNotNull(availabilityGroup, "availabilityGroup"); + this.taskResource = Preconditions.checkNotNull(taskResource, "taskResource"); this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); this.interval = Optional.fromNullable(interval); } @@ -90,9 +90,9 @@ public abstract class AbstractTask implements Task @JsonProperty @Override - public String getAvailabilityGroup() + public TaskResource getTaskResource() { - return availabilityGroup; + return taskResource; } @Override @@ -172,19 +172,16 @@ public abstract class AbstractTask implements Task AbstractTask that = (AbstractTask) o; - if (dataSource != null ? !dataSource.equals(that.dataSource) : that.dataSource != null) { - return false; - } - if (groupId != null ? !groupId.equals(that.groupId) : that.groupId != null) { - return false; - } - if (id != null ? !id.equals(that.id) : that.id != null) { - return false; - } - if (interval != null ? !interval.equals(that.interval) : that.interval != null) { + if (!id.equals(that.id)) { return false; } return true; } + + @Override + public int hashCode() + { + return id.hashCode(); + } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java index 7c7d8707a37..8388c9653e3 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java @@ -88,7 +88,6 @@ public class IndexDeterminePartitionsTask extends AbstractTask super( id != null ? id : makeTaskId(groupId, interval.getStart(), interval.getEnd()), groupId, - makeTaskId(groupId, interval.getStart(), interval.getEnd()), schema.getDataSource(), Preconditions.checkNotNull(interval, "interval") ); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java index d5dd40472f4..52dfb006193 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java @@ -105,7 +105,7 @@ public class RealtimeIndexTask extends AbstractTask @JsonCreator public RealtimeIndexTask( @JsonProperty("id") String id, - @JsonProperty("availabilityGroup") String availabilityGroup, + @JsonProperty("resource") TaskResource taskResource, @JsonProperty("schema") Schema schema, @JsonProperty("firehose") FirehoseFactory firehoseFactory, @JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig, @@ -121,9 +121,15 @@ public class RealtimeIndexTask extends AbstractTask "index_realtime_%s", schema.getDataSource() ), - availabilityGroup != null - ? availabilityGroup - : makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString()), + taskResource != null + ? taskResource + : new TaskResource( + makeTaskId( + schema.getDataSource(), + schema.getShardSpec().getPartitionNum(), + new DateTime().toString() + ), 1 + ), schema.getDataSource(), null ); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java index 6272669cda5..bf25735bcb2 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java @@ -71,12 +71,7 @@ public interface Task */ public String getGroupId(); - /** - * Returns availability group ID of this task. Tasks the same availability group cannot be assigned to the same - * worker. If tasks do not have this restriction, a common convention is to set the availability group ID to the - * task ID. - */ - public String getAvailabilityGroup(); + public TaskResource getTaskResource(); /** * Returns a descriptive label for this task type. Used for metrics emission and logging. diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskResource.java new file mode 100644 index 00000000000..29e9363b805 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskResource.java @@ -0,0 +1,39 @@ +package com.metamx.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +public class TaskResource +{ + private final String availabilityGroup; + private final int capacity; + + @JsonCreator + public TaskResource( + @JsonProperty("availabilityGroup") String availabilityGroup, + @JsonProperty("capacity") int capacity + ) + { + this.availabilityGroup = availabilityGroup; + this.capacity = capacity; + } + + /** + * Returns availability group ID of this task. Tasks the same availability group cannot be assigned to the same + * worker. If tasks do not have this restriction, a common convention is to set the availability group ID to the + * task ID. + */ + @JsonProperty + public String getAvailabilityGroup() + { + return availabilityGroup; + } + + @JsonProperty + public int getCapacity() + { + return capacity; + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java index 27e46f9ce42..4722222bf52 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java @@ -104,7 +104,7 @@ public class VersionConverterTask extends AbstractTask DataSegment segment ) { - super(id, groupId, id, dataSource, interval); + super(id, groupId, dataSource, interval); this.segment = segment; } @@ -205,13 +205,6 @@ public class VersionConverterTask extends AbstractTask segment.getShardSpec().getPartitionNum() ), groupId, - joinId( - groupId, - "sub", - segment.getInterval().getStart(), - segment.getInterval().getEnd(), - segment.getShardSpec().getPartitionNum() - ), segment.getDataSource(), segment.getInterval() ); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java index 753af4b8daa..9e7a1db3c3c 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java @@ -94,6 +94,12 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider this.jsonMapper = jsonMapper; } + @Override + public void bootstrap(List tasks) + { + // do nothing + } + @Override public ListenableFuture run(final Task task) { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index 08157d2ab43..dd150b5c3e9 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -24,22 +24,17 @@ import com.google.common.base.Charsets; import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import com.google.common.base.Stopwatch; import com.google.common.base.Throwables; import com.google.common.collect.Lists; -import com.google.common.collect.MinMaxPriorityQueue; +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.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.metamx.common.ISE; -import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.indexing.common.RetryPolicy; -import com.metamx.druid.indexing.common.RetryPolicyFactory; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; @@ -49,6 +44,8 @@ import com.metamx.druid.indexing.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.InputStreamResponseHandler; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; import com.metamx.http.client.response.ToStringResponseHandler; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; @@ -56,25 +53,22 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.curator.utils.ZKPaths; import org.apache.zookeeper.CreateMode; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.DateTime; -import org.joda.time.Duration; import java.io.IOException; import java.io.InputStream; import java.net.MalformedURLException; import java.net.URL; import java.util.Collection; -import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -96,15 +90,13 @@ import java.util.concurrent.atomic.AtomicReference; public class RemoteTaskRunner implements TaskRunner, TaskLogProvider { private static final EmittingLogger log = new EmittingLogger(RemoteTaskRunner.class); - private static final ToStringResponseHandler STRING_RESPONSE_HANDLER = new ToStringResponseHandler(Charsets.UTF_8); + private static final StatusResponseHandler RESPONSE_HANDLER = new StatusResponseHandler(Charsets.UTF_8); private static final Joiner JOINER = Joiner.on("/"); private final ObjectMapper jsonMapper; private final RemoteTaskRunnerConfig config; private final CuratorFramework cf; private final PathChildrenCache workerPathCache; - private final ScheduledExecutorService scheduledExec; - private final RetryPolicyFactory retryPolicyFactory; private final AtomicReference workerSetupData; private final HttpClient httpClient; @@ -114,8 +106,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider private final TaskRunnerWorkQueue runningTasks = new TaskRunnerWorkQueue(); // tasks that have not yet run private final TaskRunnerWorkQueue pendingTasks = new TaskRunnerWorkQueue(); - // idempotent task retry - private final Set tasksToRetry = new ConcurrentSkipListSet(); private final ExecutorService runPendingTasksExec = Executors.newSingleThreadExecutor(); @@ -128,8 +118,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider RemoteTaskRunnerConfig config, CuratorFramework cf, PathChildrenCache workerPathCache, - ScheduledExecutorService scheduledExec, - RetryPolicyFactory retryPolicyFactory, AtomicReference workerSetupData, HttpClient httpClient ) @@ -138,8 +126,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider this.config = config; this.cf = cf; this.workerPathCache = workerPathCache; - this.scheduledExec = scheduledExec; - this.retryPolicyFactory = retryPolicyFactory; this.workerSetupData = workerSetupData; this.httpClient = httpClient; } @@ -147,43 +133,11 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider @LifecycleStart public void start() { - try { - if (started) { - return; - } - - // Add listener for creation/deletion of workers - workerPathCache.getListenable().addListener( - new PathChildrenCacheListener() - { - @Override - public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception - { - if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final Worker worker = jsonMapper.readValue( - event.getData().getData(), - Worker.class - ); - log.info("Worker[%s] reportin' for duty!", worker.getHost()); - addWorker(worker); - } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { - final Worker worker = jsonMapper.readValue( - event.getData().getData(), - Worker.class - ); - log.info("Kaboom! Worker[%s] removed!", worker.getHost()); - removeWorker(worker); - } - } - } - ); - workerPathCache.start(); - - started = true; - } - catch (Exception e) { - throw Throwables.propagate(e); + if (started) { + return; } + + started = true; } @LifecycleStop @@ -197,6 +151,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider for (ZkWorker zkWorker : zkWorkers.values()) { zkWorker.close(); } + workerPathCache.close(); } catch (Exception e) { throw Throwables.propagate(e); @@ -227,18 +182,71 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider public ZkWorker findWorkerRunningTask(String taskId) { for (ZkWorker zkWorker : zkWorkers.values()) { - if (zkWorker.getRunningTasks().contains(taskId)) { + if (zkWorker.isRunningTask(taskId)) { return zkWorker; } } return null; } - public boolean isWorkerRunningTask(String workerHost, String taskId) + public boolean isWorkerRunningTask(Worker worker, Task task) { - ZkWorker zkWorker = zkWorkers.get(workerHost); + ZkWorker zkWorker = zkWorkers.get(worker.getHost()); - return (zkWorker != null && zkWorker.getRunningTasks().contains(taskId)); + return (zkWorker != null && zkWorker.isRunningTask(task.getId())); + } + + @Override + public void bootstrap(List tasks) + { + try { + Map existingTasks = Maps.newHashMap(); + Set existingWorkers = Sets.newHashSet(cf.getChildren().forPath(config.getIndexerAnnouncementPath())); + for (String existingWorker : existingWorkers) { + Worker worker = jsonMapper.readValue( + cf.getData() + .forPath( + JOINER.join( + config.getIndexerAnnouncementPath(), + existingWorker + ) + ), Worker.class + ); + ZkWorker zkWorker = addWorker(worker); + List runningTasks = cf.getChildren() + .forPath(JOINER.join(config.getIndexerStatusPath(), existingWorker)); + for (String runningTask : runningTasks) { + existingTasks.put(runningTask, zkWorker); + } + } + + // initialize data structures + for (Task task : tasks) { + TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem( + task, + SettableFuture.create(), + new DateTime() + ); + + ZkWorker zkWorker = existingTasks.remove(task.getId()); + if (zkWorker != null) { + runningTasks.put(task.getId(), taskRunnerWorkItem); + zkWorker.addRunningTask(task); + } else { + addPendingTask(taskRunnerWorkItem); + } + } + + // shutdown any tasks that we don't know about + for (String existingTask : existingTasks.keySet()) { + shutdown(existingTask); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + init(); } /** @@ -253,7 +261,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider throw new ISE("Assigned a task[%s] that is already running or pending, WTF is happening?!", task.getId()); } TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem( - task, SettableFuture.create(), retryPolicyFactory.makeRetryPolicy(), new DateTime() + task, SettableFuture.create(), new DateTime() ); addPendingTask(taskRunnerWorkItem); return taskRunnerWorkItem.getResult(); @@ -262,7 +270,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider /** * Finds the worker running the task and forwards the shutdown signal to the worker. * - * @param taskId + * @param taskId - task id to shutdown */ @Override public void shutdown(String taskId) @@ -275,40 +283,30 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider final ZkWorker zkWorker = findWorkerRunningTask(taskId); if (zkWorker == null) { - // Would be nice to have an ability to shut down pending tasks log.info("Can't shutdown! No worker running task %s", taskId); return; } - final RetryPolicy shutdownRetryPolicy = retryPolicyFactory.makeRetryPolicy(); - final URL url = workerURL(zkWorker.getWorker(), String.format("/task/%s/shutdown", taskId)); + try { + final URL url = makeWorkerURL(zkWorker.getWorker(), String.format("/task/%s/shutdown", taskId)); + final StatusResponseHolder response = httpClient.post(url) + .go(RESPONSE_HANDLER) + .get(); - while (!shutdownRetryPolicy.hasExceededRetryThreshold()) { - try { - final String response = httpClient.post(url) - .go(STRING_RESPONSE_HANDLER) - .get(); - log.info("Sent shutdown message to worker: %s, response: %s", zkWorker.getWorker().getHost(), response); + log.info( + "Sent shutdown message to worker: %s, status %s, response: %s", + zkWorker.getWorker().getHost(), + response.getStatus(), + response.getContent() + ); - return; - } - catch (Exception e) { - log.error(e, "Exception shutting down taskId: %s", taskId); - - if (shutdownRetryPolicy.hasExceededRetryThreshold()) { - throw Throwables.propagate(e); - } else { - try { - final long sleepTime = shutdownRetryPolicy.getAndIncrementRetryDelay().getMillis(); - log.info("Will try again in %s.", new Duration(sleepTime).toString()); - Thread.sleep(sleepTime); - } - catch (InterruptedException e2) { - throw Throwables.propagate(e2); - } - } + if (!response.getStatus().equals(HttpResponseStatus.ACCEPTED)) { + throw new ISE("Shutdown failed"); } } + catch (Exception e) { + throw Throwables.propagate(e); + } } @Override @@ -321,7 +319,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider return Optional.absent(); } else { // Worker is still running this task - final URL url = workerURL(zkWorker.getWorker(), String.format("/task/%s/log?offset=%d", taskId, offset)); + final URL url = makeWorkerURL(zkWorker.getWorker(), String.format("/task/%s/log?offset=%d", taskId, offset)); return Optional.>of( new InputSupplier() { @@ -347,7 +345,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider } } - private URL workerURL(Worker worker, String path) + private URL makeWorkerURL(Worker worker, String path) { Preconditions.checkArgument(path.startsWith("/"), "path must start with '/': %s", path); @@ -403,42 +401,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider ); } - /** - * Retries a task by inserting it back into the pending queue after a given delay. - * - * @param taskRunnerWorkItem - the task to retry - */ - private void retryTask(final TaskRunnerWorkItem taskRunnerWorkItem) - { - final String taskId = taskRunnerWorkItem.getTask().getId(); - - if (tasksToRetry.contains(taskId)) { - return; - } - - tasksToRetry.add(taskId); - - if (!taskRunnerWorkItem.getRetryPolicy().hasExceededRetryThreshold()) { - log.info("Retry scheduled in %s for %s", taskRunnerWorkItem.getRetryPolicy().getRetryDelay(), taskId); - scheduledExec.schedule( - new Runnable() - { - @Override - public void run() - { - runningTasks.remove(taskId); - tasksToRetry.remove(taskId); - addPendingTask(taskRunnerWorkItem); - } - }, - taskRunnerWorkItem.getRetryPolicy().getAndIncrementRetryDelay().getMillis(), - TimeUnit.MILLISECONDS - ); - } else { - log.makeAlert("Task exceeded retry threshold").addData("task", taskId).emit(); - } - } - /** * Removes a task from the running queue and clears out the ZK status path of the task. * @@ -468,17 +430,12 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider { try { final String taskId = taskRunnerWorkItem.getTask().getId(); - ZkWorker zkWorker = findWorkerRunningTask(taskId); - // If a worker is already running this task, we don't need to announce it - if (zkWorker != null) { - final Worker worker = zkWorker.getWorker(); - log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskId); - runningTasks.put(taskId, pendingTasks.remove(taskId)); - log.info("Task %s switched from pending to running", taskId); + if (runningTasks.containsKey(taskId)) { + log.info("Task[%s] already running.", taskId); } else { // Nothing running this task, announce it in ZK for a worker to run it - zkWorker = findWorkerForTask(taskRunnerWorkItem.getTask()); + ZkWorker zkWorker = findWorkerForTask(taskRunnerWorkItem.getTask()); if (zkWorker != null) { announceTask(zkWorker.getWorker(), taskRunnerWorkItem); } @@ -525,7 +482,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider Stopwatch timeoutStopwatch = new Stopwatch(); timeoutStopwatch.start(); synchronized (statusLock) { - while (!isWorkerRunningTask(theWorker.getHost(), task.getId())) { + while (!isWorkerRunningTask(theWorker, task)) { statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis()); if (timeoutStopwatch.elapsed(TimeUnit.MILLISECONDS) >= config.getTaskAssignmentTimeoutDuration().getMillis()) { log.error( @@ -534,13 +491,52 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider task.getId(), config.getTaskAssignmentTimeoutDuration() ); - retryTask(runningTasks.get(task.getId())); + + failTask(taskRunnerWorkItem); + break; } } } } + private void init() + { + try { + // Add listener for creation/deletion of workers + workerPathCache.getListenable().addListener( + new PathChildrenCacheListener() + { + @Override + public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception + { + if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { + final Worker worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); + log.info("Worker[%s] reportin' for duty!", worker.getHost()); + ZkWorker zkWorker = addWorker(worker); + initWorker(zkWorker); + runPendingTasks(); + } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { + final Worker worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); + log.info("Kaboom! Worker[%s] removed!", worker.getHost()); + removeWorker(worker); + } + } + } + ); + workerPathCache.start(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + /** * When a new worker appears, listeners are registered for status changes associated with tasks assigned to * the worker. Status changes indicate the creation or completion of a task. @@ -548,19 +544,30 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider * * @param worker - contains metadata for a worker that has appeared in ZK */ - private void addWorker(final Worker worker) + private ZkWorker addWorker(final Worker worker) { try { final String workerStatusPath = JOINER.join(config.getIndexerStatusPath(), worker.getHost()); final PathChildrenCache statusCache = new PathChildrenCache(cf, workerStatusPath, true); final ZkWorker zkWorker = new ZkWorker( worker, - statusCache, - jsonMapper + statusCache ); + zkWorkers.put(worker.getHost(), zkWorker); + + return zkWorker; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + private void initWorker(final ZkWorker zkWorker) + { + try { // Add status listener to the watcher for status changes - statusCache.getListenable().addListener( + zkWorker.addListener( new PathChildrenCacheListener() { @Override @@ -575,20 +582,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider event.getData().getData(), TaskStatus.class ); - // This can fail if a worker writes a bogus status. Retry if so. - if (!taskStatus.getId().equals(taskId)) { - retryTask(runningTasks.get(taskId)); - return; - } - log.info( "Worker[%s] wrote %s status for task: %s", - worker.getHost(), + zkWorker.getWorker().getHost(), taskStatus.getStatusCode(), taskId ); - // Synchronizing state with ZK statusLock.notify(); @@ -596,9 +596,11 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider if (taskRunnerWorkItem == null) { log.warn( "WTF?! Worker[%s] announcing a status for a task I didn't know about: %s", - worker.getHost(), + zkWorker.getWorker().getHost(), taskId ); + } else { + zkWorker.addRunningTask(taskRunnerWorkItem.getTask()); } if (taskStatus.isComplete()) { @@ -607,11 +609,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider if (result != null) { ((SettableFuture) result).set(taskStatus); } + + zkWorker.removeRunningTask(taskRunnerWorkItem.getTask()); } // Worker is done with this task zkWorker.setLastCompletedTaskTime(new DateTime()); - cleanup(worker.getHost(), taskId); + cleanup(zkWorker.getWorker().getHost(), taskId); runPendingTasks(); } } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { @@ -619,13 +623,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); if (taskRunnerWorkItem != null) { log.info("Task %s just disappeared!", taskId); - retryTask(taskRunnerWorkItem); + failTask(taskRunnerWorkItem); } } } catch (Exception e) { log.makeAlert(e, "Failed to handle new worker status") - .addData("worker", worker.getHost()) + .addData("worker", zkWorker.getWorker().getHost()) .addData("znode", event.getData().getPath()) .emit(); } @@ -633,10 +637,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider } } ); - zkWorkers.put(worker.getHost(), zkWorker); - statusCache.start(); - runPendingTasks(); + zkWorker.start(); } catch (Exception e) { throw Throwables.propagate(e); @@ -672,7 +674,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider if (cf.checkExists().forPath(taskPath) != null) { cf.delete().guaranteed().forPath(taskPath); } - retryTask(taskRunnerWorkItem); + failTask(taskRunnerWorkItem); } else { log.warn("RemoteTaskRunner has no knowledge of task %s", taskId); } @@ -691,48 +693,19 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider private ZkWorker findWorkerForTask(final Task task) { - try { - final MinMaxPriorityQueue workerQueue = MinMaxPriorityQueue.orderedBy( - new Comparator() - { - @Override - public int compare(ZkWorker w1, ZkWorker w2) - { - return -Ints.compare(w1.getRunningTasks().size(), w2.getRunningTasks().size()); - } - } - ).create( - FunctionalIterable.create(zkWorkers.values()).filter( - new Predicate() - { - @Override - public boolean apply(ZkWorker input) - { - for (String taskId : input.getRunningTasks()) { - TaskRunnerWorkItem workerTask = runningTasks.get(taskId); - if (workerTask != null && task.getAvailabilityGroup() - .equalsIgnoreCase(workerTask.getTask().getAvailabilityGroup())) { - return false; - } - } - return (!input.isAtCapacity() && - input.getWorker() - .getVersion() - .compareTo(workerSetupData.get().getMinVersion()) >= 0); - } - } - ) - ); - - if (workerQueue.isEmpty()) { - log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values()); - return null; + for (ZkWorker zkWorker : zkWorkers.values()) { + if (zkWorker.canRunTask(task) && + zkWorker.getWorker().getVersion().compareTo(workerSetupData.get().getMinVersion()) >= 0) { + return zkWorker; } - - return workerQueue.peek(); - } - catch (Exception e) { - throw Throwables.propagate(e); } + log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values()); + return null; } -} + + private void failTask(TaskRunnerWorkItem taskRunnerWorkItem) + { + final ListenableFuture result = taskRunnerWorkItem.getResult(); + ((SettableFuture) result).set(TaskStatus.failure(taskRunnerWorkItem.getTask().getId())); + } +} \ No newline at end of file diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java index 6e8fcfd6a0f..bfc3158b885 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java @@ -24,6 +24,7 @@ import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import java.util.Collection; +import java.util.List; /** * Interface for handing off tasks. Used by a {@link com.metamx.druid.indexing.coordinator.exec.TaskConsumer} to @@ -31,6 +32,8 @@ import java.util.Collection; */ public interface TaskRunner { + public void bootstrap(List tasks); + /** * Run a task. The returned status should be some kind of completed status. * diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java index 74321f5b1b6..11dddf56dec 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java @@ -35,7 +35,6 @@ public class TaskRunnerWorkItem implements Comparable { private final Task task; private final ListenableFuture result; - private final RetryPolicy retryPolicy; private final DateTime createdTime; private volatile DateTime queueInsertionTime; @@ -43,13 +42,11 @@ public class TaskRunnerWorkItem implements Comparable public TaskRunnerWorkItem( Task task, ListenableFuture result, - RetryPolicy retryPolicy, DateTime createdTime ) { this.task = task; this.result = result; - this.retryPolicy = retryPolicy; this.createdTime = createdTime; } @@ -64,11 +61,6 @@ public class TaskRunnerWorkItem implements Comparable return result; } - public RetryPolicy getRetryPolicy() - { - return retryPolicy; - } - @JsonProperty public DateTime getCreatedTime() { @@ -102,7 +94,6 @@ public class TaskRunnerWorkItem implements Comparable return "TaskRunnerWorkItem{" + "task=" + task + ", result=" + result + - ", retryPolicy=" + retryPolicy + ", createdTime=" + createdTime + '}'; } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java index 6ed4d3abf82..cd1ae10f0a3 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java @@ -77,13 +77,19 @@ 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, null, new DateTime()); + final TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem(task, statusFuture, new DateTime()); runningItems.add(taskRunnerWorkItem); Futures.addCallback( statusFuture, new FutureCallback() @@ -246,7 +252,6 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker return new TaskRunnerWorkItem( task, null, - null, createdTime ); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java index 16b0f57f5c5..3fb29f85e71 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java @@ -24,17 +24,22 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.worker.Worker; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.joda.time.DateTime; import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; +import java.util.Collection; +import java.util.Map; import java.util.Set; /** @@ -44,27 +49,28 @@ public class ZkWorker implements Closeable { private final Worker worker; private final PathChildrenCache statusCache; - private final Function cacheConverter; + private final Object lock = new Object(); + private final Map runningTasks = Maps.newHashMap(); + private final Set availabilityGroups = Sets.newHashSet(); + + private volatile int currCapacity = 0; private volatile DateTime lastCompletedTaskTime = new DateTime(); - public ZkWorker(Worker worker, PathChildrenCache statusCache, final ObjectMapper jsonMapper) + public ZkWorker(Worker worker, PathChildrenCache statusCache) { this.worker = worker; this.statusCache = statusCache; - this.cacheConverter = new Function() - { - @Override - public String apply(@Nullable ChildData input) - { - try { - return jsonMapper.readValue(input.getData(), TaskStatus.class).getId(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - }; + } + + public void start() throws Exception + { + statusCache.start(); + } + + public void addListener(PathChildrenCacheListener listener) + { + statusCache.getListenable().addListener(listener); } @JsonProperty @@ -74,14 +80,15 @@ public class ZkWorker implements Closeable } @JsonProperty - public Set getRunningTasks() + public Collection getRunningTasks() { - return Sets.newHashSet( - Lists.transform( - statusCache.getCurrentData(), - cacheConverter - ) - ); + return runningTasks.values(); + } + + @JsonProperty + public int getCurrCapacity() + { + return currCapacity; } @JsonProperty @@ -90,12 +97,47 @@ public class ZkWorker implements Closeable return lastCompletedTaskTime; } - @JsonProperty + public void addRunningTask(Task task) + { + synchronized (lock) { + runningTasks.put(task.getId(), task); + availabilityGroups.add(task.getTaskResource().getAvailabilityGroup()); + currCapacity += task.getTaskResource().getCapacity(); + } + } + + public void addRunningTasks(Collection tasks) + { + for (Task task : tasks) { + addRunningTask(task); + } + } + + public Task removeRunningTask(Task task) + { + synchronized (lock) { + currCapacity -= task.getTaskResource().getCapacity(); + availabilityGroups.remove(task.getTaskResource().getAvailabilityGroup()); + return runningTasks.remove(task.getId()); + } + } + + public boolean isRunningTask(String taskId) + { + return runningTasks.containsKey(taskId); + } + public boolean isAtCapacity() { - return statusCache.getCurrentData().size() >= worker.getCapacity(); + return currCapacity >= worker.getCapacity(); } + public boolean canRunTask(Task task) + { + return (worker.getCapacity() - currCapacity >= task.getTaskResource().getCapacity() && !availabilityGroups.contains(task.getTaskResource().getAvailabilityGroup())); + } + + public void setLastCompletedTaskTime(DateTime completedTaskTime) { lastCompletedTaskTime = completedTaskTime; @@ -111,8 +153,10 @@ public class ZkWorker implements Closeable public String toString() { return "ZkWorker{" + - "worker=" + worker + + "runningTasks=" + runningTasks + ", lastCompletedTaskTime=" + lastCompletedTaskTime + + ", currCapacity=" + currCapacity + + ", worker=" + worker + '}'; } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java index 77575b37cfd..da67e88c64e 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java @@ -21,6 +21,7 @@ package com.metamx.druid.indexing.coordinator.config; import org.skife.config.Config; import org.skife.config.Default; +import org.skife.config.DefaultNull; /** */ @@ -29,4 +30,8 @@ public abstract class EC2AutoScalingStrategyConfig @Config("druid.indexer.worker.port") @Default("8080") public abstract String getWorkerPort(); + + @Config("druid.indexer.worker.version") + @DefaultNull + public abstract String getWorkerVersion(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index f850ee5f5f3..af35351b191 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -637,29 +637,12 @@ public class IndexerCoordinatorNode extends QueryableNode WorkerSetupData setupData = workerSetupDataRef.get(); EC2NodeData workerConfig = setupData.getNodeData(); + GalaxyUserData userData = setupData.getUserData(); + if (config.getWorkerVersion() != null) { + userData = userData.withVersion(config.getWorkerVersion()); + } + RunInstancesResult result = amazonEC2Client.runInstances( new RunInstancesRequest( workerConfig.getAmiId(), @@ -84,7 +90,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy .withUserData( Base64.encodeBase64String( jsonMapper.writeValueAsBytes( - setupData.getUserData() + userData ) ) ) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/GalaxyUserData.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/GalaxyUserData.java index 60241f472ff..4a7490660cc 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/GalaxyUserData.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/GalaxyUserData.java @@ -60,6 +60,11 @@ public class GalaxyUserData return type; } + public GalaxyUserData withVersion(String ver) + { + return new GalaxyUserData(env, ver, type); + } + @Override public String toString() { diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RealtimeishTask.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RealtimeishTask.java index a43ec4fb2fc..4e1dc18ae5f 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RealtimeishTask.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RealtimeishTask.java @@ -30,6 +30,7 @@ import com.metamx.druid.indexing.common.actions.LockListAction; import com.metamx.druid.indexing.common.actions.LockReleaseAction; import com.metamx.druid.indexing.common.actions.SegmentInsertAction; import com.metamx.druid.indexing.common.task.AbstractTask; +import com.metamx.druid.indexing.common.task.TaskResource; import org.joda.time.Interval; import org.junit.Assert; @@ -41,12 +42,12 @@ public class RealtimeishTask extends AbstractTask { public RealtimeishTask() { - super("rt1", "rt", "rt1", "foo", null); + super("rt1", "rt", new TaskResource("rt1", 1), "foo", null); } - public RealtimeishTask(String id, String groupId, String availGroup, String dataSource, Interval interval) + public RealtimeishTask(String id, String groupId, TaskResource taskResource, String dataSource, Interval interval) { - super(id, groupId, availGroup, dataSource, interval); + super(id, groupId, taskResource, dataSource, interval); } @Override diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java index 0a11fcb49fb..f713cbd36b2 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -18,6 +18,8 @@ import com.metamx.druid.indexing.common.TaskToolboxFactory; import com.metamx.druid.indexing.common.config.IndexerZkConfig; import com.metamx.druid.indexing.common.config.RetryPolicyConfig; import com.metamx.druid.indexing.common.config.TaskConfig; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.common.task.TaskResource; import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.druid.indexing.worker.Worker; @@ -66,8 +68,6 @@ public class RemoteTaskRunnerTest private RemoteTaskRunner remoteTaskRunner; private WorkerTaskMonitor workerTaskMonitor; - private ScheduledExecutorService scheduledExec; - private TestTask task1; private Worker worker1; @@ -249,22 +249,26 @@ public class RemoteTaskRunnerTest Assert.assertTrue("TaskCallback was not called!", callbackCalled.booleanValue()); } - @Test public void testRunSameAvailabilityGroup() throws Exception { - TestRealtimeTask theTask = new TestRealtimeTask("rt1", "rt1", "foo", TaskStatus.running("rt1")); + TestRealtimeTask theTask = new TestRealtimeTask( + "rt1", + new TaskResource("rt1", 1), + "foo", + TaskStatus.running("rt1") + ); remoteTaskRunner.run(theTask); remoteTaskRunner.run( - new TestRealtimeTask("rt2", "rt1", "foo", TaskStatus.running("rt2")) + new TestRealtimeTask("rt2", new TaskResource("rt1", 1), "foo", TaskStatus.running("rt2")) ); remoteTaskRunner.run( - new TestRealtimeTask("rt3", "rt2", "foo", TaskStatus.running("rt3")) + new TestRealtimeTask("rt3", new TaskResource("rt2", 1), "foo", TaskStatus.running("rt3")) ); Stopwatch stopwatch = new Stopwatch(); stopwatch.start(); - while (remoteTaskRunner.getRunningTasks().isEmpty()) { + while (remoteTaskRunner.getRunningTasks().size() < 2) { Thread.sleep(100); if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { throw new ISE("Cannot find running task"); @@ -276,6 +280,36 @@ public class RemoteTaskRunnerTest Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTask().getId().equals("rt2")); } + @Test + public void testRunWithCapacity() throws Exception + { + TestRealtimeTask theTask = new TestRealtimeTask( + "rt1", + new TaskResource("rt1", 1), + "foo", + TaskStatus.running("rt1") + ); + remoteTaskRunner.run(theTask); + remoteTaskRunner.run( + new TestRealtimeTask("rt2", new TaskResource("rt2", 3), "foo", TaskStatus.running("rt2")) + ); + remoteTaskRunner.run( + new TestRealtimeTask("rt3", new TaskResource("rt3", 2), "foo", TaskStatus.running("rt3")) + ); + + Stopwatch stopwatch = new Stopwatch(); + stopwatch.start(); + while (remoteTaskRunner.getRunningTasks().size() < 2) { + Thread.sleep(100); + if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { + throw new ISE("Cannot find running task"); + } + } + + Assert.assertTrue(remoteTaskRunner.getRunningTasks().size() == 2); + Assert.assertTrue(remoteTaskRunner.getPendingTasks().size() == 1); + Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTask().getId().equals("rt2")); + } private void makeTaskMonitor() throws Exception { @@ -361,21 +395,18 @@ public class RemoteTaskRunnerTest private void makeRemoteTaskRunner() throws Exception { - scheduledExec = EasyMock.createMock(ScheduledExecutorService.class); - remoteTaskRunner = new RemoteTaskRunner( jsonMapper, new TestRemoteTaskRunnerConfig(), cf, pathChildrenCache, - scheduledExec, - new RetryPolicyFactory(new TestRetryPolicyConfig()), new AtomicReference(new WorkerSetupData("0", 0, 1, null, null)), null ); // Create a single worker and wait for things for be ready remoteTaskRunner.start(); + remoteTaskRunner.bootstrap(Lists.newArrayList()); cf.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath( String.format("%s/worker1", announcementsPath), jsonMapper.writeValueAsBytes(worker1) @@ -390,27 +421,6 @@ public class RemoteTaskRunnerTest } } - private static class TestRetryPolicyConfig extends RetryPolicyConfig - { - @Override - public Duration getRetryMinDuration() - { - return null; - } - - @Override - public Duration getRetryMaxDuration() - { - return null; - } - - @Override - public long getMaxRetryCount() - { - return 0; - } - } - private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig { @Override diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java index ef3c6412c6f..193b7d70123 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java @@ -36,6 +36,7 @@ import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.indexer.granularity.UniformGranularitySpec; +import com.metamx.druid.indexing.common.task.TaskResource; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.jackson.DefaultObjectMapper; @@ -284,7 +285,7 @@ public class TaskLifecycleTest @Test public void testSimple() throws Exception { - final Task task = new AbstractTask("id1", "id1", "id1", "ds", new Interval("2012-01-01/P1D")) + final Task task = new AbstractTask("id1", "id1", new TaskResource("id1", 1), "ds", new Interval("2012-01-01/P1D")) { @Override public String getType() @@ -321,7 +322,7 @@ public class TaskLifecycleTest @Test public void testBadInterval() throws Exception { - final Task task = new AbstractTask("id1", "id1", "id1", "ds", new Interval("2012-01-01/P1D")) + final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) { @Override public String getType() @@ -355,7 +356,7 @@ public class TaskLifecycleTest @Test public void testBadVersion() throws Exception { - final Task task = new AbstractTask("id1", "id1", "id1", "ds", new Interval("2012-01-01/P1D")) + final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) { @Override public String getType() diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskQueueTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskQueueTest.java index 4e023b736dd..d3bc9d95541 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskQueueTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskQueueTest.java @@ -346,7 +346,7 @@ public class TaskQueueTest private static Task newTask(final String id, final String groupId, final String dataSource, final Interval interval) { - return new AbstractTask(id, groupId, id, dataSource, interval) + return new AbstractTask(id, groupId, dataSource, interval) { @Override public TaskStatus run(TaskToolbox toolbox) throws Exception @@ -370,7 +370,7 @@ public class TaskQueueTest final List nextTasks ) { - return new AbstractTask(id, groupId, id, dataSource, interval) + return new AbstractTask(id, groupId, dataSource, interval) { @Override public String getType() diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java index 515b75e3e1f..4e71398b383 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java @@ -27,6 +27,7 @@ import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.task.RealtimeIndexTask; +import com.metamx.druid.indexing.common.task.TaskResource; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; @@ -40,14 +41,14 @@ public class TestRealtimeTask extends RealtimeIndexTask @JsonCreator public TestRealtimeTask( @JsonProperty("id") String id, - @JsonProperty("availabilityGroup") String availGroup, + @JsonProperty("resource") TaskResource taskResource, @JsonProperty("dataSource") String dataSource, @JsonProperty("taskStatus") TaskStatus status ) { super( id, - availGroup, + taskResource, new Schema(dataSource, null, new AggregatorFactory[]{}, QueryGranularity.NONE, new NoneShardSpec()), null, null, diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java index eba026ea620..8f512390f14 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -84,6 +84,12 @@ public class EC2AutoScalingStrategyTest { return "8080"; } + + @Override + public String getWorkerVersion() + { + return ""; + } }, workerSetupData ); diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java index c44d555f798..1cade2a5c52 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -42,6 +42,7 @@ import org.junit.Before; import org.junit.Test; import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; @@ -128,7 +129,7 @@ public class SimpleResourceManagementStrategyTest boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -156,7 +157,7 @@ public class SimpleResourceManagementStrategyTest boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -172,7 +173,7 @@ public class SimpleResourceManagementStrategyTest provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -213,7 +214,7 @@ public class SimpleResourceManagementStrategyTest boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -231,7 +232,7 @@ public class SimpleResourceManagementStrategyTest provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -265,7 +266,7 @@ public class SimpleResourceManagementStrategyTest boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -295,7 +296,7 @@ public class SimpleResourceManagementStrategyTest boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -310,7 +311,7 @@ public class SimpleResourceManagementStrategyTest terminatedSomething = simpleResourceManagementStrategy.doTerminate( Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -334,18 +335,18 @@ public class SimpleResourceManagementStrategyTest Task testTask ) { - super(new Worker("host", "ip", 3, "version"), null, null); + super(new Worker("host", "ip", 3, "version"), null); this.testTask = testTask; } @Override - public Set getRunningTasks() + public Collection getRunningTasks() { if (testTask == null) { return Sets.newHashSet(); } - return Sets.newHashSet(testTask.getId()); + return Sets.newHashSet(testTask); } } } diff --git a/pom.xml b/pom.xml index a0c7d77167b..1ce145ca1c7 100644 --- a/pom.xml +++ b/pom.xml @@ -65,7 +65,7 @@ com.metamx http-client - 0.7.1 + 0.7.2 com.metamx From 4ae8395538a1108975f1f20fc035ecf8e0d79a51 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 26 Jul 2013 14:32:08 -0700 Subject: [PATCH 03/28] 1) on bootstrap, load all initial data and do a compare with bootstrapped tasks, delete any that are extra out there 2) change autoscaling logic such that it only works with remote task runnrs 3) zk workers use their own status caches to determine what they are running --- .../cache/SimplePathChildrenCacheFactory.java | 42 +++ .../druid/indexing/common/TaskStatus.java | 21 +- .../indexing/common/config/TaskConfig.java | 28 +- .../common/task/RealtimeIndexTask.java | 16 +- .../druid/indexing/common/task/Task.java | 4 + .../indexing/common/task/TaskResource.java | 23 +- .../coordinator/RemoteTaskRunner.java | 316 ++++++++---------- .../coordinator/RemoteTaskRunnerWorkItem.java | 63 ++++ ...ue.java => RemoteTaskRunnerWorkQueue.java} | 4 +- .../coordinator/TaskMasterLifecycle.java | 14 +- .../indexing/coordinator/TaskRunner.java | 4 +- .../coordinator/TaskRunnerWorkItem.java | 16 +- .../coordinator/ThreadPoolTaskRunner.java | 12 +- .../druid/indexing/coordinator/ZkWorker.java | 93 +++--- .../config/RemoteTaskRunnerConfig.java | 4 + .../http/IndexerCoordinatorNode.java | 15 +- .../scaling/ResourceManagementScheduler.java | 5 +- .../ResourceManagementSchedulerFactory.java | 4 +- .../scaling/ResourceManagementStrategy.java | 6 +- .../SimpleResourceManagementStrategy.java | 7 +- .../coordinator/RemoteTaskRunnerTest.java | 26 +- .../coordinator/TaskLifecycleTest.java | 4 +- .../SimpleResourceManagementStrategyTest.java | 46 +-- 23 files changed, 455 insertions(+), 318 deletions(-) create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkItem.java rename indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/{TaskRunnerWorkQueue.java => RemoteTaskRunnerWorkQueue.java} (84%) diff --git a/client/src/main/java/com/metamx/druid/curator/cache/SimplePathChildrenCacheFactory.java b/client/src/main/java/com/metamx/druid/curator/cache/SimplePathChildrenCacheFactory.java index 2808ce4d7b8..fb7a3044ee4 100644 --- a/client/src/main/java/com/metamx/druid/curator/cache/SimplePathChildrenCacheFactory.java +++ b/client/src/main/java/com/metamx/druid/curator/cache/SimplePathChildrenCacheFactory.java @@ -21,8 +21,11 @@ package com.metamx.druid.curator.cache; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.utils.ThreadUtils; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; /** */ @@ -48,4 +51,43 @@ public class SimplePathChildrenCacheFactory implements PathChildrenCacheFactory { return new PathChildrenCache(curator, path, cacheData, compressed, exec); } + + public static class Builder + { + private static final ThreadFactory defaultThreadFactory = ThreadUtils.newThreadFactory("PathChildrenCache"); + + private boolean cacheData; + private boolean compressed; + private ExecutorService exec; + + public Builder() + { + cacheData = true; + compressed = false; + exec = Executors.newSingleThreadExecutor(defaultThreadFactory); + } + + public Builder withCacheData(boolean cacheData) + { + this.cacheData = cacheData; + return this; + } + + public Builder withCompressed(boolean compressed) + { + this.compressed = compressed; + return this; + } + + public Builder withExecutorService(ExecutorService exec) + { + this.exec = exec; + return this; + } + + public SimplePathChildrenCacheFactory build() + { + return new SimplePathChildrenCacheFactory(cacheData, compressed, exec); + } + } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java index 8a798d2a961..d83b7d0de40 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; +import com.metamx.druid.indexing.common.task.TaskResource; /** * Represents the status of a task. The task may be ongoing ({@link #isComplete()} false) or it may be @@ -42,33 +43,36 @@ public class TaskStatus public static TaskStatus running(String taskId) { - return new TaskStatus(taskId, Status.RUNNING, -1); + return new TaskStatus(taskId, Status.RUNNING, -1, null); } public static TaskStatus success(String taskId) { - return new TaskStatus(taskId, Status.SUCCESS, -1); + return new TaskStatus(taskId, Status.SUCCESS, -1, null); } public static TaskStatus failure(String taskId) { - return new TaskStatus(taskId, Status.FAILED, -1); + return new TaskStatus(taskId, Status.FAILED, -1, null); } private final String id; private final Status status; private final long duration; + private final TaskResource resource; @JsonCreator private TaskStatus( @JsonProperty("id") String id, @JsonProperty("status") Status status, - @JsonProperty("duration") long duration + @JsonProperty("duration") long duration, + @JsonProperty("resource") TaskResource resource ) { this.id = id; this.status = status; this.duration = duration; + this.resource = resource == null ? new TaskResource(id, 1) : resource; // Check class invariants. Preconditions.checkNotNull(id, "id"); @@ -93,6 +97,12 @@ public class TaskStatus return duration; } + @JsonProperty("resource") + public TaskResource getResource() + { + return resource; + } + /** * Signals that a task is not yet complete, and is still runnable on a worker. Exactly one of isRunnable, * isSuccess, or isFailure will be true at any one time. @@ -134,7 +144,7 @@ public class TaskStatus public TaskStatus withDuration(long _duration) { - return new TaskStatus(id, status, _duration); + return new TaskStatus(id, status, _duration, resource); } @Override @@ -144,6 +154,7 @@ public class TaskStatus .add("id", id) .add("status", status) .add("duration", duration) + .add("resource", resource) .toString(); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java index 8f7b20baee5..5e9789e9660 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexing.common.config; +import com.google.common.base.Joiner; import org.skife.config.Config; import org.skife.config.Default; @@ -26,15 +27,30 @@ import java.io.File; public abstract class TaskConfig { + private static Joiner joiner = Joiner.on("/"); + + @Config("druid.indexer.baseDir") + @Default("/tmp/") + public abstract String getBaseDir(); + @Config("druid.indexer.taskDir") - @Default("/tmp/persistent/task") - public abstract File getBaseTaskDir(); + public File getBaseTaskDir() + { + return new File(defaultPath("persistent/task")); + } + + @Config("druid.indexer.hadoopWorkingPath") + public String getHadoopWorkingPath() + { + return defaultPath("druid-indexing"); + } @Config("druid.indexer.rowFlushBoundary") @Default("500000") public abstract int getDefaultRowFlushBoundary(); - @Config("druid.indexer.hadoopWorkingPath") - @Default("/tmp/druid-indexing") - public abstract String getHadoopWorkingPath(); -} + private String defaultPath(String subPath) + { + return joiner.join(getBaseDir(), subPath); + } +} \ No newline at end of file diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java index 5b9c9e66f25..dc3ad87a9cb 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java @@ -46,13 +46,11 @@ import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.realtime.FireDepartment; import com.metamx.druid.realtime.FireDepartmentConfig; -import com.metamx.druid.realtime.FireDepartmentMetrics; import com.metamx.druid.realtime.RealtimeMetricsMonitor; import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.SegmentPublisher; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.FirehoseFactory; -import com.metamx.druid.realtime.plumber.NoopRejectionPolicyFactory; import com.metamx.druid.realtime.plumber.Plumber; import com.metamx.druid.realtime.plumber.RealtimePlumberSchool; import com.metamx.druid.realtime.plumber.RejectionPolicyFactory; @@ -115,22 +113,22 @@ public class RealtimeIndexTask extends AbstractTask ) { super( - id != null - ? id - : makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString()), + id == null + ? makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString()) + :id, String.format( "index_realtime_%s", schema.getDataSource() ), - taskResource != null - ? taskResource - : new TaskResource( + taskResource == null + ? new TaskResource( makeTaskId( schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString() ), 1 - ), + ) + : taskResource, schema.getDataSource(), null ); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java index 23a53cbfdbd..25f8b6425cd 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java @@ -71,6 +71,10 @@ public interface Task */ public String getGroupId(); + /** + * Returns a {@link com.metamx.druid.indexing.common.task.TaskResource} for this task. Task resources define specific + * worker requirements a task may require. + */ public TaskResource getTaskResource(); /** diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskResource.java index 29e9363b805..81d23a942b9 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskResource.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskResource.java @@ -8,16 +8,16 @@ import com.fasterxml.jackson.annotation.JsonProperty; public class TaskResource { private final String availabilityGroup; - private final int capacity; + private final int requiredCapacity; @JsonCreator public TaskResource( @JsonProperty("availabilityGroup") String availabilityGroup, - @JsonProperty("capacity") int capacity + @JsonProperty("requiredCapacity") int requiredCapacity ) { this.availabilityGroup = availabilityGroup; - this.capacity = capacity; + this.requiredCapacity = requiredCapacity; } /** @@ -31,9 +31,22 @@ public class TaskResource return availabilityGroup; } + + /** + * Returns the number of worker slots this task will take. + */ @JsonProperty - public int getCapacity() + public int getRequiredCapacity() { - return capacity; + return requiredCapacity; + } + + @Override + public String toString() + { + return "TaskResource{" + + "availabilityGroup='" + availabilityGroup + '\'' + + ", requiredCapacity=" + requiredCapacity + + '}'; } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index dd150b5c3e9..17d4640b25c 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -21,13 +21,13 @@ package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; +import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Stopwatch; import com.google.common.base.Throwables; 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.util.concurrent.ListenableFuture; @@ -35,6 +35,7 @@ import com.google.common.util.concurrent.SettableFuture; import com.metamx.common.ISE; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.druid.curator.cache.PathChildrenCacheFactory; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; @@ -46,8 +47,8 @@ import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.InputStreamResponseHandler; import com.metamx.http.client.response.StatusResponseHandler; import com.metamx.http.client.response.StatusResponseHolder; -import com.metamx.http.client.response.ToStringResponseHandler; 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; @@ -96,6 +97,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider private final ObjectMapper jsonMapper; private final RemoteTaskRunnerConfig config; private final CuratorFramework cf; + private final PathChildrenCacheFactory pathChildrenCacheFactory; private final PathChildrenCache workerPathCache; private final AtomicReference workerSetupData; private final HttpClient httpClient; @@ -103,9 +105,9 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider // all workers that exist in ZK private final Map zkWorkers = new ConcurrentHashMap(); // all tasks that have been assigned to a worker - private final TaskRunnerWorkQueue runningTasks = new TaskRunnerWorkQueue(); + private final RemoteTaskRunnerWorkQueue runningTasks = new RemoteTaskRunnerWorkQueue(); // tasks that have not yet run - private final TaskRunnerWorkQueue pendingTasks = new TaskRunnerWorkQueue(); + private final RemoteTaskRunnerWorkQueue pendingTasks = new RemoteTaskRunnerWorkQueue(); private final ExecutorService runPendingTasksExec = Executors.newSingleThreadExecutor(); @@ -117,7 +119,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider ObjectMapper jsonMapper, RemoteTaskRunnerConfig config, CuratorFramework cf, - PathChildrenCache workerPathCache, + PathChildrenCacheFactory pathChildrenCacheFactory, AtomicReference workerSetupData, HttpClient httpClient ) @@ -125,7 +127,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider this.jsonMapper = jsonMapper; this.config = config; this.cf = cf; - this.workerPathCache = workerPathCache; + this.pathChildrenCacheFactory = pathChildrenCacheFactory; + this.workerPathCache = pathChildrenCacheFactory.make(cf, config.getIndexerAnnouncementPath()); this.workerSetupData = workerSetupData; this.httpClient = httpClient; } @@ -168,13 +171,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider } @Override - public Collection getRunningTasks() + public Collection getRunningTasks() { return runningTasks.values(); } @Override - public Collection getPendingTasks() + public Collection getPendingTasks() { return pendingTasks.values(); } @@ -200,53 +203,73 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider public void bootstrap(List tasks) { try { - Map existingTasks = Maps.newHashMap(); - Set existingWorkers = Sets.newHashSet(cf.getChildren().forPath(config.getIndexerAnnouncementPath())); - for (String existingWorker : existingWorkers) { - Worker worker = jsonMapper.readValue( - cf.getData() - .forPath( - JOINER.join( - config.getIndexerAnnouncementPath(), - existingWorker - ) - ), Worker.class + // Add listener for creation/deletion of workers + workerPathCache.getListenable().addListener( + new PathChildrenCacheListener() + { + @Override + public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception + { + Worker worker; + switch (event.getType()) { + case CHILD_ADDED: + worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); + log.info("Worker[%s] reportin' for duty!", worker.getHost()); + addWorker(worker, PathChildrenCache.StartMode.NORMAL); + break; + case CHILD_REMOVED: + worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); + log.info("Kaboom! Worker[%s] removed!", worker.getHost()); + removeWorker(worker); + break; + default: + break; + } + } + } + ); + workerPathCache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE); + + Set existingTasks = Sets.newHashSet(); + for (ChildData childData : workerPathCache.getCurrentData()) { + final Worker worker = jsonMapper.readValue( + childData.getData(), + Worker.class ); - ZkWorker zkWorker = addWorker(worker); - List runningTasks = cf.getChildren() - .forPath(JOINER.join(config.getIndexerStatusPath(), existingWorker)); - for (String runningTask : runningTasks) { - existingTasks.put(runningTask, zkWorker); - } + log.info("Worker[%s] reportin' for duty!", worker.getHost()); + final ZkWorker zkWorker = addWorker(worker, PathChildrenCache.StartMode.BUILD_INITIAL_CACHE); + + existingTasks.addAll(zkWorker.getRunningTasks().keySet()); } - // initialize data structures - for (Task task : tasks) { - TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem( - task, - SettableFuture.create(), - new DateTime() - ); - - ZkWorker zkWorker = existingTasks.remove(task.getId()); - if (zkWorker != null) { - runningTasks.put(task.getId(), taskRunnerWorkItem); - zkWorker.addRunningTask(task); - } else { - addPendingTask(taskRunnerWorkItem); - } - } + Set bootstrappedTasks = Sets.newHashSet( + Lists.transform( + tasks, + new Function() + { + @Override + public String apply(Task input) + { + return input.getId(); + } + } + ) + ); // shutdown any tasks that we don't know about - for (String existingTask : existingTasks.keySet()) { - shutdown(existingTask); + for (String taskId : Sets.difference(existingTasks, bootstrappedTasks)) { + shutdown(taskId); } } catch (Exception e) { throw Throwables.propagate(e); } - - init(); } /** @@ -260,8 +283,9 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider if (runningTasks.containsKey(task.getId()) || pendingTasks.containsKey(task.getId())) { throw new ISE("Assigned a task[%s] that is already running or pending, WTF is happening?!", task.getId()); } - TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem( - task, SettableFuture.create(), new DateTime() + RemoteTaskRunnerWorkItem taskRunnerWorkItem = new RemoteTaskRunnerWorkItem( + task, + SettableFuture.create() ); addPendingTask(taskRunnerWorkItem); return taskRunnerWorkItem.getResult(); @@ -301,7 +325,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider ); if (!response.getStatus().equals(HttpResponseStatus.ACCEPTED)) { - throw new ISE("Shutdown failed"); + log.error("Shutdown failed for %s! Are you sure the task was running?", taskId); } } catch (Exception e) { @@ -362,7 +386,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider * * @param taskRunnerWorkItem */ - private void addPendingTask(final TaskRunnerWorkItem taskRunnerWorkItem) + private void addPendingTask(final RemoteTaskRunnerWorkItem taskRunnerWorkItem) { log.info("Added pending task %s", taskRunnerWorkItem.getTask().getId()); @@ -386,8 +410,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider try { // make a copy of the pending tasks because assignTask may delete tasks from pending and move them // into running status - List copy = Lists.newArrayList(pendingTasks.values()); - for (TaskRunnerWorkItem taskWrapper : copy) { + List copy = Lists.newArrayList(pendingTasks.values()); + for (RemoteTaskRunnerWorkItem taskWrapper : copy) { assignTask(taskWrapper); } } @@ -426,12 +450,12 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider * * @param taskRunnerWorkItem - the task to assign */ - private void assignTask(TaskRunnerWorkItem taskRunnerWorkItem) + private void assignTask(RemoteTaskRunnerWorkItem taskRunnerWorkItem) { try { final String taskId = taskRunnerWorkItem.getTask().getId(); - if (runningTasks.containsKey(taskId)) { + if (runningTasks.containsKey(taskId) || findWorkerRunningTask(taskId) != null) { log.info("Task[%s] already running.", taskId); } else { // Nothing running this task, announce it in ZK for a worker to run it @@ -453,7 +477,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider * @param theWorker The worker the task is assigned to * @param taskRunnerWorkItem The task to be assigned */ - private void announceTask(Worker theWorker, TaskRunnerWorkItem taskRunnerWorkItem) throws Exception + private void announceTask(Worker theWorker, RemoteTaskRunnerWorkItem taskRunnerWorkItem) throws Exception { final Task task = taskRunnerWorkItem.getTask(); @@ -492,51 +516,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider config.getTaskAssignmentTimeoutDuration() ); - failTask(taskRunnerWorkItem); - + taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId())); break; } } } } - private void init() - { - try { - // Add listener for creation/deletion of workers - workerPathCache.getListenable().addListener( - new PathChildrenCacheListener() - { - @Override - public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception - { - if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final Worker worker = jsonMapper.readValue( - event.getData().getData(), - Worker.class - ); - log.info("Worker[%s] reportin' for duty!", worker.getHost()); - ZkWorker zkWorker = addWorker(worker); - initWorker(zkWorker); - runPendingTasks(); - } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { - final Worker worker = jsonMapper.readValue( - event.getData().getData(), - Worker.class - ); - log.info("Kaboom! Worker[%s] removed!", worker.getHost()); - removeWorker(worker); - } - } - } - ); - workerPathCache.start(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - /** * When a new worker appears, listeners are registered for status changes associated with tasks assigned to * the worker. Status changes indicate the creation or completion of a task. @@ -544,28 +530,17 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider * * @param worker - contains metadata for a worker that has appeared in ZK */ - private ZkWorker addWorker(final Worker worker) + private ZkWorker addWorker(final Worker worker, PathChildrenCache.StartMode startMode) { try { final String workerStatusPath = JOINER.join(config.getIndexerStatusPath(), worker.getHost()); - final PathChildrenCache statusCache = new PathChildrenCache(cf, workerStatusPath, true); + final PathChildrenCache statusCache = pathChildrenCacheFactory.make(cf, workerStatusPath); final ZkWorker zkWorker = new ZkWorker( worker, - statusCache + statusCache, + jsonMapper ); - zkWorkers.put(worker.getHost(), zkWorker); - - return zkWorker; - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - - private void initWorker(final ZkWorker zkWorker) - { - try { // Add status listener to the watcher for status changes zkWorker.addListener( new PathChildrenCacheListener() @@ -573,58 +548,59 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { + String taskId; + RemoteTaskRunnerWorkItem taskRunnerWorkItem; synchronized (statusLock) { try { - if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || - event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { - final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); - final TaskStatus taskStatus = jsonMapper.readValue( - event.getData().getData(), TaskStatus.class - ); + switch (event.getType()) { + case CHILD_ADDED: + case CHILD_UPDATED: + taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); + final TaskStatus taskStatus = jsonMapper.readValue( + event.getData().getData(), TaskStatus.class + ); - log.info( - "Worker[%s] wrote %s status for task: %s", - zkWorker.getWorker().getHost(), - taskStatus.getStatusCode(), - taskId - ); - - // Synchronizing state with ZK - statusLock.notify(); - - final TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); - if (taskRunnerWorkItem == null) { - log.warn( - "WTF?! Worker[%s] announcing a status for a task I didn't know about: %s", + log.info( + "Worker[%s] wrote %s status for task: %s", zkWorker.getWorker().getHost(), + taskStatus.getStatusCode(), taskId ); - } else { - zkWorker.addRunningTask(taskRunnerWorkItem.getTask()); - } - if (taskStatus.isComplete()) { - if (taskRunnerWorkItem != null) { - final ListenableFuture result = taskRunnerWorkItem.getResult(); - if (result != null) { - ((SettableFuture) result).set(taskStatus); - } + // Synchronizing state with ZK + statusLock.notify(); - zkWorker.removeRunningTask(taskRunnerWorkItem.getTask()); + taskRunnerWorkItem = runningTasks.get(taskId); + if (taskRunnerWorkItem == null) { + log.warn( + "WTF?! Worker[%s] announcing a status for a task I didn't know about: %s", + zkWorker.getWorker().getHost(), + taskId + ); } - // Worker is done with this task - zkWorker.setLastCompletedTaskTime(new DateTime()); - cleanup(zkWorker.getWorker().getHost(), taskId); - runPendingTasks(); - } - } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { - final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); - TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); - if (taskRunnerWorkItem != null) { - log.info("Task %s just disappeared!", taskId); - failTask(taskRunnerWorkItem); - } + if (taskStatus.isComplete()) { + if (taskRunnerWorkItem != null) { + final ListenableFuture result = taskRunnerWorkItem.getResult(); + if (result != null) { + ((SettableFuture) result).set(taskStatus); + } + } + + // Worker is done with this task + zkWorker.setLastCompletedTaskTime(new DateTime()); + cleanup(zkWorker.getWorker().getHost(), taskId); + runPendingTasks(); + } + break; + case CHILD_REMOVED: + taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); + taskRunnerWorkItem = runningTasks.get(taskId); + if (taskRunnerWorkItem != null) { + log.info("Task %s just disappeared!", taskId); + taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId())); + } + break; } } catch (Exception e) { @@ -638,7 +614,10 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider } ); - zkWorker.start(); + zkWorker.start(startMode); + zkWorkers.put(worker.getHost(), zkWorker); + + return zkWorker; } catch (Exception e) { throw Throwables.propagate(e); @@ -657,35 +636,30 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider ZkWorker zkWorker = zkWorkers.get(worker.getHost()); if (zkWorker != null) { try { - Set tasksToRetry = Sets.newHashSet( - cf.getChildren() - .forPath(JOINER.join(config.getIndexerTaskPath(), worker.getHost())) - ); - tasksToRetry.addAll( - cf.getChildren() - .forPath(JOINER.join(config.getIndexerStatusPath(), worker.getHost())) - ); - log.info("%s has %d tasks to retry", worker.getHost(), tasksToRetry.size()); - - for (String taskId : tasksToRetry) { - TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); + for (String assignedTask : cf.getChildren() + .forPath(JOINER.join(config.getIndexerTaskPath(), worker.getHost()))) { + RemoteTaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(assignedTask); if (taskRunnerWorkItem != null) { - String taskPath = JOINER.join(config.getIndexerTaskPath(), worker.getHost(), taskId); + String taskPath = JOINER.join(config.getIndexerTaskPath(), worker.getHost(), assignedTask); if (cf.checkExists().forPath(taskPath) != null) { cf.delete().guaranteed().forPath(taskPath); } - failTask(taskRunnerWorkItem); + taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId())); } else { - log.warn("RemoteTaskRunner has no knowledge of task %s", taskId); + log.warn("RemoteTaskRunner has no knowledge of task %s", assignedTask); } } - - zkWorker.close(); } catch (Exception e) { throw Throwables.propagate(e); } finally { + try { + zkWorker.close(); + } + catch (Exception e) { + log.error(e, "Exception closing worker %s!", worker.getHost()); + } zkWorkers.remove(worker.getHost()); } } @@ -702,10 +676,4 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values()); return null; } - - private void failTask(TaskRunnerWorkItem taskRunnerWorkItem) - { - final ListenableFuture result = taskRunnerWorkItem.getResult(); - ((SettableFuture) result).set(TaskStatus.failure(taskRunnerWorkItem.getTask().getId())); - } } \ No newline at end of file diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkItem.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkItem.java new file mode 100644 index 00000000000..72cb7155af8 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkItem.java @@ -0,0 +1,63 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 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 com.metamx.druid.indexing.coordinator; + +import com.google.common.util.concurrent.SettableFuture; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; +import org.joda.time.DateTime; + +/** + */ +public class RemoteTaskRunnerWorkItem extends TaskRunnerWorkItem +{ + private final SettableFuture result; + + public RemoteTaskRunnerWorkItem( + Task task, + SettableFuture result + ) + { + super(task, result); + this.result = result; + } + + public RemoteTaskRunnerWorkItem( + Task task, + SettableFuture result, + DateTime createdTime, + DateTime queueInsertionTime + ) + { + super(task, result, createdTime, queueInsertionTime); + this.result = result; + } + + public void setResult(TaskStatus status) + { + result.set(status); + } + + @Override + public RemoteTaskRunnerWorkItem withQueueInsertionTime(DateTime time) + { + return new RemoteTaskRunnerWorkItem(getTask(), result, getCreatedTime(), time); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkQueue.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkQueue.java similarity index 84% rename from indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkQueue.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkQueue.java index 0562040c35b..338233c5dca 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkQueue.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkQueue.java @@ -25,10 +25,10 @@ import java.util.concurrent.ConcurrentSkipListMap; /** */ -public class TaskRunnerWorkQueue extends ConcurrentSkipListMap +public class RemoteTaskRunnerWorkQueue extends ConcurrentSkipListMap { @Override - public TaskRunnerWorkItem put(String s, TaskRunnerWorkItem taskRunnerWorkItem) + public RemoteTaskRunnerWorkItem put(String s, RemoteTaskRunnerWorkItem taskRunnerWorkItem) { return super.put(s, taskRunnerWorkItem.withQueueInsertionTime(new DateTime())); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java index 8b4e3fba6e1..bd6e375a80d 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java @@ -21,12 +21,11 @@ package com.metamx.druid.indexing.coordinator; import com.google.common.base.Optional; import com.google.common.base.Throwables; +import com.metamx.common.ISE; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.indexing.common.actions.TaskActionClient; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; import com.metamx.druid.indexing.common.task.Task; @@ -34,6 +33,8 @@ import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.indexing.coordinator.exec.TaskConsumer; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import org.apache.curator.framework.CuratorFramework; @@ -88,7 +89,6 @@ public class TaskMasterLifecycle log.info("By the power of Grayskull, I have the power!"); taskRunner = runnerFactory.build(); - resourceManagementScheduler = managementSchedulerFactory.build(taskRunner); final TaskConsumer taskConsumer = new TaskConsumer( taskQueue, taskRunner, @@ -106,7 +106,13 @@ public class TaskMasterLifecycle Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, leaderLifecycle); leaderLifecycle.addManagedInstance(taskConsumer); - leaderLifecycle.addManagedInstance(resourceManagementScheduler); + if ("remote".equalsIgnoreCase(indexerCoordinatorConfig.getRunnerImpl())) { + if (!(taskRunner instanceof RemoteTaskRunner)) { + throw new ISE("WTF?! We configured a remote runner and got %s", taskRunner.getClass()); + } + resourceManagementScheduler = managementSchedulerFactory.build((RemoteTaskRunner) taskRunner); + leaderLifecycle.addManagedInstance(resourceManagementScheduler); + } try { leaderLifecycle.start(); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java index 0f3c801028c..b205f46a5e1 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java @@ -47,9 +47,9 @@ public interface TaskRunner */ public void shutdown(String taskid); - public Collection getRunningTasks(); + public Collection getRunningTasks(); - public Collection getPendingTasks(); + public Collection getPendingTasks(); public Collection getWorkers(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java index 11dddf56dec..10a4ff5d1a3 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java @@ -22,7 +22,6 @@ package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ComparisonChain; import com.google.common.util.concurrent.ListenableFuture; -import com.metamx.druid.indexing.common.RetryPolicy; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import org.joda.time.DateTime; @@ -39,15 +38,25 @@ public class TaskRunnerWorkItem implements Comparable private volatile DateTime queueInsertionTime; + public TaskRunnerWorkItem( + Task task, + ListenableFuture result + ) + { + this(task, result, new DateTime(), new DateTime()); + } + public TaskRunnerWorkItem( Task task, ListenableFuture result, - DateTime createdTime + DateTime createdTime, + DateTime queueInsertionTime ) { this.task = task; this.result = result; this.createdTime = createdTime; + this.queueInsertionTime = queueInsertionTime; } @JsonProperty @@ -75,8 +84,7 @@ public class TaskRunnerWorkItem implements Comparable public TaskRunnerWorkItem withQueueInsertionTime(DateTime time) { - this.queueInsertionTime = time; - return this; + return new TaskRunnerWorkItem(task, result, createdTime, time); } @Override diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java index 4663d52f2e0..ac2d680ebc0 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java @@ -89,7 +89,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker final TaskToolbox toolbox = toolboxFactory.build(task); final ListenableFuture statusFuture = exec.submit(new ExecutorServiceTaskRunnerCallable(task, toolbox)); - final TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem(task, statusFuture, new DateTime()); + final TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem(task, statusFuture); runningItems.add(taskRunnerWorkItem); Futures.addCallback( statusFuture, new FutureCallback() @@ -190,14 +190,10 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker private final Task task; private final TaskToolbox toolbox; - private final DateTime createdTime; - public ExecutorServiceTaskRunnerCallable(Task task, TaskToolbox toolbox) { this.task = task; this.toolbox = toolbox; - - this.createdTime = new DateTime(); } @Override @@ -249,11 +245,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker public TaskRunnerWorkItem getTaskRunnerWorkItem() { - return new TaskRunnerWorkItem( - task, - null, - createdTime - ); + return new TaskRunnerWorkItem(task, null); } } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java index 3fb29f85e71..a20e324ed1e 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java @@ -31,14 +31,11 @@ import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.worker.Worker; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.PathChildrenCache; - import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.joda.time.DateTime; -import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; -import java.util.Collection; import java.util.Map; import java.util.Set; @@ -49,23 +46,32 @@ public class ZkWorker implements Closeable { private final Worker worker; private final PathChildrenCache statusCache; + private final Function cacheConverter; - private final Object lock = new Object(); - private final Map runningTasks = Maps.newHashMap(); - private final Set availabilityGroups = Sets.newHashSet(); - - private volatile int currCapacity = 0; private volatile DateTime lastCompletedTaskTime = new DateTime(); - public ZkWorker(Worker worker, PathChildrenCache statusCache) + public ZkWorker(Worker worker, PathChildrenCache statusCache, final ObjectMapper jsonMapper) { this.worker = worker; this.statusCache = statusCache; + this.cacheConverter = new Function() + { + @Override + public TaskStatus apply(ChildData input) + { + try { + return jsonMapper.readValue(input.getData(), TaskStatus.class); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + }; } - public void start() throws Exception + public void start(PathChildrenCache.StartMode startMode) throws Exception { - statusCache.start(); + statusCache.start(startMode); } public void addListener(PathChildrenCacheListener listener) @@ -80,64 +86,61 @@ public class ZkWorker implements Closeable } @JsonProperty - public Collection getRunningTasks() + public Map getRunningTasks() { - return runningTasks.values(); + Map retVal = Maps.newHashMap(); + for (TaskStatus taskStatus : Lists.transform( + statusCache.getCurrentData(), + cacheConverter + )) { + retVal.put(taskStatus.getId(), taskStatus); + } + + return retVal; } - @JsonProperty + @JsonProperty("currCapacity") public int getCurrCapacity() { + int currCapacity = 0; + for (TaskStatus taskStatus : getRunningTasks().values()) { + currCapacity += taskStatus.getResource().getRequiredCapacity(); + } return currCapacity; } + @JsonProperty("availabilityGroups") + public Set getAvailabilityGroups() + { + Set retVal = Sets.newHashSet(); + for (TaskStatus taskStatus : getRunningTasks().values()) { + retVal.add(taskStatus.getResource().getAvailabilityGroup()); + } + return retVal; + } + @JsonProperty public DateTime getLastCompletedTaskTime() { return lastCompletedTaskTime; } - public void addRunningTask(Task task) - { - synchronized (lock) { - runningTasks.put(task.getId(), task); - availabilityGroups.add(task.getTaskResource().getAvailabilityGroup()); - currCapacity += task.getTaskResource().getCapacity(); - } - } - - public void addRunningTasks(Collection tasks) - { - for (Task task : tasks) { - addRunningTask(task); - } - } - - public Task removeRunningTask(Task task) - { - synchronized (lock) { - currCapacity -= task.getTaskResource().getCapacity(); - availabilityGroups.remove(task.getTaskResource().getAvailabilityGroup()); - return runningTasks.remove(task.getId()); - } - } - public boolean isRunningTask(String taskId) { - return runningTasks.containsKey(taskId); + return getRunningTasks().containsKey(taskId); } public boolean isAtCapacity() { - return currCapacity >= worker.getCapacity(); + return getCurrCapacity() >= worker.getCapacity(); } public boolean canRunTask(Task task) { - return (worker.getCapacity() - currCapacity >= task.getTaskResource().getCapacity() && !availabilityGroups.contains(task.getTaskResource().getAvailabilityGroup())); + return (worker.getCapacity() - getCurrCapacity() >= task.getTaskResource().getRequiredCapacity() + && !getAvailabilityGroups().contains(task.getTaskResource().getAvailabilityGroup())); } - public void setLastCompletedTaskTime(DateTime completedTaskTime) { lastCompletedTaskTime = completedTaskTime; @@ -153,10 +156,8 @@ public class ZkWorker implements Closeable public String toString() { return "ZkWorker{" + - "runningTasks=" + runningTasks + + "worker=" + worker + ", lastCompletedTaskTime=" + lastCompletedTaskTime + - ", currCapacity=" + currCapacity + - ", worker=" + worker + '}'; } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java index 15970fc37b3..6023605ea7d 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java @@ -31,4 +31,8 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig @Config("druid.indexer.taskAssignmentTimeoutDuration") @Default("PT5M") public abstract Duration getTaskAssignmentTimeoutDuration(); + + @Config("druid.curator.compression.enable") + @Default("false") + public abstract boolean enableCompression(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index af35351b191..42a82fc8ada 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -27,7 +27,6 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Charsets; import com.google.common.base.Optional; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.io.InputSupplier; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -47,6 +46,7 @@ import com.metamx.druid.QueryableNode; import com.metamx.druid.config.ConfigManager; import com.metamx.druid.config.ConfigManagerConfig; import com.metamx.druid.config.JacksonConfigManager; +import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceInstanceFactory; @@ -56,12 +56,10 @@ import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.http.StatusServlet; -import com.metamx.druid.indexing.common.RetryPolicyFactory; import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionToolbox; import com.metamx.druid.indexing.common.config.IndexerZkConfig; -import com.metamx.druid.indexing.common.config.RetryPolicyConfig; import com.metamx.druid.indexing.common.config.TaskLogConfig; import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; @@ -115,7 +113,6 @@ import com.metamx.metrics.MonitorScheduler; import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.x.discovery.ServiceDiscovery; import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -638,11 +635,13 @@ public class IndexerCoordinatorNode extends QueryableNode runningTasks, Collection zkWorkers); + public boolean doProvision(Collection runningTasks, Collection zkWorkers); - public boolean doTerminate(Collection runningTasks, Collection zkWorkers); + public boolean doTerminate(Collection runningTasks, Collection zkWorkers); public ScalingStats getStats(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java index 023091935f8..4451d68ff76 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -25,6 +25,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; +import com.metamx.druid.indexing.coordinator.RemoteTaskRunnerWorkItem; import com.metamx.druid.indexing.coordinator.TaskRunnerWorkItem; import com.metamx.druid.indexing.coordinator.ZkWorker; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; @@ -68,7 +69,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat } @Override - public boolean doProvision(Collection pendingTasks, Collection zkWorkers) + public boolean doProvision(Collection pendingTasks, Collection zkWorkers) { if (zkWorkers.size() >= workerSetupdDataRef.get().getMaxNumWorkers()) { log.info( @@ -135,7 +136,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat } @Override - public boolean doTerminate(Collection pendingTasks, Collection zkWorkers) + public boolean doTerminate(Collection pendingTasks, Collection zkWorkers) { Set workerNodeIds = Sets.newHashSet( autoScalingStrategy.ipToIdLookup( @@ -244,7 +245,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat return scalingStats; } - private boolean hasTaskPendingBeyondThreshold(Collection pendingTasks) + private boolean hasTaskPendingBeyondThreshold(Collection pendingTasks) { long now = System.currentTimeMillis(); for (TaskRunnerWorkItem pendingTask : pendingTasks) { diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java index bc785bf26b0..dc716a0927c 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -5,12 +5,15 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Stopwatch; import com.google.common.base.Throwables; import com.google.common.collect.Lists; +import com.google.common.io.Files; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.metamx.common.ISE; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; +import com.metamx.druid.curator.cache.PathChildrenCacheFactory; +import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory; import com.metamx.druid.indexing.TestTask; import com.metamx.druid.indexing.common.RetryPolicyFactory; import com.metamx.druid.indexing.common.TaskStatus; @@ -64,7 +67,7 @@ public class RemoteTaskRunnerTest private TestingCluster testingCluster; private CuratorFramework cf; - private PathChildrenCache pathChildrenCache; + private PathChildrenCacheFactory pathChildrenCacheFactory; private RemoteTaskRunner remoteTaskRunner; private WorkerTaskMonitor workerTaskMonitor; @@ -91,7 +94,7 @@ public class RemoteTaskRunnerTest cf.create().forPath(statusPath); cf.create().forPath(String.format("%s/worker1", statusPath)); - pathChildrenCache = new PathChildrenCache(cf, announcementsPath, true); + pathChildrenCacheFactory = new SimplePathChildrenCacheFactory.Builder().build(); worker1 = new Worker( "worker1", @@ -362,14 +365,11 @@ public class RemoteTaskRunnerTest new TaskConfig() { @Override - public File getBaseTaskDir() + public String getBaseDir() { - try { - return File.createTempFile("billy", "yay"); - } - catch (Exception e) { - throw Throwables.propagate(e); - } + File tmp = Files.createTempDir(); + tmp.deleteOnExit(); + return tmp.toString(); } @Override @@ -399,7 +399,7 @@ public class RemoteTaskRunnerTest jsonMapper, new TestRemoteTaskRunnerConfig(), cf, - pathChildrenCache, + pathChildrenCacheFactory, new AtomicReference(new WorkerSetupData("0", 0, 1, null, null)), null ); @@ -423,6 +423,12 @@ public class RemoteTaskRunnerTest private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig { + @Override + public boolean enableCompression() + { + return false; + } + @Override public String getIndexerAnnouncementPath() { diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java index 4221a2b2eb0..17e718a8983 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java @@ -121,9 +121,9 @@ public class TaskLifecycleTest new TaskConfig() { @Override - public File getBaseTaskDir() + public String getBaseDir() { - return tmp; + return tmp.toString(); } @Override diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java index 1cade2a5c52..7a4e4c736f7 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -19,17 +19,20 @@ package com.metamx.druid.indexing.coordinator.scaling; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.indexing.TestTask; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; -import com.metamx.druid.indexing.coordinator.TaskRunnerWorkItem; +import com.metamx.druid.indexing.coordinator.RemoteTaskRunnerWorkItem; import com.metamx.druid.indexing.coordinator.ZkWorker; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.druid.indexing.worker.Worker; +import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; @@ -44,6 +47,7 @@ import org.junit.Test; import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; @@ -128,8 +132,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -156,8 +160,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -172,8 +176,8 @@ public class SimpleResourceManagementStrategyTest ); provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -213,8 +217,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -231,8 +235,8 @@ public class SimpleResourceManagementStrategyTest Thread.sleep(2000); provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -265,8 +269,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -295,8 +299,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -310,8 +314,8 @@ public class SimpleResourceManagementStrategyTest ); terminatedSomething = simpleResourceManagementStrategy.doTerminate( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -335,18 +339,18 @@ public class SimpleResourceManagementStrategyTest Task testTask ) { - super(new Worker("host", "ip", 3, "version"), null); + super(new Worker("host", "ip", 3, "version"), null, new DefaultObjectMapper()); this.testTask = testTask; } @Override - public Collection getRunningTasks() + public Map getRunningTasks() { if (testTask == null) { - return Sets.newHashSet(); + return Maps.newHashMap(); } - return Sets.newHashSet(testTask); + return ImmutableMap.of(testTask.getId(), TaskStatus.running(testTask.getId())); } } } From ad65c8111d36453193888a740925dcd722885dbf Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 29 Jul 2013 11:41:42 -0700 Subject: [PATCH 04/28] fix logs --- .../druid/indexing/coordinator/RemoteTaskRunner.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index 17d4640b25c..1a6325ca98d 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -74,8 +74,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; /** - * The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes and manage retries in failure - * scenarios. The RemoteTaskRunner keeps track of which workers are running which tasks and manages coordinator and + * The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes. + * The RemoteTaskRunner keeps track of which workers are running which tasks and manages coordinator and * worker interactions over Zookeeper. The RemoteTaskRunner is event driven and updates state according to ephemeral * node changes in ZK. *

@@ -83,8 +83,7 @@ import java.util.concurrent.atomic.AtomicReference; * fail. The RemoteTaskRunner depends on another component to create additional worker resources. * For example, {@link com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler} can take care of these duties. *

- * If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will automatically retry any tasks - * that were associated with the node. + * If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will fail any tasks associated with the worker. *

* The RemoteTaskRunner uses ZK for job management and assignment and http for IPC messages. */ @@ -383,8 +382,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider /** * Adds a task to the pending queue - * - * @param taskRunnerWorkItem */ private void addPendingTask(final RemoteTaskRunnerWorkItem taskRunnerWorkItem) { From eaddce06d56826fa6632d9294685c6667da3a237 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 30 Jul 2013 15:26:00 -0700 Subject: [PATCH 05/28] Call TaskRunner.bootstrap immediately after starting it --- .../coordinator/TaskMasterLifecycle.java | 18 +++++++++++++++++- .../druid/indexing/coordinator/TaskQueue.java | 15 +++++++++++++++ .../druid/indexing/coordinator/TaskRunner.java | 7 +++++++ 3 files changed, 39 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java index bd6e375a80d..f0332aa3cdf 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java @@ -101,8 +101,24 @@ public class TaskMasterLifecycle // Sensible order to start stuff: final Lifecycle leaderLifecycle = new Lifecycle(); - leaderLifecycle.addManagedInstance(taskQueue); leaderLifecycle.addManagedInstance(taskRunner); + leaderLifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + taskRunner.bootstrap(taskQueue.snapshot()); + } + + @Override + public void stop() + { + + } + } + ); + leaderLifecycle.addManagedInstance(taskQueue); Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, leaderLifecycle); leaderLifecycle.addManagedInstance(taskConsumer); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java index 630934c6d17..252728b32dc 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java @@ -23,6 +23,7 @@ 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.Lists; import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; @@ -165,6 +166,20 @@ public class TaskQueue } } + /** + * Returns an immutable snapshot of the current status of this queue. + */ + public List snapshot() + { + giant.lock(); + + try { + return ImmutableList.copyOf(queue); + } 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. diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java index b205f46a5e1..074a22d74bf 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java @@ -32,6 +32,13 @@ import java.util.List; */ public interface TaskRunner { + /** + * Provide a new task runner with a list of tasks that should 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. + * + * @param tasks the tasks + */ public void bootstrap(List tasks); /** From a4edc2221d35dc4b9ccb1259988415783d5b3e27 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 31 Jul 2013 15:28:52 -0700 Subject: [PATCH 06/28] fix RTR comments --- .../druid/indexing/coordinator/RemoteTaskRunner.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index 1a6325ca98d..14838cafa2c 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -75,9 +75,10 @@ import java.util.concurrent.atomic.AtomicReference; /** * The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes. - * The RemoteTaskRunner keeps track of which workers are running which tasks and manages coordinator and - * worker interactions over Zookeeper. The RemoteTaskRunner is event driven and updates state according to ephemeral - * node changes in ZK. + * The RemoteTaskRunner uses Zookeeper to keep track of which workers are running which tasks. Tasks are assigned by + * creating ephemeral nodes in ZK that workers must remove. Workers announce the statuses of the tasks they are running. + * Once a task completes, it is up to the RTR to remove the task status and run any necessary cleanup. + * The RemoteTaskRunner is event driven and updates state according to ephemeral node changes in ZK. *

* The RemoteTaskRunner will assign tasks to a node until the node hits capacity. At that point, task assignment will * fail. The RemoteTaskRunner depends on another component to create additional worker resources. From 2f8351b31474e34ebab44ba25e84bace43b1290a Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 1 Aug 2013 09:41:36 -0700 Subject: [PATCH 07/28] fix broken ut --- .../java/com/metamx/druid/master/DruidMasterTest.java | 8 ++++---- .../java/com/metamx/druid/master/LoadQueuePeonTester.java | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java index e47afd8d109..213e350f76b 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -47,7 +47,7 @@ public class DruidMasterTest private LoadQueueTaskMaster taskMaster; private DatabaseSegmentManager databaseSegmentManager; private SingleServerInventoryView serverInventoryView; - private ScheduledExecutorService scheduledExecutorService; + private ScheduledExecutorFactory scheduledExecutorFactory; private DruidServer druidServer; private DataSegment segment; private ConcurrentMap loadManagementPeons; @@ -65,8 +65,8 @@ public class DruidMasterTest databaseSegmentManager = EasyMock.createNiceMock(DatabaseSegmentManager.class); EasyMock.replay(databaseSegmentManager); - scheduledExecutorService = EasyMock.createNiceMock(ScheduledExecutorService.class); - EasyMock.replay(scheduledExecutorService); + scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class); + EasyMock.replay(scheduledExecutorFactory); master = new DruidMaster( new DruidMasterConfig() @@ -139,7 +139,7 @@ public class DruidMasterTest null, curator, new NoopServiceEmitter(), - scheduledExecutorService, + scheduledExecutorFactory, null, taskMaster, loadManagementPeons diff --git a/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java b/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java index 3594c660c09..366cde87e1d 100644 --- a/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java +++ b/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java @@ -10,7 +10,7 @@ public class LoadQueuePeonTester extends LoadQueuePeon public LoadQueuePeonTester() { - super(null, null, null, null, null, null); + super(null, null, null, null, null); } @Override From c33f2f06ff4f08b953944a15b07c9aee56185e56 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 2 Aug 2013 09:01:02 -0700 Subject: [PATCH 08/28] fix logic of how to assign tasks to workers --- .../indexing/coordinator/RemoteTaskRunner.java | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index 2bc272ac288..969c14b3008 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -29,6 +29,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.io.InputSupplier; +import com.google.common.primitives.Ints; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.metamx.common.ISE; @@ -61,9 +62,11 @@ import java.io.InputStream; import java.net.MalformedURLException; import java.net.URL; import java.util.Collection; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; @@ -670,7 +673,20 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider private ZkWorker findWorkerForTask(final Task task) { - for (ZkWorker zkWorker : zkWorkers.values()) { + TreeSet sortedWorkers = Sets.newTreeSet( + new Comparator() + { + @Override + public int compare( + ZkWorker zkWorker, ZkWorker zkWorker2 + ) + { + return -Ints.compare(zkWorker.getCurrCapacity(), zkWorker2.getCurrCapacity()); + } + } + ); + sortedWorkers.addAll(zkWorkers.values()); + for (ZkWorker zkWorker : sortedWorkers) { if (zkWorker.canRunTask(task) && zkWorker.getWorker().getVersion().compareTo(workerSetupData.get().getMinVersion()) >= 0) { return zkWorker; From 2b715054219dc69f4016ddd7912f413c5534af29 Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 5 Aug 2013 11:22:26 -0700 Subject: [PATCH 09/28] 1) Fix HadoopDruidIndexerConfig to no longer replace ":" with "_" on the segmentOutputDir. The segmentOutputDir is user-supplied so they should have the ability to just not set a bad directory. --- .../metamx/druid/client/SingleServerInventoryView.java | 4 +--- .../metamx/druid/indexer/HadoopDruidIndexerConfig.java | 8 ++++---- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java index 4b345dc5a29..781c4ed0c77 100644 --- a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java @@ -62,9 +62,7 @@ public class SingleServerInventoryView extends ServerInventoryView curator, exec, jsonMapper, - new TypeReference() - { - } + new TypeReference(){} ); } diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index 0fe72cf0e44..1c9e29801ee 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -61,7 +61,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.mapreduce.Job; - import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.format.ISODateTimeFormat; @@ -666,13 +665,14 @@ public class HadoopDruidIndexerConfig return new Path( String.format( "%s/%s/%s_%s/%s/%s", - getSegmentOutputDir().replace(":", "_"), - dataSource.replace(":", "_"), + getSegmentOutputDir(), + dataSource, bucketInterval.getStart().toString(ISODateTimeFormat.basicDateTime()), bucketInterval.getEnd().toString(ISODateTimeFormat.basicDateTime()), getVersion().replace(":", "_"), bucket.partitionNum - )); + ) + ); } return new Path( String.format( From 3c808b15c30597befac551ebbb94c8e0c6c1b5cd Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 5 Aug 2013 11:37:20 -0700 Subject: [PATCH 10/28] 1) Fix HadoopDruidIndexerConfigTest to actually verify the current correct behavior. --- .../druid/indexer/HadoopDruidIndexerConfigTest.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java index d0d2ee1d2c0..687ec6bb084 100644 --- a/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -26,13 +26,11 @@ import com.metamx.druid.indexer.granularity.UniformGranularitySpec; import com.metamx.druid.indexer.partitions.PartitionsSpec; import com.metamx.druid.indexer.updater.DbUpdaterJobSpec; import com.metamx.druid.jackson.DefaultObjectMapper; - import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.joda.time.DateTime; import org.joda.time.Interval; -import org.joda.time.format.ISODateTimeFormat; import org.junit.Assert; import org.junit.Test; @@ -440,13 +438,13 @@ public class HadoopDruidIndexerConfigTest try { cfg = jsonReadWriteRead( "{" - + "\"dataSource\": \"the:data:source\"," + + "\"dataSource\": \"source\"," + " \"granularitySpec\":{" + " \"type\":\"uniform\"," + " \"gran\":\"hour\"," + " \"intervals\":[\"2012-07-10/P1D\"]" + " }," - + "\"segmentOutputPath\": \"/tmp/dru:id/data:test\"" + + "\"segmentOutputPath\": \"hdfs://server:9100/tmp/druid/datatest\"" + "}", HadoopDruidIndexerConfig.class ); @@ -458,8 +456,10 @@ public class HadoopDruidIndexerConfigTest Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712); Path path = cfg.makeSegmentOutputPath(new DistributedFileSystem(), bucket); - Assert.assertEquals("/tmp/dru_id/data_test/the_data_source/20120710T050000.000Z_20120710T060000.000Z/some_brand_new_version/4712", path.toString()); - + Assert.assertEquals( + "hdfs://server:9100/tmp/druid/datatest/source/20120710T050000.000Z_20120710T060000.000Z/some_brand_new_version/4712", + path.toString() + ); } @Test From 22a90d36b7592c704b1c34f2d45cad925f313ae1 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 11:44:28 -0700 Subject: [PATCH 11/28] [maven-release-plugin] prepare release druid-0.5.23 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index d9b11deecf3..05f821553f7 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23-SNAPSHOT + 0.5.23 diff --git a/common/pom.xml b/common/pom.xml index 995b53e14cc..92e446b187a 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23-SNAPSHOT + 0.5.23 diff --git a/examples/pom.xml b/examples/pom.xml index 66daeb6635e..677987d4b54 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.23-SNAPSHOT + 0.5.23 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 34f351ce5a5..21c8208a72e 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23-SNAPSHOT + 0.5.23 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 2abed0a074a..ff39d98f4d9 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23-SNAPSHOT + 0.5.23 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 6fad0c7411c..739118c842b 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23-SNAPSHOT + 0.5.23 diff --git a/pom.xml b/pom.xml index ddfcade0d1c..435504452a1 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.23-SNAPSHOT + 0.5.23 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 793d3fbaa30..a233dd52729 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23-SNAPSHOT + 0.5.23 diff --git a/server/pom.xml b/server/pom.xml index 549334ca652..14dc7d767d2 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23-SNAPSHOT + 0.5.23 diff --git a/services/pom.xml b/services/pom.xml index 8ae3c45a492..703dbb2b599 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.23-SNAPSHOT + 0.5.23 com.metamx druid - 0.5.23-SNAPSHOT + 0.5.23 From fbb1211cbcacb8f8cd6a33d70552168fba75963d Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 11:44:36 -0700 Subject: [PATCH 12/28] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 05f821553f7..ed32e88abd0 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23 + 0.5.24-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 92e446b187a..fb4c974a659 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23 + 0.5.24-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 677987d4b54..82539f0a819 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.23 + 0.5.24-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 21c8208a72e..83f1a3816ac 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23 + 0.5.24-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index ff39d98f4d9..eac928d99ac 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23 + 0.5.24-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 739118c842b..74db6732f70 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23 + 0.5.24-SNAPSHOT diff --git a/pom.xml b/pom.xml index 435504452a1..7ae410393e2 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.23 + 0.5.24-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index a233dd52729..e0b46336a87 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23 + 0.5.24-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 14dc7d767d2..8623448d192 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.23 + 0.5.24-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 703dbb2b599..cb7ebf57645 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.23 + 0.5.24-SNAPSHOT com.metamx druid - 0.5.23 + 0.5.24-SNAPSHOT From efd34f3a8bfb5c868b33667cccb6985c210226d0 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 5 Aug 2013 14:20:31 -0700 Subject: [PATCH 13/28] TaskRunner: Fix comment --- .../com/metamx/druid/indexing/coordinator/TaskRunner.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java index 074a22d74bf..38e4874afb8 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java @@ -33,9 +33,10 @@ import java.util.List; public interface TaskRunner { /** - * Provide a new task runner with a list of tasks that should already be running. Will be called once shortly + * 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. + * 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 */ From 35f89d7232cee3ff7002e0e5c890bd5d96070680 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 14:44:01 -0700 Subject: [PATCH 14/28] make RTR idempotent to multiple run requests for same task, because higher level things in the indexing service require this behaviour --- .../coordinator/RemoteTaskRunner.java | 19 +++++++++++++------ .../config/RemoteTaskRunnerConfig.java | 5 +++++ .../coordinator/RemoteTaskRunnerTest.java | 17 ++++++----------- 3 files changed, 24 insertions(+), 17 deletions(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index 969c14b3008..68f8f9b505d 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -265,9 +265,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider task.getId(), new RemoteTaskRunnerWorkItem(task, SettableFuture.create()) ); - } else { - log.info("Bootstrap didn't find %s running. Running it again", task.getId()); - run(task); } } } @@ -284,8 +281,15 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider @Override public ListenableFuture run(final Task task) { - if (runningTasks.containsKey(task.getId()) || pendingTasks.containsKey(task.getId())) { - throw new ISE("Assigned a task[%s] that is already running or pending, WTF is happening?!", task.getId()); + RemoteTaskRunnerWorkItem runningTask = runningTasks.get(task.getId()); + if (runningTask != null) { + log.info("Assigned a task[%s] that is already running, not doing anything", task.getId()); + return runningTask.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, @@ -686,9 +690,12 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider } ); sortedWorkers.addAll(zkWorkers.values()); + final String configMinWorkerVer = workerSetupData.get().getMinVersion(); + final String minWorkerVer = configMinWorkerVer == null ? config.getWorkerVersion() : configMinWorkerVer; + for (ZkWorker zkWorker : sortedWorkers) { if (zkWorker.canRunTask(task) && - zkWorker.getWorker().getVersion().compareTo(workerSetupData.get().getMinVersion()) >= 0) { + zkWorker.getWorker().getVersion().compareTo(minWorkerVer) >= 0) { return zkWorker; } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java index 6023605ea7d..7018e37a7f4 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java @@ -23,6 +23,7 @@ import com.metamx.druid.indexing.common.config.IndexerZkConfig; import org.joda.time.Duration; import org.skife.config.Config; import org.skife.config.Default; +import org.skife.config.DefaultNull; /** */ @@ -35,4 +36,8 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig @Config("druid.curator.compression.enable") @Default("false") public abstract boolean enableCompression(); + + @Config("druid.indexer.worker.version") + @DefaultNull + public abstract String getWorkerVersion(); } diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java index c53f3e1e58e..6c6da7ccb89 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -109,17 +109,6 @@ public class RemoteTaskRunnerTest remoteTaskRunner.run(task); } - @Test(expected = ISE.class) - public void testExceptionThrownWithExistingTask() throws Exception - { - doSetup(); - - remoteTaskRunner.run(makeTask(TaskStatus.running("task"))); - remoteTaskRunner.run( - makeTask(TaskStatus.running("task")) - ); - } - @Test public void testRunTooMuchZKData() throws Exception { @@ -415,5 +404,11 @@ public class RemoteTaskRunnerTest { return 1000; } + + @Override + public String getWorkerVersion() + { + return ""; + } } } From 8e1f4d3256d44e9bf46d37181b312ec5eab8a0a3 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 14:47:26 -0700 Subject: [PATCH 15/28] [maven-release-plugin] prepare release druid-0.5.24 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index ed32e88abd0..d14082f92f2 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24-SNAPSHOT + 0.5.24 diff --git a/common/pom.xml b/common/pom.xml index fb4c974a659..ad3494f3166 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24-SNAPSHOT + 0.5.24 diff --git a/examples/pom.xml b/examples/pom.xml index 82539f0a819..4b264a8a6c2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.24-SNAPSHOT + 0.5.24 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 83f1a3816ac..bc8e1a03f46 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24-SNAPSHOT + 0.5.24 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index eac928d99ac..e0873ae8f19 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24-SNAPSHOT + 0.5.24 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 74db6732f70..6d100f27ae4 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24-SNAPSHOT + 0.5.24 diff --git a/pom.xml b/pom.xml index 7ae410393e2..705e2e5e7cd 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.24-SNAPSHOT + 0.5.24 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index e0b46336a87..a4a1604adbf 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24-SNAPSHOT + 0.5.24 diff --git a/server/pom.xml b/server/pom.xml index 8623448d192..514760f7587 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24-SNAPSHOT + 0.5.24 diff --git a/services/pom.xml b/services/pom.xml index cb7ebf57645..e15fda4918a 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.24-SNAPSHOT + 0.5.24 com.metamx druid - 0.5.24-SNAPSHOT + 0.5.24 From a95d9c46e2f7bb9bec968f8392ac62ccda6c0283 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 14:47:35 -0700 Subject: [PATCH 16/28] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index d14082f92f2..77cf3494b9d 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24 + 0.5.25-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index ad3494f3166..0a1406c185f 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24 + 0.5.25-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 4b264a8a6c2..3b0c37181c9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.24 + 0.5.25-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index bc8e1a03f46..96878b38329 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24 + 0.5.25-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index e0873ae8f19..86d339a344b 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24 + 0.5.25-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 6d100f27ae4..a7c7743843e 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24 + 0.5.25-SNAPSHOT diff --git a/pom.xml b/pom.xml index 705e2e5e7cd..a0fb1cf1190 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.24 + 0.5.25-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index a4a1604adbf..1c770440273 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24 + 0.5.25-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 514760f7587..34e54d509be 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.24 + 0.5.25-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index e15fda4918a..4b59f4c2432 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.24 + 0.5.25-SNAPSHOT com.metamx druid - 0.5.24 + 0.5.25-SNAPSHOT From 626cf14a6e42fa0a2d144e8e16940e09bafd754a Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 16:02:26 -0700 Subject: [PATCH 17/28] fix bug where the curator config name was changed in one place but not another; make some info msgs into debug msgs; fix zkworker serialization --- .../druid/indexing/coordinator/RemoteTaskRunner.java | 2 +- .../metamx/druid/indexing/coordinator/ZkWorker.java | 10 +++++----- .../coordinator/config/RemoteTaskRunnerConfig.java | 2 +- .../coordinator/scaling/EC2AutoScalingStrategy.java | 4 ++-- .../scaling/SimpleResourceManagementStrategy.java | 2 +- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index 68f8f9b505d..d13435de8a5 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -685,7 +685,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider ZkWorker zkWorker, ZkWorker zkWorker2 ) { - return -Ints.compare(zkWorker.getCurrCapacity(), zkWorker2.getCurrCapacity()); + return -Ints.compare(zkWorker.getCurrCapacityUsed(), zkWorker2.getCurrCapacityUsed()); } } ); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java index a20e324ed1e..3c65cc9137b 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java @@ -85,7 +85,7 @@ public class ZkWorker implements Closeable return worker; } - @JsonProperty + @JsonProperty("runningTasks") public Map getRunningTasks() { Map retVal = Maps.newHashMap(); @@ -99,8 +99,8 @@ public class ZkWorker implements Closeable return retVal; } - @JsonProperty("currCapacity") - public int getCurrCapacity() + @JsonProperty("currCapacityUsed") + public int getCurrCapacityUsed() { int currCapacity = 0; for (TaskStatus taskStatus : getRunningTasks().values()) { @@ -132,12 +132,12 @@ public class ZkWorker implements Closeable public boolean isAtCapacity() { - return getCurrCapacity() >= worker.getCapacity(); + return getCurrCapacityUsed() >= worker.getCapacity(); } public boolean canRunTask(Task task) { - return (worker.getCapacity() - getCurrCapacity() >= task.getTaskResource().getRequiredCapacity() + return (worker.getCapacity() - getCurrCapacityUsed() >= task.getTaskResource().getRequiredCapacity() && !getAvailabilityGroups().contains(task.getTaskResource().getAvailabilityGroup())); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java index 7018e37a7f4..2acadd3ae83 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java @@ -33,7 +33,7 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig @Default("PT5M") public abstract Duration getTaskAssignmentTimeoutDuration(); - @Config("druid.curator.compression.enable") + @Config("druid.curator.compress") @Default("false") public abstract boolean enableCompression(); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java index 5422a21773a..45b5573674a 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java @@ -218,7 +218,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy } ); - log.info("Performing lookup: %s --> %s", ips, retVal); + log.debug("Performing lookup: %s --> %s", ips, retVal); return retVal; } @@ -250,7 +250,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy } ); - log.info("Performing lookup: %s --> %s", nodeIds, retVal); + log.debug("Performing lookup: %s --> %s", nodeIds, retVal); return retVal; } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java index 4451d68ff76..2546d7e7709 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -72,7 +72,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat public boolean doProvision(Collection pendingTasks, Collection zkWorkers) { if (zkWorkers.size() >= workerSetupdDataRef.get().getMaxNumWorkers()) { - log.info( + log.debug( "Cannot scale anymore. Num workers = %d, Max num workers = %d", zkWorkers.size(), workerSetupdDataRef.get().getMaxNumWorkers() From eb276d34d752b97e3569983128a3a7f5ffc81fa3 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 16:10:11 -0700 Subject: [PATCH 18/28] [maven-release-plugin] prepare release druid-0.5.25 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 77cf3494b9d..a3f8b8c970b 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25-SNAPSHOT + 0.5.25 diff --git a/common/pom.xml b/common/pom.xml index 0a1406c185f..f1d2717a35b 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25-SNAPSHOT + 0.5.25 diff --git a/examples/pom.xml b/examples/pom.xml index 3b0c37181c9..b02ce0da247 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.25-SNAPSHOT + 0.5.25 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 96878b38329..db7fd0640fd 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25-SNAPSHOT + 0.5.25 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 86d339a344b..52f2230a3de 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25-SNAPSHOT + 0.5.25 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index a7c7743843e..10c73d60cac 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25-SNAPSHOT + 0.5.25 diff --git a/pom.xml b/pom.xml index a0fb1cf1190..09d8ad26344 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.25-SNAPSHOT + 0.5.25 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 1c770440273..566d743046d 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25-SNAPSHOT + 0.5.25 diff --git a/server/pom.xml b/server/pom.xml index 34e54d509be..da5aee2c4be 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25-SNAPSHOT + 0.5.25 diff --git a/services/pom.xml b/services/pom.xml index 4b59f4c2432..e5463222b3f 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.25-SNAPSHOT + 0.5.25 com.metamx druid - 0.5.25-SNAPSHOT + 0.5.25 From 90e082463befd2f4ff6cef95031dce013bd3ccb7 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 16:10:19 -0700 Subject: [PATCH 19/28] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index a3f8b8c970b..2f2494f2d6c 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25 + 0.5.26-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index f1d2717a35b..06c513863b9 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25 + 0.5.26-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index b02ce0da247..cf5c8fa610d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.25 + 0.5.26-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index db7fd0640fd..0d01a0e9ff0 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25 + 0.5.26-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 52f2230a3de..f191aa20113 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25 + 0.5.26-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 10c73d60cac..425183991f3 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25 + 0.5.26-SNAPSHOT diff --git a/pom.xml b/pom.xml index 09d8ad26344..c57c705337e 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.25 + 0.5.26-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 566d743046d..a70ecd579aa 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25 + 0.5.26-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index da5aee2c4be..0e48e281380 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.25 + 0.5.26-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index e5463222b3f..8881bfa878c 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.25 + 0.5.26-SNAPSHOT com.metamx druid - 0.5.25 + 0.5.26-SNAPSHOT From 479f0cefca15520f5d046120a410cd637473511b Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 17:57:59 -0700 Subject: [PATCH 20/28] fix bug with RTR not assigning tasks when a new worker is available --- .../indexing/coordinator/RemoteTaskRunner.java | 5 +++-- .../druid/indexing/coordinator/ZkWorker.java | 5 +++++ .../SimpleResourceManagementStrategy.java | 16 +++++++++++++++- .../scaling/SimpleResourceManagmentConfig.java | 5 +++++ .../SimpleResourceManagementStrategyTest.java | 6 ++++++ 5 files changed, 34 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index d13435de8a5..88b0a6d43ca 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -625,6 +625,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider zkWorker.start(startMode); zkWorkers.put(worker.getHost(), zkWorker); + runPendingTasks(); + return zkWorker; } catch (Exception e) { @@ -694,8 +696,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider final String minWorkerVer = configMinWorkerVer == null ? config.getWorkerVersion() : configMinWorkerVer; for (ZkWorker zkWorker : sortedWorkers) { - if (zkWorker.canRunTask(task) && - zkWorker.getWorker().getVersion().compareTo(minWorkerVer) >= 0) { + if (zkWorker.canRunTask(task) && zkWorker.isValidVersion(minWorkerVer)) { return zkWorker; } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java index 3c65cc9137b..fd11e9c117e 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java @@ -135,6 +135,11 @@ public class ZkWorker implements Closeable return getCurrCapacityUsed() >= worker.getCapacity(); } + public boolean isValidVersion(String minVersion) + { + return worker.getVersion().compareTo(minVersion) >= 0; + } + public boolean canRunTask(Task task) { return (worker.getCapacity() - getCurrCapacityUsed() >= task.getTaskResource().getRequiredCapacity() diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java index 2546d7e7709..0f9fd927245 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -71,7 +71,21 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat @Override public boolean doProvision(Collection pendingTasks, Collection zkWorkers) { - if (zkWorkers.size() >= workerSetupdDataRef.get().getMaxNumWorkers()) { + final WorkerSetupData workerSetupData = workerSetupdDataRef.get(); + + final String minVersion = workerSetupData.getMinVersion() == null + ? config.getWorkerVersion() + : workerSetupData.getMinVersion(); + int maxNumWorkers = workerSetupData.getMaxNumWorkers(); + + int currValidWorkers = 0; + for (ZkWorker zkWorker : zkWorkers) { + if (zkWorker.isValidVersion(minVersion)) { + currValidWorkers++; + } + } + + if (currValidWorkers >= maxNumWorkers) { log.debug( "Cannot scale anymore. Num workers = %d, Max num workers = %d", zkWorkers.size(), diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java index 1584e4b5dfc..184e1aba7ed 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java @@ -22,6 +22,7 @@ package com.metamx.druid.indexing.coordinator.scaling; import org.joda.time.Duration; import org.skife.config.Config; import org.skife.config.Default; +import org.skife.config.DefaultNull; /** */ @@ -42,4 +43,8 @@ public abstract class SimpleResourceManagmentConfig @Config("druid.indexer.maxPendingTaskDuration") @Default("PT30S") public abstract Duration getMaxPendingTaskDuration(); + + @Config("druid.indexer.worker.version") + @DefaultNull + public abstract String getWorkerVersion(); } diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java index 7a4e4c736f7..f73d289526d 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -116,6 +116,12 @@ public class SimpleResourceManagementStrategyTest { return new Duration(0); } + + @Override + public String getWorkerVersion() + { + return ""; + } }, workerSetupData ); From 1d3729906d17b8b71cfdeac303b345669129524d Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 18:00:36 -0700 Subject: [PATCH 21/28] [maven-release-plugin] prepare release druid-0.5.26 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 2f2494f2d6c..b138bf5d2b0 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26-SNAPSHOT + 0.5.26 diff --git a/common/pom.xml b/common/pom.xml index 06c513863b9..68c232cc179 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26-SNAPSHOT + 0.5.26 diff --git a/examples/pom.xml b/examples/pom.xml index cf5c8fa610d..52286957561 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.26-SNAPSHOT + 0.5.26 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 0d01a0e9ff0..e84a50ebb64 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26-SNAPSHOT + 0.5.26 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index f191aa20113..2aa4e0a3c82 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26-SNAPSHOT + 0.5.26 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 425183991f3..ad38e0ff246 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26-SNAPSHOT + 0.5.26 diff --git a/pom.xml b/pom.xml index c57c705337e..337a8cdc3b0 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.26-SNAPSHOT + 0.5.26 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index a70ecd579aa..9c6dc2ad99c 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26-SNAPSHOT + 0.5.26 diff --git a/server/pom.xml b/server/pom.xml index 0e48e281380..8b74c2ee76d 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26-SNAPSHOT + 0.5.26 diff --git a/services/pom.xml b/services/pom.xml index 8881bfa878c..6173dc4f484 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.26-SNAPSHOT + 0.5.26 com.metamx druid - 0.5.26-SNAPSHOT + 0.5.26 From 8ff0b53df442c4f776ec9f5c2c9b4c3a55871dc8 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 18:00:46 -0700 Subject: [PATCH 22/28] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index b138bf5d2b0..79803e330e3 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26 + 0.5.27-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 68c232cc179..375eece48cd 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26 + 0.5.27-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 52286957561..baaf66e6262 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.26 + 0.5.27-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index e84a50ebb64..2eb71e4fa45 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26 + 0.5.27-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 2aa4e0a3c82..7f5646b7893 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26 + 0.5.27-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index ad38e0ff246..51d960a52b2 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26 + 0.5.27-SNAPSHOT diff --git a/pom.xml b/pom.xml index 337a8cdc3b0..29b85007b11 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.26 + 0.5.27-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 9c6dc2ad99c..688fb7d8897 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26 + 0.5.27-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 8b74c2ee76d..155bb2522c2 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.26 + 0.5.27-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 6173dc4f484..1ad1686ba46 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.26 + 0.5.27-SNAPSHOT com.metamx druid - 0.5.26 + 0.5.27-SNAPSHOT From d1b2a5a4b3bb21e4b98b1250ca4c4a3e6dae63f0 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 18:22:12 -0700 Subject: [PATCH 23/28] fix indexer console serde of running tasks --- .../com/metamx/druid/indexing/coordinator/ZkWorker.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java index fd11e9c117e..c322548ffea 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java @@ -36,6 +36,7 @@ import org.joda.time.DateTime; import java.io.Closeable; import java.io.IOException; +import java.util.Collection; import java.util.Map; import java.util.Set; @@ -79,13 +80,18 @@ public class ZkWorker implements Closeable statusCache.getListenable().addListener(listener); } - @JsonProperty + @JsonProperty("worker") public Worker getWorker() { return worker; } @JsonProperty("runningTasks") + public Collection getRunningTaskIds() + { + return getRunningTasks().keySet(); + } + public Map getRunningTasks() { Map retVal = Maps.newHashMap(); From c22e5b7ee00392d683a93aff82ba393712d4e270 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 18:26:10 -0700 Subject: [PATCH 24/28] [maven-release-plugin] prepare release druid-0.5.27 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 79803e330e3..67169abe1ae 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27-SNAPSHOT + 0.5.27 diff --git a/common/pom.xml b/common/pom.xml index 375eece48cd..b39d8062e15 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27-SNAPSHOT + 0.5.27 diff --git a/examples/pom.xml b/examples/pom.xml index baaf66e6262..f575b39852b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.27-SNAPSHOT + 0.5.27 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 2eb71e4fa45..b4e662e3818 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27-SNAPSHOT + 0.5.27 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 7f5646b7893..159f37b9498 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27-SNAPSHOT + 0.5.27 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 51d960a52b2..4589c340412 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27-SNAPSHOT + 0.5.27 diff --git a/pom.xml b/pom.xml index 29b85007b11..76d2c34fc33 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.27-SNAPSHOT + 0.5.27 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 688fb7d8897..3e174dbe7ba 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27-SNAPSHOT + 0.5.27 diff --git a/server/pom.xml b/server/pom.xml index 155bb2522c2..a9f8c548e65 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27-SNAPSHOT + 0.5.27 diff --git a/services/pom.xml b/services/pom.xml index 1ad1686ba46..236e8905dc2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.27-SNAPSHOT + 0.5.27 com.metamx druid - 0.5.27-SNAPSHOT + 0.5.27 From db695f49c269cf66a4994d6d2eea9939c2dc7b9c Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 5 Aug 2013 18:26:20 -0700 Subject: [PATCH 25/28] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 67169abe1ae..83a53f18e45 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27 + 0.5.28-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index b39d8062e15..ebaddc8889e 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27 + 0.5.28-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index f575b39852b..3f54364e2b6 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.27 + 0.5.28-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index b4e662e3818..13adbabe002 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27 + 0.5.28-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 159f37b9498..83cfe5834c0 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27 + 0.5.28-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 4589c340412..1860dac772d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27 + 0.5.28-SNAPSHOT diff --git a/pom.xml b/pom.xml index 76d2c34fc33..26e38dc5cf1 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.27 + 0.5.28-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 3e174dbe7ba..3b4da848222 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27 + 0.5.28-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index a9f8c548e65..a3c90788412 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.27 + 0.5.28-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 236e8905dc2..d7ecc101ff1 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.27 + 0.5.28-SNAPSHOT com.metamx druid - 0.5.27 + 0.5.28-SNAPSHOT From a1904c9b3b2797cf1a403585d28b8dba46488fc2 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 5 Aug 2013 19:56:03 -0700 Subject: [PATCH 26/28] ChatHandlerResource: Fix Guice type errors --- .../indexing/worker/executor/ChatHandlerResource.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ChatHandlerResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ChatHandlerResource.java index 7521c1adb22..cf5a89ce219 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ChatHandlerResource.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ChatHandlerResource.java @@ -1,10 +1,9 @@ package com.metamx.druid.indexing.worker.executor; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.inject.Inject; import com.metamx.druid.indexing.common.index.ChatHandler; -import com.metamx.druid.indexing.common.index.EventReceivingChatHandlerProvider; +import com.metamx.druid.indexing.common.index.ChatHandlerProvider; import javax.ws.rs.Path; import javax.ws.rs.PathParam; @@ -13,13 +12,11 @@ import javax.ws.rs.core.Response; @Path("/druid/worker/v1") public class ChatHandlerResource { - private final ObjectMapper jsonMapper; - private final EventReceivingChatHandlerProvider handlers; + private final ChatHandlerProvider handlers; @Inject - public ChatHandlerResource(ObjectMapper jsonMapper, EventReceivingChatHandlerProvider handlers) + public ChatHandlerResource(ChatHandlerProvider handlers) { - this.jsonMapper = jsonMapper; this.handlers = handlers; } From 7101d9ad2e3bd44f8a54de99171242ecf7995e21 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 5 Aug 2013 19:58:34 -0700 Subject: [PATCH 27/28] [maven-release-plugin] prepare release druid-0.5.28 --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 83a53f18e45..a9ae3b89909 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28-SNAPSHOT + 0.5.28 diff --git a/common/pom.xml b/common/pom.xml index ebaddc8889e..a77250ae4db 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28-SNAPSHOT + 0.5.28 diff --git a/examples/pom.xml b/examples/pom.xml index 3f54364e2b6..a09aa2cc647 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.28-SNAPSHOT + 0.5.28 diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 13adbabe002..f47f746cbf1 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28-SNAPSHOT + 0.5.28 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 83cfe5834c0..4e7e69f09d0 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28-SNAPSHOT + 0.5.28 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 1860dac772d..2e72cd6a0f4 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28-SNAPSHOT + 0.5.28 diff --git a/pom.xml b/pom.xml index 26e38dc5cf1..9dd7235e436 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.28-SNAPSHOT + 0.5.28 druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index 3b4da848222..f4562cfef52 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28-SNAPSHOT + 0.5.28 diff --git a/server/pom.xml b/server/pom.xml index a3c90788412..d89f4457cdb 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28-SNAPSHOT + 0.5.28 diff --git a/services/pom.xml b/services/pom.xml index d7ecc101ff1..a653f15d3c0 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.28-SNAPSHOT + 0.5.28 com.metamx druid - 0.5.28-SNAPSHOT + 0.5.28 From ba847bff32e17c8bb169b540c9bad7669a6abf1c Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 5 Aug 2013 19:58:40 -0700 Subject: [PATCH 28/28] [maven-release-plugin] prepare for next development iteration --- client/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- indexing-common/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- pom.xml | 2 +- realtime/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 4 ++-- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index a9ae3b89909..2e83c642701 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28 + 0.5.29-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index a77250ae4db..c6b04259585 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28 + 0.5.29-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index a09aa2cc647..affa8a3db72 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.28 + 0.5.29-SNAPSHOT diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index f47f746cbf1..6ced7aae2eb 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28 + 0.5.29-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 4e7e69f09d0..9607f56d542 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28 + 0.5.29-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 2e72cd6a0f4..f1360c2e48e 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28 + 0.5.29-SNAPSHOT diff --git a/pom.xml b/pom.xml index 9dd7235e436..e019ecdc870 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.28 + 0.5.29-SNAPSHOT druid druid diff --git a/realtime/pom.xml b/realtime/pom.xml index f4562cfef52..2d94339c103 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28 + 0.5.29-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index d89f4457cdb..715256f8e1a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.28 + 0.5.29-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index a653f15d3c0..08516e7cfc7 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.28 + 0.5.29-SNAPSHOT com.metamx druid - 0.5.28 + 0.5.29-SNAPSHOT