Make Overlord auto-scaling and provisioning extensible (#4730)

* Make AutoScaler, ProvisioningStrategy and BaseWorkerBehaviorConfig extension points; More logging in PendingTaskBasedWorkerProvisioningStrategy

* Address comments and fix a bug

* Extract method

* debug logging

* Rename BaseWorkerBehaviorConfig to WorkerBehaviorConfig and WorkerBehaviorConfig to DefaultWorkerBehaviorConfig

* Fixes
This commit is contained in:
Roman Leventov 2017-10-02 20:12:23 -05:00 committed by GitHub
parent 6f91d9ca1e
commit 3f1009aaa1
23 changed files with 283 additions and 153 deletions

View File

@ -74,7 +74,6 @@ public class AuditEntry
}
/**
* @param None
* @return returns payload as String
*/
@JsonProperty
@ -82,8 +81,8 @@ public class AuditEntry
{
return payload;
}
/**
* @param None
* @return audit time as DateTime
*/
@JsonProperty

View File

@ -22,6 +22,7 @@ package io.druid.indexing.overlord;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableSet;
import io.druid.guice.annotations.PublicApi;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.worker.Worker;
import org.joda.time.DateTime;
@ -33,6 +34,7 @@ import java.util.Set;
/**
* A snapshot of a Worker and its current state i.e tasks assigned to that worker.
*/
@PublicApi
public class ImmutableWorkerInfo
{
private final Worker worker;

View File

@ -22,6 +22,7 @@ package io.druid.indexing.overlord;
import com.google.common.base.Optional;
import com.google.common.util.concurrent.ListenableFuture;
import io.druid.guice.annotations.PublicApi;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.autoscaling.ScalingStats;
@ -35,6 +36,7 @@ import java.util.concurrent.Executor;
* Interface for handing off tasks. Managed by a {@link TaskQueue}.
* Holds state
*/
@PublicApi
public interface TaskRunner
{
/**

View File

@ -19,31 +19,28 @@
package io.druid.indexing.overlord;
import com.google.common.base.Predicate;
import io.druid.guice.annotations.PublicApi;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.config.WorkerTaskRunnerConfig;
import io.druid.indexing.worker.Worker;
import java.util.Collection;
@PublicApi
public interface WorkerTaskRunner extends TaskRunner
{
/**
* List of known workers who can accept tasks
* @return A list of workers who can accept tasks for running
* List of known workers who can accept tasks for running
*/
Collection<ImmutableWorkerInfo> getWorkers();
/**
* Return a list of workers who can be reaped by autoscaling
* @return Workers which can be reaped by autoscaling
*/
Collection<Worker> getLazyWorkers();
/**
* Check which workers can be marked as lazy
* @param isLazyWorker
* @param maxWorkers
* @return
*/
Collection<Worker> markWorkersLazy(Predicate<ImmutableWorkerInfo> isLazyWorker, int maxWorkers);

View File

@ -21,8 +21,10 @@ package io.druid.indexing.overlord.autoscaling;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import io.druid.guice.annotations.ExtensionPoint;
import io.druid.indexing.overlord.autoscaling.ec2.EC2AutoScaler;
import javax.annotation.Nullable;
import java.util.List;
/**
@ -32,6 +34,7 @@ import java.util.List;
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "ec2", value = EC2AutoScaler.class)
})
@ExtensionPoint
public interface AutoScaler<T>
{
int getMinNumWorkers();
@ -40,10 +43,13 @@ public interface AutoScaler<T>
T getEnvConfig();
@Nullable
AutoScalingData provision();
@Nullable
AutoScalingData terminate(List<String> ips);
@Nullable
AutoScalingData terminateWithIds(List<String> ids);
/**

View File

@ -36,6 +36,7 @@ import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.ImmutableWorkerInfo;
import io.druid.indexing.overlord.WorkerTaskRunner;
import io.druid.indexing.overlord.config.WorkerTaskRunnerConfig;
import io.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig;
import io.druid.indexing.overlord.setup.WorkerBehaviorConfig;
import io.druid.indexing.overlord.setup.WorkerSelectStrategy;
import io.druid.indexing.worker.Worker;
@ -43,6 +44,7 @@ import io.druid.java.util.common.DateTimes;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
@ -58,6 +60,34 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
private static final String SCHEME = "http";
@Nullable
static DefaultWorkerBehaviorConfig getDefaultWorkerBehaviorConfig(
Supplier<WorkerBehaviorConfig> workerConfigRef,
String action,
EmittingLogger log
)
{
final WorkerBehaviorConfig workerBehaviorConfig = workerConfigRef.get();
if (workerBehaviorConfig == null) {
log.error("No workerConfig available, cannot %s workers.", action);
return null;
}
if (!(workerBehaviorConfig instanceof DefaultWorkerBehaviorConfig)) {
log.error(
"Only DefaultWorkerBehaviorConfig is supported as WorkerBehaviorConfig, [%s] given, cannot %s workers",
workerBehaviorConfig,
action
);
return null;
}
final DefaultWorkerBehaviorConfig workerConfig = (DefaultWorkerBehaviorConfig) workerBehaviorConfig;
if (workerConfig.getAutoScaler() == null) {
log.error("No autoScaler available, cannot %s workers", action);
return null;
}
return workerConfig;
}
private final PendingTaskBasedWorkerProvisioningConfig config;
private final Supplier<WorkerBehaviorConfig> workerConfigRef;
@ -121,11 +151,12 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
public synchronized boolean doProvision()
{
Collection<Task> pendingTasks = runner.getPendingTaskPayloads();
log.debug("Pending tasks: %d %s", pendingTasks.size(), pendingTasks);
Collection<ImmutableWorkerInfo> workers = runner.getWorkers();
log.debug("Workers: %d %s", workers.size(), workers);
boolean didProvision = false;
final WorkerBehaviorConfig workerConfig = workerConfigRef.get();
if (workerConfig == null || workerConfig.getAutoScaler() == null) {
log.error("No workerConfig available, cannot provision new workers.");
final DefaultWorkerBehaviorConfig workerConfig = getDefaultWorkerBehaviorConfig(workerConfigRef, "provision", log);
if (workerConfig == null) {
return false;
}
@ -143,25 +174,33 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
),
workerConfig
);
log.info("Currently provisioning: %d %s", currentlyProvisioning.size(), currentlyProvisioning);
currentlyProvisioning.removeAll(workerNodeIds);
log.debug(
"Currently provisioning without WorkerNodeIds: %d %s",
currentlyProvisioning.size(),
currentlyProvisioning
);
if (currentlyProvisioning.isEmpty()) {
int want = getScaleUpNodeCount(
int workersToProvision = getScaleUpNodeCount(
runner.getConfig(),
workerConfig,
pendingTasks,
workers
);
while (want > 0) {
log.info("Workers to provision: %d", workersToProvision);
while (workersToProvision > 0) {
final AutoScalingData provisioned = workerConfig.getAutoScaler().provision();
final List<String> newNodes;
if (provisioned == null || (newNodes = provisioned.getNodeIds()).isEmpty()) {
log.warn("NewNodes is empty, returning from provision loop");
break;
} else {
log.info("Provisioned: %d [%s]", provisioned.getNodeIds().size(), provisioned.getNodeIds());
currentlyProvisioning.addAll(newNodes);
lastProvisionTime = DateTimes.nowUtc();
scalingStats.addProvisionEvent(provisioned);
want -= provisioned.getNodeIds().size();
workersToProvision -= provisioned.getNodeIds().size();
didProvision = true;
}
}
@ -182,35 +221,38 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
return didProvision;
}
private Collection<String> getWorkerNodeIDs(Collection<Worker> workers, WorkerBehaviorConfig workerConfig)
private Collection<String> getWorkerNodeIDs(Collection<Worker> workers, DefaultWorkerBehaviorConfig workerConfig)
{
List<String> ips = new ArrayList<>(workers.size());
for (Worker worker : workers) {
ips.add(worker.getIp());
}
return workerConfig.getAutoScaler().ipToIdLookup(ips);
List<String> workerNodeIds = workerConfig.getAutoScaler().ipToIdLookup(ips);
log.info("WorkerNodeIds: %d %s", workerNodeIds.size(), workerNodeIds);
return workerNodeIds;
}
private int getScaleUpNodeCount(
final WorkerTaskRunnerConfig remoteTaskRunnerConfig,
final WorkerBehaviorConfig workerConfig,
final DefaultWorkerBehaviorConfig workerConfig,
final Collection<Task> pendingTasks,
final Collection<ImmutableWorkerInfo> workers
)
{
final int minWorkerCount = workerConfig.getAutoScaler().getMinNumWorkers();
final int maxWorkerCount = workerConfig.getAutoScaler().getMaxNumWorkers();
final Predicate<ImmutableWorkerInfo> isValidWorker = ProvisioningUtil.createValidWorkerPredicate(config);
final int currValidWorkers = Collections2.filter(workers, isValidWorker).size();
log.info("Min/max workers: %d/%d", minWorkerCount, maxWorkerCount);
final int currValidWorkers = getCurrValidWorkers(workers);
// If there are no worker, spin up minWorkerCount, we cannot determine the exact capacity here to fulfill the need since
// we are not aware of the expectedWorkerCapacity.
// If there are no worker, spin up minWorkerCount, we cannot determine the exact capacity here to fulfill the need
// since we are not aware of the expectedWorkerCapacity.
int moreWorkersNeeded = currValidWorkers == 0 ? minWorkerCount : getWorkersNeededToAssignTasks(
remoteTaskRunnerConfig,
workerConfig,
pendingTasks,
workers
);
log.debug("More workers needed: %d", moreWorkersNeeded);
int want = Math.max(
minWorkerCount - currValidWorkers,
@ -218,9 +260,14 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
Math.min(config.getMaxScalingStep(), moreWorkersNeeded)
// Additional workers needed to run current pending tasks
);
log.info("Want workers: %d", want);
if (want > 0 && currValidWorkers >= maxWorkerCount) {
log.warn("Unable to provision more workers. Current workerCount[%d] maximum workerCount[%d].", currValidWorkers, maxWorkerCount);
log.warn(
"Unable to provision more workers. Current workerCount[%d] maximum workerCount[%d].",
currValidWorkers,
maxWorkerCount
);
return 0;
}
want = Math.min(want, maxWorkerCount - currValidWorkers);
@ -229,7 +276,7 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
private int getWorkersNeededToAssignTasks(
final WorkerTaskRunnerConfig workerTaskRunnerConfig,
final WorkerBehaviorConfig workerConfig,
final DefaultWorkerBehaviorConfig workerConfig,
final Collection<Task> pendingTasks,
final Collection<ImmutableWorkerInfo> workers
)
@ -238,6 +285,7 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
workers,
ProvisioningUtil.createValidWorkerPredicate(config)
);
log.debug("Valid workers: %d %s", validWorkers.size(), validWorkers);
Map<String, ImmutableWorkerInfo> workersMap = Maps.newHashMap();
for (ImmutableWorkerInfo worker : validWorkers) {
@ -246,6 +294,7 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
WorkerSelectStrategy workerSelectStrategy = workerConfig.getSelectStrategy();
int need = 0;
int capacity = getExpectedWorkerCapacity(workers);
log.info("Expected worker capacity: %d", capacity);
// Simulate assigning tasks to dummy workers using configured workerSelectStrategy
// the number of additional workers needed to assign all the pending tasks is noted
@ -258,6 +307,7 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
final ImmutableWorkerInfo workerRunningTask;
if (selectedWorker != null) {
workerRunningTask = selectedWorker;
log.debug("Worker[%s] able to take the task[%s]", task, workerRunningTask);
} else {
// None of the existing worker can run this task, we need to provision one worker for it.
// create a dummy worker and try to simulate assigning task to it.
@ -267,6 +317,7 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
capacity,
workerTaskRunnerConfig.getMinWorkerVersion()
);
log.debug("Need more workers, creating a dummy worker[%s]", workerRunningTask);
need++;
}
// Update map with worker running task
@ -279,12 +330,13 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
public synchronized boolean doTerminate()
{
Collection<ImmutableWorkerInfo> zkWorkers = runner.getWorkers();
final WorkerBehaviorConfig workerConfig = workerConfigRef.get();
log.debug("Workers: %d [%s]", zkWorkers.size(), zkWorkers);
final DefaultWorkerBehaviorConfig workerConfig = getDefaultWorkerBehaviorConfig(workerConfigRef, "terminate", log);
if (workerConfig == null) {
log.warn("No workerConfig available, cannot terminate workers.");
return false;
}
log.info("Currently provisioning: %d %s", currentlyProvisioning.size(), currentlyProvisioning);
if (!currentlyProvisioning.isEmpty()) {
log.debug("Already provisioning nodes, Not Terminating any nodes.");
return false;
@ -292,17 +344,17 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
boolean didTerminate = false;
final Collection<String> workerNodeIds = getWorkerNodeIDs(runner.getLazyWorkers(), workerConfig);
final Set<String> stillExisting = Sets.newHashSet();
for (String s : currentlyTerminating) {
if (workerNodeIds.contains(s)) {
stillExisting.add(s);
}
}
currentlyTerminating.clear();
currentlyTerminating.addAll(stillExisting);
log.debug("Currently terminating: %d %s", currentlyTerminating.size(), currentlyTerminating);
currentlyTerminating.retainAll(workerNodeIds);
log.debug(
"Currently terminating among WorkerNodeIds: %d %s",
currentlyTerminating.size(),
currentlyTerminating
);
if (currentlyTerminating.isEmpty()) {
final int maxWorkersToTerminate = maxWorkersToTerminate(zkWorkers, workerConfig);
log.info("Max workers to terminate: %d", maxWorkersToTerminate);
final Predicate<ImmutableWorkerInfo> isLazyWorker = ProvisioningUtil.createLazyWorkerPredicate(config);
final Collection<String> laziestWorkerIps =
Collections2.transform(
@ -316,6 +368,7 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
}
}
);
log.info("Laziest worker ips: %d %s", laziestWorkerIps.size(), laziestWorkerIps);
if (laziestWorkerIps.isEmpty()) {
log.debug("Found no lazy workers");
} else {
@ -328,6 +381,7 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
final AutoScalingData terminated = workerConfig.getAutoScaler()
.terminate(ImmutableList.copyOf(laziestWorkerIps));
if (terminated != null) {
log.info("Terminated: %d %s", terminated.getNodeIds().size(), terminated.getNodeIds());
currentlyTerminating.addAll(terminated.getNodeIds());
lastTerminateTime = DateTimes.nowUtc();
scalingStats.addTerminateEvent(terminated);
@ -359,12 +413,12 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
}
}
private int maxWorkersToTerminate(Collection<ImmutableWorkerInfo> zkWorkers, WorkerBehaviorConfig workerConfig)
private int maxWorkersToTerminate(Collection<ImmutableWorkerInfo> zkWorkers, DefaultWorkerBehaviorConfig workerConfig)
{
final Predicate<ImmutableWorkerInfo> isValidWorker = ProvisioningUtil.createValidWorkerPredicate(config);
final int currValidWorkers = Collections2.filter(zkWorkers, isValidWorker).size();
final int currValidWorkers = getCurrValidWorkers(zkWorkers);
final int invalidWorkers = zkWorkers.size() - currValidWorkers;
final int minWorkers = workerConfig.getAutoScaler().getMinNumWorkers();
log.info("Min workers: %d", minWorkers);
// Max workers that can be terminated
// All invalid workers + any lazy workers above minCapacity
@ -377,6 +431,14 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr
);
}
private int getCurrValidWorkers(Collection<ImmutableWorkerInfo> workers)
{
final Predicate<ImmutableWorkerInfo> isValidWorker = ProvisioningUtil.createValidWorkerPredicate(config);
final int currValidWorkers = Collections2.filter(workers, isValidWorker).size();
log.debug("Current valid workers: %d", currValidWorkers);
return currValidWorkers;
}
private static int getExpectedWorkerCapacity(final Collection<ImmutableWorkerInfo> workers)
{
int size = workers.size();

View File

@ -19,7 +19,10 @@
package io.druid.indexing.overlord.autoscaling;
interface Provisioner
import io.druid.guice.annotations.PublicApi;
@PublicApi
public interface Provisioner
{
boolean doTerminate();

View File

@ -19,6 +19,8 @@
package io.druid.indexing.overlord.autoscaling;
import io.druid.guice.annotations.PublicApi;
import java.io.Closeable;
/**
@ -29,6 +31,7 @@ import java.io.Closeable;
*
* @see ProvisioningStrategy#makeProvisioningService
*/
@PublicApi
public interface ProvisioningService extends Closeable
{
/**

View File

@ -19,11 +19,13 @@
package io.druid.indexing.overlord.autoscaling;
import io.druid.guice.annotations.ExtensionPoint;
import io.druid.indexing.overlord.TaskRunner;
/**
* In general, the resource management is tied to the runner.
*/
@ExtensionPoint
public interface ProvisioningStrategy<T extends TaskRunner>
{
/**

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.collect.Lists;
import com.google.common.collect.MinMaxPriorityQueue;
import com.google.common.collect.Ordering;
import io.druid.guice.annotations.PublicApi;
import io.druid.java.util.common.DateTimes;
import org.joda.time.DateTime;
@ -33,6 +34,7 @@ import java.util.List;
/**
*/
@PublicApi
public class ScalingStats
{
public enum EVENT
@ -66,6 +68,15 @@ public class ScalingStats
}
}
public void addAll(ScalingStats stats)
{
synchronized (lock) {
synchronized (stats.lock) {
recentEvents.addAll(stats.recentEvents);
}
}
}
public void addProvisionEvent(AutoScalingData data)
{
synchronized (lock) {

View File

@ -35,6 +35,7 @@ import io.druid.indexing.overlord.ImmutableWorkerInfo;
import io.druid.indexing.overlord.TaskRunnerWorkItem;
import io.druid.indexing.overlord.WorkerTaskRunner;
import io.druid.indexing.overlord.setup.WorkerBehaviorConfig;
import io.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig;
import io.druid.indexing.worker.Worker;
import io.druid.java.util.common.DateTimes;
import org.joda.time.DateTime;
@ -118,9 +119,9 @@ public class SimpleWorkerProvisioningStrategy extends AbstractWorkerProvisioning
Collection<? extends TaskRunnerWorkItem> pendingTasks = runner.getPendingTasks();
Collection<ImmutableWorkerInfo> workers = runner.getWorkers();
boolean didProvision = false;
final WorkerBehaviorConfig workerConfig = workerConfigRef.get();
if (workerConfig == null || workerConfig.getAutoScaler() == null) {
log.error("No workerConfig available, cannot provision new workers.");
final DefaultWorkerBehaviorConfig workerConfig =
PendingTaskBasedWorkerProvisioningStrategy.getDefaultWorkerBehaviorConfig(workerConfigRef, "provision", log);
if (workerConfig == null) {
return false;
}
@ -183,9 +184,9 @@ public class SimpleWorkerProvisioningStrategy extends AbstractWorkerProvisioning
public synchronized boolean doTerminate()
{
Collection<? extends TaskRunnerWorkItem> pendingTasks = runner.getPendingTasks();
final WorkerBehaviorConfig workerConfig = workerConfigRef.get();
final DefaultWorkerBehaviorConfig workerConfig =
PendingTaskBasedWorkerProvisioningStrategy.getDefaultWorkerBehaviorConfig(workerConfigRef, "terminate", log);
if (workerConfig == null) {
log.warn("No workerConfig available, cannot terminate workers.");
return false;
}
@ -208,14 +209,7 @@ public class SimpleWorkerProvisioningStrategy extends AbstractWorkerProvisioning
)
);
final Set<String> stillExisting = Sets.newHashSet();
for (String s : currentlyTerminating) {
if (workerNodeIds.contains(s)) {
stillExisting.add(s);
}
}
currentlyTerminating.clear();
currentlyTerminating.addAll(stillExisting);
currentlyTerminating.retainAll(workerNodeIds);
Collection<ImmutableWorkerInfo> workers = runner.getWorkers();
updateTargetWorkerCount(workerConfig, pendingTasks, workers);
@ -277,7 +271,7 @@ public class SimpleWorkerProvisioningStrategy extends AbstractWorkerProvisioning
private void updateTargetWorkerCount(
final WorkerBehaviorConfig workerConfig,
final DefaultWorkerBehaviorConfig workerConfig,
final Collection<? extends TaskRunnerWorkItem> pendingTasks,
final Collection<ImmutableWorkerInfo> zkWorkers
)

View File

@ -31,6 +31,7 @@ import com.google.common.io.ByteSource;
import com.google.common.util.concurrent.SettableFuture;
import com.google.inject.Inject;
import com.sun.jersey.spi.container.ResourceFilters;
import io.druid.audit.AuditEntry;
import io.druid.audit.AuditInfo;
import io.druid.audit.AuditManager;
import io.druid.common.config.JacksonConfigManager;
@ -291,14 +292,12 @@ public class OverlordResource
Interval theInterval = interval == null ? null : Intervals.of(interval);
if (theInterval == null && count != null) {
try {
return Response.ok(
auditManager.fetchAuditHistory(
WorkerBehaviorConfig.CONFIG_KEY,
WorkerBehaviorConfig.CONFIG_KEY,
count
)
)
.build();
List<AuditEntry> workerEntryList = auditManager.fetchAuditHistory(
WorkerBehaviorConfig.CONFIG_KEY,
WorkerBehaviorConfig.CONFIG_KEY,
count
);
return Response.ok(workerEntryList).build();
}
catch (IllegalArgumentException e) {
return Response.status(Response.Status.BAD_REQUEST)
@ -306,14 +305,12 @@ public class OverlordResource
.build();
}
}
return Response.ok(
auditManager.fetchAuditHistory(
WorkerBehaviorConfig.CONFIG_KEY,
WorkerBehaviorConfig.CONFIG_KEY,
theInterval
)
)
.build();
List<AuditEntry> workerEntryList = auditManager.fetchAuditHistory(
WorkerBehaviorConfig.CONFIG_KEY,
WorkerBehaviorConfig.CONFIG_KEY,
theInterval
);
return Response.ok(workerEntryList).build();
}
@POST

View File

@ -0,0 +1,102 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.indexing.overlord.setup;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.indexing.overlord.autoscaling.AutoScaler;
import io.druid.indexing.overlord.autoscaling.NoopAutoScaler;
/**
*/
public class DefaultWorkerBehaviorConfig implements WorkerBehaviorConfig
{
private static final AutoScaler DEFAULT_AUTOSCALER = new NoopAutoScaler();
public static DefaultWorkerBehaviorConfig defaultConfig()
{
return new DefaultWorkerBehaviorConfig(DEFAULT_STRATEGY, DEFAULT_AUTOSCALER);
}
private final WorkerSelectStrategy selectStrategy;
private final AutoScaler autoScaler;
@JsonCreator
public DefaultWorkerBehaviorConfig(
@JsonProperty("selectStrategy") WorkerSelectStrategy selectStrategy,
@JsonProperty("autoScaler") AutoScaler autoScaler
)
{
this.selectStrategy = selectStrategy;
this.autoScaler = autoScaler;
}
@Override
@JsonProperty
public WorkerSelectStrategy getSelectStrategy()
{
return selectStrategy;
}
@JsonProperty
public AutoScaler<?> getAutoScaler()
{
return autoScaler;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DefaultWorkerBehaviorConfig that = (DefaultWorkerBehaviorConfig) o;
if (autoScaler != null ? !autoScaler.equals(that.autoScaler) : that.autoScaler != null) {
return false;
}
if (selectStrategy != null ? !selectStrategy.equals(that.selectStrategy) : that.selectStrategy != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = selectStrategy != null ? selectStrategy.hashCode() : 0;
result = 31 * result + (autoScaler != null ? autoScaler.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "WorkerConfiguration{" +
"selectStrategy=" + selectStrategy +
", autoScaler=" + autoScaler +
'}';
}
}

View File

@ -19,85 +19,28 @@
package io.druid.indexing.overlord.setup;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.indexing.overlord.autoscaling.AutoScaler;
import io.druid.indexing.overlord.autoscaling.NoopAutoScaler;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import io.druid.guice.annotations.ExtensionPoint;
/**
* Outside of {@link io.druid.indexing.overlord.autoscaling.PendingTaskBasedWorkerProvisioningStrategy} and
* {@link io.druid.indexing.overlord.autoscaling.SimpleWorkerProvisioningStrategy}, WorkerBehaviorConfig is used only
* in {@link io.druid.indexing.overlord.TaskRunner}, and only {@link #getSelectStrategy()} method is used. That is why
* the WorkerBehaviorConfig's interface is minimized to just this method. PendingTaskBasedWorkerProvisioningStrategy and
* SimpleWorkerProvisioningStrategy are written to work only with {@link DefaultWorkerBehaviorConfig}. Extension-defined
* WorkerBehaviorConfig implementations should likely be used with different extension-defined {@link
* io.druid.indexing.overlord.autoscaling.ProvisioningStrategy} implementations as well.
*/
public class WorkerBehaviorConfig
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultWorkerBehaviorConfig.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "default", value = DefaultWorkerBehaviorConfig.class)
})
@ExtensionPoint
public interface WorkerBehaviorConfig
{
public static final String CONFIG_KEY = "worker.config";
public static WorkerSelectStrategy DEFAULT_STRATEGY = new EqualDistributionWorkerSelectStrategy(null);
public static AutoScaler DEFAULT_AUTOSCALER = new NoopAutoScaler();
String CONFIG_KEY = "worker.config";
WorkerSelectStrategy DEFAULT_STRATEGY = new EqualDistributionWorkerSelectStrategy(null);
public static WorkerBehaviorConfig defaultConfig()
{
return new WorkerBehaviorConfig(DEFAULT_STRATEGY, DEFAULT_AUTOSCALER);
}
private final WorkerSelectStrategy selectStrategy;
private final AutoScaler autoScaler;
@JsonCreator
public WorkerBehaviorConfig(
@JsonProperty("selectStrategy") WorkerSelectStrategy selectStrategy,
@JsonProperty("autoScaler") AutoScaler autoScaler
)
{
this.selectStrategy = selectStrategy;
this.autoScaler = autoScaler;
}
@JsonProperty
public WorkerSelectStrategy getSelectStrategy()
{
return selectStrategy;
}
@JsonProperty
public AutoScaler<?> getAutoScaler()
{
return autoScaler;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
WorkerBehaviorConfig that = (WorkerBehaviorConfig) o;
if (autoScaler != null ? !autoScaler.equals(that.autoScaler) : that.autoScaler != null) {
return false;
}
if (selectStrategy != null ? !selectStrategy.equals(that.selectStrategy) : that.selectStrategy != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = selectStrategy != null ? selectStrategy.hashCode() : 0;
result = 31 * result + (autoScaler != null ? autoScaler.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "WorkerConfiguration{" +
"selectStrategy=" + selectStrategy +
", autoScaler=" + autoScaler +
'}';
}
WorkerSelectStrategy getSelectStrategy();
}

View File

@ -22,6 +22,7 @@ package io.druid.indexing.overlord.setup;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.common.collect.ImmutableMap;
import io.druid.guice.annotations.PublicApi;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.ImmutableWorkerInfo;
import io.druid.indexing.overlord.config.WorkerTaskRunnerConfig;
@ -39,6 +40,7 @@ import javax.annotation.Nullable;
@JsonSubTypes.Type(name = "equalDistributionWithAffinity", value = EqualDistributionWithAffinityWorkerSelectStrategy.class),
@JsonSubTypes.Type(name = "javascript", value = JavaScriptWorkerSelectStrategy.class)
})
@PublicApi
public interface WorkerSelectStrategy
{
/**

View File

@ -21,10 +21,12 @@ package io.druid.indexing.worker;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.guice.annotations.PublicApi;
/**
* A container for worker metadata.
*/
@PublicApi
public class Worker
{
private final String scheme;

View File

@ -23,6 +23,7 @@ import com.google.common.base.Supplier;
import io.druid.indexing.overlord.autoscaling.ProvisioningSchedulerConfig;
import io.druid.indexing.overlord.autoscaling.SimpleWorkerProvisioningConfig;
import io.druid.indexing.overlord.autoscaling.SimpleWorkerProvisioningStrategy;
import io.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig;
import io.druid.indexing.overlord.setup.WorkerBehaviorConfig;
import org.joda.time.Period;
import org.junit.After;
@ -34,11 +35,11 @@ public class OverlordBlinkLeadershipTest
{
private RemoteTaskRunnerTestUtils rtrUtils;
private final TestRemoteTaskRunnerConfig remoteTaskRunnerConfig = new TestRemoteTaskRunnerConfig(new Period("PT5M"));
private final WorkerBehaviorConfig defaultWorkerBehaviourConfig = WorkerBehaviorConfig.defaultConfig();
private final DefaultWorkerBehaviorConfig defaultWorkerBehaviourConfig = DefaultWorkerBehaviorConfig.defaultConfig();
private final Supplier<WorkerBehaviorConfig> workerBehaviorConfigSupplier = new Supplier<WorkerBehaviorConfig>()
{
@Override
public WorkerBehaviorConfig get()
public DefaultWorkerBehaviorConfig get()
{
return defaultWorkerBehaviourConfig;
}

View File

@ -36,6 +36,7 @@ import io.druid.indexing.overlord.autoscaling.NoopProvisioningStrategy;
import io.druid.indexing.overlord.autoscaling.ProvisioningStrategy;
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
import io.druid.indexing.overlord.setup.WorkerBehaviorConfig;
import io.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig;
import io.druid.indexing.worker.TaskAnnouncement;
import io.druid.indexing.worker.Worker;
import io.druid.java.util.common.StringUtils;
@ -130,7 +131,7 @@ public class RemoteTaskRunnerTestUtils
cf,
new PathChildrenCacheFactory.Builder(),
null,
DSuppliers.of(new AtomicReference<>(WorkerBehaviorConfig.defaultConfig())),
DSuppliers.of(new AtomicReference<>(DefaultWorkerBehaviorConfig.defaultConfig())),
provisioningStrategy
);

View File

@ -38,8 +38,9 @@ import io.druid.indexing.overlord.RemoteTaskRunner;
import io.druid.indexing.overlord.RemoteTaskRunnerWorkItem;
import io.druid.indexing.overlord.ZkWorker;
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
import io.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy;
import io.druid.indexing.overlord.setup.WorkerBehaviorConfig;
import io.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy;
import io.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig;
import io.druid.indexing.worker.TaskAnnouncement;
import io.druid.indexing.worker.Worker;
import io.druid.jackson.DefaultObjectMapper;
@ -85,7 +86,7 @@ public class PendingTaskBasedProvisioningStrategyTest
.setMaxScalingStep(2);
workerConfig = new AtomicReference<>(
new WorkerBehaviorConfig(
new DefaultWorkerBehaviorConfig(
new FillCapacityWorkerSelectStrategy(null),
autoScaler
)

View File

@ -39,6 +39,7 @@ import io.druid.indexing.overlord.RemoteTaskRunner;
import io.druid.indexing.overlord.RemoteTaskRunnerWorkItem;
import io.druid.indexing.overlord.ZkWorker;
import io.druid.indexing.overlord.setup.WorkerBehaviorConfig;
import io.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig;
import io.druid.indexing.worker.TaskAnnouncement;
import io.druid.indexing.worker.Worker;
import io.druid.jackson.DefaultObjectMapper;
@ -84,7 +85,7 @@ public class SimpleProvisioningStrategyTest
final ProvisioningSchedulerConfig schedulerConfig = new ProvisioningSchedulerConfig();
workerConfig = new AtomicReference<>(
new WorkerBehaviorConfig(
new DefaultWorkerBehaviorConfig(
null,
autoScaler
)

View File

@ -410,7 +410,7 @@ public class OverlordTest
}
@Override
public Collection<? extends TaskRunnerWorkItem> getPendingTasks()
public Collection<TaskRunnerWorkItem> getPendingTasks()
{
return ImmutableList.of();
}

View File

@ -40,7 +40,7 @@ public class WorkerBehaviorConfigTest
@Test
public void testSerde() throws Exception
{
WorkerBehaviorConfig config = new WorkerBehaviorConfig(
DefaultWorkerBehaviorConfig config = new DefaultWorkerBehaviorConfig(
new FillCapacityWithAffinityWorkerSelectStrategy(
new AffinityConfig(
ImmutableMap.of("foo", ImmutableSet.of("localhost")),
@ -87,6 +87,6 @@ public class WorkerBehaviorConfigTest
}
}
);
Assert.assertEquals(config, mapper.readValue(mapper.writeValueAsBytes(config), WorkerBehaviorConfig.class));
Assert.assertEquals(config, mapper.readValue(mapper.writeValueAsBytes(config), DefaultWorkerBehaviorConfig.class));
}
}

View File

@ -275,7 +275,6 @@ public class CliOverlord extends ServerRunnable
);
biddy.addBinding("simple").to(SimpleWorkerProvisioningStrategy.class);
biddy.addBinding("pendingTaskBased").to(PendingTaskBasedWorkerProvisioningStrategy.class);
}
private void configureOverlordHelpers(Binder binder)