From d0f66c1fb3611718e6e2f9e6a1441612e8c4c7a6 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 31 Oct 2012 10:28:24 -0700 Subject: [PATCH 01/15] bug fix for merging with empty strings --- .../main/java/com/metamx/druid/DruidNode.java | 55 +++++++++++++++++++ .../com/metamx/druid/kv/GenericIndexed.java | 7 ++- .../com/metamx/druid/BaseStorageAdapter.java | 1 + .../metamx/druid/index/v1/IndexMerger.java | 25 ++++++++- 4 files changed, 85 insertions(+), 3 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/DruidNode.java diff --git a/client/src/main/java/com/metamx/druid/DruidNode.java b/client/src/main/java/com/metamx/druid/DruidNode.java new file mode 100644 index 000000000000..69844e78cb83 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/DruidNode.java @@ -0,0 +1,55 @@ +package com.metamx.druid; + +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import org.codehaus.jackson.map.ObjectMapper; +import org.skife.config.ConfigurationObjectFactory; + +import java.util.Properties; + +/** + */ +public abstract class DruidNode +{ + private static final Logger log = new Logger(DruidNode.class); + + private final ObjectMapper jsonMapper; + private final Lifecycle lifecycle; + private final Properties props; + private final ConfigurationObjectFactory configFactory; + + private boolean initialized = false; + + public DruidNode( + ObjectMapper jsonMapper, + Lifecycle lifecycle, + Properties props, + ConfigurationObjectFactory configFactory + ) + { + this.jsonMapper = jsonMapper; + this.lifecycle = lifecycle; + this.props = props; + this.configFactory = configFactory; + } + + public abstract void init(); + + @LifecycleStart + public synchronized void start() throws Exception + { + if (!initialized) { + init(); + } + + lifecycle.start(); + } + + @LifecycleStop + public synchronized void stop() + { + lifecycle.stop(); + } +} diff --git a/index-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java b/index-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java index b168c4a790d4..d98abc1b670b 100644 --- a/index-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java +++ b/index-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java @@ -36,9 +36,9 @@ /** * A generic, flat storage mechanism. Use static methods fromArray() or fromIterable() to construct. If input * is sorted, supports binary search index lookups. If input is not sorted, only supports array-like index lookups. - * + *

* V1 Storage Format: - * + *

* byte 1: version (0x1) * byte 2 == 0x1 => allowReverseLookup * bytes 3-6 => numBytesUsed @@ -253,6 +253,9 @@ public String fromByteBuffer(ByteBuffer buffer, int numBytes) @Override public byte[] toBytes(String val) { + if (val == null) { + return new byte[]{}; + } return val.getBytes(Charsets.UTF_8); } diff --git a/server/src/main/java/com/metamx/druid/BaseStorageAdapter.java b/server/src/main/java/com/metamx/druid/BaseStorageAdapter.java index b40409d1ede9..be72e2526d9a 100644 --- a/server/src/main/java/com/metamx/druid/BaseStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/BaseStorageAdapter.java @@ -70,6 +70,7 @@ public Iterable searchDimensions(final SearchQuery query, final Filte Iterable dims = getDimValueLookup(dimension); if (dims != null) { for (String dimVal : dims) { + dimVal = dimVal == null ? "" : dimVal; if (searchQuerySpec.accept(dimVal)) { if (filterOffset != null) { Offset lhs = new ConciseOffset(getInvertedIndex(dimension, dimVal)); diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java index 8eea1be9eb3b..2ab0cdea015d 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IndexMerger.java @@ -469,12 +469,34 @@ private static File makeIndexFiles( } Iterable dimensionValues = CombiningIterable.createSplatted( - dimValueLookups, + Iterables.transform( + dimValueLookups, + new Function, Iterable>() + { + @Override + public Iterable apply(@Nullable Indexed indexed) + { + return Iterables.transform( + indexed, + new Function() + { + @Override + public String apply(@Nullable String input) + { + return (input == null) ? "" : input; + } + } + ); + } + } + ) + , Ordering.natural().nullsFirst() ); int count = 0; for (String value : dimensionValues) { + value = value == null ? "" : value; writer.write(value); for (int i = 0; i < indexes.size(); i++) { @@ -855,6 +877,7 @@ public void convert(String value, int index) ++currIndex; if (currIndex == dimSet.size()) { lastVal = value; + return; } currValue = dimSet.get(currIndex); } From fe9cb397a190ce60a5d5219c29ce062d7fd59619 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 31 Oct 2012 10:29:03 -0700 Subject: [PATCH 02/15] remove unncessarily created file --- .../main/java/com/metamx/druid/DruidNode.java | 55 ------------------- 1 file changed, 55 deletions(-) delete mode 100644 client/src/main/java/com/metamx/druid/DruidNode.java diff --git a/client/src/main/java/com/metamx/druid/DruidNode.java b/client/src/main/java/com/metamx/druid/DruidNode.java deleted file mode 100644 index 69844e78cb83..000000000000 --- a/client/src/main/java/com/metamx/druid/DruidNode.java +++ /dev/null @@ -1,55 +0,0 @@ -package com.metamx.druid; - -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.common.logger.Logger; -import org.codehaus.jackson.map.ObjectMapper; -import org.skife.config.ConfigurationObjectFactory; - -import java.util.Properties; - -/** - */ -public abstract class DruidNode -{ - private static final Logger log = new Logger(DruidNode.class); - - private final ObjectMapper jsonMapper; - private final Lifecycle lifecycle; - private final Properties props; - private final ConfigurationObjectFactory configFactory; - - private boolean initialized = false; - - public DruidNode( - ObjectMapper jsonMapper, - Lifecycle lifecycle, - Properties props, - ConfigurationObjectFactory configFactory - ) - { - this.jsonMapper = jsonMapper; - this.lifecycle = lifecycle; - this.props = props; - this.configFactory = configFactory; - } - - public abstract void init(); - - @LifecycleStart - public synchronized void start() throws Exception - { - if (!initialized) { - init(); - } - - lifecycle.start(); - } - - @LifecycleStop - public synchronized void stop() - { - lifecycle.stop(); - } -} From 11c64593ae852c3fda7284d2104770b744cc21a5 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 31 Oct 2012 14:35:09 -0700 Subject: [PATCH 03/15] improvements to the indexing service and auto scaling --- .../com/metamx/druid/http/BrokerMain.java | 2 +- .../druid/initialization/CuratorConfig.java | 35 ++ .../druid/initialization/Initialization.java | 11 +- .../ServiceDiscoveryConfig.java | 5 +- .../merger/coordinator/RemoteTaskRunner.java | 567 ++++++++++++------ .../druid/merger/coordinator/RetryPolicy.java | 17 - .../coordinator/TaskInventoryManager.java | 125 ---- .../druid/merger/coordinator/TaskWrapper.java | 60 ++ .../merger/coordinator/WorkerWrapper.java | 80 +++ .../config/RemoteTaskRunnerConfig.java | 40 ++ .../coordinator/config/RetryPolicyConfig.java | 4 +- .../config/S3AutoScalingStrategyConfig.java | 47 ++ .../http/IndexerCoordinatorNode.java | 62 +- .../scaling/NoopScalingStrategy.java | 124 ++++ .../scaling/S3AutoScalingStrategy.java | 186 ++++++ .../coordinator/scaling/ScalingStrategy.java | 34 ++ .../druid/merger/worker/TaskMonitor.java | 2 + .../metamx/druid/merger/worker/Worker.java | 50 +- .../worker/WorkerCuratorCoordinator.java | 12 +- .../merger/worker/config/WorkerConfig.java | 10 + .../druid/merger/worker/http/WorkerNode.java | 4 +- .../scaling/S3AutoScalingStrategyTest.java | 164 +++++ .../druid/realtime/S3SegmentPusherConfig.java | 1 - .../com/metamx/druid/http/MasterMain.java | 4 +- 24 files changed, 1243 insertions(+), 403 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/TaskInventoryManager.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/config/S3AutoScalingStrategyConfig.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java create mode 100644 merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java diff --git a/client/src/main/java/com/metamx/druid/http/BrokerMain.java b/client/src/main/java/com/metamx/druid/http/BrokerMain.java index 0386685c94ab..e46d902cf5af 100644 --- a/client/src/main/java/com/metamx/druid/http/BrokerMain.java +++ b/client/src/main/java/com/metamx/druid/http/BrokerMain.java @@ -141,7 +141,7 @@ public static void main(String[] args) throws Exception final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); CuratorFramework curatorFramework = Initialization.makeCuratorFrameworkClient( - serviceDiscoveryConfig.getZkHosts(), + serviceDiscoveryConfig, lifecycle ); diff --git a/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java b/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java new file mode 100644 index 000000000000..8b83b3c8cf94 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java @@ -0,0 +1,35 @@ +/* + * 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.initialization; + +import org.skife.config.Config; +import org.skife.config.Default; + +/** + */ +public abstract class CuratorConfig +{ + @Config("druid.zk.service.host") + public abstract String getZkHosts(); + + @Config("druid.zk.service.sessionTimeoutMs") + @Default("15000") + public abstract int getZkSessionTimeoutMs(); +} diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index f14cfecebf8a..227791b6a867 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -161,14 +161,19 @@ public static Server makeJettyServer(ServerConfig config) } public static CuratorFramework makeCuratorFrameworkClient( - String zkHosts, + CuratorConfig curatorConfig, Lifecycle lifecycle ) throws IOException { final CuratorFramework framework = CuratorFrameworkFactory.builder() - .connectString(zkHosts) - .retryPolicy(new ExponentialBackoffRetry(1000, 30)) + .connectString(curatorConfig.getZkHosts()) + .retryPolicy( + new ExponentialBackoffRetry( + 1000, + 30 + ) + ) .build(); lifecycle.addHandler( diff --git a/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java b/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java index e5a97bffdc63..62cbfe44eb9b 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java @@ -23,7 +23,7 @@ /** */ -public abstract class ServiceDiscoveryConfig +public abstract class ServiceDiscoveryConfig extends CuratorConfig { @Config("druid.service") public abstract String getServiceName(); @@ -31,9 +31,6 @@ public abstract class ServiceDiscoveryConfig @Config("druid.port") public abstract int getPort(); - @Config("druid.zk.service.host") - public abstract String getZkHosts(); - @Config("druid.zk.paths.discoveryPath") public abstract String getDiscoveryPath(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index eaf000e5276b..5ed630e607b0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -20,17 +20,21 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Joiner; +import com.google.common.base.Predicate; import com.google.common.base.Throwables; -import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.primitives.Ints; import com.metamx.common.ISE; +import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.druid.PeriodGranularity; import com.metamx.druid.merger.common.TaskHolder; import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; +import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; import com.metamx.druid.merger.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; @@ -39,15 +43,30 @@ import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.zookeeper.CreateMode; import org.codehaus.jackson.map.ObjectMapper; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Period; +import java.io.IOException; import java.util.Comparator; -import java.util.Map; -import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; /** + * The RemoteTaskRunner encapsulates all interactions with Zookeeper and keeps track of which workers + * are running which tasks. 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. RemoteTaskRunners have scaling + * strategies to help them decide when to create or delete new resources. When tasks are assigned to the remote + * task runner and no workers have capacity to handle the task, provisioning will be done according to the strategy. + * The remote task runner periodically runs a check to see if any worker nodes have not had any work for a + * specified period of time. If so, the worker node will be terminated. + *

+ * If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will automatically retry any tasks + * that were associated with the node. */ public class RemoteTaskRunner implements TaskRunner { @@ -55,105 +74,213 @@ public class RemoteTaskRunner implements TaskRunner private static final Joiner JOINER = Joiner.on("/"); private final ObjectMapper jsonMapper; - private final TaskInventoryManager taskInventoryManager; - private final IndexerZkConfig config; + private final RemoteTaskRunnerConfig config; private final CuratorFramework cf; + private final PathChildrenCache workerListener; private final ScheduledExecutorService scheduledExec; private final RetryPolicyFactory retryPolicyFactory; + private final ConcurrentHashMap zkWorkers; // all workers that exist in ZK + private final ConcurrentHashMap tasks; // all tasks that are assigned or need to be assigned + private final ScalingStrategy strategy; - private final ConcurrentHashMap monitors = new ConcurrentHashMap(); + private final Object statusLock = new Object(); + + private volatile boolean started = false; public RemoteTaskRunner( ObjectMapper jsonMapper, - TaskInventoryManager taskInventoryManager, - IndexerZkConfig config, + RemoteTaskRunnerConfig config, CuratorFramework cf, + PathChildrenCache workerListener, ScheduledExecutorService scheduledExec, - RetryPolicyFactory retryPolicyFactory + RetryPolicyFactory retryPolicyFactory, + ConcurrentHashMap zkWorkers, + ConcurrentHashMap tasks, + ScalingStrategy strategy ) { this.jsonMapper = jsonMapper; - this.taskInventoryManager = taskInventoryManager; this.config = config; this.cf = cf; + this.workerListener = workerListener; this.scheduledExec = scheduledExec; this.retryPolicyFactory = retryPolicyFactory; + this.zkWorkers = zkWorkers; + this.tasks = tasks; + this.strategy = strategy; + } + + @LifecycleStart + public void start() + { + try { + workerListener.start(); + workerListener.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( + cf.getData().forPath(event.getData().getPath()), + Worker.class + ); + + log.info("New worker[%s] found!", worker.getHost()); + addWorker(worker); + } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { + // Get the worker host from the path + String workerHost = event.getData().getPath().substring(event.getData().getPath().lastIndexOf("/") + 1); + + log.info("Worker[%s] removed!", workerHost); + removeWorker(workerHost); + } + } + } + ); + + // Schedule termination of worker nodes periodically + Period period = new Period(config.getTerminateResourcesPeriodMs()); + PeriodGranularity granularity = new PeriodGranularity(period, null, null); + final long truncatedNow = granularity.truncate(new DateTime().getMillis()); + + ScheduledExecutors.scheduleAtFixedRate( + scheduledExec, + new Duration( + System.currentTimeMillis(), + granularity.next(truncatedNow) - config.getTerminateResourcesWindowMs() + ), + new Duration(config.getTerminateResourcesPeriodMs()), + new Runnable() + { + @Override + public void run() + { + strategy.terminateIfNeeded(zkWorkers); + } + } + ); + + started = true; + } + catch (Exception e) { + throw Throwables.propagate(e); + } } @LifecycleStop public void stop() { - scheduledExec.shutdownNow(); + try { + for (WorkerWrapper workerWrapper : zkWorkers.values()) { + workerWrapper.getWatcher().close(); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + finally { + started = false; + } + } + + public boolean hasStarted() + { + return started; } @Override - public void run(final Task task, final TaskContext taskContext, final TaskCallback callback) + public void run(Task task, TaskContext context, TaskCallback callback) { - run(task, taskContext, callback, retryPolicyFactory.makeRetryPolicy()); + assignTask( + new TaskWrapper( + task, context, callback, retryPolicyFactory.makeRetryPolicy() + ) + ); } - private void run( - final Task task, - final TaskContext taskContext, - final TaskCallback callback, - final RetryPolicy retryPolicy - ) + private boolean assignTask(TaskWrapper taskWrapper) { + // If the task already exists, we don't need to announce it try { - // If a worker is already running this task, check the status - Map allRunningTasks = Maps.newHashMap(); - for (Worker worker : taskInventoryManager.getInventory()) { - for (String taskId : worker.getTasks().keySet()) { - allRunningTasks.put(taskId, worker); - } - } + WorkerWrapper workerWrapper; + if ((workerWrapper = findWorkerRunningTask(taskWrapper)) != null) { + final Worker worker = workerWrapper.getWorker(); + TaskStatus taskStatus = jsonMapper.readValue( + cf.getData().forPath(JOINER.join(config.getStatusPath(), worker.getHost(), taskWrapper.getTask().getId())), + TaskStatus.class + ); - Worker workerRunningThisTask = allRunningTasks.get(task.getId()); - if (workerRunningThisTask != null) { - // If the status is complete, just run the callback, otherwise monitor for the completion of the task - if (!verifyStatusComplete(jsonMapper, workerRunningThisTask, task, callback)) { - monitorStatus(jsonMapper, workerRunningThisTask, task, taskContext, callback, retryPolicy); + if (taskStatus.isComplete()) { + TaskCallback callback = taskWrapper.getCallback(); + if (callback != null) { + callback.notify(taskStatus); + } + new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()).run(); + } else { + tasks.put(taskWrapper.getTask().getId(), taskWrapper); } - return; + return true; } - - // Run the task if it does not currently exist - Worker theWorker = getLeastCapacityWorker(); - monitorStatus(jsonMapper, theWorker, task, taskContext, callback, retryPolicy); - announceTask(theWorker, task, taskContext); } catch (Exception e) { - log.error(e, "Failed to dispatch task. Retrying"); - retryTask(task, taskContext, callback, retryPolicy); + throw Throwables.propagate(e); + } + + // Announce the task + WorkerWrapper workerWrapper = getWorkerForTask(); + if (workerWrapper != null) { + announceTask(workerWrapper.getWorker(), taskWrapper); + return true; } + + return false; } + /** + * Retries a task that has failed. + * + * @param pre - A runnable that is executed before the retry occurs + * @param taskWrapper - a container for task properties + */ private void retryTask( - final Task task, - final TaskContext taskContext, - final TaskCallback callback, - final RetryPolicy retryPolicy + final Runnable pre, + final TaskWrapper taskWrapper ) { + final Task task = taskWrapper.getTask(); + final RetryPolicy retryPolicy = taskWrapper.getRetryPolicy(); + + log.info("Registering retry for failed task[%s]", task.getId()); + if (retryPolicy.hasExceededRetryThreshold()) { log.makeAlert("Task [%s] has failed[%d] times, giving up!", task.getId(), retryPolicy.getNumRetries()) .emit(); - callback.notify(TaskStatus.failure(task.getId())); - return; } scheduledExec.schedule( - new Callable() + new Runnable() { @Override - public Object call() throws Exception + public void run() { - retryPolicy.runRunnables(); + try { + if (pre != null) { + pre.run(); + } - log.info("Retry[%d] for task[%s]", retryPolicy.getNumRetries(), task.getId()); - run(task, taskContext, callback, retryPolicy); - return null; + if (tasks.containsKey(task.getId())) { + log.info("Retry[%d] for task[%s]", retryPolicy.getNumRetries(), task.getId()); + if (!assignTask(taskWrapper)) { + throw new ISE("Unable to find worker to send retry request to for task[%s]", task.getId()); + } + } + } + catch (Exception e) { + retryTask(null, taskWrapper); + } } }, retryPolicy.getAndIncrementRetryDelay(), @@ -161,171 +288,231 @@ public Object call() throws Exception ); } - private Worker getLeastCapacityWorker() + /** + * When a new worker appears, listeners are registered for status changes. + * Status changes indicate the creation or completion of task. + * The RemoteTaskRunner updates state according to these changes. + * + * @param worker - contains metadata for a worker that has appeared in ZK + */ + private void addWorker(final Worker worker) { - final MinMaxPriorityQueue workerQueue = MinMaxPriorityQueue.orderedBy( - new Comparator() - { - @Override - public int compare(Worker w1, Worker w2) + try { + final String workerStatus = JOINER.join(config.getStatusPath(), worker.getHost()); + final ConcurrentSkipListSet runningTasks = new ConcurrentSkipListSet( + cf.getChildren().forPath(workerStatus) + ); + final PathChildrenCache watcher = new PathChildrenCache(cf, workerStatus, false); + final WorkerWrapper workerWrapper = new WorkerWrapper( + worker, + runningTasks, + watcher + ); + + // Add status listener to the watcher for status changes + watcher.getListenable().addListener( + new PathChildrenCacheListener() { - return Ints.compare(w1.getTasks().size(), w2.getTasks().size()); + @Override + public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception + { + synchronized (statusLock) { + String taskId = null; + try { + if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { + String statusPath = event.getData().getPath(); + TaskStatus taskStatus = jsonMapper.readValue( + cf.getData().forPath(statusPath), TaskStatus.class + ); + taskId = taskStatus.getId(); + + log.info("New status[%s] appeared!", taskId); + runningTasks.add(taskId); + statusLock.notify(); + } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { + String statusPath = event.getData().getPath(); + TaskStatus taskStatus = jsonMapper.readValue( + cf.getData().forPath(statusPath), TaskStatus.class + ); + taskId = taskStatus.getId(); + + log.info("Task[%s] updated status[%s]!", taskId, taskStatus.getStatusCode()); + + if (taskStatus.isComplete()) { + workerWrapper.setLastCompletedTaskTime(new DateTime()); + TaskWrapper taskWrapper = tasks.get(taskId); + + if (taskWrapper == null) { + log.warn("A task completed that I didn't know about? WTF?!"); + } else { + TaskCallback callback = taskWrapper.getCallback(); + + // Cleanup + if (callback != null) { + callback.notify(taskStatus); + } + tasks.remove(taskId); + runningTasks.remove(taskId); + cf.delete().guaranteed().forPath(statusPath); + } + } + } + } + catch (Exception e) { + retryTask(new CleanupPaths(worker.getHost(), taskId), tasks.get(taskId)); + } + } + } } - } - ).create(taskInventoryManager.getInventory()); + ); + zkWorkers.put(worker.getHost(), workerWrapper); + watcher.start(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } - if (workerQueue.isEmpty()) { - log.error("No worker nodes found!"); - throw new RuntimeException(); + private WorkerWrapper findWorkerRunningTask(TaskWrapper taskWrapper) + { + for (WorkerWrapper workerWrapper : zkWorkers.values()) { + if (workerWrapper.getRunningTasks().contains(taskWrapper.getTask().getId())) { + return workerWrapper; + } } + return null; + } - return workerQueue.peek(); + /** + * When a ephemeral worker node disappears from ZK, we have to make sure there are no tasks still assigned + * to the worker. If tasks remain, they are retried. + * + * @param workerId - id of the removed worker + */ + private void removeWorker(final String workerId) + { + WorkerWrapper workerWrapper = zkWorkers.get(workerId); + if (workerWrapper != null) { + for (String taskId : workerWrapper.getRunningTasks()) { + TaskWrapper taskWrapper = tasks.get(taskId); + if (taskWrapper != null) { + retryTask(new CleanupPaths(workerId, taskId), tasks.get(taskId)); + } + workerWrapper.removeTask(taskId); + } + + try { + workerWrapper.getWatcher().close(); + } + catch (IOException e) { + log.error("Failed to close watcher associated with worker[%s]", workerWrapper.getWorker().getHost()); + } + } + zkWorkers.remove(workerId); } - private boolean verifyStatusComplete( - final ObjectMapper jsonMapper, - final Worker worker, - final Task task, - final TaskCallback callback - ) + private WorkerWrapper getWorkerForTask() { try { - final String taskPath = JOINER.join(config.getTaskPath(), worker.getHost(), task.getId()); - final String statusPath = JOINER.join(config.getStatusPath(), worker.getHost(), task.getId()); - - TaskStatus taskStatus = jsonMapper.readValue( - cf.getData().forPath(statusPath), TaskStatus.class + final MinMaxPriorityQueue workerQueue = MinMaxPriorityQueue.orderedBy( + new Comparator() + { + @Override + public int compare(WorkerWrapper w1, WorkerWrapper w2) + { + return -Ints.compare(w1.getRunningTasks().size(), w2.getRunningTasks().size()); + } + } + ).create( + FunctionalIterable.create(zkWorkers.values()).filter( + new Predicate() + { + @Override + public boolean apply(WorkerWrapper input) + { + return (!input.isAtCapacity() && + input.getWorker().getVersion().compareTo(config.getMinWorkerVersion()) >= 0); + } + } + ) ); - if (taskStatus.isComplete()) { - if (callback != null) { - callback.notify(taskStatus); - } - - cf.delete().guaranteed().forPath(statusPath); - cf.delete().guaranteed().forPath(taskPath); - - return true; + if (workerQueue.isEmpty()) { + log.makeAlert("There are no worker nodes with capacity to run task!").emit(); + strategy.provisionIfNeeded(zkWorkers); + return null; } + + return workerQueue.peek(); } catch (Exception e) { throw Throwables.propagate(e); } - - return false; } /** - * Creates a monitor for status updates and deletes. Worker nodes announce a status when they start a task and update - * it again upon completing the task. If a status is deleted, this means the worker node has died before completing - * its status update. + * Creates a ZK entry under a specific path associated with a worker. The worker is responsible for + * removing the task ZK entry and creating a task status ZK entry. + * + * @param theWorker The worker the task is assigned to + * @param taskWrapper The task to be assigned */ - private void monitorStatus( - final ObjectMapper jsonMapper, - final Worker worker, - final Task task, - final TaskContext taskContext, - final TaskCallback callback, - final RetryPolicy retryPolicy - ) throws Exception + private void announceTask(Worker theWorker, TaskWrapper taskWrapper) { - final String taskPath = JOINER.join(config.getTaskPath(), worker.getHost(), task.getId()); - final String statusPath = JOINER.join(config.getStatusPath(), worker.getHost(), task.getId()); - - PathChildrenCache monitor = monitors.get(worker.getHost()); - if (monitor == null) { - monitor = new PathChildrenCache( - cf, - JOINER.join(config.getStatusPath(), worker.getHost()), - false - ); - monitor.start(); - } - - final PathChildrenCache statusMonitor = monitor; - statusMonitor.getListenable().addListener( - new PathChildrenCacheListener() - { - @Override - public void childEvent(CuratorFramework curatorFramework, PathChildrenCacheEvent pathChildrenCacheEvent) - throws Exception - { - try { - if (pathChildrenCacheEvent.getData().getPath().equals(statusPath)) { - if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { - throw new ISE("Worker[%s] dropped Task[%s]!", worker.getHost(), task.getId()); - } - - TaskStatus taskStatus = jsonMapper.readValue( - cf.getData().forPath(statusPath), TaskStatus.class - ); - - if (taskStatus.isComplete()) { - if (callback != null) { - callback.notify(taskStatus); - } - - cf.delete().guaranteed().forPath(statusPath); - cf.delete().guaranteed().forPath(taskPath); - statusMonitor.close(); - } - } - } - catch (Exception e) { - log.error(e, "Exception while cleaning up task[%s]. Retrying", task.getId()); - - retryPolicy.registerRunnable( - new Runnable() - { - @Override - public void run() - { - try { - if (cf.checkExists().forPath(statusPath) != null) { - cf.delete().guaranteed().forPath(statusPath); - } - if (cf.checkExists().forPath(taskPath) != null) { - cf.delete().guaranteed().forPath(taskPath); - } - statusMonitor.close(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ); - - retryTask(task, taskContext, callback, retryPolicy); - } - } + synchronized (statusLock) { + final Task task = taskWrapper.getTask(); + final TaskContext taskContext = taskWrapper.getTaskContext(); + try { + log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId()); + + tasks.put(task.getId(), taskWrapper); + + cf.create() + .creatingParentsIfNeeded() + .withMode(CreateMode.EPHEMERAL) + .forPath( + JOINER.join( + config.getTaskPath(), + theWorker.getHost(), + task.getId() + ), + jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)) + ); + + while (findWorkerRunningTask(taskWrapper) == null) { + statusLock.wait(); } - ); + } + catch (Exception e) { + log.error(e, "Exception creating task[%s] for worker node[%s]", task.getId(), theWorker.getHost()); + throw Throwables.propagate(e); + } + } } - private void announceTask(Worker theWorker, Task task, TaskContext taskContext) + private class CleanupPaths implements Runnable { - try { - log.info( - "Coordinator asking Worker[%s] to add" - + " task[%s]", theWorker.getHost(), task.getId() - ); + private final String workerId; + private final String taskId; - cf.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.EPHEMERAL) - .forPath( - JOINER.join( - config.getTaskPath(), - theWorker.getHost(), - task.getId() - ), - jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)) - ); + private CleanupPaths(String workerId, String taskId) + { + this.workerId = workerId; + this.taskId = taskId; } - catch (Exception e) { - log.error(e, "Exception creating task[%s] for worker node[%s]", task.getId(), theWorker.getHost()); - throw Throwables.propagate(e); + + @Override + public void run() + { + try { + final String statusPath = JOINER.join(config.getStatusPath(), workerId, taskId); + final String taskPath = JOINER.join(config.getTaskPath(), workerId, taskId); + cf.delete().guaranteed().forPath(statusPath); + cf.delete().guaranteed().forPath(taskPath); + } + catch (Exception e) { + log.warn("Tried to delete a path that didn't exist! Must've gone away already!"); + } } } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java index 1754e4269774..b449ce019607 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java @@ -48,23 +48,6 @@ public RetryPolicy(RetryPolicyConfig config) this.retryCount = 0; } - /** - * Register runnables that can be run at any point in a given retry. - * @param runnable - */ - public void registerRunnable(Runnable runnable) - { - runnables.add(runnable); - } - - public void runRunnables() - { - for (Runnable runnable : runnables) { - runnable.run(); - } - runnables.clear(); - } - public long getAndIncrementRetryDelay() { long retVal = currRetryDelay; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskInventoryManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskInventoryManager.java deleted file mode 100644 index c8690bf5c363..000000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskInventoryManager.java +++ /dev/null @@ -1,125 +0,0 @@ -/* - * 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.merger.coordinator; - -import com.metamx.common.Pair; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.InventoryManagementStrategy; -import com.metamx.druid.client.InventoryManager; -import com.metamx.druid.client.InventoryManagerConfig; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.config.IndexerZkConfig; -import com.metamx.druid.merger.worker.Worker; -import com.metamx.phonebook.PhoneBook; -import com.metamx.phonebook.PhoneBookPeon; - -import java.util.Map; - -/** - * A simple {@link InventoryManager} that monitors ZK for the creation and deletion of new Workers and the - * tasks each worker is assigned. - */ -public class TaskInventoryManager extends InventoryManager -{ - public TaskInventoryManager( - IndexerZkConfig config, - PhoneBook yp - ) - { - super( - new Logger(TaskInventoryManager.class.getName() + "." + config.getStatusPath()), - new InventoryManagerConfig( - config.getAnnouncementPath(), - config.getStatusPath() - ), - yp, - new WorkerInventoryManagementStrategy( - new Logger( - TaskInventoryManager.class.getName() + "." + config.getStatusPath() - ) - ) - ); - } - - private static class WorkerInventoryManagementStrategy implements InventoryManagementStrategy - { - private final Logger log; - - public WorkerInventoryManagementStrategy( - Logger log - ) - { - this.log = log; - } - - @Override - public Class getContainerClass() - { - return Worker.class; - } - - @Override - public Pair> makeSubListener(final Worker worker) - { - return new Pair>( - worker.getHost(), - new PhoneBookPeon() - { - @Override - public Class getObjectClazz() - { - return TaskStatus.class; - } - - @Override - public void newEntry(String name, TaskStatus taskStatus) - { - worker.addTask(taskStatus); - log.info("Worker[%s] has new task[%s] in ZK", worker.getHost(), taskStatus.getId()); - } - - @Override - public void entryRemoved(String taskId) - { - worker.removeTask(taskId); - log.info("Worker[%s] removed task[%s] in ZK", worker.getHost(), taskId); - } - } - ); - } - - @Override - public void objectRemoved(Worker baseObject) - { - } - - @Override - public boolean doesSerde() - { - return false; - } - - @Override - public Worker deserialize(String name, Map properties) - { - throw new UnsupportedOperationException(); - } - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java new file mode 100644 index 000000000000..c757bb2dc33b --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java @@ -0,0 +1,60 @@ +/* + * 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.merger.coordinator; + +import com.metamx.druid.merger.common.task.Task; + +/** + */ +public class TaskWrapper +{ + private final Task task; + private final TaskContext taskContext; + private final TaskCallback callback; + private final RetryPolicy retryPolicy; + + public TaskWrapper(Task task, TaskContext taskContext, TaskCallback callback, RetryPolicy retryPolicy) + { + this.task = task; + this.taskContext = taskContext; + this.callback = callback; + this.retryPolicy = retryPolicy; + } + + public Task getTask() + { + return task; + } + + public TaskContext getTaskContext() + { + return taskContext; + } + + public TaskCallback getCallback() + { + return callback; + } + + public RetryPolicy getRetryPolicy() + { + return retryPolicy; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java new file mode 100644 index 000000000000..047533ae8589 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -0,0 +1,80 @@ +/* + * 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.merger.coordinator; + +import com.metamx.druid.merger.worker.Worker; +import com.netflix.curator.framework.recipes.cache.PathChildrenCache; +import org.joda.time.DateTime; + +import java.util.Set; +import java.util.concurrent.ConcurrentSkipListSet; + +/** + */ +public class WorkerWrapper +{ + private final Worker worker; + private final ConcurrentSkipListSet runningTasks; + private final PathChildrenCache watcher; + + private volatile DateTime lastCompletedTaskTime; + + public WorkerWrapper(Worker worker, ConcurrentSkipListSet runningTasks, PathChildrenCache watcher) + { + this.worker = worker; + this.runningTasks = runningTasks; + this.watcher = watcher; + } + + public Worker getWorker() + { + return worker; + } + + public Set getRunningTasks() + { + return runningTasks; + } + + public PathChildrenCache getWatcher() + { + return watcher; + } + + public DateTime getLastCompletedTaskTime() + { + return lastCompletedTaskTime; + } + + public boolean isAtCapacity() + { + return runningTasks.size() >= worker.getCapacity(); + } + + public void setLastCompletedTaskTime(DateTime completedTaskTime) + { + lastCompletedTaskTime = completedTaskTime; + } + + public void removeTask(String taskId) + { + runningTasks.remove(taskId); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java new file mode 100644 index 000000000000..3816468250f0 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -0,0 +1,40 @@ +/* + * 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.merger.coordinator.config; + +import com.metamx.druid.merger.common.config.IndexerZkConfig; +import org.skife.config.Config; +import org.skife.config.Default; + +/** + */ +public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig +{ + @Config("druid.indexer.terminateResources.periodMs") + @Default("3600000") // 1 hr + public abstract long getTerminateResourcesPeriodMs(); + + @Config("druid.indexer.terminateResources.windowMs") + @Default("300000") // 5 mins + public abstract long getTerminateResourcesWindowMs(); + + @Config("druid.indexer.minWorkerVersion") + public abstract String getMinWorkerVersion(); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java index bb2e6bb40e0d..044706b67edc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java @@ -27,11 +27,11 @@ public abstract class RetryPolicyConfig { @Config("druid.indexer.retry.minWaitMillis") - @Default("10000") + @Default("60000") // 1 minute public abstract long getRetryMinMillis(); @Config("druid.indexer.retry.maxWaitMillis") - @Default("60000") + @Default("600000") // 10 minutes public abstract long getRetryMaxMillis(); @Config("druid.indexer.retry.maxRetryCount") diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/S3AutoScalingStrategyConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/S3AutoScalingStrategyConfig.java new file mode 100644 index 000000000000..1aa2145cdc37 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/S3AutoScalingStrategyConfig.java @@ -0,0 +1,47 @@ +/* + * 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.merger.coordinator.config; + +import org.skife.config.Config; +import org.skife.config.Default; + +/** + */ +public abstract class S3AutoScalingStrategyConfig +{ + @Config("druid.indexer.amiId") + public abstract String getAmiId(); + + @Config("druid.indexer.worker.port") + @Default("8080") + public abstract String getWorkerPort(); + + @Config("druid.indexer.instanceType") + public abstract String getInstanceType(); + + @Config("druid.indexer.millisToWaitBeforeTerminating") + @Default("1800000") // 30 mins + public abstract long getMillisToWaitBeforeTerminating(); + + // minimum number of workers that must always be running + @Config("druid.indexer.minNumWorkers") + @Default("1") + public abstract int getMinNuMWorkers(); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 6da4c491897f..8b0d5eff7478 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.coordinator.http; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.services.ec2.AmazonEC2Client; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -42,7 +44,6 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; -import com.metamx.druid.initialization.ZkClientConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; @@ -53,15 +54,21 @@ import com.metamx.druid.merger.coordinator.MergerDBCoordinator; import com.metamx.druid.merger.coordinator.RemoteTaskRunner; import com.metamx.druid.merger.coordinator.RetryPolicyFactory; -import com.metamx.druid.merger.coordinator.TaskInventoryManager; import com.metamx.druid.merger.coordinator.TaskMaster; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskRunner; import com.metamx.druid.merger.coordinator.TaskRunnerFactory; import com.metamx.druid.merger.coordinator.TaskStorage; +import com.metamx.druid.merger.coordinator.TaskWrapper; +import com.metamx.druid.merger.coordinator.WorkerWrapper; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; +import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; +import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; +import com.metamx.druid.merger.coordinator.scaling.NoopScalingStrategy; +import com.metamx.druid.merger.coordinator.scaling.S3AutoScalingStrategy; +import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; import com.metamx.druid.realtime.S3SegmentPusher; import com.metamx.druid.realtime.S3SegmentPusherConfig; import com.metamx.druid.realtime.SegmentPusher; @@ -78,9 +85,8 @@ import com.metamx.metrics.MonitorScheduler; import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; -import com.metamx.phonebook.PhoneBook; import com.netflix.curator.framework.CuratorFramework; -import org.I0Itec.zkclient.ZkClient; +import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import org.codehaus.jackson.map.InjectableValues; import org.codehaus.jackson.map.ObjectMapper; import org.jets3t.service.S3ServiceException; @@ -96,6 +102,7 @@ import java.net.URL; import java.util.List; import java.util.Properties; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -126,7 +133,6 @@ public static Builder builder() private CuratorFramework curatorFramework = null; private ScheduledExecutorFactory scheduledExecutorFactory = null; private IndexerZkConfig indexerZkConfig; - private TaskInventoryManager taskInventoryManager; private TaskRunnerFactory taskRunnerFactory = null; private TaskMaster taskMaster = null; private Server server = null; @@ -194,7 +200,6 @@ public void init() throws Exception initializeJacksonSubtypes(); initializeCurator(); initializeIndexerZkConfig(); - initializeTaskInventoryManager(); initializeTaskRunnerFactory(); initializeTaskMaster(); initializeServer(); @@ -265,7 +270,7 @@ public URL getRedirectURL(String queryString, String requestURI) private void initializeTaskMaster() { - if(taskMaster == null) { + if (taskMaster == null) { final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); taskMaster = new TaskMaster( taskQueue, @@ -417,7 +422,7 @@ public void initializeCurator() throws Exception if (curatorFramework == null) { final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); curatorFramework = Initialization.makeCuratorFrameworkClient( - serviceDiscoveryConfig.getZkHosts(), + serviceDiscoveryConfig, lifecycle ); } @@ -430,28 +435,10 @@ public void initializeIndexerZkConfig() } } - public void initializeTaskInventoryManager() - { - if (taskInventoryManager == null) { - final ZkClient zkClient = Initialization.makeZkClient(configFactory.build(ZkClientConfig.class), lifecycle); - final PhoneBook masterYp = Initialization.createYellowPages( - jsonMapper, - zkClient, - "Master-ZKYP--%s", - lifecycle - ); - taskInventoryManager = new TaskInventoryManager( - indexerZkConfig, - masterYp - ); - lifecycle.addManagedInstance(taskInventoryManager); - } - } - public void initializeTaskStorage() { if (taskStorage == null) { - if(config.getStorageImpl().equals("local")) { + if (config.getStorageImpl().equals("local")) { taskStorage = new LocalTaskStorage(); } else if (config.getStorageImpl().equals("db")) { final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class); @@ -481,13 +468,28 @@ public TaskRunner build() .build() ); + ScalingStrategy strategy = new S3AutoScalingStrategy( + new AmazonEC2Client( + new BasicAWSCredentials( + props.getProperty("com.metamx.aws.accessKey"), + props.getProperty("com.metamx.aws.secretKey") + ) + ), + configFactory.build(S3AutoScalingStrategyConfig.class) + ); + // TODO: remove this when AMI is ready + strategy = new NoopScalingStrategy(configFactory.build(S3AutoScalingStrategyConfig.class)); + return new RemoteTaskRunner( jsonMapper, - taskInventoryManager, - indexerZkConfig, + configFactory.build(RemoteTaskRunnerConfig.class), curatorFramework, + new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), false), retryScheduledExec, - new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)) + new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)), + new ConcurrentHashMap(), + new ConcurrentHashMap(), + strategy ); } }; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java new file mode 100644 index 000000000000..d878182b30ca --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java @@ -0,0 +1,124 @@ +package com.metamx.druid.merger.coordinator.scaling; + +import com.amazonaws.services.ec2.model.Instance; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.google.common.collect.MinMaxPriorityQueue; +import com.google.common.collect.Ordering; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.druid.merger.coordinator.WorkerWrapper; +import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; +import com.metamx.emitter.EmittingLogger; + +import java.util.Comparator; +import java.util.Map; + +/** + * This class just logs when scaling should occur. + */ +public class NoopScalingStrategy implements ScalingStrategy +{ + private static final EmittingLogger log = new EmittingLogger(NoopScalingStrategy.class); + + private final S3AutoScalingStrategyConfig config; + + private final Object lock = new Object(); + + private volatile String currentlyProvisioning = null; + private volatile String currentlyTerminating = null; + + public NoopScalingStrategy( + S3AutoScalingStrategyConfig config + ) + { + this.config = config; + } + + @Override + public void provisionIfNeeded(Map zkWorkers) + { + synchronized (lock) { + if (currentlyProvisioning != null) { + if (!zkWorkers.containsKey(currentlyProvisioning)) { + log.info( + "[%s] is still provisioning. Wait for it to finish before requesting new worker.", + currentlyProvisioning + ); + return; + } + } + + Iterable availableWorkers = FunctionalIterable.create(zkWorkers.values()).filter( + new Predicate() + { + @Override + public boolean apply(WorkerWrapper input) + { + return !input.isAtCapacity(); + } + } + ); + + if (Iterables.size(availableWorkers) == 0) { + try { + log.info("If I were a real strategy I'd create something now"); + currentlyProvisioning = "willNeverBeTrue"; + } + catch (Exception e) { + log.error(e, "Unable to create instance"); + currentlyProvisioning = null; + } + } + } + } + + @Override + public Instance terminateIfNeeded(Map zkWorkers) + { + synchronized (lock) { + if (currentlyTerminating != null) { + if (zkWorkers.containsKey(currentlyTerminating)) { + log.info("[%s] has not terminated. Wait for it to finish before terminating again.", currentlyTerminating); + return null; + } + } + + MinMaxPriorityQueue currWorkers = MinMaxPriorityQueue.orderedBy( + new Comparator() + { + @Override + public int compare(WorkerWrapper w1, WorkerWrapper w2) + { + return Ordering.natural() + .nullsFirst() + .compare(w1.getLastCompletedTaskTime(), w2.getLastCompletedTaskTime()); + } + } + ).create( + zkWorkers.values() + ); + + if (currWorkers.size() <= config.getMinNuMWorkers()) { + return null; + } + + WorkerWrapper thatLazyWorker = currWorkers.poll(); + + if (System.currentTimeMillis() - thatLazyWorker.getLastCompletedTaskTime().getMillis() + > config.getMillisToWaitBeforeTerminating()) { + try { + log.info("If I were a real strategy I'd terminate something now"); + currentlyTerminating = "willNeverBeTrue"; + + return null; + } + catch (Exception e) { + log.error(e, "Unable to terminate instance"); + currentlyTerminating = null; + } + } + + return null; + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java new file mode 100644 index 000000000000..de00b7922a4a --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java @@ -0,0 +1,186 @@ +/* + * 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.merger.coordinator.scaling; + +import com.amazonaws.services.ec2.AmazonEC2Client; +import com.amazonaws.services.ec2.model.DescribeInstancesRequest; +import com.amazonaws.services.ec2.model.DescribeInstancesResult; +import com.amazonaws.services.ec2.model.Filter; +import com.amazonaws.services.ec2.model.Instance; +import com.amazonaws.services.ec2.model.InstanceType; +import com.amazonaws.services.ec2.model.RunInstancesRequest; +import com.amazonaws.services.ec2.model.RunInstancesResult; +import com.amazonaws.services.ec2.model.TerminateInstancesRequest; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.google.common.collect.MinMaxPriorityQueue; +import com.metamx.common.ISE; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.druid.merger.coordinator.WorkerWrapper; +import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; +import com.metamx.emitter.EmittingLogger; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.Map; + +/** + */ +public class S3AutoScalingStrategy implements ScalingStrategy +{ + private static final EmittingLogger log = new EmittingLogger(S3AutoScalingStrategy.class); + + private final AmazonEC2Client amazonEC2Client; + private final S3AutoScalingStrategyConfig config; + + private final Object lock = new Object(); + + private volatile String currentlyProvisioning = null; + private volatile String currentlyTerminating = null; + + public S3AutoScalingStrategy( + AmazonEC2Client amazonEC2Client, + S3AutoScalingStrategyConfig config + ) + { + this.amazonEC2Client = amazonEC2Client; + this.config = config; + } + + @Override + public void provisionIfNeeded(Map zkWorkers) + { + synchronized (lock) { + if (zkWorkers.containsKey(currentlyProvisioning)) { + currentlyProvisioning = null; + } + + if (currentlyProvisioning != null) { + log.info( + "[%s] is still provisioning. Wait for it to finish before requesting new worker.", + currentlyProvisioning + ); + return; + } + + Iterable availableWorkers = FunctionalIterable.create(zkWorkers.values()).filter( + new Predicate() + { + @Override + public boolean apply(WorkerWrapper input) + { + return !input.isAtCapacity(); + } + } + ); + + if (Iterables.size(availableWorkers) == 0) { + try { + log.info("Creating a new instance"); + RunInstancesResult result = amazonEC2Client.runInstances( + new RunInstancesRequest(config.getAmiId(), 1, 1) + .withInstanceType(InstanceType.fromValue(config.getInstanceType())) + ); + + if (result.getReservation().getInstances().size() != 1) { + throw new ISE("Created more than one instance, WTF?!"); + } + + Instance instance = result.getReservation().getInstances().get(0); + log.info("Created instance: %s", instance.getInstanceId()); + log.debug("%s", instance); + + currentlyProvisioning = String.format("%s:%s", instance.getPrivateIpAddress(), config.getWorkerPort()); + } + catch (Exception e) { + log.error(e, "Unable to create instance"); + currentlyProvisioning = null; + } + } + } + } + + @Override + public Instance terminateIfNeeded(Map zkWorkers) + { + synchronized (lock) { + if (!zkWorkers.containsKey(currentlyTerminating)) { + currentlyProvisioning = null; + } + + if (currentlyTerminating != null) { + log.info("[%s] has not terminated. Wait for it to finish before terminating again.", currentlyTerminating); + return null; + } + + MinMaxPriorityQueue currWorkers = MinMaxPriorityQueue.orderedBy( + new Comparator() + { + @Override + public int compare(WorkerWrapper w1, WorkerWrapper w2) + { + return w1.getLastCompletedTaskTime().compareTo(w2.getLastCompletedTaskTime()); + } + } + ).create( + zkWorkers.values() + ); + + if (currWorkers.size() <= config.getMinNuMWorkers()) { + return null; + } + + WorkerWrapper thatLazyWorker = currWorkers.poll(); + + if (System.currentTimeMillis() - thatLazyWorker.getLastCompletedTaskTime().getMillis() + > config.getMillisToWaitBeforeTerminating()) { + DescribeInstancesResult result = amazonEC2Client.describeInstances( + new DescribeInstancesRequest() + .withFilters( + new Filter("private-ip-address", Arrays.asList(thatLazyWorker.getWorker().getIp())) + ) + ); + + if (result.getReservations().size() != 1 || result.getReservations().get(0).getInstances().size() != 1) { + throw new ISE("More than one node with the same private IP[%s], WTF?!", thatLazyWorker.getWorker().getIp()); + } + + Instance instance = result.getReservations().get(0).getInstances().get(0); + + try { + log.info("Terminating instance[%s]", instance.getInstanceId()); + amazonEC2Client.terminateInstances( + new TerminateInstancesRequest(Arrays.asList(instance.getInstanceId())) + ); + + currentlyTerminating = String.format("%s:%s", instance.getPrivateIpAddress(), config.getWorkerPort()); + + return instance; + } + catch (Exception e) { + log.error(e, "Unable to terminate instance"); + currentlyTerminating = null; + } + } + + return null; + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java new file mode 100644 index 000000000000..7aba31b0c250 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java @@ -0,0 +1,34 @@ +/* + * 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.merger.coordinator.scaling; + +import com.amazonaws.services.ec2.model.Instance; +import com.metamx.druid.merger.coordinator.WorkerWrapper; + +import java.util.Map; + +/** + */ +public interface ScalingStrategy +{ + public void provisionIfNeeded(Map zkWorkers); + + public Instance terminateIfNeeded(Map zkWorkers); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index b70a6e9c2788..938bda933fc6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -111,6 +111,7 @@ public void run() TaskStatus taskStatus; try { + workerCuratorCoordinator.unannounceTask(task.getId()); workerCuratorCoordinator.announceStatus(TaskStatus.running(task.getId())); taskStatus = task.run(taskContext, toolbox); } @@ -165,6 +166,7 @@ public void stop() { try { pathChildrenCache.close(); + exec.shutdown(); } catch (Exception e) { log.makeAlert(e, "Exception stopping TaskMonitor") diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java b/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java index 032febb79b9b..a1ebf273521b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java @@ -19,42 +19,44 @@ package com.metamx.druid.merger.worker; -import com.google.common.collect.ImmutableMap; -import com.metamx.common.logger.Logger; -import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.worker.config.WorkerConfig; import org.codehaus.jackson.annotate.JsonCreator; import org.codehaus.jackson.annotate.JsonProperty; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - /** + * A container for worker metadata. */ public class Worker { - private static final Logger log = new Logger(Worker.class); - private final String host; - - private final ConcurrentHashMap runningTasks; + private final String ip; + private final int capacity; + private final String version; public Worker( WorkerConfig config ) { this( - config.getHost() + config.getHost(), + config.getIp(), + config.getCapacity(), + config.getVersion() ); } @JsonCreator public Worker( - @JsonProperty("host") String host + @JsonProperty("host") String host, + @JsonProperty("ip") String ip, + @JsonProperty("capacity") int capacity, + @JsonProperty("version") String version ) { this.host = host; - this.runningTasks = new ConcurrentHashMap(); + this.ip = ip; + this.capacity = capacity; + this.version = version; } @JsonProperty @@ -63,25 +65,21 @@ public String getHost() return host; } - public Map getTasks() - { - return runningTasks; - } - - public Map getStringProps() + @JsonProperty + public String getIp() { - return ImmutableMap.of( - "host", host - ); + return ip; } - public TaskStatus addTask(TaskStatus status) + @JsonProperty + public int getCapacity() { - return runningTasks.put(status.getId(), status); + return capacity; } - public TaskStatus removeTask(String taskId) + @JsonProperty + public String getVersion() { - return runningTasks.remove(taskId); + return version; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java index 8510a890e1ca..cdf056a88faf 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java @@ -92,7 +92,7 @@ public void start() throws Exception makePathIfNotExisting( getAnnouncementsPathForWorker(), CreateMode.EPHEMERAL, - worker.getStringProps() + worker ); started = true; @@ -171,6 +171,16 @@ public boolean statusExists(String id) } } + public void unannounceTask(String taskId) + { + try { + curatorFramework.delete().guaranteed().forPath(getTaskPathForId(taskId)); + } + catch (Exception e) { + log.warn("Could not delete task path for task[%s], looks like it already went away", taskId); + } + } + public void announceStatus(TaskStatus status) { synchronized (lock) { diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java index 6d258ae9dd8d..e8e68749e2d2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java @@ -32,4 +32,14 @@ public abstract class WorkerConfig @Config("druid.host") public abstract String getHost(); + + @Config("druid.worker.ip") + public abstract String getIp(); + + @Config("druid.worker.version") + public abstract String getVersion(); + + @Config("druid.worker.capacity") + @Default("3") + public abstract int getCapacity(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index ee3888e12e4c..67e1e2e089f8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -29,6 +29,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.jackson.DefaultObjectMapper; @@ -294,8 +295,9 @@ public void initializeTaskToolbox() throws S3ServiceException public void initializeCuratorFramework() throws IOException { + final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class); curatorFramework = Initialization.makeCuratorFrameworkClient( - props.getProperty("druid.zk.service.host"), + curatorConfig, lifecycle ); } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java new file mode 100644 index 000000000000..0aab85d9cf19 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java @@ -0,0 +1,164 @@ +/* + * 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.merger.coordinator.scaling; + +import com.amazonaws.services.ec2.AmazonEC2Client; +import com.amazonaws.services.ec2.model.DescribeInstancesRequest; +import com.amazonaws.services.ec2.model.DescribeInstancesResult; +import com.amazonaws.services.ec2.model.Instance; +import com.amazonaws.services.ec2.model.Reservation; +import com.amazonaws.services.ec2.model.RunInstancesRequest; +import com.amazonaws.services.ec2.model.RunInstancesResult; +import com.amazonaws.services.ec2.model.TerminateInstancesRequest; +import com.google.common.collect.Maps; +import com.metamx.druid.merger.coordinator.WorkerWrapper; +import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; +import com.metamx.druid.merger.worker.Worker; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Date; +import java.util.Map; +import java.util.concurrent.ConcurrentSkipListSet; + +/** + */ +public class S3AutoScalingStrategyTest +{ + private static final String AMI_ID = "dummy"; + private static final String INSTANCE_ID = "theInstance"; + + private AmazonEC2Client amazonEC2Client; + private RunInstancesResult runInstancesResult; + private DescribeInstancesResult describeInstancesResult; + private Reservation reservation; + private Instance instance; + private WorkerWrapper worker; + private S3AutoScalingStrategy strategy; + + @Before + public void setUp() throws Exception + { + amazonEC2Client = EasyMock.createMock(AmazonEC2Client.class); + runInstancesResult = EasyMock.createMock(RunInstancesResult.class); + describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class); + reservation = EasyMock.createMock(Reservation.class); + + instance = new Instance().withInstanceId(INSTANCE_ID).withLaunchTime(new Date()).withImageId(AMI_ID); + + worker = new WorkerWrapper( + new Worker("dummyHost", "dummyIP", 2, "0"), + new ConcurrentSkipListSet(), + null + ); + worker.setLastCompletedTaskTime(new DateTime(0)); + strategy = new S3AutoScalingStrategy( + amazonEC2Client, new S3AutoScalingStrategyConfig() + { + @Override + public String getAmiId() + { + return AMI_ID; + } + + @Override + public String getWorkerPort() + { + return "8080"; + } + + @Override + public String getInstanceType() + { + return "t1.micro"; + } + + @Override + public long getMillisToWaitBeforeTerminating() + { + return 0; + } + + @Override + public int getMinNuMWorkers() + { + return 0; + } + } + ); + } + + @After + public void tearDown() throws Exception + { + EasyMock.verify(amazonEC2Client); + EasyMock.verify(runInstancesResult); + EasyMock.verify(describeInstancesResult); + EasyMock.verify(reservation); + } + + @Test + public void testScale() + { + EasyMock.expect(amazonEC2Client.runInstances(EasyMock.anyObject(RunInstancesRequest.class))).andReturn( + runInstancesResult + ); + EasyMock.expect(amazonEC2Client.describeInstances(EasyMock.anyObject(DescribeInstancesRequest.class))) + .andReturn(describeInstancesResult); + EasyMock.expect(amazonEC2Client.terminateInstances(EasyMock.anyObject(TerminateInstancesRequest.class))) + .andReturn(null); + EasyMock.replay(amazonEC2Client); + + EasyMock.expect(runInstancesResult.getReservation()).andReturn(reservation).atLeastOnce(); + EasyMock.replay(runInstancesResult); + + EasyMock.expect(describeInstancesResult.getReservations()).andReturn(Arrays.asList(reservation)).atLeastOnce(); + EasyMock.replay(describeInstancesResult); + + EasyMock.expect(reservation.getInstances()).andReturn(Arrays.asList(instance)).atLeastOnce(); + EasyMock.replay(reservation); + + Map zkWorkers = Maps.newHashMap(); + + zkWorkers.put(worker.getWorker().getHost(), worker); + + worker.getRunningTasks().add("task1"); + + Assert.assertFalse(worker.isAtCapacity()); + + worker.getRunningTasks().add("task2"); + + Assert.assertTrue(worker.isAtCapacity()); + + strategy.provisionIfNeeded(zkWorkers); + + worker.getRunningTasks().remove("task1"); + worker.getRunningTasks().remove("task2"); + + Instance deleted = strategy.terminateIfNeeded(zkWorkers); + + Assert.assertEquals(deleted.getInstanceId(), INSTANCE_ID); + } +} diff --git a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusherConfig.java b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusherConfig.java index 8bed74b8f820..96a96eeea100 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusherConfig.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusherConfig.java @@ -26,7 +26,6 @@ */ public abstract class S3SegmentPusherConfig { - @Config("druid.pusher.s3.bucket") public abstract String getBucket(); diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index 52de6ffad81a..fe65e14b0751 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -155,13 +155,13 @@ public static void main(String[] args) throws Exception final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); CuratorFramework curatorFramework = Initialization.makeCuratorFrameworkClient( - serviceDiscoveryConfig.getZkHosts(), + serviceDiscoveryConfig, lifecycle ); final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( curatorFramework, - configFactory.build(ServiceDiscoveryConfig.class), + serviceDiscoveryConfig, lifecycle ); From 9547ea494dee120b3478302f947a732702be6208 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 31 Oct 2012 14:45:44 -0700 Subject: [PATCH 04/15] fix bug with provision --- .../merger/coordinator/RemoteTaskRunner.java | 2 +- .../scaling/NoopScalingStrategy.java | 36 +++++-------- .../scaling/S3AutoScalingStrategy.java | 54 ++++++++----------- .../coordinator/scaling/ScalingStrategy.java | 2 +- .../scaling/S3AutoScalingStrategyTest.java | 2 +- 5 files changed, 37 insertions(+), 59 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 5ed630e607b0..5e8b31078af3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -439,7 +439,7 @@ public boolean apply(WorkerWrapper input) if (workerQueue.isEmpty()) { log.makeAlert("There are no worker nodes with capacity to run task!").emit(); - strategy.provisionIfNeeded(zkWorkers); + strategy.provision(zkWorkers); return null; } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java index d878182b30ca..81f9550a24ef 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java @@ -9,6 +9,7 @@ import com.metamx.druid.merger.coordinator.WorkerWrapper; import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; import com.metamx.emitter.EmittingLogger; +import org.joda.time.DateTime; import java.util.Comparator; import java.util.Map; @@ -35,7 +36,7 @@ public NoopScalingStrategy( } @Override - public void provisionIfNeeded(Map zkWorkers) + public void provision(Map zkWorkers) { synchronized (lock) { if (currentlyProvisioning != null) { @@ -48,26 +49,13 @@ public void provisionIfNeeded(Map zkWorkers) } } - Iterable availableWorkers = FunctionalIterable.create(zkWorkers.values()).filter( - new Predicate() - { - @Override - public boolean apply(WorkerWrapper input) - { - return !input.isAtCapacity(); - } - } - ); - - if (Iterables.size(availableWorkers) == 0) { - try { - log.info("If I were a real strategy I'd create something now"); - currentlyProvisioning = "willNeverBeTrue"; - } - catch (Exception e) { - log.error(e, "Unable to create instance"); - currentlyProvisioning = null; - } + try { + log.info("If I were a real strategy I'd create something now"); + currentlyProvisioning = "willNeverBeTrue"; + } + catch (Exception e) { + log.error(e, "Unable to create instance"); + currentlyProvisioning = null; } } } @@ -89,9 +77,9 @@ public Instance terminateIfNeeded(Map zkWorkers) @Override public int compare(WorkerWrapper w1, WorkerWrapper w2) { - return Ordering.natural() - .nullsFirst() - .compare(w1.getLastCompletedTaskTime(), w2.getLastCompletedTaskTime()); + DateTime w1Time = (w1 == null) ? new DateTime(0) : w1.getLastCompletedTaskTime(); + DateTime w2Time = (w2 == null) ? new DateTime(0) : w2.getLastCompletedTaskTime(); + return w1Time.compareTo(w2Time); } } ).create( diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java index de00b7922a4a..840dba3a2a83 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java @@ -36,6 +36,7 @@ import com.metamx.druid.merger.coordinator.WorkerWrapper; import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; import com.metamx.emitter.EmittingLogger; +import org.joda.time.DateTime; import java.util.Arrays; import java.util.Comparator; @@ -65,7 +66,7 @@ public S3AutoScalingStrategy( } @Override - public void provisionIfNeeded(Map zkWorkers) + public void provision(Map zkWorkers) { synchronized (lock) { if (zkWorkers.containsKey(currentlyProvisioning)) { @@ -80,39 +81,26 @@ public void provisionIfNeeded(Map zkWorkers) return; } - Iterable availableWorkers = FunctionalIterable.create(zkWorkers.values()).filter( - new Predicate() - { - @Override - public boolean apply(WorkerWrapper input) - { - return !input.isAtCapacity(); - } - } - ); - - if (Iterables.size(availableWorkers) == 0) { - try { - log.info("Creating a new instance"); - RunInstancesResult result = amazonEC2Client.runInstances( - new RunInstancesRequest(config.getAmiId(), 1, 1) - .withInstanceType(InstanceType.fromValue(config.getInstanceType())) - ); + try { + log.info("Creating a new instance"); + RunInstancesResult result = amazonEC2Client.runInstances( + new RunInstancesRequest(config.getAmiId(), 1, 1) + .withInstanceType(InstanceType.fromValue(config.getInstanceType())) + ); - if (result.getReservation().getInstances().size() != 1) { - throw new ISE("Created more than one instance, WTF?!"); - } + if (result.getReservation().getInstances().size() != 1) { + throw new ISE("Created more than one instance, WTF?!"); + } - Instance instance = result.getReservation().getInstances().get(0); - log.info("Created instance: %s", instance.getInstanceId()); - log.debug("%s", instance); + Instance instance = result.getReservation().getInstances().get(0); + log.info("Created instance: %s", instance.getInstanceId()); + log.debug("%s", instance); - currentlyProvisioning = String.format("%s:%s", instance.getPrivateIpAddress(), config.getWorkerPort()); - } - catch (Exception e) { - log.error(e, "Unable to create instance"); - currentlyProvisioning = null; - } + currentlyProvisioning = String.format("%s:%s", instance.getPrivateIpAddress(), config.getWorkerPort()); + } + catch (Exception e) { + log.error(e, "Unable to create instance"); + currentlyProvisioning = null; } } } @@ -136,7 +124,9 @@ public Instance terminateIfNeeded(Map zkWorkers) @Override public int compare(WorkerWrapper w1, WorkerWrapper w2) { - return w1.getLastCompletedTaskTime().compareTo(w2.getLastCompletedTaskTime()); + DateTime w1Time = (w1 == null) ? new DateTime(0) : w1.getLastCompletedTaskTime(); + DateTime w2Time = (w2 == null) ? new DateTime(0) : w2.getLastCompletedTaskTime(); + return w1Time.compareTo(w2Time); } } ).create( diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java index 7aba31b0c250..6a779927b87b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java @@ -28,7 +28,7 @@ */ public interface ScalingStrategy { - public void provisionIfNeeded(Map zkWorkers); + public void provision(Map zkWorkers); public Instance terminateIfNeeded(Map zkWorkers); } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java index 0aab85d9cf19..de486cbee572 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java @@ -152,7 +152,7 @@ public void testScale() Assert.assertTrue(worker.isAtCapacity()); - strategy.provisionIfNeeded(zkWorkers); + strategy.provision(zkWorkers); worker.getRunningTasks().remove("task1"); worker.getRunningTasks().remove("task2"); From 83a51a801e24c4460c71401496c976dd347cb3a3 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 31 Oct 2012 16:43:47 -0700 Subject: [PATCH 05/15] cleanup redirects and reannounce workers on connection loss --- .../merger/coordinator/RemoteTaskRunner.java | 5 +++- .../http/IndexerCoordinatorNode.java | 4 ---- .../worker/WorkerCuratorCoordinator.java | 24 +++++++++++++++++++ .../com/metamx/druid/http/MasterMain.java | 4 ---- .../com/metamx/druid/http/RedirectFilter.java | 24 ++----------------- .../metamx/druid/http/RedirectServlet.java | 4 +++- 6 files changed, 33 insertions(+), 32 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 5e8b31078af3..16d0ff2c6d86 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -207,6 +207,9 @@ private boolean assignTask(TaskWrapper taskWrapper) WorkerWrapper workerWrapper; if ((workerWrapper = findWorkerRunningTask(taskWrapper)) != null) { final Worker worker = workerWrapper.getWorker(); + + log.info("Worker[%s] is already running task{%s].", worker.getHost(), taskWrapper.getTask().getId()); + TaskStatus taskStatus = jsonMapper.readValue( cf.getData().forPath(JOINER.join(config.getStatusPath(), worker.getHost(), taskWrapper.getTask().getId())), TaskStatus.class @@ -454,7 +457,7 @@ public boolean apply(WorkerWrapper input) * Creates a ZK entry under a specific path associated with a worker. The worker is responsible for * removing the task ZK entry and creating a task status ZK entry. * - * @param theWorker The worker the task is assigned to + * @param theWorker The worker the task is assigned to * @param taskWrapper The task to be assigned */ private void announceTask(Worker theWorker, TaskWrapper taskWrapper) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 8b0d5eff7478..30d3b4fd53eb 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -230,10 +230,6 @@ public void init() throws Exception root.addFilter( new FilterHolder( new RedirectFilter( - HttpClientInit.createClient( - HttpClientConfig.builder().withNumConnections(1).build(), - new Lifecycle() - ), new ToStringResponseHandler(Charsets.UTF_8), new RedirectInfo() { diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java index cdf056a88faf..9375ef816963 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java @@ -28,6 +28,8 @@ import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.netflix.curator.framework.CuratorFramework; +import com.netflix.curator.framework.state.ConnectionState; +import com.netflix.curator.framework.state.ConnectionStateListener; import org.apache.zookeeper.CreateMode; import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; @@ -95,6 +97,28 @@ public void start() throws Exception worker ); + curatorFramework.getConnectionStateListenable().addListener( + new ConnectionStateListener() + { + @Override + public void stateChanged(CuratorFramework client, ConnectionState newState) + { + try { + if (newState.equals(ConnectionState.RECONNECTED)) { + makePathIfNotExisting( + getAnnouncementsPathForWorker(), + CreateMode.EPHEMERAL, + worker + ); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + started = true; } } diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index fe65e14b0751..d0c77df081aa 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -267,10 +267,6 @@ public URL getRedirectURL(String queryString, String requestURI) root.addFilter( new FilterHolder( new RedirectFilter( - HttpClientInit.createClient( - HttpClientConfig.builder().withNumConnections(1).build(), - new Lifecycle() - ), new ToStringResponseHandler(Charsets.UTF_8), redirectInfo ) diff --git a/server/src/main/java/com/metamx/druid/http/RedirectFilter.java b/server/src/main/java/com/metamx/druid/http/RedirectFilter.java index 45fac15d4078..28bdb3f5417a 100644 --- a/server/src/main/java/com/metamx/druid/http/RedirectFilter.java +++ b/server/src/main/java/com/metamx/druid/http/RedirectFilter.java @@ -43,17 +43,14 @@ public class RedirectFilter implements Filter { private static final Logger log = new Logger(RedirectFilter.class); - private final HttpClient httpClient; private final HttpResponseHandler responseHandler; private final RedirectInfo redirectInfo; public RedirectFilter( - HttpClient httpClient, HttpResponseHandler responseHandler, RedirectInfo redirectInfo ) { - this.httpClient = httpClient; this.responseHandler = responseHandler; this.redirectInfo = redirectInfo; } @@ -82,28 +79,11 @@ public void doFilter(ServletRequest req, ServletResponse res, FilterChain chain) URL url = redirectInfo.getRedirectURL(request.getQueryString(), request.getRequestURI()); log.info("Forwarding request to [%s]", url); - if (request.getMethod().equals(HttpMethod.POST)) { - try { - forward(request, url); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } else { - response.sendRedirect(url.toString()); - } + response.setStatus(HttpServletResponse.SC_MOVED_TEMPORARILY); + response.setHeader("Location", url.toString()); } } @Override public void destroy() {} - - private void forward(HttpServletRequest req, URL url) throws Exception - { - byte[] requestQuery = ByteStreams.toByteArray(req.getInputStream()); - httpClient.post(url) - .setContent("application/json", requestQuery) - .go(responseHandler) - .get(); - } } diff --git a/server/src/main/java/com/metamx/druid/http/RedirectServlet.java b/server/src/main/java/com/metamx/druid/http/RedirectServlet.java index f91f03d987d6..0f7c030d6100 100644 --- a/server/src/main/java/com/metamx/druid/http/RedirectServlet.java +++ b/server/src/main/java/com/metamx/druid/http/RedirectServlet.java @@ -65,7 +65,9 @@ public void service(ServletRequest req, ServletResponse res) } else { URL url = redirectInfo.getRedirectURL(request.getQueryString(), request.getRequestURI()); log.info("Forwarding request to [%s]", url); - response.sendRedirect(url.toString()); + + response.setStatus(HttpServletResponse.SC_MOVED_TEMPORARILY); + response.setHeader("Location", url.toString()); } } } From abab35194a10a6eb1d92f30c6fbaed9cd512690b Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 2 Nov 2012 15:56:18 -0700 Subject: [PATCH 06/15] general cleanup and improvments for auto scaling --- .../merger/coordinator/RemoteTaskRunner.java | 165 +++++++++++----- .../merger/coordinator/WorkerWrapper.java | 16 +- ...java => EC2AutoScalingStrategyConfig.java} | 13 +- .../config/RemoteTaskRunnerConfig.java | 8 + .../http/IndexerCoordinatorNode.java | 19 +- .../coordinator/scaling/AutoScalingData.java | 27 +++ .../scaling/EC2AutoScalingStrategy.java | 162 ++++++++++++++++ .../scaling/NoopScalingStrategy.java | 96 +--------- .../scaling/S3AutoScalingStrategy.java | 176 ------------------ .../coordinator/scaling/ScalingStrategy.java | 11 +- .../worker/WorkerCuratorCoordinator.java | 4 +- .../merger/worker/config/WorkerConfig.java | 7 +- ...t.java => EC2AutoScalingStrategyTest.java} | 43 +++-- 13 files changed, 386 insertions(+), 361 deletions(-) rename merger/src/main/java/com/metamx/druid/merger/coordinator/config/{S3AutoScalingStrategyConfig.java => EC2AutoScalingStrategyConfig.java} (78%) create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java create mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java delete mode 100644 merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java rename merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/{S3AutoScalingStrategyTest.java => EC2AutoScalingStrategyTest.java} (78%) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 16d0ff2c6d86..3c08b397d77b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -19,9 +19,11 @@ package com.metamx.druid.merger.coordinator; +import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Predicate; import com.google.common.base.Throwables; +import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.primitives.Ints; import com.metamx.common.ISE; @@ -34,10 +36,12 @@ import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; +import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; import com.metamx.druid.merger.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; +import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; @@ -47,8 +51,10 @@ import org.joda.time.Duration; import org.joda.time.Period; +import javax.annotation.Nullable; import java.io.IOException; import java.util.Comparator; +import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ScheduledExecutorService; @@ -76,13 +82,18 @@ public class RemoteTaskRunner implements TaskRunner private final ObjectMapper jsonMapper; private final RemoteTaskRunnerConfig config; private final CuratorFramework cf; - private final PathChildrenCache workerListener; + private final PathChildrenCache workerPathCache; private final ScheduledExecutorService scheduledExec; private final RetryPolicyFactory retryPolicyFactory; - private final ConcurrentHashMap zkWorkers; // all workers that exist in ZK - private final ConcurrentHashMap tasks; // all tasks that are assigned or need to be assigned private final ScalingStrategy strategy; + // all workers that exist in ZK + private final ConcurrentHashMap zkWorkers = new ConcurrentHashMap(); + // all tasks that are assigned or need to be assigned + private final ConcurrentHashMap tasks = new ConcurrentHashMap(); + + private final ConcurrentSkipListSet currentlyProvisioning = new ConcurrentSkipListSet(); + private final ConcurrentSkipListSet currentlyTerminating = new ConcurrentSkipListSet(); private final Object statusLock = new Object(); private volatile boolean started = false; @@ -91,22 +102,18 @@ public RemoteTaskRunner( ObjectMapper jsonMapper, RemoteTaskRunnerConfig config, CuratorFramework cf, - PathChildrenCache workerListener, + PathChildrenCache workerPathCache, ScheduledExecutorService scheduledExec, RetryPolicyFactory retryPolicyFactory, - ConcurrentHashMap zkWorkers, - ConcurrentHashMap tasks, ScalingStrategy strategy ) { this.jsonMapper = jsonMapper; this.config = config; this.cf = cf; - this.workerListener = workerListener; + this.workerPathCache = workerPathCache; this.scheduledExec = scheduledExec; this.retryPolicyFactory = retryPolicyFactory; - this.zkWorkers = zkWorkers; - this.tasks = tasks; this.strategy = strategy; } @@ -114,27 +121,23 @@ public RemoteTaskRunner( public void start() { try { - workerListener.start(); - workerListener.getListenable().addListener( + workerPathCache.start(); + workerPathCache.getListenable().addListener( new PathChildrenCacheListener() { @Override public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception { + final Worker worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final Worker worker = jsonMapper.readValue( - cf.getData().forPath(event.getData().getPath()), - Worker.class - ); - log.info("New worker[%s] found!", worker.getHost()); addWorker(worker); } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { - // Get the worker host from the path - String workerHost = event.getData().getPath().substring(event.getData().getPath().lastIndexOf("/") + 1); - - log.info("Worker[%s] removed!", workerHost); - removeWorker(workerHost); + log.info("Worker[%s] removed!", worker.getHost()); + removeWorker(worker.getHost()); } } } @@ -157,11 +160,51 @@ public void childEvent(CuratorFramework client, final PathChildrenCacheEvent eve @Override public void run() { - strategy.terminateIfNeeded(zkWorkers); + if (currentlyTerminating.isEmpty()) { + if (zkWorkers.size() <= config.getMinNumWorkers()) { + return; + } + + List thoseLazyWorkers = Lists.newArrayList( + FunctionalIterable + .create(zkWorkers.values()) + .filter( + new Predicate() + { + @Override + public boolean apply(@Nullable WorkerWrapper input) + { + return System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() + > config.getmaxWorkerIdleTimeMillisBeforeDeletion(); + } + } + ) + ); + + AutoScalingData terminated = strategy.terminate( + Lists.transform( + thoseLazyWorkers, + new Function() + { + @Override + public String apply(@Nullable WorkerWrapper input) + { + return input.getWorker().getHost(); + } + } + ) + ); + + currentlyTerminating.addAll(terminated.getNodeIds()); + } else { + log.info( + "[%s] still terminating. Wait for all nodes to terminate before trying again.", + currentlyTerminating + ); + } } } ); - started = true; } catch (Exception e) { @@ -174,7 +217,7 @@ public void stop() { try { for (WorkerWrapper workerWrapper : zkWorkers.values()) { - workerWrapper.getWatcher().close(); + workerWrapper.close(); } } catch (Exception e) { @@ -202,16 +245,24 @@ public void run(Task task, TaskContext context, TaskCallback callback) private boolean assignTask(TaskWrapper taskWrapper) { - // If the task already exists, we don't need to announce it try { - WorkerWrapper workerWrapper; - if ((workerWrapper = findWorkerRunningTask(taskWrapper)) != null) { + WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); + // If the task already exists, we don't need to announce it + if (workerWrapper != null) { final Worker worker = workerWrapper.getWorker(); - log.info("Worker[%s] is already running task{%s].", worker.getHost(), taskWrapper.getTask().getId()); + log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskWrapper.getTask().getId()); TaskStatus taskStatus = jsonMapper.readValue( - cf.getData().forPath(JOINER.join(config.getStatusPath(), worker.getHost(), taskWrapper.getTask().getId())), + workerWrapper.getStatusCache() + .getCurrentData( + JOINER.join( + config.getStatusPath(), + worker.getHost(), + taskWrapper.getTask().getId() + ) + ) + .getData(), TaskStatus.class ); @@ -222,7 +273,7 @@ private boolean assignTask(TaskWrapper taskWrapper) } new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()).run(); } else { - tasks.put(taskWrapper.getTask().getId(), taskWrapper); + tasks.putIfAbsent(taskWrapper.getTask().getId(), taskWrapper); } return true; } @@ -301,19 +352,36 @@ public void run() private void addWorker(final Worker worker) { try { - final String workerStatus = JOINER.join(config.getStatusPath(), worker.getHost()); + currentlyProvisioning.remove(worker.getHost()); + + final String workerStatusPath = JOINER.join(config.getStatusPath(), worker.getHost()); + final PathChildrenCache statusCache = new PathChildrenCache(cf, workerStatusPath, true); final ConcurrentSkipListSet runningTasks = new ConcurrentSkipListSet( - cf.getChildren().forPath(workerStatus) + Lists.transform( + statusCache.getCurrentData(), + 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); + } + } + } + ) ); - final PathChildrenCache watcher = new PathChildrenCache(cf, workerStatus, false); final WorkerWrapper workerWrapper = new WorkerWrapper( worker, runningTasks, - watcher + statusCache ); // Add status listener to the watcher for status changes - watcher.getListenable().addListener( + statusCache.getListenable().addListener( new PathChildrenCacheListener() { @Override @@ -323,9 +391,8 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th String taskId = null; try { if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - String statusPath = event.getData().getPath(); TaskStatus taskStatus = jsonMapper.readValue( - cf.getData().forPath(statusPath), TaskStatus.class + event.getData().getData(), TaskStatus.class ); taskId = taskStatus.getId(); @@ -335,7 +402,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { String statusPath = event.getData().getPath(); TaskStatus taskStatus = jsonMapper.readValue( - cf.getData().forPath(statusPath), TaskStatus.class + event.getData().getData(), TaskStatus.class ); taskId = taskStatus.getId(); @@ -369,7 +436,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th } ); zkWorkers.put(worker.getHost(), workerWrapper); - watcher.start(); + statusCache.start(); } catch (Exception e) { throw Throwables.propagate(e); @@ -394,6 +461,8 @@ private WorkerWrapper findWorkerRunningTask(TaskWrapper taskWrapper) */ private void removeWorker(final String workerId) { + currentlyTerminating.remove(workerId); + WorkerWrapper workerWrapper = zkWorkers.get(workerId); if (workerWrapper != null) { for (String taskId : workerWrapper.getRunningTasks()) { @@ -405,7 +474,7 @@ private void removeWorker(final String workerId) } try { - workerWrapper.getWatcher().close(); + workerWrapper.getStatusCache().close(); } catch (IOException e) { log.error("Failed to close watcher associated with worker[%s]", workerWrapper.getWorker().getHost()); @@ -441,8 +510,19 @@ public boolean apply(WorkerWrapper input) ); if (workerQueue.isEmpty()) { - log.makeAlert("There are no worker nodes with capacity to run task!").emit(); - strategy.provision(zkWorkers); + log.info("Worker nodes do not have capacity to run any more tasks!"); + + if (currentlyProvisioning.isEmpty()) { + AutoScalingData provisioned = strategy.provision(); + if (provisioned != null) { + currentlyProvisioning.addAll(provisioned.getNodeIds()); + } + } else { + log.info( + "[%s] still provisioning. Wait for all provisioned nodes to complete before requesting new worker.", + currentlyProvisioning + ); + } return null; } @@ -471,7 +551,6 @@ private void announceTask(Worker theWorker, TaskWrapper taskWrapper) tasks.put(task.getId(), taskWrapper); cf.create() - .creatingParentsIfNeeded() .withMode(CreateMode.EPHEMERAL) .forPath( JOINER.join( diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index 047533ae8589..83440b2500b3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -23,6 +23,7 @@ import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import org.joda.time.DateTime; +import java.io.IOException; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; @@ -32,15 +33,15 @@ public class WorkerWrapper { private final Worker worker; private final ConcurrentSkipListSet runningTasks; - private final PathChildrenCache watcher; + private final PathChildrenCache statusCache; private volatile DateTime lastCompletedTaskTime; - public WorkerWrapper(Worker worker, ConcurrentSkipListSet runningTasks, PathChildrenCache watcher) + public WorkerWrapper(Worker worker, ConcurrentSkipListSet runningTasks, PathChildrenCache statusCache) { this.worker = worker; this.runningTasks = runningTasks; - this.watcher = watcher; + this.statusCache = statusCache; } public Worker getWorker() @@ -53,9 +54,9 @@ public Set getRunningTasks() return runningTasks; } - public PathChildrenCache getWatcher() + public PathChildrenCache getStatusCache() { - return watcher; + return statusCache; } public DateTime getLastCompletedTaskTime() @@ -77,4 +78,9 @@ public void removeTask(String taskId) { runningTasks.remove(taskId); } + + public void close() throws IOException + { + statusCache.close(); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/S3AutoScalingStrategyConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java similarity index 78% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/config/S3AutoScalingStrategyConfig.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java index 1aa2145cdc37..3f7b9a0171fd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/S3AutoScalingStrategyConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java @@ -24,7 +24,7 @@ /** */ -public abstract class S3AutoScalingStrategyConfig +public abstract class EC2AutoScalingStrategyConfig { @Config("druid.indexer.amiId") public abstract String getAmiId(); @@ -36,12 +36,11 @@ public abstract class S3AutoScalingStrategyConfig @Config("druid.indexer.instanceType") public abstract String getInstanceType(); - @Config("druid.indexer.millisToWaitBeforeTerminating") - @Default("1800000") // 30 mins - public abstract long getMillisToWaitBeforeTerminating(); + @Config("druid.indexer.minNumInstancesToProvision") + @Default("1") + public abstract int getMinNumInstancesToProvision(); - // minimum number of workers that must always be running - @Config("druid.indexer.minNumWorkers") + @Config("druid.indexer.maxNumInstancesToProvision") @Default("1") - public abstract int getMinNuMWorkers(); + public abstract int getMaxNumInstancesToProvision(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java index 3816468250f0..a9cf6b02f932 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -37,4 +37,12 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig @Config("druid.indexer.minWorkerVersion") public abstract String getMinWorkerVersion(); + + @Config("druid.indexer.minNumWorkers") + @Default("1") + public abstract int getMinNumWorkers(); + + @Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion") + @Default("1") + public abstract int getmaxWorkerIdleTimeMillisBeforeDeletion(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 30d3b4fd53eb..f907f2f5add7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -59,15 +59,13 @@ import com.metamx.druid.merger.coordinator.TaskRunner; import com.metamx.druid.merger.coordinator.TaskRunnerFactory; import com.metamx.druid.merger.coordinator.TaskStorage; -import com.metamx.druid.merger.coordinator.TaskWrapper; -import com.metamx.druid.merger.coordinator.WorkerWrapper; +import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; -import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; +import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.NoopScalingStrategy; -import com.metamx.druid.merger.coordinator.scaling.S3AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; import com.metamx.druid.realtime.S3SegmentPusher; import com.metamx.druid.realtime.S3SegmentPusherConfig; @@ -102,7 +100,6 @@ import java.net.URL; import java.util.List; import java.util.Properties; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -464,27 +461,25 @@ public TaskRunner build() .build() ); - ScalingStrategy strategy = new S3AutoScalingStrategy( + ScalingStrategy strategy = new EC2AutoScalingStrategy( new AmazonEC2Client( new BasicAWSCredentials( props.getProperty("com.metamx.aws.accessKey"), props.getProperty("com.metamx.aws.secretKey") ) ), - configFactory.build(S3AutoScalingStrategyConfig.class) + configFactory.build(EC2AutoScalingStrategyConfig.class) ); - // TODO: remove this when AMI is ready - strategy = new NoopScalingStrategy(configFactory.build(S3AutoScalingStrategyConfig.class)); + // TODO: use real strategy before actual deployment + strategy = new NoopScalingStrategy(); return new RemoteTaskRunner( jsonMapper, configFactory.build(RemoteTaskRunnerConfig.class), curatorFramework, - new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), false), + new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true), retryScheduledExec, new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)), - new ConcurrentHashMap(), - new ConcurrentHashMap(), strategy ); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java new file mode 100644 index 000000000000..6cce08f8731c --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java @@ -0,0 +1,27 @@ +package com.metamx.druid.merger.coordinator.scaling; + +import java.util.List; + +/** + */ +public class AutoScalingData +{ + private final List nodeIds; + private final List nodes; + + public AutoScalingData(List nodeIds, List nodes) + { + this.nodeIds = nodeIds; + this.nodes = nodes; + } + + public List getNodeIds() + { + return nodeIds; + } + + public List getNodes() + { + return nodes; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java new file mode 100644 index 000000000000..cd94b70d3ce4 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -0,0 +1,162 @@ +/* + * 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.merger.coordinator.scaling; + +import com.amazonaws.services.ec2.AmazonEC2Client; +import com.amazonaws.services.ec2.model.DescribeInstancesRequest; +import com.amazonaws.services.ec2.model.DescribeInstancesResult; +import com.amazonaws.services.ec2.model.Filter; +import com.amazonaws.services.ec2.model.Instance; +import com.amazonaws.services.ec2.model.InstanceType; +import com.amazonaws.services.ec2.model.Reservation; +import com.amazonaws.services.ec2.model.RunInstancesRequest; +import com.amazonaws.services.ec2.model.RunInstancesResult; +import com.amazonaws.services.ec2.model.TerminateInstancesRequest; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; +import com.metamx.emitter.EmittingLogger; + +import javax.annotation.Nullable; +import java.util.List; + +/** + */ +public class EC2AutoScalingStrategy implements ScalingStrategy +{ + private static final EmittingLogger log = new EmittingLogger(EC2AutoScalingStrategy.class); + + private final AmazonEC2Client amazonEC2Client; + private final EC2AutoScalingStrategyConfig config; + + public EC2AutoScalingStrategy( + AmazonEC2Client amazonEC2Client, + EC2AutoScalingStrategyConfig config + ) + { + this.amazonEC2Client = amazonEC2Client; + this.config = config; + } + + @Override + public AutoScalingData provision() + { + try { + log.info("Creating new instance(s)..."); + RunInstancesResult result = amazonEC2Client.runInstances( + new RunInstancesRequest( + config.getAmiId(), + config.getMinNumInstancesToProvision(), + config.getMaxNumInstancesToProvision() + ) + .withInstanceType(InstanceType.fromValue(config.getInstanceType())) + ); + + List instanceIds = Lists.transform( + result.getReservation().getInstances(), + new Function() + { + @Override + public String apply(@Nullable Instance input) + { + return input.getInstanceId(); + } + } + ); + + log.info("Created instances: %s", instanceIds); + + return new AutoScalingData( + Lists.transform( + result.getReservation().getInstances(), + new Function() + { + @Override + public String apply(@Nullable Instance input) + { + return String.format("%s:%s", input.getPrivateIpAddress(), config.getWorkerPort()); + } + } + ), + result.getReservation().getInstances() + ); + } + catch (Exception e) { + log.error(e, "Unable to provision any EC2 instances."); + } + + return null; + } + + @Override + public AutoScalingData terminate(List nodeIds) + { + DescribeInstancesResult result = amazonEC2Client.describeInstances( + new DescribeInstancesRequest() + .withFilters( + new Filter("private-ip-address", nodeIds) + ) + ); + + List instances = Lists.newArrayList(); + for (Reservation reservation : result.getReservations()) { + instances.addAll(reservation.getInstances()); + } + + try { + log.info("Terminating instance[%s]", instances); + amazonEC2Client.terminateInstances( + new TerminateInstancesRequest( + Lists.transform( + instances, + new Function() + { + @Override + public String apply(@Nullable Instance input) + { + return input.getInstanceId(); + } + } + ) + ) + ); + + return new AutoScalingData( + Lists.transform( + instances, + new Function() + { + @Override + public String apply(@Nullable Instance input) + { + return String.format("%s:%s", input.getPrivateIpAddress(), config.getWorkerPort()); + } + } + ), + instances + ); + } + catch (Exception e) { + log.error(e, "Unable to terminate any instances."); + } + + return null; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java index 81f9550a24ef..0331022082c3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java @@ -1,112 +1,34 @@ package com.metamx.druid.merger.coordinator.scaling; import com.amazonaws.services.ec2.model.Instance; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; import com.google.common.collect.MinMaxPriorityQueue; -import com.google.common.collect.Ordering; -import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.merger.coordinator.WorkerWrapper; -import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; +import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; import java.util.Comparator; +import java.util.List; import java.util.Map; /** * This class just logs when scaling should occur. */ -public class NoopScalingStrategy implements ScalingStrategy +public class NoopScalingStrategy implements ScalingStrategy { private static final EmittingLogger log = new EmittingLogger(NoopScalingStrategy.class); - private final S3AutoScalingStrategyConfig config; - - private final Object lock = new Object(); - - private volatile String currentlyProvisioning = null; - private volatile String currentlyTerminating = null; - - public NoopScalingStrategy( - S3AutoScalingStrategyConfig config - ) - { - this.config = config; - } - @Override - public void provision(Map zkWorkers) + public AutoScalingData provision() { - synchronized (lock) { - if (currentlyProvisioning != null) { - if (!zkWorkers.containsKey(currentlyProvisioning)) { - log.info( - "[%s] is still provisioning. Wait for it to finish before requesting new worker.", - currentlyProvisioning - ); - return; - } - } - - try { - log.info("If I were a real strategy I'd create something now"); - currentlyProvisioning = "willNeverBeTrue"; - } - catch (Exception e) { - log.error(e, "Unable to create instance"); - currentlyProvisioning = null; - } - } + log.info("If I were a real strategy I'd create something now"); + return null; } @Override - public Instance terminateIfNeeded(Map zkWorkers) + public AutoScalingData terminate(List nodeIds) { - synchronized (lock) { - if (currentlyTerminating != null) { - if (zkWorkers.containsKey(currentlyTerminating)) { - log.info("[%s] has not terminated. Wait for it to finish before terminating again.", currentlyTerminating); - return null; - } - } - - MinMaxPriorityQueue currWorkers = MinMaxPriorityQueue.orderedBy( - new Comparator() - { - @Override - public int compare(WorkerWrapper w1, WorkerWrapper w2) - { - DateTime w1Time = (w1 == null) ? new DateTime(0) : w1.getLastCompletedTaskTime(); - DateTime w2Time = (w2 == null) ? new DateTime(0) : w2.getLastCompletedTaskTime(); - return w1Time.compareTo(w2Time); - } - } - ).create( - zkWorkers.values() - ); - - if (currWorkers.size() <= config.getMinNuMWorkers()) { - return null; - } - - WorkerWrapper thatLazyWorker = currWorkers.poll(); - - if (System.currentTimeMillis() - thatLazyWorker.getLastCompletedTaskTime().getMillis() - > config.getMillisToWaitBeforeTerminating()) { - try { - log.info("If I were a real strategy I'd terminate something now"); - currentlyTerminating = "willNeverBeTrue"; - - return null; - } - catch (Exception e) { - log.error(e, "Unable to terminate instance"); - currentlyTerminating = null; - } - } - - return null; - } + log.info("If I were a real strategy I'd terminate %s now", nodeIds); + return null; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java deleted file mode 100644 index 840dba3a2a83..000000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategy.java +++ /dev/null @@ -1,176 +0,0 @@ -/* - * 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.merger.coordinator.scaling; - -import com.amazonaws.services.ec2.AmazonEC2Client; -import com.amazonaws.services.ec2.model.DescribeInstancesRequest; -import com.amazonaws.services.ec2.model.DescribeInstancesResult; -import com.amazonaws.services.ec2.model.Filter; -import com.amazonaws.services.ec2.model.Instance; -import com.amazonaws.services.ec2.model.InstanceType; -import com.amazonaws.services.ec2.model.RunInstancesRequest; -import com.amazonaws.services.ec2.model.RunInstancesResult; -import com.amazonaws.services.ec2.model.TerminateInstancesRequest; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; -import com.google.common.collect.MinMaxPriorityQueue; -import com.metamx.common.ISE; -import com.metamx.common.guava.FunctionalIterable; -import com.metamx.druid.merger.coordinator.WorkerWrapper; -import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; -import com.metamx.emitter.EmittingLogger; -import org.joda.time.DateTime; - -import java.util.Arrays; -import java.util.Comparator; -import java.util.Map; - -/** - */ -public class S3AutoScalingStrategy implements ScalingStrategy -{ - private static final EmittingLogger log = new EmittingLogger(S3AutoScalingStrategy.class); - - private final AmazonEC2Client amazonEC2Client; - private final S3AutoScalingStrategyConfig config; - - private final Object lock = new Object(); - - private volatile String currentlyProvisioning = null; - private volatile String currentlyTerminating = null; - - public S3AutoScalingStrategy( - AmazonEC2Client amazonEC2Client, - S3AutoScalingStrategyConfig config - ) - { - this.amazonEC2Client = amazonEC2Client; - this.config = config; - } - - @Override - public void provision(Map zkWorkers) - { - synchronized (lock) { - if (zkWorkers.containsKey(currentlyProvisioning)) { - currentlyProvisioning = null; - } - - if (currentlyProvisioning != null) { - log.info( - "[%s] is still provisioning. Wait for it to finish before requesting new worker.", - currentlyProvisioning - ); - return; - } - - try { - log.info("Creating a new instance"); - RunInstancesResult result = amazonEC2Client.runInstances( - new RunInstancesRequest(config.getAmiId(), 1, 1) - .withInstanceType(InstanceType.fromValue(config.getInstanceType())) - ); - - if (result.getReservation().getInstances().size() != 1) { - throw new ISE("Created more than one instance, WTF?!"); - } - - Instance instance = result.getReservation().getInstances().get(0); - log.info("Created instance: %s", instance.getInstanceId()); - log.debug("%s", instance); - - currentlyProvisioning = String.format("%s:%s", instance.getPrivateIpAddress(), config.getWorkerPort()); - } - catch (Exception e) { - log.error(e, "Unable to create instance"); - currentlyProvisioning = null; - } - } - } - - @Override - public Instance terminateIfNeeded(Map zkWorkers) - { - synchronized (lock) { - if (!zkWorkers.containsKey(currentlyTerminating)) { - currentlyProvisioning = null; - } - - if (currentlyTerminating != null) { - log.info("[%s] has not terminated. Wait for it to finish before terminating again.", currentlyTerminating); - return null; - } - - MinMaxPriorityQueue currWorkers = MinMaxPriorityQueue.orderedBy( - new Comparator() - { - @Override - public int compare(WorkerWrapper w1, WorkerWrapper w2) - { - DateTime w1Time = (w1 == null) ? new DateTime(0) : w1.getLastCompletedTaskTime(); - DateTime w2Time = (w2 == null) ? new DateTime(0) : w2.getLastCompletedTaskTime(); - return w1Time.compareTo(w2Time); - } - } - ).create( - zkWorkers.values() - ); - - if (currWorkers.size() <= config.getMinNuMWorkers()) { - return null; - } - - WorkerWrapper thatLazyWorker = currWorkers.poll(); - - if (System.currentTimeMillis() - thatLazyWorker.getLastCompletedTaskTime().getMillis() - > config.getMillisToWaitBeforeTerminating()) { - DescribeInstancesResult result = amazonEC2Client.describeInstances( - new DescribeInstancesRequest() - .withFilters( - new Filter("private-ip-address", Arrays.asList(thatLazyWorker.getWorker().getIp())) - ) - ); - - if (result.getReservations().size() != 1 || result.getReservations().get(0).getInstances().size() != 1) { - throw new ISE("More than one node with the same private IP[%s], WTF?!", thatLazyWorker.getWorker().getIp()); - } - - Instance instance = result.getReservations().get(0).getInstances().get(0); - - try { - log.info("Terminating instance[%s]", instance.getInstanceId()); - amazonEC2Client.terminateInstances( - new TerminateInstancesRequest(Arrays.asList(instance.getInstanceId())) - ); - - currentlyTerminating = String.format("%s:%s", instance.getPrivateIpAddress(), config.getWorkerPort()); - - return instance; - } - catch (Exception e) { - log.error(e, "Unable to terminate instance"); - currentlyTerminating = null; - } - } - - return null; - } - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java index 6a779927b87b..9b7da8fb3a49 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java @@ -19,16 +19,13 @@ package com.metamx.druid.merger.coordinator.scaling; -import com.amazonaws.services.ec2.model.Instance; -import com.metamx.druid.merger.coordinator.WorkerWrapper; - -import java.util.Map; +import java.util.List; /** */ -public interface ScalingStrategy +public interface ScalingStrategy { - public void provision(Map zkWorkers); + public AutoScalingData provision(); - public Instance terminateIfNeeded(Map zkWorkers); + public AutoScalingData terminate(List nodeIds); } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java index 9375ef816963..82dbd75adf89 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java @@ -145,7 +145,6 @@ public void makePathIfNotExisting(String path, CreateMode mode, Object data) thr if (curatorFramework.checkExists().forPath(path) == null) { try { curatorFramework.create() - .creatingParentsIfNeeded() .withMode(mode) .forPath(path, jsonMapper.writeValueAsBytes(data)); } @@ -201,7 +200,7 @@ public void unannounceTask(String taskId) curatorFramework.delete().guaranteed().forPath(getTaskPathForId(taskId)); } catch (Exception e) { - log.warn("Could not delete task path for task[%s], looks like it already went away", taskId); + log.warn(e, "Could not delete task path for task[%s]", taskId); } } @@ -214,7 +213,6 @@ public void announceStatus(TaskStatus status) try { curatorFramework.create() - .creatingParentsIfNeeded() .withMode(CreateMode.EPHEMERAL) .forPath( getStatusPathForId(status.getId()), diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java index e8e68749e2d2..5b5f3a0a6e71 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java @@ -39,7 +39,8 @@ public abstract class WorkerConfig @Config("druid.worker.version") public abstract String getVersion(); - @Config("druid.worker.capacity") - @Default("3") - public abstract int getCapacity(); + public int getCapacity() + { + return Runtime.getRuntime().availableProcessors() - 1; + } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java similarity index 78% rename from merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java rename to merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java index de486cbee572..f5d682ba781c 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/S3AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -29,7 +29,7 @@ import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.google.common.collect.Maps; import com.metamx.druid.merger.coordinator.WorkerWrapper; -import com.metamx.druid.merger.coordinator.config.S3AutoScalingStrategyConfig; +import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.worker.Worker; import org.easymock.EasyMock; import org.joda.time.DateTime; @@ -45,10 +45,11 @@ /** */ -public class S3AutoScalingStrategyTest +public class EC2AutoScalingStrategyTest { private static final String AMI_ID = "dummy"; private static final String INSTANCE_ID = "theInstance"; + private static final String IP = "dummyIP"; private AmazonEC2Client amazonEC2Client; private RunInstancesResult runInstancesResult; @@ -56,7 +57,7 @@ public class S3AutoScalingStrategyTest private Reservation reservation; private Instance instance; private WorkerWrapper worker; - private S3AutoScalingStrategy strategy; + private EC2AutoScalingStrategy strategy; @Before public void setUp() throws Exception @@ -66,16 +67,20 @@ public void setUp() throws Exception describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class); reservation = EasyMock.createMock(Reservation.class); - instance = new Instance().withInstanceId(INSTANCE_ID).withLaunchTime(new Date()).withImageId(AMI_ID); + instance = new Instance() + .withInstanceId(INSTANCE_ID) + .withLaunchTime(new Date()) + .withImageId(AMI_ID) + .withPrivateIpAddress(IP); worker = new WorkerWrapper( - new Worker("dummyHost", "dummyIP", 2, "0"), + new Worker("dummyHost", IP, 2, "0"), new ConcurrentSkipListSet(), null ); worker.setLastCompletedTaskTime(new DateTime(0)); - strategy = new S3AutoScalingStrategy( - amazonEC2Client, new S3AutoScalingStrategyConfig() + strategy = new EC2AutoScalingStrategy( + amazonEC2Client, new EC2AutoScalingStrategyConfig() { @Override public String getAmiId() @@ -96,15 +101,15 @@ public String getInstanceType() } @Override - public long getMillisToWaitBeforeTerminating() + public int getMinNumInstancesToProvision() { - return 0; + return 1; } @Override - public int getMinNuMWorkers() + public int getMaxNumInstancesToProvision() { - return 0; + return 1; } } ); @@ -140,10 +145,6 @@ public void testScale() EasyMock.expect(reservation.getInstances()).andReturn(Arrays.asList(instance)).atLeastOnce(); EasyMock.replay(reservation); - Map zkWorkers = Maps.newHashMap(); - - zkWorkers.put(worker.getWorker().getHost(), worker); - worker.getRunningTasks().add("task1"); Assert.assertFalse(worker.isAtCapacity()); @@ -152,13 +153,19 @@ public void testScale() Assert.assertTrue(worker.isAtCapacity()); - strategy.provision(zkWorkers); + AutoScalingData created = strategy.provision(); + + Assert.assertEquals(created.getNodeIds().size(), 1); + Assert.assertEquals(created.getNodes().size(), 1); + Assert.assertEquals(String.format("%s:8080", IP), created.getNodeIds().get(0)); worker.getRunningTasks().remove("task1"); worker.getRunningTasks().remove("task2"); - Instance deleted = strategy.terminateIfNeeded(zkWorkers); + AutoScalingData deleted = strategy.terminate(Arrays.asList("dummyHost")); - Assert.assertEquals(deleted.getInstanceId(), INSTANCE_ID); + Assert.assertEquals(deleted.getNodeIds().size(), 1); + Assert.assertEquals(deleted.getNodes().size(), 1); + Assert.assertEquals(String.format("%s:8080", IP), deleted.getNodeIds().get(0)); } } From a5c3eab2f1ce6dd09e97c208fe100db6973d85e1 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 2 Nov 2012 15:59:55 -0700 Subject: [PATCH 07/15] fix merge break --- client/src/main/java/com/metamx/druid/http/BrokerNode.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/http/BrokerNode.java b/client/src/main/java/com/metamx/druid/http/BrokerNode.java index 665a2fbd4272..b8f1f5c4d319 100644 --- a/client/src/main/java/com/metamx/druid/http/BrokerNode.java +++ b/client/src/main/java/com/metamx/druid/http/BrokerNode.java @@ -221,7 +221,7 @@ private void initializeDiscovery() throws Exception final ServiceDiscoveryConfig serviceDiscoveryConfig = getConfigFactory().build(ServiceDiscoveryConfig.class); CuratorFramework curatorFramework = Initialization.makeCuratorFrameworkClient( - serviceDiscoveryConfig.getZkHosts(), lifecycle + serviceDiscoveryConfig, lifecycle ); final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( From e9dadcae490a296f296aff80484f5ef33489dbb8 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Fri, 2 Nov 2012 16:59:49 -0700 Subject: [PATCH 08/15] more sensical logic around when to terminate worker nodes --- .../merger/coordinator/RemoteTaskRunner.java | 20 ++++++++++++------- .../config/RemoteTaskRunnerConfig.java | 14 +++++++------ 2 files changed, 21 insertions(+), 13 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 3c08b397d77b..7e8903857032 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -144,17 +144,17 @@ public void childEvent(CuratorFramework client, final PathChildrenCacheEvent eve ); // Schedule termination of worker nodes periodically - Period period = new Period(config.getTerminateResourcesPeriodMs()); - PeriodGranularity granularity = new PeriodGranularity(period, null, null); - final long truncatedNow = granularity.truncate(new DateTime().getMillis()); + Period period = new Period(config.getTerminateResourcesDuration()); + PeriodGranularity granularity = new PeriodGranularity(period, config.getTerminateResourcesOriginDateTime(), null); + final long startTime = granularity.next(granularity.truncate(new DateTime().getMillis())); ScheduledExecutors.scheduleAtFixedRate( scheduledExec, new Duration( System.currentTimeMillis(), - granularity.next(truncatedNow) - config.getTerminateResourcesWindowMs() + startTime ), - new Duration(config.getTerminateResourcesPeriodMs()), + config.getTerminateResourcesDuration(), new Runnable() { @Override @@ -588,12 +588,18 @@ public void run() { try { final String statusPath = JOINER.join(config.getStatusPath(), workerId, taskId); - final String taskPath = JOINER.join(config.getTaskPath(), workerId, taskId); cf.delete().guaranteed().forPath(statusPath); + } + catch (Exception e) { + log.warn("Tried to delete a status path that didn't exist! Must've gone away already?"); + } + + try { + final String taskPath = JOINER.join(config.getTaskPath(), workerId, taskId); cf.delete().guaranteed().forPath(taskPath); } catch (Exception e) { - log.warn("Tried to delete a path that didn't exist! Must've gone away already!"); + log.warn("Tried to delete a task path that didn't exist! Must've gone away already?"); } } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java index a9cf6b02f932..5118e6e7c593 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -20,6 +20,8 @@ package com.metamx.druid.merger.coordinator.config; import com.metamx.druid.merger.common.config.IndexerZkConfig; +import org.joda.time.DateTime; +import org.joda.time.Duration; import org.skife.config.Config; import org.skife.config.Default; @@ -27,13 +29,13 @@ */ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig { - @Config("druid.indexer.terminateResources.periodMs") - @Default("3600000") // 1 hr - public abstract long getTerminateResourcesPeriodMs(); + @Config("druid.indexer.terminateResources.duration") + @Default("PT1H") + public abstract Duration getTerminateResourcesDuration(); - @Config("druid.indexer.terminateResources.windowMs") - @Default("300000") // 5 mins - public abstract long getTerminateResourcesWindowMs(); + @Config("druid.indexer.terminateResources.originDateTime") + @Default("2012-01-01T00:55:00.000Z") + public abstract DateTime getTerminateResourcesOriginDateTime(); @Config("druid.indexer.minWorkerVersion") public abstract String getMinWorkerVersion(); From b4e3f98f4b539f26debc2be5f49d5b5d87d04d09 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 5 Nov 2012 11:18:11 -0800 Subject: [PATCH 09/15] bug fixes for retries without sufficient capacity --- .../merger/coordinator/RemoteTaskRunner.java | 63 +++++++++---------- .../druid/merger/coordinator/RetryPolicy.java | 5 -- .../merger/coordinator/WorkerWrapper.java | 2 +- 3 files changed, 29 insertions(+), 41 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 7e8903857032..77cbc0d2f5ea 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -26,7 +26,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.primitives.Ints; -import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; @@ -174,6 +173,9 @@ public void run() @Override public boolean apply(@Nullable WorkerWrapper input) { + if (!input.getRunningTasks().isEmpty()) { + return false; + } return System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() > config.getmaxWorkerIdleTimeMillisBeforeDeletion(); } @@ -243,13 +245,15 @@ public void run(Task task, TaskContext context, TaskCallback callback) ); } - private boolean assignTask(TaskWrapper taskWrapper) + private void assignTask(TaskWrapper taskWrapper) { - try { - WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); - // If the task already exists, we don't need to announce it - if (workerWrapper != null) { - final Worker worker = workerWrapper.getWorker(); + tasks.putIfAbsent(taskWrapper.getTask().getId(), taskWrapper); + WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); + + // If the task already exists, we don't need to announce it + if (workerWrapper != null) { + final Worker worker = workerWrapper.getWorker(); + try { log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskWrapper.getTask().getId()); @@ -272,24 +276,20 @@ private boolean assignTask(TaskWrapper taskWrapper) callback.notify(taskStatus); } new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()).run(); - } else { - tasks.putIfAbsent(taskWrapper.getTask().getId(), taskWrapper); } - return true; + } + catch (Exception e) { + log.error(e, "Task exists, but hit exception!"); + retryTask(new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()), taskWrapper); + } + } else { // Announce the task + workerWrapper = getWorkerForTask(); + if (workerWrapper != null) { + announceTask(workerWrapper.getWorker(), taskWrapper); + } else { + retryTask(null, taskWrapper); } } - catch (Exception e) { - throw Throwables.propagate(e); - } - - // Announce the task - WorkerWrapper workerWrapper = getWorkerForTask(); - if (workerWrapper != null) { - announceTask(workerWrapper.getWorker(), taskWrapper); - return true; - } - - return false; } /** @@ -320,20 +320,13 @@ private void retryTask( @Override public void run() { - try { - if (pre != null) { - pre.run(); - } - - if (tasks.containsKey(task.getId())) { - log.info("Retry[%d] for task[%s]", retryPolicy.getNumRetries(), task.getId()); - if (!assignTask(taskWrapper)) { - throw new ISE("Unable to find worker to send retry request to for task[%s]", task.getId()); - } - } + if (pre != null) { + pre.run(); } - catch (Exception e) { - retryTask(null, taskWrapper); + + if (tasks.containsKey(task.getId())) { + log.info("Retry[%d] for task[%s]", retryPolicy.getNumRetries(), task.getId()); + assignTask(taskWrapper); } } }, diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java index b449ce019607..24ee54290bb3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java @@ -19,12 +19,9 @@ package com.metamx.druid.merger.coordinator; -import com.google.common.collect.Lists; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.emitter.EmittingLogger; -import java.util.List; - /** */ public class RetryPolicy @@ -34,8 +31,6 @@ public class RetryPolicy private final long MAX_NUM_RETRIES; private final long MAX_RETRY_DELAY_MILLIS; - private final List runnables = Lists.newArrayList(); - private volatile long currRetryDelay; private volatile int retryCount; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index 83440b2500b3..99e330c0a3a1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -35,7 +35,7 @@ public class WorkerWrapper private final ConcurrentSkipListSet runningTasks; private final PathChildrenCache statusCache; - private volatile DateTime lastCompletedTaskTime; + private volatile DateTime lastCompletedTaskTime = new DateTime(); public WorkerWrapper(Worker worker, ConcurrentSkipListSet runningTasks, PathChildrenCache statusCache) { From 6b247f1599260309a3741afb12a3b470c6b7a212 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 5 Nov 2012 11:19:55 -0800 Subject: [PATCH 10/15] cleanup redundant code --- .../metamx/druid/merger/coordinator/RemoteTaskRunner.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 77cbc0d2f5ea..84f713604046 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -173,11 +173,9 @@ public void run() @Override public boolean apply(@Nullable WorkerWrapper input) { - if (!input.getRunningTasks().isEmpty()) { - return false; - } - return System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() - > config.getmaxWorkerIdleTimeMillisBeforeDeletion(); + return input.getRunningTasks().isEmpty() + && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() + > config.getmaxWorkerIdleTimeMillisBeforeDeletion(); } } ) From 51cd361fbe20b73f5122244ea68fb80a41fb421f Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Mon, 5 Nov 2012 13:39:16 -0800 Subject: [PATCH 11/15] some changes to how handlers are registered --- client/src/main/java/com/metamx/druid/BaseNode.java | 4 ++-- .../src/main/java/com/metamx/druid/index/v1/IndexIO.java | 2 -- .../{ComplexMetricRegistererer.java => Registererer.java} | 4 ++-- .../merger/coordinator/http/IndexerCoordinatorNode.java | 7 +++++++ .../com/metamx/druid/merger/worker/http/WorkerNode.java | 7 +++++++ 5 files changed, 18 insertions(+), 6 deletions(-) rename index-common/src/main/java/com/metamx/druid/index/v1/serde/{ComplexMetricRegistererer.java => Registererer.java} (88%) diff --git a/client/src/main/java/com/metamx/druid/BaseNode.java b/client/src/main/java/com/metamx/druid/BaseNode.java index 7e75ea383473..dc45cd6eef87 100644 --- a/client/src/main/java/com/metamx/druid/BaseNode.java +++ b/client/src/main/java/com/metamx/druid/BaseNode.java @@ -30,7 +30,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.http.RequestLogger; -import com.metamx.druid.index.v1.serde.ComplexMetricRegistererer; +import com.metamx.druid.index.v1.serde.Registererer; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ZkClientConfig; @@ -174,7 +174,7 @@ public T registerJacksonSubtype(NamedType... namedTypes) } @SuppressWarnings("unchecked") - public T registerComplexMetric(ComplexMetricRegistererer registererer) + public T registerHandler(Registererer registererer) { registererer.register(); return (T) this; diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index 75352a25137c..719b2350bbf8 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -36,12 +36,10 @@ import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.FileChannel; import java.util.Map; /** diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricRegistererer.java b/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java similarity index 88% rename from index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricRegistererer.java rename to index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java index c0dcade7ed93..f560dfdc1e60 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricRegistererer.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java @@ -20,14 +20,14 @@ package com.metamx.druid.index.v1.serde; /** - * This is a "factory" interface for registering complex metrics in the system. It exists because I'm unaware of + * This is a "factory" interface for registering handlers in the system. It exists because I'm unaware of * another way to register the complex serdes in the MR jobs that run on Hadoop. As such, instances of this interface * must be instantiatable via a no argument default constructor (the MR jobs on Hadoop use reflection to instantiate * instances). * * The name is not a typo, I felt that it needed an extra "er" to make the pronunciation that much more difficult. */ -public interface ComplexMetricRegistererer +public interface Registererer { public void register(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index f907f2f5add7..2b689c717064 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -41,6 +41,7 @@ import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.index.v1.serde.Registererer; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; @@ -182,6 +183,12 @@ public void setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory) this.taskRunnerFactory = taskRunnerFactory; } + public IndexerCoordinatorNode registerHandler(Registererer registererer) + { + registererer.register(); + return this; + } + public void init() throws Exception { scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 67e1e2e089f8..210a55bcf5fd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -29,6 +29,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.index.v1.serde.Registererer; import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; @@ -147,6 +148,12 @@ public WorkerNode setTaskMonitor(TaskMonitor taskMonitor) return this; } + public WorkerNode registerHandler(Registererer registererer) + { + registererer.register(); + return this; + } + public void init() throws Exception { initializeEmitter(); From 52214d735578630f4af6efc397d4136a0bf3d5ee Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Wed, 7 Nov 2012 17:27:23 -0800 Subject: [PATCH 12/15] additional improvements according to code review; unit tests; bug fixes for retry policies --- merger/pom.xml | 6 +- .../druid/merger/common/task/MergeTask.java | 2 +- .../merger/coordinator/RemoteTaskRunner.java | 144 +++--- .../druid/merger/coordinator/RetryPolicy.java | 19 +- .../merger/coordinator/WorkerWrapper.java | 29 +- .../config/RemoteTaskRunnerConfig.java | 10 +- .../coordinator/config/RetryPolicyConfig.java | 9 +- .../scaling/EC2AutoScalingStrategy.java | 2 +- .../scaling/NoopScalingStrategy.java | 9 +- .../coordinator/scaling/ScalingStrategy.java | 2 +- .../druid/merger/worker/TaskMonitor.java | 5 +- .../druid/merger/worker/http/WorkerNode.java | 5 +- .../coordinator/RemoteTaskRunnerTest.java | 433 ++++++++++++++++++ .../merger/coordinator/RetryPolicyTest.java | 45 ++ .../scaling/EC2AutoScalingStrategyTest.java | 26 +- pom.xml | 7 +- 16 files changed, 621 insertions(+), 132 deletions(-) create mode 100644 merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java create mode 100644 merger/src/test/java/com/metamx/druid/merger/coordinator/RetryPolicyTest.java diff --git a/merger/pom.xml b/merger/pom.xml index e5fc6f99f69e..c60bd7e2bd5b 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -19,7 +19,7 @@ --> + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> 4.0.0 com.metamx.druid druid-merger @@ -178,6 +178,10 @@ easymock test + + com.netflix.curator + curator-test + diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index db5ff0594dad..859352c2a7da 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -88,7 +88,7 @@ protected MergeTask(final String dataSource, final List segments) @Override public boolean apply(@Nullable DataSegment segment) { - return segment == null || !segment.getDataSource().equals(dataSource); + return segment == null || !segment.getDataSource().equalsIgnoreCase(dataSource); } } ) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 84f713604046..a2bc0c3688ea 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -25,7 +25,9 @@ import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; +import com.google.common.collect.Sets; import com.google.common.primitives.Ints; +import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; @@ -54,6 +56,7 @@ import java.io.IOException; import java.util.Comparator; import java.util.List; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ScheduledExecutorService; @@ -95,6 +98,8 @@ public class RemoteTaskRunner implements TaskRunner private final ConcurrentSkipListSet currentlyTerminating = new ConcurrentSkipListSet(); private final Object statusLock = new Object(); + private volatile DateTime lastProvisionTime = new DateTime(); + private volatile DateTime lastTerminateTime = new DateTime(); private volatile boolean started = false; public RemoteTaskRunner( @@ -120,27 +125,31 @@ public RemoteTaskRunner( public void start() { try { - workerPathCache.start(); workerPathCache.getListenable().addListener( new PathChildrenCacheListener() { @Override public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception { - final Worker worker = jsonMapper.readValue( - event.getData().getData(), - Worker.class - ); if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { + final Worker worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); log.info("New worker[%s] found!", 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("Worker[%s] removed!", worker.getHost()); removeWorker(worker.getHost()); } } } ); + workerPathCache.start(); // Schedule termination of worker nodes periodically Period period = new Period(config.getTerminateResourcesDuration()); @@ -175,7 +184,7 @@ public boolean apply(@Nullable WorkerWrapper input) { return input.getRunningTasks().isEmpty() && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() - > config.getmaxWorkerIdleTimeMillisBeforeDeletion(); + > config.getMaxWorkerIdleTimeMillisBeforeDeletion(); } } ) @@ -195,8 +204,19 @@ public String apply(@Nullable WorkerWrapper input) ) ); - currentlyTerminating.addAll(terminated.getNodeIds()); + if (terminated != null) { + currentlyTerminating.addAll(terminated.getNodeIds()); + lastTerminateTime = new DateTime(); + } } else { + Duration durSinceLastTerminate = new Duration(new DateTime(), lastTerminateTime); + if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration())) { + log.makeAlert( + "It has been %d millis since last scheduled termination but nodes remain", + durSinceLastTerminate.getMillis() + ).emit(); + } + log.info( "[%s] still terminating. Wait for all nodes to terminate before trying again.", currentlyTerminating @@ -233,36 +253,38 @@ public boolean hasStarted() return started; } + public int getNumWorkers() + { + return zkWorkers.size(); + } + @Override public void run(Task task, TaskContext context, TaskCallback callback) { - assignTask( - new TaskWrapper( - task, context, callback, retryPolicyFactory.makeRetryPolicy() - ) + if (tasks.contains(task.getId())) { + throw new ISE("Assigned a task[%s] that already exists, WTF is happening?!", task.getId()); + } + TaskWrapper taskWrapper = new TaskWrapper( + task, context, callback, retryPolicyFactory.makeRetryPolicy() ); + tasks.put(taskWrapper.getTask().getId(), taskWrapper); + assignTask(taskWrapper); } private void assignTask(TaskWrapper taskWrapper) { - tasks.putIfAbsent(taskWrapper.getTask().getId(), taskWrapper); WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); // If the task already exists, we don't need to announce it if (workerWrapper != null) { final Worker worker = workerWrapper.getWorker(); try { - log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskWrapper.getTask().getId()); TaskStatus taskStatus = jsonMapper.readValue( workerWrapper.getStatusCache() .getCurrentData( - JOINER.join( - config.getStatusPath(), - worker.getHost(), - taskWrapper.getTask().getId() - ) + JOINER.join(config.getStatusPath(), worker.getHost(), taskWrapper.getTask().getId()) ) .getData(), TaskStatus.class @@ -280,8 +302,9 @@ private void assignTask(TaskWrapper taskWrapper) log.error(e, "Task exists, but hit exception!"); retryTask(new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()), taskWrapper); } - } else { // Announce the task - workerWrapper = getWorkerForTask(); + } else { + // Announce the task or retry if there is not enough capacity + workerWrapper = findWorkerForTask(); if (workerWrapper != null) { announceTask(workerWrapper.getWorker(), taskWrapper); } else { @@ -328,7 +351,7 @@ public void run() } } }, - retryPolicy.getAndIncrementRetryDelay(), + retryPolicy.getAndIncrementRetryDelay().getMillis(), TimeUnit.MILLISECONDS ); } @@ -347,28 +370,22 @@ private void addWorker(final Worker worker) final String workerStatusPath = JOINER.join(config.getStatusPath(), worker.getHost()); final PathChildrenCache statusCache = new PathChildrenCache(cf, workerStatusPath, true); - final ConcurrentSkipListSet runningTasks = new ConcurrentSkipListSet( - Lists.transform( - statusCache.getCurrentData(), - 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); - } - } - } - ) - ); final WorkerWrapper workerWrapper = new WorkerWrapper( worker, - runningTasks, - statusCache + statusCache, + 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); + } + } + } ); // Add status listener to the watcher for status changes @@ -385,10 +402,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th TaskStatus taskStatus = jsonMapper.readValue( event.getData().getData(), TaskStatus.class ); - taskId = taskStatus.getId(); - - log.info("New status[%s] appeared!", taskId); - runningTasks.add(taskId); + log.info("New status[%s] appeared!", taskStatus.getId()); statusLock.notify(); } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { String statusPath = event.getData().getPath(); @@ -413,13 +427,13 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th callback.notify(taskStatus); } tasks.remove(taskId); - runningTasks.remove(taskId); - cf.delete().guaranteed().forPath(statusPath); + cf.delete().guaranteed().inBackground().forPath(statusPath); } } } } catch (Exception e) { + log.error(e, "Exception in status listener"); retryTask(new CleanupPaths(worker.getHost(), taskId), tasks.get(taskId)); } } @@ -456,25 +470,27 @@ private void removeWorker(final String workerId) WorkerWrapper workerWrapper = zkWorkers.get(workerId); if (workerWrapper != null) { - for (String taskId : workerWrapper.getRunningTasks()) { - TaskWrapper taskWrapper = tasks.get(taskId); - if (taskWrapper != null) { - retryTask(new CleanupPaths(workerId, taskId), tasks.get(taskId)); + try { + Set tasksToRetry = Sets.newHashSet(workerWrapper.getRunningTasks()); + tasksToRetry.addAll(cf.getChildren().forPath(JOINER.join(config.getTaskPath(), workerId))); + + for (String taskId : tasksToRetry) { + TaskWrapper taskWrapper = tasks.get(taskId); + if (taskWrapper != null) { + retryTask(new CleanupPaths(workerId, taskId), tasks.get(taskId)); + } } - workerWrapper.removeTask(taskId); - } - try { workerWrapper.getStatusCache().close(); } - catch (IOException e) { - log.error("Failed to close watcher associated with worker[%s]", workerWrapper.getWorker().getHost()); + catch (Exception e) { + log.error(e, "Failed to cleanly remove worker[%s]"); } } zkWorkers.remove(workerId); } - private WorkerWrapper getWorkerForTask() + private WorkerWrapper findWorkerForTask() { try { final MinMaxPriorityQueue workerQueue = MinMaxPriorityQueue.orderedBy( @@ -504,11 +520,20 @@ public boolean apply(WorkerWrapper input) log.info("Worker nodes do not have capacity to run any more tasks!"); if (currentlyProvisioning.isEmpty()) { - AutoScalingData provisioned = strategy.provision(); + AutoScalingData provisioned = strategy.provision(currentlyProvisioning.size()); if (provisioned != null) { currentlyProvisioning.addAll(provisioned.getNodeIds()); + lastProvisionTime = new DateTime(); } } else { + Duration durSinceLastProvision = new Duration(new DateTime(), lastProvisionTime); + if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration())) { + log.makeAlert( + "It has been %d millis since last scheduled provision but nodes remain", + durSinceLastProvision.getMillis() + ).emit(); + } + log.info( "[%s] still provisioning. Wait for all provisioned nodes to complete before requesting new worker.", currentlyProvisioning @@ -552,8 +577,9 @@ private void announceTask(Worker theWorker, TaskWrapper taskWrapper) jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext)) ); + // Syncing state with Zookeeper while (findWorkerRunningTask(taskWrapper) == null) { - statusLock.wait(); + statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis()); } } catch (Exception e) { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java index 24ee54290bb3..fbb27d76c180 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java @@ -21,6 +21,7 @@ import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.emitter.EmittingLogger; +import org.joda.time.Duration; /** */ @@ -29,29 +30,25 @@ public class RetryPolicy private static final EmittingLogger log = new EmittingLogger(RetryPolicy.class); private final long MAX_NUM_RETRIES; - private final long MAX_RETRY_DELAY_MILLIS; + private final Duration MAX_RETRY_DURATION; - private volatile long currRetryDelay; + private volatile Duration currRetryDelay; private volatile int retryCount; public RetryPolicy(RetryPolicyConfig config) { this.MAX_NUM_RETRIES = config.getMaxRetryCount(); - this.MAX_RETRY_DELAY_MILLIS = config.getRetryMaxMillis(); + this.MAX_RETRY_DURATION = config.getRetryMaxDuration(); - this.currRetryDelay = config.getRetryMinMillis(); + this.currRetryDelay = config.getRetryMinDuration(); this.retryCount = 0; } - public long getAndIncrementRetryDelay() + public Duration getAndIncrementRetryDelay() { - long retVal = currRetryDelay; - if (currRetryDelay < MAX_RETRY_DELAY_MILLIS) { - currRetryDelay *= 2; - } - + Duration retVal = new Duration(currRetryDelay); + currRetryDelay = new Duration(Math.min(currRetryDelay.getMillis() * 2, MAX_RETRY_DURATION.getMillis())); retryCount++; - return retVal; } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index 99e330c0a3a1..68d4f0a128c9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -19,29 +19,33 @@ package com.metamx.druid.merger.coordinator; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import com.metamx.druid.merger.worker.Worker; +import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import org.joda.time.DateTime; +import java.io.Closeable; import java.io.IOException; import java.util.Set; -import java.util.concurrent.ConcurrentSkipListSet; /** */ -public class WorkerWrapper +public class WorkerWrapper implements Closeable { private final Worker worker; - private final ConcurrentSkipListSet runningTasks; private final PathChildrenCache statusCache; + private final Function cacheConverter; private volatile DateTime lastCompletedTaskTime = new DateTime(); - public WorkerWrapper(Worker worker, ConcurrentSkipListSet runningTasks, PathChildrenCache statusCache) + public WorkerWrapper(Worker worker, PathChildrenCache statusCache, Function cacheConverter) { this.worker = worker; - this.runningTasks = runningTasks; this.statusCache = statusCache; + this.cacheConverter = cacheConverter; } public Worker getWorker() @@ -51,7 +55,12 @@ public Worker getWorker() public Set getRunningTasks() { - return runningTasks; + return Sets.newHashSet( + Lists.transform( + statusCache.getCurrentData(), + cacheConverter + ) + ); } public PathChildrenCache getStatusCache() @@ -66,7 +75,7 @@ public DateTime getLastCompletedTaskTime() public boolean isAtCapacity() { - return runningTasks.size() >= worker.getCapacity(); + return statusCache.getCurrentData().size() >= worker.getCapacity(); } public void setLastCompletedTaskTime(DateTime completedTaskTime) @@ -74,11 +83,7 @@ public void setLastCompletedTaskTime(DateTime completedTaskTime) lastCompletedTaskTime = completedTaskTime; } - public void removeTask(String taskId) - { - runningTasks.remove(taskId); - } - + @Override public void close() throws IOException { statusCache.close(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java index 5118e6e7c593..00b869ea6da1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -46,5 +46,13 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig @Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion") @Default("1") - public abstract int getmaxWorkerIdleTimeMillisBeforeDeletion(); + public abstract int getMaxWorkerIdleTimeMillisBeforeDeletion(); + + @Config("druid.indexer.maxScalingDuration") + @Default("PT1H") + public abstract Duration getMaxScalingDuration(); + + @Config("druid.indexer.taskAssignmentTimeoutDuration") + @Default("PT5M") + public abstract Duration getTaskAssignmentTimeoutDuration(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java index 044706b67edc..47c8eaf4d1a5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RetryPolicyConfig.java @@ -19,6 +19,7 @@ package com.metamx.druid.merger.coordinator.config; +import org.joda.time.Duration; import org.skife.config.Config; import org.skife.config.Default; @@ -27,12 +28,12 @@ public abstract class RetryPolicyConfig { @Config("druid.indexer.retry.minWaitMillis") - @Default("60000") // 1 minute - public abstract long getRetryMinMillis(); + @Default("PT1M") // 1 minute + public abstract Duration getRetryMinDuration(); @Config("druid.indexer.retry.maxWaitMillis") - @Default("600000") // 10 minutes - public abstract long getRetryMaxMillis(); + @Default("PT10M") // 10 minutes + public abstract Duration getRetryMaxDuration(); @Config("druid.indexer.retry.maxRetryCount") @Default("10") diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index cd94b70d3ce4..a85c3ade8fde 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -56,7 +56,7 @@ public EC2AutoScalingStrategy( } @Override - public AutoScalingData provision() + public AutoScalingData provision(long numUnassignedTasks) { try { log.info("Creating new instance(s)..."); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java index 0331022082c3..923de4638706 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java @@ -1,15 +1,8 @@ package com.metamx.druid.merger.coordinator.scaling; -import com.amazonaws.services.ec2.model.Instance; -import com.google.common.collect.MinMaxPriorityQueue; -import com.metamx.druid.merger.coordinator.WorkerWrapper; -import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.emitter.EmittingLogger; -import org.joda.time.DateTime; -import java.util.Comparator; import java.util.List; -import java.util.Map; /** * This class just logs when scaling should occur. @@ -19,7 +12,7 @@ public class NoopScalingStrategy implements ScalingStrategy private static final EmittingLogger log = new EmittingLogger(NoopScalingStrategy.class); @Override - public AutoScalingData provision() + public AutoScalingData provision(long numUnassignedTasks) { log.info("If I were a real strategy I'd create something now"); return null; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java index 9b7da8fb3a49..ec71d8563016 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java @@ -25,7 +25,7 @@ */ public interface ScalingStrategy { - public AutoScalingData provision(); + public AutoScalingData provision(long numUnassignedTasks); public AutoScalingData terminate(List nodeIds); } diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index 938bda933fc6..5fc49788fcdf 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -47,7 +47,6 @@ public class TaskMonitor private final PathChildrenCache pathChildrenCache; private final CuratorFramework cf; - private final ObjectMapper jsonMapper; private final WorkerCuratorCoordinator workerCuratorCoordinator; private final TaskToolbox toolbox; private final ExecutorService exec; @@ -55,7 +54,6 @@ public class TaskMonitor public TaskMonitor( PathChildrenCache pathChildrenCache, CuratorFramework cf, - ObjectMapper jsonMapper, WorkerCuratorCoordinator workerCuratorCoordinator, TaskToolbox toolbox, ExecutorService exec @@ -63,7 +61,6 @@ public TaskMonitor( { this.pathChildrenCache = pathChildrenCache; this.cf = cf; - this.jsonMapper = jsonMapper; this.workerCuratorCoordinator = workerCuratorCoordinator; this.toolbox = toolbox; this.exec = exec; @@ -87,7 +84,7 @@ public void childEvent(CuratorFramework curatorFramework, PathChildrenCacheEvent throws Exception { if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final TaskHolder taskHolder = jsonMapper.readValue( + final TaskHolder taskHolder = toolbox.getObjectMapper().readValue( cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()), TaskHolder.class ); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 210a55bcf5fd..cc30b914367d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -287,8 +287,8 @@ public void initializeTaskToolbox() throws S3ServiceException if (taskToolbox == null) { final RestS3Service s3Client = new RestS3Service( new AWSCredentials( - props.getProperty("com.metamx.aws.accessKey"), - props.getProperty("com.metamx.aws.secretKey") + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ); final SegmentPusher segmentPusher = new S3SegmentPusher( @@ -334,7 +334,6 @@ public void initializeTaskMonitor() taskMonitor = new TaskMonitor( pathChildrenCache, curatorFramework, - jsonMapper, workerCuratorCoordinator, taskToolbox, workerExec diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java new file mode 100644 index 000000000000..e219c2984a0b --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -0,0 +1,433 @@ +package com.metamx.druid.merger.coordinator; + +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.config.IndexerZkConfig; +import com.metamx.druid.merger.common.task.DefaultMergeTask; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; +import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; +import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; +import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; +import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; +import com.metamx.druid.merger.worker.TaskMonitor; +import com.metamx.druid.merger.worker.Worker; +import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; +import com.netflix.curator.framework.CuratorFramework; +import com.netflix.curator.framework.CuratorFrameworkFactory; +import com.netflix.curator.framework.recipes.cache.PathChildrenCache; +import com.netflix.curator.retry.ExponentialBackoffRetry; +import com.netflix.curator.test.TestingCluster; +import org.apache.commons.lang.mutable.MutableBoolean; +import org.apache.zookeeper.CreateMode; +import org.codehaus.jackson.annotate.JsonProperty; +import org.codehaus.jackson.annotate.JsonTypeName; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.map.jsontype.NamedType; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.util.List; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; + +/** + */ +public class RemoteTaskRunnerTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private static final String basePath = "/test/druid/indexer"; + private static final String announcementsPath = String.format("%s/announcements", basePath); + private static final String tasksPath = String.format("%s/tasks", basePath); + private static final String statusPath = String.format("%s/status", basePath); + + private TestingCluster testingCluster; + private CuratorFramework cf; + private PathChildrenCache pathChildrenCache; + private RemoteTaskRunner remoteTaskRunner; + private TaskMonitor taskMonitor; + + private ScheduledExecutorService scheduledExec; + + private Task task1; + + private Worker worker1; + + + @Before + public void setUp() throws Exception + { + testingCluster = new TestingCluster(1); + testingCluster.start(); + + cf = CuratorFrameworkFactory.builder() + .connectString(testingCluster.getConnectString()) + .retryPolicy(new ExponentialBackoffRetry(1, 10)) + .build(); + cf.start(); + + cf.create().creatingParentsIfNeeded().forPath(announcementsPath); + cf.create().forPath(tasksPath); + cf.create().forPath(String.format("%s/worker1", tasksPath)); + cf.create().forPath(statusPath); + cf.create().forPath(String.format("%s/worker1", statusPath)); + + pathChildrenCache = new PathChildrenCache(cf, announcementsPath, true); + + worker1 = new Worker( + "worker1", + "localhost", + 3, + "0" + ); + + makeRemoteTaskRunner(); + makeTaskMonitor(); + } + + @After + public void tearDown() throws Exception + { + testingCluster.stop(); + remoteTaskRunner.stop(); + taskMonitor.stop(); + } + + @Test + public void testRunNoExistingTask() throws Exception + { + remoteTaskRunner.run( + task1, + new TaskContext(new DateTime().toString(), Sets.newHashSet()), + null + ); + } + + @Test + public void testRunWithExistingCompletedTask() throws Exception + { + cf.create().creatingParentsIfNeeded().forPath( + String.format("%s/worker1/task1", statusPath), + jsonMapper.writeValueAsBytes( + TaskStatus.success( + "task1", + Lists.newArrayList() + ) + ) + ); + + // Really don't like this way of waiting for the task to appear + while (remoteTaskRunner.getNumWorkers() == 0) { + Thread.sleep(500); + } + + final MutableBoolean callbackCalled = new MutableBoolean(false); + remoteTaskRunner.run( + task1, + null, + new TaskCallback() + { + @Override + public void notify(TaskStatus status) + { + callbackCalled.setValue(true); + } + } + ); + + Assert.assertTrue("TaskCallback was not called!", callbackCalled.booleanValue()); + } + + private void makeTaskMonitor() throws Exception + { + WorkerCuratorCoordinator workerCuratorCoordinator = new WorkerCuratorCoordinator( + jsonMapper, + new IndexerZkConfig() + { + @Override + public String getAnnouncementPath() + { + return announcementsPath; + } + + @Override + public String getTaskPath() + { + return tasksPath; + } + + @Override + public String getStatusPath() + { + return statusPath; + } + }, + cf, + worker1 + ); + workerCuratorCoordinator.start(); + + taskMonitor = new TaskMonitor( + new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true), + cf, + workerCuratorCoordinator, + new TaskToolbox( + new IndexerCoordinatorConfig() + { + @Override + public String getServerName() + { + return "worker1"; + } + + @Override + public String getLeaderLatchPath() + { + return null; + } + + @Override + public int getNumLocalThreads() + { + return 1; + } + + @Override + public String getRunnerImpl() + { + return null; + } + + @Override + public String getStorageImpl() + { + return null; + } + + @Override + public File getBaseTaskDir() + { + try { + return File.createTempFile("billy", "yay"); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @Override + public boolean isWhitelistEnabled() + { + return false; + } + + @Override + public String getWhitelistDatasourcesString() + { + return null; + } + + @Override + public long getRowFlushBoundary() + { + return 0; + } + }, null, null, null, jsonMapper + ), + Executors.newSingleThreadExecutor() + ); + jsonMapper.registerSubtypes(new NamedType(TestTask.class, "test")); + taskMonitor.start(); + } + + private void makeRemoteTaskRunner() throws Exception + { + scheduledExec = EasyMock.createMock(ScheduledExecutorService.class); + + remoteTaskRunner = new RemoteTaskRunner( + jsonMapper, + new TestRemoteTaskRunnerConfig(), + cf, + pathChildrenCache, + scheduledExec, + new RetryPolicyFactory(new TestRetryPolicyConfig()), + new TestScalingStrategy() + ); + + task1 = new TestTask( + "task1", + "dummyDs", + Lists.newArrayList( + new DataSegment( + "dummyDs", + new Interval(new DateTime(), new DateTime()), + new DateTime().toString(), + null, + null, + null, + null, + 0 + ) + ), Lists.newArrayList() + ); + + // Create a single worker and wait for things for be ready + remoteTaskRunner.start(); + cf.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath( + String.format("%s/worker1", announcementsPath), + jsonMapper.writeValueAsBytes(worker1) + ); + while (remoteTaskRunner.getNumWorkers() == 0) { + Thread.sleep(500); + } + } + + 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 TestScalingStrategy implements ScalingStrategy + { + @Override + public AutoScalingData provision(long numUnassignedTasks) + { + return null; + } + + @Override + public AutoScalingData terminate(List nodeIds) + { + return null; + } + } + + private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig + { + @Override + public Duration getTerminateResourcesDuration() + { + return null; + } + + @Override + public DateTime getTerminateResourcesOriginDateTime() + { + return null; + } + + @Override + public String getMinWorkerVersion() + { + return "0"; + } + + @Override + public int getMinNumWorkers() + { + return 0; + } + + @Override + public int getMaxWorkerIdleTimeMillisBeforeDeletion() + { + return 0; + } + + @Override + public Duration getMaxScalingDuration() + { + return null; + } + + @Override + public String getAnnouncementPath() + { + return announcementsPath; + } + + @Override + public String getTaskPath() + { + return tasksPath; + } + + @Override + public String getStatusPath() + { + return statusPath; + } + + @Override + public Duration getTaskAssignmentTimeoutDuration() + { + return new Duration(60000); + } + } + + @JsonTypeName("test") + private static class TestTask extends DefaultMergeTask + { + private final String id; + + public TestTask( + @JsonProperty("id") String id, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segments") List segments, + @JsonProperty("aggregations") List aggregators + ) + { + super(dataSource, segments, aggregators); + + this.id = id; + } + + @Override + @JsonProperty + public String getId() + { + return id; + } + + @Override + public Type getType() + { + return Type.TEST; + } + + @Override + public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + { + return TaskStatus.success("task1", Lists.newArrayList()); + } + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RetryPolicyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RetryPolicyTest.java new file mode 100644 index 000000000000..5445c05e7dd9 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RetryPolicyTest.java @@ -0,0 +1,45 @@ +package com.metamx.druid.merger.coordinator; + +import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; +import junit.framework.Assert; +import org.joda.time.Duration; +import org.junit.Test; + +/** + */ +public class RetryPolicyTest +{ + @Test + public void testGetAndIncrementRetryDelay() throws Exception + { + RetryPolicy retryPolicy = new RetryPolicy( + new RetryPolicyConfig() + { + @Override + public Duration getRetryMinDuration() + { + return new Duration("PT1S"); + } + + @Override + public Duration getRetryMaxDuration() + { + return new Duration("PT10S"); + } + + @Override + public long getMaxRetryCount() + { + return 10; + } + } + ); + + Assert.assertEquals(new Duration("PT1S"), retryPolicy.getAndIncrementRetryDelay()); + Assert.assertEquals(new Duration("PT2S"), retryPolicy.getAndIncrementRetryDelay()); + Assert.assertEquals(new Duration("PT4S"), retryPolicy.getAndIncrementRetryDelay()); + Assert.assertEquals(new Duration("PT8S"), retryPolicy.getAndIncrementRetryDelay()); + Assert.assertEquals(new Duration("PT10S"), retryPolicy.getAndIncrementRetryDelay()); + Assert.assertEquals(new Duration("PT10S"), retryPolicy.getAndIncrementRetryDelay()); + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java index f5d682ba781c..11a837196cc2 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -27,12 +27,8 @@ import com.amazonaws.services.ec2.model.RunInstancesRequest; import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; -import com.google.common.collect.Maps; -import com.metamx.druid.merger.coordinator.WorkerWrapper; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; -import com.metamx.druid.merger.worker.Worker; import org.easymock.EasyMock; -import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -40,8 +36,6 @@ import java.util.Arrays; import java.util.Date; -import java.util.Map; -import java.util.concurrent.ConcurrentSkipListSet; /** */ @@ -56,7 +50,6 @@ public class EC2AutoScalingStrategyTest private DescribeInstancesResult describeInstancesResult; private Reservation reservation; private Instance instance; - private WorkerWrapper worker; private EC2AutoScalingStrategy strategy; @Before @@ -73,12 +66,6 @@ public void setUp() throws Exception .withImageId(AMI_ID) .withPrivateIpAddress(IP); - worker = new WorkerWrapper( - new Worker("dummyHost", IP, 2, "0"), - new ConcurrentSkipListSet(), - null - ); - worker.setLastCompletedTaskTime(new DateTime(0)); strategy = new EC2AutoScalingStrategy( amazonEC2Client, new EC2AutoScalingStrategyConfig() { @@ -145,23 +132,12 @@ public void testScale() EasyMock.expect(reservation.getInstances()).andReturn(Arrays.asList(instance)).atLeastOnce(); EasyMock.replay(reservation); - worker.getRunningTasks().add("task1"); - - Assert.assertFalse(worker.isAtCapacity()); - - worker.getRunningTasks().add("task2"); - - Assert.assertTrue(worker.isAtCapacity()); - - AutoScalingData created = strategy.provision(); + AutoScalingData created = strategy.provision(0); Assert.assertEquals(created.getNodeIds().size(), 1); Assert.assertEquals(created.getNodes().size(), 1); Assert.assertEquals(String.format("%s:8080", IP), created.getNodeIds().get(0)); - worker.getRunningTasks().remove("task1"); - worker.getRunningTasks().remove("task2"); - AutoScalingData deleted = strategy.terminate(Arrays.asList("dummyHost")); Assert.assertEquals(deleted.getNodeIds().size(), 1); diff --git a/pom.xml b/pom.xml index cbeb399f8edf..8a4b8b3d6506 100644 --- a/pom.xml +++ b/pom.xml @@ -19,7 +19,7 @@ --> + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> 4.0.0 com.metamx druid @@ -131,6 +131,11 @@ curator-x-discovery 1.2.2 + + com.netflix.curator + curator-test + 1.2.2 + it.uniroma3.mat extendedset From f39af83be4cf828df2dbe5bc41e8ec902427b273 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Thu, 8 Nov 2012 09:30:32 -0800 Subject: [PATCH 13/15] make switching between scaling strategies config based --- .../config/IndexerCoordinatorConfig.java | 4 +++ .../http/IndexerCoordinatorNode.java | 32 ++++++++++++------- 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java index 18f317525fe7..6d6218c3bffa 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java @@ -83,4 +83,8 @@ public Set getWhitelistDatasources() @Config("druid.merger.rowFlushBoundary") @Default("500000") public abstract long getRowFlushBoundary(); + + @Config("druid.indexer.strategy") + @Default("noop") + public abstract String getStrategyImpl(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 2b689c717064..3bcfca0deff6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -468,17 +468,27 @@ public TaskRunner build() .build() ); - ScalingStrategy strategy = new EC2AutoScalingStrategy( - new AmazonEC2Client( - new BasicAWSCredentials( - props.getProperty("com.metamx.aws.accessKey"), - props.getProperty("com.metamx.aws.secretKey") - ) - ), - configFactory.build(EC2AutoScalingStrategyConfig.class) - ); - // TODO: use real strategy before actual deployment - strategy = new NoopScalingStrategy(); + ScalingStrategy strategy; + if (config.getStrategyImpl().equalsIgnoreCase("ec2")) { + strategy = new EC2AutoScalingStrategy( + new AmazonEC2Client( + new BasicAWSCredentials( + props.getProperty("com.metamx.aws.accessKey"), + props.getProperty("com.metamx.aws.secretKey") + ) + ), + configFactory.build(EC2AutoScalingStrategyConfig.class) + ); + } else if (config.getStorageImpl().equalsIgnoreCase("noop")) { + strategy = new NoopScalingStrategy(); + } else { + throw new IllegalStateException( + String.format( + "Invalid strategy implementation: %s", + config.getStrategyImpl() + ) + ); + } return new RemoteTaskRunner( jsonMapper, From cdd9cdb2bddb838d45b97be820bced1ea4aefdc2 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Thu, 8 Nov 2012 09:31:22 -0800 Subject: [PATCH 14/15] use PropUtils for indexer coordinator node --- .../merger/coordinator/http/IndexerCoordinatorNode.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 3bcfca0deff6..c8fc5cebb78d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -384,8 +384,8 @@ public void initializeTaskToolbox() throws S3ServiceException if (taskToolbox == null) { final RestS3Service s3Client = new RestS3Service( new AWSCredentials( - props.getProperty("com.metamx.aws.accessKey"), - props.getProperty("com.metamx.aws.secretKey") + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ); final SegmentPusher segmentPusher = new S3SegmentPusher( @@ -473,8 +473,8 @@ public TaskRunner build() strategy = new EC2AutoScalingStrategy( new AmazonEC2Client( new BasicAWSCredentials( - props.getProperty("com.metamx.aws.accessKey"), - props.getProperty("com.metamx.aws.secretKey") + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") ) ), configFactory.build(EC2AutoScalingStrategyConfig.class) From 7083821fbff153c61c17c999d6330023f20301d2 Mon Sep 17 00:00:00 2001 From: Fangjin Yang Date: Thu, 8 Nov 2012 10:08:40 -0800 Subject: [PATCH 15/15] no param for scaling provision and minor fix for initialization --- .../druid/initialization/Initialization.java | 46 +++++++++++++------ .../merger/coordinator/RemoteTaskRunner.java | 18 +------- .../merger/coordinator/WorkerWrapper.java | 20 +++++++- .../scaling/EC2AutoScalingStrategy.java | 2 +- .../scaling/NoopScalingStrategy.java | 2 +- .../coordinator/scaling/ScalingStrategy.java | 2 +- 6 files changed, 54 insertions(+), 36 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index addf3ac6ebb6..1d2c5c12a111 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -69,7 +69,8 @@ public class Initialization "druid.zk.paths.announcementsPath", "druid.zk.paths.servedSegmentsPath", "druid.zk.paths.loadQueuePath", - "druid.zk.paths.masterPath"}; + "druid.zk.paths.masterPath" + }; public static final String DEFAULT_ZPATH = "/druid"; public static ZkClient makeZkClient(ZkClientConfig config, Lifecycle lifecycle) @@ -119,10 +120,12 @@ public static ZKPhoneBook createPhoneBook( } - /** Load properties. + /** + * Load properties. * Properties are layered, high to low precedence: cmdLine -D, runtime.properties file, stored in zookeeper. * Idempotent. Thread-safe. Properties are only loaded once. * If property druid.zk.service.host=none then do not load properties from zookeeper. + * * @return Properties ready to use. */ public synchronized static Properties loadProperties() @@ -139,7 +142,9 @@ public synchronized static Properties loadProperties() final InputStream stream = ClassLoader.getSystemResourceAsStream("runtime.properties"); if (stream == null) { - log.info("runtime.properties not found as a resource in classpath, relying only on system properties, and zookeeper now."); + log.info( + "runtime.properties not found as a resource in classpath, relying only on system properties, and zookeeper now." + ); } else { log.info("Loading properties from runtime.properties"); try { @@ -202,7 +207,7 @@ public String getZkHosts() log.warn("property druid.zk.service.host is not set, so no way to contact zookeeper for coordination."); } // validate properties now that all levels of precedence are loaded - if (! validateResolveProps(tmp_props)) { + if (!validateResolveProps(tmp_props)) { log.error("Properties failed to validate, cannot continue"); throw new RuntimeException("Properties failed to validate"); } @@ -358,12 +363,15 @@ public static String makePropPath(String basePath) return String.format("%s/%s", basePath, PROP_SUBPATH); } - /** Validate and Resolve Properties. + /** + * Validate and Resolve Properties. * Resolve zpaths with props like druid.zk.paths.*Path using druid.zk.paths.base value. * Check validity so that if druid.zk.paths.*Path props are set, all are set, * if none set, then construct defaults relative to druid.zk.paths.base and add these * to the properties chain. + * * @param props + * * @return true if valid zpath properties. */ public static boolean validateResolveProps(Properties props) @@ -379,7 +387,9 @@ public static boolean validateResolveProps(Properties props) final String propertiesZpathOverride = props.getProperty("druid.zk.paths.propertiesPath"); - if (!zpathValidateFailed) System.out.println("Effective zpath prefix=" + zpathEffective); + if (!zpathValidateFailed) { + System.out.println("Effective zpath prefix=" + zpathEffective); + } // validate druid.zk.paths.*Path properties // @@ -408,22 +418,25 @@ public static boolean validateResolveProps(Properties props) } } if (zpathOverridesNotAbs) { - System.err.println("When overriding zk zpaths, with properties like druid.zk.paths.*Path " + - "the znode path must start with '/' (slash) ; problem overrides:"); + System.err.println( + "When overriding zk zpaths, with properties like druid.zk.paths.*Path " + + "the znode path must start with '/' (slash) ; problem overrides:" + ); System.err.print(sbErrors.toString()); } if (zpathOverrideCount > 0) { - if (zpathOverrideCount < SUB_PATH_PROPS.length + 1) { + if (zpathOverrideCount < SUB_PATH_PROPS.length) { zpathValidateFailed = true; - System.err.println("When overriding zk zpaths, with properties of form druid.zk.paths.*Path " + - "all must be overridden together; missing overrides:"); + System.err.println( + "When overriding zk zpaths, with properties of form druid.zk.paths.*Path " + + "all must be overridden together; missing overrides:" + ); for (int i = 0; i < SUB_PATH_PROPS.length; i++) { String val = props.getProperty(SUB_PATH_PROPS[i]); if (val == null) { System.err.println(" " + SUB_PATH_PROPS[i]); } } - if (propertiesZpathOverride == null) System.err.println(" " + "druid.zk.paths.propertiesPath"); } else { // proper overrides // do not prefix with property druid.zk.paths.base ; // fallthru @@ -440,13 +453,16 @@ public static boolean validateResolveProps(Properties props) } props.setProperty("druid.zk.paths.propertiesPath", zpathEffective + "/properties"); } - return ! zpathValidateFailed; + return !zpathValidateFailed; } - /** Check znode zpath base for proper slash, no trailing slash. - * @param zpathBase znode base path, if null then this method does nothing. + /** + * Check znode zpath base for proper slash, no trailing slash. + * + * @param zpathBase znode base path, if null then this method does nothing. * @param errorMsgPrefix error context to use if errors are emitted, should indicate * where the zpathBase value came from. + * * @return true if validate failed. */ public static boolean zpathBaseCheck(String zpathBase, String errorMsgPrefix) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index a2bc0c3688ea..fce83b8618ae 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -42,7 +42,6 @@ import com.metamx.druid.merger.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; -import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; @@ -53,7 +52,6 @@ import org.joda.time.Period; import javax.annotation.Nullable; -import java.io.IOException; import java.util.Comparator; import java.util.List; import java.util.Set; @@ -373,19 +371,7 @@ private void addWorker(final Worker worker) final WorkerWrapper workerWrapper = new WorkerWrapper( worker, statusCache, - 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); - } - } - } + jsonMapper ); // Add status listener to the watcher for status changes @@ -520,7 +506,7 @@ public boolean apply(WorkerWrapper input) log.info("Worker nodes do not have capacity to run any more tasks!"); if (currentlyProvisioning.isEmpty()) { - AutoScalingData provisioned = strategy.provision(currentlyProvisioning.size()); + AutoScalingData provisioned = strategy.provision(); if (provisioned != null) { currentlyProvisioning.addAll(provisioned.getNodeIds()); lastProvisionTime = new DateTime(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java index 68d4f0a128c9..c6353698cbd3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java @@ -20,13 +20,17 @@ package com.metamx.druid.merger.coordinator; import com.google.common.base.Function; +import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.worker.Worker; import com.netflix.curator.framework.recipes.cache.ChildData; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; +import org.codehaus.jackson.map.ObjectMapper; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.util.Set; @@ -41,11 +45,23 @@ public class WorkerWrapper implements Closeable private volatile DateTime lastCompletedTaskTime = new DateTime(); - public WorkerWrapper(Worker worker, PathChildrenCache statusCache, Function cacheConverter) + public WorkerWrapper(Worker worker, PathChildrenCache statusCache, final ObjectMapper jsonMapper) { this.worker = worker; this.statusCache = statusCache; - this.cacheConverter = cacheConverter; + 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 Worker getWorker() diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index a85c3ade8fde..cd94b70d3ce4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -56,7 +56,7 @@ public EC2AutoScalingStrategy( } @Override - public AutoScalingData provision(long numUnassignedTasks) + public AutoScalingData provision() { try { log.info("Creating new instance(s)..."); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java index 923de4638706..67eb99293e45 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java @@ -12,7 +12,7 @@ public class NoopScalingStrategy implements ScalingStrategy private static final EmittingLogger log = new EmittingLogger(NoopScalingStrategy.class); @Override - public AutoScalingData provision(long numUnassignedTasks) + public AutoScalingData provision() { log.info("If I were a real strategy I'd create something now"); return null; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java index ec71d8563016..9b7da8fb3a49 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java @@ -25,7 +25,7 @@ */ public interface ScalingStrategy { - public AutoScalingData provision(long numUnassignedTasks); + public AutoScalingData provision(); public AutoScalingData terminate(List nodeIds); }