From cb25fd91c84cc60516ab4e3221e8deecadf4cbfe Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Wed, 1 Oct 2025 17:25:33 +0800 Subject: [PATCH 01/19] Implement dynamic capacity for kubernetes task runner --- .../overlord/KubernetesOverlordModule.java | 18 +- .../k8s/overlord/KubernetesTaskRunner.java | 54 +- .../overlord/KubernetesTaskRunnerConfig.java | 514 +--------------- .../KubernetesTaskRunnerEffectiveConfig.java | 185 ++++++ .../overlord/KubernetesTaskRunnerFactory.java | 4 +- .../KubernetesTaskRunnerStaticConfig.java | 562 ++++++++++++++++++ ...aultKubernetesTaskRunnerDynamicConfig.java | 42 +- ...KubernetesTaskExecutionConfigResource.java | 20 +- .../KubernetesTaskRunnerDynamicConfig.java | 22 + .../DynamicConfigPodTemplateSelector.java | 19 +- .../taskadapter/PodTemplateTaskAdapter.java | 2 +- .../KubernetesTaskRunnerFactoryTest.java | 5 +- ...KubernetesTaskRunnerStaticConfigTest.java} | 10 +- .../overlord/KubernetesTaskRunnerTest.java | 12 +- ...KubernetesTaskRunnerDynamicConfigTest.java | 13 +- ...rnetesTaskExecutionConfigResourceTest.java | 121 ++++ ...KubernetesTaskRunnerDynamicConfigTest.java | 13 +- .../DruidPeonClientIntegrationTest.java | 8 +- .../DynamicConfigPodTemplateSelectorTest.java | 46 +- .../taskadapter/K8sTaskAdapterTest.java | 74 +-- .../MultiContainerTaskAdapterTest.java | 26 +- .../PodTemplateTaskAdapterTest.java | 8 +- .../SingleContainerTaskAdapterTest.java | 8 +- 23 files changed, 1158 insertions(+), 628 deletions(-) create mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfig.java create mode 100644 extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java rename extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/{KubernetesTaskRunnerConfigTest.java => KubernetesTaskRunnerStaticConfigTest.java} (94%) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java index 55fb08627b2c..ece8397c1fd1 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java @@ -85,7 +85,7 @@ public class KubernetesOverlordModule implements DruidModule public void configure(Binder binder) { // druid.indexer.runner.type=k8s - JsonConfigProvider.bind(binder, IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX, KubernetesTaskRunnerConfig.class); + JsonConfigProvider.bind(binder, IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX, KubernetesTaskRunnerStaticConfig.class); JsonConfigProvider.bind(binder, K8SANDWORKER_PROPERTIES_PREFIX, KubernetesAndWorkerTaskRunnerConfig.class); JsonConfigProvider.bind(binder, "druid.indexer.queue", TaskQueueConfig.class); JacksonConfigProvider.bind(binder, KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, KubernetesTaskRunnerDynamicConfig.class, null); @@ -117,10 +117,20 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, HTTPCLIENT_PROPERITES_PREFIX, DruidKubernetesHttpClientConfig.class); } + @Provides + @LazySingleton + public KubernetesTaskRunnerEffectiveConfig provideEffectiveConfig( + KubernetesTaskRunnerStaticConfig staticConfig, + Supplier dynamicConfigSupplier + ) + { + return new KubernetesTaskRunnerEffectiveConfig(staticConfig, dynamicConfigSupplier); + } + @Provides @LazySingleton public DruidKubernetesClient makeKubernetesClient( - KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig, + KubernetesTaskRunnerStaticConfig kubernetesTaskRunnerConfig, DruidKubernetesHttpClientConfig httpClientConfig, Lifecycle lifecycle ) @@ -184,7 +194,7 @@ TaskRunnerFactory provideWorkerTaskRunner( TaskAdapter provideTaskAdapter( DruidKubernetesClient client, Properties properties, - KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig, + KubernetesTaskRunnerEffectiveConfig kubernetesTaskRunnerConfig, TaskConfig taskConfig, StartupLoggingConfig startupLoggingConfig, @Self DruidNode druidNode, @@ -227,7 +237,7 @@ TaskAdapter provideTaskAdapter( druidNode, smileMapper, taskLogs, - new DynamicConfigPodTemplateSelector(properties, dynamicConfigRef) + new DynamicConfigPodTemplateSelector(properties, kubernetesTaskRunnerConfig) ); } else { return new SingleContainerTaskAdapter( diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 51913d13a71e..62b9f19da761 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -76,7 +76,9 @@ 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.stream.Collectors; @@ -85,13 +87,13 @@ * The KubernetesTaskRunner runs tasks by transforming the task spec into a K8s Job spec based * on the TaskAdapter it is configured with. The KubernetesTaskRunner has a pool of threads * (configurable with the capacity configuration) to track the jobs (1 thread tracks 1 job). - * + *

* Each thread calls down to the KubernetesPeonLifecycle class to submit the Job to K8s and then * waits for the lifecycle class to report back with the Job's status (success/failure). - * + *

* If there are not enough threads in the thread pool to execute and wait for a job, then the * task is put in a queue and left in WAITING state until another task completes. - * + *

* When the KubernetesTaskRunner comes up it attempts to restore its internal mapping of tasks * from Kubernetes by listing running jobs and calling join on each job, which spawns a thread to * wait for the fabric8 client library to report back, similar to what happens when a new @@ -109,17 +111,20 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner protected final TaskAdapter adapter; private final KubernetesPeonClient client; - private final KubernetesTaskRunnerConfig config; + private final KubernetesTaskRunnerEffectiveConfig 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 int currentCapacity; + public KubernetesTaskRunner( TaskAdapter adapter, - KubernetesTaskRunnerConfig config, + KubernetesTaskRunnerEffectiveConfig config, KubernetesPeonClient client, HttpClient httpClient, PeonLifecycleFactory peonLifecycleFactory, @@ -132,10 +137,11 @@ 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 = config.getCapacity(); + this.tpe = new ThreadPoolExecutor(currentCapacity, currentCapacity, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue(), Execs.makeThreadFactory("k8s-task-runner-%d", null)); + this.exec = MoreExecutors.listeningDecorator(this.tpe); } @Override @@ -151,6 +157,8 @@ public Optional streamTaskLog(String taskid, long offset) @Override public ListenableFuture run(Task task) { + syncCapacityWithDynamicConfig(); + synchronized (tasks) { return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem( task, @@ -166,6 +174,8 @@ public ListenableFuture run(Task task) protected KubernetesWorkItem joinAsync(Task task) { + syncCapacityWithDynamicConfig(); + synchronized (tasks) { return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem( task, @@ -179,6 +189,24 @@ protected KubernetesWorkItem joinAsync(Task task) } } + private void syncCapacityWithDynamicConfig() + { + int newCapacity = config.getCapacity(); + if (newCapacity == currentCapacity) { + return; + } + log.info("Adjusting k8s task runner capacity from [%d] to [%d]", currentCapacity, newCapacity); + // maximum pool size must always be greater than or equal to the core pool size + if (newCapacity < currentCapacity) { + tpe.setCorePoolSize(newCapacity); + tpe.setMaximumPoolSize(newCapacity); + } else { + tpe.setMaximumPoolSize(newCapacity); + tpe.setCorePoolSize(newCapacity); + } + currentCapacity = newCapacity; + } + private TaskStatus runTask(Task task) { return doTask(task, true); @@ -294,7 +322,7 @@ public void shutdown(String taskid, String reason) synchronized (tasks) { tasks.remove(taskid); } - + } @Override @@ -420,7 +448,7 @@ public void stop() @Override public Map getTotalTaskSlotCount() { - return ImmutableMap.of(WORKER_CATEGORY, (long) config.getCapacity()); + return ImmutableMap.of(WORKER_CATEGORY, (long) currentCapacity); } @Override @@ -438,13 +466,13 @@ public Optional getScalingStats() @Override public Map getIdleTaskSlotCount() { - return ImmutableMap.of(WORKER_CATEGORY, (long) Math.max(0, config.getCapacity() - tasks.size())); + return ImmutableMap.of(WORKER_CATEGORY, (long) Math.max(0, currentCapacity - tasks.size())); } @Override public Map getUsedTaskSlotCount() { - return ImmutableMap.of(WORKER_CATEGORY, (long) Math.min(config.getCapacity(), tasks.size())); + return ImmutableMap.of(WORKER_CATEGORY, (long) Math.min(currentCapacity, tasks.size())); } @Override @@ -535,7 +563,7 @@ public RunnerTaskState getRunnerTaskState(String taskId) @Override public int getTotalCapacity() { - return config.getCapacity(); + return currentCapacity; } @Override diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index d810c9ee23a2..cdf08374ad9f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -19,523 +19,53 @@ package org.apache.druid.k8s.overlord; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.apache.commons.lang3.ObjectUtils; import org.joda.time.Period; -import javax.annotation.Nonnull; -import javax.validation.constraints.Max; -import javax.validation.constraints.Min; -import javax.validation.constraints.NotNull; import java.util.List; import java.util.Map; -public class KubernetesTaskRunnerConfig +public interface KubernetesTaskRunnerConfig { - @JsonProperty - @NotNull - private String namespace; + String getNamespace(); - @JsonProperty - private String k8sTaskPodNamePrefix = ""; + String getOverlordNamespace(); - // This property is the namespace that the Overlord is running in. - // For cases where we want task pods to run on different namespace from the overlord, we need to specify the namespace of the overlord here. - // Else, we can simply leave this field alone. - @JsonProperty - private String overlordNamespace = ""; + String getK8sTaskPodNamePrefix(); - @JsonProperty - private boolean debugJobs = false; + boolean isDebugJobs(); - /** - * Deprecated, please specify adapter type runtime property instead - *

- * I.E `druid.indexer.runner.k8s.adapter.type: overlordMultiContainer` - */ @Deprecated - @JsonProperty - private boolean sidecarSupport = false; + boolean isSidecarSupport(); - @JsonProperty - // if this is not set, then the first container in your pod spec is assumed to be the overlord container. - // usually this is fine, but when you are dynamically adding sidecars like istio, the service mesh could - // in fact place the istio-proxy container as the first container. Thus, you would specify this value to - // the name of your primary container. e.g. druid-overlord - private String primaryContainerName = null; + String getPrimaryContainerName(); - @JsonProperty - // for multi-container jobs, we need this image to shut down sidecars after the main container - // has completed - private String kubexitImage = "karlkfi/kubexit:v0.3.2"; + String getKubexitImage(); - // how much time to wait for preStop hooks to execute - // lower number speeds up pod termination time to release locks - // faster, defaults to your k8s setup, usually 30 seconds. - private Long graceTerminationPeriodSeconds = null; + Long getGraceTerminationPeriodSeconds(); - @JsonProperty - // disable using http / https proxy environment variables - private boolean disableClientProxy; + boolean isDisableClientProxy(); - @JsonProperty - @NotNull - private Period maxTaskDuration = new Period("PT4H"); + Period getTaskTimeout(); - @JsonProperty - @NotNull - // how long to wait for the jobs to be cleaned up. - private Period taskCleanupDelay = new Period("P2D"); + Period getTaskJoinTimeout(); - @JsonProperty - @NotNull - // interval for k8s job cleanup to run - private Period taskCleanupInterval = new Period("PT10m"); + Period getTaskCleanupDelay(); - @JsonProperty - @NotNull - // how long to wait to join peon k8s jobs on startup - private Period taskJoinTimeout = new Period("PT1M"); + Period getTaskCleanupInterval(); + Period getTaskLaunchTimeout(); - @JsonProperty - @NotNull - // how long to wait for the peon k8s job to launch - private Period k8sjobLaunchTimeout = new Period("PT1H"); + Period getLogSaveTimeout(); - @JsonProperty - @NotNull - // how long to wait for log saving operations to complete - private Period logSaveTimeout = new Period("PT300S"); + List getPeonMonitors(); - @JsonProperty - // ForkingTaskRunner inherits the monitors from the MM, in k8s mode - // the peon inherits the monitors from the overlord, so if someone specifies - // a TaskCountStatsMonitor in the overlord for example, the peon process - // fails because it can not inject this monitor in the peon process. - private List peonMonitors = ImmutableList.of(); + List getJavaOptsArray(); - @JsonProperty - @NotNull - private List javaOptsArray = ImmutableList.of(); + int getCpuCoreInMicro(); - @JsonProperty - @NotNull - private int cpuCoreInMicro = 0; + Map getLabels(); - @JsonProperty - @NotNull - private Map labels = ImmutableMap.of(); + Map getAnnotations(); - @JsonProperty - @NotNull - private Map annotations = ImmutableMap.of(); - - @JsonProperty - @Min(1) - @Max(Integer.MAX_VALUE) - @NotNull - private Integer capacity = Integer.MAX_VALUE; - - public KubernetesTaskRunnerConfig() - { - } - - private KubernetesTaskRunnerConfig( - @Nonnull String namespace, - String overlordNamespace, - String k8sTaskPodNamePrefix, - boolean debugJobs, - boolean sidecarSupport, - String primaryContainerName, - String kubexitImage, - Long graceTerminationPeriodSeconds, - boolean disableClientProxy, - Period maxTaskDuration, - Period taskCleanupDelay, - Period taskCleanupInterval, - Period k8sjobLaunchTimeout, - Period logSaveTimeout, - List peonMonitors, - List javaOptsArray, - int cpuCoreInMicro, - Map labels, - Map annotations, - Integer capacity, - Period taskJoinTimeout - ) - { - this.namespace = namespace; - this.overlordNamespace = ObjectUtils.defaultIfNull( - overlordNamespace, - this.overlordNamespace - ); - this.k8sTaskPodNamePrefix = k8sTaskPodNamePrefix; - this.debugJobs = ObjectUtils.defaultIfNull( - debugJobs, - this.debugJobs - ); - this.sidecarSupport = ObjectUtils.defaultIfNull( - sidecarSupport, - this.sidecarSupport - ); - this.primaryContainerName = ObjectUtils.defaultIfNull( - primaryContainerName, - this.primaryContainerName - ); - this.kubexitImage = ObjectUtils.defaultIfNull( - kubexitImage, - this.kubexitImage - ); - this.graceTerminationPeriodSeconds = ObjectUtils.defaultIfNull( - graceTerminationPeriodSeconds, - this.graceTerminationPeriodSeconds - ); - this.disableClientProxy = disableClientProxy; - this.maxTaskDuration = ObjectUtils.defaultIfNull( - maxTaskDuration, - this.maxTaskDuration - ); - this.taskCleanupDelay = ObjectUtils.defaultIfNull( - taskCleanupDelay, - this.taskCleanupDelay - ); - this.taskCleanupInterval = ObjectUtils.defaultIfNull( - taskCleanupInterval, - this.taskCleanupInterval - ); - this.k8sjobLaunchTimeout = ObjectUtils.defaultIfNull( - k8sjobLaunchTimeout, - this.k8sjobLaunchTimeout - ); - this.taskJoinTimeout = ObjectUtils.defaultIfNull( - taskJoinTimeout, - this.taskJoinTimeout - ); - this.logSaveTimeout = ObjectUtils.defaultIfNull( - logSaveTimeout, - this.logSaveTimeout - ); - this.peonMonitors = ObjectUtils.defaultIfNull( - peonMonitors, - this.peonMonitors - ); - this.javaOptsArray = ObjectUtils.defaultIfNull( - javaOptsArray, - this.javaOptsArray - ); - this.cpuCoreInMicro = ObjectUtils.defaultIfNull( - cpuCoreInMicro, - this.cpuCoreInMicro - ); - this.labels = ObjectUtils.defaultIfNull( - labels, - this.labels - ); - this.annotations = ObjectUtils.defaultIfNull( - annotations, - this.annotations - ); - this.capacity = ObjectUtils.defaultIfNull( - capacity, - this.capacity - ); - } - - public String getNamespace() - { - return namespace; - } - - public String getOverlordNamespace() - { - return overlordNamespace; - } - - public String getK8sTaskPodNamePrefix() - { - return k8sTaskPodNamePrefix; - } - - public boolean isDebugJobs() - { - return debugJobs; - } - - @Deprecated - public boolean isSidecarSupport() - { - return sidecarSupport; - } - - public String getPrimaryContainerName() - { - return primaryContainerName; - } - - public String getKubexitImage() - { - return kubexitImage; - } - - public Long getGraceTerminationPeriodSeconds() - { - return graceTerminationPeriodSeconds; - } - - public boolean isDisableClientProxy() - { - return disableClientProxy; - } - - public Period getTaskTimeout() - { - return maxTaskDuration; - } - - public Period getTaskJoinTimeout() - { - return taskJoinTimeout; - } - - - public Period getTaskCleanupDelay() - { - return taskCleanupDelay; - } - - public Period getTaskCleanupInterval() - { - return taskCleanupInterval; - } - - public Period getTaskLaunchTimeout() - { - return k8sjobLaunchTimeout; - } - - public Period getLogSaveTimeout() - { - return logSaveTimeout; - } - - public List getPeonMonitors() - { - return peonMonitors; - } - - public List getJavaOptsArray() - { - return javaOptsArray; - } - - public int getCpuCoreInMicro() - { - return cpuCoreInMicro; - } - - public Map getLabels() - { - return labels; - } - - public Map getAnnotations() - { - return annotations; - } - - public Integer getCapacity() - { - return capacity; - } - - public static Builder builder() - { - return new Builder(); - } - - public static class Builder - { - private String namespace; - private String overlordNamespace; - private String k8sTaskPodNamePrefix; - private boolean debugJob; - private boolean sidecarSupport; - private String primaryContainerName; - private String kubexitImage; - private Long graceTerminationPeriodSeconds; - private boolean disableClientProxy; - private Period maxTaskDuration; - private Period taskCleanupDelay; - private Period taskCleanupInterval; - private Period k8sjobLaunchTimeout; - private List peonMonitors; - private List javaOptsArray; - private int cpuCoreInMicro; - private Map labels; - private Map annotations; - private Integer capacity; - private Period taskJoinTimeout; - private Period logSaveTimeout; - - public Builder() - { - } - - public Builder withNamespace(String namespace) - { - this.namespace = namespace; - return this; - } - - public Builder withOverlordNamespace(String overlordNamespace) - { - this.overlordNamespace = overlordNamespace; - return this; - } - - public Builder withK8sTaskPodNamePrefix(String k8sTaskPodNamePrefix) - { - this.k8sTaskPodNamePrefix = k8sTaskPodNamePrefix; - return this; - } - - public Builder withDebugJob(boolean debugJob) - { - this.debugJob = debugJob; - return this; - } - - public Builder withSidecarSupport(boolean sidecarSupport) - { - this.sidecarSupport = sidecarSupport; - return this; - } - - public Builder withPrimaryContainerName(String primaryContainerName) - { - this.primaryContainerName = primaryContainerName; - return this; - } - - public Builder withKubexitImage(String kubexitImage) - { - this.kubexitImage = kubexitImage; - return this; - } - - public Builder withGraceTerminationPeriodSeconds(Long graceTerminationPeriodSeconds) - { - this.graceTerminationPeriodSeconds = graceTerminationPeriodSeconds; - return this; - } - - public Builder withDisableClientProxy(boolean disableClientProxy) - { - this.disableClientProxy = disableClientProxy; - return this; - } - - public Builder withTaskTimeout(Period taskTimeout) - { - this.maxTaskDuration = taskTimeout; - return this; - } - - public Builder withTaskCleanupDelay(Period taskCleanupDelay) - { - this.taskCleanupDelay = taskCleanupDelay; - return this; - } - - public Builder withTaskCleanupInterval(Period taskCleanupInterval) - { - this.taskCleanupInterval = taskCleanupInterval; - return this; - } - - public Builder withK8sJobLaunchTimeout(Period k8sjobLaunchTimeout) - { - this.k8sjobLaunchTimeout = k8sjobLaunchTimeout; - return this; - } - - public Builder withPeonMonitors(List peonMonitors) - { - this.peonMonitors = peonMonitors; - return this; - } - - public Builder withCpuCore(int cpuCore) - { - this.cpuCoreInMicro = cpuCore; - return this; - } - - public Builder withJavaOptsArray(List javaOptsArray) - { - this.javaOptsArray = javaOptsArray; - return this; - } - - public Builder withLabels(Map labels) - { - this.labels = labels; - return this; - } - - public Builder withAnnotations(Map annotations) - { - this.annotations = annotations; - return this; - } - - - public Builder withCapacity(@Min(0) @Max(Integer.MAX_VALUE) Integer capacity) - { - this.capacity = capacity; - return this; - } - - public Builder withTaskJoinTimeout(Period taskJoinTimeout) - { - this.taskJoinTimeout = taskJoinTimeout; - return this; - } - - public Builder withLogSaveTimeout(Period logSaveTimeout) - { - this.logSaveTimeout = logSaveTimeout; - return this; - } - - public KubernetesTaskRunnerConfig build() - { - return new KubernetesTaskRunnerConfig( - this.namespace, - this.overlordNamespace, - this.k8sTaskPodNamePrefix, - this.debugJob, - this.sidecarSupport, - this.primaryContainerName, - this.kubexitImage, - this.graceTerminationPeriodSeconds, - this.disableClientProxy, - this.maxTaskDuration, - this.taskCleanupDelay, - this.taskCleanupInterval, - this.k8sjobLaunchTimeout, - this.logSaveTimeout, - this.peonMonitors, - this.javaOptsArray, - this.cpuCoreInMicro, - this.labels, - this.annotations, - this.capacity, - this.taskJoinTimeout - ); - } - } + Integer getCapacity(); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfig.java new file mode 100644 index 000000000000..92f5ceb7d0d7 --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfig.java @@ -0,0 +1,185 @@ +/* + * 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.k8s.overlord; + +import com.google.common.base.Supplier; +import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; +import org.apache.druid.k8s.overlord.execution.PodTemplateSelectStrategy; +import org.joda.time.Period; + +import java.util.List; +import java.util.Map; + +/** + * Effective config object that combines static {@link KubernetesTaskRunnerConfig} + * with dynamic overrides from {@link KubernetesTaskRunnerDynamicConfig}. + */ +public class KubernetesTaskRunnerEffectiveConfig implements KubernetesTaskRunnerConfig +{ + private final KubernetesTaskRunnerStaticConfig staticConfig; + private final Supplier dynamicConfigSupplier; + + public KubernetesTaskRunnerEffectiveConfig( + KubernetesTaskRunnerStaticConfig staticConfig, + Supplier dynamicConfigSupplier + ) + { + this.staticConfig = staticConfig; + this.dynamicConfigSupplier = dynamicConfigSupplier; + } + + @Override + public String getNamespace() + { + return staticConfig.getNamespace(); + } + + @Override + public String getOverlordNamespace() + { + return staticConfig.getOverlordNamespace(); + } + + @Override + public String getK8sTaskPodNamePrefix() + { + return staticConfig.getK8sTaskPodNamePrefix(); + } + + @Override + public boolean isDebugJobs() + { + return staticConfig.isDebugJobs(); + } + + @Override + public boolean isSidecarSupport() + { + return staticConfig.isSidecarSupport(); + } + + @Override + public String getPrimaryContainerName() + { + return staticConfig.getPrimaryContainerName(); + } + + @Override + public String getKubexitImage() + { + return staticConfig.getKubexitImage(); + } + + @Override + public Long getGraceTerminationPeriodSeconds() + { + return staticConfig.getGraceTerminationPeriodSeconds(); + } + + @Override + public boolean isDisableClientProxy() + { + return staticConfig.isDisableClientProxy(); + } + + @Override + public Period getTaskTimeout() + { + return staticConfig.getTaskTimeout(); + } + + @Override + public Period getTaskJoinTimeout() + { + return staticConfig.getTaskJoinTimeout(); + } + + @Override + public Period getTaskCleanupDelay() + { + return staticConfig.getTaskCleanupDelay(); + } + + @Override + public Period getTaskCleanupInterval() + { + return staticConfig.getTaskCleanupInterval(); + } + + @Override + public Period getTaskLaunchTimeout() + { + return staticConfig.getTaskLaunchTimeout(); + } + + @Override + public Period getLogSaveTimeout() + { + return staticConfig.getLogSaveTimeout(); + } + + @Override + public List getPeonMonitors() + { + return staticConfig.getPeonMonitors(); + } + + @Override + public List getJavaOptsArray() + { + return staticConfig.getJavaOptsArray(); + } + + @Override + public int getCpuCoreInMicro() + { + return staticConfig.getCpuCoreInMicro(); + } + + @Override + public Map getLabels() + { + return staticConfig.getLabels(); + } + + @Override + public Map getAnnotations() + { + return staticConfig.getAnnotations(); + } + + @Override + public Integer getCapacity() + { + if (dynamicConfigSupplier == null || dynamicConfigSupplier.get() == null || dynamicConfigSupplier.get().getCapacity() == null) { + return staticConfig.getCapacity(); + } + return dynamicConfigSupplier.get().getCapacity(); + } + + public PodTemplateSelectStrategy getPodTemplateSelectStrategy() + { + if (dynamicConfigSupplier == null || dynamicConfigSupplier.get() == null || dynamicConfigSupplier.get().getPodTemplateSelectStrategy() == null) { + return KubernetesTaskRunnerDynamicConfig.DEFAULT_STRATEGY; + } + return dynamicConfigSupplier.get().getPodTemplateSelectStrategy(); + } +} + diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index dd8111ed49ed..41dab987d1d3 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -39,7 +39,7 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory + * I.E `druid.indexer.runner.k8s.adapter.type: overlordMultiContainer` + */ + @Deprecated + @JsonProperty + private boolean sidecarSupport = false; + + @JsonProperty + // if this is not set, then the first container in your pod spec is assumed to be the overlord container. + // usually this is fine, but when you are dynamically adding sidecars like istio, the service mesh could + // in fact place the istio-proxy container as the first container. Thus, you would specify this value to + // the name of your primary container. e.g. druid-overlord + private String primaryContainerName = null; + + @JsonProperty + // for multi-container jobs, we need this image to shut down sidecars after the main container + // has completed + private String kubexitImage = "karlkfi/kubexit:v0.3.2"; + + // how much time to wait for preStop hooks to execute + // lower number speeds up pod termination time to release locks + // faster, defaults to your k8s setup, usually 30 seconds. + private Long graceTerminationPeriodSeconds = null; + + @JsonProperty + // disable using http / https proxy environment variables + private boolean disableClientProxy; + + @JsonProperty + @NotNull + private Period maxTaskDuration = new Period("PT4H"); + + @JsonProperty + @NotNull + // how long to wait for the jobs to be cleaned up. + private Period taskCleanupDelay = new Period("P2D"); + + @JsonProperty + @NotNull + // interval for k8s job cleanup to run + private Period taskCleanupInterval = new Period("PT10m"); + + @JsonProperty + @NotNull + // how long to wait to join peon k8s jobs on startup + private Period taskJoinTimeout = new Period("PT1M"); + + + @JsonProperty + @NotNull + // how long to wait for the peon k8s job to launch + private Period k8sjobLaunchTimeout = new Period("PT1H"); + + @JsonProperty + @NotNull + // how long to wait for log saving operations to complete + private Period logSaveTimeout = new Period("PT300S"); + + @JsonProperty + // ForkingTaskRunner inherits the monitors from the MM, in k8s mode + // the peon inherits the monitors from the overlord, so if someone specifies + // a TaskCountStatsMonitor in the overlord for example, the peon process + // fails because it can not inject this monitor in the peon process. + private List peonMonitors = ImmutableList.of(); + + @JsonProperty + @NotNull + private List javaOptsArray = ImmutableList.of(); + + @JsonProperty + @NotNull + private int cpuCoreInMicro = 0; + + @JsonProperty + @NotNull + private Map labels = ImmutableMap.of(); + + @JsonProperty + @NotNull + private Map annotations = ImmutableMap.of(); + + @JsonProperty + @Min(1) + @Max(Integer.MAX_VALUE) + @NotNull + private Integer capacity = Integer.MAX_VALUE; + + public KubernetesTaskRunnerStaticConfig() + { + } + + private KubernetesTaskRunnerStaticConfig( + @Nonnull String namespace, + String overlordNamespace, + String k8sTaskPodNamePrefix, + boolean debugJobs, + boolean sidecarSupport, + String primaryContainerName, + String kubexitImage, + Long graceTerminationPeriodSeconds, + boolean disableClientProxy, + Period maxTaskDuration, + Period taskCleanupDelay, + Period taskCleanupInterval, + Period k8sjobLaunchTimeout, + Period logSaveTimeout, + List peonMonitors, + List javaOptsArray, + int cpuCoreInMicro, + Map labels, + Map annotations, + Integer capacity, + Period taskJoinTimeout + ) + { + this.namespace = namespace; + this.overlordNamespace = ObjectUtils.defaultIfNull( + overlordNamespace, + this.overlordNamespace + ); + this.k8sTaskPodNamePrefix = k8sTaskPodNamePrefix; + this.debugJobs = ObjectUtils.defaultIfNull( + debugJobs, + this.debugJobs + ); + this.sidecarSupport = ObjectUtils.defaultIfNull( + sidecarSupport, + this.sidecarSupport + ); + this.primaryContainerName = ObjectUtils.defaultIfNull( + primaryContainerName, + this.primaryContainerName + ); + this.kubexitImage = ObjectUtils.defaultIfNull( + kubexitImage, + this.kubexitImage + ); + this.graceTerminationPeriodSeconds = ObjectUtils.defaultIfNull( + graceTerminationPeriodSeconds, + this.graceTerminationPeriodSeconds + ); + this.disableClientProxy = disableClientProxy; + this.maxTaskDuration = ObjectUtils.defaultIfNull( + maxTaskDuration, + this.maxTaskDuration + ); + this.taskCleanupDelay = ObjectUtils.defaultIfNull( + taskCleanupDelay, + this.taskCleanupDelay + ); + this.taskCleanupInterval = ObjectUtils.defaultIfNull( + taskCleanupInterval, + this.taskCleanupInterval + ); + this.k8sjobLaunchTimeout = ObjectUtils.defaultIfNull( + k8sjobLaunchTimeout, + this.k8sjobLaunchTimeout + ); + this.taskJoinTimeout = ObjectUtils.defaultIfNull( + taskJoinTimeout, + this.taskJoinTimeout + ); + this.logSaveTimeout = ObjectUtils.defaultIfNull( + logSaveTimeout, + this.logSaveTimeout + ); + this.peonMonitors = ObjectUtils.defaultIfNull( + peonMonitors, + this.peonMonitors + ); + this.javaOptsArray = ObjectUtils.defaultIfNull( + javaOptsArray, + this.javaOptsArray + ); + this.cpuCoreInMicro = ObjectUtils.defaultIfNull( + cpuCoreInMicro, + this.cpuCoreInMicro + ); + this.labels = ObjectUtils.defaultIfNull( + labels, + this.labels + ); + this.annotations = ObjectUtils.defaultIfNull( + annotations, + this.annotations + ); + this.capacity = ObjectUtils.defaultIfNull( + capacity, + this.capacity + ); + } + + @Override + public String getNamespace() + { + return namespace; + } + + @Override + public String getOverlordNamespace() + { + return overlordNamespace; + } + + @Override + public String getK8sTaskPodNamePrefix() + { + return k8sTaskPodNamePrefix; + } + + @Override + public boolean isDebugJobs() + { + return debugJobs; + } + + @Override + @Deprecated + public boolean isSidecarSupport() + { + return sidecarSupport; + } + + @Override + public String getPrimaryContainerName() + { + return primaryContainerName; + } + + @Override + public String getKubexitImage() + { + return kubexitImage; + } + + @Override + public Long getGraceTerminationPeriodSeconds() + { + return graceTerminationPeriodSeconds; + } + + @Override + public boolean isDisableClientProxy() + { + return disableClientProxy; + } + + @Override + public Period getTaskTimeout() + { + return maxTaskDuration; + } + + @Override + public Period getTaskJoinTimeout() + { + return taskJoinTimeout; + } + + + @Override + public Period getTaskCleanupDelay() + { + return taskCleanupDelay; + } + + @Override + public Period getTaskCleanupInterval() + { + return taskCleanupInterval; + } + + @Override + public Period getTaskLaunchTimeout() + { + return k8sjobLaunchTimeout; + } + + @Override + public Period getLogSaveTimeout() + { + return logSaveTimeout; + } + + @Override + public List getPeonMonitors() + { + return peonMonitors; + } + + @Override + public List getJavaOptsArray() + { + return javaOptsArray; + } + + @Override + public int getCpuCoreInMicro() + { + return cpuCoreInMicro; + } + + @Override + public Map getLabels() + { + return labels; + } + + @Override + public Map getAnnotations() + { + return annotations; + } + + @Override + public Integer getCapacity() + { + return capacity; + } + + public static Builder builder() + { + return new Builder(); + } + + public static class Builder + { + private String namespace; + private String overlordNamespace; + private String k8sTaskPodNamePrefix; + private boolean debugJob; + private boolean sidecarSupport; + private String primaryContainerName; + private String kubexitImage; + private Long graceTerminationPeriodSeconds; + private boolean disableClientProxy; + private Period maxTaskDuration; + private Period taskCleanupDelay; + private Period taskCleanupInterval; + private Period k8sjobLaunchTimeout; + private List peonMonitors; + private List javaOptsArray; + private int cpuCoreInMicro; + private Map labels; + private Map annotations; + private Integer capacity; + private Period taskJoinTimeout; + private Period logSaveTimeout; + + public Builder() + { + } + + public Builder withNamespace(String namespace) + { + this.namespace = namespace; + return this; + } + + public Builder withOverlordNamespace(String overlordNamespace) + { + this.overlordNamespace = overlordNamespace; + return this; + } + + public Builder withK8sTaskPodNamePrefix(String k8sTaskPodNamePrefix) + { + this.k8sTaskPodNamePrefix = k8sTaskPodNamePrefix; + return this; + } + + public Builder withDebugJob(boolean debugJob) + { + this.debugJob = debugJob; + return this; + } + + public Builder withSidecarSupport(boolean sidecarSupport) + { + this.sidecarSupport = sidecarSupport; + return this; + } + + public Builder withPrimaryContainerName(String primaryContainerName) + { + this.primaryContainerName = primaryContainerName; + return this; + } + + public Builder withKubexitImage(String kubexitImage) + { + this.kubexitImage = kubexitImage; + return this; + } + + public Builder withGraceTerminationPeriodSeconds(Long graceTerminationPeriodSeconds) + { + this.graceTerminationPeriodSeconds = graceTerminationPeriodSeconds; + return this; + } + + public Builder withDisableClientProxy(boolean disableClientProxy) + { + this.disableClientProxy = disableClientProxy; + return this; + } + + public Builder withTaskTimeout(Period taskTimeout) + { + this.maxTaskDuration = taskTimeout; + return this; + } + + public Builder withTaskCleanupDelay(Period taskCleanupDelay) + { + this.taskCleanupDelay = taskCleanupDelay; + return this; + } + + public Builder withTaskCleanupInterval(Period taskCleanupInterval) + { + this.taskCleanupInterval = taskCleanupInterval; + return this; + } + + public Builder withK8sJobLaunchTimeout(Period k8sjobLaunchTimeout) + { + this.k8sjobLaunchTimeout = k8sjobLaunchTimeout; + return this; + } + + public Builder withPeonMonitors(List peonMonitors) + { + this.peonMonitors = peonMonitors; + return this; + } + + public Builder withCpuCore(int cpuCore) + { + this.cpuCoreInMicro = cpuCore; + return this; + } + + public Builder withJavaOptsArray(List javaOptsArray) + { + this.javaOptsArray = javaOptsArray; + return this; + } + + public Builder withLabels(Map labels) + { + this.labels = labels; + return this; + } + + public Builder withAnnotations(Map annotations) + { + this.annotations = annotations; + return this; + } + + + public Builder withCapacity(@Min(0) @Max(Integer.MAX_VALUE) Integer capacity) + { + this.capacity = capacity; + return this; + } + + public Builder withTaskJoinTimeout(Period taskJoinTimeout) + { + this.taskJoinTimeout = taskJoinTimeout; + return this; + } + + public Builder withLogSaveTimeout(Period logSaveTimeout) + { + this.logSaveTimeout = logSaveTimeout; + return this; + } + + public KubernetesTaskRunnerStaticConfig build() + { + return new KubernetesTaskRunnerStaticConfig( + this.namespace, + this.overlordNamespace, + this.k8sTaskPodNamePrefix, + this.debugJob, + this.sidecarSupport, + this.primaryContainerName, + this.kubexitImage, + this.graceTerminationPeriodSeconds, + this.disableClientProxy, + this.maxTaskDuration, + this.taskCleanupDelay, + this.taskCleanupInterval, + this.k8sjobLaunchTimeout, + this.logSaveTimeout, + this.peonMonitors, + this.javaOptsArray, + this.cpuCoreInMicro, + this.labels, + this.annotations, + this.capacity, + this.taskJoinTimeout + ); + } + } +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfig.java index eddd5e4a1ee1..fde4dcc7b0bf 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfig.java @@ -21,21 +21,26 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; +import javax.annotation.Nullable; import java.util.Objects; public class DefaultKubernetesTaskRunnerDynamicConfig implements KubernetesTaskRunnerDynamicConfig { + @Nullable private final PodTemplateSelectStrategy podTemplateSelectStrategy; + @Nullable + private final Integer capacity; + @JsonCreator public DefaultKubernetesTaskRunnerDynamicConfig( - @JsonProperty("podTemplateSelectStrategy") PodTemplateSelectStrategy podTemplateSelectStrategy + @JsonProperty("podTemplateSelectStrategy") PodTemplateSelectStrategy podTemplateSelectStrategy, + @JsonProperty("capacity") Integer capacity ) { - Preconditions.checkNotNull(podTemplateSelectStrategy); this.podTemplateSelectStrategy = podTemplateSelectStrategy; + this.capacity = capacity; } @Override @@ -45,6 +50,31 @@ public PodTemplateSelectStrategy getPodTemplateSelectStrategy() return podTemplateSelectStrategy; } + @Override + @JsonProperty + public Integer getCapacity() + { + return capacity; + } + + @Override + public KubernetesTaskRunnerDynamicConfig merge(KubernetesTaskRunnerDynamicConfig other) + { + if (other == null) { + return this; + } + Integer mergeCapacity = getCapacity(); + if (other.getCapacity() != null) { + mergeCapacity = other.getCapacity(); + } + + PodTemplateSelectStrategy mergePodTemplateSelectStrategy = getPodTemplateSelectStrategy(); + if (other.getPodTemplateSelectStrategy() != null) { + mergePodTemplateSelectStrategy = other.getPodTemplateSelectStrategy(); + } + return new DefaultKubernetesTaskRunnerDynamicConfig(mergePodTemplateSelectStrategy, mergeCapacity); + } + @Override public boolean equals(Object o) { @@ -55,13 +85,14 @@ public boolean equals(Object o) return false; } DefaultKubernetesTaskRunnerDynamicConfig that = (DefaultKubernetesTaskRunnerDynamicConfig) o; - return Objects.equals(podTemplateSelectStrategy, that.podTemplateSelectStrategy); + return Objects.equals(capacity, that.capacity) && + Objects.equals(podTemplateSelectStrategy, that.podTemplateSelectStrategy); } @Override public int hashCode() { - return Objects.hashCode(podTemplateSelectStrategy); + return Objects.hash(podTemplateSelectStrategy, capacity); } @Override @@ -69,6 +100,7 @@ public String toString() { return "DefaultKubernetesTaskRunnerDynamicConfig{" + "podTemplateSelectStrategy=" + podTemplateSelectStrategy + + "capacity=" + capacity + '}'; } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java index ec03b045f503..432a41933ede 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java @@ -73,7 +73,7 @@ public KubernetesTaskExecutionConfigResource( * Updates the Kubernetes execution configuration. * * @param dynamicConfig the new execution configuration to set - * @param req the HTTP servlet request providing context for audit information + * @param req the HTTP servlet request providing context for audit information * @return a response indicating the success or failure of the update operation */ @POST @@ -84,13 +84,19 @@ public Response setExecutionConfig( @Context final HttpServletRequest req ) { + KubernetesTaskRunnerDynamicConfig currentConfig = getDynamicConfig(); + KubernetesTaskRunnerDynamicConfig mergedConfig = dynamicConfig; + + if (currentConfig != null) { + mergedConfig = currentConfig.merge(dynamicConfig); + } final ConfigManager.SetResult setResult = configManager.set( KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, - dynamicConfig, + mergedConfig, AuthorizationUtils.buildAuditInfo(req) ); if (setResult.isOk()) { - log.info("Updating K8s execution configs: %s", dynamicConfig); + log.info("Updating K8s execution configs: %s", mergedConfig); return Response.ok().build(); } else { @@ -147,11 +153,15 @@ public Response getExecutionConfigHistory( @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(ConfigResourceFilter.class) public Response getExecutionConfig() + { + return Response.ok(getDynamicConfig()).build(); + } + + private KubernetesTaskRunnerDynamicConfig getDynamicConfig() { if (dynamicConfigRef == null) { dynamicConfigRef = configManager.watch(KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, KubernetesTaskRunnerDynamicConfig.class); } - - return Response.ok(dynamicConfigRef.get()).build(); + return dynamicConfigRef.get(); } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfig.java index 4f6d4b07c41d..fd9c9b465313 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfig.java @@ -22,6 +22,9 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import javax.validation.constraints.Max; +import javax.validation.constraints.Min; + /** * Represents the configuration for task execution within a Kubernetes environment. * This interface allows for dynamic configuration of task execution strategies based @@ -38,7 +41,26 @@ public interface KubernetesTaskRunnerDynamicConfig /** * Retrieves the execution behavior strategy associated with this configuration. + * * @return the execution behavior strategy */ PodTemplateSelectStrategy getPodTemplateSelectStrategy(); + + /** + * Retrieves the capacity associated with this configuration. + * + * @return the capacity + */ + @Min(0) + @Max(Integer.MAX_VALUE) + Integer getCapacity(); + + /** + * Merges this configuration with another, preferring values from {@code other} + * and falling back to this configuration when not present. + * + * @param other the configuration to merge with + * @return the merged configuration + */ + KubernetesTaskRunnerDynamicConfig merge(KubernetesTaskRunnerDynamicConfig other); } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelector.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelector.java index 92832b2ff660..ff4a4c2cd727 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelector.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelector.java @@ -26,8 +26,7 @@ import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; -import org.apache.druid.k8s.overlord.execution.PodTemplateSelectStrategy; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerEffectiveConfig; import java.io.File; import java.nio.file.Files; @@ -43,17 +42,17 @@ public class DynamicConfigPodTemplateSelector implements PodTemplateSelector + ".k8s.podTemplate."; private final Properties properties; - private final Supplier dynamicConfigRef; + private final KubernetesTaskRunnerEffectiveConfig effectiveConfig; // Supplier allows Overlord to read the most recent pod template file without calling initializeTemplatesFromFileSystem() again. private HashMap> podTemplates; public DynamicConfigPodTemplateSelector( Properties properties, - Supplier dynamicConfigRef + KubernetesTaskRunnerEffectiveConfig effectiveConfig ) { this.properties = properties; - this.dynamicConfigRef = dynamicConfigRef; + this.effectiveConfig = effectiveConfig; initializeTemplatesFromFileSystem(); } @@ -120,14 +119,6 @@ private void validateTemplateSupplier(Supplier templateSupplier) th @Override public Optional getPodTemplateForTask(Task task) { - PodTemplateSelectStrategy podTemplateSelectStrategy; - KubernetesTaskRunnerDynamicConfig dynamicConfig = dynamicConfigRef.get(); - if (dynamicConfig == null || dynamicConfig.getPodTemplateSelectStrategy() == null) { - podTemplateSelectStrategy = KubernetesTaskRunnerDynamicConfig.DEFAULT_STRATEGY; - } else { - podTemplateSelectStrategy = dynamicConfig.getPodTemplateSelectStrategy(); - } - - return Optional.of(podTemplateSelectStrategy.getPodTemplateForTask(task, podTemplates)); + return Optional.of(effectiveConfig.getPodTemplateSelectStrategy().getPodTemplateForTask(task, podTemplates)); } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index 2744b82b126d..8c68d9324e83 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -269,7 +269,7 @@ private Map getPodTemplateAnnotations(Task task) throws IOExcept } return podTemplateAnnotationBuilder.build(); } - + private Map getJobLabels(KubernetesTaskRunnerConfig config, Task task) { Preconditions.checkNotNull(config.getNamespace(), "When using Custom Pod Templates, druid.indexer.runner.namespace cannot be null."); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index 49720642216f..fe56e1946df0 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -41,7 +41,7 @@ public class KubernetesTaskRunnerFactoryTest { private ObjectMapper objectMapper; - private KubernetesTaskRunnerConfig kubernetesTaskRunnerConfig; + private KubernetesTaskRunnerEffectiveConfig kubernetesTaskRunnerConfig; private TaskLogs taskLogs; private DruidKubernetesClient druidKubernetesClient; @@ -52,13 +52,14 @@ public class KubernetesTaskRunnerFactoryTest public void setup() { objectMapper = new TestUtils().getTestObjectMapper(); - kubernetesTaskRunnerConfig = KubernetesTaskRunnerConfig.builder() + KubernetesTaskRunnerStaticConfig kubernetesTaskRunnerStaticConfig = KubernetesTaskRunnerStaticConfig.builder() .withCapacity(1) .build(); taskLogs = new NoopTaskLogs(); druidKubernetesClient = new DruidKubernetesClient(new DruidKubernetesHttpClientConfig(), new ConfigBuilder().build()); taskAdapter = new TestTaskAdapter(); + kubernetesTaskRunnerConfig = new KubernetesTaskRunnerEffectiveConfig(kubernetesTaskRunnerStaticConfig, () -> null); } @Test diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfigTest.java similarity index 94% rename from extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java rename to extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfigTest.java index 1f4a7281f649..999d29e5387b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfigTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfigTest.java @@ -29,15 +29,15 @@ import java.io.IOException; -public class KubernetesTaskRunnerConfigTest +public class KubernetesTaskRunnerStaticConfigTest { @Test public void test_deserializable() throws IOException { ObjectMapper mapper = new DefaultObjectMapper(); - KubernetesTaskRunnerConfig config = mapper.readValue( + KubernetesTaskRunnerStaticConfig config = mapper.readValue( this.getClass().getClassLoader().getResource("kubernetesTaskRunnerConfig.json"), - KubernetesTaskRunnerConfig.class + KubernetesTaskRunnerStaticConfig.class ); Assert.assertEquals("namespace", config.getNamespace()); @@ -60,7 +60,7 @@ public void test_deserializable() throws IOException @Test public void test_builder_preservesDefaults() { - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() .withNamespace("namespace") .withDisableClientProxy(true) .build(); @@ -85,7 +85,7 @@ public void test_builder_preservesDefaults() @Test public void test_builder() { - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() .withNamespace("namespace") .withDebugJob(true) .withSidecarSupport(true) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index 083bf2db0e6c..d942b49d6c1c 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -20,6 +20,7 @@ package org.apache.druid.k8s.overlord; import com.google.common.base.Optional; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -39,6 +40,8 @@ import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; +import org.apache.druid.k8s.overlord.execution.DefaultKubernetesTaskRunnerDynamicConfig; +import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; @@ -79,17 +82,22 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport @Mock private ServiceEmitter emitter; @Mock private ListenableFuture statusFuture; - private KubernetesTaskRunnerConfig config; + private KubernetesTaskRunnerStaticConfig staticConfig; + private KubernetesTaskRunnerEffectiveConfig config; private KubernetesTaskRunner runner; private Task task; @Before public void setup() { - config = KubernetesTaskRunnerConfig.builder() + staticConfig = KubernetesTaskRunnerStaticConfig.builder() .withCapacity(1) .build(); + Supplier dynamicConfigRef = () -> new DefaultKubernetesTaskRunnerDynamicConfig(null, 1); + + config = new KubernetesTaskRunnerEffectiveConfig(staticConfig, dynamicConfigRef); + task = K8sTestUtils.createTask(ID, 0); runner = new KubernetesTaskRunner( diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfigTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfigTest.java index de8919e329de..31c6d7fdd6de 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfigTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfigTest.java @@ -31,17 +31,26 @@ public class DefaultKubernetesTaskRunnerDynamicConfigTest public void getPodTemplateSelectStrategyTest() { PodTemplateSelectStrategy strategy = new TaskTypePodTemplateSelectStrategy(); - DefaultKubernetesTaskRunnerDynamicConfig config = new DefaultKubernetesTaskRunnerDynamicConfig(strategy); + DefaultKubernetesTaskRunnerDynamicConfig config = new DefaultKubernetesTaskRunnerDynamicConfig(strategy, 1); Assert.assertEquals(strategy, config.getPodTemplateSelectStrategy()); } + @Test + public void getCapacityTest() + { + Integer capacity = 4; + DefaultKubernetesTaskRunnerDynamicConfig config = new DefaultKubernetesTaskRunnerDynamicConfig(null, 4); + + Assert.assertEquals(capacity, config.getCapacity()); + } + @Test public void testSerde() throws Exception { final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); PodTemplateSelectStrategy strategy = new TaskTypePodTemplateSelectStrategy(); - DefaultKubernetesTaskRunnerDynamicConfig config = new DefaultKubernetesTaskRunnerDynamicConfig(strategy); + DefaultKubernetesTaskRunnerDynamicConfig config = new DefaultKubernetesTaskRunnerDynamicConfig(strategy, 1); DefaultKubernetesTaskRunnerDynamicConfig config2 = objectMapper.readValue( objectMapper.writeValueAsBytes(config), diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResourceTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResourceTest.java index b76b7eaf0cfe..c06056f01133 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResourceTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResourceTest.java @@ -30,6 +30,7 @@ import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; +import java.util.concurrent.atomic.AtomicReference; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -52,6 +53,10 @@ public void setUp() @Test public void setExecutionConfigSuccessfulUpdate() { + EasyMock.expect(configManager.watch( + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, + KubernetesTaskRunnerDynamicConfig.class + )).andReturn(new AtomicReference<>(null)); KubernetesTaskExecutionConfigResource testedResource = new KubernetesTaskExecutionConfigResource( configManager, auditManager @@ -75,6 +80,10 @@ public void setExecutionConfigSuccessfulUpdate() @Test public void setExecutionConfigFailedUpdate() { + EasyMock.expect(configManager.watch( + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, + KubernetesTaskRunnerDynamicConfig.class + )).andReturn(new AtomicReference<>(null)); KubernetesTaskExecutionConfigResource testedResource = new KubernetesTaskExecutionConfigResource( configManager, auditManager @@ -94,4 +103,116 @@ public void setExecutionConfigFailedUpdate() Response result = testedResource.setExecutionConfig(dynamicConfig, req); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), result.getStatus()); } + + @Test + public void setExecutionConfig_MergeUsesCurrentCapacityWhenRequestCapacityNull() + { + KubernetesTaskExecutionConfigResource testedResource = new KubernetesTaskExecutionConfigResource( + configManager, + auditManager + ); + + PodTemplateSelectStrategy currentStrategy = new TaskTypePodTemplateSelectStrategy(); + KubernetesTaskRunnerDynamicConfig currentConfig = new DefaultKubernetesTaskRunnerDynamicConfig(currentStrategy, 5); + EasyMock.expect(configManager.watch( + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, + KubernetesTaskRunnerDynamicConfig.class + )).andReturn(new AtomicReference<>(currentConfig)); + + PodTemplateSelectStrategy requestStrategy = new TaskTypePodTemplateSelectStrategy(); + KubernetesTaskRunnerDynamicConfig requestConfig = new DefaultKubernetesTaskRunnerDynamicConfig(requestStrategy, null); + + KubernetesTaskRunnerDynamicConfig expectedMergedConfig = new DefaultKubernetesTaskRunnerDynamicConfig(requestStrategy, 5); + + EasyMock.expect(req.getHeader(AuditManager.X_DRUID_AUTHOR)).andReturn(null).anyTimes(); + EasyMock.expect(req.getHeader(AuditManager.X_DRUID_COMMENT)).andReturn(null).anyTimes(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(null).anyTimes(); + EasyMock.expect(req.getRemoteAddr()).andReturn("127.0.0.1").anyTimes(); + EasyMock.replay(req); + + EasyMock.expect(configManager.set( + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, + expectedMergedConfig, + AuthorizationUtils.buildAuditInfo(req) + )).andReturn(ConfigManager.SetResult.ok()); + + EasyMock.replay(configManager, auditManager); + + Response result = testedResource.setExecutionConfig(requestConfig, req); + assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); + } + + @Test + public void setExecutionConfig_MergeUsesCurrentStrategyWhenRequestStrategyNull() + { + KubernetesTaskExecutionConfigResource testedResource = new KubernetesTaskExecutionConfigResource( + configManager, + auditManager + ); + + PodTemplateSelectStrategy currentStrategy = new TaskTypePodTemplateSelectStrategy(); + KubernetesTaskRunnerDynamicConfig currentConfig = new DefaultKubernetesTaskRunnerDynamicConfig(currentStrategy, 2); + EasyMock.expect(configManager.watch( + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, + KubernetesTaskRunnerDynamicConfig.class + )).andReturn(new AtomicReference<>(currentConfig)); + + KubernetesTaskRunnerDynamicConfig requestConfig = new DefaultKubernetesTaskRunnerDynamicConfig(null, 7); + + KubernetesTaskRunnerDynamicConfig expectedMergedConfig = new DefaultKubernetesTaskRunnerDynamicConfig(currentStrategy, 7); + + EasyMock.expect(req.getHeader(AuditManager.X_DRUID_AUTHOR)).andReturn(null).anyTimes(); + EasyMock.expect(req.getHeader(AuditManager.X_DRUID_COMMENT)).andReturn(null).anyTimes(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(null).anyTimes(); + EasyMock.expect(req.getRemoteAddr()).andReturn("127.0.0.1").anyTimes(); + EasyMock.replay(req); + + EasyMock.expect(configManager.set( + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, + expectedMergedConfig, + AuthorizationUtils.buildAuditInfo(req) + )).andReturn(ConfigManager.SetResult.ok()); + + EasyMock.replay(configManager, auditManager); + + Response result = testedResource.setExecutionConfig(requestConfig, req); + assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); + } + + @Test + public void setExecutionConfig_MergeUsesCurrentWhenBothRequestFieldsNull() + { + KubernetesTaskExecutionConfigResource testedResource = new KubernetesTaskExecutionConfigResource( + configManager, + auditManager + ); + + PodTemplateSelectStrategy currentStrategy = new TaskTypePodTemplateSelectStrategy(); + KubernetesTaskRunnerDynamicConfig currentConfig = new DefaultKubernetesTaskRunnerDynamicConfig(currentStrategy, 9); + EasyMock.expect(configManager.watch( + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, + KubernetesTaskRunnerDynamicConfig.class + )).andReturn(new AtomicReference<>(currentConfig)); + + KubernetesTaskRunnerDynamicConfig requestConfig = new DefaultKubernetesTaskRunnerDynamicConfig(null, null); + + KubernetesTaskRunnerDynamicConfig expectedMergedConfig = new DefaultKubernetesTaskRunnerDynamicConfig(currentStrategy, 9); + + EasyMock.expect(req.getHeader(AuditManager.X_DRUID_AUTHOR)).andReturn(null).anyTimes(); + EasyMock.expect(req.getHeader(AuditManager.X_DRUID_COMMENT)).andReturn(null).anyTimes(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(null).anyTimes(); + EasyMock.expect(req.getRemoteAddr()).andReturn("127.0.0.1").anyTimes(); + EasyMock.replay(req); + + EasyMock.expect(configManager.set( + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, + expectedMergedConfig, + AuthorizationUtils.buildAuditInfo(req) + )).andReturn(ConfigManager.SetResult.ok()); + + EasyMock.replay(configManager, auditManager); + + Response result = testedResource.setExecutionConfig(requestConfig, req); + assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); + } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java index 77a819dde9c1..b5fac1233c09 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java @@ -36,7 +36,8 @@ public void testSerde() throws JsonProcessingException + " \"type\": \"default\",\n" + " \"podTemplateSelectStrategy\": {\n" + " \"type\": \"default\"\n" - + " }\n" + + " },\n" + + " \"capacity\": 3\n" + "}"; KubernetesTaskRunnerDynamicConfig deserialized = jsonMapper.readValue( @@ -45,6 +46,7 @@ public void testSerde() throws JsonProcessingException ); PodTemplateSelectStrategy selectStrategy = deserialized.getPodTemplateSelectStrategy(); Assert.assertTrue(selectStrategy instanceof TaskTypePodTemplateSelectStrategy); + Assert.assertEquals(Integer.valueOf(3), deserialized.getCapacity()); json = "{\n" + " \"type\": \"default\",\n" @@ -72,5 +74,14 @@ public void testSerde() throws JsonProcessingException selectStrategy = deserialized.getPodTemplateSelectStrategy(); Assert.assertTrue(selectStrategy instanceof SelectorBasedPodTemplateSelectStrategy); Assert.assertEquals(2, ((SelectorBasedPodTemplateSelectStrategy) selectStrategy).getSelectors().size()); + Assert.assertNull(deserialized.getCapacity()); + + json = "{\n" + + " \"type\": \"default\",\n" + + " \"capacity\": 12" + + "}"; + deserialized = jsonMapper.readValue(json, KubernetesTaskRunnerDynamicConfig.class); + Assert.assertEquals(Integer.valueOf(12), deserialized.getCapacity()); + Assert.assertNull(deserialized.getPodTemplateSelectStrategy()); } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 446c2502fa11..ead3c84fe6b7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -33,7 +33,7 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; -import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesHttpClientConfig; import org.apache.druid.k8s.overlord.common.JobResponse; @@ -109,9 +109,9 @@ public void testDeployingSomethingToKind(@TempDir Path tempDir) throws Exception PodSpec podSpec = K8sTestUtils.getDummyPodSpec(); Task task = K8sTestUtils.getTask(); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("default") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("default") + .build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( k8sClient, config, diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelectorTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelectorTest.java index 106a98fa6a8e..762c93c6607d 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelectorTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelectorTest.java @@ -29,6 +29,8 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerEffectiveConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.execution.DefaultKubernetesTaskRunnerDynamicConfig; import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; @@ -53,14 +55,16 @@ public class DynamicConfigPodTemplateSelectorTest private Path tempDir; private ObjectMapper mapper; private PodTemplate podTemplateSpec; - private Supplier dynamicConfigRef; + private KubernetesTaskRunnerEffectiveConfig effectiveConfig; @BeforeEach public void setup() { mapper = new TestUtils().getTestObjectMapper(); podTemplateSpec = K8sTestUtils.fileToResource("basePodTemplate.yaml", PodTemplate.class); - dynamicConfigRef = () -> new DefaultKubernetesTaskRunnerDynamicConfig(KubernetesTaskRunnerDynamicConfig.DEFAULT_STRATEGY); + Supplier dynamicConfigRef = () -> new DefaultKubernetesTaskRunnerDynamicConfig(KubernetesTaskRunnerDynamicConfig.DEFAULT_STRATEGY, 1); + KubernetesTaskRunnerStaticConfig staticConfig = KubernetesTaskRunnerStaticConfig.builder().build(); + effectiveConfig = new KubernetesTaskRunnerEffectiveConfig(staticConfig, dynamicConfigRef); } @Test @@ -71,7 +75,7 @@ public void test_fromTask_withoutBasePodTemplateInRuntimeProperites_raisesIAE() IAE.class, () -> new DynamicConfigPodTemplateSelector( new Properties(), - dynamicConfigRef + effectiveConfig ) ); Assertions.assertEquals( @@ -93,7 +97,7 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites_withEmptyFile_r IAE.class, () -> new DynamicConfigPodTemplateSelector( props, - dynamicConfigRef + effectiveConfig ) ); @@ -111,7 +115,7 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites() throws IOExce DynamicConfigPodTemplateSelector adapter = new DynamicConfigPodTemplateSelector( props, - dynamicConfigRef + effectiveConfig ); Task task = new NoopTask("id", "id", "datasource", 0, 0, null); @@ -146,7 +150,7 @@ public void test_fromTask_withIndexKafkaPodTemplateInRuntimeProperties() throws DynamicConfigPodTemplateSelector selector = new DynamicConfigPodTemplateSelector( props, - dynamicConfigRef + effectiveConfig ); Task kafkaTask = new NoopTask("id", "id", "datasource", 0, 0, null) @@ -191,9 +195,10 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperties_withEmptyFile_r props.setProperty("druid.indexer.runner.k8s.podTemplate.noop", noopTemplatePath.toString()); Assert.assertThrows(IAE.class, () -> new DynamicConfigPodTemplateSelector( - props, - dynamicConfigRef - )); + props, + effectiveConfig + ) + ); } @Test @@ -208,7 +213,7 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperites() throws IOExce DynamicConfigPodTemplateSelector podTemplateSelector = new DynamicConfigPodTemplateSelector( props, - dynamicConfigRef + effectiveConfig ); Task task = new NoopTask("id", "id", "datasource", 0, 0, null); @@ -242,16 +247,21 @@ public void test_fromTask_matchPodTemplateBasedOnStrategy() throws IOException Properties props = new Properties(); props.setProperty("druid.indexer.runner.k8s.podTemplate.base", baseTemplatePath.toString()); props.setProperty("druid.indexer.runner.k8s.podTemplate.lowThroughput", lowThroughputTemplatePath.toString()); - dynamicConfigRef = () -> new DefaultKubernetesTaskRunnerDynamicConfig(new SelectorBasedPodTemplateSelectStrategy( - Collections.singletonList( - new Selector("lowThroughput", null, null, Sets.newSet(dataSource) + Supplier dynamicConfigRef = () -> new DefaultKubernetesTaskRunnerDynamicConfig( + new SelectorBasedPodTemplateSelectStrategy( + Collections.singletonList( + new Selector("lowThroughput", null, null, Sets.newSet(dataSource) + ) ) - ) - )); + ), 1 + ); + + KubernetesTaskRunnerStaticConfig staticConfig = KubernetesTaskRunnerStaticConfig.builder().build(); + effectiveConfig = new KubernetesTaskRunnerEffectiveConfig(staticConfig, dynamicConfigRef); DynamicConfigPodTemplateSelector podTemplateSelector = new DynamicConfigPodTemplateSelector( props, - dynamicConfigRef + effectiveConfig ); Task taskWithMatchedDatasource = new NoopTask("id", "id", dataSource, 0, 0, null); @@ -276,7 +286,7 @@ public void test_fromTask_LazyLoadInvalidPodTemplateThrowsError() throws IOExcep DynamicConfigPodTemplateSelector adapter = new DynamicConfigPodTemplateSelector( props, - dynamicConfigRef + effectiveConfig ); Task task = new NoopTask("id", "id", "datasource", 0, 0, null); @@ -311,7 +321,7 @@ public void test_fromTask_LazyLoadPodTemplateChangesPodSpecs() throws IOExceptio DynamicConfigPodTemplateSelector adapter = new DynamicConfigPodTemplateSelector( props, - dynamicConfigRef + effectiveConfig ); Task task = new NoopTask("id", "id", "datasource", 0, 0, null); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index 16cccf04e280..547c2e3c63e8 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -51,7 +51,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.java.util.common.HumanReadableBytes; -import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; @@ -138,12 +138,12 @@ public PodSpec getSpec() } }; - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("test") - .withOverlordNamespace("test_different") - .withAnnotations(ImmutableMap.of("annotation_key", "annotation_value")) - .withLabels(ImmutableMap.of("label_key", "label_value")) - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("test") + .withOverlordNamespace("test_different") + .withAnnotations(ImmutableMap.of("annotation_key", "annotation_value")) + .withLabels(ImmutableMap.of("label_key", "label_value")) + .build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, @@ -175,9 +175,9 @@ public void serializingAndDeserializingATask() throws IOException { // given a task create a k8s job TestKubernetesClient testClient = new TestKubernetesClient(client); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("test") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("test") + .build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, @@ -229,9 +229,9 @@ public PodSpec getSpec() } }; - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("test") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("test") + .build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, @@ -277,9 +277,9 @@ public PodSpec getSpec() public void toTask_useTaskPayloadManager() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("test") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("test") + .build(); Task taskInTaskPayloadManager = K8sTestUtils.getTask(); TaskLogs mockTestLogs = Mockito.mock(TaskLogs.class); Mockito.when(mockTestLogs.streamTaskPayload("ID")).thenReturn(com.google.common.base.Optional.of( @@ -309,7 +309,7 @@ public void toTask_useTaskPayloadManager() throws IOException public void getTaskId() { TestKubernetesClient testClient = new TestKubernetesClient(client); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, @@ -331,7 +331,7 @@ public void getTaskId() public void getTaskId_noAnnotations() { TestKubernetesClient testClient = new TestKubernetesClient(client); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, @@ -353,7 +353,7 @@ public void getTaskId_noAnnotations() public void getTaskId_missingTaskIdAnnotation() { TestKubernetesClient testClient = new TestKubernetesClient(client); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder().build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, @@ -458,9 +458,9 @@ void testAddingMonitors() throws IOException new File("/tmp/"), 0 ); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("test") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("test") + .build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, @@ -481,10 +481,10 @@ void testAddingMonitors() throws IOException ); // we have an override, but nothing in the overlord - config = KubernetesTaskRunnerConfig.builder() - .withNamespace("test") - .withPeonMonitors(ImmutableList.of("org.apache.druid.java.util.metrics.JvmMonitor")) - .build(); + config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("test") + .withPeonMonitors(ImmutableList.of("org.apache.druid.java.util.metrics.JvmMonitor")) + .build(); adapter = new SingleContainerTaskAdapter( testClient, config, @@ -532,9 +532,9 @@ void testEphemeralStorageIsRespected() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("ephemeralPodSpec.yaml", Pod.class); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("namespace") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("namespace") + .build(); SingleContainerTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, @@ -582,9 +582,9 @@ void testProbesRemoved() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("probesPodSpec.yaml", Pod.class); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("test") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("test") + .build(); SingleContainerTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, @@ -635,11 +635,11 @@ void testCPUResourceIsRespected() throws IOException List javaOpts = new ArrayList<>(); javaOpts.add("-Xms1G -Xmx2G -XX:MaxDirectMemorySize=3G"); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("namespace") - .withJavaOptsArray(javaOpts) - .withCpuCore(2000) - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("namespace") + .withJavaOptsArray(javaOpts) + .withCpuCore(2000) + .build(); SingleContainerTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java index 48330d625069..9a5b91b4b7d9 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java @@ -33,7 +33,7 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; -import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.PeonCommandContext; import org.apache.druid.k8s.overlord.common.TestKubernetesClient; @@ -87,9 +87,9 @@ public void testMultiContainerSupport() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpec.yaml", Pod.class); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("namespace") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("namespace") + .build(); MultiContainerTaskAdapter adapter = new MultiContainerTaskAdapter( testClient, config, @@ -138,10 +138,10 @@ public void testMultiContainerSupportWithNamedContainer() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpecOrder.yaml", Pod.class); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("namespace") - .withPrimaryContainerName("primary") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("namespace") + .withPrimaryContainerName("primary") + .build(); MultiContainerTaskAdapter adapter = new MultiContainerTaskAdapter( testClient, config, @@ -192,11 +192,11 @@ public void testOverridingPeonMonitors() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("podSpec.yaml", Pod.class); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("namespace") - .withPrimaryContainerName("primary") - .withPeonMonitors(ImmutableList.of("org.apache.druid.java.util.metrics.JvmMonitor")) - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("namespace") + .withPrimaryContainerName("primary") + .withPeonMonitors(ImmutableList.of("org.apache.druid.java.util.metrics.JvmMonitor")) + .build(); MultiContainerTaskAdapter adapter = new MultiContainerTaskAdapter( testClient, diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 45f2c356cfe0..649986d238cc 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -32,7 +32,7 @@ import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.Base64Compression; import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTaskId; @@ -58,7 +58,7 @@ public class PodTemplateTaskAdapterTest { - private KubernetesTaskRunnerConfig taskRunnerConfig; + private KubernetesTaskRunnerStaticConfig taskRunnerConfig; private PodTemplate podTemplateSpec; private TaskConfig taskConfig; private DruidNode node; @@ -68,7 +68,7 @@ public class PodTemplateTaskAdapterTest @BeforeEach public void setup() { - taskRunnerConfig = KubernetesTaskRunnerConfig.builder().withNamespace("namespace").build(); + taskRunnerConfig = KubernetesTaskRunnerStaticConfig.builder().withNamespace("namespace").build(); taskConfig = new TaskConfigBuilder().setBaseDir("/tmp").build(); node = new DruidNode( "test", @@ -189,7 +189,7 @@ public void test_getTaskId() public void test_getTaskIdWithK8sTaskPodNamePrefix() { TestPodTemplateSelector podTemplateSelector = new TestPodTemplateSelector(podTemplateSpec); - taskRunnerConfig = KubernetesTaskRunnerConfig.builder().withK8sTaskPodNamePrefix("k8sTaskPodNamePrefix").build(); + taskRunnerConfig = KubernetesTaskRunnerStaticConfig.builder().withK8sTaskPodNamePrefix("k8sTaskPodNamePrefix").build(); PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( taskRunnerConfig, diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java index d7d74cf1812a..f5a486b43d8b 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java @@ -31,7 +31,7 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; -import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.PeonCommandContext; import org.apache.druid.k8s.overlord.common.TestKubernetesClient; @@ -86,9 +86,9 @@ public void testSingleContainerSupport() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpec.yaml", Pod.class); - KubernetesTaskRunnerConfig config = KubernetesTaskRunnerConfig.builder() - .withNamespace("namespace") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + .withNamespace("namespace") + .build(); SingleContainerTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, From c4386f18c4ddf24c0de7e1afa319cf1814c3a2cc Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Thu, 2 Oct 2025 09:51:20 +0800 Subject: [PATCH 02/19] Update docs --- docs/development/extensions-core/k8s-jobs.md | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/docs/development/extensions-core/k8s-jobs.md b/docs/development/extensions-core/k8s-jobs.md index 4d75a65ba6ff..b8c066a5dae5 100644 --- a/docs/development/extensions-core/k8s-jobs.md +++ b/docs/development/extensions-core/k8s-jobs.md @@ -48,9 +48,9 @@ Other configurations required are: Druid operators can dynamically tune certain features within this extension. You don't need to restart the Overlord service for these changes to take effect. -Druid can dynamically tune [pod template selection](#pod-template-selection), which allows you to configure the pod -template based on the task to be run. To enable dynamic pod template selection, first configure the -[custom template pod adapter](#custom-template-pod-adapter). +Druid can dynamically tune [pod template selection](#pod-template-selection) and [capacity](#properties). + +Pod template selection allows you to configure the pod template based on the task to be run. To enable dynamic pod template selection, first configure the [custom template pod adapter](#custom-template-pod-adapter). Use the following APIs to view and update the dynamic configuration for the Kubernetes task runner. @@ -126,7 +126,8 @@ Host: http://ROUTER_IP:ROUTER_PORT "type": ["index_kafka"] } ] - } + }, + "capacity": 12 } ``` @@ -193,7 +194,8 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/k8s/taskrunner/executionconf "type": ["index_kafka"] } ] - } + }, + "capacity": 6 }' ``` @@ -225,7 +227,8 @@ Content-Type: application/json "type": ["index_kafka"] } ] - } + }, + "capacity": 6 } ``` @@ -309,7 +312,7 @@ Host: http://ROUTER_IP:ROUTER_PORT "comment": "", "ip": "127.0.0.1" }, - "payload": "{\"type\": \"default\",\"podTemplateSelectStrategy\":{\"type\": \"taskType\"}", + "payload": "{\"type\": \"default\",\"podTemplateSelectStrategy\":{\"type\": \"taskType\"},\"capacity\":6", "auditTime": "2024-06-13T20:59:51.622Z" } ] From 579a12c6fc1833ab798304f386d2abf3f69e31ce Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Fri, 3 Oct 2025 16:59:28 +0800 Subject: [PATCH 03/19] Refactor to use getIfNull --- .../KubernetesTaskRunnerStaticConfig.java | 230 ++---------------- 1 file changed, 19 insertions(+), 211 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java index 39f857f13f3d..1813b2ee7bff 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java @@ -139,11 +139,7 @@ public class KubernetesTaskRunnerStaticConfig implements KubernetesTaskRunnerCon @NotNull private Integer capacity = Integer.MAX_VALUE; - public KubernetesTaskRunnerStaticConfig() - { - } - - private KubernetesTaskRunnerStaticConfig( + public KubernetesTaskRunnerStaticConfig( @Nonnull String namespace, String overlordNamespace, String k8sTaskPodNamePrefix, @@ -168,77 +164,77 @@ private KubernetesTaskRunnerStaticConfig( ) { this.namespace = namespace; - this.overlordNamespace = ObjectUtils.defaultIfNull( + this.overlordNamespace = ObjectUtils.getIfNull( overlordNamespace, this.overlordNamespace ); this.k8sTaskPodNamePrefix = k8sTaskPodNamePrefix; - this.debugJobs = ObjectUtils.defaultIfNull( + this.debugJobs = ObjectUtils.getIfNull( debugJobs, this.debugJobs ); - this.sidecarSupport = ObjectUtils.defaultIfNull( + this.sidecarSupport = ObjectUtils.getIfNull( sidecarSupport, this.sidecarSupport ); - this.primaryContainerName = ObjectUtils.defaultIfNull( + this.primaryContainerName = ObjectUtils.getIfNull( primaryContainerName, this.primaryContainerName ); - this.kubexitImage = ObjectUtils.defaultIfNull( + this.kubexitImage = ObjectUtils.getIfNull( kubexitImage, this.kubexitImage ); - this.graceTerminationPeriodSeconds = ObjectUtils.defaultIfNull( + this.graceTerminationPeriodSeconds = ObjectUtils.getIfNull( graceTerminationPeriodSeconds, this.graceTerminationPeriodSeconds ); this.disableClientProxy = disableClientProxy; - this.maxTaskDuration = ObjectUtils.defaultIfNull( + this.maxTaskDuration = ObjectUtils.getIfNull( maxTaskDuration, this.maxTaskDuration ); - this.taskCleanupDelay = ObjectUtils.defaultIfNull( + this.taskCleanupDelay = ObjectUtils.getIfNull( taskCleanupDelay, this.taskCleanupDelay ); - this.taskCleanupInterval = ObjectUtils.defaultIfNull( + this.taskCleanupInterval = ObjectUtils.getIfNull( taskCleanupInterval, this.taskCleanupInterval ); - this.k8sjobLaunchTimeout = ObjectUtils.defaultIfNull( + this.k8sjobLaunchTimeout = ObjectUtils.getIfNull( k8sjobLaunchTimeout, this.k8sjobLaunchTimeout ); - this.taskJoinTimeout = ObjectUtils.defaultIfNull( + this.taskJoinTimeout = ObjectUtils.getIfNull( taskJoinTimeout, this.taskJoinTimeout ); - this.logSaveTimeout = ObjectUtils.defaultIfNull( + this.logSaveTimeout = ObjectUtils.getIfNull( logSaveTimeout, this.logSaveTimeout ); - this.peonMonitors = ObjectUtils.defaultIfNull( + this.peonMonitors = ObjectUtils.getIfNull( peonMonitors, this.peonMonitors ); - this.javaOptsArray = ObjectUtils.defaultIfNull( + this.javaOptsArray = ObjectUtils.getIfNull( javaOptsArray, this.javaOptsArray ); - this.cpuCoreInMicro = ObjectUtils.defaultIfNull( + this.cpuCoreInMicro = ObjectUtils.getIfNull( cpuCoreInMicro, this.cpuCoreInMicro ); - this.labels = ObjectUtils.defaultIfNull( + this.labels = ObjectUtils.getIfNull( labels, this.labels ); - this.annotations = ObjectUtils.defaultIfNull( + this.annotations = ObjectUtils.getIfNull( annotations, this.annotations ); - this.capacity = ObjectUtils.defaultIfNull( + this.capacity = ObjectUtils.getIfNull( capacity, this.capacity ); @@ -371,192 +367,4 @@ public Integer getCapacity() { return capacity; } - - public static Builder builder() - { - return new Builder(); - } - - public static class Builder - { - private String namespace; - private String overlordNamespace; - private String k8sTaskPodNamePrefix; - private boolean debugJob; - private boolean sidecarSupport; - private String primaryContainerName; - private String kubexitImage; - private Long graceTerminationPeriodSeconds; - private boolean disableClientProxy; - private Period maxTaskDuration; - private Period taskCleanupDelay; - private Period taskCleanupInterval; - private Period k8sjobLaunchTimeout; - private List peonMonitors; - private List javaOptsArray; - private int cpuCoreInMicro; - private Map labels; - private Map annotations; - private Integer capacity; - private Period taskJoinTimeout; - private Period logSaveTimeout; - - public Builder() - { - } - - public Builder withNamespace(String namespace) - { - this.namespace = namespace; - return this; - } - - public Builder withOverlordNamespace(String overlordNamespace) - { - this.overlordNamespace = overlordNamespace; - return this; - } - - public Builder withK8sTaskPodNamePrefix(String k8sTaskPodNamePrefix) - { - this.k8sTaskPodNamePrefix = k8sTaskPodNamePrefix; - return this; - } - - public Builder withDebugJob(boolean debugJob) - { - this.debugJob = debugJob; - return this; - } - - public Builder withSidecarSupport(boolean sidecarSupport) - { - this.sidecarSupport = sidecarSupport; - return this; - } - - public Builder withPrimaryContainerName(String primaryContainerName) - { - this.primaryContainerName = primaryContainerName; - return this; - } - - public Builder withKubexitImage(String kubexitImage) - { - this.kubexitImage = kubexitImage; - return this; - } - - public Builder withGraceTerminationPeriodSeconds(Long graceTerminationPeriodSeconds) - { - this.graceTerminationPeriodSeconds = graceTerminationPeriodSeconds; - return this; - } - - public Builder withDisableClientProxy(boolean disableClientProxy) - { - this.disableClientProxy = disableClientProxy; - return this; - } - - public Builder withTaskTimeout(Period taskTimeout) - { - this.maxTaskDuration = taskTimeout; - return this; - } - - public Builder withTaskCleanupDelay(Period taskCleanupDelay) - { - this.taskCleanupDelay = taskCleanupDelay; - return this; - } - - public Builder withTaskCleanupInterval(Period taskCleanupInterval) - { - this.taskCleanupInterval = taskCleanupInterval; - return this; - } - - public Builder withK8sJobLaunchTimeout(Period k8sjobLaunchTimeout) - { - this.k8sjobLaunchTimeout = k8sjobLaunchTimeout; - return this; - } - - public Builder withPeonMonitors(List peonMonitors) - { - this.peonMonitors = peonMonitors; - return this; - } - - public Builder withCpuCore(int cpuCore) - { - this.cpuCoreInMicro = cpuCore; - return this; - } - - public Builder withJavaOptsArray(List javaOptsArray) - { - this.javaOptsArray = javaOptsArray; - return this; - } - - public Builder withLabels(Map labels) - { - this.labels = labels; - return this; - } - - public Builder withAnnotations(Map annotations) - { - this.annotations = annotations; - return this; - } - - - public Builder withCapacity(@Min(0) @Max(Integer.MAX_VALUE) Integer capacity) - { - this.capacity = capacity; - return this; - } - - public Builder withTaskJoinTimeout(Period taskJoinTimeout) - { - this.taskJoinTimeout = taskJoinTimeout; - return this; - } - - public Builder withLogSaveTimeout(Period logSaveTimeout) - { - this.logSaveTimeout = logSaveTimeout; - return this; - } - - public KubernetesTaskRunnerStaticConfig build() - { - return new KubernetesTaskRunnerStaticConfig( - this.namespace, - this.overlordNamespace, - this.k8sTaskPodNamePrefix, - this.debugJob, - this.sidecarSupport, - this.primaryContainerName, - this.kubexitImage, - this.graceTerminationPeriodSeconds, - this.disableClientProxy, - this.maxTaskDuration, - this.taskCleanupDelay, - this.taskCleanupInterval, - this.k8sjobLaunchTimeout, - this.logSaveTimeout, - this.peonMonitors, - this.javaOptsArray, - this.cpuCoreInMicro, - this.labels, - this.annotations, - this.capacity, - this.taskJoinTimeout - ); - } - } } From 9f9e845898d7702717c9de1278fbbdc9bf1acdcc Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Fri, 3 Oct 2025 17:05:24 +0800 Subject: [PATCH 04/19] Update docs wording --- docs/development/extensions-core/k8s-jobs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/development/extensions-core/k8s-jobs.md b/docs/development/extensions-core/k8s-jobs.md index b8c066a5dae5..375fc95dd296 100644 --- a/docs/development/extensions-core/k8s-jobs.md +++ b/docs/development/extensions-core/k8s-jobs.md @@ -48,7 +48,7 @@ Other configurations required are: Druid operators can dynamically tune certain features within this extension. You don't need to restart the Overlord service for these changes to take effect. -Druid can dynamically tune [pod template selection](#pod-template-selection) and [capacity](#properties). +Druid can dynamically tune [pod template selection](#pod-template-selection) and [capacity](#properties). Where capacity refers to `druid.indexer.runner.capacity`. Pod template selection allows you to configure the pod template based on the task to be run. To enable dynamic pod template selection, first configure the [custom template pod adapter](#custom-template-pod-adapter). From 5ae5d4edc90e08e92c20034e6072d10004899416 Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Fri, 3 Oct 2025 17:15:21 +0800 Subject: [PATCH 05/19] Fix based on comments --- .../k8s/overlord/KubernetesTaskRunner.java | 4 +- .../overlord/KubernetesTaskRunnerConfig.java | 191 ++++++++++++++++++ .../KubernetesTaskRunnerEffectiveConfig.java | 5 +- .../KubernetesTaskRunnerStaticConfig.java | 6 + .../KubernetesTaskRunnerFactoryTest.java | 2 +- .../KubernetesTaskRunnerStaticConfigTest.java | 4 +- .../overlord/KubernetesTaskRunnerTest.java | 2 +- .../DruidPeonClientIntegrationTest.java | 3 +- .../DynamicConfigPodTemplateSelectorTest.java | 5 +- .../taskadapter/K8sTaskAdapterTest.java | 25 +-- .../MultiContainerTaskAdapterTest.java | 11 +- .../PodTemplateTaskAdapterTest.java | 5 +- .../SingleContainerTaskAdapterTest.java | 7 +- 13 files changed, 238 insertions(+), 32 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 62b9f19da761..824100b51d31 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -111,7 +111,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner protected final TaskAdapter adapter; private final KubernetesPeonClient client; - private final KubernetesTaskRunnerEffectiveConfig config; + private final KubernetesTaskRunnerConfig config; private final ListeningExecutorService exec; private final ThreadPoolExecutor tpe; private final HttpClient httpClient; @@ -124,7 +124,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner public KubernetesTaskRunner( TaskAdapter adapter, - KubernetesTaskRunnerEffectiveConfig config, + KubernetesTaskRunnerConfig config, KubernetesPeonClient client, HttpClient httpClient, PeonLifecycleFactory peonLifecycleFactory, diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java index cdf08374ad9f..e7b9eb2b8043 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerConfig.java @@ -21,6 +21,9 @@ import org.joda.time.Period; +import javax.validation.constraints.Max; +import javax.validation.constraints.Min; + import java.util.List; import java.util.Map; @@ -68,4 +71,192 @@ public interface KubernetesTaskRunnerConfig Map getAnnotations(); Integer getCapacity(); + + static Builder builder() + { + return new Builder(); + } + + public static class Builder + { + private String namespace; + private String overlordNamespace; + private String k8sTaskPodNamePrefix; + private boolean debugJob; + private boolean sidecarSupport; + private String primaryContainerName; + private String kubexitImage; + private Long graceTerminationPeriodSeconds; + private boolean disableClientProxy; + private Period maxTaskDuration; + private Period taskCleanupDelay; + private Period taskCleanupInterval; + private Period k8sjobLaunchTimeout; + private List peonMonitors; + private List javaOptsArray; + private int cpuCoreInMicro; + private Map labels; + private Map annotations; + private Integer capacity; + private Period taskJoinTimeout; + private Period logSaveTimeout; + + public Builder() + { + } + + public Builder withNamespace(String namespace) + { + this.namespace = namespace; + return this; + } + + public Builder withOverlordNamespace(String overlordNamespace) + { + this.overlordNamespace = overlordNamespace; + return this; + } + + public Builder withK8sTaskPodNamePrefix(String k8sTaskPodNamePrefix) + { + this.k8sTaskPodNamePrefix = k8sTaskPodNamePrefix; + return this; + } + + public Builder withDebugJob(boolean debugJob) + { + this.debugJob = debugJob; + return this; + } + + public Builder withSidecarSupport(boolean sidecarSupport) + { + this.sidecarSupport = sidecarSupport; + return this; + } + + public Builder withPrimaryContainerName(String primaryContainerName) + { + this.primaryContainerName = primaryContainerName; + return this; + } + + public Builder withKubexitImage(String kubexitImage) + { + this.kubexitImage = kubexitImage; + return this; + } + + public Builder withGraceTerminationPeriodSeconds(Long graceTerminationPeriodSeconds) + { + this.graceTerminationPeriodSeconds = graceTerminationPeriodSeconds; + return this; + } + + public Builder withDisableClientProxy(boolean disableClientProxy) + { + this.disableClientProxy = disableClientProxy; + return this; + } + + public Builder withTaskTimeout(Period taskTimeout) + { + this.maxTaskDuration = taskTimeout; + return this; + } + + public Builder withTaskCleanupDelay(Period taskCleanupDelay) + { + this.taskCleanupDelay = taskCleanupDelay; + return this; + } + + public Builder withTaskCleanupInterval(Period taskCleanupInterval) + { + this.taskCleanupInterval = taskCleanupInterval; + return this; + } + + public Builder withK8sJobLaunchTimeout(Period k8sjobLaunchTimeout) + { + this.k8sjobLaunchTimeout = k8sjobLaunchTimeout; + return this; + } + + public Builder withPeonMonitors(List peonMonitors) + { + this.peonMonitors = peonMonitors; + return this; + } + + public Builder withCpuCore(int cpuCore) + { + this.cpuCoreInMicro = cpuCore; + return this; + } + + public Builder withJavaOptsArray(List javaOptsArray) + { + this.javaOptsArray = javaOptsArray; + return this; + } + + public Builder withLabels(Map labels) + { + this.labels = labels; + return this; + } + + public Builder withAnnotations(Map annotations) + { + this.annotations = annotations; + return this; + } + + + public Builder withCapacity(@Min(0) @Max(Integer.MAX_VALUE) Integer capacity) + { + this.capacity = capacity; + return this; + } + + public Builder withTaskJoinTimeout(Period taskJoinTimeout) + { + this.taskJoinTimeout = taskJoinTimeout; + return this; + } + + public Builder withLogSaveTimeout(Period logSaveTimeout) + { + this.logSaveTimeout = logSaveTimeout; + return this; + } + + public KubernetesTaskRunnerStaticConfig build() + { + return new KubernetesTaskRunnerStaticConfig( + this.namespace, + this.overlordNamespace, + this.k8sTaskPodNamePrefix, + this.debugJob, + this.sidecarSupport, + this.primaryContainerName, + this.kubexitImage, + this.graceTerminationPeriodSeconds, + this.disableClientProxy, + this.maxTaskDuration, + this.taskCleanupDelay, + this.taskCleanupInterval, + this.k8sjobLaunchTimeout, + this.logSaveTimeout, + this.peonMonitors, + this.javaOptsArray, + this.cpuCoreInMicro, + this.labels, + this.annotations, + this.capacity, + this.taskJoinTimeout + ); + } + } } diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfig.java index 92f5ceb7d0d7..c1593c3577db 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfig.java @@ -28,8 +28,11 @@ import java.util.Map; /** - * Effective config object that combines static {@link KubernetesTaskRunnerConfig} + * Provides a flexible mechanism to configure Kubernetes task pods, + * by merging the static base settings from {@link KubernetesTaskRunnerConfig} * with dynamic overrides from {@link KubernetesTaskRunnerDynamicConfig}. + *

+ * Kubernetes will always use this effective config to run new tasks. */ public class KubernetesTaskRunnerEffectiveConfig implements KubernetesTaskRunnerConfig { diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java index 1813b2ee7bff..959361605ab7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java @@ -32,6 +32,12 @@ import java.util.List; import java.util.Map; +/** + * Static configuration for the Kubernetes-based task runner used by the Overlord. + *

+ * This configuration is populated from runtime properties with the prefix + * {@code druid.indexer.runner}. + */ public class KubernetesTaskRunnerStaticConfig implements KubernetesTaskRunnerConfig { @JsonProperty diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index fe56e1946df0..51b35fb167a3 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -52,7 +52,7 @@ public class KubernetesTaskRunnerFactoryTest public void setup() { objectMapper = new TestUtils().getTestObjectMapper(); - KubernetesTaskRunnerStaticConfig kubernetesTaskRunnerStaticConfig = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig kubernetesTaskRunnerStaticConfig = KubernetesTaskRunnerConfig.builder() .withCapacity(1) .build(); taskLogs = new NoopTaskLogs(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfigTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfigTest.java index 999d29e5387b..91b5148e2688 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfigTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfigTest.java @@ -60,7 +60,7 @@ public void test_deserializable() throws IOException @Test public void test_builder_preservesDefaults() { - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") .withDisableClientProxy(true) .build(); @@ -85,7 +85,7 @@ public void test_builder_preservesDefaults() @Test public void test_builder() { - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") .withDebugJob(true) .withSidecarSupport(true) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index d942b49d6c1c..ea13f7f41fd5 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -90,7 +90,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport @Before public void setup() { - staticConfig = KubernetesTaskRunnerStaticConfig.builder() + staticConfig = KubernetesTaskRunnerConfig.builder() .withCapacity(1) .build(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index ead3c84fe6b7..c44c0f5460bd 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesHttpClientConfig; @@ -109,7 +110,7 @@ public void testDeployingSomethingToKind(@TempDir Path tempDir) throws Exception PodSpec podSpec = K8sTestUtils.getDummyPodSpec(); Task task = K8sTestUtils.getTask(); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("default") .build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelectorTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelectorTest.java index 762c93c6607d..1052c97c5377 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelectorTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DynamicConfigPodTemplateSelectorTest.java @@ -29,6 +29,7 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerEffectiveConfig; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.K8sTestUtils; @@ -63,7 +64,7 @@ public void setup() mapper = new TestUtils().getTestObjectMapper(); podTemplateSpec = K8sTestUtils.fileToResource("basePodTemplate.yaml", PodTemplate.class); Supplier dynamicConfigRef = () -> new DefaultKubernetesTaskRunnerDynamicConfig(KubernetesTaskRunnerDynamicConfig.DEFAULT_STRATEGY, 1); - KubernetesTaskRunnerStaticConfig staticConfig = KubernetesTaskRunnerStaticConfig.builder().build(); + KubernetesTaskRunnerStaticConfig staticConfig = KubernetesTaskRunnerConfig.builder().build(); effectiveConfig = new KubernetesTaskRunnerEffectiveConfig(staticConfig, dynamicConfigRef); } @@ -256,7 +257,7 @@ public void test_fromTask_matchPodTemplateBasedOnStrategy() throws IOException ), 1 ); - KubernetesTaskRunnerStaticConfig staticConfig = KubernetesTaskRunnerStaticConfig.builder().build(); + KubernetesTaskRunnerStaticConfig staticConfig = KubernetesTaskRunnerConfig.builder().build(); effectiveConfig = new KubernetesTaskRunnerEffectiveConfig(staticConfig, dynamicConfigRef); DynamicConfigPodTemplateSelector podTemplateSelector = new DynamicConfigPodTemplateSelector( diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java index 547c2e3c63e8..552d7201fd05 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/K8sTaskAdapterTest.java @@ -51,6 +51,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTaskId; @@ -138,7 +139,7 @@ public PodSpec getSpec() } }; - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .withOverlordNamespace("test_different") .withAnnotations(ImmutableMap.of("annotation_key", "annotation_value")) @@ -175,7 +176,7 @@ public void serializingAndDeserializingATask() throws IOException { // given a task create a k8s job TestKubernetesClient testClient = new TestKubernetesClient(client); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( @@ -229,7 +230,7 @@ public PodSpec getSpec() } }; - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( @@ -277,7 +278,7 @@ public PodSpec getSpec() public void toTask_useTaskPayloadManager() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .build(); Task taskInTaskPayloadManager = K8sTestUtils.getTask(); @@ -309,7 +310,7 @@ public void toTask_useTaskPayloadManager() throws IOException public void getTaskId() { TestKubernetesClient testClient = new TestKubernetesClient(client); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder().build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, @@ -331,7 +332,7 @@ public void getTaskId() public void getTaskId_noAnnotations() { TestKubernetesClient testClient = new TestKubernetesClient(client); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder().build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, @@ -353,7 +354,7 @@ public void getTaskId_noAnnotations() public void getTaskId_missingTaskIdAnnotation() { TestKubernetesClient testClient = new TestKubernetesClient(client); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder().build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder().build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, @@ -458,7 +459,7 @@ void testAddingMonitors() throws IOException new File("/tmp/"), 0 ); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .build(); K8sTaskAdapter adapter = new SingleContainerTaskAdapter( @@ -481,7 +482,7 @@ void testAddingMonitors() throws IOException ); // we have an override, but nothing in the overlord - config = KubernetesTaskRunnerStaticConfig.builder() + config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .withPeonMonitors(ImmutableList.of("org.apache.druid.java.util.metrics.JvmMonitor")) .build(); @@ -532,7 +533,7 @@ void testEphemeralStorageIsRespected() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("ephemeralPodSpec.yaml", Pod.class); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") .build(); @@ -582,7 +583,7 @@ void testProbesRemoved() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("probesPodSpec.yaml", Pod.class); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("test") .build(); @@ -635,7 +636,7 @@ void testCPUResourceIsRespected() throws IOException List javaOpts = new ArrayList<>(); javaOpts.add("-Xms1G -Xmx2G -XX:MaxDirectMemorySize=3G"); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") .withJavaOptsArray(javaOpts) .withCpuCore(2000) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java index 9a5b91b4b7d9..477758d9ac4e 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/MultiContainerTaskAdapterTest.java @@ -33,6 +33,7 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.PeonCommandContext; @@ -87,9 +88,9 @@ public void testMultiContainerSupport() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpec.yaml", Pod.class); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() - .withNamespace("namespace") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() + .withNamespace("namespace") + .build(); MultiContainerTaskAdapter adapter = new MultiContainerTaskAdapter( testClient, config, @@ -138,7 +139,7 @@ public void testMultiContainerSupportWithNamedContainer() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpecOrder.yaml", Pod.class); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") .withPrimaryContainerName("primary") .build(); @@ -192,7 +193,7 @@ public void testOverridingPeonMonitors() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("podSpec.yaml", Pod.class); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() .withNamespace("namespace") .withPrimaryContainerName("primary") .withPeonMonitors(ImmutableList.of("org.apache.druid.java.util.metrics.JvmMonitor")) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 649986d238cc..01d99a896f62 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -32,6 +32,7 @@ import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.Base64Compression; import org.apache.druid.k8s.overlord.common.DruidK8sConstants; @@ -68,7 +69,7 @@ public class PodTemplateTaskAdapterTest @BeforeEach public void setup() { - taskRunnerConfig = KubernetesTaskRunnerStaticConfig.builder().withNamespace("namespace").build(); + taskRunnerConfig = KubernetesTaskRunnerConfig.builder().withNamespace("namespace").build(); taskConfig = new TaskConfigBuilder().setBaseDir("/tmp").build(); node = new DruidNode( "test", @@ -189,7 +190,7 @@ public void test_getTaskId() public void test_getTaskIdWithK8sTaskPodNamePrefix() { TestPodTemplateSelector podTemplateSelector = new TestPodTemplateSelector(podTemplateSpec); - taskRunnerConfig = KubernetesTaskRunnerStaticConfig.builder().withK8sTaskPodNamePrefix("k8sTaskPodNamePrefix").build(); + taskRunnerConfig = KubernetesTaskRunnerConfig.builder().withK8sTaskPodNamePrefix("k8sTaskPodNamePrefix").build(); PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( taskRunnerConfig, diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java index f5a486b43d8b..832a7292304f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/SingleContainerTaskAdapterTest.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; +import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerStaticConfig; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.common.PeonCommandContext; @@ -86,9 +87,9 @@ public void testSingleContainerSupport() throws IOException { TestKubernetesClient testClient = new TestKubernetesClient(client); Pod pod = K8sTestUtils.fileToResource("multiContainerPodSpec.yaml", Pod.class); - KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerStaticConfig.builder() - .withNamespace("namespace") - .build(); + KubernetesTaskRunnerStaticConfig config = KubernetesTaskRunnerConfig.builder() + .withNamespace("namespace") + .build(); SingleContainerTaskAdapter adapter = new SingleContainerTaskAdapter( testClient, config, From d0e0081b47373638ceb91fd2eeb76c83713b1495 Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Fri, 3 Oct 2025 17:26:22 +0800 Subject: [PATCH 06/19] Update wording for docs --- docs/development/extensions-core/k8s-jobs.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/development/extensions-core/k8s-jobs.md b/docs/development/extensions-core/k8s-jobs.md index 375fc95dd296..c04db2bd1cb1 100644 --- a/docs/development/extensions-core/k8s-jobs.md +++ b/docs/development/extensions-core/k8s-jobs.md @@ -136,6 +136,8 @@ Host: http://ROUTER_IP:ROUTER_PORT Updates the dynamic configuration for the Kubernetes Task Runner +Note: Both `podTemplateSelectStrategy` and `capacity` are optional fields. A POST request may include either, both, or neither. + ##### URL `POST` `/druid/indexer/v1/k8s/taskrunner/executionconfig` From 40530833668f37babf2cff151d7ece273e46f9ec Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Fri, 3 Oct 2025 17:57:50 +0800 Subject: [PATCH 07/19] Upate static config java doc --- .../druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java index 959361605ab7..2d7f820f167f 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java @@ -33,10 +33,9 @@ import java.util.Map; /** - * Static configuration for the Kubernetes-based task runner used by the Overlord. - *

* This configuration is populated from runtime properties with the prefix - * {@code druid.indexer.runner}. + * {@code druid.indexer.runner}. It is the base configuration that + * {@link KubernetesTaskRunnerEffectiveConfig} will use if no dynamic config is provided. */ public class KubernetesTaskRunnerStaticConfig implements KubernetesTaskRunnerConfig { From 1da285cd6145be7f89e4872ffb6522f8315ffbbb Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Fri, 3 Oct 2025 18:01:15 +0800 Subject: [PATCH 08/19] Undo removal of constructor --- .../druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java index 2d7f820f167f..b68e70075dbf 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerStaticConfig.java @@ -144,6 +144,10 @@ public class KubernetesTaskRunnerStaticConfig implements KubernetesTaskRunnerCon @NotNull private Integer capacity = Integer.MAX_VALUE; + public KubernetesTaskRunnerStaticConfig() + { + } + public KubernetesTaskRunnerStaticConfig( @Nonnull String namespace, String overlordNamespace, From 128eb6ee8a8ef3a3c784af3754acf03c01b708e3 Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Tue, 21 Oct 2025 10:57:21 +0800 Subject: [PATCH 09/19] Initial config observer implementation --- .../k8s/overlord/KubernetesTaskRunner.java | 13 +++--- .../overlord/KubernetesTaskRunnerFactory.java | 9 +++- .../KubernetesTaskRunnerFactoryTest.java | 5 ++- .../overlord/KubernetesTaskRunnerTest.java | 17 ++++--- .../druid/common/config/ConfigManager.java | 44 +++++++++++++++++++ 5 files changed, 74 insertions(+), 14 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 824100b51d31..3368ac28e441 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -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; @@ -56,6 +57,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; @@ -102,6 +104,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> listeners = new CopyOnWriteArrayList<>(); // to cleanup old jobs that might not have been deleted. @@ -128,7 +131,8 @@ public KubernetesTaskRunner( KubernetesPeonClient client, HttpClient httpClient, PeonLifecycleFactory peonLifecycleFactory, - ServiceEmitter emitter + ServiceEmitter emitter, + ConfigManager configManager ) { this.adapter = adapter; @@ -142,6 +146,7 @@ public KubernetesTaskRunner( this.currentCapacity = config.getCapacity(); this.tpe = new ThreadPoolExecutor(currentCapacity, currentCapacity, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue(), Execs.makeThreadFactory("k8s-task-runner-%d", null)); this.exec = MoreExecutors.listeningDecorator(this.tpe); + configManager.addListener(KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, String.format(OBSERVER_KEY, Thread.currentThread().getId()), this::syncCapacityWithDynamicConfig); } @Override @@ -157,8 +162,6 @@ public Optional streamTaskLog(String taskid, long offset) @Override public ListenableFuture run(Task task) { - syncCapacityWithDynamicConfig(); - synchronized (tasks) { return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem( task, @@ -174,8 +177,6 @@ public ListenableFuture run(Task task) protected KubernetesWorkItem joinAsync(Task task) { - syncCapacityWithDynamicConfig(); - synchronized (tasks) { return tasks.computeIfAbsent(task.getId(), k -> new KubernetesWorkItem( task, @@ -189,7 +190,7 @@ protected KubernetesWorkItem joinAsync(Task task) } } - private void syncCapacityWithDynamicConfig() + private void syncCapacityWithDynamicConfig(KubernetesTaskRunnerDynamicConfig config) { int newCapacity = config.getCapacity(); if (newCapacity == currentCapacity) { diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index 41dab987d1d3..516d229c8917 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; +import org.apache.druid.common.config.ConfigManager; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.indexing.overlord.TaskRunnerFactory; @@ -45,6 +46,7 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory adapterTypeAllowingTasksInDifferentNamespaces = Set.of(PodTemplateTaskAdapter.TYPE); @Inject @@ -55,7 +57,8 @@ public KubernetesTaskRunnerFactory( TaskLogs taskLogs, DruidKubernetesClient druidKubernetesClient, ServiceEmitter emitter, - TaskAdapter taskAdapter + TaskAdapter taskAdapter, + ConfigManager configManager ) { this.smileMapper = smileMapper; @@ -65,6 +68,7 @@ public KubernetesTaskRunnerFactory( this.druidKubernetesClient = druidKubernetesClient; this.emitter = emitter; this.taskAdapter = taskAdapter; + this.configManager = configManager; } @Override @@ -99,7 +103,8 @@ public KubernetesTaskRunner build() smileMapper, kubernetesTaskRunnerConfig.getLogSaveTimeout().toStandardDuration().getMillis() ), - emitter + emitter, + configManager ); return runner; } diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index 51b35fb167a3..382e75418ac7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.ConfigBuilder; +import org.apache.druid.common.config.ConfigManager; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -47,6 +48,7 @@ public class KubernetesTaskRunnerFactoryTest private DruidKubernetesClient druidKubernetesClient; @Mock private ServiceEmitter emitter; private TaskAdapter taskAdapter; + @Mock private ConfigManager configManager; @Before public void setup() @@ -72,7 +74,8 @@ public void test_get_returnsSameKuberentesTaskRunner_asBuild() taskLogs, druidKubernetesClient, emitter, - taskAdapter + taskAdapter, + configManager ); KubernetesTaskRunner expectedRunner = factory.build(); diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index ea13f7f41fd5..7636b0c841bd 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -28,6 +28,7 @@ import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.api.model.batch.v1.JobBuilder; import org.apache.commons.io.IOUtils; +import org.apache.druid.common.config.ConfigManager; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; @@ -81,6 +82,7 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport @Mock private KubernetesPeonLifecycle kubernetesPeonLifecycle; @Mock private ServiceEmitter emitter; @Mock private ListenableFuture statusFuture; + @Mock private ConfigManager configManager; private KubernetesTaskRunnerStaticConfig staticConfig; private KubernetesTaskRunnerEffectiveConfig config; @@ -106,7 +108,8 @@ public void setup() peonClient, httpClient, new TestPeonLifecycleFactory(kubernetesPeonLifecycle), - emitter + emitter, + configManager ); } @@ -121,7 +124,8 @@ public void test_start_withExistingJobs() throws IOException peonClient, httpClient, new TestPeonLifecycleFactory(kubernetesPeonLifecycle), - emitter + emitter, + configManager ) { @Override @@ -171,7 +175,8 @@ public void test_start_withExistingJobs_oneJobFails() throws IOException peonClient, httpClient, new TestPeonLifecycleFactory(kubernetesPeonLifecycle), - emitter + emitter, + configManager ) { @Override @@ -227,7 +232,8 @@ public void test_start_whenDeserializationExceptionThrown_isIgnored() throws IOE peonClient, httpClient, new TestPeonLifecycleFactory(kubernetesPeonLifecycle), - emitter + emitter, + configManager ) { @Override @@ -779,7 +785,8 @@ public void test_stop() peonClient, httpClient, new TestPeonLifecycleFactory(kubernetesPeonLifecycle), - emitter + emitter, + configManager ); kubernetesTaskRunner.stop(); Assert.assertThrows(RejectedExecutionException.class, () -> kubernetesTaskRunner.run(task)); diff --git a/processing/src/main/java/org/apache/druid/common/config/ConfigManager.java b/processing/src/main/java/org/apache/druid/common/config/ConfigManager.java index 87cbad379410..63d21e323142 100644 --- a/processing/src/main/java/org/apache/druid/common/config/ConfigManager.java +++ b/processing/src/main/java/org/apache/druid/common/config/ConfigManager.java @@ -41,6 +41,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; /** */ @@ -219,6 +220,26 @@ public SetResult set(final String key, final ConfigSerde serde, @Nullable } } + public boolean addListener(String configKey, String listenerKey, Consumer listener) + { + ConfigHolder holder = watchedConfigs.get(configKey); + if (holder == null) { + log.warn("ConfigHolder not found for configKey[%s]", configKey); + return false; + } + return holder.addListener(listenerKey, listener); + } + + public boolean removeListener(String configKey, String listenerKey, Consumer listener) + { + ConfigHolder holder = watchedConfigs.get(configKey); + if (holder == null) { + log.warn("ConfigHolder not found for configKey[%s]", configKey); + return false; + } + return holder.removeListener(listenerKey, listener); + } + @Nonnull private MetadataCASUpdate createMetadataCASUpdate( String keyValue, @@ -285,6 +306,7 @@ private static class ConfigHolder private final AtomicReference rawBytes; private final ConfigSerde serde; private final AtomicReference reference; + private final ConcurrentMap> listeners; ConfigHolder( byte[] rawBytes, @@ -294,6 +316,7 @@ private static class ConfigHolder this.rawBytes = new AtomicReference<>(rawBytes); this.serde = serde; this.reference = new AtomicReference<>(serde.deserialize(rawBytes)); + this.listeners = new ConcurrentHashMap<>(); } public AtomicReference getReference() @@ -306,10 +329,31 @@ public boolean swapIfNew(byte[] newBytes) if (!Arrays.equals(newBytes, rawBytes.get())) { reference.set(serde.deserialize(newBytes)); rawBytes.set(newBytes); + listeners.forEach((_key, listener) -> listener.accept(reference.get())); return true; } return false; } + + public boolean addListener(String key, Consumer listener) + { + if (listeners.containsKey(key)) { + log.warn("listener key[%s] already exists", key); + return false; + } + listeners.put(key, listener); + return true; + } + + public boolean removeListener(String key, Consumer listener) + { + if (!listeners.containsKey(key)) { + log.warn("listener key[%s] not found", key); + return false; + } + listeners.remove(key, listener); + return true; + } } private class PollingCallable implements Callable From afec8ecca5c44b33d6185f08f7a26fe1a10ab070 Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Tue, 21 Oct 2025 14:40:19 +0800 Subject: [PATCH 10/19] Use StringUtils.format --- .../org/apache/druid/k8s/overlord/KubernetesTaskRunner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 3368ac28e441..c7ecfc32ebe3 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -146,7 +146,7 @@ public KubernetesTaskRunner( this.currentCapacity = config.getCapacity(); this.tpe = new ThreadPoolExecutor(currentCapacity, currentCapacity, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue(), Execs.makeThreadFactory("k8s-task-runner-%d", null)); this.exec = MoreExecutors.listeningDecorator(this.tpe); - configManager.addListener(KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, String.format(OBSERVER_KEY, Thread.currentThread().getId()), this::syncCapacityWithDynamicConfig); + configManager.addListener(KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, StringUtils.format(OBSERVER_KEY, Thread.currentThread().getId()), this::syncCapacityWithDynamicConfig); } @Override From c02abc3f4ea47f5f1b117777be688645b70f524b Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Tue, 21 Oct 2025 14:58:28 +0800 Subject: [PATCH 11/19] Add missing import --- .../java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java | 1 + 1 file changed, 1 insertion(+) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index c7ecfc32ebe3..805013f1a139 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -45,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; From 2a3193882bdad99f670316e9f7441ea379c9e48a Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Wed, 22 Oct 2025 11:17:09 +0800 Subject: [PATCH 12/19] Update listener operations to be atomic --- .../druid/common/config/ConfigManager.java | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/common/config/ConfigManager.java b/processing/src/main/java/org/apache/druid/common/config/ConfigManager.java index 63d21e323142..12265df31e27 100644 --- a/processing/src/main/java/org/apache/druid/common/config/ConfigManager.java +++ b/processing/src/main/java/org/apache/druid/common/config/ConfigManager.java @@ -329,7 +329,14 @@ public boolean swapIfNew(byte[] newBytes) if (!Arrays.equals(newBytes, rawBytes.get())) { reference.set(serde.deserialize(newBytes)); rawBytes.set(newBytes); - listeners.forEach((_key, listener) -> listener.accept(reference.get())); + listeners.forEach((key, listener) -> { + try { + listener.accept(reference.get()); + } + catch (Exception e) { + log.warn(e, "Exception when calling listener for key[%s]", key); + } + }); return true; } return false; @@ -337,21 +344,21 @@ public boolean swapIfNew(byte[] newBytes) public boolean addListener(String key, Consumer listener) { - if (listeners.containsKey(key)) { - log.warn("listener key[%s] already exists", key); + Consumer val = listeners.putIfAbsent(key, listener); + if (val != null) { + log.warn("Listener key[%s] already exists", key); return false; } - listeners.put(key, listener); return true; } public boolean removeListener(String key, Consumer listener) { - if (!listeners.containsKey(key)) { - log.warn("listener key[%s] not found", key); + boolean isRemoved = listeners.remove(key, listener); + if (!isRemoved) { + log.warn("Listener key[%s] not found", key); return false; } - listeners.remove(key, listener); return true; } } From 0ac26f8a314339c1e13facae6d4b190fad4430e2 Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Wed, 22 Oct 2025 15:09:52 +0800 Subject: [PATCH 13/19] Fix config manager intialisation in tests --- .../druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java | 3 +++ .../apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java | 3 +++ 2 files changed, 6 insertions(+) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index 799dcd61749d..a67ab70a0a8a 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -33,6 +33,7 @@ import org.apache.druid.k8s.overlord.taskadapter.TaskAdapter; import org.apache.druid.tasklogs.NoopTaskLogs; import org.apache.druid.tasklogs.TaskLogs; +import org.easymock.EasyMock; import org.easymock.Mock; import org.junit.Assert; import org.junit.Before; @@ -63,6 +64,8 @@ public void setup() new DruidKubernetesClient(new DruidKubernetesVertxHttpClientFactory(new DruidKubernetesVertxHttpClientConfig()), new ConfigBuilder().build()); taskAdapter = new TestTaskAdapter(); kubernetesTaskRunnerConfig = new KubernetesTaskRunnerEffectiveConfig(kubernetesTaskRunnerStaticConfig, () -> null); + configManager = EasyMock.createNiceMock(ConfigManager.class); + EasyMock.replay(configManager); } @Test diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index 7636b0c841bd..e94478784850 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -102,6 +102,9 @@ public void setup() task = K8sTestUtils.createTask(ID, 0); + configManager = EasyMock.createNiceMock(ConfigManager.class); + EasyMock.replay(configManager); + runner = new KubernetesTaskRunner( taskAdapter, config, From 6050eea124eedc1cef4a7c06cd6e8b96e9b387ea Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Wed, 22 Oct 2025 15:10:11 +0800 Subject: [PATCH 14/19] Add tests for effective config --- ...bernetesTaskRunnerEffectiveConfigTest.java | 90 +++++++++++++++++++ 1 file changed, 90 insertions(+) create mode 100644 extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfigTest.java diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfigTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfigTest.java new file mode 100644 index 000000000000..b54c7f4d6150 --- /dev/null +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfigTest.java @@ -0,0 +1,90 @@ +/* + * 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.k8s.overlord; + +import com.google.common.base.Supplier; +import org.apache.druid.k8s.overlord.execution.DefaultKubernetesTaskRunnerDynamicConfig; +import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; +import org.apache.druid.k8s.overlord.execution.PodTemplateSelectStrategy; +import org.apache.druid.k8s.overlord.execution.TaskTypePodTemplateSelectStrategy; +import org.junit.Assert; +import org.junit.Test; + +public class KubernetesTaskRunnerEffectiveConfigTest +{ + @Test + public void test_getCapacity_usesStaticWhenDynamicNull() + { + KubernetesTaskRunnerStaticConfig staticConfig = KubernetesTaskRunnerConfig.builder() + .withCapacity(7) + .build(); + Supplier dynamicSupplier = () -> null; + KubernetesTaskRunnerEffectiveConfig effective = new KubernetesTaskRunnerEffectiveConfig(staticConfig, dynamicSupplier); + + Assert.assertEquals(7, effective.getCapacity().intValue()); + } + + @Test + public void test_getCapacity_usesDynamicWhenProvided() + { + KubernetesTaskRunnerStaticConfig staticConfig = KubernetesTaskRunnerConfig.builder() + .withCapacity(2) + .build(); + Supplier dynamicSupplier = () -> new DefaultKubernetesTaskRunnerDynamicConfig(null, 9); + KubernetesTaskRunnerEffectiveConfig effective = new KubernetesTaskRunnerEffectiveConfig(staticConfig, dynamicSupplier); + + Assert.assertEquals(9, effective.getCapacity().intValue()); + } + + @Test + public void test_getCapacity_usesStaticWhenDynamicNullCapacity() + { + KubernetesTaskRunnerStaticConfig staticConfig = KubernetesTaskRunnerConfig.builder() + .withCapacity(7) + .build(); + Supplier dynamicSupplier = () -> new DefaultKubernetesTaskRunnerDynamicConfig(null, null); + KubernetesTaskRunnerEffectiveConfig effective = new KubernetesTaskRunnerEffectiveConfig(staticConfig, dynamicSupplier); + + Assert.assertEquals(7, effective.getCapacity().intValue()); + } + + @Test + public void test_getPodTemplateSelectStrategy_usesDefaultWhenDynamicNull() + { + KubernetesTaskRunnerStaticConfig staticConfig = KubernetesTaskRunnerConfig.builder().build(); + Supplier dynamicSupplier = () -> null; + KubernetesTaskRunnerEffectiveConfig effective = new KubernetesTaskRunnerEffectiveConfig(staticConfig, dynamicSupplier); + + PodTemplateSelectStrategy strategy = effective.getPodTemplateSelectStrategy(); + Assert.assertTrue(strategy instanceof TaskTypePodTemplateSelectStrategy); + Assert.assertEquals(KubernetesTaskRunnerDynamicConfig.DEFAULT_STRATEGY, strategy); + } + + @Test + public void test_getPodTemplateSelectStrategy_usesDynamicWhenProvided() + { + KubernetesTaskRunnerStaticConfig staticConfig = KubernetesTaskRunnerConfig.builder().build(); + PodTemplateSelectStrategy custom = new TaskTypePodTemplateSelectStrategy(); + Supplier dynamicSupplier = () -> new DefaultKubernetesTaskRunnerDynamicConfig(custom, null); + KubernetesTaskRunnerEffectiveConfig effective = new KubernetesTaskRunnerEffectiveConfig(staticConfig, dynamicSupplier); + + Assert.assertEquals(custom, effective.getPodTemplateSelectStrategy()); + } +} \ No newline at end of file From fcc3d3e2ca5b169da2b6ca29301922064b05a136 Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Wed, 22 Oct 2025 15:10:43 +0800 Subject: [PATCH 15/19] Test syncCapacityWithDynamicConfig --- .../overlord/KubernetesTaskRunnerTest.java | 68 +++++++++++++++++++ 1 file changed, 68 insertions(+) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index e94478784850..730fa466351a 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -55,6 +55,8 @@ import org.junit.Test; import org.junit.runner.RunWith; +import java.lang.reflect.Field; +import java.lang.reflect.Method; import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; @@ -65,6 +67,7 @@ import java.util.concurrent.Executor; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.ThreadPoolExecutor; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -794,4 +797,69 @@ public void test_stop() kubernetesTaskRunner.stop(); Assert.assertThrows(RejectedExecutionException.class, () -> kubernetesTaskRunner.run(task)); } + + @Test + public void test_syncCapacityWithDynamicConfig_increase_updatesExecutorAndCapacity() throws Exception + { + Method method = KubernetesTaskRunner.class.getDeclaredMethod( + "syncCapacityWithDynamicConfig", + KubernetesTaskRunnerDynamicConfig.class + ); + method.setAccessible(true); + + // increase from 1 -> 3 + method.invoke(runner, new DefaultKubernetesTaskRunnerDynamicConfig(null, 3)); + + Field tpeField = KubernetesTaskRunner.class.getDeclaredField("tpe"); + tpeField.setAccessible(true); + ThreadPoolExecutor executor = (ThreadPoolExecutor) tpeField.get(runner); + + Assert.assertEquals(3, executor.getCorePoolSize()); + Assert.assertEquals(3, executor.getMaximumPoolSize()); + Assert.assertEquals(3, runner.getTotalCapacity()); + } + + @Test + public void test_syncCapacityWithDynamicConfig_decrease_updatesExecutorAndCapacity() throws Exception + { + Method method = KubernetesTaskRunner.class.getDeclaredMethod( + "syncCapacityWithDynamicConfig", + KubernetesTaskRunnerDynamicConfig.class + ); + method.setAccessible(true); + + // first increase to 4 to ensure we can decrease after + method.invoke(runner, new DefaultKubernetesTaskRunnerDynamicConfig(null, 4)); + // then decrease 4 -> 2 + method.invoke(runner, new DefaultKubernetesTaskRunnerDynamicConfig(null, 2)); + + Field tpeField = KubernetesTaskRunner.class.getDeclaredField("tpe"); + tpeField.setAccessible(true); + ThreadPoolExecutor executor = (ThreadPoolExecutor) tpeField.get(runner); + + Assert.assertEquals(2, executor.getCorePoolSize()); + Assert.assertEquals(2, executor.getMaximumPoolSize()); + Assert.assertEquals(2, runner.getTotalCapacity()); + } + + @Test + public void test_syncCapacityWithDynamicConfig_sameCapacity_noChangeAndNoError() throws Exception + { + Method method = KubernetesTaskRunner.class.getDeclaredMethod( + "syncCapacityWithDynamicConfig", + KubernetesTaskRunnerDynamicConfig.class + ); + method.setAccessible(true); + + // initial capacity is 1 in setup; calling with 1 should be a no-op + method.invoke(runner, new DefaultKubernetesTaskRunnerDynamicConfig(null, 1)); + + Field tpeField = KubernetesTaskRunner.class.getDeclaredField("tpe"); + tpeField.setAccessible(true); + ThreadPoolExecutor executor = (ThreadPoolExecutor) tpeField.get(runner); + + Assert.assertEquals(1, executor.getCorePoolSize()); + Assert.assertEquals(1, executor.getMaximumPoolSize()); + Assert.assertEquals(1, runner.getTotalCapacity()); + } } From 2c4f068867571c97d9ac39666aa271091a7506df Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Wed, 22 Oct 2025 15:16:57 +0800 Subject: [PATCH 16/19] Fix checkstyle --- .../org/apache/druid/k8s/overlord/KubernetesTaskRunner.java | 6 +++--- .../overlord/KubernetesTaskRunnerEffectiveConfigTest.java | 2 +- .../apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index 805013f1a139..c62fde1a7731 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -90,13 +90,13 @@ * The KubernetesTaskRunner runs tasks by transforming the task spec into a K8s Job spec based * on the TaskAdapter it is configured with. The KubernetesTaskRunner has a pool of threads * (configurable with the capacity configuration) to track the jobs (1 thread tracks 1 job). - *

+ * * Each thread calls down to the KubernetesPeonLifecycle class to submit the Job to K8s and then * waits for the lifecycle class to report back with the Job's status (success/failure). - *

+ * * If there are not enough threads in the thread pool to execute and wait for a job, then the * task is put in a queue and left in WAITING state until another task completes. - *

+ * * When the KubernetesTaskRunner comes up it attempts to restore its internal mapping of tasks * from Kubernetes by listing running jobs and calling join on each job, which spawns a thread to * wait for the fabric8 client library to report back, similar to what happens when a new diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfigTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfigTest.java index b54c7f4d6150..c44f8a34b4e6 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfigTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerEffectiveConfigTest.java @@ -87,4 +87,4 @@ public void test_getPodTemplateSelectStrategy_usesDynamicWhenProvided() Assert.assertEquals(custom, effective.getPodTemplateSelectStrategy()); } -} \ No newline at end of file +} diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java index 730fa466351a..1c72078a52e7 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerTest.java @@ -55,10 +55,10 @@ import org.junit.Test; import org.junit.runner.RunWith; -import java.lang.reflect.Field; -import java.lang.reflect.Method; import java.io.IOException; import java.io.InputStream; +import java.lang.reflect.Field; +import java.lang.reflect.Method; import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.Collections; @@ -66,8 +66,8 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.TimeUnit; import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; From 55de05d1f0e4e6031bc79919c68b0eb48a0ab43f Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Wed, 22 Oct 2025 18:26:31 +0800 Subject: [PATCH 17/19] Update KubernetesOverlordModuleTest to setup ConfigManager --- .../KubernetesOverlordModuleTest.java | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java index 59c9508005f6..dd8960864e0e 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java +++ b/extensions-core/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java @@ -27,6 +27,7 @@ import com.google.inject.ProvisionException; import com.google.inject.TypeLiteral; import org.apache.druid.audit.AuditManager; +import org.apache.druid.common.config.ConfigManager; import org.apache.druid.common.config.ConfigManagerConfig; import org.apache.druid.guice.ConfigModule; import org.apache.druid.guice.DruidGuiceExtensions; @@ -44,6 +45,7 @@ import org.apache.druid.k8s.overlord.common.httpclient.jdk.DruidKubernetesJdkHttpClientFactory; import org.apache.druid.k8s.overlord.common.httpclient.okhttp.DruidKubernetesOkHttpHttpClientFactory; import org.apache.druid.k8s.overlord.common.httpclient.vertx.DruidKubernetesVertxHttpClientFactory; +import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; import org.apache.druid.k8s.overlord.taskadapter.MultiContainerTaskAdapter; import org.apache.druid.k8s.overlord.taskadapter.PodTemplateTaskAdapter; import org.apache.druid.k8s.overlord.taskadapter.SingleContainerTaskAdapter; @@ -51,14 +53,18 @@ import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.server.DruidNode; +import org.easymock.EasyMock; import org.easymock.EasyMockRunner; import org.easymock.Mock; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import java.net.URL; import java.util.Properties; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; @RunWith(EasyMockRunner.class) public class KubernetesOverlordModuleTest @@ -81,8 +87,26 @@ public class KubernetesOverlordModuleTest private AuditManager auditManager; @Mock private MetadataStorageConnector metadataStorageConnector; + @Mock + private ConfigManager configManager; private Injector injector; + @Before + public void setUpConfigManagerMock() + { + EasyMock.reset(configManager); + EasyMock.expect(configManager.watchConfig( + EasyMock.anyString(), + EasyMock.anyObject() + )).andReturn(new AtomicReference<>(null)).anyTimes(); + EasyMock.expect(configManager.addListener( + EasyMock.eq(KubernetesTaskRunnerDynamicConfig.CONFIG_KEY), + EasyMock.anyString(), + EasyMock.anyObject(Consumer.class) + )).andReturn(true).anyTimes(); + EasyMock.replay(configManager); + } + @Test public void testDefaultHttpRemoteTaskRunnerFactoryBindSuccessfully() { @@ -325,6 +349,7 @@ private Injector makeInjectorWithProperties( }).toInstance(Suppliers.ofInstance(metadataStorageTablesConfig)); binder.bind(AuditManager.class).toInstance(auditManager); binder.bind(MetadataStorageConnector.class).toInstance(metadataStorageConnector); + binder.bind(ConfigManager.class).toInstance(configManager); }, new ConfigModule(), new IndexingServiceTaskLogsModule(props), From a3c2c0a56da55d57829eaf0c4c98abbca96d6567 Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Tue, 28 Oct 2025 10:53:56 +0800 Subject: [PATCH 18/19] Update docs --- docs/development/extensions-core/k8s-jobs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/development/extensions-core/k8s-jobs.md b/docs/development/extensions-core/k8s-jobs.md index a42353a04eca..6dc7bdd70ea9 100644 --- a/docs/development/extensions-core/k8s-jobs.md +++ b/docs/development/extensions-core/k8s-jobs.md @@ -795,7 +795,7 @@ Should you require the needed permissions for interacting across Kubernetes name | `druid.indexer.runner.annotations` | `JsonObject` | Additional annotations you want to add to peon pod. | `{}` | No | | `druid.indexer.runner.peonMonitors` | `JsonArray` | Overrides `druid.monitoring.monitors`. Use this property if you don't want to inherit monitors from the Overlord. | `[]` | No | | `druid.indexer.runner.graceTerminationPeriodSeconds` | `Long` | Number of seconds you want to wait after a sigterm for container lifecycle hooks to complete. Keep at a smaller value if you want tasks to hold locks for shorter periods. | `PT30S` (K8s default) | No | -| `druid.indexer.runner.capacity` | `Integer` | Number of concurrent jobs that can be sent to Kubernetes. | `2147483647` | No | +| `druid.indexer.runner.capacity` | `Integer` | Number of concurrent jobs that can be sent to Kubernetes. Value will be overridden if a dynamic config value has been set. | `2147483647` | No | | `druid.indexer.runner.cpuCoreInMicro` | `Integer` | Number of CPU micro core for the task. | `1000` | No | | `druid.indexer.runner.logSaveTimeout` | `Duration` | The peon executing the ingestion task makes a best effort to persist the pod logs from `k8s` to persistent task log storage. The timeout ensures that `k8s` connection issues do not cause the pod to hang indefinitely thereby blocking Overlord operations. If the timeout occurs before the logs are saved, those logs will not be available in Druid. | `PT300S` | NO | From 1c64afb4ad4efcfa2d9ce33f298eec52703f8a60 Mon Sep 17 00:00:00 2001 From: Gabriel Chang <77312579+GabrielCWT@users.noreply.github.com> Date: Tue, 28 Oct 2025 10:55:08 +0800 Subject: [PATCH 19/19] Use AtomicInteger for currentCapacity --- .../k8s/overlord/KubernetesTaskRunner.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java index c62fde1a7731..48d709d1f841 100644 --- a/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java +++ b/extensions-core/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunner.java @@ -83,6 +83,7 @@ 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; /** @@ -124,7 +125,7 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner // currently worker categories aren't supported, so it's hardcoded. protected static final String WORKER_CATEGORY = "_k8s_worker_category"; - private int currentCapacity; + private final AtomicInteger currentCapacity; public KubernetesTaskRunner( TaskAdapter adapter, @@ -144,8 +145,8 @@ public KubernetesTaskRunner( this.cleanupExecutor = Executors.newScheduledThreadPool(1); this.emitter = emitter; - this.currentCapacity = config.getCapacity(); - this.tpe = new ThreadPoolExecutor(currentCapacity, currentCapacity, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue(), Execs.makeThreadFactory("k8s-task-runner-%d", null)); + this.currentCapacity = new AtomicInteger(config.getCapacity()); + this.tpe = new ThreadPoolExecutor(currentCapacity.get(), currentCapacity.get(), 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue(), 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); } @@ -194,19 +195,19 @@ protected KubernetesWorkItem joinAsync(Task task) private void syncCapacityWithDynamicConfig(KubernetesTaskRunnerDynamicConfig config) { int newCapacity = config.getCapacity(); - if (newCapacity == currentCapacity) { + if (newCapacity == currentCapacity.get()) { return; } - log.info("Adjusting k8s task runner capacity from [%d] to [%d]", currentCapacity, newCapacity); + 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) { + if (newCapacity < currentCapacity.get()) { tpe.setCorePoolSize(newCapacity); tpe.setMaximumPoolSize(newCapacity); } else { tpe.setMaximumPoolSize(newCapacity); tpe.setCorePoolSize(newCapacity); } - currentCapacity = newCapacity; + currentCapacity.set(newCapacity); } private TaskStatus runTask(Task task) @@ -450,7 +451,7 @@ public void stop() @Override public Map getTotalTaskSlotCount() { - return ImmutableMap.of(WORKER_CATEGORY, (long) currentCapacity); + return ImmutableMap.of(WORKER_CATEGORY, (long) currentCapacity.get()); } @Override @@ -468,13 +469,13 @@ public Optional getScalingStats() @Override public Map getIdleTaskSlotCount() { - return ImmutableMap.of(WORKER_CATEGORY, (long) Math.max(0, currentCapacity - tasks.size())); + return ImmutableMap.of(WORKER_CATEGORY, (long) Math.max(0, currentCapacity.get() - tasks.size())); } @Override public Map getUsedTaskSlotCount() { - return ImmutableMap.of(WORKER_CATEGORY, (long) Math.min(currentCapacity, tasks.size())); + return ImmutableMap.of(WORKER_CATEGORY, (long) Math.min(currentCapacity.get(), tasks.size())); } @Override @@ -565,7 +566,7 @@ public RunnerTaskState getRunnerTaskState(String taskId) @Override public int getTotalCapacity() { - return currentCapacity; + return currentCapacity.get(); } @Override