From 71bf2324a256a1017802977a41662e84990a950b Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Mon, 4 Nov 2019 10:15:52 +0200 Subject: [PATCH 01/23] Support different types of MiddleManagers by Autoscaler #8695 Initial implementation. --- .../CategoriedProvisioningConfig.java | 74 +++ .../CategoriedProvisioningStrategy.java | 628 ++++++++++++++++++ .../setup/CategoriedWorkerBehaviorConfig.java | 107 +++ .../overlord/setup/WorkerBehaviorConfig.java | 3 +- .../CategoriedProvisioningStrategyTest.java | 175 +++++ .../org/apache/druid/cli/CliOverlord.java | 4 + 6 files changed, 990 insertions(+), 1 deletion(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningConfig.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategy.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/CategoriedWorkerBehaviorConfig.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java 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..bd8fecd6fc1a --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategy.java @@ -0,0 +1,628 @@ +/* + * 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.EqualDistributionWithCategorySpecWorkerSelectStrategy; +import org.apache.druid.indexing.overlord.setup.FillCapacityWithCategorySpecWorkerSelectStrategy; +import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +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.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 DefaultWorkerBehaviorConfig is supported as WorkerBehaviorConfig, [%s] given, cannot %s workers", + workerBehaviorConfig, + action + ); + return null; + } + final CategoriedWorkerBehaviorConfig workerConfig = (CategoriedWorkerBehaviorConfig) workerBehaviorConfig; + if (workerConfig.getDefaultAutoScaler() == null) { + log.error("No default autoScaler available, cannot %s workers", action); + return null; + } + if (!((workerConfig.getSelectStrategy() instanceof FillCapacityWithCategorySpecWorkerSelectStrategy) + || (workerConfig.getSelectStrategy() instanceof EqualDistributionWithCategorySpecWorkerSelectStrategy))) { + log.error("Select strategy %s is not supported", workerConfig.getSelectStrategy()); + 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 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 = zkWorkers.stream().collect(Collectors.groupingBy( + immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) + ); + + Set allCategories = workersByCategories.keySet(); + log.debug( + "Workers of %d categories: %s", + workersByCategories.size(), + allCategories + ); + + 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 = getCategoryAutoscaler(category, workerConfig); + + didTerminate = doTerminate( + category, + categoryWorkers, + currentlyTerminating, + groupAutoscaler + ) || didTerminate; + } + + return didTerminate; + } + + private boolean doTerminate( + String category, + Collection zkWorkers, + Set currentlyTerminating, + AutoScaler autoScaler + ) + { + 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 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) { + return false; + } + + // Group tasks by categories + Map> tasksByCategories = pendingTasks.stream().collect(Collectors.groupingBy(Task::getType)); + + Map> workersByCategories = workers.stream().collect(Collectors.groupingBy( + immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) + ); + + // Merge categories of tasks and workers + Set allCategories = new HashSet<>(tasksByCategories.keySet()); + allCategories.addAll(workersByCategories.keySet()); + + log.debug( + "Pending Tasks of %d categories, Workers of %d categories (%d common categories: %s)", + tasksByCategories.size(), + workersByCategories.size(), + allCategories.size(), + allCategories + ); + + if (allCategories.isEmpty()) { + // Likely empty categories means initialization. Just spinup required amount of workers + currentlyProvisioning.putIfAbsent(WorkerConfig.DEFAULT_CATEGORY, new HashSet<>()); + Set currentlyProvisioning = this.currentlyProvisioning.get(WorkerConfig.DEFAULT_CATEGORY); + didProvision = doProvision( + WorkerConfig.DEFAULT_CATEGORY, + Collections.emptyList(), + Collections.emptyList(), + workerConfig, + currentlyProvisioning, + workerConfig.getDefaultAutoScaler() + ); + return didProvision; + } + + 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 = getCategoryAutoscaler(category, workerConfig); + + 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; + } + + @Override + public ScalingStats getStats() + { + return scalingStats; + } + + 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, we cannot determine the exact capacity here to fulfill the need + // since we are not aware of the expectedWorkerCapacity. + int moreWorkersNeeded = currValidWorkers == 0 ? minWorkerCount : getWorkersNeededToAssignTasks( + remoteTaskRunnerConfig, + workerConfig, + pendingTasks, + workers + ); + log.debug("More workers needed: %d", moreWorkersNeeded); + + int want = Math.max( + minWorkerCount - currValidWorkers, + // 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; + } + + 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 + ) + ); + } + + @Nullable + private AutoScaler getCategoryAutoscaler(String category, CategoriedWorkerBehaviorConfig workerConfig) + { + AutoScaler autoScaler = workerConfig.getCategoryAutoScalers().get(category); + if (autoScaler == null && workerConfig.isStrong()) { + 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 ? workerConfig.getDefaultAutoScaler() : 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..640c76d1ce9a --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/CategoriedWorkerBehaviorConfig.java @@ -0,0 +1,107 @@ +/* + * 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 java.util.Map; +import java.util.Objects; + +public class CategoriedWorkerBehaviorConfig implements WorkerBehaviorConfig +{ + private final WorkerSelectStrategy selectStrategy; + private final AutoScaler defaultAutoScaler; + private final Map categoryAutoScalers; + private final boolean strong; + + @JsonCreator + public CategoriedWorkerBehaviorConfig( + @JsonProperty("selectStrategy") WorkerSelectStrategy selectStrategy, + @JsonProperty("defaultAutoScaler") AutoScaler defaultAutoScaler, + @JsonProperty("categoryAutoScalers") Map categoryAutoScalers, + @JsonProperty("strong") boolean strong + ) + { + this.selectStrategy = selectStrategy; + this.defaultAutoScaler = defaultAutoScaler; + this.categoryAutoScalers = categoryAutoScalers; + this.strong = strong; + } + + @Override + @JsonProperty + public WorkerSelectStrategy getSelectStrategy() + { + return selectStrategy; + } + + @JsonProperty + public AutoScaler getDefaultAutoScaler() + { + return defaultAutoScaler; + } + + @JsonProperty + public Map getCategoryAutoScalers() + { + return categoryAutoScalers; + } + + @JsonProperty + public boolean isStrong() + { + return strong; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CategoriedWorkerBehaviorConfig that = (CategoriedWorkerBehaviorConfig) o; + return strong == that.strong && + Objects.equals(selectStrategy, that.selectStrategy) && + Objects.equals(defaultAutoScaler, that.defaultAutoScaler) && + Objects.equals(categoryAutoScalers, that.categoryAutoScalers); + } + + @Override + public int hashCode() + { + return Objects.hash(selectStrategy, defaultAutoScaler, categoryAutoScalers, strong); + } + + @Override + public String toString() + { + return "WorkerConfiguration{" + + "selectStrategy=" + selectStrategy + + ", defaultAutoScaler=" + defaultAutoScaler + + ", categoryAutoScalers=" + categoryAutoScalers + + ", strong=" + strong + + '}'; + } +} 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/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java new file mode 100644 index 000000000000..94da161ce81a --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -0,0 +1,175 @@ +/* + * 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.apache.druid.common.guava.DSuppliers; +import org.apache.druid.indexing.common.TestTasks; +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.FillCapacityWithCategorySpecWorkerSelectStrategy; +import org.apache.druid.indexing.overlord.setup.WorkerBehaviorConfig; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.easymock.EasyMock; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; + +public class CategoriedProvisioningStrategyTest +{ + private AutoScaler autoScalerDefault; + private AutoScaler autoScalerCategory1; + private AutoScaler autoScalerCategory2; + private final Map categoryAutoScaler = new HashMap<>(); + private Task testTask; + private CategoriedProvisioningConfig config; + private final ScheduledExecutorService executorService = Execs.scheduledSingleThreaded("test service"); + private static final String MIN_VERSION = "2014-01-00T00:01:00Z"; + private static final String INVALID_VERSION = "0"; + + @Before + public void setup() + { + autoScalerDefault = EasyMock.createMock(AutoScaler.class); + + autoScalerCategory1 = EasyMock.createMock(AutoScaler.class); + autoScalerCategory2 = EasyMock.createMock(AutoScaler.class); + + categoryAutoScaler.clear(); + categoryAutoScaler.put("category1", autoScalerCategory1); + categoryAutoScaler.put("category2", autoScalerCategory2); + + testTask = TestTasks.immediateSuccess("task1"); + + config = new CategoriedProvisioningConfig() + .setMaxScalingDuration(new Period(1000)) + .setNumEventsToTrack(10) + .setPendingTaskTimeout(new Period(0)) + .setWorkerVersion(MIN_VERSION) + .setMaxScalingStep(2); + } + + @Test + public void testDefaultAutoscalerSuccessfullInitialMinWorkers() + { + AtomicReference workerConfig = createWorkerConfigRef(false); + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // No pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); + EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ).times(3); + + EasyMock.replay(runner, autoScalerDefault); + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + Assert.assertEquals(3, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + } + + @Test + public void testStrongAssigmentDoesntInitialMinWorkers() + { + AtomicReference workerConfig = createWorkerConfigRef(false); + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.emptyList()); + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // No pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( + Collections.emptyList() + ); + EasyMock.expect(runner.getWorkers()).andReturn( + Collections.emptyList() + ); + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ).times(3); + EasyMock.replay(runner, autoScalerDefault); + + + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + Assert.assertTrue(provisioner.getStats().toList().size() == 3); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertTrue( + event.getEvent() == ScalingStats.EVENT.PROVISION + ); + } + } + + + private void setupAutoscaler(AutoScaler autoScaler, int minWorkers, int maxWorkers, List pendingTasks) { + EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(minWorkers); + EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(maxWorkers); + EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) + .andReturn(pendingTasks); + } + + private AtomicReference createWorkerConfigRef(boolean isStrong) + { + return new AtomicReference<>( + new CategoriedWorkerBehaviorConfig( + new FillCapacityWithCategorySpecWorkerSelectStrategy(null), + autoScalerDefault, + categoryAutoScaler, + isStrong + ) + ); + } + + private CategoriedProvisioningStrategy createStrategy(AtomicReference workerConfigRef) + { + return new CategoriedProvisioningStrategy( + config, + DSuppliers.of(workerConfigRef), + new ProvisioningSchedulerConfig(), + () -> executorService + ); + } +} 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..46e0a284f45b 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,7 @@ 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.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 +125,7 @@ import java.util.List; /** + * */ @Command( name = "overlord", @@ -326,6 +328,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 +348,7 @@ private void configureOverlordHelpers(Binder binder) } /** + * */ private static class OverlordJettyServerInitializer implements JettyServerInitializer { From 22f75292b6139e9f8edf2a03c8d6cd357efdebb6 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Wed, 6 Nov 2019 14:56:55 +0200 Subject: [PATCH 02/23] Support different types of MiddleManagers by Autoscaler #8695 - Changed behavior config structure. - Small refactoring - Extended unit tests --- .../CategoriedProvisioningStrategy.java | 59 ++- .../setup/CategoriedWorkerBehaviorConfig.java | 18 +- .../overlord/setup/WorkerSelectUtils.java | 43 ++- .../CategoriedProvisioningStrategyTest.java | 349 ++++++++++++++++-- 4 files changed, 398 insertions(+), 71 deletions(-) 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 index bd8fecd6fc1a..20db45f6c010 100644 --- 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 @@ -37,7 +37,9 @@ import org.apache.druid.indexing.overlord.setup.EqualDistributionWithCategorySpecWorkerSelectStrategy; 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.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; @@ -66,6 +68,7 @@ public class CategoriedProvisioningStrategy extends AbstractWorkerProvisioningSt private final CategoriedProvisioningConfig config; private final Supplier workerConfigRef; + private final WorkerCategorySpec workerCategorySpec; @Nullable private static CategoriedWorkerBehaviorConfig getCategoriedWorkerBehaviorConfig( @@ -88,10 +91,6 @@ private static CategoriedWorkerBehaviorConfig getCategoriedWorkerBehaviorConfig( return null; } final CategoriedWorkerBehaviorConfig workerConfig = (CategoriedWorkerBehaviorConfig) workerBehaviorConfig; - if (workerConfig.getDefaultAutoScaler() == null) { - log.error("No default autoScaler available, cannot %s workers", action); - return null; - } if (!((workerConfig.getSelectStrategy() instanceof FillCapacityWithCategorySpecWorkerSelectStrategy) || (workerConfig.getSelectStrategy() instanceof EqualDistributionWithCategorySpecWorkerSelectStrategy))) { log.error("Select strategy %s is not supported", workerConfig.getSelectStrategy()); @@ -104,12 +103,14 @@ private static CategoriedWorkerBehaviorConfig getCategoriedWorkerBehaviorConfig( public CategoriedProvisioningStrategy( CategoriedProvisioningConfig config, Supplier workerConfigRef, + WorkerCategorySpec workerCategorySpec, ProvisioningSchedulerConfig provisioningSchedulerConfig ) { this( config, workerConfigRef, + workerCategorySpec, provisioningSchedulerConfig, () -> ScheduledExecutors.fixed(1, "CategoriedProvisioning-manager--%d") ); @@ -118,6 +119,7 @@ public CategoriedProvisioningStrategy( public CategoriedProvisioningStrategy( CategoriedProvisioningConfig config, Supplier workerConfigRef, + WorkerCategorySpec workerCategorySpec, ProvisioningSchedulerConfig provisioningSchedulerConfig, Supplier execFactory ) @@ -125,6 +127,7 @@ public CategoriedProvisioningStrategy( super(provisioningSchedulerConfig, execFactory); this.config = config; this.workerConfigRef = workerConfigRef; + this.workerCategorySpec = workerCategorySpec; } @Override @@ -299,11 +302,18 @@ public boolean doProvision() log ); if (workerConfig == null) { + log.info("No worker config found. Skip provisioning."); return false; } // Group tasks by categories - Map> tasksByCategories = pendingTasks.stream().collect(Collectors.groupingBy(Task::getType)); + Map> tasksByCategories = pendingTasks.stream().collect(Collectors.groupingBy( + task -> WorkerSelectUtils.getTaskCategory( + task, + workerCategorySpec, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY + ) + )); Map> workersByCategories = workers.stream().collect(Collectors.groupingBy( immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) @@ -322,17 +332,15 @@ public boolean doProvision() ); if (allCategories.isEmpty()) { - // Likely empty categories means initialization. Just spinup required amount of workers - currentlyProvisioning.putIfAbsent(WorkerConfig.DEFAULT_CATEGORY, new HashSet<>()); - Set currentlyProvisioning = this.currentlyProvisioning.get(WorkerConfig.DEFAULT_CATEGORY); - didProvision = doProvision( - WorkerConfig.DEFAULT_CATEGORY, - Collections.emptyList(), - Collections.emptyList(), - workerConfig, - currentlyProvisioning, - workerConfig.getDefaultAutoScaler() - ); + // Likely empty categories means initialization. Just try to spinup required amount of workers of each non empty autoscalers + if (workerConfig.getDefaultAutoScaler() != null) { + didProvision = initAutoscaler(workerConfig.getDefaultAutoScaler(), CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, workerConfig); + } + for (Map.Entry autoscalerInfo : workerConfig.getAutoScalers().entrySet()) { + String category = autoscalerInfo.getKey(); + AutoScaler autoScaler = autoscalerInfo.getValue(); + didProvision = initAutoscaler(autoScaler, category, workerConfig) || didProvision; + } return didProvision; } @@ -611,10 +619,27 @@ private int maxWorkersToTerminate(Collection zkWorkers, Aut ); } + 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 AutoScaler getCategoryAutoscaler(String category, CategoriedWorkerBehaviorConfig workerConfig) { - AutoScaler autoScaler = workerConfig.getCategoryAutoScalers().get(category); + AutoScaler autoScaler = workerConfig.getAutoScalers().get(category); if (autoScaler == null && workerConfig.isStrong()) { log.warn( "No autoscaler found for category %s. Tasks of this category will not be assigned to default autoscaler because of strong affinity.", 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 index 640c76d1ce9a..31ccc1f1390a 100644 --- 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 @@ -28,23 +28,22 @@ public class CategoriedWorkerBehaviorConfig implements WorkerBehaviorConfig { + public static final String DEFAULT_AUTOSCALER_CATEGORY = "_default_autoscaler_category"; + private final WorkerSelectStrategy selectStrategy; private final AutoScaler defaultAutoScaler; private final Map categoryAutoScalers; - private final boolean strong; @JsonCreator public CategoriedWorkerBehaviorConfig( @JsonProperty("selectStrategy") WorkerSelectStrategy selectStrategy, @JsonProperty("defaultAutoScaler") AutoScaler defaultAutoScaler, - @JsonProperty("categoryAutoScalers") Map categoryAutoScalers, - @JsonProperty("strong") boolean strong + @JsonProperty("categoryAutoScalers") Map categoryAutoScalers ) { this.selectStrategy = selectStrategy; this.defaultAutoScaler = defaultAutoScaler; this.categoryAutoScalers = categoryAutoScalers; - this.strong = strong; } @Override @@ -61,15 +60,14 @@ public AutoScaler getDefaultAutoScaler() } @JsonProperty - public Map getCategoryAutoScalers() + public Map getAutoScalers() { return categoryAutoScalers; } - @JsonProperty public boolean isStrong() { - return strong; + return defaultAutoScaler == null; } @Override @@ -82,8 +80,7 @@ public boolean equals(Object o) return false; } CategoriedWorkerBehaviorConfig that = (CategoriedWorkerBehaviorConfig) o; - return strong == that.strong && - Objects.equals(selectStrategy, that.selectStrategy) && + return Objects.equals(selectStrategy, that.selectStrategy) && Objects.equals(defaultAutoScaler, that.defaultAutoScaler) && Objects.equals(categoryAutoScalers, that.categoryAutoScalers); } @@ -91,7 +88,7 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(selectStrategy, defaultAutoScaler, categoryAutoScalers, strong); + return Objects.hash(selectStrategy, defaultAutoScaler, categoryAutoScalers); } @Override @@ -101,7 +98,6 @@ public String toString() "selectStrategy=" + selectStrategy + ", defaultAutoScaler=" + defaultAutoScaler + ", categoryAutoScalers=" + categoryAutoScalers + - ", strong=" + strong + '}'; } } 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..24c9103fecc4 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 @@ -110,6 +110,29 @@ public static ImmutableWorkerInfo selectWorker( { 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 +142,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. diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index 94da161ce81a..93a7bd49d8a3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -19,14 +19,24 @@ package org.apache.druid.indexing.overlord.autoscaling; +import com.google.common.collect.ImmutableMap; import org.apache.druid.common.guava.DSuppliers; +import org.apache.druid.indexer.TaskLocation; +import org.apache.druid.indexer.TaskStatus; 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.ZkWorker; import org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig; 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.apache.druid.java.util.common.concurrent.Execs; import org.easymock.EasyMock; import org.joda.time.Period; @@ -34,6 +44,7 @@ import org.junit.Before; import org.junit.Test; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -43,6 +54,8 @@ public class CategoriedProvisioningStrategyTest { + public static final String CATEGORY_1 = "category1"; + public static final String CATEGORY_2 = "category2"; private AutoScaler autoScalerDefault; private AutoScaler autoScalerCategory1; private AutoScaler autoScalerCategory2; @@ -62,8 +75,8 @@ public void setup() autoScalerCategory2 = EasyMock.createMock(AutoScaler.class); categoryAutoScaler.clear(); - categoryAutoScaler.put("category1", autoScalerCategory1); - categoryAutoScaler.put("category2", autoScalerCategory2); + categoryAutoScaler.put(CATEGORY_1, autoScalerCategory1); + categoryAutoScaler.put(CATEGORY_2, autoScalerCategory2); testTask = TestTasks.immediateSuccess("task1"); @@ -76,11 +89,12 @@ public void setup() } @Test - public void testDefaultAutoscalerSuccessfullInitialMinWorkers() + public void testDefaultAutoscalerSuccessfulInitialMinWorkers() { AtomicReference workerConfig = createWorkerConfigRef(false); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); @@ -88,88 +102,371 @@ public void testDefaultAutoscalerSuccessfullInitialMinWorkers() 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()); - EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(3); EasyMock.expect(autoScalerDefault.provision()).andReturn( new AutoScalingData(Collections.singletonList("aNode")) ).times(3); - EasyMock.replay(runner, autoScalerDefault); + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ).times(2); + + EasyMock.expect(autoScalerCategory2.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category2Node")) + ).times(4); + + EasyMock.replay(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); Provisioner provisioner = strategy.makeProvisioner(runner); boolean provisionedSomething = provisioner.doProvision(); Assert.assertTrue(provisionedSomething); - Assert.assertEquals(3, provisioner.getStats().toList().size()); + Assert.assertEquals(9, provisioner.getStats().toList().size()); for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); } } @Test - public void testStrongAssigmentDoesntInitialMinWorkers() + public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() + { + AtomicReference workerConfig = createWorkerConfigRef(true); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); + + 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 each categorizied autoscaler will call it + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(2); + + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ).times(2); + EasyMock.expect(autoScalerCategory2.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category2Node")) + ).times(4); + + 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); + } + } + + @Test + public void testDefaultAutoscalerSuccessfulMinWorkers() { AtomicReference workerConfig = createWorkerConfigRef(false); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); - EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); - EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); - EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) - .andReturn(Collections.emptyList()); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // No pending tasks - EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( - Collections.emptyList() + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); + // 1 node already running, only provision 2 more. + EasyMock.expect(runner.getWorkers()).andReturn( + Collections.singletonList( + new TestZkWorker(testTask).toImmutable() + ) ); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + + 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); + Assert.assertEquals(2, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + } + + @Test + public void testDefaultAutoscalerDidntSpawnMinWorkers() + { + AtomicReference workerConfig = createWorkerConfigRef(true); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); + + 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.emptyList() + Collections.singletonList( + new TestZkWorker(testTask).toImmutable() + ) ); + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + EasyMock.expect(autoScalerDefault.provision()).andReturn( new AutoScalingData(Collections.singletonList("aNode")) - ).times(3); + ).times(2); + EasyMock.replay(runner, autoScalerDefault); + Provisioner provisioner = strategy.makeProvisioner(runner); + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertFalse(provisionedSomething); + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + } + + @Test + public void testCategoriedAutoscalerSpawnedMinWorkers() + { + AtomicReference workerConfig = createWorkerConfigRef(true); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false, "noop", CATEGORY_1, "", ""); + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, 5, 7, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // One pending task + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.singletonList(NoopTask.create())); + // No workers + EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + + 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); - Assert.assertTrue(provisioner.getStats().toList().size() == 3); + Assert.assertEquals(5, provisioner.getStats().toList().size()); for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { - Assert.assertTrue( - event.getEvent() == ScalingStats.EVENT.PROVISION - ); + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); } } + @Test + public void testCategoriedAutoscalerSpawnedAdditionalWorker() + { + AtomicReference workerConfig = createWorkerConfigRef(true); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false, "noop", CATEGORY_1, "", ""); - private void setupAutoscaler(AutoScaler autoScaler, int minWorkers, int maxWorkers, List pendingTasks) { + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, 2, 3, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // One pending task + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.singletonList(NoopTask.create())); + // Min workers are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new 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); + Assert.assertEquals(1, provisioner.getStats().toList().size()); + Assert.assertSame(provisioner.getStats().toList().get(0).getEvent(), ScalingStats.EVENT.PROVISION); + } + + @Test + public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() + { + AtomicReference workerConfig = createWorkerConfigRef(true); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false, "noop", CATEGORY_1, "", ""); + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, 2, 3, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // Two pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Arrays.asList(NoopTask.create(), NoopTask.create())); + // Min workers are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new 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); + Assert.assertEquals(1, provisioner.getStats().toList().size()); + Assert.assertSame(provisioner.getStats().toList().get(0).getEvent(), ScalingStats.EVENT.PROVISION); + } + + + private void setupAutoscaler(AutoScaler autoScaler, int minWorkers, int maxWorkers, List pendingTasks) + { EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(minWorkers); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(maxWorkers); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) .andReturn(pendingTasks); } + private WorkerCategorySpec createWorkerCategorySpec(boolean isStrong) + { + Map categoryMap = new HashMap<>(); + return new WorkerCategorySpec(categoryMap, isStrong); + } + + private 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); + } + private AtomicReference createWorkerConfigRef(boolean isStrong) { return new AtomicReference<>( new CategoriedWorkerBehaviorConfig( new FillCapacityWithCategorySpecWorkerSelectStrategy(null), - autoScalerDefault, - categoryAutoScaler, - isStrong + isStrong ? null : autoScalerDefault, + categoryAutoScaler ) ); } - private CategoriedProvisioningStrategy createStrategy(AtomicReference workerConfigRef) + private CategoriedProvisioningStrategy createStrategy( + AtomicReference workerConfigRef, + WorkerCategorySpec workerCategorySpec + ) { return new CategoriedProvisioningStrategy( config, DSuppliers.of(workerConfigRef), + workerCategorySpec, new ProvisioningSchedulerConfig(), () -> executorService ); } + + public static class TestZkWorker extends ZkWorker + { + private final Task testTask; + + public TestZkWorker( + Task testTask + ) + { + this(testTask, "http", "host", "ip", MIN_VERSION, WorkerConfig.DEFAULT_CATEGORY); + } + + public TestZkWorker( + Task testTask, + String category + ) + { + this(testTask, "http", "host", "ip", MIN_VERSION, category); + } + + public TestZkWorker( + Task testTask, + String scheme, + String host, + String ip, + String version, + String category + ) + { + this(testTask, scheme, host, ip, 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() + ) + ); + } + } } From b61fee903f3cdc8e9fb178ae25b658b89f5d04d8 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Wed, 6 Nov 2019 16:27:19 +0200 Subject: [PATCH 03/23] Support different types of MiddleManagers by Autoscaler #8695 - Extended unit tests --- .../CategoriedProvisioningStrategyTest.java | 206 +++++++++++++++++- 1 file changed, 205 insertions(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index 93a7bd49d8a3..d6e3dc999375 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.common.guava.DSuppliers; +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.TestTasks; @@ -315,7 +316,7 @@ public void testCategoriedAutoscalerSpawnedAdditionalWorker() public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() { AtomicReference workerConfig = createWorkerConfigRef(true); - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false, "noop", CATEGORY_1, "", ""); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false, "noop", CATEGORY_1, "ds1", CATEGORY_2); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); @@ -348,6 +349,162 @@ public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() Assert.assertSame(provisioner.getStats().toList().get(0).getEvent(), ScalingStats.EVENT.PROVISION); } + @Test + public void testAllCategoriedAutoscalersStrongly() + { + String taskType1 = "taskType1"; + String taskType2 = "taskType2"; + String taskType3 = "taskType3"; + String dataSource1 = "ds1"; + String dataSource2 = "ds2"; + AtomicReference workerConfig = createWorkerConfigRef(true); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + false, + taskType1, + new WorkerCategorySpec.CategoryConfig( + "test1", + ImmutableMap.of( + dataSource1, + CATEGORY_1, + dataSource2, + CATEGORY_2 + ) + ), + taskType2, + new WorkerCategorySpec.CategoryConfig( + "test2", + ImmutableMap.of( + dataSource1, + CATEGORY_1, + dataSource2, + CATEGORY_2 + ) + ) + ); + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, 1, 3, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 1, 3, Collections.emptyList()); + + 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( + createTask(taskType1, dataSource1), + createTask(taskType2, dataSource2), + createTask(taskType1, dataSource2), + createTask(taskType3, dataSource2) + )); + // Min workers are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(2); + + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + 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); + Assert.assertEquals(3, provisioner.getStats().toList().size()); + for (ScalingStats.ScalingEvent event : provisioner.getStats().toList()) { + Assert.assertSame(event.getEvent(), ScalingStats.EVENT.PROVISION); + } + } + + @Test + public void testAllCategoriedAutoscalersNotStrongMode() + { + String taskType1 = "taskType1"; + String taskType2 = "taskType2"; + String taskType3 = "taskType3"; + String dataSource1 = "ds1"; + String dataSource2 = "ds2"; + AtomicReference workerConfig = createWorkerConfigRef(false); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + false, + taskType1, + new WorkerCategorySpec.CategoryConfig( + "test1", + ImmutableMap.of( + dataSource1, + CATEGORY_1, + dataSource2, + CATEGORY_2 + ) + ), + taskType2, + new WorkerCategorySpec.CategoryConfig( + "test2", + ImmutableMap.of( + dataSource1, + CATEGORY_1, + dataSource2, + CATEGORY_2 + ) + ) + ); + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, 1, 3, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 1, 3, Collections.emptyList()); + + 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( + createTask(taskType1, dataSource1), + createTask(taskType2, dataSource2), + createTask(taskType1, dataSource2), + createTask(taskType3, dataSource2) + )); + // Min workers of two categoriez are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()).times(3); + + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("aNode")) + ).times(3); + + EasyMock.expect(autoScalerCategory1.provision()).andReturn( + new AutoScalingData(Collections.singletonList("category1Node")) + ); + + 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); + } + } + private void setupAutoscaler(AutoScaler autoScaler, int minWorkers, int maxWorkers, List pendingTasks) { @@ -382,6 +539,20 @@ private WorkerCategorySpec createWorkerCategorySpec( return new WorkerCategorySpec(categoryMap, isStrong); } + private 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); + } + private AtomicReference createWorkerConfigRef(boolean isStrong) { return new AtomicReference<>( @@ -407,6 +578,11 @@ private CategoriedProvisioningStrategy createStrategy( ); } + private Task createTask(String taskType, String dataSource) + { + return new TestTask(null, null, dataSource, 0, 0, null, null, null, taskType); + } + public static class TestZkWorker extends ZkWorker { private final Task testTask; @@ -469,4 +645,32 @@ public Map getRunningTasks() ); } } + + private 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; + } + + @Override + public String getType() + { + return type; + } + } } From 66d3ebeac498d54ddbd752df8e18fef1c32bc7e6 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Wed, 6 Nov 2019 17:26:21 +0200 Subject: [PATCH 04/23] Support different types of MiddleManagers by Autoscaler #8695 - Extended and refactored unit tests --- .../CategoriedProvisioningStrategyTest.java | 168 +++++++++++------- 1 file changed, 107 insertions(+), 61 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index d6e3dc999375..3e29302722ea 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -57,6 +57,11 @@ public class CategoriedProvisioningStrategyTest { public static final String CATEGORY_1 = "category1"; public static final String CATEGORY_2 = "category2"; + public static final String TASK_TYPE_1 = "taskType1"; + public static final String TASK_TYPE_2 = "taskType2"; + public static final String TASK_TYPE_3 = "taskType3"; + public static final String DATA_SOURCE_1 = "ds1"; + public static final String DATA_SOURCE_2 = "ds2"; private AutoScaler autoScalerDefault; private AutoScaler autoScalerCategory1; private AutoScaler autoScalerCategory2; @@ -92,6 +97,7 @@ public void setup() @Test public void testDefaultAutoscalerSuccessfulInitialMinWorkers() { + // Not strong affinity autoscaling mode will use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(false); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); @@ -108,14 +114,17 @@ public void testDefaultAutoscalerSuccessfulInitialMinWorkers() 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 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); @@ -125,20 +134,23 @@ public void testDefaultAutoscalerSuccessfulInitialMinWorkers() Provisioner provisioner = strategy.makeProvisioner(runner); boolean provisionedSomething = provisioner.doProvision(); Assert.assertTrue(provisionedSomething); + // In total expect provisioning of 2 + 3 + 4 = 9 workers Assert.assertEquals(9, 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 testDefaultAutoscalerDidntSpawnInitialMinWorkers() { + // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(true); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); - setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); @@ -147,37 +159,43 @@ public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); // No workers EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); - // Expect this call two times because each categorizied autoscaler will call it + // 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, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + 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() { + // Not strong affinity autoscaling mode will use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(false); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); - setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // No pending tasks @@ -191,6 +209,7 @@ public void testDefaultAutoscalerSuccessfulMinWorkers() EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); + // Expect to create 2 workers EasyMock.expect(autoScalerDefault.provision()).andReturn( new AutoScalingData(Collections.singletonList("aNode")) ).times(2); @@ -200,66 +219,71 @@ public void testDefaultAutoscalerSuccessfulMinWorkers() 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 testDefaultAutoscalerDidntSpawnMinWorkers() + public void testAnyAutoscalerDontSpawnMinWorkers() { + // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(true); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); - setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); - setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // No pending tasks EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.emptyList()); - // 1 node already running, only provision 2 more. + + // 1 worker already running. That means no initialization is required. EasyMock.expect(runner.getWorkers()).andReturn( Collections.singletonList( new TestZkWorker(testTask).toImmutable() ) ); - EasyMock.expect(runner.getConfig()).andReturn(new RemoteTaskRunnerConfig()); - - EasyMock.expect(autoScalerDefault.provision()).andReturn( - new AutoScalingData(Collections.singletonList("aNode")) - ).times(2); - - EasyMock.replay(runner, autoScalerDefault); + 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() { + // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(true); - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false, "noop", CATEGORY_1, "", ""); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + false, + TASK_TYPE_1, + CATEGORY_1, + DATA_SOURCE_1, + CATEGORY_2 + ); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); - setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, 5, 7, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // One pending task - EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.singletonList(NoopTask.create())); + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Collections.singletonList(createTask(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); @@ -269,26 +293,35 @@ public void testCategoriedAutoscalerSpawnedMinWorkers() 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() { + // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(true); - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false, "noop", CATEGORY_1, "", ""); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + false, + TASK_TYPE_1, + CATEGORY_1, + DATA_SOURCE_1, + CATEGORY_2 + ); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); - setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, 2, 3, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // One pending task - EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Collections.singletonList(NoopTask.create())); + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Collections.singletonList(createTask(TASK_TYPE_1, DATA_SOURCE_2))); // Min workers are running EasyMock.expect(runner.getWorkers()).andReturn( Arrays.asList( @@ -308,24 +341,33 @@ public void testCategoriedAutoscalerSpawnedAdditionalWorker() 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() { + // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(true); - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false, "noop", CATEGORY_1, "ds1", CATEGORY_2); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + false, + TASK_TYPE_1, + CATEGORY_1, + DATA_SOURCE_1, + CATEGORY_2 + ); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); - setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, 2, 3, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // Two pending tasks - EasyMock.expect(runner.getPendingTaskPayloads()).andReturn(Arrays.asList(NoopTask.create(), NoopTask.create())); + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList(createTask(TASK_TYPE_1, DATA_SOURCE_2), createTask(TASK_TYPE_1, DATA_SOURCE_2))); // Min workers are running EasyMock.expect(runner.getWorkers()).andReturn( Arrays.asList( @@ -345,45 +387,43 @@ public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() 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() { - String taskType1 = "taskType1"; - String taskType2 = "taskType2"; - String taskType3 = "taskType3"; - String dataSource1 = "ds1"; - String dataSource2 = "ds2"; + // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(true); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( false, - taskType1, + TASK_TYPE_1, new WorkerCategorySpec.CategoryConfig( "test1", ImmutableMap.of( - dataSource1, + DATA_SOURCE_1, CATEGORY_1, - dataSource2, + DATA_SOURCE_2, CATEGORY_2 ) ), - taskType2, + TASK_TYPE_2, new WorkerCategorySpec.CategoryConfig( "test2", ImmutableMap.of( - dataSource1, + DATA_SOURCE_1, CATEGORY_1, - dataSource2, + DATA_SOURCE_2, CATEGORY_2 ) ) ); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); - setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, 1, 3, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, 1, 3, Collections.emptyList()); @@ -391,12 +431,12 @@ public void testAllCategoriedAutoscalersStrongly() // Four pending tasks: three have their categorized autoscalers and one for default autoscaler EasyMock.expect(runner.getPendingTaskPayloads()) .andReturn(Arrays.asList( - createTask(taskType1, dataSource1), - createTask(taskType2, dataSource2), - createTask(taskType1, dataSource2), - createTask(taskType3, dataSource2) + createTask(TASK_TYPE_1, DATA_SOURCE_1), + createTask(TASK_TYPE_2, DATA_SOURCE_2), + createTask(TASK_TYPE_1, DATA_SOURCE_2), + createTask(TASK_TYPE_3, DATA_SOURCE_2) )); - // Min workers are running + // Min workers number of the each category are running EasyMock.expect(runner.getWorkers()).andReturn( Arrays.asList( new TestZkWorker(testTask, CATEGORY_1).toImmutable(), @@ -406,10 +446,12 @@ public void testAllCategoriedAutoscalersStrongly() 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); @@ -419,40 +461,38 @@ public void testAllCategoriedAutoscalersStrongly() 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() { - String taskType1 = "taskType1"; - String taskType2 = "taskType2"; - String taskType3 = "taskType3"; - String dataSource1 = "ds1"; - String dataSource2 = "ds2"; AtomicReference workerConfig = createWorkerConfigRef(false); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( false, - taskType1, + TASK_TYPE_1, new WorkerCategorySpec.CategoryConfig( "test1", ImmutableMap.of( - dataSource1, + DATA_SOURCE_1, CATEGORY_1, - dataSource2, + DATA_SOURCE_2, CATEGORY_2 ) ), - taskType2, + TASK_TYPE_2, new WorkerCategorySpec.CategoryConfig( "test2", ImmutableMap.of( - dataSource1, + DATA_SOURCE_1, CATEGORY_1, - dataSource2, + DATA_SOURCE_2, CATEGORY_2 ) ) @@ -467,10 +507,10 @@ public void testAllCategoriedAutoscalersNotStrongMode() // Four pending tasks: three have their categorized autoscalers and one for default autoscaler EasyMock.expect(runner.getPendingTaskPayloads()) .andReturn(Arrays.asList( - createTask(taskType1, dataSource1), - createTask(taskType2, dataSource2), - createTask(taskType1, dataSource2), - createTask(taskType3, dataSource2) + createTask(TASK_TYPE_1, DATA_SOURCE_1), + createTask(TASK_TYPE_2, DATA_SOURCE_2), + createTask(TASK_TYPE_1, DATA_SOURCE_2), + createTask(TASK_TYPE_3, DATA_SOURCE_2) )); // Min workers of two categoriez are running EasyMock.expect(runner.getWorkers()).andReturn( @@ -482,14 +522,17 @@ public void testAllCategoriedAutoscalersNotStrongMode() 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); @@ -499,10 +542,13 @@ public void testAllCategoriedAutoscalersNotStrongMode() 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); } From ba88a3ca22a89bdf7cd9f86694997d81035fa875 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Thu, 7 Nov 2019 13:21:03 +0200 Subject: [PATCH 05/23] Support different types of MiddleManagers by Autoscaler #8695 - Extended and refactored unit tests --- .../CategoriedProvisioningStrategy.java | 13 +- .../setup/CategoriedWorkerBehaviorConfig.java | 4 +- .../CategoriedProvisioningStrategyTest.java | 361 +++++++++++++++--- 3 files changed, 331 insertions(+), 47 deletions(-) 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 index 20db45f6c010..b3e664bdb96c 100644 --- 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 @@ -324,9 +324,11 @@ public boolean doProvision() allCategories.addAll(workersByCategories.keySet()); log.debug( - "Pending Tasks of %d categories, Workers of %d categories (%d common categories: %s)", + "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 ); @@ -334,7 +336,11 @@ public boolean doProvision() if (allCategories.isEmpty()) { // Likely empty categories means initialization. Just try to spinup required amount of workers of each non empty autoscalers if (workerConfig.getDefaultAutoScaler() != null) { - didProvision = initAutoscaler(workerConfig.getDefaultAutoScaler(), CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, workerConfig); + didProvision = initAutoscaler( + workerConfig.getDefaultAutoScaler(), + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + workerConfig + ); } for (Map.Entry autoscalerInfo : workerConfig.getAutoScalers().entrySet()) { String category = autoscalerInfo.getKey(); @@ -639,6 +645,9 @@ private boolean initAutoscaler(AutoScaler autoScaler, String category, Categorie @Nullable private AutoScaler getCategoryAutoscaler(String category, CategoriedWorkerBehaviorConfig workerConfig) { + if (CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY.equals(category)) { + return workerConfig.getDefaultAutoScaler(); + } AutoScaler autoScaler = workerConfig.getAutoScalers().get(category); if (autoScaler == null && workerConfig.isStrong()) { log.warn( 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 index 31ccc1f1390a..07139a3120f8 100644 --- 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 @@ -22,13 +22,15 @@ 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.Map; import java.util.Objects; public class CategoriedWorkerBehaviorConfig implements WorkerBehaviorConfig { - public static final String DEFAULT_AUTOSCALER_CATEGORY = "_default_autoscaler_category"; + // 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 AutoScaler defaultAutoScaler; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index 3e29302722ea..d2eb0c439b73 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -55,13 +55,15 @@ public class CategoriedProvisioningStrategyTest { - public static final String CATEGORY_1 = "category1"; - public static final String CATEGORY_2 = "category2"; - public static final String TASK_TYPE_1 = "taskType1"; - public static final String TASK_TYPE_2 = "taskType2"; - public static final String TASK_TYPE_3 = "taskType3"; - public static final String DATA_SOURCE_1 = "ds1"; - public static final String DATA_SOURCE_2 = "ds2"; + 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; @@ -70,7 +72,6 @@ public class CategoriedProvisioningStrategyTest private CategoriedProvisioningConfig config; private final ScheduledExecutorService executorService = Execs.scheduledSingleThreaded("test service"); private static final String MIN_VERSION = "2014-01-00T00:01:00Z"; - private static final String INVALID_VERSION = "0"; @Before public void setup() @@ -277,7 +278,7 @@ public void testCategoriedAutoscalerSpawnedMinWorkers() RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // One pending task EasyMock.expect(runner.getPendingTaskPayloads()) - .andReturn(Collections.singletonList(createTask(TASK_TYPE_1, DATA_SOURCE_2))); + .andReturn(Collections.singletonList(TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); // No workers EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); @@ -321,7 +322,7 @@ public void testCategoriedAutoscalerSpawnedAdditionalWorker() RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // One pending task EasyMock.expect(runner.getPendingTaskPayloads()) - .andReturn(Collections.singletonList(createTask(TASK_TYPE_1, DATA_SOURCE_2))); + .andReturn(Collections.singletonList(TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); // Min workers are running EasyMock.expect(runner.getWorkers()).andReturn( Arrays.asList( @@ -367,7 +368,10 @@ public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // Two pending tasks EasyMock.expect(runner.getPendingTaskPayloads()) - .andReturn(Arrays.asList(createTask(TASK_TYPE_1, DATA_SOURCE_2), createTask(TASK_TYPE_1, DATA_SOURCE_2))); + .andReturn(Arrays.asList( + TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), + TestTask.create(TASK_TYPE_1, DATA_SOURCE_2) + )); // Min workers are running EasyMock.expect(runner.getWorkers()).andReturn( Arrays.asList( @@ -403,7 +407,7 @@ public void testAllCategoriedAutoscalersStrongly() false, TASK_TYPE_1, new WorkerCategorySpec.CategoryConfig( - "test1", + DEFAULT_CATEGORY_1, ImmutableMap.of( DATA_SOURCE_1, CATEGORY_1, @@ -413,7 +417,7 @@ public void testAllCategoriedAutoscalersStrongly() ), TASK_TYPE_2, new WorkerCategorySpec.CategoryConfig( - "test2", + DEFAULT_CATEGORY_2, ImmutableMap.of( DATA_SOURCE_1, CATEGORY_1, @@ -431,10 +435,10 @@ public void testAllCategoriedAutoscalersStrongly() // Four pending tasks: three have their categorized autoscalers and one for default autoscaler EasyMock.expect(runner.getPendingTaskPayloads()) .andReturn(Arrays.asList( - createTask(TASK_TYPE_1, DATA_SOURCE_1), - createTask(TASK_TYPE_2, DATA_SOURCE_2), - createTask(TASK_TYPE_1, DATA_SOURCE_2), - createTask(TASK_TYPE_3, DATA_SOURCE_2) + TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), + TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), + TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), + TestTask.create(TASK_TYPE_3, DATA_SOURCE_2) )); // Min workers number of the each category are running EasyMock.expect(runner.getWorkers()).andReturn( @@ -478,7 +482,7 @@ public void testAllCategoriedAutoscalersNotStrongMode() false, TASK_TYPE_1, new WorkerCategorySpec.CategoryConfig( - "test1", + DEFAULT_CATEGORY_1, ImmutableMap.of( DATA_SOURCE_1, CATEGORY_1, @@ -488,7 +492,7 @@ public void testAllCategoriedAutoscalersNotStrongMode() ), TASK_TYPE_2, new WorkerCategorySpec.CategoryConfig( - "test2", + DEFAULT_CATEGORY_2, ImmutableMap.of( DATA_SOURCE_1, CATEGORY_1, @@ -507,10 +511,10 @@ public void testAllCategoriedAutoscalersNotStrongMode() // Four pending tasks: three have their categorized autoscalers and one for default autoscaler EasyMock.expect(runner.getPendingTaskPayloads()) .andReturn(Arrays.asList( - createTask(TASK_TYPE_1, DATA_SOURCE_1), - createTask(TASK_TYPE_2, DATA_SOURCE_2), - createTask(TASK_TYPE_1, DATA_SOURCE_2), - createTask(TASK_TYPE_3, DATA_SOURCE_2) + TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), + TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), + TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), + TestTask.create(TASK_TYPE_3, DATA_SOURCE_2) )); // Min workers of two categoriez are running EasyMock.expect(runner.getWorkers()).andReturn( @@ -551,11 +555,292 @@ public void testAllCategoriedAutoscalersNotStrongMode() EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); } + @Test + public void testNullWorkerConfig() + { + AtomicReference workerConfig = new AtomicReference<>(null); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + false, + TASK_TYPE_1, + CATEGORY_1, + DATA_SOURCE_1, + CATEGORY_2 + ); + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + // One pending task + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Collections.singletonList(TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); + // Min workers of two categoriez are running + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new 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() + { + AtomicReference workerConfig = createWorkerConfigRef(false); + WorkerCategorySpec workerCategorySpec = null; + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + + setupAutoscaler(autoScalerDefault, 1, 3, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, 1, 2, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 1, 4, Collections.emptyList()); + + // 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( + TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), + TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), + 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 TestZkWorker(testTask).toImmutable(), + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new 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() + { + AtomicReference workerConfig = createWorkerConfigRef(true); + WorkerCategorySpec workerCategorySpec = null; + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + + setupAutoscaler(autoScalerCategory1, 1, 2, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 1, 4, Collections.emptyList()); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + + // Three pending tasks + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Arrays.asList( + TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), + TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), + 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 TestZkWorker(testTask).toImmutable(), + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new 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 + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + + EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + } + + @Test + public void testNoActionNeeded() + { + Assert.fail(); + } + + @Test + public void testDoSuccessfulTerminateForAllCategories() + { + AtomicReference workerConfig = createWorkerConfigRef(false); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + + setupAutoscaler(autoScalerDefault, 1, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, 1, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, 1, Collections.emptyList()); + + 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 TestZkWorker(testTask).toImmutable(), + new TestZkWorker(testTask).toImmutable(), + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new TestZkWorker(testTask, CATEGORY_2).toImmutable(), + new TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn( + Arrays.asList( + new TestZkWorker(testTask).getWorker(), + new TestZkWorker(testTask).getWorker(), + new TestZkWorker(testTask, CATEGORY_1).getWorker(), + new TestZkWorker(testTask, CATEGORY_1).getWorker(), + new TestZkWorker(testTask, CATEGORY_2).getWorker(), + new 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 testSomethingTerminating() + { + AtomicReference workerConfig = createWorkerConfigRef(false); + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(1); + 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.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.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.singletonList("ip")).times(2); + EasyMock.expect(autoScalerCategory2.terminate(EasyMock.anyObject())).andReturn( + new AutoScalingData(Collections.singletonList("ip")) + ); + + EasyMock.replay(autoScalerDefault, autoScalerCategory1, autoScalerCategory2); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + + EasyMock.expect(runner.getWorkers()).andReturn( + Arrays.asList( + new TestZkWorker(testTask).toImmutable(), + new TestZkWorker(testTask).toImmutable(), + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new TestZkWorker(testTask, CATEGORY_2).toImmutable(), + new TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ).times(2); + + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn( + Arrays.asList( + new TestZkWorker(testTask).getWorker(), + new TestZkWorker(testTask).getWorker(), + new TestZkWorker(testTask, CATEGORY_1).getWorker(), + new TestZkWorker(testTask, CATEGORY_1).getWorker(), + new TestZkWorker(testTask, CATEGORY_2).getWorker(), + new 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); + } + private void setupAutoscaler(AutoScaler autoScaler, int minWorkers, int maxWorkers, List pendingTasks) { - EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(minWorkers); + setupAutoscaler(autoScaler, minWorkers, pendingTasks); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(maxWorkers); + } + + private void setupAutoscaler(AutoScaler autoScaler, int minWorkers, List pendingTasks) + { + EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(minWorkers); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) .andReturn(pendingTasks); } @@ -624,11 +909,6 @@ private CategoriedProvisioningStrategy createStrategy( ); } - private Task createTask(String taskType, String dataSource) - { - return new TestTask(null, null, dataSource, 0, 0, null, null, null, taskType); - } - public static class TestZkWorker extends ZkWorker { private final Task testTask; @@ -637,7 +917,7 @@ public TestZkWorker( Task testTask ) { - this(testTask, "http", "host", "ip", MIN_VERSION, WorkerConfig.DEFAULT_CATEGORY); + this(testTask, "http", "host", "ip", MIN_VERSION, 1, WorkerConfig.DEFAULT_CATEGORY); } public TestZkWorker( @@ -645,19 +925,7 @@ public TestZkWorker( String category ) { - this(testTask, "http", "host", "ip", MIN_VERSION, category); - } - - public TestZkWorker( - Task testTask, - String scheme, - String host, - String ip, - String version, - String category - ) - { - this(testTask, scheme, host, ip, version, 1, category); + this(testTask, "http", "host", "ip", MIN_VERSION, 1, category); } public TestZkWorker( @@ -692,7 +960,7 @@ public Map getRunningTasks() } } - private class TestTask extends NoopTask + private static class TestTask extends NoopTask { private final String type; @@ -713,6 +981,11 @@ public TestTask( this.type = type; } + public static TestTask create(String taskType, String dataSource) + { + return new TestTask(null, null, dataSource, 0, 0, null, null, null, taskType); + } + @Override public String getType() { From d929a4e3b819528f1a6ca3e6355685ca6a89c217 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Fri, 8 Nov 2019 20:43:08 +0200 Subject: [PATCH 06/23] Support different types of MiddleManagers by Autoscaler #8695 - Extended and refactored unit tests --- .../CategoriedProvisioningStrategyTest.java | 131 ++++++++++++++++-- 1 file changed, 123 insertions(+), 8 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index d2eb0c439b73..8718036100c6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -39,12 +39,17 @@ import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.jackson.DefaultObjectMapper; 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.Assert; import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -555,6 +560,123 @@ public void testAllCategoriedAutoscalersNotStrongMode() 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); + + AtomicReference workerConfig = createWorkerConfigRef(false); + WorkerCategorySpec workerCategorySpec = 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 + ) + ) + ); + + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); + EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); + 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.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.ipToIdLookup(EasyMock.anyObject())) + .andReturn(Collections.emptyList()).times(2); + EasyMock.expect(autoScalerCategory2.terminateWithIds(EasyMock.anyObject())) + .andReturn(null); + + 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( + TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), + TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), + TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), + 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 TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new 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() { @@ -679,18 +801,12 @@ public void testNullWorkerCategorySpecStrong() Provisioner provisioner = strategy.makeProvisioner(runner); boolean provisionedSomething = provisioner.doProvision(); Assert.assertFalse(provisionedSomething); - // No workers should be created + // No workers should be created because of strong affinity Assert.assertTrue(provisioner.getStats().toList().isEmpty()); EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); } - @Test - public void testNoActionNeeded() - { - Assert.fail(); - } - @Test public void testDoSuccessfulTerminateForAllCategories() { @@ -962,7 +1078,6 @@ public Map getRunningTasks() private static class TestTask extends NoopTask { - private final String type; public TestTask( From 66f63ccf66009e58f8d5c7e34f1344fd55ae8ed1 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Tue, 12 Nov 2019 17:53:10 +0200 Subject: [PATCH 07/23] Support different types of MiddleManagers by Autoscaler #8695 - Fixed formatting --- .../overlord/setup/WorkerSelectUtils.java | 30 +++++++++++++------ .../CategoriedProvisioningStrategyTest.java | 1 - 2 files changed, 21 insertions(+), 10 deletions(-) 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 24c9103fecc4..f372797deff9 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,7 +112,11 @@ 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) { @@ -116,7 +124,10 @@ public static ImmutableWorkerInfo selectWorker( if (preferredCategory != null) { // select worker from preferred category - final ImmutableMap categoryWorkers = getCategoryWorkers(preferredCategory, runnableWorkers); + final ImmutableMap categoryWorkers = getCategoryWorkers( + preferredCategory, + runnableWorkers + ); final ImmutableWorkerInfo selected = workerSelector.apply(categoryWorkers); if (selected != null) { @@ -132,7 +143,8 @@ public static ImmutableWorkerInfo selectWorker( } @Nullable - public static String getTaskCategory(Task task, WorkerCategorySpec workerCategorySpec, String defaultValue) { + 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()); @@ -167,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 */ diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index 8718036100c6..81fe408f35ca 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -49,7 +49,6 @@ import org.junit.Before; import org.junit.Test; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; From 320f0c24808c90fc1b0629839fbfea5de9347cc4 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Wed, 13 Nov 2019 13:46:04 +0200 Subject: [PATCH 08/23] Support different types of MiddleManagers by Autoscaler #8695 - Fixed unit tests --- .../CategoriedProvisioningStrategy.java | 21 +++-- .../CategoriedProvisioningStrategyTest.java | 87 +++++++++---------- 2 files changed, 55 insertions(+), 53 deletions(-) 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 index b3e664bdb96c..ab9a152d52fb 100644 --- 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 @@ -68,7 +68,6 @@ public class CategoriedProvisioningStrategy extends AbstractWorkerProvisioningSt private final CategoriedProvisioningConfig config; private final Supplier workerConfigRef; - private final WorkerCategorySpec workerCategorySpec; @Nullable private static CategoriedWorkerBehaviorConfig getCategoriedWorkerBehaviorConfig( @@ -103,14 +102,12 @@ private static CategoriedWorkerBehaviorConfig getCategoriedWorkerBehaviorConfig( public CategoriedProvisioningStrategy( CategoriedProvisioningConfig config, Supplier workerConfigRef, - WorkerCategorySpec workerCategorySpec, ProvisioningSchedulerConfig provisioningSchedulerConfig ) { this( config, workerConfigRef, - workerCategorySpec, provisioningSchedulerConfig, () -> ScheduledExecutors.fixed(1, "CategoriedProvisioning-manager--%d") ); @@ -119,7 +116,6 @@ public CategoriedProvisioningStrategy( public CategoriedProvisioningStrategy( CategoriedProvisioningConfig config, Supplier workerConfigRef, - WorkerCategorySpec workerCategorySpec, ProvisioningSchedulerConfig provisioningSchedulerConfig, Supplier execFactory ) @@ -127,7 +123,6 @@ public CategoriedProvisioningStrategy( super(provisioningSchedulerConfig, execFactory); this.config = config; this.workerConfigRef = workerConfigRef; - this.workerCategorySpec = workerCategorySpec; } @Override @@ -306,6 +301,8 @@ public boolean doProvision() return false; } + WorkerCategorySpec workerCategorySpec = getWorkerCategorySpec(workerConfig); + // Group tasks by categories Map> tasksByCategories = pendingTasks.stream().collect(Collectors.groupingBy( task -> WorkerSelectUtils.getTaskCategory( @@ -658,5 +655,19 @@ private AutoScaler getCategoryAutoscaler(String category, CategoriedWorkerBehavi } return autoScaler == null ? workerConfig.getDefaultAutoScaler() : autoScaler; } + + @Nullable + private WorkerCategorySpec getWorkerCategorySpec(CategoriedWorkerBehaviorConfig workerConfig) + { + if (workerConfig != null && workerConfig.getSelectStrategy() != null) { + WorkerSelectStrategy selectStrategy = workerConfig.getSelectStrategy(); + if (selectStrategy instanceof FillCapacityWithCategorySpecWorkerSelectStrategy) { + return ((FillCapacityWithCategorySpecWorkerSelectStrategy) selectStrategy).getWorkerCategorySpec(); + } else if (selectStrategy instanceof EqualDistributionWithCategorySpecWorkerSelectStrategy) { + return ((EqualDistributionWithCategorySpecWorkerSelectStrategy) selectStrategy).getWorkerCategorySpec(); + } + } + return null; + } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index 81fe408f35ca..f8fb31a01763 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -103,10 +103,10 @@ public void setup() public void testDefaultAutoscalerSuccessfulInitialMinWorkers() { // Not strong affinity autoscaling mode will use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(false); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); @@ -151,11 +151,11 @@ public void testDefaultAutoscalerSuccessfulInitialMinWorkers() @Test public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() { - // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); @@ -195,11 +195,11 @@ public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() @Test public void testDefaultAutoscalerSuccessfulMinWorkers() { - // Not strong affinity autoscaling mode will use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(false); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + // Not strong affinity autoscaling mode will use default autoscaler + AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -236,11 +236,11 @@ public void testDefaultAutoscalerSuccessfulMinWorkers() @Test public void testAnyAutoscalerDontSpawnMinWorkers() { - // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // No pending tasks @@ -266,8 +266,6 @@ public void testAnyAutoscalerDontSpawnMinWorkers() @Test public void testCategoriedAutoscalerSpawnedMinWorkers() { - // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( false, TASK_TYPE_1, @@ -275,8 +273,10 @@ public void testCategoriedAutoscalerSpawnedMinWorkers() DATA_SOURCE_1, CATEGORY_2 ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, 5, 7, Collections.emptyList()); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -310,8 +310,6 @@ public void testCategoriedAutoscalerSpawnedMinWorkers() @Test public void testCategoriedAutoscalerSpawnedAdditionalWorker() { - // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( false, TASK_TYPE_1, @@ -319,8 +317,10 @@ public void testCategoriedAutoscalerSpawnedAdditionalWorker() DATA_SOURCE_1, CATEGORY_2 ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, 2, 3, Collections.emptyList()); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -356,8 +356,6 @@ public void testCategoriedAutoscalerSpawnedAdditionalWorker() @Test public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() { - // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( false, TASK_TYPE_1, @@ -365,8 +363,10 @@ public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() DATA_SOURCE_1, CATEGORY_2 ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, 2, 3, Collections.emptyList()); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -405,8 +405,6 @@ public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() @Test public void testAllCategoriedAutoscalersStrongly() { - // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( false, TASK_TYPE_1, @@ -430,8 +428,10 @@ public void testAllCategoriedAutoscalersStrongly() ) ) ); + // Strong affinity autoscaling mode will not use default autoscaler + AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, 1, 3, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, 1, 3, Collections.emptyList()); @@ -481,7 +481,6 @@ public void testAllCategoriedAutoscalersStrongly() @Test public void testAllCategoriedAutoscalersNotStrongMode() { - AtomicReference workerConfig = createWorkerConfigRef(false); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( false, TASK_TYPE_1, @@ -505,8 +504,9 @@ public void testAllCategoriedAutoscalersNotStrongMode() ) ) ); + AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, 1, 3, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, 1, 3, Collections.emptyList()); @@ -568,7 +568,6 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException EasyMock.expectLastCall().times(3); EasyMock.replay(emitter); - AtomicReference workerConfig = createWorkerConfigRef(false); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( false, TASK_TYPE_1, @@ -592,8 +591,9 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException ) ) ); + AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); @@ -680,15 +680,8 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException public void testNullWorkerConfig() { AtomicReference workerConfig = new AtomicReference<>(null); - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( - false, - TASK_TYPE_1, - CATEGORY_1, - DATA_SOURCE_1, - CATEGORY_2 - ); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // One pending task @@ -716,10 +709,10 @@ public void testNullWorkerConfig() @Test public void testNullWorkerCategorySpecNotStrong() { - AtomicReference workerConfig = createWorkerConfigRef(false); WorkerCategorySpec workerCategorySpec = null; + AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerDefault, 1, 3, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, 1, 2, Collections.emptyList()); @@ -766,10 +759,10 @@ public void testNullWorkerCategorySpecNotStrong() @Test public void testNullWorkerCategorySpecStrong() { - AtomicReference workerConfig = createWorkerConfigRef(true); WorkerCategorySpec workerCategorySpec = null; + AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, 1, 2, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, 1, 4, Collections.emptyList()); @@ -809,9 +802,9 @@ public void testNullWorkerCategorySpecStrong() @Test public void testDoSuccessfulTerminateForAllCategories() { - AtomicReference workerConfig = createWorkerConfigRef(false); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerDefault, 1, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, 1, Collections.emptyList()); @@ -871,9 +864,9 @@ public void testDoSuccessfulTerminateForAllCategories() @Test public void testSomethingTerminating() { - AtomicReference workerConfig = createWorkerConfigRef(false); WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(1); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) @@ -999,11 +992,11 @@ private WorkerCategorySpec createWorkerCategorySpec( return new WorkerCategorySpec(categoryMap, isStrong); } - private AtomicReference createWorkerConfigRef(boolean isStrong) + private AtomicReference createWorkerConfigRef(boolean isStrong, WorkerCategorySpec workerCategorySpec) { return new AtomicReference<>( new CategoriedWorkerBehaviorConfig( - new FillCapacityWithCategorySpecWorkerSelectStrategy(null), + new FillCapacityWithCategorySpecWorkerSelectStrategy(workerCategorySpec), isStrong ? null : autoScalerDefault, categoryAutoScaler ) @@ -1011,14 +1004,12 @@ private AtomicReference createWorkerConfigRef(boolean isSt } private CategoriedProvisioningStrategy createStrategy( - AtomicReference workerConfigRef, - WorkerCategorySpec workerCategorySpec + AtomicReference workerConfigRef ) { return new CategoriedProvisioningStrategy( config, DSuppliers.of(workerConfigRef), - workerCategorySpec, new ProvisioningSchedulerConfig(), () -> executorService ); From 9fba4d2f7bb369f8a240a0e9a85a8701c1f615b2 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Mon, 18 Nov 2019 19:08:26 +0200 Subject: [PATCH 09/23] Support different types of MiddleManagers by Autoscaler #8695 - Added "category" field into Autoscaler - Changed config format to support the "category" field - Fixed parameters of unit tests --- .../autoscaling/ec2/EC2AutoScaler.java | 12 +- .../autoscaling/ec2/EC2AutoScalerTest.java | 10 +- .../setup/WorkerBehaviorConfigTest.java | 3 +- .../overlord/autoscaling/AutoScaler.java | 2 + .../CategoriedProvisioningStrategy.java | 59 ++++++--- .../overlord/autoscaling/NoopAutoScaler.java | 20 ++- .../setup/CategoriedWorkerBehaviorConfig.java | 34 ++---- .../setup/DefaultWorkerBehaviorConfig.java | 2 +- .../CategoriedProvisioningStrategyTest.java | 115 +++++++++++------- 9 files changed, 158 insertions(+), 99 deletions(-) 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..54abe18d506a 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 @@ -40,6 +40,7 @@ import org.apache.druid.indexing.overlord.autoscaling.AutoScaler; 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.java.util.emitter.EmittingLogger; import java.util.ArrayList; @@ -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(value = "category", defaultValue = CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY) String category ) { this.minNumWorkers = minNumWorkers; this.maxNumWorkers = maxNumWorkers; + this.category = category; this.envConfig = envConfig; this.amazonEC2Client = amazonEC2Client; this.config = config; @@ -89,6 +93,12 @@ public int getMaxNumWorkers() return maxNumWorkers; } + @Override + public String getCategory() + { + return category; + } + @Override @JsonProperty public EC2EnvironmentConfig getEnvConfig() 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..6ed048a64ace 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 @@ -36,6 +36,8 @@ public interface AutoScaler int getMaxNumWorkers(); + String getCategory(); + /** * 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/CategoriedProvisioningStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategy.java index ab9a152d52fb..579a96dc57e2 100644 --- 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 @@ -60,6 +60,8 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; +import static org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY; + @JsonTypeName("categoriedTaskBased") public class CategoriedProvisioningStrategy extends AbstractWorkerProvisioningStrategy { @@ -90,11 +92,18 @@ private static CategoriedWorkerBehaviorConfig getCategoriedWorkerBehaviorConfig( return null; } final CategoriedWorkerBehaviorConfig workerConfig = (CategoriedWorkerBehaviorConfig) workerBehaviorConfig; + // TODO Create a superclass if (!((workerConfig.getSelectStrategy() instanceof FillCapacityWithCategorySpecWorkerSelectStrategy) || (workerConfig.getSelectStrategy() instanceof EqualDistributionWithCategorySpecWorkerSelectStrategy))) { 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; } @@ -174,6 +183,8 @@ public boolean doTerminate() allCategories ); + Map autoscalersByCategory = mapAutoscalerByCategory(workerConfig.getAutoScalers()); + for (String category : allCategories) { Set currentlyProvisioning = this.currentlyProvisioning.getOrDefault(category, Collections.emptySet()); log.info( @@ -190,7 +201,7 @@ public boolean doTerminate() List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); currentlyTerminating.putIfAbsent(category, new HashSet<>()); Set currentlyTerminating = this.currentlyTerminating.get(category); - AutoScaler groupAutoscaler = getCategoryAutoscaler(category, workerConfig); + AutoScaler groupAutoscaler = getCategoryAutoscaler(category, autoscalersByCategory); didTerminate = doTerminate( category, @@ -308,7 +319,7 @@ public boolean doProvision() task -> WorkerSelectUtils.getTaskCategory( task, workerCategorySpec, - CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY + DEFAULT_AUTOSCALER_CATEGORY ) )); @@ -332,27 +343,21 @@ public boolean doProvision() if (allCategories.isEmpty()) { // Likely empty categories means initialization. Just try to spinup required amount of workers of each non empty autoscalers - if (workerConfig.getDefaultAutoScaler() != null) { - didProvision = initAutoscaler( - workerConfig.getDefaultAutoScaler(), - CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, - workerConfig - ); - } - for (Map.Entry autoscalerInfo : workerConfig.getAutoScalers().entrySet()) { - String category = autoscalerInfo.getKey(); - AutoScaler autoScaler = autoscalerInfo.getValue(); + for (AutoScaler autoScaler : workerConfig.getAutoScalers()) { + String category = autoScaler.getCategory(); didProvision = initAutoscaler(autoScaler, category, workerConfig) || didProvision; } return didProvision; } + Map autoscalersByCategory = 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 = getCategoryAutoscaler(category, workerConfig); + AutoScaler groupAutoscaler = getCategoryAutoscaler(category, autoscalersByCategory); didProvision = doProvision( category, @@ -640,20 +645,33 @@ private boolean initAutoscaler(AutoScaler autoScaler, String category, Categorie } @Nullable - private AutoScaler getCategoryAutoscaler(String category, CategoriedWorkerBehaviorConfig workerConfig) + private AutoScaler getCategoryAutoscaler(String category, Map autoscalersByCategory) { - if (CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY.equals(category)) { - return workerConfig.getDefaultAutoScaler(); - } - AutoScaler autoScaler = workerConfig.getAutoScalers().get(category); - if (autoScaler == null && workerConfig.isStrong()) { + AutoScaler autoScaler = autoscalersByCategory.get(category); + boolean isStrongAssignment = !autoscalersByCategory.containsKey(DEFAULT_AUTOSCALER_CATEGORY); + + 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 ? workerConfig.getDefaultAutoScaler() : autoScaler; + return autoScaler == null ? autoscalersByCategory.get(DEFAULT_AUTOSCALER_CATEGORY) : autoScaler; + } + + private Map mapAutoscalerByCategory(List autoScalers) + { + Map result = autoScalers.stream().collect(Collectors.groupingBy( + AutoScaler::getCategory, + 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 will be used."); + } + + return result; } @Nullable @@ -661,6 +679,7 @@ private WorkerCategorySpec getWorkerCategorySpec(CategoriedWorkerBehaviorConfig { if (workerConfig != null && workerConfig.getSelectStrategy() != null) { WorkerSelectStrategy selectStrategy = workerConfig.getSelectStrategy(); + // TODO Replace by superclass if (selectStrategy instanceof FillCapacityWithCategorySpecWorkerSelectStrategy) { return ((FillCapacityWithCategorySpecWorkerSelectStrategy) selectStrategy).getWorkerCategorySpec(); } else if (selectStrategy instanceof EqualDistributionWithCategorySpecWorkerSelectStrategy) { 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..6a01885c6c81 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.JsonProperty; +import org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -30,6 +32,14 @@ public class NoopAutoScaler implements AutoScaler { private static final EmittingLogger log = new EmittingLogger(NoopAutoScaler.class); + private final String category; + + public NoopAutoScaler( + @JsonProperty(value = "category", defaultValue = CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY) String category + ) + { + this.category = category; + } @Override public int getMinNumWorkers() @@ -43,6 +53,12 @@ public int getMaxNumWorkers() return 0; } + @Override + public String getCategory() + { + return category; + } + @Override public Void getEnvConfig() { @@ -52,14 +68,14 @@ 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 in 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 in category %s", ips, category); return null; } 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 index 07139a3120f8..fe98e570c773 100644 --- 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 @@ -24,7 +24,7 @@ import org.apache.druid.indexing.overlord.autoscaling.AutoScaler; import org.apache.druid.indexing.worker.config.WorkerConfig; -import java.util.Map; +import java.util.List; import java.util.Objects; public class CategoriedWorkerBehaviorConfig implements WorkerBehaviorConfig @@ -33,19 +33,16 @@ public class CategoriedWorkerBehaviorConfig implements WorkerBehaviorConfig public static final String DEFAULT_AUTOSCALER_CATEGORY = WorkerConfig.DEFAULT_CATEGORY; private final WorkerSelectStrategy selectStrategy; - private final AutoScaler defaultAutoScaler; - private final Map categoryAutoScalers; + private final List autoScalers; @JsonCreator public CategoriedWorkerBehaviorConfig( @JsonProperty("selectStrategy") WorkerSelectStrategy selectStrategy, - @JsonProperty("defaultAutoScaler") AutoScaler defaultAutoScaler, - @JsonProperty("categoryAutoScalers") Map categoryAutoScalers + @JsonProperty("autoScalers") List autoScalers ) { this.selectStrategy = selectStrategy; - this.defaultAutoScaler = defaultAutoScaler; - this.categoryAutoScalers = categoryAutoScalers; + this.autoScalers = autoScalers; } @Override @@ -56,20 +53,9 @@ public WorkerSelectStrategy getSelectStrategy() } @JsonProperty - public AutoScaler getDefaultAutoScaler() + public List getAutoScalers() { - return defaultAutoScaler; - } - - @JsonProperty - public Map getAutoScalers() - { - return categoryAutoScalers; - } - - public boolean isStrong() - { - return defaultAutoScaler == null; + return autoScalers; } @Override @@ -83,14 +69,13 @@ public boolean equals(Object o) } CategoriedWorkerBehaviorConfig that = (CategoriedWorkerBehaviorConfig) o; return Objects.equals(selectStrategy, that.selectStrategy) && - Objects.equals(defaultAutoScaler, that.defaultAutoScaler) && - Objects.equals(categoryAutoScalers, that.categoryAutoScalers); + Objects.equals(autoScalers, that.autoScalers); } @Override public int hashCode() { - return Objects.hash(selectStrategy, defaultAutoScaler, categoryAutoScalers); + return Objects.hash(selectStrategy, autoScalers); } @Override @@ -98,8 +83,7 @@ public String toString() { return "WorkerConfiguration{" + "selectStrategy=" + selectStrategy + - ", defaultAutoScaler=" + defaultAutoScaler + - ", categoryAutoScalers=" + categoryAutoScalers + + ", autoScalers=" + autoScalers + '}'; } } 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/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index f8fb31a01763..395f4c06bfc5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -49,6 +49,7 @@ import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -57,6 +58,8 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY; + public class CategoriedProvisioningStrategyTest { private static final String DEFAULT_CATEGORY_1 = "default_category1"; @@ -71,7 +74,8 @@ public class CategoriedProvisioningStrategyTest private AutoScaler autoScalerDefault; private AutoScaler autoScalerCategory1; private AutoScaler autoScalerCategory2; - private final Map categoryAutoScaler = new HashMap<>(); + private final List autoScalers = new ArrayList<>(); + private final List autoScalersStrong = new ArrayList<>(); private Task testTask; private CategoriedProvisioningConfig config; private final ScheduledExecutorService executorService = Execs.scheduledSingleThreaded("test service"); @@ -85,9 +89,7 @@ public void setup() autoScalerCategory1 = EasyMock.createMock(AutoScaler.class); autoScalerCategory2 = EasyMock.createMock(AutoScaler.class); - categoryAutoScaler.clear(); - categoryAutoScaler.put(CATEGORY_1, autoScalerCategory1); - categoryAutoScaler.put(CATEGORY_2, autoScalerCategory2); + autoScalers.clear(); testTask = TestTasks.immediateSuccess("task1"); @@ -104,12 +106,13 @@ public void testDefaultAutoscalerSuccessfulInitialMinWorkers() { // Not strong affinity autoscaling mode will use default autoscaler WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); - setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); + setupAutoscaler(autoScalerDefault, DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, CATEGORY_2,4, 6, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // No pending tasks @@ -153,11 +156,12 @@ public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() { WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, 2, 4, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 4, 6, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // No pending tasks @@ -197,10 +201,11 @@ public void testDefaultAutoscalerSuccessfulMinWorkers() { WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); // Not strong affinity autoscaling mode will use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerDefault, DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); + autoScalers.add(autoScalerDefault); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // No pending tasks @@ -238,7 +243,7 @@ public void testAnyAutoscalerDontSpawnMinWorkers() { WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); @@ -274,10 +279,11 @@ public void testCategoriedAutoscalerSpawnedMinWorkers() CATEGORY_2 ); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, 5, 7, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1,5, 7, Collections.emptyList()); + autoScalers.add(autoScalerCategory1); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // One pending task @@ -318,10 +324,11 @@ public void testCategoriedAutoscalerSpawnedAdditionalWorker() CATEGORY_2 ); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, 2, 3, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 3, Collections.emptyList()); + autoScalers.add(autoScalerCategory1); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // One pending task @@ -364,10 +371,11 @@ public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() CATEGORY_2 ); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, 2, 3, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 3, Collections.emptyList()); + autoScalers.add(autoScalerCategory1); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // Two pending tasks @@ -429,11 +437,12 @@ public void testAllCategoriedAutoscalersStrongly() ) ); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, 1, 3, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 1, 3, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 3, Collections.emptyList()); + 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 @@ -504,12 +513,13 @@ public void testAllCategoriedAutoscalersNotStrongMode() ) ) ); - AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, 3, 5, Collections.emptyList()); - setupAutoscaler(autoScalerCategory1, 1, 3, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 1, 3, Collections.emptyList()); + setupAutoscaler(autoScalerDefault, DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 3, Collections.emptyList()); + 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 @@ -591,12 +601,13 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException ) ) ); - AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(DEFAULT_AUTOSCALER_CATEGORY).times(2); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.emptyList()).times(2); EasyMock.expect(autoScalerDefault.terminateWithIds(EasyMock.anyObject())) @@ -604,6 +615,7 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException EasyMock.expect(autoScalerCategory1.getMinNumWorkers()).andReturn(1); EasyMock.expect(autoScalerCategory1.getMaxNumWorkers()).andReturn(3); + EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(2); EasyMock.expect(autoScalerCategory1.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.emptyList()).times(2); EasyMock.expect(autoScalerCategory1.terminateWithIds(EasyMock.anyObject())) @@ -611,11 +623,14 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException EasyMock.expect(autoScalerCategory2.getMinNumWorkers()).andReturn(1); EasyMock.expect(autoScalerCategory2.getMaxNumWorkers()).andReturn(3); + EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(2); 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()) @@ -710,13 +725,14 @@ public void testNullWorkerConfig() public void testNullWorkerCategorySpecNotStrong() { WorkerCategorySpec workerCategorySpec = null; - AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, 1, 3, Collections.emptyList()); - setupAutoscaler(autoScalerCategory1, 1, 2, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 1, 4, Collections.emptyList()); + setupAutoscaler(autoScalerDefault, DEFAULT_AUTOSCALER_CATEGORY, 1, 3, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 2, Collections.emptyList()); + 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( @@ -760,12 +776,13 @@ public void testNullWorkerCategorySpecNotStrong() public void testNullWorkerCategorySpecStrong() { WorkerCategorySpec workerCategorySpec = null; - AtomicReference workerConfig = createWorkerConfigRef(true, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, 1, 2, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 1, 4, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 2, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 4, Collections.emptyList()); + autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -803,12 +820,13 @@ public void testNullWorkerCategorySpecStrong() public void testDoSuccessfulTerminateForAllCategories() { WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, 1, Collections.emptyList()); - setupAutoscaler(autoScalerCategory1, 1, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, 1, Collections.emptyList()); + setupAutoscaler(autoScalerDefault, DEFAULT_AUTOSCALER_CATEGORY, 1, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, Collections.emptyList()); + 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()) @@ -865,10 +883,11 @@ public void testDoSuccessfulTerminateForAllCategories() public void testSomethingTerminating() { WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - AtomicReference workerConfig = createWorkerConfigRef(false, workerCategorySpec); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(1); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(DEFAULT_AUTOSCALER_CATEGORY).times(2); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")).times(2); EasyMock.expect(autoScalerDefault.terminate(EasyMock.anyObject())).andReturn( @@ -876,6 +895,7 @@ public void testSomethingTerminating() ); EasyMock.expect(autoScalerCategory1.getMinNumWorkers()).andReturn(1); + EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(2); EasyMock.expect(autoScalerCategory1.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")).times(2); EasyMock.expect(autoScalerCategory1.terminate(EasyMock.anyObject())).andReturn( @@ -883,12 +903,15 @@ public void testSomethingTerminating() ); EasyMock.expect(autoScalerCategory2.getMinNumWorkers()).andReturn(1); + EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(2); 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); @@ -940,15 +963,16 @@ public void testSomethingTerminating() } - private void setupAutoscaler(AutoScaler autoScaler, int minWorkers, int maxWorkers, List pendingTasks) + private void setupAutoscaler(AutoScaler autoScaler, String category, int minWorkers, int maxWorkers, List pendingTasks) { - setupAutoscaler(autoScaler, minWorkers, pendingTasks); + setupAutoscaler(autoScaler, category, minWorkers, pendingTasks); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(maxWorkers); } - private void setupAutoscaler(AutoScaler autoScaler, int minWorkers, List pendingTasks) + private void setupAutoscaler(AutoScaler autoScaler, String category, int minWorkers, List pendingTasks) { EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(minWorkers); + EasyMock.expect(autoScaler.getCategory()).andReturn(category); EasyMock.expect(autoScaler.ipToIdLookup(EasyMock.anyObject())) .andReturn(pendingTasks); } @@ -992,13 +1016,12 @@ private WorkerCategorySpec createWorkerCategorySpec( return new WorkerCategorySpec(categoryMap, isStrong); } - private AtomicReference createWorkerConfigRef(boolean isStrong, WorkerCategorySpec workerCategorySpec) + private AtomicReference createWorkerConfigRef(WorkerCategorySpec workerCategorySpec) { return new AtomicReference<>( new CategoriedWorkerBehaviorConfig( new FillCapacityWithCategorySpecWorkerSelectStrategy(workerCategorySpec), - isStrong ? null : autoScalerDefault, - categoryAutoScaler + autoScalers ) ); } From 5b2e0c68641971485f58b49121ce16ef5ada6d21 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Tue, 19 Nov 2019 10:02:16 +0200 Subject: [PATCH 10/23] Support different types of MiddleManagers by Autoscaler #8695 - Fixed build --- .../CategoriedProvisioningStrategy.java | 8 +++----- .../CategoriedProvisioningStrategyTest.java | 15 +++++++-------- 2 files changed, 10 insertions(+), 13 deletions(-) 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 index 579a96dc57e2..52afe28b2e50 100644 --- 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 @@ -60,8 +60,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; -import static org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY; - @JsonTypeName("categoriedTaskBased") public class CategoriedProvisioningStrategy extends AbstractWorkerProvisioningStrategy { @@ -319,7 +317,7 @@ public boolean doProvision() task -> WorkerSelectUtils.getTaskCategory( task, workerCategorySpec, - DEFAULT_AUTOSCALER_CATEGORY + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY ) )); @@ -648,7 +646,7 @@ private boolean initAutoscaler(AutoScaler autoScaler, String category, Categorie private AutoScaler getCategoryAutoscaler(String category, Map autoscalersByCategory) { AutoScaler autoScaler = autoscalersByCategory.get(category); - boolean isStrongAssignment = !autoscalersByCategory.containsKey(DEFAULT_AUTOSCALER_CATEGORY); + boolean isStrongAssignment = !autoscalersByCategory.containsKey(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY); if (autoScaler == null && isStrongAssignment) { log.warn( @@ -657,7 +655,7 @@ private AutoScaler getCategoryAutoscaler(String category, Map mapAutoscalerByCategory(List autoScalers) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index 395f4c06bfc5..644718e3cc8d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -58,7 +58,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; -import static org.apache.druid.indexing.overlord.setup.CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY; public class CategoriedProvisioningStrategyTest { @@ -109,7 +108,7 @@ public void testDefaultAutoscalerSuccessfulInitialMinWorkers() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, CATEGORY_2,4, 6, Collections.emptyList()); autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); @@ -204,7 +203,7 @@ public void testDefaultAutoscalerSuccessfulMinWorkers() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); autoScalers.add(autoScalerDefault); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -516,7 +515,7 @@ public void testAllCategoriedAutoscalersNotStrongMode() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); + setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 3, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 3, Collections.emptyList()); autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); @@ -607,7 +606,7 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); - EasyMock.expect(autoScalerDefault.getCategory()).andReturn(DEFAULT_AUTOSCALER_CATEGORY).times(2); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY).times(2); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.emptyList()).times(2); EasyMock.expect(autoScalerDefault.terminateWithIds(EasyMock.anyObject())) @@ -729,7 +728,7 @@ public void testNullWorkerCategorySpecNotStrong() CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, DEFAULT_AUTOSCALER_CATEGORY, 1, 3, Collections.emptyList()); + setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 1, 3, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 2, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 4, Collections.emptyList()); autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); @@ -823,7 +822,7 @@ public void testDoSuccessfulTerminateForAllCategories() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, DEFAULT_AUTOSCALER_CATEGORY, 1, Collections.emptyList()); + setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 1, Collections.emptyList()); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, Collections.emptyList()); autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); @@ -887,7 +886,7 @@ public void testSomethingTerminating() CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(1); - EasyMock.expect(autoScalerDefault.getCategory()).andReturn(DEFAULT_AUTOSCALER_CATEGORY).times(2); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY).times(2); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")).times(2); EasyMock.expect(autoScalerDefault.terminate(EasyMock.anyObject())).andReturn( From 66fbec5fefb89a1e54ebd67058fdaa064b9071c4 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Tue, 19 Nov 2019 10:43:21 +0200 Subject: [PATCH 11/23] Support different types of MiddleManagers by Autoscaler #8695 - Fixed formatting --- .../CategoriedProvisioningStrategyTest.java | 59 +++++++++++++++---- 1 file changed, 49 insertions(+), 10 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index 644718e3cc8d..cc0a97db2681 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -108,9 +108,15 @@ public void testDefaultAutoscalerSuccessfulInitialMinWorkers() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); + setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 3, + 5, + Collections.emptyList() + ); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, CATEGORY_2,4, 6, Collections.emptyList()); + setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList()); autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -203,7 +209,13 @@ public void testDefaultAutoscalerSuccessfulMinWorkers() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); + setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 3, + 5, + Collections.emptyList() + ); autoScalers.add(autoScalerDefault); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -281,7 +293,7 @@ public void testCategoriedAutoscalerSpawnedMinWorkers() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, CATEGORY_1,5, 7, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 5, 7, Collections.emptyList()); autoScalers.add(autoScalerCategory1); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -515,7 +527,13 @@ public void testAllCategoriedAutoscalersNotStrongMode() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList()); + setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 3, + 5, + Collections.emptyList() + ); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 3, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 3, Collections.emptyList()); autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); @@ -606,7 +624,9 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); - EasyMock.expect(autoScalerDefault.getCategory()).andReturn(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY).times(2); + EasyMock.expect(autoScalerDefault.getCategory()) + .andReturn(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY) + .times(2); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.emptyList()).times(2); EasyMock.expect(autoScalerDefault.terminateWithIds(EasyMock.anyObject())) @@ -728,7 +748,13 @@ public void testNullWorkerCategorySpecNotStrong() CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 1, 3, Collections.emptyList()); + setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 1, + 3, + Collections.emptyList() + ); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 2, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 4, Collections.emptyList()); autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); @@ -822,7 +848,12 @@ public void testDoSuccessfulTerminateForAllCategories() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 1, Collections.emptyList()); + setupAutoscaler( + autoScalerDefault, + CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, + 1, + Collections.emptyList() + ); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, Collections.emptyList()); autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); @@ -886,7 +917,9 @@ public void testSomethingTerminating() CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(1); - EasyMock.expect(autoScalerDefault.getCategory()).andReturn(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY).times(2); + EasyMock.expect(autoScalerDefault.getCategory()) + .andReturn(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY) + .times(2); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")).times(2); EasyMock.expect(autoScalerDefault.terminate(EasyMock.anyObject())).andReturn( @@ -962,7 +995,13 @@ public void testSomethingTerminating() } - private void setupAutoscaler(AutoScaler autoScaler, String category, int minWorkers, int maxWorkers, List pendingTasks) + private void setupAutoscaler( + AutoScaler autoScaler, + String category, + int minWorkers, + int maxWorkers, + List pendingTasks + ) { setupAutoscaler(autoScaler, category, minWorkers, pendingTasks); EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(maxWorkers); From 8d94ff267ab5428663d4b8b7914e1862bbd23747 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Fri, 22 Nov 2019 17:13:47 +0200 Subject: [PATCH 12/23] Support different types of MiddleManagers by Autoscaler #8695 - Put category information into Autoscaler - Changed structure of behavior config - Changed CategoriedProvisioningStrategy accordingly - Fixed bug in PendingTaskBasedWorkerProvisioningStrategy - Extended unit tests - Code cleanup and refactoring --- .../autoscaling/ec2/EC2AutoScaler.java | 28 +-- .../overlord/autoscaling/AutoScaler.java | 6 +- .../CategoriedProvisioningStrategy.java | 22 +- .../overlord/autoscaling/NoopAutoScaler.java | 24 +- ...ngTaskBasedWorkerProvisioningStrategy.java | 4 +- .../SimpleWorkerProvisioningStrategy.java | 1 + .../CategoriedProvisioningStrategyTest.java | 229 ++++++++++++++++-- ...dingTaskBasedProvisioningStrategyTest.java | 29 +++ .../SimpleProvisioningStrategyTest.java | 33 +++ .../org/apache/druid/cli/CliOverlord.java | 2 + 10 files changed, 324 insertions(+), 54 deletions(-) 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 54abe18d506a..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 @@ -40,11 +40,11 @@ import org.apache.druid.indexing.overlord.autoscaling.AutoScaler; 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.java.util.emitter.EmittingLogger; import java.util.ArrayList; import java.util.List; +import java.util.Objects; /** */ @@ -68,7 +68,7 @@ public EC2AutoScaler( @JsonProperty("envConfig") EC2EnvironmentConfig envConfig, @JacksonInject AmazonEC2 amazonEC2Client, @JacksonInject SimpleWorkerProvisioningConfig config, - @JsonProperty(value = "category", defaultValue = CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY) String category + @JsonProperty("category") String category ) { this.minNumWorkers = minNumWorkers; @@ -94,6 +94,7 @@ public int getMaxNumWorkers() } @Override + @JsonProperty public String getCategory() { return category; @@ -341,6 +342,7 @@ public String toString() "envConfig=" + envConfig + ", maxNumWorkers=" + maxNumWorkers + ", minNumWorkers=" + minNumWorkers + + ", category=" + category + '}'; } @@ -353,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/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 6ed048a64ace..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,7 +37,10 @@ public interface AutoScaler int getMaxNumWorkers(); - String getCategory(); + 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/CategoriedProvisioningStrategy.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategy.java index 52afe28b2e50..8121bf6933ca 100644 --- 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 @@ -83,7 +83,7 @@ private static CategoriedWorkerBehaviorConfig getCategoriedWorkerBehaviorConfig( } if (!(workerBehaviorConfig instanceof CategoriedWorkerBehaviorConfig)) { log.error( - "Only DefaultWorkerBehaviorConfig is supported as WorkerBehaviorConfig, [%s] given, cannot %s workers", + "Only CategoriedWorkerBehaviorConfig is supported as WorkerBehaviorConfig, [%s] given, cannot %s workers", workerBehaviorConfig, action ); @@ -219,6 +219,11 @@ private boolean doTerminate( 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( @@ -472,9 +477,9 @@ private int getScaleUpNodeCount( 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), 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( + int moreWorkersNeeded = currValidWorkers == 0 ? Math.max(minWorkerCount, 1) : getWorkersNeededToAssignTasks( remoteTaskRunnerConfig, workerConfig, pendingTasks, @@ -655,18 +660,23 @@ private AutoScaler getCategoryAutoscaler(String category, Map mapAutoscalerByCategory(List autoScalers) { Map result = autoScalers.stream().collect(Collectors.groupingBy( - AutoScaler::getCategory, + autoScaler -> autoScaler.getCategory() == null + ? CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY + : autoScaler.getCategory(), 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 will be used."); + log.warn( + "Probably autoscalers with duplicated categories were defined. The first instance of each duplicate category will be used."); } return result; 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 6a01885c6c81..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,8 +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.indexing.overlord.setup.CategoriedWorkerBehaviorConfig; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -34,8 +34,9 @@ public class NoopAutoScaler implements AutoScaler private static final EmittingLogger log = new EmittingLogger(NoopAutoScaler.class); private final String category; + @JsonCreator public NoopAutoScaler( - @JsonProperty(value = "category", defaultValue = CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY) String category + @JsonProperty("category") String category ) { this.category = category; @@ -54,6 +55,7 @@ public int getMaxNumWorkers() } @Override + @JsonProperty public String getCategory() { return category; @@ -68,35 +70,43 @@ public Void getEnvConfig() @Override public AutoScalingData provision() { - log.info("If I were a real strategy I'd create something now in category %s", category); + 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 in category %s", ips, category); + 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..6ae9812d2551 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 @@ -246,9 +246,9 @@ private int getScaleUpNodeCount( 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), 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( + int moreWorkersNeeded = currValidWorkers == 0 ? Math.max(minWorkerCount, 1) : getWorkersNeededToAssignTasks( remoteTaskRunnerConfig, workerConfig, pendingTasks, 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..719d76a5fd8e 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 @@ -149,6 +149,7 @@ public String apply(ImmutableWorkerInfo input) updateTargetWorkerCount(workerConfig, pendingTasks, workers); int want = targetWorkerCount - (currValidWorkers + currentlyProvisioning.size()); + log.info("Want workers: %d", want); while (want > 0) { final AutoScalingData provisioned = workerConfig.getAutoScaler().provision(); final List newNodes; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index cc0a97db2681..f08d4d593fb8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -108,15 +108,11 @@ public void testDefaultAutoscalerSuccessfulInitialMinWorkers() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler( - autoScalerDefault, - CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, - 3, - 5, - Collections.emptyList() + setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 3, 5, + Collections.emptyList(), 1, 1, 1, 1 ); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList(), 1, 1, 1, 1); + setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList(), 1, 1, 1, 1); autoScalers.addAll(Arrays.asList(autoScalerDefault, autoScalerCategory1, autoScalerCategory2)); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -164,8 +160,8 @@ public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList()); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList(), 1, 1, 1, 1); + setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList(), 1, 1, 1, 1); autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -626,7 +622,7 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); EasyMock.expect(autoScalerDefault.getCategory()) .andReturn(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY) - .times(2); + .times(4); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.emptyList()).times(2); EasyMock.expect(autoScalerDefault.terminateWithIds(EasyMock.anyObject())) @@ -634,7 +630,7 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException EasyMock.expect(autoScalerCategory1.getMinNumWorkers()).andReturn(1); EasyMock.expect(autoScalerCategory1.getMaxNumWorkers()).andReturn(3); - EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(2); + EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(4); EasyMock.expect(autoScalerCategory1.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.emptyList()).times(2); EasyMock.expect(autoScalerCategory1.terminateWithIds(EasyMock.anyObject())) @@ -642,7 +638,7 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException EasyMock.expect(autoScalerCategory2.getMinNumWorkers()).andReturn(1); EasyMock.expect(autoScalerCategory2.getMaxNumWorkers()).andReturn(3); - EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(2); + EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(4); EasyMock.expect(autoScalerCategory2.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.emptyList()).times(2); EasyMock.expect(autoScalerCategory2.terminateWithIds(EasyMock.anyObject())) @@ -909,6 +905,65 @@ public void testDoSuccessfulTerminateForAllCategories() EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); } + @Test + public void testDoSuccessfulTerminateWithoutDefaultAutoscaler() + { + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, Collections.emptyList()); + 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 TestZkWorker(testTask).toImmutable(), + new TestZkWorker(testTask).toImmutable(), + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new TestZkWorker(testTask, CATEGORY_2).toImmutable(), + new TestZkWorker(testTask, CATEGORY_2).toImmutable() + ) + ); + + EasyMock.expect(runner.markWorkersLazy(EasyMock.anyObject(), EasyMock.anyInt())) + .andReturn( + Arrays.asList( + new TestZkWorker(testTask).getWorker(), + new TestZkWorker(testTask).getWorker(), + new TestZkWorker(testTask, CATEGORY_1).getWorker(), + new TestZkWorker(testTask, CATEGORY_1).getWorker(), + new TestZkWorker(testTask, CATEGORY_2).getWorker(), + new 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() { @@ -919,7 +974,7 @@ public void testSomethingTerminating() EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(1); EasyMock.expect(autoScalerDefault.getCategory()) .andReturn(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY) - .times(2); + .times(4); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")).times(2); EasyMock.expect(autoScalerDefault.terminate(EasyMock.anyObject())).andReturn( @@ -927,7 +982,7 @@ public void testSomethingTerminating() ); EasyMock.expect(autoScalerCategory1.getMinNumWorkers()).andReturn(1); - EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(2); + EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(4); EasyMock.expect(autoScalerCategory1.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")).times(2); EasyMock.expect(autoScalerCategory1.terminate(EasyMock.anyObject())).andReturn( @@ -935,7 +990,7 @@ public void testSomethingTerminating() ); EasyMock.expect(autoScalerCategory2.getMinNumWorkers()).andReturn(1); - EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(2); + EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(4); EasyMock.expect(autoScalerCategory2.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")).times(2); EasyMock.expect(autoScalerCategory2.terminate(EasyMock.anyObject())).andReturn( @@ -994,6 +1049,113 @@ public void testSomethingTerminating() EasyMock.verify(runner, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); } + @Test + public void testMinCountIncrease() + { + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + CategoriedProvisioningStrategy 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); + 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 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); + EasyMock.replay(autoScalerDefault); + + boolean provisionedSomething = provisioner.doProvision(); + Assert.assertFalse(provisionedSomething); + Assert.assertTrue(provisioner.getStats().toList().isEmpty()); + EasyMock.verify(autoScalerDefault); + + EasyMock.reset(autoScalerDefault); + // Increase minNumWorkers + EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); + 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("h3")) + ); + // Should provision two new workers + EasyMock.expect(autoScalerDefault.provision()).andReturn( + new AutoScalingData(Collections.singletonList("h4")) + ); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null); + EasyMock.replay(autoScalerDefault); + provisionedSomething = provisioner.doProvision(); + Assert.assertTrue(provisionedSomething); + EasyMock.verify(autoScalerDefault); + EasyMock.verify(runner); + } + + @Test + public void testMinCountIncreaseNoWorkers() + { + WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + CategoriedProvisioningStrategy 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); + + EasyMock.replay(autoScalerDefault); + autoScalers.add(autoScalerDefault); + + RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); + EasyMock.expect(runner.getPendingTaskPayloads()) + .andReturn(Collections.singletonList(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 void setupAutoscaler( AutoScaler autoScaler, @@ -1007,12 +1169,41 @@ private void setupAutoscaler( EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(maxWorkers); } + private 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); + } + private void setupAutoscaler(AutoScaler autoScaler, String category, int minWorkers, List pendingTasks) { - EasyMock.expect(autoScaler.getMinNumWorkers()).andReturn(minWorkers); - EasyMock.expect(autoScaler.getCategory()).andReturn(category); + setupAutoscaler(autoScaler, category, minWorkers, pendingTasks, 2, 1, 1); + } + + private 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); + .andReturn(pendingTasks).times(pendingTasksTimes); } private WorkerCategorySpec createWorkerCategorySpec(boolean isStrong) 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..5a314b66fcb6 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 @@ -516,6 +516,35 @@ public void testMinCountIncrease() EasyMock.verify(runner); } + @Test + public void testMinCountIncreaseNoWorkers() + { + EasyMock.reset(autoScaler); + // 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() { 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..8edeb2d3a05b 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 @@ -457,6 +457,39 @@ public void testMinCountIncrease() EasyMock.verify(runner); } + @Test + public void testMinCountIncreaseNoWorkers() + { + EasyMock.reset(autoScaler); + // 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.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() { 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 46e0a284f45b..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,7 @@ 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; @@ -315,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, From e12c2a6f11e07c32f830b8add3331841951b557a Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Tue, 26 Nov 2019 11:26:55 +0200 Subject: [PATCH 13/23] Support different types of MiddleManagers by Autoscaler #8695 - Code refactoring --- .../CategoriedProvisioningStrategy.java | 14 ++++---------- .../setup/CategoriedWorkerSelectStrategy.java | 6 ++++++ ...butionWithCategorySpecWorkerSelectStrategy.java | 3 ++- ...pacityWithCategorySpecWorkerSelectStrategy.java | 3 ++- 4 files changed, 14 insertions(+), 12 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/CategoriedWorkerSelectStrategy.java 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 index 8121bf6933ca..9a762829ae91 100644 --- 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 @@ -34,8 +34,7 @@ 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.EqualDistributionWithCategorySpecWorkerSelectStrategy; -import org.apache.druid.indexing.overlord.setup.FillCapacityWithCategorySpecWorkerSelectStrategy; +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; @@ -90,9 +89,7 @@ private static CategoriedWorkerBehaviorConfig getCategoriedWorkerBehaviorConfig( return null; } final CategoriedWorkerBehaviorConfig workerConfig = (CategoriedWorkerBehaviorConfig) workerBehaviorConfig; - // TODO Create a superclass - if (!((workerConfig.getSelectStrategy() instanceof FillCapacityWithCategorySpecWorkerSelectStrategy) - || (workerConfig.getSelectStrategy() instanceof EqualDistributionWithCategorySpecWorkerSelectStrategy))) { + if (!(workerConfig.getSelectStrategy() instanceof CategoriedWorkerSelectStrategy)) { log.error("Select strategy %s is not supported", workerConfig.getSelectStrategy()); return null; } @@ -687,11 +684,8 @@ private WorkerCategorySpec getWorkerCategorySpec(CategoriedWorkerBehaviorConfig { if (workerConfig != null && workerConfig.getSelectStrategy() != null) { WorkerSelectStrategy selectStrategy = workerConfig.getSelectStrategy(); - // TODO Replace by superclass - if (selectStrategy instanceof FillCapacityWithCategorySpecWorkerSelectStrategy) { - return ((FillCapacityWithCategorySpecWorkerSelectStrategy) selectStrategy).getWorkerCategorySpec(); - } else if (selectStrategy instanceof EqualDistributionWithCategorySpecWorkerSelectStrategy) { - return ((EqualDistributionWithCategorySpecWorkerSelectStrategy) selectStrategy).getWorkerCategorySpec(); + if (selectStrategy instanceof CategoriedWorkerSelectStrategy) { + return ((CategoriedWorkerSelectStrategy) selectStrategy).getWorkerCategorySpec(); } } return null; 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..53a1576567ad --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/CategoriedWorkerSelectStrategy.java @@ -0,0 +1,6 @@ +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/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; From 8bd1959e1b5cc51f2a1dc5a0ca368a3765498e82 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Wed, 27 Nov 2019 11:19:03 +0200 Subject: [PATCH 14/23] Support different types of MiddleManagers by Autoscaler #8695 - Code refactoring --- .../CategoriedProvisioningStrategy.java | 300 +++++++++--------- 1 file changed, 150 insertions(+), 150 deletions(-) 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 index 9a762829ae91..d496a2d31763 100644 --- 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 @@ -152,150 +152,7 @@ private CategoriedProvisioner(WorkerTaskRunner runner) } @Override - public 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 = zkWorkers.stream().collect(Collectors.groupingBy( - immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) - ); - - Set allCategories = workersByCategories.keySet(); - log.debug( - "Workers of %d categories: %s", - workersByCategories.size(), - allCategories - ); - - Map autoscalersByCategory = 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 = getCategoryAutoscaler(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 boolean doProvision() + public synchronized boolean doProvision() { Collection pendingTasks = runner.getPendingTaskPayloads(); log.debug("Pending tasks: %d %s", pendingTasks.size(), pendingTasks); @@ -444,12 +301,6 @@ private boolean doProvision( return didProvision; } - @Override - public ScalingStats getStats() - { - return scalingStats; - } - private Collection getWorkerNodeIDs(Collection workers, AutoScaler autoScaler) { List ips = new ArrayList<>(workers.size()); @@ -556,6 +407,155 @@ private int getWorkersNeededToAssignTasks( 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 = zkWorkers.stream().collect(Collectors.groupingBy( + immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) + ); + + Set allCategories = workersByCategories.keySet(); + log.debug( + "Workers of %d categories: %s", + workersByCategories.size(), + allCategories + ); + + Map autoscalersByCategory = 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 = getCategoryAutoscaler(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); From d530f03a9d0affc74279f0a9ca1290fdfb9a6783 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Mon, 2 Dec 2019 19:27:21 +0200 Subject: [PATCH 15/23] Support different types of MiddleManagers by Autoscaler #8695 - Introduced new logic into the existing strategies --- ...ngTaskBasedWorkerProvisioningStrategy.java | 321 +++++++++++++----- .../autoscaling/ProvisioningUtil.java | 110 ++++++ .../SimpleWorkerProvisioningStrategy.java | 302 +++++++++++++--- .../CategoriedProvisioningStrategyTest.java | 104 ++---- ...dingTaskBasedProvisioningStrategyTest.java | 17 + .../SimpleProvisioningStrategyTest.java | 55 ++- 6 files changed, 686 insertions(+), 223 deletions(-) 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 6ae9812d2551..d04dc0846978 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> currentlyProvisioning = new HashMap<>(); + private final Map> currentlyTerminating = new HashMap<>(); - private DateTime lastProvisionTime = DateTimes.nowUtc(); - private DateTime lastTerminateTime = lastProvisionTime; + private final Map lastProvisionTimeMap = new HashMap<>();//DateTimes.nowUtc(); + private final Map lastTerminateTimeMap = new HashMap<>();//lastProvisionTime; private PendingProvisioner(WorkerTaskRunner runner) { @@ -157,24 +125,99 @@ 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 = workers.stream().collect(Collectors.groupingBy( + immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) + ); + + // 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 = ProvisioningUtil.getAutoscalerCategory(autoScaler); + didProvision = initAutoscaler(autoScaler, category, workerConfig, currentlyProvisioning) || didProvision; + } + return didProvision; + } + + 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); + return false; + } + // Correct category name by selected autoscaler + category = ProvisioningUtil.getAutoscalerCategory(categoryAutoscaler); + + 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); + + 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 +226,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 +245,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 +261,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,26 +269,27 @@ 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); @@ -278,7 +325,7 @@ private int getScaleUpNodeCount( private int getWorkersNeededToAssignTasks( final WorkerTaskRunnerConfig workerTaskRunnerConfig, - final DefaultWorkerBehaviorConfig workerConfig, + final CategoriedWorkerBehaviorConfig workerConfig, final Collection pendingTasks, final Collection workers ) @@ -333,67 +380,135 @@ 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 = zkWorkers.stream().collect(Collectors.groupingBy( + immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) + ); + + 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; + } + + AutoScaler categoryAutoscaler = ProvisioningUtil.getAutoscalerByCategory(category, autoscalersByCategory); + if (categoryAutoscaler == null) { + log.error("No autoScaler available, cannot execute doTerminate for workers of category %s", category); + return false; + } + // Correct category name by selected autoscaler + category = ProvisioningUtil.getAutoscalerCategory(categoryAutoscaler); + + List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); + currentlyTerminating.putIfAbsent(category, new HashSet<>()); + Set currentlyTerminating = this.currentlyTerminating.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 +523,28 @@ public String apply(Worker zkWorker) return didTerminate; } + 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 +552,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..ed456e8f6718 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,29 @@ 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.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 +79,96 @@ 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); + + 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(); + } } 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 719d76a5fd8e..8b9ea6ed7dfe 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,63 +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> currentlyProvisioning = new HashMap<>(); + private final Map> currentlyTerminating = 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(); Collection workers = runner.getWorkers(); 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 = workers.stream().collect(Collectors.groupingBy( + immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) + ); + + // 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()); + currentlyProvisioning.putIfAbsent(category, new HashSet<>()); + Set currentlyProvisioning = this.currentlyProvisioning.get(category); + currentlyTerminating.putIfAbsent(category, new HashSet<>()); + Set currentlyTerminating = this.currentlyTerminating.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; @@ -166,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())) { @@ -174,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(); } } @@ -185,27 +286,92 @@ 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 = workers.stream().collect(Collectors.groupingBy( + immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) + ); + + 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()); + currentlyProvisioning.putIfAbsent(category, new HashSet<>()); + Set currentlyProvisioning = this.currentlyProvisioning.get(category); + currentlyTerminating.putIfAbsent(category, new HashSet<>()); + Set currentlyTerminating = this.currentlyTerminating.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 ) ) ) @@ -213,8 +379,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()) { @@ -243,17 +409,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); @@ -273,9 +439,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( @@ -283,14 +452,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( @@ -347,6 +518,8 @@ private void updateTargetWorkerCount( maxWorkerCount ); } + + targetWorkerCountMap.put(category, targetWorkerCount); } private boolean hasTaskPendingBeyondThreshold(Collection pendingTasks) @@ -367,6 +540,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) + { + currentlyProvisioning.putIfAbsent(category, new HashSet<>()); + Set currentlyProvisioning = this.currentlyProvisioning.get(category); + currentlyTerminating.putIfAbsent(category, new HashSet<>()); + Set currentlyTerminating = this.currentlyTerminating.get(category); + return doProvision( + category, + Collections.emptyList(), + Collections.emptyList(), + currentlyProvisioning, + currentlyTerminating, + autoScaler + ); + } + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index f08d4d593fb8..91be206f8d1e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -76,7 +76,7 @@ public class CategoriedProvisioningStrategyTest private final List autoScalers = new ArrayList<>(); private final List autoScalersStrong = new ArrayList<>(); private Task testTask; - private CategoriedProvisioningConfig config; + private PendingTaskBasedWorkerProvisioningConfig config; private final ScheduledExecutorService executorService = Execs.scheduledSingleThreaded("test service"); private static final String MIN_VERSION = "2014-01-00T00:01:00Z"; @@ -92,7 +92,7 @@ public void setup() testTask = TestTasks.immediateSuccess("task1"); - config = new CategoriedProvisioningConfig() + config = new PendingTaskBasedWorkerProvisioningConfig() .setMaxScalingDuration(new Period(1000)) .setNumEventsToTrack(10) .setPendingTaskTimeout(new Period(0)) @@ -100,58 +100,6 @@ public void setup() .setMaxScalingStep(2); } - @Test - public void testDefaultAutoscalerSuccessfulInitialMinWorkers() - { - // Not strong affinity autoscaling mode will use default autoscaler - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 3, 5, - Collections.emptyList(), 1, 1, 1, 1 - ); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList(), 1, 1, 1, 1); - setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList(), 1, 1, 1, 1); - autoScalers.addAll(Arrays.asList(autoScalerDefault, 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()); - - 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 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, autoScalerDefault, autoScalerCategory1, autoScalerCategory2); - - Provisioner provisioner = strategy.makeProvisioner(runner); - boolean provisionedSomething = provisioner.doProvision(); - Assert.assertTrue(provisionedSomething); - // In total expect provisioning of 2 + 3 + 4 = 9 workers - Assert.assertEquals(9, 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 testDefaultAutoscalerDidntSpawnInitialMinWorkers() { @@ -159,7 +107,7 @@ public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList(), 1, 1, 1, 1); setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList(), 1, 1, 1, 1); autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); @@ -204,7 +152,7 @@ public void testDefaultAutoscalerSuccessfulMinWorkers() // Not strong affinity autoscaling mode will use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler( autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, @@ -252,7 +200,7 @@ public void testAnyAutoscalerDontSpawnMinWorkers() // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // No pending tasks @@ -288,7 +236,7 @@ public void testCategoriedAutoscalerSpawnedMinWorkers() // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 5, 7, Collections.emptyList()); autoScalers.add(autoScalerCategory1); @@ -333,7 +281,7 @@ public void testCategoriedAutoscalerSpawnedAdditionalWorker() // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 3, Collections.emptyList()); autoScalers.add(autoScalerCategory1); @@ -380,7 +328,7 @@ public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 3, Collections.emptyList()); autoScalers.add(autoScalerCategory1); @@ -446,7 +394,7 @@ public void testAllCategoriedAutoscalersStrongly() // Strong affinity autoscaling mode will not use default autoscaler AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 3, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 3, Collections.emptyList()); autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); @@ -522,7 +470,7 @@ public void testAllCategoriedAutoscalersNotStrongMode() ); AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler( autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, @@ -616,12 +564,12 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException ); AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(3); EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(5); EasyMock.expect(autoScalerDefault.getCategory()) - .andReturn(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY) + .andReturn(null) .times(4); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.emptyList()).times(2); @@ -711,7 +659,7 @@ public void testNullWorkerConfig() { AtomicReference workerConfig = new AtomicReference<>(null); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // One pending task @@ -742,7 +690,7 @@ public void testNullWorkerCategorySpecNotStrong() WorkerCategorySpec workerCategorySpec = null; AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler( autoScalerDefault, @@ -799,7 +747,7 @@ public void testNullWorkerCategorySpecStrong() WorkerCategorySpec workerCategorySpec = null; AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 2, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 4, Collections.emptyList()); @@ -842,7 +790,7 @@ public void testDoSuccessfulTerminateForAllCategories() { WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler( autoScalerDefault, @@ -910,7 +858,7 @@ public void testDoSuccessfulTerminateWithoutDefaultAutoscaler() { WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, Collections.emptyList()); setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, Collections.emptyList()); @@ -969,11 +917,11 @@ public void testSomethingTerminating() { WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(1); EasyMock.expect(autoScalerDefault.getCategory()) - .andReturn(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY) + .andReturn(null) .times(4); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")).times(2); @@ -1054,13 +1002,13 @@ public void testMinCountIncrease() { WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + 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); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); EasyMock.replay(autoScalerDefault); autoScalers.add(autoScalerDefault); @@ -1091,7 +1039,7 @@ public void testMinCountIncrease() EasyMock.expect(autoScalerDefault.getMaxNumWorkers()).andReturn(2); EasyMock.expect(autoScalerDefault.ipToIdLookup(EasyMock.anyObject())) .andReturn(Collections.singletonList("ip")); - EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); EasyMock.replay(autoScalerDefault); boolean provisionedSomething = provisioner.doProvision(); @@ -1125,7 +1073,7 @@ public void testMinCountIncreaseNoWorkers() { WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); - CategoriedProvisioningStrategy strategy = createStrategy(workerConfig); + 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); @@ -1187,7 +1135,7 @@ private void setupAutoscaler( private void setupAutoscaler(AutoScaler autoScaler, String category, int minWorkers, List pendingTasks) { - setupAutoscaler(autoScaler, category, minWorkers, pendingTasks, 2, 1, 1); + setupAutoscaler(autoScaler, category, minWorkers, pendingTasks, 4, 1, 1); } private void setupAutoscaler( @@ -1255,11 +1203,11 @@ private AtomicReference createWorkerConfigRef(WorkerCatego ); } - private CategoriedProvisioningStrategy createStrategy( + private PendingTaskBasedWorkerProvisioningStrategy createStrategy( AtomicReference workerConfigRef ) { - return new CategoriedProvisioningStrategy( + return new PendingTaskBasedWorkerProvisioningStrategy( config, DSuppliers.of(workerConfigRef), new ProvisioningSchedulerConfig(), 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 5a314b66fcb6..925b338b9634 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,9 +31,11 @@ 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; +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; @@ -54,6 +56,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; @@ -62,6 +65,7 @@ */ public class PendingTaskBasedProvisioningStrategyTest { + public static final String DEFAULT_CATEGORY = CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY; private AutoScaler autoScaler; private Task testTask; private PendingTaskBasedWorkerProvisioningStrategy strategy; @@ -109,6 +113,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 +145,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 +180,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 +216,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 +274,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 +332,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 +377,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 +422,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 +454,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 +473,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 +501,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 +512,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); @@ -520,6 +536,7 @@ public void testMinCountIncrease() 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); 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 8edeb2d3a05b..81885bcae2b8 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; @@ -61,6 +62,7 @@ */ public class SimpleProvisioningStrategyTest { + public static final String DEFAULT_CATEGORY = CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY; private AutoScaler autoScaler; private Task testTask; private SimpleWorkerProvisioningStrategy strategy; @@ -113,6 +115,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 +130,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 +159,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 +174,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 +221,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 +239,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 +282,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 +298,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 +328,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 +345,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 +385,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 +399,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 +422,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 +441,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 +451,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 +473,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 +485,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())) @@ -461,6 +508,7 @@ public void testMinCountIncrease() 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); @@ -478,6 +526,11 @@ public void testMinCountIncreaseNoWorkers() .withQueueInsertionTime(DateTimes.nowUtc()) ) ); + EasyMock.expect(runner.getPendingTaskPayloads()).andReturn( + Collections.singletonList( + testTask + ) + ); EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); EasyMock.replay(runner); @@ -507,7 +560,7 @@ public void testNullWorkerConfig() Collections.singletonList( new TestZkWorker(null).toImmutable() ) - ).times(1); + ).times(2); EasyMock.replay(runner); Provisioner provisioner = strategy.makeProvisioner(runner); From 16deb7b349809c5e25edd09e714532aa8977a49f Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Tue, 3 Dec 2019 11:26:45 +0200 Subject: [PATCH 16/23] Support different types of MiddleManagers by Autoscaler #8695 - Fixes and refactoring --- ...ngTaskBasedWorkerProvisioningStrategy.java | 56 +++++++++++-------- .../SimpleWorkerProvisioningStrategy.java | 28 +++++----- .../CategoriedProvisioningStrategyTest.java | 25 ++++----- 3 files changed, 59 insertions(+), 50 deletions(-) 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 d04dc0846978..b1d10bac36f5 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 @@ -106,8 +106,8 @@ private class PendingProvisioner 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 final Map> currentlyProvisioningMap = new HashMap<>(); + private final Map> currentlyTerminatingMap = new HashMap<>(); private final Map lastProvisionTimeMap = new HashMap<>();//DateTimes.nowUtc(); private final Map lastTerminateTimeMap = new HashMap<>();//lastProvisionTime; @@ -164,12 +164,9 @@ public synchronized boolean doProvision() ); 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 = ProvisioningUtil.getAutoscalerCategory(autoScaler); - didProvision = initAutoscaler(autoScaler, category, workerConfig, currentlyProvisioning) || didProvision; - } - return didProvision; + // 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()); @@ -178,15 +175,15 @@ public synchronized boolean doProvision() AutoScaler categoryAutoscaler = ProvisioningUtil.getAutoscalerByCategory(category, autoscalersByCategory); if (categoryAutoscaler == null) { log.error("No autoScaler available, cannot execute doProvision for workers of category %s", category); - return false; + 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()); - currentlyProvisioning.putIfAbsent(category, new HashSet<>()); - Set currentlyProvisioning = this.currentlyProvisioning.get(category); + currentlyProvisioningMap.putIfAbsent(category, new HashSet<>()); + Set currentlyProvisioning = this.currentlyProvisioningMap.get(category); didProvision = doProvision( category, @@ -293,14 +290,16 @@ private int getScaleUpNodeCount( 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 + // 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 ? Math.max(minWorkerCount, 1) : 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( @@ -405,7 +404,10 @@ public synchronized boolean doTerminate() Map autoscalersByCategory = ProvisioningUtil.mapAutoscalerByCategory(workerConfig.getAutoScalers()); for (String category : allCategories) { - Set currentlyProvisioning = this.currentlyProvisioning.getOrDefault(category, Collections.emptySet()); + Set currentlyProvisioning = this.currentlyProvisioningMap.getOrDefault( + category, + Collections.emptySet() + ); log.info( "Currently provisioning of category %s: %d %s", category, @@ -420,14 +422,14 @@ public synchronized boolean doTerminate() AutoScaler categoryAutoscaler = ProvisioningUtil.getAutoscalerByCategory(category, autoscalersByCategory); if (categoryAutoscaler == null) { log.error("No autoScaler available, cannot execute doTerminate for workers of category %s", category); - return false; + continue; } // Correct category name by selected autoscaler category = ProvisioningUtil.getAutoscalerCategory(categoryAutoscaler); List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); - currentlyTerminating.putIfAbsent(category, new HashSet<>()); - Set currentlyTerminating = this.currentlyTerminating.get(category); + currentlyTerminatingMap.putIfAbsent(category, new HashSet<>()); + Set currentlyTerminating = this.currentlyTerminatingMap.get(category); didTerminate = doTerminate( category, @@ -523,6 +525,16 @@ private boolean doTerminate( 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, 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 8b9ea6ed7dfe..20ba1f8992bb 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 @@ -103,8 +103,8 @@ private class SimpleProvisioner 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 final Map> currentlyProvisioningMap = new HashMap<>(); + private final Map> currentlyTerminatingMap = new HashMap<>(); private final Map targetWorkerCountMap = new HashMap<>(); private final Map lastProvisionTimeMap = new HashMap<>(); @@ -203,10 +203,10 @@ public synchronized boolean doProvision() category = ProvisioningUtil.getAutoscalerCategory(categoryAutoscaler); List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); - currentlyProvisioning.putIfAbsent(category, new HashSet<>()); - Set currentlyProvisioning = this.currentlyProvisioning.get(category); - currentlyTerminating.putIfAbsent(category, new HashSet<>()); - Set currentlyTerminating = this.currentlyTerminating.get(category); + 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, @@ -332,10 +332,10 @@ public synchronized boolean doTerminate() // Correct category name by selected autoscaler category = ProvisioningUtil.getAutoscalerCategory(categoryAutoscaler); List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); - currentlyProvisioning.putIfAbsent(category, new HashSet<>()); - Set currentlyProvisioning = this.currentlyProvisioning.get(category); - currentlyTerminating.putIfAbsent(category, new HashSet<>()); - Set currentlyTerminating = this.currentlyTerminating.get(category); + 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() @@ -553,10 +553,10 @@ private boolean initAutoscalers(CategoriedWorkerBehaviorConfig workerConfig) private boolean initAutoscaler(AutoScaler autoScaler, String category) { - currentlyProvisioning.putIfAbsent(category, new HashSet<>()); - Set currentlyProvisioning = this.currentlyProvisioning.get(category); - currentlyTerminating.putIfAbsent(category, new HashSet<>()); - Set currentlyTerminating = this.currentlyTerminating.get(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(), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java index 91be206f8d1e..13f145b69beb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java @@ -108,8 +108,8 @@ public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList(), 1, 1, 1, 1); - setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList(), 1, 1, 1, 1); + setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList(), 2, 1, 1, 1); + setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList(), 2, 1, 1, 1); autoScalers.addAll(Arrays.asList(autoScalerCategory1, autoScalerCategory2)); RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); @@ -578,7 +578,7 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException EasyMock.expect(autoScalerCategory1.getMinNumWorkers()).andReturn(1); EasyMock.expect(autoScalerCategory1.getMaxNumWorkers()).andReturn(3); - EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(4); + 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())) @@ -586,7 +586,7 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException EasyMock.expect(autoScalerCategory2.getMinNumWorkers()).andReturn(1); EasyMock.expect(autoScalerCategory2.getMaxNumWorkers()).andReturn(3); - EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(4); + 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())) @@ -930,7 +930,7 @@ public void testSomethingTerminating() ); EasyMock.expect(autoScalerCategory1.getMinNumWorkers()).andReturn(1); - EasyMock.expect(autoScalerCategory1.getCategory()).andReturn(CATEGORY_1).times(4); + 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( @@ -938,7 +938,7 @@ public void testSomethingTerminating() ); EasyMock.expect(autoScalerCategory2.getMinNumWorkers()).andReturn(1); - EasyMock.expect(autoScalerCategory2.getCategory()).andReturn(CATEGORY_2).times(4); + 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( @@ -1048,19 +1048,16 @@ public void testMinCountIncrease() EasyMock.verify(autoScalerDefault); EasyMock.reset(autoScalerDefault); - // Increase minNumWorkers + // 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")); - EasyMock.expect(autoScalerDefault.provision()).andReturn( - new AutoScalingData(Collections.singletonList("h3")) - ); // Should provision two new workers EasyMock.expect(autoScalerDefault.provision()).andReturn( - new AutoScalingData(Collections.singletonList("h4")) - ); - EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null); + new AutoScalingData(Collections.singletonList("h3")) + ).times(3); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); EasyMock.replay(autoScalerDefault); provisionedSomething = provisioner.doProvision(); Assert.assertTrue(provisionedSomething); @@ -1083,7 +1080,7 @@ public void testMinCountIncreaseNoWorkers() EasyMock.expect(autoScalerDefault.provision()).andReturn( new AutoScalingData(Collections.singletonList("aNode")) ); - EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null); + EasyMock.expect(autoScalerDefault.getCategory()).andReturn(null).times(2); EasyMock.replay(autoScalerDefault); autoScalers.add(autoScalerDefault); From c5f1f9393c748704f6987c6a4c88416b6ef80ef1 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Tue, 3 Dec 2019 17:45:04 +0200 Subject: [PATCH 17/23] Support different types of MiddleManagers by Autoscaler #8695 - Added more test cases - Refactored unit tests --- .../SimpleWorkerProvisioningStrategy.java | 2 + ...dingTaskBasedProvisioningStrategyTest.java | 9 +- ...asedProvisioningStrategyTestExtended.java} | 522 +++---- .../SimpleProvisioningStrategyTest.java | 1 + ...impleProvisioningStrategyTestExtended.java | 1220 +++++++++++++++++ .../autoscaling/StrategyTestUtils.java | 215 +++ 6 files changed, 1640 insertions(+), 329 deletions(-) rename indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/{CategoriedProvisioningStrategyTest.java => PendingTaskBasedProvisioningStrategyTestExtended.java} (70%) create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTestExtended.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/StrategyTestUtils.java 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 20ba1f8992bb..ad93be92f4a7 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 @@ -142,7 +142,9 @@ private Map> groupTasksByCategories( 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 CategoriedWorkerBehaviorConfig workerConfig = ProvisioningUtil.getCategoriedWorkerBehaviorConfig( workerConfigRef, 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 925b338b9634..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 @@ -35,7 +35,6 @@ import org.apache.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; import org.apache.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy; 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; @@ -56,12 +55,12 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; /** + * */ public class PendingTaskBasedProvisioningStrategyTest { @@ -624,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/CategoriedProvisioningStrategyTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTestExtended.java similarity index 70% rename from indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTestExtended.java index 13f145b69beb..236371259c50 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/CategoriedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTestExtended.java @@ -21,23 +21,14 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.common.guava.DSuppliers; -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.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.ZkWorker; import org.apache.druid.indexing.overlord.config.RemoteTaskRunnerConfig; 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.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -45,6 +36,7 @@ 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; @@ -52,14 +44,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; -public class CategoriedProvisioningStrategyTest +public class PendingTaskBasedProvisioningStrategyTestExtended { private static final String DEFAULT_CATEGORY_1 = "default_category1"; private static final String DEFAULT_CATEGORY_2 = "default_category2"; @@ -74,7 +64,6 @@ public class CategoriedProvisioningStrategyTest private AutoScaler autoScalerCategory1; private AutoScaler autoScalerCategory2; private final List autoScalers = new ArrayList<>(); - private final List autoScalersStrong = new ArrayList<>(); private Task testTask; private PendingTaskBasedWorkerProvisioningConfig config; private final ScheduledExecutorService executorService = Execs.scheduledSingleThreaded("test service"); @@ -100,16 +89,25 @@ public void setup() .setMaxScalingStep(2); } + @After + public void tearDown() + { + executorService.shutdownNow(); + } + @Test public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 4, Collections.emptyList(), 2, 1, 1, 1); - setupAutoscaler(autoScalerCategory2, CATEGORY_2, 4, 6, Collections.emptyList(), 2, 1, 1, 1); + 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); @@ -148,12 +146,15 @@ public void testDefaultAutoscalerDidntSpawnInitialMinWorkers() @Test public void testDefaultAutoscalerSuccessfulMinWorkers() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); // Not strong affinity autoscaling mode will use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler( + StrategyTestUtils.setupAutoscaler( autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 3, @@ -168,7 +169,7 @@ public void testDefaultAutoscalerSuccessfulMinWorkers() // 1 node already running, only provision 2 more. EasyMock.expect(runner.getWorkers()).andReturn( Collections.singletonList( - new TestZkWorker(testTask).toImmutable() + new StrategyTestUtils.TestZkWorker(testTask).toImmutable() ) ); @@ -196,9 +197,12 @@ public void testDefaultAutoscalerSuccessfulMinWorkers() @Test public void testAnyAutoscalerDontSpawnMinWorkers() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); @@ -209,7 +213,7 @@ public void testAnyAutoscalerDontSpawnMinWorkers() // 1 worker already running. That means no initialization is required. EasyMock.expect(runner.getWorkers()).andReturn( Collections.singletonList( - new TestZkWorker(testTask).toImmutable() + new StrategyTestUtils.TestZkWorker(testTask).toImmutable() ) ); @@ -226,7 +230,7 @@ public void testAnyAutoscalerDontSpawnMinWorkers() @Test public void testCategoriedAutoscalerSpawnedMinWorkers() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( false, TASK_TYPE_1, CATEGORY_1, @@ -234,16 +238,19 @@ public void testCategoriedAutoscalerSpawnedMinWorkers() CATEGORY_2 ); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 5, 7, Collections.emptyList()); + 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(TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); + .andReturn(Collections.singletonList(StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); // No workers EasyMock.expect(runner.getWorkers()).andReturn(Collections.emptyList()); @@ -271,7 +278,7 @@ public void testCategoriedAutoscalerSpawnedMinWorkers() @Test public void testCategoriedAutoscalerSpawnedAdditionalWorker() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( false, TASK_TYPE_1, CATEGORY_1, @@ -279,21 +286,24 @@ public void testCategoriedAutoscalerSpawnedAdditionalWorker() CATEGORY_2 ); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 3, Collections.emptyList()); + 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(TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); + .andReturn(Collections.singletonList(StrategyTestUtils.TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); // Min workers are running EasyMock.expect(runner.getWorkers()).andReturn( Arrays.asList( - new TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_1).toImmutable() + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable() ) ); @@ -318,7 +328,7 @@ public void testCategoriedAutoscalerSpawnedAdditionalWorker() @Test public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( false, TASK_TYPE_1, CATEGORY_1, @@ -326,24 +336,27 @@ public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() CATEGORY_2 ); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 2, 3, Collections.emptyList()); + 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( - TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), - TestTask.create(TASK_TYPE_1, DATA_SOURCE_2) + 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 TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_1).toImmutable() + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable() ) ); @@ -368,7 +381,7 @@ public void testCategoriedAutoscalerSpawnedUpToMaxWorkers() @Test public void testAllCategoriedAutoscalersStrongly() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( false, TASK_TYPE_1, new WorkerCategorySpec.CategoryConfig( @@ -392,27 +405,30 @@ public void testAllCategoriedAutoscalersStrongly() ) ); // Strong affinity autoscaling mode will not use default autoscaler - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 3, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 3, Collections.emptyList()); + 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( - TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), - TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), - TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), - TestTask.create(TASK_TYPE_3, DATA_SOURCE_2) + 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 TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable() + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() ) ); @@ -445,7 +461,7 @@ public void testAllCategoriedAutoscalersStrongly() @Test public void testAllCategoriedAutoscalersNotStrongMode() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( false, TASK_TYPE_1, new WorkerCategorySpec.CategoryConfig( @@ -468,34 +484,37 @@ public void testAllCategoriedAutoscalersNotStrongMode() ) ) ); - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler( + StrategyTestUtils.setupAutoscaler( autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 3, 5, Collections.emptyList() ); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 3, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 3, 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( - TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), - TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), - TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), - TestTask.create(TASK_TYPE_3, DATA_SOURCE_2) + 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 TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable() + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() ) ); @@ -539,7 +558,7 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException EasyMock.expectLastCall().times(3); EasyMock.replay(emitter); - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec( + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec( false, TASK_TYPE_1, new WorkerCategorySpec.CategoryConfig( @@ -562,7 +581,10 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException ) ) ); - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); @@ -598,16 +620,16 @@ public void testAllCategoriedAutoscalersAlert() throws InterruptedException // Four pending tasks: three have their categorized autoscalers and one for default autoscaler EasyMock.expect(runner.getPendingTaskPayloads()) .andReturn(Arrays.asList( - TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), - TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), - TestTask.create(TASK_TYPE_1, DATA_SOURCE_2), - TestTask.create(TASK_TYPE_3, DATA_SOURCE_2) + 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 TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable() + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() ) ).times(2); @@ -664,12 +686,12 @@ public void testNullWorkerConfig() RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); // One pending task EasyMock.expect(runner.getPendingTaskPayloads()) - .andReturn(Collections.singletonList(TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); + .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 TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable() + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() ) ); @@ -688,19 +710,22 @@ public void testNullWorkerConfig() public void testNullWorkerCategorySpecNotStrong() { WorkerCategorySpec workerCategorySpec = null; - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler( + StrategyTestUtils.setupAutoscaler( autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 1, 3, Collections.emptyList() ); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 2, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 4, 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 @@ -712,16 +737,16 @@ public void testNullWorkerCategorySpecNotStrong() // Three pending tasks EasyMock.expect(runner.getPendingTaskPayloads()) .andReturn(Arrays.asList( - TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), - TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), - TestTask.create(TASK_TYPE_1, DATA_SOURCE_2) + 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 TestZkWorker(testTask).toImmutable(), - new TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable() + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() ) ); @@ -745,12 +770,15 @@ public void testNullWorkerCategorySpecNotStrong() public void testNullWorkerCategorySpecStrong() { WorkerCategorySpec workerCategorySpec = null; - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, 2, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, 4, Collections.emptyList()); + 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); @@ -758,17 +786,17 @@ public void testNullWorkerCategorySpecStrong() // Three pending tasks EasyMock.expect(runner.getPendingTaskPayloads()) .andReturn(Arrays.asList( - TestTask.create(TASK_TYPE_1, DATA_SOURCE_1), - TestTask.create(TASK_TYPE_2, DATA_SOURCE_2), - TestTask.create(TASK_TYPE_1, DATA_SOURCE_2) + 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 TestZkWorker(testTask).toImmutable(), - new TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable() + new StrategyTestUtils.TestZkWorker(testTask).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_1).toImmutable(), + new StrategyTestUtils.TestZkWorker(testTask, CATEGORY_2).toImmutable() ) ); @@ -788,18 +816,21 @@ public void testNullWorkerCategorySpecStrong() @Test public void testDoSuccessfulTerminateForAllCategories() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler( + StrategyTestUtils.setupAutoscaler( autoScalerDefault, CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY, 1, Collections.emptyList() ); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, CATEGORY_2, 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( @@ -817,24 +848,24 @@ public void testDoSuccessfulTerminateForAllCategories() EasyMock.expect(runner.getWorkers()).andReturn( Arrays.asList( - new TestZkWorker(testTask).toImmutable(), - new TestZkWorker(testTask).toImmutable(), - new TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable() + 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 TestZkWorker(testTask).getWorker(), - new TestZkWorker(testTask).getWorker(), - new TestZkWorker(testTask, CATEGORY_1).getWorker(), - new TestZkWorker(testTask, CATEGORY_1).getWorker(), - new TestZkWorker(testTask, CATEGORY_2).getWorker(), - new TestZkWorker(testTask, CATEGORY_2).getWorker() + 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); @@ -856,12 +887,15 @@ public void testDoSuccessfulTerminateForAllCategories() @Test public void testDoSuccessfulTerminateWithoutDefaultAutoscaler() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); - setupAutoscaler(autoScalerCategory1, CATEGORY_1, 1, Collections.emptyList()); - setupAutoscaler(autoScalerCategory2, CATEGORY_2, 1, Collections.emptyList()); + 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( @@ -876,24 +910,24 @@ public void testDoSuccessfulTerminateWithoutDefaultAutoscaler() EasyMock.expect(runner.getWorkers()).andReturn( Arrays.asList( - new TestZkWorker(testTask).toImmutable(), - new TestZkWorker(testTask).toImmutable(), - new TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable() + 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 TestZkWorker(testTask).getWorker(), - new TestZkWorker(testTask).getWorker(), - new TestZkWorker(testTask, CATEGORY_1).getWorker(), - new TestZkWorker(testTask, CATEGORY_1).getWorker(), - new TestZkWorker(testTask, CATEGORY_2).getWorker(), - new TestZkWorker(testTask, CATEGORY_2).getWorker() + 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); @@ -915,8 +949,11 @@ public void testDoSuccessfulTerminateWithoutDefaultAutoscaler() @Test public void testSomethingTerminating() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); EasyMock.expect(autoScalerDefault.getMinNumWorkers()).andReturn(1); @@ -953,24 +990,24 @@ public void testSomethingTerminating() EasyMock.expect(runner.getWorkers()).andReturn( Arrays.asList( - new TestZkWorker(testTask).toImmutable(), - new TestZkWorker(testTask).toImmutable(), - new TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_1).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable(), - new TestZkWorker(testTask, CATEGORY_2).toImmutable() + 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 TestZkWorker(testTask).getWorker(), - new TestZkWorker(testTask).getWorker(), - new TestZkWorker(testTask, CATEGORY_1).getWorker(), - new TestZkWorker(testTask, CATEGORY_1).getWorker(), - new TestZkWorker(testTask, CATEGORY_2).getWorker(), - new TestZkWorker(testTask, CATEGORY_2).getWorker() + 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); @@ -1000,8 +1037,11 @@ public void testSomethingTerminating() @Test public void testMinCountIncrease() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + WorkerCategorySpec workerCategorySpec = StrategyTestUtils.createWorkerCategorySpec(false); + AtomicReference workerConfig = StrategyTestUtils.createWorkerConfigRef( + workerCategorySpec, + autoScalers + ); PendingTaskBasedWorkerProvisioningStrategy strategy = createStrategy(workerConfig); // Don't terminate anything @@ -1018,7 +1058,15 @@ public void testMinCountIncrease() ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( Collections.singletonList( - new TestZkWorker(NoopTask.create(), "http", "h1", "i1", MIN_VERSION, 1, DEFAULT_CATEGORY_1).toImmutable() + 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); @@ -1068,8 +1116,11 @@ public void testMinCountIncrease() @Test public void testMinCountIncreaseNoWorkers() { - WorkerCategorySpec workerCategorySpec = createWorkerCategorySpec(false); - AtomicReference workerConfig = createWorkerConfigRef(workerCategorySpec); + 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 @@ -1087,7 +1138,7 @@ public void testMinCountIncreaseNoWorkers() RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); EasyMock.expect(runner.getPendingTaskPayloads()) - .andReturn(Collections.singletonList(TestTask.create(TASK_TYPE_1, DATA_SOURCE_2))); + .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()); @@ -1102,104 +1153,6 @@ public void testMinCountIncreaseNoWorkers() EasyMock.verify(autoScalerDefault, runner); } - private void setupAutoscaler( - AutoScaler autoScaler, - String category, - int minWorkers, - int maxWorkers, - List pendingTasks - ) - { - setupAutoscaler(autoScaler, category, minWorkers, pendingTasks); - EasyMock.expect(autoScaler.getMaxNumWorkers()).andReturn(maxWorkers); - } - - private 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); - } - - private void setupAutoscaler(AutoScaler autoScaler, String category, int minWorkers, List pendingTasks) - { - setupAutoscaler(autoScaler, category, minWorkers, pendingTasks, 4, 1, 1); - } - - private 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); - } - - private WorkerCategorySpec createWorkerCategorySpec(boolean isStrong) - { - Map categoryMap = new HashMap<>(); - return new WorkerCategorySpec(categoryMap, isStrong); - } - - private 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); - } - - private 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); - } - - private AtomicReference createWorkerConfigRef(WorkerCategorySpec workerCategorySpec) - { - return new AtomicReference<>( - new CategoriedWorkerBehaviorConfig( - new FillCapacityWithCategorySpecWorkerSelectStrategy(workerCategorySpec), - autoScalers - ) - ); - } - private PendingTaskBasedWorkerProvisioningStrategy createStrategy( AtomicReference workerConfigRef ) @@ -1211,87 +1164,4 @@ private PendingTaskBasedWorkerProvisioningStrategy createStrategy( () -> executorService ); } - - 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() - ) - ); - } - } - - private 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); - } - - @Override - public String getType() - { - return type; - } - } } 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 81885bcae2b8..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 @@ -59,6 +59,7 @@ import java.util.concurrent.atomic.AtomicReference; /** + * */ public class SimpleProvisioningStrategyTest { 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..5aee469440af --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/autoscaling/StrategyTestUtils.java @@ -0,0 +1,215 @@ +package org.apache.druid.indexing.overlord.autoscaling; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.guava.DSuppliers; +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; + } + } +} From 3028f32359ad773b0a49228dc1a122b558a2f7b4 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Thu, 5 Dec 2019 12:04:23 +0200 Subject: [PATCH 18/23] Support different types of MiddleManagers by Autoscaler #8695 - Fixed code styles --- ...ngTaskBasedWorkerProvisioningStrategy.java | 4 +-- .../autoscaling/ProvisioningUtil.java | 6 +++-- .../setup/CategoriedWorkerSelectStrategy.java | 19 ++++++++++++++ .../autoscaling/StrategyTestUtils.java | 25 +++++++++++++++++-- 4 files changed, 48 insertions(+), 6 deletions(-) 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 b1d10bac36f5..38f89b447244 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 @@ -109,8 +109,8 @@ private class PendingProvisioner implements Provisioner private final Map> currentlyProvisioningMap = new HashMap<>(); private final Map> currentlyTerminatingMap = new HashMap<>(); - private final Map lastProvisionTimeMap = new HashMap<>();//DateTimes.nowUtc(); - private final Map lastTerminateTimeMap = new HashMap<>();//lastProvisionTime; + private final Map lastProvisionTimeMap = new HashMap<>(); + private final Map lastTerminateTimeMap = new HashMap<>(); private PendingProvisioner(WorkerTaskRunner runner) { 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 ed456e8f6718..bb9d4cdbb6c7 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 @@ -162,11 +162,13 @@ public static AutoScaler getAutoscalerByCategory(String category, Map getWorkersOfCategory(Collection workers, String category) { + 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) { + public static String getAutoscalerCategory(AutoScaler autoScaler) + { return autoScaler.getCategory() == null ? CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY : autoScaler.getCategory(); 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 index 53a1576567ad..baeb28922249 100644 --- 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 @@ -1,3 +1,22 @@ +/* + * 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 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 index 5aee469440af..b83f8c61135d 100644 --- 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 @@ -1,7 +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.autoscaling; import com.google.common.collect.ImmutableMap; -import org.apache.druid.common.guava.DSuppliers; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; @@ -115,7 +133,10 @@ public static WorkerCategorySpec createWorkerCategorySpec( return new WorkerCategorySpec(categoryMap, isStrong); } - public static AtomicReference createWorkerConfigRef(WorkerCategorySpec workerCategorySpec, List autoScalers) + public static AtomicReference createWorkerConfigRef( + WorkerCategorySpec workerCategorySpec, + List autoScalers + ) { return new AtomicReference<>( new CategoriedWorkerBehaviorConfig( From 7c0a379c5bef6b38a533e45aaec5d38dd96751a1 Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Wed, 29 Jan 2020 12:10:31 +0200 Subject: [PATCH 19/23] Support different types of MiddleManagers by Autoscaler #8695 - Used Default Worker Category as a fallback for null worker categories --- .../autoscaling/CategoriedProvisioningStrategy.java | 8 ++------ .../PendingTaskBasedWorkerProvisioningStrategy.java | 8 ++------ .../overlord/autoscaling/ProvisioningUtil.java | 10 ++++++++++ .../autoscaling/SimpleWorkerProvisioningStrategy.java | 8 ++------ 4 files changed, 16 insertions(+), 18 deletions(-) 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 index d496a2d31763..9b3824518ad1 100644 --- 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 @@ -180,9 +180,7 @@ public synchronized boolean doProvision() ) )); - Map> workersByCategories = workers.stream().collect(Collectors.groupingBy( - immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) - ); + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(workers); // Merge categories of tasks and workers Set allCategories = new HashSet<>(tasksByCategories.keySet()); @@ -423,9 +421,7 @@ public synchronized boolean doTerminate() boolean didTerminate = false; - Map> workersByCategories = zkWorkers.stream().collect(Collectors.groupingBy( - immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) - ); + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(zkWorkers); Set allCategories = workersByCategories.keySet(); log.debug( 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 38f89b447244..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 @@ -145,9 +145,7 @@ public synchronized boolean doProvision() ) )); - Map> workersByCategories = workers.stream().collect(Collectors.groupingBy( - immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) - ); + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(workers); // Merge categories of tasks and workers Set allCategories = new HashSet<>(tasksByCategories.keySet()); @@ -390,9 +388,7 @@ public synchronized boolean doTerminate() boolean didTerminate = false; - Map> workersByCategories = zkWorkers.stream().collect(Collectors.groupingBy( - immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) - ); + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(zkWorkers); Set allCategories = workersByCategories.keySet(); log.debug( 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 bb9d4cdbb6c7..719c2be05d30 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 @@ -29,6 +29,7 @@ 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; @@ -173,4 +174,13 @@ public static String getAutoscalerCategory(AutoScaler autoScaler) ? 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 ad93be92f4a7..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 @@ -164,9 +164,7 @@ public synchronized boolean doProvision() workerCategorySpec ); - Map> workersByCategories = workers.stream().collect(Collectors.groupingBy( - immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) - ); + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(workers); // Merge categories of tasks and workers Set allCategories = new HashSet<>(tasksByCategories.keySet()); @@ -310,9 +308,7 @@ public synchronized boolean doTerminate() workerCategorySpec ); - Map> workersByCategories = workers.stream().collect(Collectors.groupingBy( - immutableWorkerInfo -> immutableWorkerInfo.getWorker().getCategory()) - ); + Map> workersByCategories = ProvisioningUtil.getWorkersByCategories(workers); Set allCategories = workersByCategories.keySet(); log.debug( From 10f51a78f136288b85bb87ba1cf6f6639552e18f Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Thu, 30 Jan 2020 18:18:12 +0200 Subject: [PATCH 20/23] Support different types of MiddleManagers by Autoscaler #8695 - Refactored legacy code to avoid NPE - Removed duplicates --- .../CategoriedProvisioningStrategy.java | 21 ++----------------- .../overlord/setup/WorkerSelectUtils.java | 2 +- 2 files changed, 3 insertions(+), 20 deletions(-) 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 index 9b3824518ad1..9b825dbd54ea 100644 --- 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 @@ -205,7 +205,7 @@ public synchronized boolean doProvision() return didProvision; } - Map autoscalersByCategory = mapAutoscalerByCategory(workerConfig.getAutoScalers()); + Map autoscalersByCategory = ProvisioningUtil.mapAutoscalerByCategory(workerConfig.getAutoScalers()); for (String category : allCategories) { List categoryTasks = tasksByCategories.getOrDefault(category, Collections.emptyList()); @@ -430,7 +430,7 @@ public synchronized boolean doTerminate() allCategories ); - Map autoscalersByCategory = mapAutoscalerByCategory(workerConfig.getAutoScalers()); + Map autoscalersByCategory = ProvisioningUtil.mapAutoscalerByCategory(workerConfig.getAutoScalers()); for (String category : allCategories) { Set currentlyProvisioning = this.currentlyProvisioning.getOrDefault(category, Collections.emptySet()); @@ -658,23 +658,6 @@ private AutoScaler getCategoryAutoscaler(String category, Map mapAutoscalerByCategory(List autoScalers) - { - Map result = autoScalers.stream().collect(Collectors.groupingBy( - autoScaler -> autoScaler.getCategory() == null - ? CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY - : autoScaler.getCategory(), - 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 private WorkerCategorySpec getWorkerCategorySpec(CategoriedWorkerBehaviorConfig workerConfig) { 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 f372797deff9..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 @@ -190,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())) ); } From 931ebbac00678f275581cc4c51d3f6c89399bdfc Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Fri, 31 Jan 2020 16:08:44 +0200 Subject: [PATCH 21/23] Support different types of MiddleManagers by Autoscaler #8695 - Small fix --- .../overlord/autoscaling/CategoriedProvisioningStrategy.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 9b825dbd54ea..511347ee8feb 100644 --- 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 @@ -325,7 +325,7 @@ private int getScaleUpNodeCount( // 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, 1) : getWorkersNeededToAssignTasks( + int moreWorkersNeeded = currValidWorkers == 0 ? Math.max(minWorkerCount, pendingTasks.isEmpty() ? 0 : 1) : getWorkersNeededToAssignTasks( remoteTaskRunnerConfig, workerConfig, pendingTasks, From 03f5e6201685d179c3fc59779bc5b66e630fc8bd Mon Sep 17 00:00:00 2001 From: Vladimir Iordanov Date: Tue, 4 Feb 2020 13:31:25 +0200 Subject: [PATCH 22/23] Support different types of MiddleManagers by Autoscaler #8695 - Added more debug info - Removed duplicated code --- .../CategoriedProvisioningStrategy.java | 22 ++----------------- .../autoscaling/ProvisioningUtil.java | 2 +- .../overlord/setup/WorkerSelectUtils.java | 5 ++++- 3 files changed, 7 insertions(+), 22 deletions(-) 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 index 511347ee8feb..bd72b68336c1 100644 --- 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 @@ -212,7 +212,7 @@ public synchronized boolean doProvision() List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); currentlyProvisioning.putIfAbsent(category, new HashSet<>()); Set currentlyProvisioning = this.currentlyProvisioning.get(category); - AutoScaler groupAutoscaler = getCategoryAutoscaler(category, autoscalersByCategory); + AutoScaler groupAutoscaler = ProvisioningUtil.getAutoscalerByCategory(category, autoscalersByCategory); didProvision = doProvision( category, @@ -448,7 +448,7 @@ public synchronized boolean doTerminate() List categoryWorkers = workersByCategories.getOrDefault(category, Collections.emptyList()); currentlyTerminating.putIfAbsent(category, new HashSet<>()); Set currentlyTerminating = this.currentlyTerminating.get(category); - AutoScaler groupAutoscaler = getCategoryAutoscaler(category, autoscalersByCategory); + AutoScaler groupAutoscaler = ProvisioningUtil.getAutoscalerByCategory(category, autoscalersByCategory); didTerminate = doTerminate( category, @@ -640,24 +640,6 @@ private boolean initAutoscaler(AutoScaler autoScaler, String category, Categorie ); } - @Nullable - private AutoScaler getCategoryAutoscaler(String category, Map autoscalersByCategory) - { - AutoScaler autoScaler = autoscalersByCategory.get(category); - boolean isStrongAssignment = !autoscalersByCategory.containsKey(CategoriedWorkerBehaviorConfig.DEFAULT_AUTOSCALER_CATEGORY); - - 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; - } - @Nullable private WorkerCategorySpec getWorkerCategorySpec(CategoriedWorkerBehaviorConfig workerConfig) { 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 719c2be05d30..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 @@ -150,7 +150,7 @@ public static AutoScaler getAutoscalerByCategory(String category, Map Date: Wed, 5 Feb 2020 11:22:54 +0200 Subject: [PATCH 23/23] Support different types of MiddleManagers by Autoscaler #8695 - Removed debug logging --- .../druid/indexing/overlord/setup/WorkerSelectUtils.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) 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 cf09848140d2..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 @@ -24,7 +24,6 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.ImmutableWorkerInfo; import org.apache.druid.indexing.overlord.config.WorkerTaskRunnerConfig; -import org.apache.druid.java.util.emitter.EmittingLogger; import javax.annotation.Nullable; import java.util.Map; @@ -34,8 +33,6 @@ public class WorkerSelectUtils { - private static final EmittingLogger log = new EmittingLogger(WorkerSelectUtils.class); - private WorkerSelectUtils() { // No instantiation. @@ -132,7 +129,7 @@ public static ImmutableWorkerInfo selectWorker( runnableWorkers ); final ImmutableWorkerInfo selected = workerSelector.apply(categoryWorkers); - log.debug("Selected worker %s for category %s. Spec strong assignment is %b", selected, preferredCategory, workerCategorySpec.isStrong()); + if (selected != null) { return selected; } else if (workerCategorySpec.isStrong()) {