-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Implement dynamic capacity for kubernetes task runner #18591
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
GabrielCWT
wants to merge
20
commits into
apache:master
Choose a base branch
from
GabrielCWT:gh-56-dynamic-runner-capacity
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+1,255
−445
Open
Changes from all commits
Commits
Show all changes
20 commits
Select commit
Hold shift + click to select a range
cb25fd9
Implement dynamic capacity for kubernetes task runner
GabrielCWT c4386f1
Update docs
GabrielCWT 579a12c
Refactor to use getIfNull
GabrielCWT 9f9e845
Update docs wording
GabrielCWT 5ae5d4e
Fix based on comments
GabrielCWT d0e0081
Update wording for docs
GabrielCWT 4053083
Upate static config java doc
GabrielCWT 1da285c
Undo removal of constructor
GabrielCWT 128eb6e
Initial config observer implementation
GabrielCWT bc093fb
Merge branch 'master' into gh-56-dynamic-runner-capacity
GabrielCWT afec8ec
Use StringUtils.format
GabrielCWT c02abc3
Add missing import
GabrielCWT 2a31938
Update listener operations to be atomic
GabrielCWT 0ac26f8
Fix config manager intialisation in tests
GabrielCWT 6050eea
Add tests for effective config
GabrielCWT fcc3d3e
Test syncCapacityWithDynamicConfig
GabrielCWT 2c4f068
Fix checkstyle
GabrielCWT 55de05d
Update KubernetesOverlordModuleTest to setup ConfigManager
GabrielCWT a3c2c0a
Update docs
GabrielCWT 1c64afb
Use AtomicInteger for currentCapacity
GabrielCWT File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,6 +29,7 @@ | |
| import com.google.common.util.concurrent.ListeningExecutorService; | ||
| import com.google.common.util.concurrent.MoreExecutors; | ||
| import io.fabric8.kubernetes.api.model.batch.v1.Job; | ||
| import org.apache.druid.common.config.ConfigManager; | ||
| import org.apache.druid.common.guava.FutureUtils; | ||
| import org.apache.druid.error.DruidException; | ||
| import org.apache.druid.indexer.RunnerTaskState; | ||
|
|
@@ -44,6 +45,7 @@ | |
| import org.apache.druid.java.util.common.DateTimes; | ||
| import org.apache.druid.java.util.common.ISE; | ||
| import org.apache.druid.java.util.common.Pair; | ||
| import org.apache.druid.java.util.common.StringUtils; | ||
| import org.apache.druid.java.util.common.concurrent.Execs; | ||
| import org.apache.druid.java.util.common.lifecycle.LifecycleStart; | ||
| import org.apache.druid.java.util.common.lifecycle.LifecycleStop; | ||
|
|
@@ -56,6 +58,7 @@ | |
| import org.apache.druid.k8s.overlord.common.K8sTaskId; | ||
| import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; | ||
| import org.apache.druid.k8s.overlord.common.KubernetesResourceNotFoundException; | ||
| import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; | ||
| import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; | ||
| import org.apache.druid.tasklogs.TaskLogStreamer; | ||
| import org.jboss.netty.handler.codec.http.HttpMethod; | ||
|
|
@@ -76,8 +79,11 @@ | |
| import java.util.concurrent.ExecutionException; | ||
| import java.util.concurrent.Executor; | ||
| import java.util.concurrent.Executors; | ||
| import java.util.concurrent.LinkedBlockingQueue; | ||
| import java.util.concurrent.ScheduledExecutorService; | ||
| import java.util.concurrent.ThreadPoolExecutor; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| /** | ||
|
|
@@ -100,6 +106,7 @@ | |
| public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner | ||
| { | ||
| private static final EmittingLogger log = new EmittingLogger(KubernetesTaskRunner.class); | ||
| private static final String OBSERVER_KEY = "k8s-task-runner-capacity-%s"; | ||
| private final CopyOnWriteArrayList<Pair<TaskRunnerListener, Executor>> listeners = new CopyOnWriteArrayList<>(); | ||
|
|
||
| // to cleanup old jobs that might not have been deleted. | ||
|
|
@@ -111,19 +118,23 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner | |
| private final KubernetesPeonClient client; | ||
| private final KubernetesTaskRunnerConfig config; | ||
| private final ListeningExecutorService exec; | ||
| private final ThreadPoolExecutor tpe; | ||
| private final HttpClient httpClient; | ||
| private final PeonLifecycleFactory peonLifecycleFactory; | ||
| private final ServiceEmitter emitter; | ||
| // currently worker categories aren't supported, so it's hardcoded. | ||
| protected static final String WORKER_CATEGORY = "_k8s_worker_category"; | ||
|
|
||
| private final AtomicInteger currentCapacity; | ||
|
|
||
| public KubernetesTaskRunner( | ||
| TaskAdapter adapter, | ||
| KubernetesTaskRunnerConfig config, | ||
| KubernetesPeonClient client, | ||
| HttpClient httpClient, | ||
| PeonLifecycleFactory peonLifecycleFactory, | ||
| ServiceEmitter emitter | ||
| ServiceEmitter emitter, | ||
| ConfigManager configManager | ||
| ) | ||
| { | ||
| this.adapter = adapter; | ||
|
|
@@ -132,10 +143,12 @@ public KubernetesTaskRunner( | |
| this.httpClient = httpClient; | ||
| this.peonLifecycleFactory = peonLifecycleFactory; | ||
| this.cleanupExecutor = Executors.newScheduledThreadPool(1); | ||
| this.exec = MoreExecutors.listeningDecorator( | ||
| Execs.multiThreaded(config.getCapacity(), "k8s-task-runner-%d") | ||
| ); | ||
| this.emitter = emitter; | ||
|
|
||
| this.currentCapacity = new AtomicInteger(config.getCapacity()); | ||
| this.tpe = new ThreadPoolExecutor(currentCapacity.get(), currentCapacity.get(), 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(), Execs.makeThreadFactory("k8s-task-runner-%d", null)); | ||
| this.exec = MoreExecutors.listeningDecorator(this.tpe); | ||
| configManager.addListener(KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, StringUtils.format(OBSERVER_KEY, Thread.currentThread().getId()), this::syncCapacityWithDynamicConfig); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -179,6 +192,24 @@ protected KubernetesWorkItem joinAsync(Task task) | |
| } | ||
| } | ||
|
|
||
| private void syncCapacityWithDynamicConfig(KubernetesTaskRunnerDynamicConfig config) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This looks much cleaner. Thanks @FrankChen021 for the suggestion and @GabrielCWT for the impl. |
||
| { | ||
| int newCapacity = config.getCapacity(); | ||
| if (newCapacity == currentCapacity.get()) { | ||
| return; | ||
| } | ||
| log.info("Adjusting k8s task runner capacity from [%d] to [%d]", currentCapacity.get(), newCapacity); | ||
| // maximum pool size must always be greater than or equal to the core pool size | ||
| if (newCapacity < currentCapacity.get()) { | ||
| tpe.setCorePoolSize(newCapacity); | ||
| tpe.setMaximumPoolSize(newCapacity); | ||
| } else { | ||
| tpe.setMaximumPoolSize(newCapacity); | ||
| tpe.setCorePoolSize(newCapacity); | ||
| } | ||
| currentCapacity.set(newCapacity); | ||
| } | ||
|
|
||
| private TaskStatus runTask(Task task) | ||
| { | ||
| return doTask(task, true); | ||
|
|
@@ -294,7 +325,7 @@ public void shutdown(String taskid, String reason) | |
| synchronized (tasks) { | ||
| tasks.remove(taskid); | ||
| } | ||
|
|
||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -420,7 +451,7 @@ public void stop() | |
| @Override | ||
| public Map<String, Long> getTotalTaskSlotCount() | ||
| { | ||
| return ImmutableMap.of(WORKER_CATEGORY, (long) config.getCapacity()); | ||
| return ImmutableMap.of(WORKER_CATEGORY, (long) currentCapacity.get()); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -438,13 +469,13 @@ public Optional<ScalingStats> getScalingStats() | |
| @Override | ||
| public Map<String, Long> getIdleTaskSlotCount() | ||
| { | ||
| return ImmutableMap.of(WORKER_CATEGORY, (long) Math.max(0, config.getCapacity() - tasks.size())); | ||
| return ImmutableMap.of(WORKER_CATEGORY, (long) Math.max(0, currentCapacity.get() - tasks.size())); | ||
| } | ||
|
|
||
| @Override | ||
| public Map<String, Long> getUsedTaskSlotCount() | ||
| { | ||
| return ImmutableMap.of(WORKER_CATEGORY, (long) Math.min(config.getCapacity(), tasks.size())); | ||
| return ImmutableMap.of(WORKER_CATEGORY, (long) Math.min(currentCapacity.get(), tasks.size())); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -535,7 +566,7 @@ public RunnerTaskState getRunnerTaskState(String taskId) | |
| @Override | ||
| public int getTotalCapacity() | ||
| { | ||
| return config.getCapacity(); | ||
| return currentCapacity.get(); | ||
| } | ||
|
|
||
| @Override | ||
|
|
||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would this run in the jetty thread?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes it will run in the same thread as the one handling the API call. Would your concern be that if more listeners were to be added in the future, the updating of dynamic configs would be a long blocking call?