diff --git a/extensions-core/ec2-extensions/src/main/java/org/apache/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java b/extensions-core/ec2-extensions/src/main/java/org/apache/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java index 56ffc6b74efc..4fc60eaa09cc 100644 --- a/extensions-core/ec2-extensions/src/main/java/org/apache/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java +++ b/extensions-core/ec2-extensions/src/main/java/org/apache/druid/indexing/overlord/autoscaling/ec2/EC2AutoScaler.java @@ -44,6 +44,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Objects; /** */ @@ -55,6 +56,7 @@ public class EC2AutoScaler implements AutoScaler private final int minNumWorkers; private final int maxNumWorkers; + private final String category; private final EC2EnvironmentConfig envConfig; private final AmazonEC2 amazonEC2Client; private final SimpleWorkerProvisioningConfig config; @@ -65,11 +67,13 @@ public EC2AutoScaler( @JsonProperty("maxNumWorkers") int maxNumWorkers, @JsonProperty("envConfig") EC2EnvironmentConfig envConfig, @JacksonInject AmazonEC2 amazonEC2Client, - @JacksonInject SimpleWorkerProvisioningConfig config + @JacksonInject SimpleWorkerProvisioningConfig config, + @JsonProperty("category") String category ) { this.minNumWorkers = minNumWorkers; this.maxNumWorkers = maxNumWorkers; + this.category = category; this.envConfig = envConfig; this.amazonEC2Client = amazonEC2Client; this.config = config; @@ -89,6 +93,13 @@ public int getMaxNumWorkers() return maxNumWorkers; } + @Override + @JsonProperty + public String getCategory() + { + return category; + } + @Override @JsonProperty public EC2EnvironmentConfig getEnvConfig() @@ -331,6 +342,7 @@ public String toString() "envConfig=" + envConfig + ", maxNumWorkers=" + maxNumWorkers + ", minNumWorkers=" + minNumWorkers + + ", category=" + category + '}'; } @@ -343,28 +355,16 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - EC2AutoScaler that = (EC2AutoScaler) o; - - if (maxNumWorkers != that.maxNumWorkers) { - return false; - } - if (minNumWorkers != that.minNumWorkers) { - return false; - } - if (envConfig != null ? !envConfig.equals(that.envConfig) : that.envConfig != null) { - return false; - } - - return true; + return minNumWorkers == that.minNumWorkers && + maxNumWorkers == that.maxNumWorkers && + Objects.equals(category, that.category) && + Objects.equals(envConfig, that.envConfig); } @Override public int hashCode() { - int result = minNumWorkers; - result = 31 * result + maxNumWorkers; - result = 31 * result + (envConfig != null ? envConfig.hashCode() : 0); - return result; + return Objects.hash(minNumWorkers, maxNumWorkers, category, envConfig); } } diff --git a/extensions-core/ec2-extensions/src/test/java/org/apache/druid/indexing/overlord/autoscaling/ec2/EC2AutoScalerTest.java b/extensions-core/ec2-extensions/src/test/java/org/apache/druid/indexing/overlord/autoscaling/ec2/EC2AutoScalerTest.java index bdeb9e81221a..64da2148c529 100644 --- a/extensions-core/ec2-extensions/src/test/java/org/apache/druid/indexing/overlord/autoscaling/ec2/EC2AutoScalerTest.java +++ b/extensions-core/ec2-extensions/src/test/java/org/apache/druid/indexing/overlord/autoscaling/ec2/EC2AutoScalerTest.java @@ -35,6 +35,7 @@ import com.google.common.collect.Range; import org.apache.druid.indexing.overlord.autoscaling.AutoScalingData; import org.apache.druid.indexing.overlord.autoscaling.SimpleWorkerProvisioningConfig; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.easymock.EasyMock; import org.junit.After; @@ -101,7 +102,8 @@ public void testScale() 1, ENV_CONFIG, amazonEC2Client, - managementConfig + managementConfig, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY ); EasyMock.expect(amazonEC2Client.runInstances(EasyMock.anyObject(RunInstancesRequest.class))).andReturn( @@ -145,7 +147,8 @@ public void testIptoIdLookup() 1, ENV_CONFIG, amazonEC2Client, - managementConfig + managementConfig, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY ); final int n = 150; @@ -198,7 +201,8 @@ public void testIdToIpLookup() 1, ENV_CONFIG, amazonEC2Client, - managementConfig + managementConfig, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY ); final int n = 150; diff --git a/extensions-core/ec2-extensions/src/test/java/org/apache/druid/indexing/overlord/setup/WorkerBehaviorConfigTest.java b/extensions-core/ec2-extensions/src/test/java/org/apache/druid/indexing/overlord/setup/WorkerBehaviorConfigTest.java index a47a79fa9bc1..2af0763d0e73 100644 --- a/extensions-core/ec2-extensions/src/test/java/org/apache/druid/indexing/overlord/setup/WorkerBehaviorConfigTest.java +++ b/extensions-core/ec2-extensions/src/test/java/org/apache/druid/indexing/overlord/setup/WorkerBehaviorConfigTest.java @@ -70,7 +70,8 @@ public void testSerde() throws Exception ) ), null, - null + null, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY ) ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/AutoScaler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/AutoScaler.java index 76b20cf42287..18c8e54cd0ed 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/AutoScaler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/AutoScaler.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; import javax.annotation.Nullable; import java.util.List; @@ -36,6 +37,11 @@ public interface AutoScaler int getMaxNumWorkers(); + default String getCategory() + { + return CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY; + } + /** * This method is unused, but AutoScaler is an {@link ExtensionPoint}, so we cannot remove it. */ diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningConfig.java new file mode 100644 index 000000000000..4e7aec2b7b6d --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningConfig.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.overlord.autoscaling; + +import org.joda.time.Period; + +public class CategoriedProvisioningConfig extends PendingTaskBasedWorkerProvisioningConfig +{ + @Override + public CategoriedProvisioningConfig setMaxScalingStep(int maxScalingStep) + { + super.setMaxScalingStep(maxScalingStep); + return this; + } + + @Override + public CategoriedProvisioningConfig setWorkerIdleTimeout(Period workerIdleTimeout) + { + super.setWorkerIdleTimeout(workerIdleTimeout); + return this; + } + + @Override + public CategoriedProvisioningConfig setMaxScalingDuration(Period maxScalingDuration) + { + super.setMaxScalingDuration(maxScalingDuration); + return this; + } + + @Override + public CategoriedProvisioningConfig setNumEventsToTrack(int numEventsToTrack) + { + super.setNumEventsToTrack(numEventsToTrack); + return this; + } + + @Override + public CategoriedProvisioningConfig setWorkerVersion(String workerVersion) + { + super.setWorkerVersion(workerVersion); + return this; + } + + @Override + public CategoriedProvisioningConfig setPendingTaskTimeout(Period pendingTaskTimeout) + { + super.setPendingTaskTimeout(pendingTaskTimeout); + return this; + } + + @Override + public CategoriedProvisioningConfig setWorkerPort(int workerPort) + { + super.setWorkerPort(workerPort); + return this; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategy.java new file mode 100644 index 000000000000..bd72b68336c1 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategy.java @@ -0,0 +1,655 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.overlord.autoscaling; + +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Joiner; +import com.google.common.base.Predicate; +import com.google.common.base.Supplier; +import com.google.common.collect.Collections2; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.inject.Inject; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.ImmutableWorkerInfo; +import org.apache.druid.indexing.overlord.WorkerTaskRunner; +import org.apache.druid.indexing.overlord.config.WorkerTaskRunnerConfig; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerSelectStrategy; +import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerCategorySpec; +import org.apache.druid.indexing.overlord.setup.WorkerSelectStrategy; +import org.apache.druid.indexing.overlord.setup.WorkerSelectUtils; +import org.apache.druid.indexing.worker.Worker; +import org.apache.druid.indexing.worker.config.WorkerConfig; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; +import org.apache.druid.java.util.emitter.EmittingLogger; +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.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; + +@JsonTypeName("categoriedTaskBased") +public class CategoriedProvisioningStrategy extends AbstractWorkerProvisioningStrategy +{ + private static final String SCHEME = "http"; + private static final EmittingLogger log = new EmittingLogger(CategoriedProvisioningStrategy.class); + + private final CategoriedProvisioningConfig config; + private final Supplier workerConfigRef; + + @Nullable + private static CategoriedWorkerBehaviorConfig getCategoriedWorkerBehaviorConfig( + Supplier 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 CategoriedWorkerBehaviorConfig)) { + log.error( + "Only CategoriedWorkerBehaviorConfig is supported as WorkerBehaviorConfig, [%s] given, cannot %s workers", + workerBehaviorConfig, + action + ); + return null; + } + final CategoriedWorkerBehaviorConfig workerConfig = (CategoriedWorkerBehaviorConfig) workerBehaviorConfig; + if (!(workerConfig.getSelectStrategy() instanceof CategoriedWorkerSelectStrategy)) { + log.error("Select strategy %s is not supported", workerConfig.getSelectStrategy()); + return null; + } + + if (workerConfig.getAutoScalers() == null || workerConfig.getAutoScalers().isEmpty()) { + log.error("At least one autoscaler should be specified."); + return null; + } + + return workerConfig; + } + + @Inject + public CategoriedProvisioningStrategy( + CategoriedProvisioningConfig config, + Supplier workerConfigRef, + ProvisioningSchedulerConfig provisioningSchedulerConfig + ) + { + this( + config, + workerConfigRef, + provisioningSchedulerConfig, + () -> ScheduledExecutors.fixed(1, "CategoriedProvisioning-manager--%d") + ); + } + + public CategoriedProvisioningStrategy( + CategoriedProvisioningConfig config, + Supplier workerConfigRef, + ProvisioningSchedulerConfig provisioningSchedulerConfig, + Supplier execFactory + ) + { + super(provisioningSchedulerConfig, execFactory); + this.config = config; + this.workerConfigRef = workerConfigRef; + } + + @Override + protected Provisioner makeProvisioner(WorkerTaskRunner runner) + { + return new CategoriedProvisioner(runner); + } + + private class CategoriedProvisioner implements Provisioner + { + private final WorkerTaskRunner runner; + private final ScalingStats scalingStats = new ScalingStats(config.getNumEventsToTrack()); + + private final Map> currentlyProvisioning = new HashMap<>(); + private final Map> currentlyTerminating = new HashMap<>(); + + private DateTime lastProvisionTime = DateTimes.nowUtc(); + private DateTime lastTerminateTime = DateTimes.nowUtc(); + + private CategoriedProvisioner(WorkerTaskRunner runner) + { + this.runner = runner; + } + + @Override + public synchronized boolean doProvision() + { + Collection pendingTasks = runner.getPendingTaskPayloads(); + log.debug("Pending tasks: %d %s", pendingTasks.size(), pendingTasks); + Collection workers = runner.getWorkers(); + log.debug("Workers: %d %s", workers.size(), workers); + boolean didProvision = false; + final CategoriedWorkerBehaviorConfig workerConfig = getCategoriedWorkerBehaviorConfig( + workerConfigRef, + "provision", + log + ); + if (workerConfig == null) { + log.info("No worker config found. Skip provisioning."); + return false; + } + + WorkerCategorySpec workerCategorySpec = getWorkerCategorySpec(workerConfig); + + // Group tasks by categories + Map> tasksByCategories = pendingTasks.stream().collect(Collectors.groupingBy( + task -> WorkerSelectUtils.getTaskCategory( + task, + workerCategorySpec, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY + ) + )); + + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(workers); + + // Merge categories of tasks and workers + Set allCategories = new HashSet<>(tasksByCategories.keySet()); + allCategories.addAll(workersByCategories.keySet()); + + log.debug( + "Pending Tasks of %d categories (%s), Workers of %d categories (%s). %d common categories: %s", + tasksByCategories.size(), + tasksByCategories.keySet(), + workersByCategories.size(), + workersByCategories.keySet(), + allCategories.size(), + allCategories + ); + + if (allCategories.isEmpty()) { + // Likely empty categories means initialization. Just try to spinup required amount of workers of each non empty autoscalers + for (AutoScaler autoScaler : workerConfig.getAutoScalers()) { + String category = autoScaler.getCategory(); + didProvision = initAutoscaler(autoScaler, category, workerConfig) || didProvision; + } + return didProvision; + } + + Map autoscalersByCategory = ProvisioningUtil.mapAutoscalerByCategory(workerConfig.getAutoScalers()); + + for (String category : allCategories) { + List categoryTasks = tasksByCategories.getOrDefault(category, Collections.emptyList()); + List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); + currentlyProvisioning.putIfAbsent(category, new HashSet<>()); + Set currentlyProvisioning = this.currentlyProvisioning.get(category); + AutoScaler groupAutoscaler = ProvisioningUtil.getAutoscalerByCategory(category, autoscalersByCategory); + + didProvision = doProvision( + category, + categoryWorkers, + categoryTasks, + workerConfig, + currentlyProvisioning, + groupAutoscaler + ) || didProvision; + } + + return didProvision; + } + + private boolean doProvision( + String category, + Collection workers, + Collection pendingTasks, + CategoriedWorkerBehaviorConfig workerConfig, + Set currentlyProvisioning, + AutoScaler autoScaler + ) + { + boolean didProvision = false; + + if (autoScaler == null) { + log.error("No autoScaler available, cannot execute doProvision for workers of category %s", category); + return false; + } + + final Collection workerNodeIds = getWorkerNodeIDs( + Collections2.transform( + workers, + ImmutableWorkerInfo::getWorker + ), + autoScaler + ); + 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 workersToProvision = getScaleUpNodeCount( + runner.getConfig(), + workerConfig, + pendingTasks, + workers, + autoScaler + ); + log.info("Workers to provision: %d", workersToProvision); + while (workersToProvision > 0) { + final AutoScalingData provisioned = autoScaler.provision(); + final List 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); + workersToProvision -= provisioned.getNodeIds().size(); + didProvision = true; + } + } + } else { + Duration durSinceLastProvision = new Duration(lastProvisionTime, DateTimes.nowUtc()); + log.info("%s provisioning. Current wait time: %s", currentlyProvisioning, durSinceLastProvision); + if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { + log.makeAlert("Worker node provisioning taking too long!") + .addData("millisSinceLastProvision", durSinceLastProvision.getMillis()) + .addData("provisioningCount", currentlyProvisioning.size()) + .emit(); + + autoScaler.terminateWithIds(Lists.newArrayList(currentlyProvisioning)); + currentlyProvisioning.clear(); + } + } + + return didProvision; + } + + private Collection getWorkerNodeIDs(Collection workers, AutoScaler autoScaler) + { + List ips = new ArrayList<>(workers.size()); + for (Worker worker : workers) { + ips.add(worker.getIp()); + } + List workerNodeIds = autoScaler.ipToIdLookup(ips); + log.info("WorkerNodeIds: %d %s", workerNodeIds.size(), workerNodeIds); + return workerNodeIds; + } + + private int getScaleUpNodeCount( + final WorkerTaskRunnerConfig remoteTaskRunnerConfig, + final CategoriedWorkerBehaviorConfig workerConfig, + final Collection pendingTasks, + final Collection workers, + AutoScaler autoScaler + ) + { + final int minWorkerCount = autoScaler.getMinNumWorkers(); + final int maxWorkerCount = autoScaler.getMaxNumWorkers(); + log.info("Min/max workers: %d/%d", minWorkerCount, maxWorkerCount); + final int currValidWorkers = getCurrValidWorkers(workers); + + // If there are no worker, spin up minWorkerCount (or 1 if minWorkerCount is 0), we cannot determine the exact capacity here to fulfill the need + // since we are not aware of the expectedWorkerCapacity. + int moreWorkersNeeded = currValidWorkers == 0 ? Math.max(minWorkerCount, pendingTasks.isEmpty() ? 0 : 1) : getWorkersNeededToAssignTasks( + remoteTaskRunnerConfig, + workerConfig, + pendingTasks, + workers + ); + log.debug("More workers needed: %d", moreWorkersNeeded); + + int want = Math.max( + minWorkerCount - currValidWorkers, + // Additional workers needed to reach minWorkerCount + 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 + ); + return 0; + } + want = Math.min(want, maxWorkerCount - currValidWorkers); + return want; + } + + private int getWorkersNeededToAssignTasks( + final WorkerTaskRunnerConfig workerTaskRunnerConfig, + final CategoriedWorkerBehaviorConfig workerConfig, + final Collection pendingTasks, + final Collection workers + ) + { + final Collection validWorkers = Collections2.filter( + workers, + ProvisioningUtil.createValidWorkerPredicate(config) + ); + log.debug("Valid workers: %d %s", validWorkers.size(), validWorkers); + + Map workersMap = new HashMap<>(); + for (ImmutableWorkerInfo worker : validWorkers) { + workersMap.put(worker.getWorker().getHost(), worker); + } + 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 + for (Task task : pendingTasks) { + final ImmutableWorkerInfo selectedWorker = workerSelectStrategy.findWorkerForTask( + workerTaskRunnerConfig, + ImmutableMap.copyOf(workersMap), + task + ); + 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. + workerRunningTask = createDummyWorker( + SCHEME, + "dummy" + need, + capacity, + workerTaskRunnerConfig.getMinWorkerVersion() + ); + log.debug("Need more workers, creating a dummy worker[%s]", workerRunningTask); + need++; + } + // Update map with worker running task + workersMap.put(workerRunningTask.getWorker().getHost(), workerWithTask(workerRunningTask, task)); + } + return need; + } + + @Override + public synchronized boolean doTerminate() + { + Collection zkWorkers = runner.getWorkers(); + log.debug("Workers: %d [%s]", zkWorkers.size(), zkWorkers); + final CategoriedWorkerBehaviorConfig workerConfig = getCategoriedWorkerBehaviorConfig( + workerConfigRef, + "terminate", + log + ); + if (workerConfig == null) { + return false; + } + + boolean didTerminate = false; + + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(zkWorkers); + + Set allCategories = workersByCategories.keySet(); + log.debug( + "Workers of %d categories: %s", + workersByCategories.size(), + allCategories + ); + + Map autoscalersByCategory = ProvisioningUtil.mapAutoscalerByCategory(workerConfig.getAutoScalers()); + + for (String category : allCategories) { + Set currentlyProvisioning = this.currentlyProvisioning.getOrDefault(category, Collections.emptySet()); + log.info( + "Currently provisioning of category %s: %d %s", + category, + currentlyProvisioning.size(), + currentlyProvisioning + ); + if (!currentlyProvisioning.isEmpty()) { + log.debug("Already provisioning nodes of category %s, Not Terminating any nodes.", category); + return false; + } + + List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); + currentlyTerminating.putIfAbsent(category, new HashSet<>()); + Set currentlyTerminating = this.currentlyTerminating.get(category); + AutoScaler groupAutoscaler = ProvisioningUtil.getAutoscalerByCategory(category, autoscalersByCategory); + + didTerminate = doTerminate( + category, + categoryWorkers, + currentlyTerminating, + groupAutoscaler + ) || didTerminate; + } + + return didTerminate; + } + + private boolean doTerminate( + String category, + Collection zkWorkers, + Set currentlyTerminating, + AutoScaler autoScaler + ) + { + if (autoScaler == null) { + log.error("No autoScaler available, cannot execute doTerminate for workers of category %s", category); + return false; + } + + boolean didTerminate = false; + final Collection workerNodeIds = getWorkerNodeIDs(runner.getLazyWorkers(), autoScaler); + log.debug( + "Currently terminating of category %s: %d %s", + category, + currentlyTerminating.size(), + currentlyTerminating + ); + currentlyTerminating.retainAll(workerNodeIds); + log.debug( + "Currently terminating of category %s among WorkerNodeIds: %d %s", + category, + currentlyTerminating.size(), + currentlyTerminating + ); + + if (currentlyTerminating.isEmpty()) { + final int maxWorkersToTerminate = maxWorkersToTerminate(zkWorkers, autoScaler); + log.info("Max workers to terminate of category %s: %d", category, maxWorkersToTerminate); + final Predicate isLazyWorker = ProvisioningUtil.createLazyWorkerPredicate(config); + final Collection laziestWorkerIps = + Collections2.transform( + runner.markWorkersLazy(isLazyWorker, maxWorkersToTerminate), + Worker::getIp + ); + log.info("Laziest worker ips of category %s: %d %s", category, laziestWorkerIps.size(), laziestWorkerIps); + if (laziestWorkerIps.isEmpty()) { + log.debug("Found no lazy workers for category %s", category); + } else { + log.info( + "Terminating %,d lazy workers of category %s: %s", + laziestWorkerIps.size(), + category, + Joiner.on(", ").join(laziestWorkerIps) + ); + + final AutoScalingData terminated = autoScaler.terminate(ImmutableList.copyOf(laziestWorkerIps)); + if (terminated != null) { + log.info( + "Terminated of category %s: %d %s", + category, + terminated.getNodeIds().size(), + terminated.getNodeIds() + ); + currentlyTerminating.addAll(terminated.getNodeIds()); + lastTerminateTime = DateTimes.nowUtc(); + scalingStats.addTerminateEvent(terminated); + didTerminate = true; + } + } + } else { + Duration durSinceLastTerminate = new Duration(lastTerminateTime, DateTimes.nowUtc()); + + log.info( + "%s terminating of category %s. Current wait time: %s", + currentlyTerminating, + category, + durSinceLastTerminate + ); + + if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { + log.makeAlert("Worker node termination taking too long!") + .addData("millisSinceLastTerminate", durSinceLastTerminate.getMillis()) + .addData("terminatingCount", currentlyTerminating.size()) + .emit(); + + currentlyTerminating.clear(); + } + } + + return didTerminate; + } + + @Override + public ScalingStats getStats() + { + return scalingStats; + } + + private int getCurrValidWorkers(Collection workers) + { + final Predicate isValidWorker = ProvisioningUtil.createValidWorkerPredicate(config); + final int currValidWorkers = Collections2.filter(workers, isValidWorker).size(); + log.debug("Current valid workers: %d", currValidWorkers); + return currValidWorkers; + } + + private int getExpectedWorkerCapacity(final Collection workers) + { + int size = workers.size(); + if (size == 0) { + // No existing workers assume capacity per worker as 1 + return 1; + } else { + // Assume all workers have same capacity + return workers.iterator().next().getWorker().getCapacity(); + } + } + + private ImmutableWorkerInfo createDummyWorker(String scheme, String host, int capacity, String version) + { + return new ImmutableWorkerInfo( + new Worker(scheme, host, "-2", capacity, version, WorkerConfig.DEFAULT_CATEGORY), + 0, + new HashSet<>(), + new HashSet<>(), + DateTimes.nowUtc() + ); + } + + private ImmutableWorkerInfo workerWithTask(ImmutableWorkerInfo immutableWorker, Task task) + { + return new ImmutableWorkerInfo( + immutableWorker.getWorker(), + immutableWorker.getCurrCapacityUsed() + 1, + Sets.union( + immutableWorker.getAvailabilityGroups(), + Sets.newHashSet( + task.getTaskResource() + .getAvailabilityGroup() + ) + ), + Sets.union( + immutableWorker.getRunningTasks(), + Sets.newHashSet( + task.getId() + ) + ), + DateTimes.nowUtc() + ); + } + + private int maxWorkersToTerminate(Collection zkWorkers, AutoScaler autoScaler) + { + final int currValidWorkers = getCurrValidWorkers(zkWorkers); + final int invalidWorkers = zkWorkers.size() - currValidWorkers; + final int minWorkers = autoScaler.getMinNumWorkers(); + log.info("Min workers: %d", minWorkers); + + // Max workers that can be terminated + // All invalid workers + any lazy workers above minCapacity + return invalidWorkers + Math.max( + 0, + Math.min( + config.getMaxScalingStep(), + currValidWorkers - minWorkers + ) + ); + } + + private boolean initAutoscaler(AutoScaler autoScaler, String category, CategoriedWorkerBehaviorConfig workerConfig) + { + currentlyProvisioning.putIfAbsent( + category, + new HashSet<>() + ); + Set currentlyProvisioning = this.currentlyProvisioning.get(category); + return doProvision( + category, + Collections.emptyList(), + Collections.emptyList(), + workerConfig, + currentlyProvisioning, + autoScaler + ); + } + + @Nullable + private WorkerCategorySpec getWorkerCategorySpec(CategoriedWorkerBehaviorConfig workerConfig) + { + if (workerConfig != null && workerConfig.getSelectStrategy() != null) { + WorkerSelectStrategy selectStrategy = workerConfig.getSelectStrategy(); + if (selectStrategy instanceof CategoriedWorkerSelectStrategy) { + return ((CategoriedWorkerSelectStrategy) selectStrategy).getWorkerCategorySpec(); + } + } + return null; + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/NoopAutoScaler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/NoopAutoScaler.java index 457f08bce2c8..6448bf874e0c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/NoopAutoScaler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/NoopAutoScaler.java @@ -19,6 +19,8 @@ package org.apache.druid.indexing.overlord.autoscaling; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -30,6 +32,15 @@ public class NoopAutoScaler implements AutoScaler { private static final EmittingLogger log = new EmittingLogger(NoopAutoScaler.class); + private final String category; + + @JsonCreator + public NoopAutoScaler( + @JsonProperty("category") String category + ) + { + this.category = category; + } @Override public int getMinNumWorkers() @@ -43,6 +54,13 @@ public int getMaxNumWorkers() return 0; } + @Override + @JsonProperty + public String getCategory() + { + return category; + } + @Override public Void getEnvConfig() { @@ -52,35 +70,43 @@ public Void getEnvConfig() @Override public AutoScalingData provision() { - log.info("If I were a real strategy I'd create something now"); + log.info("If I were a real strategy I'd create something now [category: %s]", category); return null; } @Override public AutoScalingData terminate(List ips) { - log.info("If I were a real strategy I'd terminate %s now", ips); + log.info("If I were a real strategy I'd terminate %s now [category: %s]", ips, category); return null; } @Override public AutoScalingData terminateWithIds(List ids) { - log.info("If I were a real strategy I'd terminate %s now", ids); + log.info("If I were a real strategy I'd terminate %s now [category: %s]", ids, category); return null; } @Override public List ipToIdLookup(List ips) { - log.info("I'm not a real strategy so I'm returning what I got %s", ips); + log.info("I'm not a real strategy so I'm returning what I got %s [category: %s]", ips, category); return ips; } @Override public List idToIpLookup(List nodeIds) { - log.info("I'm not a real strategy so I'm returning what I got %s", nodeIds); + log.info("I'm not a real strategy so I'm returning what I got %s [category: %s]", nodeIds, category); return nodeIds; } + + @Override + public String toString() + { + return "NoopAutoScaler{" + + "category='" + category + '\'' + + '}'; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerProvisioningStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerProvisioningStrategy.java index 52624f41e4e0..6b3dd07e4aee 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerProvisioningStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedWorkerProvisioningStrategy.java @@ -19,7 +19,6 @@ package org.apache.druid.indexing.overlord.autoscaling; -import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Predicate; import com.google.common.base.Supplier; @@ -33,9 +32,11 @@ import org.apache.druid.indexing.overlord.ImmutableWorkerInfo; import org.apache.druid.indexing.overlord.WorkerTaskRunner; import org.apache.druid.indexing.overlord.config.WorkerTaskRunnerConfig; -import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerCategorySpec; import org.apache.druid.indexing.overlord.setup.WorkerSelectStrategy; +import org.apache.druid.indexing.overlord.setup.WorkerSelectUtils; import org.apache.druid.indexing.worker.Worker; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.DateTimes; @@ -44,17 +45,19 @@ 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.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; /** + * */ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerProvisioningStrategy { @@ -62,34 +65,6 @@ public class PendingTaskBasedWorkerProvisioningStrategy extends AbstractWorkerPr private static final String SCHEME = "http"; - @Nullable - static DefaultWorkerBehaviorConfig getDefaultWorkerBehaviorConfig( - Supplier 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 workerConfigRef; @@ -104,14 +79,7 @@ public PendingTaskBasedWorkerProvisioningStrategy( config, workerConfigRef, provisioningSchedulerConfig, - new Supplier() - { - @Override - public ScheduledExecutorService get() - { - return ScheduledExecutors.fixed(1, "PendingTaskBasedWorkerProvisioning-manager--%d"); - } - } + () -> ScheduledExecutors.fixed(1, "PendingTaskBasedWorkerProvisioning-manager--%d") ); } @@ -138,11 +106,11 @@ private class PendingProvisioner implements Provisioner private final WorkerTaskRunner runner; private final ScalingStats scalingStats = new ScalingStats(config.getNumEventsToTrack()); - private final Set currentlyProvisioning = new HashSet<>(); - private final Set currentlyTerminating = new HashSet<>(); + private final Map> currentlyProvisioningMap = new HashMap<>(); + private final Map> currentlyTerminatingMap = new HashMap<>(); - private DateTime lastProvisionTime = DateTimes.nowUtc(); - private DateTime lastTerminateTime = lastProvisionTime; + private final Map lastProvisionTimeMap = new HashMap<>(); + private final Map lastTerminateTimeMap = new HashMap<>(); private PendingProvisioner(WorkerTaskRunner runner) { @@ -157,24 +125,94 @@ public synchronized boolean doProvision() Collection workers = runner.getWorkers(); log.debug("Workers: %d %s", workers.size(), workers); boolean didProvision = false; - final DefaultWorkerBehaviorConfig workerConfig = getDefaultWorkerBehaviorConfig(workerConfigRef, "provision", log); + final CategoriedWorkerBehaviorConfig workerConfig = ProvisioningUtil.getCategoriedWorkerBehaviorConfig( + workerConfigRef, + "provision" + ); if (workerConfig == null) { + log.info("No worker config found. Skip provisioning."); return false; } + WorkerCategorySpec workerCategorySpec = ProvisioningUtil.getWorkerCategorySpec(workerConfig); + + // Group tasks by categories + Map> tasksByCategories = pendingTasks.stream().collect(Collectors.groupingBy( + task -> WorkerSelectUtils.getTaskCategory( + task, + workerCategorySpec, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY + ) + )); + + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(workers); + + // Merge categories of tasks and workers + Set allCategories = new HashSet<>(tasksByCategories.keySet()); + allCategories.addAll(workersByCategories.keySet()); + + log.debug( + "Pending Tasks of %d categories (%s), Workers of %d categories (%s). %d common categories: %s", + tasksByCategories.size(), + tasksByCategories.keySet(), + workersByCategories.size(), + workersByCategories.keySet(), + allCategories.size(), + allCategories + ); + + if (allCategories.isEmpty()) { + // Likely empty categories means initialization. + // Just try to spinup required amount of workers of each non empty autoscalers + return initAutoscalers(workerConfig); + } + + Map autoscalersByCategory = ProvisioningUtil.mapAutoscalerByCategory(workerConfig.getAutoScalers()); + + for (String category : allCategories) { + AutoScaler categoryAutoscaler = ProvisioningUtil.getAutoscalerByCategory(category, autoscalersByCategory); + if (categoryAutoscaler == null) { + log.error("No autoScaler available, cannot execute doProvision for workers of category %s", category); + continue; + } + // Correct category name by selected autoscaler + category = ProvisioningUtil.getAutoscalerCategory(categoryAutoscaler); + + List categoryTasks = tasksByCategories.getOrDefault(category, Collections.emptyList()); + List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); + currentlyProvisioningMap.putIfAbsent(category, new HashSet<>()); + Set currentlyProvisioning = this.currentlyProvisioningMap.get(category); + + didProvision = doProvision( + category, + categoryWorkers, + categoryTasks, + workerConfig, + currentlyProvisioning, + categoryAutoscaler + ) || didProvision; + } + + return didProvision; + } + + private boolean doProvision( + String category, + Collection workers, + Collection pendingTasks, + CategoriedWorkerBehaviorConfig workerConfig, + Set currentlyProvisioning, + AutoScaler autoScaler + ) + { + boolean didProvision = false; + final Collection workerNodeIds = getWorkerNodeIDs( Collections2.transform( workers, - new Function() - { - @Override - public Worker apply(ImmutableWorkerInfo input) - { - return input.getWorker(); - } - } + ImmutableWorkerInfo::getWorker ), - workerConfig + autoScaler ); log.info("Currently provisioning: %d %s", currentlyProvisioning.size(), currentlyProvisioning); currentlyProvisioning.removeAll(workerNodeIds); @@ -183,16 +221,18 @@ public Worker apply(ImmutableWorkerInfo input) currentlyProvisioning.size(), currentlyProvisioning ); + if (currentlyProvisioning.isEmpty()) { int workersToProvision = getScaleUpNodeCount( runner.getConfig(), workerConfig, pendingTasks, - workers + workers, + autoScaler ); log.info("Workers to provision: %d", workersToProvision); while (workersToProvision > 0) { - final AutoScalingData provisioned = workerConfig.getAutoScaler().provision(); + final AutoScalingData provisioned = autoScaler.provision(); final List newNodes; if (provisioned == null || (newNodes = provisioned.getNodeIds()).isEmpty()) { log.warn("NewNodes is empty, returning from provision loop"); @@ -200,13 +240,14 @@ public Worker apply(ImmutableWorkerInfo input) } else { log.info("Provisioned: %d [%s]", provisioned.getNodeIds().size(), provisioned.getNodeIds()); currentlyProvisioning.addAll(newNodes); - lastProvisionTime = DateTimes.nowUtc(); + lastProvisionTimeMap.put(category, DateTimes.nowUtc()); scalingStats.addProvisionEvent(provisioned); workersToProvision -= provisioned.getNodeIds().size(); didProvision = true; } } } else { + DateTime lastProvisionTime = lastProvisionTimeMap.getOrDefault(category, DateTimes.nowUtc()); Duration durSinceLastProvision = new Duration(lastProvisionTime, DateTimes.nowUtc()); log.info("%s provisioning. Current wait time: %s", currentlyProvisioning, durSinceLastProvision); if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { @@ -215,7 +256,7 @@ public Worker apply(ImmutableWorkerInfo input) .addData("provisioningCount", currentlyProvisioning.size()) .emit(); - workerConfig.getAutoScaler().terminateWithIds(Lists.newArrayList(currentlyProvisioning)); + autoScaler.terminateWithIds(Lists.newArrayList(currentlyProvisioning)); currentlyProvisioning.clear(); } } @@ -223,37 +264,40 @@ public Worker apply(ImmutableWorkerInfo input) return didProvision; } - private Collection getWorkerNodeIDs(Collection workers, DefaultWorkerBehaviorConfig workerConfig) + private Collection getWorkerNodeIDs(Collection workers, AutoScaler autoScaler) { List ips = new ArrayList<>(workers.size()); for (Worker worker : workers) { ips.add(worker.getIp()); } - List workerNodeIds = workerConfig.getAutoScaler().ipToIdLookup(ips); + List workerNodeIds = autoScaler.ipToIdLookup(ips); log.info("WorkerNodeIds: %d %s", workerNodeIds.size(), workerNodeIds); return workerNodeIds; } private int getScaleUpNodeCount( final WorkerTaskRunnerConfig remoteTaskRunnerConfig, - final DefaultWorkerBehaviorConfig workerConfig, + final CategoriedWorkerBehaviorConfig workerConfig, final Collection pendingTasks, - final Collection workers + final Collection workers, + AutoScaler autoScaler ) { - final int minWorkerCount = workerConfig.getAutoScaler().getMinNumWorkers(); - final int maxWorkerCount = workerConfig.getAutoScaler().getMaxNumWorkers(); + final int minWorkerCount = autoScaler.getMinNumWorkers(); + final int maxWorkerCount = autoScaler.getMaxNumWorkers(); 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 + // If there are no worker, spin up minWorkerCount (or 1 if minWorkerCount is 0 and there are pending tasks), 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 - ); + int moreWorkersNeeded = currValidWorkers == 0 + ? Math.max(minWorkerCount, pendingTasks.isEmpty() ? 0 : 1) + : getWorkersNeededToAssignTasks( + remoteTaskRunnerConfig, + workerConfig, + pendingTasks, + workers + ); log.debug("More workers needed: %d", moreWorkersNeeded); int want = Math.max( @@ -278,7 +322,7 @@ private int getScaleUpNodeCount( private int getWorkersNeededToAssignTasks( final WorkerTaskRunnerConfig workerTaskRunnerConfig, - final DefaultWorkerBehaviorConfig workerConfig, + final CategoriedWorkerBehaviorConfig workerConfig, final Collection pendingTasks, final Collection workers ) @@ -333,67 +377,136 @@ public synchronized boolean doTerminate() { Collection zkWorkers = runner.getWorkers(); log.debug("Workers: %d [%s]", zkWorkers.size(), zkWorkers); - final DefaultWorkerBehaviorConfig workerConfig = getDefaultWorkerBehaviorConfig(workerConfigRef, "terminate", log); + final CategoriedWorkerBehaviorConfig workerConfig = ProvisioningUtil.getCategoriedWorkerBehaviorConfig( + workerConfigRef, + "terminate" + ); if (workerConfig == null) { + log.info("No worker config found. Skip terminating."); 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; + boolean didTerminate = false; + + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(zkWorkers); + + Set allCategories = workersByCategories.keySet(); + log.debug( + "Workers of %d categories: %s", + workersByCategories.size(), + allCategories + ); + + Map autoscalersByCategory = ProvisioningUtil.mapAutoscalerByCategory(workerConfig.getAutoScalers()); + + for (String category : allCategories) { + Set currentlyProvisioning = this.currentlyProvisioningMap.getOrDefault( + category, + Collections.emptySet() + ); + log.info( + "Currently provisioning of category %s: %d %s", + category, + currentlyProvisioning.size(), + currentlyProvisioning + ); + if (!currentlyProvisioning.isEmpty()) { + log.debug("Already provisioning nodes of category %s, Not Terminating any nodes.", category); + return false; + } + + AutoScaler categoryAutoscaler = ProvisioningUtil.getAutoscalerByCategory(category, autoscalersByCategory); + if (categoryAutoscaler == null) { + log.error("No autoScaler available, cannot execute doTerminate for workers of category %s", category); + continue; + } + // Correct category name by selected autoscaler + category = ProvisioningUtil.getAutoscalerCategory(categoryAutoscaler); + + List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); + currentlyTerminatingMap.putIfAbsent(category, new HashSet<>()); + Set currentlyTerminating = this.currentlyTerminatingMap.get(category); + + didTerminate = doTerminate( + category, + categoryWorkers, + currentlyTerminating, + categoryAutoscaler + ) || didTerminate; } + return didTerminate; + } + + private boolean doTerminate( + String category, + Collection zkWorkers, + Set currentlyTerminating, + AutoScaler autoScaler + ) + { boolean didTerminate = false; - final Collection workerNodeIds = getWorkerNodeIDs(runner.getLazyWorkers(), workerConfig); - log.debug("Currently terminating: %d %s", currentlyTerminating.size(), currentlyTerminating); + + Collection lazyWorkers = ProvisioningUtil.getWorkersOfCategory(runner.getLazyWorkers(), category); + final Collection workerNodeIds = getWorkerNodeIDs(lazyWorkers, autoScaler); + log.debug( + "Currently terminating of category %s: %d %s", + category, + currentlyTerminating.size(), + currentlyTerminating + ); currentlyTerminating.retainAll(workerNodeIds); log.debug( - "Currently terminating among WorkerNodeIds: %d %s", + "Currently terminating of category %s among WorkerNodeIds: %d %s", + category, currentlyTerminating.size(), currentlyTerminating ); if (currentlyTerminating.isEmpty()) { - final int maxWorkersToTerminate = maxWorkersToTerminate(zkWorkers, workerConfig); - log.info("Max workers to terminate: %d", maxWorkersToTerminate); + final int maxWorkersToTerminate = maxWorkersToTerminate(zkWorkers, autoScaler); + log.info("Max workers to terminate of category %s: %d", category, maxWorkersToTerminate); final Predicate isLazyWorker = ProvisioningUtil.createLazyWorkerPredicate(config); final Collection laziestWorkerIps = Collections2.transform( runner.markWorkersLazy(isLazyWorker, maxWorkersToTerminate), - new Function() - { - @Override - public String apply(Worker zkWorker) - { - return zkWorker.getIp(); - } - } + Worker::getIp ); - log.info("Laziest worker ips: %d %s", laziestWorkerIps.size(), laziestWorkerIps); + log.info("Laziest worker ips of category %s: %d %s", category, laziestWorkerIps.size(), laziestWorkerIps); if (laziestWorkerIps.isEmpty()) { - log.debug("Found no lazy workers"); + log.debug("Found no lazy workers for category %s", category); } else { log.info( - "Terminating %,d lazy workers: %s", + "Terminating %,d lazy workers of category %s: %s", laziestWorkerIps.size(), + category, Joiner.on(", ").join(laziestWorkerIps) ); - final AutoScalingData terminated = workerConfig.getAutoScaler() - .terminate(ImmutableList.copyOf(laziestWorkerIps)); + final AutoScalingData terminated = autoScaler.terminate(ImmutableList.copyOf(laziestWorkerIps)); if (terminated != null) { - log.info("Terminated: %d %s", terminated.getNodeIds().size(), terminated.getNodeIds()); + log.info( + "Terminated of category %s: %d %s", + category, + terminated.getNodeIds().size(), + terminated.getNodeIds() + ); currentlyTerminating.addAll(terminated.getNodeIds()); - lastTerminateTime = DateTimes.nowUtc(); + lastTerminateTimeMap.put(category, DateTimes.nowUtc()); scalingStats.addTerminateEvent(terminated); didTerminate = true; } } } else { + DateTime lastTerminateTime = lastTerminateTimeMap.getOrDefault(category, DateTimes.nowUtc()); Duration durSinceLastTerminate = new Duration(lastTerminateTime, DateTimes.nowUtc()); - log.info("%s terminating. Current wait time: %s", currentlyTerminating, durSinceLastTerminate); + log.info( + "%s terminating of category %s. Current wait time: %s", + currentlyTerminating, + category, + durSinceLastTerminate + ); if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { log.makeAlert("Worker node termination taking too long!") @@ -408,6 +521,38 @@ public String apply(Worker zkWorker) return didTerminate; } + private boolean initAutoscalers(CategoriedWorkerBehaviorConfig workerConfig) + { + boolean didProvision = false; + for (AutoScaler autoScaler : workerConfig.getAutoScalers()) { + String category = ProvisioningUtil.getAutoscalerCategory(autoScaler); + didProvision = initAutoscaler(autoScaler, category, workerConfig, currentlyProvisioningMap) || didProvision; + } + return didProvision; + } + + private boolean initAutoscaler( + AutoScaler autoScaler, + String category, + CategoriedWorkerBehaviorConfig workerConfig, + Map> currentlyProvisioningMap + ) + { + currentlyProvisioningMap.putIfAbsent( + category, + new HashSet<>() + ); + Set currentlyProvisioning = currentlyProvisioningMap.get(category); + return doProvision( + category, + Collections.emptyList(), + Collections.emptyList(), + workerConfig, + currentlyProvisioning, + autoScaler + ); + } + @Override public ScalingStats getStats() { @@ -415,11 +560,11 @@ public ScalingStats getStats() } } - private int maxWorkersToTerminate(Collection zkWorkers, DefaultWorkerBehaviorConfig workerConfig) + private int maxWorkersToTerminate(Collection zkWorkers, AutoScaler autoScaler) { final int currValidWorkers = getCurrValidWorkers(zkWorkers); final int invalidWorkers = zkWorkers.size() - currValidWorkers; - final int minWorkers = workerConfig.getAutoScaler().getMinNumWorkers(); + final int minWorkers = autoScaler.getMinNumWorkers(); log.info("Min workers: %d", minWorkers); // Max workers that can be terminated diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/ProvisioningUtil.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/ProvisioningUtil.java index 6eb7e05305cf..a8d0412ab2c5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/ProvisioningUtil.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/ProvisioningUtil.java @@ -20,11 +20,30 @@ package org.apache.druid.indexing.overlord.autoscaling; import com.google.common.base.Predicate; +import com.google.common.base.Supplier; import org.apache.druid.indexing.overlord.ImmutableWorkerInfo; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerSelectStrategy; +import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerCategorySpec; +import org.apache.druid.indexing.overlord.setup.WorkerSelectStrategy; +import org.apache.druid.indexing.worker.Worker; +import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.emitter.EmittingLogger; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; public class ProvisioningUtil { + private static final EmittingLogger log = new EmittingLogger(ProvisioningUtil.class); + public static Predicate createValidWorkerPredicate( final SimpleWorkerProvisioningConfig config ) @@ -61,4 +80,107 @@ public boolean apply(ImmutableWorkerInfo worker) }; } + @Nullable + public static CategoriedWorkerBehaviorConfig getCategoriedWorkerBehaviorConfig( + Supplier workerConfigRef, + String action + ) + { + final WorkerBehaviorConfig workerBehaviorConfig = workerConfigRef.get(); + if (workerBehaviorConfig == null) { + log.error("No workerConfig available, cannot %s workers.", action); + return null; + } + + CategoriedWorkerBehaviorConfig workerConfig; + if (workerBehaviorConfig instanceof DefaultWorkerBehaviorConfig) { + AutoScaler autoscaler = ((DefaultWorkerBehaviorConfig) workerBehaviorConfig).getAutoScaler(); + WorkerSelectStrategy workerSelectStrategy = workerBehaviorConfig.getSelectStrategy(); + List autoscalers = autoscaler == null + ? Collections.emptyList() + : Collections.singletonList(autoscaler); + workerConfig = new CategoriedWorkerBehaviorConfig(workerSelectStrategy, autoscalers); + } else if (workerBehaviorConfig instanceof CategoriedWorkerBehaviorConfig) { + workerConfig = (CategoriedWorkerBehaviorConfig) workerBehaviorConfig; + } else { + log.error( + "Only DefaultWorkerBehaviorConfig or CategoriedWorkerBehaviorConfig are supported as WorkerBehaviorConfig, [%s] given, cannot %s workers", + workerBehaviorConfig, + action + ); + return null; + } + if (workerConfig.getAutoScalers() == null || workerConfig.getAutoScalers().isEmpty()) { + log.error("No autoScaler available, cannot %s workers", action); + return null; + } + + return workerConfig; + } + + @Nullable + public static WorkerCategorySpec getWorkerCategorySpec(CategoriedWorkerBehaviorConfig workerConfig) + { + if (workerConfig != null && workerConfig.getSelectStrategy() != null) { + WorkerSelectStrategy selectStrategy = workerConfig.getSelectStrategy(); + if (selectStrategy instanceof CategoriedWorkerSelectStrategy) { + return ((CategoriedWorkerSelectStrategy) selectStrategy).getWorkerCategorySpec(); + } + } + return null; + } + + public static Map mapAutoscalerByCategory(List autoScalers) + { + Map result = autoScalers.stream().collect(Collectors.groupingBy( + ProvisioningUtil::getAutoscalerCategory, + Collectors.collectingAndThen(Collectors.toList(), values -> values.get(0)) + )); + + if (result.size() != autoScalers.size()) { + log.warn( + "Probably autoscalers with duplicated categories were defined. The first instance of each duplicate category will be used."); + } + + return result; + } + + @Nullable + public static AutoScaler getAutoscalerByCategory(String category, Map autoscalersByCategory) + { + AutoScaler autoScaler = autoscalersByCategory.get(category); + boolean isStrongAssignment = !autoscalersByCategory.containsKey(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY); + log.debug("Found autoscaler %s for category %s in available categories %s. Is strong assignment=%b. ", autoScaler, category, autoscalersByCategory, isStrongAssignment); + if (autoScaler == null && isStrongAssignment) { + log.warn( + "No autoscaler found for category %s. Tasks of this category will not be assigned to default autoscaler because of strong affinity.", + category + ); + return null; + } + return autoScaler == null + ? autoscalersByCategory.get(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY) + : autoScaler; + } + + public static Collection getWorkersOfCategory(Collection workers, String category) + { + return workers.stream().filter(worker -> category.equals(worker.getCategory())).collect(Collectors.toList()); + } + + public static String getAutoscalerCategory(AutoScaler autoScaler) + { + return autoScaler.getCategory() == null + ? CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY + : autoScaler.getCategory(); + } + + public static Map> getWorkersByCategories(Collection workers) + { + return workers.stream().collect(Collectors.groupingBy( + immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory() == null + ? WorkerConfig.DEFAULT_CATEGORY + : immutableWorkerInfo.getWorker().getCategory()) + ); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/SimpleWorkerProvisioningStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/SimpleWorkerProvisioningStrategy.java index a17014c29c56..bd8af5be4dc4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/SimpleWorkerProvisioningStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/SimpleWorkerProvisioningStrategy.java @@ -29,11 +29,14 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.inject.Inject; +import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.ImmutableWorkerInfo; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; import org.apache.druid.indexing.overlord.WorkerTaskRunner; -import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerCategorySpec; +import org.apache.druid.indexing.overlord.setup.WorkerSelectUtils; import org.apache.druid.indexing.worker.Worker; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; @@ -42,16 +45,21 @@ import org.joda.time.Duration; import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; /** + * */ - public class SimpleWorkerProvisioningStrategy extends AbstractWorkerProvisioningStrategy { + public static final Integer TARGET_WORKER_DEFAULT = -1; private static final EmittingLogger log = new EmittingLogger(SimpleWorkerProvisioningStrategy.class); private final SimpleWorkerProvisioningConfig config; @@ -68,14 +76,7 @@ public SimpleWorkerProvisioningStrategy( config, workerConfigRef, provisioningSchedulerConfig, - new Supplier() - { - @Override - public ScheduledExecutorService get() - { - return ScheduledExecutors.fixed(1, "SimpleResourceManagement-manager--%d"); - } - } + () -> ScheduledExecutors.fixed(1, "SimpleResourceManagement-manager--%d") ); } @@ -102,62 +103,162 @@ private class SimpleProvisioner implements Provisioner private final WorkerTaskRunner runner; private final ScalingStats scalingStats = new ScalingStats(config.getNumEventsToTrack()); - private final Set currentlyProvisioning = new HashSet<>(); - private final Set currentlyTerminating = new HashSet<>(); + private final Map> currentlyProvisioningMap = new HashMap<>(); + private final Map> currentlyTerminatingMap = new HashMap<>(); - private int targetWorkerCount = -1; - private DateTime lastProvisionTime = DateTimes.nowUtc(); - private DateTime lastTerminateTime = lastProvisionTime; + private final Map targetWorkerCountMap = new HashMap<>(); + private final Map lastProvisionTimeMap = new HashMap<>(); + private final Map lastTerminateTimeMap = new HashMap<>(); SimpleProvisioner(WorkerTaskRunner runner) { this.runner = runner; } + private Map> groupTasksByCategories( + Collection pendingTasks, + WorkerTaskRunner runner, + WorkerCategorySpec workerCategorySpec + ) + { + Collection pendingTasksPayload = runner.getPendingTaskPayloads(); + Map> taskPayloadsById = pendingTasksPayload.stream() + .collect(Collectors.groupingBy(Task::getId)); + + return pendingTasks.stream().collect(Collectors.groupingBy(task -> { + List taskPayloads = taskPayloadsById.get(task.getTaskId()); + if (taskPayloads == null || taskPayloads.isEmpty()) { + return CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY; + } + return WorkerSelectUtils.getTaskCategory( + taskPayloads.get(0), + workerCategorySpec, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY + ); + })); + } + @Override public synchronized boolean doProvision() { Collection pendingTasks = runner.getPendingTasks(); + log.debug("Pending tasks: %d %s", pendingTasks.size(), pendingTasks); Collection workers = runner.getWorkers(); + log.debug("Workers: %d %s", workers.size(), workers); boolean didProvision = false; - final DefaultWorkerBehaviorConfig workerConfig = - PendingTaskBasedWorkerProvisioningStrategy.getDefaultWorkerBehaviorConfig(workerConfigRef, "provision", log); + final CategoriedWorkerBehaviorConfig workerConfig = ProvisioningUtil.getCategoriedWorkerBehaviorConfig( + workerConfigRef, + "provision" + ); if (workerConfig == null) { + log.info("No worker config found. Skip provisioning."); return false; } + WorkerCategorySpec workerCategorySpec = ProvisioningUtil.getWorkerCategorySpec(workerConfig); + + // Group tasks by categories + Map> tasksByCategories = groupTasksByCategories( + pendingTasks, + runner, + workerCategorySpec + ); + + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(workers); + + // Merge categories of tasks and workers + Set allCategories = new HashSet<>(tasksByCategories.keySet()); + allCategories.addAll(workersByCategories.keySet()); + + log.debug( + "Pending Tasks of %d categories (%s), Workers of %d categories (%s). %d common categories: %s", + tasksByCategories.size(), + tasksByCategories.keySet(), + workersByCategories.size(), + workersByCategories.keySet(), + allCategories.size(), + allCategories + ); + + if (allCategories.isEmpty()) { + // Likely empty categories means initialization. + // Just try to spinup required amount of workers of each non empty autoscalers + return initAutoscalers(workerConfig); + } + + Map autoscalersByCategory = ProvisioningUtil.mapAutoscalerByCategory(workerConfig.getAutoScalers()); + + for (String category : allCategories) { + List categoryTasks = tasksByCategories.getOrDefault( + category, + Collections.emptyList() + ); + AutoScaler categoryAutoscaler = ProvisioningUtil.getAutoscalerByCategory(category, autoscalersByCategory); + + if (categoryAutoscaler == null) { + log.error("No autoScaler available, cannot execute doProvision for workers of category %s", category); + continue; + } + // Correct category name by selected autoscaler + category = ProvisioningUtil.getAutoscalerCategory(categoryAutoscaler); + + List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); + currentlyProvisioningMap.putIfAbsent(category, new HashSet<>()); + Set currentlyProvisioning = this.currentlyProvisioningMap.get(category); + currentlyTerminatingMap.putIfAbsent(category, new HashSet<>()); + Set currentlyTerminating = this.currentlyTerminatingMap.get(category); + + didProvision = doProvision( + category, + categoryWorkers, + categoryTasks, + currentlyProvisioning, + currentlyTerminating, + categoryAutoscaler + ) || didProvision; + } + + return didProvision; + } + + private boolean doProvision( + String category, + Collection workers, + Collection pendingTasks, + Set currentlyProvisioning, + Set currentlyTerminating, + AutoScaler autoScaler + ) + { + boolean didProvision = false; + final Predicate isValidWorker = ProvisioningUtil.createValidWorkerPredicate(config); final int currValidWorkers = Collections2.filter(workers, isValidWorker).size(); - final List workerNodeIds = workerConfig.getAutoScaler().ipToIdLookup( + final List workerNodeIds = autoScaler.ipToIdLookup( Lists.newArrayList( Iterables.transform( workers, - new Function() - { - @Override - public String apply(ImmutableWorkerInfo input) - { - return input.getWorker().getIp(); - } - } + input -> input.getWorker().getIp() ) ) ); currentlyProvisioning.removeAll(workerNodeIds); - updateTargetWorkerCount(workerConfig, pendingTasks, workers); + updateTargetWorkerCount(autoScaler, pendingTasks, workers, category, currentlyProvisioning, currentlyTerminating); + int targetWorkerCount = targetWorkerCountMap.getOrDefault(category, TARGET_WORKER_DEFAULT); int want = targetWorkerCount - (currValidWorkers + currentlyProvisioning.size()); + log.info("Want workers: %d", want); while (want > 0) { - final AutoScalingData provisioned = workerConfig.getAutoScaler().provision(); + final AutoScalingData provisioned = autoScaler.provision(); final List newNodes; if (provisioned == null || (newNodes = provisioned.getNodeIds()).isEmpty()) { log.warn("NewNodes is empty, returning from provision loop"); break; } else { currentlyProvisioning.addAll(newNodes); - lastProvisionTime = DateTimes.nowUtc(); + lastProvisionTimeMap.put(category, DateTimes.nowUtc()); scalingStats.addProvisionEvent(provisioned); want -= provisioned.getNodeIds().size(); didProvision = true; @@ -165,6 +266,7 @@ public String apply(ImmutableWorkerInfo input) } if (!currentlyProvisioning.isEmpty()) { + DateTime lastProvisionTime = lastProvisionTimeMap.getOrDefault(category, DateTimes.nowUtc()); Duration durSinceLastProvision = new Duration(lastProvisionTime, DateTimes.nowUtc()); log.info("%s provisioning. Current wait time: %s", currentlyProvisioning, durSinceLastProvision); if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { @@ -173,7 +275,7 @@ public String apply(ImmutableWorkerInfo input) .addData("provisioningCount", currentlyProvisioning.size()) .emit(); - workerConfig.getAutoScaler().terminateWithIds(Lists.newArrayList(currentlyProvisioning)); + autoScaler.terminateWithIds(Lists.newArrayList(currentlyProvisioning)); currentlyProvisioning.clear(); } } @@ -184,27 +286,90 @@ public String apply(ImmutableWorkerInfo input) @Override public synchronized boolean doTerminate() { + Collection workers = runner.getWorkers(); Collection pendingTasks = runner.getPendingTasks(); - final DefaultWorkerBehaviorConfig workerConfig = - PendingTaskBasedWorkerProvisioningStrategy.getDefaultWorkerBehaviorConfig(workerConfigRef, "terminate", log); + final CategoriedWorkerBehaviorConfig workerConfig = ProvisioningUtil.getCategoriedWorkerBehaviorConfig( + workerConfigRef, + "terminate" + ); if (workerConfig == null) { + log.info("No worker config found. Skip terminating."); return false; } boolean didTerminate = false; + + WorkerCategorySpec workerCategorySpec = ProvisioningUtil.getWorkerCategorySpec(workerConfig); + + // Group tasks by categories + Map> pendingTasksByCategories = groupTasksByCategories( + pendingTasks, + runner, + workerCategorySpec + ); + + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(workers); + + Set allCategories = workersByCategories.keySet(); + log.debug( + "Workers of %d categories: %s", + workersByCategories.size(), + allCategories + ); + + Map autoscalersByCategory = ProvisioningUtil.mapAutoscalerByCategory(workerConfig.getAutoScalers()); + + for (String category : allCategories) { + AutoScaler categoryAutoscaler = ProvisioningUtil.getAutoscalerByCategory(category, autoscalersByCategory); + + if (categoryAutoscaler == null) { + log.error("No autoScaler available, cannot execute doTerminate for workers of category %s", category); + continue; + } + + // Correct category name by selected autoscaler + category = ProvisioningUtil.getAutoscalerCategory(categoryAutoscaler); + List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); + currentlyProvisioningMap.putIfAbsent(category, new HashSet<>()); + Set currentlyProvisioning = this.currentlyProvisioningMap.get(category); + currentlyTerminatingMap.putIfAbsent(category, new HashSet<>()); + Set currentlyTerminating = this.currentlyTerminatingMap.get(category); + List categoryPendingTasks = pendingTasksByCategories.getOrDefault( + category, + Collections.emptyList() + ); + + didTerminate = doTerminate( + category, + categoryWorkers, + currentlyProvisioning, + currentlyTerminating, + categoryAutoscaler, + categoryPendingTasks + ) || didTerminate; + } + + return didTerminate; + } + + private boolean doTerminate( + String category, + Collection workers, + Set currentlyProvisioning, + Set currentlyTerminating, + AutoScaler autoScaler, + List pendingTasks + ) + { + boolean didTerminate = false; + + Collection lazyWorkers = ProvisioningUtil.getWorkersOfCategory(runner.getLazyWorkers(), category); final Set workerNodeIds = Sets.newHashSet( - workerConfig.getAutoScaler().ipToIdLookup( + autoScaler.ipToIdLookup( Lists.newArrayList( Iterables.transform( - runner.getLazyWorkers(), - new Function() - { - @Override - public String apply(Worker input) - { - return input.getIp(); - } - } + lazyWorkers, + Worker::getIp ) ) ) @@ -212,8 +377,8 @@ public String apply(Worker input) currentlyTerminating.retainAll(workerNodeIds); - Collection workers = runner.getWorkers(); - updateTargetWorkerCount(workerConfig, pendingTasks, workers); + updateTargetWorkerCount(autoScaler, pendingTasks, workers, category, currentlyProvisioning, currentlyTerminating); + int targetWorkerCount = targetWorkerCountMap.getOrDefault(category, TARGET_WORKER_DEFAULT); if (currentlyTerminating.isEmpty()) { @@ -242,17 +407,17 @@ public String apply(Worker worker) Joiner.on(", ").join(laziestWorkerIps) ); - final AutoScalingData terminated = workerConfig.getAutoScaler() - .terminate(ImmutableList.copyOf(laziestWorkerIps)); + final AutoScalingData terminated = autoScaler.terminate(ImmutableList.copyOf(laziestWorkerIps)); if (terminated != null) { currentlyTerminating.addAll(terminated.getNodeIds()); - lastTerminateTime = DateTimes.nowUtc(); + lastTerminateTimeMap.put(category, DateTimes.nowUtc()); scalingStats.addTerminateEvent(terminated); didTerminate = true; } } } } else { + DateTime lastTerminateTime = lastTerminateTimeMap.getOrDefault(category, DateTimes.nowUtc()); Duration durSinceLastTerminate = new Duration(lastTerminateTime, DateTimes.nowUtc()); log.info("%s terminating. Current wait time: %s", currentlyTerminating, durSinceLastTerminate); @@ -272,9 +437,12 @@ public String apply(Worker worker) private void updateTargetWorkerCount( - final DefaultWorkerBehaviorConfig workerConfig, + final AutoScaler autoScaler, final Collection pendingTasks, - final Collection zkWorkers + final Collection zkWorkers, + String category, + Set currentlyProvisioning, + Set currentlyTerminating ) { final Collection validWorkers = Collections2.filter( @@ -282,14 +450,16 @@ private void updateTargetWorkerCount( ProvisioningUtil.createValidWorkerPredicate(config) ); final Predicate isLazyWorker = ProvisioningUtil.createLazyWorkerPredicate(config); - final int minWorkerCount = workerConfig.getAutoScaler().getMinNumWorkers(); - final int maxWorkerCount = workerConfig.getAutoScaler().getMaxNumWorkers(); + final int minWorkerCount = autoScaler.getMinNumWorkers(); + final int maxWorkerCount = autoScaler.getMaxNumWorkers(); if (minWorkerCount > maxWorkerCount) { log.error("Huh? minWorkerCount[%d] > maxWorkerCount[%d]. I give up!", minWorkerCount, maxWorkerCount); return; } + int targetWorkerCount = targetWorkerCountMap.getOrDefault(category, TARGET_WORKER_DEFAULT); + if (targetWorkerCount < 0) { // Initialize to size of current worker pool, subject to pool size limits targetWorkerCount = Math.max( @@ -346,6 +516,8 @@ private void updateTargetWorkerCount( maxWorkerCount ); } + + targetWorkerCountMap.put(category, targetWorkerCount); } private boolean hasTaskPendingBeyondThreshold(Collection pendingTasks) @@ -366,6 +538,31 @@ public ScalingStats getStats() { return scalingStats; } - } + private boolean initAutoscalers(CategoriedWorkerBehaviorConfig workerConfig) + { + boolean didProvision = false; + for (AutoScaler autoScaler : workerConfig.getAutoScalers()) { + String category = ProvisioningUtil.getAutoscalerCategory(autoScaler); + didProvision = initAutoscaler(autoScaler, category) || didProvision; + } + return didProvision; + } + + private boolean initAutoscaler(AutoScaler autoScaler, String category) + { + currentlyProvisioningMap.putIfAbsent(category, new HashSet<>()); + Set currentlyProvisioning = this.currentlyProvisioningMap.get(category); + currentlyTerminatingMap.putIfAbsent(category, new HashSet<>()); + Set currentlyTerminating = this.currentlyTerminatingMap.get(category); + return doProvision( + category, + Collections.emptyList(), + Collections.emptyList(), + currentlyProvisioning, + currentlyTerminating, + autoScaler + ); + } + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/CategoriedWorkerBehaviorConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/CategoriedWorkerBehaviorConfig.java new file mode 100644 index 000000000000..fe98e570c773 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/CategoriedWorkerBehaviorConfig.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.overlord.setup; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.overlord.autoscaling.AutoScaler; +import org.apache.druid.indexing.worker.config.WorkerConfig; + +import java.util.List; +import java.util.Objects; + +public class CategoriedWorkerBehaviorConfig implements WorkerBehaviorConfig +{ + // Use the same category constant as for worker category to match default workers and autoscalers + public static final String DEFAULT_AUTOSCALER_CATEGORY = WorkerConfig.DEFAULT_CATEGORY; + + private final WorkerSelectStrategy selectStrategy; + private final List autoScalers; + + @JsonCreator + public CategoriedWorkerBehaviorConfig( + @JsonProperty("selectStrategy") WorkerSelectStrategy selectStrategy, + @JsonProperty("autoScalers") List autoScalers + ) + { + this.selectStrategy = selectStrategy; + this.autoScalers = autoScalers; + } + + @Override + @JsonProperty + public WorkerSelectStrategy getSelectStrategy() + { + return selectStrategy; + } + + @JsonProperty + public List getAutoScalers() + { + return autoScalers; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CategoriedWorkerBehaviorConfig that = (CategoriedWorkerBehaviorConfig) o; + return Objects.equals(selectStrategy, that.selectStrategy) && + Objects.equals(autoScalers, that.autoScalers); + } + + @Override + public int hashCode() + { + return Objects.hash(selectStrategy, autoScalers); + } + + @Override + public String toString() + { + return "WorkerConfiguration{" + + "selectStrategy=" + selectStrategy + + ", autoScalers=" + autoScalers + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/CategoriedWorkerSelectStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/CategoriedWorkerSelectStrategy.java new file mode 100644 index 000000000000..baeb28922249 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/CategoriedWorkerSelectStrategy.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.overlord.setup; + +public interface CategoriedWorkerSelectStrategy extends WorkerSelectStrategy +{ + WorkerCategorySpec getWorkerCategorySpec(); +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/DefaultWorkerBehaviorConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/DefaultWorkerBehaviorConfig.java index 99bf90ff18f9..faae4e84eca0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/DefaultWorkerBehaviorConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/DefaultWorkerBehaviorConfig.java @@ -28,7 +28,7 @@ */ public class DefaultWorkerBehaviorConfig implements WorkerBehaviorConfig { - private static final AutoScaler DEFAULT_AUTOSCALER = new NoopAutoScaler(); + private static final AutoScaler DEFAULT_AUTOSCALER = new NoopAutoScaler(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY); public static DefaultWorkerBehaviorConfig defaultConfig() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWithCategorySpecWorkerSelectStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWithCategorySpecWorkerSelectStrategy.java index ec65693ac3dd..05c32de11079 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWithCategorySpecWorkerSelectStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/EqualDistributionWithCategorySpecWorkerSelectStrategy.java @@ -29,7 +29,7 @@ import javax.annotation.Nullable; import java.util.Objects; -public class EqualDistributionWithCategorySpecWorkerSelectStrategy implements WorkerSelectStrategy +public class EqualDistributionWithCategorySpecWorkerSelectStrategy implements CategoriedWorkerSelectStrategy { private final WorkerCategorySpec workerCategorySpec; @@ -42,6 +42,7 @@ public EqualDistributionWithCategorySpecWorkerSelectStrategy( } @JsonProperty + @Override public WorkerCategorySpec getWorkerCategorySpec() { return workerCategorySpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/FillCapacityWithCategorySpecWorkerSelectStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/FillCapacityWithCategorySpecWorkerSelectStrategy.java index 3dcdfe9a5401..71006c906dea 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/FillCapacityWithCategorySpecWorkerSelectStrategy.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/FillCapacityWithCategorySpecWorkerSelectStrategy.java @@ -29,7 +29,7 @@ import javax.annotation.Nullable; import java.util.Objects; -public class FillCapacityWithCategorySpecWorkerSelectStrategy implements WorkerSelectStrategy +public class FillCapacityWithCategorySpecWorkerSelectStrategy implements CategoriedWorkerSelectStrategy { private final WorkerCategorySpec workerCategorySpec; @@ -42,6 +42,7 @@ public FillCapacityWithCategorySpecWorkerSelectStrategy( } @JsonProperty + @Override public WorkerCategorySpec getWorkerCategorySpec() { return workerCategorySpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerBehaviorConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerBehaviorConfig.java index ffc30f55e648..65461ca08a27 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerBehaviorConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerBehaviorConfig.java @@ -34,7 +34,8 @@ */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultWorkerBehaviorConfig.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "default", value = DefaultWorkerBehaviorConfig.class) + @JsonSubTypes.Type(name = "default", value = DefaultWorkerBehaviorConfig.class), + @JsonSubTypes.Type(name = "categoried", value = CategoriedWorkerBehaviorConfig.class) }) @ExtensionPoint public interface WorkerBehaviorConfig diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerSelectUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerSelectUtils.java index 24721e85d068..a1fa14ee4840 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerSelectUtils.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerSelectUtils.java @@ -57,7 +57,11 @@ public static ImmutableWorkerInfo selectWorker( final Function, ImmutableWorkerInfo> workerSelector ) { - final Map runnableWorkers = getRunnableWorkers(task, allWorkers, workerTaskRunnerConfig); + final Map runnableWorkers = getRunnableWorkers( + task, + allWorkers, + workerTaskRunnerConfig + ); if (affinityConfig == null) { // All runnable workers are valid. @@ -92,10 +96,10 @@ public static ImmutableWorkerInfo selectWorker( /** * Helper for {@link WorkerSelectStrategy} implementations. * - * @param allWorkers map of all workers, in the style provided to {@link WorkerSelectStrategy} + * @param allWorkers map of all workers, in the style provided to {@link WorkerSelectStrategy} * @param workerCategorySpec worker category spec, or null - * @param workerSelector function that receives a list of eligible workers: version is high enough, worker can run - * the task, and worker satisfies the worker category spec. may return null. + * @param workerSelector function that receives a list of eligible workers: version is high enough, worker can run + * the task, and worker satisfies the worker category spec. may return null. * * @return selected worker from "allWorkers", or null. */ @@ -108,8 +112,39 @@ public static ImmutableWorkerInfo selectWorker( final Function, ImmutableWorkerInfo> workerSelector ) { - final Map runnableWorkers = getRunnableWorkers(task, allWorkers, workerTaskRunnerConfig); + final Map runnableWorkers = getRunnableWorkers( + task, + allWorkers, + workerTaskRunnerConfig + ); + + // select worker according to worker category spec + if (workerCategorySpec != null) { + String preferredCategory = getTaskCategory(task, workerCategorySpec, null); + + if (preferredCategory != null) { + // select worker from preferred category + final ImmutableMap categoryWorkers = getCategoryWorkers( + preferredCategory, + runnableWorkers + ); + final ImmutableWorkerInfo selected = workerSelector.apply(categoryWorkers); + + if (selected != null) { + return selected; + } else if (workerCategorySpec.isStrong()) { + return null; + } + } + } + + // select worker from all runnable workers by default + return workerSelector.apply(ImmutableMap.copyOf(runnableWorkers)); + } + @Nullable + public static String getTaskCategory(Task task, WorkerCategorySpec workerCategorySpec, String defaultValue) + { // select worker according to worker category spec if (workerCategorySpec != null) { final WorkerCategorySpec.CategoryConfig categoryConfig = workerCategorySpec.getCategoryMap().get(task.getType()); @@ -119,26 +154,12 @@ public static ImmutableWorkerInfo selectWorker( final Map categoryAffinity = categoryConfig.getCategoryAffinity(); String preferredCategory = categoryAffinity.get(task.getDataSource()); - // If there is no preferred category for the datasource, then using the defaultCategory. However, the defaultCategory - // may be null too, so we need to do one more null check (see below). - preferredCategory = preferredCategory == null ? defaultCategory : preferredCategory; - - if (preferredCategory != null) { - // select worker from preferred category - final ImmutableMap categoryWorkers = getCategoryWorkers(preferredCategory, runnableWorkers); - final ImmutableWorkerInfo selected = workerSelector.apply(categoryWorkers); - - if (selected != null) { - return selected; - } else if (workerCategorySpec.isStrong()) { - return null; - } - } + // If there is no preferred category for the datasource, then using the defaultCategory. + return preferredCategory == null ? defaultCategory : preferredCategory; } } - // select worker from all runnable workers by default - return workerSelector.apply(ImmutableMap.copyOf(runnableWorkers)); + return defaultValue; } // Get workers that could potentially run this task, ignoring affinityConfig/workerCategorySpec. @@ -158,8 +179,8 @@ private static Map getRunnableWorkers( /** * Return workers belong to this category. * - * @param category worker category name - * @param workerMap map of worker hostname to worker info + * @param category worker category name + * @param workerMap map of worker hostname to worker info * * @return map of worker hostname to worker info */ @@ -169,7 +190,7 @@ private static ImmutableMap getCategoryWorkers( ) { return ImmutableMap.copyOf( - Maps.filterValues(workerMap, workerInfo -> workerInfo.getWorker().getCategory().equals(category)) + Maps.filterValues(workerMap, workerInfo -> category.equals(workerInfo.getWorker().getCategory())) ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java index 45207940668e..cf01aaeb18f3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.overlord.RemoteTaskRunnerWorkItem; import org.apache.druid.indexing.overlord.ZkWorker; import org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; import org.apache.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy; import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; @@ -59,9 +60,11 @@ import java.util.concurrent.atomic.AtomicReference; /** + * */ public class PendingTaskBasedProvisioningStrategyTest { + public static final String DEFAULT_CATEGORY = CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY; private AutoScaler autoScaler; private Task testTask; private PendingTaskBasedWorkerProvisioningStrategy strategy; @@ -109,6 +112,7 @@ public ScheduledExecutorService get() @Test public void testSuccessfulInitialMinWorkersProvision() { + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(3); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(5); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -140,6 +144,7 @@ public void testSuccessfulInitialMinWorkersProvision() @Test public void testSuccessfulMinWorkersProvision() { + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(3); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(5); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -174,6 +179,7 @@ public void testSuccessfulMinWorkersProvision() @Test public void testSuccessfulMinWorkersProvisionWithOldVersionNodeRunning() { + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(3); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(5); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -209,6 +215,7 @@ public void testSuccessfulMinWorkersProvisionWithOldVersionNodeRunning() @Test public void testSomethingProvisioning() { + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(8); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0).times(1); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2).times(1); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -266,6 +273,7 @@ public void testProvisionAlert() throws Exception EasyMock.expectLastCall(); EasyMock.replay(emitter); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(8); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0).times(1); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2).times(1); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -323,6 +331,7 @@ public void testProvisionAlert() throws Exception @Test public void testDoSuccessfulTerminate() { + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) .andReturn(new ArrayList()); @@ -367,6 +376,7 @@ public void testDoSuccessfulTerminate() @Test public void testSomethingTerminating() { + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(8); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0).times(1); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")).times(2); @@ -411,6 +421,7 @@ public void testSomethingTerminating() public void testNoActionNeeded() { EasyMock.reset(autoScaler); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")); @@ -442,6 +453,7 @@ public void testNoActionNeeded() EasyMock.verify(autoScaler); EasyMock.reset(autoScaler); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -460,6 +472,7 @@ public void testMinCountIncrease() { // Don't terminate anything EasyMock.reset(autoScaler); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")); @@ -487,6 +500,7 @@ public void testMinCountIncrease() // Don't provision anything EasyMock.reset(autoScaler); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -497,6 +511,7 @@ public void testMinCountIncrease() EasyMock.verify(autoScaler); EasyMock.reset(autoScaler); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); // Increase minNumWorkers EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(3); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(5); @@ -516,6 +531,36 @@ public void testMinCountIncrease() EasyMock.verify(runner); } + @Test + public void testMinCountIncreaseNoWorkers() + { + EasyMock.reset(autoScaler); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); + // Expect min number to be zero, but autoscaling should work for that case as well even there is no workers running + EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); + EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")); + EasyMock.expect(autoScaler.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ); + EasyMock.replay(autoScaler); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.singletonList(NoopTask.create())); + EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + Assert.assertEquals(1, provisioner.getStats().toList().size()); + Assert.assertSame(provisioner.getStats().toList().get(0).getEvent(), ScalingStats.EVENT.PROVISION); + EasyMock.verify(autoScaler, runner); + } + @Test public void testNullWorkerConfig() { @@ -578,7 +623,11 @@ public TestZkWorker( int capacity ) { - super(new Worker(scheme, host, ip, capacity, version, WorkerConfig.DEFAULT_CATEGORY), null, new DefaultObjectMapper()); + super( + new Worker(scheme, host, ip, capacity, version, WorkerConfig.DEFAULT_CATEGORY), + null, + new DefaultObjectMapper() + ); this.testTask = testTask; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTestExtended.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTestExtended.java new file mode 100644 index 000000000000..236371259c50 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTestExtended.java @@ -0,0 +1,1167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.overlord.autoscaling; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.guava.DSuppliers; +import org.apache.druid.indexing.common.TestTasks; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.RemoteTaskRunner; +import org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerCategorySpec; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceEventBuilder; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; + + +public class PendingTaskBasedProvisioningStrategyTestExtended +{ + private static final String DEFAULT_CATEGORY_1 = "default_category1"; + private static final String DEFAULT_CATEGORY_2 = "default_category2"; + private static final String CATEGORY_1 = "category1"; + private static final String CATEGORY_2 = "category2"; + private static final String TASK_TYPE_1 = "taskType1"; + private static final String TASK_TYPE_2 = "taskType2"; + private static final String TASK_TYPE_3 = "taskType3"; + private static final String DATA_SOURCE_1 = "ds1"; + private static final String DATA_SOURCE_2 = "ds2"; + private AutoScaler autoScalerDefault; + private AutoScaler autoScalerCategory1; + private AutoScaler autoScalerCategory2; + private final List autoScalers = new ArrayList<>(); + private Task testTask; + private PendingTaskBasedWorkerProvisioningConfig config; + private final ScheduledExecutorService executorService = Execs.scheduledSingleThreaded("test service"); + private static final String MIN_VERSION = "2014-01-00T00:01:00Z"; + + @Before + public void setup() + { + autoScalerDefault = EasyMock.createMock(AutoScaler.class); + + autoScalerCategory1 = EasyMock.createMock(AutoScaler.class); + autoScalerCategory2 = EasyMock.createMock(AutoScaler.class); + + autoScalers.clear(); + + testTask = TestTasks.immediateSuccess("task1"); + + config = new PendingTaskBasedWorkerProvisioningConfig() + .setMaxScalingDuration(new Period(1000)) + .setNumEventsToTrack(10) + .setPendingTaskTimeout(new Period(0)) + .setWorkerVersion(MIN_VERSION) + .setMaxScalingStep(2); + } + + @After + public void tearDown() + { + executorService.shutdownNow(); + } + + @Test + public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList(), 2, 1, 1, 1); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList(), 2, 1, 1, 1); + autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // No pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); + // No workers + EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); + // Expect this call two times because the both categorizied autoscalers will call it + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(2); + + // Expect to create 2 workers + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ).times(2); + + // Expect to create 4 workers + EasyMock.expect(autoScalerCategory2.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category2Node")) + ).times(4); + + EasyMock.replay(runner, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + + // In total expect provisioning of 2 + 4 = 6 workers + Assert.assertEquals(6, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testDefaultAutoscalerSuccessfulMinWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + // Not strong affinity autoscaling mode will use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 3, + 5, + Collections.emptyList() + ); + autoScalers.add(autoScalerDefault); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // No pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); + // 1 node already running, only provision 2 more. + EasyMock.expect(runner.getWorkers()).andReturn( + Collections.singletonList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable() + ) + ); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + + // Expect to create 2 workers + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ).times(2); + + EasyMock.replay(runner, autoScalerDefault); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // Two workers should be provisioned + Assert.assertEquals(2, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerDefault); + } + + @Test + public void testAnyAutoscalerDontSpawnMinWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // No pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); + + // 1 worker already running. That means no initialization is required. + EasyMock.expect(runner.getWorkers()).andReturn( + Collections.singletonList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable() + ) + ); + + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertFalse(provisionedSomething); + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + + EasyMock.verify(runner); + } + + @Test + public void testCategoriedAutoscalerSpawnedMinWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + CATEGORY_1, + DATA_SOURCE_1, + CATEGORY_2 + ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 5, 7, Collections.emptyList()); + autoScalers.add(autoScalerCategory1); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // One pending task + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Collections.singletonList(StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); + // No workers + EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + + // Expect to create 5 workers + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ).times(5); + + EasyMock.replay(runner, autoScalerCategory1); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // Five workers should be created + Assert.assertEquals(5, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerCategory1); + } + + @Test + public void testCategoriedAutoscalerSpawnedAdditionalWorker() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + CATEGORY_1, + DATA_SOURCE_1, + CATEGORY_2 + ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 3, Collections.emptyList()); + autoScalers.add(autoScalerCategory1); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // One pending task + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Collections.singletonList(StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); + // Min workers are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable() + ) + ); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + EasyMock.replay(runner, autoScalerCategory1); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // Expecting provisioning of one node for the pending task + Assert.assertEquals(1, provisioner.getStats().toList().size()); + Assert.assertSame(provisioner.getStats().toList().get(0).getEvent(), ScalingStats.EVENT.PROVISION); + + EasyMock.verify(runner, autoScalerCategory1); + } + + @Test + public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + CATEGORY_1, + DATA_SOURCE_1, + CATEGORY_2 + ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 3, Collections.emptyList()); + autoScalers.add(autoScalerCategory1); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // Two pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2) + )); + // Min workers are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable() + ) + ); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + EasyMock.replay(runner, autoScalerCategory1); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // Can only spawn one worker because of maximum limit + Assert.assertEquals(1, provisioner.getStats().toList().size()); + Assert.assertSame(provisioner.getStats().toList().get(0).getEvent(), ScalingStats.EVENT.PROVISION); + + EasyMock.verify(runner, autoScalerCategory1); + } + + @Test + public void testAllCategoriedAutoscalersStrongly() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_1, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ), + TASK_TYPE_2, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_2, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ) + ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 3, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 3, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // Four pending tasks: three have their categorized autoscalers and one for default autoscaler + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), + StrategyTestUtils.TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(TASK_TYPE_3, DATA_SOURCE_2) + )); + // Min workers number of the each category are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(2); + + // Expect to create 1 worker + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + // Expect to create 2 workers + EasyMock.expect(autoScalerCategory2.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category2Node")) + ).times(2); + + EasyMock.replay(runner, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // In total expect provisioning of 1 + 2 = 3 workers + Assert.assertEquals(3, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testAllCategoriedAutoscalersNotStrongMode() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_1, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ), + TASK_TYPE_2, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_2, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ) + ); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 3, + 5, + Collections.emptyList() + ); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 3, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 3, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // Four pending tasks: three have their categorized autoscalers and one for default autoscaler + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), + StrategyTestUtils.TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(TASK_TYPE_3, DATA_SOURCE_2) + )); + // Min workers of two categoriez are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(3); + + // Expect to create 3 workers + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ).times(3); + + // Expect to create 1 worker + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + // Expect to create 2 workers + EasyMock.expect(autoScalerCategory2.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category2Node")) + ).times(2); + + EasyMock.replay(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // In total expect provisioning of 3 + 1 + 2 = 6 workers + Assert.assertEquals(6, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testAllCategoriedAutoscalersAlert() throws InterruptedException + { + ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class); + EmittingLogger.registerEmitter(emitter); + emitter.emit(EasyMock.anyObject()); + EasyMock.expectLastCall().times(3); + EasyMock.replay(emitter); + + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_1, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ), + TASK_TYPE_2, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_2, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ) + ); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScalerDefault.getCategory()) + .andReturn(null) + .times(4); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.emptyList()).times(2); + EasyMock.expect(autoScalerDefault.terminateWithIds(EasyMock.anyObject())) + .andReturn(null); + + EasyMock.expect(autoScalerCategory1.getMinNumWorkers()).andReturn(1); + EasyMock.expect(autoScalerCategory1.getMaxNumWorkers()).andReturn(3); + EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(8); + EasyMock.expect(autoScalerCategory1.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.emptyList()).times(2); + EasyMock.expect(autoScalerCategory1.terminateWithIds(EasyMock.anyObject())) + .andReturn(null); + + EasyMock.expect(autoScalerCategory2.getMinNumWorkers()).andReturn(1); + EasyMock.expect(autoScalerCategory2.getMaxNumWorkers()).andReturn(3); + EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(8); + EasyMock.expect(autoScalerCategory2.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.emptyList()).times(2); + EasyMock.expect(autoScalerCategory2.terminateWithIds(EasyMock.anyObject())) + .andReturn(null); + + autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // Four pending tasks: three have their categorized autoscalers and one for default autoscaler + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), + StrategyTestUtils.TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(TASK_TYPE_3, DATA_SOURCE_2) + )).times(2); + // Min workers of two categoriez are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ).times(2); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(3); + + // Expect to create 3 workers + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ).times(3); + + // Expect to create 1 worker + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + // Expect to create 2 workers + EasyMock.expect(autoScalerCategory2.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category2Node")) + ).times(2); + + EasyMock.replay(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + Assert.assertEquals(6, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + DateTime createdTime = provisioner.getStats().toList().get(0).getTimestamp(); + + Thread.sleep(2000); + + provisionedSomething = provisioner.doProvision(); + + Assert.assertFalse(provisionedSomething); + Assert.assertEquals(6, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + DateTime anotherCreatedTime = provisioner.getStats().toList().get(0).getTimestamp(); + Assert.assertEquals(createdTime, anotherCreatedTime); + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2, emitter); + } + + @Test + public void testNullWorkerConfig() + { + AtomicReference workerConfig = new AtomicReference<>(null); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // One pending task + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Collections.singletonList(StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); + // Min workers of two categoriez are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.replay(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertFalse(provisionedSomething); + // No workers should be created + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testNullWorkerCategorySpecNotStrong() + { + WorkerCategorySpec workerCategorySpec = null; + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + StrategyTestUtils.setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 1, + 3, + Collections.emptyList() + ); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 2, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 4, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); + + // Expect to create 2 workers for 3 tasks because of maxLimit + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ).times(2); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // Three pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), + StrategyTestUtils.TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2) + )); + // Min workers of two categories and one default are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(3); + + EasyMock.replay(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // Two workers for default autoscaler should be created + Assert.assertEquals(2, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testNullWorkerCategorySpecStrong() + { + WorkerCategorySpec workerCategorySpec = null; + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 2, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 4, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + + // Three pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), + StrategyTestUtils.TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2) + )); + + // Min workers of two categories and one default are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(2); + + EasyMock.replay(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertFalse(provisionedSomething); + // No workers should be created because of strong affinity + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testDoSuccessfulTerminateForAllCategories() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + StrategyTestUtils.setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 1, + Collections.emptyList() + ); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); + + EasyMock.expect(autoScalerDefault.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.emptyList()) + ); + EasyMock.expect(autoScalerCategory1.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.emptyList()) + ); + EasyMock.expect(autoScalerCategory2.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.emptyList()) + ); + EasyMock.replay(autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).getWorker() + ) + ).times(3); + + EasyMock.expect(runner.getLazyWorkers()).andReturn(Collections.emptyList()).times(3); + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean terminatedSomething = provisioner.doTerminate(); + + Assert.assertTrue(terminatedSomething); + Assert.assertEquals(3, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.TERMINATE); + } + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testDoSuccessfulTerminateWithoutDefaultAutoscaler() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); + + EasyMock.expect(autoScalerCategory1.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.emptyList()) + ); + EasyMock.expect(autoScalerCategory2.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.emptyList()) + ); + EasyMock.replay(autoScalerCategory1, autoScalerCategory2); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).getWorker() + ) + ).times(2); + + EasyMock.expect(runner.getLazyWorkers()).andReturn(Collections.emptyList()).times(2); + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean terminatedSomething = provisioner.doTerminate(); + + Assert.assertTrue(terminatedSomething); + Assert.assertEquals(2, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.TERMINATE); + } + + EasyMock.verify(runner, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testSomethingTerminating() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(1); + EasyMock.expect(autoScalerDefault.getCategory()) + .andReturn(null) + .times(4); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")).times(2); + EasyMock.expect(autoScalerDefault.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.singletonList("ip")) + ); + + EasyMock.expect(autoScalerCategory1.getMinNumWorkers()).andReturn(1); + EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(8); + EasyMock.expect(autoScalerCategory1.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")).times(2); + EasyMock.expect(autoScalerCategory1.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.singletonList("ip")) + ); + + EasyMock.expect(autoScalerCategory2.getMinNumWorkers()).andReturn(1); + EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(8); + EasyMock.expect(autoScalerCategory2.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")).times(2); + EasyMock.expect(autoScalerCategory2.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.singletonList("ip")) + ); + + autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); + + EasyMock.replay(autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ).times(2); + + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).getWorker() + ) + ).times(3); + + EasyMock.expect(runner.getLazyWorkers()).andReturn(Collections.emptyList()).times(6); + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean terminatedSomething = provisioner.doTerminate(); + + Assert.assertTrue(terminatedSomething); + Assert.assertEquals(3, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.TERMINATE); + } + + terminatedSomething = provisioner.doTerminate(); + + Assert.assertFalse(terminatedSomething); + Assert.assertEquals(3, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.TERMINATE); + } + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testMinCountIncrease() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + // Don't terminate anything + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(0); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); + EasyMock.replay(autoScalerDefault); + autoScalers.add(autoScalerDefault); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( + Collections.emptyList() + ).times(2); + EasyMock.expect(runner.getWorkers()).andReturn( + Collections.singletonList( + new StrategyTestUtils.TestZkWorker( + NoopTask.create(), + "http", + "h1", + "i1", + MIN_VERSION, + 1, + DEFAULT_CATEGORY_1 + ).toImmutable() + ) + ).times(3); + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(2); + + EasyMock.expect(runner.getLazyWorkers()).andReturn(new ArrayList<>()); + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn(Collections.emptyList()); + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean terminatedSomething = provisioner.doTerminate(); + Assert.assertFalse(terminatedSomething); + EasyMock.verify(autoScalerDefault); + + // Don't provision anything + EasyMock.reset(autoScalerDefault); + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(0); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(2); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); + EasyMock.replay(autoScalerDefault); + + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertFalse(provisionedSomething); + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + EasyMock.verify(autoScalerDefault); + + EasyMock.reset(autoScalerDefault); + // Increase minNumWorkers and expect provisioning + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")); + // Should provision two new workers + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("h3")) + ).times(3); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); + EasyMock.replay(autoScalerDefault); + provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + EasyMock.verify(autoScalerDefault); + EasyMock.verify(runner); + } + + @Test + public void testMinCountIncreaseNoWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + // Expect min number to be zero, but autoscaling should work for that case as well even there is no workers running + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(0); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")); + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); + + EasyMock.replay(autoScalerDefault); + autoScalers.add(autoScalerDefault); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Collections.singletonList(StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); + EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + Assert.assertEquals(1, provisioner.getStats().toList().size()); + Assert.assertSame(provisioner.getStats().toList().get(0).getEvent(), ScalingStats.EVENT.PROVISION); + + EasyMock.verify(autoScalerDefault, runner); + } + + private PendingTaskBasedWorkerProvisioningStrategy createStrategy( + AtomicReference workerConfigRef + ) + { + return new PendingTaskBasedWorkerProvisioningStrategy( + config, + DSuppliers.of(workerConfigRef), + new ProvisioningSchedulerConfig(), + () -> executorService + ); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java index 7f9739ad55ff..7f721bb1c94c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java @@ -30,6 +30,7 @@ import org.apache.druid.indexing.overlord.RemoteTaskRunner; import org.apache.druid.indexing.overlord.RemoteTaskRunnerWorkItem; import org.apache.druid.indexing.overlord.ZkWorker; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; import org.apache.druid.indexing.worker.TaskAnnouncement; @@ -58,9 +59,11 @@ import java.util.concurrent.atomic.AtomicReference; /** + * */ public class SimpleProvisioningStrategyTest { + public static final String DEFAULT_CATEGORY = CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY; private AutoScaler autoScaler; private Task testTask; private SimpleWorkerProvisioningStrategy strategy; @@ -113,6 +116,7 @@ public void tearDown() @Test public void testSuccessfulProvision() { + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -127,6 +131,11 @@ public void testSuccessfulProvision() .withQueueInsertionTime(DateTimes.nowUtc()) ) ); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( + Collections.singletonList( + testTask + ) + ); EasyMock.expect(runner.getWorkers()).andReturn( Collections.singletonList( new TestZkWorker(testTask).toImmutable() @@ -151,6 +160,7 @@ public void testSuccessfulProvision() @Test public void testSomethingProvisioning() { + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(8); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0).times(2); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2).times(2); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -165,6 +175,11 @@ public void testSomethingProvisioning() .withQueueInsertionTime(DateTimes.nowUtc()) ) ).times(2); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( + Collections.singletonList( + testTask + ) + ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( Collections.singletonList( new TestZkWorker(testTask).toImmutable() @@ -207,6 +222,7 @@ public void testProvisionAlert() throws Exception EasyMock.expectLastCall().atLeastOnce(); EasyMock.replay(emitter); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(8); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0).times(2); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2).times(2); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -224,6 +240,11 @@ public void testProvisionAlert() throws Exception .withQueueInsertionTime(DateTimes.nowUtc()) ) ).times(2); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( + Collections.singletonList( + testTask + ) + ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( Collections.singletonList( new TestZkWorker(testTask).toImmutable() @@ -262,6 +283,7 @@ public void testProvisionAlert() throws Exception @Test public void testDoSuccessfulTerminate() { + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(1); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -277,6 +299,11 @@ public void testDoSuccessfulTerminate() .withQueueInsertionTime(DateTimes.nowUtc()) ) ).times(2); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( + Collections.singletonList( + testTask + ) + ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( Collections.singletonList( new TestZkWorker(testTask).toImmutable() @@ -302,6 +329,7 @@ public void testDoSuccessfulTerminate() @Test public void testSomethingTerminating() { + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(8); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0).times(2); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(1).times(2); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -318,6 +346,11 @@ public void testSomethingTerminating() .withQueueInsertionTime(DateTimes.nowUtc()) ) ).times(2); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( + Collections.singletonList( + testTask + ) + ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( Collections.singletonList( new TestZkWorker(testTask).toImmutable() @@ -353,6 +386,7 @@ public void testSomethingTerminating() public void testNoActionNeeded() { EasyMock.reset(autoScaler); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -366,6 +400,11 @@ public void testNoActionNeeded() .withQueueInsertionTime(DateTimes.nowUtc()) ) ).times(2); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( + Collections.singletonList( + testTask + ) + ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( Arrays.asList( new TestZkWorker(NoopTask.create()).toImmutable(), @@ -384,6 +423,7 @@ public void testNoActionNeeded() EasyMock.verify(autoScaler); EasyMock.reset(autoScaler); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -402,6 +442,7 @@ public void testMinCountIncrease() { // Don't terminate anything EasyMock.reset(autoScaler); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -411,6 +452,11 @@ public void testMinCountIncrease() EasyMock.expect(runner.getPendingTasks()).andReturn( Collections.emptyList() ).times(3); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( + Collections.singletonList( + testTask + ) + ).times(3); EasyMock.expect(runner.getWorkers()).andReturn( Collections.singletonList( new TestZkWorker(NoopTask.create(), "http", "h1", "i1", "0").toImmutable() @@ -428,6 +474,7 @@ public void testMinCountIncrease() // Don't provision anything EasyMock.reset(autoScaler); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(2); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -439,6 +486,7 @@ public void testMinCountIncrease() EasyMock.reset(autoScaler); // Increase minNumWorkers + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(3); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(5); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) @@ -457,6 +505,45 @@ public void testMinCountIncrease() EasyMock.verify(runner); } + @Test + public void testMinCountIncreaseNoWorkers() + { + EasyMock.reset(autoScaler); + EasyMock.expect(autoScaler.getCategory()).andReturn(DEFAULT_CATEGORY).times(4); + // Expect min number to be zero, but autoscaling should work for that case as well even there is no workers running + EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(0); + EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")); + EasyMock.expect(autoScaler.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ); + EasyMock.replay(autoScaler); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + EasyMock.expect(runner.getPendingTasks()).andReturn( + Collections.singletonList( + new RemoteTaskRunnerWorkItem(testTask.getId(), testTask.getType(), null, null, testTask.getDataSource()) + .withQueueInsertionTime(DateTimes.nowUtc()) + ) + ); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( + Collections.singletonList( + testTask + ) + ); + EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); + + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + Assert.assertEquals(1, provisioner.getStats().toList().size()); + Assert.assertSame(provisioner.getStats().toList().get(0).getEvent(), ScalingStats.EVENT.PROVISION); + EasyMock.verify(autoScaler, runner); + } + @Test public void testNullWorkerConfig() { @@ -474,7 +561,7 @@ public void testNullWorkerConfig() Collections.singletonList( new TestZkWorker(null).toImmutable() ) - ).times(1); + ).times(2); EasyMock.replay(runner); Provisioner provisioner = strategy.makeProvisioner(runner); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTestExtended.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTestExtended.java new file mode 100644 index 000000000000..cbbf158ca63c --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTestExtended.java @@ -0,0 +1,1220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.overlord.autoscaling; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.guava.DSuppliers; +import org.apache.druid.indexing.common.TestTasks; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.RemoteTaskRunner; +import org.apache.druid.indexing.overlord.RemoteTaskRunnerWorkItem; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerCategorySpec; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceEventBuilder; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; + +/** + * + */ +public class SimpleProvisioningStrategyTestExtended +{ + public static final String DEFAULT_CATEGORY = CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY; + private static final String DEFAULT_CATEGORY_1 = "default_category1"; + private static final String DEFAULT_CATEGORY_2 = "default_category2"; + private static final String CATEGORY_1 = "category1"; + private static final String CATEGORY_2 = "category2"; + private static final String TASK_TYPE_1 = "taskType1"; + private static final String TASK_TYPE_2 = "taskType2"; + private static final String TASK_TYPE_3 = "taskType3"; + private static final String DATA_SOURCE_1 = "ds1"; + private static final String DATA_SOURCE_2 = "ds2"; + private AutoScaler autoScalerDefault; + private AutoScaler autoScalerCategory1; + private AutoScaler autoScalerCategory2; + private final List autoScalers = new ArrayList<>(); + private Task testTask; + private final ScheduledExecutorService executorService = Execs.scheduledSingleThreaded("test service"); + private SimpleWorkerProvisioningConfig config; + + @Before + public void setUp() + { + autoScalerDefault = EasyMock.createMock(AutoScaler.class); + + autoScalerCategory1 = EasyMock.createMock(AutoScaler.class); + autoScalerCategory2 = EasyMock.createMock(AutoScaler.class); + + autoScalers.clear(); + + testTask = TestTasks.immediateSuccess("task1"); + + config = new SimpleWorkerProvisioningConfig() + .setWorkerIdleTimeout(new Period(0)) + .setMaxScalingDuration(new Period(1000)) + .setNumEventsToTrack(10) + .setPendingTaskTimeout(new Period(0)) + .setWorkerVersion(""); + } + + @After + public void tearDown() + { + executorService.shutdownNow(); + } + + @Test + public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList(), 2, 1, 1, 1); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList(), 2, 1, 1, 1); + autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // No pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); + EasyMock.expect(runner.getPendingTasks()).andReturn(Collections.emptyList()); + // No workers + EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); + + // Expect to create 2 workers + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ).times(2); + + // Expect to create 4 workers + EasyMock.expect(autoScalerCategory2.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category2Node")) + ).times(4); + + EasyMock.replay(runner, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + + // In total expect provisioning of 2 + 4 = 6 workers + Assert.assertEquals(6, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testDefaultAutoscalerSuccessfulMinWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + // Not strong affinity autoscaling mode will use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 3, + 5, + Collections.emptyList() + ); + autoScalers.add(autoScalerDefault); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // No pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); + EasyMock.expect(runner.getPendingTasks()).andReturn(Collections.emptyList()); + // 1 node already running, only provision 2 more. + EasyMock.expect(runner.getWorkers()).andReturn( + Collections.singletonList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable() + ) + ); + + // Expect to create 2 workers + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ).times(2); + + EasyMock.replay(runner, autoScalerDefault); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // Two workers should be provisioned + Assert.assertEquals(2, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerDefault); + } + + @Test + public void testAnyAutoscalerDontSpawnMinWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // No pending tasks + EasyMock.expect(runner.getPendingTasks()).andReturn(Collections.emptyList()); + + // 1 worker already running. That means no initialization is required. + EasyMock.expect(runner.getWorkers()).andReturn( + Collections.singletonList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable() + ) + ); + + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertFalse(provisionedSomething); + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + + EasyMock.verify(runner); + } + + @Test + public void testCategoriedAutoscalerSpawnedMinWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + CATEGORY_1, + DATA_SOURCE_1, + CATEGORY_2 + ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 5, 7, Collections.emptyList()); + autoScalers.add(autoScalerCategory1); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // One pending task + String taskId = "taskId"; + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Collections.singletonList(StrategyTestUtils.TestTask.create( + taskId, + TASK_TYPE_1, + DATA_SOURCE_2 + ))); + EasyMock.expect(runner.getPendingTasks()) + .andReturn(Collections.singletonList(new RemoteTaskRunnerWorkItem( + taskId, + TASK_TYPE_1, + null, + null, + DATA_SOURCE_2 + ))); + // No workers + EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); + + // Expect to create 5 workers + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ).times(5); + + EasyMock.replay(runner, autoScalerCategory1); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // Five workers should be created + Assert.assertEquals(5, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerCategory1); + } + + @Test + public void testCategoriedAutoscalerSpawnedAdditionalWorker() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + CATEGORY_1, + DATA_SOURCE_1, + CATEGORY_2 + ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 3, Collections.emptyList()); + autoScalers.add(autoScalerCategory1); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // One pending task + String taskId = "taskId"; + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Collections.singletonList(StrategyTestUtils.TestTask.create( + taskId, + TASK_TYPE_1, + DATA_SOURCE_2 + ))); + EasyMock.expect(runner.getPendingTasks()) + .andReturn(Collections.singletonList(new RemoteTaskRunnerWorkItem( + taskId, + TASK_TYPE_1, + null, + null, + DATA_SOURCE_2 + ))); + // Min workers are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable() + ) + ); + + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + EasyMock.replay(runner, autoScalerCategory1); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // Expecting provisioning of one node for the pending task + Assert.assertEquals(1, provisioner.getStats().toList().size()); + Assert.assertSame(provisioner.getStats().toList().get(0).getEvent(), ScalingStats.EVENT.PROVISION); + + EasyMock.verify(runner, autoScalerCategory1); + } + + @Test + public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + CATEGORY_1, + DATA_SOURCE_1, + CATEGORY_2 + ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 3, Collections.emptyList()); + autoScalers.add(autoScalerCategory1); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // Two pending tasks + String task1Id = "task1Id"; + String task2Id = "task2Id"; + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(task1Id, TASK_TYPE_1, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(task2Id, TASK_TYPE_1, DATA_SOURCE_2) + )); + EasyMock.expect(runner.getPendingTasks()) + .andReturn(Arrays.asList( + new RemoteTaskRunnerWorkItem(task1Id, TASK_TYPE_1, null, null, DATA_SOURCE_2), + new RemoteTaskRunnerWorkItem(task2Id, TASK_TYPE_1, null, null, DATA_SOURCE_2) + )); + // Min workers are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable() + ) + ); + + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + EasyMock.replay(runner, autoScalerCategory1); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // Can only spawn one worker because of maximum limit + Assert.assertEquals(1, provisioner.getStats().toList().size()); + Assert.assertSame(provisioner.getStats().toList().get(0).getEvent(), ScalingStats.EVENT.PROVISION); + + EasyMock.verify(runner, autoScalerCategory1); + } + + @Test + public void testAllCategoriedAutoscalersStrongly() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_1, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ), + TASK_TYPE_2, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_2, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ) + ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 3, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 3, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // Four pending tasks: three have their categorized autoscalers and one for default autoscaler + String task1Id = "task1Id"; + String task2Id = "task2Id"; + String task3Id = "task3Id"; + String task4Id = "task4Id"; + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(task1Id, TASK_TYPE_1, DATA_SOURCE_1), + StrategyTestUtils.TestTask.create(task2Id, TASK_TYPE_2, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(task3Id, TASK_TYPE_1, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(task4Id, TASK_TYPE_3, DATA_SOURCE_2) + )); + EasyMock.expect(runner.getPendingTasks()) + .andReturn(Arrays.asList( + new RemoteTaskRunnerWorkItem(task1Id, TASK_TYPE_1, null, null, DATA_SOURCE_1), + new RemoteTaskRunnerWorkItem(task2Id, TASK_TYPE_2, null, null, DATA_SOURCE_2), + new RemoteTaskRunnerWorkItem(task3Id, TASK_TYPE_1, null, null, DATA_SOURCE_2), + new RemoteTaskRunnerWorkItem(task4Id, TASK_TYPE_3, null, null, DATA_SOURCE_2) + )); + // Min workers number of the each category are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + // Expect to create 1 worker + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + // Expect to create 2 workers + EasyMock.expect(autoScalerCategory2.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category2Node")) + ); + + EasyMock.replay(runner, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // In total expect provisioning of 1 + (2 - 1) = 2 workers + Assert.assertEquals(2, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testAllCategoriedAutoscalersNotStrongMode() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_1, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ), + TASK_TYPE_2, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_2, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ) + ); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + StrategyTestUtils.setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 3, + 5, + Collections.emptyList() + ); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 3, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 3, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // Four pending tasks: three have their categorized autoscalers and one for default autoscaler + String task1Id = "task1Id"; + String task2Id = "task2Id"; + String task3Id = "task3Id"; + String task4Id = "task4Id"; + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(task1Id, TASK_TYPE_1, DATA_SOURCE_1), + StrategyTestUtils.TestTask.create(task2Id, TASK_TYPE_2, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(task3Id, TASK_TYPE_1, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(task4Id, TASK_TYPE_3, DATA_SOURCE_2) + )); + EasyMock.expect(runner.getPendingTasks()) + .andReturn(Arrays.asList( + new RemoteTaskRunnerWorkItem(task1Id, TASK_TYPE_1, null, null, DATA_SOURCE_1), + new RemoteTaskRunnerWorkItem(task2Id, TASK_TYPE_2, null, null, DATA_SOURCE_2), + new RemoteTaskRunnerWorkItem(task3Id, TASK_TYPE_1, null, null, DATA_SOURCE_2), + new RemoteTaskRunnerWorkItem(task4Id, TASK_TYPE_3, null, null, DATA_SOURCE_2) + )); + // Min workers of two categoriez are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + // Expect to create 3 workers + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ).times(3); + + // Expect to create 1 worker + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + // Expect to create 1 worker + EasyMock.expect(autoScalerCategory2.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category2Node")) + ); + + EasyMock.replay(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // In total expect provisioning of 3 + 1 + 1 = 5 workers + Assert.assertEquals(5, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testAllCategoriedAutoscalersAlert() throws InterruptedException + { + ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class); + EmittingLogger.registerEmitter(emitter); + emitter.emit(EasyMock.anyObject()); + EasyMock.expectLastCall().times(2); + EasyMock.replay(emitter); + + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( + false, + TASK_TYPE_1, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_1, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ), + TASK_TYPE_2, + new WorkerCategorySpec.CategoryConfig( + DEFAULT_CATEGORY_2, + ImmutableMap.of( + DATA_SOURCE_1, + CATEGORY_1, + DATA_SOURCE_2, + CATEGORY_2 + ) + ) + ); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3).times(2); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5).times(2); + EasyMock.expect(autoScalerDefault.getCategory()) + .andReturn(null) + .times(4); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.emptyList()).times(2); + + EasyMock.expect(autoScalerCategory1.getMinNumWorkers()).andReturn(1).times(2); + EasyMock.expect(autoScalerCategory1.getMaxNumWorkers()).andReturn(3).times(2); + EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(8); + EasyMock.expect(autoScalerCategory1.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.emptyList()).times(2); + EasyMock.expect(autoScalerCategory1.terminateWithIds(EasyMock.anyObject())) + .andReturn(null); + + EasyMock.expect(autoScalerCategory2.getMinNumWorkers()).andReturn(1).times(2); + EasyMock.expect(autoScalerCategory2.getMaxNumWorkers()).andReturn(3).times(2); + EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(8); + EasyMock.expect(autoScalerCategory2.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.emptyList()).times(2); + EasyMock.expect(autoScalerCategory2.terminateWithIds(EasyMock.anyObject())) + .andReturn(null); + + autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // Four pending tasks: three have their categorized autoscalers and one for default autoscaler + String task1Id = "task1Id"; + String task2Id = "task2Id"; + String task3Id = "task3Id"; + String task4Id = "task4Id"; + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(task1Id, TASK_TYPE_1, DATA_SOURCE_1), + StrategyTestUtils.TestTask.create(task2Id, TASK_TYPE_2, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(task3Id, TASK_TYPE_1, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(task4Id, TASK_TYPE_3, DATA_SOURCE_2) + )).times(2); + EasyMock.expect(runner.getPendingTasks()) + .andReturn(Arrays.asList( + new RemoteTaskRunnerWorkItem(task1Id, TASK_TYPE_1, null, null, DATA_SOURCE_1), + new RemoteTaskRunnerWorkItem(task2Id, TASK_TYPE_2, null, null, DATA_SOURCE_2), + new RemoteTaskRunnerWorkItem(task3Id, TASK_TYPE_1, null, null, DATA_SOURCE_2), + new RemoteTaskRunnerWorkItem(task4Id, TASK_TYPE_3, null, null, DATA_SOURCE_2) + )).times(2); + // Min workers of two categoriez are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ).times(2); + + // Expect to create 5 workers + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ).times(5); + + // Expect to create 1 worker + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + // Expect to create 1 worker + EasyMock.expect(autoScalerCategory2.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category2Node")) + ); + + EasyMock.replay(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + Assert.assertEquals(5, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + DateTime createdTime = provisioner.getStats().toList().get(0).getTimestamp(); + + Thread.sleep(2000); + + provisionedSomething = provisioner.doProvision(); + + Assert.assertTrue(provisionedSomething); + Assert.assertEquals(7, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2, emitter); + } + + @Test + public void testNullWorkerConfig() + { + AtomicReference workerConfig = new AtomicReference<>(null); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // One pending task + String taskId = "taskId"; + EasyMock.expect(runner.getPendingTasks()) + .andReturn(Collections.singletonList( + new RemoteTaskRunnerWorkItem(taskId, TASK_TYPE_1, null, null, DATA_SOURCE_1) + )); + // Min workers of two categoriez are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.replay(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertFalse(provisionedSomething); + // No workers should be created + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testNullWorkerCategorySpecNotStrong() + { + WorkerCategorySpec workerCategorySpec = null; + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + StrategyTestUtils.setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 1, + 3, + Collections.emptyList() + ); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 2, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 4, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); + + // Expect to create 1 worker for 3 tasks because of maxLimit + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // Three pending tasks + String task1Id = "task1Id"; + String task2Id = "task2Id"; + String task3Id = "task3Id"; + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(task1Id, TASK_TYPE_1, DATA_SOURCE_1), + StrategyTestUtils.TestTask.create(task2Id, TASK_TYPE_2, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(task3Id, TASK_TYPE_1, DATA_SOURCE_2) + )); + EasyMock.expect(runner.getPendingTasks()) + .andReturn(Arrays.asList( + new RemoteTaskRunnerWorkItem(task1Id, TASK_TYPE_1, null, null, DATA_SOURCE_1), + new RemoteTaskRunnerWorkItem(task2Id, TASK_TYPE_2, null, null, DATA_SOURCE_2), + new RemoteTaskRunnerWorkItem(task3Id, TASK_TYPE_1, null, null, DATA_SOURCE_2) + )); + // Min workers of two categories and one default are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.replay(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + // Two workers for default autoscaler should be created + Assert.assertEquals(1, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testNullWorkerCategorySpecStrong() + { + WorkerCategorySpec workerCategorySpec = null; + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 2, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 4, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + + // Three pending tasks + String task1Id = "task1Id"; + String task2Id = "task2Id"; + String task3Id = "task3Id"; + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + StrategyTestUtils.TestTask.create(task1Id, TASK_TYPE_1, DATA_SOURCE_1), + StrategyTestUtils.TestTask.create(task2Id, TASK_TYPE_2, DATA_SOURCE_2), + StrategyTestUtils.TestTask.create(task3Id, TASK_TYPE_1, DATA_SOURCE_2) + )); + EasyMock.expect(runner.getPendingTasks()) + .andReturn(Arrays.asList( + new RemoteTaskRunnerWorkItem(task1Id, TASK_TYPE_1, null, null, DATA_SOURCE_1), + new RemoteTaskRunnerWorkItem(task2Id, TASK_TYPE_2, null, null, DATA_SOURCE_2), + new RemoteTaskRunnerWorkItem(task3Id, TASK_TYPE_1, null, null, DATA_SOURCE_2) + )); + + // Min workers of two categories and one default are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.replay(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertFalse(provisionedSomething); + // No workers should be created because of strong affinity + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testDoSuccessfulTerminateForAllCategories() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + StrategyTestUtils.setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 1, + 1, + Collections.emptyList() + ); + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 1, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 1, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); + + EasyMock.expect(autoScalerDefault.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.emptyList()) + ); + EasyMock.expect(autoScalerCategory1.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.emptyList()) + ); + EasyMock.expect(autoScalerCategory2.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.emptyList()) + ); + EasyMock.replay(autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); + EasyMock.expect(runner.getPendingTasks()).andReturn(Collections.emptyList()); + + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).getWorker() + ) + ).times(3); + + EasyMock.expect(runner.getLazyWorkers()).andReturn(Collections.emptyList()).times(3); + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean terminatedSomething = provisioner.doTerminate(); + + Assert.assertTrue(terminatedSomething); + Assert.assertEquals(3, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.TERMINATE); + } + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testDoSuccessfulTerminateWithoutDefaultAutoscaler() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + StrategyTestUtils.setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 1, Collections.emptyList()); + StrategyTestUtils.setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 1, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); + + EasyMock.expect(autoScalerCategory1.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.emptyList()) + ); + EasyMock.expect(autoScalerCategory2.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.emptyList()) + ); + EasyMock.replay(autoScalerCategory1, autoScalerCategory2); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); + EasyMock.expect(runner.getPendingTasks()).andReturn(Collections.emptyList()); + + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).getWorker(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).getWorker() + ) + ).times(2); + + EasyMock.expect(runner.getLazyWorkers()).andReturn(Collections.emptyList()).times(2); + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean terminatedSomething = provisioner.doTerminate(); + + Assert.assertTrue(terminatedSomething); + Assert.assertEquals(2, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.TERMINATE); + } + + EasyMock.verify(runner, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testSomethingTerminating() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(2); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScalerDefault.getCategory()) + .andReturn(null) + .times(2); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")).times(1); + + EasyMock.expect(autoScalerCategory1.getMinNumWorkers()).andReturn(2); + EasyMock.expect(autoScalerCategory1.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(4); + EasyMock.expect(autoScalerCategory1.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")).times(1); + + EasyMock.expect(autoScalerCategory2.getMinNumWorkers()).andReturn(2); + EasyMock.expect(autoScalerCategory2.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(4); + EasyMock.expect(autoScalerCategory2.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")).times(1); + + autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); + + EasyMock.replay(autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); + EasyMock.expect(runner.getPendingTasks()).andReturn(Collections.emptyList()); + + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.getLazyWorkers()).andReturn(Collections.emptyList()).times(3); + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean terminatedSomething = provisioner.doTerminate(); + + Assert.assertFalse(terminatedSomething); + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testMinCountIncrease() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + // Don't terminate anything + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(0); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); + EasyMock.replay(autoScalerDefault); + autoScalers.add(autoScalerDefault); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()).times(3); + EasyMock.expect(runner.getPendingTasks()).andReturn(Collections.emptyList()).times(3); + EasyMock.expect(runner.getWorkers()).andReturn( + Collections.singletonList( + new StrategyTestUtils.TestZkWorker( + NoopTask.create(), + "http", + "h1", + "i1", + StrategyTestUtils.MIN_VERSION, + 1, + DEFAULT_CATEGORY_1 + ).toImmutable() + ) + ).times(3); + + EasyMock.expect(runner.getLazyWorkers()).andReturn(new ArrayList<>()); + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean terminatedSomething = provisioner.doTerminate(); + Assert.assertFalse(terminatedSomething); + EasyMock.verify(autoScalerDefault); + + // Don't provision anything + EasyMock.reset(autoScalerDefault); + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(0); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(2); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); + EasyMock.replay(autoScalerDefault); + + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertFalse(provisionedSomething); + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + EasyMock.verify(autoScalerDefault); + + EasyMock.reset(autoScalerDefault); + // Increase minNumWorkers and expect provisioning + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")); + // Should provision two new workers + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("h3")) + ).times(3); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); + EasyMock.replay(autoScalerDefault); + provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + EasyMock.verify(autoScalerDefault); + EasyMock.verify(runner); + } + + @Test + public void testMinCountIncreaseNoWorkers() + { + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); + SimpleWorkerProvisioningStrategy strategy = createStrategy(workerConfig); + + // Expect min number to be zero, but autoscaling should work for that case as well even there is no workers running + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(0); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")); + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); + + EasyMock.replay(autoScalerDefault); + autoScalers.add(autoScalerDefault); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + String taskId = "taskId"; + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Collections.singletonList(StrategyTestUtils.TestTask.create( + taskId, + TASK_TYPE_1, + DATA_SOURCE_2 + ))); + EasyMock.expect(runner.getPendingTasks()) + .andReturn(Collections.singletonList( + new RemoteTaskRunnerWorkItem(taskId, TASK_TYPE_1, null, null, DATA_SOURCE_2) + )); + EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); + + EasyMock.replay(runner); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + Assert.assertEquals(1, provisioner.getStats().toList().size()); + Assert.assertSame(provisioner.getStats().toList().get(0).getEvent(), ScalingStats.EVENT.PROVISION); + + EasyMock.verify(autoScalerDefault, runner); + } + + //******************************************************** + private SimpleWorkerProvisioningStrategy createStrategy( + AtomicReference workerConfigRef + ) + { + return new SimpleWorkerProvisioningStrategy( + config, + DSuppliers.of(workerConfigRef), + new ProvisioningSchedulerConfig(), + () -> executorService + ); + } + + +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/StrategyTestUtils.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/StrategyTestUtils.java new file mode 100644 index 000000000000..b83f8c61135d --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/StrategyTestUtils.java @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.indexing.overlord.autoscaling; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.ZkWorker; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.FillCapacityWithCategorySpecWorkerSelectStrategy; +import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import org.apache.druid.indexing.overlord.setup.WorkerCategorySpec; +import org.apache.druid.indexing.worker.TaskAnnouncement; +import org.apache.druid.indexing.worker.Worker; +import org.apache.druid.indexing.worker.config.WorkerConfig; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.easymock.EasyMock; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +public class StrategyTestUtils +{ + public static final String MIN_VERSION = "2014-01-00T00:01:00Z"; + + public static void setupAutoscaler( + AutoScaler autoScaler, + String category, + int minWorkers, + int maxWorkers, + List pendingTasks + ) + { + setupAutoscaler(autoScaler, category, minWorkers, pendingTasks); + EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(maxWorkers); + } + + public static void setupAutoscaler( + AutoScaler autoScaler, + String category, + int minWorkers, + int maxWorkers, + List pendingTasks, + int categoryTimes, + int minWorkersTimes, + int maxWorkersTimes, + int pendingTasksTimes + ) + { + setupAutoscaler(autoScaler, category, minWorkers, pendingTasks, categoryTimes, minWorkersTimes, pendingTasksTimes); + EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(maxWorkers).times(maxWorkersTimes); + } + + public static void setupAutoscaler(AutoScaler autoScaler, String category, int minWorkers, List pendingTasks) + { + setupAutoscaler(autoScaler, category, minWorkers, pendingTasks, 4, 1, 1); + } + + public static void setupAutoscaler( + AutoScaler autoScaler, + String category, + int minWorkers, + List pendingTasks, + int categoryTimes, + int minWorkersTimes, + int pendingTasksTimes + ) + { + EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(minWorkers).times(minWorkersTimes); + EasyMock.expect(autoScaler.getCategory()).andReturn(category).times(categoryTimes); + EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) + .andReturn(pendingTasks).times(pendingTasksTimes); + } + + public static WorkerCategorySpec createWorkerCategorySpec(boolean isStrong) + { + Map categoryMap = new HashMap<>(); + return new WorkerCategorySpec(categoryMap, isStrong); + } + + public static WorkerCategorySpec createWorkerCategorySpec( + boolean isStrong, + String taskType, + String defaultCategory, + String datasource, + String category + ) + { + Map categoryAffinity = new HashMap<>(); + categoryAffinity.put(datasource, category); + WorkerCategorySpec.CategoryConfig categoryConfig = new WorkerCategorySpec.CategoryConfig( + defaultCategory, + categoryAffinity + ); + Map categoryMap = new HashMap<>(); + categoryMap.put(taskType, categoryConfig); + return new WorkerCategorySpec(categoryMap, isStrong); + } + + public static WorkerCategorySpec createWorkerCategorySpec( + boolean isStrong, + String taskType1, + WorkerCategorySpec.CategoryConfig categoryConfig1, + String taskType2, + WorkerCategorySpec.CategoryConfig categoryConfig2 + ) + { + Map categoryMap = new HashMap<>(); + categoryMap.put(taskType1, categoryConfig1); + categoryMap.put(taskType2, categoryConfig2); + return new WorkerCategorySpec(categoryMap, isStrong); + } + + public static AtomicReference createWorkerConfigRef( + WorkerCategorySpec workerCategorySpec, + List autoScalers + ) + { + return new AtomicReference<>( + new CategoriedWorkerBehaviorConfig( + new FillCapacityWithCategorySpecWorkerSelectStrategy(workerCategorySpec), + autoScalers + ) + ); + } + + public static class TestZkWorker extends ZkWorker + { + private final Task testTask; + + public TestZkWorker( + Task testTask + ) + { + this(testTask, "http", "host", "ip", MIN_VERSION, 1, WorkerConfig.DEFAULT_CATEGORY); + } + + public TestZkWorker( + Task testTask, + String category + ) + { + this(testTask, "http", "host", "ip", MIN_VERSION, 1, category); + } + + public TestZkWorker( + Task testTask, + String scheme, + String host, + String ip, + String version, + int capacity, + String category + ) + { + super(new Worker(scheme, host, ip, capacity, version, category), null, new DefaultObjectMapper()); + + this.testTask = testTask; + } + + @Override + public Map getRunningTasks() + { + if (testTask == null) { + return new HashMap<>(); + } + return ImmutableMap.of( + testTask.getId(), + TaskAnnouncement.create( + testTask, + TaskStatus.running(testTask.getId()), + TaskLocation.unknown() + ) + ); + } + } + + public static class TestTask extends NoopTask + { + private final String type; + + public TestTask( + String id, + String groupId, + String dataSource, + long runTime, + long isReadyTime, + String isReadyResult, + FirehoseFactory firehoseFactory, + Map context, + String type + ) + { + super(id, groupId, dataSource, runTime, isReadyTime, isReadyResult, firehoseFactory, context); + this.type = type; + } + + public static TestTask create(String taskType, String dataSource) + { + return new TestTask(null, null, dataSource, 0, 0, null, null, null, taskType); + } + + public static TestTask create(String id, String taskType, String dataSource) + { + return new TestTask(id, null, dataSource, 0, 0, null, null, null, taskType); + } + + @Override + public String getType() + { + return type; + } + } +} diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 0cd3730de43a..8873b851920e 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -73,6 +73,8 @@ import org.apache.druid.indexing.overlord.TaskRunnerFactory; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.TaskStorageQueryAdapter; +import org.apache.druid.indexing.overlord.autoscaling.CategoriedProvisioningConfig; +import org.apache.druid.indexing.overlord.autoscaling.CategoriedProvisioningStrategy; import org.apache.druid.indexing.overlord.autoscaling.PendingTaskBasedWorkerProvisioningConfig; import org.apache.druid.indexing.overlord.autoscaling.PendingTaskBasedWorkerProvisioningStrategy; import org.apache.druid.indexing.overlord.autoscaling.ProvisioningSchedulerConfig; @@ -124,6 +126,7 @@ import java.util.List; /** + * */ @Command( name = "overlord", @@ -313,6 +316,7 @@ private void configureAutoscale(Binder binder) PendingTaskBasedWorkerProvisioningConfig.class ); JsonConfigProvider.bind(binder, "druid.indexer.autoscale", SimpleWorkerProvisioningConfig.class); + JsonConfigProvider.bind(binder, "druid.indexer.autoscale", CategoriedProvisioningConfig.class); PolyBind.createChoice( binder, @@ -326,6 +330,7 @@ private void configureAutoscale(Binder binder) ); biddy.addBinding("simple").to(SimpleWorkerProvisioningStrategy.class); biddy.addBinding("pendingTaskBased").to(PendingTaskBasedWorkerProvisioningStrategy.class); + biddy.addBinding("categoriedTaskBased").to(CategoriedProvisioningStrategy.class); } private void configureOverlordHelpers(Binder binder) @@ -345,6 +350,7 @@ private void configureOverlordHelpers(Binder binder) } /** + * */ private static class OverlordJettyServerInitializer implements JettyServerInitializer {