From b74e5b81dfa9fb183918116a2b9f470d5514d7fc Mon Sep 17 00:00:00 2001 From: YongGang Date: Mon, 27 May 2024 22:21:47 -0700 Subject: [PATCH 01/16] initial commit --- .../overlord/KubernetesOverlordModule.java | 7 + .../overlord/KubernetesTaskRunnerFactory.java | 10 +- .../execution/DefaultExecutionConfig.java | 77 ++++++++++ .../DynamicTaskExecutionBehaviorStrategy.java | 86 +++++++++++ .../execution/ExecutionBehaviorStrategy.java | 36 +++++ .../overlord/execution/ExecutionConfig.java | 34 +++++ .../execution/KubernetesResource.java | 134 ++++++++++++++++++ .../k8s/overlord/execution/MappingRule.java | 112 +++++++++++++++ .../taskadapter/PodTemplateTaskAdapter.java | 19 ++- .../KubernetesOverlordModuleTest.java | 8 ++ .../KubernetesTaskRunnerFactoryTest.java | 27 ++-- .../PodTemplateTaskAdapterTest.java | 60 +++++--- 12 files changed, 579 insertions(+), 31 deletions(-) create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java index 1e52a3583b27..de09de268e8a 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java @@ -32,6 +32,8 @@ import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Binders; import org.apache.druid.guice.IndexingServiceModuleHelper; +import org.apache.druid.guice.JacksonConfigProvider; +import org.apache.druid.guice.Jerseys; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.JsonConfigurator; import org.apache.druid.guice.LazySingleton; @@ -49,6 +51,8 @@ import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; +import org.apache.druid.k8s.overlord.execution.ExecutionConfig; +import org.apache.druid.k8s.overlord.execution.KubernetesResource; import org.apache.druid.k8s.overlord.runnerstrategy.RunnerStrategy; import org.apache.druid.tasklogs.NoopTaskLogs; import org.apache.druid.tasklogs.TaskLogKiller; @@ -75,6 +79,7 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX, KubernetesTaskRunnerConfig.class); JsonConfigProvider.bind(binder, K8SANDWORKER_PROPERTIES_PREFIX, KubernetesAndWorkerTaskRunnerConfig.class); JsonConfigProvider.bind(binder, "druid.indexer.queue", TaskQueueConfig.class); + JacksonConfigProvider.bind(binder, ExecutionConfig.CONFIG_KEY, ExecutionConfig.class, null); PolyBind.createChoice( binder, "druid.indexer.runner.type", @@ -98,6 +103,8 @@ public void configure(Binder binder) .toProvider(RunnerStrategyProvider.class) .in(LazySingleton.class); configureTaskLogs(binder); + + Jerseys.addResource(binder, KubernetesResource.class); } @Provides diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index 72d7ef0c00d4..874220403311 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -20,6 +20,7 @@ package org.apache.druid.k8s.overlord; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; import com.google.inject.Inject; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.guice.annotations.EscalatedGlobal; @@ -32,6 +33,7 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; +import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import org.apache.druid.k8s.overlord.taskadapter.MultiContainerTaskAdapter; import org.apache.druid.k8s.overlord.taskadapter.PodTemplateTaskAdapter; import org.apache.druid.k8s.overlord.taskadapter.SingleContainerTaskAdapter; @@ -56,6 +58,7 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory executionConfigRef; private KubernetesTaskRunner runner; @Inject @@ -69,7 +72,8 @@ public KubernetesTaskRunnerFactory( TaskConfig taskConfig, Properties properties, DruidKubernetesClient druidKubernetesClient, - ServiceEmitter emitter + ServiceEmitter emitter, + Supplier executionConfigRef ) { this.smileMapper = smileMapper; @@ -82,6 +86,7 @@ public KubernetesTaskRunnerFactory( this.properties = properties; this.druidKubernetesClient = druidKubernetesClient; this.emitter = emitter; + this.executionConfigRef = executionConfigRef; } @Override @@ -146,7 +151,8 @@ private TaskAdapter buildTaskAdapter(DruidKubernetesClient client) druidNode, smileMapper, properties, - taskLogs + taskLogs, + executionConfigRef ); } else { return new SingleContainerTaskAdapter( diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java new file mode 100644 index 000000000000..054ceb05f20f --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java @@ -0,0 +1,77 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +public class DefaultExecutionConfig implements ExecutionConfig +{ + private final ExecutionBehaviorStrategy behaviorStrategy; + + public static DefaultExecutionConfig defaultConfig() + { + return new DefaultExecutionConfig(new DynamicTaskExecutionBehaviorStrategy(null)); + } + + @JsonCreator + public DefaultExecutionConfig( + @JsonProperty("behaviorStrategy") ExecutionBehaviorStrategy behaviorStrategy + ) + { + this.behaviorStrategy = behaviorStrategy; + } + + @Override + @JsonProperty + public ExecutionBehaviorStrategy getBehaviorStrategy() + { + return behaviorStrategy; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultExecutionConfig that = (DefaultExecutionConfig) o; + return Objects.equals(behaviorStrategy, that.behaviorStrategy); + } + + @Override + public int hashCode() + { + return Objects.hashCode(behaviorStrategy); + } + + @Override + public String toString() + { + return "DefaultExecutionConfig{" + + "behaviorStrategy=" + behaviorStrategy + + '}'; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java new file mode 100644 index 000000000000..eea1b527a26f --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java @@ -0,0 +1,86 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.common.task.Task; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Objects; + +public class DynamicTaskExecutionBehaviorStrategy implements ExecutionBehaviorStrategy +{ + private LinkedHashMap categoryRuleMap; + + @JsonCreator + public DynamicTaskExecutionBehaviorStrategy( + @JsonProperty("categoryMap") LinkedHashMap categoryRuleMap + ) + { + this.categoryRuleMap = categoryRuleMap; + } + + @Override + public String getTaskCategory(Task task) + { + return categoryRuleMap.entrySet() + .stream() + .filter(categoryEntry -> + categoryEntry.getValue().evaluate(task)) + .findFirst() + .map(Map.Entry::getKey) + .orElse(null); + } + + @JsonProperty("categoryMap") + public LinkedHashMap getCategoryRuleMap() + { + return categoryRuleMap; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DynamicTaskExecutionBehaviorStrategy that = (DynamicTaskExecutionBehaviorStrategy) o; + return Objects.equals(categoryRuleMap, that.categoryRuleMap); + } + + @Override + public int hashCode() + { + return Objects.hashCode(categoryRuleMap); + } + + @Override + public String toString() + { + return "DynamicTaskExecutionBehaviorStrategy{" + + "categoryMap=" + categoryRuleMap + + '}'; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java new file mode 100644 index 000000000000..54726f325f41 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java @@ -0,0 +1,36 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.indexing.common.task.Task; + +import javax.annotation.Nullable; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DynamicTaskExecutionBehaviorStrategy.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "default", value = DynamicTaskExecutionBehaviorStrategy.class) +}) +public interface ExecutionBehaviorStrategy +{ + @Nullable + String getTaskCategory(Task task); +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java new file mode 100644 index 000000000000..a154440a603b --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java @@ -0,0 +1,34 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultExecutionConfig.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "default", value = DefaultExecutionConfig.class) +}) +public interface ExecutionConfig +{ + String CONFIG_KEY = "k8s.taskrunner.config"; + + ExecutionBehaviorStrategy getBehaviorStrategy(); +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java new file mode 100644 index 000000000000..5faaccfec81e --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java @@ -0,0 +1,134 @@ +/* + * 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.execution; + +import com.google.common.collect.ImmutableMap; +import com.sun.jersey.spi.container.ResourceFilters; +import org.apache.druid.audit.AuditEntry; +import org.apache.druid.audit.AuditManager; +import org.apache.druid.common.config.ConfigManager; +import org.apache.druid.common.config.JacksonConfigManager; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.http.security.ConfigResourceFilter; +import org.apache.druid.server.security.AuthorizationUtils; +import org.joda.time.Interval; + +import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +@Path("/druid/indexer/v1/k8s/runner") +public class KubernetesResource +{ + private static final Logger log = new Logger(KubernetesResource.class); + private final JacksonConfigManager configManager; + private final AuditManager auditManager; + private AtomicReference executionConfigRef = null; + + @Inject + public KubernetesResource( + final JacksonConfigManager configManager, + final AuditManager auditManager + ) + { + this.configManager = configManager; + this.auditManager = auditManager; + } + + @POST + @Path("/execution") + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(ConfigResourceFilter.class) + public Response setExecutionConfig( + final ExecutionConfig executionConfig, + @Context final HttpServletRequest req + ) + { + final ConfigManager.SetResult setResult = configManager.set( + ExecutionConfig.CONFIG_KEY, + executionConfig, + AuthorizationUtils.buildAuditInfo(req) + ); + if (setResult.isOk()) { + log.info("Updating K8s execution configs: %s", executionConfig); + + return Response.ok().build(); + } else { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + } + + @GET + @Path("/execution/history") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(ConfigResourceFilter.class) + public Response getExecutionConfigHistory( + @QueryParam("interval") final String interval, + @QueryParam("count") final Integer count + ) + { + Interval theInterval = interval == null ? null : Intervals.of(interval); + if (theInterval == null && count != null) { + try { + List executionEntryList = auditManager.fetchAuditHistory( + ExecutionConfig.CONFIG_KEY, + ExecutionConfig.CONFIG_KEY, + count + ); + return Response.ok(executionEntryList).build(); + } + catch (IllegalArgumentException e) { + return Response.status(Response.Status.BAD_REQUEST) + .entity(ImmutableMap.of("error", e.getMessage())) + .build(); + } + } + List executionEntryList = auditManager.fetchAuditHistory( + ExecutionConfig.CONFIG_KEY, + ExecutionConfig.CONFIG_KEY, + theInterval + ); + return Response.ok(executionEntryList).build(); + } + + @GET + @Path("/execution") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(ConfigResourceFilter.class) + public Response getExecutionConfig() + { + if (executionConfigRef == null) { + executionConfigRef = configManager.watch(ExecutionConfig.CONFIG_KEY, ExecutionConfig.class); + } + + return Response.ok(executionConfigRef.get()).build(); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java new file mode 100644 index 000000000000..875e34d0f44d --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java @@ -0,0 +1,112 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.query.DruidMetrics; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class MappingRule +{ + private List selectors; + + @JsonCreator + public MappingRule(@JsonProperty("selectors") List selectors) + { + this.selectors = selectors; + } + + public boolean evaluate(Task task) + { + return selectors.stream().anyMatch(selector -> selector.isSatisfied(task)); + } + + @JsonProperty + public List getSelectors() + { + return selectors; + } + + static class Selector + { + private Map> cxtTagsConditions; + private Map> taskFieldsConditions; + + @JsonCreator + public Selector( + @JsonProperty("context.tags") Map> cxtTagsConditions, + @JsonProperty("task") Map> taskFieldsConditions + ) + { + this.cxtTagsConditions = cxtTagsConditions; + this.taskFieldsConditions = taskFieldsConditions; + } + + public boolean isSatisfied(Task task) + { + boolean tagsMatch = cxtTagsConditions.entrySet().stream().allMatch(entry -> { + String expectedTag = entry.getKey(); + Set expectedTagSet = entry.getValue(); + Map tags = task.getContextValue(DruidMetrics.TAGS); + if (tags == null || tags.isEmpty()) { + return false; + } + Object tagValue = tags.get(expectedTag); + + return tagValue == null ? false : expectedTagSet.contains((String) tagValue); + }); + + if (!tagsMatch) { + return false; + } + + return taskFieldsConditions.entrySet().stream().allMatch(entry -> { + String expectedField = entry.getKey(); + Set expectedFieldSet = entry.getValue(); + if ("datasource".equalsIgnoreCase(expectedField)) { + return expectedFieldSet.contains(task.getDataSource()); + } + + if ("type".equalsIgnoreCase(expectedField)) { + return expectedFieldSet.contains(task.getType()); + } + + return false; + }); + } + + @JsonProperty("context.tags") + public Map> getCxtTagsConditions() + { + return cxtTagsConditions; + } + + @JsonProperty("task") + public Map> getTaskFieldsConditions() + { + return taskFieldsConditions; + } + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index c22fa5869d8c..31e5d17f22d7 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -20,6 +20,7 @@ package org.apache.druid.k8s.overlord.taskadapter; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import io.fabric8.kubernetes.api.model.EnvVar; @@ -46,6 +47,7 @@ import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.KubernetesOverlordUtils; +import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import org.apache.druid.server.DruidNode; import org.apache.druid.tasklogs.TaskLogs; @@ -92,6 +94,7 @@ public class PodTemplateTaskAdapter implements TaskAdapter private final ObjectMapper mapper; private final HashMap templates; private final TaskLogs taskLogs; + private final Supplier executionConfigRef; public PodTemplateTaskAdapter( KubernetesTaskRunnerConfig taskRunnerConfig, @@ -99,7 +102,8 @@ public PodTemplateTaskAdapter( DruidNode node, ObjectMapper mapper, Properties properties, - TaskLogs taskLogs + TaskLogs taskLogs, + Supplier executionConfigRef ) { this.taskRunnerConfig = taskRunnerConfig; @@ -108,6 +112,7 @@ public PodTemplateTaskAdapter( this.mapper = mapper; this.templates = initializePodTemplates(properties); this.taskLogs = taskLogs; + this.executionConfigRef = executionConfigRef; } /** @@ -126,7 +131,17 @@ public PodTemplateTaskAdapter( @Override public Job fromTask(Task task) throws IOException { - PodTemplate podTemplate = templates.getOrDefault(task.getType(), templates.get("base")); + PodTemplate podTemplate = null; + ExecutionConfig executionConfig = executionConfigRef.get(); + if (executionConfig != null && executionConfig.getBehaviorStrategy() != null) { + String category = executionConfig.getBehaviorStrategy().getTaskCategory(task); + if (category != null) { + podTemplate = templates.get(category); + } + } + if (podTemplate == null) { + podTemplate = templates.getOrDefault(task.getType(), templates.get("base")); + } if (podTemplate == null) { throw new ISE("Pod template spec not found for task type [%s]", task.getType()); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java index b83ec562bda7..df6adb09ac95 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java @@ -19,10 +19,13 @@ package org.apache.druid.k8s.overlord; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.ProvisionException; +import com.google.inject.TypeLiteral; import org.apache.druid.guice.ConfigModule; import org.apache.druid.guice.DruidGuiceExtensions; import org.apache.druid.guice.annotations.EscalatedGlobal; @@ -33,6 +36,7 @@ import org.apache.druid.jackson.JacksonModule; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import org.apache.druid.server.DruidNode; import org.easymock.EasyMockRunner; import org.easymock.Mock; @@ -111,6 +115,10 @@ private Injector makeInjectorWithProperties( if (isWorkerTypeHttpRemote) { binder.bind(HttpRemoteTaskRunnerFactory.class).toInstance(httpRemoteTaskRunnerFactory); } + binder.bind( + new TypeLiteral>() + { + }).toInstance(Suppliers.ofInstance(() -> null)); }, new ConfigModule(), new KubernetesOverlordModule() diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index ba9d2accf170..08d377b6d105 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -20,12 +20,14 @@ package org.apache.druid.k8s.overlord; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.config.TaskConfigBuilder; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; +import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import org.apache.druid.k8s.overlord.taskadapter.MultiContainerTaskAdapter; import org.apache.druid.k8s.overlord.taskadapter.PodTemplateTaskAdapter; import org.apache.druid.k8s.overlord.taskadapter.SingleContainerTaskAdapter; @@ -53,6 +55,7 @@ public class KubernetesTaskRunnerFactoryTest private DruidKubernetesClient druidKubernetesClient; @Mock private ServiceEmitter emitter; + @Mock private Supplier executionConfigRef; @Before public void setup() @@ -90,7 +93,8 @@ public void test_get_returnsSameKuberentesTaskRunner_asBuild() taskConfig, properties, druidKubernetesClient, - emitter + emitter, + executionConfigRef ); KubernetesTaskRunner expectedRunner = factory.build(); @@ -112,7 +116,8 @@ public void test_build_withoutSidecarSupport_returnsKubernetesTaskRunnerWithSing taskConfig, properties, druidKubernetesClient, - emitter + emitter, + executionConfigRef ); KubernetesTaskRunner runner = factory.build(); @@ -139,7 +144,8 @@ public void test_build_withSidecarSupport_returnsKubernetesTaskRunnerWithMultiCo taskConfig, properties, druidKubernetesClient, - emitter + emitter, + executionConfigRef ); KubernetesTaskRunner runner = factory.build(); @@ -164,7 +170,8 @@ public void test_build_withSingleContainerAdapterType_returnsKubernetesTaskRunne taskConfig, props, druidKubernetesClient, - emitter + emitter, + executionConfigRef ); KubernetesTaskRunner runner = factory.build(); @@ -194,7 +201,8 @@ public void test_build_withSingleContainerAdapterTypeAndSidecarSupport_throwsIAE taskConfig, props, druidKubernetesClient, - emitter + emitter, + executionConfigRef ); Assert.assertThrows( @@ -225,7 +233,8 @@ public void test_build_withMultiContainerAdapterType_returnsKubernetesTaskRunner taskConfig, props, druidKubernetesClient, - emitter + emitter, + executionConfigRef ); KubernetesTaskRunner runner = factory.build(); @@ -250,7 +259,8 @@ public void test_build_withMultiContainerAdapterTypeAndSidecarSupport_returnsKub taskConfig, props, druidKubernetesClient, - emitter + emitter, + executionConfigRef ); KubernetesTaskRunner runner = factory.build(); @@ -278,7 +288,8 @@ public void test_build_withPodTemplateAdapterType_returnsKubernetesTaskRunnerWit taskConfig, props, druidKubernetesClient, - emitter + emitter, + executionConfigRef ); KubernetesTaskRunner runner = factory.build(); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 1796bb2b4396..6ecc8209b1b0 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import io.fabric8.kubernetes.api.model.PodTemplate; import io.fabric8.kubernetes.api.model.PodTemplateBuilder; @@ -40,10 +41,10 @@ import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; +import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import org.apache.druid.server.DruidNode; import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; -import org.easymock.Mock; import org.junit.Assert; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -71,7 +72,8 @@ public class PodTemplateTaskAdapterTest private TaskConfig taskConfig; private DruidNode node; private ObjectMapper mapper; - @Mock private TaskLogs taskLogs; + private TaskLogs taskLogs; + private Supplier executionConfigRef; @BeforeEach public void setup() @@ -89,6 +91,9 @@ public void setup() ); mapper = new TestUtils().getTestObjectMapper(); podTemplateSpec = K8sTestUtils.fileToResource("basePodTemplate.yaml", PodTemplate.class); + + taskLogs = EasyMock.createMock(TaskLogs.class); + executionConfigRef = EasyMock.createMock(Supplier.class); } @Test @@ -103,7 +108,8 @@ public void test_fromTask_withoutBasePodTemplateInRuntimeProperites_raisesIAE() node, mapper, new Properties(), - taskLogs + taskLogs, + executionConfigRef )); Assert.assertEquals(exception.getMessage(), "Pod template task adapter requires a base pod template to be specified under druid.indexer.runner.k8s.podTemplate.base"); } @@ -125,7 +131,8 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites_withEmptyFile_r node, mapper, props, - taskLogs + taskLogs, + executionConfigRef )); Assert.assertTrue(exception.getMessage().contains("Failed to load pod template file for")); @@ -147,7 +154,8 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites() throws IOExce node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Task task = new NoopTask("id", "id", "datasource", 0, 0, null); @@ -180,7 +188,8 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites_andTlsEnabled() ), mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Task task = new NoopTask("id", "id", "datasource", 0, 0, null); @@ -207,7 +216,8 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperties_withEmptyFile_r node, mapper, props, - taskLogs + taskLogs, + executionConfigRef )); } @@ -227,7 +237,8 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperites() throws IOExce node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Task task = new NoopTask("id", "id", "datasource", 0, 0, null); @@ -253,7 +264,8 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperites_dontSetTaskJSON node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Task task = new NoopTask( @@ -286,7 +298,8 @@ public void test_fromTask_withoutAnnotations_throwsDruidException() throws IOExc node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Job job = K8sTestUtils.fileToResource("baseJobWithoutAnnotations.yaml", Job.class); @@ -309,7 +322,8 @@ public void test_getTaskId() throws IOException node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Job job = new JobBuilder() .editSpec().editTemplate().editMetadata() @@ -333,7 +347,8 @@ public void test_getTaskId_noAnnotations() throws IOException node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Job job = new JobBuilder() .editSpec().editTemplate().editMetadata() @@ -357,7 +372,8 @@ public void test_getTaskId_missingTaskIdAnnotation() throws IOException node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Job job = new JobBuilder() .editSpec().editTemplate().editMetadata() @@ -383,7 +399,8 @@ public void test_toTask_withoutTaskAnnotation_throwsIOE() throws IOException node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Job baseJob = K8sTestUtils.fileToResource("baseJobWithoutAnnotations.yaml", Job.class); @@ -415,7 +432,8 @@ public void test_toTask() throws IOException node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Job job = K8sTestUtils.fileToResource("baseJob.yaml", Job.class); @@ -446,7 +464,8 @@ public void test_toTask_useTaskPayloadManager() throws IOException node, mapper, props, - mockTestLogs + mockTestLogs, + executionConfigRef ); Job job = K8sTestUtils.fileToResource("expectedNoopJob.yaml", Job.class); @@ -470,7 +489,8 @@ public void test_fromTask_withRealIds() throws IOException node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Task task = new NoopTask( @@ -504,7 +524,8 @@ public void test_fromTask_taskSupportsQueries() throws IOException node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Task task = EasyMock.mock(Task.class); @@ -552,7 +573,8 @@ public void test_fromTask_withIndexKafkaPodTemplateInRuntimeProperites() throws node, mapper, props, - taskLogs + taskLogs, + executionConfigRef ); Task kafkaTask = new NoopTask("id", "id", "datasource", 0, 0, null) { From 615669bda489d230b405f4e7ae9b36bbed19b04e Mon Sep 17 00:00:00 2001 From: YongGang Date: Tue, 28 May 2024 12:51:43 -0700 Subject: [PATCH 02/16] add Javadocs --- .../execution/DefaultExecutionConfig.java | 5 - .../DynamicTaskExecutionBehaviorStrategy.java | 19 +++- .../execution/ExecutionBehaviorStrategy.java | 11 +++ .../overlord/execution/ExecutionConfig.java | 9 ++ .../execution/KubernetesResource.java | 6 ++ .../k8s/overlord/execution/MappingRule.java | 97 +++++++++++++++++-- 6 files changed, 133 insertions(+), 14 deletions(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java index 054ceb05f20f..79d96527be16 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java @@ -28,11 +28,6 @@ public class DefaultExecutionConfig implements ExecutionConfig { private final ExecutionBehaviorStrategy behaviorStrategy; - public static DefaultExecutionConfig defaultConfig() - { - return new DefaultExecutionConfig(new DynamicTaskExecutionBehaviorStrategy(null)); - } - @JsonCreator public DefaultExecutionConfig( @JsonProperty("behaviorStrategy") ExecutionBehaviorStrategy behaviorStrategy diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java index eea1b527a26f..71746c7facf9 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java @@ -27,10 +27,19 @@ import java.util.Map; import java.util.Objects; +/** + * Implements {@link ExecutionBehaviorStrategy} by dynamically evaluating a series of mapping rules. + * Each rule corresponds to a potential task category. + */ public class DynamicTaskExecutionBehaviorStrategy implements ExecutionBehaviorStrategy { private LinkedHashMap categoryRuleMap; + /** + * Constructs a new strategy with a predefined map of category rules. + * + * @param categoryRuleMap a map linking categories to their respective {@link MappingRule} + */ @JsonCreator public DynamicTaskExecutionBehaviorStrategy( @JsonProperty("categoryMap") LinkedHashMap categoryRuleMap @@ -39,13 +48,19 @@ public DynamicTaskExecutionBehaviorStrategy( this.categoryRuleMap = categoryRuleMap; } + /** + * Evaluates the provided task against the set mapping rules to determine its category. + * + * @param task the task to be categorized + * @return the category if a rule matches, otherwise null + */ @Override public String getTaskCategory(Task task) { return categoryRuleMap.entrySet() .stream() - .filter(categoryEntry -> - categoryEntry.getValue().evaluate(task)) + .filter(categoryRule -> + categoryRule.getValue().evaluate(task)) .findFirst() .map(Map.Entry::getKey) .orElse(null); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java index 54726f325f41..5875a13446de 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java @@ -25,12 +25,23 @@ import javax.annotation.Nullable; +/** + * Defines a strategy for determining the execution behavior of tasks based on specific conditions. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DynamicTaskExecutionBehaviorStrategy.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "default", value = DynamicTaskExecutionBehaviorStrategy.class) }) public interface ExecutionBehaviorStrategy { + /** + * Determines the category of a task based on its properties. The identified category is used + * to map different Peon Pod templates, which allows for tailored resource allocation and management + * according to the task's requirements. + * + * @param task the task to evaluate + * @return the category of the task, or null if no category matches + */ @Nullable String getTaskCategory(Task task); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java index a154440a603b..c258aec0be49 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java @@ -22,6 +22,11 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +/** + * Represents the configuration for task execution within a Kubernetes environment. + * This interface allows for dynamic configuration of task execution strategies based + * on specified behavior strategies. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultExecutionConfig.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "default", value = DefaultExecutionConfig.class) @@ -30,5 +35,9 @@ public interface ExecutionConfig { String CONFIG_KEY = "k8s.taskrunner.config"; + /** + * Retrieves the execution behavior strategy associated with this configuration. + * @return the execution behavior strategy + */ ExecutionBehaviorStrategy getBehaviorStrategy(); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java index 5faaccfec81e..185f442616f9 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java @@ -45,6 +45,12 @@ import java.util.List; import java.util.concurrent.atomic.AtomicReference; +/** + * Resource that manages Kubernetes-specific execution configurations for running tasks. + * + *

This class handles the CRUD operations for execution configurations and provides + * endpoints to update, retrieve, and manage the history of these configurations.

+ */ @Path("/druid/indexer/v1/k8s/runner") public class KubernetesResource { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java index 875e34d0f44d..5c2ccb8a9234 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java @@ -26,8 +26,12 @@ import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; +/** + * Represents a rule that can evaluate whether a task satisfies a set of conditions. + */ public class MappingRule { private List selectors; @@ -38,6 +42,12 @@ public MappingRule(@JsonProperty("selectors") List selectors) this.selectors = selectors; } + /** + * Evaluates the specified task against all selectors, determining if any selector is satisfied by the task. + * + * @param task the task to evaluate + * @return true if any selector is satisfied, otherwise false + */ public boolean evaluate(Task task) { return selectors.stream().anyMatch(selector -> selector.isSatisfied(task)); @@ -49,11 +59,47 @@ public List getSelectors() return selectors; } + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MappingRule that = (MappingRule) o; + return Objects.equals(selectors, that.selectors); + } + + @Override + public int hashCode() + { + return Objects.hashCode(selectors); + } + + @Override + public String toString() + { + return "MappingRule{" + + "selectors=" + selectors + + '}'; + } + + /** + * Represents a condition or set of conditions that can be used to evaluate whether a task meets specific criteria. + */ static class Selector { private Map> cxtTagsConditions; private Map> taskFieldsConditions; + /** + * Creates a selector with specified conditions for context tags and task fields. + * + * @param cxtTagsConditions conditions on context tags + * @param taskFieldsConditions conditions on task fields + */ @JsonCreator public Selector( @JsonProperty("context.tags") Map> cxtTagsConditions, @@ -64,9 +110,15 @@ public Selector( this.taskFieldsConditions = taskFieldsConditions; } + /** + * Evaluates this selector against a given task. + * + * @param task the task to evaluate + * @return true if the task meets all the conditions specified by this selector, otherwise false + */ public boolean isSatisfied(Task task) { - boolean tagsMatch = cxtTagsConditions.entrySet().stream().allMatch(entry -> { + boolean tagsMatch = cxtTagsConditions.entrySet().stream().anyMatch(entry -> { String expectedTag = entry.getKey(); Set expectedTagSet = entry.getValue(); Map tags = task.getContextValue(DruidMetrics.TAGS); @@ -78,19 +130,19 @@ public boolean isSatisfied(Task task) return tagValue == null ? false : expectedTagSet.contains((String) tagValue); }); - if (!tagsMatch) { - return false; + if (tagsMatch) { + return true; } - return taskFieldsConditions.entrySet().stream().allMatch(entry -> { + return taskFieldsConditions.entrySet().stream().anyMatch(entry -> { String expectedField = entry.getKey(); - Set expectedFieldSet = entry.getValue(); + Set expectedValueSet = entry.getValue(); if ("datasource".equalsIgnoreCase(expectedField)) { - return expectedFieldSet.contains(task.getDataSource()); + return expectedValueSet.contains(task.getDataSource()); } if ("type".equalsIgnoreCase(expectedField)) { - return expectedFieldSet.contains(task.getType()); + return expectedValueSet.contains(task.getType()); } return false; @@ -108,5 +160,36 @@ public Map> getTaskFieldsConditions() { return taskFieldsConditions; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Selector selector = (Selector) o; + return Objects.equals(cxtTagsConditions, selector.cxtTagsConditions) && Objects.equals( + taskFieldsConditions, + selector.taskFieldsConditions + ); + } + + @Override + public int hashCode() + { + return Objects.hash(cxtTagsConditions, taskFieldsConditions); + } + + @Override + public String toString() + { + return "Selector{" + + "context.tags=" + cxtTagsConditions + + ", task=" + taskFieldsConditions + + '}'; + } } } From f34033afef05f1bbcdd1e222fef1396a5c695874 Mon Sep 17 00:00:00 2001 From: YongGang Date: Tue, 28 May 2024 22:18:02 -0700 Subject: [PATCH 03/16] refine JSON input config --- .../DynamicTaskExecutionBehaviorStrategy.java | 46 ++--- .../execution/KubernetesResource.java | 19 ++ .../k8s/overlord/execution/MappingRule.java | 195 ------------------ .../k8s/overlord/execution/Selector.java | 148 +++++++++++++ .../execution/ExecutionConfigTest.java | 69 +++++++ 5 files changed, 255 insertions(+), 222 deletions(-) delete mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java index 71746c7facf9..a567b08783cb 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java @@ -23,53 +23,45 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.common.task.Task; -import java.util.LinkedHashMap; -import java.util.Map; +import java.util.List; import java.util.Objects; /** - * Implements {@link ExecutionBehaviorStrategy} by dynamically evaluating a series of mapping rules. - * Each rule corresponds to a potential task category. + * Implements {@link ExecutionBehaviorStrategy} by dynamically evaluating a series of selectors. + * Each selector corresponds to a potential task category. */ public class DynamicTaskExecutionBehaviorStrategy implements ExecutionBehaviorStrategy { - private LinkedHashMap categoryRuleMap; + private List categorySelectors; - /** - * Constructs a new strategy with a predefined map of category rules. - * - * @param categoryRuleMap a map linking categories to their respective {@link MappingRule} - */ @JsonCreator public DynamicTaskExecutionBehaviorStrategy( - @JsonProperty("categoryMap") LinkedHashMap categoryRuleMap + @JsonProperty("categorySelectors") List categorySelectors ) { - this.categoryRuleMap = categoryRuleMap; + this.categorySelectors = categorySelectors; } /** - * Evaluates the provided task against the set mapping rules to determine its category. + * Evaluates the provided task against the set selectors to determine its category. * * @param task the task to be categorized - * @return the category if a rule matches, otherwise null + * @return the category if a selector matches, otherwise null */ @Override public String getTaskCategory(Task task) { - return categoryRuleMap.entrySet() - .stream() - .filter(categoryRule -> - categoryRule.getValue().evaluate(task)) - .findFirst() - .map(Map.Entry::getKey) - .orElse(null); + return categorySelectors.stream() + .filter(selector -> selector.evaluate(task)) + .findFirst() + .map(Selector::getName) + .orElse(null); } - @JsonProperty("categoryMap") - public LinkedHashMap getCategoryRuleMap() + @JsonProperty + public List getCategorySelectors() { - return categoryRuleMap; + return categorySelectors; } @Override @@ -82,20 +74,20 @@ public boolean equals(Object o) return false; } DynamicTaskExecutionBehaviorStrategy that = (DynamicTaskExecutionBehaviorStrategy) o; - return Objects.equals(categoryRuleMap, that.categoryRuleMap); + return Objects.equals(categorySelectors, that.categorySelectors); } @Override public int hashCode() { - return Objects.hashCode(categoryRuleMap); + return Objects.hashCode(categorySelectors); } @Override public String toString() { return "DynamicTaskExecutionBehaviorStrategy{" + - "categoryMap=" + categoryRuleMap + + "categorySelectors=" + categorySelectors + '}'; } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java index 185f442616f9..63106e1ad6bc 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java @@ -69,6 +69,13 @@ public KubernetesResource( this.auditManager = auditManager; } + /** + * Updates the Kubernetes execution configuration. + * + * @param executionConfig the new execution configuration to set + * @param req the HTTP servlet request providing context for audit information + * @return a response indicating the success or failure of the update operation + */ @POST @Path("/execution") @Consumes(MediaType.APPLICATION_JSON) @@ -92,6 +99,13 @@ public Response setExecutionConfig( } } + /** + * Retrieves the history of changes to the Kubernetes execution configuration. + * + * @param interval the time interval for fetching historical data (optional) + * @param count the maximum number of historical entries to fetch (optional) + * @return a response containing a list of audit entries or an error message + */ @GET @Path("/execution/history") @Produces(MediaType.APPLICATION_JSON) @@ -125,6 +139,11 @@ public Response getExecutionConfigHistory( return Response.ok(executionEntryList).build(); } + /** + * Retrieves the current execution configuration for tasks running in Kubernetes. + * + * @return a Response object containing the current execution configuration in JSON format. + */ @GET @Path("/execution") @Produces(MediaType.APPLICATION_JSON) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java deleted file mode 100644 index 5c2ccb8a9234..000000000000 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/MappingRule.java +++ /dev/null @@ -1,195 +0,0 @@ -/* - * 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.execution; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.query.DruidMetrics; - -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; - -/** - * Represents a rule that can evaluate whether a task satisfies a set of conditions. - */ -public class MappingRule -{ - private List selectors; - - @JsonCreator - public MappingRule(@JsonProperty("selectors") List selectors) - { - this.selectors = selectors; - } - - /** - * Evaluates the specified task against all selectors, determining if any selector is satisfied by the task. - * - * @param task the task to evaluate - * @return true if any selector is satisfied, otherwise false - */ - public boolean evaluate(Task task) - { - return selectors.stream().anyMatch(selector -> selector.isSatisfied(task)); - } - - @JsonProperty - public List getSelectors() - { - return selectors; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - MappingRule that = (MappingRule) o; - return Objects.equals(selectors, that.selectors); - } - - @Override - public int hashCode() - { - return Objects.hashCode(selectors); - } - - @Override - public String toString() - { - return "MappingRule{" + - "selectors=" + selectors + - '}'; - } - - /** - * Represents a condition or set of conditions that can be used to evaluate whether a task meets specific criteria. - */ - static class Selector - { - private Map> cxtTagsConditions; - private Map> taskFieldsConditions; - - /** - * Creates a selector with specified conditions for context tags and task fields. - * - * @param cxtTagsConditions conditions on context tags - * @param taskFieldsConditions conditions on task fields - */ - @JsonCreator - public Selector( - @JsonProperty("context.tags") Map> cxtTagsConditions, - @JsonProperty("task") Map> taskFieldsConditions - ) - { - this.cxtTagsConditions = cxtTagsConditions; - this.taskFieldsConditions = taskFieldsConditions; - } - - /** - * Evaluates this selector against a given task. - * - * @param task the task to evaluate - * @return true if the task meets all the conditions specified by this selector, otherwise false - */ - public boolean isSatisfied(Task task) - { - boolean tagsMatch = cxtTagsConditions.entrySet().stream().anyMatch(entry -> { - String expectedTag = entry.getKey(); - Set expectedTagSet = entry.getValue(); - Map tags = task.getContextValue(DruidMetrics.TAGS); - if (tags == null || tags.isEmpty()) { - return false; - } - Object tagValue = tags.get(expectedTag); - - return tagValue == null ? false : expectedTagSet.contains((String) tagValue); - }); - - if (tagsMatch) { - return true; - } - - return taskFieldsConditions.entrySet().stream().anyMatch(entry -> { - String expectedField = entry.getKey(); - Set expectedValueSet = entry.getValue(); - if ("datasource".equalsIgnoreCase(expectedField)) { - return expectedValueSet.contains(task.getDataSource()); - } - - if ("type".equalsIgnoreCase(expectedField)) { - return expectedValueSet.contains(task.getType()); - } - - return false; - }); - } - - @JsonProperty("context.tags") - public Map> getCxtTagsConditions() - { - return cxtTagsConditions; - } - - @JsonProperty("task") - public Map> getTaskFieldsConditions() - { - return taskFieldsConditions; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Selector selector = (Selector) o; - return Objects.equals(cxtTagsConditions, selector.cxtTagsConditions) && Objects.equals( - taskFieldsConditions, - selector.taskFieldsConditions - ); - } - - @Override - public int hashCode() - { - return Objects.hash(cxtTagsConditions, taskFieldsConditions); - } - - @Override - public String toString() - { - return "Selector{" + - "context.tags=" + cxtTagsConditions + - ", task=" + taskFieldsConditions + - '}'; - } - } -} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java new file mode 100644 index 000000000000..1a8ae9b8ac6a --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java @@ -0,0 +1,148 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.query.DruidMetrics; + +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + * Represents a condition-based selector that evaluates whether a given task meets specified criteria. + * The selector uses conditions defined on context tags and task fields to determine if a task matches. + */ +public class Selector +{ + private final String name; + private final Map> cxtTagsConditions; + private final Map> taskFieldsConditions; + + /** + * Creates a selector with specified conditions for context tags and task fields. + * + * @param name the identifier representing the outcome when a task matches the conditions + * @param cxtTagsConditions conditions on context tags + * @param taskFieldsConditions conditions on task fields + */ + @JsonCreator + public Selector( + @JsonProperty("name") String name, + @JsonProperty("context.tags") Map> cxtTagsConditions, + @JsonProperty("task") Map> taskFieldsConditions + ) + { + this.name = name; + this.cxtTagsConditions = cxtTagsConditions; + this.taskFieldsConditions = taskFieldsConditions; + } + + /** + * Evaluates this selector against a given task. + * + * @param task the task to evaluate + * @return true if the task meets all the conditions specified by this selector, otherwise false + */ + public boolean evaluate(Task task) + { + boolean tagsMatch = cxtTagsConditions.entrySet().stream().allMatch(entry -> { + String tagKey = entry.getKey(); + Set tagValues = entry.getValue(); + Map tags = task.getContextValue(DruidMetrics.TAGS); + if (tags == null || tags.isEmpty()) { + return false; + } + Object tagValue = tags.get(tagKey); + + return tagValue == null ? false : tagValues.contains((String) tagValue); + }); + + if (!tagsMatch) { + return false; + } + + return taskFieldsConditions.entrySet().stream().allMatch(entry -> { + String fieldKey = entry.getKey(); + Set fieldValues = entry.getValue(); + if ("datasource".equalsIgnoreCase(fieldKey)) { + return fieldValues.contains(task.getDataSource()); + } + + if ("type".equalsIgnoreCase(fieldKey)) { + return fieldValues.contains(task.getType()); + } + + return false; + }); + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty("context.tags") + public Map> getCxtTagsConditions() + { + return cxtTagsConditions; + } + + @JsonProperty("task") + public Map> getTaskFieldsConditions() + { + return taskFieldsConditions; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Selector selector = (Selector) o; + return Objects.equals(name, selector.name) && Objects.equals( + cxtTagsConditions, + selector.cxtTagsConditions + ) && Objects.equals(taskFieldsConditions, selector.taskFieldsConditions); + } + + @Override + public int hashCode() + { + return Objects.hash(name, cxtTagsConditions, taskFieldsConditions); + } + + @Override + public String toString() + { + return "Selector{" + + "name=" + name + + ", context.tags=" + cxtTagsConditions + + ", task=" + taskFieldsConditions + + '}'; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java new file mode 100644 index 000000000000..fa733f1cf865 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java @@ -0,0 +1,69 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +public class ExecutionConfigTest +{ + private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + @Test + public void testDeserializeExecutionConfig() throws JsonProcessingException + { + String json = "{\n" + + " \"type\": \"default\",\n" + + " \"behaviorStrategy\": {\n" + + " \"type\": \"default\",\n" + + " \"categorySelectors\": [\n" + + " {\n" + + " \"name\": \"low-throughput\",\n" + + " \"context.tags\": {\n" + + " \"billingCategory\": [\n" + + " \"streaming_ingestion\"\n" + + " ]\n" + + " },\n" + + " \"task\": {\n" + + " \"datasource\": [\n" + + " \"wikipedia\"\n" + + " ]\n" + + " }\n" + + " },\n" + + " {\n" + + " \"name\": \"medium-throughput\",\n" + + " \"task\": {\n" + + " \"type\": [\n" + + " \"async_query\"\n" + + " ]\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + "}"; + + ExecutionConfig deserialized = jsonMapper.readValue(json, ExecutionConfig.class); + ExecutionBehaviorStrategy behaviorStrategy = deserialized.getBehaviorStrategy(); + Assert.assertTrue(behaviorStrategy instanceof DynamicTaskExecutionBehaviorStrategy); + } +} From c3b03124d71bd2b7414dae4f72b8b187e66a3310 Mon Sep 17 00:00:00 2001 From: YongGang Date: Thu, 30 May 2024 13:49:29 -0700 Subject: [PATCH 04/16] more test and fix build --- .../overlord/KubernetesTaskRunnerFactory.java | 3 +- .../overlord/common/KubernetesPeonClient.java | 11 +- .../execution/DefaultExecutionConfig.java | 2 + .../DynamicTaskExecutionBehaviorStrategy.java | 4 + .../k8s/overlord/execution/Selector.java | 51 ++++---- .../KubernetesOverlordModuleTest.java | 23 +++- .../common/KubernetesPeonClientTest.java | 14 ++- ...amicTaskExecutionBehaviorStrategyTest.java | 89 ++++++++++++++ .../k8s/overlord/execution/SelectorTest.java | 115 ++++++++++++++++++ .../DruidPeonClientIntegrationTest.java | 10 +- .../PodTemplateTaskAdapterTest.java | 4 +- 11 files changed, 294 insertions(+), 32 deletions(-) create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategyTest.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index 874220403311..0189651d51ac 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -97,7 +97,8 @@ public KubernetesTaskRunner build() druidKubernetesClient, kubernetesTaskRunnerConfig.getNamespace(), kubernetesTaskRunnerConfig.isDebugJobs(), - emitter + emitter, + executionConfigRef ); runner = new KubernetesTaskRunner( diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index 63487e4e373e..689526630a19 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; +import com.google.common.base.Supplier; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; @@ -31,6 +32,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import java.io.InputStream; import java.sql.Timestamp; @@ -47,18 +49,21 @@ public class KubernetesPeonClient private final String namespace; private final boolean debugJobs; private final ServiceEmitter emitter; + private final Supplier executionConfigRef; public KubernetesPeonClient( KubernetesClientApi clientApi, String namespace, boolean debugJobs, - ServiceEmitter emitter + ServiceEmitter emitter, + Supplier executionConfigRef ) { this.clientApi = clientApi; this.namespace = namespace; this.debugJobs = debugJobs; this.emitter = emitter; + this.executionConfigRef = executionConfigRef; } public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUnit timeUnit) throws IllegalStateException @@ -274,6 +279,10 @@ private void emitK8sPodMetrics(Task task, String metric, long durationMs) { ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); IndexTaskUtils.setTaskDimensions(metricBuilder, task); + metricBuilder.setDimensionIfNotNull( + "category", + executionConfigRef.get().getBehaviorStrategy().getTaskCategory(task) + ); emitter.emit(metricBuilder.setMetric(metric, durationMs)); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java index 79d96527be16..190b278cb59d 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; import java.util.Objects; @@ -33,6 +34,7 @@ public DefaultExecutionConfig( @JsonProperty("behaviorStrategy") ExecutionBehaviorStrategy behaviorStrategy ) { + Preconditions.checkNotNull(behaviorStrategy); this.behaviorStrategy = behaviorStrategy; } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java index a567b08783cb..2b85832e7176 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java @@ -51,6 +51,10 @@ public DynamicTaskExecutionBehaviorStrategy( @Override public String getTaskCategory(Task task) { + if (categorySelectors == null) { + return null; + } + return categorySelectors.stream() .filter(selector -> selector.evaluate(task)) .findFirst() diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java index 1a8ae9b8ac6a..bec3c9dc9060 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java @@ -65,35 +65,42 @@ public Selector( */ public boolean evaluate(Task task) { - boolean tagsMatch = cxtTagsConditions.entrySet().stream().allMatch(entry -> { - String tagKey = entry.getKey(); - Set tagValues = entry.getValue(); - Map tags = task.getContextValue(DruidMetrics.TAGS); - if (tags == null || tags.isEmpty()) { - return false; - } - Object tagValue = tags.get(tagKey); - - return tagValue == null ? false : tagValues.contains((String) tagValue); - }); + boolean tagsMatch = true; + if (cxtTagsConditions != null) { + tagsMatch = cxtTagsConditions.entrySet().stream().allMatch(entry -> { + String tagKey = entry.getKey(); + Set tagValues = entry.getValue(); + Map tags = task.getContextValue(DruidMetrics.TAGS); + if (tags == null || tags.isEmpty()) { + return false; + } + Object tagValue = tags.get(tagKey); + + return tagValue == null ? false : tagValues.contains((String) tagValue); + }); + } if (!tagsMatch) { return false; } - return taskFieldsConditions.entrySet().stream().allMatch(entry -> { - String fieldKey = entry.getKey(); - Set fieldValues = entry.getValue(); - if ("datasource".equalsIgnoreCase(fieldKey)) { - return fieldValues.contains(task.getDataSource()); - } + if (taskFieldsConditions != null) { + return taskFieldsConditions.entrySet().stream().allMatch(entry -> { + String fieldKey = entry.getKey(); + Set fieldValues = entry.getValue(); + if ("datasource".equalsIgnoreCase(fieldKey)) { + return fieldValues.contains(task.getDataSource()); + } - if ("type".equalsIgnoreCase(fieldKey)) { - return fieldValues.contains(task.getType()); - } + if ("type".equalsIgnoreCase(fieldKey)) { + return fieldValues.contains(task.getType()); + } - return false; - }); + return false; + }); + } + + return true; } @JsonProperty diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java index df6adb09ac95..369f29726a40 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesOverlordModuleTest.java @@ -26,6 +26,8 @@ import com.google.inject.Injector; import com.google.inject.ProvisionException; import com.google.inject.TypeLiteral; +import org.apache.druid.audit.AuditManager; +import org.apache.druid.common.config.ConfigManagerConfig; import org.apache.druid.guice.ConfigModule; import org.apache.druid.guice.DruidGuiceExtensions; import org.apache.druid.guice.annotations.EscalatedGlobal; @@ -36,7 +38,8 @@ import org.apache.druid.jackson.JacksonModule; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.k8s.overlord.execution.ExecutionConfig; +import org.apache.druid.metadata.MetadataStorageConnector; +import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.server.DruidNode; import org.easymock.EasyMockRunner; import org.easymock.Mock; @@ -59,6 +62,14 @@ public class KubernetesOverlordModuleTest private RemoteTaskRunnerFactory remoteTaskRunnerFactory; @Mock private HttpRemoteTaskRunnerFactory httpRemoteTaskRunnerFactory; + @Mock + private ConfigManagerConfig configManagerConfig; + @Mock + private MetadataStorageTablesConfig metadataStorageTablesConfig; + @Mock + private AuditManager auditManager; + @Mock + private MetadataStorageConnector metadataStorageConnector; private Injector injector; @Test @@ -116,9 +127,15 @@ private Injector makeInjectorWithProperties( binder.bind(HttpRemoteTaskRunnerFactory.class).toInstance(httpRemoteTaskRunnerFactory); } binder.bind( - new TypeLiteral>() + new TypeLiteral>() + { + }).toInstance(Suppliers.ofInstance(configManagerConfig)); + binder.bind( + new TypeLiteral>() { - }).toInstance(Suppliers.ofInstance(() -> null)); + }).toInstance(Suppliers.ofInstance(metadataStorageTablesConfig)); + binder.bind(AuditManager.class).toInstance(auditManager); + binder.bind(MetadataStorageConnector.class).toInstance(metadataStorageConnector); }, new ConfigModule(), new KubernetesOverlordModule() diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java index fa0da14fab73..175ea50d9783 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java @@ -20,6 +20,7 @@ package org.apache.druid.k8s.overlord.common; import com.google.common.base.Optional; +import com.google.common.base.Supplier; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.PodBuilder; import io.fabric8.kubernetes.api.model.PodListBuilder; @@ -33,6 +34,9 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.metrics.StubServiceEmitter; +import org.apache.druid.k8s.overlord.execution.DefaultExecutionConfig; +import org.apache.druid.k8s.overlord.execution.DynamicTaskExecutionBehaviorStrategy; +import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -57,13 +61,15 @@ public class KubernetesPeonClientTest private KubernetesClientApi clientApi; private KubernetesPeonClient instance; private StubServiceEmitter serviceEmitter; + private Supplier executionConfigRef; @BeforeEach public void setup() { clientApi = new TestKubernetesClient(this.client); serviceEmitter = new StubServiceEmitter("service", "host"); - instance = new KubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); + executionConfigRef = () -> new DefaultExecutionConfig(new DynamicTaskExecutionBehaviorStrategy(null)); + instance = new KubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter, executionConfigRef); } @Test @@ -237,7 +243,8 @@ void test_deletePeonJob_withJob_withDebugJobsTrue_skipsDelete() new TestKubernetesClient(this.client), NAMESPACE, true, - serviceEmitter + serviceEmitter, + executionConfigRef ); Job job = new JobBuilder() @@ -262,7 +269,8 @@ void test_deletePeonJob_withoutJob_withDebugJobsTrue_skipsDelete() new TestKubernetesClient(this.client), NAMESPACE, true, - serviceEmitter + serviceEmitter, + executionConfigRef ); Assertions.assertTrue(instance.deletePeonJob(new K8sTaskId(ID))); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategyTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategyTest.java new file mode 100644 index 000000000000..532fa79d84f2 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategyTest.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.k8s.overlord.execution; + +import com.google.common.collect.Lists; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; + +public class DynamicTaskExecutionBehaviorStrategyTest +{ + + @Test + public void testGetTaskCategory_nullSelectors() + { + DynamicTaskExecutionBehaviorStrategy strategy = new DynamicTaskExecutionBehaviorStrategy(null); + Task task = NoopTask.create(); + Assert.assertNull(strategy.getTaskCategory(task)); + } + + @Test + public void testGetTaskCategory_emptySelectors() + { + List emptySelectors = Collections.emptyList(); + DynamicTaskExecutionBehaviorStrategy strategy = new DynamicTaskExecutionBehaviorStrategy(emptySelectors); + Task task = NoopTask.create(); + Assert.assertNull(strategy.getTaskCategory(task)); + } + + @Test + public void testGetTaskCategory_noMatchSelectors() + { + Selector noMatchSelector = new MockSelector(false, "mock"); + List selectors = Collections.singletonList(noMatchSelector); + DynamicTaskExecutionBehaviorStrategy strategy = new DynamicTaskExecutionBehaviorStrategy(selectors); + Task task = NoopTask.create(); + Assert.assertNull(strategy.getTaskCategory(task)); + } + + @Test + public void testGetTaskCategory_withMatchSelectors() + { + Selector noMatchSelector = new MockSelector(false, "no_match"); + Selector matchSelector = new MockSelector(true, "match"); + List selectors = Lists.newArrayList(noMatchSelector, matchSelector); + DynamicTaskExecutionBehaviorStrategy strategy = new DynamicTaskExecutionBehaviorStrategy(selectors); + Task task = NoopTask.create(); + Assert.assertEquals("match", strategy.getTaskCategory(task) + ); + } + + static class MockSelector extends Selector + { + private final boolean matches; + + MockSelector(boolean matches, String name) + { + super(name, null, null); + this.matches = matches; + } + + @Override + public boolean evaluate(final Task task) + { + return matches; + } + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java new file mode 100644 index 000000000000..dc0f2aa89081 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java @@ -0,0 +1,115 @@ +/* + * 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.execution; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.query.DruidMetrics; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class SelectorTest +{ + + @Test + public void shouldReturnTrueWhenAllTagsAndTasksMatch() + { + String dataSource = "my_table"; + Map> cxtTagsConditions = new HashMap<>(); + cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); + + Map> taskFieldsConditions = new HashMap<>(); + taskFieldsConditions.put("datasource", Sets.newHashSet(dataSource)); + + Task task = NoopTask.forDatasource(dataSource); + task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); + + Selector selector = new Selector( + "TestSelector", + cxtTagsConditions, + taskFieldsConditions + ); + + Assert.assertTrue(selector.evaluate(task)); + } + + @Test + public void shouldReturnFalseWhenTagsDoNotMatch() + { + String dataSource = "my_table"; + Map> cxtTagsConditions = new HashMap<>(); + cxtTagsConditions.put("nonexistentTag", Sets.newHashSet("tag1value")); + + Map> taskFieldsConditions = new HashMap<>(); + taskFieldsConditions.put("datasource", Sets.newHashSet(dataSource)); + + Task task = NoopTask.forDatasource(dataSource); + + Selector selector = new Selector( + "TestSelector", + cxtTagsConditions, + taskFieldsConditions + ); + + Assert.assertFalse(selector.evaluate(task)); + } + + @Test + public void shouldReturnFalseWhenTaskFieldsDoNotMatch() + { + Map> cxtTagsConditions = new HashMap<>(); + cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); + + Map> taskFieldsConditions = new HashMap<>(); + taskFieldsConditions.put("datasource", Sets.newHashSet("my_table")); + + Task task = NoopTask.forDatasource("another_table"); + task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); + + Selector selector = new Selector( + "TestSelector", + cxtTagsConditions, + taskFieldsConditions + ); + + Assert.assertFalse(selector.evaluate(task)); + } + + @Test + public void shouldReturnTrueWhenNoConditionsSpecified() + { + Task task = NoopTask.forDatasource("my_table"); + task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); + + Selector selector = new Selector( + "TestSelector", + null, + null + ); + + Assert.assertTrue(selector.evaluate(task)); + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 241b4d9fc68f..67dd5503411f 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -43,6 +43,8 @@ import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.common.PeonCommandContext; import org.apache.druid.k8s.overlord.common.PeonPhase; +import org.apache.druid.k8s.overlord.execution.DefaultExecutionConfig; +import org.apache.druid.k8s.overlord.execution.DynamicTaskExecutionBehaviorStrategy; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -91,7 +93,13 @@ public void setup() new NamedType(IndexTask.IndexTuningConfig.class, "index") ); k8sClient = new DruidKubernetesClient(); - peonClient = new KubernetesPeonClient(k8sClient, "default", false, new NoopServiceEmitter()); + peonClient = new KubernetesPeonClient( + k8sClient, + "default", + false, + new NoopServiceEmitter(), + () -> new DefaultExecutionConfig(new DynamicTaskExecutionBehaviorStrategy(null)) + ); druidNode = new DruidNode( "test", null, diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 6ecc8209b1b0..480d933ed44d 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -41,6 +41,8 @@ import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; +import org.apache.druid.k8s.overlord.execution.DefaultExecutionConfig; +import org.apache.druid.k8s.overlord.execution.DynamicTaskExecutionBehaviorStrategy; import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import org.apache.druid.server.DruidNode; import org.apache.druid.tasklogs.TaskLogs; @@ -93,7 +95,7 @@ public void setup() podTemplateSpec = K8sTestUtils.fileToResource("basePodTemplate.yaml", PodTemplate.class); taskLogs = EasyMock.createMock(TaskLogs.class); - executionConfigRef = EasyMock.createMock(Supplier.class); + executionConfigRef = () -> new DefaultExecutionConfig(new DynamicTaskExecutionBehaviorStrategy(null)); } @Test From ee9b8d116ec3dea7b188a3e5eed16adb913000cf Mon Sep 17 00:00:00 2001 From: YongGang Date: Sun, 2 Jun 2024 22:09:28 -0700 Subject: [PATCH 05/16] extract existing behavior as default strategy --- .../kubernetes-overlord-extensions/pom.xml | 20 +++++++ .../overlord/common/KubernetesPeonClient.java | 12 ++-- .../DefaultExecutionBehaviorStrategy.java | 50 ++++++++++++++++ .../DynamicTaskExecutionBehaviorStrategy.java | 2 +- .../execution/ExecutionBehaviorStrategy.java | 5 +- .../overlord/execution/ExecutionConfig.java | 1 + .../k8s/overlord/execution/Selector.java | 18 +++--- .../taskadapter/PodTemplateTaskAdapter.java | 25 ++++---- .../execution/ExecutionConfigTest.java | 59 +++++++++++-------- .../DruidPeonClientIntegrationTest.java | 4 +- .../PodTemplateTaskAdapterTest.java | 51 +++++++++++++++- 11 files changed, 193 insertions(+), 54 deletions(-) create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionBehaviorStrategy.java diff --git a/extensions-contrib/kubernetes-overlord-extensions/pom.xml b/extensions-contrib/kubernetes-overlord-extensions/pom.xml index 4baf4fde84ee..be40224054da 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/pom.xml +++ b/extensions-contrib/kubernetes-overlord-extensions/pom.xml @@ -139,6 +139,26 @@ 6.7.2 runtime + + javax.ws.rs + jsr311-api + provided + + + javax.servlet + javax.servlet-api + provided + + + com.sun.jersey + jersey-server + provided + + + jakarta.inject + jakarta.inject-api + provided + diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index 689526630a19..802640e2db5b 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -279,10 +279,14 @@ private void emitK8sPodMetrics(Task task, String metric, long durationMs) { ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); IndexTaskUtils.setTaskDimensions(metricBuilder, task); - metricBuilder.setDimensionIfNotNull( - "category", - executionConfigRef.get().getBehaviorStrategy().getTaskCategory(task) - ); + ExecutionConfig executionConfig = executionConfigRef.get(); + if (executionConfig != null && executionConfig.getBehaviorStrategy() != null) { + metricBuilder.setDimensionIfNotNull( + "category", + executionConfig.getBehaviorStrategy().getTaskCategory(task) + ); + } + emitter.emit(metricBuilder.setMetric(metric, durationMs)); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionBehaviorStrategy.java new file mode 100644 index 000000000000..7ccb0ab04c06 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionBehaviorStrategy.java @@ -0,0 +1,50 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.druid.indexing.common.task.Task; + +/** + * This strategy defines how tasks are categorized based on their type for execution purposes. + * + * This implementation categorizes tasks by simply returning the type of the task, + * making it a straightforward, type-based categorization strategy. + */ +public class DefaultExecutionBehaviorStrategy implements ExecutionBehaviorStrategy +{ + @JsonCreator + public DefaultExecutionBehaviorStrategy() + { + } + + @Override + public String getTaskCategory(Task task) + { + return task.getType(); + } + + @Override + public String toString() + { + return "DefaultExecutionBehaviorStrategy{" + + '}'; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java index 2b85832e7176..1f3799a02bb2 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java @@ -58,7 +58,7 @@ public String getTaskCategory(Task task) return categorySelectors.stream() .filter(selector -> selector.evaluate(task)) .findFirst() - .map(Selector::getName) + .map(Selector::getSelectionKey) .orElse(null); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java index 5875a13446de..b4980c0a9ea3 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java @@ -28,9 +28,10 @@ /** * Defines a strategy for determining the execution behavior of tasks based on specific conditions. */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DynamicTaskExecutionBehaviorStrategy.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultExecutionBehaviorStrategy.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "default", value = DynamicTaskExecutionBehaviorStrategy.class) + @JsonSubTypes.Type(name = "default", value = DefaultExecutionBehaviorStrategy.class), + @JsonSubTypes.Type(name = "dynamicTask", value = DynamicTaskExecutionBehaviorStrategy.class), }) public interface ExecutionBehaviorStrategy { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java index c258aec0be49..1dc3e01e2fc3 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java @@ -34,6 +34,7 @@ public interface ExecutionConfig { String CONFIG_KEY = "k8s.taskrunner.config"; + ExecutionBehaviorStrategy DEFAULT_STRATEGY = new DefaultExecutionBehaviorStrategy(); /** * Retrieves the execution behavior strategy associated with this configuration. diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java index bec3c9dc9060..70284fe728e4 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java @@ -34,25 +34,25 @@ */ public class Selector { - private final String name; + private final String selectionKey; private final Map> cxtTagsConditions; private final Map> taskFieldsConditions; /** * Creates a selector with specified conditions for context tags and task fields. * - * @param name the identifier representing the outcome when a task matches the conditions + * @param selectionKey the identifier representing the outcome when a task matches the conditions * @param cxtTagsConditions conditions on context tags * @param taskFieldsConditions conditions on task fields */ @JsonCreator public Selector( - @JsonProperty("name") String name, + @JsonProperty("selectionKey") String selectionKey, @JsonProperty("context.tags") Map> cxtTagsConditions, @JsonProperty("task") Map> taskFieldsConditions ) { - this.name = name; + this.selectionKey = selectionKey; this.cxtTagsConditions = cxtTagsConditions; this.taskFieldsConditions = taskFieldsConditions; } @@ -104,9 +104,9 @@ public boolean evaluate(Task task) } @JsonProperty - public String getName() + public String getSelectionKey() { - return name; + return selectionKey; } @JsonProperty("context.tags") @@ -131,7 +131,7 @@ public boolean equals(Object o) return false; } Selector selector = (Selector) o; - return Objects.equals(name, selector.name) && Objects.equals( + return Objects.equals(selectionKey, selector.selectionKey) && Objects.equals( cxtTagsConditions, selector.cxtTagsConditions ) && Objects.equals(taskFieldsConditions, selector.taskFieldsConditions); @@ -140,14 +140,14 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(name, cxtTagsConditions, taskFieldsConditions); + return Objects.hash(selectionKey, cxtTagsConditions, taskFieldsConditions); } @Override public String toString() { return "Selector{" + - "name=" + name + + "selectionKey=" + selectionKey + ", context.tags=" + cxtTagsConditions + ", task=" + taskFieldsConditions + '}'; diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index 31e5d17f22d7..055baede175a 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -47,6 +47,7 @@ import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.KubernetesOverlordUtils; +import org.apache.druid.k8s.overlord.execution.ExecutionBehaviorStrategy; import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import org.apache.druid.server.DruidNode; import org.apache.druid.tasklogs.TaskLogs; @@ -131,17 +132,19 @@ public PodTemplateTaskAdapter( @Override public Job fromTask(Task task) throws IOException { - PodTemplate podTemplate = null; + ExecutionBehaviorStrategy behaviorStrategy; ExecutionConfig executionConfig = executionConfigRef.get(); - if (executionConfig != null && executionConfig.getBehaviorStrategy() != null) { - String category = executionConfig.getBehaviorStrategy().getTaskCategory(task); - if (category != null) { - podTemplate = templates.get(category); - } - } - if (podTemplate == null) { - podTemplate = templates.getOrDefault(task.getType(), templates.get("base")); + if (executionConfig == null || executionConfig.getBehaviorStrategy() == null) { + behaviorStrategy = ExecutionConfig.DEFAULT_STRATEGY; + } else { + behaviorStrategy = executionConfig.getBehaviorStrategy(); } + String category = behaviorStrategy.getTaskCategory(task); + PodTemplate podTemplate = templates.getOrDefault( + category != null ? category : task.getType(), + templates.get("base") + ); + if (podTemplate == null) { throw new ISE("Pod template spec not found for task type [%s]", task.getType()); } @@ -167,7 +170,9 @@ public Job fromTask(Task task) throws IOException .endTemplate() .withActiveDeadlineSeconds(taskRunnerConfig.getTaskTimeout().toStandardDuration().getStandardSeconds()) .withBackoffLimit(0) // druid does not support an external system retrying failed tasks - .withTtlSecondsAfterFinished((int) taskRunnerConfig.getTaskCleanupDelay().toStandardDuration().getStandardSeconds()) + .withTtlSecondsAfterFinished((int) taskRunnerConfig.getTaskCleanupDelay() + .toStandardDuration() + .getStandardSeconds()) .endSpec() .build(); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java index fa733f1cf865..12e59c8fb5b2 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java @@ -35,35 +35,46 @@ public void testDeserializeExecutionConfig() throws JsonProcessingException String json = "{\n" + " \"type\": \"default\",\n" + " \"behaviorStrategy\": {\n" - + " \"type\": \"default\",\n" - + " \"categorySelectors\": [\n" - + " {\n" - + " \"name\": \"low-throughput\",\n" - + " \"context.tags\": {\n" - + " \"billingCategory\": [\n" - + " \"streaming_ingestion\"\n" - + " ]\n" - + " },\n" - + " \"task\": {\n" - + " \"datasource\": [\n" - + " \"wikipedia\"\n" - + " ]\n" - + " }\n" - + " },\n" - + " {\n" - + " \"name\": \"medium-throughput\",\n" - + " \"task\": {\n" - + " \"type\": [\n" - + " \"async_query\"\n" - + " ]\n" - + " }\n" - + " }\n" - + " ]\n" + + " \"type\": \"default\"\n" + " }\n" + "}"; ExecutionConfig deserialized = jsonMapper.readValue(json, ExecutionConfig.class); ExecutionBehaviorStrategy behaviorStrategy = deserialized.getBehaviorStrategy(); + Assert.assertTrue(behaviorStrategy instanceof DefaultExecutionBehaviorStrategy); + + json = "{\n" + + " \"type\": \"default\",\n" + + " \"behaviorStrategy\": {\n" + + " \"type\": \"dynamicTask\",\n" + + " \"categorySelectors\": [\n" + + " {\n" + + " \"selectionKey\": \"low-throughput\",\n" + + " \"context.tags\": {\n" + + " \"billingCategory\": [\n" + + " \"streaming_ingestion\"\n" + + " ]\n" + + " },\n" + + " \"task\": {\n" + + " \"datasource\": [\n" + + " \"wikipedia\"\n" + + " ]\n" + + " }\n" + + " },\n" + + " {\n" + + " \"selectionKey\": \"medium-throughput\",\n" + + " \"task\": {\n" + + " \"type\": [\n" + + " \"async_query\"\n" + + " ]\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + + "}"; + + deserialized = jsonMapper.readValue(json, ExecutionConfig.class); + behaviorStrategy = deserialized.getBehaviorStrategy(); Assert.assertTrue(behaviorStrategy instanceof DynamicTaskExecutionBehaviorStrategy); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 67dd5503411f..2a4994bac034 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -44,7 +44,7 @@ import org.apache.druid.k8s.overlord.common.PeonCommandContext; import org.apache.druid.k8s.overlord.common.PeonPhase; import org.apache.druid.k8s.overlord.execution.DefaultExecutionConfig; -import org.apache.druid.k8s.overlord.execution.DynamicTaskExecutionBehaviorStrategy; +import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -98,7 +98,7 @@ public void setup() "default", false, new NoopServiceEmitter(), - () -> new DefaultExecutionConfig(new DynamicTaskExecutionBehaviorStrategy(null)) + () -> new DefaultExecutionConfig(ExecutionConfig.DEFAULT_STRATEGY) ); druidNode = new DruidNode( "test", diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 480d933ed44d..078a38d51a0f 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -44,6 +44,7 @@ import org.apache.druid.k8s.overlord.execution.DefaultExecutionConfig; import org.apache.druid.k8s.overlord.execution.DynamicTaskExecutionBehaviorStrategy; import org.apache.druid.k8s.overlord.execution.ExecutionConfig; +import org.apache.druid.k8s.overlord.execution.Selector; import org.apache.druid.server.DruidNode; import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; @@ -53,6 +54,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.mockito.Mockito; +import org.mockito.internal.util.collections.Sets; import java.io.ByteArrayInputStream; import java.io.IOException; @@ -95,7 +97,7 @@ public void setup() podTemplateSpec = K8sTestUtils.fileToResource("basePodTemplate.yaml", PodTemplate.class); taskLogs = EasyMock.createMock(TaskLogs.class); - executionConfigRef = () -> new DefaultExecutionConfig(new DynamicTaskExecutionBehaviorStrategy(null)); + executionConfigRef = () -> new DefaultExecutionConfig(ExecutionConfig.DEFAULT_STRATEGY); } @Test @@ -138,7 +140,6 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites_withEmptyFile_r )); Assert.assertTrue(exception.getMessage().contains("Failed to load pod template file for")); - } @Test @@ -595,6 +596,52 @@ public String getType() Assert.assertEquals(0, actual.getSpec().getTemplate().getSpec().getVolumes().size(), 1); } + @Test + public void test_fromTask_matchPodTemplateBasedOnCategory() throws IOException + { + String dataSource = "my_table"; + executionConfigRef = () -> new DefaultExecutionConfig(new DynamicTaskExecutionBehaviorStrategy(Collections.singletonList( + new Selector("lowThrougput", null, ImmutableMap.of( + "datasource", + Sets.newSet(dataSource) + ))))); + Path baseTemplatePath = Files.createFile(tempDir.resolve("base.yaml")); + mapper.writeValue(baseTemplatePath.toFile(), podTemplateSpec); + + Path lowThroughputTemplatePath = Files.createFile(tempDir.resolve("low-throughput.yaml")); + PodTemplate lowThroughputPodTemplate = new PodTemplateBuilder(podTemplateSpec) + .editTemplate() + .editSpec() + .setNewVolumeLike(0, new VolumeBuilder().withName("volume").build()) + .endVolume() + .endSpec() + .endTemplate() + .build(); + mapper.writeValue(lowThroughputTemplatePath.toFile(), lowThroughputPodTemplate); + + Properties props = new Properties(); + props.setProperty("druid.indexer.runner.k8s.podTemplate.base", baseTemplatePath.toString()); + props.setProperty("druid.indexer.runner.k8s.podTemplate.lowThroughput", lowThroughputTemplatePath.toString()); + + PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( + taskRunnerConfig, + taskConfig, + node, + mapper, + props, + taskLogs, + executionConfigRef + ); + + Task taskWithMatchedDatasource = new NoopTask("id", "id", dataSource, 0, 0, null); + Task noopTask = new NoopTask("id", "id", "datasource", 0, 0, null); + Job actual = adapter.fromTask(taskWithMatchedDatasource); + Assert.assertEquals(1, actual.getSpec().getTemplate().getSpec().getVolumes().size(), 1); + + actual = adapter.fromTask(noopTask); + Assert.assertEquals(0, actual.getSpec().getTemplate().getSpec().getVolumes().size(), 1); + } + private void assertJobSpecsEqual(Job actual, Job expected) throws IOException { Map actualAnnotations = actual.getSpec().getTemplate().getMetadata().getAnnotations(); From 0dd6df2c8f9628e220d247e23e46f8fd789068fe Mon Sep 17 00:00:00 2001 From: YongGang Date: Mon, 3 Jun 2024 11:00:37 -0700 Subject: [PATCH 06/16] change template mapping fallback --- .../taskadapter/PodTemplateTaskAdapter.java | 5 +- ...amicTaskExecutionBehaviorStrategyTest.java | 32 +++++++++ .../execution/ExecutionConfigTest.java | 2 +- .../k8s/overlord/execution/SelectorTest.java | 70 +++++++++++++++++++ 4 files changed, 104 insertions(+), 5 deletions(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index 055baede175a..4393cf3b719c 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -140,10 +140,7 @@ public Job fromTask(Task task) throws IOException behaviorStrategy = executionConfig.getBehaviorStrategy(); } String category = behaviorStrategy.getTaskCategory(task); - PodTemplate podTemplate = templates.getOrDefault( - category != null ? category : task.getType(), - templates.get("base") - ); + PodTemplate podTemplate = templates.getOrDefault(category, templates.get("base")); if (podTemplate == null) { throw new ISE("Pod template spec not found for task type [%s]", task.getType()); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategyTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategyTest.java index 532fa79d84f2..6ea206ef96bc 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategyTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategyTest.java @@ -19,14 +19,20 @@ package org.apache.druid.k8s.overlord.execution; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.segment.TestHelper; import org.junit.Assert; import org.junit.Test; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Set; public class DynamicTaskExecutionBehaviorStrategyTest { @@ -70,6 +76,32 @@ public void testGetTaskCategory_withMatchSelectors() ); } + @Test + public void testSerde() throws Exception + { + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + Map> cxtTagsConditions = new HashMap<>(); + cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); + + Map> taskFieldsConditions = new HashMap<>(); + taskFieldsConditions.put("type", Sets.newHashSet(NoopTask.TYPE)); + + Selector selector = new Selector( + "TestSelector", + cxtTagsConditions, + taskFieldsConditions + ); + + DynamicTaskExecutionBehaviorStrategy strategy = new DynamicTaskExecutionBehaviorStrategy(Collections.singletonList( + selector)); + + DynamicTaskExecutionBehaviorStrategy strategy2 = objectMapper.readValue( + objectMapper.writeValueAsBytes(strategy), + DynamicTaskExecutionBehaviorStrategy.class + ); + Assert.assertEquals(strategy, strategy2); + } + static class MockSelector extends Selector { private final boolean matches; diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java index 12e59c8fb5b2..bf6ce7b74a79 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java @@ -30,7 +30,7 @@ public class ExecutionConfigTest private final ObjectMapper jsonMapper = new DefaultObjectMapper(); @Test - public void testDeserializeExecutionConfig() throws JsonProcessingException + public void testSerde() throws JsonProcessingException { String json = "{\n" + " \"type\": \"default\",\n" diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java index dc0f2aa89081..8ab89a2c572e 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java @@ -19,11 +19,13 @@ package org.apache.druid.k8s.overlord.execution; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.TestHelper; import org.junit.Assert; import org.junit.Test; @@ -77,6 +79,29 @@ public void shouldReturnFalseWhenTagsDoNotMatch() Assert.assertFalse(selector.evaluate(task)); } + @Test + public void shouldReturnFalseWhenSomeTagsDoNotMatch() + { + String dataSource = "my_table"; + Map> cxtTagsConditions = new HashMap<>(); + cxtTagsConditions.put("nonexistentTag", Sets.newHashSet("nonexistentTagValue")); + cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); + + Map> taskFieldsConditions = new HashMap<>(); + taskFieldsConditions.put("datasource", Sets.newHashSet(dataSource)); + + Task task = NoopTask.forDatasource(dataSource); + task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); + + Selector selector = new Selector( + "TestSelector", + cxtTagsConditions, + taskFieldsConditions + ); + + Assert.assertFalse(selector.evaluate(task)); + } + @Test public void shouldReturnFalseWhenTaskFieldsDoNotMatch() { @@ -98,6 +123,28 @@ public void shouldReturnFalseWhenTaskFieldsDoNotMatch() Assert.assertFalse(selector.evaluate(task)); } + @Test + public void shouldReturnFalseWhenSomeTaskFieldsDoNotMatch() + { + Map> cxtTagsConditions = new HashMap<>(); + cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); + + Map> taskFieldsConditions = new HashMap<>(); + taskFieldsConditions.put("type", Sets.newHashSet(NoopTask.TYPE)); + taskFieldsConditions.put("datasource", Sets.newHashSet("my_table")); + + Task task = NoopTask.forDatasource("another_table"); + task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); + + Selector selector = new Selector( + "TestSelector", + cxtTagsConditions, + taskFieldsConditions + ); + + Assert.assertFalse(selector.evaluate(task)); + } + @Test public void shouldReturnTrueWhenNoConditionsSpecified() { @@ -112,4 +159,27 @@ public void shouldReturnTrueWhenNoConditionsSpecified() Assert.assertTrue(selector.evaluate(task)); } + + @Test + public void testSerde() throws Exception + { + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + Map> cxtTagsConditions = new HashMap<>(); + cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); + + Map> taskFieldsConditions = new HashMap<>(); + taskFieldsConditions.put("type", Sets.newHashSet(NoopTask.TYPE)); + + Selector selector = new Selector( + "TestSelector", + cxtTagsConditions, + taskFieldsConditions + ); + + Selector selector2 = objectMapper.readValue( + objectMapper.writeValueAsBytes(selector), + Selector.class + ); + Assert.assertEquals(selector, selector2); + } } From de59b8ae7464e74fe960200eceeb0e3db068cba2 Mon Sep 17 00:00:00 2001 From: YongGang Date: Mon, 3 Jun 2024 21:25:49 -0700 Subject: [PATCH 07/16] add docs --- .../extensions-contrib/k8s-jobs.md | 62 ++++++++++++++++++- .../DefaultExecutionBehaviorStrategy.java | 12 ++++ .../execution/DefaultExecutionConfigTest.java | 52 ++++++++++++++++ 3 files changed, 125 insertions(+), 1 deletion(-) create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfigTest.java diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index bcc8805b3a52..347e758d1398 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -217,6 +217,66 @@ data: druid.peon.mode=remote druid.indexer.task.encapsulatedTask=true ``` +#### Dynamic Pod Template Selection Config +The Dynamic Pod Template Selection feature enhances the K8s extension by enabling more flexible and dynamic selection of pod templates based on task properties. This process is governed by the `ExecutionBehaviorStrategy`. Below are the two strategies implemented: + +|Property|Description|Default| +|--------|-----------|-------| +|`DefaultExecutionBehaviorStrategy`| This strategy categorizes tasks based on their type for execution purposes, implementing the existing behavior that maps pod templates according to task type. | true | +|`DynamicTaskExecutionBehaviorStrategy`| This strategy dynamically evaluates a series of selectors, with each selector corresponding to a potential task category.| false | + +`DynamicTaskExecutionBehaviorStrategy`, the strategy implementing this new feature, is based on conditional selectors that match against task properties. Conditions are specified in the dynamic config, and the selection process uses these conditions to determine the category of a task based on context tags and task fields. The identified category is then used to map to different Peon Pod templates, allowing tailored resource allocation and management according to the task’s requirements. + +Example Configuration: + +We define two categories in the configuration—`low-throughput` and `medium-throughput`—each associated with specific task conditions. + +- Low Throughput Category: Tasks that have a context tag `billingCategory=streaming_ingestion` and a datasource of `wikipedia` will be classified under the `low-throughput` category. This classification directs such tasks to utilize a predefined pod template optimized for low throughput requirements. + +- Medium Throughput Category: If a task does not meet the low-throughput criteria, the system will then evaluate it against the next selector. In this example, if the task type is index_kafka, it will fall into the `medium-throughput` category. +``` +{ + "type": "default", + "behaviorStrategy": { + "type": "dynamicTask", + "categorySelectors": [ + { + "selectionKey": "low-throughput", + "context.tags": { + "billingCategory": [ + "streaming_ingestion" + ] + }, + "task": { + "datasource": [ + "wikipedia" + ] + } + }, + { + "selectionKey": "medium-throughput", + "task": { + "type": [ + "index_kafka" + ] + } + } + ] + } +} +``` +Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{category}: /path/to/taskSpecificPodSpec.yaml` where {category} is the matched selectionKey of the behaviorStrategy i.e `low-throughput`. + +Similar to Overlord dynamic configuration, the following API endpoints are defined to retrieve and manage dynamic configurations of Pod Template Selection config: + +- Get dynamic configuration: +`POST` `/druid/indexer/v1/k8s/runner/execution` + +- Update dynamic configuration: +`GET` `/druid/indexer/v1/k8s/runner/execution` + +- Get dynamic configuration history: +`GET` `/druid/indexer/v1/k8s/runner/execution/history` ### Properties |Property| Possible Values | Description |Default|required| @@ -242,7 +302,7 @@ data: |Metric|Description|Dimensions|Normal value| |------|-----------|----------|------------| -| `k8s/peon/startup/time` | Metric indicating the milliseconds for peon pod to startup. | `dataSource`, `taskId`, `taskType`, `groupId`, `taskStatus`, `tags` |Varies| +| `k8s/peon/startup/time` | Metric indicating the milliseconds for peon pod to startup. | `dataSource`, `taskId`, `taskType`, `groupId`, `taskStatus`, `tags`, `category` |Varies| ### Gotchas diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionBehaviorStrategy.java index 7ccb0ab04c06..0ab1257bba3a 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionBehaviorStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionBehaviorStrategy.java @@ -41,6 +41,18 @@ public String getTaskCategory(Task task) return task.getType(); } + @Override + public boolean equals(Object o) + { + return o instanceof DefaultExecutionBehaviorStrategy; + } + + @Override + public int hashCode() + { + return 1; // Any constant will work here + } + @Override public String toString() { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfigTest.java new file mode 100644 index 000000000000..16aa5182e48a --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfigTest.java @@ -0,0 +1,52 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class DefaultExecutionConfigTest +{ + + @Test + public void getBehaviorStrategyTest() + { + ExecutionBehaviorStrategy strategy = new DefaultExecutionBehaviorStrategy(); + DefaultExecutionConfig config = new DefaultExecutionConfig(strategy); + + Assert.assertEquals(strategy, config.getBehaviorStrategy()); + } + + @Test + public void testSerde() throws Exception + { + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + ExecutionBehaviorStrategy strategy = new DefaultExecutionBehaviorStrategy(); + DefaultExecutionConfig config = new DefaultExecutionConfig(strategy); + + DefaultExecutionConfig config2 = objectMapper.readValue( + objectMapper.writeValueAsBytes(config), + DefaultExecutionConfig.class + ); + Assert.assertEquals(config, config2); + } +} From b4b3c31243a5029b8db123cf18d624202cd3595e Mon Sep 17 00:00:00 2001 From: YongGang Date: Tue, 4 Jun 2024 09:41:29 -0700 Subject: [PATCH 08/16] update doc --- docs/development/extensions-contrib/k8s-jobs.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index 347e758d1398..ad5e14cefce7 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -225,15 +225,15 @@ The Dynamic Pod Template Selection feature enhances the K8s extension by enablin |`DefaultExecutionBehaviorStrategy`| This strategy categorizes tasks based on their type for execution purposes, implementing the existing behavior that maps pod templates according to task type. | true | |`DynamicTaskExecutionBehaviorStrategy`| This strategy dynamically evaluates a series of selectors, with each selector corresponding to a potential task category.| false | -`DynamicTaskExecutionBehaviorStrategy`, the strategy implementing this new feature, is based on conditional selectors that match against task properties. Conditions are specified in the dynamic config, and the selection process uses these conditions to determine the category of a task based on context tags and task fields. The identified category is then used to map to different Peon Pod templates, allowing tailored resource allocation and management according to the task’s requirements. +`DynamicTaskExecutionBehaviorStrategy`, the strategy implementing this new feature, is based on conditional selectors `categorySelectors` that match against task properties. These selectors are ordered in the dynamic configuration, with the first selector given the highest priority during the evaluation process. This means that the selection process uses these ordered conditions to determine a task’s category based on context tags and task fields. The first matching condition immediately determines the category, thereby prioritizing certain configurations over others. Once a category is identified, it is used to map to different Peon Pod templates, enabling tailored resource allocation and management that aligns with the specific requirements of each task. Example Configuration: -We define two categories in the configuration—`low-throughput` and `medium-throughput`—each associated with specific task conditions. +We define two categories in the configuration—`low-throughput` and `medium-throughput`—each associated with specific task conditions and arranged in a priority order. -- Low Throughput Category: Tasks that have a context tag `billingCategory=streaming_ingestion` and a datasource of `wikipedia` will be classified under the `low-throughput` category. This classification directs such tasks to utilize a predefined pod template optimized for low throughput requirements. +- Low Throughput Category: This is the first category evaluated and has the highest priority. Tasks that have a context tag `billingCategory=streaming_ingestion` and a datasource of `wikipedia` will be classified under the `low-throughput` category. This classification directs such tasks to utilize a predefined pod template optimized for low throughput requirements. -- Medium Throughput Category: If a task does not meet the low-throughput criteria, the system will then evaluate it against the next selector. In this example, if the task type is index_kafka, it will fall into the `medium-throughput` category. +- Medium Throughput Category: If a task does not meet the low-throughput criteria, the system will then evaluate it against the next selector in order. In this example, if the task type is index_kafka, it will fall into the `medium-throughput` category. ``` { "type": "default", From 07c0209dc8c3a13fae317adb3862514e4a506520 Mon Sep 17 00:00:00 2001 From: YongGang Date: Tue, 4 Jun 2024 09:51:27 -0700 Subject: [PATCH 09/16] fix doc --- docs/development/extensions-contrib/k8s-jobs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index ad5e14cefce7..08f0bfdfaf34 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -265,7 +265,7 @@ We define two categories in the configuration—`low-throughput` and `medium-thr } } ``` -Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{category}: /path/to/taskSpecificPodSpec.yaml` where {category} is the matched selectionKey of the behaviorStrategy i.e `low-throughput`. +Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{category}: /path/to/taskSpecificPodSpec.yaml` where {category} is the matched `selectionKey` of the `behaviorStrategy` i.e low-throughput. Similar to Overlord dynamic configuration, the following API endpoints are defined to retrieve and manage dynamic configurations of Pod Template Selection config: From 903d815b43cb5c9241d77c2e6146440990e36694 Mon Sep 17 00:00:00 2001 From: YongGang Date: Thu, 6 Jun 2024 11:56:37 -0700 Subject: [PATCH 10/16] address comments --- .../extensions-contrib/k8s-jobs.md | 28 ++--- .../overlord/KubernetesOverlordModule.java | 8 +- .../overlord/KubernetesTaskRunnerFactory.java | 13 +-- .../overlord/common/DruidK8sConstants.java | 1 + .../overlord/common/KubernetesPeonClient.java | 19 +--- ...ultKubernetesTaskRunnerDynamicConfig.java} | 26 ++--- .../DynamicTaskExecutionBehaviorStrategy.java | 97 ----------------- .../DynamicTaskPodTemplateSelectStrategy.java | 103 ++++++++++++++++++ .../execution/ExecutionBehaviorStrategy.java | 48 -------- ...ubernetesTaskExecutionConfigResource.java} | 40 +++---- ...=> KubernetesTaskRunnerDynamicConfig.java} | 10 +- .../execution/PodTemplateSelectStrategy.java | 63 +++++++++++ ...=> TaskTypePodTemplateSelectStrategy.java} | 23 ++-- .../taskadapter/PodTemplateTaskAdapter.java | 47 ++++---- .../KubernetesTaskRunnerFactoryTest.java | 20 ++-- .../common/KubernetesPeonClientTest.java | 14 +-- ...ubernetesTaskRunnerDynamicConfigTest.java} | 18 +-- ...micTaskPodTemplateSelectStrategyTest.java} | 59 +++++++--- ...ubernetesTaskRunnerDynamicConfigTest.java} | 20 ++-- .../DruidPeonClientIntegrationTest.java | 10 +- .../PodTemplateTaskAdapterTest.java | 60 +++++----- .../src/test/resources/expectedNoopJob.yaml | 1 + .../expectedNoopJobBaseTemplate.yaml | 53 +++++++++ .../resources/expectedNoopJobLongIds.yaml | 1 + .../resources/expectedNoopJobNoTaskJson.yaml | 1 + .../resources/expectedNoopJobTlsEnabled.yaml | 1 + 26 files changed, 435 insertions(+), 349 deletions(-) rename extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/{DefaultExecutionConfig.java => DefaultKubernetesTaskRunnerDynamicConfig.java} (60%) delete mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategy.java delete mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java rename extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/{KubernetesResource.java => KubernetesTaskExecutionConfigResource.java} (80%) rename extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/{ExecutionConfig.java => KubernetesTaskRunnerDynamicConfig.java} (80%) create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java rename extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/{DefaultExecutionBehaviorStrategy.java => TaskTypePodTemplateSelectStrategy.java} (61%) rename extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/{DefaultExecutionConfigTest.java => DefaultKubernetesTaskRunnerDynamicConfigTest.java} (64%) rename extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/{DynamicTaskExecutionBehaviorStrategyTest.java => DynamicTaskPodTemplateSelectStrategyTest.java} (59%) rename extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/{ExecutionConfigTest.java => KubernetesTaskRunnerDynamicConfigTest.java} (75%) create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBaseTemplate.yaml diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index 08f0bfdfaf34..b1f346978b5d 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -218,28 +218,28 @@ data: druid.indexer.task.encapsulatedTask=true ``` #### Dynamic Pod Template Selection Config -The Dynamic Pod Template Selection feature enhances the K8s extension by enabling more flexible and dynamic selection of pod templates based on task properties. This process is governed by the `ExecutionBehaviorStrategy`. Below are the two strategies implemented: +The Dynamic Pod Template Selection feature enhances the K8s extension by enabling more flexible and dynamic selection of pod templates based on task properties. This process is governed by the `PodTemplateSelectStrategy`. Below are the two strategies implemented: |Property|Description|Default| |--------|-----------|-------| -|`DefaultExecutionBehaviorStrategy`| This strategy categorizes tasks based on their type for execution purposes, implementing the existing behavior that maps pod templates according to task type. | true | -|`DynamicTaskExecutionBehaviorStrategy`| This strategy dynamically evaluates a series of selectors, with each selector corresponding to a potential task category.| false | +|`TaskTypePodTemplateSelectStrategy`| This strategy selects pod templates based on task type for execution purposes, implementing the behavior that maps templates to specific task types. | true | +|`DynamicTaskPodTemplateSelectStrategy`| This strategy dynamically evaluates a series of selectors, known as templateSelectors, which are aligned with potential task properties. | false | -`DynamicTaskExecutionBehaviorStrategy`, the strategy implementing this new feature, is based on conditional selectors `categorySelectors` that match against task properties. These selectors are ordered in the dynamic configuration, with the first selector given the highest priority during the evaluation process. This means that the selection process uses these ordered conditions to determine a task’s category based on context tags and task fields. The first matching condition immediately determines the category, thereby prioritizing certain configurations over others. Once a category is identified, it is used to map to different Peon Pod templates, enabling tailored resource allocation and management that aligns with the specific requirements of each task. +`DynamicTaskPodTemplateSelectStrategy`, the strategy implementing this new feature, is based on conditional selectors `templateSelectors` that match against task properties. These selectors are ordered in the dynamic configuration, with the first selector given the highest priority during the evaluation process. This means that the selection process uses these ordered conditions to determine a task’s Pod template based on context tags and task fields. The first matching condition immediately determines the Pod template, thereby prioritizing certain configurations over others. Example Configuration: -We define two categories in the configuration—`low-throughput` and `medium-throughput`—each associated with specific task conditions and arranged in a priority order. +We define two template keys in the configuration—`low-throughput` and `medium-throughput`—each associated with specific task conditions and arranged in a priority order. -- Low Throughput Category: This is the first category evaluated and has the highest priority. Tasks that have a context tag `billingCategory=streaming_ingestion` and a datasource of `wikipedia` will be classified under the `low-throughput` category. This classification directs such tasks to utilize a predefined pod template optimized for low throughput requirements. +- Low Throughput Template: This is the first template evaluated and has the highest priority. Tasks that have a context tag `billingCategory=streaming_ingestion` and a datasource of `wikipedia` will be classified under the `low-throughput` template. This classification directs such tasks to utilize a predefined pod template optimized for low throughput requirements. -- Medium Throughput Category: If a task does not meet the low-throughput criteria, the system will then evaluate it against the next selector in order. In this example, if the task type is index_kafka, it will fall into the `medium-throughput` category. +- Medium Throughput Template: If a task does not meet the low-throughput criteria, the system will then evaluate it against the next selector in order. In this example, if the task type is index_kafka, it will fall into the `medium-throughput` template. ``` { "type": "default", - "behaviorStrategy": { + "podTemplateSelectStrategy": { "type": "dynamicTask", - "categorySelectors": [ + "templateSelectors": [ { "selectionKey": "low-throughput", "context.tags": { @@ -265,18 +265,18 @@ We define two categories in the configuration—`low-throughput` and `medium-thr } } ``` -Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{category}: /path/to/taskSpecificPodSpec.yaml` where {category} is the matched `selectionKey` of the `behaviorStrategy` i.e low-throughput. +Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{template}: /path/to/taskSpecificPodSpec.yaml` where {template} is the matched `selectionKey` of the `podTemplateSelectStrategy` i.e low-throughput. Similar to Overlord dynamic configuration, the following API endpoints are defined to retrieve and manage dynamic configurations of Pod Template Selection config: - Get dynamic configuration: -`POST` `/druid/indexer/v1/k8s/runner/execution` +`POST` `/druid/indexer/v1/k8s/taskRunner/executionConfig` - Update dynamic configuration: -`GET` `/druid/indexer/v1/k8s/runner/execution` +`GET` `/druid/indexer/v1/k8s/taskRunner/executionConfig` - Get dynamic configuration history: -`GET` `/druid/indexer/v1/k8s/runner/execution/history` +`GET` `/druid/indexer/v1/k8s/taskRunner/executionConfig/history` ### Properties |Property| Possible Values | Description |Default|required| @@ -302,7 +302,7 @@ Similar to Overlord dynamic configuration, the following API endpoints are defin |Metric|Description|Dimensions|Normal value| |------|-----------|----------|------------| -| `k8s/peon/startup/time` | Metric indicating the milliseconds for peon pod to startup. | `dataSource`, `taskId`, `taskType`, `groupId`, `taskStatus`, `tags`, `category` |Varies| +| `k8s/peon/startup/time` | Metric indicating the milliseconds for peon pod to startup. | `dataSource`, `taskId`, `taskType`, `groupId`, `taskStatus`, `tags`, `podTemplate` |Varies| ### Gotchas diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java index de09de268e8a..eacbabbd19e0 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesOverlordModule.java @@ -51,8 +51,8 @@ import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; -import org.apache.druid.k8s.overlord.execution.ExecutionConfig; -import org.apache.druid.k8s.overlord.execution.KubernetesResource; +import org.apache.druid.k8s.overlord.execution.KubernetesTaskExecutionConfigResource; +import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; import org.apache.druid.k8s.overlord.runnerstrategy.RunnerStrategy; import org.apache.druid.tasklogs.NoopTaskLogs; import org.apache.druid.tasklogs.TaskLogKiller; @@ -79,7 +79,7 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX, KubernetesTaskRunnerConfig.class); JsonConfigProvider.bind(binder, K8SANDWORKER_PROPERTIES_PREFIX, KubernetesAndWorkerTaskRunnerConfig.class); JsonConfigProvider.bind(binder, "druid.indexer.queue", TaskQueueConfig.class); - JacksonConfigProvider.bind(binder, ExecutionConfig.CONFIG_KEY, ExecutionConfig.class, null); + JacksonConfigProvider.bind(binder, KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, KubernetesTaskRunnerDynamicConfig.class, null); PolyBind.createChoice( binder, "druid.indexer.runner.type", @@ -104,7 +104,7 @@ public void configure(Binder binder) .in(LazySingleton.class); configureTaskLogs(binder); - Jerseys.addResource(binder, KubernetesResource.class); + Jerseys.addResource(binder, KubernetesTaskExecutionConfigResource.class); } @Provides diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java index 0189651d51ac..2807053f4586 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactory.java @@ -33,7 +33,7 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; -import org.apache.druid.k8s.overlord.execution.ExecutionConfig; +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; @@ -58,7 +58,7 @@ public class KubernetesTaskRunnerFactory implements TaskRunnerFactory executionConfigRef; + private final Supplier dynamicConfigRef; private KubernetesTaskRunner runner; @Inject @@ -73,7 +73,7 @@ public KubernetesTaskRunnerFactory( Properties properties, DruidKubernetesClient druidKubernetesClient, ServiceEmitter emitter, - Supplier executionConfigRef + Supplier dynamicConfigRef ) { this.smileMapper = smileMapper; @@ -86,7 +86,7 @@ public KubernetesTaskRunnerFactory( this.properties = properties; this.druidKubernetesClient = druidKubernetesClient; this.emitter = emitter; - this.executionConfigRef = executionConfigRef; + this.dynamicConfigRef = dynamicConfigRef; } @Override @@ -97,8 +97,7 @@ public KubernetesTaskRunner build() druidKubernetesClient, kubernetesTaskRunnerConfig.getNamespace(), kubernetesTaskRunnerConfig.isDebugJobs(), - emitter, - executionConfigRef + emitter ); runner = new KubernetesTaskRunner( @@ -153,7 +152,7 @@ private TaskAdapter buildTaskAdapter(DruidKubernetesClient client) smileMapper, properties, taskLogs, - executionConfigRef + dynamicConfigRef ); } else { return new SingleContainerTaskAdapter( diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java index 6c195ed15151..ca32106334ed 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java @@ -43,5 +43,6 @@ public class DruidK8sConstants public static final String LABEL_KEY = "druid.k8s.peons"; public static final String DRUID_LABEL_PREFIX = "druid."; public static final long MAX_ENV_VARIABLE_KBS = 130048; // 127 KB + public static final String POD_TEMPLATE_KEY = "template.key"; static final Predicate IS_TRANSIENT = e -> e instanceof KubernetesResourceNotFoundException; } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index 802640e2db5b..b7f15f1f9505 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -21,7 +21,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; -import com.google.common.base.Supplier; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.batch.v1.Job; import io.fabric8.kubernetes.client.KubernetesClient; @@ -32,7 +31,6 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import java.io.InputStream; import java.sql.Timestamp; @@ -49,21 +47,18 @@ public class KubernetesPeonClient private final String namespace; private final boolean debugJobs; private final ServiceEmitter emitter; - private final Supplier executionConfigRef; public KubernetesPeonClient( KubernetesClientApi clientApi, String namespace, boolean debugJobs, - ServiceEmitter emitter, - Supplier executionConfigRef + ServiceEmitter emitter ) { this.clientApi = clientApi; this.namespace = namespace; this.debugJobs = debugJobs; this.emitter = emitter; - this.executionConfigRef = executionConfigRef; } public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUnit timeUnit) throws IllegalStateException @@ -88,7 +83,7 @@ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUn throw new IllegalStateException("K8s pod for the task [%s] appeared and disappeared. It can happen if the task was canceled"); } long duration = System.currentTimeMillis() - start; - emitK8sPodMetrics(task, "k8s/peon/startup/time", duration); + emitK8sPodMetrics(task, job, "k8s/peon/startup/time", duration); return result; }); } @@ -275,18 +270,16 @@ Pod getPeonPodWithRetries(KubernetesClient client, String jobName, int quietTrie } } - private void emitK8sPodMetrics(Task task, String metric, long durationMs) + private void emitK8sPodMetrics(Task task, Job job, String metric, long durationMs) { ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); IndexTaskUtils.setTaskDimensions(metricBuilder, task); - ExecutionConfig executionConfig = executionConfigRef.get(); - if (executionConfig != null && executionConfig.getBehaviorStrategy() != null) { + if (job.getMetadata() != null && job.getMetadata().getAnnotations() != null) { metricBuilder.setDimensionIfNotNull( - "category", - executionConfig.getBehaviorStrategy().getTaskCategory(task) + "podTemplate", + job.getMetadata().getAnnotations().get(DruidK8sConstants.POD_TEMPLATE_KEY) ); } - emitter.emit(metricBuilder.setMetric(metric, durationMs)); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfig.java similarity index 60% rename from extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java rename to extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfig.java index 190b278cb59d..eddd5e4a1ee1 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfig.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfig.java @@ -25,24 +25,24 @@ import java.util.Objects; -public class DefaultExecutionConfig implements ExecutionConfig +public class DefaultKubernetesTaskRunnerDynamicConfig implements KubernetesTaskRunnerDynamicConfig { - private final ExecutionBehaviorStrategy behaviorStrategy; + private final PodTemplateSelectStrategy podTemplateSelectStrategy; @JsonCreator - public DefaultExecutionConfig( - @JsonProperty("behaviorStrategy") ExecutionBehaviorStrategy behaviorStrategy + public DefaultKubernetesTaskRunnerDynamicConfig( + @JsonProperty("podTemplateSelectStrategy") PodTemplateSelectStrategy podTemplateSelectStrategy ) { - Preconditions.checkNotNull(behaviorStrategy); - this.behaviorStrategy = behaviorStrategy; + Preconditions.checkNotNull(podTemplateSelectStrategy); + this.podTemplateSelectStrategy = podTemplateSelectStrategy; } @Override @JsonProperty - public ExecutionBehaviorStrategy getBehaviorStrategy() + public PodTemplateSelectStrategy getPodTemplateSelectStrategy() { - return behaviorStrategy; + return podTemplateSelectStrategy; } @Override @@ -54,21 +54,21 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - DefaultExecutionConfig that = (DefaultExecutionConfig) o; - return Objects.equals(behaviorStrategy, that.behaviorStrategy); + DefaultKubernetesTaskRunnerDynamicConfig that = (DefaultKubernetesTaskRunnerDynamicConfig) o; + return Objects.equals(podTemplateSelectStrategy, that.podTemplateSelectStrategy); } @Override public int hashCode() { - return Objects.hashCode(behaviorStrategy); + return Objects.hashCode(podTemplateSelectStrategy); } @Override public String toString() { - return "DefaultExecutionConfig{" + - "behaviorStrategy=" + behaviorStrategy + + return "DefaultKubernetesTaskRunnerDynamicConfig{" + + "podTemplateSelectStrategy=" + podTemplateSelectStrategy + '}'; } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java deleted file mode 100644 index 1f3799a02bb2..000000000000 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategy.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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.execution; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.common.task.Task; - -import java.util.List; -import java.util.Objects; - -/** - * Implements {@link ExecutionBehaviorStrategy} by dynamically evaluating a series of selectors. - * Each selector corresponds to a potential task category. - */ -public class DynamicTaskExecutionBehaviorStrategy implements ExecutionBehaviorStrategy -{ - private List categorySelectors; - - @JsonCreator - public DynamicTaskExecutionBehaviorStrategy( - @JsonProperty("categorySelectors") List categorySelectors - ) - { - this.categorySelectors = categorySelectors; - } - - /** - * Evaluates the provided task against the set selectors to determine its category. - * - * @param task the task to be categorized - * @return the category if a selector matches, otherwise null - */ - @Override - public String getTaskCategory(Task task) - { - if (categorySelectors == null) { - return null; - } - - return categorySelectors.stream() - .filter(selector -> selector.evaluate(task)) - .findFirst() - .map(Selector::getSelectionKey) - .orElse(null); - } - - @JsonProperty - public List getCategorySelectors() - { - return categorySelectors; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DynamicTaskExecutionBehaviorStrategy that = (DynamicTaskExecutionBehaviorStrategy) o; - return Objects.equals(categorySelectors, that.categorySelectors); - } - - @Override - public int hashCode() - { - return Objects.hashCode(categorySelectors); - } - - @Override - public String toString() - { - return "DynamicTaskExecutionBehaviorStrategy{" + - "categorySelectors=" + categorySelectors + - '}'; - } -} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategy.java new file mode 100644 index 000000000000..f6e1634ee521 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategy.java @@ -0,0 +1,103 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.fabric8.kubernetes.api.model.PodTemplate; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.Pair; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Implements {@link PodTemplateSelectStrategy} by dynamically evaluating a series of selectors. + * Each selector corresponds to a potential task template key. + */ +public class DynamicTaskPodTemplateSelectStrategy implements PodTemplateSelectStrategy +{ + private List templateSelectors; + + @JsonCreator + public DynamicTaskPodTemplateSelectStrategy( + @JsonProperty("templateSelectors") List templateSelectors + ) + { + this.templateSelectors = templateSelectors; + } + + /** + * Evaluates the provided task against the set selectors to determine its template. + * + * @param task the task to be checked + * @return the template if a selector matches, otherwise fallback to base template + */ + @Override + public Pair getPodTemplateForTask(Task task, Map templates) + { + String templateKey; + if (templateSelectors == null) { + templateKey = null; + } else { + templateKey = templateSelectors.stream() + .filter(selector -> selector.evaluate(task)) + .findFirst() + .map(Selector::getSelectionKey) + .orElse(null); + } + + return getTemplateOrDefault(templateKey, templates); + } + + @JsonProperty + public List getTemplateSelectors() + { + return templateSelectors; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DynamicTaskPodTemplateSelectStrategy that = (DynamicTaskPodTemplateSelectStrategy) o; + return Objects.equals(templateSelectors, that.templateSelectors); + } + + @Override + public int hashCode() + { + return Objects.hashCode(templateSelectors); + } + + @Override + public String toString() + { + return "DynamicTaskPodTemplateSelectStrategy{" + + "templateSelectors=" + templateSelectors + + '}'; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java deleted file mode 100644 index b4980c0a9ea3..000000000000 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionBehaviorStrategy.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.execution; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.indexing.common.task.Task; - -import javax.annotation.Nullable; - -/** - * Defines a strategy for determining the execution behavior of tasks based on specific conditions. - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultExecutionBehaviorStrategy.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "default", value = DefaultExecutionBehaviorStrategy.class), - @JsonSubTypes.Type(name = "dynamicTask", value = DynamicTaskExecutionBehaviorStrategy.class), -}) -public interface ExecutionBehaviorStrategy -{ - /** - * Determines the category of a task based on its properties. The identified category is used - * to map different Peon Pod templates, which allows for tailored resource allocation and management - * according to the task's requirements. - * - * @param task the task to evaluate - * @return the category of the task, or null if no category matches - */ - @Nullable - String getTaskCategory(Task task); -} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java similarity index 80% rename from extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java rename to extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java index 63106e1ad6bc..228c07450eb5 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesResource.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java @@ -51,16 +51,16 @@ *

This class handles the CRUD operations for execution configurations and provides * endpoints to update, retrieve, and manage the history of these configurations.

*/ -@Path("/druid/indexer/v1/k8s/runner") -public class KubernetesResource +@Path("/druid/indexer/v1/k8s/taskRunner") +public class KubernetesTaskExecutionConfigResource { - private static final Logger log = new Logger(KubernetesResource.class); + private static final Logger log = new Logger(KubernetesTaskExecutionConfigResource.class); private final JacksonConfigManager configManager; private final AuditManager auditManager; - private AtomicReference executionConfigRef = null; + private AtomicReference dynamicConfigRef = null; @Inject - public KubernetesResource( + public KubernetesTaskExecutionConfigResource( final JacksonConfigManager configManager, final AuditManager auditManager ) @@ -72,26 +72,26 @@ public KubernetesResource( /** * Updates the Kubernetes execution configuration. * - * @param executionConfig the new execution configuration to set + * @param dynamicConfig the new execution configuration to set * @param req the HTTP servlet request providing context for audit information * @return a response indicating the success or failure of the update operation */ @POST - @Path("/execution") + @Path("/executionConfig") @Consumes(MediaType.APPLICATION_JSON) @ResourceFilters(ConfigResourceFilter.class) public Response setExecutionConfig( - final ExecutionConfig executionConfig, + final KubernetesTaskRunnerDynamicConfig dynamicConfig, @Context final HttpServletRequest req ) { final ConfigManager.SetResult setResult = configManager.set( - ExecutionConfig.CONFIG_KEY, - executionConfig, + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, + dynamicConfig, AuthorizationUtils.buildAuditInfo(req) ); if (setResult.isOk()) { - log.info("Updating K8s execution configs: %s", executionConfig); + log.info("Updating K8s execution configs: %s", dynamicConfig); return Response.ok().build(); } else { @@ -107,7 +107,7 @@ public Response setExecutionConfig( * @return a response containing a list of audit entries or an error message */ @GET - @Path("/execution/history") + @Path("/executionConfig/history") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(ConfigResourceFilter.class) public Response getExecutionConfigHistory( @@ -119,8 +119,8 @@ public Response getExecutionConfigHistory( if (theInterval == null && count != null) { try { List executionEntryList = auditManager.fetchAuditHistory( - ExecutionConfig.CONFIG_KEY, - ExecutionConfig.CONFIG_KEY, + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, count ); return Response.ok(executionEntryList).build(); @@ -132,8 +132,8 @@ public Response getExecutionConfigHistory( } } List executionEntryList = auditManager.fetchAuditHistory( - ExecutionConfig.CONFIG_KEY, - ExecutionConfig.CONFIG_KEY, + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, + KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, theInterval ); return Response.ok(executionEntryList).build(); @@ -145,15 +145,15 @@ public Response getExecutionConfigHistory( * @return a Response object containing the current execution configuration in JSON format. */ @GET - @Path("/execution") + @Path("/executionConfig") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(ConfigResourceFilter.class) public Response getExecutionConfig() { - if (executionConfigRef == null) { - executionConfigRef = configManager.watch(ExecutionConfig.CONFIG_KEY, ExecutionConfig.class); + if (dynamicConfigRef == null) { + dynamicConfigRef = configManager.watch(KubernetesTaskRunnerDynamicConfig.CONFIG_KEY, KubernetesTaskRunnerDynamicConfig.class); } - return Response.ok(executionConfigRef.get()).build(); + return Response.ok(dynamicConfigRef.get()).build(); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfig.java similarity index 80% rename from extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java rename to extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfig.java index 1dc3e01e2fc3..4f6d4b07c41d 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/ExecutionConfig.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfig.java @@ -27,18 +27,18 @@ * This interface allows for dynamic configuration of task execution strategies based * on specified behavior strategies. */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultExecutionConfig.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultKubernetesTaskRunnerDynamicConfig.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "default", value = DefaultExecutionConfig.class) + @JsonSubTypes.Type(name = "default", value = DefaultKubernetesTaskRunnerDynamicConfig.class) }) -public interface ExecutionConfig +public interface KubernetesTaskRunnerDynamicConfig { String CONFIG_KEY = "k8s.taskrunner.config"; - ExecutionBehaviorStrategy DEFAULT_STRATEGY = new DefaultExecutionBehaviorStrategy(); + PodTemplateSelectStrategy DEFAULT_STRATEGY = new TaskTypePodTemplateSelectStrategy(); /** * Retrieves the execution behavior strategy associated with this configuration. * @return the execution behavior strategy */ - ExecutionBehaviorStrategy getBehaviorStrategy(); + PodTemplateSelectStrategy getPodTemplateSelectStrategy(); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java new file mode 100644 index 000000000000..16c7171430b0 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java @@ -0,0 +1,63 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.fabric8.kubernetes.api.model.PodTemplate; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.Pair; + +import java.util.Map; + +/** + * Defines a strategy for selecting the Pod template of tasks based on specific conditions. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = TaskTypePodTemplateSelectStrategy.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "default", value = TaskTypePodTemplateSelectStrategy.class), + @JsonSubTypes.Type(name = "dynamicTask", value = DynamicTaskPodTemplateSelectStrategy.class), +}) +public interface PodTemplateSelectStrategy +{ + /** + * Determines the appropriate Pod template for a task by evaluating its properties. This selection + * allows for customized resource allocation and management tailored to the task's specific requirements. + * + * @param task The task for which the Pod template is determined. + * @return A key-value pair representing the selected Pod template. If no matching template is found, + * the method falls back to a base template. + */ + Pair getPodTemplateForTask(Task task, Map templates); + + default Pair getTemplateOrDefault(String templateKey, Map templates) + { + if (templates == null) { + return null; + } + + PodTemplate podTemplate = templates.get(templateKey); + if (podTemplate != null) { + return Pair.of(templateKey, podTemplate); + } else { + return Pair.of("base", templates.get("base")); + } + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionBehaviorStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskTypePodTemplateSelectStrategy.java similarity index 61% rename from extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionBehaviorStrategy.java rename to extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskTypePodTemplateSelectStrategy.java index 0ab1257bba3a..8b4f7f015349 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionBehaviorStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskTypePodTemplateSelectStrategy.java @@ -20,31 +20,36 @@ package org.apache.druid.k8s.overlord.execution; import com.fasterxml.jackson.annotation.JsonCreator; +import io.fabric8.kubernetes.api.model.PodTemplate; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.java.util.common.Pair; + +import java.util.Map; /** - * This strategy defines how tasks are categorized based on their type for execution purposes. + * This strategy defines how task template is selected based on their type for execution purposes. * - * This implementation categorizes tasks by simply returning the type of the task, - * making it a straightforward, type-based categorization strategy. + * This implementation selects pod template by looking at the type of the task, + * making it a straightforward, type-based template selection strategy. */ -public class DefaultExecutionBehaviorStrategy implements ExecutionBehaviorStrategy +public class TaskTypePodTemplateSelectStrategy implements PodTemplateSelectStrategy { + @JsonCreator - public DefaultExecutionBehaviorStrategy() + public TaskTypePodTemplateSelectStrategy() { } @Override - public String getTaskCategory(Task task) + public Pair getPodTemplateForTask(Task task, Map templates) { - return task.getType(); + return getTemplateOrDefault(task.getType(), templates); } @Override public boolean equals(Object o) { - return o instanceof DefaultExecutionBehaviorStrategy; + return o instanceof TaskTypePodTemplateSelectStrategy; } @Override @@ -56,7 +61,7 @@ public int hashCode() @Override public String toString() { - return "DefaultExecutionBehaviorStrategy{" + + return "TaskTypePodTemplateSelectStrategy{" + '}'; } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index 4393cf3b719c..f700fc8ab7d5 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -40,6 +40,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.IAE; 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.logger.Logger; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; @@ -47,8 +48,8 @@ import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.KubernetesOverlordUtils; -import org.apache.druid.k8s.overlord.execution.ExecutionBehaviorStrategy; -import org.apache.druid.k8s.overlord.execution.ExecutionConfig; +import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; +import org.apache.druid.k8s.overlord.execution.PodTemplateSelectStrategy; import org.apache.druid.server.DruidNode; import org.apache.druid.tasklogs.TaskLogs; @@ -86,7 +87,6 @@ public class PodTemplateTaskAdapter implements TaskAdapter private static final Logger log = new Logger(PodTemplateTaskAdapter.class); - private static final String TASK_PROPERTY = IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX + ".k8s.podTemplate."; private final KubernetesTaskRunnerConfig taskRunnerConfig; @@ -95,7 +95,7 @@ public class PodTemplateTaskAdapter implements TaskAdapter private final ObjectMapper mapper; private final HashMap templates; private final TaskLogs taskLogs; - private final Supplier executionConfigRef; + private final Supplier dynamicConfigRef; public PodTemplateTaskAdapter( KubernetesTaskRunnerConfig taskRunnerConfig, @@ -104,7 +104,7 @@ public PodTemplateTaskAdapter( ObjectMapper mapper, Properties properties, TaskLogs taskLogs, - Supplier executionConfigRef + Supplier dynamicConfigRef ) { this.taskRunnerConfig = taskRunnerConfig; @@ -113,7 +113,7 @@ public PodTemplateTaskAdapter( this.mapper = mapper; this.templates = initializePodTemplates(properties); this.taskLogs = taskLogs; - this.executionConfigRef = executionConfigRef; + this.dynamicConfigRef = dynamicConfigRef; } /** @@ -132,17 +132,17 @@ public PodTemplateTaskAdapter( @Override public Job fromTask(Task task) throws IOException { - ExecutionBehaviorStrategy behaviorStrategy; - ExecutionConfig executionConfig = executionConfigRef.get(); - if (executionConfig == null || executionConfig.getBehaviorStrategy() == null) { - behaviorStrategy = ExecutionConfig.DEFAULT_STRATEGY; + PodTemplateSelectStrategy podTemplateSelectStrategy; + KubernetesTaskRunnerDynamicConfig dynamicConfig = dynamicConfigRef.get(); + if (dynamicConfig == null || dynamicConfig.getPodTemplateSelectStrategy() == null) { + podTemplateSelectStrategy = KubernetesTaskRunnerDynamicConfig.DEFAULT_STRATEGY; } else { - behaviorStrategy = executionConfig.getBehaviorStrategy(); + podTemplateSelectStrategy = dynamicConfig.getPodTemplateSelectStrategy(); } - String category = behaviorStrategy.getTaskCategory(task); - PodTemplate podTemplate = templates.getOrDefault(category, templates.get("base")); - if (podTemplate == null) { + Pair podTemplatePair = podTemplateSelectStrategy.getPodTemplateForTask(task, templates); + + if (podTemplatePair == null || podTemplatePair.rhs == null) { throw new ISE("Pod template spec not found for task type [%s]", task.getType()); } @@ -150,10 +150,10 @@ public Job fromTask(Task task) throws IOException .withNewMetadata() .withName(new K8sTaskId(task).getK8sJobName()) .addToLabels(getJobLabels(taskRunnerConfig, task)) - .addToAnnotations(getJobAnnotations(taskRunnerConfig, task)) + .addToAnnotations(getJobAnnotations(taskRunnerConfig, task, podTemplatePair.lhs)) .endMetadata() .withNewSpec() - .withTemplate(podTemplate.getTemplate()) + .withTemplate(podTemplatePair.rhs.getTemplate()) .editTemplate() .editOrNewMetadata() .addToAnnotations(getPodTemplateAnnotations(task)) @@ -334,15 +334,16 @@ private Map getJobLabels(KubernetesTaskRunnerConfig config, Task .build(); } - private Map getJobAnnotations(KubernetesTaskRunnerConfig config, Task task) + private Map getJobAnnotations(KubernetesTaskRunnerConfig config, Task task, String templateName) { return ImmutableMap.builder() - .putAll(config.getAnnotations()) - .put(DruidK8sConstants.TASK_ID, task.getId()) - .put(DruidK8sConstants.TASK_TYPE, task.getType()) - .put(DruidK8sConstants.TASK_GROUP_ID, task.getGroupId()) - .put(DruidK8sConstants.TASK_DATASOURCE, task.getDataSource()) - .build(); + .putAll(config.getAnnotations()) + .put(DruidK8sConstants.TASK_ID, task.getId()) + .put(DruidK8sConstants.TASK_TYPE, task.getType()) + .put(DruidK8sConstants.TASK_GROUP_ID, task.getGroupId()) + .put(DruidK8sConstants.TASK_DATASOURCE, task.getDataSource()) + .put(DruidK8sConstants.POD_TEMPLATE_KEY, templateName) + .build(); } private String getDruidLabel(String baseLabel) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java index 08d377b6d105..ec347a5714d3 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/KubernetesTaskRunnerFactoryTest.java @@ -27,7 +27,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.k8s.overlord.common.DruidKubernetesClient; -import org.apache.druid.k8s.overlord.execution.ExecutionConfig; +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; @@ -55,7 +55,7 @@ public class KubernetesTaskRunnerFactoryTest private DruidKubernetesClient druidKubernetesClient; @Mock private ServiceEmitter emitter; - @Mock private Supplier executionConfigRef; + @Mock private Supplier dynamicConfigRef; @Before public void setup() @@ -94,7 +94,7 @@ public void test_get_returnsSameKuberentesTaskRunner_asBuild() properties, druidKubernetesClient, emitter, - executionConfigRef + dynamicConfigRef ); KubernetesTaskRunner expectedRunner = factory.build(); @@ -117,7 +117,7 @@ public void test_build_withoutSidecarSupport_returnsKubernetesTaskRunnerWithSing properties, druidKubernetesClient, emitter, - executionConfigRef + dynamicConfigRef ); KubernetesTaskRunner runner = factory.build(); @@ -145,7 +145,7 @@ public void test_build_withSidecarSupport_returnsKubernetesTaskRunnerWithMultiCo properties, druidKubernetesClient, emitter, - executionConfigRef + dynamicConfigRef ); KubernetesTaskRunner runner = factory.build(); @@ -171,7 +171,7 @@ public void test_build_withSingleContainerAdapterType_returnsKubernetesTaskRunne props, druidKubernetesClient, emitter, - executionConfigRef + dynamicConfigRef ); KubernetesTaskRunner runner = factory.build(); @@ -202,7 +202,7 @@ public void test_build_withSingleContainerAdapterTypeAndSidecarSupport_throwsIAE props, druidKubernetesClient, emitter, - executionConfigRef + dynamicConfigRef ); Assert.assertThrows( @@ -234,7 +234,7 @@ public void test_build_withMultiContainerAdapterType_returnsKubernetesTaskRunner props, druidKubernetesClient, emitter, - executionConfigRef + dynamicConfigRef ); KubernetesTaskRunner runner = factory.build(); @@ -260,7 +260,7 @@ public void test_build_withMultiContainerAdapterTypeAndSidecarSupport_returnsKub props, druidKubernetesClient, emitter, - executionConfigRef + dynamicConfigRef ); KubernetesTaskRunner runner = factory.build(); @@ -289,7 +289,7 @@ public void test_build_withPodTemplateAdapterType_returnsKubernetesTaskRunnerWit props, druidKubernetesClient, emitter, - executionConfigRef + dynamicConfigRef ); KubernetesTaskRunner runner = factory.build(); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java index 175ea50d9783..fa0da14fab73 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClientTest.java @@ -20,7 +20,6 @@ package org.apache.druid.k8s.overlord.common; import com.google.common.base.Optional; -import com.google.common.base.Supplier; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.PodBuilder; import io.fabric8.kubernetes.api.model.PodListBuilder; @@ -34,9 +33,6 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.k8s.overlord.execution.DefaultExecutionConfig; -import org.apache.druid.k8s.overlord.execution.DynamicTaskExecutionBehaviorStrategy; -import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -61,15 +57,13 @@ public class KubernetesPeonClientTest private KubernetesClientApi clientApi; private KubernetesPeonClient instance; private StubServiceEmitter serviceEmitter; - private Supplier executionConfigRef; @BeforeEach public void setup() { clientApi = new TestKubernetesClient(this.client); serviceEmitter = new StubServiceEmitter("service", "host"); - executionConfigRef = () -> new DefaultExecutionConfig(new DynamicTaskExecutionBehaviorStrategy(null)); - instance = new KubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter, executionConfigRef); + instance = new KubernetesPeonClient(clientApi, NAMESPACE, false, serviceEmitter); } @Test @@ -243,8 +237,7 @@ void test_deletePeonJob_withJob_withDebugJobsTrue_skipsDelete() new TestKubernetesClient(this.client), NAMESPACE, true, - serviceEmitter, - executionConfigRef + serviceEmitter ); Job job = new JobBuilder() @@ -269,8 +262,7 @@ void test_deletePeonJob_withoutJob_withDebugJobsTrue_skipsDelete() new TestKubernetesClient(this.client), NAMESPACE, true, - serviceEmitter, - executionConfigRef + serviceEmitter ); Assertions.assertTrue(instance.deletePeonJob(new K8sTaskId(ID))); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfigTest.java similarity index 64% rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfigTest.java rename to extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfigTest.java index 16aa5182e48a..de8919e329de 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultExecutionConfigTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DefaultKubernetesTaskRunnerDynamicConfigTest.java @@ -24,28 +24,28 @@ import org.junit.Assert; import org.junit.Test; -public class DefaultExecutionConfigTest +public class DefaultKubernetesTaskRunnerDynamicConfigTest { @Test - public void getBehaviorStrategyTest() + public void getPodTemplateSelectStrategyTest() { - ExecutionBehaviorStrategy strategy = new DefaultExecutionBehaviorStrategy(); - DefaultExecutionConfig config = new DefaultExecutionConfig(strategy); + PodTemplateSelectStrategy strategy = new TaskTypePodTemplateSelectStrategy(); + DefaultKubernetesTaskRunnerDynamicConfig config = new DefaultKubernetesTaskRunnerDynamicConfig(strategy); - Assert.assertEquals(strategy, config.getBehaviorStrategy()); + Assert.assertEquals(strategy, config.getPodTemplateSelectStrategy()); } @Test public void testSerde() throws Exception { final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); - ExecutionBehaviorStrategy strategy = new DefaultExecutionBehaviorStrategy(); - DefaultExecutionConfig config = new DefaultExecutionConfig(strategy); + PodTemplateSelectStrategy strategy = new TaskTypePodTemplateSelectStrategy(); + DefaultKubernetesTaskRunnerDynamicConfig config = new DefaultKubernetesTaskRunnerDynamicConfig(strategy); - DefaultExecutionConfig config2 = objectMapper.readValue( + DefaultKubernetesTaskRunnerDynamicConfig config2 = objectMapper.readValue( objectMapper.writeValueAsBytes(config), - DefaultExecutionConfig.class + DefaultKubernetesTaskRunnerDynamicConfig.class ); Assert.assertEquals(config, config2); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategyTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategyTest.java similarity index 59% rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategyTest.java rename to extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategyTest.java index 6ea206ef96bc..0cd27c164f70 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskExecutionBehaviorStrategyTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategyTest.java @@ -20,13 +20,20 @@ package org.apache.druid.k8s.overlord.execution; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import io.fabric8.kubernetes.api.model.PodTemplate; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.segment.TestHelper; +import org.easymock.EasyMockRunner; +import org.easymock.EasyMockSupport; +import org.easymock.Mock; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; import java.util.Collections; import java.util.HashMap; @@ -34,46 +41,64 @@ import java.util.Map; import java.util.Set; -public class DynamicTaskExecutionBehaviorStrategyTest +@RunWith(EasyMockRunner.class) +public class DynamicTaskPodTemplateSelectStrategyTest extends EasyMockSupport { + @Mock + private PodTemplate podTemplate; + private Map templates; + + @Before + public void setup() + { + templates = ImmutableMap.of( + "mock", + podTemplate, + "no_match", + podTemplate, + "match", + podTemplate, + "base", + podTemplate + ); + } @Test - public void testGetTaskCategory_nullSelectors() + public void testGetPodTemplate_ForTask_nullSelectorsFallbackToBaseTemplate() { - DynamicTaskExecutionBehaviorStrategy strategy = new DynamicTaskExecutionBehaviorStrategy(null); + DynamicTaskPodTemplateSelectStrategy strategy = new DynamicTaskPodTemplateSelectStrategy(null); Task task = NoopTask.create(); - Assert.assertNull(strategy.getTaskCategory(task)); + Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).lhs); } @Test - public void testGetTaskCategory_emptySelectors() + public void testGetPodTemplate_ForTask_emptySelectorsFallbackToBaseTemplate() { List emptySelectors = Collections.emptyList(); - DynamicTaskExecutionBehaviorStrategy strategy = new DynamicTaskExecutionBehaviorStrategy(emptySelectors); + DynamicTaskPodTemplateSelectStrategy strategy = new DynamicTaskPodTemplateSelectStrategy(emptySelectors); Task task = NoopTask.create(); - Assert.assertNull(strategy.getTaskCategory(task)); + Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).lhs); } @Test - public void testGetTaskCategory_noMatchSelectors() + public void testGetPodTemplate_ForTask_noMatchSelectorsFallbackToBaseTemplate() { Selector noMatchSelector = new MockSelector(false, "mock"); List selectors = Collections.singletonList(noMatchSelector); - DynamicTaskExecutionBehaviorStrategy strategy = new DynamicTaskExecutionBehaviorStrategy(selectors); + DynamicTaskPodTemplateSelectStrategy strategy = new DynamicTaskPodTemplateSelectStrategy(selectors); Task task = NoopTask.create(); - Assert.assertNull(strategy.getTaskCategory(task)); + Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).lhs); } @Test - public void testGetTaskCategory_withMatchSelectors() + public void testGetPodTemplate_ForTask_withMatchSelectors() { Selector noMatchSelector = new MockSelector(false, "no_match"); Selector matchSelector = new MockSelector(true, "match"); List selectors = Lists.newArrayList(noMatchSelector, matchSelector); - DynamicTaskExecutionBehaviorStrategy strategy = new DynamicTaskExecutionBehaviorStrategy(selectors); + DynamicTaskPodTemplateSelectStrategy strategy = new DynamicTaskPodTemplateSelectStrategy(selectors); Task task = NoopTask.create(); - Assert.assertEquals("match", strategy.getTaskCategory(task) - ); + Assert.assertEquals("match", strategy.getPodTemplateForTask(task, templates).lhs); } @Test @@ -92,12 +117,12 @@ public void testSerde() throws Exception taskFieldsConditions ); - DynamicTaskExecutionBehaviorStrategy strategy = new DynamicTaskExecutionBehaviorStrategy(Collections.singletonList( + DynamicTaskPodTemplateSelectStrategy strategy = new DynamicTaskPodTemplateSelectStrategy(Collections.singletonList( selector)); - DynamicTaskExecutionBehaviorStrategy strategy2 = objectMapper.readValue( + DynamicTaskPodTemplateSelectStrategy strategy2 = objectMapper.readValue( objectMapper.writeValueAsBytes(strategy), - DynamicTaskExecutionBehaviorStrategy.class + DynamicTaskPodTemplateSelectStrategy.class ); Assert.assertEquals(strategy, strategy2); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java similarity index 75% rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java rename to extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java index bf6ce7b74a79..041c4ca00a8b 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/ExecutionConfigTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java @@ -25,7 +25,7 @@ import org.junit.Assert; import org.junit.Test; -public class ExecutionConfigTest +public class KubernetesTaskRunnerDynamicConfigTest { private final ObjectMapper jsonMapper = new DefaultObjectMapper(); @@ -34,20 +34,20 @@ public void testSerde() throws JsonProcessingException { String json = "{\n" + " \"type\": \"default\",\n" - + " \"behaviorStrategy\": {\n" + + " \"podTemplateSelectStrategy\": {\n" + " \"type\": \"default\"\n" + " }\n" + "}"; - ExecutionConfig deserialized = jsonMapper.readValue(json, ExecutionConfig.class); - ExecutionBehaviorStrategy behaviorStrategy = deserialized.getBehaviorStrategy(); - Assert.assertTrue(behaviorStrategy instanceof DefaultExecutionBehaviorStrategy); + KubernetesTaskRunnerDynamicConfig deserialized = jsonMapper.readValue(json, KubernetesTaskRunnerDynamicConfig.class); + PodTemplateSelectStrategy behaviorStrategy = deserialized.getPodTemplateSelectStrategy(); + Assert.assertTrue(behaviorStrategy instanceof TaskTypePodTemplateSelectStrategy); json = "{\n" + " \"type\": \"default\",\n" - + " \"behaviorStrategy\": {\n" + + " \"podTemplateSelectStrategy\": {\n" + " \"type\": \"dynamicTask\",\n" - + " \"categorySelectors\": [\n" + + " \"templateSelectors\": [\n" + " {\n" + " \"selectionKey\": \"low-throughput\",\n" + " \"context.tags\": {\n" @@ -73,8 +73,8 @@ public void testSerde() throws JsonProcessingException + " }\n" + "}"; - deserialized = jsonMapper.readValue(json, ExecutionConfig.class); - behaviorStrategy = deserialized.getBehaviorStrategy(); - Assert.assertTrue(behaviorStrategy instanceof DynamicTaskExecutionBehaviorStrategy); + deserialized = jsonMapper.readValue(json, KubernetesTaskRunnerDynamicConfig.class); + behaviorStrategy = deserialized.getPodTemplateSelectStrategy(); + Assert.assertTrue(behaviorStrategy instanceof DynamicTaskPodTemplateSelectStrategy); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java index 2a4994bac034..241b4d9fc68f 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java @@ -43,8 +43,6 @@ import org.apache.druid.k8s.overlord.common.KubernetesPeonClient; import org.apache.druid.k8s.overlord.common.PeonCommandContext; import org.apache.druid.k8s.overlord.common.PeonPhase; -import org.apache.druid.k8s.overlord.execution.DefaultExecutionConfig; -import org.apache.druid.k8s.overlord.execution.ExecutionConfig; import org.apache.druid.server.DruidNode; import org.apache.druid.server.log.StartupLoggingConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -93,13 +91,7 @@ public void setup() new NamedType(IndexTask.IndexTuningConfig.class, "index") ); k8sClient = new DruidKubernetesClient(); - peonClient = new KubernetesPeonClient( - k8sClient, - "default", - false, - new NoopServiceEmitter(), - () -> new DefaultExecutionConfig(ExecutionConfig.DEFAULT_STRATEGY) - ); + peonClient = new KubernetesPeonClient(k8sClient, "default", false, new NoopServiceEmitter()); druidNode = new DruidNode( "test", null, diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 078a38d51a0f..85ed052acf8d 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -41,9 +41,9 @@ import org.apache.druid.k8s.overlord.common.DruidK8sConstants; import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; -import org.apache.druid.k8s.overlord.execution.DefaultExecutionConfig; -import org.apache.druid.k8s.overlord.execution.DynamicTaskExecutionBehaviorStrategy; -import org.apache.druid.k8s.overlord.execution.ExecutionConfig; +import org.apache.druid.k8s.overlord.execution.DefaultKubernetesTaskRunnerDynamicConfig; +import org.apache.druid.k8s.overlord.execution.DynamicTaskPodTemplateSelectStrategy; +import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; import org.apache.druid.k8s.overlord.execution.Selector; import org.apache.druid.server.DruidNode; import org.apache.druid.tasklogs.TaskLogs; @@ -77,7 +77,7 @@ public class PodTemplateTaskAdapterTest private DruidNode node; private ObjectMapper mapper; private TaskLogs taskLogs; - private Supplier executionConfigRef; + private Supplier dynamicConfigRef; @BeforeEach public void setup() @@ -97,7 +97,7 @@ public void setup() podTemplateSpec = K8sTestUtils.fileToResource("basePodTemplate.yaml", PodTemplate.class); taskLogs = EasyMock.createMock(TaskLogs.class); - executionConfigRef = () -> new DefaultExecutionConfig(ExecutionConfig.DEFAULT_STRATEGY); + dynamicConfigRef = () -> new DefaultKubernetesTaskRunnerDynamicConfig(KubernetesTaskRunnerDynamicConfig.DEFAULT_STRATEGY); } @Test @@ -113,7 +113,7 @@ public void test_fromTask_withoutBasePodTemplateInRuntimeProperites_raisesIAE() mapper, new Properties(), taskLogs, - executionConfigRef + dynamicConfigRef )); Assert.assertEquals(exception.getMessage(), "Pod template task adapter requires a base pod template to be specified under druid.indexer.runner.k8s.podTemplate.base"); } @@ -136,7 +136,7 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites_withEmptyFile_r mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef )); Assert.assertTrue(exception.getMessage().contains("Failed to load pod template file for")); @@ -158,12 +158,12 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites() throws IOExce mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Task task = new NoopTask("id", "id", "datasource", 0, 0, null); Job actual = adapter.fromTask(task); - Job expected = K8sTestUtils.fileToResource("expectedNoopJob.yaml", Job.class); + Job expected = K8sTestUtils.fileToResource("expectedNoopJobBaseTemplate.yaml", Job.class); assertJobSpecsEqual(actual, expected); } @@ -192,7 +192,7 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites_andTlsEnabled() mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Task task = new NoopTask("id", "id", "datasource", 0, 0, null); @@ -220,7 +220,7 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperties_withEmptyFile_r mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef )); } @@ -241,7 +241,7 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperites() throws IOExce mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Task task = new NoopTask("id", "id", "datasource", 0, 0, null); @@ -268,7 +268,7 @@ public void test_fromTask_withNoopPodTemplateInRuntimeProperites_dontSetTaskJSON mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Task task = new NoopTask( @@ -302,7 +302,7 @@ public void test_fromTask_withoutAnnotations_throwsDruidException() throws IOExc mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Job job = K8sTestUtils.fileToResource("baseJobWithoutAnnotations.yaml", Job.class); @@ -326,7 +326,7 @@ public void test_getTaskId() throws IOException mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Job job = new JobBuilder() .editSpec().editTemplate().editMetadata() @@ -351,7 +351,7 @@ public void test_getTaskId_noAnnotations() throws IOException mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Job job = new JobBuilder() .editSpec().editTemplate().editMetadata() @@ -376,7 +376,7 @@ public void test_getTaskId_missingTaskIdAnnotation() throws IOException mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Job job = new JobBuilder() .editSpec().editTemplate().editMetadata() @@ -403,7 +403,7 @@ public void test_toTask_withoutTaskAnnotation_throwsIOE() throws IOException mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Job baseJob = K8sTestUtils.fileToResource("baseJobWithoutAnnotations.yaml", Job.class); @@ -436,7 +436,7 @@ public void test_toTask() throws IOException mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Job job = K8sTestUtils.fileToResource("baseJob.yaml", Job.class); @@ -468,7 +468,7 @@ public void test_toTask_useTaskPayloadManager() throws IOException mapper, props, mockTestLogs, - executionConfigRef + dynamicConfigRef ); Job job = K8sTestUtils.fileToResource("expectedNoopJob.yaml", Job.class); @@ -493,7 +493,7 @@ public void test_fromTask_withRealIds() throws IOException mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Task task = new NoopTask( @@ -528,7 +528,7 @@ public void test_fromTask_taskSupportsQueries() throws IOException mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Task task = EasyMock.mock(Task.class); @@ -577,7 +577,7 @@ public void test_fromTask_withIndexKafkaPodTemplateInRuntimeProperites() throws mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Task kafkaTask = new NoopTask("id", "id", "datasource", 0, 0, null) { @@ -597,14 +597,9 @@ public String getType() } @Test - public void test_fromTask_matchPodTemplateBasedOnCategory() throws IOException + public void test_fromTask_matchPodTemplateBasedOnStrategy() throws IOException { String dataSource = "my_table"; - executionConfigRef = () -> new DefaultExecutionConfig(new DynamicTaskExecutionBehaviorStrategy(Collections.singletonList( - new Selector("lowThrougput", null, ImmutableMap.of( - "datasource", - Sets.newSet(dataSource) - ))))); Path baseTemplatePath = Files.createFile(tempDir.resolve("base.yaml")); mapper.writeValue(baseTemplatePath.toFile(), podTemplateSpec); @@ -622,6 +617,11 @@ public void test_fromTask_matchPodTemplateBasedOnCategory() 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 DynamicTaskPodTemplateSelectStrategy(Collections.singletonList( + new Selector("lowThrougput", null, ImmutableMap.of( + "datasource", + Sets.newSet(dataSource) + ))))); PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( taskRunnerConfig, @@ -630,7 +630,7 @@ public void test_fromTask_matchPodTemplateBasedOnCategory() throws IOException mapper, props, taskLogs, - executionConfigRef + dynamicConfigRef ); Task taskWithMatchedDatasource = new NoopTask("id", "id", dataSource, 0, 0, null); diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml index 2cef837f3972..7ceb160ed369 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml @@ -13,6 +13,7 @@ metadata: task.type: "noop" task.group.id: "id" task.datasource: "datasource" + template.key: "noop" spec: activeDeadlineSeconds: 14400 backoffLimit: 0 diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBaseTemplate.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBaseTemplate.yaml new file mode 100644 index 000000000000..b0a2742d3ef6 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBaseTemplate.yaml @@ -0,0 +1,53 @@ +apiVersion: batch/v1 +kind: Job +metadata: + name: "id-3e70afe5cd823dfc7dd308eea616426b" + labels: + druid.k8s.peons: "true" + druid.task.id: "id" + druid.task.type: "noop" + druid.task.group.id: "id" + druid.task.datasource: "datasource" + annotations: + task.id: "id" + task.type: "noop" + task.group.id: "id" + task.datasource: "datasource" + template.key: "base" +spec: + activeDeadlineSeconds: 14400 + backoffLimit: 0 + ttlSecondsAfterFinished: 172800 + template: + metadata: + labels: + druid.k8s.peons: "true" + druid.task.id: "id" + druid.task.type: "noop" + druid.task.group.id: "id" + druid.task.datasource: "datasource" + annotations: + task: "H4sIAAAAAAAAAD2MvQ4CIRCE32VqijsTG1qLi7W+wArEbHICrmC8EN7dJf40k/lmJtNQthxgEVPKMGCvXsXgKqnm4x89FTqlKm6MBzw+YCA1nvmm8W4/TQYuxRJeBbZ17cJ3ZhvoSbzShVcu2zLOf9cS7pUl+ANlclrCzr2/AQUK0FqZAAAA" + tls.enabled: "false" + task.id: "id" + task.type: "noop" + task.group.id: "id" + task.datasource: "datasource" + spec: + containers: + - command: + - sleep + - "3600" + env: + - name: "TASK_DIR" + value: "/tmp" + - name: "TASK_ID" + value: "id" + - name: "LOAD_BROADCAST_SEGMENTS" + value: "false" + - name: "TASK_JSON" + valueFrom: + fieldRef: + fieldPath: "metadata.annotations['task']" + image: one + name: primary diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml index cf16c49c5db1..e404b49bc5d1 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml @@ -13,6 +13,7 @@ metadata: task.type: "noop" task.group.id: "api-issued_kill_wikipedia3_omjobnbc_1000-01-01T00:00:00.000Z_2023-05-14T00:00:00.000Z_2023-05-15T17:03:01.220Z" task.datasource: "data_source" + template.key: "noop" spec: activeDeadlineSeconds: 14400 backoffLimit: 0 diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml index d72d0ef37b03..0467cc2b739e 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml @@ -13,6 +13,7 @@ metadata: task.type: "noop" task.group.id: "id" task.datasource: "datasource" + template.key: "noop" spec: activeDeadlineSeconds: 14400 backoffLimit: 0 diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml index a230ac913a60..d880dee3a85f 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml @@ -13,6 +13,7 @@ metadata: task.type: "noop" task.group.id: "id" task.datasource: "datasource" + template.key: "base" spec: activeDeadlineSeconds: 14400 backoffLimit: 0 From d882598d40479e8cc712323978cf2150e10d8624 Mon Sep 17 00:00:00 2001 From: YongGang Date: Mon, 10 Jun 2024 09:30:22 -0700 Subject: [PATCH 11/16] define Matcher interface --- .../extensions-contrib/k8s-jobs.md | 68 ++++---- .../overlord/common/DruidK8sConstants.java | 1 - .../overlord/common/KubernetesPeonClient.java | 10 +- .../DynamicTaskPodTemplateSelectStrategy.java | 103 ----------- ...KubernetesTaskExecutionConfigResource.java | 6 +- .../druid/k8s/overlord/execution/Matcher.java | 38 ++++ .../execution/PodTemplateSelectStrategy.java | 21 +-- ...lector.java => TaskPropertiesMatcher.java} | 42 ++--- ...skPropertiesPodTemplateSelectStrategy.java | 157 +++++++++++++++++ .../TaskTypePodTemplateSelectStrategy.java | 5 +- .../taskadapter/PodTemplateTaskAdapter.java | 12 +- ...amicTaskPodTemplateSelectStrategyTest.java | 146 ---------------- ...KubernetesTaskRunnerDynamicConfigTest.java | 74 ++++---- ...st.java => TaskPropertiesMatcherTest.java} | 43 ++--- ...opertiesPodTemplateSelectStrategyTest.java | 162 ++++++++++++++++++ .../PodTemplateTaskAdapterTest.java | 20 ++- .../src/test/resources/expectedNoopJob.yaml | 1 - .../expectedNoopJobBaseTemplate.yaml | 53 ------ .../resources/expectedNoopJobLongIds.yaml | 1 - .../resources/expectedNoopJobNoTaskJson.yaml | 1 - .../resources/expectedNoopJobTlsEnabled.yaml | 1 - 21 files changed, 490 insertions(+), 475 deletions(-) delete mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategy.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java rename extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/{Selector.java => TaskPropertiesMatcher.java} (71%) create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategy.java delete mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategyTest.java rename extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/{SelectorTest.java => TaskPropertiesMatcherTest.java} (83%) create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategyTest.java delete mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBaseTemplate.yaml diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index b1f346978b5d..2e89241fc1ce 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -223,9 +223,9 @@ The Dynamic Pod Template Selection feature enhances the K8s extension by enablin |Property|Description|Default| |--------|-----------|-------| |`TaskTypePodTemplateSelectStrategy`| This strategy selects pod templates based on task type for execution purposes, implementing the behavior that maps templates to specific task types. | true | -|`DynamicTaskPodTemplateSelectStrategy`| This strategy dynamically evaluates a series of selectors, known as templateSelectors, which are aligned with potential task properties. | false | +|`TaskPropertiesPodTemplateSelectStrategy`| This strategy evaluates a series of selectors, known as `templateSelectors`, which are aligned with potential task properties. | false | -`DynamicTaskPodTemplateSelectStrategy`, the strategy implementing this new feature, is based on conditional selectors `templateSelectors` that match against task properties. These selectors are ordered in the dynamic configuration, with the first selector given the highest priority during the evaluation process. This means that the selection process uses these ordered conditions to determine a task’s Pod template based on context tags and task fields. The first matching condition immediately determines the Pod template, thereby prioritizing certain configurations over others. +`TaskPropertiesPodTemplateSelectStrategy`, the strategy implementing this new feature, is based on conditional selectors `templateSelectors` that match against task properties. These selectors are ordered in the dynamic configuration, with the first selector given the highest priority during the evaluation process. This means that the selection process uses these ordered conditions to determine a task’s Pod template based on context tags and task fields. The first matching condition immediately determines the Pod template, thereby prioritizing certain configurations over others. Example Configuration: @@ -236,36 +236,42 @@ We define two template keys in the configuration—`low-throughput` and `medium- - Medium Throughput Template: If a task does not meet the low-throughput criteria, the system will then evaluate it against the next selector in order. In this example, if the task type is index_kafka, it will fall into the `medium-throughput` template. ``` { - "type": "default", - "podTemplateSelectStrategy": { - "type": "dynamicTask", - "templateSelectors": [ - { - "selectionKey": "low-throughput", - "context.tags": { - "billingCategory": [ - "streaming_ingestion" - ] - }, - "task": { - "datasource": [ - "wikipedia" - ] - } - }, - { - "selectionKey": "medium-throughput", - "task": { - "type": [ - "index_kafka" - ] - } - } - ] - } + "type":"default", + "podTemplateSelectStrategy":{ + "type":"taskProperties", + "templateSelectors":[ + { + "templateKey":"low-throughput", + "matcher":{ + "type":"default", + "context.tags":{ + "billingCategory":[ + "streaming_ingestion" + ] + }, + "task":{ + "datasource":[ + "wikipedia" + ] + } + } + }, + { + "templateKey":"medium-throughput", + "matcher":{ + "type":"default", + "task":{ + "type":[ + "index_kafka" + ] + } + } + } + ] + } } ``` -Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{template}: /path/to/taskSpecificPodSpec.yaml` where {template} is the matched `selectionKey` of the `podTemplateSelectStrategy` i.e low-throughput. +Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{template}: /path/to/taskSpecificPodSpec.yaml` where {template} is the matched `templateKey` of the `podTemplateSelectStrategy` i.e low-throughput. Similar to Overlord dynamic configuration, the following API endpoints are defined to retrieve and manage dynamic configurations of Pod Template Selection config: @@ -302,7 +308,7 @@ Similar to Overlord dynamic configuration, the following API endpoints are defin |Metric|Description|Dimensions|Normal value| |------|-----------|----------|------------| -| `k8s/peon/startup/time` | Metric indicating the milliseconds for peon pod to startup. | `dataSource`, `taskId`, `taskType`, `groupId`, `taskStatus`, `tags`, `podTemplate` |Varies| +| `k8s/peon/startup/time` | Metric indicating the milliseconds for peon pod to startup. | `dataSource`, `taskId`, `taskType`, `groupId`, `taskStatus`, `tags` |Varies| ### Gotchas diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java index ca32106334ed..6c195ed15151 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/DruidK8sConstants.java @@ -43,6 +43,5 @@ public class DruidK8sConstants public static final String LABEL_KEY = "druid.k8s.peons"; public static final String DRUID_LABEL_PREFIX = "druid."; public static final long MAX_ENV_VARIABLE_KBS = 130048; // 127 KB - public static final String POD_TEMPLATE_KEY = "template.key"; static final Predicate IS_TRANSIENT = e -> e instanceof KubernetesResourceNotFoundException; } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java index b7f15f1f9505..63487e4e373e 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/common/KubernetesPeonClient.java @@ -83,7 +83,7 @@ public Pod launchPeonJobAndWaitForStart(Job job, Task task, long howLong, TimeUn throw new IllegalStateException("K8s pod for the task [%s] appeared and disappeared. It can happen if the task was canceled"); } long duration = System.currentTimeMillis() - start; - emitK8sPodMetrics(task, job, "k8s/peon/startup/time", duration); + emitK8sPodMetrics(task, "k8s/peon/startup/time", duration); return result; }); } @@ -270,16 +270,10 @@ Pod getPeonPodWithRetries(KubernetesClient client, String jobName, int quietTrie } } - private void emitK8sPodMetrics(Task task, Job job, String metric, long durationMs) + private void emitK8sPodMetrics(Task task, String metric, long durationMs) { ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder(); IndexTaskUtils.setTaskDimensions(metricBuilder, task); - if (job.getMetadata() != null && job.getMetadata().getAnnotations() != null) { - metricBuilder.setDimensionIfNotNull( - "podTemplate", - job.getMetadata().getAnnotations().get(DruidK8sConstants.POD_TEMPLATE_KEY) - ); - } emitter.emit(metricBuilder.setMetric(metric, durationMs)); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategy.java deleted file mode 100644 index f6e1634ee521..000000000000 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategy.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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.execution; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import io.fabric8.kubernetes.api.model.PodTemplate; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.java.util.common.Pair; - -import java.util.List; -import java.util.Map; -import java.util.Objects; - -/** - * Implements {@link PodTemplateSelectStrategy} by dynamically evaluating a series of selectors. - * Each selector corresponds to a potential task template key. - */ -public class DynamicTaskPodTemplateSelectStrategy implements PodTemplateSelectStrategy -{ - private List templateSelectors; - - @JsonCreator - public DynamicTaskPodTemplateSelectStrategy( - @JsonProperty("templateSelectors") List templateSelectors - ) - { - this.templateSelectors = templateSelectors; - } - - /** - * Evaluates the provided task against the set selectors to determine its template. - * - * @param task the task to be checked - * @return the template if a selector matches, otherwise fallback to base template - */ - @Override - public Pair getPodTemplateForTask(Task task, Map templates) - { - String templateKey; - if (templateSelectors == null) { - templateKey = null; - } else { - templateKey = templateSelectors.stream() - .filter(selector -> selector.evaluate(task)) - .findFirst() - .map(Selector::getSelectionKey) - .orElse(null); - } - - return getTemplateOrDefault(templateKey, templates); - } - - @JsonProperty - public List getTemplateSelectors() - { - return templateSelectors; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DynamicTaskPodTemplateSelectStrategy that = (DynamicTaskPodTemplateSelectStrategy) o; - return Objects.equals(templateSelectors, that.templateSelectors); - } - - @Override - public int hashCode() - { - return Objects.hashCode(templateSelectors); - } - - @Override - public String toString() - { - return "DynamicTaskPodTemplateSelectStrategy{" + - "templateSelectors=" + templateSelectors + - '}'; - } -} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java index 228c07450eb5..9be7b2428a08 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java @@ -51,7 +51,7 @@ *

This class handles the CRUD operations for execution configurations and provides * endpoints to update, retrieve, and manage the history of these configurations.

*/ -@Path("/druid/indexer/v1/k8s/taskRunner") +@Path("/druid/indexer/v1/k8s/taskRunner/executionConfig") public class KubernetesTaskExecutionConfigResource { private static final Logger log = new Logger(KubernetesTaskExecutionConfigResource.class); @@ -77,7 +77,6 @@ public KubernetesTaskExecutionConfigResource( * @return a response indicating the success or failure of the update operation */ @POST - @Path("/executionConfig") @Consumes(MediaType.APPLICATION_JSON) @ResourceFilters(ConfigResourceFilter.class) public Response setExecutionConfig( @@ -107,7 +106,7 @@ public Response setExecutionConfig( * @return a response containing a list of audit entries or an error message */ @GET - @Path("/executionConfig/history") + @Path("/history") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(ConfigResourceFilter.class) public Response getExecutionConfigHistory( @@ -145,7 +144,6 @@ public Response getExecutionConfigHistory( * @return a Response object containing the current execution configuration in JSON format. */ @GET - @Path("/executionConfig") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(ConfigResourceFilter.class) public Response getExecutionConfig() diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java new file mode 100644 index 000000000000..495ed84f78d4 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java @@ -0,0 +1,38 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.indexing.common.task.Task; + +/** + * Represents a condition-based matcher that evaluates whether a given task meets specified criteria. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = TaskPropertiesMatcher.class) +public interface Matcher +{ + /** + * Evaluates this matcher against a given task. + * + * @param task the task to evaluate + * @return true if the task meets all the conditions specified by this matcher, otherwise false + */ + boolean evaluate(Task task); +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java index 16c7171430b0..e76de1d345dd 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.fabric8.kubernetes.api.model.PodTemplate; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.java.util.common.Pair; import java.util.Map; @@ -33,7 +32,7 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = TaskTypePodTemplateSelectStrategy.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "default", value = TaskTypePodTemplateSelectStrategy.class), - @JsonSubTypes.Type(name = "dynamicTask", value = DynamicTaskPodTemplateSelectStrategy.class), + @JsonSubTypes.Type(name = "taskProperties", value = TaskPropertiesPodTemplateSelectStrategy.class), }) public interface PodTemplateSelectStrategy { @@ -42,22 +41,8 @@ public interface PodTemplateSelectStrategy * allows for customized resource allocation and management tailored to the task's specific requirements. * * @param task The task for which the Pod template is determined. - * @return A key-value pair representing the selected Pod template. If no matching template is found, + * @return The selected Pod template. If no matching template is found, * the method falls back to a base template. */ - Pair getPodTemplateForTask(Task task, Map templates); - - default Pair getTemplateOrDefault(String templateKey, Map templates) - { - if (templates == null) { - return null; - } - - PodTemplate podTemplate = templates.get(templateKey); - if (podTemplate != null) { - return Pair.of(templateKey, podTemplate); - } else { - return Pair.of("base", templates.get("base")); - } - } + PodTemplate getPodTemplateForTask(Task task, Map templates); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcher.java similarity index 71% rename from extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java rename to extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcher.java index 70284fe728e4..8eca337b5ab4 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcher.java @@ -28,41 +28,28 @@ import java.util.Objects; import java.util.Set; -/** - * Represents a condition-based selector that evaluates whether a given task meets specified criteria. - * The selector uses conditions defined on context tags and task fields to determine if a task matches. - */ -public class Selector +public class TaskPropertiesMatcher implements Matcher { - private final String selectionKey; private final Map> cxtTagsConditions; private final Map> taskFieldsConditions; /** - * Creates a selector with specified conditions for context tags and task fields. + * Creates a matcher with specified conditions for context tags and task fields. * - * @param selectionKey the identifier representing the outcome when a task matches the conditions * @param cxtTagsConditions conditions on context tags * @param taskFieldsConditions conditions on task fields */ @JsonCreator - public Selector( - @JsonProperty("selectionKey") String selectionKey, + public TaskPropertiesMatcher( @JsonProperty("context.tags") Map> cxtTagsConditions, @JsonProperty("task") Map> taskFieldsConditions ) { - this.selectionKey = selectionKey; this.cxtTagsConditions = cxtTagsConditions; this.taskFieldsConditions = taskFieldsConditions; } - /** - * Evaluates this selector against a given task. - * - * @param task the task to evaluate - * @return true if the task meets all the conditions specified by this selector, otherwise false - */ + @Override public boolean evaluate(Task task) { boolean tagsMatch = true; @@ -103,12 +90,6 @@ public boolean evaluate(Task task) return true; } - @JsonProperty - public String getSelectionKey() - { - return selectionKey; - } - @JsonProperty("context.tags") public Map> getCxtTagsConditions() { @@ -130,25 +111,22 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - Selector selector = (Selector) o; - return Objects.equals(selectionKey, selector.selectionKey) && Objects.equals( - cxtTagsConditions, - selector.cxtTagsConditions - ) && Objects.equals(taskFieldsConditions, selector.taskFieldsConditions); + TaskPropertiesMatcher matcher = (TaskPropertiesMatcher) o; + return Objects.equals(cxtTagsConditions, matcher.cxtTagsConditions + ) && Objects.equals(taskFieldsConditions, matcher.taskFieldsConditions); } @Override public int hashCode() { - return Objects.hash(selectionKey, cxtTagsConditions, taskFieldsConditions); + return Objects.hash(cxtTagsConditions, taskFieldsConditions); } @Override public String toString() { - return "Selector{" + - "selectionKey=" + selectionKey + - ", context.tags=" + cxtTagsConditions + + return "TaskPropertiesMatcher{" + + "context.tags=" + cxtTagsConditions + ", task=" + taskFieldsConditions + '}'; } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategy.java new file mode 100644 index 000000000000..2fd2da71fb5b --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategy.java @@ -0,0 +1,157 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import io.fabric8.kubernetes.api.model.PodTemplate; +import org.apache.druid.indexing.common.task.Task; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Implements {@link PodTemplateSelectStrategy} by dynamically evaluating a series of selectors. + * Each selector corresponds to a potential task template key. + */ +public class TaskPropertiesPodTemplateSelectStrategy implements PodTemplateSelectStrategy +{ + private List templateSelectors; + + @JsonCreator + public TaskPropertiesPodTemplateSelectStrategy( + @JsonProperty("templateSelectors") List templateSelectors + ) + { + Preconditions.checkNotNull(templateSelectors, "templateSelectors"); + this.templateSelectors = templateSelectors; + } + + /** + * Evaluates the provided task against the set selectors to determine its template. + * + * @param task the task to be checked + * @return the template if a selector matches, otherwise fallback to base template + */ + @Override + public PodTemplate getPodTemplateForTask(Task task, Map templates) + { + String templateKey = templateSelectors.stream() + .filter(selector -> selector.getMatcher().evaluate(task)) + .findFirst() + .map(TemplateSelector::getTemplateKey) + .orElse(null); + + return templates.getOrDefault(templateKey, templates.get("base")); + } + + @JsonProperty + public List getTemplateSelectors() + { + return templateSelectors; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TaskPropertiesPodTemplateSelectStrategy that = (TaskPropertiesPodTemplateSelectStrategy) o; + return Objects.equals(templateSelectors, that.templateSelectors); + } + + @Override + public int hashCode() + { + return Objects.hashCode(templateSelectors); + } + + @Override + public String toString() + { + return "DynamicTaskPodTemplateSelectStrategy{" + + "templateSelectors=" + templateSelectors + + '}'; + } + + public static class TemplateSelector + { + private final Matcher matcher; + private final String templateKey; + + @JsonCreator + public TemplateSelector( + @JsonProperty("templateKey") String templateKey, + @JsonProperty("matcher") Matcher matcher + ) + { + Preconditions.checkNotNull(templateKey, "templateKey"); + Preconditions.checkNotNull(matcher, "matcher"); + this.templateKey = templateKey; + this.matcher = matcher; + } + + @JsonProperty + public String getTemplateKey() + { + return templateKey; + } + + @JsonProperty + public Matcher getMatcher() + { + return matcher; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TemplateSelector that = (TemplateSelector) o; + return Objects.equals(matcher, that.matcher) && Objects.equals(templateKey, that.templateKey); + } + + @Override + public int hashCode() + { + return Objects.hash(matcher, templateKey); + } + + @Override + public String toString() + { + return "TemplateSelector{" + + "templateKey=" + templateKey + + ", matcher=" + matcher + + '}'; + } + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskTypePodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskTypePodTemplateSelectStrategy.java index 8b4f7f015349..b374e0b6ff40 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskTypePodTemplateSelectStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskTypePodTemplateSelectStrategy.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import io.fabric8.kubernetes.api.model.PodTemplate; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.java.util.common.Pair; import java.util.Map; @@ -41,9 +40,9 @@ public TaskTypePodTemplateSelectStrategy() } @Override - public Pair getPodTemplateForTask(Task task, Map templates) + public PodTemplate getPodTemplateForTask(Task task, Map templates) { - return getTemplateOrDefault(task.getType(), templates); + return templates.getOrDefault(task.getType(), templates.get("base")); } @Override diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java index f700fc8ab7d5..8e3788e31e15 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapter.java @@ -40,7 +40,6 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.java.util.common.IAE; 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.logger.Logger; import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig; @@ -140,9 +139,9 @@ public Job fromTask(Task task) throws IOException podTemplateSelectStrategy = dynamicConfig.getPodTemplateSelectStrategy(); } - Pair podTemplatePair = podTemplateSelectStrategy.getPodTemplateForTask(task, templates); + PodTemplate podTemplate = podTemplateSelectStrategy.getPodTemplateForTask(task, templates); - if (podTemplatePair == null || podTemplatePair.rhs == null) { + if (podTemplate == null) { throw new ISE("Pod template spec not found for task type [%s]", task.getType()); } @@ -150,10 +149,10 @@ public Job fromTask(Task task) throws IOException .withNewMetadata() .withName(new K8sTaskId(task).getK8sJobName()) .addToLabels(getJobLabels(taskRunnerConfig, task)) - .addToAnnotations(getJobAnnotations(taskRunnerConfig, task, podTemplatePair.lhs)) + .addToAnnotations(getJobAnnotations(taskRunnerConfig, task)) .endMetadata() .withNewSpec() - .withTemplate(podTemplatePair.rhs.getTemplate()) + .withTemplate(podTemplate.getTemplate()) .editTemplate() .editOrNewMetadata() .addToAnnotations(getPodTemplateAnnotations(task)) @@ -334,7 +333,7 @@ private Map getJobLabels(KubernetesTaskRunnerConfig config, Task .build(); } - private Map getJobAnnotations(KubernetesTaskRunnerConfig config, Task task, String templateName) + private Map getJobAnnotations(KubernetesTaskRunnerConfig config, Task task) { return ImmutableMap.builder() .putAll(config.getAnnotations()) @@ -342,7 +341,6 @@ private Map getJobAnnotations(KubernetesTaskRunnerConfig config, .put(DruidK8sConstants.TASK_TYPE, task.getType()) .put(DruidK8sConstants.TASK_GROUP_ID, task.getGroupId()) .put(DruidK8sConstants.TASK_DATASOURCE, task.getDataSource()) - .put(DruidK8sConstants.POD_TEMPLATE_KEY, templateName) .build(); } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategyTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategyTest.java deleted file mode 100644 index 0cd27c164f70..000000000000 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/DynamicTaskPodTemplateSelectStrategyTest.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * 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.execution; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import io.fabric8.kubernetes.api.model.PodTemplate; -import org.apache.druid.indexing.common.task.NoopTask; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.segment.TestHelper; -import org.easymock.EasyMockRunner; -import org.easymock.EasyMockSupport; -import org.easymock.Mock; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; - -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -@RunWith(EasyMockRunner.class) -public class DynamicTaskPodTemplateSelectStrategyTest extends EasyMockSupport -{ - @Mock - private PodTemplate podTemplate; - private Map templates; - - @Before - public void setup() - { - templates = ImmutableMap.of( - "mock", - podTemplate, - "no_match", - podTemplate, - "match", - podTemplate, - "base", - podTemplate - ); - } - - @Test - public void testGetPodTemplate_ForTask_nullSelectorsFallbackToBaseTemplate() - { - DynamicTaskPodTemplateSelectStrategy strategy = new DynamicTaskPodTemplateSelectStrategy(null); - Task task = NoopTask.create(); - Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).lhs); - } - - @Test - public void testGetPodTemplate_ForTask_emptySelectorsFallbackToBaseTemplate() - { - List emptySelectors = Collections.emptyList(); - DynamicTaskPodTemplateSelectStrategy strategy = new DynamicTaskPodTemplateSelectStrategy(emptySelectors); - Task task = NoopTask.create(); - Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).lhs); - } - - @Test - public void testGetPodTemplate_ForTask_noMatchSelectorsFallbackToBaseTemplate() - { - Selector noMatchSelector = new MockSelector(false, "mock"); - List selectors = Collections.singletonList(noMatchSelector); - DynamicTaskPodTemplateSelectStrategy strategy = new DynamicTaskPodTemplateSelectStrategy(selectors); - Task task = NoopTask.create(); - Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).lhs); - } - - @Test - public void testGetPodTemplate_ForTask_withMatchSelectors() - { - Selector noMatchSelector = new MockSelector(false, "no_match"); - Selector matchSelector = new MockSelector(true, "match"); - List selectors = Lists.newArrayList(noMatchSelector, matchSelector); - DynamicTaskPodTemplateSelectStrategy strategy = new DynamicTaskPodTemplateSelectStrategy(selectors); - Task task = NoopTask.create(); - Assert.assertEquals("match", strategy.getPodTemplateForTask(task, templates).lhs); - } - - @Test - public void testSerde() throws Exception - { - final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); - Map> cxtTagsConditions = new HashMap<>(); - cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); - - Map> taskFieldsConditions = new HashMap<>(); - taskFieldsConditions.put("type", Sets.newHashSet(NoopTask.TYPE)); - - Selector selector = new Selector( - "TestSelector", - cxtTagsConditions, - taskFieldsConditions - ); - - DynamicTaskPodTemplateSelectStrategy strategy = new DynamicTaskPodTemplateSelectStrategy(Collections.singletonList( - selector)); - - DynamicTaskPodTemplateSelectStrategy strategy2 = objectMapper.readValue( - objectMapper.writeValueAsBytes(strategy), - DynamicTaskPodTemplateSelectStrategy.class - ); - Assert.assertEquals(strategy, strategy2); - } - - static class MockSelector extends Selector - { - private final boolean matches; - - MockSelector(boolean matches, String name) - { - super(name, null, null); - this.matches = matches; - } - - @Override - public boolean evaluate(final Task task) - { - return matches; - } - } -} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java index 041c4ca00a8b..f97d6224a369 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java @@ -39,42 +39,52 @@ public void testSerde() throws JsonProcessingException + " }\n" + "}"; - KubernetesTaskRunnerDynamicConfig deserialized = jsonMapper.readValue(json, KubernetesTaskRunnerDynamicConfig.class); - PodTemplateSelectStrategy behaviorStrategy = deserialized.getPodTemplateSelectStrategy(); - Assert.assertTrue(behaviorStrategy instanceof TaskTypePodTemplateSelectStrategy); + KubernetesTaskRunnerDynamicConfig deserialized = jsonMapper.readValue( + json, + KubernetesTaskRunnerDynamicConfig.class + ); + PodTemplateSelectStrategy selectStrategy = deserialized.getPodTemplateSelectStrategy(); + Assert.assertTrue(selectStrategy instanceof TaskTypePodTemplateSelectStrategy); json = "{\n" - + " \"type\": \"default\",\n" - + " \"podTemplateSelectStrategy\": {\n" - + " \"type\": \"dynamicTask\",\n" - + " \"templateSelectors\": [\n" - + " {\n" - + " \"selectionKey\": \"low-throughput\",\n" - + " \"context.tags\": {\n" - + " \"billingCategory\": [\n" - + " \"streaming_ingestion\"\n" - + " ]\n" - + " },\n" - + " \"task\": {\n" - + " \"datasource\": [\n" - + " \"wikipedia\"\n" - + " ]\n" - + " }\n" - + " },\n" - + " {\n" - + " \"selectionKey\": \"medium-throughput\",\n" - + " \"task\": {\n" - + " \"type\": [\n" - + " \"async_query\"\n" - + " ]\n" - + " }\n" - + " }\n" - + " ]\n" - + " }\n" + + " \"type\":\"default\",\n" + + " \"podTemplateSelectStrategy\":{\n" + + " \"type\":\"taskProperties\",\n" + + " \"templateSelectors\":[\n" + + " {\n" + + " \"templateKey\":\"low-throughput\",\n" + + " \"matcher\":{\n" + + " \"type\":\"default\",\n" + + " \"context.tags\":{\n" + + " \"billingCategory\":[\n" + + " \"streaming_ingestion\"\n" + + " ]\n" + + " },\n" + + " \"task\":{\n" + + " \"datasource\":[\n" + + " \"wikipedia\"\n" + + " ]\n" + + " }\n" + + " }\n" + + " },\n" + + " {\n" + + " \"templateKey\":\"medium-throughput\",\n" + + " \"matcher\":{\n" + + " \"type\":\"default\",\n" + + " \"task\":{\n" + + " \"type\":[\n" + + " \"index_kafka\"\n" + + " ]\n" + + " }\n" + + " }\n" + + " }\n" + + " ]\n" + + " }\n" + "}"; deserialized = jsonMapper.readValue(json, KubernetesTaskRunnerDynamicConfig.class); - behaviorStrategy = deserialized.getPodTemplateSelectStrategy(); - Assert.assertTrue(behaviorStrategy instanceof DynamicTaskPodTemplateSelectStrategy); + selectStrategy = deserialized.getPodTemplateSelectStrategy(); + Assert.assertTrue(selectStrategy instanceof TaskPropertiesPodTemplateSelectStrategy); + Assert.assertEquals(2, ((TaskPropertiesPodTemplateSelectStrategy) selectStrategy).getTemplateSelectors().size()); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcherTest.java similarity index 83% rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java rename to extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcherTest.java index 8ab89a2c572e..e207b61ed616 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcherTest.java @@ -33,7 +33,7 @@ import java.util.Map; import java.util.Set; -public class SelectorTest +public class TaskPropertiesMatcherTest { @Test @@ -49,13 +49,12 @@ public void shouldReturnTrueWhenAllTagsAndTasksMatch() Task task = NoopTask.forDatasource(dataSource); task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); - Selector selector = new Selector( - "TestSelector", + TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( cxtTagsConditions, taskFieldsConditions ); - Assert.assertTrue(selector.evaluate(task)); + Assert.assertTrue(matcher.evaluate(task)); } @Test @@ -70,13 +69,12 @@ public void shouldReturnFalseWhenTagsDoNotMatch() Task task = NoopTask.forDatasource(dataSource); - Selector selector = new Selector( - "TestSelector", + TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( cxtTagsConditions, taskFieldsConditions ); - Assert.assertFalse(selector.evaluate(task)); + Assert.assertFalse(matcher.evaluate(task)); } @Test @@ -93,13 +91,12 @@ public void shouldReturnFalseWhenSomeTagsDoNotMatch() Task task = NoopTask.forDatasource(dataSource); task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); - Selector selector = new Selector( - "TestSelector", + TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( cxtTagsConditions, taskFieldsConditions ); - Assert.assertFalse(selector.evaluate(task)); + Assert.assertFalse(matcher.evaluate(task)); } @Test @@ -114,13 +111,12 @@ public void shouldReturnFalseWhenTaskFieldsDoNotMatch() Task task = NoopTask.forDatasource("another_table"); task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); - Selector selector = new Selector( - "TestSelector", + TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( cxtTagsConditions, taskFieldsConditions ); - Assert.assertFalse(selector.evaluate(task)); + Assert.assertFalse(matcher.evaluate(task)); } @Test @@ -136,13 +132,12 @@ public void shouldReturnFalseWhenSomeTaskFieldsDoNotMatch() Task task = NoopTask.forDatasource("another_table"); task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); - Selector selector = new Selector( - "TestSelector", + TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( cxtTagsConditions, taskFieldsConditions ); - Assert.assertFalse(selector.evaluate(task)); + Assert.assertFalse(matcher.evaluate(task)); } @Test @@ -151,13 +146,12 @@ public void shouldReturnTrueWhenNoConditionsSpecified() Task task = NoopTask.forDatasource("my_table"); task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); - Selector selector = new Selector( - "TestSelector", + TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( null, null ); - Assert.assertTrue(selector.evaluate(task)); + Assert.assertTrue(matcher.evaluate(task)); } @Test @@ -170,16 +164,15 @@ public void testSerde() throws Exception Map> taskFieldsConditions = new HashMap<>(); taskFieldsConditions.put("type", Sets.newHashSet(NoopTask.TYPE)); - Selector selector = new Selector( - "TestSelector", + TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( cxtTagsConditions, taskFieldsConditions ); - Selector selector2 = objectMapper.readValue( - objectMapper.writeValueAsBytes(selector), - Selector.class + TaskPropertiesMatcher matcher2 = objectMapper.readValue( + objectMapper.writeValueAsBytes(matcher), + TaskPropertiesMatcher.class ); - Assert.assertEquals(selector, selector2); + Assert.assertEquals(matcher, matcher2); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategyTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategyTest.java new file mode 100644 index 000000000000..155e583e7f71 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategyTest.java @@ -0,0 +1,162 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import io.fabric8.kubernetes.api.model.ObjectMeta; +import io.fabric8.kubernetes.api.model.PodTemplate; +import org.apache.druid.indexing.common.task.NoopTask; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class TaskPropertiesPodTemplateSelectStrategyTest +{ + private Map templates; + + @Before + public void setup() + { + templates = ImmutableMap.of( + "mock", + new PodTemplate(null, null, new ObjectMeta() + { + @Override + public String getName() + { + return "mock"; + } + }, null), + "no_match", + new PodTemplate(null, null, new ObjectMeta() + { + @Override + public String getName() + { + return "no_match"; + } + }, null), + "match", + new PodTemplate(null, null, new ObjectMeta() + { + @Override + public String getName() + { + return "match"; + } + }, null), + "base", + new PodTemplate(null, "base", new ObjectMeta() + { + @Override + public String getName() + { + return "base"; + } + }, null) + ); + } + + @Test(expected = NullPointerException.class) + public void testExceptionThrownIfNoTemplateSelectors() + { + new TaskPropertiesPodTemplateSelectStrategy(null); + } + + @Test + public void testGetPodTemplate_ForTask_emptySelectorsFallbackToBaseTemplate() + { + List emptySelectors = Collections.emptyList(); + TaskPropertiesPodTemplateSelectStrategy strategy = new TaskPropertiesPodTemplateSelectStrategy(emptySelectors); + Task task = NoopTask.create(); + Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).getMetadata().getName()); + } + + @Test + public void testGetPodTemplate_ForTask_noMatchSelectorsFallbackToBaseTemplate() + { + TaskPropertiesPodTemplateSelectStrategy.TemplateSelector noMatchSelector = new MockTemplateSelector(false, "mock"); + List selectors = Collections.singletonList(noMatchSelector); + TaskPropertiesPodTemplateSelectStrategy strategy = new TaskPropertiesPodTemplateSelectStrategy(selectors); + Task task = NoopTask.create(); + Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).getMetadata().getName()); + } + + @Test + public void testGetPodTemplate_ForTask_withMatchSelectors() + { + TaskPropertiesPodTemplateSelectStrategy.TemplateSelector noMatchSelector = new MockTemplateSelector( + false, + "no_match" + ); + TaskPropertiesPodTemplateSelectStrategy.TemplateSelector matchSelector = new MockTemplateSelector(true, "match"); + List selectors = Lists.newArrayList( + noMatchSelector, + matchSelector + ); + TaskPropertiesPodTemplateSelectStrategy strategy = new TaskPropertiesPodTemplateSelectStrategy(selectors); + Task task = NoopTask.create(); + Assert.assertEquals("match", strategy.getPodTemplateForTask(task, templates).getMetadata().getName()); + } + + @Test + public void testSerde() throws Exception + { + final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); + Map> cxtTagsConditions = new HashMap<>(); + cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); + + Map> taskFieldsConditions = new HashMap<>(); + taskFieldsConditions.put("type", Sets.newHashSet(NoopTask.TYPE)); + + TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( + cxtTagsConditions, + taskFieldsConditions + ); + + TaskPropertiesPodTemplateSelectStrategy strategy = new TaskPropertiesPodTemplateSelectStrategy(Collections.singletonList( + new TaskPropertiesPodTemplateSelectStrategy.TemplateSelector("TestSelector", matcher))); + + TaskPropertiesPodTemplateSelectStrategy strategy2 = objectMapper.readValue( + objectMapper.writeValueAsBytes(strategy), + TaskPropertiesPodTemplateSelectStrategy.class + ); + Assert.assertEquals(strategy, strategy2); + } + + static class MockTemplateSelector extends TaskPropertiesPodTemplateSelectStrategy.TemplateSelector + { + MockTemplateSelector(boolean matches, String name) + { + super(name, task -> matches); + } + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 85ed052acf8d..5a5635010a41 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -42,9 +42,9 @@ import org.apache.druid.k8s.overlord.common.K8sTaskId; import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.execution.DefaultKubernetesTaskRunnerDynamicConfig; -import org.apache.druid.k8s.overlord.execution.DynamicTaskPodTemplateSelectStrategy; import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; -import org.apache.druid.k8s.overlord.execution.Selector; +import org.apache.druid.k8s.overlord.execution.TaskPropertiesMatcher; +import org.apache.druid.k8s.overlord.execution.TaskPropertiesPodTemplateSelectStrategy; import org.apache.druid.server.DruidNode; import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; @@ -163,7 +163,7 @@ public void test_fromTask_withBasePodTemplateInRuntimeProperites() throws IOExce Task task = new NoopTask("id", "id", "datasource", 0, 0, null); Job actual = adapter.fromTask(task); - Job expected = K8sTestUtils.fileToResource("expectedNoopJobBaseTemplate.yaml", Job.class); + Job expected = K8sTestUtils.fileToResource("expectedNoopJob.yaml", Job.class); assertJobSpecsEqual(actual, expected); } @@ -617,11 +617,15 @@ 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 DynamicTaskPodTemplateSelectStrategy(Collections.singletonList( - new Selector("lowThrougput", null, ImmutableMap.of( - "datasource", - Sets.newSet(dataSource) - ))))); + dynamicConfigRef = () -> new DefaultKubernetesTaskRunnerDynamicConfig(new TaskPropertiesPodTemplateSelectStrategy( + Collections.singletonList( + new TaskPropertiesPodTemplateSelectStrategy.TemplateSelector( + "lowThrougput", + new TaskPropertiesMatcher(null, ImmutableMap.of( + "datasource", + Sets.newSet(dataSource) + )) + )))); PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( taskRunnerConfig, diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml index 7ceb160ed369..2cef837f3972 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJob.yaml @@ -13,7 +13,6 @@ metadata: task.type: "noop" task.group.id: "id" task.datasource: "datasource" - template.key: "noop" spec: activeDeadlineSeconds: 14400 backoffLimit: 0 diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBaseTemplate.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBaseTemplate.yaml deleted file mode 100644 index b0a2742d3ef6..000000000000 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobBaseTemplate.yaml +++ /dev/null @@ -1,53 +0,0 @@ -apiVersion: batch/v1 -kind: Job -metadata: - name: "id-3e70afe5cd823dfc7dd308eea616426b" - labels: - druid.k8s.peons: "true" - druid.task.id: "id" - druid.task.type: "noop" - druid.task.group.id: "id" - druid.task.datasource: "datasource" - annotations: - task.id: "id" - task.type: "noop" - task.group.id: "id" - task.datasource: "datasource" - template.key: "base" -spec: - activeDeadlineSeconds: 14400 - backoffLimit: 0 - ttlSecondsAfterFinished: 172800 - template: - metadata: - labels: - druid.k8s.peons: "true" - druid.task.id: "id" - druid.task.type: "noop" - druid.task.group.id: "id" - druid.task.datasource: "datasource" - annotations: - task: "H4sIAAAAAAAAAD2MvQ4CIRCE32VqijsTG1qLi7W+wArEbHICrmC8EN7dJf40k/lmJtNQthxgEVPKMGCvXsXgKqnm4x89FTqlKm6MBzw+YCA1nvmm8W4/TQYuxRJeBbZ17cJ3ZhvoSbzShVcu2zLOf9cS7pUl+ANlclrCzr2/AQUK0FqZAAAA" - tls.enabled: "false" - task.id: "id" - task.type: "noop" - task.group.id: "id" - task.datasource: "datasource" - spec: - containers: - - command: - - sleep - - "3600" - env: - - name: "TASK_DIR" - value: "/tmp" - - name: "TASK_ID" - value: "id" - - name: "LOAD_BROADCAST_SEGMENTS" - value: "false" - - name: "TASK_JSON" - valueFrom: - fieldRef: - fieldPath: "metadata.annotations['task']" - image: one - name: primary diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml index e404b49bc5d1..cf16c49c5db1 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobLongIds.yaml @@ -13,7 +13,6 @@ metadata: task.type: "noop" task.group.id: "api-issued_kill_wikipedia3_omjobnbc_1000-01-01T00:00:00.000Z_2023-05-14T00:00:00.000Z_2023-05-15T17:03:01.220Z" task.datasource: "data_source" - template.key: "noop" spec: activeDeadlineSeconds: 14400 backoffLimit: 0 diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml index 0467cc2b739e..d72d0ef37b03 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobNoTaskJson.yaml @@ -13,7 +13,6 @@ metadata: task.type: "noop" task.group.id: "id" task.datasource: "datasource" - template.key: "noop" spec: activeDeadlineSeconds: 14400 backoffLimit: 0 diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml index d880dee3a85f..a230ac913a60 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/resources/expectedNoopJobTlsEnabled.yaml @@ -13,7 +13,6 @@ metadata: task.type: "noop" task.group.id: "id" task.datasource: "datasource" - template.key: "base" spec: activeDeadlineSeconds: 14400 backoffLimit: 0 From e1c9711ba0783d37625c85d55d1b6e37e0f2c437 Mon Sep 17 00:00:00 2001 From: YongGang Date: Mon, 10 Jun 2024 10:33:00 -0700 Subject: [PATCH 12/16] fix test coverage --- ...askPropertiesPodTemplateSelectStrategyTest.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategyTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategyTest.java index 155e583e7f71..532753170860 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategyTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategyTest.java @@ -86,11 +86,23 @@ public String getName() } @Test(expected = NullPointerException.class) - public void testExceptionThrownIfNoTemplateSelectors() + public void shouldThrowNullPointerExceptionWhenTemplateSelectorsAreNull() { new TaskPropertiesPodTemplateSelectStrategy(null); } + @Test(expected = NullPointerException.class) + public void shouldThrowNullPointerExceptionWhenTemplateKeyIsNull() + { + new TaskPropertiesPodTemplateSelectStrategy.TemplateSelector(null, task -> false); + } + + @Test(expected = NullPointerException.class) + public void shouldThrowNullPointerExceptionWhenMatcherIsNull() + { + new TaskPropertiesPodTemplateSelectStrategy.TemplateSelector("myTemplate", null); + } + @Test public void testGetPodTemplate_ForTask_emptySelectorsFallbackToBaseTemplate() { From 8e9308ef83191000fd3855e7f7199b359eda54e2 Mon Sep 17 00:00:00 2001 From: YongGang Date: Mon, 10 Jun 2024 11:11:11 -0700 Subject: [PATCH 13/16] use lower case for endpoint path --- .../execution/KubernetesTaskExecutionConfigResource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java index 9be7b2428a08..ec03b045f503 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResource.java @@ -51,7 +51,7 @@ *

This class handles the CRUD operations for execution configurations and provides * endpoints to update, retrieve, and manage the history of these configurations.

*/ -@Path("/druid/indexer/v1/k8s/taskRunner/executionConfig") +@Path("/druid/indexer/v1/k8s/taskrunner/executionconfig") public class KubernetesTaskExecutionConfigResource { private static final Logger log = new Logger(KubernetesTaskExecutionConfigResource.class); From c71c805bd0c88f608fc795e5dd9953b4f59e6f42 Mon Sep 17 00:00:00 2001 From: YongGang Date: Mon, 10 Jun 2024 11:21:17 -0700 Subject: [PATCH 14/16] update Json name --- docs/development/extensions-contrib/k8s-jobs.md | 4 ++-- .../java/org/apache/druid/k8s/overlord/execution/Matcher.java | 4 ++++ .../execution/KubernetesTaskRunnerDynamicConfigTest.java | 4 ++-- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index 2e89241fc1ce..85a8657a5057 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -243,7 +243,7 @@ We define two template keys in the configuration—`low-throughput` and `medium- { "templateKey":"low-throughput", "matcher":{ - "type":"default", + "type":"taskProperties", "context.tags":{ "billingCategory":[ "streaming_ingestion" @@ -259,7 +259,7 @@ We define two template keys in the configuration—`low-throughput` and `medium- { "templateKey":"medium-throughput", "matcher":{ - "type":"default", + "type":"taskProperties", "task":{ "type":[ "index_kafka" diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java index 495ed84f78d4..6fed7d3208ee 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java @@ -19,6 +19,7 @@ package org.apache.druid.k8s.overlord.execution; +import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.indexing.common.task.Task; @@ -26,6 +27,9 @@ * Represents a condition-based matcher that evaluates whether a given task meets specified criteria. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = TaskPropertiesMatcher.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "taskProperties", value = TaskPropertiesMatcher.class) +}) public interface Matcher { /** diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java index f97d6224a369..202cab0d4bf8 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java @@ -54,7 +54,7 @@ public void testSerde() throws JsonProcessingException + " {\n" + " \"templateKey\":\"low-throughput\",\n" + " \"matcher\":{\n" - + " \"type\":\"default\",\n" + + " \"type\":\"taskProperties\",\n" + " \"context.tags\":{\n" + " \"billingCategory\":[\n" + " \"streaming_ingestion\"\n" @@ -70,7 +70,7 @@ public void testSerde() throws JsonProcessingException + " {\n" + " \"templateKey\":\"medium-throughput\",\n" + " \"matcher\":{\n" - + " \"type\":\"default\",\n" + + " \"type\":\"taskProperties\",\n" + " \"task\":{\n" + " \"type\":[\n" + " \"index_kafka\"\n" From 18432bf308cac1cc81efd0d43c81798806aa6475 Mon Sep 17 00:00:00 2001 From: YongGang Date: Mon, 10 Jun 2024 14:58:06 -0700 Subject: [PATCH 15/16] add more tests --- ...rnetesTaskExecutionConfigResourceTest.java | 97 +++++++++++++++++++ 1 file changed, 97 insertions(+) create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResourceTest.java diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResourceTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResourceTest.java new file mode 100644 index 000000000000..b76b7eaf0cfe --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskExecutionConfigResourceTest.java @@ -0,0 +1,97 @@ +/* + * 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.execution; + +import org.apache.druid.audit.AuditManager; +import org.apache.druid.common.config.ConfigManager; +import org.apache.druid.common.config.JacksonConfigManager; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthorizationUtils; +import org.easymock.EasyMock; +import org.junit.Before; +import org.junit.Test; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.core.Response; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class KubernetesTaskExecutionConfigResourceTest +{ + private JacksonConfigManager configManager; + private AuditManager auditManager; + private HttpServletRequest req; + private KubernetesTaskRunnerDynamicConfig dynamicConfig; + + @Before + public void setUp() + { + configManager = EasyMock.createMock(JacksonConfigManager.class); + auditManager = EasyMock.createMock(AuditManager.class); + req = EasyMock.createMock(HttpServletRequest.class); + dynamicConfig = EasyMock.createMock(KubernetesTaskRunnerDynamicConfig.class); + } + + @Test + public void setExecutionConfigSuccessfulUpdate() + { + KubernetesTaskExecutionConfigResource testedResource = new KubernetesTaskExecutionConfigResource( + configManager, + auditManager + ); + 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, + dynamicConfig, + AuthorizationUtils.buildAuditInfo(req) + )).andReturn(ConfigManager.SetResult.ok()); + EasyMock.replay(configManager, auditManager, dynamicConfig); + + Response result = testedResource.setExecutionConfig(dynamicConfig, req); + assertEquals(Response.Status.OK.getStatusCode(), result.getStatus()); + } + + @Test + public void setExecutionConfigFailedUpdate() + { + KubernetesTaskExecutionConfigResource testedResource = new KubernetesTaskExecutionConfigResource( + configManager, + auditManager + ); + 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, + dynamicConfig, + AuthorizationUtils.buildAuditInfo(req) + )).andReturn(ConfigManager.SetResult.failure(new RuntimeException())); + EasyMock.replay(configManager, auditManager, dynamicConfig); + + Response result = testedResource.setExecutionConfig(dynamicConfig, req); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), result.getStatus()); + } +} From 57e28317fa075815a3b79efd19430e9cd9f740fa Mon Sep 17 00:00:00 2001 From: YongGang Date: Wed, 12 Jun 2024 12:25:15 -0700 Subject: [PATCH 16/16] refactoring Selector class --- .../extensions-contrib/k8s-jobs.md | 59 +++---- .../druid/k8s/overlord/execution/Matcher.java | 42 ----- .../execution/PodTemplateSelectStrategy.java | 2 +- .../k8s/overlord/execution/Selector.java | 157 ++++++++++++++++++ ...electorBasedPodTemplateSelectStrategy.java | 112 +++++++++++++ .../execution/TaskPropertiesMatcher.java | 133 --------------- ...skPropertiesPodTemplateSelectStrategy.java | 157 ------------------ ...KubernetesTaskRunnerDynamicConfigTest.java | 57 +++---- ...orBasedPodTemplateSelectStrategyTest.java} | 78 +++++---- ...tiesMatcherTest.java => SelectorTest.java} | 81 +++++---- .../PodTemplateTaskAdapterTest.java | 15 +- 11 files changed, 400 insertions(+), 493 deletions(-) delete mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java create mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategy.java delete mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcher.java delete mode 100644 extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategy.java rename extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/{TaskPropertiesPodTemplateSelectStrategyTest.java => SelectorBasedPodTemplateSelectStrategyTest.java} (60%) rename extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/{TaskPropertiesMatcherTest.java => SelectorTest.java} (65%) diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md index 85a8657a5057..1d75e2b7efe7 100644 --- a/docs/development/extensions-contrib/k8s-jobs.md +++ b/docs/development/extensions-contrib/k8s-jobs.md @@ -223,9 +223,9 @@ The Dynamic Pod Template Selection feature enhances the K8s extension by enablin |Property|Description|Default| |--------|-----------|-------| |`TaskTypePodTemplateSelectStrategy`| This strategy selects pod templates based on task type for execution purposes, implementing the behavior that maps templates to specific task types. | true | -|`TaskPropertiesPodTemplateSelectStrategy`| This strategy evaluates a series of selectors, known as `templateSelectors`, which are aligned with potential task properties. | false | +|`SelectorBasedPodTemplateSelectStrategy`| This strategy evaluates a series of selectors, known as `selectors`, which are aligned with potential task properties. | false | -`TaskPropertiesPodTemplateSelectStrategy`, the strategy implementing this new feature, is based on conditional selectors `templateSelectors` that match against task properties. These selectors are ordered in the dynamic configuration, with the first selector given the highest priority during the evaluation process. This means that the selection process uses these ordered conditions to determine a task’s Pod template based on context tags and task fields. The first matching condition immediately determines the Pod template, thereby prioritizing certain configurations over others. +`SelectorBasedPodTemplateSelectStrategy`, the strategy implementing this new feature, is based on conditional `selectors` that match against top-level keys from the task payload. Currently, it supports matching based on task context tags, task type, and dataSource. These selectors are ordered in the dynamic configuration, with the first selector given the highest priority during the evaluation process. This means that the selection process uses these ordered conditions to determine a task’s Pod template. The first matching condition immediately determines the Pod template, thereby prioritizing certain configurations over others. If no selector matches, it will fall back to an optional `defaultKey` if configured; if there is still no match, it will use the `base` template. Example Configuration: @@ -236,42 +236,29 @@ We define two template keys in the configuration—`low-throughput` and `medium- - Medium Throughput Template: If a task does not meet the low-throughput criteria, the system will then evaluate it against the next selector in order. In this example, if the task type is index_kafka, it will fall into the `medium-throughput` template. ``` { - "type":"default", - "podTemplateSelectStrategy":{ - "type":"taskProperties", - "templateSelectors":[ - { - "templateKey":"low-throughput", - "matcher":{ - "type":"taskProperties", - "context.tags":{ - "billingCategory":[ - "streaming_ingestion" - ] - }, - "task":{ - "datasource":[ - "wikipedia" - ] - } - } - }, - { - "templateKey":"medium-throughput", - "matcher":{ - "type":"taskProperties", - "task":{ - "type":[ - "index_kafka" - ] - } - } - } - ] - } + "type": "default", + "podTemplateSelectStrategy": + { + "type": "selectorBased", + "selectors": [ + { + "selectionKey": "low-throughput", + "context.tags": + { + "billingCategory": ["streaming_ingestion"] + }, + "dataSource": ["wikipedia"] + }, + { + "selectionKey": "medium-throughput", + "type": ["index_kafka"] + } + ], + "defaultKey"" "base" + } } ``` -Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{template}: /path/to/taskSpecificPodSpec.yaml` where {template} is the matched `templateKey` of the `podTemplateSelectStrategy` i.e low-throughput. +Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{template}: /path/to/taskSpecificPodSpec.yaml` where {template} is the matched `selectionKey` of the `podTemplateSelectStrategy` i.e low-throughput. Similar to Overlord dynamic configuration, the following API endpoints are defined to retrieve and manage dynamic configurations of Pod Template Selection config: diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java deleted file mode 100644 index 6fed7d3208ee..000000000000 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Matcher.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.execution; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.indexing.common.task.Task; - -/** - * Represents a condition-based matcher that evaluates whether a given task meets specified criteria. - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = TaskPropertiesMatcher.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "taskProperties", value = TaskPropertiesMatcher.class) -}) -public interface Matcher -{ - /** - * Evaluates this matcher against a given task. - * - * @param task the task to evaluate - * @return true if the task meets all the conditions specified by this matcher, otherwise false - */ - boolean evaluate(Task task); -} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java index e76de1d345dd..1b8d57419d28 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/PodTemplateSelectStrategy.java @@ -32,7 +32,7 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = TaskTypePodTemplateSelectStrategy.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "default", value = TaskTypePodTemplateSelectStrategy.class), - @JsonSubTypes.Type(name = "taskProperties", value = TaskPropertiesPodTemplateSelectStrategy.class), + @JsonSubTypes.Type(name = "selectorBased", value = SelectorBasedPodTemplateSelectStrategy.class), }) public interface PodTemplateSelectStrategy { diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java new file mode 100644 index 000000000000..a314a69b3811 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/Selector.java @@ -0,0 +1,157 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.query.DruidMetrics; + +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + * Represents a condition-based selector that evaluates whether a given task meets specified criteria. + * The selector uses conditions defined on context tags and task fields to determine if a task matches. + */ +public class Selector +{ + private final String selectionKey; + private final Map> cxtTagsConditions; + private final Set taskTypeCondition; + private final Set dataSourceCondition; + + /** + * Creates a selector with specified conditions for context tags and task fields. + * + * @param selectionKey the identifier representing the outcome when a task matches the conditions + * @param cxtTagsConditions conditions on context tags + * @param taskTypeCondition conditions on task type + * @param dataSourceCondition conditions on task dataSource + */ + @JsonCreator + public Selector( + @JsonProperty("selectionKey") String selectionKey, + @JsonProperty("context.tags") Map> cxtTagsConditions, + @JsonProperty("type") Set taskTypeCondition, + @JsonProperty("dataSource") Set dataSourceCondition + ) + { + this.selectionKey = selectionKey; + this.cxtTagsConditions = cxtTagsConditions; + this.taskTypeCondition = taskTypeCondition; + this.dataSourceCondition = dataSourceCondition; + } + + /** + * Evaluates this selector against a given task. + * + * @param task the task to evaluate + * @return true if the task meets all the conditions specified by this selector, otherwise false + */ + public boolean evaluate(Task task) + { + boolean isMatch = true; + if (cxtTagsConditions != null) { + isMatch = cxtTagsConditions.entrySet().stream().allMatch(entry -> { + String tagKey = entry.getKey(); + Set tagValues = entry.getValue(); + Map tags = task.getContextValue(DruidMetrics.TAGS); + if (tags == null || tags.isEmpty()) { + return false; + } + Object tagValue = tags.get(tagKey); + + return tagValue == null ? false : tagValues.contains((String) tagValue); + }); + } + + if (isMatch && taskTypeCondition != null) { + isMatch = taskTypeCondition.contains(task.getType()); + } + + if (isMatch && dataSourceCondition != null) { + isMatch = dataSourceCondition.contains(task.getDataSource()); + } + + return isMatch; + } + + @JsonProperty + public String getSelectionKey() + { + return selectionKey; + } + + @JsonProperty("context.tags") + public Map> getCxtTagsConditions() + { + return cxtTagsConditions; + } + + @JsonProperty("type") + public Set getTaskTypeCondition() + { + return taskTypeCondition; + } + + @JsonProperty("dataSource") + public Set getDataSourceCondition() + { + return dataSourceCondition; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Selector selector = (Selector) o; + return Objects.equals(selectionKey, selector.selectionKey) && Objects.equals( + cxtTagsConditions, + selector.cxtTagsConditions + ) && Objects.equals(taskTypeCondition, selector.taskTypeCondition) && Objects.equals( + dataSourceCondition, + selector.dataSourceCondition + ); + } + + @Override + public int hashCode() + { + return Objects.hash(selectionKey, cxtTagsConditions, taskTypeCondition, dataSourceCondition); + } + + @Override + public String toString() + { + return "Selector{" + + "selectionKey=" + selectionKey + + ", context.tags=" + cxtTagsConditions + + ", type=" + taskTypeCondition + + ", dataSource=" + dataSourceCondition + + '}'; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategy.java new file mode 100644 index 000000000000..938ed04e6a60 --- /dev/null +++ b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategy.java @@ -0,0 +1,112 @@ +/* + * 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.execution; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import io.fabric8.kubernetes.api.model.PodTemplate; +import org.apache.druid.indexing.common.task.Task; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Implements {@link PodTemplateSelectStrategy} by dynamically evaluating a series of selectors. + * Each selector corresponds to a potential task template key. + */ +public class SelectorBasedPodTemplateSelectStrategy implements PodTemplateSelectStrategy +{ + @Nullable + private String defaultKey; + private List selectors; + + @JsonCreator + public SelectorBasedPodTemplateSelectStrategy( + @JsonProperty("selectors") List selectors, + @JsonProperty("defaultKey") @Nullable String defaultKey + ) + { + Preconditions.checkNotNull(selectors, "selectors"); + this.selectors = selectors; + this.defaultKey = defaultKey; + } + + /** + * Evaluates the provided task against the set selectors to determine its template. + * + * @param task the task to be checked + * @return the template if a selector matches, otherwise fallback to base template + */ + @Override + public PodTemplate getPodTemplateForTask(Task task, Map templates) + { + String templateKey = selectors.stream() + .filter(selector -> selector.evaluate(task)) + .findFirst() + .map(Selector::getSelectionKey) + .orElse(defaultKey); + + return templates.getOrDefault(templateKey, templates.get("base")); + } + + @JsonProperty + public List getSelectors() + { + return selectors; + } + + @Nullable + @JsonProperty + public String getDefaultKey() + { + return defaultKey; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SelectorBasedPodTemplateSelectStrategy that = (SelectorBasedPodTemplateSelectStrategy) o; + return Objects.equals(defaultKey, that.defaultKey) && Objects.equals(selectors, that.selectors); + } + + @Override + public int hashCode() + { + return Objects.hash(defaultKey, selectors); + } + + @Override + public String toString() + { + return "SelectorBasedPodTemplateSelectStrategy{" + + "selectors=" + selectors + + ", defaultKey=" + defaultKey + + '}'; + } +} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcher.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcher.java deleted file mode 100644 index 8eca337b5ab4..000000000000 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcher.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * 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.execution; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.query.DruidMetrics; - -import java.util.Map; -import java.util.Objects; -import java.util.Set; - -public class TaskPropertiesMatcher implements Matcher -{ - private final Map> cxtTagsConditions; - private final Map> taskFieldsConditions; - - /** - * Creates a matcher with specified conditions for context tags and task fields. - * - * @param cxtTagsConditions conditions on context tags - * @param taskFieldsConditions conditions on task fields - */ - @JsonCreator - public TaskPropertiesMatcher( - @JsonProperty("context.tags") Map> cxtTagsConditions, - @JsonProperty("task") Map> taskFieldsConditions - ) - { - this.cxtTagsConditions = cxtTagsConditions; - this.taskFieldsConditions = taskFieldsConditions; - } - - @Override - public boolean evaluate(Task task) - { - boolean tagsMatch = true; - if (cxtTagsConditions != null) { - tagsMatch = cxtTagsConditions.entrySet().stream().allMatch(entry -> { - String tagKey = entry.getKey(); - Set tagValues = entry.getValue(); - Map tags = task.getContextValue(DruidMetrics.TAGS); - if (tags == null || tags.isEmpty()) { - return false; - } - Object tagValue = tags.get(tagKey); - - return tagValue == null ? false : tagValues.contains((String) tagValue); - }); - } - - if (!tagsMatch) { - return false; - } - - if (taskFieldsConditions != null) { - return taskFieldsConditions.entrySet().stream().allMatch(entry -> { - String fieldKey = entry.getKey(); - Set fieldValues = entry.getValue(); - if ("datasource".equalsIgnoreCase(fieldKey)) { - return fieldValues.contains(task.getDataSource()); - } - - if ("type".equalsIgnoreCase(fieldKey)) { - return fieldValues.contains(task.getType()); - } - - return false; - }); - } - - return true; - } - - @JsonProperty("context.tags") - public Map> getCxtTagsConditions() - { - return cxtTagsConditions; - } - - @JsonProperty("task") - public Map> getTaskFieldsConditions() - { - return taskFieldsConditions; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - TaskPropertiesMatcher matcher = (TaskPropertiesMatcher) o; - return Objects.equals(cxtTagsConditions, matcher.cxtTagsConditions - ) && Objects.equals(taskFieldsConditions, matcher.taskFieldsConditions); - } - - @Override - public int hashCode() - { - return Objects.hash(cxtTagsConditions, taskFieldsConditions); - } - - @Override - public String toString() - { - return "TaskPropertiesMatcher{" + - "context.tags=" + cxtTagsConditions + - ", task=" + taskFieldsConditions + - '}'; - } -} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategy.java b/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategy.java deleted file mode 100644 index 2fd2da71fb5b..000000000000 --- a/extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategy.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * 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.execution; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import io.fabric8.kubernetes.api.model.PodTemplate; -import org.apache.druid.indexing.common.task.Task; - -import java.util.List; -import java.util.Map; -import java.util.Objects; - -/** - * Implements {@link PodTemplateSelectStrategy} by dynamically evaluating a series of selectors. - * Each selector corresponds to a potential task template key. - */ -public class TaskPropertiesPodTemplateSelectStrategy implements PodTemplateSelectStrategy -{ - private List templateSelectors; - - @JsonCreator - public TaskPropertiesPodTemplateSelectStrategy( - @JsonProperty("templateSelectors") List templateSelectors - ) - { - Preconditions.checkNotNull(templateSelectors, "templateSelectors"); - this.templateSelectors = templateSelectors; - } - - /** - * Evaluates the provided task against the set selectors to determine its template. - * - * @param task the task to be checked - * @return the template if a selector matches, otherwise fallback to base template - */ - @Override - public PodTemplate getPodTemplateForTask(Task task, Map templates) - { - String templateKey = templateSelectors.stream() - .filter(selector -> selector.getMatcher().evaluate(task)) - .findFirst() - .map(TemplateSelector::getTemplateKey) - .orElse(null); - - return templates.getOrDefault(templateKey, templates.get("base")); - } - - @JsonProperty - public List getTemplateSelectors() - { - return templateSelectors; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - TaskPropertiesPodTemplateSelectStrategy that = (TaskPropertiesPodTemplateSelectStrategy) o; - return Objects.equals(templateSelectors, that.templateSelectors); - } - - @Override - public int hashCode() - { - return Objects.hashCode(templateSelectors); - } - - @Override - public String toString() - { - return "DynamicTaskPodTemplateSelectStrategy{" + - "templateSelectors=" + templateSelectors + - '}'; - } - - public static class TemplateSelector - { - private final Matcher matcher; - private final String templateKey; - - @JsonCreator - public TemplateSelector( - @JsonProperty("templateKey") String templateKey, - @JsonProperty("matcher") Matcher matcher - ) - { - Preconditions.checkNotNull(templateKey, "templateKey"); - Preconditions.checkNotNull(matcher, "matcher"); - this.templateKey = templateKey; - this.matcher = matcher; - } - - @JsonProperty - public String getTemplateKey() - { - return templateKey; - } - - @JsonProperty - public Matcher getMatcher() - { - return matcher; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - TemplateSelector that = (TemplateSelector) o; - return Objects.equals(matcher, that.matcher) && Objects.equals(templateKey, that.templateKey); - } - - @Override - public int hashCode() - { - return Objects.hash(matcher, templateKey); - } - - @Override - public String toString() - { - return "TemplateSelector{" + - "templateKey=" + templateKey + - ", matcher=" + matcher + - '}'; - } - } -} diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java index 202cab0d4bf8..6236794d3661 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/KubernetesTaskRunnerDynamicConfigTest.java @@ -47,44 +47,31 @@ public void testSerde() throws JsonProcessingException Assert.assertTrue(selectStrategy instanceof TaskTypePodTemplateSelectStrategy); json = "{\n" - + " \"type\":\"default\",\n" - + " \"podTemplateSelectStrategy\":{\n" - + " \"type\":\"taskProperties\",\n" - + " \"templateSelectors\":[\n" - + " {\n" - + " \"templateKey\":\"low-throughput\",\n" - + " \"matcher\":{\n" - + " \"type\":\"taskProperties\",\n" - + " \"context.tags\":{\n" - + " \"billingCategory\":[\n" - + " \"streaming_ingestion\"\n" - + " ]\n" - + " },\n" - + " \"task\":{\n" - + " \"datasource\":[\n" - + " \"wikipedia\"\n" - + " ]\n" - + " }\n" - + " }\n" - + " },\n" - + " {\n" - + " \"templateKey\":\"medium-throughput\",\n" - + " \"matcher\":{\n" - + " \"type\":\"taskProperties\",\n" - + " \"task\":{\n" - + " \"type\":[\n" - + " \"index_kafka\"\n" - + " ]\n" - + " }\n" - + " }\n" - + " }\n" - + " ]\n" - + " }\n" + + " \"type\": \"default\",\n" + + " \"podTemplateSelectStrategy\":\n" + + " {\n" + + " \"type\": \"selectorBased\",\n" + + " \"selectors\": [\n" + + " {\n" + + " \"selectionKey\": \"low-throughput\",\n" + + " \"context.tags\":\n" + + " {\n" + + " \"billingCategory\": [\"streaming_ingestion\"]\n" + + " },\n" + + " \"dataSource\": [\"wikipedia\"]\n" + + " },\n" + + " {\n" + + " \"selectionKey\": \"medium-throughput\",\n" + + " \"type\": [\"index_kafka\"]\n" + + " }\n" + + " ],\n" + + " \"defaultKey\": \"base\"\n" + + " }\n" + "}"; deserialized = jsonMapper.readValue(json, KubernetesTaskRunnerDynamicConfig.class); selectStrategy = deserialized.getPodTemplateSelectStrategy(); - Assert.assertTrue(selectStrategy instanceof TaskPropertiesPodTemplateSelectStrategy); - Assert.assertEquals(2, ((TaskPropertiesPodTemplateSelectStrategy) selectStrategy).getTemplateSelectors().size()); + Assert.assertTrue(selectStrategy instanceof SelectorBasedPodTemplateSelectStrategy); + Assert.assertEquals(2, ((SelectorBasedPodTemplateSelectStrategy) selectStrategy).getSelectors().size()); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategyTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategyTest.java similarity index 60% rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategyTest.java rename to extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategyTest.java index 532753170860..9aa1376a5157 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesPodTemplateSelectStrategyTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorBasedPodTemplateSelectStrategyTest.java @@ -38,7 +38,7 @@ import java.util.Map; import java.util.Set; -public class TaskPropertiesPodTemplateSelectStrategyTest +public class SelectorBasedPodTemplateSelectStrategyTest { private Map templates; @@ -86,55 +86,53 @@ public String getName() } @Test(expected = NullPointerException.class) - public void shouldThrowNullPointerExceptionWhenTemplateSelectorsAreNull() + public void shouldThrowNullPointerExceptionWhenSelectorsAreNull() { - new TaskPropertiesPodTemplateSelectStrategy(null); + new SelectorBasedPodTemplateSelectStrategy(null, null); } - @Test(expected = NullPointerException.class) - public void shouldThrowNullPointerExceptionWhenTemplateKeyIsNull() - { - new TaskPropertiesPodTemplateSelectStrategy.TemplateSelector(null, task -> false); - } - - @Test(expected = NullPointerException.class) - public void shouldThrowNullPointerExceptionWhenMatcherIsNull() + @Test + public void testGetPodTemplate_ForTask_emptySelectorsFallbackToBaseTemplate() { - new TaskPropertiesPodTemplateSelectStrategy.TemplateSelector("myTemplate", null); + List emptySelectors = Collections.emptyList(); + SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(emptySelectors, null); + Task task = NoopTask.create(); + Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).getMetadata().getName()); } @Test - public void testGetPodTemplate_ForTask_emptySelectorsFallbackToBaseTemplate() + public void testGetPodTemplate_ForTask_noMatchSelectorsFallbackToBaseTemplateIfNullDefaultKey() { - List emptySelectors = Collections.emptyList(); - TaskPropertiesPodTemplateSelectStrategy strategy = new TaskPropertiesPodTemplateSelectStrategy(emptySelectors); + Selector noMatchSelector = new MockSelector(false, "mock"); + List selectors = Collections.singletonList(noMatchSelector); + SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors, null); Task task = NoopTask.create(); Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).getMetadata().getName()); } @Test - public void testGetPodTemplate_ForTask_noMatchSelectorsFallbackToBaseTemplate() + public void testGetPodTemplate_ForTask_noMatchSelectorsFallbackToDefaultKeyTemplate() { - TaskPropertiesPodTemplateSelectStrategy.TemplateSelector noMatchSelector = new MockTemplateSelector(false, "mock"); - List selectors = Collections.singletonList(noMatchSelector); - TaskPropertiesPodTemplateSelectStrategy strategy = new TaskPropertiesPodTemplateSelectStrategy(selectors); + Selector noMatchSelector = new MockSelector(false, "mock"); + List selectors = Collections.singletonList(noMatchSelector); + SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors, "match"); Task task = NoopTask.create(); - Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).getMetadata().getName()); + Assert.assertEquals("match", strategy.getPodTemplateForTask(task, templates).getMetadata().getName()); } @Test public void testGetPodTemplate_ForTask_withMatchSelectors() { - TaskPropertiesPodTemplateSelectStrategy.TemplateSelector noMatchSelector = new MockTemplateSelector( + Selector noMatchSelector = new MockSelector( false, "no_match" ); - TaskPropertiesPodTemplateSelectStrategy.TemplateSelector matchSelector = new MockTemplateSelector(true, "match"); - List selectors = Lists.newArrayList( + Selector matchSelector = new MockSelector(true, "match"); + List selectors = Lists.newArrayList( noMatchSelector, matchSelector ); - TaskPropertiesPodTemplateSelectStrategy strategy = new TaskPropertiesPodTemplateSelectStrategy(selectors); + SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors, null); Task task = NoopTask.create(); Assert.assertEquals("match", strategy.getPodTemplateForTask(task, templates).getMetadata().getName()); } @@ -146,29 +144,37 @@ public void testSerde() throws Exception Map> cxtTagsConditions = new HashMap<>(); cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); - Map> taskFieldsConditions = new HashMap<>(); - taskFieldsConditions.put("type", Sets.newHashSet(NoopTask.TYPE)); - - TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( + Selector selector = new Selector( + "TestSelector", cxtTagsConditions, - taskFieldsConditions + Sets.newHashSet(NoopTask.TYPE), + Sets.newHashSet("my_table") ); - TaskPropertiesPodTemplateSelectStrategy strategy = new TaskPropertiesPodTemplateSelectStrategy(Collections.singletonList( - new TaskPropertiesPodTemplateSelectStrategy.TemplateSelector("TestSelector", matcher))); + SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy( + Collections.singletonList(selector), "default"); - TaskPropertiesPodTemplateSelectStrategy strategy2 = objectMapper.readValue( + SelectorBasedPodTemplateSelectStrategy strategy2 = objectMapper.readValue( objectMapper.writeValueAsBytes(strategy), - TaskPropertiesPodTemplateSelectStrategy.class + SelectorBasedPodTemplateSelectStrategy.class ); Assert.assertEquals(strategy, strategy2); } - static class MockTemplateSelector extends TaskPropertiesPodTemplateSelectStrategy.TemplateSelector + static class MockSelector extends Selector { - MockTemplateSelector(boolean matches, String name) + private final boolean matches; + + MockSelector(boolean matches, String name) + { + super(name, null, null, null); + this.matches = matches; + } + + @Override + public boolean evaluate(final Task task) { - super(name, task -> matches); + return matches; } } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcherTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java similarity index 65% rename from extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcherTest.java rename to extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java index e207b61ed616..0ecff67408e3 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/TaskPropertiesMatcherTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/execution/SelectorTest.java @@ -33,7 +33,7 @@ import java.util.Map; import java.util.Set; -public class TaskPropertiesMatcherTest +public class SelectorTest { @Test @@ -43,18 +43,17 @@ public void shouldReturnTrueWhenAllTagsAndTasksMatch() Map> cxtTagsConditions = new HashMap<>(); cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); - Map> taskFieldsConditions = new HashMap<>(); - taskFieldsConditions.put("datasource", Sets.newHashSet(dataSource)); - Task task = NoopTask.forDatasource(dataSource); task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); - TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( + Selector selector = new Selector( + "TestSelector", cxtTagsConditions, - taskFieldsConditions + null, + Sets.newHashSet(dataSource) ); - Assert.assertTrue(matcher.evaluate(task)); + Assert.assertTrue(selector.evaluate(task)); } @Test @@ -64,17 +63,16 @@ public void shouldReturnFalseWhenTagsDoNotMatch() Map> cxtTagsConditions = new HashMap<>(); cxtTagsConditions.put("nonexistentTag", Sets.newHashSet("tag1value")); - Map> taskFieldsConditions = new HashMap<>(); - taskFieldsConditions.put("datasource", Sets.newHashSet(dataSource)); - Task task = NoopTask.forDatasource(dataSource); - TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( + Selector selector = new Selector( + "TestSelector", cxtTagsConditions, - taskFieldsConditions + null, + Sets.newHashSet(dataSource) ); - Assert.assertFalse(matcher.evaluate(task)); + Assert.assertFalse(selector.evaluate(task)); } @Test @@ -85,18 +83,17 @@ public void shouldReturnFalseWhenSomeTagsDoNotMatch() cxtTagsConditions.put("nonexistentTag", Sets.newHashSet("nonexistentTagValue")); cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); - Map> taskFieldsConditions = new HashMap<>(); - taskFieldsConditions.put("datasource", Sets.newHashSet(dataSource)); - Task task = NoopTask.forDatasource(dataSource); task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); - TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( + Selector selector = new Selector( + "TestSelector", cxtTagsConditions, - taskFieldsConditions + null, + Sets.newHashSet(dataSource) ); - Assert.assertFalse(matcher.evaluate(task)); + Assert.assertFalse(selector.evaluate(task)); } @Test @@ -105,18 +102,17 @@ public void shouldReturnFalseWhenTaskFieldsDoNotMatch() Map> cxtTagsConditions = new HashMap<>(); cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); - Map> taskFieldsConditions = new HashMap<>(); - taskFieldsConditions.put("datasource", Sets.newHashSet("my_table")); - Task task = NoopTask.forDatasource("another_table"); task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); - TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( + Selector selector = new Selector( + "TestSelector", cxtTagsConditions, - taskFieldsConditions + null, + Sets.newHashSet("my_table") ); - Assert.assertFalse(matcher.evaluate(task)); + Assert.assertFalse(selector.evaluate(task)); } @Test @@ -125,19 +121,17 @@ public void shouldReturnFalseWhenSomeTaskFieldsDoNotMatch() Map> cxtTagsConditions = new HashMap<>(); cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); - Map> taskFieldsConditions = new HashMap<>(); - taskFieldsConditions.put("type", Sets.newHashSet(NoopTask.TYPE)); - taskFieldsConditions.put("datasource", Sets.newHashSet("my_table")); - Task task = NoopTask.forDatasource("another_table"); task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); - TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( + Selector selector = new Selector( + "TestSelector", cxtTagsConditions, - taskFieldsConditions + Sets.newHashSet(NoopTask.TYPE), + Sets.newHashSet("my_table") ); - Assert.assertFalse(matcher.evaluate(task)); + Assert.assertFalse(selector.evaluate(task)); } @Test @@ -146,12 +140,14 @@ public void shouldReturnTrueWhenNoConditionsSpecified() Task task = NoopTask.forDatasource("my_table"); task.addToContext(DruidMetrics.TAGS, ImmutableMap.of("tag1", "tag1value")); - TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( + Selector selector = new Selector( + "TestSelector", + null, null, null ); - Assert.assertTrue(matcher.evaluate(task)); + Assert.assertTrue(selector.evaluate(task)); } @Test @@ -161,18 +157,17 @@ public void testSerde() throws Exception Map> cxtTagsConditions = new HashMap<>(); cxtTagsConditions.put("tag1", Sets.newHashSet("tag1value")); - Map> taskFieldsConditions = new HashMap<>(); - taskFieldsConditions.put("type", Sets.newHashSet(NoopTask.TYPE)); - - TaskPropertiesMatcher matcher = new TaskPropertiesMatcher( + Selector selector = new Selector( + "TestSelector", cxtTagsConditions, - taskFieldsConditions + Sets.newHashSet(NoopTask.TYPE), + Sets.newHashSet("my_table") ); - TaskPropertiesMatcher matcher2 = objectMapper.readValue( - objectMapper.writeValueAsBytes(matcher), - TaskPropertiesMatcher.class + Selector selector2 = objectMapper.readValue( + objectMapper.writeValueAsBytes(selector), + Selector.class ); - Assert.assertEquals(matcher, matcher2); + Assert.assertEquals(selector, selector2); } } diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java index 5a5635010a41..36fc77631a1c 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/PodTemplateTaskAdapterTest.java @@ -43,8 +43,8 @@ import org.apache.druid.k8s.overlord.common.K8sTestUtils; import org.apache.druid.k8s.overlord.execution.DefaultKubernetesTaskRunnerDynamicConfig; import org.apache.druid.k8s.overlord.execution.KubernetesTaskRunnerDynamicConfig; -import org.apache.druid.k8s.overlord.execution.TaskPropertiesMatcher; -import org.apache.druid.k8s.overlord.execution.TaskPropertiesPodTemplateSelectStrategy; +import org.apache.druid.k8s.overlord.execution.Selector; +import org.apache.druid.k8s.overlord.execution.SelectorBasedPodTemplateSelectStrategy; import org.apache.druid.server.DruidNode; import org.apache.druid.tasklogs.TaskLogs; import org.easymock.EasyMock; @@ -617,15 +617,10 @@ 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 TaskPropertiesPodTemplateSelectStrategy( + dynamicConfigRef = () -> new DefaultKubernetesTaskRunnerDynamicConfig(new SelectorBasedPodTemplateSelectStrategy( Collections.singletonList( - new TaskPropertiesPodTemplateSelectStrategy.TemplateSelector( - "lowThrougput", - new TaskPropertiesMatcher(null, ImmutableMap.of( - "datasource", - Sets.newSet(dataSource) - )) - )))); + new Selector("lowThrougput", null, null, Sets.newSet(dataSource) + )), null)); PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter( taskRunnerConfig,